From e8b96e66cc734f5bb3414faf4899cd8d46faba16 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 27 Feb 2019 11:35:49 -0800 Subject: [PATCH 01/67] Initial commit for multi-part lists --- posting/index.go | 15 +- posting/list.go | 536 ++++++++++++++++++++++++++++++++---- posting/list_test.go | 97 +++++-- posting/mvcc_test.go | 4 +- protos/pb.proto | 5 + protos/pb/pb.pb.go | 572 +++++++++++++++++++++++++-------------- worker/draft.go | 13 +- worker/predicate_move.go | 6 +- x/keys.go | 18 ++ 9 files changed, 960 insertions(+), 306 deletions(-) diff --git a/posting/index.go b/posting/index.go index ee0df6d98ec..e7d2f9583c1 100644 --- a/posting/index.go +++ b/posting/index.go @@ -550,15 +550,18 @@ func (r *rebuild) Run(ctx context.Context) error { if le == 0 { continue } - kv, err := pl.MarshalToKv() + kvs, err := pl.MarshalToKv() if err != nil { return err } - // We choose to write the PL at r.startTs, so it won't be read by txns, - // which occurred before this schema mutation. Typically, we use - // kv.Version as the timestamp. - if err = writer.SetAt(kv.Key, kv.Value, kv.UserMeta[0], r.startTs); err != nil { - return err + + for _, kv := range kvs { + // We choose to write the PL at r.startTs, so it won't be read by txns, + // which occurred before this schema mutation. Typically, we use + // kv.Version as the timestamp. + if err = writer.SetAt(kv.Key, kv.Value, kv.UserMeta[0], r.startTs); err != nil { + return err + } } // This locking is just to catch any future issues. We shouldn't need // to release this lock, because each posting list must only be accessed diff --git a/posting/list.go b/posting/list.go index aade0d7405d..4d65069ee6f 100644 --- a/posting/list.go +++ b/posting/list.go @@ -19,6 +19,7 @@ package posting import ( "bytes" "context" + "encoding/binary" "errors" "fmt" "log" @@ -29,6 +30,7 @@ import ( "github.com/dgryski/go-farm" "github.com/golang/glog" + "github.com/dgraph-io/badger" bpb "github.com/dgraph-io/badger/pb" "github.com/dgraph-io/dgo/protos/api" "github.com/dgraph-io/dgo/y" @@ -50,6 +52,7 @@ var ( ErrInvalidTxn = fmt.Errorf("Invalid transaction") ErrStopIteration = errors.New("Stop iteration") emptyPosting = &pb.Posting{} + maxListLength = 2000000 ) const ( @@ -75,6 +78,73 @@ type List struct { pendingTxns int32 // Using atomic for this, to avoid locking in SetForDeletion operation. deleteMe int32 // Using atomic for this, to avoid expensive SetForDeletion operation. + next *List // If a multi-part list, this is a pointer to the next list. + first *List // If a multi-part list, points to the first part of the list. +} + +func appendNextStartToKey(key []byte, nextPartStart uint64) []byte { + keyCopy := make([]byte, len(key)) + copy(keyCopy, key) + + encNexStart := make([]byte, 8) + binary.BigEndian.PutUint64(encNexStart, nextPartStart) + keyCopy = append(keyCopy, encNexStart...) + return keyCopy +} + +func replaceNextStartInKey(key []byte, nextPartStart uint64) []byte { + keyCopy := make([]byte, len(key)) + copy(keyCopy, key) + + rest := keyCopy[len(keyCopy)-8:] + binary.BigEndian.PutUint64(rest, nextPartStart) + return keyCopy +} + +func (l *List) getNextPartKey() []byte { + if l.plist != nil { + return nil + } + + if !l.plist.MultiPart { + return nil + } + + if l.plist.EndUid == math.MaxUint64 { + return nil + } + + if l.plist.FirstPart { + // Add the start of the next list to the end of the key. + return appendNextStartToKey(l.key, l.plist.EndUid+1) + } + + // In this case, the key already includes the extra bytes to encode the start + // UID of this part. Replace those bytes with the start UID of the next part. + return replaceNextStartInKey(l.key, l.plist.EndUid+1) +} + +func generateNextPartKey(currKey []byte, currPl *pb.PostingList, nextPartStart uint64) []byte { + appendToKey := currPl.FirstPart || !currPl.MultiPart + if appendToKey { + return appendNextStartToKey(currKey, nextPartStart) + } + return replaceNextStartInKey(currKey, nextPartStart) +} + +func (l *List) updateMinTs(readTs, minTs uint64) error { + l.AssertLock() + + curr := l + for curr != nil { + curr.minTs = minTs + if err := curr.loadNextPart(readTs); err != nil { + return err + } + curr = curr.next + } + + return nil } func (l *List) maxVersion() uint64 { @@ -84,41 +154,84 @@ func (l *List) maxVersion() uint64 { } type PIterator struct { - pl *pb.PostingList + l *List + opts PItrOpts uidPosting *pb.Posting pidx int // index of postings plen int + firstPart bool dec *codec.Decoder uids []uint64 uidx int // Offset into the uids slice } -func (it *PIterator) Init(pl *pb.PostingList, afterUid uint64) { - it.pl = pl +type PItrOpts struct { + discardPl bool + afterUid uint64 + partialIteration bool + readTs uint64 +} + +func (it *PIterator) Init(l *List, opts PItrOpts) { + if l.plist.FirstPart { + l.loadNextPart(opts.readTs) + it.Init(l.next, opts) + return + } + + it.l = l + it.opts = opts it.uidPosting = &pb.Posting{} - it.dec = &codec.Decoder{Pack: pl.Pack} - it.uids = it.dec.Seek(afterUid) + it.dec = &codec.Decoder{Pack: l.plist.Pack} + it.uids = it.dec.Seek(opts.afterUid) it.uidx = 0 - it.plen = len(pl.Postings) + it.plen = len(l.plist.Postings) it.pidx = sort.Search(it.plen, func(idx int) bool { - p := pl.Postings[idx] - return afterUid < p.Uid + p := l.plist.Postings[idx] + return it.opts.afterUid < p.Uid }) } -func (it *PIterator) Next() { +func (it *PIterator) Next() error { + if it.opts.discardPl { + return nil + } + it.uidx++ if it.uidx < len(it.uids) { - return + return nil } it.uidx = 0 it.uids = it.dec.Next() + return nil } func (it *PIterator) Valid() bool { + if it.opts.discardPl { + return false + } + + if len(it.uids) > 0 { + return true + } + + // Only iterate through this part of the list. + if it.opts.partialIteration { + return false + } + + // Load the next part of the list if it exists, and reinitialize the + // iterator to move to the start of this new part. + if err := it.l.loadNextPart(it.l.plist.CommitTs); err != nil { + return false + } + if it.l.next == nil { + return false + } + it.Init(it.l.next, it.opts) return len(it.uids) > 0 } @@ -126,7 +239,7 @@ func (it *PIterator) Posting() *pb.Posting { uid := it.uids[it.uidx] for it.pidx < it.plen { - p := it.pl.Postings[it.pidx] + p := it.l.plist.Postings[it.pidx] if p.Uid > uid { break } @@ -444,7 +557,15 @@ func (l *List) Conflicts(readTs uint64) []uint64 { return conflicts } -func (l *List) pickPostings(readTs uint64) (*pb.PostingList, []*pb.Posting) { +func (l *List) pickPostings(readTs uint64) (uint64, []*pb.Posting) { + return l.pickPostingsInternal(readTs, false) +} + +func (l *List) pickPartPostings(readTs uint64) (uint64, []*pb.Posting) { + return l.pickPostingsInternal(readTs, true) +} + +func (l *List) pickPostingsInternal(readTs uint64, partial bool) (uint64, []*pb.Posting) { // This function would return zero ts for entries above readTs. effective := func(start, commit uint64) uint64 { if commit > 0 && commit <= readTs { @@ -461,12 +582,19 @@ func (l *List) pickPostings(readTs uint64) (*pb.PostingList, []*pb.Posting) { // First pick up the postings. var deleteBelow uint64 var posts []*pb.Posting - for startTs, plist := range l.mutationMap { + mutationMap := l.mutationMap + if l.first != nil { + mutationMap = l.first.mutationMap + } + for startTs, plist := range mutationMap { // Pick up the transactions which are either committed, or the one which is ME. effectiveTs := effective(startTs, plist.CommitTs) if effectiveTs > deleteBelow { // We're above the deleteBelow marker. We wouldn't reach here if effectiveTs is zero. for _, mpost := range plist.Postings { + if partial && (mpost.Uid < l.plist.StartUid || l.plist.EndUid < mpost.Uid) { + continue + } if hasDeleteAll(mpost) { deleteBelow = effectiveTs continue @@ -476,16 +604,9 @@ func (l *List) pickPostings(readTs uint64) (*pb.PostingList, []*pb.Posting) { } } - storedList := l.plist if deleteBelow > 0 { // There was a delete all marker. So, trim down the list of postings. - - // Create an empty posting list at the same commit ts as the deletion marker. This is - // important, so that after rollup happens, we are left with a posting list at the - // highest commit timestamp. - storedList = &pb.PostingList{CommitTs: deleteBelow} result := posts[:0] - // Trim the posts. for _, post := range posts { effectiveTs := effective(post.StartTs, post.CommitTs) if effectiveTs < deleteBelow { // Do pick the posts at effectiveTs == deleteBelow. @@ -507,13 +628,13 @@ func (l *List) pickPostings(readTs uint64) (*pb.PostingList, []*pb.Posting) { } return pi.Uid < pj.Uid }) - return storedList, posts + return deleteBelow, posts } func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) error) error { l.AssertRLock() - plist, mposts := l.pickPostings(readTs) + deleteBelow, mposts := l.pickPostings(readTs) if readTs < l.minTs { return x.Errorf("readTs: %d less than minTs: %d for key: %q", readTs, l.minTs, l.key) } @@ -528,7 +649,11 @@ func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) e var mp, pp *pb.Posting var pitr PIterator - pitr.Init(plist, afterUid) + pitr.Init(l, PItrOpts{ + afterUid: afterUid, + discardPl: deleteBelow > 0, + readTs: readTs, + }) prevUid := uint64(0) var err error for err == nil { @@ -554,7 +679,9 @@ func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) e case mp.Uid == 0 || (pp.Uid > 0 && pp.Uid < mp.Uid): // Either mp is empty, or pp is lower than mp. err = f(pp) - pitr.Next() + if err := pitr.Next(); err != nil { + return err + } case pp.Uid == 0 || (mp.Uid > 0 && mp.Uid < pp.Uid): // Either pp is empty, or mp is lower than pp. if mp.Op != Del { @@ -567,7 +694,84 @@ func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) e err = f(mp) } prevUid = mp.Uid - pitr.Next() + if err := pitr.Next(); err != nil { + return err + } + midx++ + default: + log.Fatalf("Unhandled case during iteration of posting list.") + } + } + if err == ErrStopIteration { + return nil + } + return err +} + +func (l *List) partIterate(readTs uint64, f func(obj *pb.Posting) error) error { + if !l.plist.MultiPart { + return l.iterate(readTs, 0, f) + } + + if l.plist.FirstPart { + return nil + } + deleteBelow, mposts := l.pickPartPostings(readTs) + if readTs < l.minTs { + return x.Errorf("readTs: %d less than minTs: %d for key: %q", readTs, l.minTs, l.key) + } + + midx, mlen := 0, len(mposts) + var mp, pp *pb.Posting + var pitr PIterator + pitr.Init(l, PItrOpts{ + afterUid: 0, + discardPl: deleteBelow > 0, + partialIteration: true, + }) + prevUid := uint64(0) + var err error + for err == nil { + if midx < mlen { + mp = mposts[midx] + } else { + mp = emptyPosting + } + if pitr.Valid() { + pp = pitr.Posting() + } else { + pp = emptyPosting + } + + switch { + case mp.Uid > 0 && mp.Uid == prevUid: + // Only pick the latest version of this posting. + // mp.Uid can be zero if it's an empty posting. + midx++ + case pp.Uid == 0 && mp.Uid == 0: + // Reached empty posting for both iterators. + return nil + case mp.Uid == 0 || (pp.Uid > 0 && pp.Uid < mp.Uid): + // Either mp is empty, or pp is lower than mp. + err = f(pp) + if err := pitr.Next(); err != nil { + return err + } + case pp.Uid == 0 || (mp.Uid > 0 && mp.Uid < pp.Uid): + // Either pp is empty, or mp is lower than pp. + if mp.Op != Del { + err = f(mp) + } + prevUid = mp.Uid + midx++ + case pp.Uid == mp.Uid: + if mp.Op != Del { + err = f(mp) + } + prevUid = mp.Uid + if err := pitr.Next(); err != nil { + return err + } midx++ default: log.Fatalf("Unhandled case during iteration of posting list.") @@ -606,6 +810,18 @@ func (l *List) length(readTs, afterUid uint64) int { return count } +func (l *List) partialLength(readTs uint64) int { + count := 0 + err := l.partIterate(readTs, func(p *pb.Posting) error { + count++ + return nil + }) + if err != nil { + return -1 + } + return count +} + // Length iterates over the mutation layer and counts number of elements. func (l *List) Length(readTs, afterUid uint64) int { l.RLock() @@ -613,20 +829,32 @@ func (l *List) Length(readTs, afterUid uint64) int { return l.length(readTs, afterUid) } -func (l *List) MarshalToKv() (*bpb.KV, error) { +func (l *List) MarshalToKv() ([]*bpb.KV, error) { l.Lock() defer l.Unlock() if err := l.rollup(math.MaxUint64); err != nil { return nil, err } - kv := &bpb.KV{} - kv.Version = l.minTs - kv.Key = l.key - val, meta := marshalPostingList(l.plist) - kv.UserMeta = []byte{meta} - kv.Value = val - return kv, nil + var kvs []*bpb.KV + curr := l + for curr != nil { + kv := &bpb.KV{} + kv.Version = curr.minTs + kv.Key = curr.key + val, meta := marshalPostingList(curr.plist) + kv.UserMeta = []byte{meta} + kv.Value = val + kvs = append(kvs, kv) + + // Load next part of the list if necessary. + if err := curr.loadNextPart(curr.plist.CommitTs); err != nil { + return nil, err + } + curr = curr.next + } + + return kvs, nil } func marshalPostingList(plist *pb.PostingList) (data []byte, meta byte) { @@ -658,25 +886,47 @@ func (l *List) rollup(readTs uint64) error { return nil } - final := new(pb.PostingList) - enc := codec.Encoder{BlockSize: blockSize} - err := l.iterate(readTs, 0, func(p *pb.Posting) error { - // iterate already takes care of not returning entries whose commitTs is above l.commitTs. - // So, we don't need to do any filtering here. In fact, doing filtering here could result - // in a bug. - enc.Add(p.Uid) + if l.plist.MultiPart && !l.plist.FirstPart { + return fmt.Errorf("rollup can only be called from the first part of a multi-part list") + } - // We want to add the posting if it has facets or has a value. - if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { - // I think it's okay to take the pointer from the iterator, because we have a lock - // over List; which won't be released until final has been marshalled. Thus, the - // underlying data wouldn't be changed. - final.Postings = append(final.Postings, p) - } + // Check if the list (or any of it's parts if it's been previously split) have + // become too big. Split the list if that is the case. + if err := l.splitList(readTs); err != nil { return nil - }) - x.Check(err) - final.Pack = enc.Done() + } + + curr := l + for curr != nil { + final := new(pb.PostingList) + + enc := codec.Encoder{BlockSize: blockSize} + err := curr.partIterate(readTs, func(p *pb.Posting) error { + // iterate already takes care of not returning entries whose commitTs + // is above curr.commitTs. + // So, we don't need to do any filtering here. In fact, doing filtering + // here could result in a bug. + enc.Add(p.Uid) + + // We want to add the posting if it has facets or has a value. + if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { + // I think it's okay to take the pointer from the iterator, because + // we have a lock over List; which won't be released until final has + // been marshalled. Thus, the underlying data wouldn't be changed. + final.Postings = append(final.Postings, p) + } + return nil + }) + x.Check(err) + final.Pack = enc.Done() + curr.plist.Pack = final.Pack + curr.plist.Postings = final.Postings + + if err := curr.loadNextPart(readTs); err != nil { + return err + } + curr = curr.next + } maxCommitTs := l.minTs { @@ -684,8 +934,8 @@ func (l *List) rollup(readTs uint64) error { // postings which had deletions to provide a sorted view of the list. Therefore, the safest // way to get the max commit timestamp is to pick all the relevant postings for the given // readTs and calculate the maxCommitTs. - plist, mposts := l.pickPostings(readTs) - maxCommitTs = x.Max(maxCommitTs, plist.CommitTs) + deleteBelow, mposts := l.pickPostings(readTs) + maxCommitTs = x.Max(maxCommitTs, deleteBelow) for _, mp := range mposts { maxCommitTs = x.Max(maxCommitTs, mp.CommitTs) } @@ -703,8 +953,7 @@ func (l *List) rollup(readTs uint64) error { } } - l.minTs = maxCommitTs - l.plist = final + l.updateMinTs(readTs, maxCommitTs) return nil } @@ -960,3 +1209,182 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs } return facets.CopyFacets(p.Facets, param), nil } + +func (l *List) loadNextPart(readTs uint64) error { + // No plist so there's nothing that can be loaded. + if l.plist == nil { + return nil + } + // This is not a multi-part list so nothing to load. + if !l.plist.MultiPart { + return nil + } + // The next part has already been loaded so nothing else to do. + if l.next != nil { + return nil + } + // This is the end of the multi-part list so there's nothing to do. + if l.plist.EndUid == math.MaxUint64 { + return nil + } + + txn := pstore.NewTransactionAt(readTs, false) + opts := badger.DefaultIteratorOptions + opts.AllVersions = true + it := txn.NewIterator(opts) + + nextPartKey := l.getNextPartKey() + if nextPartKey == nil { + return fmt.Errorf( + "Could not get find key for next part of the posting list. Current key %v", l.key) + } + + nextListPart, err := ReadPostingList(nextPartKey, it) + if err != nil { + return err + } + l.next = nextListPart + + return nil +} + +func (l *List) needsSplit(readTs uint64) bool { + length := l.partialLength(readTs) + return length >= maxListLength +} + +func (l *List) splitList(readTs uint64) error { + l.AssertLock() + + curr := l + var prev *List + for curr != nil { + + if curr.needsSplit(readTs) { + newLists := curr.splitListPart(readTs) + + // If splitting a list for the first time, initialize it as a + // multi-part list. + if !curr.plist.MultiPart { + curr.plist = &pb.PostingList{ + CommitTs: curr.plist.CommitTs, + MultiPart: true, + FirstPart: true, + } + curr.next = newLists[0] + } else { + prev.next = newLists[0] + } + + curr = newLists[1] + } + + if err := curr.loadNextPart(readTs); err != nil { + return err + } + + prev = curr + curr = curr.next + } + return nil +} + +func (l *List) splitListPart(readTs uint64) []*List { + var uids []uint64 + err := l.partIterate(readTs, func(p *pb.Posting) error { + uids = append(uids, p.Uid) + return nil + }) + x.Check(err) + + sort.Slice(uids, func(i, j int) bool { return uids[i] < uids[j] }) + midUid := uids[len(uids)/2] + + // Generate posting list holding the first half of the current list's postings. + lowPl := new(pb.PostingList) + lowEnc := codec.Encoder{BlockSize: blockSize} + err = l.partIterate(readTs, func(p *pb.Posting) error { + // Skip all postings with an UID greater than or equal to midUid. + if p.Uid >= midUid { + return nil + } + + lowEnc.Add(p.Uid) + + if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { + lowPl.Postings = append(lowPl.Postings, p) + } + return nil + }) + x.Check(err) + lowPl.Pack = lowEnc.Done() + lowPl.CommitTs = l.plist.CommitTs + lowPl.MultiPart = true + lowPl.FirstPart = false + lowPl.StartUid = l.plist.StartUid + lowPl.EndUid = midUid - 1 + + // Generate first list. + lowList := &List{} + if !l.plist.MultiPart { + lowList.key = generateNextPartKey(l.key, l.plist, 0) + } else { + lowList.key = generateNextPartKey(l.key, l.plist, l.plist.StartUid) + } + lowList.minTs = l.minTs + lowList.maxTs = l.maxTs + lowList.plist = lowPl + + // Generate posting list holding the second half of the current list's postings. + highPl := new(pb.PostingList) + highEnc := codec.Encoder{BlockSize: blockSize} + err = l.partIterate(readTs, func(p *pb.Posting) error { + // Skip all postings with an UID less than midUid. + if p.Uid < midUid { + return nil + } + + highEnc.Add(p.Uid) + if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { + highPl.Postings = append(highPl.Postings, p) + } + return nil + }) + x.Check(err) + highPl.Pack = highEnc.Done() + highPl.CommitTs = l.plist.CommitTs + highPl.MultiPart = true + highPl.FirstPart = false + highPl.StartUid = midUid + if !l.plist.MultiPart { + // We are splitting a list that was previously non-split. So the new + // list should end in this part. + highPl.EndUid = math.MaxUint64 + } else { + // Else, this posting list should point to the part the original list + // is pointing to. + highPl.EndUid = l.plist.EndUid + } + + // Generate second list. + highList := &List{} + highList.key = generateNextPartKey(l.key, l.plist, midUid) + highList.minTs = l.minTs + highList.maxTs = l.maxTs + highList.plist = highPl + + // Link the two lists together and to the rest of the list. + lowList.next = highList + highList.next = l.next + + // Add a pointer to the start of the list. + if !l.plist.MultiPart { + lowList.first = l + highList.first = l + } else { + lowList.first = l.first + highList.first = l.first + } + + return []*List{lowList, highList} +} diff --git a/posting/list_test.go b/posting/list_test.go index b9766c705cd..f9c9d82e258 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -414,36 +414,37 @@ func TestAddMutation_mrjn1(t *testing.T) { require.Equal(t, 0, ol.Length(txn.StartTs, 0)) } -func TestMillion(t *testing.T) { - key := x.DataKey("bal", 1331) - ol, err := getNew(key, ps) - require.NoError(t, err) - var commits int - N := int(1e6) - for i := 2; i <= N; i += 2 { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - } - txn := Txn{StartTs: uint64(i)} - addMutationHelper(t, ol, edge, Set, &txn) - require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%10000 == 0 { - // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting - // list. - t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) - require.NoError(t, ol.Rollup(math.MaxUint64)) - } - commits++ - } - t.Logf("Completed a million writes.\n") - opt := ListOptions{ReadTs: uint64(N) + 1} - l, err := ol.Uids(opt) - require.NoError(t, err) - require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) - for i, uid := range l.Uids { - require.Equal(t, uint64(i+1)*2, uid) - } -} +// TODO: size calculations are slow so this test is extremely slow. Fix that. +// func TestMillion(t *testing.T) { +// key := x.DataKey("bal", 1331) +// ol, err := getNew(key, ps) +// require.NoError(t, err) +// var commits int +// N := int(1e6) +// for i := 2; i <= N; i += 2 { +// edge := &pb.DirectedEdge{ +// ValueId: uint64(i), +// } +// txn := Txn{StartTs: uint64(i)} +// addMutationHelper(t, ol, edge, Set, &txn) +// require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) +// if i%10000 == 0 { +// // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting +// // list. +// t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) +// require.NoError(t, ol.Rollup(math.MaxUint64)) +// } +// commits++ +// } +// t.Logf("Completed a million writes.\n") +// opt := ListOptions{ReadTs: uint64(N) + 1} +// l, err := ol.Uids(opt) +// require.NoError(t, err) +// require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) +// for i, uid := range l.Uids { +// require.Equal(t, uint64(i+1)*2, uid) +// } +// } // Test the various mutate, commit and abort sequences. func TestAddMutation_mrjn2(t *testing.T) { @@ -901,3 +902,39 @@ func BenchmarkAddMutations(b *testing.B) { } } } + +func TestMultiPartList(t *testing.T) { + // For testing, set the max list length to a lower threshold. + maxListLength = 2000 + defer func() { + maxListLength = 2000000 + }() + + key := x.DataKey("bal", 1331) + ol, err := getNew(key, ps) + require.NoError(t, err) + var commits int + N := int(1e5) + for i := 2; i <= N; i += 2 { + edge := &pb.DirectedEdge{ + ValueId: uint64(i), + } + txn := Txn{StartTs: uint64(i)} + addMutationHelper(t, ol, edge, Set, &txn) + require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) + if i%2000 == 0 { + // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting + // list. + t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) + require.NoError(t, ol.Rollup(math.MaxUint64)) + } + commits++ + } + opt := ListOptions{ReadTs: uint64(N) + 1} + l, err := ol.Uids(opt) + require.NoError(t, err) + require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) + for i, uid := range l.Uids { + require.Equal(t, uint64(i+1)*2, uid) + } +} diff --git a/posting/mvcc_test.go b/posting/mvcc_test.go index aaa6a333f0c..6a1708b74cf 100644 --- a/posting/mvcc_test.go +++ b/posting/mvcc_test.go @@ -56,9 +56,9 @@ func TestRollupTimestamp(t *testing.T) { // Now check that we don't lost the highest version during a rollup operation, despite the STAR // delete marker being the most recent update. require.NoError(t, nl.Rollup(11)) - kv, err := nl.MarshalToKv() + kvs, err := nl.MarshalToKv() require.NoError(t, err) - require.Equal(t, uint64(10), kv.Version) + require.Equal(t, uint64(10), kvs[0].Version) } func TestPostingListRead(t *testing.T) { diff --git a/protos/pb.proto b/protos/pb.proto index d74a6ca8555..bdd88e1051c 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -277,6 +277,11 @@ message PostingList { UidPack pack = 1; // Encoded list of uids in this posting list. repeated Posting postings = 2; uint64 commit_ts = 3; // More inclination towards smaller values. + + bool multi_part = 4; + bool first_part = 5; + uint64 start_uid = 6; + uint64 end_uid = 7; } message FacetParam { diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index 96d5104a3a4..4de52e1a643 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -2100,6 +2100,10 @@ type PostingList struct { Pack *UidPack `protobuf:"bytes,1,opt,name=pack,proto3" json:"pack,omitempty"` Postings []*Posting `protobuf:"bytes,2,rep,name=postings,proto3" json:"postings,omitempty"` CommitTs uint64 `protobuf:"varint,3,opt,name=commit_ts,json=commitTs,proto3" json:"commit_ts,omitempty"` + MultiPart bool `protobuf:"varint,4,opt,name=multi_part,json=multiPart,proto3" json:"multi_part,omitempty"` + FirstPart bool `protobuf:"varint,5,opt,name=first_part,json=firstPart,proto3" json:"first_part,omitempty"` + StartUid uint64 `protobuf:"varint,6,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` + EndUid uint64 `protobuf:"varint,7,opt,name=end_uid,json=endUid,proto3" json:"end_uid,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2159,6 +2163,34 @@ func (m *PostingList) GetCommitTs() uint64 { return 0 } +func (m *PostingList) GetMultiPart() bool { + if m != nil { + return m.MultiPart + } + return false +} + +func (m *PostingList) GetFirstPart() bool { + if m != nil { + return m.FirstPart + } + return false +} + +func (m *PostingList) GetStartUid() uint64 { + if m != nil { + return m.StartUid + } + return 0 +} + +func (m *PostingList) GetEndUid() uint64 { + if m != nil { + return m.EndUid + } + return 0 +} + type FacetParam struct { Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` Alias string `protobuf:"bytes,2,opt,name=alias,proto3" json:"alias,omitempty"` @@ -3621,214 +3653,218 @@ func init() { func init() { proto.RegisterFile("pb.proto", fileDescriptor_f80abaa17e25ccc8) } var fileDescriptor_f80abaa17e25ccc8 = []byte{ - // 3307 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4b, 0x73, 0x1b, 0x49, - 0x72, 0x66, 0x37, 0x80, 0x46, 0x77, 0x02, 0xa4, 0xb0, 0x25, 0xad, 0x06, 0x43, 0xed, 0x52, 0x9c, - 0x1e, 0x69, 0x44, 0x49, 0x23, 0x4a, 0xc3, 0x19, 0xdb, 0xab, 0x75, 0xf8, 0xc0, 0x07, 0x24, 0x73, - 0x86, 0x2f, 0x17, 0x40, 0xad, 0x3d, 0xe1, 0x30, 0xa2, 0xd0, 0x5d, 0x04, 0x7b, 0xd9, 0xe8, 0x6e, - 0x77, 0x35, 0x18, 0xa0, 0x6e, 0x3e, 0xd8, 0xbf, 0x61, 0x0f, 0x0e, 0xdf, 0x7c, 0xf1, 0xc5, 0xd7, - 0x39, 0xfa, 0xe2, 0x08, 0x1f, 0x1d, 0xfe, 0x05, 0x0e, 0xd9, 0xff, 0xc0, 0x11, 0x8e, 0xf0, 0xcd, - 0x51, 0x59, 0xd5, 0x0f, 0x40, 0xa4, 0x34, 0xe3, 0x08, 0x9f, 0xd0, 0xf9, 0xa8, 0xd7, 0x57, 0x99, - 0x59, 0x99, 0x09, 0xb0, 0x93, 0xd1, 0x66, 0x92, 0xc6, 0x59, 0x4c, 0xcc, 0x64, 0xb4, 0xea, 0xb0, - 0x24, 0x50, 0xe4, 0xea, 0xa3, 0x71, 0x90, 0x9d, 0x4f, 0x47, 0x9b, 0x5e, 0x3c, 0x79, 0xee, 0x8f, - 0x53, 0x96, 0x9c, 0x3f, 0x0b, 0xe2, 0xe7, 0x23, 0xe6, 0x8f, 0x79, 0xfa, 0x3c, 0x19, 0x3d, 0xcf, - 0xc7, 0xb9, 0xab, 0x50, 0x3f, 0x08, 0x44, 0x46, 0x08, 0xd4, 0xa7, 0x81, 0x2f, 0xba, 0xc6, 0x7a, - 0x6d, 0xc3, 0xa2, 0xf8, 0xed, 0x1e, 0x82, 0x33, 0x60, 0xe2, 0xe2, 0x0d, 0x0b, 0xa7, 0x9c, 0x74, - 0xa0, 0x76, 0xc9, 0xc2, 0xae, 0xb1, 0x6e, 0x6c, 0xb4, 0xa9, 0xfc, 0x24, 0x9b, 0x60, 0x5f, 0xb2, - 0x70, 0x98, 0x5d, 0x25, 0xbc, 0x6b, 0xae, 0x1b, 0x1b, 0x2b, 0x5b, 0xb7, 0x37, 0x93, 0xd1, 0xe6, - 0x49, 0x2c, 0xb2, 0x20, 0x1a, 0x6f, 0xbe, 0x61, 0xe1, 0xe0, 0x2a, 0xe1, 0xb4, 0x79, 0xa9, 0x3e, - 0xdc, 0x63, 0x68, 0xf5, 0x53, 0xef, 0xd5, 0x34, 0xf2, 0xb2, 0x20, 0x8e, 0xe4, 0x8a, 0x11, 0x9b, - 0x70, 0x9c, 0xd1, 0xa1, 0xf8, 0x2d, 0x79, 0x2c, 0x1d, 0x8b, 0x6e, 0x6d, 0xbd, 0x26, 0x79, 0xf2, - 0x9b, 0x74, 0xa1, 0x19, 0x88, 0xdd, 0x78, 0x1a, 0x65, 0xdd, 0xfa, 0xba, 0xb1, 0x61, 0xd3, 0x9c, - 0x74, 0xff, 0xcb, 0x84, 0xc6, 0x9f, 0x4c, 0x79, 0x7a, 0x85, 0xe3, 0xb2, 0x2c, 0xcd, 0xe7, 0x92, - 0xdf, 0xe4, 0x0e, 0x34, 0x42, 0x16, 0x8d, 0x45, 0xd7, 0xc4, 0xc9, 0x14, 0x41, 0xee, 0x81, 0xc3, - 0xce, 0x32, 0x9e, 0x0e, 0xa7, 0x81, 0xdf, 0xad, 0xad, 0x1b, 0x1b, 0x16, 0xb5, 0x91, 0x71, 0x1a, - 0xf8, 0xe4, 0x53, 0xb0, 0xfd, 0x78, 0xe8, 0x55, 0xd7, 0xf2, 0x63, 0x5c, 0x8b, 0x7c, 0x0e, 0xf6, - 0x34, 0xf0, 0x87, 0x61, 0x20, 0xb2, 0x6e, 0x63, 0xdd, 0xd8, 0x68, 0x6d, 0xd9, 0xf2, 0xb0, 0x12, - 0x3b, 0xda, 0x9c, 0x06, 0x3e, 0x82, 0xf8, 0x04, 0x6c, 0x91, 0x7a, 0xc3, 0xb3, 0x69, 0xe4, 0x75, - 0x2d, 0x54, 0xba, 0x25, 0x95, 0x2a, 0xa7, 0xa6, 0x4d, 0xa1, 0x08, 0x79, 0xac, 0x94, 0x5f, 0xf2, - 0x54, 0xf0, 0x6e, 0x53, 0x2d, 0xa5, 0x49, 0xf2, 0x02, 0x5a, 0x67, 0xcc, 0xe3, 0xd9, 0x30, 0x61, - 0x29, 0x9b, 0x74, 0xed, 0x72, 0xa2, 0x57, 0x92, 0x7d, 0x22, 0xb9, 0x82, 0xc2, 0x59, 0x41, 0x90, - 0xaf, 0x61, 0x19, 0x29, 0x31, 0x3c, 0x0b, 0xc2, 0x8c, 0xa7, 0x5d, 0x07, 0xc7, 0xac, 0xe0, 0x18, - 0xe4, 0x0c, 0x52, 0xce, 0x69, 0x5b, 0x29, 0x29, 0x0e, 0xf9, 0x25, 0x00, 0x9f, 0x25, 0x2c, 0xf2, - 0x87, 0x2c, 0x0c, 0xbb, 0x80, 0x7b, 0x70, 0x14, 0x67, 0x3b, 0x0c, 0xc9, 0x27, 0x72, 0x7f, 0xcc, - 0x1f, 0x66, 0xa2, 0xbb, 0xbc, 0x6e, 0x6c, 0xd4, 0xa9, 0x25, 0xc9, 0x81, 0x70, 0xb7, 0xc0, 0x41, - 0x8b, 0xc0, 0x13, 0x3f, 0x04, 0xeb, 0x52, 0x12, 0xca, 0x70, 0x5a, 0x5b, 0xcb, 0x72, 0xc9, 0xc2, - 0x68, 0xa8, 0x16, 0xba, 0x6b, 0x60, 0x1f, 0xb0, 0x68, 0x9c, 0x5b, 0x9a, 0xbc, 0x0a, 0x1c, 0xe0, - 0x50, 0xfc, 0x76, 0x7f, 0x67, 0x82, 0x45, 0xb9, 0x98, 0x86, 0x19, 0x79, 0x04, 0x20, 0x81, 0x9e, - 0xb0, 0x2c, 0x0d, 0x66, 0x7a, 0xd6, 0x12, 0x6a, 0x67, 0x1a, 0xf8, 0x87, 0x28, 0x22, 0x2f, 0xa0, - 0x8d, 0xb3, 0xe7, 0xaa, 0x66, 0xb9, 0x81, 0x62, 0x7f, 0xb4, 0x85, 0x2a, 0x7a, 0xc4, 0x5d, 0xb0, - 0xf0, 0x6e, 0x95, 0x7d, 0x2d, 0x53, 0x4d, 0x91, 0x87, 0xb0, 0x12, 0x44, 0x99, 0xc4, 0xde, 0xcb, - 0x86, 0x3e, 0x17, 0xf9, 0xe5, 0x2f, 0x17, 0xdc, 0x3d, 0x2e, 0x32, 0xf2, 0x15, 0x28, 0x00, 0xf3, - 0x05, 0x1b, 0xb8, 0xe0, 0x4a, 0x71, 0x31, 0x42, 0xad, 0x88, 0x3a, 0x7a, 0xc5, 0x67, 0xd0, 0x92, - 0xe7, 0xcb, 0x47, 0x58, 0x38, 0xa2, 0x8d, 0xa7, 0xd1, 0x70, 0x50, 0x90, 0x0a, 0x5a, 0x5d, 0x42, - 0x23, 0x0d, 0x4c, 0x19, 0x04, 0x7e, 0xbb, 0x3d, 0x68, 0x1c, 0xa7, 0x3e, 0x4f, 0xaf, 0xb5, 0x71, - 0x02, 0x75, 0x9f, 0x0b, 0x0f, 0xdd, 0xcf, 0xa6, 0xf8, 0x5d, 0xda, 0x7d, 0xad, 0x62, 0xf7, 0xee, - 0xdf, 0x19, 0xd0, 0xea, 0xc7, 0x69, 0x76, 0xc8, 0x85, 0x60, 0x63, 0x4e, 0xee, 0x43, 0x23, 0x96, - 0xd3, 0x6a, 0x84, 0x1d, 0xb9, 0x27, 0x5c, 0x87, 0x2a, 0xfe, 0xc2, 0x3d, 0x98, 0x37, 0xdf, 0xc3, - 0x1d, 0x68, 0x28, 0x8f, 0x91, 0xde, 0xd4, 0xa0, 0x8a, 0x90, 0x58, 0xc7, 0x67, 0x67, 0x82, 0x2b, - 0x2c, 0x1b, 0x54, 0x53, 0x37, 0x9b, 0xd5, 0xef, 0x01, 0xc8, 0xfd, 0xfd, 0x44, 0x2b, 0x70, 0xcf, - 0xa1, 0x45, 0xd9, 0x59, 0xb6, 0x1b, 0x47, 0x19, 0x9f, 0x65, 0x64, 0x05, 0xcc, 0xc0, 0x47, 0x88, - 0x2c, 0x6a, 0x06, 0xbe, 0xdc, 0xdc, 0x38, 0x8d, 0xa7, 0x09, 0x22, 0xb4, 0x4c, 0x15, 0x81, 0x50, - 0xfa, 0x7e, 0x8a, 0x3b, 0x96, 0x50, 0xfa, 0x7e, 0x4a, 0xee, 0x43, 0x4b, 0x44, 0x2c, 0x11, 0xe7, - 0x71, 0x26, 0x37, 0x57, 0xc7, 0xcd, 0x41, 0xce, 0x1a, 0x08, 0xf7, 0x9f, 0x0d, 0xb0, 0x0e, 0xf9, - 0x64, 0xc4, 0xd3, 0xf7, 0x56, 0xf9, 0x14, 0x6c, 0x9c, 0x78, 0x18, 0xf8, 0x7a, 0xa1, 0x26, 0xd2, - 0xfb, 0xfe, 0xb5, 0x4b, 0xdd, 0x05, 0x2b, 0xe4, 0x4c, 0x82, 0xaf, 0xec, 0x4c, 0x53, 0x12, 0x1b, - 0x36, 0x19, 0xfa, 0x9c, 0xf9, 0x18, 0x62, 0x6c, 0x6a, 0xb1, 0xc9, 0x1e, 0x67, 0xbe, 0xdc, 0x5b, - 0xc8, 0x44, 0x36, 0x9c, 0x26, 0x3e, 0xcb, 0x38, 0x86, 0x96, 0xba, 0x34, 0x1c, 0x91, 0x9d, 0x22, - 0x87, 0x3c, 0x81, 0x9f, 0x79, 0xe1, 0x54, 0xc8, 0xb8, 0x16, 0x44, 0x67, 0xf1, 0x30, 0x8e, 0xc2, - 0x2b, 0xc4, 0xd7, 0xa6, 0xb7, 0xb4, 0x60, 0x3f, 0x3a, 0x8b, 0x8f, 0xa3, 0xf0, 0xca, 0xfd, 0xc1, - 0x84, 0xc6, 0x6b, 0x84, 0xe1, 0x05, 0x34, 0x27, 0x78, 0xa0, 0xdc, 0x7b, 0xef, 0x4a, 0x84, 0x51, - 0xb6, 0xa9, 0x4e, 0x2a, 0x7a, 0x51, 0x96, 0x5e, 0xd1, 0x5c, 0x4d, 0x8e, 0xc8, 0xd8, 0x28, 0xe4, - 0x99, 0xd0, 0x16, 0x51, 0x19, 0x31, 0x50, 0x02, 0x3d, 0x42, 0xab, 0x2d, 0xc2, 0x5a, 0x5b, 0x84, - 0x95, 0xac, 0x82, 0xed, 0x9d, 0x73, 0xef, 0x42, 0x4c, 0x27, 0x1a, 0xf4, 0x82, 0x5e, 0x7d, 0x05, - 0xed, 0xea, 0x3e, 0xe4, 0x1b, 0x74, 0xc1, 0xaf, 0x10, 0xf8, 0x3a, 0x95, 0x9f, 0x64, 0x1d, 0x1a, - 0xe8, 0xe1, 0x08, 0x7b, 0x6b, 0x0b, 0xe4, 0x76, 0xd4, 0x10, 0xaa, 0x04, 0xbf, 0x36, 0x7f, 0x65, - 0xc8, 0x79, 0xaa, 0xbb, 0xab, 0xce, 0xe3, 0xdc, 0x3c, 0x8f, 0x1a, 0x52, 0x99, 0xc7, 0xfd, 0x1f, - 0x13, 0xda, 0xdf, 0xf3, 0x34, 0x3e, 0x49, 0xe3, 0x24, 0x16, 0x2c, 0x24, 0xdb, 0xf3, 0xa7, 0x53, - 0x28, 0xae, 0xcb, 0xc1, 0x55, 0xb5, 0xcd, 0x7e, 0x71, 0x5c, 0x85, 0x4e, 0xf5, 0xfc, 0x2e, 0x58, - 0x0a, 0xdd, 0x6b, 0x8e, 0xa0, 0x25, 0x52, 0x47, 0xe1, 0x89, 0xf8, 0xcd, 0x6f, 0x4f, 0x4b, 0xc8, - 0x1a, 0xc0, 0x84, 0xcd, 0x0e, 0x38, 0x13, 0x7c, 0xdf, 0xcf, 0xcd, 0xb7, 0xe4, 0x48, 0x9c, 0x27, - 0x6c, 0x36, 0x98, 0x45, 0x03, 0x81, 0xd6, 0x55, 0xa7, 0x05, 0x4d, 0x7e, 0x01, 0xce, 0x84, 0xcd, - 0xa4, 0x1f, 0xed, 0xfb, 0xda, 0xba, 0x4a, 0x06, 0xf9, 0x0c, 0x6a, 0xd9, 0x2c, 0xc2, 0xa0, 0x24, - 0xdf, 0x21, 0x99, 0x64, 0x0c, 0x66, 0x91, 0xf6, 0x38, 0x2a, 0x65, 0x39, 0xa0, 0x76, 0x09, 0x68, - 0x07, 0x6a, 0x5e, 0xe0, 0xe3, 0x43, 0xe4, 0x50, 0xf9, 0xb9, 0xfa, 0x47, 0x70, 0x6b, 0x01, 0x87, - 0xea, 0x3d, 0x2c, 0xab, 0x61, 0x77, 0xaa, 0xf7, 0x50, 0xaf, 0x62, 0xff, 0x43, 0x0d, 0x6e, 0x69, - 0x63, 0x38, 0x0f, 0x92, 0x7e, 0x26, 0xcd, 0xbe, 0x0b, 0x4d, 0x8c, 0x36, 0x3c, 0xd5, 0x36, 0x91, - 0x93, 0xe4, 0x0f, 0xc0, 0x42, 0x0f, 0xcc, 0xed, 0xf4, 0x7e, 0x89, 0x6a, 0x31, 0x5c, 0xd9, 0xad, - 0xbe, 0x12, 0xad, 0x4e, 0xbe, 0x81, 0xc6, 0x5b, 0x9e, 0xc6, 0x2a, 0x7a, 0xb6, 0xb6, 0xd6, 0xae, - 0x1b, 0x27, 0xef, 0x56, 0x0f, 0x53, 0xca, 0xff, 0x8f, 0xe0, 0x3f, 0x90, 0xf1, 0x72, 0x12, 0x5f, - 0x72, 0xbf, 0xdb, 0xc4, 0x1d, 0x55, 0xed, 0x23, 0x17, 0xe5, 0x68, 0xdb, 0x25, 0xda, 0x7b, 0xd0, - 0xaa, 0x1c, 0xef, 0x1a, 0xa4, 0xef, 0xcf, 0x5b, 0xbc, 0x53, 0x38, 0x72, 0xd5, 0x71, 0xf6, 0x00, - 0xca, 0xc3, 0xfe, 0x5f, 0xdd, 0xcf, 0xfd, 0x2b, 0x03, 0x6e, 0xed, 0xc6, 0x51, 0xc4, 0x31, 0x05, - 0x52, 0x57, 0x57, 0x9a, 0xbd, 0x71, 0xa3, 0xd9, 0x3f, 0x86, 0x86, 0x90, 0xca, 0x7a, 0xf6, 0xdb, - 0xd7, 0xdc, 0x05, 0x55, 0x1a, 0x32, 0xcc, 0x4c, 0xd8, 0x6c, 0x98, 0xf0, 0xc8, 0x0f, 0xa2, 0x71, - 0x1e, 0x66, 0x26, 0x6c, 0x76, 0xa2, 0x38, 0xee, 0x5f, 0x1b, 0x60, 0x29, 0x8f, 0x99, 0x8b, 0xd6, - 0xc6, 0x7c, 0xb4, 0xfe, 0x05, 0x38, 0x49, 0xca, 0xfd, 0xc0, 0xcb, 0x57, 0x75, 0x68, 0xc9, 0x90, - 0xc6, 0x79, 0x16, 0xa7, 0x1e, 0xc7, 0xe9, 0x6d, 0xaa, 0x08, 0xc9, 0x15, 0x09, 0xf3, 0x54, 0x1a, - 0x57, 0xa3, 0x8a, 0x90, 0x31, 0x5e, 0x5d, 0x0e, 0x5e, 0x8a, 0x4d, 0x35, 0xe5, 0xfe, 0x83, 0x09, - 0xed, 0xbd, 0x20, 0xe5, 0x5e, 0xc6, 0xfd, 0x9e, 0x3f, 0x46, 0x45, 0x1e, 0x65, 0x41, 0x76, 0xa5, - 0xdf, 0x13, 0x4d, 0x15, 0xcf, 0xbd, 0x39, 0x9f, 0xd2, 0x2a, 0xb8, 0x6b, 0x98, 0x85, 0x2b, 0x82, - 0x6c, 0x01, 0xa8, 0x44, 0x08, 0x33, 0xf1, 0xfa, 0xcd, 0x99, 0xb8, 0x83, 0x6a, 0xf2, 0x53, 0x62, - 0xa0, 0xc6, 0x04, 0xea, 0xad, 0xb1, 0x30, 0x4d, 0x9f, 0x4a, 0x5b, 0xc5, 0xfc, 0x61, 0xc4, 0x43, - 0xb4, 0x45, 0xcc, 0x1f, 0x46, 0x3c, 0x2c, 0xb2, 0xb6, 0xa6, 0xda, 0x8e, 0xfc, 0x26, 0x9f, 0x83, - 0x19, 0x27, 0x78, 0x3e, 0xbd, 0x60, 0xf5, 0x60, 0x9b, 0xc7, 0x09, 0x35, 0xe3, 0x44, 0x5e, 0xb4, - 0x4a, 0x3b, 0xbb, 0x8e, 0xb6, 0x5f, 0x19, 0x40, 0x30, 0x61, 0xa2, 0x5a, 0xe2, 0xde, 0x05, 0xf3, - 0x38, 0x21, 0x4d, 0xa8, 0xf5, 0x7b, 0x83, 0xce, 0x92, 0xfc, 0xd8, 0xeb, 0x1d, 0x74, 0x0c, 0xf7, - 0x9d, 0x01, 0xce, 0xe1, 0x34, 0x63, 0xd2, 0x6c, 0xc4, 0x87, 0xee, 0xed, 0x53, 0xb0, 0x45, 0xc6, - 0x52, 0x0c, 0xc2, 0x2a, 0x72, 0x34, 0x91, 0x1e, 0x08, 0xf2, 0x05, 0x34, 0xb8, 0x3f, 0xe6, 0xb9, - 0x43, 0x77, 0x16, 0xf7, 0x49, 0x95, 0x98, 0x6c, 0x80, 0x25, 0xbc, 0x73, 0x3e, 0x61, 0xdd, 0x7a, - 0xa9, 0xd8, 0x47, 0x8e, 0x7a, 0x64, 0xa9, 0x96, 0x63, 0x95, 0x90, 0xc6, 0x09, 0xa6, 0xcd, 0x0d, - 0x5d, 0x25, 0xa4, 0x71, 0x22, 0x93, 0xe6, 0x2d, 0xf8, 0x79, 0x30, 0x8e, 0xe2, 0x94, 0x0f, 0x83, - 0xc8, 0xe7, 0xb3, 0xa1, 0x17, 0x47, 0x67, 0x61, 0xe0, 0x65, 0x88, 0xa5, 0x4d, 0x6f, 0x2b, 0xe1, - 0xbe, 0x94, 0xed, 0x6a, 0x91, 0x3b, 0x03, 0x3b, 0x8f, 0x8b, 0xe4, 0xb1, 0x0c, 0x68, 0x18, 0x57, - 0xb5, 0x5b, 0x60, 0xda, 0x5f, 0x49, 0x70, 0x68, 0x2e, 0x97, 0xd7, 0x84, 0x6b, 0xe4, 0x91, 0x12, - 0x89, 0x6a, 0x7a, 0x55, 0xab, 0xa6, 0x57, 0x98, 0x29, 0xc6, 0x11, 0xd7, 0x19, 0x07, 0x7e, 0xbb, - 0x7f, 0x6b, 0x82, 0x5d, 0x3c, 0x65, 0x4f, 0xc1, 0x99, 0xe4, 0x50, 0x6b, 0x87, 0xc3, 0x5c, 0xba, - 0xc0, 0x9f, 0x96, 0x72, 0x72, 0x17, 0xcc, 0x8b, 0x4b, 0x0d, 0x94, 0x25, 0xb5, 0xbe, 0x7b, 0x43, - 0xcd, 0x8b, 0xcb, 0xd2, 0x63, 0x1b, 0x1f, 0xf5, 0xd8, 0x47, 0x70, 0xcb, 0x0b, 0x39, 0x8b, 0x86, - 0xa5, 0xc3, 0x29, 0x83, 0x5b, 0x41, 0xf6, 0x49, 0xe1, 0x75, 0x3a, 0xea, 0x34, 0xcb, 0xb7, 0xe5, - 0x21, 0x34, 0x7c, 0x1e, 0x66, 0xac, 0x5a, 0x1a, 0x1d, 0xa7, 0xcc, 0x0b, 0xf9, 0x9e, 0x64, 0x53, - 0x25, 0x25, 0x1b, 0x60, 0xe7, 0xef, 0xac, 0x2e, 0x88, 0x30, 0xf3, 0xce, 0xc1, 0xa6, 0x85, 0xb4, - 0xc4, 0x12, 0x2a, 0x58, 0xba, 0x5f, 0x41, 0xed, 0xbb, 0x37, 0x7d, 0x7d, 0x56, 0xe3, 0xbd, 0xb3, - 0xe6, 0x88, 0x9a, 0x15, 0x44, 0xff, 0xbb, 0x06, 0x4d, 0xed, 0x75, 0x72, 0xdf, 0xd3, 0x22, 0x4b, - 0x94, 0x9f, 0xf3, 0x8f, 0x5b, 0xe1, 0xbe, 0xd5, 0x32, 0xba, 0xf6, 0xf1, 0x32, 0x9a, 0xfc, 0x1a, - 0xda, 0x89, 0x92, 0x55, 0x1d, 0xfe, 0x93, 0xea, 0x18, 0xfd, 0x8b, 0xe3, 0x5a, 0x49, 0x49, 0x48, - 0xd3, 0xc5, 0x7a, 0x24, 0x63, 0x63, 0xbc, 0xa2, 0x36, 0x6d, 0x4a, 0x7a, 0xc0, 0xc6, 0x37, 0xb8, - 0xfd, 0x8f, 0xf0, 0x5e, 0x99, 0x0d, 0xc7, 0x49, 0xb7, 0x8d, 0x1e, 0x29, 0x3d, 0xbe, 0xea, 0x8c, - 0xcb, 0xf3, 0xce, 0x78, 0x0f, 0x1c, 0x2f, 0x9e, 0x4c, 0x02, 0x94, 0xad, 0xe8, 0x6c, 0x0f, 0x19, - 0x03, 0xe1, 0xfe, 0x8d, 0x01, 0x4d, 0x7d, 0x5a, 0xd2, 0x82, 0xe6, 0x5e, 0xef, 0xd5, 0xf6, 0xe9, - 0x81, 0x8c, 0x07, 0x00, 0xd6, 0xce, 0xfe, 0xd1, 0x36, 0xfd, 0xb3, 0x8e, 0x21, 0x63, 0xc3, 0xfe, - 0xd1, 0xa0, 0x63, 0x12, 0x07, 0x1a, 0xaf, 0x0e, 0x8e, 0xb7, 0x07, 0x9d, 0x1a, 0xb1, 0xa1, 0xbe, - 0x73, 0x7c, 0x7c, 0xd0, 0xa9, 0x93, 0x36, 0xd8, 0x7b, 0xdb, 0x83, 0xde, 0x60, 0xff, 0xb0, 0xd7, - 0x69, 0x48, 0xdd, 0xd7, 0xbd, 0xe3, 0x8e, 0x25, 0x3f, 0x4e, 0xf7, 0xf7, 0x3a, 0x4d, 0x29, 0x3f, - 0xd9, 0xee, 0xf7, 0x7f, 0x73, 0x4c, 0xf7, 0x3a, 0xb6, 0x9c, 0xb7, 0x3f, 0xa0, 0xfb, 0x47, 0xaf, - 0x3b, 0x8e, 0xfc, 0x3e, 0xde, 0xf9, 0xb6, 0xb7, 0x3b, 0xe8, 0x80, 0xfb, 0x15, 0xb4, 0x2a, 0x08, - 0xca, 0xd1, 0xb4, 0xf7, 0xaa, 0xb3, 0x24, 0x97, 0x7c, 0xb3, 0x7d, 0x70, 0xda, 0xeb, 0x18, 0x64, - 0x05, 0x00, 0x3f, 0x87, 0x07, 0xdb, 0x47, 0xaf, 0x3b, 0xa6, 0xfb, 0xfb, 0x60, 0x9f, 0x06, 0xfe, - 0x4e, 0x18, 0x7b, 0x17, 0xd2, 0x30, 0x46, 0x4c, 0x70, 0xfd, 0x4e, 0xe2, 0xb7, 0x8c, 0xf2, 0x68, - 0x94, 0x42, 0xdf, 0xbd, 0xa6, 0xdc, 0x23, 0x68, 0x9e, 0x06, 0xfe, 0x09, 0xf3, 0x2e, 0x64, 0x3d, - 0x3e, 0x92, 0xe3, 0x87, 0x22, 0x78, 0xcb, 0x75, 0x80, 0x73, 0x90, 0xd3, 0x0f, 0xde, 0x72, 0xf2, - 0x00, 0x2c, 0x24, 0xf2, 0x8c, 0x06, 0x6d, 0x39, 0x5f, 0x93, 0x6a, 0x99, 0x9b, 0x15, 0x5b, 0xc7, - 0x5a, 0xfb, 0x3e, 0xd4, 0x13, 0xe6, 0x5d, 0xe8, 0x60, 0xd2, 0xd2, 0x43, 0xe4, 0x72, 0x14, 0x05, - 0xe4, 0x11, 0xd8, 0xda, 0x3e, 0xf2, 0x79, 0x5b, 0x15, 0x43, 0xa2, 0x85, 0x70, 0xfe, 0xe6, 0x6a, - 0x0b, 0x37, 0xf7, 0x0d, 0x40, 0xd9, 0x9a, 0xb8, 0x26, 0xbb, 0xbe, 0x03, 0x0d, 0x16, 0x06, 0xfa, - 0xf0, 0x0e, 0x55, 0x84, 0x7b, 0x04, 0xad, 0x4a, 0x43, 0x43, 0x9a, 0x0d, 0x0b, 0xc3, 0xe1, 0x05, - 0xbf, 0x12, 0x38, 0xd6, 0xa6, 0x4d, 0x16, 0x86, 0xdf, 0xf1, 0x2b, 0x41, 0x1e, 0x40, 0x43, 0xf5, - 0x42, 0xcc, 0x85, 0x92, 0x1b, 0x87, 0x52, 0x25, 0x74, 0xbf, 0x04, 0x4b, 0xd5, 0xe1, 0x15, 0xab, - 0x35, 0x6e, 0x7c, 0x73, 0x5e, 0xea, 0x3d, 0x63, 0xd5, 0x4e, 0x9e, 0xea, 0x9e, 0x8b, 0x50, 0x1d, - 0x1e, 0xa3, 0x4c, 0xb5, 0x94, 0x92, 0x6e, 0xb7, 0xa0, 0xb2, 0xbb, 0x07, 0xf6, 0x07, 0xbb, 0x58, - 0x1a, 0x00, 0xb3, 0x04, 0xe0, 0x9a, 0xbe, 0x96, 0xfb, 0x5b, 0x80, 0xb2, 0x37, 0xa3, 0x9d, 0x48, - 0xcd, 0x22, 0x9d, 0xe8, 0x89, 0x2c, 0x8b, 0x82, 0xd0, 0x4f, 0x79, 0x34, 0x77, 0xea, 0xb2, 0x9b, - 0x53, 0xc8, 0xc9, 0x3a, 0xd4, 0xb1, 0xe5, 0x54, 0x2b, 0x83, 0x5c, 0xd1, 0x6f, 0x42, 0x89, 0x3b, - 0x82, 0x65, 0xf5, 0x94, 0x51, 0xfe, 0x97, 0x53, 0x2e, 0x3e, 0x98, 0x03, 0xad, 0x01, 0x14, 0x21, - 0x39, 0x6f, 0x9e, 0x55, 0x38, 0xd2, 0x94, 0xcf, 0x02, 0x1e, 0xfa, 0xf9, 0x69, 0x34, 0xe5, 0xfe, - 0x21, 0xb4, 0xf3, 0x35, 0xb0, 0x84, 0x7f, 0x5a, 0x3c, 0xa8, 0x0a, 0x4d, 0x55, 0x39, 0x28, 0x95, - 0xa3, 0xd8, 0xe7, 0x3b, 0x66, 0xd7, 0xc8, 0xdf, 0x54, 0xf7, 0xdf, 0x6a, 0xf9, 0x68, 0x5d, 0xd1, - 0xce, 0x65, 0x62, 0xc6, 0x62, 0x26, 0x36, 0x9f, 0xf2, 0x98, 0x3f, 0x2a, 0xe5, 0xf9, 0x15, 0x38, - 0x3e, 0xbe, 0xfb, 0xc1, 0x65, 0x1e, 0x68, 0x57, 0x17, 0xdf, 0x78, 0x9d, 0x19, 0x04, 0x97, 0x9c, - 0x96, 0xca, 0x72, 0x2f, 0x59, 0x7c, 0xc1, 0xa3, 0xe0, 0x2d, 0x96, 0xec, 0xf2, 0xd0, 0x25, 0xa3, - 0xec, 0x7f, 0xa8, 0x5c, 0x40, 0xf7, 0x3f, 0xf2, 0x56, 0x8e, 0x55, 0xb6, 0x72, 0x24, 0x72, 0xd3, - 0x44, 0xf0, 0x34, 0xcb, 0x73, 0x42, 0x45, 0x15, 0xb9, 0x95, 0xa3, 0x75, 0x65, 0x6e, 0xf5, 0x19, - 0xb4, 0xa3, 0x38, 0x1a, 0x46, 0xd3, 0x30, 0x94, 0x59, 0xab, 0xee, 0xcf, 0xb5, 0xa2, 0x38, 0x3a, - 0xd2, 0x2c, 0x59, 0xf4, 0x57, 0x55, 0x94, 0xe5, 0xb6, 0x54, 0xd1, 0x5f, 0xd1, 0x43, 0xfb, 0xde, - 0x80, 0x4e, 0x3c, 0xfa, 0x2d, 0xf7, 0x32, 0x44, 0x6c, 0x88, 0x26, 0xdb, 0x56, 0xcf, 0xad, 0xe2, - 0x4b, 0x88, 0x8e, 0xd8, 0x84, 0xbb, 0x2f, 0xc1, 0x29, 0x40, 0x90, 0x91, 0xf5, 0xe8, 0xf8, 0xa8, - 0xa7, 0x62, 0xdf, 0xfe, 0xd1, 0x5e, 0xef, 0x4f, 0x3b, 0x86, 0x8c, 0xcd, 0xb4, 0xf7, 0xa6, 0x47, - 0xfb, 0xbd, 0x8e, 0x29, 0xe3, 0xe6, 0x5e, 0xef, 0xa0, 0x37, 0xe8, 0x75, 0x6a, 0xdf, 0xd6, 0xed, - 0x66, 0xc7, 0xa6, 0x36, 0x9f, 0x25, 0x61, 0xe0, 0x05, 0x99, 0xdb, 0x07, 0x90, 0xd3, 0xea, 0x1b, - 0xbd, 0x07, 0x4e, 0xb9, 0xb6, 0xba, 0x51, 0x3b, 0xd3, 0xab, 0xca, 0xec, 0x4b, 0x1b, 0x95, 0x79, - 0x53, 0xf6, 0xa5, 0xcd, 0xec, 0x14, 0xec, 0x43, 0x96, 0xbc, 0x57, 0x90, 0xb4, 0x8b, 0xb2, 0x73, - 0xaa, 0x9b, 0x30, 0xfa, 0xd1, 0x7d, 0x08, 0x4d, 0x1d, 0xc4, 0xb4, 0x7f, 0xcc, 0x05, 0xb8, 0x5c, - 0x26, 0xeb, 0x83, 0x3b, 0x87, 0xf1, 0x25, 0x2f, 0xf2, 0x8e, 0x13, 0x76, 0x15, 0xc6, 0xcc, 0xff, - 0x88, 0x21, 0xfe, 0x12, 0x40, 0xc4, 0xd3, 0xd4, 0xe3, 0xc3, 0x71, 0xd1, 0xfb, 0x71, 0x14, 0xe7, - 0xb5, 0x6e, 0x28, 0x73, 0x91, 0xa1, 0xb0, 0xa6, 0xdc, 0x4c, 0xd2, 0x52, 0xf4, 0x73, 0xb0, 0xb2, - 0x59, 0x54, 0xb6, 0x9a, 0x1a, 0x99, 0xac, 0x06, 0xdd, 0x5d, 0x70, 0x06, 0x33, 0xac, 0x91, 0xa6, - 0x62, 0xee, 0x25, 0x35, 0x3e, 0xf0, 0x92, 0x9a, 0x0b, 0xf1, 0xf8, 0x3f, 0x0d, 0x68, 0x55, 0x12, - 0x22, 0xf2, 0x19, 0xd4, 0xb3, 0x59, 0x34, 0xdf, 0xa3, 0xcd, 0x17, 0xa1, 0x28, 0x92, 0xf6, 0x26, - 0x0b, 0x28, 0x26, 0x44, 0x30, 0x8e, 0xb8, 0xaf, 0xa7, 0x94, 0x45, 0xd5, 0xb6, 0x66, 0x91, 0x03, - 0xb8, 0xa5, 0x62, 0x46, 0xde, 0x9f, 0xc9, 0x73, 0xea, 0xcf, 0x17, 0x12, 0x30, 0x55, 0x47, 0xee, - 0xe6, 0x5a, 0xaa, 0x52, 0x5e, 0x19, 0xcf, 0x31, 0x57, 0xb7, 0xe1, 0xf6, 0x35, 0x6a, 0x3f, 0xa9, - 0x25, 0x70, 0x1f, 0x96, 0x65, 0x09, 0x1d, 0x4c, 0xb8, 0xc8, 0xd8, 0x24, 0xc1, 0x4c, 0x44, 0xc7, - 0xfc, 0x3a, 0x35, 0x33, 0xe1, 0x7e, 0x01, 0xed, 0x13, 0xce, 0x53, 0xca, 0x45, 0x12, 0x47, 0xea, - 0x15, 0x16, 0x78, 0x68, 0xfd, 0xc0, 0x68, 0xca, 0xfd, 0x0b, 0x70, 0x64, 0x8e, 0xbd, 0xc3, 0x32, - 0xef, 0xfc, 0xa7, 0xe4, 0xe0, 0x5f, 0x40, 0x33, 0x51, 0x66, 0xa2, 0x33, 0xe6, 0x36, 0xc6, 0x38, - 0x6d, 0x3a, 0x34, 0x17, 0xba, 0xdf, 0x40, 0xed, 0x68, 0x3a, 0xa9, 0xfe, 0x85, 0x52, 0x57, 0x7f, - 0xa1, 0xdc, 0x03, 0x07, 0xd3, 0x75, 0xec, 0xd7, 0xa9, 0x44, 0xd2, 0x96, 0x0c, 0x6c, 0xd4, 0x7d, - 0x0f, 0xad, 0x1c, 0xfb, 0x7d, 0x1f, 0xff, 0x07, 0xc1, 0xcb, 0xdf, 0xf7, 0xe7, 0x6c, 0x41, 0x55, - 0x6c, 0x3c, 0xf2, 0xf7, 0xf3, 0x4b, 0x53, 0xc4, 0xfc, 0xdc, 0xba, 0xe9, 0x50, 0xcc, 0xfd, 0x0a, - 0xda, 0x79, 0x1e, 0x7c, 0xc8, 0x33, 0x86, 0xe6, 0x14, 0x06, 0x3c, 0xaa, 0x98, 0x9a, 0xad, 0x18, - 0x03, 0xf1, 0x81, 0xf6, 0xa6, 0xbb, 0x09, 0x96, 0xb6, 0x55, 0x02, 0x75, 0x2f, 0xf6, 0x95, 0x8b, - 0x34, 0x28, 0x7e, 0xcb, 0x03, 0x4f, 0xc4, 0x38, 0x7f, 0x08, 0x27, 0x62, 0xec, 0xce, 0x60, 0x79, - 0x87, 0x79, 0x17, 0xd3, 0x24, 0x7f, 0x88, 0x2a, 0x05, 0x8b, 0x31, 0x57, 0xb0, 0x7c, 0xa0, 0xa7, - 0xfa, 0x09, 0x34, 0xa7, 0x51, 0x30, 0xcb, 0x33, 0x11, 0x87, 0x5a, 0x92, 0x54, 0xbd, 0xc4, 0x30, - 0xf6, 0xb0, 0x46, 0x41, 0xaf, 0x72, 0x68, 0x41, 0xbb, 0x7f, 0x0e, 0xcb, 0xbd, 0x59, 0x82, 0x1d, - 0xe6, 0x8f, 0x3e, 0x81, 0x95, 0x4d, 0x99, 0x73, 0x9b, 0x5a, 0x58, 0xb9, 0x96, 0xaf, 0xbc, 0xf5, - 0x4f, 0x06, 0xd4, 0xa5, 0x89, 0xc8, 0x32, 0xea, 0x8f, 0x39, 0x4b, 0xb3, 0x11, 0x67, 0x19, 0x99, - 0x33, 0x87, 0xd5, 0x39, 0xca, 0x5d, 0x7a, 0x61, 0x90, 0x2f, 0x55, 0xf3, 0x3a, 0xef, 0xc9, 0x2f, - 0xe7, 0x86, 0x86, 0x86, 0xb8, 0xa8, 0x4f, 0x36, 0xa1, 0xf5, 0x6d, 0x1c, 0x44, 0xbb, 0xaa, 0x9f, - 0x4b, 0x16, 0xcd, 0xf2, 0x3d, 0xfd, 0x67, 0x60, 0xed, 0x0b, 0x69, 0xff, 0xef, 0xab, 0x62, 0x78, - 0xad, 0xba, 0x86, 0xbb, 0xb4, 0xf5, 0x8f, 0x35, 0xa8, 0x7f, 0xcf, 0xd3, 0x98, 0x7c, 0x09, 0x4d, - 0xdd, 0xad, 0x21, 0x95, 0xae, 0xcc, 0x2a, 0xbe, 0xaf, 0x0b, 0x6d, 0x1c, 0x5c, 0xa5, 0xa3, 0x22, - 0x74, 0x59, 0xe7, 0x91, 0xb2, 0x99, 0xf4, 0xde, 0xa6, 0x5e, 0x42, 0xa7, 0x9f, 0xa5, 0x9c, 0x4d, - 0x2a, 0xea, 0xf3, 0x30, 0x5d, 0x57, 0x34, 0x22, 0x5a, 0x4f, 0xc1, 0x52, 0x41, 0x66, 0x61, 0xc0, - 0x62, 0xfd, 0x87, 0xca, 0x8f, 0xa0, 0xd5, 0x3f, 0x8f, 0xa7, 0xa1, 0xdf, 0xe7, 0xe9, 0x25, 0x27, - 0x95, 0x8e, 0xe9, 0x6a, 0xe5, 0xdb, 0x5d, 0x22, 0x1b, 0x00, 0xca, 0xcb, 0x4e, 0x03, 0x5f, 0x90, - 0xa6, 0x94, 0x1d, 0x4d, 0x27, 0x6a, 0xd2, 0x8a, 0xfb, 0x29, 0xcd, 0x4a, 0xac, 0xf9, 0x90, 0xe6, - 0xd7, 0xb0, 0xbc, 0x8b, 0xb1, 0xf8, 0x38, 0xdd, 0x1e, 0xc5, 0x69, 0x46, 0x16, 0xbb, 0xa6, 0xab, - 0x8b, 0x0c, 0x77, 0x89, 0xbc, 0x00, 0x7b, 0x90, 0x5e, 0x29, 0xfd, 0x9f, 0xe9, 0x10, 0x5d, 0xae, - 0x77, 0xcd, 0x29, 0xb7, 0xfe, 0xbe, 0x06, 0xd6, 0x6f, 0xe2, 0xf4, 0x82, 0xa7, 0xe4, 0x09, 0x58, - 0x58, 0xa8, 0x6b, 0x23, 0x2a, 0x8a, 0xf6, 0xeb, 0x16, 0x7a, 0x00, 0x0e, 0x82, 0x32, 0x60, 0xe2, - 0x42, 0x5d, 0x15, 0xfe, 0x89, 0xaa, 0x70, 0x51, 0xc9, 0x1b, 0xde, 0xeb, 0x8a, 0xba, 0xa8, 0xa2, - 0x39, 0x31, 0x57, 0x3d, 0xaf, 0x36, 0x55, 0x29, 0xdc, 0x77, 0x97, 0x36, 0x8c, 0x17, 0x06, 0x79, - 0x0c, 0xf5, 0xbe, 0x3a, 0xa9, 0x54, 0x2a, 0xff, 0x68, 0x5a, 0x5d, 0xc9, 0x19, 0xc5, 0xcc, 0xcf, - 0xc1, 0x52, 0x2f, 0xbb, 0x3a, 0xe6, 0x5c, 0x62, 0xba, 0xda, 0xa9, 0xb2, 0xf4, 0x80, 0xc7, 0x60, - 0xa9, 0xa0, 0xa1, 0x06, 0xcc, 0x05, 0x10, 0xb5, 0x6b, 0x15, 0x83, 0x94, 0xaa, 0xf2, 0x72, 0xa5, - 0x3a, 0xe7, 0xf1, 0x0b, 0xaa, 0xcf, 0xa0, 0x43, 0xb9, 0xc7, 0x83, 0xca, 0x9b, 0x4f, 0xf2, 0x43, - 0x2d, 0x9a, 0xed, 0x86, 0x41, 0x5e, 0xc2, 0xf2, 0x5c, 0x7e, 0x40, 0xba, 0x08, 0xf4, 0x35, 0x29, - 0xc3, 0xe2, 0xe0, 0x9d, 0xce, 0xbf, 0xbc, 0x5b, 0x33, 0xfe, 0xf5, 0xdd, 0x9a, 0xf1, 0xef, 0xef, - 0xd6, 0x8c, 0xdf, 0xfd, 0xc7, 0xda, 0xd2, 0xc8, 0xc2, 0x3f, 0xdf, 0xbf, 0xfe, 0xdf, 0x00, 0x00, - 0x00, 0xff, 0xff, 0x3a, 0xde, 0xb8, 0x2c, 0xc0, 0x1f, 0x00, 0x00, + // 3368 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0xcd, 0x73, 0xdb, 0x48, + 0x76, 0x17, 0x40, 0x12, 0x04, 0x1e, 0x29, 0x99, 0xd3, 0xf6, 0x7a, 0x38, 0xf2, 0xae, 0xac, 0xc1, + 0xd8, 0x63, 0xd9, 0x1e, 0xcb, 0x1e, 0xcd, 0x24, 0x59, 0x6f, 0x2a, 0x07, 0x7d, 0xd0, 0x8e, 0x66, + 0xf4, 0x95, 0x26, 0xe5, 0x4d, 0xa6, 0x52, 0x61, 0x35, 0x81, 0x16, 0x85, 0x15, 0x08, 0x20, 0x68, + 0x50, 0x45, 0xf9, 0x96, 0x43, 0xf2, 0x37, 0xec, 0x21, 0x95, 0x5b, 0x2e, 0xb9, 0xe4, 0xba, 0xc7, + 0x5c, 0x52, 0x95, 0x63, 0x2a, 0x7f, 0x41, 0xca, 0xc9, 0x21, 0xf7, 0x54, 0xa5, 0x2a, 0xb7, 0x54, + 0xbf, 0x6e, 0x7c, 0x90, 0x96, 0xec, 0x9d, 0x54, 0xed, 0x89, 0x78, 0x1f, 0xfd, 0xf5, 0xeb, 0xf7, + 0x5e, 0xbf, 0xf7, 0x08, 0x76, 0x32, 0xda, 0x4c, 0xd2, 0x38, 0x8b, 0x89, 0x99, 0x8c, 0x56, 0x1d, + 0x96, 0x04, 0x8a, 0x5c, 0x7d, 0x34, 0x0e, 0xb2, 0xf3, 0xe9, 0x68, 0xd3, 0x8b, 0x27, 0xcf, 0xfd, + 0x71, 0xca, 0x92, 0xf3, 0x67, 0x41, 0xfc, 0x7c, 0xc4, 0xfc, 0x31, 0x4f, 0x9f, 0x27, 0xa3, 0xe7, + 0xf9, 0x38, 0x77, 0x15, 0xea, 0x07, 0x81, 0xc8, 0x08, 0x81, 0xfa, 0x34, 0xf0, 0x45, 0xd7, 0x58, + 0xaf, 0x6d, 0x58, 0x14, 0xbf, 0xdd, 0x43, 0x70, 0x06, 0x4c, 0x5c, 0xbc, 0x61, 0xe1, 0x94, 0x93, + 0x0e, 0xd4, 0x2e, 0x59, 0xd8, 0x35, 0xd6, 0x8d, 0x8d, 0x36, 0x95, 0x9f, 0x64, 0x13, 0xec, 0x4b, + 0x16, 0x0e, 0xb3, 0xab, 0x84, 0x77, 0xcd, 0x75, 0x63, 0x63, 0x65, 0xeb, 0xf6, 0x66, 0x32, 0xda, + 0x3c, 0x89, 0x45, 0x16, 0x44, 0xe3, 0xcd, 0x37, 0x2c, 0x1c, 0x5c, 0x25, 0x9c, 0x36, 0x2f, 0xd5, + 0x87, 0x7b, 0x0c, 0xad, 0x7e, 0xea, 0xbd, 0x9a, 0x46, 0x5e, 0x16, 0xc4, 0x91, 0x5c, 0x31, 0x62, + 0x13, 0x8e, 0x33, 0x3a, 0x14, 0xbf, 0x25, 0x8f, 0xa5, 0x63, 0xd1, 0xad, 0xad, 0xd7, 0x24, 0x4f, + 0x7e, 0x93, 0x2e, 0x34, 0x03, 0xb1, 0x1b, 0x4f, 0xa3, 0xac, 0x5b, 0x5f, 0x37, 0x36, 0x6c, 0x9a, + 0x93, 0xee, 0x7f, 0x9b, 0xd0, 0xf8, 0x93, 0x29, 0x4f, 0xaf, 0x70, 0x5c, 0x96, 0xa5, 0xf9, 0x5c, + 0xf2, 0x9b, 0xdc, 0x81, 0x46, 0xc8, 0xa2, 0xb1, 0xe8, 0x9a, 0x38, 0x99, 0x22, 0xc8, 0x3d, 0x70, + 0xd8, 0x59, 0xc6, 0xd3, 0xe1, 0x34, 0xf0, 0xbb, 0xb5, 0x75, 0x63, 0xc3, 0xa2, 0x36, 0x32, 0x4e, + 0x03, 0x9f, 0x7c, 0x06, 0xb6, 0x1f, 0x0f, 0xbd, 0xea, 0x5a, 0x7e, 0x8c, 0x6b, 0x91, 0x2f, 0xc0, + 0x9e, 0x06, 0xfe, 0x30, 0x0c, 0x44, 0xd6, 0x6d, 0xac, 0x1b, 0x1b, 0xad, 0x2d, 0x5b, 0x1e, 0x56, + 0x62, 0x47, 0x9b, 0xd3, 0xc0, 0x47, 0x10, 0x9f, 0x80, 0x2d, 0x52, 0x6f, 0x78, 0x36, 0x8d, 0xbc, + 0xae, 0x85, 0x4a, 0xb7, 0xa4, 0x52, 0xe5, 0xd4, 0xb4, 0x29, 0x14, 0x21, 0x8f, 0x95, 0xf2, 0x4b, + 0x9e, 0x0a, 0xde, 0x6d, 0xaa, 0xa5, 0x34, 0x49, 0x5e, 0x40, 0xeb, 0x8c, 0x79, 0x3c, 0x1b, 0x26, + 0x2c, 0x65, 0x93, 0xae, 0x5d, 0x4e, 0xf4, 0x4a, 0xb2, 0x4f, 0x24, 0x57, 0x50, 0x38, 0x2b, 0x08, + 0xf2, 0x0d, 0x2c, 0x23, 0x25, 0x86, 0x67, 0x41, 0x98, 0xf1, 0xb4, 0xeb, 0xe0, 0x98, 0x15, 0x1c, + 0x83, 0x9c, 0x41, 0xca, 0x39, 0x6d, 0x2b, 0x25, 0xc5, 0x21, 0x3f, 0x03, 0xe0, 0xb3, 0x84, 0x45, + 0xfe, 0x90, 0x85, 0x61, 0x17, 0x70, 0x0f, 0x8e, 0xe2, 0x6c, 0x87, 0x21, 0xf9, 0x54, 0xee, 0x8f, + 0xf9, 0xc3, 0x4c, 0x74, 0x97, 0xd7, 0x8d, 0x8d, 0x3a, 0xb5, 0x24, 0x39, 0x10, 0xee, 0x16, 0x38, + 0x68, 0x11, 0x78, 0xe2, 0x87, 0x60, 0x5d, 0x4a, 0x42, 0x19, 0x4e, 0x6b, 0x6b, 0x59, 0x2e, 0x59, + 0x18, 0x0d, 0xd5, 0x42, 0x77, 0x0d, 0xec, 0x03, 0x16, 0x8d, 0x73, 0x4b, 0x93, 0x57, 0x81, 0x03, + 0x1c, 0x8a, 0xdf, 0xee, 0xaf, 0x4d, 0xb0, 0x28, 0x17, 0xd3, 0x30, 0x23, 0x8f, 0x00, 0x24, 0xd0, + 0x13, 0x96, 0xa5, 0xc1, 0x4c, 0xcf, 0x5a, 0x42, 0xed, 0x4c, 0x03, 0xff, 0x10, 0x45, 0xe4, 0x05, + 0xb4, 0x71, 0xf6, 0x5c, 0xd5, 0x2c, 0x37, 0x50, 0xec, 0x8f, 0xb6, 0x50, 0x45, 0x8f, 0xb8, 0x0b, + 0x16, 0xde, 0xad, 0xb2, 0xaf, 0x65, 0xaa, 0x29, 0xf2, 0x10, 0x56, 0x82, 0x28, 0x93, 0xd8, 0x7b, + 0xd9, 0xd0, 0xe7, 0x22, 0xbf, 0xfc, 0xe5, 0x82, 0xbb, 0xc7, 0x45, 0x46, 0xbe, 0x06, 0x05, 0x60, + 0xbe, 0x60, 0x03, 0x17, 0x5c, 0x29, 0x2e, 0x46, 0xa8, 0x15, 0x51, 0x47, 0xaf, 0xf8, 0x0c, 0x5a, + 0xf2, 0x7c, 0xf9, 0x08, 0x0b, 0x47, 0xb4, 0xf1, 0x34, 0x1a, 0x0e, 0x0a, 0x52, 0x41, 0xab, 0x4b, + 0x68, 0xa4, 0x81, 0x29, 0x83, 0xc0, 0x6f, 0xb7, 0x07, 0x8d, 0xe3, 0xd4, 0xe7, 0xe9, 0xb5, 0x36, + 0x4e, 0xa0, 0xee, 0x73, 0xe1, 0xa1, 0xfb, 0xd9, 0x14, 0xbf, 0x4b, 0xbb, 0xaf, 0x55, 0xec, 0xde, + 0xfd, 0x3b, 0x03, 0x5a, 0xfd, 0x38, 0xcd, 0x0e, 0xb9, 0x10, 0x6c, 0xcc, 0xc9, 0x7d, 0x68, 0xc4, + 0x72, 0x5a, 0x8d, 0xb0, 0x23, 0xf7, 0x84, 0xeb, 0x50, 0xc5, 0x5f, 0xb8, 0x07, 0xf3, 0xe6, 0x7b, + 0xb8, 0x03, 0x0d, 0xe5, 0x31, 0xd2, 0x9b, 0x1a, 0x54, 0x11, 0x12, 0xeb, 0xf8, 0xec, 0x4c, 0x70, + 0x85, 0x65, 0x83, 0x6a, 0xea, 0x66, 0xb3, 0xfa, 0x3d, 0x00, 0xb9, 0xbf, 0x1f, 0x69, 0x05, 0xee, + 0x39, 0xb4, 0x28, 0x3b, 0xcb, 0x76, 0xe3, 0x28, 0xe3, 0xb3, 0x8c, 0xac, 0x80, 0x19, 0xf8, 0x08, + 0x91, 0x45, 0xcd, 0xc0, 0x97, 0x9b, 0x1b, 0xa7, 0xf1, 0x34, 0x41, 0x84, 0x96, 0xa9, 0x22, 0x10, + 0x4a, 0xdf, 0x4f, 0x71, 0xc7, 0x12, 0x4a, 0xdf, 0x4f, 0xc9, 0x7d, 0x68, 0x89, 0x88, 0x25, 0xe2, + 0x3c, 0xce, 0xe4, 0xe6, 0xea, 0xb8, 0x39, 0xc8, 0x59, 0x03, 0xe1, 0xfe, 0xb3, 0x01, 0xd6, 0x21, + 0x9f, 0x8c, 0x78, 0xfa, 0xde, 0x2a, 0x9f, 0x81, 0x8d, 0x13, 0x0f, 0x03, 0x5f, 0x2f, 0xd4, 0x44, + 0x7a, 0xdf, 0xbf, 0x76, 0xa9, 0xbb, 0x60, 0x85, 0x9c, 0x49, 0xf0, 0x95, 0x9d, 0x69, 0x4a, 0x62, + 0xc3, 0x26, 0x43, 0x9f, 0x33, 0x1f, 0x43, 0x8c, 0x4d, 0x2d, 0x36, 0xd9, 0xe3, 0xcc, 0x97, 0x7b, + 0x0b, 0x99, 0xc8, 0x86, 0xd3, 0xc4, 0x67, 0x19, 0xc7, 0xd0, 0x52, 0x97, 0x86, 0x23, 0xb2, 0x53, + 0xe4, 0x90, 0x27, 0xf0, 0x89, 0x17, 0x4e, 0x85, 0x8c, 0x6b, 0x41, 0x74, 0x16, 0x0f, 0xe3, 0x28, + 0xbc, 0x42, 0x7c, 0x6d, 0x7a, 0x4b, 0x0b, 0xf6, 0xa3, 0xb3, 0xf8, 0x38, 0x0a, 0xaf, 0xdc, 0xdf, + 0x98, 0xd0, 0x78, 0x8d, 0x30, 0xbc, 0x80, 0xe6, 0x04, 0x0f, 0x94, 0x7b, 0xef, 0x5d, 0x89, 0x30, + 0xca, 0x36, 0xd5, 0x49, 0x45, 0x2f, 0xca, 0xd2, 0x2b, 0x9a, 0xab, 0xc9, 0x11, 0x19, 0x1b, 0x85, + 0x3c, 0x13, 0xda, 0x22, 0x2a, 0x23, 0x06, 0x4a, 0xa0, 0x47, 0x68, 0xb5, 0x45, 0x58, 0x6b, 0x8b, + 0xb0, 0x92, 0x55, 0xb0, 0xbd, 0x73, 0xee, 0x5d, 0x88, 0xe9, 0x44, 0x83, 0x5e, 0xd0, 0xab, 0xaf, + 0xa0, 0x5d, 0xdd, 0x87, 0x7c, 0x83, 0x2e, 0xf8, 0x15, 0x02, 0x5f, 0xa7, 0xf2, 0x93, 0xac, 0x43, + 0x03, 0x3d, 0x1c, 0x61, 0x6f, 0x6d, 0x81, 0xdc, 0x8e, 0x1a, 0x42, 0x95, 0xe0, 0x17, 0xe6, 0xcf, + 0x0d, 0x39, 0x4f, 0x75, 0x77, 0xd5, 0x79, 0x9c, 0x9b, 0xe7, 0x51, 0x43, 0x2a, 0xf3, 0xb8, 0xff, + 0x6b, 0x42, 0xfb, 0x07, 0x9e, 0xc6, 0x27, 0x69, 0x9c, 0xc4, 0x82, 0x85, 0x64, 0x7b, 0xfe, 0x74, + 0x0a, 0xc5, 0x75, 0x39, 0xb8, 0xaa, 0xb6, 0xd9, 0x2f, 0x8e, 0xab, 0xd0, 0xa9, 0x9e, 0xdf, 0x05, + 0x4b, 0xa1, 0x7b, 0xcd, 0x11, 0xb4, 0x44, 0xea, 0x28, 0x3c, 0x11, 0xbf, 0xf9, 0xed, 0x69, 0x09, + 0x59, 0x03, 0x98, 0xb0, 0xd9, 0x01, 0x67, 0x82, 0xef, 0xfb, 0xb9, 0xf9, 0x96, 0x1c, 0x89, 0xf3, + 0x84, 0xcd, 0x06, 0xb3, 0x68, 0x20, 0xd0, 0xba, 0xea, 0xb4, 0xa0, 0xc9, 0x4f, 0xc1, 0x99, 0xb0, + 0x99, 0xf4, 0xa3, 0x7d, 0x5f, 0x5b, 0x57, 0xc9, 0x20, 0x9f, 0x43, 0x2d, 0x9b, 0x45, 0x18, 0x94, + 0xe4, 0x3b, 0x24, 0x93, 0x8c, 0xc1, 0x2c, 0xd2, 0x1e, 0x47, 0xa5, 0x2c, 0x07, 0xd4, 0x2e, 0x01, + 0xed, 0x40, 0xcd, 0x0b, 0x7c, 0x7c, 0x88, 0x1c, 0x2a, 0x3f, 0x57, 0xff, 0x08, 0x6e, 0x2d, 0xe0, + 0x50, 0xbd, 0x87, 0x65, 0x35, 0xec, 0x4e, 0xf5, 0x1e, 0xea, 0x55, 0xec, 0x7f, 0x53, 0x83, 0x5b, + 0xda, 0x18, 0xce, 0x83, 0xa4, 0x9f, 0x49, 0xb3, 0xef, 0x42, 0x13, 0xa3, 0x0d, 0x4f, 0xb5, 0x4d, + 0xe4, 0x24, 0xf9, 0x03, 0xb0, 0xd0, 0x03, 0x73, 0x3b, 0xbd, 0x5f, 0xa2, 0x5a, 0x0c, 0x57, 0x76, + 0xab, 0xaf, 0x44, 0xab, 0x93, 0x6f, 0xa1, 0xf1, 0x96, 0xa7, 0xb1, 0x8a, 0x9e, 0xad, 0xad, 0xb5, + 0xeb, 0xc6, 0xc9, 0xbb, 0xd5, 0xc3, 0x94, 0xf2, 0xef, 0x10, 0xfc, 0x07, 0x32, 0x5e, 0x4e, 0xe2, + 0x4b, 0xee, 0x77, 0x9b, 0xb8, 0xa3, 0xaa, 0x7d, 0xe4, 0xa2, 0x1c, 0x6d, 0xbb, 0x44, 0x7b, 0x0f, + 0x5a, 0x95, 0xe3, 0x5d, 0x83, 0xf4, 0xfd, 0x79, 0x8b, 0x77, 0x0a, 0x47, 0xae, 0x3a, 0xce, 0x1e, + 0x40, 0x79, 0xd8, 0xff, 0xaf, 0xfb, 0xb9, 0x7f, 0x65, 0xc0, 0xad, 0xdd, 0x38, 0x8a, 0x38, 0xa6, + 0x40, 0xea, 0xea, 0x4a, 0xb3, 0x37, 0x6e, 0x34, 0xfb, 0xc7, 0xd0, 0x10, 0x52, 0x59, 0xcf, 0x7e, + 0xfb, 0x9a, 0xbb, 0xa0, 0x4a, 0x43, 0x86, 0x99, 0x09, 0x9b, 0x0d, 0x13, 0x1e, 0xf9, 0x41, 0x34, + 0xce, 0xc3, 0xcc, 0x84, 0xcd, 0x4e, 0x14, 0xc7, 0xfd, 0x6b, 0x03, 0x2c, 0xe5, 0x31, 0x73, 0xd1, + 0xda, 0x98, 0x8f, 0xd6, 0x3f, 0x05, 0x27, 0x49, 0xb9, 0x1f, 0x78, 0xf9, 0xaa, 0x0e, 0x2d, 0x19, + 0xd2, 0x38, 0xcf, 0xe2, 0xd4, 0xe3, 0x38, 0xbd, 0x4d, 0x15, 0x21, 0xb9, 0x22, 0x61, 0x9e, 0x4a, + 0xe3, 0x6a, 0x54, 0x11, 0x32, 0xc6, 0xab, 0xcb, 0xc1, 0x4b, 0xb1, 0xa9, 0xa6, 0xdc, 0x7f, 0x30, + 0xa1, 0xbd, 0x17, 0xa4, 0xdc, 0xcb, 0xb8, 0xdf, 0xf3, 0xc7, 0xa8, 0xc8, 0xa3, 0x2c, 0xc8, 0xae, + 0xf4, 0x7b, 0xa2, 0xa9, 0xe2, 0xb9, 0x37, 0xe7, 0x53, 0x5a, 0x05, 0x77, 0x0d, 0xb3, 0x70, 0x45, + 0x90, 0x2d, 0x00, 0x95, 0x08, 0x61, 0x26, 0x5e, 0xbf, 0x39, 0x13, 0x77, 0x50, 0x4d, 0x7e, 0x4a, + 0x0c, 0xd4, 0x98, 0x40, 0xbd, 0x35, 0x16, 0xa6, 0xe9, 0x53, 0x69, 0xab, 0x98, 0x3f, 0x8c, 0x78, + 0x88, 0xb6, 0x88, 0xf9, 0xc3, 0x88, 0x87, 0x45, 0xd6, 0xd6, 0x54, 0xdb, 0x91, 0xdf, 0xe4, 0x0b, + 0x30, 0xe3, 0x04, 0xcf, 0xa7, 0x17, 0xac, 0x1e, 0x6c, 0xf3, 0x38, 0xa1, 0x66, 0x9c, 0xc8, 0x8b, + 0x56, 0x69, 0x67, 0xd7, 0xd1, 0xf6, 0x2b, 0x03, 0x08, 0x26, 0x4c, 0x54, 0x4b, 0xdc, 0xbb, 0x60, + 0x1e, 0x27, 0xa4, 0x09, 0xb5, 0x7e, 0x6f, 0xd0, 0x59, 0x92, 0x1f, 0x7b, 0xbd, 0x83, 0x8e, 0xe1, + 0xbe, 0x33, 0xc0, 0x39, 0x9c, 0x66, 0x4c, 0x9a, 0x8d, 0xf8, 0xd0, 0xbd, 0x7d, 0x06, 0xb6, 0xc8, + 0x58, 0x8a, 0x41, 0x58, 0x45, 0x8e, 0x26, 0xd2, 0x03, 0x41, 0xbe, 0x84, 0x06, 0xf7, 0xc7, 0x3c, + 0x77, 0xe8, 0xce, 0xe2, 0x3e, 0xa9, 0x12, 0x93, 0x0d, 0xb0, 0x84, 0x77, 0xce, 0x27, 0xac, 0x5b, + 0x2f, 0x15, 0xfb, 0xc8, 0x51, 0x8f, 0x2c, 0xd5, 0x72, 0xac, 0x12, 0xd2, 0x38, 0xc1, 0xb4, 0xb9, + 0xa1, 0xab, 0x84, 0x34, 0x4e, 0x64, 0xd2, 0xbc, 0x05, 0x3f, 0x09, 0xc6, 0x51, 0x9c, 0xf2, 0x61, + 0x10, 0xf9, 0x7c, 0x36, 0xf4, 0xe2, 0xe8, 0x2c, 0x0c, 0xbc, 0x0c, 0xb1, 0xb4, 0xe9, 0x6d, 0x25, + 0xdc, 0x97, 0xb2, 0x5d, 0x2d, 0x72, 0x67, 0x60, 0xe7, 0x71, 0x91, 0x3c, 0x96, 0x01, 0x0d, 0xe3, + 0xaa, 0x76, 0x0b, 0x4c, 0xfb, 0x2b, 0x09, 0x0e, 0xcd, 0xe5, 0xf2, 0x9a, 0x70, 0x8d, 0x3c, 0x52, + 0x22, 0x51, 0x4d, 0xaf, 0x6a, 0xd5, 0xf4, 0x0a, 0x33, 0xc5, 0x38, 0xe2, 0x3a, 0xe3, 0xc0, 0x6f, + 0xf7, 0x6f, 0x4d, 0xb0, 0x8b, 0xa7, 0xec, 0x29, 0x38, 0x93, 0x1c, 0x6a, 0xed, 0x70, 0x98, 0x4b, + 0x17, 0xf8, 0xd3, 0x52, 0x4e, 0xee, 0x82, 0x79, 0x71, 0xa9, 0x81, 0xb2, 0xa4, 0xd6, 0xf7, 0x6f, + 0xa8, 0x79, 0x71, 0x59, 0x7a, 0x6c, 0xe3, 0xa3, 0x1e, 0xfb, 0x08, 0x6e, 0x79, 0x21, 0x67, 0xd1, + 0xb0, 0x74, 0x38, 0x65, 0x70, 0x2b, 0xc8, 0x3e, 0x29, 0xbc, 0x4e, 0x47, 0x9d, 0x66, 0xf9, 0xb6, + 0x3c, 0x84, 0x86, 0xcf, 0xc3, 0x8c, 0x55, 0x4b, 0xa3, 0xe3, 0x94, 0x79, 0x21, 0xdf, 0x93, 0x6c, + 0xaa, 0xa4, 0x64, 0x03, 0xec, 0xfc, 0x9d, 0xd5, 0x05, 0x11, 0x66, 0xde, 0x39, 0xd8, 0xb4, 0x90, + 0x96, 0x58, 0x42, 0x05, 0x4b, 0xf7, 0x6b, 0xa8, 0x7d, 0xff, 0xa6, 0xaf, 0xcf, 0x6a, 0xbc, 0x77, + 0xd6, 0x1c, 0x51, 0xb3, 0x82, 0xe8, 0xff, 0xd4, 0xa0, 0xa9, 0xbd, 0x4e, 0xee, 0x7b, 0x5a, 0x64, + 0x89, 0xf2, 0x73, 0xfe, 0x71, 0x2b, 0xdc, 0xb7, 0x5a, 0x46, 0xd7, 0x3e, 0x5e, 0x46, 0x93, 0x5f, + 0x40, 0x3b, 0x51, 0xb2, 0xaa, 0xc3, 0x7f, 0x5a, 0x1d, 0xa3, 0x7f, 0x71, 0x5c, 0x2b, 0x29, 0x09, + 0x69, 0xba, 0x58, 0x8f, 0x64, 0x6c, 0x8c, 0x57, 0xd4, 0xa6, 0x4d, 0x49, 0x0f, 0xd8, 0xf8, 0x06, + 0xb7, 0xff, 0x2d, 0xbc, 0x57, 0x66, 0xc3, 0x71, 0xd2, 0x6d, 0xa3, 0x47, 0x4a, 0x8f, 0xaf, 0x3a, + 0xe3, 0xf2, 0xbc, 0x33, 0xde, 0x03, 0xc7, 0x8b, 0x27, 0x93, 0x00, 0x65, 0x2b, 0x3a, 0xdb, 0x43, + 0xc6, 0x40, 0xb8, 0x7f, 0x63, 0x40, 0x53, 0x9f, 0x96, 0xb4, 0xa0, 0xb9, 0xd7, 0x7b, 0xb5, 0x7d, + 0x7a, 0x20, 0xe3, 0x01, 0x80, 0xb5, 0xb3, 0x7f, 0xb4, 0x4d, 0xff, 0xac, 0x63, 0xc8, 0xd8, 0xb0, + 0x7f, 0x34, 0xe8, 0x98, 0xc4, 0x81, 0xc6, 0xab, 0x83, 0xe3, 0xed, 0x41, 0xa7, 0x46, 0x6c, 0xa8, + 0xef, 0x1c, 0x1f, 0x1f, 0x74, 0xea, 0xa4, 0x0d, 0xf6, 0xde, 0xf6, 0xa0, 0x37, 0xd8, 0x3f, 0xec, + 0x75, 0x1a, 0x52, 0xf7, 0x75, 0xef, 0xb8, 0x63, 0xc9, 0x8f, 0xd3, 0xfd, 0xbd, 0x4e, 0x53, 0xca, + 0x4f, 0xb6, 0xfb, 0xfd, 0x5f, 0x1e, 0xd3, 0xbd, 0x8e, 0x2d, 0xe7, 0xed, 0x0f, 0xe8, 0xfe, 0xd1, + 0xeb, 0x8e, 0x23, 0xbf, 0x8f, 0x77, 0xbe, 0xeb, 0xed, 0x0e, 0x3a, 0xe0, 0x7e, 0x0d, 0xad, 0x0a, + 0x82, 0x72, 0x34, 0xed, 0xbd, 0xea, 0x2c, 0xc9, 0x25, 0xdf, 0x6c, 0x1f, 0x9c, 0xf6, 0x3a, 0x06, + 0x59, 0x01, 0xc0, 0xcf, 0xe1, 0xc1, 0xf6, 0xd1, 0xeb, 0x8e, 0xe9, 0xfe, 0x3e, 0xd8, 0xa7, 0x81, + 0xbf, 0x13, 0xc6, 0xde, 0x85, 0x34, 0x8c, 0x11, 0x13, 0x5c, 0xbf, 0x93, 0xf8, 0x2d, 0xa3, 0x3c, + 0x1a, 0xa5, 0xd0, 0x77, 0xaf, 0x29, 0xf7, 0x08, 0x9a, 0xa7, 0x81, 0x7f, 0xc2, 0xbc, 0x0b, 0x59, + 0x8f, 0x8f, 0xe4, 0xf8, 0xa1, 0x08, 0xde, 0x72, 0x1d, 0xe0, 0x1c, 0xe4, 0xf4, 0x83, 0xb7, 0x9c, + 0x3c, 0x00, 0x0b, 0x89, 0x3c, 0xa3, 0x41, 0x5b, 0xce, 0xd7, 0xa4, 0x5a, 0xe6, 0xfe, 0x97, 0x51, + 0xec, 0x1d, 0x8b, 0xed, 0xfb, 0x50, 0x4f, 0x98, 0x77, 0xa1, 0xa3, 0x49, 0x4b, 0x8f, 0x91, 0xeb, + 0x51, 0x14, 0x90, 0x47, 0x60, 0x6b, 0x03, 0xc9, 0x27, 0x6e, 0x55, 0x2c, 0x89, 0x16, 0xc2, 0xf9, + 0xab, 0xab, 0xcd, 0x5f, 0x9d, 0xdc, 0xfb, 0x64, 0x1a, 0x66, 0xc1, 0x30, 0x61, 0x69, 0x5e, 0x3d, + 0x3b, 0xc8, 0x39, 0x61, 0x69, 0x26, 0xc5, 0x67, 0x41, 0x2a, 0x32, 0x25, 0x56, 0x31, 0xd3, 0x41, + 0x0e, 0x8a, 0xef, 0x81, 0xa3, 0x0c, 0x46, 0xfa, 0x8b, 0xca, 0x80, 0x94, 0x05, 0x9d, 0x06, 0xbe, + 0x8c, 0x68, 0x3c, 0xf2, 0x51, 0xd4, 0x54, 0x11, 0x8d, 0x47, 0xfe, 0x69, 0xe0, 0xbb, 0xdf, 0x02, + 0x94, 0xfd, 0x90, 0x6b, 0x52, 0xfa, 0x3b, 0xd0, 0x60, 0x61, 0xa0, 0x11, 0x77, 0xa8, 0x22, 0xdc, + 0x23, 0x68, 0x55, 0xba, 0x28, 0xd2, 0x56, 0x59, 0x18, 0x0e, 0x2f, 0xf8, 0x95, 0xc0, 0xb1, 0x36, + 0x6d, 0xb2, 0x30, 0xfc, 0x9e, 0x5f, 0x09, 0xf2, 0x00, 0x1a, 0xaa, 0x01, 0x63, 0x2e, 0xd4, 0xf9, + 0x38, 0x94, 0x2a, 0xa1, 0xfb, 0x15, 0x58, 0xaa, 0xf8, 0xaf, 0xb8, 0x8a, 0x71, 0xe3, 0x43, 0xf7, + 0x52, 0xef, 0x19, 0x5b, 0x05, 0xe4, 0xa9, 0x6e, 0xf4, 0x08, 0xd5, 0x56, 0x32, 0xca, 0xfc, 0x4e, + 0x29, 0xe9, 0x1e, 0x0f, 0x2a, 0xbb, 0x7b, 0x60, 0x7f, 0xb0, 0x75, 0xa6, 0x01, 0x30, 0x4b, 0x00, + 0xae, 0x69, 0xa6, 0xb9, 0xbf, 0x02, 0x28, 0x1b, 0x42, 0xda, 0x73, 0xd5, 0x2c, 0xd2, 0x73, 0x9f, + 0xc8, 0x5a, 0x2c, 0x08, 0xfd, 0x94, 0x47, 0x73, 0xa7, 0x2e, 0x5b, 0x48, 0x85, 0x9c, 0xac, 0x43, + 0x1d, 0xfb, 0x5c, 0xb5, 0x32, 0xb2, 0x16, 0x4d, 0x2e, 0x94, 0xb8, 0x23, 0x58, 0x56, 0xef, 0x27, + 0xe5, 0x7f, 0x39, 0xe5, 0xe2, 0x83, 0x89, 0xd7, 0x1a, 0x40, 0xf1, 0x0e, 0xe4, 0x1d, 0xbb, 0x0a, + 0x47, 0xfa, 0xcf, 0x59, 0xc0, 0x43, 0x3f, 0x3f, 0x8d, 0xa6, 0xdc, 0x3f, 0x84, 0x76, 0xbe, 0x06, + 0xf6, 0x0d, 0x9e, 0x16, 0xaf, 0xb8, 0x42, 0x53, 0x95, 0x2b, 0x4a, 0xe5, 0x28, 0xf6, 0xf9, 0x8e, + 0xd9, 0x35, 0xf2, 0x87, 0xdc, 0xfd, 0xb7, 0x5a, 0x3e, 0x5a, 0x97, 0xd1, 0x73, 0xe9, 0x9f, 0xb1, + 0x98, 0xfe, 0xcd, 0xe7, 0x59, 0xe6, 0x6f, 0x95, 0x67, 0xfd, 0x1c, 0x1c, 0x1f, 0x93, 0x8d, 0xe0, + 0x32, 0x8f, 0xee, 0xab, 0x8b, 0x89, 0x85, 0x4e, 0x47, 0x82, 0x4b, 0x4e, 0x4b, 0x65, 0xb9, 0x97, + 0x2c, 0xbe, 0xe0, 0x51, 0xf0, 0x16, 0xfb, 0x04, 0xf2, 0xd0, 0x25, 0xa3, 0x6c, 0xba, 0x28, 0x67, + 0xd2, 0x4d, 0x97, 0xbc, 0x7f, 0x64, 0x95, 0xfd, 0x23, 0x89, 0xdc, 0x34, 0x11, 0x3c, 0xcd, 0xf2, + 0x44, 0x54, 0x51, 0x45, 0x42, 0xe7, 0x68, 0x5d, 0x99, 0xd0, 0x7d, 0x0e, 0xed, 0x28, 0x8e, 0x86, + 0xd1, 0x34, 0x0c, 0x65, 0xaa, 0xac, 0x9b, 0x82, 0xad, 0x28, 0x8e, 0x8e, 0x34, 0x8b, 0x3c, 0x81, + 0x4f, 0xaa, 0x2a, 0xca, 0x72, 0x5b, 0xaa, 0xd3, 0x50, 0xd1, 0x43, 0xfb, 0xde, 0x80, 0x4e, 0x3c, + 0xfa, 0x15, 0xf7, 0x32, 0x44, 0x6c, 0x88, 0x26, 0xdb, 0x56, 0x6f, 0xbc, 0xe2, 0x4b, 0x88, 0x8e, + 0xd8, 0x84, 0xbb, 0x2f, 0xc1, 0x29, 0x40, 0x90, 0xe1, 0xfc, 0xe8, 0xf8, 0xa8, 0xa7, 0x02, 0xee, + 0xfe, 0xd1, 0x5e, 0xef, 0x4f, 0x3b, 0x86, 0x7c, 0x10, 0x68, 0xef, 0x4d, 0x8f, 0xf6, 0x7b, 0x1d, + 0x53, 0x06, 0xeb, 0xbd, 0xde, 0x41, 0x6f, 0xd0, 0xeb, 0xd4, 0xbe, 0xab, 0xdb, 0xcd, 0x8e, 0x4d, + 0x6d, 0x3e, 0x4b, 0xc2, 0xc0, 0x0b, 0x32, 0xb7, 0x0f, 0x20, 0xa7, 0xd5, 0x37, 0x7a, 0x0f, 0x9c, + 0x72, 0x6d, 0x75, 0xa3, 0x76, 0xa6, 0x57, 0x95, 0x29, 0x9f, 0x36, 0x2a, 0xf3, 0xa6, 0x94, 0x4f, + 0x9b, 0xd9, 0x29, 0xd8, 0x87, 0x2c, 0x79, 0xaf, 0x0a, 0x6a, 0x17, 0xb5, 0xee, 0x54, 0x77, 0x7e, + 0xf4, 0x4b, 0xff, 0x10, 0x9a, 0x3a, 0x70, 0x6a, 0xff, 0x98, 0x0b, 0xaa, 0xb9, 0x4c, 0x16, 0x25, + 0x77, 0x0e, 0xe3, 0x4b, 0x5e, 0x24, 0x3b, 0x27, 0xec, 0x2a, 0x8c, 0x99, 0xff, 0x11, 0x43, 0xfc, + 0x19, 0x80, 0x88, 0xa7, 0xa9, 0xc7, 0x87, 0xe3, 0xa2, 0xe1, 0xe4, 0x28, 0xce, 0x6b, 0xdd, 0xc5, + 0xe6, 0x22, 0x43, 0x61, 0x4d, 0xb9, 0x99, 0xa4, 0xa5, 0xe8, 0x27, 0x60, 0x65, 0xb3, 0xa8, 0xec, + 0x6f, 0x35, 0x32, 0x59, 0x82, 0xba, 0xbb, 0xe0, 0x0c, 0x66, 0x58, 0x98, 0x4d, 0xc5, 0xdc, 0xf3, + 0x6d, 0x7c, 0xe0, 0xf9, 0x36, 0x17, 0x9e, 0xef, 0xff, 0x34, 0xa0, 0x55, 0xc9, 0xc2, 0xc8, 0xe7, + 0x50, 0xcf, 0x66, 0xd1, 0x7c, 0x63, 0x38, 0x5f, 0x84, 0xa2, 0x48, 0xda, 0x9b, 0xac, 0xda, 0x98, + 0x10, 0xc1, 0x38, 0xe2, 0xbe, 0x9e, 0x52, 0x56, 0x72, 0xdb, 0x9a, 0x45, 0x0e, 0xe0, 0x96, 0x8a, + 0x19, 0x79, 0x53, 0x28, 0x4f, 0xe4, 0xbf, 0x58, 0xc8, 0xfa, 0x54, 0xf1, 0xba, 0x9b, 0x6b, 0xa9, + 0xf2, 0x7c, 0x65, 0x3c, 0xc7, 0x5c, 0xdd, 0x86, 0xdb, 0xd7, 0xa8, 0xfd, 0xa8, 0x3e, 0xc4, 0x7d, + 0x58, 0x96, 0x75, 0x7b, 0x30, 0xe1, 0x22, 0x63, 0x93, 0x04, 0xd3, 0x1f, 0x1d, 0xf3, 0xeb, 0xd4, + 0xcc, 0x84, 0xfb, 0x25, 0xb4, 0x4f, 0x38, 0x4f, 0x29, 0x17, 0x49, 0x1c, 0xa9, 0xa7, 0x5f, 0xe0, + 0xa1, 0xf5, 0x03, 0xa3, 0x29, 0xf7, 0x2f, 0xc0, 0x91, 0x89, 0xfd, 0x0e, 0xcb, 0xbc, 0xf3, 0x1f, + 0x93, 0xf8, 0x7f, 0x09, 0xcd, 0x44, 0x99, 0x89, 0x4e, 0xd3, 0xdb, 0x18, 0xe3, 0xb4, 0xe9, 0xd0, + 0x5c, 0xe8, 0x7e, 0x0b, 0xb5, 0xa3, 0xe9, 0xa4, 0xfa, 0xbf, 0x4d, 0x5d, 0xfd, 0x6f, 0x73, 0x0f, + 0x1c, 0xac, 0x11, 0xb0, 0x49, 0xa8, 0xb2, 0x57, 0x5b, 0x32, 0xb0, 0x3b, 0xf8, 0x03, 0xb4, 0x72, + 0xec, 0xf7, 0x7d, 0xfc, 0xf3, 0x05, 0x2f, 0x7f, 0xdf, 0x9f, 0xb3, 0x05, 0x55, 0x26, 0xf2, 0xc8, + 0xdf, 0xcf, 0x2f, 0x4d, 0x11, 0xf3, 0x73, 0xeb, 0x4e, 0x47, 0x31, 0xf7, 0x2b, 0x68, 0xe7, 0xc9, + 0xf7, 0x21, 0xcf, 0x18, 0x9a, 0x53, 0x18, 0xf0, 0xa8, 0x62, 0x6a, 0xb6, 0x62, 0x0c, 0xc4, 0x07, + 0x7a, 0xaa, 0xee, 0x26, 0x58, 0xda, 0x56, 0x09, 0xd4, 0xbd, 0xd8, 0x57, 0x2e, 0xd2, 0xa0, 0xf8, + 0x2d, 0x0f, 0x3c, 0x11, 0xe3, 0xfc, 0x21, 0x9c, 0x88, 0xb1, 0x3b, 0x83, 0xe5, 0x1d, 0xe6, 0x5d, + 0x4c, 0x93, 0xfc, 0x21, 0xaa, 0x54, 0x49, 0xc6, 0x5c, 0x95, 0xf4, 0x81, 0x46, 0xee, 0xa7, 0xd0, + 0x9c, 0x46, 0xc1, 0x2c, 0xcf, 0x7e, 0x1c, 0x6a, 0x49, 0x52, 0x35, 0x30, 0xc3, 0xd8, 0xc3, 0xc2, + 0x08, 0xbd, 0xca, 0xa1, 0x05, 0xed, 0xfe, 0x39, 0x2c, 0xf7, 0x66, 0x09, 0xb6, 0xb5, 0x3f, 0xfa, + 0x04, 0x56, 0x36, 0x65, 0xce, 0x6d, 0x6a, 0x61, 0xe5, 0x5a, 0xbe, 0xf2, 0xd6, 0x3f, 0x19, 0x50, + 0x97, 0x26, 0x22, 0x6b, 0xb7, 0x3f, 0xe6, 0x2c, 0xcd, 0x46, 0x9c, 0x65, 0x64, 0xce, 0x1c, 0x56, + 0xe7, 0x28, 0x77, 0xe9, 0x85, 0x41, 0xbe, 0x52, 0x1d, 0xf3, 0xfc, 0x8f, 0x80, 0xe5, 0xdc, 0xd0, + 0xd0, 0x10, 0x17, 0xf5, 0xc9, 0x26, 0xb4, 0xbe, 0x8b, 0x83, 0x68, 0x57, 0x35, 0x91, 0xc9, 0xa2, + 0x59, 0xbe, 0xa7, 0xff, 0x0c, 0xac, 0x7d, 0x21, 0xed, 0xff, 0x7d, 0x55, 0x0c, 0xaf, 0x55, 0xd7, + 0x70, 0x97, 0xb6, 0xfe, 0xb1, 0x06, 0xf5, 0x1f, 0x78, 0x1a, 0x93, 0xaf, 0xa0, 0xa9, 0x5b, 0x44, + 0xa4, 0xd2, 0x0a, 0x5a, 0xc5, 0xf7, 0x75, 0xa1, 0x77, 0x84, 0xab, 0x74, 0x54, 0x84, 0x2e, 0x8b, + 0x4b, 0x52, 0x76, 0xb0, 0xde, 0xdb, 0xd4, 0x4b, 0xe8, 0xf4, 0xb3, 0x94, 0xb3, 0x49, 0x45, 0x7d, + 0x1e, 0xa6, 0xeb, 0x2a, 0x55, 0x44, 0xeb, 0x29, 0x58, 0x2a, 0xc8, 0x2c, 0x0c, 0x58, 0x2c, 0x3a, + 0x51, 0xf9, 0x11, 0xb4, 0xfa, 0xe7, 0xf1, 0x34, 0xf4, 0xfb, 0x3c, 0xbd, 0xe4, 0xa4, 0xd2, 0xa6, + 0x5d, 0xad, 0x7c, 0xbb, 0x4b, 0x64, 0x03, 0x40, 0x79, 0xd9, 0x69, 0xe0, 0x0b, 0xd2, 0x94, 0xb2, + 0xa3, 0xe9, 0x44, 0x4d, 0x5a, 0x71, 0x3f, 0xa5, 0x59, 0x89, 0x35, 0x1f, 0xd2, 0xfc, 0x06, 0x96, + 0x77, 0x31, 0x16, 0x1f, 0xa7, 0xdb, 0xa3, 0x38, 0xcd, 0xc8, 0x62, 0xab, 0x76, 0x75, 0x91, 0xe1, + 0x2e, 0x91, 0x17, 0x60, 0x0f, 0xd2, 0x2b, 0xa5, 0xff, 0x89, 0x0e, 0xd1, 0xe5, 0x7a, 0xd7, 0x9c, + 0x72, 0xeb, 0xef, 0x6b, 0x60, 0xfd, 0x32, 0x4e, 0x2f, 0x78, 0x4a, 0x9e, 0x80, 0x85, 0xdd, 0x01, + 0x6d, 0x44, 0x45, 0xa7, 0xe0, 0xba, 0x85, 0x1e, 0x80, 0x83, 0xa0, 0x0c, 0x98, 0xb8, 0x50, 0x57, + 0x85, 0xff, 0xdc, 0x2a, 0x5c, 0x54, 0xf2, 0x86, 0xf7, 0xba, 0xa2, 0x2e, 0xaa, 0xe8, 0x88, 0xcc, + 0x95, 0xec, 0xab, 0x4d, 0x55, 0x7f, 0xf7, 0xdd, 0xa5, 0x0d, 0xe3, 0x85, 0x41, 0x1e, 0x43, 0xbd, + 0xaf, 0x4e, 0x2a, 0x95, 0xca, 0x7f, 0xb7, 0x56, 0x57, 0x72, 0x46, 0x31, 0xf3, 0x73, 0xb0, 0xd4, + 0xcb, 0xae, 0x8e, 0x39, 0x97, 0x98, 0xae, 0x76, 0xaa, 0x2c, 0x3d, 0xe0, 0x31, 0x58, 0x2a, 0x68, + 0xa8, 0x01, 0x73, 0x01, 0x44, 0xed, 0x5a, 0xc5, 0x20, 0xa5, 0xaa, 0xbc, 0x5c, 0xa9, 0xce, 0x79, + 0xfc, 0x82, 0xea, 0x33, 0xe8, 0x50, 0xee, 0xf1, 0xa0, 0xf2, 0xe6, 0x93, 0xfc, 0x50, 0x8b, 0x66, + 0xbb, 0x61, 0x90, 0x97, 0xb0, 0x3c, 0x97, 0x1f, 0x90, 0x2e, 0x02, 0x7d, 0x4d, 0xca, 0xb0, 0x38, + 0x78, 0xa7, 0xf3, 0x2f, 0xef, 0xd6, 0x8c, 0x7f, 0x7d, 0xb7, 0x66, 0xfc, 0xfb, 0xbb, 0x35, 0xe3, + 0xd7, 0xff, 0xb1, 0xb6, 0x34, 0xb2, 0xf0, 0x1f, 0xff, 0x6f, 0xfe, 0x2f, 0x00, 0x00, 0xff, 0xff, + 0xcf, 0xaa, 0xfb, 0xee, 0x35, 0x20, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -6412,6 +6448,36 @@ func (m *PostingList) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintPb(dAtA, i, uint64(m.CommitTs)) } + if m.MultiPart { + dAtA[i] = 0x20 + i++ + if m.MultiPart { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.FirstPart { + dAtA[i] = 0x28 + i++ + if m.FirstPart { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if m.StartUid != 0 { + dAtA[i] = 0x30 + i++ + i = encodeVarintPb(dAtA, i, uint64(m.StartUid)) + } + if m.EndUid != 0 { + dAtA[i] = 0x38 + i++ + i = encodeVarintPb(dAtA, i, uint64(m.EndUid)) + } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) } @@ -8221,6 +8287,18 @@ func (m *PostingList) Size() (n int) { if m.CommitTs != 0 { n += 1 + sovPb(uint64(m.CommitTs)) } + if m.MultiPart { + n += 2 + } + if m.FirstPart { + n += 2 + } + if m.StartUid != 0 { + n += 1 + sovPb(uint64(m.StartUid)) + } + if m.EndUid != 0 { + n += 1 + sovPb(uint64(m.EndUid)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -13531,6 +13609,84 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { break } } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MultiPart", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.MultiPart = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FirstPart", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.FirstPart = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartUid", wireType) + } + m.StartUid = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StartUid |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EndUid", wireType) + } + m.EndUid = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.EndUid |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipPb(dAtA[iNdEx:]) diff --git a/worker/draft.go b/worker/draft.go index 0bfed7a2455..5b3679983ed 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -892,9 +892,16 @@ func (n *node) rollupLists(readTs uint64) error { return nil, err } atomic.AddUint64(&numKeys, 1) - kv, err := l.MarshalToKv() - addTo(key, int64(kv.Size())) - return listWrap(kv), err + kvs, err := l.MarshalToKv() + + // If there are multiple keys, the posting list was split into multiple + // parts. The key of the first part is the right key to use for tablet + // size calculations. + for _, kv := range kvs { + addTo(kvs[0].Key, int64(kv.Size())) + } + + return &bpb.KVList{Kv: kvs}, err } stream.Send = func(list *bpb.KVList) error { return writer.Send(&pb.KVS{Kv: list.Kv}) diff --git a/worker/predicate_move.go b/worker/predicate_move.go index 2dec7326900..a4eef343da6 100644 --- a/worker/predicate_move.go +++ b/worker/predicate_move.go @@ -257,12 +257,12 @@ func movePredicateHelper(ctx context.Context, in *pb.MovePredicatePayload) error if err != nil { return nil, err } - kv, err := l.MarshalToKv() - if kv != nil { + kvs, err := l.MarshalToKv() + for _, kv := range kvs { // Let's set all of them at this move timestamp. kv.Version = in.TxnTs } - return listWrap(kv), err + return &bpb.KVList{Kv: kvs}, err } stream.Send = func(list *bpb.KVList) error { return s.Send(&pb.KVS{Kv: list.Kv}) diff --git a/x/keys.go b/x/keys.go index c6018ae3d36..3cac8db2f29 100644 --- a/x/keys.go +++ b/x/keys.go @@ -72,6 +72,24 @@ func DataKey(attr string, uid uint64) []byte { return buf } +func DataKeyMultiPart(attr string, uid, startUid uint64) []byte { + buf := make([]byte, 2+len(attr)+2+8+8) + buf[0] = defaultPrefix + rest := buf[1:] + + rest = writeAttr(rest, attr) + rest[0] = ByteData + + rest = rest[1:] + binary.BigEndian.PutUint64(rest, uid) + + // This list is split in multiple parts. startUid represents the first UID + // in the range of UIDs stored by this part of the list. + rest = rest[8:] + binary.BigEndian.PutUint64(rest, startUid) + return buf +} + func ReverseKey(attr string, uid uint64) []byte { buf := make([]byte, 2+len(attr)+2+8) buf[0] = defaultPrefix From 8aff4a7e783a3df1971c7e9adb6e788bb8924234 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 11:33:10 -0800 Subject: [PATCH 02/67] Decide if list needs to be split based on approxLen. --- posting/list.go | 23 +++++------------ posting/list_test.go | 61 ++++++++++++++++++++++---------------------- 2 files changed, 37 insertions(+), 47 deletions(-) diff --git a/posting/list.go b/posting/list.go index 4d65069ee6f..0c8726dc3d0 100644 --- a/posting/list.go +++ b/posting/list.go @@ -810,18 +810,6 @@ func (l *List) length(readTs, afterUid uint64) int { return count } -func (l *List) partialLength(readTs uint64) int { - count := 0 - err := l.partIterate(readTs, func(p *pb.Posting) error { - count++ - return nil - }) - if err != nil { - return -1 - } - return count -} - // Length iterates over the mutation layer and counts number of elements. func (l *List) Length(readTs, afterUid uint64) int { l.RLock() @@ -960,6 +948,10 @@ func (l *List) rollup(readTs uint64) error { func (l *List) ApproxLen() int { l.RLock() defer l.RUnlock() + return l.approxLen() +} + +func (l *List) approxLen() int { return len(l.mutationMap) + codec.ApproxLen(l.plist.Pack) } @@ -1248,9 +1240,8 @@ func (l *List) loadNextPart(readTs uint64) error { return nil } -func (l *List) needsSplit(readTs uint64) bool { - length := l.partialLength(readTs) - return length >= maxListLength +func (l *List) needsSplit() bool { + return l.approxLen() >= maxListLength } func (l *List) splitList(readTs uint64) error { @@ -1260,7 +1251,7 @@ func (l *List) splitList(readTs uint64) error { var prev *List for curr != nil { - if curr.needsSplit(readTs) { + if curr.needsSplit() { newLists := curr.splitListPart(readTs) // If splitting a list for the first time, initialize it as a diff --git a/posting/list_test.go b/posting/list_test.go index f9c9d82e258..54033d7d03b 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -414,37 +414,36 @@ func TestAddMutation_mrjn1(t *testing.T) { require.Equal(t, 0, ol.Length(txn.StartTs, 0)) } -// TODO: size calculations are slow so this test is extremely slow. Fix that. -// func TestMillion(t *testing.T) { -// key := x.DataKey("bal", 1331) -// ol, err := getNew(key, ps) -// require.NoError(t, err) -// var commits int -// N := int(1e6) -// for i := 2; i <= N; i += 2 { -// edge := &pb.DirectedEdge{ -// ValueId: uint64(i), -// } -// txn := Txn{StartTs: uint64(i)} -// addMutationHelper(t, ol, edge, Set, &txn) -// require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) -// if i%10000 == 0 { -// // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting -// // list. -// t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) -// require.NoError(t, ol.Rollup(math.MaxUint64)) -// } -// commits++ -// } -// t.Logf("Completed a million writes.\n") -// opt := ListOptions{ReadTs: uint64(N) + 1} -// l, err := ol.Uids(opt) -// require.NoError(t, err) -// require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) -// for i, uid := range l.Uids { -// require.Equal(t, uint64(i+1)*2, uid) -// } -// } +func TestMillion(t *testing.T) { + key := x.DataKey("bal", 1331) + ol, err := getNew(key, ps) + require.NoError(t, err) + var commits int + N := int(1e6) + for i := 2; i <= N; i += 2 { + edge := &pb.DirectedEdge{ + ValueId: uint64(i), + } + txn := Txn{StartTs: uint64(i)} + addMutationHelper(t, ol, edge, Set, &txn) + require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) + if i%10000 == 0 { + // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting + // list. + t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) + require.NoError(t, ol.Rollup(math.MaxUint64)) + } + commits++ + } + t.Logf("Completed a million writes.\n") + opt := ListOptions{ReadTs: uint64(N) + 1} + l, err := ol.Uids(opt) + require.NoError(t, err) + require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) + for i, uid := range l.Uids { + require.Equal(t, uint64(i+1)*2, uid) + } +} // Test the various mutate, commit and abort sequences. func TestAddMutation_mrjn2(t *testing.T) { From dd630c49cab56a75ff292166b282d3752100bf68 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 13:04:28 -0800 Subject: [PATCH 03/67] Use size instead of length to do split decisions. --- posting/list.go | 8 ++------ posting/list_test.go | 4 ++-- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/posting/list.go b/posting/list.go index 0c8726dc3d0..19f0ca69f00 100644 --- a/posting/list.go +++ b/posting/list.go @@ -52,7 +52,7 @@ var ( ErrInvalidTxn = fmt.Errorf("Invalid transaction") ErrStopIteration = errors.New("Stop iteration") emptyPosting = &pb.Posting{} - maxListLength = 2000000 + maxListSize = 2000000 ) const ( @@ -948,10 +948,6 @@ func (l *List) rollup(readTs uint64) error { func (l *List) ApproxLen() int { l.RLock() defer l.RUnlock() - return l.approxLen() -} - -func (l *List) approxLen() int { return len(l.mutationMap) + codec.ApproxLen(l.plist.Pack) } @@ -1241,7 +1237,7 @@ func (l *List) loadNextPart(readTs uint64) error { } func (l *List) needsSplit() bool { - return l.approxLen() >= maxListLength + return l.plist.Size() >= maxListSize } func (l *List) splitList(readTs uint64) error { diff --git a/posting/list_test.go b/posting/list_test.go index 54033d7d03b..c2b874dc751 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -904,9 +904,9 @@ func BenchmarkAddMutations(b *testing.B) { func TestMultiPartList(t *testing.T) { // For testing, set the max list length to a lower threshold. - maxListLength = 2000 + maxListSize = 2000 defer func() { - maxListLength = 2000000 + maxListSize = 2000000 }() key := x.DataKey("bal", 1331) From f6125bae38afe0cb000369a73dc468d2caff2a4a Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 16:00:04 -0800 Subject: [PATCH 04/67] Several changes. - Load whole list at once. - Eliminate linked list. - Store all plists inside the List object. --- posting/list.go | 314 ++++++++++++++++++++----------------------- posting/list_test.go | 6 +- posting/mvcc.go | 3 + 3 files changed, 155 insertions(+), 168 deletions(-) diff --git a/posting/list.go b/posting/list.go index 19f0ca69f00..c7cbce0cf00 100644 --- a/posting/list.go +++ b/posting/list.go @@ -30,7 +30,6 @@ import ( "github.com/dgryski/go-farm" "github.com/golang/glog" - "github.com/dgraph-io/badger" bpb "github.com/dgraph-io/badger/pb" "github.com/dgraph-io/dgo/protos/api" "github.com/dgraph-io/dgo/y" @@ -72,14 +71,23 @@ type List struct { x.SafeMutex key []byte plist *pb.PostingList + parts []*pb.PostingList // If a multi-part list, the parts will be stored here. mutationMap map[uint64]*pb.PostingList minTs uint64 // commit timestamp of immutable layer, reject reads before this ts. maxTs uint64 // max commit timestamp seen for this list. pendingTxns int32 // Using atomic for this, to avoid locking in SetForDeletion operation. deleteMe int32 // Using atomic for this, to avoid expensive SetForDeletion operation. - next *List // If a multi-part list, this is a pointer to the next list. - first *List // If a multi-part list, points to the first part of the list. +} + +func getNextPartKey(baseKey []byte, nextPartStart uint64) []byte { + keyCopy := make([]byte, len(baseKey)) + copy(keyCopy, baseKey) + + encNexStart := make([]byte, 8) + binary.BigEndian.PutUint64(encNexStart, nextPartStart) + keyCopy = append(keyCopy, encNexStart...) + return keyCopy } func appendNextStartToKey(key []byte, nextPartStart uint64) []byte { @@ -132,21 +140,6 @@ func generateNextPartKey(currKey []byte, currPl *pb.PostingList, nextPartStart u return replaceNextStartInKey(currKey, nextPartStart) } -func (l *List) updateMinTs(readTs, minTs uint64) error { - l.AssertLock() - - curr := l - for curr != nil { - curr.minTs = minTs - if err := curr.loadNextPart(readTs); err != nil { - return err - } - curr = curr.next - } - - return nil -} - func (l *List) maxVersion() uint64 { l.RLock() defer l.RUnlock() @@ -155,6 +148,7 @@ func (l *List) maxVersion() uint64 { type PIterator struct { l *List + plist *pb.PostingList opts PItrOpts uidPosting *pb.Posting pidx int // index of postings @@ -164,33 +158,37 @@ type PIterator struct { dec *codec.Decoder uids []uint64 uidx int // Offset into the uids slice + + partIndex int // Offset into the parts list (if a multi-part list). } type PItrOpts struct { discardPl bool afterUid uint64 partialIteration bool + startPart int readTs uint64 } func (it *PIterator) Init(l *List, opts PItrOpts) { - if l.plist.FirstPart { - l.loadNextPart(opts.readTs) - it.Init(l.next, opts) - return + if l.plist.MultiPart { + it.plist = l.parts[opts.startPart] + } else { + it.plist = l.plist } + log.Printf("parts %v, currentPart %v", len(l.parts), opts.startPart) it.l = l it.opts = opts it.uidPosting = &pb.Posting{} - it.dec = &codec.Decoder{Pack: l.plist.Pack} + it.dec = &codec.Decoder{Pack: it.plist.Pack} it.uids = it.dec.Seek(opts.afterUid) it.uidx = 0 - it.plen = len(l.plist.Postings) + it.plen = len(it.plist.Postings) it.pidx = sort.Search(it.plen, func(idx int) bool { - p := l.plist.Postings[idx] + p := it.plist.Postings[idx] return it.opts.afterUid < p.Uid }) } @@ -223,15 +221,20 @@ func (it *PIterator) Valid() bool { return false } - // Load the next part of the list if it exists, and reinitialize the - // iterator to move to the start of this new part. - if err := it.l.loadNextPart(it.l.plist.CommitTs); err != nil { + // Not a multi-part list, so nothing else to iterate through + if !it.l.plist.MultiPart { return false } - if it.l.next == nil { + + // No more parts to iterate through + if len(it.l.parts) == it.opts.startPart+1 { return false } - it.Init(it.l.next, it.opts) + + it.opts.startPart++ + it.Init(it.l, it.opts) + + // TODO: corner case, what if next part is empty but the one after that is not. return len(it.uids) > 0 } @@ -239,7 +242,7 @@ func (it *PIterator) Posting() *pb.Posting { uid := it.uids[it.uidx] for it.pidx < it.plen { - p := it.l.plist.Postings[it.pidx] + p := it.plist.Postings[it.pidx] if p.Uid > uid { break } @@ -558,14 +561,15 @@ func (l *List) Conflicts(readTs uint64) []uint64 { } func (l *List) pickPostings(readTs uint64) (uint64, []*pb.Posting) { - return l.pickPostingsInternal(readTs, false) + return l.pickPostingsInternal(readTs, false, 0) } -func (l *List) pickPartPostings(readTs uint64) (uint64, []*pb.Posting) { - return l.pickPostingsInternal(readTs, true) +func (l *List) pickPartPostings(readTs uint64, partIdx int) (uint64, []*pb.Posting) { + return l.pickPostingsInternal(readTs, true, partIdx) } -func (l *List) pickPostingsInternal(readTs uint64, partial bool) (uint64, []*pb.Posting) { +func (l *List) pickPostingsInternal(readTs uint64, partial bool, partIdx int) ( + uint64, []*pb.Posting) { // This function would return zero ts for entries above readTs. effective := func(start, commit uint64) uint64 { if commit > 0 && commit <= readTs { @@ -582,17 +586,14 @@ func (l *List) pickPostingsInternal(readTs uint64, partial bool) (uint64, []*pb. // First pick up the postings. var deleteBelow uint64 var posts []*pb.Posting - mutationMap := l.mutationMap - if l.first != nil { - mutationMap = l.first.mutationMap - } - for startTs, plist := range mutationMap { + for startTs, plist := range l.mutationMap { // Pick up the transactions which are either committed, or the one which is ME. effectiveTs := effective(startTs, plist.CommitTs) if effectiveTs > deleteBelow { // We're above the deleteBelow marker. We wouldn't reach here if effectiveTs is zero. for _, mpost := range plist.Postings { - if partial && (mpost.Uid < l.plist.StartUid || l.plist.EndUid < mpost.Uid) { + if partial && (mpost.Uid < l.parts[partIdx].StartUid || + l.parts[partIdx].EndUid < mpost.Uid) { continue } if hasDeleteAll(mpost) { @@ -708,15 +709,12 @@ func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) e return err } -func (l *List) partIterate(readTs uint64, f func(obj *pb.Posting) error) error { +func (l *List) partIterate(readTs uint64, partIdx int, f func(obj *pb.Posting) error) error { if !l.plist.MultiPart { return l.iterate(readTs, 0, f) } - if l.plist.FirstPart { - return nil - } - deleteBelow, mposts := l.pickPartPostings(readTs) + deleteBelow, mposts := l.pickPartPostings(readTs, partIdx) if readTs < l.minTs { return x.Errorf("readTs: %d less than minTs: %d for key: %q", readTs, l.minTs, l.key) } @@ -728,17 +726,20 @@ func (l *List) partIterate(readTs uint64, f func(obj *pb.Posting) error) error { afterUid: 0, discardPl: deleteBelow > 0, partialIteration: true, + startPart: partIdx, }) prevUid := uint64(0) var err error for err == nil { if midx < mlen { mp = mposts[midx] + log.Printf("mp %v", mp.Uid) } else { mp = emptyPosting } if pitr.Valid() { pp = pitr.Posting() + log.Printf("pp %v", pp.Uid) } else { pp = emptyPosting } @@ -825,21 +826,22 @@ func (l *List) MarshalToKv() ([]*bpb.KV, error) { } var kvs []*bpb.KV - curr := l - for curr != nil { + kv := &bpb.KV{} + kv.Version = l.minTs + kv.Key = l.key + val, meta := marshalPostingList(l.plist) + kv.UserMeta = []byte{meta} + kv.Value = val + kvs = append(kvs, kv) + + for _, part := range l.parts { kv := &bpb.KV{} - kv.Version = curr.minTs - kv.Key = curr.key - val, meta := marshalPostingList(curr.plist) + kv.Version = l.minTs + kv.Key = getNextPartKey(l.key, part.StartUid) + val, meta := marshalPostingList(part) kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) - - // Load next part of the list if necessary. - if err := curr.loadNextPart(curr.plist.CommitTs); err != nil { - return nil, err - } - curr = curr.next } return kvs, nil @@ -878,18 +880,11 @@ func (l *List) rollup(readTs uint64) error { return fmt.Errorf("rollup can only be called from the first part of a multi-part list") } - // Check if the list (or any of it's parts if it's been previously split) have - // become too big. Split the list if that is the case. - if err := l.splitList(readTs); err != nil { - return nil - } - - curr := l - for curr != nil { + if !l.plist.MultiPart { final := new(pb.PostingList) enc := codec.Encoder{BlockSize: blockSize} - err := curr.partIterate(readTs, func(p *pb.Posting) error { + err := l.iterate(readTs, 0, func(p *pb.Posting) error { // iterate already takes care of not returning entries whose commitTs // is above curr.commitTs. // So, we don't need to do any filtering here. In fact, doing filtering @@ -907,13 +902,36 @@ func (l *List) rollup(readTs uint64) error { }) x.Check(err) final.Pack = enc.Done() - curr.plist.Pack = final.Pack - curr.plist.Postings = final.Postings - - if err := curr.loadNextPart(readTs); err != nil { - return err + l.plist.Pack = final.Pack + l.plist.Postings = final.Postings + } else { + for partIdx, part := range l.parts { + final := new(pb.PostingList) + + log.Printf("part start %v part end %v", part.StartUid, part.EndUid) + enc := codec.Encoder{BlockSize: blockSize} + err := l.partIterate(readTs, partIdx, func(p *pb.Posting) error { + // iterate already takes care of not returning entries whose commitTs + // is above curr.commitTs. + // So, we don't need to do any filtering here. In fact, doing filtering + // here could result in a bug. + enc.Add(p.Uid) + + log.Printf("rolling up %v", p.Uid) + // We want to add the posting if it has facets or has a value. + if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { + // I think it's okay to take the pointer from the iterator, because + // we have a lock over List; which won't be released until final has + // been marshalled. Thus, the underlying data wouldn't be changed. + final.Postings = append(final.Postings, p) + } + return nil + }) + x.Check(err) + final.Pack = enc.Done() + part.Pack = final.Pack + part.Postings = final.Postings } - curr = curr.next } maxCommitTs := l.minTs @@ -941,7 +959,14 @@ func (l *List) rollup(readTs uint64) error { } } - l.updateMinTs(readTs, maxCommitTs) + l.minTs = maxCommitTs + + // Check if the list (or any of it's parts if it's been previously split) have + // become too big. Split the list if that is the case. + if err := l.splitList(readTs); err != nil { + return nil + } + return nil } @@ -1198,87 +1223,73 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs return facets.CopyFacets(p.Facets, param), nil } -func (l *List) loadNextPart(readTs uint64) error { - // No plist so there's nothing that can be loaded. - if l.plist == nil { - return nil - } - // This is not a multi-part list so nothing to load. +func (l *List) readListParts(baseKey []byte, readTs uint64) error { if !l.plist.MultiPart { return nil } - // The next part has already been loaded so nothing else to do. - if l.next != nil { - return nil - } - // This is the end of the multi-part list so there's nothing to do. - if l.plist.EndUid == math.MaxUint64 { - return nil - } - - txn := pstore.NewTransactionAt(readTs, false) - opts := badger.DefaultIteratorOptions - opts.AllVersions = true - it := txn.NewIterator(opts) - nextPartKey := l.getNextPartKey() - if nextPartKey == nil { - return fmt.Errorf( - "Could not get find key for next part of the posting list. Current key %v", l.key) - } + var nextPartStart uint64 + for { + nextKey := getNextPartKey(baseKey, nextPartStart) + txn := pstore.NewTransactionAt(readTs, false) + item, err := txn.Get(nextKey) + if err != nil { + return err + } + var part pb.PostingList + if err := unmarshalOrCopy(&part, item); err != nil { + return err + } + l.parts = append(l.parts, &part) - nextListPart, err := ReadPostingList(nextPartKey, it) - if err != nil { - return err + nextPartStart = part.EndUid + if nextPartStart == math.MaxUint64 { + break + } + nextPartStart++ } - l.next = nextListPart - return nil } -func (l *List) needsSplit() bool { - return l.plist.Size() >= maxListSize +func needsSplit(plist *pb.PostingList) bool { + return plist.Size() >= maxListSize } func (l *List) splitList(readTs uint64) error { l.AssertLock() - curr := l - var prev *List - for curr != nil { - - if curr.needsSplit() { - newLists := curr.splitListPart(readTs) - - // If splitting a list for the first time, initialize it as a - // multi-part list. - if !curr.plist.MultiPart { - curr.plist = &pb.PostingList{ - CommitTs: curr.plist.CommitTs, - MultiPart: true, - FirstPart: true, - } - curr.next = newLists[0] - } else { - prev.next = newLists[0] + if !l.plist.MultiPart { + if needsSplit(l.plist) { + log.Printf("first split") + l.parts = l.splitPostingList(readTs, 0) + l.plist = &pb.PostingList{ + CommitTs: l.plist.CommitTs, + MultiPart: true, + FirstPart: true, } - - curr = newLists[1] } + return nil + } - if err := curr.loadNextPart(readTs); err != nil { - return err + var newParts []*pb.PostingList + for partIdx, part := range l.parts { + if needsSplit(part) { + log.Printf("split part %v", partIdx) + splitParts := l.splitPostingList(readTs, partIdx) + newParts = append(newParts, splitParts...) + } else { + newParts = append(newParts, part) } - - prev = curr - curr = curr.next } + + l.parts = newParts return nil } -func (l *List) splitListPart(readTs uint64) []*List { +func (l *List) splitPostingList(readTs uint64, partIdx int) []*pb.PostingList { + log.Printf("split posting list") var uids []uint64 - err := l.partIterate(readTs, func(p *pb.Posting) error { + err := l.partIterate(readTs, partIdx, func(p *pb.Posting) error { uids = append(uids, p.Uid) return nil }) @@ -1290,7 +1301,7 @@ func (l *List) splitListPart(readTs uint64) []*List { // Generate posting list holding the first half of the current list's postings. lowPl := new(pb.PostingList) lowEnc := codec.Encoder{BlockSize: blockSize} - err = l.partIterate(readTs, func(p *pb.Posting) error { + err = l.partIterate(readTs, partIdx, func(p *pb.Posting) error { // Skip all postings with an UID greater than or equal to midUid. if p.Uid >= midUid { return nil @@ -1308,24 +1319,17 @@ func (l *List) splitListPart(readTs uint64) []*List { lowPl.CommitTs = l.plist.CommitTs lowPl.MultiPart = true lowPl.FirstPart = false - lowPl.StartUid = l.plist.StartUid - lowPl.EndUid = midUid - 1 - - // Generate first list. - lowList := &List{} if !l.plist.MultiPart { - lowList.key = generateNextPartKey(l.key, l.plist, 0) + lowPl.StartUid = 0 } else { - lowList.key = generateNextPartKey(l.key, l.plist, l.plist.StartUid) + lowPl.StartUid = l.parts[partIdx].StartUid } - lowList.minTs = l.minTs - lowList.maxTs = l.maxTs - lowList.plist = lowPl + lowPl.EndUid = midUid - 1 // Generate posting list holding the second half of the current list's postings. highPl := new(pb.PostingList) highEnc := codec.Encoder{BlockSize: blockSize} - err = l.partIterate(readTs, func(p *pb.Posting) error { + err = l.partIterate(readTs, partIdx, func(p *pb.Posting) error { // Skip all postings with an UID less than midUid. if p.Uid < midUid { return nil @@ -1350,28 +1354,8 @@ func (l *List) splitListPart(readTs uint64) []*List { } else { // Else, this posting list should point to the part the original list // is pointing to. - highPl.EndUid = l.plist.EndUid - } - - // Generate second list. - highList := &List{} - highList.key = generateNextPartKey(l.key, l.plist, midUid) - highList.minTs = l.minTs - highList.maxTs = l.maxTs - highList.plist = highPl - - // Link the two lists together and to the rest of the list. - lowList.next = highList - highList.next = l.next - - // Add a pointer to the start of the list. - if !l.plist.MultiPart { - lowList.first = l - highList.first = l - } else { - lowList.first = l.first - highList.first = l.first + highPl.EndUid = l.parts[partIdx].EndUid } - return []*List{lowList, highList} + return []*pb.PostingList{lowPl, highPl} } diff --git a/posting/list_test.go b/posting/list_test.go index c2b874dc751..923c2a36590 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -904,7 +904,7 @@ func BenchmarkAddMutations(b *testing.B) { func TestMultiPartList(t *testing.T) { // For testing, set the max list length to a lower threshold. - maxListSize = 2000 + maxListSize = 50 defer func() { maxListSize = 2000000 }() @@ -913,7 +913,7 @@ func TestMultiPartList(t *testing.T) { ol, err := getNew(key, ps) require.NoError(t, err) var commits int - N := int(1e5) + N := int(200) for i := 2; i <= N; i += 2 { edge := &pb.DirectedEdge{ ValueId: uint64(i), @@ -921,7 +921,7 @@ func TestMultiPartList(t *testing.T) { txn := Txn{StartTs: uint64(i)} addMutationHelper(t, ol, edge, Set, &txn) require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%2000 == 0 { + if i%50 == 0 { // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting // list. t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) diff --git a/posting/mvcc.go b/posting/mvcc.go index 34bb5d365f0..30d78f25656 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -208,6 +208,9 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { return nil, err } l.minTs = item.Version() + if err := l.readListParts(key, item.Version()); err != nil { + return nil, err + } // No need to do Next here. The outer loop can take care of skipping more versions of // the same key. return l, nil From b572a390cec2853408e47b30083f22d2cc48519a Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 16:21:18 -0800 Subject: [PATCH 05/67] Increase test size and remove print statements --- posting/list.go | 8 -------- posting/list_test.go | 6 +++--- 2 files changed, 3 insertions(+), 11 deletions(-) diff --git a/posting/list.go b/posting/list.go index c7cbce0cf00..59f5871d1aa 100644 --- a/posting/list.go +++ b/posting/list.go @@ -176,7 +176,6 @@ func (it *PIterator) Init(l *List, opts PItrOpts) { } else { it.plist = l.plist } - log.Printf("parts %v, currentPart %v", len(l.parts), opts.startPart) it.l = l it.opts = opts @@ -733,13 +732,11 @@ func (l *List) partIterate(readTs uint64, partIdx int, f func(obj *pb.Posting) e for err == nil { if midx < mlen { mp = mposts[midx] - log.Printf("mp %v", mp.Uid) } else { mp = emptyPosting } if pitr.Valid() { pp = pitr.Posting() - log.Printf("pp %v", pp.Uid) } else { pp = emptyPosting } @@ -908,7 +905,6 @@ func (l *List) rollup(readTs uint64) error { for partIdx, part := range l.parts { final := new(pb.PostingList) - log.Printf("part start %v part end %v", part.StartUid, part.EndUid) enc := codec.Encoder{BlockSize: blockSize} err := l.partIterate(readTs, partIdx, func(p *pb.Posting) error { // iterate already takes care of not returning entries whose commitTs @@ -917,7 +913,6 @@ func (l *List) rollup(readTs uint64) error { // here could result in a bug. enc.Add(p.Uid) - log.Printf("rolling up %v", p.Uid) // We want to add the posting if it has facets or has a value. if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { // I think it's okay to take the pointer from the iterator, because @@ -1260,7 +1255,6 @@ func (l *List) splitList(readTs uint64) error { if !l.plist.MultiPart { if needsSplit(l.plist) { - log.Printf("first split") l.parts = l.splitPostingList(readTs, 0) l.plist = &pb.PostingList{ CommitTs: l.plist.CommitTs, @@ -1274,7 +1268,6 @@ func (l *List) splitList(readTs uint64) error { var newParts []*pb.PostingList for partIdx, part := range l.parts { if needsSplit(part) { - log.Printf("split part %v", partIdx) splitParts := l.splitPostingList(readTs, partIdx) newParts = append(newParts, splitParts...) } else { @@ -1287,7 +1280,6 @@ func (l *List) splitList(readTs uint64) error { } func (l *List) splitPostingList(readTs uint64, partIdx int) []*pb.PostingList { - log.Printf("split posting list") var uids []uint64 err := l.partIterate(readTs, partIdx, func(p *pb.Posting) error { uids = append(uids, p.Uid) diff --git a/posting/list_test.go b/posting/list_test.go index 923c2a36590..694b68fe501 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -904,7 +904,7 @@ func BenchmarkAddMutations(b *testing.B) { func TestMultiPartList(t *testing.T) { // For testing, set the max list length to a lower threshold. - maxListSize = 50 + maxListSize = 1000 defer func() { maxListSize = 2000000 }() @@ -913,7 +913,7 @@ func TestMultiPartList(t *testing.T) { ol, err := getNew(key, ps) require.NoError(t, err) var commits int - N := int(200) + N := int(1e5) for i := 2; i <= N; i += 2 { edge := &pb.DirectedEdge{ ValueId: uint64(i), @@ -921,7 +921,7 @@ func TestMultiPartList(t *testing.T) { txn := Txn{StartTs: uint64(i)} addMutationHelper(t, ol, edge, Set, &txn) require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%50 == 0 { + if i%1000 == 0 { // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting // list. t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) From e9d3f05cbdd267c6edec6b2ab06439a24edc7a96 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 16:32:30 -0800 Subject: [PATCH 06/67] Combine partial and complete iterations into the same function. --- posting/list.go | 94 +++++++++----------------------------------- posting/list_test.go | 3 +- 2 files changed, 21 insertions(+), 76 deletions(-) diff --git a/posting/list.go b/posting/list.go index 59f5871d1aa..5b9d482d9ff 100644 --- a/posting/list.go +++ b/posting/list.go @@ -632,9 +632,24 @@ func (l *List) pickPostingsInternal(readTs uint64, partial bool, partIdx int) ( } func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) error) error { + return l.iterateInternal(readTs, afterUid, false, 0, f) +} + +func (l *List) partIterate(readTs uint64, partIdx int, f func(obj *pb.Posting) error) error { + return l.iterateInternal(readTs, 0, true, partIdx, f) +} + +func (l *List) iterateInternal(readTs uint64, afterUid uint64, partial bool, partIdx int, + f func(obj *pb.Posting) error) error { l.AssertRLock() - deleteBelow, mposts := l.pickPostings(readTs) + // If not a multi-part list iterate through the whole list. + if !l.plist.MultiPart { + partial = false + partIdx = 0 + } + + deleteBelow, mposts := l.pickPostingsInternal(readTs, partial, partIdx) if readTs < l.minTs { return x.Errorf("readTs: %d less than minTs: %d for key: %q", readTs, l.minTs, l.key) } @@ -650,81 +665,10 @@ func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) e var mp, pp *pb.Posting var pitr PIterator pitr.Init(l, PItrOpts{ - afterUid: afterUid, - discardPl: deleteBelow > 0, - readTs: readTs, - }) - prevUid := uint64(0) - var err error - for err == nil { - if midx < mlen { - mp = mposts[midx] - } else { - mp = emptyPosting - } - if pitr.Valid() { - pp = pitr.Posting() - } else { - pp = emptyPosting - } - - switch { - case mp.Uid > 0 && mp.Uid == prevUid: - // Only pick the latest version of this posting. - // mp.Uid can be zero if it's an empty posting. - midx++ - case pp.Uid == 0 && mp.Uid == 0: - // Reached empty posting for both iterators. - return nil - case mp.Uid == 0 || (pp.Uid > 0 && pp.Uid < mp.Uid): - // Either mp is empty, or pp is lower than mp. - err = f(pp) - if err := pitr.Next(); err != nil { - return err - } - case pp.Uid == 0 || (mp.Uid > 0 && mp.Uid < pp.Uid): - // Either pp is empty, or mp is lower than pp. - if mp.Op != Del { - err = f(mp) - } - prevUid = mp.Uid - midx++ - case pp.Uid == mp.Uid: - if mp.Op != Del { - err = f(mp) - } - prevUid = mp.Uid - if err := pitr.Next(); err != nil { - return err - } - midx++ - default: - log.Fatalf("Unhandled case during iteration of posting list.") - } - } - if err == ErrStopIteration { - return nil - } - return err -} - -func (l *List) partIterate(readTs uint64, partIdx int, f func(obj *pb.Posting) error) error { - if !l.plist.MultiPart { - return l.iterate(readTs, 0, f) - } - - deleteBelow, mposts := l.pickPartPostings(readTs, partIdx) - if readTs < l.minTs { - return x.Errorf("readTs: %d less than minTs: %d for key: %q", readTs, l.minTs, l.key) - } - - midx, mlen := 0, len(mposts) - var mp, pp *pb.Posting - var pitr PIterator - pitr.Init(l, PItrOpts{ - afterUid: 0, + afterUid: afterUid, discardPl: deleteBelow > 0, - partialIteration: true, + readTs: readTs, + partialIteration: partial, startPart: partIdx, }) prevUid := uint64(0) diff --git a/posting/list_test.go b/posting/list_test.go index 694b68fe501..46812cf506b 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -904,7 +904,7 @@ func BenchmarkAddMutations(b *testing.B) { func TestMultiPartList(t *testing.T) { // For testing, set the max list length to a lower threshold. - maxListSize = 1000 + maxListSize = 10000 defer func() { maxListSize = 2000000 }() @@ -929,6 +929,7 @@ func TestMultiPartList(t *testing.T) { } commits++ } + t.Logf("List parts %v", len(ol.parts)) opt := ListOptions{ReadTs: uint64(N) + 1} l, err := ol.Uids(opt) require.NoError(t, err) From 9f8d54f070520678e4251f06053d124a6c1dfb00 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 17:08:35 -0800 Subject: [PATCH 07/67] Remove unused function and optimize rollup. --- posting/list.go | 57 ++++++------------------------------------------- 1 file changed, 7 insertions(+), 50 deletions(-) diff --git a/posting/list.go b/posting/list.go index 5b9d482d9ff..0436d58db96 100644 --- a/posting/list.go +++ b/posting/list.go @@ -90,56 +90,6 @@ func getNextPartKey(baseKey []byte, nextPartStart uint64) []byte { return keyCopy } -func appendNextStartToKey(key []byte, nextPartStart uint64) []byte { - keyCopy := make([]byte, len(key)) - copy(keyCopy, key) - - encNexStart := make([]byte, 8) - binary.BigEndian.PutUint64(encNexStart, nextPartStart) - keyCopy = append(keyCopy, encNexStart...) - return keyCopy -} - -func replaceNextStartInKey(key []byte, nextPartStart uint64) []byte { - keyCopy := make([]byte, len(key)) - copy(keyCopy, key) - - rest := keyCopy[len(keyCopy)-8:] - binary.BigEndian.PutUint64(rest, nextPartStart) - return keyCopy -} - -func (l *List) getNextPartKey() []byte { - if l.plist != nil { - return nil - } - - if !l.plist.MultiPart { - return nil - } - - if l.plist.EndUid == math.MaxUint64 { - return nil - } - - if l.plist.FirstPart { - // Add the start of the next list to the end of the key. - return appendNextStartToKey(l.key, l.plist.EndUid+1) - } - - // In this case, the key already includes the extra bytes to encode the start - // UID of this part. Replace those bytes with the start UID of the next part. - return replaceNextStartInKey(l.key, l.plist.EndUid+1) -} - -func generateNextPartKey(currKey []byte, currPl *pb.PostingList, nextPartStart uint64) []byte { - appendToKey := currPl.FirstPart || !currPl.MultiPart - if appendToKey { - return appendNextStartToKey(currKey, nextPartStart) - } - return replaceNextStartInKey(currKey, nextPartStart) -} - func (l *List) maxVersion() uint64 { l.RLock() defer l.RUnlock() @@ -847,6 +797,13 @@ func (l *List) rollup(readTs uint64) error { l.plist.Postings = final.Postings } else { for partIdx, part := range l.parts { + // If no changes have been made to this part, rollup does not need to + // change this list part. + deleteBelow, mposts := l.pickPartPostings(readTs, partIdx) + if deleteBelow == 0 && len(mposts) == 0 { + continue + } + final := new(pb.PostingList) enc := codec.Encoder{BlockSize: blockSize} From 4dc93976cc996a255b2375d3245fc3415d156fed Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 17:08:51 -0800 Subject: [PATCH 08/67] Add extra test. --- posting/list_test.go | 44 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/posting/list_test.go b/posting/list_test.go index 46812cf506b..f288773a1bf 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -938,3 +938,47 @@ func TestMultiPartList(t *testing.T) { require.Equal(t, uint64(i+1)*2, uid) } } + +func TestMultiPartListWithPostings(t *testing.T) { + // For testing, set the max list length to a lower threshold. + maxListSize = 10000 + defer func() { + maxListSize = 2000000 + }() + + key := x.DataKey("bal", 1331) + ol, err := getNew(key, ps) + require.NoError(t, err) + var commits int + N := int(1e5) + for i := 2; i <= N; i += 2 { + edge := &pb.DirectedEdge{ + ValueId: uint64(i), + Label: strconv.Itoa(i), + } + txn := Txn{StartTs: uint64(i)} + addMutationHelper(t, ol, edge, Set, &txn) + require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) + if i%1000 == 0 { + // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting + // list. + t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) + require.NoError(t, ol.Rollup(math.MaxUint64)) + } + commits++ + } + t.Logf("List parts %v", len(ol.parts)) + + var labels []string + ol.Iterate(uint64(N) + 1, 0, func(p *pb.Posting) error { + if len(p.Label) > 0 { + labels = append(labels, p.Label) + } + return nil + }) + require.NoError(t, err) + require.Equal(t, commits, len(labels)) + for i, label := range labels { + require.Equal(t, label, strconv.Itoa(int(i+1)*2)) + } +} From fc4c183c942ee41cef325c93c2b6c1d2ce95f927 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 18:00:21 -0800 Subject: [PATCH 09/67] Add extra tests --- posting/list_test.go | 42 ++++++++++++++++++++++++++++++++++++++---- 1 file changed, 38 insertions(+), 4 deletions(-) diff --git a/posting/list_test.go b/posting/list_test.go index f288773a1bf..cfb03cde988 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -902,7 +902,12 @@ func BenchmarkAddMutations(b *testing.B) { } } -func TestMultiPartList(t *testing.T) { +func createMultiPartList(t *testing.T, size int) (*List, int) { + // For testing, set the max list size to a lower threshold. + maxListSize = 10000 + defer func() { + maxListSize = 2000000 + }() // For testing, set the max list length to a lower threshold. maxListSize = 10000 defer func() { @@ -912,9 +917,8 @@ func TestMultiPartList(t *testing.T) { key := x.DataKey("bal", 1331) ol, err := getNew(key, ps) require.NoError(t, err) - var commits int - N := int(1e5) - for i := 2; i <= N; i += 2 { + commits := 0 + for i := 2; i <= size; i += 2 { edge := &pb.DirectedEdge{ ValueId: uint64(i), } @@ -929,6 +933,13 @@ func TestMultiPartList(t *testing.T) { } commits++ } + + return ol, commits +} + +func TestMultiPartList(t *testing.T) { + N := int(1e5) + ol, commits := createMultiPartList(t, N) t.Logf("List parts %v", len(ol.parts)) opt := ListOptions{ReadTs: uint64(N) + 1} l, err := ol.Uids(opt) @@ -982,3 +993,26 @@ func TestMultiPartListWithPostings(t *testing.T) { require.Equal(t, label, strconv.Itoa(int(i+1)*2)) } } + +func TestMultiPartListMarshal(t *testing.T) { + N := int(1e5) + ol, _ := createMultiPartList(t, N) + t.Logf("List parts %v", len(ol.parts)) + + kvs, err := ol.MarshalToKv() + require.NoError(t, err) + require.Equal(t, len(kvs), len(ol.parts) + 1) + + key := x.DataKey("bal", 1331) + require.Equal(t, key, kvs[0].Key) + + for i, part := range ol.parts { + partKey := getNextPartKey(key, part.StartUid) + require.Equal(t, partKey, kvs[i+1].Key) + data, err := part.Marshal() + require.NoError(t, err) + require.Equal(t, data, kvs[i+1].Value) + require.Equal(t, []byte{BitCompletePosting}, kvs[i+1].UserMeta) + require.Equal(t, ol.minTs, kvs[i+1].Version) + } +} From 75db0f5af4e62b58713d20b575f7c69ee48388ab Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 18:08:16 -0800 Subject: [PATCH 10/67] Remove first_part field from proto. --- posting/list.go | 7 - protos/pb.proto | 5 +- protos/pb/pb.pb.go | 474 +++++++++++++++++++++------------------------ 3 files changed, 218 insertions(+), 268 deletions(-) diff --git a/posting/list.go b/posting/list.go index 0436d58db96..d1ebf06db32 100644 --- a/posting/list.go +++ b/posting/list.go @@ -767,10 +767,6 @@ func (l *List) rollup(readTs uint64) error { return nil } - if l.plist.MultiPart && !l.plist.FirstPart { - return fmt.Errorf("rollup can only be called from the first part of a multi-part list") - } - if !l.plist.MultiPart { final := new(pb.PostingList) @@ -1160,7 +1156,6 @@ func (l *List) splitList(readTs uint64) error { l.plist = &pb.PostingList{ CommitTs: l.plist.CommitTs, MultiPart: true, - FirstPart: true, } } return nil @@ -1211,7 +1206,6 @@ func (l *List) splitPostingList(readTs uint64, partIdx int) []*pb.PostingList { lowPl.Pack = lowEnc.Done() lowPl.CommitTs = l.plist.CommitTs lowPl.MultiPart = true - lowPl.FirstPart = false if !l.plist.MultiPart { lowPl.StartUid = 0 } else { @@ -1238,7 +1232,6 @@ func (l *List) splitPostingList(readTs uint64, partIdx int) []*pb.PostingList { highPl.Pack = highEnc.Done() highPl.CommitTs = l.plist.CommitTs highPl.MultiPart = true - highPl.FirstPart = false highPl.StartUid = midUid if !l.plist.MultiPart { // We are splitting a list that was previously non-split. So the new diff --git a/protos/pb.proto b/protos/pb.proto index bdd88e1051c..b6a3892ec0c 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -279,9 +279,8 @@ message PostingList { uint64 commit_ts = 3; // More inclination towards smaller values. bool multi_part = 4; - bool first_part = 5; - uint64 start_uid = 6; - uint64 end_uid = 7; + uint64 start_uid = 5; + uint64 end_uid = 6; } message FacetParam { diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index 4de52e1a643..c2595995d8c 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -2101,9 +2101,8 @@ type PostingList struct { Postings []*Posting `protobuf:"bytes,2,rep,name=postings,proto3" json:"postings,omitempty"` CommitTs uint64 `protobuf:"varint,3,opt,name=commit_ts,json=commitTs,proto3" json:"commit_ts,omitempty"` MultiPart bool `protobuf:"varint,4,opt,name=multi_part,json=multiPart,proto3" json:"multi_part,omitempty"` - FirstPart bool `protobuf:"varint,5,opt,name=first_part,json=firstPart,proto3" json:"first_part,omitempty"` - StartUid uint64 `protobuf:"varint,6,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` - EndUid uint64 `protobuf:"varint,7,opt,name=end_uid,json=endUid,proto3" json:"end_uid,omitempty"` + StartUid uint64 `protobuf:"varint,5,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` + EndUid uint64 `protobuf:"varint,6,opt,name=end_uid,json=endUid,proto3" json:"end_uid,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2170,13 +2169,6 @@ func (m *PostingList) GetMultiPart() bool { return false } -func (m *PostingList) GetFirstPart() bool { - if m != nil { - return m.FirstPart - } - return false -} - func (m *PostingList) GetStartUid() uint64 { if m != nil { return m.StartUid @@ -3653,218 +3645,217 @@ func init() { func init() { proto.RegisterFile("pb.proto", fileDescriptor_f80abaa17e25ccc8) } var fileDescriptor_f80abaa17e25ccc8 = []byte{ - // 3368 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0xcd, 0x73, 0xdb, 0x48, - 0x76, 0x17, 0x40, 0x12, 0x04, 0x1e, 0x29, 0x99, 0xd3, 0xf6, 0x7a, 0x38, 0xf2, 0xae, 0xac, 0xc1, - 0xd8, 0x63, 0xd9, 0x1e, 0xcb, 0x1e, 0xcd, 0x24, 0x59, 0x6f, 0x2a, 0x07, 0x7d, 0xd0, 0x8e, 0x66, - 0xf4, 0x95, 0x26, 0xe5, 0x4d, 0xa6, 0x52, 0x61, 0x35, 0x81, 0x16, 0x85, 0x15, 0x08, 0x20, 0x68, - 0x50, 0x45, 0xf9, 0x96, 0x43, 0xf2, 0x37, 0xec, 0x21, 0x95, 0x5b, 0x2e, 0xb9, 0xe4, 0xba, 0xc7, - 0x5c, 0x52, 0x95, 0x63, 0x2a, 0x7f, 0x41, 0xca, 0xc9, 0x21, 0xf7, 0x54, 0xa5, 0x2a, 0xb7, 0x54, - 0xbf, 0x6e, 0x7c, 0x90, 0x96, 0xec, 0x9d, 0x54, 0xed, 0x89, 0x78, 0x1f, 0xfd, 0xf5, 0xeb, 0xf7, - 0x5e, 0xbf, 0xf7, 0x08, 0x76, 0x32, 0xda, 0x4c, 0xd2, 0x38, 0x8b, 0x89, 0x99, 0x8c, 0x56, 0x1d, - 0x96, 0x04, 0x8a, 0x5c, 0x7d, 0x34, 0x0e, 0xb2, 0xf3, 0xe9, 0x68, 0xd3, 0x8b, 0x27, 0xcf, 0xfd, - 0x71, 0xca, 0x92, 0xf3, 0x67, 0x41, 0xfc, 0x7c, 0xc4, 0xfc, 0x31, 0x4f, 0x9f, 0x27, 0xa3, 0xe7, - 0xf9, 0x38, 0x77, 0x15, 0xea, 0x07, 0x81, 0xc8, 0x08, 0x81, 0xfa, 0x34, 0xf0, 0x45, 0xd7, 0x58, - 0xaf, 0x6d, 0x58, 0x14, 0xbf, 0xdd, 0x43, 0x70, 0x06, 0x4c, 0x5c, 0xbc, 0x61, 0xe1, 0x94, 0x93, - 0x0e, 0xd4, 0x2e, 0x59, 0xd8, 0x35, 0xd6, 0x8d, 0x8d, 0x36, 0x95, 0x9f, 0x64, 0x13, 0xec, 0x4b, - 0x16, 0x0e, 0xb3, 0xab, 0x84, 0x77, 0xcd, 0x75, 0x63, 0x63, 0x65, 0xeb, 0xf6, 0x66, 0x32, 0xda, - 0x3c, 0x89, 0x45, 0x16, 0x44, 0xe3, 0xcd, 0x37, 0x2c, 0x1c, 0x5c, 0x25, 0x9c, 0x36, 0x2f, 0xd5, - 0x87, 0x7b, 0x0c, 0xad, 0x7e, 0xea, 0xbd, 0x9a, 0x46, 0x5e, 0x16, 0xc4, 0x91, 0x5c, 0x31, 0x62, - 0x13, 0x8e, 0x33, 0x3a, 0x14, 0xbf, 0x25, 0x8f, 0xa5, 0x63, 0xd1, 0xad, 0xad, 0xd7, 0x24, 0x4f, - 0x7e, 0x93, 0x2e, 0x34, 0x03, 0xb1, 0x1b, 0x4f, 0xa3, 0xac, 0x5b, 0x5f, 0x37, 0x36, 0x6c, 0x9a, - 0x93, 0xee, 0x7f, 0x9b, 0xd0, 0xf8, 0x93, 0x29, 0x4f, 0xaf, 0x70, 0x5c, 0x96, 0xa5, 0xf9, 0x5c, - 0xf2, 0x9b, 0xdc, 0x81, 0x46, 0xc8, 0xa2, 0xb1, 0xe8, 0x9a, 0x38, 0x99, 0x22, 0xc8, 0x3d, 0x70, - 0xd8, 0x59, 0xc6, 0xd3, 0xe1, 0x34, 0xf0, 0xbb, 0xb5, 0x75, 0x63, 0xc3, 0xa2, 0x36, 0x32, 0x4e, - 0x03, 0x9f, 0x7c, 0x06, 0xb6, 0x1f, 0x0f, 0xbd, 0xea, 0x5a, 0x7e, 0x8c, 0x6b, 0x91, 0x2f, 0xc0, - 0x9e, 0x06, 0xfe, 0x30, 0x0c, 0x44, 0xd6, 0x6d, 0xac, 0x1b, 0x1b, 0xad, 0x2d, 0x5b, 0x1e, 0x56, - 0x62, 0x47, 0x9b, 0xd3, 0xc0, 0x47, 0x10, 0x9f, 0x80, 0x2d, 0x52, 0x6f, 0x78, 0x36, 0x8d, 0xbc, - 0xae, 0x85, 0x4a, 0xb7, 0xa4, 0x52, 0xe5, 0xd4, 0xb4, 0x29, 0x14, 0x21, 0x8f, 0x95, 0xf2, 0x4b, - 0x9e, 0x0a, 0xde, 0x6d, 0xaa, 0xa5, 0x34, 0x49, 0x5e, 0x40, 0xeb, 0x8c, 0x79, 0x3c, 0x1b, 0x26, - 0x2c, 0x65, 0x93, 0xae, 0x5d, 0x4e, 0xf4, 0x4a, 0xb2, 0x4f, 0x24, 0x57, 0x50, 0x38, 0x2b, 0x08, - 0xf2, 0x0d, 0x2c, 0x23, 0x25, 0x86, 0x67, 0x41, 0x98, 0xf1, 0xb4, 0xeb, 0xe0, 0x98, 0x15, 0x1c, - 0x83, 0x9c, 0x41, 0xca, 0x39, 0x6d, 0x2b, 0x25, 0xc5, 0x21, 0x3f, 0x03, 0xe0, 0xb3, 0x84, 0x45, - 0xfe, 0x90, 0x85, 0x61, 0x17, 0x70, 0x0f, 0x8e, 0xe2, 0x6c, 0x87, 0x21, 0xf9, 0x54, 0xee, 0x8f, - 0xf9, 0xc3, 0x4c, 0x74, 0x97, 0xd7, 0x8d, 0x8d, 0x3a, 0xb5, 0x24, 0x39, 0x10, 0xee, 0x16, 0x38, - 0x68, 0x11, 0x78, 0xe2, 0x87, 0x60, 0x5d, 0x4a, 0x42, 0x19, 0x4e, 0x6b, 0x6b, 0x59, 0x2e, 0x59, - 0x18, 0x0d, 0xd5, 0x42, 0x77, 0x0d, 0xec, 0x03, 0x16, 0x8d, 0x73, 0x4b, 0x93, 0x57, 0x81, 0x03, - 0x1c, 0x8a, 0xdf, 0xee, 0xaf, 0x4d, 0xb0, 0x28, 0x17, 0xd3, 0x30, 0x23, 0x8f, 0x00, 0x24, 0xd0, - 0x13, 0x96, 0xa5, 0xc1, 0x4c, 0xcf, 0x5a, 0x42, 0xed, 0x4c, 0x03, 0xff, 0x10, 0x45, 0xe4, 0x05, - 0xb4, 0x71, 0xf6, 0x5c, 0xd5, 0x2c, 0x37, 0x50, 0xec, 0x8f, 0xb6, 0x50, 0x45, 0x8f, 0xb8, 0x0b, - 0x16, 0xde, 0xad, 0xb2, 0xaf, 0x65, 0xaa, 0x29, 0xf2, 0x10, 0x56, 0x82, 0x28, 0x93, 0xd8, 0x7b, - 0xd9, 0xd0, 0xe7, 0x22, 0xbf, 0xfc, 0xe5, 0x82, 0xbb, 0xc7, 0x45, 0x46, 0xbe, 0x06, 0x05, 0x60, - 0xbe, 0x60, 0x03, 0x17, 0x5c, 0x29, 0x2e, 0x46, 0xa8, 0x15, 0x51, 0x47, 0xaf, 0xf8, 0x0c, 0x5a, - 0xf2, 0x7c, 0xf9, 0x08, 0x0b, 0x47, 0xb4, 0xf1, 0x34, 0x1a, 0x0e, 0x0a, 0x52, 0x41, 0xab, 0x4b, - 0x68, 0xa4, 0x81, 0x29, 0x83, 0xc0, 0x6f, 0xb7, 0x07, 0x8d, 0xe3, 0xd4, 0xe7, 0xe9, 0xb5, 0x36, - 0x4e, 0xa0, 0xee, 0x73, 0xe1, 0xa1, 0xfb, 0xd9, 0x14, 0xbf, 0x4b, 0xbb, 0xaf, 0x55, 0xec, 0xde, - 0xfd, 0x3b, 0x03, 0x5a, 0xfd, 0x38, 0xcd, 0x0e, 0xb9, 0x10, 0x6c, 0xcc, 0xc9, 0x7d, 0x68, 0xc4, - 0x72, 0x5a, 0x8d, 0xb0, 0x23, 0xf7, 0x84, 0xeb, 0x50, 0xc5, 0x5f, 0xb8, 0x07, 0xf3, 0xe6, 0x7b, - 0xb8, 0x03, 0x0d, 0xe5, 0x31, 0xd2, 0x9b, 0x1a, 0x54, 0x11, 0x12, 0xeb, 0xf8, 0xec, 0x4c, 0x70, - 0x85, 0x65, 0x83, 0x6a, 0xea, 0x66, 0xb3, 0xfa, 0x3d, 0x00, 0xb9, 0xbf, 0x1f, 0x69, 0x05, 0xee, - 0x39, 0xb4, 0x28, 0x3b, 0xcb, 0x76, 0xe3, 0x28, 0xe3, 0xb3, 0x8c, 0xac, 0x80, 0x19, 0xf8, 0x08, - 0x91, 0x45, 0xcd, 0xc0, 0x97, 0x9b, 0x1b, 0xa7, 0xf1, 0x34, 0x41, 0x84, 0x96, 0xa9, 0x22, 0x10, - 0x4a, 0xdf, 0x4f, 0x71, 0xc7, 0x12, 0x4a, 0xdf, 0x4f, 0xc9, 0x7d, 0x68, 0x89, 0x88, 0x25, 0xe2, - 0x3c, 0xce, 0xe4, 0xe6, 0xea, 0xb8, 0x39, 0xc8, 0x59, 0x03, 0xe1, 0xfe, 0xb3, 0x01, 0xd6, 0x21, - 0x9f, 0x8c, 0x78, 0xfa, 0xde, 0x2a, 0x9f, 0x81, 0x8d, 0x13, 0x0f, 0x03, 0x5f, 0x2f, 0xd4, 0x44, - 0x7a, 0xdf, 0xbf, 0x76, 0xa9, 0xbb, 0x60, 0x85, 0x9c, 0x49, 0xf0, 0x95, 0x9d, 0x69, 0x4a, 0x62, - 0xc3, 0x26, 0x43, 0x9f, 0x33, 0x1f, 0x43, 0x8c, 0x4d, 0x2d, 0x36, 0xd9, 0xe3, 0xcc, 0x97, 0x7b, - 0x0b, 0x99, 0xc8, 0x86, 0xd3, 0xc4, 0x67, 0x19, 0xc7, 0xd0, 0x52, 0x97, 0x86, 0x23, 0xb2, 0x53, - 0xe4, 0x90, 0x27, 0xf0, 0x89, 0x17, 0x4e, 0x85, 0x8c, 0x6b, 0x41, 0x74, 0x16, 0x0f, 0xe3, 0x28, - 0xbc, 0x42, 0x7c, 0x6d, 0x7a, 0x4b, 0x0b, 0xf6, 0xa3, 0xb3, 0xf8, 0x38, 0x0a, 0xaf, 0xdc, 0xdf, - 0x98, 0xd0, 0x78, 0x8d, 0x30, 0xbc, 0x80, 0xe6, 0x04, 0x0f, 0x94, 0x7b, 0xef, 0x5d, 0x89, 0x30, - 0xca, 0x36, 0xd5, 0x49, 0x45, 0x2f, 0xca, 0xd2, 0x2b, 0x9a, 0xab, 0xc9, 0x11, 0x19, 0x1b, 0x85, - 0x3c, 0x13, 0xda, 0x22, 0x2a, 0x23, 0x06, 0x4a, 0xa0, 0x47, 0x68, 0xb5, 0x45, 0x58, 0x6b, 0x8b, - 0xb0, 0x92, 0x55, 0xb0, 0xbd, 0x73, 0xee, 0x5d, 0x88, 0xe9, 0x44, 0x83, 0x5e, 0xd0, 0xab, 0xaf, - 0xa0, 0x5d, 0xdd, 0x87, 0x7c, 0x83, 0x2e, 0xf8, 0x15, 0x02, 0x5f, 0xa7, 0xf2, 0x93, 0xac, 0x43, - 0x03, 0x3d, 0x1c, 0x61, 0x6f, 0x6d, 0x81, 0xdc, 0x8e, 0x1a, 0x42, 0x95, 0xe0, 0x17, 0xe6, 0xcf, - 0x0d, 0x39, 0x4f, 0x75, 0x77, 0xd5, 0x79, 0x9c, 0x9b, 0xe7, 0x51, 0x43, 0x2a, 0xf3, 0xb8, 0xff, - 0x6b, 0x42, 0xfb, 0x07, 0x9e, 0xc6, 0x27, 0x69, 0x9c, 0xc4, 0x82, 0x85, 0x64, 0x7b, 0xfe, 0x74, - 0x0a, 0xc5, 0x75, 0x39, 0xb8, 0xaa, 0xb6, 0xd9, 0x2f, 0x8e, 0xab, 0xd0, 0xa9, 0x9e, 0xdf, 0x05, - 0x4b, 0xa1, 0x7b, 0xcd, 0x11, 0xb4, 0x44, 0xea, 0x28, 0x3c, 0x11, 0xbf, 0xf9, 0xed, 0x69, 0x09, - 0x59, 0x03, 0x98, 0xb0, 0xd9, 0x01, 0x67, 0x82, 0xef, 0xfb, 0xb9, 0xf9, 0x96, 0x1c, 0x89, 0xf3, - 0x84, 0xcd, 0x06, 0xb3, 0x68, 0x20, 0xd0, 0xba, 0xea, 0xb4, 0xa0, 0xc9, 0x4f, 0xc1, 0x99, 0xb0, - 0x99, 0xf4, 0xa3, 0x7d, 0x5f, 0x5b, 0x57, 0xc9, 0x20, 0x9f, 0x43, 0x2d, 0x9b, 0x45, 0x18, 0x94, - 0xe4, 0x3b, 0x24, 0x93, 0x8c, 0xc1, 0x2c, 0xd2, 0x1e, 0x47, 0xa5, 0x2c, 0x07, 0xd4, 0x2e, 0x01, - 0xed, 0x40, 0xcd, 0x0b, 0x7c, 0x7c, 0x88, 0x1c, 0x2a, 0x3f, 0x57, 0xff, 0x08, 0x6e, 0x2d, 0xe0, - 0x50, 0xbd, 0x87, 0x65, 0x35, 0xec, 0x4e, 0xf5, 0x1e, 0xea, 0x55, 0xec, 0x7f, 0x53, 0x83, 0x5b, - 0xda, 0x18, 0xce, 0x83, 0xa4, 0x9f, 0x49, 0xb3, 0xef, 0x42, 0x13, 0xa3, 0x0d, 0x4f, 0xb5, 0x4d, - 0xe4, 0x24, 0xf9, 0x03, 0xb0, 0xd0, 0x03, 0x73, 0x3b, 0xbd, 0x5f, 0xa2, 0x5a, 0x0c, 0x57, 0x76, - 0xab, 0xaf, 0x44, 0xab, 0x93, 0x6f, 0xa1, 0xf1, 0x96, 0xa7, 0xb1, 0x8a, 0x9e, 0xad, 0xad, 0xb5, - 0xeb, 0xc6, 0xc9, 0xbb, 0xd5, 0xc3, 0x94, 0xf2, 0xef, 0x10, 0xfc, 0x07, 0x32, 0x5e, 0x4e, 0xe2, - 0x4b, 0xee, 0x77, 0x9b, 0xb8, 0xa3, 0xaa, 0x7d, 0xe4, 0xa2, 0x1c, 0x6d, 0xbb, 0x44, 0x7b, 0x0f, - 0x5a, 0x95, 0xe3, 0x5d, 0x83, 0xf4, 0xfd, 0x79, 0x8b, 0x77, 0x0a, 0x47, 0xae, 0x3a, 0xce, 0x1e, - 0x40, 0x79, 0xd8, 0xff, 0xaf, 0xfb, 0xb9, 0x7f, 0x65, 0xc0, 0xad, 0xdd, 0x38, 0x8a, 0x38, 0xa6, - 0x40, 0xea, 0xea, 0x4a, 0xb3, 0x37, 0x6e, 0x34, 0xfb, 0xc7, 0xd0, 0x10, 0x52, 0x59, 0xcf, 0x7e, - 0xfb, 0x9a, 0xbb, 0xa0, 0x4a, 0x43, 0x86, 0x99, 0x09, 0x9b, 0x0d, 0x13, 0x1e, 0xf9, 0x41, 0x34, - 0xce, 0xc3, 0xcc, 0x84, 0xcd, 0x4e, 0x14, 0xc7, 0xfd, 0x6b, 0x03, 0x2c, 0xe5, 0x31, 0x73, 0xd1, - 0xda, 0x98, 0x8f, 0xd6, 0x3f, 0x05, 0x27, 0x49, 0xb9, 0x1f, 0x78, 0xf9, 0xaa, 0x0e, 0x2d, 0x19, - 0xd2, 0x38, 0xcf, 0xe2, 0xd4, 0xe3, 0x38, 0xbd, 0x4d, 0x15, 0x21, 0xb9, 0x22, 0x61, 0x9e, 0x4a, - 0xe3, 0x6a, 0x54, 0x11, 0x32, 0xc6, 0xab, 0xcb, 0xc1, 0x4b, 0xb1, 0xa9, 0xa6, 0xdc, 0x7f, 0x30, - 0xa1, 0xbd, 0x17, 0xa4, 0xdc, 0xcb, 0xb8, 0xdf, 0xf3, 0xc7, 0xa8, 0xc8, 0xa3, 0x2c, 0xc8, 0xae, - 0xf4, 0x7b, 0xa2, 0xa9, 0xe2, 0xb9, 0x37, 0xe7, 0x53, 0x5a, 0x05, 0x77, 0x0d, 0xb3, 0x70, 0x45, - 0x90, 0x2d, 0x00, 0x95, 0x08, 0x61, 0x26, 0x5e, 0xbf, 0x39, 0x13, 0x77, 0x50, 0x4d, 0x7e, 0x4a, - 0x0c, 0xd4, 0x98, 0x40, 0xbd, 0x35, 0x16, 0xa6, 0xe9, 0x53, 0x69, 0xab, 0x98, 0x3f, 0x8c, 0x78, - 0x88, 0xb6, 0x88, 0xf9, 0xc3, 0x88, 0x87, 0x45, 0xd6, 0xd6, 0x54, 0xdb, 0x91, 0xdf, 0xe4, 0x0b, - 0x30, 0xe3, 0x04, 0xcf, 0xa7, 0x17, 0xac, 0x1e, 0x6c, 0xf3, 0x38, 0xa1, 0x66, 0x9c, 0xc8, 0x8b, - 0x56, 0x69, 0x67, 0xd7, 0xd1, 0xf6, 0x2b, 0x03, 0x08, 0x26, 0x4c, 0x54, 0x4b, 0xdc, 0xbb, 0x60, - 0x1e, 0x27, 0xa4, 0x09, 0xb5, 0x7e, 0x6f, 0xd0, 0x59, 0x92, 0x1f, 0x7b, 0xbd, 0x83, 0x8e, 0xe1, - 0xbe, 0x33, 0xc0, 0x39, 0x9c, 0x66, 0x4c, 0x9a, 0x8d, 0xf8, 0xd0, 0xbd, 0x7d, 0x06, 0xb6, 0xc8, - 0x58, 0x8a, 0x41, 0x58, 0x45, 0x8e, 0x26, 0xd2, 0x03, 0x41, 0xbe, 0x84, 0x06, 0xf7, 0xc7, 0x3c, - 0x77, 0xe8, 0xce, 0xe2, 0x3e, 0xa9, 0x12, 0x93, 0x0d, 0xb0, 0x84, 0x77, 0xce, 0x27, 0xac, 0x5b, - 0x2f, 0x15, 0xfb, 0xc8, 0x51, 0x8f, 0x2c, 0xd5, 0x72, 0xac, 0x12, 0xd2, 0x38, 0xc1, 0xb4, 0xb9, - 0xa1, 0xab, 0x84, 0x34, 0x4e, 0x64, 0xd2, 0xbc, 0x05, 0x3f, 0x09, 0xc6, 0x51, 0x9c, 0xf2, 0x61, - 0x10, 0xf9, 0x7c, 0x36, 0xf4, 0xe2, 0xe8, 0x2c, 0x0c, 0xbc, 0x0c, 0xb1, 0xb4, 0xe9, 0x6d, 0x25, - 0xdc, 0x97, 0xb2, 0x5d, 0x2d, 0x72, 0x67, 0x60, 0xe7, 0x71, 0x91, 0x3c, 0x96, 0x01, 0x0d, 0xe3, - 0xaa, 0x76, 0x0b, 0x4c, 0xfb, 0x2b, 0x09, 0x0e, 0xcd, 0xe5, 0xf2, 0x9a, 0x70, 0x8d, 0x3c, 0x52, - 0x22, 0x51, 0x4d, 0xaf, 0x6a, 0xd5, 0xf4, 0x0a, 0x33, 0xc5, 0x38, 0xe2, 0x3a, 0xe3, 0xc0, 0x6f, - 0xf7, 0x6f, 0x4d, 0xb0, 0x8b, 0xa7, 0xec, 0x29, 0x38, 0x93, 0x1c, 0x6a, 0xed, 0x70, 0x98, 0x4b, - 0x17, 0xf8, 0xd3, 0x52, 0x4e, 0xee, 0x82, 0x79, 0x71, 0xa9, 0x81, 0xb2, 0xa4, 0xd6, 0xf7, 0x6f, - 0xa8, 0x79, 0x71, 0x59, 0x7a, 0x6c, 0xe3, 0xa3, 0x1e, 0xfb, 0x08, 0x6e, 0x79, 0x21, 0x67, 0xd1, - 0xb0, 0x74, 0x38, 0x65, 0x70, 0x2b, 0xc8, 0x3e, 0x29, 0xbc, 0x4e, 0x47, 0x9d, 0x66, 0xf9, 0xb6, - 0x3c, 0x84, 0x86, 0xcf, 0xc3, 0x8c, 0x55, 0x4b, 0xa3, 0xe3, 0x94, 0x79, 0x21, 0xdf, 0x93, 0x6c, - 0xaa, 0xa4, 0x64, 0x03, 0xec, 0xfc, 0x9d, 0xd5, 0x05, 0x11, 0x66, 0xde, 0x39, 0xd8, 0xb4, 0x90, - 0x96, 0x58, 0x42, 0x05, 0x4b, 0xf7, 0x6b, 0xa8, 0x7d, 0xff, 0xa6, 0xaf, 0xcf, 0x6a, 0xbc, 0x77, - 0xd6, 0x1c, 0x51, 0xb3, 0x82, 0xe8, 0xff, 0xd4, 0xa0, 0xa9, 0xbd, 0x4e, 0xee, 0x7b, 0x5a, 0x64, - 0x89, 0xf2, 0x73, 0xfe, 0x71, 0x2b, 0xdc, 0xb7, 0x5a, 0x46, 0xd7, 0x3e, 0x5e, 0x46, 0x93, 0x5f, - 0x40, 0x3b, 0x51, 0xb2, 0xaa, 0xc3, 0x7f, 0x5a, 0x1d, 0xa3, 0x7f, 0x71, 0x5c, 0x2b, 0x29, 0x09, - 0x69, 0xba, 0x58, 0x8f, 0x64, 0x6c, 0x8c, 0x57, 0xd4, 0xa6, 0x4d, 0x49, 0x0f, 0xd8, 0xf8, 0x06, - 0xb7, 0xff, 0x2d, 0xbc, 0x57, 0x66, 0xc3, 0x71, 0xd2, 0x6d, 0xa3, 0x47, 0x4a, 0x8f, 0xaf, 0x3a, - 0xe3, 0xf2, 0xbc, 0x33, 0xde, 0x03, 0xc7, 0x8b, 0x27, 0x93, 0x00, 0x65, 0x2b, 0x3a, 0xdb, 0x43, - 0xc6, 0x40, 0xb8, 0x7f, 0x63, 0x40, 0x53, 0x9f, 0x96, 0xb4, 0xa0, 0xb9, 0xd7, 0x7b, 0xb5, 0x7d, - 0x7a, 0x20, 0xe3, 0x01, 0x80, 0xb5, 0xb3, 0x7f, 0xb4, 0x4d, 0xff, 0xac, 0x63, 0xc8, 0xd8, 0xb0, - 0x7f, 0x34, 0xe8, 0x98, 0xc4, 0x81, 0xc6, 0xab, 0x83, 0xe3, 0xed, 0x41, 0xa7, 0x46, 0x6c, 0xa8, - 0xef, 0x1c, 0x1f, 0x1f, 0x74, 0xea, 0xa4, 0x0d, 0xf6, 0xde, 0xf6, 0xa0, 0x37, 0xd8, 0x3f, 0xec, - 0x75, 0x1a, 0x52, 0xf7, 0x75, 0xef, 0xb8, 0x63, 0xc9, 0x8f, 0xd3, 0xfd, 0xbd, 0x4e, 0x53, 0xca, - 0x4f, 0xb6, 0xfb, 0xfd, 0x5f, 0x1e, 0xd3, 0xbd, 0x8e, 0x2d, 0xe7, 0xed, 0x0f, 0xe8, 0xfe, 0xd1, - 0xeb, 0x8e, 0x23, 0xbf, 0x8f, 0x77, 0xbe, 0xeb, 0xed, 0x0e, 0x3a, 0xe0, 0x7e, 0x0d, 0xad, 0x0a, - 0x82, 0x72, 0x34, 0xed, 0xbd, 0xea, 0x2c, 0xc9, 0x25, 0xdf, 0x6c, 0x1f, 0x9c, 0xf6, 0x3a, 0x06, - 0x59, 0x01, 0xc0, 0xcf, 0xe1, 0xc1, 0xf6, 0xd1, 0xeb, 0x8e, 0xe9, 0xfe, 0x3e, 0xd8, 0xa7, 0x81, - 0xbf, 0x13, 0xc6, 0xde, 0x85, 0x34, 0x8c, 0x11, 0x13, 0x5c, 0xbf, 0x93, 0xf8, 0x2d, 0xa3, 0x3c, - 0x1a, 0xa5, 0xd0, 0x77, 0xaf, 0x29, 0xf7, 0x08, 0x9a, 0xa7, 0x81, 0x7f, 0xc2, 0xbc, 0x0b, 0x59, - 0x8f, 0x8f, 0xe4, 0xf8, 0xa1, 0x08, 0xde, 0x72, 0x1d, 0xe0, 0x1c, 0xe4, 0xf4, 0x83, 0xb7, 0x9c, - 0x3c, 0x00, 0x0b, 0x89, 0x3c, 0xa3, 0x41, 0x5b, 0xce, 0xd7, 0xa4, 0x5a, 0xe6, 0xfe, 0x97, 0x51, - 0xec, 0x1d, 0x8b, 0xed, 0xfb, 0x50, 0x4f, 0x98, 0x77, 0xa1, 0xa3, 0x49, 0x4b, 0x8f, 0x91, 0xeb, - 0x51, 0x14, 0x90, 0x47, 0x60, 0x6b, 0x03, 0xc9, 0x27, 0x6e, 0x55, 0x2c, 0x89, 0x16, 0xc2, 0xf9, - 0xab, 0xab, 0xcd, 0x5f, 0x9d, 0xdc, 0xfb, 0x64, 0x1a, 0x66, 0xc1, 0x30, 0x61, 0x69, 0x5e, 0x3d, - 0x3b, 0xc8, 0x39, 0x61, 0x69, 0x26, 0xc5, 0x67, 0x41, 0x2a, 0x32, 0x25, 0x56, 0x31, 0xd3, 0x41, - 0x0e, 0x8a, 0xef, 0x81, 0xa3, 0x0c, 0x46, 0xfa, 0x8b, 0xca, 0x80, 0x94, 0x05, 0x9d, 0x06, 0xbe, - 0x8c, 0x68, 0x3c, 0xf2, 0x51, 0xd4, 0x54, 0x11, 0x8d, 0x47, 0xfe, 0x69, 0xe0, 0xbb, 0xdf, 0x02, - 0x94, 0xfd, 0x90, 0x6b, 0x52, 0xfa, 0x3b, 0xd0, 0x60, 0x61, 0xa0, 0x11, 0x77, 0xa8, 0x22, 0xdc, - 0x23, 0x68, 0x55, 0xba, 0x28, 0xd2, 0x56, 0x59, 0x18, 0x0e, 0x2f, 0xf8, 0x95, 0xc0, 0xb1, 0x36, - 0x6d, 0xb2, 0x30, 0xfc, 0x9e, 0x5f, 0x09, 0xf2, 0x00, 0x1a, 0xaa, 0x01, 0x63, 0x2e, 0xd4, 0xf9, - 0x38, 0x94, 0x2a, 0xa1, 0xfb, 0x15, 0x58, 0xaa, 0xf8, 0xaf, 0xb8, 0x8a, 0x71, 0xe3, 0x43, 0xf7, - 0x52, 0xef, 0x19, 0x5b, 0x05, 0xe4, 0xa9, 0x6e, 0xf4, 0x08, 0xd5, 0x56, 0x32, 0xca, 0xfc, 0x4e, - 0x29, 0xe9, 0x1e, 0x0f, 0x2a, 0xbb, 0x7b, 0x60, 0x7f, 0xb0, 0x75, 0xa6, 0x01, 0x30, 0x4b, 0x00, - 0xae, 0x69, 0xa6, 0xb9, 0xbf, 0x02, 0x28, 0x1b, 0x42, 0xda, 0x73, 0xd5, 0x2c, 0xd2, 0x73, 0x9f, - 0xc8, 0x5a, 0x2c, 0x08, 0xfd, 0x94, 0x47, 0x73, 0xa7, 0x2e, 0x5b, 0x48, 0x85, 0x9c, 0xac, 0x43, - 0x1d, 0xfb, 0x5c, 0xb5, 0x32, 0xb2, 0x16, 0x4d, 0x2e, 0x94, 0xb8, 0x23, 0x58, 0x56, 0xef, 0x27, - 0xe5, 0x7f, 0x39, 0xe5, 0xe2, 0x83, 0x89, 0xd7, 0x1a, 0x40, 0xf1, 0x0e, 0xe4, 0x1d, 0xbb, 0x0a, - 0x47, 0xfa, 0xcf, 0x59, 0xc0, 0x43, 0x3f, 0x3f, 0x8d, 0xa6, 0xdc, 0x3f, 0x84, 0x76, 0xbe, 0x06, - 0xf6, 0x0d, 0x9e, 0x16, 0xaf, 0xb8, 0x42, 0x53, 0x95, 0x2b, 0x4a, 0xe5, 0x28, 0xf6, 0xf9, 0x8e, - 0xd9, 0x35, 0xf2, 0x87, 0xdc, 0xfd, 0xb7, 0x5a, 0x3e, 0x5a, 0x97, 0xd1, 0x73, 0xe9, 0x9f, 0xb1, - 0x98, 0xfe, 0xcd, 0xe7, 0x59, 0xe6, 0x6f, 0x95, 0x67, 0xfd, 0x1c, 0x1c, 0x1f, 0x93, 0x8d, 0xe0, - 0x32, 0x8f, 0xee, 0xab, 0x8b, 0x89, 0x85, 0x4e, 0x47, 0x82, 0x4b, 0x4e, 0x4b, 0x65, 0xb9, 0x97, - 0x2c, 0xbe, 0xe0, 0x51, 0xf0, 0x16, 0xfb, 0x04, 0xf2, 0xd0, 0x25, 0xa3, 0x6c, 0xba, 0x28, 0x67, - 0xd2, 0x4d, 0x97, 0xbc, 0x7f, 0x64, 0x95, 0xfd, 0x23, 0x89, 0xdc, 0x34, 0x11, 0x3c, 0xcd, 0xf2, - 0x44, 0x54, 0x51, 0x45, 0x42, 0xe7, 0x68, 0x5d, 0x99, 0xd0, 0x7d, 0x0e, 0xed, 0x28, 0x8e, 0x86, - 0xd1, 0x34, 0x0c, 0x65, 0xaa, 0xac, 0x9b, 0x82, 0xad, 0x28, 0x8e, 0x8e, 0x34, 0x8b, 0x3c, 0x81, - 0x4f, 0xaa, 0x2a, 0xca, 0x72, 0x5b, 0xaa, 0xd3, 0x50, 0xd1, 0x43, 0xfb, 0xde, 0x80, 0x4e, 0x3c, - 0xfa, 0x15, 0xf7, 0x32, 0x44, 0x6c, 0x88, 0x26, 0xdb, 0x56, 0x6f, 0xbc, 0xe2, 0x4b, 0x88, 0x8e, - 0xd8, 0x84, 0xbb, 0x2f, 0xc1, 0x29, 0x40, 0x90, 0xe1, 0xfc, 0xe8, 0xf8, 0xa8, 0xa7, 0x02, 0xee, - 0xfe, 0xd1, 0x5e, 0xef, 0x4f, 0x3b, 0x86, 0x7c, 0x10, 0x68, 0xef, 0x4d, 0x8f, 0xf6, 0x7b, 0x1d, - 0x53, 0x06, 0xeb, 0xbd, 0xde, 0x41, 0x6f, 0xd0, 0xeb, 0xd4, 0xbe, 0xab, 0xdb, 0xcd, 0x8e, 0x4d, - 0x6d, 0x3e, 0x4b, 0xc2, 0xc0, 0x0b, 0x32, 0xb7, 0x0f, 0x20, 0xa7, 0xd5, 0x37, 0x7a, 0x0f, 0x9c, - 0x72, 0x6d, 0x75, 0xa3, 0x76, 0xa6, 0x57, 0x95, 0x29, 0x9f, 0x36, 0x2a, 0xf3, 0xa6, 0x94, 0x4f, - 0x9b, 0xd9, 0x29, 0xd8, 0x87, 0x2c, 0x79, 0xaf, 0x0a, 0x6a, 0x17, 0xb5, 0xee, 0x54, 0x77, 0x7e, - 0xf4, 0x4b, 0xff, 0x10, 0x9a, 0x3a, 0x70, 0x6a, 0xff, 0x98, 0x0b, 0xaa, 0xb9, 0x4c, 0x16, 0x25, - 0x77, 0x0e, 0xe3, 0x4b, 0x5e, 0x24, 0x3b, 0x27, 0xec, 0x2a, 0x8c, 0x99, 0xff, 0x11, 0x43, 0xfc, - 0x19, 0x80, 0x88, 0xa7, 0xa9, 0xc7, 0x87, 0xe3, 0xa2, 0xe1, 0xe4, 0x28, 0xce, 0x6b, 0xdd, 0xc5, - 0xe6, 0x22, 0x43, 0x61, 0x4d, 0xb9, 0x99, 0xa4, 0xa5, 0xe8, 0x27, 0x60, 0x65, 0xb3, 0xa8, 0xec, - 0x6f, 0x35, 0x32, 0x59, 0x82, 0xba, 0xbb, 0xe0, 0x0c, 0x66, 0x58, 0x98, 0x4d, 0xc5, 0xdc, 0xf3, - 0x6d, 0x7c, 0xe0, 0xf9, 0x36, 0x17, 0x9e, 0xef, 0xff, 0x34, 0xa0, 0x55, 0xc9, 0xc2, 0xc8, 0xe7, - 0x50, 0xcf, 0x66, 0xd1, 0x7c, 0x63, 0x38, 0x5f, 0x84, 0xa2, 0x48, 0xda, 0x9b, 0xac, 0xda, 0x98, - 0x10, 0xc1, 0x38, 0xe2, 0xbe, 0x9e, 0x52, 0x56, 0x72, 0xdb, 0x9a, 0x45, 0x0e, 0xe0, 0x96, 0x8a, - 0x19, 0x79, 0x53, 0x28, 0x4f, 0xe4, 0xbf, 0x58, 0xc8, 0xfa, 0x54, 0xf1, 0xba, 0x9b, 0x6b, 0xa9, - 0xf2, 0x7c, 0x65, 0x3c, 0xc7, 0x5c, 0xdd, 0x86, 0xdb, 0xd7, 0xa8, 0xfd, 0xa8, 0x3e, 0xc4, 0x7d, - 0x58, 0x96, 0x75, 0x7b, 0x30, 0xe1, 0x22, 0x63, 0x93, 0x04, 0xd3, 0x1f, 0x1d, 0xf3, 0xeb, 0xd4, - 0xcc, 0x84, 0xfb, 0x25, 0xb4, 0x4f, 0x38, 0x4f, 0x29, 0x17, 0x49, 0x1c, 0xa9, 0xa7, 0x5f, 0xe0, - 0xa1, 0xf5, 0x03, 0xa3, 0x29, 0xf7, 0x2f, 0xc0, 0x91, 0x89, 0xfd, 0x0e, 0xcb, 0xbc, 0xf3, 0x1f, - 0x93, 0xf8, 0x7f, 0x09, 0xcd, 0x44, 0x99, 0x89, 0x4e, 0xd3, 0xdb, 0x18, 0xe3, 0xb4, 0xe9, 0xd0, - 0x5c, 0xe8, 0x7e, 0x0b, 0xb5, 0xa3, 0xe9, 0xa4, 0xfa, 0xbf, 0x4d, 0x5d, 0xfd, 0x6f, 0x73, 0x0f, - 0x1c, 0xac, 0x11, 0xb0, 0x49, 0xa8, 0xb2, 0x57, 0x5b, 0x32, 0xb0, 0x3b, 0xf8, 0x03, 0xb4, 0x72, - 0xec, 0xf7, 0x7d, 0xfc, 0xf3, 0x05, 0x2f, 0x7f, 0xdf, 0x9f, 0xb3, 0x05, 0x55, 0x26, 0xf2, 0xc8, - 0xdf, 0xcf, 0x2f, 0x4d, 0x11, 0xf3, 0x73, 0xeb, 0x4e, 0x47, 0x31, 0xf7, 0x2b, 0x68, 0xe7, 0xc9, - 0xf7, 0x21, 0xcf, 0x18, 0x9a, 0x53, 0x18, 0xf0, 0xa8, 0x62, 0x6a, 0xb6, 0x62, 0x0c, 0xc4, 0x07, - 0x7a, 0xaa, 0xee, 0x26, 0x58, 0xda, 0x56, 0x09, 0xd4, 0xbd, 0xd8, 0x57, 0x2e, 0xd2, 0xa0, 0xf8, - 0x2d, 0x0f, 0x3c, 0x11, 0xe3, 0xfc, 0x21, 0x9c, 0x88, 0xb1, 0x3b, 0x83, 0xe5, 0x1d, 0xe6, 0x5d, - 0x4c, 0x93, 0xfc, 0x21, 0xaa, 0x54, 0x49, 0xc6, 0x5c, 0x95, 0xf4, 0x81, 0x46, 0xee, 0xa7, 0xd0, - 0x9c, 0x46, 0xc1, 0x2c, 0xcf, 0x7e, 0x1c, 0x6a, 0x49, 0x52, 0x35, 0x30, 0xc3, 0xd8, 0xc3, 0xc2, - 0x08, 0xbd, 0xca, 0xa1, 0x05, 0xed, 0xfe, 0x39, 0x2c, 0xf7, 0x66, 0x09, 0xb6, 0xb5, 0x3f, 0xfa, - 0x04, 0x56, 0x36, 0x65, 0xce, 0x6d, 0x6a, 0x61, 0xe5, 0x5a, 0xbe, 0xf2, 0xd6, 0x3f, 0x19, 0x50, - 0x97, 0x26, 0x22, 0x6b, 0xb7, 0x3f, 0xe6, 0x2c, 0xcd, 0x46, 0x9c, 0x65, 0x64, 0xce, 0x1c, 0x56, - 0xe7, 0x28, 0x77, 0xe9, 0x85, 0x41, 0xbe, 0x52, 0x1d, 0xf3, 0xfc, 0x8f, 0x80, 0xe5, 0xdc, 0xd0, - 0xd0, 0x10, 0x17, 0xf5, 0xc9, 0x26, 0xb4, 0xbe, 0x8b, 0x83, 0x68, 0x57, 0x35, 0x91, 0xc9, 0xa2, - 0x59, 0xbe, 0xa7, 0xff, 0x0c, 0xac, 0x7d, 0x21, 0xed, 0xff, 0x7d, 0x55, 0x0c, 0xaf, 0x55, 0xd7, - 0x70, 0x97, 0xb6, 0xfe, 0xb1, 0x06, 0xf5, 0x1f, 0x78, 0x1a, 0x93, 0xaf, 0xa0, 0xa9, 0x5b, 0x44, - 0xa4, 0xd2, 0x0a, 0x5a, 0xc5, 0xf7, 0x75, 0xa1, 0x77, 0x84, 0xab, 0x74, 0x54, 0x84, 0x2e, 0x8b, - 0x4b, 0x52, 0x76, 0xb0, 0xde, 0xdb, 0xd4, 0x4b, 0xe8, 0xf4, 0xb3, 0x94, 0xb3, 0x49, 0x45, 0x7d, - 0x1e, 0xa6, 0xeb, 0x2a, 0x55, 0x44, 0xeb, 0x29, 0x58, 0x2a, 0xc8, 0x2c, 0x0c, 0x58, 0x2c, 0x3a, - 0x51, 0xf9, 0x11, 0xb4, 0xfa, 0xe7, 0xf1, 0x34, 0xf4, 0xfb, 0x3c, 0xbd, 0xe4, 0xa4, 0xd2, 0xa6, - 0x5d, 0xad, 0x7c, 0xbb, 0x4b, 0x64, 0x03, 0x40, 0x79, 0xd9, 0x69, 0xe0, 0x0b, 0xd2, 0x94, 0xb2, - 0xa3, 0xe9, 0x44, 0x4d, 0x5a, 0x71, 0x3f, 0xa5, 0x59, 0x89, 0x35, 0x1f, 0xd2, 0xfc, 0x06, 0x96, - 0x77, 0x31, 0x16, 0x1f, 0xa7, 0xdb, 0xa3, 0x38, 0xcd, 0xc8, 0x62, 0xab, 0x76, 0x75, 0x91, 0xe1, - 0x2e, 0x91, 0x17, 0x60, 0x0f, 0xd2, 0x2b, 0xa5, 0xff, 0x89, 0x0e, 0xd1, 0xe5, 0x7a, 0xd7, 0x9c, - 0x72, 0xeb, 0xef, 0x6b, 0x60, 0xfd, 0x32, 0x4e, 0x2f, 0x78, 0x4a, 0x9e, 0x80, 0x85, 0xdd, 0x01, - 0x6d, 0x44, 0x45, 0xa7, 0xe0, 0xba, 0x85, 0x1e, 0x80, 0x83, 0xa0, 0x0c, 0x98, 0xb8, 0x50, 0x57, - 0x85, 0xff, 0xdc, 0x2a, 0x5c, 0x54, 0xf2, 0x86, 0xf7, 0xba, 0xa2, 0x2e, 0xaa, 0xe8, 0x88, 0xcc, - 0x95, 0xec, 0xab, 0x4d, 0x55, 0x7f, 0xf7, 0xdd, 0xa5, 0x0d, 0xe3, 0x85, 0x41, 0x1e, 0x43, 0xbd, - 0xaf, 0x4e, 0x2a, 0x95, 0xca, 0x7f, 0xb7, 0x56, 0x57, 0x72, 0x46, 0x31, 0xf3, 0x73, 0xb0, 0xd4, - 0xcb, 0xae, 0x8e, 0x39, 0x97, 0x98, 0xae, 0x76, 0xaa, 0x2c, 0x3d, 0xe0, 0x31, 0x58, 0x2a, 0x68, - 0xa8, 0x01, 0x73, 0x01, 0x44, 0xed, 0x5a, 0xc5, 0x20, 0xa5, 0xaa, 0xbc, 0x5c, 0xa9, 0xce, 0x79, - 0xfc, 0x82, 0xea, 0x33, 0xe8, 0x50, 0xee, 0xf1, 0xa0, 0xf2, 0xe6, 0x93, 0xfc, 0x50, 0x8b, 0x66, - 0xbb, 0x61, 0x90, 0x97, 0xb0, 0x3c, 0x97, 0x1f, 0x90, 0x2e, 0x02, 0x7d, 0x4d, 0xca, 0xb0, 0x38, - 0x78, 0xa7, 0xf3, 0x2f, 0xef, 0xd6, 0x8c, 0x7f, 0x7d, 0xb7, 0x66, 0xfc, 0xfb, 0xbb, 0x35, 0xe3, - 0xd7, 0xff, 0xb1, 0xb6, 0x34, 0xb2, 0xf0, 0x1f, 0xff, 0x6f, 0xfe, 0x2f, 0x00, 0x00, 0xff, 0xff, - 0xcf, 0xaa, 0xfb, 0xee, 0x35, 0x20, 0x00, 0x00, + // 3355 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4b, 0x73, 0x23, 0x47, + 0x72, 0x66, 0x37, 0x80, 0x46, 0x77, 0x02, 0xe4, 0x40, 0xa5, 0xd9, 0x11, 0x44, 0xed, 0x72, 0xa8, + 0xd6, 0x63, 0xa8, 0x91, 0xc4, 0x19, 0x51, 0xb2, 0xbd, 0x5a, 0x87, 0x0f, 0x7c, 0x60, 0xc6, 0x94, + 0xf8, 0x72, 0x01, 0x9c, 0xb5, 0x15, 0x0e, 0x23, 0x0a, 0xdd, 0x45, 0xb0, 0x97, 0x8d, 0xee, 0x76, + 0x57, 0x37, 0x03, 0x9c, 0x9b, 0x0f, 0xf6, 0x6f, 0xd8, 0x83, 0xc3, 0x37, 0x5f, 0x7c, 0xf1, 0x75, + 0x8f, 0xbe, 0x38, 0xc2, 0xbe, 0x39, 0xfc, 0x0b, 0x1c, 0x63, 0xff, 0x03, 0x47, 0x38, 0xc2, 0x37, + 0x47, 0x65, 0x55, 0x3f, 0x80, 0x21, 0x67, 0x56, 0x8e, 0xd8, 0x13, 0x3a, 0x1f, 0xf5, 0xfa, 0x2a, + 0x33, 0x2b, 0x33, 0x01, 0x76, 0x32, 0xd9, 0x4e, 0xd2, 0x38, 0x8b, 0x89, 0x99, 0x4c, 0xd6, 0x1d, + 0x96, 0x04, 0x8a, 0x5c, 0x7f, 0x34, 0x0d, 0xb2, 0xcb, 0x7c, 0xb2, 0xed, 0xc5, 0xb3, 0x27, 0xfe, + 0x34, 0x65, 0xc9, 0xe5, 0x97, 0x41, 0xfc, 0x64, 0xc2, 0xfc, 0x29, 0x4f, 0x9f, 0x24, 0x93, 0x27, + 0xc5, 0x38, 0x77, 0x1d, 0x9a, 0x47, 0x81, 0xc8, 0x08, 0x81, 0x66, 0x1e, 0xf8, 0xa2, 0x6f, 0x6c, + 0x36, 0xb6, 0x2c, 0x8a, 0xdf, 0xee, 0x31, 0x38, 0x23, 0x26, 0xae, 0x5e, 0xb0, 0x30, 0xe7, 0xa4, + 0x07, 0x8d, 0x6b, 0x16, 0xf6, 0x8d, 0x4d, 0x63, 0xab, 0x4b, 0xe5, 0x27, 0xd9, 0x06, 0xfb, 0x9a, + 0x85, 0xe3, 0xec, 0x26, 0xe1, 0x7d, 0x73, 0xd3, 0xd8, 0x5a, 0xdb, 0x79, 0x77, 0x3b, 0x99, 0x6c, + 0x9f, 0xc5, 0x22, 0x0b, 0xa2, 0xe9, 0xf6, 0x0b, 0x16, 0x8e, 0x6e, 0x12, 0x4e, 0xdb, 0xd7, 0xea, + 0xc3, 0x3d, 0x85, 0xce, 0x30, 0xf5, 0x9e, 0xe5, 0x91, 0x97, 0x05, 0x71, 0x24, 0x57, 0x8c, 0xd8, + 0x8c, 0xe3, 0x8c, 0x0e, 0xc5, 0x6f, 0xc9, 0x63, 0xe9, 0x54, 0xf4, 0x1b, 0x9b, 0x0d, 0xc9, 0x93, + 0xdf, 0xa4, 0x0f, 0xed, 0x40, 0xec, 0xc7, 0x79, 0x94, 0xf5, 0x9b, 0x9b, 0xc6, 0x96, 0x4d, 0x0b, + 0xd2, 0xfd, 0x6f, 0x13, 0x5a, 0x7f, 0x92, 0xf3, 0xf4, 0x06, 0xc7, 0x65, 0x59, 0x5a, 0xcc, 0x25, + 0xbf, 0xc9, 0x7d, 0x68, 0x85, 0x2c, 0x9a, 0x8a, 0xbe, 0x89, 0x93, 0x29, 0x82, 0x7c, 0x00, 0x0e, + 0xbb, 0xc8, 0x78, 0x3a, 0xce, 0x03, 0xbf, 0xdf, 0xd8, 0x34, 0xb6, 0x2c, 0x6a, 0x23, 0xe3, 0x3c, + 0xf0, 0xc9, 0xfb, 0x60, 0xfb, 0xf1, 0xd8, 0xab, 0xaf, 0xe5, 0xc7, 0xb8, 0x16, 0xf9, 0x08, 0xec, + 0x3c, 0xf0, 0xc7, 0x61, 0x20, 0xb2, 0x7e, 0x6b, 0xd3, 0xd8, 0xea, 0xec, 0xd8, 0xf2, 0xb0, 0x12, + 0x3b, 0xda, 0xce, 0x03, 0x1f, 0x41, 0x7c, 0x0c, 0xb6, 0x48, 0xbd, 0xf1, 0x45, 0x1e, 0x79, 0x7d, + 0x0b, 0x95, 0xee, 0x49, 0xa5, 0xda, 0xa9, 0x69, 0x5b, 0x28, 0x42, 0x1e, 0x2b, 0xe5, 0xd7, 0x3c, + 0x15, 0xbc, 0xdf, 0x56, 0x4b, 0x69, 0x92, 0x3c, 0x85, 0xce, 0x05, 0xf3, 0x78, 0x36, 0x4e, 0x58, + 0xca, 0x66, 0x7d, 0xbb, 0x9a, 0xe8, 0x99, 0x64, 0x9f, 0x49, 0xae, 0xa0, 0x70, 0x51, 0x12, 0xe4, + 0x6b, 0x58, 0x45, 0x4a, 0x8c, 0x2f, 0x82, 0x30, 0xe3, 0x69, 0xdf, 0xc1, 0x31, 0x6b, 0x38, 0x06, + 0x39, 0xa3, 0x94, 0x73, 0xda, 0x55, 0x4a, 0x8a, 0x43, 0x7e, 0x06, 0xc0, 0xe7, 0x09, 0x8b, 0xfc, + 0x31, 0x0b, 0xc3, 0x3e, 0xe0, 0x1e, 0x1c, 0xc5, 0xd9, 0x0d, 0x43, 0xf2, 0x9e, 0xdc, 0x1f, 0xf3, + 0xc7, 0x99, 0xe8, 0xaf, 0x6e, 0x1a, 0x5b, 0x4d, 0x6a, 0x49, 0x72, 0x24, 0xdc, 0x1d, 0x70, 0xd0, + 0x22, 0xf0, 0xc4, 0x9f, 0x80, 0x75, 0x2d, 0x09, 0x65, 0x38, 0x9d, 0x9d, 0x55, 0xb9, 0x64, 0x69, + 0x34, 0x54, 0x0b, 0xdd, 0x0d, 0xb0, 0x8f, 0x58, 0x34, 0x2d, 0x2c, 0x4d, 0x5e, 0x05, 0x0e, 0x70, + 0x28, 0x7e, 0xbb, 0xbf, 0x36, 0xc1, 0xa2, 0x5c, 0xe4, 0x61, 0x46, 0x1e, 0x01, 0x48, 0xa0, 0x67, + 0x2c, 0x4b, 0x83, 0xb9, 0x9e, 0xb5, 0x82, 0xda, 0xc9, 0x03, 0xff, 0x18, 0x45, 0xe4, 0x29, 0x74, + 0x71, 0xf6, 0x42, 0xd5, 0xac, 0x36, 0x50, 0xee, 0x8f, 0x76, 0x50, 0x45, 0x8f, 0x78, 0x00, 0x16, + 0xde, 0xad, 0xb2, 0xaf, 0x55, 0xaa, 0x29, 0xf2, 0x09, 0xac, 0x05, 0x51, 0x26, 0xb1, 0xf7, 0xb2, + 0xb1, 0xcf, 0x45, 0x71, 0xf9, 0xab, 0x25, 0xf7, 0x80, 0x8b, 0x8c, 0x7c, 0x05, 0x0a, 0xc0, 0x62, + 0xc1, 0x16, 0x2e, 0xb8, 0x56, 0x5e, 0x8c, 0x50, 0x2b, 0xa2, 0x8e, 0x5e, 0xf1, 0x4b, 0xe8, 0xc8, + 0xf3, 0x15, 0x23, 0x2c, 0x1c, 0xd1, 0xc5, 0xd3, 0x68, 0x38, 0x28, 0x48, 0x05, 0xad, 0x2e, 0xa1, + 0x91, 0x06, 0xa6, 0x0c, 0x02, 0xbf, 0xdd, 0x01, 0xb4, 0x4e, 0x53, 0x9f, 0xa7, 0xb7, 0xda, 0x38, + 0x81, 0xa6, 0xcf, 0x85, 0x87, 0xee, 0x67, 0x53, 0xfc, 0xae, 0xec, 0xbe, 0x51, 0xb3, 0x7b, 0xf7, + 0xef, 0x0c, 0xe8, 0x0c, 0xe3, 0x34, 0x3b, 0xe6, 0x42, 0xb0, 0x29, 0x27, 0x0f, 0xa1, 0x15, 0xcb, + 0x69, 0x35, 0xc2, 0x8e, 0xdc, 0x13, 0xae, 0x43, 0x15, 0x7f, 0xe9, 0x1e, 0xcc, 0xbb, 0xef, 0xe1, + 0x3e, 0xb4, 0x94, 0xc7, 0x48, 0x6f, 0x6a, 0x51, 0x45, 0x48, 0xac, 0xe3, 0x8b, 0x0b, 0xc1, 0x15, + 0x96, 0x2d, 0xaa, 0xa9, 0xbb, 0xcd, 0xea, 0xf7, 0x00, 0xe4, 0xfe, 0x7e, 0xa4, 0x15, 0xb8, 0x97, + 0xd0, 0xa1, 0xec, 0x22, 0xdb, 0x8f, 0xa3, 0x8c, 0xcf, 0x33, 0xb2, 0x06, 0x66, 0xe0, 0x23, 0x44, + 0x16, 0x35, 0x03, 0x5f, 0x6e, 0x6e, 0x9a, 0xc6, 0x79, 0x82, 0x08, 0xad, 0x52, 0x45, 0x20, 0x94, + 0xbe, 0x9f, 0xe2, 0x8e, 0x25, 0x94, 0xbe, 0x9f, 0x92, 0x87, 0xd0, 0x11, 0x11, 0x4b, 0xc4, 0x65, + 0x9c, 0xc9, 0xcd, 0x35, 0x71, 0x73, 0x50, 0xb0, 0x46, 0xc2, 0xfd, 0x67, 0x03, 0xac, 0x63, 0x3e, + 0x9b, 0xf0, 0xf4, 0xb5, 0x55, 0xde, 0x07, 0x1b, 0x27, 0x1e, 0x07, 0xbe, 0x5e, 0xa8, 0x8d, 0xf4, + 0xa1, 0x7f, 0xeb, 0x52, 0x0f, 0xc0, 0x0a, 0x39, 0x93, 0xe0, 0x2b, 0x3b, 0xd3, 0x94, 0xc4, 0x86, + 0xcd, 0xc6, 0x3e, 0x67, 0x3e, 0x86, 0x18, 0x9b, 0x5a, 0x6c, 0x76, 0xc0, 0x99, 0x2f, 0xf7, 0x16, + 0x32, 0x91, 0x8d, 0xf3, 0xc4, 0x67, 0x19, 0xc7, 0xd0, 0xd2, 0x94, 0x86, 0x23, 0xb2, 0x73, 0xe4, + 0x90, 0xc7, 0xf0, 0x8e, 0x17, 0xe6, 0x42, 0xc6, 0xb5, 0x20, 0xba, 0x88, 0xc7, 0x71, 0x14, 0xde, + 0x20, 0xbe, 0x36, 0xbd, 0xa7, 0x05, 0x87, 0xd1, 0x45, 0x7c, 0x1a, 0x85, 0x37, 0xee, 0x6f, 0x4c, + 0x68, 0x3d, 0x47, 0x18, 0x9e, 0x42, 0x7b, 0x86, 0x07, 0x2a, 0xbc, 0xf7, 0x81, 0x44, 0x18, 0x65, + 0xdb, 0xea, 0xa4, 0x62, 0x10, 0x65, 0xe9, 0x0d, 0x2d, 0xd4, 0xe4, 0x88, 0x8c, 0x4d, 0x42, 0x9e, + 0x09, 0x6d, 0x11, 0xb5, 0x11, 0x23, 0x25, 0xd0, 0x23, 0xb4, 0xda, 0x32, 0xac, 0x8d, 0x65, 0x58, + 0xc9, 0x3a, 0xd8, 0xde, 0x25, 0xf7, 0xae, 0x44, 0x3e, 0xd3, 0xa0, 0x97, 0xf4, 0xfa, 0x33, 0xe8, + 0xd6, 0xf7, 0x21, 0xdf, 0xa0, 0x2b, 0x7e, 0x83, 0xc0, 0x37, 0xa9, 0xfc, 0x24, 0x9b, 0xd0, 0x42, + 0x0f, 0x47, 0xd8, 0x3b, 0x3b, 0x20, 0xb7, 0xa3, 0x86, 0x50, 0x25, 0xf8, 0x85, 0xf9, 0x73, 0x43, + 0xce, 0x53, 0xdf, 0x5d, 0x7d, 0x1e, 0xe7, 0xee, 0x79, 0xd4, 0x90, 0xda, 0x3c, 0xee, 0xff, 0x9a, + 0xd0, 0xfd, 0x81, 0xa7, 0xf1, 0x59, 0x1a, 0x27, 0xb1, 0x60, 0x21, 0xd9, 0x5d, 0x3c, 0x9d, 0x42, + 0x71, 0x53, 0x0e, 0xae, 0xab, 0x6d, 0x0f, 0xcb, 0xe3, 0x2a, 0x74, 0xea, 0xe7, 0x77, 0xc1, 0x52, + 0xe8, 0xde, 0x72, 0x04, 0x2d, 0x91, 0x3a, 0x0a, 0x4f, 0xc4, 0x6f, 0x71, 0x7b, 0x5a, 0x42, 0x36, + 0x00, 0x66, 0x6c, 0x7e, 0xc4, 0x99, 0xe0, 0x87, 0x7e, 0x61, 0xbe, 0x15, 0x47, 0xe2, 0x3c, 0x63, + 0xf3, 0xd1, 0x3c, 0x1a, 0x09, 0xb4, 0xae, 0x26, 0x2d, 0x69, 0xf2, 0x53, 0x70, 0x66, 0x6c, 0x2e, + 0xfd, 0xe8, 0xd0, 0xd7, 0xd6, 0x55, 0x31, 0xc8, 0x87, 0xd0, 0xc8, 0xe6, 0x11, 0x06, 0x25, 0xf9, + 0x0e, 0xc9, 0x24, 0x63, 0x34, 0x8f, 0xb4, 0xc7, 0x51, 0x29, 0x2b, 0x00, 0xb5, 0x2b, 0x40, 0x7b, + 0xd0, 0xf0, 0x02, 0x1f, 0x1f, 0x22, 0x87, 0xca, 0xcf, 0xf5, 0x3f, 0x82, 0x7b, 0x4b, 0x38, 0xd4, + 0xef, 0x61, 0x55, 0x0d, 0xbb, 0x5f, 0xbf, 0x87, 0x66, 0x1d, 0xfb, 0xdf, 0x34, 0xe0, 0x9e, 0x36, + 0x86, 0xcb, 0x20, 0x19, 0x66, 0xd2, 0xec, 0xfb, 0xd0, 0xc6, 0x68, 0xc3, 0x53, 0x6d, 0x13, 0x05, + 0x49, 0xfe, 0x00, 0x2c, 0xf4, 0xc0, 0xc2, 0x4e, 0x1f, 0x56, 0xa8, 0x96, 0xc3, 0x95, 0xdd, 0xea, + 0x2b, 0xd1, 0xea, 0xe4, 0x1b, 0x68, 0xbd, 0xe4, 0x69, 0xac, 0xa2, 0x67, 0x67, 0x67, 0xe3, 0xb6, + 0x71, 0xf2, 0x6e, 0xf5, 0x30, 0xa5, 0xfc, 0x3b, 0x04, 0xff, 0x63, 0x19, 0x2f, 0x67, 0xf1, 0x35, + 0xf7, 0xfb, 0x6d, 0xdc, 0x51, 0xdd, 0x3e, 0x0a, 0x51, 0x81, 0xb6, 0x5d, 0xa1, 0x7d, 0x00, 0x9d, + 0xda, 0xf1, 0x6e, 0x41, 0xfa, 0xe1, 0xa2, 0xc5, 0x3b, 0xa5, 0x23, 0xd7, 0x1d, 0xe7, 0x00, 0xa0, + 0x3a, 0xec, 0xff, 0xd7, 0xfd, 0xdc, 0xbf, 0x32, 0xe0, 0xde, 0x7e, 0x1c, 0x45, 0x1c, 0x53, 0x20, + 0x75, 0x75, 0x95, 0xd9, 0x1b, 0x77, 0x9a, 0xfd, 0x67, 0xd0, 0x12, 0x52, 0x59, 0xcf, 0xfe, 0xee, + 0x2d, 0x77, 0x41, 0x95, 0x86, 0x0c, 0x33, 0x33, 0x36, 0x1f, 0x27, 0x3c, 0xf2, 0x83, 0x68, 0x5a, + 0x84, 0x99, 0x19, 0x9b, 0x9f, 0x29, 0x8e, 0xfb, 0xd7, 0x06, 0x58, 0xca, 0x63, 0x16, 0xa2, 0xb5, + 0xb1, 0x18, 0xad, 0x7f, 0x0a, 0x4e, 0x92, 0x72, 0x3f, 0xf0, 0x8a, 0x55, 0x1d, 0x5a, 0x31, 0xa4, + 0x71, 0x5e, 0xc4, 0xa9, 0xc7, 0x71, 0x7a, 0x9b, 0x2a, 0x42, 0x72, 0x45, 0xc2, 0x3c, 0x95, 0xc6, + 0x35, 0xa8, 0x22, 0x64, 0x8c, 0x57, 0x97, 0x83, 0x97, 0x62, 0x53, 0x4d, 0xb9, 0xff, 0x60, 0x42, + 0xf7, 0x20, 0x48, 0xb9, 0x97, 0x71, 0x7f, 0xe0, 0x4f, 0x51, 0x91, 0x47, 0x59, 0x90, 0xdd, 0xe8, + 0xf7, 0x44, 0x53, 0xe5, 0x73, 0x6f, 0x2e, 0xa6, 0xb4, 0x0a, 0xee, 0x06, 0x66, 0xe1, 0x8a, 0x20, + 0x3b, 0x00, 0x2a, 0x11, 0xc2, 0x4c, 0xbc, 0x79, 0x77, 0x26, 0xee, 0xa0, 0x9a, 0xfc, 0x94, 0x18, + 0xa8, 0x31, 0x81, 0x7a, 0x6b, 0x2c, 0x4c, 0xd3, 0x73, 0x69, 0xab, 0x98, 0x3f, 0x4c, 0x78, 0x88, + 0xb6, 0x88, 0xf9, 0xc3, 0x84, 0x87, 0x65, 0xd6, 0xd6, 0x56, 0xdb, 0x91, 0xdf, 0xe4, 0x23, 0x30, + 0xe3, 0x04, 0xcf, 0xa7, 0x17, 0xac, 0x1f, 0x6c, 0xfb, 0x34, 0xa1, 0x66, 0x9c, 0xc8, 0x8b, 0x56, + 0x69, 0x67, 0xdf, 0xd1, 0xf6, 0x2b, 0x03, 0x08, 0x26, 0x4c, 0x54, 0x4b, 0xdc, 0x07, 0x60, 0x9e, + 0x26, 0xa4, 0x0d, 0x8d, 0xe1, 0x60, 0xd4, 0x5b, 0x91, 0x1f, 0x07, 0x83, 0xa3, 0x9e, 0xe1, 0xbe, + 0x32, 0xc0, 0x39, 0xce, 0x33, 0x26, 0xcd, 0x46, 0xbc, 0xe9, 0xde, 0xde, 0x07, 0x5b, 0x64, 0x2c, + 0xc5, 0x20, 0xac, 0x22, 0x47, 0x1b, 0xe9, 0x91, 0x20, 0x9f, 0x42, 0x8b, 0xfb, 0x53, 0x5e, 0x38, + 0x74, 0x6f, 0x79, 0x9f, 0x54, 0x89, 0xc9, 0x16, 0x58, 0xc2, 0xbb, 0xe4, 0x33, 0xd6, 0x6f, 0x56, + 0x8a, 0x43, 0xe4, 0xa8, 0x47, 0x96, 0x6a, 0x39, 0x56, 0x09, 0x69, 0x9c, 0x60, 0xda, 0xdc, 0xd2, + 0x55, 0x42, 0x1a, 0x27, 0x32, 0x69, 0xde, 0x81, 0x9f, 0x04, 0xd3, 0x28, 0x4e, 0xf9, 0x38, 0x88, + 0x7c, 0x3e, 0x1f, 0x7b, 0x71, 0x74, 0x11, 0x06, 0x5e, 0x86, 0x58, 0xda, 0xf4, 0x5d, 0x25, 0x3c, + 0x94, 0xb2, 0x7d, 0x2d, 0x72, 0xe7, 0x60, 0x17, 0x71, 0x91, 0x7c, 0x26, 0x03, 0x1a, 0xc6, 0x55, + 0xed, 0x16, 0x98, 0xf6, 0xd7, 0x12, 0x1c, 0x5a, 0xc8, 0xe5, 0x35, 0xe1, 0x1a, 0x45, 0xa4, 0x44, + 0xa2, 0x9e, 0x5e, 0x35, 0xea, 0xe9, 0x15, 0x66, 0x8a, 0x71, 0xc4, 0x75, 0xc6, 0x81, 0xdf, 0xee, + 0xdf, 0x9a, 0x60, 0x97, 0x4f, 0xd9, 0xe7, 0xe0, 0xcc, 0x0a, 0xa8, 0xb5, 0xc3, 0x61, 0x2e, 0x5d, + 0xe2, 0x4f, 0x2b, 0x39, 0x79, 0x00, 0xe6, 0xd5, 0xb5, 0x06, 0xca, 0x92, 0x5a, 0xdf, 0xbf, 0xa0, + 0xe6, 0xd5, 0x75, 0xe5, 0xb1, 0xad, 0xb7, 0x7a, 0xec, 0x23, 0xb8, 0xe7, 0x85, 0x9c, 0x45, 0xe3, + 0xca, 0xe1, 0x94, 0xc1, 0xad, 0x21, 0xfb, 0xac, 0xf4, 0x3a, 0x1d, 0x75, 0xda, 0xd5, 0xdb, 0xf2, + 0x09, 0xb4, 0x7c, 0x1e, 0x66, 0xac, 0x5e, 0x1a, 0x9d, 0xa6, 0xcc, 0x0b, 0xf9, 0x81, 0x64, 0x53, + 0x25, 0x25, 0x5b, 0x60, 0x17, 0xef, 0xac, 0x2e, 0x88, 0x30, 0xf3, 0x2e, 0xc0, 0xa6, 0xa5, 0xb4, + 0xc2, 0x12, 0x6a, 0x58, 0xba, 0x5f, 0x41, 0xe3, 0xfb, 0x17, 0x43, 0x7d, 0x56, 0xe3, 0xb5, 0xb3, + 0x16, 0x88, 0x9a, 0x35, 0x44, 0xff, 0xa7, 0x01, 0x6d, 0xed, 0x75, 0x72, 0xdf, 0x79, 0x99, 0x25, + 0xca, 0xcf, 0xc5, 0xc7, 0xad, 0x74, 0xdf, 0x7a, 0x19, 0xdd, 0x78, 0x7b, 0x19, 0x4d, 0x7e, 0x01, + 0xdd, 0x44, 0xc9, 0xea, 0x0e, 0xff, 0x5e, 0x7d, 0x8c, 0xfe, 0xc5, 0x71, 0x9d, 0xa4, 0x22, 0xa4, + 0xe9, 0x62, 0x3d, 0x92, 0xb1, 0x29, 0x5e, 0x51, 0x97, 0xb6, 0x25, 0x3d, 0x62, 0xd3, 0x3b, 0xdc, + 0xfe, 0xb7, 0xf0, 0x5e, 0x99, 0x0d, 0xc7, 0x49, 0xbf, 0x8b, 0x1e, 0x29, 0x3d, 0xbe, 0xee, 0x8c, + 0xab, 0x8b, 0xce, 0xf8, 0x01, 0x38, 0x5e, 0x3c, 0x9b, 0x05, 0x28, 0x5b, 0xd3, 0xd9, 0x1e, 0x32, + 0x46, 0xc2, 0xfd, 0x1b, 0x03, 0xda, 0xfa, 0xb4, 0xa4, 0x03, 0xed, 0x83, 0xc1, 0xb3, 0xdd, 0xf3, + 0x23, 0x19, 0x0f, 0x00, 0xac, 0xbd, 0xc3, 0x93, 0x5d, 0xfa, 0x67, 0x3d, 0x43, 0xc6, 0x86, 0xc3, + 0x93, 0x51, 0xcf, 0x24, 0x0e, 0xb4, 0x9e, 0x1d, 0x9d, 0xee, 0x8e, 0x7a, 0x0d, 0x62, 0x43, 0x73, + 0xef, 0xf4, 0xf4, 0xa8, 0xd7, 0x24, 0x5d, 0xb0, 0x0f, 0x76, 0x47, 0x83, 0xd1, 0xe1, 0xf1, 0xa0, + 0xd7, 0x92, 0xba, 0xcf, 0x07, 0xa7, 0x3d, 0x4b, 0x7e, 0x9c, 0x1f, 0x1e, 0xf4, 0xda, 0x52, 0x7e, + 0xb6, 0x3b, 0x1c, 0xfe, 0xf2, 0x94, 0x1e, 0xf4, 0x6c, 0x39, 0xef, 0x70, 0x44, 0x0f, 0x4f, 0x9e, + 0xf7, 0x1c, 0xf9, 0x7d, 0xba, 0xf7, 0xdd, 0x60, 0x7f, 0xd4, 0x03, 0xf7, 0x2b, 0xe8, 0xd4, 0x10, + 0x94, 0xa3, 0xe9, 0xe0, 0x59, 0x6f, 0x45, 0x2e, 0xf9, 0x62, 0xf7, 0xe8, 0x7c, 0xd0, 0x33, 0xc8, + 0x1a, 0x00, 0x7e, 0x8e, 0x8f, 0x76, 0x4f, 0x9e, 0xf7, 0x4c, 0xf7, 0xf7, 0xc1, 0x3e, 0x0f, 0xfc, + 0xbd, 0x30, 0xf6, 0xae, 0xa4, 0x61, 0x4c, 0x98, 0xe0, 0xfa, 0x9d, 0xc4, 0x6f, 0x19, 0xe5, 0xd1, + 0x28, 0x85, 0xbe, 0x7b, 0x4d, 0xb9, 0x27, 0xd0, 0x3e, 0x0f, 0xfc, 0x33, 0xe6, 0x5d, 0xc9, 0x7a, + 0x7c, 0x22, 0xc7, 0x8f, 0x45, 0xf0, 0x92, 0xeb, 0x00, 0xe7, 0x20, 0x67, 0x18, 0xbc, 0xe4, 0xe4, + 0x63, 0xb0, 0x90, 0x28, 0x32, 0x1a, 0xb4, 0xe5, 0x62, 0x4d, 0xaa, 0x65, 0xee, 0xbf, 0x1a, 0xe5, + 0xde, 0xb1, 0xd8, 0x7e, 0x08, 0xcd, 0x84, 0x79, 0x57, 0x3a, 0x9a, 0x74, 0xf4, 0x18, 0xb9, 0x1e, + 0x45, 0x01, 0x79, 0x04, 0xb6, 0x36, 0x90, 0x62, 0xe2, 0x4e, 0xcd, 0x92, 0x68, 0x29, 0x5c, 0xbc, + 0xba, 0xc6, 0xe2, 0xd5, 0xc9, 0xbd, 0xcf, 0xf2, 0x30, 0x0b, 0xc6, 0x09, 0x4b, 0x8b, 0xea, 0xd9, + 0x41, 0xce, 0x19, 0x4b, 0x33, 0x39, 0x56, 0x59, 0x44, 0xae, 0x9f, 0x9b, 0x26, 0x55, 0x26, 0x72, + 0x1e, 0xf8, 0x32, 0x64, 0xf1, 0xc8, 0x47, 0x91, 0xca, 0x7e, 0x2c, 0x1e, 0xf9, 0xe7, 0x81, 0xef, + 0x7e, 0x03, 0x50, 0x35, 0x3c, 0x6e, 0xc9, 0xd9, 0xef, 0x43, 0x8b, 0x85, 0x81, 0x86, 0xd4, 0xa1, + 0x8a, 0x70, 0x4f, 0xa0, 0x53, 0x6b, 0x93, 0x48, 0x63, 0x64, 0x61, 0x38, 0xbe, 0xe2, 0x37, 0x02, + 0xc7, 0xda, 0xb4, 0xcd, 0xc2, 0xf0, 0x7b, 0x7e, 0x23, 0xc8, 0xc7, 0xd0, 0x52, 0x1d, 0x16, 0x73, + 0xa9, 0x90, 0xc7, 0xa1, 0x54, 0x09, 0xdd, 0x2f, 0xc0, 0x52, 0xd5, 0x7d, 0xcd, 0x17, 0x8c, 0x3b, + 0x5f, 0xb2, 0x6f, 0xf5, 0x9e, 0xb1, 0x17, 0x40, 0x3e, 0xd7, 0x9d, 0x1c, 0xa1, 0xfa, 0x46, 0x46, + 0x95, 0xc0, 0x29, 0x25, 0xdd, 0xc4, 0x41, 0x65, 0xf7, 0x00, 0xec, 0x37, 0xf6, 0xc6, 0x34, 0x00, + 0x66, 0x05, 0xc0, 0x2d, 0xdd, 0x32, 0xf7, 0x57, 0x00, 0x55, 0xc7, 0x47, 0xbb, 0xa6, 0x9a, 0x45, + 0xba, 0xe6, 0x63, 0x59, 0x6c, 0x05, 0xa1, 0x9f, 0xf2, 0x68, 0xe1, 0xd4, 0x55, 0x8f, 0xa8, 0x94, + 0x93, 0x4d, 0x68, 0x62, 0x23, 0xab, 0x51, 0x85, 0xce, 0xb2, 0x8b, 0x85, 0x12, 0x77, 0x02, 0xab, + 0xea, 0x81, 0xa4, 0xfc, 0x2f, 0x73, 0x2e, 0xde, 0x98, 0x59, 0x6d, 0x00, 0x94, 0x81, 0xbe, 0x68, + 0xc9, 0xd5, 0x38, 0xd2, 0x41, 0x2e, 0x02, 0x1e, 0xfa, 0xc5, 0x69, 0x34, 0xe5, 0xfe, 0x21, 0x74, + 0x8b, 0x35, 0xb0, 0x31, 0xf0, 0x79, 0xf9, 0x4c, 0x2b, 0x34, 0x55, 0x3d, 0xa2, 0x54, 0x4e, 0x62, + 0x9f, 0xef, 0x99, 0x7d, 0xa3, 0x78, 0xa9, 0xdd, 0x7f, 0x6f, 0x14, 0xa3, 0x75, 0x9d, 0xbc, 0x90, + 0xdf, 0x19, 0xcb, 0xf9, 0xdd, 0x62, 0x22, 0x65, 0xfe, 0x56, 0x89, 0xd4, 0xcf, 0xc1, 0xf1, 0x31, + 0x9b, 0x08, 0xae, 0x8b, 0xf0, 0xbd, 0xbe, 0x9c, 0x39, 0xe8, 0x7c, 0x23, 0xb8, 0xe6, 0xb4, 0x52, + 0x96, 0x7b, 0xc9, 0xe2, 0x2b, 0x1e, 0x05, 0x2f, 0xb1, 0x11, 0x20, 0x0f, 0x5d, 0x31, 0xaa, 0xae, + 0x8a, 0xca, 0x30, 0x74, 0x57, 0xa5, 0x68, 0x10, 0x59, 0x55, 0x83, 0x48, 0x22, 0x97, 0x27, 0x82, + 0xa7, 0x59, 0x91, 0x69, 0x2a, 0xaa, 0xcc, 0xd8, 0x1c, 0xad, 0x2b, 0x33, 0xb6, 0x0f, 0xa1, 0x1b, + 0xc5, 0xd1, 0x38, 0xca, 0xc3, 0x50, 0xe6, 0xc2, 0xba, 0xeb, 0xd7, 0x89, 0xe2, 0xe8, 0x44, 0xb3, + 0xc8, 0x63, 0x78, 0xa7, 0xae, 0xa2, 0x2c, 0xb7, 0xa3, 0x5a, 0x09, 0x35, 0x3d, 0xb4, 0xef, 0x2d, + 0xe8, 0xc5, 0x93, 0x5f, 0x71, 0x2f, 0x43, 0xc4, 0xc6, 0x68, 0xb2, 0x5d, 0xf5, 0x88, 0x2b, 0xbe, + 0x84, 0xe8, 0x84, 0xcd, 0xb8, 0xfb, 0x2d, 0x38, 0x25, 0x08, 0x32, 0x5e, 0x9f, 0x9c, 0x9e, 0x0c, + 0x54, 0x44, 0x3d, 0x3c, 0x39, 0x18, 0xfc, 0x69, 0xcf, 0x90, 0x11, 0x9f, 0x0e, 0x5e, 0x0c, 0xe8, + 0x70, 0xd0, 0x33, 0x65, 0x34, 0x3e, 0x18, 0x1c, 0x0d, 0x46, 0x83, 0x5e, 0xe3, 0xbb, 0xa6, 0xdd, + 0xee, 0xd9, 0xd4, 0xe6, 0xf3, 0x24, 0x0c, 0xbc, 0x20, 0x73, 0x87, 0x00, 0x72, 0x5a, 0x7d, 0xa3, + 0x1f, 0x80, 0x53, 0xad, 0xad, 0x6e, 0xd4, 0xce, 0xf4, 0xaa, 0x32, 0xa7, 0xd3, 0x46, 0x65, 0xde, + 0x95, 0xd3, 0x69, 0x33, 0x3b, 0x07, 0xfb, 0x98, 0x25, 0xaf, 0x95, 0x39, 0xdd, 0xb2, 0x98, 0xcd, + 0x75, 0x6b, 0x47, 0x3f, 0xe5, 0x9f, 0x40, 0x5b, 0x47, 0x46, 0xed, 0x1f, 0x0b, 0x51, 0xb3, 0x90, + 0xc9, 0xaa, 0xe3, 0xfe, 0x71, 0x7c, 0xcd, 0xcb, 0x6c, 0xe6, 0x8c, 0xdd, 0x84, 0x31, 0xf3, 0xdf, + 0x62, 0x88, 0x3f, 0x03, 0x10, 0x71, 0x9e, 0x7a, 0x7c, 0x3c, 0x2d, 0x3b, 0x4a, 0x8e, 0xe2, 0x3c, + 0xd7, 0x6d, 0x6a, 0x2e, 0x32, 0x14, 0x36, 0x94, 0x9b, 0x49, 0x5a, 0x8a, 0x7e, 0x02, 0x56, 0x36, + 0x8f, 0xaa, 0x06, 0x56, 0x2b, 0x93, 0x35, 0xa6, 0xbb, 0x0f, 0xce, 0x68, 0x8e, 0x95, 0x57, 0x2e, + 0x16, 0xde, 0x67, 0xe3, 0x0d, 0xef, 0xb3, 0xb9, 0xf4, 0x3e, 0xff, 0x97, 0x01, 0x9d, 0x5a, 0x9a, + 0x45, 0x3e, 0x84, 0x66, 0x36, 0x8f, 0x16, 0x3b, 0xbf, 0xc5, 0x22, 0x14, 0x45, 0xd2, 0xde, 0x64, + 0x59, 0xc6, 0x84, 0x08, 0xa6, 0x11, 0xf7, 0xf5, 0x94, 0xb2, 0x54, 0xdb, 0xd5, 0x2c, 0x72, 0x04, + 0xf7, 0x54, 0xcc, 0x28, 0xba, 0x3e, 0x45, 0xa6, 0xfe, 0xd1, 0x52, 0x5a, 0xa7, 0xaa, 0xd3, 0xfd, + 0x42, 0x4b, 0xd5, 0xdf, 0x6b, 0xd3, 0x05, 0xe6, 0xfa, 0x2e, 0xbc, 0x7b, 0x8b, 0xda, 0x8f, 0x6a, + 0x34, 0x3c, 0x84, 0x55, 0x59, 0x98, 0x07, 0x33, 0x2e, 0x32, 0x36, 0x4b, 0x30, 0xbf, 0xd1, 0x31, + 0xbf, 0x49, 0xcd, 0x4c, 0xb8, 0x9f, 0x42, 0xf7, 0x8c, 0xf3, 0x94, 0x72, 0x91, 0xc4, 0x91, 0x7a, + 0xdb, 0x05, 0x1e, 0x5a, 0x3f, 0x30, 0x9a, 0x72, 0xff, 0x02, 0x1c, 0x99, 0xb9, 0xef, 0xb1, 0xcc, + 0xbb, 0xfc, 0x31, 0x99, 0xfd, 0xa7, 0xd0, 0x4e, 0x94, 0x99, 0xe8, 0x3c, 0xbc, 0x8b, 0x31, 0x4e, + 0x9b, 0x0e, 0x2d, 0x84, 0xee, 0x37, 0xd0, 0x38, 0xc9, 0x67, 0xf5, 0x3f, 0x66, 0x9a, 0xea, 0x8f, + 0x99, 0x0f, 0xc0, 0xc1, 0x22, 0x00, 0xbb, 0x80, 0x2a, 0x3d, 0xb5, 0x25, 0x03, 0xdb, 0x7f, 0x3f, + 0x40, 0xa7, 0xc0, 0xfe, 0xd0, 0xc7, 0x7f, 0x57, 0xf0, 0xf2, 0x0f, 0xfd, 0x05, 0x5b, 0x50, 0x75, + 0x20, 0x8f, 0xfc, 0xc3, 0xe2, 0xd2, 0x14, 0xb1, 0x38, 0xb7, 0x7e, 0xca, 0xcb, 0xb9, 0x9f, 0x41, + 0xb7, 0xc8, 0xae, 0x8f, 0x79, 0xc6, 0xd0, 0x9c, 0xc2, 0x80, 0x47, 0x35, 0x53, 0xb3, 0x15, 0x63, + 0x24, 0xde, 0xd0, 0x34, 0x75, 0xb7, 0xc1, 0xd2, 0xb6, 0x4a, 0xa0, 0xe9, 0xc5, 0xbe, 0x72, 0x91, + 0x16, 0xc5, 0x6f, 0x79, 0xe0, 0x99, 0x98, 0x16, 0x0f, 0xe1, 0x4c, 0x4c, 0xdd, 0x39, 0xac, 0xee, + 0x31, 0xef, 0x2a, 0x4f, 0x8a, 0x87, 0xa8, 0x56, 0x06, 0x19, 0x0b, 0x65, 0xd0, 0x1b, 0x3a, 0xb5, + 0xef, 0x41, 0x3b, 0x8f, 0x82, 0x79, 0x91, 0xde, 0x38, 0xd4, 0x92, 0xa4, 0xea, 0x50, 0x86, 0xb1, + 0x87, 0x95, 0x0f, 0x7a, 0x95, 0x43, 0x4b, 0xda, 0xfd, 0x73, 0x58, 0x1d, 0xcc, 0x13, 0xec, 0x5b, + 0xbf, 0xf5, 0x09, 0xac, 0x6d, 0xca, 0x5c, 0xd8, 0xd4, 0xd2, 0xca, 0x8d, 0x62, 0xe5, 0x9d, 0x7f, + 0x32, 0xa0, 0x29, 0x4d, 0x44, 0x16, 0x67, 0x7f, 0xcc, 0x59, 0x9a, 0x4d, 0x38, 0xcb, 0xc8, 0x82, + 0x39, 0xac, 0x2f, 0x50, 0xee, 0xca, 0x53, 0x83, 0x7c, 0xa1, 0x5a, 0xe2, 0x45, 0xa7, 0x7f, 0xb5, + 0x30, 0x34, 0x34, 0xc4, 0x65, 0x7d, 0xb2, 0x0d, 0x9d, 0xef, 0xe2, 0x20, 0xda, 0x57, 0x5d, 0x62, + 0xb2, 0x6c, 0x96, 0xaf, 0xe9, 0x7f, 0x09, 0xd6, 0xa1, 0x90, 0xf6, 0xff, 0xba, 0x2a, 0x86, 0xd7, + 0xba, 0x6b, 0xb8, 0x2b, 0x3b, 0xff, 0xd8, 0x80, 0xe6, 0x0f, 0x3c, 0x8d, 0xc9, 0x17, 0xd0, 0xd6, + 0x3d, 0x20, 0x52, 0xeb, 0xf5, 0xac, 0xe3, 0xfb, 0xba, 0xd4, 0x1c, 0xc2, 0x55, 0x7a, 0x2a, 0x42, + 0x57, 0xd5, 0x23, 0xa9, 0x5a, 0x54, 0xaf, 0x6d, 0xea, 0x5b, 0xe8, 0x0d, 0xb3, 0x94, 0xb3, 0x59, + 0x4d, 0x7d, 0x11, 0xa6, 0xdb, 0x4a, 0x51, 0x44, 0xeb, 0x73, 0xb0, 0x54, 0x90, 0x59, 0x1a, 0xb0, + 0x5c, 0x55, 0xa2, 0xf2, 0x23, 0xe8, 0x0c, 0x2f, 0xe3, 0x3c, 0xf4, 0x87, 0x3c, 0xbd, 0xe6, 0xa4, + 0xd6, 0x87, 0x5d, 0xaf, 0x7d, 0xbb, 0x2b, 0x64, 0x0b, 0x40, 0x79, 0xd9, 0x79, 0xe0, 0x0b, 0xd2, + 0x96, 0xb2, 0x93, 0x7c, 0xa6, 0x26, 0xad, 0xb9, 0x9f, 0xd2, 0xac, 0xc5, 0x9a, 0x37, 0x69, 0x7e, + 0x0d, 0xab, 0xfb, 0x18, 0x8b, 0x4f, 0xd3, 0xdd, 0x49, 0x9c, 0x66, 0x64, 0xb9, 0x17, 0xbb, 0xbe, + 0xcc, 0x70, 0x57, 0xc8, 0x53, 0xb0, 0x47, 0xe9, 0x8d, 0xd2, 0x7f, 0x47, 0x87, 0xe8, 0x6a, 0xbd, + 0x5b, 0x4e, 0xb9, 0xf3, 0xf7, 0x0d, 0xb0, 0x7e, 0x19, 0xa7, 0x57, 0x3c, 0x25, 0x8f, 0xc1, 0xc2, + 0xf2, 0x5f, 0x1b, 0x51, 0xd9, 0x0a, 0xb8, 0x6d, 0xa1, 0x8f, 0xc1, 0x41, 0x50, 0x46, 0x4c, 0x5c, + 0xa9, 0xab, 0xc2, 0xbf, 0x66, 0x15, 0x2e, 0x2a, 0x79, 0xc3, 0x7b, 0x5d, 0x53, 0x17, 0x55, 0xb6, + 0x3c, 0x16, 0x6a, 0xf2, 0xf5, 0xb6, 0x2a, 0xb0, 0x87, 0xee, 0xca, 0x96, 0xf1, 0xd4, 0x20, 0x9f, + 0x41, 0x73, 0xa8, 0x4e, 0x2a, 0x95, 0xaa, 0xbf, 0xaf, 0xd6, 0xd7, 0x0a, 0x46, 0x39, 0xf3, 0x13, + 0xb0, 0xd4, 0xcb, 0xae, 0x8e, 0xb9, 0x90, 0x98, 0xae, 0xf7, 0xea, 0x2c, 0x3d, 0xe0, 0x33, 0xb0, + 0x54, 0xd0, 0x50, 0x03, 0x16, 0x02, 0x88, 0xda, 0xb5, 0x8a, 0x41, 0x4a, 0x55, 0x79, 0xb9, 0x52, + 0x5d, 0xf0, 0xf8, 0x25, 0xd5, 0x2f, 0xa1, 0x47, 0xb9, 0xc7, 0x83, 0xda, 0x9b, 0x4f, 0x8a, 0x43, + 0x2d, 0x9b, 0xed, 0x96, 0x41, 0xbe, 0x85, 0xd5, 0x85, 0xfc, 0x80, 0xf4, 0x11, 0xe8, 0x5b, 0x52, + 0x86, 0xe5, 0xc1, 0x7b, 0xbd, 0x7f, 0x79, 0xb5, 0x61, 0xfc, 0xdb, 0xab, 0x0d, 0xe3, 0x3f, 0x5e, + 0x6d, 0x18, 0xbf, 0xfe, 0xcf, 0x8d, 0x95, 0x89, 0x85, 0x7f, 0xe9, 0x7f, 0xfd, 0x7f, 0x01, 0x00, + 0x00, 0xff, 0xff, 0xc2, 0xe7, 0x33, 0xd9, 0x16, 0x20, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -6458,23 +6449,13 @@ func (m *PostingList) MarshalTo(dAtA []byte) (int, error) { } i++ } - if m.FirstPart { - dAtA[i] = 0x28 - i++ - if m.FirstPart { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ - } if m.StartUid != 0 { - dAtA[i] = 0x30 + dAtA[i] = 0x28 i++ i = encodeVarintPb(dAtA, i, uint64(m.StartUid)) } if m.EndUid != 0 { - dAtA[i] = 0x38 + dAtA[i] = 0x30 i++ i = encodeVarintPb(dAtA, i, uint64(m.EndUid)) } @@ -8290,9 +8271,6 @@ func (m *PostingList) Size() (n int) { if m.MultiPart { n += 2 } - if m.FirstPart { - n += 2 - } if m.StartUid != 0 { n += 1 + sovPb(uint64(m.StartUid)) } @@ -13630,26 +13608,6 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { } m.MultiPart = bool(v != 0) case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field FirstPart", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.FirstPart = bool(v != 0) - case 6: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field StartUid", wireType) } @@ -13668,7 +13626,7 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { break } } - case 7: + case 6: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field EndUid", wireType) } From e29a15dd07771ebde2ed5c00ade4c43071e9b02c Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 18:15:55 -0800 Subject: [PATCH 11/67] Fix bug in Uids function. --- posting/list.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/list.go b/posting/list.go index d1ebf06db32..c74d6e7165b 100644 --- a/posting/list.go +++ b/posting/list.go @@ -877,7 +877,7 @@ func (l *List) Uids(opt ListOptions) (*pb.List, error) { // Use approximate length for initial capacity. res := make([]uint64, 0, len(l.mutationMap)+codec.ApproxLen(l.plist.Pack)) out := &pb.List{} - if len(l.mutationMap) == 0 && opt.Intersect != nil { + if len(l.mutationMap) == 0 && opt.Intersect != nil && !l.plist.MultiPart { if opt.ReadTs < l.minTs { l.RUnlock() return out, ErrTsTooOld From ebd83032d0d4babb532aa068f48763f4068c3ec9 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 18:16:30 -0800 Subject: [PATCH 12/67] Removed unused field --- posting/list.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/posting/list.go b/posting/list.go index c74d6e7165b..214d183c957 100644 --- a/posting/list.go +++ b/posting/list.go @@ -108,8 +108,6 @@ type PIterator struct { dec *codec.Decoder uids []uint64 uidx int // Offset into the uids slice - - partIndex int // Offset into the parts list (if a multi-part list). } type PItrOpts struct { From 025063606daa4e6a4c8354632b711e413a3dc036 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 18:26:51 -0800 Subject: [PATCH 13/67] Simplify rollup logic. --- posting/list.go | 48 +++++++++++------------------------------------- 1 file changed, 11 insertions(+), 37 deletions(-) diff --git a/posting/list.go b/posting/list.go index 214d183c957..b0fea96efc0 100644 --- a/posting/list.go +++ b/posting/list.go @@ -765,11 +765,18 @@ func (l *List) rollup(readTs uint64) error { return nil } - if !l.plist.MultiPart { + var plsToRollup []*pb.PostingList + if l.plist.MultiPart { + plsToRollup = append(plsToRollup, l.parts...) + } else { + plsToRollup = append(plsToRollup, l.plist) + } + + for partIdx, part := range plsToRollup { final := new(pb.PostingList) enc := codec.Encoder{BlockSize: blockSize} - err := l.iterate(readTs, 0, func(p *pb.Posting) error { + err := l.iterateInternal(readTs, 0, true, partIdx, func(p *pb.Posting) error { // iterate already takes care of not returning entries whose commitTs // is above curr.commitTs. // So, we don't need to do any filtering here. In fact, doing filtering @@ -787,41 +794,8 @@ func (l *List) rollup(readTs uint64) error { }) x.Check(err) final.Pack = enc.Done() - l.plist.Pack = final.Pack - l.plist.Postings = final.Postings - } else { - for partIdx, part := range l.parts { - // If no changes have been made to this part, rollup does not need to - // change this list part. - deleteBelow, mposts := l.pickPartPostings(readTs, partIdx) - if deleteBelow == 0 && len(mposts) == 0 { - continue - } - - final := new(pb.PostingList) - - enc := codec.Encoder{BlockSize: blockSize} - err := l.partIterate(readTs, partIdx, func(p *pb.Posting) error { - // iterate already takes care of not returning entries whose commitTs - // is above curr.commitTs. - // So, we don't need to do any filtering here. In fact, doing filtering - // here could result in a bug. - enc.Add(p.Uid) - - // We want to add the posting if it has facets or has a value. - if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { - // I think it's okay to take the pointer from the iterator, because - // we have a lock over List; which won't be released until final has - // been marshalled. Thus, the underlying data wouldn't be changed. - final.Postings = append(final.Postings, p) - } - return nil - }) - x.Check(err) - final.Pack = enc.Done() - part.Pack = final.Pack - part.Postings = final.Postings - } + part.Pack = final.Pack + part.Postings = final.Postings } maxCommitTs := l.minTs From b0eddc42f2d33d0da7f3acf994cdf1b333b1e778 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 6 Mar 2019 19:53:25 -0800 Subject: [PATCH 14/67] Change order of if-statements. --- posting/list.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/posting/list.go b/posting/list.go index b0fea96efc0..6213260dab4 100644 --- a/posting/list.go +++ b/posting/list.go @@ -163,13 +163,13 @@ func (it *PIterator) Valid() bool { return true } - // Only iterate through this part of the list. - if it.opts.partialIteration { + // Not a multi-part list, so nothing else to iterate through + if !it.l.plist.MultiPart { return false } - // Not a multi-part list, so nothing else to iterate through - if !it.l.plist.MultiPart { + // Only iterate through this part of the list. + if it.opts.partialIteration { return false } From aad3f20de6cb27826ca67f868461a912cb7f7521 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 10:16:04 -0800 Subject: [PATCH 15/67] Remove unused field. --- posting/list.go | 1 - 1 file changed, 1 deletion(-) diff --git a/posting/list.go b/posting/list.go index 6213260dab4..5107d006494 100644 --- a/posting/list.go +++ b/posting/list.go @@ -103,7 +103,6 @@ type PIterator struct { uidPosting *pb.Posting pidx int // index of postings plen int - firstPart bool dec *codec.Decoder uids []uint64 From 60296300d0a70429b4000b95120fe3feb4a7bfb8 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 15:58:19 -0800 Subject: [PATCH 16/67] Load list parts on demand. Also simpler split logic. --- posting/list.go | 291 +++++++++++----------- posting/list_test.go | 37 +-- posting/mvcc.go | 4 +- protos/pb.proto | 5 +- protos/pb/pb.pb.go | 568 +++++++++++++++++++++++++------------------ 5 files changed, 511 insertions(+), 394 deletions(-) diff --git a/posting/list.go b/posting/list.go index 5107d006494..ba4b606b0c0 100644 --- a/posting/list.go +++ b/posting/list.go @@ -71,7 +71,7 @@ type List struct { x.SafeMutex key []byte plist *pb.PostingList - parts []*pb.PostingList // If a multi-part list, the parts will be stored here. + newParts map[uint64]*pb.PostingList mutationMap map[uint64]*pb.PostingList minTs uint64 // commit timestamp of immutable layer, reject reads before this ts. maxTs uint64 // max commit timestamp seen for this list. @@ -117,9 +117,13 @@ type PItrOpts struct { readTs uint64 } -func (it *PIterator) Init(l *List, opts PItrOpts) { - if l.plist.MultiPart { - it.plist = l.parts[opts.startPart] +func (it *PIterator) Init(l *List, opts PItrOpts) error { + if len(l.plist.Parts) > 0 { + plist, err := l.readListPart(l.key, l.plist.Parts[opts.startPart]) + if err != nil { + return err + } + it.plist = plist } else { it.plist = l.plist } @@ -137,6 +141,7 @@ func (it *PIterator) Init(l *List, opts PItrOpts) { p := it.plist.Postings[idx] return it.opts.afterUid < p.Uid }) + return nil } func (it *PIterator) Next() error { @@ -173,7 +178,7 @@ func (it *PIterator) Valid() bool { } // No more parts to iterate through - if len(it.l.parts) == it.opts.startPart+1 { + if len(it.l.plist.Parts) == it.opts.startPart+1 { return false } @@ -507,15 +512,6 @@ func (l *List) Conflicts(readTs uint64) []uint64 { } func (l *List) pickPostings(readTs uint64) (uint64, []*pb.Posting) { - return l.pickPostingsInternal(readTs, false, 0) -} - -func (l *List) pickPartPostings(readTs uint64, partIdx int) (uint64, []*pb.Posting) { - return l.pickPostingsInternal(readTs, true, partIdx) -} - -func (l *List) pickPostingsInternal(readTs uint64, partial bool, partIdx int) ( - uint64, []*pb.Posting) { // This function would return zero ts for entries above readTs. effective := func(start, commit uint64) uint64 { if commit > 0 && commit <= readTs { @@ -538,10 +534,6 @@ func (l *List) pickPostingsInternal(readTs uint64, partial bool, partIdx int) ( if effectiveTs > deleteBelow { // We're above the deleteBelow marker. We wouldn't reach here if effectiveTs is zero. for _, mpost := range plist.Postings { - if partial && (mpost.Uid < l.parts[partIdx].StartUid || - l.parts[partIdx].EndUid < mpost.Uid) { - continue - } if hasDeleteAll(mpost) { deleteBelow = effectiveTs continue @@ -579,24 +571,9 @@ func (l *List) pickPostingsInternal(readTs uint64, partial bool, partIdx int) ( } func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) error) error { - return l.iterateInternal(readTs, afterUid, false, 0, f) -} - -func (l *List) partIterate(readTs uint64, partIdx int, f func(obj *pb.Posting) error) error { - return l.iterateInternal(readTs, 0, true, partIdx, f) -} - -func (l *List) iterateInternal(readTs uint64, afterUid uint64, partial bool, partIdx int, - f func(obj *pb.Posting) error) error { l.AssertRLock() - // If not a multi-part list iterate through the whole list. - if !l.plist.MultiPart { - partial = false - partIdx = 0 - } - - deleteBelow, mposts := l.pickPostingsInternal(readTs, partial, partIdx) + deleteBelow, mposts := l.pickPostings(readTs) if readTs < l.minTs { return x.Errorf("readTs: %d less than minTs: %d for key: %q", readTs, l.minTs, l.key) } @@ -611,15 +588,17 @@ func (l *List) iterateInternal(readTs uint64, afterUid uint64, partial bool, par var mp, pp *pb.Posting var pitr PIterator - pitr.Init(l, PItrOpts{ - afterUid: afterUid, - discardPl: deleteBelow > 0, - readTs: readTs, - partialIteration: partial, - startPart: partIdx, + err := pitr.Init(l, PItrOpts{ + afterUid: afterUid, + discardPl: deleteBelow > 0, + readTs: readTs, + startPart: 0, }) + if err != nil { + return err + } + prevUid := uint64(0) - var err error for err == nil { if midx < mlen { mp = mposts[midx] @@ -722,11 +701,15 @@ func (l *List) MarshalToKv() ([]*bpb.KV, error) { kv.Value = val kvs = append(kvs, kv) - for _, part := range l.parts { + for _, startUid := range l.plist.Parts { kv := &bpb.KV{} kv.Version = l.minTs - kv.Key = getNextPartKey(l.key, part.StartUid) - val, meta := marshalPostingList(part) + kv.Key = getNextPartKey(l.key, startUid) + plist, err := l.readListPart(l.key, startUid) + if err != nil { + return nil, err + } + val, meta := marshalPostingList(plist) kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) @@ -764,20 +747,13 @@ func (l *List) rollup(readTs uint64) error { return nil } - var plsToRollup []*pb.PostingList - if l.plist.MultiPart { - plsToRollup = append(plsToRollup, l.parts...) - } else { - plsToRollup = append(plsToRollup, l.plist) - } - - for partIdx, part := range plsToRollup { + if len(l.plist.Parts) == 0 { + // This is not a multi-part list so use a single encoder to collect all the changes. final := new(pb.PostingList) - enc := codec.Encoder{BlockSize: blockSize} - err := l.iterateInternal(readTs, 0, true, partIdx, func(p *pb.Posting) error { + err := l.iterate(readTs, 0, func(p *pb.Posting) error { // iterate already takes care of not returning entries whose commitTs - // is above curr.commitTs. + // is above l.commitTs. // So, we don't need to do any filtering here. In fact, doing filtering // here could result in a bug. enc.Add(p.Uid) @@ -793,8 +769,63 @@ func (l *List) rollup(readTs uint64) error { }) x.Check(err) final.Pack = enc.Done() - part.Pack = final.Pack - part.Postings = final.Postings + l.plist.Pack = final.Pack + l.plist.Postings = final.Postings + } else { + // This is a multi-part list. Use a single iteration but multiple encoders, + // one for each part of the list. + var plist *pb.PostingList + var final *pb.PostingList + var enc codec.Encoder + var endUid uint64 + var splitIdx int + + init := func() error { + startUid := l.plist.Parts[splitIdx] + if splitIdx+1 == len(l.plist.Parts) { + endUid = math.MaxUint64 + } else { + endUid = l.plist.Parts[splitIdx+1] - 1 + } + + pl, err := l.readListPart(l.key, startUid) + if err != nil { + return err + } + + plist = pl + final = new(pb.PostingList) + enc = codec.Encoder{BlockSize: blockSize} + return nil + } + if err := init(); err != nil { + return err + } + + err := l.iterate(readTs, 0, func(p *pb.Posting) error { + if p.Uid > endUid { + final.Pack = enc.Done() + plist.Pack = final.Pack + plist.Postings = final.Postings + l.newParts[plist.StartUid] = plist + + splitIdx++ + init() + } + + enc.Add(p.Uid) + if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { + final.Postings = append(final.Postings, p) + } + return nil + }) + x.Check(err) + + // Finish writing the last part of the list. + final.Pack = enc.Done() + plist.Pack = final.Pack + plist.Postings = final.Postings + l.newParts[plist.StartUid] = plist } maxCommitTs := l.minTs @@ -1086,132 +1117,104 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs return facets.CopyFacets(p.Facets, param), nil } -func (l *List) readListParts(baseKey []byte, readTs uint64) error { +func (l *List) readListPart(baseKey []byte, startUid uint64) (*pb.PostingList, error) { if !l.plist.MultiPart { - return nil + return nil, nil } - var nextPartStart uint64 - for { - nextKey := getNextPartKey(baseKey, nextPartStart) - txn := pstore.NewTransactionAt(readTs, false) - item, err := txn.Get(nextKey) - if err != nil { - return err - } - var part pb.PostingList - if err := unmarshalOrCopy(&part, item); err != nil { - return err - } - l.parts = append(l.parts, &part) + if part, ok := l.newParts[startUid]; ok { + return part, nil + } - nextPartStart = part.EndUid - if nextPartStart == math.MaxUint64 { - break - } - nextPartStart++ + nextKey := getNextPartKey(baseKey, startUid) + txn := pstore.NewTransactionAt(l.minTs, false) + item, err := txn.Get(nextKey) + if err != nil { + return nil, err } - return nil + var part pb.PostingList + if err := unmarshalOrCopy(&part, item); err != nil { + return nil, err + } + return &part, nil } func needsSplit(plist *pb.PostingList) bool { - return plist.Size() >= maxListSize + return plist.Size() >= maxListSize && len(plist.Pack.Blocks) > 1 } func (l *List) splitList(readTs uint64) error { l.AssertLock() - if !l.plist.MultiPart { + if len(l.plist.Parts) == 0 { if needsSplit(l.plist) { - l.parts = l.splitPostingList(readTs, 0) + var newParts []uint64 + parts := splitPostingList(l.plist) + for _, part := range parts { + l.newParts[part.StartUid] = part + newParts = append(newParts, part.StartUid) + } + l.plist = &pb.PostingList{ CommitTs: l.plist.CommitTs, MultiPart: true, + Parts: newParts, } } return nil } - var newParts []*pb.PostingList - for partIdx, part := range l.parts { + var newParts []uint64 + for _, startUid := range l.plist.Parts { + part, err := l.readListPart(l.key, startUid) + if err != nil { + return err + } + if needsSplit(part) { - splitParts := l.splitPostingList(readTs, partIdx) - newParts = append(newParts, splitParts...) + splitParts := splitPostingList(part) + for _, part := range splitParts { + l.newParts[part.StartUid] = part + newParts = append(newParts, part.StartUid) + } } else { - newParts = append(newParts, part) + newParts = append(newParts, part.StartUid) } } + l.plist.Parts = newParts - l.parts = newParts return nil } -func (l *List) splitPostingList(readTs uint64, partIdx int) []*pb.PostingList { - var uids []uint64 - err := l.partIterate(readTs, partIdx, func(p *pb.Posting) error { - uids = append(uids, p.Uid) - return nil - }) - x.Check(err) - - sort.Slice(uids, func(i, j int) bool { return uids[i] < uids[j] }) - midUid := uids[len(uids)/2] +func splitPostingList(plist *pb.PostingList) []*pb.PostingList { + midBlock := len(plist.Pack.Blocks) / 2 + midUid := plist.Pack.Blocks[midBlock].GetBase() // Generate posting list holding the first half of the current list's postings. lowPl := new(pb.PostingList) - lowEnc := codec.Encoder{BlockSize: blockSize} - err = l.partIterate(readTs, partIdx, func(p *pb.Posting) error { - // Skip all postings with an UID greater than or equal to midUid. - if p.Uid >= midUid { - return nil - } - - lowEnc.Add(p.Uid) - - if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { - lowPl.Postings = append(lowPl.Postings, p) - } - return nil - }) - x.Check(err) - lowPl.Pack = lowEnc.Done() - lowPl.CommitTs = l.plist.CommitTs - lowPl.MultiPart = true - if !l.plist.MultiPart { - lowPl.StartUid = 0 - } else { - lowPl.StartUid = l.parts[partIdx].StartUid + lowPl.Pack = &pb.UidPack{ + BlockSize: plist.Pack.BlockSize, + Blocks: plist.Pack.Blocks[:midBlock], } - lowPl.EndUid = midUid - 1 + lowPl.CommitTs = plist.CommitTs + lowPl.StartUid = plist.StartUid // Generate posting list holding the second half of the current list's postings. highPl := new(pb.PostingList) - highEnc := codec.Encoder{BlockSize: blockSize} - err = l.partIterate(readTs, partIdx, func(p *pb.Posting) error { - // Skip all postings with an UID less than midUid. - if p.Uid < midUid { - return nil - } + highPl.Pack = &pb.UidPack{ + BlockSize: plist.Pack.BlockSize, + Blocks: plist.Pack.Blocks[midBlock:], + } + highPl.CommitTs = plist.CommitTs + highPl.StartUid = midUid - highEnc.Add(p.Uid) - if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { - highPl.Postings = append(highPl.Postings, p) + // Add elements in plist.Postings to the corresponding list. + for _, posting := range plist.Postings { + if posting.Uid < midUid { + lowPl.Postings = append(lowPl.Postings, posting) + } else { + highPl.Postings = append(highPl.Postings, posting) } - return nil - }) - x.Check(err) - highPl.Pack = highEnc.Done() - highPl.CommitTs = l.plist.CommitTs - highPl.MultiPart = true - highPl.StartUid = midUid - if !l.plist.MultiPart { - // We are splitting a list that was previously non-split. So the new - // list should end in this part. - highPl.EndUid = math.MaxUint64 - } else { - // Else, this posting list should point to the part the original list - // is pointing to. - highPl.EndUid = l.parts[partIdx].EndUid } return []*pb.PostingList{lowPl, highPl} diff --git a/posting/list_test.go b/posting/list_test.go index cfb03cde988..af1f7421dfe 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -908,11 +908,6 @@ func createMultiPartList(t *testing.T, size int) (*List, int) { defer func() { maxListSize = 2000000 }() - // For testing, set the max list length to a lower threshold. - maxListSize = 10000 - defer func() { - maxListSize = 2000000 - }() key := x.DataKey("bal", 1331) ol, err := getNew(key, ps) @@ -940,7 +935,7 @@ func createMultiPartList(t *testing.T, size int) (*List, int) { func TestMultiPartList(t *testing.T) { N := int(1e5) ol, commits := createMultiPartList(t, N) - t.Logf("List parts %v", len(ol.parts)) + t.Logf("List parts %v", len(ol.plist.Parts)) opt := ListOptions{ReadTs: uint64(N) + 1} l, err := ol.Uids(opt) require.NoError(t, err) @@ -965,7 +960,7 @@ func TestMultiPartListWithPostings(t *testing.T) { for i := 2; i <= N; i += 2 { edge := &pb.DirectedEdge{ ValueId: uint64(i), - Label: strconv.Itoa(i), + Label: strconv.Itoa(i), } txn := Txn{StartTs: uint64(i)} addMutationHelper(t, ol, edge, Set, &txn) @@ -978,10 +973,10 @@ func TestMultiPartListWithPostings(t *testing.T) { } commits++ } - t.Logf("List parts %v", len(ol.parts)) + t.Logf("List parts %v", len(ol.plist.Parts)) var labels []string - ol.Iterate(uint64(N) + 1, 0, func(p *pb.Posting) error { + ol.Iterate(uint64(N)+1, 0, func(p *pb.Posting) error { if len(p.Label) > 0 { labels = append(labels, p.Label) } @@ -996,23 +991,37 @@ func TestMultiPartListWithPostings(t *testing.T) { func TestMultiPartListMarshal(t *testing.T) { N := int(1e5) - ol, _ := createMultiPartList(t, N) - t.Logf("List parts %v", len(ol.parts)) + ol, commits := createMultiPartList(t, N) + t.Logf("List parts %v", len(ol.plist.Parts)) kvs, err := ol.MarshalToKv() require.NoError(t, err) - require.Equal(t, len(kvs), len(ol.parts) + 1) + require.Equal(t, len(kvs), len(ol.plist.Parts)+1) key := x.DataKey("bal", 1331) require.Equal(t, key, kvs[0].Key) - for i, part := range ol.parts { - partKey := getNextPartKey(key, part.StartUid) + for i, startUid := range ol.plist.Parts { + partKey := getNextPartKey(key, startUid) require.Equal(t, partKey, kvs[i+1].Key) + part, err := ol.readListPart(ol.key, startUid) + require.NoError(t, err) data, err := part.Marshal() require.NoError(t, err) require.Equal(t, data, kvs[i+1].Value) require.Equal(t, []byte{BitCompletePosting}, kvs[i+1].UserMeta) require.Equal(t, ol.minTs, kvs[i+1].Version) } + + // Marshalling causes the mutation map to be cleaned up. Verify all changes + // can still be seen even though the posting lists have not been committed to + // disk yet. + opt := ListOptions{ReadTs: uint64(N) + 1} + l, err := ol.Uids(opt) + require.NoError(t, err) + require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) + for i, uid := range l.Uids { + require.Equal(t, uint64(i+1)*2, uid) + } + } diff --git a/posting/mvcc.go b/posting/mvcc.go index 30d78f25656..d4538647ada 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -185,6 +185,7 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { l := new(List) l.key = key l.mutationMap = make(map[uint64]*pb.PostingList) + l.newParts = make(map[uint64]*pb.PostingList) l.plist = new(pb.PostingList) // Iterates from highest Ts to lowest Ts @@ -208,9 +209,6 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { return nil, err } l.minTs = item.Version() - if err := l.readListParts(key, item.Version()); err != nil { - return nil, err - } // No need to do Next here. The outer loop can take care of skipping more versions of // the same key. return l, nil diff --git a/protos/pb.proto b/protos/pb.proto index b6a3892ec0c..789774bdc5f 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -279,8 +279,9 @@ message PostingList { uint64 commit_ts = 3; // More inclination towards smaller values. bool multi_part = 4; - uint64 start_uid = 5; - uint64 end_uid = 6; + repeated uint64 parts = 5; + uint64 start_uid = 6; + uint64 end_uid = 7; } message FacetParam { diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index c2595995d8c..33085ad7777 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -2101,8 +2101,9 @@ type PostingList struct { Postings []*Posting `protobuf:"bytes,2,rep,name=postings,proto3" json:"postings,omitempty"` CommitTs uint64 `protobuf:"varint,3,opt,name=commit_ts,json=commitTs,proto3" json:"commit_ts,omitempty"` MultiPart bool `protobuf:"varint,4,opt,name=multi_part,json=multiPart,proto3" json:"multi_part,omitempty"` - StartUid uint64 `protobuf:"varint,5,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` - EndUid uint64 `protobuf:"varint,6,opt,name=end_uid,json=endUid,proto3" json:"end_uid,omitempty"` + Parts []uint64 `protobuf:"varint,5,rep,packed,name=parts,proto3" json:"parts,omitempty"` + StartUid uint64 `protobuf:"varint,6,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` + EndUid uint64 `protobuf:"varint,7,opt,name=end_uid,json=endUid,proto3" json:"end_uid,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2169,6 +2170,13 @@ func (m *PostingList) GetMultiPart() bool { return false } +func (m *PostingList) GetParts() []uint64 { + if m != nil { + return m.Parts + } + return nil +} + func (m *PostingList) GetStartUid() uint64 { if m != nil { return m.StartUid @@ -3645,217 +3653,218 @@ func init() { func init() { proto.RegisterFile("pb.proto", fileDescriptor_f80abaa17e25ccc8) } var fileDescriptor_f80abaa17e25ccc8 = []byte{ - // 3355 bytes of a gzipped FileDescriptorProto + // 3363 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4b, 0x73, 0x23, 0x47, - 0x72, 0x66, 0x37, 0x80, 0x46, 0x77, 0x02, 0xe4, 0x40, 0xa5, 0xd9, 0x11, 0x44, 0xed, 0x72, 0xa8, - 0xd6, 0x63, 0xa8, 0x91, 0xc4, 0x19, 0x51, 0xb2, 0xbd, 0x5a, 0x87, 0x0f, 0x7c, 0x60, 0xc6, 0x94, - 0xf8, 0x72, 0x01, 0x9c, 0xb5, 0x15, 0x0e, 0x23, 0x0a, 0xdd, 0x45, 0xb0, 0x97, 0x8d, 0xee, 0x76, - 0x57, 0x37, 0x03, 0x9c, 0x9b, 0x0f, 0xf6, 0x6f, 0xd8, 0x83, 0xc3, 0x37, 0x5f, 0x7c, 0xf1, 0x75, - 0x8f, 0xbe, 0x38, 0xc2, 0xbe, 0x39, 0xfc, 0x0b, 0x1c, 0x63, 0xff, 0x03, 0x47, 0x38, 0xc2, 0x37, - 0x47, 0x65, 0x55, 0x3f, 0x80, 0x21, 0x67, 0x56, 0x8e, 0xd8, 0x13, 0x3a, 0x1f, 0xf5, 0xfa, 0x2a, - 0x33, 0x2b, 0x33, 0x01, 0x76, 0x32, 0xd9, 0x4e, 0xd2, 0x38, 0x8b, 0x89, 0x99, 0x4c, 0xd6, 0x1d, - 0x96, 0x04, 0x8a, 0x5c, 0x7f, 0x34, 0x0d, 0xb2, 0xcb, 0x7c, 0xb2, 0xed, 0xc5, 0xb3, 0x27, 0xfe, - 0x34, 0x65, 0xc9, 0xe5, 0x97, 0x41, 0xfc, 0x64, 0xc2, 0xfc, 0x29, 0x4f, 0x9f, 0x24, 0x93, 0x27, - 0xc5, 0x38, 0x77, 0x1d, 0x9a, 0x47, 0x81, 0xc8, 0x08, 0x81, 0x66, 0x1e, 0xf8, 0xa2, 0x6f, 0x6c, - 0x36, 0xb6, 0x2c, 0x8a, 0xdf, 0xee, 0x31, 0x38, 0x23, 0x26, 0xae, 0x5e, 0xb0, 0x30, 0xe7, 0xa4, - 0x07, 0x8d, 0x6b, 0x16, 0xf6, 0x8d, 0x4d, 0x63, 0xab, 0x4b, 0xe5, 0x27, 0xd9, 0x06, 0xfb, 0x9a, - 0x85, 0xe3, 0xec, 0x26, 0xe1, 0x7d, 0x73, 0xd3, 0xd8, 0x5a, 0xdb, 0x79, 0x77, 0x3b, 0x99, 0x6c, - 0x9f, 0xc5, 0x22, 0x0b, 0xa2, 0xe9, 0xf6, 0x0b, 0x16, 0x8e, 0x6e, 0x12, 0x4e, 0xdb, 0xd7, 0xea, - 0xc3, 0x3d, 0x85, 0xce, 0x30, 0xf5, 0x9e, 0xe5, 0x91, 0x97, 0x05, 0x71, 0x24, 0x57, 0x8c, 0xd8, - 0x8c, 0xe3, 0x8c, 0x0e, 0xc5, 0x6f, 0xc9, 0x63, 0xe9, 0x54, 0xf4, 0x1b, 0x9b, 0x0d, 0xc9, 0x93, - 0xdf, 0xa4, 0x0f, 0xed, 0x40, 0xec, 0xc7, 0x79, 0x94, 0xf5, 0x9b, 0x9b, 0xc6, 0x96, 0x4d, 0x0b, - 0xd2, 0xfd, 0x6f, 0x13, 0x5a, 0x7f, 0x92, 0xf3, 0xf4, 0x06, 0xc7, 0x65, 0x59, 0x5a, 0xcc, 0x25, - 0xbf, 0xc9, 0x7d, 0x68, 0x85, 0x2c, 0x9a, 0x8a, 0xbe, 0x89, 0x93, 0x29, 0x82, 0x7c, 0x00, 0x0e, - 0xbb, 0xc8, 0x78, 0x3a, 0xce, 0x03, 0xbf, 0xdf, 0xd8, 0x34, 0xb6, 0x2c, 0x6a, 0x23, 0xe3, 0x3c, - 0xf0, 0xc9, 0xfb, 0x60, 0xfb, 0xf1, 0xd8, 0xab, 0xaf, 0xe5, 0xc7, 0xb8, 0x16, 0xf9, 0x08, 0xec, - 0x3c, 0xf0, 0xc7, 0x61, 0x20, 0xb2, 0x7e, 0x6b, 0xd3, 0xd8, 0xea, 0xec, 0xd8, 0xf2, 0xb0, 0x12, - 0x3b, 0xda, 0xce, 0x03, 0x1f, 0x41, 0x7c, 0x0c, 0xb6, 0x48, 0xbd, 0xf1, 0x45, 0x1e, 0x79, 0x7d, - 0x0b, 0x95, 0xee, 0x49, 0xa5, 0xda, 0xa9, 0x69, 0x5b, 0x28, 0x42, 0x1e, 0x2b, 0xe5, 0xd7, 0x3c, - 0x15, 0xbc, 0xdf, 0x56, 0x4b, 0x69, 0x92, 0x3c, 0x85, 0xce, 0x05, 0xf3, 0x78, 0x36, 0x4e, 0x58, - 0xca, 0x66, 0x7d, 0xbb, 0x9a, 0xe8, 0x99, 0x64, 0x9f, 0x49, 0xae, 0xa0, 0x70, 0x51, 0x12, 0xe4, - 0x6b, 0x58, 0x45, 0x4a, 0x8c, 0x2f, 0x82, 0x30, 0xe3, 0x69, 0xdf, 0xc1, 0x31, 0x6b, 0x38, 0x06, - 0x39, 0xa3, 0x94, 0x73, 0xda, 0x55, 0x4a, 0x8a, 0x43, 0x7e, 0x06, 0xc0, 0xe7, 0x09, 0x8b, 0xfc, - 0x31, 0x0b, 0xc3, 0x3e, 0xe0, 0x1e, 0x1c, 0xc5, 0xd9, 0x0d, 0x43, 0xf2, 0x9e, 0xdc, 0x1f, 0xf3, - 0xc7, 0x99, 0xe8, 0xaf, 0x6e, 0x1a, 0x5b, 0x4d, 0x6a, 0x49, 0x72, 0x24, 0xdc, 0x1d, 0x70, 0xd0, - 0x22, 0xf0, 0xc4, 0x9f, 0x80, 0x75, 0x2d, 0x09, 0x65, 0x38, 0x9d, 0x9d, 0x55, 0xb9, 0x64, 0x69, - 0x34, 0x54, 0x0b, 0xdd, 0x0d, 0xb0, 0x8f, 0x58, 0x34, 0x2d, 0x2c, 0x4d, 0x5e, 0x05, 0x0e, 0x70, - 0x28, 0x7e, 0xbb, 0xbf, 0x36, 0xc1, 0xa2, 0x5c, 0xe4, 0x61, 0x46, 0x1e, 0x01, 0x48, 0xa0, 0x67, - 0x2c, 0x4b, 0x83, 0xb9, 0x9e, 0xb5, 0x82, 0xda, 0xc9, 0x03, 0xff, 0x18, 0x45, 0xe4, 0x29, 0x74, - 0x71, 0xf6, 0x42, 0xd5, 0xac, 0x36, 0x50, 0xee, 0x8f, 0x76, 0x50, 0x45, 0x8f, 0x78, 0x00, 0x16, - 0xde, 0xad, 0xb2, 0xaf, 0x55, 0xaa, 0x29, 0xf2, 0x09, 0xac, 0x05, 0x51, 0x26, 0xb1, 0xf7, 0xb2, - 0xb1, 0xcf, 0x45, 0x71, 0xf9, 0xab, 0x25, 0xf7, 0x80, 0x8b, 0x8c, 0x7c, 0x05, 0x0a, 0xc0, 0x62, - 0xc1, 0x16, 0x2e, 0xb8, 0x56, 0x5e, 0x8c, 0x50, 0x2b, 0xa2, 0x8e, 0x5e, 0xf1, 0x4b, 0xe8, 0xc8, - 0xf3, 0x15, 0x23, 0x2c, 0x1c, 0xd1, 0xc5, 0xd3, 0x68, 0x38, 0x28, 0x48, 0x05, 0xad, 0x2e, 0xa1, - 0x91, 0x06, 0xa6, 0x0c, 0x02, 0xbf, 0xdd, 0x01, 0xb4, 0x4e, 0x53, 0x9f, 0xa7, 0xb7, 0xda, 0x38, - 0x81, 0xa6, 0xcf, 0x85, 0x87, 0xee, 0x67, 0x53, 0xfc, 0xae, 0xec, 0xbe, 0x51, 0xb3, 0x7b, 0xf7, - 0xef, 0x0c, 0xe8, 0x0c, 0xe3, 0x34, 0x3b, 0xe6, 0x42, 0xb0, 0x29, 0x27, 0x0f, 0xa1, 0x15, 0xcb, - 0x69, 0x35, 0xc2, 0x8e, 0xdc, 0x13, 0xae, 0x43, 0x15, 0x7f, 0xe9, 0x1e, 0xcc, 0xbb, 0xef, 0xe1, - 0x3e, 0xb4, 0x94, 0xc7, 0x48, 0x6f, 0x6a, 0x51, 0x45, 0x48, 0xac, 0xe3, 0x8b, 0x0b, 0xc1, 0x15, - 0x96, 0x2d, 0xaa, 0xa9, 0xbb, 0xcd, 0xea, 0xf7, 0x00, 0xe4, 0xfe, 0x7e, 0xa4, 0x15, 0xb8, 0x97, - 0xd0, 0xa1, 0xec, 0x22, 0xdb, 0x8f, 0xa3, 0x8c, 0xcf, 0x33, 0xb2, 0x06, 0x66, 0xe0, 0x23, 0x44, - 0x16, 0x35, 0x03, 0x5f, 0x6e, 0x6e, 0x9a, 0xc6, 0x79, 0x82, 0x08, 0xad, 0x52, 0x45, 0x20, 0x94, - 0xbe, 0x9f, 0xe2, 0x8e, 0x25, 0x94, 0xbe, 0x9f, 0x92, 0x87, 0xd0, 0x11, 0x11, 0x4b, 0xc4, 0x65, - 0x9c, 0xc9, 0xcd, 0x35, 0x71, 0x73, 0x50, 0xb0, 0x46, 0xc2, 0xfd, 0x67, 0x03, 0xac, 0x63, 0x3e, - 0x9b, 0xf0, 0xf4, 0xb5, 0x55, 0xde, 0x07, 0x1b, 0x27, 0x1e, 0x07, 0xbe, 0x5e, 0xa8, 0x8d, 0xf4, - 0xa1, 0x7f, 0xeb, 0x52, 0x0f, 0xc0, 0x0a, 0x39, 0x93, 0xe0, 0x2b, 0x3b, 0xd3, 0x94, 0xc4, 0x86, - 0xcd, 0xc6, 0x3e, 0x67, 0x3e, 0x86, 0x18, 0x9b, 0x5a, 0x6c, 0x76, 0xc0, 0x99, 0x2f, 0xf7, 0x16, - 0x32, 0x91, 0x8d, 0xf3, 0xc4, 0x67, 0x19, 0xc7, 0xd0, 0xd2, 0x94, 0x86, 0x23, 0xb2, 0x73, 0xe4, - 0x90, 0xc7, 0xf0, 0x8e, 0x17, 0xe6, 0x42, 0xc6, 0xb5, 0x20, 0xba, 0x88, 0xc7, 0x71, 0x14, 0xde, - 0x20, 0xbe, 0x36, 0xbd, 0xa7, 0x05, 0x87, 0xd1, 0x45, 0x7c, 0x1a, 0x85, 0x37, 0xee, 0x6f, 0x4c, - 0x68, 0x3d, 0x47, 0x18, 0x9e, 0x42, 0x7b, 0x86, 0x07, 0x2a, 0xbc, 0xf7, 0x81, 0x44, 0x18, 0x65, - 0xdb, 0xea, 0xa4, 0x62, 0x10, 0x65, 0xe9, 0x0d, 0x2d, 0xd4, 0xe4, 0x88, 0x8c, 0x4d, 0x42, 0x9e, - 0x09, 0x6d, 0x11, 0xb5, 0x11, 0x23, 0x25, 0xd0, 0x23, 0xb4, 0xda, 0x32, 0xac, 0x8d, 0x65, 0x58, - 0xc9, 0x3a, 0xd8, 0xde, 0x25, 0xf7, 0xae, 0x44, 0x3e, 0xd3, 0xa0, 0x97, 0xf4, 0xfa, 0x33, 0xe8, - 0xd6, 0xf7, 0x21, 0xdf, 0xa0, 0x2b, 0x7e, 0x83, 0xc0, 0x37, 0xa9, 0xfc, 0x24, 0x9b, 0xd0, 0x42, - 0x0f, 0x47, 0xd8, 0x3b, 0x3b, 0x20, 0xb7, 0xa3, 0x86, 0x50, 0x25, 0xf8, 0x85, 0xf9, 0x73, 0x43, - 0xce, 0x53, 0xdf, 0x5d, 0x7d, 0x1e, 0xe7, 0xee, 0x79, 0xd4, 0x90, 0xda, 0x3c, 0xee, 0xff, 0x9a, - 0xd0, 0xfd, 0x81, 0xa7, 0xf1, 0x59, 0x1a, 0x27, 0xb1, 0x60, 0x21, 0xd9, 0x5d, 0x3c, 0x9d, 0x42, - 0x71, 0x53, 0x0e, 0xae, 0xab, 0x6d, 0x0f, 0xcb, 0xe3, 0x2a, 0x74, 0xea, 0xe7, 0x77, 0xc1, 0x52, - 0xe8, 0xde, 0x72, 0x04, 0x2d, 0x91, 0x3a, 0x0a, 0x4f, 0xc4, 0x6f, 0x71, 0x7b, 0x5a, 0x42, 0x36, - 0x00, 0x66, 0x6c, 0x7e, 0xc4, 0x99, 0xe0, 0x87, 0x7e, 0x61, 0xbe, 0x15, 0x47, 0xe2, 0x3c, 0x63, - 0xf3, 0xd1, 0x3c, 0x1a, 0x09, 0xb4, 0xae, 0x26, 0x2d, 0x69, 0xf2, 0x53, 0x70, 0x66, 0x6c, 0x2e, - 0xfd, 0xe8, 0xd0, 0xd7, 0xd6, 0x55, 0x31, 0xc8, 0x87, 0xd0, 0xc8, 0xe6, 0x11, 0x06, 0x25, 0xf9, - 0x0e, 0xc9, 0x24, 0x63, 0x34, 0x8f, 0xb4, 0xc7, 0x51, 0x29, 0x2b, 0x00, 0xb5, 0x2b, 0x40, 0x7b, - 0xd0, 0xf0, 0x02, 0x1f, 0x1f, 0x22, 0x87, 0xca, 0xcf, 0xf5, 0x3f, 0x82, 0x7b, 0x4b, 0x38, 0xd4, - 0xef, 0x61, 0x55, 0x0d, 0xbb, 0x5f, 0xbf, 0x87, 0x66, 0x1d, 0xfb, 0xdf, 0x34, 0xe0, 0x9e, 0x36, - 0x86, 0xcb, 0x20, 0x19, 0x66, 0xd2, 0xec, 0xfb, 0xd0, 0xc6, 0x68, 0xc3, 0x53, 0x6d, 0x13, 0x05, - 0x49, 0xfe, 0x00, 0x2c, 0xf4, 0xc0, 0xc2, 0x4e, 0x1f, 0x56, 0xa8, 0x96, 0xc3, 0x95, 0xdd, 0xea, - 0x2b, 0xd1, 0xea, 0xe4, 0x1b, 0x68, 0xbd, 0xe4, 0x69, 0xac, 0xa2, 0x67, 0x67, 0x67, 0xe3, 0xb6, - 0x71, 0xf2, 0x6e, 0xf5, 0x30, 0xa5, 0xfc, 0x3b, 0x04, 0xff, 0x63, 0x19, 0x2f, 0x67, 0xf1, 0x35, - 0xf7, 0xfb, 0x6d, 0xdc, 0x51, 0xdd, 0x3e, 0x0a, 0x51, 0x81, 0xb6, 0x5d, 0xa1, 0x7d, 0x00, 0x9d, - 0xda, 0xf1, 0x6e, 0x41, 0xfa, 0xe1, 0xa2, 0xc5, 0x3b, 0xa5, 0x23, 0xd7, 0x1d, 0xe7, 0x00, 0xa0, - 0x3a, 0xec, 0xff, 0xd7, 0xfd, 0xdc, 0xbf, 0x32, 0xe0, 0xde, 0x7e, 0x1c, 0x45, 0x1c, 0x53, 0x20, - 0x75, 0x75, 0x95, 0xd9, 0x1b, 0x77, 0x9a, 0xfd, 0x67, 0xd0, 0x12, 0x52, 0x59, 0xcf, 0xfe, 0xee, - 0x2d, 0x77, 0x41, 0x95, 0x86, 0x0c, 0x33, 0x33, 0x36, 0x1f, 0x27, 0x3c, 0xf2, 0x83, 0x68, 0x5a, - 0x84, 0x99, 0x19, 0x9b, 0x9f, 0x29, 0x8e, 0xfb, 0xd7, 0x06, 0x58, 0xca, 0x63, 0x16, 0xa2, 0xb5, - 0xb1, 0x18, 0xad, 0x7f, 0x0a, 0x4e, 0x92, 0x72, 0x3f, 0xf0, 0x8a, 0x55, 0x1d, 0x5a, 0x31, 0xa4, - 0x71, 0x5e, 0xc4, 0xa9, 0xc7, 0x71, 0x7a, 0x9b, 0x2a, 0x42, 0x72, 0x45, 0xc2, 0x3c, 0x95, 0xc6, - 0x35, 0xa8, 0x22, 0x64, 0x8c, 0x57, 0x97, 0x83, 0x97, 0x62, 0x53, 0x4d, 0xb9, 0xff, 0x60, 0x42, - 0xf7, 0x20, 0x48, 0xb9, 0x97, 0x71, 0x7f, 0xe0, 0x4f, 0x51, 0x91, 0x47, 0x59, 0x90, 0xdd, 0xe8, - 0xf7, 0x44, 0x53, 0xe5, 0x73, 0x6f, 0x2e, 0xa6, 0xb4, 0x0a, 0xee, 0x06, 0x66, 0xe1, 0x8a, 0x20, - 0x3b, 0x00, 0x2a, 0x11, 0xc2, 0x4c, 0xbc, 0x79, 0x77, 0x26, 0xee, 0xa0, 0x9a, 0xfc, 0x94, 0x18, - 0xa8, 0x31, 0x81, 0x7a, 0x6b, 0x2c, 0x4c, 0xd3, 0x73, 0x69, 0xab, 0x98, 0x3f, 0x4c, 0x78, 0x88, - 0xb6, 0x88, 0xf9, 0xc3, 0x84, 0x87, 0x65, 0xd6, 0xd6, 0x56, 0xdb, 0x91, 0xdf, 0xe4, 0x23, 0x30, - 0xe3, 0x04, 0xcf, 0xa7, 0x17, 0xac, 0x1f, 0x6c, 0xfb, 0x34, 0xa1, 0x66, 0x9c, 0xc8, 0x8b, 0x56, - 0x69, 0x67, 0xdf, 0xd1, 0xf6, 0x2b, 0x03, 0x08, 0x26, 0x4c, 0x54, 0x4b, 0xdc, 0x07, 0x60, 0x9e, - 0x26, 0xa4, 0x0d, 0x8d, 0xe1, 0x60, 0xd4, 0x5b, 0x91, 0x1f, 0x07, 0x83, 0xa3, 0x9e, 0xe1, 0xbe, - 0x32, 0xc0, 0x39, 0xce, 0x33, 0x26, 0xcd, 0x46, 0xbc, 0xe9, 0xde, 0xde, 0x07, 0x5b, 0x64, 0x2c, - 0xc5, 0x20, 0xac, 0x22, 0x47, 0x1b, 0xe9, 0x91, 0x20, 0x9f, 0x42, 0x8b, 0xfb, 0x53, 0x5e, 0x38, - 0x74, 0x6f, 0x79, 0x9f, 0x54, 0x89, 0xc9, 0x16, 0x58, 0xc2, 0xbb, 0xe4, 0x33, 0xd6, 0x6f, 0x56, - 0x8a, 0x43, 0xe4, 0xa8, 0x47, 0x96, 0x6a, 0x39, 0x56, 0x09, 0x69, 0x9c, 0x60, 0xda, 0xdc, 0xd2, - 0x55, 0x42, 0x1a, 0x27, 0x32, 0x69, 0xde, 0x81, 0x9f, 0x04, 0xd3, 0x28, 0x4e, 0xf9, 0x38, 0x88, - 0x7c, 0x3e, 0x1f, 0x7b, 0x71, 0x74, 0x11, 0x06, 0x5e, 0x86, 0x58, 0xda, 0xf4, 0x5d, 0x25, 0x3c, - 0x94, 0xb2, 0x7d, 0x2d, 0x72, 0xe7, 0x60, 0x17, 0x71, 0x91, 0x7c, 0x26, 0x03, 0x1a, 0xc6, 0x55, - 0xed, 0x16, 0x98, 0xf6, 0xd7, 0x12, 0x1c, 0x5a, 0xc8, 0xe5, 0x35, 0xe1, 0x1a, 0x45, 0xa4, 0x44, - 0xa2, 0x9e, 0x5e, 0x35, 0xea, 0xe9, 0x15, 0x66, 0x8a, 0x71, 0xc4, 0x75, 0xc6, 0x81, 0xdf, 0xee, - 0xdf, 0x9a, 0x60, 0x97, 0x4f, 0xd9, 0xe7, 0xe0, 0xcc, 0x0a, 0xa8, 0xb5, 0xc3, 0x61, 0x2e, 0x5d, - 0xe2, 0x4f, 0x2b, 0x39, 0x79, 0x00, 0xe6, 0xd5, 0xb5, 0x06, 0xca, 0x92, 0x5a, 0xdf, 0xbf, 0xa0, - 0xe6, 0xd5, 0x75, 0xe5, 0xb1, 0xad, 0xb7, 0x7a, 0xec, 0x23, 0xb8, 0xe7, 0x85, 0x9c, 0x45, 0xe3, - 0xca, 0xe1, 0x94, 0xc1, 0xad, 0x21, 0xfb, 0xac, 0xf4, 0x3a, 0x1d, 0x75, 0xda, 0xd5, 0xdb, 0xf2, - 0x09, 0xb4, 0x7c, 0x1e, 0x66, 0xac, 0x5e, 0x1a, 0x9d, 0xa6, 0xcc, 0x0b, 0xf9, 0x81, 0x64, 0x53, - 0x25, 0x25, 0x5b, 0x60, 0x17, 0xef, 0xac, 0x2e, 0x88, 0x30, 0xf3, 0x2e, 0xc0, 0xa6, 0xa5, 0xb4, - 0xc2, 0x12, 0x6a, 0x58, 0xba, 0x5f, 0x41, 0xe3, 0xfb, 0x17, 0x43, 0x7d, 0x56, 0xe3, 0xb5, 0xb3, - 0x16, 0x88, 0x9a, 0x35, 0x44, 0xff, 0xa7, 0x01, 0x6d, 0xed, 0x75, 0x72, 0xdf, 0x79, 0x99, 0x25, - 0xca, 0xcf, 0xc5, 0xc7, 0xad, 0x74, 0xdf, 0x7a, 0x19, 0xdd, 0x78, 0x7b, 0x19, 0x4d, 0x7e, 0x01, - 0xdd, 0x44, 0xc9, 0xea, 0x0e, 0xff, 0x5e, 0x7d, 0x8c, 0xfe, 0xc5, 0x71, 0x9d, 0xa4, 0x22, 0xa4, - 0xe9, 0x62, 0x3d, 0x92, 0xb1, 0x29, 0x5e, 0x51, 0x97, 0xb6, 0x25, 0x3d, 0x62, 0xd3, 0x3b, 0xdc, - 0xfe, 0xb7, 0xf0, 0x5e, 0x99, 0x0d, 0xc7, 0x49, 0xbf, 0x8b, 0x1e, 0x29, 0x3d, 0xbe, 0xee, 0x8c, - 0xab, 0x8b, 0xce, 0xf8, 0x01, 0x38, 0x5e, 0x3c, 0x9b, 0x05, 0x28, 0x5b, 0xd3, 0xd9, 0x1e, 0x32, - 0x46, 0xc2, 0xfd, 0x1b, 0x03, 0xda, 0xfa, 0xb4, 0xa4, 0x03, 0xed, 0x83, 0xc1, 0xb3, 0xdd, 0xf3, - 0x23, 0x19, 0x0f, 0x00, 0xac, 0xbd, 0xc3, 0x93, 0x5d, 0xfa, 0x67, 0x3d, 0x43, 0xc6, 0x86, 0xc3, - 0x93, 0x51, 0xcf, 0x24, 0x0e, 0xb4, 0x9e, 0x1d, 0x9d, 0xee, 0x8e, 0x7a, 0x0d, 0x62, 0x43, 0x73, - 0xef, 0xf4, 0xf4, 0xa8, 0xd7, 0x24, 0x5d, 0xb0, 0x0f, 0x76, 0x47, 0x83, 0xd1, 0xe1, 0xf1, 0xa0, - 0xd7, 0x92, 0xba, 0xcf, 0x07, 0xa7, 0x3d, 0x4b, 0x7e, 0x9c, 0x1f, 0x1e, 0xf4, 0xda, 0x52, 0x7e, - 0xb6, 0x3b, 0x1c, 0xfe, 0xf2, 0x94, 0x1e, 0xf4, 0x6c, 0x39, 0xef, 0x70, 0x44, 0x0f, 0x4f, 0x9e, - 0xf7, 0x1c, 0xf9, 0x7d, 0xba, 0xf7, 0xdd, 0x60, 0x7f, 0xd4, 0x03, 0xf7, 0x2b, 0xe8, 0xd4, 0x10, - 0x94, 0xa3, 0xe9, 0xe0, 0x59, 0x6f, 0x45, 0x2e, 0xf9, 0x62, 0xf7, 0xe8, 0x7c, 0xd0, 0x33, 0xc8, - 0x1a, 0x00, 0x7e, 0x8e, 0x8f, 0x76, 0x4f, 0x9e, 0xf7, 0x4c, 0xf7, 0xf7, 0xc1, 0x3e, 0x0f, 0xfc, - 0xbd, 0x30, 0xf6, 0xae, 0xa4, 0x61, 0x4c, 0x98, 0xe0, 0xfa, 0x9d, 0xc4, 0x6f, 0x19, 0xe5, 0xd1, - 0x28, 0x85, 0xbe, 0x7b, 0x4d, 0xb9, 0x27, 0xd0, 0x3e, 0x0f, 0xfc, 0x33, 0xe6, 0x5d, 0xc9, 0x7a, - 0x7c, 0x22, 0xc7, 0x8f, 0x45, 0xf0, 0x92, 0xeb, 0x00, 0xe7, 0x20, 0x67, 0x18, 0xbc, 0xe4, 0xe4, - 0x63, 0xb0, 0x90, 0x28, 0x32, 0x1a, 0xb4, 0xe5, 0x62, 0x4d, 0xaa, 0x65, 0xee, 0xbf, 0x1a, 0xe5, - 0xde, 0xb1, 0xd8, 0x7e, 0x08, 0xcd, 0x84, 0x79, 0x57, 0x3a, 0x9a, 0x74, 0xf4, 0x18, 0xb9, 0x1e, - 0x45, 0x01, 0x79, 0x04, 0xb6, 0x36, 0x90, 0x62, 0xe2, 0x4e, 0xcd, 0x92, 0x68, 0x29, 0x5c, 0xbc, - 0xba, 0xc6, 0xe2, 0xd5, 0xc9, 0xbd, 0xcf, 0xf2, 0x30, 0x0b, 0xc6, 0x09, 0x4b, 0x8b, 0xea, 0xd9, - 0x41, 0xce, 0x19, 0x4b, 0x33, 0x39, 0x56, 0x59, 0x44, 0xae, 0x9f, 0x9b, 0x26, 0x55, 0x26, 0x72, - 0x1e, 0xf8, 0x32, 0x64, 0xf1, 0xc8, 0x47, 0x91, 0xca, 0x7e, 0x2c, 0x1e, 0xf9, 0xe7, 0x81, 0xef, - 0x7e, 0x03, 0x50, 0x35, 0x3c, 0x6e, 0xc9, 0xd9, 0xef, 0x43, 0x8b, 0x85, 0x81, 0x86, 0xd4, 0xa1, - 0x8a, 0x70, 0x4f, 0xa0, 0x53, 0x6b, 0x93, 0x48, 0x63, 0x64, 0x61, 0x38, 0xbe, 0xe2, 0x37, 0x02, - 0xc7, 0xda, 0xb4, 0xcd, 0xc2, 0xf0, 0x7b, 0x7e, 0x23, 0xc8, 0xc7, 0xd0, 0x52, 0x1d, 0x16, 0x73, - 0xa9, 0x90, 0xc7, 0xa1, 0x54, 0x09, 0xdd, 0x2f, 0xc0, 0x52, 0xd5, 0x7d, 0xcd, 0x17, 0x8c, 0x3b, - 0x5f, 0xb2, 0x6f, 0xf5, 0x9e, 0xb1, 0x17, 0x40, 0x3e, 0xd7, 0x9d, 0x1c, 0xa1, 0xfa, 0x46, 0x46, - 0x95, 0xc0, 0x29, 0x25, 0xdd, 0xc4, 0x41, 0x65, 0xf7, 0x00, 0xec, 0x37, 0xf6, 0xc6, 0x34, 0x00, - 0x66, 0x05, 0xc0, 0x2d, 0xdd, 0x32, 0xf7, 0x57, 0x00, 0x55, 0xc7, 0x47, 0xbb, 0xa6, 0x9a, 0x45, - 0xba, 0xe6, 0x63, 0x59, 0x6c, 0x05, 0xa1, 0x9f, 0xf2, 0x68, 0xe1, 0xd4, 0x55, 0x8f, 0xa8, 0x94, - 0x93, 0x4d, 0x68, 0x62, 0x23, 0xab, 0x51, 0x85, 0xce, 0xb2, 0x8b, 0x85, 0x12, 0x77, 0x02, 0xab, - 0xea, 0x81, 0xa4, 0xfc, 0x2f, 0x73, 0x2e, 0xde, 0x98, 0x59, 0x6d, 0x00, 0x94, 0x81, 0xbe, 0x68, - 0xc9, 0xd5, 0x38, 0xd2, 0x41, 0x2e, 0x02, 0x1e, 0xfa, 0xc5, 0x69, 0x34, 0xe5, 0xfe, 0x21, 0x74, - 0x8b, 0x35, 0xb0, 0x31, 0xf0, 0x79, 0xf9, 0x4c, 0x2b, 0x34, 0x55, 0x3d, 0xa2, 0x54, 0x4e, 0x62, - 0x9f, 0xef, 0x99, 0x7d, 0xa3, 0x78, 0xa9, 0xdd, 0x7f, 0x6f, 0x14, 0xa3, 0x75, 0x9d, 0xbc, 0x90, - 0xdf, 0x19, 0xcb, 0xf9, 0xdd, 0x62, 0x22, 0x65, 0xfe, 0x56, 0x89, 0xd4, 0xcf, 0xc1, 0xf1, 0x31, - 0x9b, 0x08, 0xae, 0x8b, 0xf0, 0xbd, 0xbe, 0x9c, 0x39, 0xe8, 0x7c, 0x23, 0xb8, 0xe6, 0xb4, 0x52, - 0x96, 0x7b, 0xc9, 0xe2, 0x2b, 0x1e, 0x05, 0x2f, 0xb1, 0x11, 0x20, 0x0f, 0x5d, 0x31, 0xaa, 0xae, - 0x8a, 0xca, 0x30, 0x74, 0x57, 0xa5, 0x68, 0x10, 0x59, 0x55, 0x83, 0x48, 0x22, 0x97, 0x27, 0x82, - 0xa7, 0x59, 0x91, 0x69, 0x2a, 0xaa, 0xcc, 0xd8, 0x1c, 0xad, 0x2b, 0x33, 0xb6, 0x0f, 0xa1, 0x1b, - 0xc5, 0xd1, 0x38, 0xca, 0xc3, 0x50, 0xe6, 0xc2, 0xba, 0xeb, 0xd7, 0x89, 0xe2, 0xe8, 0x44, 0xb3, - 0xc8, 0x63, 0x78, 0xa7, 0xae, 0xa2, 0x2c, 0xb7, 0xa3, 0x5a, 0x09, 0x35, 0x3d, 0xb4, 0xef, 0x2d, - 0xe8, 0xc5, 0x93, 0x5f, 0x71, 0x2f, 0x43, 0xc4, 0xc6, 0x68, 0xb2, 0x5d, 0xf5, 0x88, 0x2b, 0xbe, - 0x84, 0xe8, 0x84, 0xcd, 0xb8, 0xfb, 0x2d, 0x38, 0x25, 0x08, 0x32, 0x5e, 0x9f, 0x9c, 0x9e, 0x0c, - 0x54, 0x44, 0x3d, 0x3c, 0x39, 0x18, 0xfc, 0x69, 0xcf, 0x90, 0x11, 0x9f, 0x0e, 0x5e, 0x0c, 0xe8, - 0x70, 0xd0, 0x33, 0x65, 0x34, 0x3e, 0x18, 0x1c, 0x0d, 0x46, 0x83, 0x5e, 0xe3, 0xbb, 0xa6, 0xdd, - 0xee, 0xd9, 0xd4, 0xe6, 0xf3, 0x24, 0x0c, 0xbc, 0x20, 0x73, 0x87, 0x00, 0x72, 0x5a, 0x7d, 0xa3, - 0x1f, 0x80, 0x53, 0xad, 0xad, 0x6e, 0xd4, 0xce, 0xf4, 0xaa, 0x32, 0xa7, 0xd3, 0x46, 0x65, 0xde, - 0x95, 0xd3, 0x69, 0x33, 0x3b, 0x07, 0xfb, 0x98, 0x25, 0xaf, 0x95, 0x39, 0xdd, 0xb2, 0x98, 0xcd, - 0x75, 0x6b, 0x47, 0x3f, 0xe5, 0x9f, 0x40, 0x5b, 0x47, 0x46, 0xed, 0x1f, 0x0b, 0x51, 0xb3, 0x90, - 0xc9, 0xaa, 0xe3, 0xfe, 0x71, 0x7c, 0xcd, 0xcb, 0x6c, 0xe6, 0x8c, 0xdd, 0x84, 0x31, 0xf3, 0xdf, - 0x62, 0x88, 0x3f, 0x03, 0x10, 0x71, 0x9e, 0x7a, 0x7c, 0x3c, 0x2d, 0x3b, 0x4a, 0x8e, 0xe2, 0x3c, - 0xd7, 0x6d, 0x6a, 0x2e, 0x32, 0x14, 0x36, 0x94, 0x9b, 0x49, 0x5a, 0x8a, 0x7e, 0x02, 0x56, 0x36, - 0x8f, 0xaa, 0x06, 0x56, 0x2b, 0x93, 0x35, 0xa6, 0xbb, 0x0f, 0xce, 0x68, 0x8e, 0x95, 0x57, 0x2e, - 0x16, 0xde, 0x67, 0xe3, 0x0d, 0xef, 0xb3, 0xb9, 0xf4, 0x3e, 0xff, 0x97, 0x01, 0x9d, 0x5a, 0x9a, - 0x45, 0x3e, 0x84, 0x66, 0x36, 0x8f, 0x16, 0x3b, 0xbf, 0xc5, 0x22, 0x14, 0x45, 0xd2, 0xde, 0x64, - 0x59, 0xc6, 0x84, 0x08, 0xa6, 0x11, 0xf7, 0xf5, 0x94, 0xb2, 0x54, 0xdb, 0xd5, 0x2c, 0x72, 0x04, - 0xf7, 0x54, 0xcc, 0x28, 0xba, 0x3e, 0x45, 0xa6, 0xfe, 0xd1, 0x52, 0x5a, 0xa7, 0xaa, 0xd3, 0xfd, - 0x42, 0x4b, 0xd5, 0xdf, 0x6b, 0xd3, 0x05, 0xe6, 0xfa, 0x2e, 0xbc, 0x7b, 0x8b, 0xda, 0x8f, 0x6a, - 0x34, 0x3c, 0x84, 0x55, 0x59, 0x98, 0x07, 0x33, 0x2e, 0x32, 0x36, 0x4b, 0x30, 0xbf, 0xd1, 0x31, - 0xbf, 0x49, 0xcd, 0x4c, 0xb8, 0x9f, 0x42, 0xf7, 0x8c, 0xf3, 0x94, 0x72, 0x91, 0xc4, 0x91, 0x7a, - 0xdb, 0x05, 0x1e, 0x5a, 0x3f, 0x30, 0x9a, 0x72, 0xff, 0x02, 0x1c, 0x99, 0xb9, 0xef, 0xb1, 0xcc, - 0xbb, 0xfc, 0x31, 0x99, 0xfd, 0xa7, 0xd0, 0x4e, 0x94, 0x99, 0xe8, 0x3c, 0xbc, 0x8b, 0x31, 0x4e, - 0x9b, 0x0e, 0x2d, 0x84, 0xee, 0x37, 0xd0, 0x38, 0xc9, 0x67, 0xf5, 0x3f, 0x66, 0x9a, 0xea, 0x8f, - 0x99, 0x0f, 0xc0, 0xc1, 0x22, 0x00, 0xbb, 0x80, 0x2a, 0x3d, 0xb5, 0x25, 0x03, 0xdb, 0x7f, 0x3f, - 0x40, 0xa7, 0xc0, 0xfe, 0xd0, 0xc7, 0x7f, 0x57, 0xf0, 0xf2, 0x0f, 0xfd, 0x05, 0x5b, 0x50, 0x75, - 0x20, 0x8f, 0xfc, 0xc3, 0xe2, 0xd2, 0x14, 0xb1, 0x38, 0xb7, 0x7e, 0xca, 0xcb, 0xb9, 0x9f, 0x41, - 0xb7, 0xc8, 0xae, 0x8f, 0x79, 0xc6, 0xd0, 0x9c, 0xc2, 0x80, 0x47, 0x35, 0x53, 0xb3, 0x15, 0x63, - 0x24, 0xde, 0xd0, 0x34, 0x75, 0xb7, 0xc1, 0xd2, 0xb6, 0x4a, 0xa0, 0xe9, 0xc5, 0xbe, 0x72, 0x91, - 0x16, 0xc5, 0x6f, 0x79, 0xe0, 0x99, 0x98, 0x16, 0x0f, 0xe1, 0x4c, 0x4c, 0xdd, 0x39, 0xac, 0xee, - 0x31, 0xef, 0x2a, 0x4f, 0x8a, 0x87, 0xa8, 0x56, 0x06, 0x19, 0x0b, 0x65, 0xd0, 0x1b, 0x3a, 0xb5, - 0xef, 0x41, 0x3b, 0x8f, 0x82, 0x79, 0x91, 0xde, 0x38, 0xd4, 0x92, 0xa4, 0xea, 0x50, 0x86, 0xb1, - 0x87, 0x95, 0x0f, 0x7a, 0x95, 0x43, 0x4b, 0xda, 0xfd, 0x73, 0x58, 0x1d, 0xcc, 0x13, 0xec, 0x5b, - 0xbf, 0xf5, 0x09, 0xac, 0x6d, 0xca, 0x5c, 0xd8, 0xd4, 0xd2, 0xca, 0x8d, 0x62, 0xe5, 0x9d, 0x7f, - 0x32, 0xa0, 0x29, 0x4d, 0x44, 0x16, 0x67, 0x7f, 0xcc, 0x59, 0x9a, 0x4d, 0x38, 0xcb, 0xc8, 0x82, - 0x39, 0xac, 0x2f, 0x50, 0xee, 0xca, 0x53, 0x83, 0x7c, 0xa1, 0x5a, 0xe2, 0x45, 0xa7, 0x7f, 0xb5, - 0x30, 0x34, 0x34, 0xc4, 0x65, 0x7d, 0xb2, 0x0d, 0x9d, 0xef, 0xe2, 0x20, 0xda, 0x57, 0x5d, 0x62, - 0xb2, 0x6c, 0x96, 0xaf, 0xe9, 0x7f, 0x09, 0xd6, 0xa1, 0x90, 0xf6, 0xff, 0xba, 0x2a, 0x86, 0xd7, - 0xba, 0x6b, 0xb8, 0x2b, 0x3b, 0xff, 0xd8, 0x80, 0xe6, 0x0f, 0x3c, 0x8d, 0xc9, 0x17, 0xd0, 0xd6, - 0x3d, 0x20, 0x52, 0xeb, 0xf5, 0xac, 0xe3, 0xfb, 0xba, 0xd4, 0x1c, 0xc2, 0x55, 0x7a, 0x2a, 0x42, - 0x57, 0xd5, 0x23, 0xa9, 0x5a, 0x54, 0xaf, 0x6d, 0xea, 0x5b, 0xe8, 0x0d, 0xb3, 0x94, 0xb3, 0x59, - 0x4d, 0x7d, 0x11, 0xa6, 0xdb, 0x4a, 0x51, 0x44, 0xeb, 0x73, 0xb0, 0x54, 0x90, 0x59, 0x1a, 0xb0, - 0x5c, 0x55, 0xa2, 0xf2, 0x23, 0xe8, 0x0c, 0x2f, 0xe3, 0x3c, 0xf4, 0x87, 0x3c, 0xbd, 0xe6, 0xa4, - 0xd6, 0x87, 0x5d, 0xaf, 0x7d, 0xbb, 0x2b, 0x64, 0x0b, 0x40, 0x79, 0xd9, 0x79, 0xe0, 0x0b, 0xd2, - 0x96, 0xb2, 0x93, 0x7c, 0xa6, 0x26, 0xad, 0xb9, 0x9f, 0xd2, 0xac, 0xc5, 0x9a, 0x37, 0x69, 0x7e, - 0x0d, 0xab, 0xfb, 0x18, 0x8b, 0x4f, 0xd3, 0xdd, 0x49, 0x9c, 0x66, 0x64, 0xb9, 0x17, 0xbb, 0xbe, - 0xcc, 0x70, 0x57, 0xc8, 0x53, 0xb0, 0x47, 0xe9, 0x8d, 0xd2, 0x7f, 0x47, 0x87, 0xe8, 0x6a, 0xbd, - 0x5b, 0x4e, 0xb9, 0xf3, 0xf7, 0x0d, 0xb0, 0x7e, 0x19, 0xa7, 0x57, 0x3c, 0x25, 0x8f, 0xc1, 0xc2, - 0xf2, 0x5f, 0x1b, 0x51, 0xd9, 0x0a, 0xb8, 0x6d, 0xa1, 0x8f, 0xc1, 0x41, 0x50, 0x46, 0x4c, 0x5c, - 0xa9, 0xab, 0xc2, 0xbf, 0x66, 0x15, 0x2e, 0x2a, 0x79, 0xc3, 0x7b, 0x5d, 0x53, 0x17, 0x55, 0xb6, - 0x3c, 0x16, 0x6a, 0xf2, 0xf5, 0xb6, 0x2a, 0xb0, 0x87, 0xee, 0xca, 0x96, 0xf1, 0xd4, 0x20, 0x9f, - 0x41, 0x73, 0xa8, 0x4e, 0x2a, 0x95, 0xaa, 0xbf, 0xaf, 0xd6, 0xd7, 0x0a, 0x46, 0x39, 0xf3, 0x13, - 0xb0, 0xd4, 0xcb, 0xae, 0x8e, 0xb9, 0x90, 0x98, 0xae, 0xf7, 0xea, 0x2c, 0x3d, 0xe0, 0x33, 0xb0, - 0x54, 0xd0, 0x50, 0x03, 0x16, 0x02, 0x88, 0xda, 0xb5, 0x8a, 0x41, 0x4a, 0x55, 0x79, 0xb9, 0x52, - 0x5d, 0xf0, 0xf8, 0x25, 0xd5, 0x2f, 0xa1, 0x47, 0xb9, 0xc7, 0x83, 0xda, 0x9b, 0x4f, 0x8a, 0x43, - 0x2d, 0x9b, 0xed, 0x96, 0x41, 0xbe, 0x85, 0xd5, 0x85, 0xfc, 0x80, 0xf4, 0x11, 0xe8, 0x5b, 0x52, - 0x86, 0xe5, 0xc1, 0x7b, 0xbd, 0x7f, 0x79, 0xb5, 0x61, 0xfc, 0xdb, 0xab, 0x0d, 0xe3, 0x3f, 0x5e, - 0x6d, 0x18, 0xbf, 0xfe, 0xcf, 0x8d, 0x95, 0x89, 0x85, 0x7f, 0xe9, 0x7f, 0xfd, 0x7f, 0x01, 0x00, - 0x00, 0xff, 0xff, 0xc2, 0xe7, 0x33, 0xd9, 0x16, 0x20, 0x00, 0x00, + 0x72, 0x9e, 0xc6, 0xb3, 0x3b, 0x01, 0x72, 0xa0, 0xd2, 0xec, 0x08, 0xa2, 0x76, 0x67, 0xa8, 0xd6, + 0x63, 0xa8, 0x91, 0xc4, 0x19, 0x51, 0xb2, 0xbd, 0x5a, 0x87, 0x0f, 0x1c, 0x12, 0x33, 0xa6, 0xc4, + 0x97, 0x0b, 0xe0, 0xac, 0xad, 0x70, 0x18, 0x51, 0xe8, 0x2e, 0x82, 0xbd, 0x6c, 0x74, 0xb7, 0xbb, + 0x1a, 0x0c, 0x50, 0x37, 0x1f, 0xec, 0xdf, 0xb0, 0x07, 0x87, 0x6f, 0xbe, 0xf8, 0xe2, 0xeb, 0x1e, + 0x7d, 0x71, 0x84, 0x8f, 0x0e, 0xff, 0x01, 0x3b, 0x64, 0xff, 0x03, 0x47, 0x38, 0xc2, 0x37, 0x47, + 0x66, 0x55, 0x3f, 0x80, 0x21, 0x67, 0x56, 0x8e, 0xd8, 0x13, 0x3a, 0x1f, 0xf5, 0xfa, 0x2a, 0x33, + 0x2b, 0x33, 0x01, 0x76, 0x32, 0xd9, 0x4e, 0xd2, 0x38, 0x8b, 0x59, 0x2d, 0x99, 0x6c, 0x38, 0x22, + 0x09, 0x34, 0xb9, 0xf1, 0x68, 0x1a, 0x64, 0x17, 0xf3, 0xc9, 0xb6, 0x17, 0xcf, 0x9e, 0xf8, 0xd3, + 0x54, 0x24, 0x17, 0x9f, 0x07, 0xf1, 0x93, 0x89, 0xf0, 0xa7, 0x32, 0x7d, 0x92, 0x4c, 0x9e, 0xe4, + 0xe3, 0xdc, 0x0d, 0x68, 0x1c, 0x06, 0x2a, 0x63, 0x0c, 0x1a, 0xf3, 0xc0, 0x57, 0x7d, 0x6b, 0xb3, + 0xbe, 0xd5, 0xe2, 0xf4, 0xed, 0x1e, 0x81, 0x33, 0x12, 0xea, 0xf2, 0xa5, 0x08, 0xe7, 0x92, 0xf5, + 0xa0, 0x7e, 0x25, 0xc2, 0xbe, 0xb5, 0x69, 0x6d, 0x75, 0x39, 0x7e, 0xb2, 0x6d, 0xb0, 0xaf, 0x44, + 0x38, 0xce, 0xae, 0x13, 0xd9, 0xaf, 0x6d, 0x5a, 0x5b, 0xeb, 0x3b, 0x6f, 0x6f, 0x27, 0x93, 0xed, + 0xd3, 0x58, 0x65, 0x41, 0x34, 0xdd, 0x7e, 0x29, 0xc2, 0xd1, 0x75, 0x22, 0x79, 0xfb, 0x4a, 0x7f, + 0xb8, 0x27, 0xd0, 0x19, 0xa6, 0xde, 0xf3, 0x79, 0xe4, 0x65, 0x41, 0x1c, 0xe1, 0x8a, 0x91, 0x98, + 0x49, 0x9a, 0xd1, 0xe1, 0xf4, 0x8d, 0x3c, 0x91, 0x4e, 0x55, 0xbf, 0xbe, 0x59, 0x47, 0x1e, 0x7e, + 0xb3, 0x3e, 0xb4, 0x03, 0xb5, 0x17, 0xcf, 0xa3, 0xac, 0xdf, 0xd8, 0xb4, 0xb6, 0x6c, 0x9e, 0x93, + 0xee, 0x7f, 0xd7, 0xa0, 0xf9, 0x27, 0x73, 0x99, 0x5e, 0xd3, 0xb8, 0x2c, 0x4b, 0xf3, 0xb9, 0xf0, + 0x9b, 0xdd, 0x83, 0x66, 0x28, 0xa2, 0xa9, 0xea, 0xd7, 0x68, 0x32, 0x4d, 0xb0, 0xf7, 0xc0, 0x11, + 0xe7, 0x99, 0x4c, 0xc7, 0xf3, 0xc0, 0xef, 0xd7, 0x37, 0xad, 0xad, 0x16, 0xb7, 0x89, 0x71, 0x16, + 0xf8, 0xec, 0x5d, 0xb0, 0xfd, 0x78, 0xec, 0x55, 0xd7, 0xf2, 0x63, 0x5a, 0x8b, 0x7d, 0x00, 0xf6, + 0x3c, 0xf0, 0xc7, 0x61, 0xa0, 0xb2, 0x7e, 0x73, 0xd3, 0xda, 0xea, 0xec, 0xd8, 0x78, 0x58, 0xc4, + 0x8e, 0xb7, 0xe7, 0x81, 0x4f, 0x20, 0x3e, 0x06, 0x5b, 0xa5, 0xde, 0xf8, 0x7c, 0x1e, 0x79, 0xfd, + 0x16, 0x29, 0xdd, 0x45, 0xa5, 0xca, 0xa9, 0x79, 0x5b, 0x69, 0x02, 0x8f, 0x95, 0xca, 0x2b, 0x99, + 0x2a, 0xd9, 0x6f, 0xeb, 0xa5, 0x0c, 0xc9, 0x9e, 0x42, 0xe7, 0x5c, 0x78, 0x32, 0x1b, 0x27, 0x22, + 0x15, 0xb3, 0xbe, 0x5d, 0x4e, 0xf4, 0x1c, 0xd9, 0xa7, 0xc8, 0x55, 0x1c, 0xce, 0x0b, 0x82, 0x7d, + 0x09, 0x6b, 0x44, 0xa9, 0xf1, 0x79, 0x10, 0x66, 0x32, 0xed, 0x3b, 0x34, 0x66, 0x9d, 0xc6, 0x10, + 0x67, 0x94, 0x4a, 0xc9, 0xbb, 0x5a, 0x49, 0x73, 0xd8, 0xcf, 0x00, 0xe4, 0x22, 0x11, 0x91, 0x3f, + 0x16, 0x61, 0xd8, 0x07, 0xda, 0x83, 0xa3, 0x39, 0xbb, 0x61, 0xc8, 0xde, 0xc1, 0xfd, 0x09, 0x7f, + 0x9c, 0xa9, 0xfe, 0xda, 0xa6, 0xb5, 0xd5, 0xe0, 0x2d, 0x24, 0x47, 0xca, 0xdd, 0x01, 0x87, 0x2c, + 0x82, 0x4e, 0xfc, 0x11, 0xb4, 0xae, 0x90, 0xd0, 0x86, 0xd3, 0xd9, 0x59, 0xc3, 0x25, 0x0b, 0xa3, + 0xe1, 0x46, 0xe8, 0x3e, 0x00, 0xfb, 0x50, 0x44, 0xd3, 0xdc, 0xd2, 0xf0, 0x2a, 0x68, 0x80, 0xc3, + 0xe9, 0xdb, 0xfd, 0x75, 0x0d, 0x5a, 0x5c, 0xaa, 0x79, 0x98, 0xb1, 0x47, 0x00, 0x08, 0xf4, 0x4c, + 0x64, 0x69, 0xb0, 0x30, 0xb3, 0x96, 0x50, 0x3b, 0xf3, 0xc0, 0x3f, 0x22, 0x11, 0x7b, 0x0a, 0x5d, + 0x9a, 0x3d, 0x57, 0xad, 0x95, 0x1b, 0x28, 0xf6, 0xc7, 0x3b, 0xa4, 0x62, 0x46, 0xdc, 0x87, 0x16, + 0xdd, 0xad, 0xb6, 0xaf, 0x35, 0x6e, 0x28, 0xf6, 0x11, 0xac, 0x07, 0x51, 0x86, 0xd8, 0x7b, 0xd9, + 0xd8, 0x97, 0x2a, 0xbf, 0xfc, 0xb5, 0x82, 0xbb, 0x2f, 0x55, 0xc6, 0xbe, 0x00, 0x0d, 0x60, 0xbe, + 0x60, 0x93, 0x16, 0x5c, 0x2f, 0x2e, 0x46, 0xe9, 0x15, 0x49, 0xc7, 0xac, 0xf8, 0x39, 0x74, 0xf0, + 0x7c, 0xf9, 0x88, 0x16, 0x8d, 0xe8, 0xd2, 0x69, 0x0c, 0x1c, 0x1c, 0x50, 0xc1, 0xa8, 0x23, 0x34, + 0x68, 0x60, 0xda, 0x20, 0xe8, 0xdb, 0x1d, 0x40, 0xf3, 0x24, 0xf5, 0x65, 0x7a, 0xa3, 0x8d, 0x33, + 0x68, 0xf8, 0x52, 0x79, 0xe4, 0x7e, 0x36, 0xa7, 0xef, 0xd2, 0xee, 0xeb, 0x15, 0xbb, 0x77, 0xff, + 0xce, 0x82, 0xce, 0x30, 0x4e, 0xb3, 0x23, 0xa9, 0x94, 0x98, 0x4a, 0xf6, 0x10, 0x9a, 0x31, 0x4e, + 0x6b, 0x10, 0x76, 0x70, 0x4f, 0xb4, 0x0e, 0xd7, 0xfc, 0x95, 0x7b, 0xa8, 0xdd, 0x7e, 0x0f, 0xf7, + 0xa0, 0xa9, 0x3d, 0x06, 0xbd, 0xa9, 0xc9, 0x35, 0x81, 0x58, 0xc7, 0xe7, 0xe7, 0x4a, 0x6a, 0x2c, + 0x9b, 0xdc, 0x50, 0xb7, 0x9b, 0xd5, 0xef, 0x01, 0xe0, 0xfe, 0x7e, 0xa4, 0x15, 0xb8, 0x17, 0xd0, + 0xe1, 0xe2, 0x3c, 0xdb, 0x8b, 0xa3, 0x4c, 0x2e, 0x32, 0xb6, 0x0e, 0xb5, 0xc0, 0x27, 0x88, 0x5a, + 0xbc, 0x16, 0xf8, 0xb8, 0xb9, 0x69, 0x1a, 0xcf, 0x13, 0x42, 0x68, 0x8d, 0x6b, 0x82, 0xa0, 0xf4, + 0xfd, 0x94, 0x76, 0x8c, 0x50, 0xfa, 0x7e, 0xca, 0x1e, 0x42, 0x47, 0x45, 0x22, 0x51, 0x17, 0x71, + 0x86, 0x9b, 0x6b, 0xd0, 0xe6, 0x20, 0x67, 0x8d, 0x94, 0xfb, 0xcf, 0x16, 0xb4, 0x8e, 0xe4, 0x6c, + 0x22, 0xd3, 0x57, 0x56, 0x79, 0x17, 0x6c, 0x9a, 0x78, 0x1c, 0xf8, 0x66, 0xa1, 0x36, 0xd1, 0x07, + 0xfe, 0x8d, 0x4b, 0xdd, 0x87, 0x56, 0x28, 0x05, 0x82, 0xaf, 0xed, 0xcc, 0x50, 0x88, 0x8d, 0x98, + 0x8d, 0x7d, 0x29, 0x7c, 0x0a, 0x31, 0x36, 0x6f, 0x89, 0xd9, 0xbe, 0x14, 0x3e, 0xee, 0x2d, 0x14, + 0x2a, 0x1b, 0xcf, 0x13, 0x5f, 0x64, 0x92, 0x42, 0x4b, 0x03, 0x0d, 0x47, 0x65, 0x67, 0xc4, 0x61, + 0x8f, 0xe1, 0x2d, 0x2f, 0x9c, 0x2b, 0x8c, 0x6b, 0x41, 0x74, 0x1e, 0x8f, 0xe3, 0x28, 0xbc, 0x26, + 0x7c, 0x6d, 0x7e, 0xd7, 0x08, 0x0e, 0xa2, 0xf3, 0xf8, 0x24, 0x0a, 0xaf, 0xdd, 0xdf, 0xd4, 0xa0, + 0xf9, 0x82, 0x60, 0x78, 0x0a, 0xed, 0x19, 0x1d, 0x28, 0xf7, 0xde, 0xfb, 0x88, 0x30, 0xc9, 0xb6, + 0xf5, 0x49, 0xd5, 0x20, 0xca, 0xd2, 0x6b, 0x9e, 0xab, 0xe1, 0x88, 0x4c, 0x4c, 0x42, 0x99, 0x29, + 0x63, 0x11, 0x95, 0x11, 0x23, 0x2d, 0x30, 0x23, 0x8c, 0xda, 0x2a, 0xac, 0xf5, 0x55, 0x58, 0xd9, + 0x06, 0xd8, 0xde, 0x85, 0xf4, 0x2e, 0xd5, 0x7c, 0x66, 0x40, 0x2f, 0xe8, 0x8d, 0xe7, 0xd0, 0xad, + 0xee, 0x03, 0xdf, 0xa0, 0x4b, 0x79, 0x4d, 0xc0, 0x37, 0x38, 0x7e, 0xb2, 0x4d, 0x68, 0x92, 0x87, + 0x13, 0xec, 0x9d, 0x1d, 0xc0, 0xed, 0xe8, 0x21, 0x5c, 0x0b, 0x7e, 0x51, 0xfb, 0xb9, 0x85, 0xf3, + 0x54, 0x77, 0x57, 0x9d, 0xc7, 0xb9, 0x7d, 0x1e, 0x3d, 0xa4, 0x32, 0x8f, 0xfb, 0xbf, 0x35, 0xe8, + 0x7e, 0x27, 0xd3, 0xf8, 0x34, 0x8d, 0x93, 0x58, 0x89, 0x90, 0xed, 0x2e, 0x9f, 0x4e, 0xa3, 0xb8, + 0x89, 0x83, 0xab, 0x6a, 0xdb, 0xc3, 0xe2, 0xb8, 0x1a, 0x9d, 0xea, 0xf9, 0x5d, 0x68, 0x69, 0x74, + 0x6f, 0x38, 0x82, 0x91, 0xa0, 0x8e, 0xc6, 0x93, 0xf0, 0x5b, 0xde, 0x9e, 0x91, 0xb0, 0x07, 0x00, + 0x33, 0xb1, 0x38, 0x94, 0x42, 0xc9, 0x03, 0x3f, 0x37, 0xdf, 0x92, 0x83, 0x38, 0xcf, 0xc4, 0x62, + 0xb4, 0x88, 0x46, 0x8a, 0xac, 0xab, 0xc1, 0x0b, 0x9a, 0xfd, 0x14, 0x9c, 0x99, 0x58, 0xa0, 0x1f, + 0x1d, 0xf8, 0xc6, 0xba, 0x4a, 0x06, 0x7b, 0x1f, 0xea, 0xd9, 0x22, 0xa2, 0xa0, 0x84, 0xef, 0x10, + 0x26, 0x19, 0xa3, 0x45, 0x64, 0x3c, 0x8e, 0xa3, 0x2c, 0x07, 0xd4, 0x2e, 0x01, 0xed, 0x41, 0xdd, + 0x0b, 0x7c, 0x7a, 0x88, 0x1c, 0x8e, 0x9f, 0x1b, 0x7f, 0x04, 0x77, 0x57, 0x70, 0xa8, 0xde, 0xc3, + 0x9a, 0x1e, 0x76, 0xaf, 0x7a, 0x0f, 0x8d, 0x2a, 0xf6, 0xbf, 0xa9, 0xc3, 0x5d, 0x63, 0x0c, 0x17, + 0x41, 0x32, 0xcc, 0xd0, 0xec, 0xfb, 0xd0, 0xa6, 0x68, 0x23, 0x53, 0x63, 0x13, 0x39, 0xc9, 0xfe, + 0x00, 0x5a, 0xe4, 0x81, 0xb9, 0x9d, 0x3e, 0x2c, 0x51, 0x2d, 0x86, 0x6b, 0xbb, 0x35, 0x57, 0x62, + 0xd4, 0xd9, 0x57, 0xd0, 0xfc, 0x5e, 0xa6, 0xb1, 0x8e, 0x9e, 0x9d, 0x9d, 0x07, 0x37, 0x8d, 0xc3, + 0xbb, 0x35, 0xc3, 0xb4, 0xf2, 0xef, 0x10, 0xfc, 0x0f, 0x31, 0x5e, 0xce, 0xe2, 0x2b, 0xe9, 0xf7, + 0xdb, 0xb4, 0xa3, 0xaa, 0x7d, 0xe4, 0xa2, 0x1c, 0x6d, 0xbb, 0x44, 0x7b, 0x1f, 0x3a, 0x95, 0xe3, + 0xdd, 0x80, 0xf4, 0xc3, 0x65, 0x8b, 0x77, 0x0a, 0x47, 0xae, 0x3a, 0xce, 0x3e, 0x40, 0x79, 0xd8, + 0xff, 0xaf, 0xfb, 0xb9, 0x7f, 0x65, 0xc1, 0xdd, 0xbd, 0x38, 0x8a, 0x24, 0xa5, 0x40, 0xfa, 0xea, + 0x4a, 0xb3, 0xb7, 0x6e, 0x35, 0xfb, 0x4f, 0xa0, 0xa9, 0x50, 0xd9, 0xcc, 0xfe, 0xf6, 0x0d, 0x77, + 0xc1, 0xb5, 0x06, 0x86, 0x99, 0x99, 0x58, 0x8c, 0x13, 0x19, 0xf9, 0x41, 0x34, 0xcd, 0xc3, 0xcc, + 0x4c, 0x2c, 0x4e, 0x35, 0xc7, 0xfd, 0x6b, 0x0b, 0x5a, 0xda, 0x63, 0x96, 0xa2, 0xb5, 0xb5, 0x1c, + 0xad, 0x7f, 0x0a, 0x4e, 0x92, 0x4a, 0x3f, 0xf0, 0xf2, 0x55, 0x1d, 0x5e, 0x32, 0xd0, 0x38, 0xcf, + 0xe3, 0xd4, 0x93, 0x34, 0xbd, 0xcd, 0x35, 0x81, 0x5c, 0x95, 0x08, 0x4f, 0xa7, 0x71, 0x75, 0xae, + 0x09, 0x8c, 0xf1, 0xfa, 0x72, 0xe8, 0x52, 0x6c, 0x6e, 0x28, 0xf7, 0x1f, 0x6a, 0xd0, 0xdd, 0x0f, + 0x52, 0xe9, 0x65, 0xd2, 0x1f, 0xf8, 0x53, 0x52, 0x94, 0x51, 0x16, 0x64, 0xd7, 0xe6, 0x3d, 0x31, + 0x54, 0xf1, 0xdc, 0xd7, 0x96, 0x53, 0x5a, 0x0d, 0x77, 0x9d, 0xb2, 0x70, 0x4d, 0xb0, 0x1d, 0x00, + 0x9d, 0x08, 0x51, 0x26, 0xde, 0xb8, 0x3d, 0x13, 0x77, 0x48, 0x0d, 0x3f, 0x11, 0x03, 0x3d, 0x26, + 0xd0, 0x6f, 0x4d, 0x8b, 0xd2, 0xf4, 0x39, 0xda, 0x2a, 0xe5, 0x0f, 0x13, 0x19, 0x92, 0x2d, 0x52, + 0xfe, 0x30, 0x91, 0x61, 0x91, 0xb5, 0xb5, 0xf5, 0x76, 0xf0, 0x9b, 0x7d, 0x00, 0xb5, 0x38, 0xa1, + 0xf3, 0x99, 0x05, 0xab, 0x07, 0xdb, 0x3e, 0x49, 0x78, 0x2d, 0x4e, 0xf0, 0xa2, 0x75, 0xda, 0xd9, + 0x77, 0x8c, 0xfd, 0x62, 0x00, 0xa1, 0x84, 0x89, 0x1b, 0x89, 0x7b, 0x1f, 0x6a, 0x27, 0x09, 0x6b, + 0x43, 0x7d, 0x38, 0x18, 0xf5, 0xee, 0xe0, 0xc7, 0xfe, 0xe0, 0xb0, 0x67, 0xb9, 0x3f, 0x58, 0xe0, + 0x1c, 0xcd, 0x33, 0x81, 0x66, 0xa3, 0x5e, 0x77, 0x6f, 0xef, 0x82, 0xad, 0x32, 0x91, 0x52, 0x10, + 0xd6, 0x91, 0xa3, 0x4d, 0xf4, 0x48, 0xb1, 0x8f, 0xa1, 0x29, 0xfd, 0xa9, 0xcc, 0x1d, 0xba, 0xb7, + 0xba, 0x4f, 0xae, 0xc5, 0x6c, 0x0b, 0x5a, 0xca, 0xbb, 0x90, 0x33, 0xd1, 0x6f, 0x94, 0x8a, 0x43, + 0xe2, 0xe8, 0x47, 0x96, 0x1b, 0x39, 0x55, 0x09, 0x69, 0x9c, 0x50, 0xda, 0xdc, 0x34, 0x55, 0x42, + 0x1a, 0x27, 0x98, 0x34, 0xef, 0xc0, 0x4f, 0x82, 0x69, 0x14, 0xa7, 0x72, 0x1c, 0x44, 0xbe, 0x5c, + 0x8c, 0xbd, 0x38, 0x3a, 0x0f, 0x03, 0x2f, 0x23, 0x2c, 0x6d, 0xfe, 0xb6, 0x16, 0x1e, 0xa0, 0x6c, + 0xcf, 0x88, 0xdc, 0x05, 0xd8, 0x79, 0x5c, 0x64, 0x9f, 0x60, 0x40, 0xa3, 0xb8, 0x6a, 0xdc, 0x82, + 0xd2, 0xfe, 0x4a, 0x82, 0xc3, 0x73, 0x39, 0x5e, 0x13, 0xad, 0x91, 0x47, 0x4a, 0x22, 0xaa, 0xe9, + 0x55, 0xbd, 0x9a, 0x5e, 0x51, 0xa6, 0x18, 0x47, 0xd2, 0x64, 0x1c, 0xf4, 0xed, 0xfe, 0x6d, 0x0d, + 0xec, 0xe2, 0x29, 0xfb, 0x14, 0x9c, 0x59, 0x0e, 0xb5, 0x71, 0x38, 0xca, 0xa5, 0x0b, 0xfc, 0x79, + 0x29, 0x67, 0xf7, 0xa1, 0x76, 0x79, 0x65, 0x80, 0x6a, 0xa1, 0xd6, 0xb7, 0x2f, 0x79, 0xed, 0xf2, + 0xaa, 0xf4, 0xd8, 0xe6, 0x1b, 0x3d, 0xf6, 0x11, 0xdc, 0xf5, 0x42, 0x29, 0xa2, 0x71, 0xe9, 0x70, + 0xda, 0xe0, 0xd6, 0x89, 0x7d, 0x5a, 0x78, 0x9d, 0x89, 0x3a, 0xed, 0xf2, 0x6d, 0xf9, 0x08, 0x9a, + 0xbe, 0x0c, 0x33, 0x51, 0x2d, 0x8d, 0x4e, 0x52, 0xe1, 0x85, 0x72, 0x1f, 0xd9, 0x5c, 0x4b, 0xd9, + 0x16, 0xd8, 0xf9, 0x3b, 0x6b, 0x0a, 0x22, 0xca, 0xbc, 0x73, 0xb0, 0x79, 0x21, 0x2d, 0xb1, 0x84, + 0x0a, 0x96, 0xee, 0x17, 0x50, 0xff, 0xf6, 0xe5, 0xd0, 0x9c, 0xd5, 0x7a, 0xe5, 0xac, 0x39, 0xa2, + 0xb5, 0x0a, 0xa2, 0xff, 0x53, 0x87, 0xb6, 0xf1, 0x3a, 0xdc, 0xf7, 0xbc, 0xc8, 0x12, 0xf1, 0x73, + 0xf9, 0x71, 0x2b, 0xdc, 0xb7, 0x5a, 0x46, 0xd7, 0xdf, 0x5c, 0x46, 0xb3, 0x5f, 0x40, 0x37, 0xd1, + 0xb2, 0xaa, 0xc3, 0xbf, 0x53, 0x1d, 0x63, 0x7e, 0x69, 0x5c, 0x27, 0x29, 0x09, 0x34, 0x5d, 0xaa, + 0x47, 0x32, 0x31, 0xa5, 0x2b, 0xea, 0xf2, 0x36, 0xd2, 0x23, 0x31, 0xbd, 0xc5, 0xed, 0x7f, 0x0b, + 0xef, 0xc5, 0x6c, 0x38, 0x4e, 0xfa, 0x5d, 0xf2, 0x48, 0xf4, 0xf8, 0xaa, 0x33, 0xae, 0x2d, 0x3b, + 0xe3, 0x7b, 0xe0, 0x78, 0xf1, 0x6c, 0x16, 0x90, 0x6c, 0xdd, 0x64, 0x7b, 0xc4, 0x18, 0x29, 0xf7, + 0x6f, 0x2c, 0x68, 0x9b, 0xd3, 0xb2, 0x0e, 0xb4, 0xf7, 0x07, 0xcf, 0x77, 0xcf, 0x0e, 0x31, 0x1e, + 0x00, 0xb4, 0x9e, 0x1d, 0x1c, 0xef, 0xf2, 0x3f, 0xeb, 0x59, 0x18, 0x1b, 0x0e, 0x8e, 0x47, 0xbd, + 0x1a, 0x73, 0xa0, 0xf9, 0xfc, 0xf0, 0x64, 0x77, 0xd4, 0xab, 0x33, 0x1b, 0x1a, 0xcf, 0x4e, 0x4e, + 0x0e, 0x7b, 0x0d, 0xd6, 0x05, 0x7b, 0x7f, 0x77, 0x34, 0x18, 0x1d, 0x1c, 0x0d, 0x7a, 0x4d, 0xd4, + 0x7d, 0x31, 0x38, 0xe9, 0xb5, 0xf0, 0xe3, 0xec, 0x60, 0xbf, 0xd7, 0x46, 0xf9, 0xe9, 0xee, 0x70, + 0xf8, 0xcb, 0x13, 0xbe, 0xdf, 0xb3, 0x71, 0xde, 0xe1, 0x88, 0x1f, 0x1c, 0xbf, 0xe8, 0x39, 0xf8, + 0x7d, 0xf2, 0xec, 0x9b, 0xc1, 0xde, 0xa8, 0x07, 0xee, 0x17, 0xd0, 0xa9, 0x20, 0x88, 0xa3, 0xf9, + 0xe0, 0x79, 0xef, 0x0e, 0x2e, 0xf9, 0x72, 0xf7, 0xf0, 0x6c, 0xd0, 0xb3, 0xd8, 0x3a, 0x00, 0x7d, + 0x8e, 0x0f, 0x77, 0x8f, 0x5f, 0xf4, 0x6a, 0xee, 0xef, 0x83, 0x7d, 0x16, 0xf8, 0xcf, 0xc2, 0xd8, + 0xbb, 0x44, 0xc3, 0x98, 0x08, 0x25, 0xcd, 0x3b, 0x49, 0xdf, 0x18, 0xe5, 0xc9, 0x28, 0x95, 0xb9, + 0x7b, 0x43, 0xb9, 0xc7, 0xd0, 0x3e, 0x0b, 0xfc, 0x53, 0xe1, 0x5d, 0x62, 0x3d, 0x3e, 0xc1, 0xf1, + 0x63, 0x15, 0x7c, 0x2f, 0x4d, 0x80, 0x73, 0x88, 0x33, 0x0c, 0xbe, 0x97, 0xec, 0x43, 0x68, 0x11, + 0x91, 0x67, 0x34, 0x64, 0xcb, 0xf9, 0x9a, 0xdc, 0xc8, 0xdc, 0x7f, 0xb7, 0x8a, 0xbd, 0x53, 0xb1, + 0xfd, 0x10, 0x1a, 0x89, 0xf0, 0x2e, 0x4d, 0x34, 0xe9, 0x98, 0x31, 0xb8, 0x1e, 0x27, 0x01, 0x7b, + 0x04, 0xb6, 0x31, 0x90, 0x7c, 0xe2, 0x4e, 0xc5, 0x92, 0x78, 0x21, 0x5c, 0xbe, 0xba, 0xfa, 0xf2, + 0xd5, 0xe1, 0xde, 0x67, 0xf3, 0x30, 0x0b, 0xc6, 0x89, 0x48, 0xf3, 0xea, 0xd9, 0x21, 0xce, 0xa9, + 0x48, 0xc9, 0xbf, 0x50, 0xa0, 0xa8, 0x64, 0x6e, 0x70, 0x4d, 0xe0, 0x8c, 0xda, 0x4e, 0xd0, 0x4d, + 0x74, 0xe2, 0xa3, 0x0d, 0xe7, 0x2c, 0xf0, 0x31, 0x90, 0xc9, 0xc8, 0x27, 0x51, 0x5b, 0x07, 0x32, + 0x19, 0xf9, 0x67, 0x81, 0xef, 0x7e, 0x05, 0x50, 0xb6, 0x41, 0x6e, 0xc8, 0xe4, 0xef, 0x41, 0x53, + 0x84, 0x81, 0x01, 0xda, 0xe1, 0x9a, 0x70, 0x8f, 0xa1, 0x53, 0x69, 0x9e, 0xa0, 0x89, 0x8a, 0x30, + 0x1c, 0x5f, 0xca, 0x6b, 0x45, 0x63, 0x6d, 0xde, 0x16, 0x61, 0xf8, 0xad, 0xbc, 0x56, 0xec, 0x43, + 0xda, 0xab, 0x98, 0x19, 0x34, 0xd6, 0x97, 0xfb, 0x2e, 0x5c, 0x0b, 0xdd, 0xcf, 0xa0, 0xa5, 0x6b, + 0xfe, 0x8a, 0x87, 0x58, 0xb7, 0xbe, 0x6f, 0x5f, 0x9b, 0x3d, 0x53, 0x87, 0x80, 0x7d, 0x6a, 0xfa, + 0x3b, 0x4a, 0x77, 0x93, 0xac, 0x32, 0xad, 0xd3, 0x4a, 0xa6, 0xb5, 0x43, 0xca, 0xee, 0x3e, 0xd8, + 0xaf, 0xed, 0x98, 0x19, 0x00, 0x6a, 0x25, 0x00, 0x37, 0xf4, 0xd0, 0xdc, 0x5f, 0x01, 0x94, 0x7d, + 0x20, 0xe3, 0xb0, 0x7a, 0x16, 0x74, 0xd8, 0xc7, 0x58, 0x82, 0x05, 0xa1, 0x9f, 0xca, 0x68, 0xe9, + 0xd4, 0x65, 0xe7, 0xa8, 0x90, 0xb3, 0x4d, 0x68, 0x50, 0x7b, 0xab, 0x5e, 0x06, 0xd4, 0xa2, 0xb7, + 0x45, 0x12, 0x77, 0x02, 0x6b, 0xfa, 0xd9, 0xe4, 0xf2, 0x2f, 0xe7, 0x52, 0xbd, 0x36, 0xdf, 0x7a, + 0x00, 0x50, 0x84, 0xff, 0xbc, 0x51, 0x57, 0xe1, 0xa0, 0xdb, 0x9c, 0x07, 0x32, 0xf4, 0xf3, 0xd3, + 0x18, 0xca, 0xfd, 0x43, 0xe8, 0xe6, 0x6b, 0x50, 0xbb, 0xe0, 0xd3, 0xe2, 0xf1, 0xd6, 0x68, 0xea, + 0x2a, 0x45, 0xab, 0x1c, 0xc7, 0xbe, 0x7c, 0x56, 0xeb, 0x5b, 0xf9, 0xfb, 0xed, 0xfe, 0x5b, 0x3d, + 0x1f, 0x6d, 0xaa, 0xe7, 0xa5, 0xac, 0xcf, 0x5a, 0xcd, 0xfa, 0x96, 0xd3, 0xab, 0xda, 0x6f, 0x95, + 0x5e, 0xfd, 0x1c, 0x1c, 0x9f, 0x72, 0x8c, 0xe0, 0x2a, 0x0f, 0xea, 0x1b, 0xab, 0xf9, 0x84, 0xc9, + 0x42, 0x82, 0x2b, 0xc9, 0x4b, 0x65, 0xdc, 0x4b, 0x16, 0x5f, 0xca, 0x28, 0xf8, 0x9e, 0xda, 0x03, + 0x78, 0xe8, 0x92, 0x51, 0xf6, 0x5a, 0x74, 0xde, 0x61, 0x7a, 0x2d, 0x79, 0xdb, 0xa8, 0x55, 0xb6, + 0x8d, 0x10, 0xb9, 0x79, 0xa2, 0x64, 0x9a, 0xe5, 0xf9, 0xa7, 0xa6, 0x8a, 0x3c, 0xce, 0x31, 0xba, + 0x98, 0xc7, 0xbd, 0x0f, 0xdd, 0x28, 0x8e, 0xc6, 0xd1, 0x3c, 0x0c, 0x31, 0x43, 0x36, 0xbd, 0xc0, + 0x4e, 0x14, 0x47, 0xc7, 0x86, 0xc5, 0x1e, 0xc3, 0x5b, 0x55, 0x15, 0x6d, 0xb9, 0x1d, 0xdd, 0x60, + 0xa8, 0xe8, 0x91, 0x7d, 0x6f, 0x41, 0x2f, 0x9e, 0xfc, 0x4a, 0x7a, 0x19, 0x21, 0x36, 0x26, 0x93, + 0xed, 0xea, 0xa7, 0x5d, 0xf3, 0x11, 0xa2, 0x63, 0x31, 0x93, 0xee, 0xd7, 0xe0, 0x14, 0x20, 0x60, + 0x14, 0x3f, 0x3e, 0x39, 0x1e, 0xe8, 0x38, 0x7b, 0x70, 0xbc, 0x3f, 0xf8, 0xd3, 0x9e, 0x85, 0xef, + 0x00, 0x1f, 0xbc, 0x1c, 0xf0, 0xe1, 0xa0, 0x57, 0xc3, 0x18, 0xbd, 0x3f, 0x38, 0x1c, 0x8c, 0x06, + 0xbd, 0xfa, 0x37, 0x0d, 0xbb, 0xdd, 0xb3, 0xb9, 0x2d, 0x17, 0x49, 0x18, 0x78, 0x41, 0xe6, 0x0e, + 0x01, 0x70, 0x5a, 0x73, 0xa3, 0xef, 0x81, 0x53, 0xae, 0xad, 0x6f, 0xd4, 0xce, 0xcc, 0xaa, 0x98, + 0xe9, 0x19, 0xa3, 0xaa, 0xdd, 0x96, 0xe9, 0x19, 0x33, 0x3b, 0x03, 0xfb, 0x48, 0x24, 0xaf, 0x14, + 0x3f, 0xdd, 0xa2, 0xc4, 0x9d, 0x9b, 0x86, 0x8f, 0x79, 0xe0, 0x3f, 0x82, 0xb6, 0x89, 0x97, 0xc6, + 0x3f, 0x96, 0x62, 0x69, 0x2e, 0xc3, 0x5a, 0xe4, 0xde, 0x51, 0x7c, 0x25, 0x8b, 0x1c, 0xe7, 0x54, + 0x5c, 0x87, 0xb1, 0xf0, 0xdf, 0x60, 0x88, 0x3f, 0x03, 0x50, 0xf1, 0x3c, 0xf5, 0xe4, 0x78, 0x5a, + 0xf4, 0x99, 0x1c, 0xcd, 0x79, 0x61, 0x9a, 0xd7, 0x52, 0x65, 0x24, 0xac, 0x6b, 0x37, 0x43, 0x1a, + 0x45, 0x3f, 0x81, 0x56, 0xb6, 0x88, 0xca, 0xb6, 0x56, 0x33, 0xc3, 0xca, 0xd3, 0xdd, 0x03, 0x67, + 0xb4, 0xa0, 0x7a, 0x6c, 0xae, 0x96, 0x5e, 0x6d, 0xeb, 0x35, 0xaf, 0x76, 0x6d, 0xe5, 0xd5, 0xfe, + 0x2f, 0x0b, 0x3a, 0x95, 0xe4, 0x8b, 0xbd, 0x0f, 0x8d, 0x6c, 0x11, 0x2d, 0xf7, 0x83, 0xf3, 0x45, + 0x38, 0x89, 0xd0, 0xde, 0xb0, 0x58, 0x13, 0x4a, 0x05, 0xd3, 0x48, 0xfa, 0x66, 0x4a, 0x2c, 0xe0, + 0x76, 0x0d, 0x8b, 0x1d, 0xc2, 0x5d, 0x1d, 0x33, 0xf2, 0x5e, 0x50, 0x9e, 0xbf, 0x7f, 0xb0, 0x92, + 0xec, 0xe9, 0x9a, 0x75, 0x2f, 0xd7, 0xd2, 0x55, 0xf9, 0xfa, 0x74, 0x89, 0xb9, 0xb1, 0x0b, 0x6f, + 0xdf, 0xa0, 0xf6, 0xa3, 0xda, 0x0f, 0x0f, 0x61, 0x0d, 0xcb, 0xf5, 0x60, 0x26, 0x55, 0x26, 0x66, + 0x09, 0x65, 0x3d, 0x26, 0xe6, 0x37, 0x78, 0x2d, 0x53, 0xee, 0xc7, 0xd0, 0x3d, 0x95, 0x32, 0xe5, + 0x52, 0x25, 0x71, 0xa4, 0x5f, 0x7c, 0x45, 0x87, 0x36, 0x0f, 0x8c, 0xa1, 0xdc, 0xbf, 0x00, 0x07, + 0xf3, 0xf9, 0x67, 0x22, 0xf3, 0x2e, 0x7e, 0x4c, 0xbe, 0xff, 0x31, 0xb4, 0x13, 0x6d, 0x26, 0x26, + 0x3b, 0xef, 0x52, 0x8c, 0x33, 0xa6, 0xc3, 0x73, 0xa1, 0xfb, 0x15, 0xd4, 0x8f, 0xe7, 0xb3, 0xea, + 0xdf, 0x35, 0x0d, 0xfd, 0x77, 0xcd, 0x7b, 0xe0, 0x50, 0x69, 0x40, 0xbd, 0x41, 0x9d, 0xb4, 0xda, + 0xc8, 0xa0, 0xa6, 0xe0, 0x77, 0xd0, 0xc9, 0xb1, 0x3f, 0xf0, 0xe9, 0x3f, 0x17, 0xba, 0xfc, 0x03, + 0x7f, 0xc9, 0x16, 0x74, 0x75, 0x28, 0x23, 0xff, 0x20, 0xbf, 0x34, 0x4d, 0x2c, 0xcf, 0x6d, 0x1a, + 0x1c, 0xc5, 0xdc, 0xcf, 0xa1, 0x9b, 0xe7, 0xdc, 0x47, 0x32, 0x13, 0x64, 0x4e, 0x61, 0x20, 0xa3, + 0x8a, 0xa9, 0xd9, 0x9a, 0x31, 0x52, 0xaf, 0x69, 0xa5, 0xba, 0xdb, 0xd0, 0x32, 0xb6, 0xca, 0xa0, + 0xe1, 0xc5, 0xbe, 0x76, 0x91, 0x26, 0xa7, 0x6f, 0x3c, 0xf0, 0x4c, 0x4d, 0xf3, 0x87, 0x70, 0xa6, + 0xa6, 0xee, 0x02, 0xd6, 0x9e, 0x09, 0xef, 0x72, 0x9e, 0xe4, 0x0f, 0x51, 0xa5, 0x38, 0xb2, 0x96, + 0x8a, 0xa3, 0xd7, 0xf4, 0x6f, 0xdf, 0x81, 0xf6, 0x3c, 0x0a, 0x16, 0x79, 0xd2, 0xe3, 0xf0, 0x16, + 0x92, 0xba, 0x6f, 0x19, 0xc6, 0x1e, 0xd5, 0x43, 0xe4, 0x55, 0x0e, 0x2f, 0x68, 0xf7, 0xcf, 0x61, + 0x6d, 0xb0, 0x48, 0xa8, 0x9b, 0xfd, 0xc6, 0x27, 0xb0, 0xb2, 0xa9, 0xda, 0xd2, 0xa6, 0x56, 0x56, + 0xae, 0xe7, 0x2b, 0xef, 0xfc, 0x93, 0x05, 0x0d, 0x34, 0x11, 0x2c, 0xd9, 0xfe, 0x58, 0x8a, 0x34, + 0x9b, 0x48, 0x91, 0xb1, 0x25, 0x73, 0xd8, 0x58, 0xa2, 0xdc, 0x3b, 0x4f, 0x2d, 0xf6, 0x99, 0x6e, + 0x94, 0xe7, 0xfd, 0xff, 0xb5, 0xdc, 0xd0, 0xc8, 0x10, 0x57, 0xf5, 0xd9, 0x36, 0x74, 0xbe, 0x89, + 0x83, 0x68, 0x4f, 0xf7, 0x8e, 0xd9, 0xaa, 0x59, 0xbe, 0xa2, 0xff, 0x39, 0xb4, 0x0e, 0x14, 0xda, + 0xff, 0xab, 0xaa, 0x14, 0x5e, 0xab, 0xae, 0xe1, 0xde, 0xd9, 0xf9, 0xc7, 0x3a, 0x34, 0xbe, 0x93, + 0x69, 0xcc, 0x3e, 0x83, 0xb6, 0xe9, 0x0c, 0xb1, 0x4a, 0x07, 0x68, 0x83, 0xde, 0xd7, 0x95, 0x96, + 0x11, 0xad, 0xd2, 0xd3, 0x11, 0xba, 0xac, 0x29, 0x59, 0xd9, 0xb8, 0x7a, 0x65, 0x53, 0x5f, 0x43, + 0x6f, 0x98, 0xa5, 0x52, 0xcc, 0x2a, 0xea, 0xcb, 0x30, 0xdd, 0x54, 0xa0, 0x12, 0x5a, 0x9f, 0x42, + 0x4b, 0x07, 0x99, 0x95, 0x01, 0xab, 0xb5, 0x26, 0x29, 0x3f, 0x82, 0xce, 0xf0, 0x22, 0x9e, 0x87, + 0xfe, 0x50, 0xa6, 0x57, 0x92, 0x55, 0xba, 0xb3, 0x1b, 0x95, 0x6f, 0xf7, 0x0e, 0xdb, 0x02, 0xd0, + 0x5e, 0x76, 0x16, 0xf8, 0x8a, 0xb5, 0x51, 0x76, 0x3c, 0x9f, 0xe9, 0x49, 0x2b, 0xee, 0xa7, 0x35, + 0x2b, 0xb1, 0xe6, 0x75, 0x9a, 0x5f, 0xc2, 0xda, 0x1e, 0xc5, 0xe2, 0x93, 0x74, 0x77, 0x12, 0xa7, + 0x19, 0x5b, 0xed, 0xd0, 0x6e, 0xac, 0x32, 0xdc, 0x3b, 0xec, 0x29, 0xd8, 0xa3, 0xf4, 0x5a, 0xeb, + 0xbf, 0x65, 0x42, 0x74, 0xb9, 0xde, 0x0d, 0xa7, 0xdc, 0xf9, 0xfb, 0x3a, 0xb4, 0x7e, 0x19, 0xa7, + 0x97, 0x32, 0x65, 0x8f, 0xa1, 0x45, 0x4d, 0x01, 0x63, 0x44, 0x45, 0x83, 0xe0, 0xa6, 0x85, 0x3e, + 0x04, 0x87, 0x40, 0x19, 0x09, 0x75, 0xa9, 0xaf, 0x8a, 0xfe, 0xb0, 0xd5, 0xb8, 0xe8, 0xe4, 0x8d, + 0xee, 0x75, 0x5d, 0x5f, 0x54, 0xd1, 0x08, 0x59, 0xaa, 0xd4, 0x37, 0xda, 0xba, 0xec, 0x1e, 0xba, + 0x77, 0xb6, 0xac, 0xa7, 0x16, 0xfb, 0x04, 0x1a, 0x43, 0x7d, 0x52, 0x54, 0x2a, 0xff, 0xd4, 0xda, + 0x58, 0xcf, 0x19, 0xc5, 0xcc, 0x4f, 0xa0, 0xa5, 0x5f, 0x76, 0x7d, 0xcc, 0xa5, 0xc4, 0x74, 0xa3, + 0x57, 0x65, 0x99, 0x01, 0x9f, 0x40, 0x4b, 0x07, 0x0d, 0x3d, 0x60, 0x29, 0x80, 0xe8, 0x5d, 0xeb, + 0x18, 0xa4, 0x55, 0xb5, 0x97, 0x6b, 0xd5, 0x25, 0x8f, 0x5f, 0x51, 0xfd, 0x1c, 0x7a, 0x5c, 0x7a, + 0x32, 0xa8, 0xbc, 0xf9, 0x2c, 0x3f, 0xd4, 0xaa, 0xd9, 0x6e, 0x59, 0xec, 0x6b, 0x58, 0x5b, 0xca, + 0x0f, 0x58, 0x9f, 0x80, 0xbe, 0x21, 0x65, 0x58, 0x1d, 0xfc, 0xac, 0xf7, 0x2f, 0x3f, 0x3c, 0xb0, + 0xfe, 0xf5, 0x87, 0x07, 0xd6, 0x7f, 0xfc, 0xf0, 0xc0, 0xfa, 0xf5, 0x7f, 0x3e, 0xb8, 0x33, 0x69, + 0xd1, 0x1f, 0xfd, 0x5f, 0xfe, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x59, 0x4a, 0xb6, 0x37, 0x2c, + 0x20, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -6449,13 +6458,30 @@ func (m *PostingList) MarshalTo(dAtA []byte) (int, error) { } i++ } + if len(m.Parts) > 0 { + dAtA23 := make([]byte, len(m.Parts)*10) + var j22 int + for _, num := range m.Parts { + for num >= 1<<7 { + dAtA23[j22] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j22++ + } + dAtA23[j22] = uint8(num) + j22++ + } + dAtA[i] = 0x2a + i++ + i = encodeVarintPb(dAtA, i, uint64(j22)) + i += copy(dAtA[i:], dAtA23[:j22]) + } if m.StartUid != 0 { - dAtA[i] = 0x28 + dAtA[i] = 0x30 i++ i = encodeVarintPb(dAtA, i, uint64(m.StartUid)) } if m.EndUid != 0 { - dAtA[i] = 0x30 + dAtA[i] = 0x38 i++ i = encodeVarintPb(dAtA, i, uint64(m.EndUid)) } @@ -6692,11 +6718,11 @@ func (m *FilterTree) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintPb(dAtA, i, uint64(m.Func.Size())) - n22, err := m.Func.MarshalTo(dAtA[i:]) + n24, err := m.Func.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n24 } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) @@ -6981,11 +7007,11 @@ func (m *MapEntry) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintPb(dAtA, i, uint64(m.Posting.Size())) - n23, err := m.Posting.MarshalTo(dAtA[i:]) + n25, err := m.Posting.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n25 } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) @@ -7135,21 +7161,21 @@ func (m *TxnTimestamps) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.Ts) > 0 { - dAtA25 := make([]byte, len(m.Ts)*10) - var j24 int + dAtA27 := make([]byte, len(m.Ts)*10) + var j26 int for _, num := range m.Ts { for num >= 1<<7 { - dAtA25[j24] = uint8(uint64(num)&0x7f | 0x80) + dAtA27[j26] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j24++ + j26++ } - dAtA25[j24] = uint8(num) - j24++ + dAtA27[j26] = uint8(num) + j26++ } dAtA[i] = 0xa i++ - i = encodeVarintPb(dAtA, i, uint64(j24)) - i += copy(dAtA[i:], dAtA25[:j24]) + i = encodeVarintPb(dAtA, i, uint64(j26)) + i += copy(dAtA[i:], dAtA27[:j26]) } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) @@ -7207,21 +7233,21 @@ func (m *RaftBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPb(dAtA, i, uint64(m.Context.Size())) - n26, err := m.Context.MarshalTo(dAtA[i:]) + n28, err := m.Context.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n28 } if m.Payload != nil { dAtA[i] = 0x12 i++ i = encodeVarintPb(dAtA, i, uint64(m.Payload.Size())) - n27, err := m.Payload.MarshalTo(dAtA[i:]) + n29, err := m.Payload.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n29 } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) @@ -8271,6 +8297,13 @@ func (m *PostingList) Size() (n int) { if m.MultiPart { n += 2 } + if len(m.Parts) > 0 { + l = 0 + for _, e := range m.Parts { + l += sovPb(uint64(e)) + } + n += 1 + sovPb(uint64(l)) + l + } if m.StartUid != 0 { n += 1 + sovPb(uint64(m.StartUid)) } @@ -13608,6 +13641,79 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { } m.MultiPart = bool(v != 0) case 5: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Parts = append(m.Parts, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthPb + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Parts) == 0 { + m.Parts = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Parts = append(m.Parts, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Parts", wireType) + } + case 6: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field StartUid", wireType) } @@ -13626,7 +13732,7 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { break } } - case 6: + case 7: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field EndUid", wireType) } From 78d35fb0b50d9e7a2aadeba1c0d798f4f5a65271 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 16:02:49 -0800 Subject: [PATCH 17/67] Remove partialIteration field. --- posting/list.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/posting/list.go b/posting/list.go index ba4b606b0c0..c018a60180e 100644 --- a/posting/list.go +++ b/posting/list.go @@ -112,7 +112,6 @@ type PIterator struct { type PItrOpts struct { discardPl bool afterUid uint64 - partialIteration bool startPart int readTs uint64 } @@ -172,11 +171,6 @@ func (it *PIterator) Valid() bool { return false } - // Only iterate through this part of the list. - if it.opts.partialIteration { - return false - } - // No more parts to iterate through if len(it.l.plist.Parts) == it.opts.startPart+1 { return false From 68473785cf2d4a22149e50b615c40528d139b7c0 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 16:16:30 -0800 Subject: [PATCH 18/67] Remove unused proto fields. --- posting/list.go | 10 +- protos/pb.proto | 6 +- protos/pb/pb.pb.go | 508 +++++++++++++++++++-------------------------- 3 files changed, 219 insertions(+), 305 deletions(-) diff --git a/posting/list.go b/posting/list.go index c018a60180e..4bb41d5596f 100644 --- a/posting/list.go +++ b/posting/list.go @@ -166,8 +166,7 @@ func (it *PIterator) Valid() bool { return true } - // Not a multi-part list, so nothing else to iterate through - if !it.l.plist.MultiPart { + if len(it.l.plist.Parts) == 0 { return false } @@ -873,7 +872,7 @@ func (l *List) Uids(opt ListOptions) (*pb.List, error) { // Use approximate length for initial capacity. res := make([]uint64, 0, len(l.mutationMap)+codec.ApproxLen(l.plist.Pack)) out := &pb.List{} - if len(l.mutationMap) == 0 && opt.Intersect != nil && !l.plist.MultiPart { + if len(l.mutationMap) == 0 && opt.Intersect != nil && len(l.plist.Parts) == 0 { if opt.ReadTs < l.minTs { l.RUnlock() return out, ErrTsTooOld @@ -1112,10 +1111,6 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs } func (l *List) readListPart(baseKey []byte, startUid uint64) (*pb.PostingList, error) { - if !l.plist.MultiPart { - return nil, nil - } - if part, ok := l.newParts[startUid]; ok { return part, nil } @@ -1151,7 +1146,6 @@ func (l *List) splitList(readTs uint64) error { l.plist = &pb.PostingList{ CommitTs: l.plist.CommitTs, - MultiPart: true, Parts: newParts, } } diff --git a/protos/pb.proto b/protos/pb.proto index 789774bdc5f..017781f9064 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -278,10 +278,8 @@ message PostingList { repeated Posting postings = 2; uint64 commit_ts = 3; // More inclination towards smaller values. - bool multi_part = 4; - repeated uint64 parts = 5; - uint64 start_uid = 6; - uint64 end_uid = 7; + repeated uint64 parts = 4; + uint64 start_uid = 5; } message FacetParam { diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index 33085ad7777..baaa676fcc9 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -2100,10 +2100,8 @@ type PostingList struct { Pack *UidPack `protobuf:"bytes,1,opt,name=pack,proto3" json:"pack,omitempty"` Postings []*Posting `protobuf:"bytes,2,rep,name=postings,proto3" json:"postings,omitempty"` CommitTs uint64 `protobuf:"varint,3,opt,name=commit_ts,json=commitTs,proto3" json:"commit_ts,omitempty"` - MultiPart bool `protobuf:"varint,4,opt,name=multi_part,json=multiPart,proto3" json:"multi_part,omitempty"` - Parts []uint64 `protobuf:"varint,5,rep,packed,name=parts,proto3" json:"parts,omitempty"` - StartUid uint64 `protobuf:"varint,6,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` - EndUid uint64 `protobuf:"varint,7,opt,name=end_uid,json=endUid,proto3" json:"end_uid,omitempty"` + Parts []uint64 `protobuf:"varint,4,rep,packed,name=parts,proto3" json:"parts,omitempty"` + StartUid uint64 `protobuf:"varint,5,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2163,13 +2161,6 @@ func (m *PostingList) GetCommitTs() uint64 { return 0 } -func (m *PostingList) GetMultiPart() bool { - if m != nil { - return m.MultiPart - } - return false -} - func (m *PostingList) GetParts() []uint64 { if m != nil { return m.Parts @@ -2184,13 +2175,6 @@ func (m *PostingList) GetStartUid() uint64 { return 0 } -func (m *PostingList) GetEndUid() uint64 { - if m != nil { - return m.EndUid - } - return 0 -} - type FacetParam struct { Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` Alias string `protobuf:"bytes,2,opt,name=alias,proto3" json:"alias,omitempty"` @@ -3653,218 +3637,216 @@ func init() { func init() { proto.RegisterFile("pb.proto", fileDescriptor_f80abaa17e25ccc8) } var fileDescriptor_f80abaa17e25ccc8 = []byte{ - // 3363 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4b, 0x73, 0x23, 0x47, - 0x72, 0x9e, 0xc6, 0xb3, 0x3b, 0x01, 0x72, 0xa0, 0xd2, 0xec, 0x08, 0xa2, 0x76, 0x67, 0xa8, 0xd6, - 0x63, 0xa8, 0x91, 0xc4, 0x19, 0x51, 0xb2, 0xbd, 0x5a, 0x87, 0x0f, 0x1c, 0x12, 0x33, 0xa6, 0xc4, - 0x97, 0x0b, 0xe0, 0xac, 0xad, 0x70, 0x18, 0x51, 0xe8, 0x2e, 0x82, 0xbd, 0x6c, 0x74, 0xb7, 0xbb, - 0x1a, 0x0c, 0x50, 0x37, 0x1f, 0xec, 0xdf, 0xb0, 0x07, 0x87, 0x6f, 0xbe, 0xf8, 0xe2, 0xeb, 0x1e, - 0x7d, 0x71, 0x84, 0x8f, 0x0e, 0xff, 0x01, 0x3b, 0x64, 0xff, 0x03, 0x47, 0x38, 0xc2, 0x37, 0x47, - 0x66, 0x55, 0x3f, 0x80, 0x21, 0x67, 0x56, 0x8e, 0xd8, 0x13, 0x3a, 0x1f, 0xf5, 0xfa, 0x2a, 0x33, - 0x2b, 0x33, 0x01, 0x76, 0x32, 0xd9, 0x4e, 0xd2, 0x38, 0x8b, 0x59, 0x2d, 0x99, 0x6c, 0x38, 0x22, - 0x09, 0x34, 0xb9, 0xf1, 0x68, 0x1a, 0x64, 0x17, 0xf3, 0xc9, 0xb6, 0x17, 0xcf, 0x9e, 0xf8, 0xd3, - 0x54, 0x24, 0x17, 0x9f, 0x07, 0xf1, 0x93, 0x89, 0xf0, 0xa7, 0x32, 0x7d, 0x92, 0x4c, 0x9e, 0xe4, - 0xe3, 0xdc, 0x0d, 0x68, 0x1c, 0x06, 0x2a, 0x63, 0x0c, 0x1a, 0xf3, 0xc0, 0x57, 0x7d, 0x6b, 0xb3, - 0xbe, 0xd5, 0xe2, 0xf4, 0xed, 0x1e, 0x81, 0x33, 0x12, 0xea, 0xf2, 0xa5, 0x08, 0xe7, 0x92, 0xf5, - 0xa0, 0x7e, 0x25, 0xc2, 0xbe, 0xb5, 0x69, 0x6d, 0x75, 0x39, 0x7e, 0xb2, 0x6d, 0xb0, 0xaf, 0x44, - 0x38, 0xce, 0xae, 0x13, 0xd9, 0xaf, 0x6d, 0x5a, 0x5b, 0xeb, 0x3b, 0x6f, 0x6f, 0x27, 0x93, 0xed, - 0xd3, 0x58, 0x65, 0x41, 0x34, 0xdd, 0x7e, 0x29, 0xc2, 0xd1, 0x75, 0x22, 0x79, 0xfb, 0x4a, 0x7f, - 0xb8, 0x27, 0xd0, 0x19, 0xa6, 0xde, 0xf3, 0x79, 0xe4, 0x65, 0x41, 0x1c, 0xe1, 0x8a, 0x91, 0x98, - 0x49, 0x9a, 0xd1, 0xe1, 0xf4, 0x8d, 0x3c, 0x91, 0x4e, 0x55, 0xbf, 0xbe, 0x59, 0x47, 0x1e, 0x7e, - 0xb3, 0x3e, 0xb4, 0x03, 0xb5, 0x17, 0xcf, 0xa3, 0xac, 0xdf, 0xd8, 0xb4, 0xb6, 0x6c, 0x9e, 0x93, - 0xee, 0x7f, 0xd7, 0xa0, 0xf9, 0x27, 0x73, 0x99, 0x5e, 0xd3, 0xb8, 0x2c, 0x4b, 0xf3, 0xb9, 0xf0, - 0x9b, 0xdd, 0x83, 0x66, 0x28, 0xa2, 0xa9, 0xea, 0xd7, 0x68, 0x32, 0x4d, 0xb0, 0xf7, 0xc0, 0x11, - 0xe7, 0x99, 0x4c, 0xc7, 0xf3, 0xc0, 0xef, 0xd7, 0x37, 0xad, 0xad, 0x16, 0xb7, 0x89, 0x71, 0x16, - 0xf8, 0xec, 0x5d, 0xb0, 0xfd, 0x78, 0xec, 0x55, 0xd7, 0xf2, 0x63, 0x5a, 0x8b, 0x7d, 0x00, 0xf6, - 0x3c, 0xf0, 0xc7, 0x61, 0xa0, 0xb2, 0x7e, 0x73, 0xd3, 0xda, 0xea, 0xec, 0xd8, 0x78, 0x58, 0xc4, - 0x8e, 0xb7, 0xe7, 0x81, 0x4f, 0x20, 0x3e, 0x06, 0x5b, 0xa5, 0xde, 0xf8, 0x7c, 0x1e, 0x79, 0xfd, - 0x16, 0x29, 0xdd, 0x45, 0xa5, 0xca, 0xa9, 0x79, 0x5b, 0x69, 0x02, 0x8f, 0x95, 0xca, 0x2b, 0x99, - 0x2a, 0xd9, 0x6f, 0xeb, 0xa5, 0x0c, 0xc9, 0x9e, 0x42, 0xe7, 0x5c, 0x78, 0x32, 0x1b, 0x27, 0x22, - 0x15, 0xb3, 0xbe, 0x5d, 0x4e, 0xf4, 0x1c, 0xd9, 0xa7, 0xc8, 0x55, 0x1c, 0xce, 0x0b, 0x82, 0x7d, - 0x09, 0x6b, 0x44, 0xa9, 0xf1, 0x79, 0x10, 0x66, 0x32, 0xed, 0x3b, 0x34, 0x66, 0x9d, 0xc6, 0x10, - 0x67, 0x94, 0x4a, 0xc9, 0xbb, 0x5a, 0x49, 0x73, 0xd8, 0xcf, 0x00, 0xe4, 0x22, 0x11, 0x91, 0x3f, - 0x16, 0x61, 0xd8, 0x07, 0xda, 0x83, 0xa3, 0x39, 0xbb, 0x61, 0xc8, 0xde, 0xc1, 0xfd, 0x09, 0x7f, - 0x9c, 0xa9, 0xfe, 0xda, 0xa6, 0xb5, 0xd5, 0xe0, 0x2d, 0x24, 0x47, 0xca, 0xdd, 0x01, 0x87, 0x2c, - 0x82, 0x4e, 0xfc, 0x11, 0xb4, 0xae, 0x90, 0xd0, 0x86, 0xd3, 0xd9, 0x59, 0xc3, 0x25, 0x0b, 0xa3, - 0xe1, 0x46, 0xe8, 0x3e, 0x00, 0xfb, 0x50, 0x44, 0xd3, 0xdc, 0xd2, 0xf0, 0x2a, 0x68, 0x80, 0xc3, - 0xe9, 0xdb, 0xfd, 0x75, 0x0d, 0x5a, 0x5c, 0xaa, 0x79, 0x98, 0xb1, 0x47, 0x00, 0x08, 0xf4, 0x4c, - 0x64, 0x69, 0xb0, 0x30, 0xb3, 0x96, 0x50, 0x3b, 0xf3, 0xc0, 0x3f, 0x22, 0x11, 0x7b, 0x0a, 0x5d, - 0x9a, 0x3d, 0x57, 0xad, 0x95, 0x1b, 0x28, 0xf6, 0xc7, 0x3b, 0xa4, 0x62, 0x46, 0xdc, 0x87, 0x16, - 0xdd, 0xad, 0xb6, 0xaf, 0x35, 0x6e, 0x28, 0xf6, 0x11, 0xac, 0x07, 0x51, 0x86, 0xd8, 0x7b, 0xd9, - 0xd8, 0x97, 0x2a, 0xbf, 0xfc, 0xb5, 0x82, 0xbb, 0x2f, 0x55, 0xc6, 0xbe, 0x00, 0x0d, 0x60, 0xbe, - 0x60, 0x93, 0x16, 0x5c, 0x2f, 0x2e, 0x46, 0xe9, 0x15, 0x49, 0xc7, 0xac, 0xf8, 0x39, 0x74, 0xf0, - 0x7c, 0xf9, 0x88, 0x16, 0x8d, 0xe8, 0xd2, 0x69, 0x0c, 0x1c, 0x1c, 0x50, 0xc1, 0xa8, 0x23, 0x34, - 0x68, 0x60, 0xda, 0x20, 0xe8, 0xdb, 0x1d, 0x40, 0xf3, 0x24, 0xf5, 0x65, 0x7a, 0xa3, 0x8d, 0x33, - 0x68, 0xf8, 0x52, 0x79, 0xe4, 0x7e, 0x36, 0xa7, 0xef, 0xd2, 0xee, 0xeb, 0x15, 0xbb, 0x77, 0xff, - 0xce, 0x82, 0xce, 0x30, 0x4e, 0xb3, 0x23, 0xa9, 0x94, 0x98, 0x4a, 0xf6, 0x10, 0x9a, 0x31, 0x4e, - 0x6b, 0x10, 0x76, 0x70, 0x4f, 0xb4, 0x0e, 0xd7, 0xfc, 0x95, 0x7b, 0xa8, 0xdd, 0x7e, 0x0f, 0xf7, - 0xa0, 0xa9, 0x3d, 0x06, 0xbd, 0xa9, 0xc9, 0x35, 0x81, 0x58, 0xc7, 0xe7, 0xe7, 0x4a, 0x6a, 0x2c, - 0x9b, 0xdc, 0x50, 0xb7, 0x9b, 0xd5, 0xef, 0x01, 0xe0, 0xfe, 0x7e, 0xa4, 0x15, 0xb8, 0x17, 0xd0, - 0xe1, 0xe2, 0x3c, 0xdb, 0x8b, 0xa3, 0x4c, 0x2e, 0x32, 0xb6, 0x0e, 0xb5, 0xc0, 0x27, 0x88, 0x5a, - 0xbc, 0x16, 0xf8, 0xb8, 0xb9, 0x69, 0x1a, 0xcf, 0x13, 0x42, 0x68, 0x8d, 0x6b, 0x82, 0xa0, 0xf4, - 0xfd, 0x94, 0x76, 0x8c, 0x50, 0xfa, 0x7e, 0xca, 0x1e, 0x42, 0x47, 0x45, 0x22, 0x51, 0x17, 0x71, - 0x86, 0x9b, 0x6b, 0xd0, 0xe6, 0x20, 0x67, 0x8d, 0x94, 0xfb, 0xcf, 0x16, 0xb4, 0x8e, 0xe4, 0x6c, - 0x22, 0xd3, 0x57, 0x56, 0x79, 0x17, 0x6c, 0x9a, 0x78, 0x1c, 0xf8, 0x66, 0xa1, 0x36, 0xd1, 0x07, - 0xfe, 0x8d, 0x4b, 0xdd, 0x87, 0x56, 0x28, 0x05, 0x82, 0xaf, 0xed, 0xcc, 0x50, 0x88, 0x8d, 0x98, - 0x8d, 0x7d, 0x29, 0x7c, 0x0a, 0x31, 0x36, 0x6f, 0x89, 0xd9, 0xbe, 0x14, 0x3e, 0xee, 0x2d, 0x14, - 0x2a, 0x1b, 0xcf, 0x13, 0x5f, 0x64, 0x92, 0x42, 0x4b, 0x03, 0x0d, 0x47, 0x65, 0x67, 0xc4, 0x61, - 0x8f, 0xe1, 0x2d, 0x2f, 0x9c, 0x2b, 0x8c, 0x6b, 0x41, 0x74, 0x1e, 0x8f, 0xe3, 0x28, 0xbc, 0x26, - 0x7c, 0x6d, 0x7e, 0xd7, 0x08, 0x0e, 0xa2, 0xf3, 0xf8, 0x24, 0x0a, 0xaf, 0xdd, 0xdf, 0xd4, 0xa0, - 0xf9, 0x82, 0x60, 0x78, 0x0a, 0xed, 0x19, 0x1d, 0x28, 0xf7, 0xde, 0xfb, 0x88, 0x30, 0xc9, 0xb6, - 0xf5, 0x49, 0xd5, 0x20, 0xca, 0xd2, 0x6b, 0x9e, 0xab, 0xe1, 0x88, 0x4c, 0x4c, 0x42, 0x99, 0x29, - 0x63, 0x11, 0x95, 0x11, 0x23, 0x2d, 0x30, 0x23, 0x8c, 0xda, 0x2a, 0xac, 0xf5, 0x55, 0x58, 0xd9, - 0x06, 0xd8, 0xde, 0x85, 0xf4, 0x2e, 0xd5, 0x7c, 0x66, 0x40, 0x2f, 0xe8, 0x8d, 0xe7, 0xd0, 0xad, - 0xee, 0x03, 0xdf, 0xa0, 0x4b, 0x79, 0x4d, 0xc0, 0x37, 0x38, 0x7e, 0xb2, 0x4d, 0x68, 0x92, 0x87, - 0x13, 0xec, 0x9d, 0x1d, 0xc0, 0xed, 0xe8, 0x21, 0x5c, 0x0b, 0x7e, 0x51, 0xfb, 0xb9, 0x85, 0xf3, - 0x54, 0x77, 0x57, 0x9d, 0xc7, 0xb9, 0x7d, 0x1e, 0x3d, 0xa4, 0x32, 0x8f, 0xfb, 0xbf, 0x35, 0xe8, - 0x7e, 0x27, 0xd3, 0xf8, 0x34, 0x8d, 0x93, 0x58, 0x89, 0x90, 0xed, 0x2e, 0x9f, 0x4e, 0xa3, 0xb8, - 0x89, 0x83, 0xab, 0x6a, 0xdb, 0xc3, 0xe2, 0xb8, 0x1a, 0x9d, 0xea, 0xf9, 0x5d, 0x68, 0x69, 0x74, - 0x6f, 0x38, 0x82, 0x91, 0xa0, 0x8e, 0xc6, 0x93, 0xf0, 0x5b, 0xde, 0x9e, 0x91, 0xb0, 0x07, 0x00, - 0x33, 0xb1, 0x38, 0x94, 0x42, 0xc9, 0x03, 0x3f, 0x37, 0xdf, 0x92, 0x83, 0x38, 0xcf, 0xc4, 0x62, - 0xb4, 0x88, 0x46, 0x8a, 0xac, 0xab, 0xc1, 0x0b, 0x9a, 0xfd, 0x14, 0x9c, 0x99, 0x58, 0xa0, 0x1f, - 0x1d, 0xf8, 0xc6, 0xba, 0x4a, 0x06, 0x7b, 0x1f, 0xea, 0xd9, 0x22, 0xa2, 0xa0, 0x84, 0xef, 0x10, - 0x26, 0x19, 0xa3, 0x45, 0x64, 0x3c, 0x8e, 0xa3, 0x2c, 0x07, 0xd4, 0x2e, 0x01, 0xed, 0x41, 0xdd, - 0x0b, 0x7c, 0x7a, 0x88, 0x1c, 0x8e, 0x9f, 0x1b, 0x7f, 0x04, 0x77, 0x57, 0x70, 0xa8, 0xde, 0xc3, - 0x9a, 0x1e, 0x76, 0xaf, 0x7a, 0x0f, 0x8d, 0x2a, 0xf6, 0xbf, 0xa9, 0xc3, 0x5d, 0x63, 0x0c, 0x17, - 0x41, 0x32, 0xcc, 0xd0, 0xec, 0xfb, 0xd0, 0xa6, 0x68, 0x23, 0x53, 0x63, 0x13, 0x39, 0xc9, 0xfe, - 0x00, 0x5a, 0xe4, 0x81, 0xb9, 0x9d, 0x3e, 0x2c, 0x51, 0x2d, 0x86, 0x6b, 0xbb, 0x35, 0x57, 0x62, - 0xd4, 0xd9, 0x57, 0xd0, 0xfc, 0x5e, 0xa6, 0xb1, 0x8e, 0x9e, 0x9d, 0x9d, 0x07, 0x37, 0x8d, 0xc3, - 0xbb, 0x35, 0xc3, 0xb4, 0xf2, 0xef, 0x10, 0xfc, 0x0f, 0x31, 0x5e, 0xce, 0xe2, 0x2b, 0xe9, 0xf7, - 0xdb, 0xb4, 0xa3, 0xaa, 0x7d, 0xe4, 0xa2, 0x1c, 0x6d, 0xbb, 0x44, 0x7b, 0x1f, 0x3a, 0x95, 0xe3, - 0xdd, 0x80, 0xf4, 0xc3, 0x65, 0x8b, 0x77, 0x0a, 0x47, 0xae, 0x3a, 0xce, 0x3e, 0x40, 0x79, 0xd8, - 0xff, 0xaf, 0xfb, 0xb9, 0x7f, 0x65, 0xc1, 0xdd, 0xbd, 0x38, 0x8a, 0x24, 0xa5, 0x40, 0xfa, 0xea, - 0x4a, 0xb3, 0xb7, 0x6e, 0x35, 0xfb, 0x4f, 0xa0, 0xa9, 0x50, 0xd9, 0xcc, 0xfe, 0xf6, 0x0d, 0x77, - 0xc1, 0xb5, 0x06, 0x86, 0x99, 0x99, 0x58, 0x8c, 0x13, 0x19, 0xf9, 0x41, 0x34, 0xcd, 0xc3, 0xcc, - 0x4c, 0x2c, 0x4e, 0x35, 0xc7, 0xfd, 0x6b, 0x0b, 0x5a, 0xda, 0x63, 0x96, 0xa2, 0xb5, 0xb5, 0x1c, - 0xad, 0x7f, 0x0a, 0x4e, 0x92, 0x4a, 0x3f, 0xf0, 0xf2, 0x55, 0x1d, 0x5e, 0x32, 0xd0, 0x38, 0xcf, - 0xe3, 0xd4, 0x93, 0x34, 0xbd, 0xcd, 0x35, 0x81, 0x5c, 0x95, 0x08, 0x4f, 0xa7, 0x71, 0x75, 0xae, - 0x09, 0x8c, 0xf1, 0xfa, 0x72, 0xe8, 0x52, 0x6c, 0x6e, 0x28, 0xf7, 0x1f, 0x6a, 0xd0, 0xdd, 0x0f, - 0x52, 0xe9, 0x65, 0xd2, 0x1f, 0xf8, 0x53, 0x52, 0x94, 0x51, 0x16, 0x64, 0xd7, 0xe6, 0x3d, 0x31, - 0x54, 0xf1, 0xdc, 0xd7, 0x96, 0x53, 0x5a, 0x0d, 0x77, 0x9d, 0xb2, 0x70, 0x4d, 0xb0, 0x1d, 0x00, - 0x9d, 0x08, 0x51, 0x26, 0xde, 0xb8, 0x3d, 0x13, 0x77, 0x48, 0x0d, 0x3f, 0x11, 0x03, 0x3d, 0x26, - 0xd0, 0x6f, 0x4d, 0x8b, 0xd2, 0xf4, 0x39, 0xda, 0x2a, 0xe5, 0x0f, 0x13, 0x19, 0x92, 0x2d, 0x52, - 0xfe, 0x30, 0x91, 0x61, 0x91, 0xb5, 0xb5, 0xf5, 0x76, 0xf0, 0x9b, 0x7d, 0x00, 0xb5, 0x38, 0xa1, - 0xf3, 0x99, 0x05, 0xab, 0x07, 0xdb, 0x3e, 0x49, 0x78, 0x2d, 0x4e, 0xf0, 0xa2, 0x75, 0xda, 0xd9, - 0x77, 0x8c, 0xfd, 0x62, 0x00, 0xa1, 0x84, 0x89, 0x1b, 0x89, 0x7b, 0x1f, 0x6a, 0x27, 0x09, 0x6b, - 0x43, 0x7d, 0x38, 0x18, 0xf5, 0xee, 0xe0, 0xc7, 0xfe, 0xe0, 0xb0, 0x67, 0xb9, 0x3f, 0x58, 0xe0, - 0x1c, 0xcd, 0x33, 0x81, 0x66, 0xa3, 0x5e, 0x77, 0x6f, 0xef, 0x82, 0xad, 0x32, 0x91, 0x52, 0x10, - 0xd6, 0x91, 0xa3, 0x4d, 0xf4, 0x48, 0xb1, 0x8f, 0xa1, 0x29, 0xfd, 0xa9, 0xcc, 0x1d, 0xba, 0xb7, - 0xba, 0x4f, 0xae, 0xc5, 0x6c, 0x0b, 0x5a, 0xca, 0xbb, 0x90, 0x33, 0xd1, 0x6f, 0x94, 0x8a, 0x43, - 0xe2, 0xe8, 0x47, 0x96, 0x1b, 0x39, 0x55, 0x09, 0x69, 0x9c, 0x50, 0xda, 0xdc, 0x34, 0x55, 0x42, - 0x1a, 0x27, 0x98, 0x34, 0xef, 0xc0, 0x4f, 0x82, 0x69, 0x14, 0xa7, 0x72, 0x1c, 0x44, 0xbe, 0x5c, - 0x8c, 0xbd, 0x38, 0x3a, 0x0f, 0x03, 0x2f, 0x23, 0x2c, 0x6d, 0xfe, 0xb6, 0x16, 0x1e, 0xa0, 0x6c, - 0xcf, 0x88, 0xdc, 0x05, 0xd8, 0x79, 0x5c, 0x64, 0x9f, 0x60, 0x40, 0xa3, 0xb8, 0x6a, 0xdc, 0x82, - 0xd2, 0xfe, 0x4a, 0x82, 0xc3, 0x73, 0x39, 0x5e, 0x13, 0xad, 0x91, 0x47, 0x4a, 0x22, 0xaa, 0xe9, - 0x55, 0xbd, 0x9a, 0x5e, 0x51, 0xa6, 0x18, 0x47, 0xd2, 0x64, 0x1c, 0xf4, 0xed, 0xfe, 0x6d, 0x0d, - 0xec, 0xe2, 0x29, 0xfb, 0x14, 0x9c, 0x59, 0x0e, 0xb5, 0x71, 0x38, 0xca, 0xa5, 0x0b, 0xfc, 0x79, - 0x29, 0x67, 0xf7, 0xa1, 0x76, 0x79, 0x65, 0x80, 0x6a, 0xa1, 0xd6, 0xb7, 0x2f, 0x79, 0xed, 0xf2, - 0xaa, 0xf4, 0xd8, 0xe6, 0x1b, 0x3d, 0xf6, 0x11, 0xdc, 0xf5, 0x42, 0x29, 0xa2, 0x71, 0xe9, 0x70, - 0xda, 0xe0, 0xd6, 0x89, 0x7d, 0x5a, 0x78, 0x9d, 0x89, 0x3a, 0xed, 0xf2, 0x6d, 0xf9, 0x08, 0x9a, - 0xbe, 0x0c, 0x33, 0x51, 0x2d, 0x8d, 0x4e, 0x52, 0xe1, 0x85, 0x72, 0x1f, 0xd9, 0x5c, 0x4b, 0xd9, - 0x16, 0xd8, 0xf9, 0x3b, 0x6b, 0x0a, 0x22, 0xca, 0xbc, 0x73, 0xb0, 0x79, 0x21, 0x2d, 0xb1, 0x84, - 0x0a, 0x96, 0xee, 0x17, 0x50, 0xff, 0xf6, 0xe5, 0xd0, 0x9c, 0xd5, 0x7a, 0xe5, 0xac, 0x39, 0xa2, - 0xb5, 0x0a, 0xa2, 0xff, 0x53, 0x87, 0xb6, 0xf1, 0x3a, 0xdc, 0xf7, 0xbc, 0xc8, 0x12, 0xf1, 0x73, - 0xf9, 0x71, 0x2b, 0xdc, 0xb7, 0x5a, 0x46, 0xd7, 0xdf, 0x5c, 0x46, 0xb3, 0x5f, 0x40, 0x37, 0xd1, - 0xb2, 0xaa, 0xc3, 0xbf, 0x53, 0x1d, 0x63, 0x7e, 0x69, 0x5c, 0x27, 0x29, 0x09, 0x34, 0x5d, 0xaa, - 0x47, 0x32, 0x31, 0xa5, 0x2b, 0xea, 0xf2, 0x36, 0xd2, 0x23, 0x31, 0xbd, 0xc5, 0xed, 0x7f, 0x0b, - 0xef, 0xc5, 0x6c, 0x38, 0x4e, 0xfa, 0x5d, 0xf2, 0x48, 0xf4, 0xf8, 0xaa, 0x33, 0xae, 0x2d, 0x3b, - 0xe3, 0x7b, 0xe0, 0x78, 0xf1, 0x6c, 0x16, 0x90, 0x6c, 0xdd, 0x64, 0x7b, 0xc4, 0x18, 0x29, 0xf7, - 0x6f, 0x2c, 0x68, 0x9b, 0xd3, 0xb2, 0x0e, 0xb4, 0xf7, 0x07, 0xcf, 0x77, 0xcf, 0x0e, 0x31, 0x1e, - 0x00, 0xb4, 0x9e, 0x1d, 0x1c, 0xef, 0xf2, 0x3f, 0xeb, 0x59, 0x18, 0x1b, 0x0e, 0x8e, 0x47, 0xbd, - 0x1a, 0x73, 0xa0, 0xf9, 0xfc, 0xf0, 0x64, 0x77, 0xd4, 0xab, 0x33, 0x1b, 0x1a, 0xcf, 0x4e, 0x4e, - 0x0e, 0x7b, 0x0d, 0xd6, 0x05, 0x7b, 0x7f, 0x77, 0x34, 0x18, 0x1d, 0x1c, 0x0d, 0x7a, 0x4d, 0xd4, - 0x7d, 0x31, 0x38, 0xe9, 0xb5, 0xf0, 0xe3, 0xec, 0x60, 0xbf, 0xd7, 0x46, 0xf9, 0xe9, 0xee, 0x70, - 0xf8, 0xcb, 0x13, 0xbe, 0xdf, 0xb3, 0x71, 0xde, 0xe1, 0x88, 0x1f, 0x1c, 0xbf, 0xe8, 0x39, 0xf8, - 0x7d, 0xf2, 0xec, 0x9b, 0xc1, 0xde, 0xa8, 0x07, 0xee, 0x17, 0xd0, 0xa9, 0x20, 0x88, 0xa3, 0xf9, - 0xe0, 0x79, 0xef, 0x0e, 0x2e, 0xf9, 0x72, 0xf7, 0xf0, 0x6c, 0xd0, 0xb3, 0xd8, 0x3a, 0x00, 0x7d, - 0x8e, 0x0f, 0x77, 0x8f, 0x5f, 0xf4, 0x6a, 0xee, 0xef, 0x83, 0x7d, 0x16, 0xf8, 0xcf, 0xc2, 0xd8, - 0xbb, 0x44, 0xc3, 0x98, 0x08, 0x25, 0xcd, 0x3b, 0x49, 0xdf, 0x18, 0xe5, 0xc9, 0x28, 0x95, 0xb9, - 0x7b, 0x43, 0xb9, 0xc7, 0xd0, 0x3e, 0x0b, 0xfc, 0x53, 0xe1, 0x5d, 0x62, 0x3d, 0x3e, 0xc1, 0xf1, - 0x63, 0x15, 0x7c, 0x2f, 0x4d, 0x80, 0x73, 0x88, 0x33, 0x0c, 0xbe, 0x97, 0xec, 0x43, 0x68, 0x11, - 0x91, 0x67, 0x34, 0x64, 0xcb, 0xf9, 0x9a, 0xdc, 0xc8, 0xdc, 0x7f, 0xb7, 0x8a, 0xbd, 0x53, 0xb1, - 0xfd, 0x10, 0x1a, 0x89, 0xf0, 0x2e, 0x4d, 0x34, 0xe9, 0x98, 0x31, 0xb8, 0x1e, 0x27, 0x01, 0x7b, - 0x04, 0xb6, 0x31, 0x90, 0x7c, 0xe2, 0x4e, 0xc5, 0x92, 0x78, 0x21, 0x5c, 0xbe, 0xba, 0xfa, 0xf2, - 0xd5, 0xe1, 0xde, 0x67, 0xf3, 0x30, 0x0b, 0xc6, 0x89, 0x48, 0xf3, 0xea, 0xd9, 0x21, 0xce, 0xa9, - 0x48, 0xc9, 0xbf, 0x50, 0xa0, 0xa8, 0x64, 0x6e, 0x70, 0x4d, 0xe0, 0x8c, 0xda, 0x4e, 0xd0, 0x4d, - 0x74, 0xe2, 0xa3, 0x0d, 0xe7, 0x2c, 0xf0, 0x31, 0x90, 0xc9, 0xc8, 0x27, 0x51, 0x5b, 0x07, 0x32, - 0x19, 0xf9, 0x67, 0x81, 0xef, 0x7e, 0x05, 0x50, 0xb6, 0x41, 0x6e, 0xc8, 0xe4, 0xef, 0x41, 0x53, - 0x84, 0x81, 0x01, 0xda, 0xe1, 0x9a, 0x70, 0x8f, 0xa1, 0x53, 0x69, 0x9e, 0xa0, 0x89, 0x8a, 0x30, - 0x1c, 0x5f, 0xca, 0x6b, 0x45, 0x63, 0x6d, 0xde, 0x16, 0x61, 0xf8, 0xad, 0xbc, 0x56, 0xec, 0x43, - 0xda, 0xab, 0x98, 0x19, 0x34, 0xd6, 0x97, 0xfb, 0x2e, 0x5c, 0x0b, 0xdd, 0xcf, 0xa0, 0xa5, 0x6b, - 0xfe, 0x8a, 0x87, 0x58, 0xb7, 0xbe, 0x6f, 0x5f, 0x9b, 0x3d, 0x53, 0x87, 0x80, 0x7d, 0x6a, 0xfa, - 0x3b, 0x4a, 0x77, 0x93, 0xac, 0x32, 0xad, 0xd3, 0x4a, 0xa6, 0xb5, 0x43, 0xca, 0xee, 0x3e, 0xd8, - 0xaf, 0xed, 0x98, 0x19, 0x00, 0x6a, 0x25, 0x00, 0x37, 0xf4, 0xd0, 0xdc, 0x5f, 0x01, 0x94, 0x7d, - 0x20, 0xe3, 0xb0, 0x7a, 0x16, 0x74, 0xd8, 0xc7, 0x58, 0x82, 0x05, 0xa1, 0x9f, 0xca, 0x68, 0xe9, - 0xd4, 0x65, 0xe7, 0xa8, 0x90, 0xb3, 0x4d, 0x68, 0x50, 0x7b, 0xab, 0x5e, 0x06, 0xd4, 0xa2, 0xb7, - 0x45, 0x12, 0x77, 0x02, 0x6b, 0xfa, 0xd9, 0xe4, 0xf2, 0x2f, 0xe7, 0x52, 0xbd, 0x36, 0xdf, 0x7a, - 0x00, 0x50, 0x84, 0xff, 0xbc, 0x51, 0x57, 0xe1, 0xa0, 0xdb, 0x9c, 0x07, 0x32, 0xf4, 0xf3, 0xd3, - 0x18, 0xca, 0xfd, 0x43, 0xe8, 0xe6, 0x6b, 0x50, 0xbb, 0xe0, 0xd3, 0xe2, 0xf1, 0xd6, 0x68, 0xea, - 0x2a, 0x45, 0xab, 0x1c, 0xc7, 0xbe, 0x7c, 0x56, 0xeb, 0x5b, 0xf9, 0xfb, 0xed, 0xfe, 0x5b, 0x3d, - 0x1f, 0x6d, 0xaa, 0xe7, 0xa5, 0xac, 0xcf, 0x5a, 0xcd, 0xfa, 0x96, 0xd3, 0xab, 0xda, 0x6f, 0x95, - 0x5e, 0xfd, 0x1c, 0x1c, 0x9f, 0x72, 0x8c, 0xe0, 0x2a, 0x0f, 0xea, 0x1b, 0xab, 0xf9, 0x84, 0xc9, - 0x42, 0x82, 0x2b, 0xc9, 0x4b, 0x65, 0xdc, 0x4b, 0x16, 0x5f, 0xca, 0x28, 0xf8, 0x9e, 0xda, 0x03, - 0x78, 0xe8, 0x92, 0x51, 0xf6, 0x5a, 0x74, 0xde, 0x61, 0x7a, 0x2d, 0x79, 0xdb, 0xa8, 0x55, 0xb6, - 0x8d, 0x10, 0xb9, 0x79, 0xa2, 0x64, 0x9a, 0xe5, 0xf9, 0xa7, 0xa6, 0x8a, 0x3c, 0xce, 0x31, 0xba, - 0x98, 0xc7, 0xbd, 0x0f, 0xdd, 0x28, 0x8e, 0xc6, 0xd1, 0x3c, 0x0c, 0x31, 0x43, 0x36, 0xbd, 0xc0, - 0x4e, 0x14, 0x47, 0xc7, 0x86, 0xc5, 0x1e, 0xc3, 0x5b, 0x55, 0x15, 0x6d, 0xb9, 0x1d, 0xdd, 0x60, - 0xa8, 0xe8, 0x91, 0x7d, 0x6f, 0x41, 0x2f, 0x9e, 0xfc, 0x4a, 0x7a, 0x19, 0x21, 0x36, 0x26, 0x93, - 0xed, 0xea, 0xa7, 0x5d, 0xf3, 0x11, 0xa2, 0x63, 0x31, 0x93, 0xee, 0xd7, 0xe0, 0x14, 0x20, 0x60, - 0x14, 0x3f, 0x3e, 0x39, 0x1e, 0xe8, 0x38, 0x7b, 0x70, 0xbc, 0x3f, 0xf8, 0xd3, 0x9e, 0x85, 0xef, - 0x00, 0x1f, 0xbc, 0x1c, 0xf0, 0xe1, 0xa0, 0x57, 0xc3, 0x18, 0xbd, 0x3f, 0x38, 0x1c, 0x8c, 0x06, - 0xbd, 0xfa, 0x37, 0x0d, 0xbb, 0xdd, 0xb3, 0xb9, 0x2d, 0x17, 0x49, 0x18, 0x78, 0x41, 0xe6, 0x0e, - 0x01, 0x70, 0x5a, 0x73, 0xa3, 0xef, 0x81, 0x53, 0xae, 0xad, 0x6f, 0xd4, 0xce, 0xcc, 0xaa, 0x98, - 0xe9, 0x19, 0xa3, 0xaa, 0xdd, 0x96, 0xe9, 0x19, 0x33, 0x3b, 0x03, 0xfb, 0x48, 0x24, 0xaf, 0x14, - 0x3f, 0xdd, 0xa2, 0xc4, 0x9d, 0x9b, 0x86, 0x8f, 0x79, 0xe0, 0x3f, 0x82, 0xb6, 0x89, 0x97, 0xc6, - 0x3f, 0x96, 0x62, 0x69, 0x2e, 0xc3, 0x5a, 0xe4, 0xde, 0x51, 0x7c, 0x25, 0x8b, 0x1c, 0xe7, 0x54, - 0x5c, 0x87, 0xb1, 0xf0, 0xdf, 0x60, 0x88, 0x3f, 0x03, 0x50, 0xf1, 0x3c, 0xf5, 0xe4, 0x78, 0x5a, - 0xf4, 0x99, 0x1c, 0xcd, 0x79, 0x61, 0x9a, 0xd7, 0x52, 0x65, 0x24, 0xac, 0x6b, 0x37, 0x43, 0x1a, - 0x45, 0x3f, 0x81, 0x56, 0xb6, 0x88, 0xca, 0xb6, 0x56, 0x33, 0xc3, 0xca, 0xd3, 0xdd, 0x03, 0x67, - 0xb4, 0xa0, 0x7a, 0x6c, 0xae, 0x96, 0x5e, 0x6d, 0xeb, 0x35, 0xaf, 0x76, 0x6d, 0xe5, 0xd5, 0xfe, - 0x2f, 0x0b, 0x3a, 0x95, 0xe4, 0x8b, 0xbd, 0x0f, 0x8d, 0x6c, 0x11, 0x2d, 0xf7, 0x83, 0xf3, 0x45, - 0x38, 0x89, 0xd0, 0xde, 0xb0, 0x58, 0x13, 0x4a, 0x05, 0xd3, 0x48, 0xfa, 0x66, 0x4a, 0x2c, 0xe0, - 0x76, 0x0d, 0x8b, 0x1d, 0xc2, 0x5d, 0x1d, 0x33, 0xf2, 0x5e, 0x50, 0x9e, 0xbf, 0x7f, 0xb0, 0x92, - 0xec, 0xe9, 0x9a, 0x75, 0x2f, 0xd7, 0xd2, 0x55, 0xf9, 0xfa, 0x74, 0x89, 0xb9, 0xb1, 0x0b, 0x6f, - 0xdf, 0xa0, 0xf6, 0xa3, 0xda, 0x0f, 0x0f, 0x61, 0x0d, 0xcb, 0xf5, 0x60, 0x26, 0x55, 0x26, 0x66, - 0x09, 0x65, 0x3d, 0x26, 0xe6, 0x37, 0x78, 0x2d, 0x53, 0xee, 0xc7, 0xd0, 0x3d, 0x95, 0x32, 0xe5, - 0x52, 0x25, 0x71, 0xa4, 0x5f, 0x7c, 0x45, 0x87, 0x36, 0x0f, 0x8c, 0xa1, 0xdc, 0xbf, 0x00, 0x07, - 0xf3, 0xf9, 0x67, 0x22, 0xf3, 0x2e, 0x7e, 0x4c, 0xbe, 0xff, 0x31, 0xb4, 0x13, 0x6d, 0x26, 0x26, - 0x3b, 0xef, 0x52, 0x8c, 0x33, 0xa6, 0xc3, 0x73, 0xa1, 0xfb, 0x15, 0xd4, 0x8f, 0xe7, 0xb3, 0xea, - 0xdf, 0x35, 0x0d, 0xfd, 0x77, 0xcd, 0x7b, 0xe0, 0x50, 0x69, 0x40, 0xbd, 0x41, 0x9d, 0xb4, 0xda, - 0xc8, 0xa0, 0xa6, 0xe0, 0x77, 0xd0, 0xc9, 0xb1, 0x3f, 0xf0, 0xe9, 0x3f, 0x17, 0xba, 0xfc, 0x03, - 0x7f, 0xc9, 0x16, 0x74, 0x75, 0x28, 0x23, 0xff, 0x20, 0xbf, 0x34, 0x4d, 0x2c, 0xcf, 0x6d, 0x1a, - 0x1c, 0xc5, 0xdc, 0xcf, 0xa1, 0x9b, 0xe7, 0xdc, 0x47, 0x32, 0x13, 0x64, 0x4e, 0x61, 0x20, 0xa3, - 0x8a, 0xa9, 0xd9, 0x9a, 0x31, 0x52, 0xaf, 0x69, 0xa5, 0xba, 0xdb, 0xd0, 0x32, 0xb6, 0xca, 0xa0, - 0xe1, 0xc5, 0xbe, 0x76, 0x91, 0x26, 0xa7, 0x6f, 0x3c, 0xf0, 0x4c, 0x4d, 0xf3, 0x87, 0x70, 0xa6, - 0xa6, 0xee, 0x02, 0xd6, 0x9e, 0x09, 0xef, 0x72, 0x9e, 0xe4, 0x0f, 0x51, 0xa5, 0x38, 0xb2, 0x96, - 0x8a, 0xa3, 0xd7, 0xf4, 0x6f, 0xdf, 0x81, 0xf6, 0x3c, 0x0a, 0x16, 0x79, 0xd2, 0xe3, 0xf0, 0x16, - 0x92, 0xba, 0x6f, 0x19, 0xc6, 0x1e, 0xd5, 0x43, 0xe4, 0x55, 0x0e, 0x2f, 0x68, 0xf7, 0xcf, 0x61, - 0x6d, 0xb0, 0x48, 0xa8, 0x9b, 0xfd, 0xc6, 0x27, 0xb0, 0xb2, 0xa9, 0xda, 0xd2, 0xa6, 0x56, 0x56, - 0xae, 0xe7, 0x2b, 0xef, 0xfc, 0x93, 0x05, 0x0d, 0x34, 0x11, 0x2c, 0xd9, 0xfe, 0x58, 0x8a, 0x34, - 0x9b, 0x48, 0x91, 0xb1, 0x25, 0x73, 0xd8, 0x58, 0xa2, 0xdc, 0x3b, 0x4f, 0x2d, 0xf6, 0x99, 0x6e, - 0x94, 0xe7, 0xfd, 0xff, 0xb5, 0xdc, 0xd0, 0xc8, 0x10, 0x57, 0xf5, 0xd9, 0x36, 0x74, 0xbe, 0x89, - 0x83, 0x68, 0x4f, 0xf7, 0x8e, 0xd9, 0xaa, 0x59, 0xbe, 0xa2, 0xff, 0x39, 0xb4, 0x0e, 0x14, 0xda, - 0xff, 0xab, 0xaa, 0x14, 0x5e, 0xab, 0xae, 0xe1, 0xde, 0xd9, 0xf9, 0xc7, 0x3a, 0x34, 0xbe, 0x93, - 0x69, 0xcc, 0x3e, 0x83, 0xb6, 0xe9, 0x0c, 0xb1, 0x4a, 0x07, 0x68, 0x83, 0xde, 0xd7, 0x95, 0x96, - 0x11, 0xad, 0xd2, 0xd3, 0x11, 0xba, 0xac, 0x29, 0x59, 0xd9, 0xb8, 0x7a, 0x65, 0x53, 0x5f, 0x43, - 0x6f, 0x98, 0xa5, 0x52, 0xcc, 0x2a, 0xea, 0xcb, 0x30, 0xdd, 0x54, 0xa0, 0x12, 0x5a, 0x9f, 0x42, - 0x4b, 0x07, 0x99, 0x95, 0x01, 0xab, 0xb5, 0x26, 0x29, 0x3f, 0x82, 0xce, 0xf0, 0x22, 0x9e, 0x87, - 0xfe, 0x50, 0xa6, 0x57, 0x92, 0x55, 0xba, 0xb3, 0x1b, 0x95, 0x6f, 0xf7, 0x0e, 0xdb, 0x02, 0xd0, - 0x5e, 0x76, 0x16, 0xf8, 0x8a, 0xb5, 0x51, 0x76, 0x3c, 0x9f, 0xe9, 0x49, 0x2b, 0xee, 0xa7, 0x35, - 0x2b, 0xb1, 0xe6, 0x75, 0x9a, 0x5f, 0xc2, 0xda, 0x1e, 0xc5, 0xe2, 0x93, 0x74, 0x77, 0x12, 0xa7, - 0x19, 0x5b, 0xed, 0xd0, 0x6e, 0xac, 0x32, 0xdc, 0x3b, 0xec, 0x29, 0xd8, 0xa3, 0xf4, 0x5a, 0xeb, - 0xbf, 0x65, 0x42, 0x74, 0xb9, 0xde, 0x0d, 0xa7, 0xdc, 0xf9, 0xfb, 0x3a, 0xb4, 0x7e, 0x19, 0xa7, - 0x97, 0x32, 0x65, 0x8f, 0xa1, 0x45, 0x4d, 0x01, 0x63, 0x44, 0x45, 0x83, 0xe0, 0xa6, 0x85, 0x3e, - 0x04, 0x87, 0x40, 0x19, 0x09, 0x75, 0xa9, 0xaf, 0x8a, 0xfe, 0xb0, 0xd5, 0xb8, 0xe8, 0xe4, 0x8d, - 0xee, 0x75, 0x5d, 0x5f, 0x54, 0xd1, 0x08, 0x59, 0xaa, 0xd4, 0x37, 0xda, 0xba, 0xec, 0x1e, 0xba, - 0x77, 0xb6, 0xac, 0xa7, 0x16, 0xfb, 0x04, 0x1a, 0x43, 0x7d, 0x52, 0x54, 0x2a, 0xff, 0xd4, 0xda, - 0x58, 0xcf, 0x19, 0xc5, 0xcc, 0x4f, 0xa0, 0xa5, 0x5f, 0x76, 0x7d, 0xcc, 0xa5, 0xc4, 0x74, 0xa3, - 0x57, 0x65, 0x99, 0x01, 0x9f, 0x40, 0x4b, 0x07, 0x0d, 0x3d, 0x60, 0x29, 0x80, 0xe8, 0x5d, 0xeb, - 0x18, 0xa4, 0x55, 0xb5, 0x97, 0x6b, 0xd5, 0x25, 0x8f, 0x5f, 0x51, 0xfd, 0x1c, 0x7a, 0x5c, 0x7a, - 0x32, 0xa8, 0xbc, 0xf9, 0x2c, 0x3f, 0xd4, 0xaa, 0xd9, 0x6e, 0x59, 0xec, 0x6b, 0x58, 0x5b, 0xca, - 0x0f, 0x58, 0x9f, 0x80, 0xbe, 0x21, 0x65, 0x58, 0x1d, 0xfc, 0xac, 0xf7, 0x2f, 0x3f, 0x3c, 0xb0, - 0xfe, 0xf5, 0x87, 0x07, 0xd6, 0x7f, 0xfc, 0xf0, 0xc0, 0xfa, 0xf5, 0x7f, 0x3e, 0xb8, 0x33, 0x69, - 0xd1, 0x1f, 0xfd, 0x5f, 0xfe, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x59, 0x4a, 0xb6, 0x37, 0x2c, - 0x20, 0x00, 0x00, + // 3336 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0xcd, 0x6f, 0x1c, 0x47, + 0x76, 0x67, 0xcf, 0x47, 0x4f, 0xf7, 0x9b, 0x21, 0x35, 0x2e, 0xc9, 0xf2, 0x98, 0xb2, 0x29, 0xba, + 0x2d, 0x59, 0x94, 0x64, 0x51, 0x32, 0xed, 0x24, 0x96, 0x83, 0x1c, 0xf8, 0x31, 0x52, 0x68, 0xf3, + 0x2b, 0x35, 0x43, 0x39, 0x31, 0x82, 0x0c, 0x6a, 0xba, 0x8b, 0xc3, 0x36, 0x7b, 0xba, 0x3b, 0x5d, + 0x3d, 0xc4, 0x50, 0xb7, 0x1c, 0x92, 0xbf, 0xc1, 0x87, 0x20, 0xb7, 0x20, 0x40, 0x2e, 0xb9, 0xfa, + 0x98, 0xcb, 0x02, 0x7b, 0x5c, 0xec, 0x5f, 0xb0, 0xd0, 0xee, 0x7f, 0xb0, 0xc0, 0x02, 0x7b, 0x5b, + 0xd4, 0xab, 0xea, 0x8f, 0x19, 0x91, 0x92, 0xbd, 0xc0, 0x9e, 0xa6, 0xdf, 0x47, 0x7d, 0xbd, 0xfa, + 0xbd, 0x57, 0xef, 0xbd, 0x01, 0x2b, 0x1e, 0xae, 0xc7, 0x49, 0x94, 0x46, 0xa4, 0x12, 0x0f, 0x97, + 0x6d, 0x16, 0xfb, 0x8a, 0x5c, 0xbe, 0x37, 0xf2, 0xd3, 0xd3, 0xc9, 0x70, 0xdd, 0x8d, 0xc6, 0x8f, + 0xbd, 0x51, 0xc2, 0xe2, 0xd3, 0x47, 0x7e, 0xf4, 0x78, 0xc8, 0xbc, 0x11, 0x4f, 0x1e, 0xc7, 0xc3, + 0xc7, 0xd9, 0x38, 0x67, 0x19, 0x6a, 0x7b, 0xbe, 0x48, 0x09, 0x81, 0xda, 0xc4, 0xf7, 0x44, 0xc7, + 0x58, 0xad, 0xae, 0x99, 0x14, 0xbf, 0x9d, 0x7d, 0xb0, 0xfb, 0x4c, 0x9c, 0xbd, 0x60, 0xc1, 0x84, + 0x93, 0x36, 0x54, 0xcf, 0x59, 0xd0, 0x31, 0x56, 0x8d, 0xb5, 0x16, 0x95, 0x9f, 0x64, 0x1d, 0xac, + 0x73, 0x16, 0x0c, 0xd2, 0x8b, 0x98, 0x77, 0x2a, 0xab, 0xc6, 0xda, 0xd2, 0xc6, 0xf5, 0xf5, 0x78, + 0xb8, 0x7e, 0x14, 0x89, 0xd4, 0x0f, 0x47, 0xeb, 0x2f, 0x58, 0xd0, 0xbf, 0x88, 0x39, 0x6d, 0x9c, + 0xab, 0x0f, 0xe7, 0x10, 0x9a, 0xbd, 0xc4, 0x7d, 0x36, 0x09, 0xdd, 0xd4, 0x8f, 0x42, 0xb9, 0x62, + 0xc8, 0xc6, 0x1c, 0x67, 0xb4, 0x29, 0x7e, 0x4b, 0x1e, 0x4b, 0x46, 0xa2, 0x53, 0x5d, 0xad, 0x4a, + 0x9e, 0xfc, 0x26, 0x1d, 0x68, 0xf8, 0x62, 0x3b, 0x9a, 0x84, 0x69, 0xa7, 0xb6, 0x6a, 0xac, 0x59, + 0x34, 0x23, 0x9d, 0xdf, 0x57, 0xa0, 0xfe, 0x0f, 0x13, 0x9e, 0x5c, 0xe0, 0xb8, 0x34, 0x4d, 0xb2, + 0xb9, 0xe4, 0x37, 0xb9, 0x01, 0xf5, 0x80, 0x85, 0x23, 0xd1, 0xa9, 0xe0, 0x64, 0x8a, 0x20, 0xb7, + 0xc0, 0x66, 0x27, 0x29, 0x4f, 0x06, 0x13, 0xdf, 0xeb, 0x54, 0x57, 0x8d, 0x35, 0x93, 0x5a, 0xc8, + 0x38, 0xf6, 0x3d, 0xf2, 0x3e, 0x58, 0x5e, 0x34, 0x70, 0xcb, 0x6b, 0x79, 0x11, 0xae, 0x45, 0x3e, + 0x06, 0x6b, 0xe2, 0x7b, 0x83, 0xc0, 0x17, 0x69, 0xa7, 0xbe, 0x6a, 0xac, 0x35, 0x37, 0x2c, 0x79, + 0x58, 0x69, 0x3b, 0xda, 0x98, 0xf8, 0x1e, 0x1a, 0xf1, 0x01, 0x58, 0x22, 0x71, 0x07, 0x27, 0x93, + 0xd0, 0xed, 0x98, 0xa8, 0x74, 0x4d, 0x2a, 0x95, 0x4e, 0x4d, 0x1b, 0x42, 0x11, 0xf2, 0x58, 0x09, + 0x3f, 0xe7, 0x89, 0xe0, 0x9d, 0x86, 0x5a, 0x4a, 0x93, 0xe4, 0x09, 0x34, 0x4f, 0x98, 0xcb, 0xd3, + 0x41, 0xcc, 0x12, 0x36, 0xee, 0x58, 0xc5, 0x44, 0xcf, 0x24, 0xfb, 0x48, 0x72, 0x05, 0x85, 0x93, + 0x9c, 0x20, 0x9f, 0xc3, 0x22, 0x52, 0x62, 0x70, 0xe2, 0x07, 0x29, 0x4f, 0x3a, 0x36, 0x8e, 0x59, + 0xc2, 0x31, 0xc8, 0xe9, 0x27, 0x9c, 0xd3, 0x96, 0x52, 0x52, 0x1c, 0xf2, 0x21, 0x00, 0x9f, 0xc6, + 0x2c, 0xf4, 0x06, 0x2c, 0x08, 0x3a, 0x80, 0x7b, 0xb0, 0x15, 0x67, 0x33, 0x08, 0xc8, 0x7b, 0x72, + 0x7f, 0xcc, 0x1b, 0xa4, 0xa2, 0xb3, 0xb8, 0x6a, 0xac, 0xd5, 0xa8, 0x29, 0xc9, 0xbe, 0x70, 0x36, + 0xc0, 0x46, 0x44, 0xe0, 0x89, 0xef, 0x82, 0x79, 0x2e, 0x09, 0x05, 0x9c, 0xe6, 0xc6, 0xa2, 0x5c, + 0x32, 0x07, 0x0d, 0xd5, 0x42, 0x67, 0x05, 0xac, 0x3d, 0x16, 0x8e, 0x32, 0xa4, 0xc9, 0xab, 0xc0, + 0x01, 0x36, 0xc5, 0x6f, 0xe7, 0x87, 0x0a, 0x98, 0x94, 0x8b, 0x49, 0x90, 0x92, 0x7b, 0x00, 0xd2, + 0xd0, 0x63, 0x96, 0x26, 0xfe, 0x54, 0xcf, 0x5a, 0x98, 0xda, 0x9e, 0xf8, 0xde, 0x3e, 0x8a, 0xc8, + 0x13, 0x68, 0xe1, 0xec, 0x99, 0x6a, 0xa5, 0xd8, 0x40, 0xbe, 0x3f, 0xda, 0x44, 0x15, 0x3d, 0xe2, + 0x26, 0x98, 0x78, 0xb7, 0x0a, 0x5f, 0x8b, 0x54, 0x53, 0xe4, 0x2e, 0x2c, 0xf9, 0x61, 0x2a, 0x6d, + 0xef, 0xa6, 0x03, 0x8f, 0x8b, 0xec, 0xf2, 0x17, 0x73, 0xee, 0x0e, 0x17, 0x29, 0xf9, 0x0c, 0x94, + 0x01, 0xb3, 0x05, 0xeb, 0xb8, 0xe0, 0x52, 0x7e, 0x31, 0x42, 0xad, 0x88, 0x3a, 0x7a, 0xc5, 0x47, + 0xd0, 0x94, 0xe7, 0xcb, 0x46, 0x98, 0x38, 0xa2, 0x85, 0xa7, 0xd1, 0xe6, 0xa0, 0x20, 0x15, 0xb4, + 0xba, 0x34, 0x8d, 0x04, 0x98, 0x02, 0x04, 0x7e, 0x3b, 0x5d, 0xa8, 0x1f, 0x26, 0x1e, 0x4f, 0x2e, + 0xc5, 0x38, 0x81, 0x9a, 0xc7, 0x85, 0x8b, 0xee, 0x67, 0x51, 0xfc, 0x2e, 0x70, 0x5f, 0x2d, 0xe1, + 0xde, 0xf9, 0x2f, 0x03, 0x9a, 0xbd, 0x28, 0x49, 0xf7, 0xb9, 0x10, 0x6c, 0xc4, 0xc9, 0x6d, 0xa8, + 0x47, 0x72, 0x5a, 0x6d, 0x61, 0x5b, 0xee, 0x09, 0xd7, 0xa1, 0x8a, 0x3f, 0x77, 0x0f, 0x95, 0xab, + 0xef, 0xe1, 0x06, 0xd4, 0x95, 0xc7, 0x48, 0x6f, 0xaa, 0x53, 0x45, 0x48, 0x5b, 0x47, 0x27, 0x27, + 0x82, 0x2b, 0x5b, 0xd6, 0xa9, 0xa6, 0xae, 0x86, 0xd5, 0x5f, 0x01, 0xc8, 0xfd, 0xfd, 0x4c, 0x14, + 0x38, 0xa7, 0xd0, 0xa4, 0xec, 0x24, 0xdd, 0x8e, 0xc2, 0x94, 0x4f, 0x53, 0xb2, 0x04, 0x15, 0xdf, + 0x43, 0x13, 0x99, 0xb4, 0xe2, 0x7b, 0x72, 0x73, 0xa3, 0x24, 0x9a, 0xc4, 0x68, 0xa1, 0x45, 0xaa, + 0x08, 0x34, 0xa5, 0xe7, 0x25, 0xb8, 0x63, 0x69, 0x4a, 0xcf, 0x4b, 0xc8, 0x6d, 0x68, 0x8a, 0x90, + 0xc5, 0xe2, 0x34, 0x4a, 0xe5, 0xe6, 0x6a, 0xb8, 0x39, 0xc8, 0x58, 0x7d, 0xe1, 0xfc, 0xc2, 0x00, + 0x73, 0x9f, 0x8f, 0x87, 0x3c, 0x79, 0x6d, 0x95, 0xf7, 0xc1, 0xc2, 0x89, 0x07, 0xbe, 0xa7, 0x17, + 0x6a, 0x20, 0xbd, 0xeb, 0x5d, 0xba, 0xd4, 0x4d, 0x30, 0x03, 0xce, 0xa4, 0xf1, 0x15, 0xce, 0x34, + 0x25, 0x6d, 0xc3, 0xc6, 0x03, 0x8f, 0x33, 0x0f, 0x43, 0x8c, 0x45, 0x4d, 0x36, 0xde, 0xe1, 0xcc, + 0x93, 0x7b, 0x0b, 0x98, 0x48, 0x07, 0x93, 0xd8, 0x63, 0x29, 0xc7, 0xd0, 0x52, 0x93, 0xc0, 0x11, + 0xe9, 0x31, 0x72, 0xc8, 0x03, 0x78, 0xc7, 0x0d, 0x26, 0x42, 0xc6, 0x35, 0x3f, 0x3c, 0x89, 0x06, + 0x51, 0x18, 0x5c, 0xa0, 0x7d, 0x2d, 0x7a, 0x4d, 0x0b, 0x76, 0xc3, 0x93, 0xe8, 0x30, 0x0c, 0x2e, + 0x9c, 0x1f, 0x2b, 0x50, 0x7f, 0x8e, 0x66, 0x78, 0x02, 0x8d, 0x31, 0x1e, 0x28, 0xf3, 0xde, 0x9b, + 0xd2, 0xc2, 0x28, 0x5b, 0x57, 0x27, 0x15, 0xdd, 0x30, 0x4d, 0x2e, 0x68, 0xa6, 0x26, 0x47, 0xa4, + 0x6c, 0x18, 0xf0, 0x54, 0x68, 0x44, 0x94, 0x46, 0xf4, 0x95, 0x40, 0x8f, 0xd0, 0x6a, 0xf3, 0x66, + 0xad, 0xce, 0x9b, 0x95, 0x2c, 0x83, 0xe5, 0x9e, 0x72, 0xf7, 0x4c, 0x4c, 0xc6, 0xda, 0xe8, 0x39, + 0xbd, 0xfc, 0x0c, 0x5a, 0xe5, 0x7d, 0xc8, 0x37, 0xe8, 0x8c, 0x5f, 0xa0, 0xe1, 0x6b, 0x54, 0x7e, + 0x92, 0x55, 0xa8, 0xa3, 0x87, 0xa3, 0xd9, 0x9b, 0x1b, 0x20, 0xb7, 0xa3, 0x86, 0x50, 0x25, 0xf8, + 0xaa, 0xf2, 0xa5, 0x21, 0xe7, 0x29, 0xef, 0xae, 0x3c, 0x8f, 0x7d, 0xf5, 0x3c, 0x6a, 0x48, 0x69, + 0x1e, 0xe7, 0x8f, 0x15, 0x68, 0x7d, 0xc7, 0x93, 0xe8, 0x28, 0x89, 0xe2, 0x48, 0xb0, 0x80, 0x6c, + 0xce, 0x9e, 0x4e, 0x59, 0x71, 0x55, 0x0e, 0x2e, 0xab, 0xad, 0xf7, 0xf2, 0xe3, 0x2a, 0xeb, 0x94, + 0xcf, 0xef, 0x80, 0xa9, 0xac, 0x7b, 0xc9, 0x11, 0xb4, 0x44, 0xea, 0x28, 0x7b, 0xa2, 0xfd, 0x66, + 0xb7, 0xa7, 0x25, 0x64, 0x05, 0x60, 0xcc, 0xa6, 0x7b, 0x9c, 0x09, 0xbe, 0xeb, 0x65, 0xf0, 0x2d, + 0x38, 0xd2, 0xce, 0x63, 0x36, 0xed, 0x4f, 0xc3, 0xbe, 0x40, 0x74, 0xd5, 0x68, 0x4e, 0x93, 0x0f, + 0xc0, 0x1e, 0xb3, 0xa9, 0xf4, 0xa3, 0x5d, 0x4f, 0xa3, 0xab, 0x60, 0x90, 0x8f, 0xa0, 0x9a, 0x4e, + 0x43, 0x0c, 0x4a, 0xf2, 0x1d, 0x92, 0x49, 0x46, 0x7f, 0x1a, 0x6a, 0x8f, 0xa3, 0x52, 0x96, 0x19, + 0xd4, 0x2a, 0x0c, 0xda, 0x86, 0xaa, 0xeb, 0x7b, 0xf8, 0x10, 0xd9, 0x54, 0x7e, 0x2e, 0xff, 0x1d, + 0x5c, 0x9b, 0xb3, 0x43, 0xf9, 0x1e, 0x16, 0xd5, 0xb0, 0x1b, 0xe5, 0x7b, 0xa8, 0x95, 0x6d, 0xff, + 0x63, 0x15, 0xae, 0x69, 0x30, 0x9c, 0xfa, 0x71, 0x2f, 0x95, 0xb0, 0xef, 0x40, 0x03, 0xa3, 0x0d, + 0x4f, 0x34, 0x26, 0x32, 0x92, 0xfc, 0x0d, 0x98, 0xe8, 0x81, 0x19, 0x4e, 0x6f, 0x17, 0x56, 0xcd, + 0x87, 0x2b, 0xdc, 0xea, 0x2b, 0xd1, 0xea, 0xe4, 0x0b, 0xa8, 0xbf, 0xe4, 0x49, 0xa4, 0xa2, 0x67, + 0x73, 0x63, 0xe5, 0xb2, 0x71, 0xf2, 0x6e, 0xf5, 0x30, 0xa5, 0xfc, 0x17, 0x34, 0xfe, 0x1d, 0x19, + 0x2f, 0xc7, 0xd1, 0x39, 0xf7, 0x3a, 0x0d, 0xdc, 0x51, 0x19, 0x1f, 0x99, 0x28, 0xb3, 0xb6, 0x55, + 0x58, 0x7b, 0x07, 0x9a, 0xa5, 0xe3, 0x5d, 0x62, 0xe9, 0xdb, 0xb3, 0x88, 0xb7, 0x73, 0x47, 0x2e, + 0x3b, 0xce, 0x0e, 0x40, 0x71, 0xd8, 0x3f, 0xd7, 0xfd, 0x9c, 0x7f, 0x33, 0xe0, 0xda, 0x76, 0x14, + 0x86, 0x1c, 0x53, 0x20, 0x75, 0x75, 0x05, 0xec, 0x8d, 0x2b, 0x61, 0x7f, 0x1f, 0xea, 0x42, 0x2a, + 0xeb, 0xd9, 0xaf, 0x5f, 0x72, 0x17, 0x54, 0x69, 0xc8, 0x30, 0x33, 0x66, 0xd3, 0x41, 0xcc, 0x43, + 0xcf, 0x0f, 0x47, 0x59, 0x98, 0x19, 0xb3, 0xe9, 0x91, 0xe2, 0x38, 0xff, 0x6e, 0x80, 0xa9, 0x3c, + 0x66, 0x26, 0x5a, 0x1b, 0xb3, 0xd1, 0xfa, 0x03, 0xb0, 0xe3, 0x84, 0x7b, 0xbe, 0x9b, 0xad, 0x6a, + 0xd3, 0x82, 0x21, 0xc1, 0x79, 0x12, 0x25, 0x2e, 0xc7, 0xe9, 0x2d, 0xaa, 0x08, 0xc9, 0x15, 0x31, + 0x73, 0x55, 0x1a, 0x57, 0xa5, 0x8a, 0x90, 0x31, 0x5e, 0x5d, 0x0e, 0x5e, 0x8a, 0x45, 0x35, 0xe5, + 0xfc, 0x6f, 0x05, 0x5a, 0x3b, 0x7e, 0xc2, 0xdd, 0x94, 0x7b, 0x5d, 0x6f, 0x84, 0x8a, 0x3c, 0x4c, + 0xfd, 0xf4, 0x42, 0xbf, 0x27, 0x9a, 0xca, 0x9f, 0xfb, 0xca, 0x6c, 0x4a, 0xab, 0xcc, 0x5d, 0xc5, + 0x2c, 0x5c, 0x11, 0x64, 0x03, 0x40, 0x25, 0x42, 0x98, 0x89, 0xd7, 0xae, 0xce, 0xc4, 0x6d, 0x54, + 0x93, 0x9f, 0xd2, 0x06, 0x6a, 0x8c, 0xaf, 0xde, 0x1a, 0x13, 0xd3, 0xf4, 0x89, 0xc4, 0x2a, 0xe6, + 0x0f, 0x43, 0x1e, 0x20, 0x16, 0x31, 0x7f, 0x18, 0xf2, 0x20, 0xcf, 0xda, 0x1a, 0x6a, 0x3b, 0xf2, + 0x9b, 0x7c, 0x0c, 0x95, 0x28, 0xc6, 0xf3, 0xe9, 0x05, 0xcb, 0x07, 0x5b, 0x3f, 0x8c, 0x69, 0x25, + 0x8a, 0xe5, 0x45, 0xab, 0xb4, 0xb3, 0x63, 0x6b, 0xfc, 0xca, 0x00, 0x82, 0x09, 0x13, 0xd5, 0x12, + 0xe7, 0x26, 0x54, 0x0e, 0x63, 0xd2, 0x80, 0x6a, 0xaf, 0xdb, 0x6f, 0x2f, 0xc8, 0x8f, 0x9d, 0xee, + 0x5e, 0xdb, 0x70, 0x5e, 0x19, 0x60, 0xef, 0x4f, 0x52, 0x26, 0x61, 0x23, 0xde, 0x74, 0x6f, 0xef, + 0x83, 0x25, 0x52, 0x96, 0x60, 0x10, 0x56, 0x91, 0xa3, 0x81, 0x74, 0x5f, 0x90, 0x4f, 0xa0, 0xce, + 0xbd, 0x11, 0xcf, 0x1c, 0xba, 0x3d, 0xbf, 0x4f, 0xaa, 0xc4, 0x64, 0x0d, 0x4c, 0xe1, 0x9e, 0xf2, + 0x31, 0xeb, 0xd4, 0x0a, 0xc5, 0x1e, 0x72, 0xd4, 0x23, 0x4b, 0xb5, 0x1c, 0xab, 0x84, 0x24, 0x8a, + 0x31, 0x6d, 0xae, 0xeb, 0x2a, 0x21, 0x89, 0x62, 0x99, 0x34, 0x6f, 0xc0, 0xbb, 0xfe, 0x28, 0x8c, + 0x12, 0x3e, 0xf0, 0x43, 0x8f, 0x4f, 0x07, 0x6e, 0x14, 0x9e, 0x04, 0xbe, 0x9b, 0xa2, 0x2d, 0x2d, + 0x7a, 0x5d, 0x09, 0x77, 0xa5, 0x6c, 0x5b, 0x8b, 0x9c, 0x29, 0x58, 0x59, 0x5c, 0x24, 0xf7, 0x65, + 0x40, 0xc3, 0xb8, 0xaa, 0xdd, 0x02, 0xd3, 0xfe, 0x52, 0x82, 0x43, 0x33, 0xb9, 0xbc, 0x26, 0x5c, + 0x23, 0x8b, 0x94, 0x48, 0x94, 0xd3, 0xab, 0x6a, 0x39, 0xbd, 0xc2, 0x4c, 0x31, 0x0a, 0xb9, 0xce, + 0x38, 0xf0, 0xdb, 0xf9, 0xcf, 0x0a, 0x58, 0xf9, 0x53, 0xf6, 0x10, 0xec, 0x71, 0x66, 0x6a, 0xed, + 0x70, 0x98, 0x4b, 0xe7, 0xf6, 0xa7, 0x85, 0x9c, 0xdc, 0x84, 0xca, 0xd9, 0xb9, 0x36, 0x94, 0x29, + 0xb5, 0xbe, 0x79, 0x41, 0x2b, 0x67, 0xe7, 0x85, 0xc7, 0xd6, 0xdf, 0xea, 0xb1, 0xf7, 0xe0, 0x9a, + 0x1b, 0x70, 0x16, 0x0e, 0x0a, 0x87, 0x53, 0x80, 0x5b, 0x42, 0xf6, 0x51, 0xee, 0x75, 0x3a, 0xea, + 0x34, 0x8a, 0xb7, 0xe5, 0x2e, 0xd4, 0x3d, 0x1e, 0xa4, 0xac, 0x5c, 0x1a, 0x1d, 0x26, 0xcc, 0x0d, + 0xf8, 0x8e, 0x64, 0x53, 0x25, 0x25, 0x6b, 0x60, 0x65, 0xef, 0xac, 0x2e, 0x88, 0x30, 0xf3, 0xce, + 0x8c, 0x4d, 0x73, 0x69, 0x61, 0x4b, 0x28, 0xd9, 0xd2, 0xf9, 0x0c, 0xaa, 0xdf, 0xbc, 0xe8, 0xe9, + 0xb3, 0x1a, 0xaf, 0x9d, 0x35, 0xb3, 0x68, 0xa5, 0x64, 0xd1, 0x3f, 0x54, 0xa1, 0xa1, 0xbd, 0x4e, + 0xee, 0x7b, 0x92, 0x67, 0x89, 0xf2, 0x73, 0xf6, 0x71, 0xcb, 0xdd, 0xb7, 0x5c, 0x46, 0x57, 0xdf, + 0x5e, 0x46, 0x93, 0xaf, 0xa0, 0x15, 0x2b, 0x59, 0xd9, 0xe1, 0xdf, 0x2b, 0x8f, 0xd1, 0xbf, 0x38, + 0xae, 0x19, 0x17, 0x84, 0x84, 0x2e, 0xd6, 0x23, 0x29, 0x1b, 0xe1, 0x15, 0xb5, 0x68, 0x43, 0xd2, + 0x7d, 0x36, 0xba, 0xc2, 0xed, 0x7f, 0x82, 0xf7, 0xca, 0x6c, 0x38, 0x8a, 0x3b, 0x2d, 0xf4, 0x48, + 0xe9, 0xf1, 0x65, 0x67, 0x5c, 0x9c, 0x75, 0xc6, 0x5b, 0x60, 0xbb, 0xd1, 0x78, 0xec, 0xa3, 0x6c, + 0x49, 0x67, 0x7b, 0xc8, 0xe8, 0x0b, 0xe7, 0x3f, 0x0c, 0x68, 0xe8, 0xd3, 0x92, 0x26, 0x34, 0x76, + 0xba, 0xcf, 0x36, 0x8f, 0xf7, 0x64, 0x3c, 0x00, 0x30, 0xb7, 0x76, 0x0f, 0x36, 0xe9, 0x3f, 0xb5, + 0x0d, 0x19, 0x1b, 0x76, 0x0f, 0xfa, 0xed, 0x0a, 0xb1, 0xa1, 0xfe, 0x6c, 0xef, 0x70, 0xb3, 0xdf, + 0xae, 0x12, 0x0b, 0x6a, 0x5b, 0x87, 0x87, 0x7b, 0xed, 0x1a, 0x69, 0x81, 0xb5, 0xb3, 0xd9, 0xef, + 0xf6, 0x77, 0xf7, 0xbb, 0xed, 0xba, 0xd4, 0x7d, 0xde, 0x3d, 0x6c, 0x9b, 0xf2, 0xe3, 0x78, 0x77, + 0xa7, 0xdd, 0x90, 0xf2, 0xa3, 0xcd, 0x5e, 0xef, 0xdb, 0x43, 0xba, 0xd3, 0xb6, 0xe4, 0xbc, 0xbd, + 0x3e, 0xdd, 0x3d, 0x78, 0xde, 0xb6, 0xe5, 0xf7, 0xe1, 0xd6, 0xd7, 0xdd, 0xed, 0x7e, 0x1b, 0x9c, + 0xcf, 0xa0, 0x59, 0xb2, 0xa0, 0x1c, 0x4d, 0xbb, 0xcf, 0xda, 0x0b, 0x72, 0xc9, 0x17, 0x9b, 0x7b, + 0xc7, 0xdd, 0xb6, 0x41, 0x96, 0x00, 0xf0, 0x73, 0xb0, 0xb7, 0x79, 0xf0, 0xbc, 0x5d, 0x71, 0xfe, + 0x1a, 0xac, 0x63, 0xdf, 0xdb, 0x0a, 0x22, 0xf7, 0x4c, 0x02, 0x63, 0xc8, 0x04, 0xd7, 0xef, 0x24, + 0x7e, 0xcb, 0x28, 0x8f, 0xa0, 0x14, 0xfa, 0xee, 0x35, 0xe5, 0x1c, 0x40, 0xe3, 0xd8, 0xf7, 0x8e, + 0x98, 0x7b, 0x26, 0xeb, 0xf1, 0xa1, 0x1c, 0x3f, 0x10, 0xfe, 0x4b, 0xae, 0x03, 0x9c, 0x8d, 0x9c, + 0x9e, 0xff, 0x92, 0x93, 0x3b, 0x60, 0x22, 0x91, 0x65, 0x34, 0x88, 0xe5, 0x6c, 0x4d, 0xaa, 0x65, + 0xce, 0xff, 0x18, 0xf9, 0xde, 0xb1, 0xd8, 0xbe, 0x0d, 0xb5, 0x98, 0xb9, 0x67, 0x3a, 0x9a, 0x34, + 0xf5, 0x18, 0xb9, 0x1e, 0x45, 0x01, 0xb9, 0x07, 0x96, 0x06, 0x48, 0x36, 0x71, 0xb3, 0x84, 0x24, + 0x9a, 0x0b, 0x67, 0xaf, 0xae, 0x3a, 0x7b, 0x75, 0x12, 0x3c, 0x31, 0x4b, 0xb0, 0x6c, 0xaa, 0x4a, + 0x07, 0x42, 0x42, 0x0e, 0x51, 0x40, 0x98, 0xe8, 0x57, 0xa6, 0x46, 0x15, 0x32, 0x8e, 0x7d, 0xcf, + 0xf9, 0x02, 0xa0, 0x68, 0x67, 0x5c, 0x92, 0x91, 0xdf, 0x80, 0x3a, 0x0b, 0x7c, 0x6d, 0x30, 0x9b, + 0x2a, 0xc2, 0x39, 0x80, 0x66, 0xa9, 0x09, 0x22, 0xa1, 0xc6, 0x82, 0x60, 0x70, 0xc6, 0x2f, 0x04, + 0x8e, 0xb5, 0x68, 0x83, 0x05, 0xc1, 0x37, 0xfc, 0x42, 0x90, 0x3b, 0xb8, 0x25, 0x36, 0xd6, 0xa7, + 0x5a, 0x9a, 0xed, 0x9f, 0x50, 0x25, 0x74, 0x3e, 0x05, 0x53, 0xd5, 0xee, 0x25, 0xa4, 0x1b, 0x57, + 0xbe, 0x53, 0x4f, 0xf5, 0x9e, 0xb1, 0xd2, 0x27, 0x0f, 0x75, 0x9f, 0x46, 0xa8, 0xae, 0x90, 0x51, + 0xa4, 0x67, 0x4a, 0x49, 0xb7, 0x68, 0x50, 0xd9, 0xd9, 0x01, 0xeb, 0x8d, 0x9d, 0x2f, 0x6d, 0x80, + 0x4a, 0x61, 0x80, 0x4b, 0x7a, 0x61, 0xce, 0xf7, 0x00, 0x45, 0x3f, 0x47, 0x3b, 0x9e, 0x9a, 0x45, + 0x3a, 0xde, 0x03, 0x59, 0x4a, 0xf9, 0x81, 0x97, 0xf0, 0x70, 0xe6, 0xd4, 0x45, 0x07, 0x28, 0x97, + 0x93, 0x55, 0xa8, 0x61, 0x9b, 0xaa, 0x5a, 0x04, 0xc6, 0xbc, 0x47, 0x85, 0x12, 0x67, 0x08, 0x8b, + 0xea, 0xf9, 0xa3, 0xfc, 0x5f, 0x27, 0x5c, 0xbc, 0x31, 0x6f, 0x5a, 0x01, 0xc8, 0xc3, 0x78, 0xd6, + 0x70, 0x2b, 0x71, 0x24, 0xfc, 0x4f, 0x7c, 0x1e, 0x78, 0xd9, 0x69, 0x34, 0xe5, 0xfc, 0x2d, 0xb4, + 0xb2, 0x35, 0xb0, 0xec, 0x7f, 0x98, 0x3f, 0xc2, 0xca, 0x9a, 0xaa, 0xda, 0x50, 0x2a, 0x07, 0x91, + 0xc7, 0xb7, 0x2a, 0x1d, 0x23, 0x7b, 0x87, 0x9d, 0x5f, 0x57, 0xb3, 0xd1, 0xba, 0x0a, 0x9e, 0xc9, + 0xde, 0x8c, 0xf9, 0xec, 0x6d, 0x36, 0x4d, 0xaa, 0xfc, 0xa4, 0x34, 0xe9, 0x4b, 0xb0, 0x3d, 0xcc, + 0x15, 0xfc, 0xf3, 0x2c, 0x38, 0x2f, 0xcf, 0xe7, 0x05, 0x3a, 0x9b, 0xf0, 0xcf, 0x39, 0x2d, 0x94, + 0xe5, 0x5e, 0xd2, 0xe8, 0x8c, 0x87, 0xfe, 0x4b, 0x2c, 0xf3, 0xe5, 0xa1, 0x0b, 0x46, 0xd1, 0x33, + 0x51, 0xf9, 0x83, 0xee, 0x99, 0x64, 0xed, 0x1f, 0xb3, 0x68, 0xff, 0x48, 0xcb, 0x4d, 0x62, 0xc1, + 0x93, 0x34, 0xcb, 0x23, 0x15, 0x95, 0xe7, 0x63, 0xb6, 0xd6, 0x95, 0xf9, 0xd8, 0x47, 0xd0, 0x0a, + 0xa3, 0x70, 0x10, 0x4e, 0x82, 0x40, 0x66, 0xba, 0xba, 0xa7, 0xd7, 0x0c, 0xa3, 0xf0, 0x40, 0xb3, + 0xc8, 0x03, 0x78, 0xa7, 0xac, 0xa2, 0x90, 0xdb, 0x54, 0x8d, 0x82, 0x92, 0x1e, 0xe2, 0x7b, 0x0d, + 0xda, 0xd1, 0xf0, 0x7b, 0xee, 0xa6, 0x68, 0xb1, 0x01, 0x42, 0xb6, 0xa5, 0x9e, 0x68, 0xc5, 0x97, + 0x26, 0x3a, 0x60, 0x63, 0xee, 0x3c, 0x05, 0x3b, 0x37, 0x82, 0x8c, 0xc6, 0x07, 0x87, 0x07, 0x5d, + 0x15, 0x2f, 0x77, 0x0f, 0x76, 0xba, 0xff, 0xd8, 0x36, 0x64, 0x3c, 0xa7, 0xdd, 0x17, 0x5d, 0xda, + 0xeb, 0xb6, 0x2b, 0x32, 0xd6, 0xee, 0x74, 0xf7, 0xba, 0xfd, 0x6e, 0xbb, 0xfa, 0x75, 0xcd, 0x6a, + 0xb4, 0x2d, 0x6a, 0xf1, 0x69, 0x1c, 0xf8, 0xae, 0x9f, 0x3a, 0x3d, 0x00, 0x39, 0xad, 0xbe, 0xd1, + 0x5b, 0x60, 0x17, 0x6b, 0xab, 0x1b, 0xb5, 0x52, 0xbd, 0xaa, 0xcc, 0xd8, 0x34, 0xa8, 0x2a, 0x57, + 0x65, 0x6c, 0x1a, 0x66, 0xc7, 0x60, 0xed, 0xb3, 0xf8, 0xb5, 0x22, 0xa6, 0x95, 0x97, 0xaa, 0x13, + 0xdd, 0xb8, 0xd1, 0x0f, 0xf5, 0x5d, 0x68, 0xe8, 0xb8, 0xa7, 0xfd, 0x63, 0x26, 0x26, 0x66, 0x32, + 0x59, 0x53, 0xdc, 0xd8, 0x8f, 0xce, 0x79, 0x9e, 0xab, 0x1c, 0xb1, 0x8b, 0x20, 0x62, 0xde, 0x5b, + 0x80, 0xf8, 0x21, 0x80, 0x88, 0x26, 0x89, 0xcb, 0x07, 0xa3, 0xbc, 0x5f, 0x64, 0x2b, 0xce, 0x73, + 0xdd, 0x84, 0xe6, 0x22, 0x45, 0x61, 0x55, 0xb9, 0x99, 0xa4, 0xa5, 0xe8, 0x5d, 0x30, 0xd3, 0x69, + 0x58, 0xb4, 0xa7, 0xea, 0xa9, 0xac, 0x20, 0x9d, 0x6d, 0xb0, 0xfb, 0x53, 0xac, 0xab, 0x26, 0x62, + 0xe6, 0xf5, 0x35, 0xde, 0xf0, 0xfa, 0x56, 0xe6, 0x5e, 0xdf, 0xdf, 0x19, 0xd0, 0x2c, 0x25, 0x51, + 0xe4, 0x23, 0xa8, 0xa5, 0xd3, 0x70, 0xb6, 0xaf, 0x9b, 0x2d, 0x42, 0x51, 0x24, 0xf1, 0x26, 0x8b, + 0x2e, 0x26, 0x84, 0x3f, 0x0a, 0xb9, 0xa7, 0xa7, 0x94, 0x85, 0xd8, 0xa6, 0x66, 0x91, 0x3d, 0xb8, + 0xa6, 0x62, 0x46, 0xd6, 0xd3, 0xc9, 0xf2, 0xf0, 0x8f, 0xe7, 0x92, 0x36, 0x55, 0x7b, 0x6e, 0x67, + 0x5a, 0xaa, 0xba, 0x5e, 0x1a, 0xcd, 0x30, 0x97, 0x37, 0xe1, 0xfa, 0x25, 0x6a, 0x3f, 0xab, 0x8d, + 0x70, 0x1b, 0x16, 0x65, 0xd9, 0xed, 0x8f, 0xb9, 0x48, 0xd9, 0x38, 0xc6, 0xec, 0x45, 0xc7, 0xfc, + 0x1a, 0xad, 0xa4, 0xc2, 0xf9, 0x04, 0x5a, 0x47, 0x9c, 0x27, 0x94, 0x8b, 0x38, 0x0a, 0xd5, 0xcb, + 0x2d, 0xf0, 0xd0, 0xfa, 0x81, 0xd1, 0x94, 0xf3, 0x2f, 0x60, 0xcb, 0xbc, 0x7c, 0x8b, 0xa5, 0xee, + 0xe9, 0xcf, 0xc9, 0xdb, 0x3f, 0x81, 0x46, 0xac, 0x60, 0xa2, 0xb3, 0xec, 0x16, 0xc6, 0x38, 0x0d, + 0x1d, 0x9a, 0x09, 0x9d, 0x2f, 0xa0, 0x7a, 0x30, 0x19, 0x97, 0xff, 0x76, 0xa9, 0xa9, 0xbf, 0x5d, + 0x6e, 0x81, 0x8d, 0x29, 0x3e, 0xf6, 0xf8, 0x54, 0xf2, 0x69, 0x49, 0x06, 0x36, 0xf7, 0xbe, 0x83, + 0x66, 0x66, 0xfb, 0x5d, 0x0f, 0xff, 0x3b, 0xc1, 0xcb, 0xdf, 0xf5, 0x66, 0xb0, 0xa0, 0xaa, 0x3c, + 0x1e, 0x7a, 0xbb, 0xd9, 0xa5, 0x29, 0x62, 0x76, 0x6e, 0xfd, 0x62, 0xe7, 0x73, 0x3f, 0x83, 0x56, + 0x96, 0x3b, 0xef, 0xf3, 0x94, 0x21, 0x9c, 0x02, 0x9f, 0x87, 0x25, 0xa8, 0x59, 0x8a, 0xd1, 0x17, + 0x6f, 0x68, 0x89, 0x3a, 0xeb, 0x60, 0x6a, 0xac, 0x12, 0xa8, 0xb9, 0x91, 0xa7, 0x5c, 0xa4, 0x4e, + 0xf1, 0x5b, 0x1e, 0x78, 0x2c, 0x46, 0xd9, 0x43, 0x38, 0x16, 0x23, 0x67, 0x0a, 0x8b, 0x5b, 0xcc, + 0x3d, 0x9b, 0xc4, 0xd9, 0x43, 0x54, 0x2a, 0x72, 0x8c, 0x99, 0x22, 0xe7, 0x0d, 0x7d, 0xd8, 0xf7, + 0xa0, 0x31, 0x09, 0xfd, 0x69, 0x96, 0xbc, 0xd8, 0xd4, 0x94, 0xa4, 0xea, 0x3f, 0x06, 0x91, 0x8b, + 0x75, 0x0d, 0x7a, 0x95, 0x4d, 0x73, 0xda, 0xf9, 0x67, 0x58, 0xec, 0x4e, 0x63, 0xec, 0x4a, 0xbf, + 0xf5, 0x09, 0x2c, 0x6d, 0xaa, 0x32, 0xb3, 0xa9, 0xb9, 0x95, 0xab, 0xd9, 0xca, 0x1b, 0xff, 0x6f, + 0x40, 0x4d, 0x42, 0x44, 0x96, 0x5e, 0x7f, 0xcf, 0x59, 0x92, 0x0e, 0x39, 0x4b, 0xc9, 0x0c, 0x1c, + 0x96, 0x67, 0x28, 0x67, 0xe1, 0x89, 0x41, 0x3e, 0x55, 0x0d, 0xef, 0xac, 0x8f, 0xbf, 0x98, 0x01, + 0x0d, 0x81, 0x38, 0xaf, 0x4f, 0xd6, 0xa1, 0xf9, 0x75, 0xe4, 0x87, 0xdb, 0xaa, 0x07, 0x4c, 0xe6, + 0x61, 0xf9, 0x9a, 0xfe, 0x23, 0x30, 0x77, 0x85, 0xc4, 0xff, 0xeb, 0xaa, 0x18, 0x5e, 0xcb, 0xae, + 0xe1, 0x2c, 0x6c, 0xfc, 0x5f, 0x15, 0x6a, 0xdf, 0xf1, 0x24, 0x22, 0x9f, 0x42, 0x43, 0x77, 0x78, + 0x48, 0xa9, 0x93, 0xb3, 0x8c, 0xef, 0xeb, 0x5c, 0xeb, 0x07, 0x57, 0x69, 0xab, 0x08, 0x5d, 0xd4, + 0x86, 0xa4, 0x68, 0x40, 0xbd, 0xb6, 0xa9, 0xa7, 0xd0, 0xee, 0xa5, 0x09, 0x67, 0xe3, 0x92, 0xfa, + 0xac, 0x99, 0x2e, 0x2b, 0x34, 0xd1, 0x5a, 0x0f, 0xc1, 0x54, 0x41, 0x66, 0x6e, 0xc0, 0x7c, 0xcd, + 0x88, 0xca, 0xf7, 0xa0, 0xd9, 0x3b, 0x8d, 0x26, 0x81, 0xd7, 0xe3, 0xc9, 0x39, 0x27, 0xa5, 0x2e, + 0xeb, 0x72, 0xe9, 0xdb, 0x59, 0x20, 0x6b, 0x00, 0xca, 0xcb, 0x8e, 0x7d, 0x4f, 0x90, 0x86, 0x94, + 0x1d, 0x4c, 0xc6, 0x6a, 0xd2, 0x92, 0xfb, 0x29, 0xcd, 0x52, 0xac, 0x79, 0x93, 0xe6, 0xe7, 0xb0, + 0xb8, 0x8d, 0xb1, 0xf8, 0x30, 0xd9, 0x1c, 0x46, 0x49, 0x4a, 0xe6, 0x3b, 0xad, 0xcb, 0xf3, 0x0c, + 0x67, 0x81, 0x3c, 0x01, 0xab, 0x9f, 0x5c, 0x28, 0xfd, 0x77, 0x74, 0x88, 0x2e, 0xd6, 0xbb, 0xe4, + 0x94, 0x1b, 0xff, 0x5d, 0x05, 0xf3, 0xdb, 0x28, 0x39, 0xe3, 0x09, 0x79, 0x00, 0x26, 0x16, 0xf7, + 0x1a, 0x44, 0x79, 0xa1, 0x7f, 0xd9, 0x42, 0x77, 0xc0, 0x46, 0xa3, 0xf4, 0x99, 0x38, 0x53, 0x57, + 0x85, 0x7f, 0xbc, 0x2a, 0xbb, 0xa8, 0xe4, 0x0d, 0xef, 0x75, 0x49, 0x5d, 0x54, 0xde, 0xd0, 0x98, + 0xa9, 0xb8, 0x97, 0x1b, 0xaa, 0x7c, 0xee, 0x39, 0x0b, 0x6b, 0xc6, 0x13, 0x83, 0xdc, 0x87, 0x5a, + 0x4f, 0x9d, 0x54, 0x2a, 0x15, 0x7f, 0x4e, 0x2d, 0x2f, 0x65, 0x8c, 0x7c, 0xe6, 0xc7, 0x60, 0xaa, + 0x97, 0x5d, 0x1d, 0x73, 0x26, 0x31, 0x5d, 0x6e, 0x97, 0x59, 0x7a, 0xc0, 0x7d, 0x30, 0x55, 0xd0, + 0x50, 0x03, 0x66, 0x02, 0x88, 0xda, 0xb5, 0x8a, 0x41, 0x4a, 0x55, 0x79, 0xb9, 0x52, 0x9d, 0xf1, + 0xf8, 0x39, 0xd5, 0x47, 0xd0, 0xa6, 0xdc, 0xe5, 0x7e, 0xe9, 0xcd, 0x27, 0xd9, 0xa1, 0xe6, 0x61, + 0xbb, 0x66, 0x90, 0xa7, 0xb0, 0x38, 0x93, 0x1f, 0x90, 0x0e, 0x1a, 0xfa, 0x92, 0x94, 0x61, 0x7e, + 0xf0, 0x56, 0xfb, 0x97, 0xaf, 0x56, 0x8c, 0x5f, 0xbd, 0x5a, 0x31, 0x7e, 0xf3, 0x6a, 0xc5, 0xf8, + 0xe1, 0xb7, 0x2b, 0x0b, 0x43, 0x13, 0xff, 0xb0, 0xff, 0xfc, 0x4f, 0x01, 0x00, 0x00, 0xff, 0xff, + 0xa0, 0x45, 0x23, 0xbc, 0xf4, 0x1f, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -6448,16 +6430,6 @@ func (m *PostingList) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintPb(dAtA, i, uint64(m.CommitTs)) } - if m.MultiPart { - dAtA[i] = 0x20 - i++ - if m.MultiPart { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ - } if len(m.Parts) > 0 { dAtA23 := make([]byte, len(m.Parts)*10) var j22 int @@ -6470,21 +6442,16 @@ func (m *PostingList) MarshalTo(dAtA []byte) (int, error) { dAtA23[j22] = uint8(num) j22++ } - dAtA[i] = 0x2a + dAtA[i] = 0x22 i++ i = encodeVarintPb(dAtA, i, uint64(j22)) i += copy(dAtA[i:], dAtA23[:j22]) } if m.StartUid != 0 { - dAtA[i] = 0x30 + dAtA[i] = 0x28 i++ i = encodeVarintPb(dAtA, i, uint64(m.StartUid)) } - if m.EndUid != 0 { - dAtA[i] = 0x38 - i++ - i = encodeVarintPb(dAtA, i, uint64(m.EndUid)) - } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) } @@ -8294,9 +8261,6 @@ func (m *PostingList) Size() (n int) { if m.CommitTs != 0 { n += 1 + sovPb(uint64(m.CommitTs)) } - if m.MultiPart { - n += 2 - } if len(m.Parts) > 0 { l = 0 for _, e := range m.Parts { @@ -8307,9 +8271,6 @@ func (m *PostingList) Size() (n int) { if m.StartUid != 0 { n += 1 + sovPb(uint64(m.StartUid)) } - if m.EndUid != 0 { - n += 1 + sovPb(uint64(m.EndUid)) - } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -13621,26 +13582,6 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { } } case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MultiPart", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.MultiPart = bool(v != 0) - case 5: if wireType == 0 { var v uint64 for shift := uint(0); ; shift += 7 { @@ -13713,7 +13654,7 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field Parts", wireType) } - case 6: + case 5: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field StartUid", wireType) } @@ -13732,25 +13673,6 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { break } } - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field EndUid", wireType) - } - m.EndUid = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.EndUid |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skipPb(dAtA[iNdEx:]) From 29df604879a593c5ed6f444f4903850f4e8bb476 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 16:19:28 -0800 Subject: [PATCH 19/67] Remove unnecessary parameter from method. --- posting/list.go | 12 ++++++------ posting/list_test.go | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/posting/list.go b/posting/list.go index 4bb41d5596f..7f8d2b4f88d 100644 --- a/posting/list.go +++ b/posting/list.go @@ -118,7 +118,7 @@ type PItrOpts struct { func (it *PIterator) Init(l *List, opts PItrOpts) error { if len(l.plist.Parts) > 0 { - plist, err := l.readListPart(l.key, l.plist.Parts[opts.startPart]) + plist, err := l.readListPart(l.plist.Parts[opts.startPart]) if err != nil { return err } @@ -698,7 +698,7 @@ func (l *List) MarshalToKv() ([]*bpb.KV, error) { kv := &bpb.KV{} kv.Version = l.minTs kv.Key = getNextPartKey(l.key, startUid) - plist, err := l.readListPart(l.key, startUid) + plist, err := l.readListPart(startUid) if err != nil { return nil, err } @@ -781,7 +781,7 @@ func (l *List) rollup(readTs uint64) error { endUid = l.plist.Parts[splitIdx+1] - 1 } - pl, err := l.readListPart(l.key, startUid) + pl, err := l.readListPart(startUid) if err != nil { return err } @@ -1110,12 +1110,12 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs return facets.CopyFacets(p.Facets, param), nil } -func (l *List) readListPart(baseKey []byte, startUid uint64) (*pb.PostingList, error) { +func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { if part, ok := l.newParts[startUid]; ok { return part, nil } - nextKey := getNextPartKey(baseKey, startUid) + nextKey := getNextPartKey(l.key, startUid) txn := pstore.NewTransactionAt(l.minTs, false) item, err := txn.Get(nextKey) if err != nil { @@ -1154,7 +1154,7 @@ func (l *List) splitList(readTs uint64) error { var newParts []uint64 for _, startUid := range l.plist.Parts { - part, err := l.readListPart(l.key, startUid) + part, err := l.readListPart(startUid) if err != nil { return err } diff --git a/posting/list_test.go b/posting/list_test.go index af1f7421dfe..32631c1d9fc 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -1004,7 +1004,7 @@ func TestMultiPartListMarshal(t *testing.T) { for i, startUid := range ol.plist.Parts { partKey := getNextPartKey(key, startUid) require.Equal(t, partKey, kvs[i+1].Key) - part, err := ol.readListPart(ol.key, startUid) + part, err := ol.readListPart(startUid) require.NoError(t, err) data, err := part.Marshal() require.NoError(t, err) From d1d6bce120f843e1f3a8a599bad0168f01414ca7 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 16:46:28 -0800 Subject: [PATCH 20/67] Variable rename --- posting/list.go | 28 ++++++++++++++++------------ posting/mvcc.go | 2 +- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/posting/list.go b/posting/list.go index 7f8d2b4f88d..4d54f75d1d6 100644 --- a/posting/list.go +++ b/posting/list.go @@ -71,13 +71,17 @@ type List struct { x.SafeMutex key []byte plist *pb.PostingList - newParts map[uint64]*pb.PostingList mutationMap map[uint64]*pb.PostingList minTs uint64 // commit timestamp of immutable layer, reject reads before this ts. maxTs uint64 // max commit timestamp seen for this list. pendingTxns int32 // Using atomic for this, to avoid locking in SetForDeletion operation. deleteMe int32 // Using atomic for this, to avoid expensive SetForDeletion operation. + + // Rolling up might create new parts (either because there were changes or + // a split occurred). These parts need to be kept in memory until they are + // committed to disk. + uncommittedParts map[uint64]*pb.PostingList } func getNextPartKey(baseKey []byte, nextPartStart uint64) []byte { @@ -110,10 +114,10 @@ type PIterator struct { } type PItrOpts struct { - discardPl bool - afterUid uint64 - startPart int - readTs uint64 + discardPl bool + afterUid uint64 + startPart int + readTs uint64 } func (it *PIterator) Init(l *List, opts PItrOpts) error { @@ -800,7 +804,7 @@ func (l *List) rollup(readTs uint64) error { final.Pack = enc.Done() plist.Pack = final.Pack plist.Postings = final.Postings - l.newParts[plist.StartUid] = plist + l.uncommittedParts[plist.StartUid] = plist splitIdx++ init() @@ -818,7 +822,7 @@ func (l *List) rollup(readTs uint64) error { final.Pack = enc.Done() plist.Pack = final.Pack plist.Postings = final.Postings - l.newParts[plist.StartUid] = plist + l.uncommittedParts[plist.StartUid] = plist } maxCommitTs := l.minTs @@ -1111,7 +1115,7 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs } func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { - if part, ok := l.newParts[startUid]; ok { + if part, ok := l.uncommittedParts[startUid]; ok { return part, nil } @@ -1140,13 +1144,13 @@ func (l *List) splitList(readTs uint64) error { var newParts []uint64 parts := splitPostingList(l.plist) for _, part := range parts { - l.newParts[part.StartUid] = part + l.uncommittedParts[part.StartUid] = part newParts = append(newParts, part.StartUid) } l.plist = &pb.PostingList{ - CommitTs: l.plist.CommitTs, - Parts: newParts, + CommitTs: l.plist.CommitTs, + Parts: newParts, } } return nil @@ -1162,7 +1166,7 @@ func (l *List) splitList(readTs uint64) error { if needsSplit(part) { splitParts := splitPostingList(part) for _, part := range splitParts { - l.newParts[part.StartUid] = part + l.uncommittedParts[part.StartUid] = part newParts = append(newParts, part.StartUid) } } else { diff --git a/posting/mvcc.go b/posting/mvcc.go index d4538647ada..310b501b2ca 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -185,7 +185,7 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { l := new(List) l.key = key l.mutationMap = make(map[uint64]*pb.PostingList) - l.newParts = make(map[uint64]*pb.PostingList) + l.uncommittedParts = make(map[uint64]*pb.PostingList) l.plist = new(pb.PostingList) // Iterates from highest Ts to lowest Ts From a9eca0ffa50057c534da3ca07de71a3142225867 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 17:31:07 -0800 Subject: [PATCH 21/67] Clean map of uncommitted lists. --- posting/list.go | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/posting/list.go b/posting/list.go index 4d54f75d1d6..05a2434ee63 100644 --- a/posting/list.go +++ b/posting/list.go @@ -858,6 +858,20 @@ func (l *List) rollup(readTs uint64) error { return nil } + // Delete lists from the uncommittedParts map that are already in disk. + for _, startUid := range l.plist.Parts { + pl, version, err := l.readListPartFromDisk(startUid) + if err != nil || pl == nil { + // Ignore errors since this might be that the list has never + // been committed to disk. + continue + } + + if version >= l.minTs { + delete(l.uncommittedParts, startUid) + } + } + return nil } @@ -1118,18 +1132,22 @@ func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { if part, ok := l.uncommittedParts[startUid]; ok { return part, nil } + part, _, err := l.readListPartFromDisk(startUid) + return part, err +} +func (l *List) readListPartFromDisk(startUid uint64) (*pb.PostingList, uint64, error) { nextKey := getNextPartKey(l.key, startUid) txn := pstore.NewTransactionAt(l.minTs, false) item, err := txn.Get(nextKey) if err != nil { - return nil, err + return nil, 0, err } var part pb.PostingList if err := unmarshalOrCopy(&part, item); err != nil { - return nil, err + return nil, 0, err } - return &part, nil + return &part, item.Version(), nil } func needsSplit(plist *pb.PostingList) bool { From 76704570f574da59f53fad5123aa2d6761246903 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 17:38:43 -0800 Subject: [PATCH 22/67] Move logic to clean uncommitted list map at the beginning of rollup. --- posting/list.go | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/posting/list.go b/posting/list.go index 05a2434ee63..779474f8990 100644 --- a/posting/list.go +++ b/posting/list.go @@ -744,6 +744,20 @@ func (l *List) rollup(readTs uint64) error { return nil } + // Delete lists from the uncommittedParts map that are already in disk. + for _, startUid := range l.plist.Parts { + pl, version, err := l.readListPartFromDisk(startUid) + if err != nil || pl == nil { + // Ignore errors since this might be that the list has never + // been committed to disk. + continue + } + + if version >= l.minTs { + delete(l.uncommittedParts, startUid) + } + } + if len(l.plist.Parts) == 0 { // This is not a multi-part list so use a single encoder to collect all the changes. final := new(pb.PostingList) @@ -858,20 +872,6 @@ func (l *List) rollup(readTs uint64) error { return nil } - // Delete lists from the uncommittedParts map that are already in disk. - for _, startUid := range l.plist.Parts { - pl, version, err := l.readListPartFromDisk(startUid) - if err != nil || pl == nil { - // Ignore errors since this might be that the list has never - // been committed to disk. - continue - } - - if version >= l.minTs { - delete(l.uncommittedParts, startUid) - } - } - return nil } From 8cef9e07c8e78421d9b4aba392444b201fd9bb0b Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 17:49:11 -0800 Subject: [PATCH 23/67] Loop until finding a part with valid entries. --- posting/list.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/posting/list.go b/posting/list.go index 779474f8990..f458970bb3a 100644 --- a/posting/list.go +++ b/posting/list.go @@ -174,16 +174,16 @@ func (it *PIterator) Valid() bool { return false } - // No more parts to iterate through - if len(it.l.plist.Parts) == it.opts.startPart+1 { - return false - } + for it.opts.startPart + 1 < len(it.l.plist.Parts) { + it.opts.startPart++ + it.Init(it.l, it.opts) - it.opts.startPart++ - it.Init(it.l, it.opts) + if len(it.uids) > 0 { + return true + } + } - // TODO: corner case, what if next part is empty but the one after that is not. - return len(it.uids) > 0 + return false } func (it *PIterator) Posting() *pb.Posting { From a709f8e70cfd3609f0678cacfc6351d8760d61c0 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 7 Mar 2019 17:59:52 -0800 Subject: [PATCH 24/67] Simplify rollup logic. --- posting/list.go | 125 ++++++++++++++++++++---------------------------- 1 file changed, 52 insertions(+), 73 deletions(-) diff --git a/posting/list.go b/posting/list.go index f458970bb3a..4c28feb772a 100644 --- a/posting/list.go +++ b/posting/list.go @@ -174,7 +174,7 @@ func (it *PIterator) Valid() bool { return false } - for it.opts.startPart + 1 < len(it.l.plist.Parts) { + for it.opts.startPart+1 < len(it.l.plist.Parts) { it.opts.startPart++ it.Init(it.l, it.opts) @@ -758,87 +758,66 @@ func (l *List) rollup(readTs uint64) error { } } - if len(l.plist.Parts) == 0 { - // This is not a multi-part list so use a single encoder to collect all the changes. - final := new(pb.PostingList) - enc := codec.Encoder{BlockSize: blockSize} - err := l.iterate(readTs, 0, func(p *pb.Posting) error { - // iterate already takes care of not returning entries whose commitTs - // is above l.commitTs. - // So, we don't need to do any filtering here. In fact, doing filtering - // here could result in a bug. - enc.Add(p.Uid) - - // We want to add the posting if it has facets or has a value. - if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { - // I think it's okay to take the pointer from the iterator, because - // we have a lock over List; which won't be released until final has - // been marshalled. Thus, the underlying data wouldn't be changed. - final.Postings = append(final.Postings, p) - } - return nil - }) - x.Check(err) - final.Pack = enc.Done() - l.plist.Pack = final.Pack - l.plist.Postings = final.Postings - } else { - // This is a multi-part list. Use a single iteration but multiple encoders, - // one for each part of the list. - var plist *pb.PostingList - var final *pb.PostingList - var enc codec.Encoder - var endUid uint64 - var splitIdx int - - init := func() error { - startUid := l.plist.Parts[splitIdx] - if splitIdx+1 == len(l.plist.Parts) { - endUid = math.MaxUint64 - } else { - endUid = l.plist.Parts[splitIdx+1] - 1 - } + var plist *pb.PostingList + var final *pb.PostingList + var enc codec.Encoder + var endUid uint64 + var splitIdx int - pl, err := l.readListPart(startUid) - if err != nil { - return err - } + // Method to properly initialize all the variables described above. + init := func() error { + final = new(pb.PostingList) + enc = codec.Encoder{BlockSize: blockSize} - plist = pl - final = new(pb.PostingList) - enc = codec.Encoder{BlockSize: blockSize} + // If not a multi-part list, all uids go to the same encoder. + if len(l.plist.Parts) == 0 { + plist = l.plist + endUid = math.MaxUint64 return nil } - if err := init(); err != nil { - return err - } - err := l.iterate(readTs, 0, func(p *pb.Posting) error { - if p.Uid > endUid { - final.Pack = enc.Done() - plist.Pack = final.Pack - plist.Postings = final.Postings - l.uncommittedParts[plist.StartUid] = plist - - splitIdx++ - init() - } + // Otherwise, load the corresponding part and set endUid to correctly + // detect the end of the list. + startUid := l.plist.Parts[splitIdx] + if splitIdx+1 == len(l.plist.Parts) { + endUid = math.MaxUint64 + } else { + endUid = l.plist.Parts[splitIdx+1] - 1 + } - enc.Add(p.Uid) - if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { - final.Postings = append(final.Postings, p) - } - return nil - }) - x.Check(err) + var err error + plist, err = l.readListPart(startUid) + return err + } - // Finish writing the last part of the list. - final.Pack = enc.Done() - plist.Pack = final.Pack - plist.Postings = final.Postings - l.uncommittedParts[plist.StartUid] = plist + if err := init(); err != nil { + return err } + err := l.iterate(readTs, 0, func(p *pb.Posting) error { + if p.Uid > endUid { + final.Pack = enc.Done() + plist.Pack = final.Pack + plist.Postings = final.Postings + l.uncommittedParts[plist.StartUid] = plist + + splitIdx++ + init() + } + + enc.Add(p.Uid) + if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { + final.Postings = append(final.Postings, p) + } + return nil + }) + // Finish writing the last part of the list (or the whole list if not a multi-part list). + x.Check(err) + final.Pack = enc.Done() + plist.Pack = final.Pack + plist.Postings = final.Postings + l.uncommittedParts[plist.StartUid] = plist + maxCommitTs := l.minTs { // We can't rely upon iterate to give us the max commit timestamp, because it can skip over From e3de2e19ab7dcc853dff7517631e10d107471d4c Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 8 Mar 2019 15:06:58 -0800 Subject: [PATCH 25/67] Rename proto field and simplify split logic. --- posting/list.go | 73 ++++--- posting/list_test.go | 10 +- protos/pb.proto | 2 +- protos/pb/pb.pb.go | 444 +++++++++++++++++++++---------------------- 4 files changed, 264 insertions(+), 265 deletions(-) diff --git a/posting/list.go b/posting/list.go index 4c28feb772a..29365c66126 100644 --- a/posting/list.go +++ b/posting/list.go @@ -121,8 +121,8 @@ type PItrOpts struct { } func (it *PIterator) Init(l *List, opts PItrOpts) error { - if len(l.plist.Parts) > 0 { - plist, err := l.readListPart(l.plist.Parts[opts.startPart]) + if len(l.plist.Splits) > 0 { + plist, err := l.readListPart(l.plist.Splits[opts.startPart]) if err != nil { return err } @@ -170,11 +170,11 @@ func (it *PIterator) Valid() bool { return true } - if len(it.l.plist.Parts) == 0 { + if len(it.l.plist.Splits) == 0 { return false } - for it.opts.startPart+1 < len(it.l.plist.Parts) { + for it.opts.startPart+1 < len(it.l.plist.Splits) { it.opts.startPart++ it.Init(it.l, it.opts) @@ -698,7 +698,7 @@ func (l *List) MarshalToKv() ([]*bpb.KV, error) { kv.Value = val kvs = append(kvs, kv) - for _, startUid := range l.plist.Parts { + for _, startUid := range l.plist.Splits { kv := &bpb.KV{} kv.Version = l.minTs kv.Key = getNextPartKey(l.key, startUid) @@ -745,7 +745,7 @@ func (l *List) rollup(readTs uint64) error { } // Delete lists from the uncommittedParts map that are already in disk. - for _, startUid := range l.plist.Parts { + for _, startUid := range l.plist.Splits { pl, version, err := l.readListPartFromDisk(startUid) if err != nil || pl == nil { // Ignore errors since this might be that the list has never @@ -770,7 +770,7 @@ func (l *List) rollup(readTs uint64) error { enc = codec.Encoder{BlockSize: blockSize} // If not a multi-part list, all uids go to the same encoder. - if len(l.plist.Parts) == 0 { + if len(l.plist.Splits) == 0 { plist = l.plist endUid = math.MaxUint64 return nil @@ -778,11 +778,11 @@ func (l *List) rollup(readTs uint64) error { // Otherwise, load the corresponding part and set endUid to correctly // detect the end of the list. - startUid := l.plist.Parts[splitIdx] - if splitIdx+1 == len(l.plist.Parts) { + startUid := l.plist.Splits[splitIdx] + if splitIdx+1 == len(l.plist.Splits) { endUid = math.MaxUint64 } else { - endUid = l.plist.Parts[splitIdx+1] - 1 + endUid = l.plist.Splits[splitIdx+1] - 1 } var err error @@ -869,7 +869,7 @@ func (l *List) Uids(opt ListOptions) (*pb.List, error) { // Use approximate length for initial capacity. res := make([]uint64, 0, len(l.mutationMap)+codec.ApproxLen(l.plist.Pack)) out := &pb.List{} - if len(l.mutationMap) == 0 && opt.Intersect != nil && len(l.plist.Parts) == 0 { + if len(l.mutationMap) == 0 && opt.Intersect != nil && len(l.plist.Splits) == 0 { if opt.ReadTs < l.minTs { l.RUnlock() return out, ErrTsTooOld @@ -1136,42 +1136,41 @@ func needsSplit(plist *pb.PostingList) bool { func (l *List) splitList(readTs uint64) error { l.AssertLock() - if len(l.plist.Parts) == 0 { - if needsSplit(l.plist) { - var newParts []uint64 - parts := splitPostingList(l.plist) - for _, part := range parts { - l.uncommittedParts[part.StartUid] = part - newParts = append(newParts, part.StartUid) - } - - l.plist = &pb.PostingList{ - CommitTs: l.plist.CommitTs, - Parts: newParts, + var lists []*pb.PostingList + if len(l.plist.Splits) == 0 { + lists = append(lists, l.plist) + } else { + for _, startUid := range l.plist.Splits { + part, err := l.readListPart(startUid) + if err != nil { + return err } + lists = append(lists, part) } - return nil } - var newParts []uint64 - for _, startUid := range l.plist.Parts { - part, err := l.readListPart(startUid) - if err != nil { - return err - } - - if needsSplit(part) { - splitParts := splitPostingList(part) - for _, part := range splitParts { + var newLists []*pb.PostingList + for _, list := range lists { + if needsSplit(list) { + splitList := splitPostingList(list) + for _, part := range splitList { l.uncommittedParts[part.StartUid] = part - newParts = append(newParts, part.StartUid) + newLists = append(newLists, part) } } else { - newParts = append(newParts, part.StartUid) + newLists = append(newLists, list) } } - l.plist.Parts = newParts + if len(newLists) == 1 || len(newLists) == len(l.plist.Splits) { + return nil + } else { + var splits []uint64 + for _, list := range newLists { + splits = append(splits, list.StartUid) + } + l.plist.Splits = splits + } return nil } diff --git a/posting/list_test.go b/posting/list_test.go index 32631c1d9fc..56c3654f3ab 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -935,7 +935,7 @@ func createMultiPartList(t *testing.T, size int) (*List, int) { func TestMultiPartList(t *testing.T) { N := int(1e5) ol, commits := createMultiPartList(t, N) - t.Logf("List parts %v", len(ol.plist.Parts)) + t.Logf("List parts %v", len(ol.plist.Splits)) opt := ListOptions{ReadTs: uint64(N) + 1} l, err := ol.Uids(opt) require.NoError(t, err) @@ -973,7 +973,7 @@ func TestMultiPartListWithPostings(t *testing.T) { } commits++ } - t.Logf("List parts %v", len(ol.plist.Parts)) + t.Logf("List parts %v", len(ol.plist.Splits)) var labels []string ol.Iterate(uint64(N)+1, 0, func(p *pb.Posting) error { @@ -992,16 +992,16 @@ func TestMultiPartListWithPostings(t *testing.T) { func TestMultiPartListMarshal(t *testing.T) { N := int(1e5) ol, commits := createMultiPartList(t, N) - t.Logf("List parts %v", len(ol.plist.Parts)) + t.Logf("List parts %v", len(ol.plist.Splits)) kvs, err := ol.MarshalToKv() require.NoError(t, err) - require.Equal(t, len(kvs), len(ol.plist.Parts)+1) + require.Equal(t, len(kvs), len(ol.plist.Splits)+1) key := x.DataKey("bal", 1331) require.Equal(t, key, kvs[0].Key) - for i, startUid := range ol.plist.Parts { + for i, startUid := range ol.plist.Splits { partKey := getNextPartKey(key, startUid) require.Equal(t, partKey, kvs[i+1].Key) part, err := ol.readListPart(startUid) diff --git a/protos/pb.proto b/protos/pb.proto index 017781f9064..41db7e1f36e 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -278,7 +278,7 @@ message PostingList { repeated Posting postings = 2; uint64 commit_ts = 3; // More inclination towards smaller values. - repeated uint64 parts = 4; + repeated uint64 splits = 4; uint64 start_uid = 5; } diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index baaa676fcc9..a2932f50a05 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -2100,7 +2100,7 @@ type PostingList struct { Pack *UidPack `protobuf:"bytes,1,opt,name=pack,proto3" json:"pack,omitempty"` Postings []*Posting `protobuf:"bytes,2,rep,name=postings,proto3" json:"postings,omitempty"` CommitTs uint64 `protobuf:"varint,3,opt,name=commit_ts,json=commitTs,proto3" json:"commit_ts,omitempty"` - Parts []uint64 `protobuf:"varint,4,rep,packed,name=parts,proto3" json:"parts,omitempty"` + Splits []uint64 `protobuf:"varint,4,rep,packed,name=splits,proto3" json:"splits,omitempty"` StartUid uint64 `protobuf:"varint,5,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -2161,9 +2161,9 @@ func (m *PostingList) GetCommitTs() uint64 { return 0 } -func (m *PostingList) GetParts() []uint64 { +func (m *PostingList) GetSplits() []uint64 { if m != nil { - return m.Parts + return m.Splits } return nil } @@ -3638,215 +3638,215 @@ func init() { proto.RegisterFile("pb.proto", fileDescriptor_f80abaa17e25ccc8) } var fileDescriptor_f80abaa17e25ccc8 = []byte{ // 3336 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0xcd, 0x6f, 0x1c, 0x47, - 0x76, 0x67, 0xcf, 0x47, 0x4f, 0xf7, 0x9b, 0x21, 0x35, 0x2e, 0xc9, 0xf2, 0x98, 0xb2, 0x29, 0xba, - 0x2d, 0x59, 0x94, 0x64, 0x51, 0x32, 0xed, 0x24, 0x96, 0x83, 0x1c, 0xf8, 0x31, 0x52, 0x68, 0xf3, - 0x2b, 0x35, 0x43, 0x39, 0x31, 0x82, 0x0c, 0x6a, 0xba, 0x8b, 0xc3, 0x36, 0x7b, 0xba, 0x3b, 0x5d, - 0x3d, 0xc4, 0x50, 0xb7, 0x1c, 0x92, 0xbf, 0xc1, 0x87, 0x20, 0xb7, 0x20, 0x40, 0x2e, 0xb9, 0xfa, - 0x98, 0xcb, 0x02, 0x7b, 0x5c, 0xec, 0x5f, 0xb0, 0xd0, 0xee, 0x7f, 0xb0, 0xc0, 0x02, 0x7b, 0x5b, - 0xd4, 0xab, 0xea, 0x8f, 0x19, 0x91, 0x92, 0xbd, 0xc0, 0x9e, 0xa6, 0xdf, 0x47, 0x7d, 0xbd, 0xfa, - 0xbd, 0x57, 0xef, 0xbd, 0x01, 0x2b, 0x1e, 0xae, 0xc7, 0x49, 0x94, 0x46, 0xa4, 0x12, 0x0f, 0x97, - 0x6d, 0x16, 0xfb, 0x8a, 0x5c, 0xbe, 0x37, 0xf2, 0xd3, 0xd3, 0xc9, 0x70, 0xdd, 0x8d, 0xc6, 0x8f, - 0xbd, 0x51, 0xc2, 0xe2, 0xd3, 0x47, 0x7e, 0xf4, 0x78, 0xc8, 0xbc, 0x11, 0x4f, 0x1e, 0xc7, 0xc3, - 0xc7, 0xd9, 0x38, 0x67, 0x19, 0x6a, 0x7b, 0xbe, 0x48, 0x09, 0x81, 0xda, 0xc4, 0xf7, 0x44, 0xc7, - 0x58, 0xad, 0xae, 0x99, 0x14, 0xbf, 0x9d, 0x7d, 0xb0, 0xfb, 0x4c, 0x9c, 0xbd, 0x60, 0xc1, 0x84, - 0x93, 0x36, 0x54, 0xcf, 0x59, 0xd0, 0x31, 0x56, 0x8d, 0xb5, 0x16, 0x95, 0x9f, 0x64, 0x1d, 0xac, - 0x73, 0x16, 0x0c, 0xd2, 0x8b, 0x98, 0x77, 0x2a, 0xab, 0xc6, 0xda, 0xd2, 0xc6, 0xf5, 0xf5, 0x78, - 0xb8, 0x7e, 0x14, 0x89, 0xd4, 0x0f, 0x47, 0xeb, 0x2f, 0x58, 0xd0, 0xbf, 0x88, 0x39, 0x6d, 0x9c, - 0xab, 0x0f, 0xe7, 0x10, 0x9a, 0xbd, 0xc4, 0x7d, 0x36, 0x09, 0xdd, 0xd4, 0x8f, 0x42, 0xb9, 0x62, - 0xc8, 0xc6, 0x1c, 0x67, 0xb4, 0x29, 0x7e, 0x4b, 0x1e, 0x4b, 0x46, 0xa2, 0x53, 0x5d, 0xad, 0x4a, - 0x9e, 0xfc, 0x26, 0x1d, 0x68, 0xf8, 0x62, 0x3b, 0x9a, 0x84, 0x69, 0xa7, 0xb6, 0x6a, 0xac, 0x59, - 0x34, 0x23, 0x9d, 0xdf, 0x57, 0xa0, 0xfe, 0x0f, 0x13, 0x9e, 0x5c, 0xe0, 0xb8, 0x34, 0x4d, 0xb2, - 0xb9, 0xe4, 0x37, 0xb9, 0x01, 0xf5, 0x80, 0x85, 0x23, 0xd1, 0xa9, 0xe0, 0x64, 0x8a, 0x20, 0xb7, - 0xc0, 0x66, 0x27, 0x29, 0x4f, 0x06, 0x13, 0xdf, 0xeb, 0x54, 0x57, 0x8d, 0x35, 0x93, 0x5a, 0xc8, - 0x38, 0xf6, 0x3d, 0xf2, 0x3e, 0x58, 0x5e, 0x34, 0x70, 0xcb, 0x6b, 0x79, 0x11, 0xae, 0x45, 0x3e, - 0x06, 0x6b, 0xe2, 0x7b, 0x83, 0xc0, 0x17, 0x69, 0xa7, 0xbe, 0x6a, 0xac, 0x35, 0x37, 0x2c, 0x79, - 0x58, 0x69, 0x3b, 0xda, 0x98, 0xf8, 0x1e, 0x1a, 0xf1, 0x01, 0x58, 0x22, 0x71, 0x07, 0x27, 0x93, - 0xd0, 0xed, 0x98, 0xa8, 0x74, 0x4d, 0x2a, 0x95, 0x4e, 0x4d, 0x1b, 0x42, 0x11, 0xf2, 0x58, 0x09, - 0x3f, 0xe7, 0x89, 0xe0, 0x9d, 0x86, 0x5a, 0x4a, 0x93, 0xe4, 0x09, 0x34, 0x4f, 0x98, 0xcb, 0xd3, - 0x41, 0xcc, 0x12, 0x36, 0xee, 0x58, 0xc5, 0x44, 0xcf, 0x24, 0xfb, 0x48, 0x72, 0x05, 0x85, 0x93, - 0x9c, 0x20, 0x9f, 0xc3, 0x22, 0x52, 0x62, 0x70, 0xe2, 0x07, 0x29, 0x4f, 0x3a, 0x36, 0x8e, 0x59, - 0xc2, 0x31, 0xc8, 0xe9, 0x27, 0x9c, 0xd3, 0x96, 0x52, 0x52, 0x1c, 0xf2, 0x21, 0x00, 0x9f, 0xc6, - 0x2c, 0xf4, 0x06, 0x2c, 0x08, 0x3a, 0x80, 0x7b, 0xb0, 0x15, 0x67, 0x33, 0x08, 0xc8, 0x7b, 0x72, - 0x7f, 0xcc, 0x1b, 0xa4, 0xa2, 0xb3, 0xb8, 0x6a, 0xac, 0xd5, 0xa8, 0x29, 0xc9, 0xbe, 0x70, 0x36, - 0xc0, 0x46, 0x44, 0xe0, 0x89, 0xef, 0x82, 0x79, 0x2e, 0x09, 0x05, 0x9c, 0xe6, 0xc6, 0xa2, 0x5c, - 0x32, 0x07, 0x0d, 0xd5, 0x42, 0x67, 0x05, 0xac, 0x3d, 0x16, 0x8e, 0x32, 0xa4, 0xc9, 0xab, 0xc0, - 0x01, 0x36, 0xc5, 0x6f, 0xe7, 0x87, 0x0a, 0x98, 0x94, 0x8b, 0x49, 0x90, 0x92, 0x7b, 0x00, 0xd2, - 0xd0, 0x63, 0x96, 0x26, 0xfe, 0x54, 0xcf, 0x5a, 0x98, 0xda, 0x9e, 0xf8, 0xde, 0x3e, 0x8a, 0xc8, - 0x13, 0x68, 0xe1, 0xec, 0x99, 0x6a, 0xa5, 0xd8, 0x40, 0xbe, 0x3f, 0xda, 0x44, 0x15, 0x3d, 0xe2, - 0x26, 0x98, 0x78, 0xb7, 0x0a, 0x5f, 0x8b, 0x54, 0x53, 0xe4, 0x2e, 0x2c, 0xf9, 0x61, 0x2a, 0x6d, - 0xef, 0xa6, 0x03, 0x8f, 0x8b, 0xec, 0xf2, 0x17, 0x73, 0xee, 0x0e, 0x17, 0x29, 0xf9, 0x0c, 0x94, - 0x01, 0xb3, 0x05, 0xeb, 0xb8, 0xe0, 0x52, 0x7e, 0x31, 0x42, 0xad, 0x88, 0x3a, 0x7a, 0xc5, 0x47, - 0xd0, 0x94, 0xe7, 0xcb, 0x46, 0x98, 0x38, 0xa2, 0x85, 0xa7, 0xd1, 0xe6, 0xa0, 0x20, 0x15, 0xb4, - 0xba, 0x34, 0x8d, 0x04, 0x98, 0x02, 0x04, 0x7e, 0x3b, 0x5d, 0xa8, 0x1f, 0x26, 0x1e, 0x4f, 0x2e, - 0xc5, 0x38, 0x81, 0x9a, 0xc7, 0x85, 0x8b, 0xee, 0x67, 0x51, 0xfc, 0x2e, 0x70, 0x5f, 0x2d, 0xe1, - 0xde, 0xf9, 0x2f, 0x03, 0x9a, 0xbd, 0x28, 0x49, 0xf7, 0xb9, 0x10, 0x6c, 0xc4, 0xc9, 0x6d, 0xa8, - 0x47, 0x72, 0x5a, 0x6d, 0x61, 0x5b, 0xee, 0x09, 0xd7, 0xa1, 0x8a, 0x3f, 0x77, 0x0f, 0x95, 0xab, - 0xef, 0xe1, 0x06, 0xd4, 0x95, 0xc7, 0x48, 0x6f, 0xaa, 0x53, 0x45, 0x48, 0x5b, 0x47, 0x27, 0x27, - 0x82, 0x2b, 0x5b, 0xd6, 0xa9, 0xa6, 0xae, 0x86, 0xd5, 0x5f, 0x01, 0xc8, 0xfd, 0xfd, 0x4c, 0x14, - 0x38, 0xa7, 0xd0, 0xa4, 0xec, 0x24, 0xdd, 0x8e, 0xc2, 0x94, 0x4f, 0x53, 0xb2, 0x04, 0x15, 0xdf, - 0x43, 0x13, 0x99, 0xb4, 0xe2, 0x7b, 0x72, 0x73, 0xa3, 0x24, 0x9a, 0xc4, 0x68, 0xa1, 0x45, 0xaa, - 0x08, 0x34, 0xa5, 0xe7, 0x25, 0xb8, 0x63, 0x69, 0x4a, 0xcf, 0x4b, 0xc8, 0x6d, 0x68, 0x8a, 0x90, - 0xc5, 0xe2, 0x34, 0x4a, 0xe5, 0xe6, 0x6a, 0xb8, 0x39, 0xc8, 0x58, 0x7d, 0xe1, 0xfc, 0xc2, 0x00, - 0x73, 0x9f, 0x8f, 0x87, 0x3c, 0x79, 0x6d, 0x95, 0xf7, 0xc1, 0xc2, 0x89, 0x07, 0xbe, 0xa7, 0x17, - 0x6a, 0x20, 0xbd, 0xeb, 0x5d, 0xba, 0xd4, 0x4d, 0x30, 0x03, 0xce, 0xa4, 0xf1, 0x15, 0xce, 0x34, - 0x25, 0x6d, 0xc3, 0xc6, 0x03, 0x8f, 0x33, 0x0f, 0x43, 0x8c, 0x45, 0x4d, 0x36, 0xde, 0xe1, 0xcc, - 0x93, 0x7b, 0x0b, 0x98, 0x48, 0x07, 0x93, 0xd8, 0x63, 0x29, 0xc7, 0xd0, 0x52, 0x93, 0xc0, 0x11, - 0xe9, 0x31, 0x72, 0xc8, 0x03, 0x78, 0xc7, 0x0d, 0x26, 0x42, 0xc6, 0x35, 0x3f, 0x3c, 0x89, 0x06, - 0x51, 0x18, 0x5c, 0xa0, 0x7d, 0x2d, 0x7a, 0x4d, 0x0b, 0x76, 0xc3, 0x93, 0xe8, 0x30, 0x0c, 0x2e, - 0x9c, 0x1f, 0x2b, 0x50, 0x7f, 0x8e, 0x66, 0x78, 0x02, 0x8d, 0x31, 0x1e, 0x28, 0xf3, 0xde, 0x9b, - 0xd2, 0xc2, 0x28, 0x5b, 0x57, 0x27, 0x15, 0xdd, 0x30, 0x4d, 0x2e, 0x68, 0xa6, 0x26, 0x47, 0xa4, - 0x6c, 0x18, 0xf0, 0x54, 0x68, 0x44, 0x94, 0x46, 0xf4, 0x95, 0x40, 0x8f, 0xd0, 0x6a, 0xf3, 0x66, - 0xad, 0xce, 0x9b, 0x95, 0x2c, 0x83, 0xe5, 0x9e, 0x72, 0xf7, 0x4c, 0x4c, 0xc6, 0xda, 0xe8, 0x39, - 0xbd, 0xfc, 0x0c, 0x5a, 0xe5, 0x7d, 0xc8, 0x37, 0xe8, 0x8c, 0x5f, 0xa0, 0xe1, 0x6b, 0x54, 0x7e, - 0x92, 0x55, 0xa8, 0xa3, 0x87, 0xa3, 0xd9, 0x9b, 0x1b, 0x20, 0xb7, 0xa3, 0x86, 0x50, 0x25, 0xf8, - 0xaa, 0xf2, 0xa5, 0x21, 0xe7, 0x29, 0xef, 0xae, 0x3c, 0x8f, 0x7d, 0xf5, 0x3c, 0x6a, 0x48, 0x69, - 0x1e, 0xe7, 0x8f, 0x15, 0x68, 0x7d, 0xc7, 0x93, 0xe8, 0x28, 0x89, 0xe2, 0x48, 0xb0, 0x80, 0x6c, - 0xce, 0x9e, 0x4e, 0x59, 0x71, 0x55, 0x0e, 0x2e, 0xab, 0xad, 0xf7, 0xf2, 0xe3, 0x2a, 0xeb, 0x94, - 0xcf, 0xef, 0x80, 0xa9, 0xac, 0x7b, 0xc9, 0x11, 0xb4, 0x44, 0xea, 0x28, 0x7b, 0xa2, 0xfd, 0x66, - 0xb7, 0xa7, 0x25, 0x64, 0x05, 0x60, 0xcc, 0xa6, 0x7b, 0x9c, 0x09, 0xbe, 0xeb, 0x65, 0xf0, 0x2d, - 0x38, 0xd2, 0xce, 0x63, 0x36, 0xed, 0x4f, 0xc3, 0xbe, 0x40, 0x74, 0xd5, 0x68, 0x4e, 0x93, 0x0f, - 0xc0, 0x1e, 0xb3, 0xa9, 0xf4, 0xa3, 0x5d, 0x4f, 0xa3, 0xab, 0x60, 0x90, 0x8f, 0xa0, 0x9a, 0x4e, - 0x43, 0x0c, 0x4a, 0xf2, 0x1d, 0x92, 0x49, 0x46, 0x7f, 0x1a, 0x6a, 0x8f, 0xa3, 0x52, 0x96, 0x19, - 0xd4, 0x2a, 0x0c, 0xda, 0x86, 0xaa, 0xeb, 0x7b, 0xf8, 0x10, 0xd9, 0x54, 0x7e, 0x2e, 0xff, 0x1d, - 0x5c, 0x9b, 0xb3, 0x43, 0xf9, 0x1e, 0x16, 0xd5, 0xb0, 0x1b, 0xe5, 0x7b, 0xa8, 0x95, 0x6d, 0xff, - 0x63, 0x15, 0xae, 0x69, 0x30, 0x9c, 0xfa, 0x71, 0x2f, 0x95, 0xb0, 0xef, 0x40, 0x03, 0xa3, 0x0d, - 0x4f, 0x34, 0x26, 0x32, 0x92, 0xfc, 0x0d, 0x98, 0xe8, 0x81, 0x19, 0x4e, 0x6f, 0x17, 0x56, 0xcd, - 0x87, 0x2b, 0xdc, 0xea, 0x2b, 0xd1, 0xea, 0xe4, 0x0b, 0xa8, 0xbf, 0xe4, 0x49, 0xa4, 0xa2, 0x67, - 0x73, 0x63, 0xe5, 0xb2, 0x71, 0xf2, 0x6e, 0xf5, 0x30, 0xa5, 0xfc, 0x17, 0x34, 0xfe, 0x1d, 0x19, - 0x2f, 0xc7, 0xd1, 0x39, 0xf7, 0x3a, 0x0d, 0xdc, 0x51, 0x19, 0x1f, 0x99, 0x28, 0xb3, 0xb6, 0x55, - 0x58, 0x7b, 0x07, 0x9a, 0xa5, 0xe3, 0x5d, 0x62, 0xe9, 0xdb, 0xb3, 0x88, 0xb7, 0x73, 0x47, 0x2e, - 0x3b, 0xce, 0x0e, 0x40, 0x71, 0xd8, 0x3f, 0xd7, 0xfd, 0x9c, 0x7f, 0x33, 0xe0, 0xda, 0x76, 0x14, - 0x86, 0x1c, 0x53, 0x20, 0x75, 0x75, 0x05, 0xec, 0x8d, 0x2b, 0x61, 0x7f, 0x1f, 0xea, 0x42, 0x2a, - 0xeb, 0xd9, 0xaf, 0x5f, 0x72, 0x17, 0x54, 0x69, 0xc8, 0x30, 0x33, 0x66, 0xd3, 0x41, 0xcc, 0x43, - 0xcf, 0x0f, 0x47, 0x59, 0x98, 0x19, 0xb3, 0xe9, 0x91, 0xe2, 0x38, 0xff, 0x6e, 0x80, 0xa9, 0x3c, - 0x66, 0x26, 0x5a, 0x1b, 0xb3, 0xd1, 0xfa, 0x03, 0xb0, 0xe3, 0x84, 0x7b, 0xbe, 0x9b, 0xad, 0x6a, - 0xd3, 0x82, 0x21, 0xc1, 0x79, 0x12, 0x25, 0x2e, 0xc7, 0xe9, 0x2d, 0xaa, 0x08, 0xc9, 0x15, 0x31, - 0x73, 0x55, 0x1a, 0x57, 0xa5, 0x8a, 0x90, 0x31, 0x5e, 0x5d, 0x0e, 0x5e, 0x8a, 0x45, 0x35, 0xe5, - 0xfc, 0x6f, 0x05, 0x5a, 0x3b, 0x7e, 0xc2, 0xdd, 0x94, 0x7b, 0x5d, 0x6f, 0x84, 0x8a, 0x3c, 0x4c, - 0xfd, 0xf4, 0x42, 0xbf, 0x27, 0x9a, 0xca, 0x9f, 0xfb, 0xca, 0x6c, 0x4a, 0xab, 0xcc, 0x5d, 0xc5, - 0x2c, 0x5c, 0x11, 0x64, 0x03, 0x40, 0x25, 0x42, 0x98, 0x89, 0xd7, 0xae, 0xce, 0xc4, 0x6d, 0x54, - 0x93, 0x9f, 0xd2, 0x06, 0x6a, 0x8c, 0xaf, 0xde, 0x1a, 0x13, 0xd3, 0xf4, 0x89, 0xc4, 0x2a, 0xe6, - 0x0f, 0x43, 0x1e, 0x20, 0x16, 0x31, 0x7f, 0x18, 0xf2, 0x20, 0xcf, 0xda, 0x1a, 0x6a, 0x3b, 0xf2, - 0x9b, 0x7c, 0x0c, 0x95, 0x28, 0xc6, 0xf3, 0xe9, 0x05, 0xcb, 0x07, 0x5b, 0x3f, 0x8c, 0x69, 0x25, - 0x8a, 0xe5, 0x45, 0xab, 0xb4, 0xb3, 0x63, 0x6b, 0xfc, 0xca, 0x00, 0x82, 0x09, 0x13, 0xd5, 0x12, - 0xe7, 0x26, 0x54, 0x0e, 0x63, 0xd2, 0x80, 0x6a, 0xaf, 0xdb, 0x6f, 0x2f, 0xc8, 0x8f, 0x9d, 0xee, - 0x5e, 0xdb, 0x70, 0x5e, 0x19, 0x60, 0xef, 0x4f, 0x52, 0x26, 0x61, 0x23, 0xde, 0x74, 0x6f, 0xef, - 0x83, 0x25, 0x52, 0x96, 0x60, 0x10, 0x56, 0x91, 0xa3, 0x81, 0x74, 0x5f, 0x90, 0x4f, 0xa0, 0xce, - 0xbd, 0x11, 0xcf, 0x1c, 0xba, 0x3d, 0xbf, 0x4f, 0xaa, 0xc4, 0x64, 0x0d, 0x4c, 0xe1, 0x9e, 0xf2, - 0x31, 0xeb, 0xd4, 0x0a, 0xc5, 0x1e, 0x72, 0xd4, 0x23, 0x4b, 0xb5, 0x1c, 0xab, 0x84, 0x24, 0x8a, - 0x31, 0x6d, 0xae, 0xeb, 0x2a, 0x21, 0x89, 0x62, 0x99, 0x34, 0x6f, 0xc0, 0xbb, 0xfe, 0x28, 0x8c, - 0x12, 0x3e, 0xf0, 0x43, 0x8f, 0x4f, 0x07, 0x6e, 0x14, 0x9e, 0x04, 0xbe, 0x9b, 0xa2, 0x2d, 0x2d, - 0x7a, 0x5d, 0x09, 0x77, 0xa5, 0x6c, 0x5b, 0x8b, 0x9c, 0x29, 0x58, 0x59, 0x5c, 0x24, 0xf7, 0x65, - 0x40, 0xc3, 0xb8, 0xaa, 0xdd, 0x02, 0xd3, 0xfe, 0x52, 0x82, 0x43, 0x33, 0xb9, 0xbc, 0x26, 0x5c, - 0x23, 0x8b, 0x94, 0x48, 0x94, 0xd3, 0xab, 0x6a, 0x39, 0xbd, 0xc2, 0x4c, 0x31, 0x0a, 0xb9, 0xce, - 0x38, 0xf0, 0xdb, 0xf9, 0xcf, 0x0a, 0x58, 0xf9, 0x53, 0xf6, 0x10, 0xec, 0x71, 0x66, 0x6a, 0xed, - 0x70, 0x98, 0x4b, 0xe7, 0xf6, 0xa7, 0x85, 0x9c, 0xdc, 0x84, 0xca, 0xd9, 0xb9, 0x36, 0x94, 0x29, - 0xb5, 0xbe, 0x79, 0x41, 0x2b, 0x67, 0xe7, 0x85, 0xc7, 0xd6, 0xdf, 0xea, 0xb1, 0xf7, 0xe0, 0x9a, - 0x1b, 0x70, 0x16, 0x0e, 0x0a, 0x87, 0x53, 0x80, 0x5b, 0x42, 0xf6, 0x51, 0xee, 0x75, 0x3a, 0xea, - 0x34, 0x8a, 0xb7, 0xe5, 0x2e, 0xd4, 0x3d, 0x1e, 0xa4, 0xac, 0x5c, 0x1a, 0x1d, 0x26, 0xcc, 0x0d, - 0xf8, 0x8e, 0x64, 0x53, 0x25, 0x25, 0x6b, 0x60, 0x65, 0xef, 0xac, 0x2e, 0x88, 0x30, 0xf3, 0xce, - 0x8c, 0x4d, 0x73, 0x69, 0x61, 0x4b, 0x28, 0xd9, 0xd2, 0xf9, 0x0c, 0xaa, 0xdf, 0xbc, 0xe8, 0xe9, - 0xb3, 0x1a, 0xaf, 0x9d, 0x35, 0xb3, 0x68, 0xa5, 0x64, 0xd1, 0x3f, 0x54, 0xa1, 0xa1, 0xbd, 0x4e, - 0xee, 0x7b, 0x92, 0x67, 0x89, 0xf2, 0x73, 0xf6, 0x71, 0xcb, 0xdd, 0xb7, 0x5c, 0x46, 0x57, 0xdf, - 0x5e, 0x46, 0x93, 0xaf, 0xa0, 0x15, 0x2b, 0x59, 0xd9, 0xe1, 0xdf, 0x2b, 0x8f, 0xd1, 0xbf, 0x38, - 0xae, 0x19, 0x17, 0x84, 0x84, 0x2e, 0xd6, 0x23, 0x29, 0x1b, 0xe1, 0x15, 0xb5, 0x68, 0x43, 0xd2, - 0x7d, 0x36, 0xba, 0xc2, 0xed, 0x7f, 0x82, 0xf7, 0xca, 0x6c, 0x38, 0x8a, 0x3b, 0x2d, 0xf4, 0x48, - 0xe9, 0xf1, 0x65, 0x67, 0x5c, 0x9c, 0x75, 0xc6, 0x5b, 0x60, 0xbb, 0xd1, 0x78, 0xec, 0xa3, 0x6c, - 0x49, 0x67, 0x7b, 0xc8, 0xe8, 0x0b, 0xe7, 0x3f, 0x0c, 0x68, 0xe8, 0xd3, 0x92, 0x26, 0x34, 0x76, - 0xba, 0xcf, 0x36, 0x8f, 0xf7, 0x64, 0x3c, 0x00, 0x30, 0xb7, 0x76, 0x0f, 0x36, 0xe9, 0x3f, 0xb5, - 0x0d, 0x19, 0x1b, 0x76, 0x0f, 0xfa, 0xed, 0x0a, 0xb1, 0xa1, 0xfe, 0x6c, 0xef, 0x70, 0xb3, 0xdf, - 0xae, 0x12, 0x0b, 0x6a, 0x5b, 0x87, 0x87, 0x7b, 0xed, 0x1a, 0x69, 0x81, 0xb5, 0xb3, 0xd9, 0xef, - 0xf6, 0x77, 0xf7, 0xbb, 0xed, 0xba, 0xd4, 0x7d, 0xde, 0x3d, 0x6c, 0x9b, 0xf2, 0xe3, 0x78, 0x77, - 0xa7, 0xdd, 0x90, 0xf2, 0xa3, 0xcd, 0x5e, 0xef, 0xdb, 0x43, 0xba, 0xd3, 0xb6, 0xe4, 0xbc, 0xbd, - 0x3e, 0xdd, 0x3d, 0x78, 0xde, 0xb6, 0xe5, 0xf7, 0xe1, 0xd6, 0xd7, 0xdd, 0xed, 0x7e, 0x1b, 0x9c, - 0xcf, 0xa0, 0x59, 0xb2, 0xa0, 0x1c, 0x4d, 0xbb, 0xcf, 0xda, 0x0b, 0x72, 0xc9, 0x17, 0x9b, 0x7b, - 0xc7, 0xdd, 0xb6, 0x41, 0x96, 0x00, 0xf0, 0x73, 0xb0, 0xb7, 0x79, 0xf0, 0xbc, 0x5d, 0x71, 0xfe, - 0x1a, 0xac, 0x63, 0xdf, 0xdb, 0x0a, 0x22, 0xf7, 0x4c, 0x02, 0x63, 0xc8, 0x04, 0xd7, 0xef, 0x24, - 0x7e, 0xcb, 0x28, 0x8f, 0xa0, 0x14, 0xfa, 0xee, 0x35, 0xe5, 0x1c, 0x40, 0xe3, 0xd8, 0xf7, 0x8e, - 0x98, 0x7b, 0x26, 0xeb, 0xf1, 0xa1, 0x1c, 0x3f, 0x10, 0xfe, 0x4b, 0xae, 0x03, 0x9c, 0x8d, 0x9c, - 0x9e, 0xff, 0x92, 0x93, 0x3b, 0x60, 0x22, 0x91, 0x65, 0x34, 0x88, 0xe5, 0x6c, 0x4d, 0xaa, 0x65, - 0xce, 0xff, 0x18, 0xf9, 0xde, 0xb1, 0xd8, 0xbe, 0x0d, 0xb5, 0x98, 0xb9, 0x67, 0x3a, 0x9a, 0x34, - 0xf5, 0x18, 0xb9, 0x1e, 0x45, 0x01, 0xb9, 0x07, 0x96, 0x06, 0x48, 0x36, 0x71, 0xb3, 0x84, 0x24, - 0x9a, 0x0b, 0x67, 0xaf, 0xae, 0x3a, 0x7b, 0x75, 0x12, 0x3c, 0x31, 0x4b, 0xb0, 0x6c, 0xaa, 0x4a, - 0x07, 0x42, 0x42, 0x0e, 0x51, 0x40, 0x98, 0xe8, 0x57, 0xa6, 0x46, 0x15, 0x32, 0x8e, 0x7d, 0xcf, - 0xf9, 0x02, 0xa0, 0x68, 0x67, 0x5c, 0x92, 0x91, 0xdf, 0x80, 0x3a, 0x0b, 0x7c, 0x6d, 0x30, 0x9b, - 0x2a, 0xc2, 0x39, 0x80, 0x66, 0xa9, 0x09, 0x22, 0xa1, 0xc6, 0x82, 0x60, 0x70, 0xc6, 0x2f, 0x04, - 0x8e, 0xb5, 0x68, 0x83, 0x05, 0xc1, 0x37, 0xfc, 0x42, 0x90, 0x3b, 0xb8, 0x25, 0x36, 0xd6, 0xa7, - 0x5a, 0x9a, 0xed, 0x9f, 0x50, 0x25, 0x74, 0x3e, 0x05, 0x53, 0xd5, 0xee, 0x25, 0xa4, 0x1b, 0x57, - 0xbe, 0x53, 0x4f, 0xf5, 0x9e, 0xb1, 0xd2, 0x27, 0x0f, 0x75, 0x9f, 0x46, 0xa8, 0xae, 0x90, 0x51, - 0xa4, 0x67, 0x4a, 0x49, 0xb7, 0x68, 0x50, 0xd9, 0xd9, 0x01, 0xeb, 0x8d, 0x9d, 0x2f, 0x6d, 0x80, - 0x4a, 0x61, 0x80, 0x4b, 0x7a, 0x61, 0xce, 0xf7, 0x00, 0x45, 0x3f, 0x47, 0x3b, 0x9e, 0x9a, 0x45, - 0x3a, 0xde, 0x03, 0x59, 0x4a, 0xf9, 0x81, 0x97, 0xf0, 0x70, 0xe6, 0xd4, 0x45, 0x07, 0x28, 0x97, - 0x93, 0x55, 0xa8, 0x61, 0x9b, 0xaa, 0x5a, 0x04, 0xc6, 0xbc, 0x47, 0x85, 0x12, 0x67, 0x08, 0x8b, - 0xea, 0xf9, 0xa3, 0xfc, 0x5f, 0x27, 0x5c, 0xbc, 0x31, 0x6f, 0x5a, 0x01, 0xc8, 0xc3, 0x78, 0xd6, - 0x70, 0x2b, 0x71, 0x24, 0xfc, 0x4f, 0x7c, 0x1e, 0x78, 0xd9, 0x69, 0x34, 0xe5, 0xfc, 0x2d, 0xb4, - 0xb2, 0x35, 0xb0, 0xec, 0x7f, 0x98, 0x3f, 0xc2, 0xca, 0x9a, 0xaa, 0xda, 0x50, 0x2a, 0x07, 0x91, - 0xc7, 0xb7, 0x2a, 0x1d, 0x23, 0x7b, 0x87, 0x9d, 0x5f, 0x57, 0xb3, 0xd1, 0xba, 0x0a, 0x9e, 0xc9, - 0xde, 0x8c, 0xf9, 0xec, 0x6d, 0x36, 0x4d, 0xaa, 0xfc, 0xa4, 0x34, 0xe9, 0x4b, 0xb0, 0x3d, 0xcc, - 0x15, 0xfc, 0xf3, 0x2c, 0x38, 0x2f, 0xcf, 0xe7, 0x05, 0x3a, 0x9b, 0xf0, 0xcf, 0x39, 0x2d, 0x94, - 0xe5, 0x5e, 0xd2, 0xe8, 0x8c, 0x87, 0xfe, 0x4b, 0x2c, 0xf3, 0xe5, 0xa1, 0x0b, 0x46, 0xd1, 0x33, - 0x51, 0xf9, 0x83, 0xee, 0x99, 0x64, 0xed, 0x1f, 0xb3, 0x68, 0xff, 0x48, 0xcb, 0x4d, 0x62, 0xc1, - 0x93, 0x34, 0xcb, 0x23, 0x15, 0x95, 0xe7, 0x63, 0xb6, 0xd6, 0x95, 0xf9, 0xd8, 0x47, 0xd0, 0x0a, - 0xa3, 0x70, 0x10, 0x4e, 0x82, 0x40, 0x66, 0xba, 0xba, 0xa7, 0xd7, 0x0c, 0xa3, 0xf0, 0x40, 0xb3, - 0xc8, 0x03, 0x78, 0xa7, 0xac, 0xa2, 0x90, 0xdb, 0x54, 0x8d, 0x82, 0x92, 0x1e, 0xe2, 0x7b, 0x0d, - 0xda, 0xd1, 0xf0, 0x7b, 0xee, 0xa6, 0x68, 0xb1, 0x01, 0x42, 0xb6, 0xa5, 0x9e, 0x68, 0xc5, 0x97, - 0x26, 0x3a, 0x60, 0x63, 0xee, 0x3c, 0x05, 0x3b, 0x37, 0x82, 0x8c, 0xc6, 0x07, 0x87, 0x07, 0x5d, - 0x15, 0x2f, 0x77, 0x0f, 0x76, 0xba, 0xff, 0xd8, 0x36, 0x64, 0x3c, 0xa7, 0xdd, 0x17, 0x5d, 0xda, - 0xeb, 0xb6, 0x2b, 0x32, 0xd6, 0xee, 0x74, 0xf7, 0xba, 0xfd, 0x6e, 0xbb, 0xfa, 0x75, 0xcd, 0x6a, - 0xb4, 0x2d, 0x6a, 0xf1, 0x69, 0x1c, 0xf8, 0xae, 0x9f, 0x3a, 0x3d, 0x00, 0x39, 0xad, 0xbe, 0xd1, - 0x5b, 0x60, 0x17, 0x6b, 0xab, 0x1b, 0xb5, 0x52, 0xbd, 0xaa, 0xcc, 0xd8, 0x34, 0xa8, 0x2a, 0x57, - 0x65, 0x6c, 0x1a, 0x66, 0xc7, 0x60, 0xed, 0xb3, 0xf8, 0xb5, 0x22, 0xa6, 0x95, 0x97, 0xaa, 0x13, - 0xdd, 0xb8, 0xd1, 0x0f, 0xf5, 0x5d, 0x68, 0xe8, 0xb8, 0xa7, 0xfd, 0x63, 0x26, 0x26, 0x66, 0x32, - 0x59, 0x53, 0xdc, 0xd8, 0x8f, 0xce, 0x79, 0x9e, 0xab, 0x1c, 0xb1, 0x8b, 0x20, 0x62, 0xde, 0x5b, - 0x80, 0xf8, 0x21, 0x80, 0x88, 0x26, 0x89, 0xcb, 0x07, 0xa3, 0xbc, 0x5f, 0x64, 0x2b, 0xce, 0x73, - 0xdd, 0x84, 0xe6, 0x22, 0x45, 0x61, 0x55, 0xb9, 0x99, 0xa4, 0xa5, 0xe8, 0x5d, 0x30, 0xd3, 0x69, - 0x58, 0xb4, 0xa7, 0xea, 0xa9, 0xac, 0x20, 0x9d, 0x6d, 0xb0, 0xfb, 0x53, 0xac, 0xab, 0x26, 0x62, - 0xe6, 0xf5, 0x35, 0xde, 0xf0, 0xfa, 0x56, 0xe6, 0x5e, 0xdf, 0xdf, 0x19, 0xd0, 0x2c, 0x25, 0x51, - 0xe4, 0x23, 0xa8, 0xa5, 0xd3, 0x70, 0xb6, 0xaf, 0x9b, 0x2d, 0x42, 0x51, 0x24, 0xf1, 0x26, 0x8b, - 0x2e, 0x26, 0x84, 0x3f, 0x0a, 0xb9, 0xa7, 0xa7, 0x94, 0x85, 0xd8, 0xa6, 0x66, 0x91, 0x3d, 0xb8, - 0xa6, 0x62, 0x46, 0xd6, 0xd3, 0xc9, 0xf2, 0xf0, 0x8f, 0xe7, 0x92, 0x36, 0x55, 0x7b, 0x6e, 0x67, - 0x5a, 0xaa, 0xba, 0x5e, 0x1a, 0xcd, 0x30, 0x97, 0x37, 0xe1, 0xfa, 0x25, 0x6a, 0x3f, 0xab, 0x8d, - 0x70, 0x1b, 0x16, 0x65, 0xd9, 0xed, 0x8f, 0xb9, 0x48, 0xd9, 0x38, 0xc6, 0xec, 0x45, 0xc7, 0xfc, - 0x1a, 0xad, 0xa4, 0xc2, 0xf9, 0x04, 0x5a, 0x47, 0x9c, 0x27, 0x94, 0x8b, 0x38, 0x0a, 0xd5, 0xcb, - 0x2d, 0xf0, 0xd0, 0xfa, 0x81, 0xd1, 0x94, 0xf3, 0x2f, 0x60, 0xcb, 0xbc, 0x7c, 0x8b, 0xa5, 0xee, - 0xe9, 0xcf, 0xc9, 0xdb, 0x3f, 0x81, 0x46, 0xac, 0x60, 0xa2, 0xb3, 0xec, 0x16, 0xc6, 0x38, 0x0d, - 0x1d, 0x9a, 0x09, 0x9d, 0x2f, 0xa0, 0x7a, 0x30, 0x19, 0x97, 0xff, 0x76, 0xa9, 0xa9, 0xbf, 0x5d, - 0x6e, 0x81, 0x8d, 0x29, 0x3e, 0xf6, 0xf8, 0x54, 0xf2, 0x69, 0x49, 0x06, 0x36, 0xf7, 0xbe, 0x83, - 0x66, 0x66, 0xfb, 0x5d, 0x0f, 0xff, 0x3b, 0xc1, 0xcb, 0xdf, 0xf5, 0x66, 0xb0, 0xa0, 0xaa, 0x3c, - 0x1e, 0x7a, 0xbb, 0xd9, 0xa5, 0x29, 0x62, 0x76, 0x6e, 0xfd, 0x62, 0xe7, 0x73, 0x3f, 0x83, 0x56, - 0x96, 0x3b, 0xef, 0xf3, 0x94, 0x21, 0x9c, 0x02, 0x9f, 0x87, 0x25, 0xa8, 0x59, 0x8a, 0xd1, 0x17, - 0x6f, 0x68, 0x89, 0x3a, 0xeb, 0x60, 0x6a, 0xac, 0x12, 0xa8, 0xb9, 0x91, 0xa7, 0x5c, 0xa4, 0x4e, - 0xf1, 0x5b, 0x1e, 0x78, 0x2c, 0x46, 0xd9, 0x43, 0x38, 0x16, 0x23, 0x67, 0x0a, 0x8b, 0x5b, 0xcc, - 0x3d, 0x9b, 0xc4, 0xd9, 0x43, 0x54, 0x2a, 0x72, 0x8c, 0x99, 0x22, 0xe7, 0x0d, 0x7d, 0xd8, 0xf7, - 0xa0, 0x31, 0x09, 0xfd, 0x69, 0x96, 0xbc, 0xd8, 0xd4, 0x94, 0xa4, 0xea, 0x3f, 0x06, 0x91, 0x8b, - 0x75, 0x0d, 0x7a, 0x95, 0x4d, 0x73, 0xda, 0xf9, 0x67, 0x58, 0xec, 0x4e, 0x63, 0xec, 0x4a, 0xbf, - 0xf5, 0x09, 0x2c, 0x6d, 0xaa, 0x32, 0xb3, 0xa9, 0xb9, 0x95, 0xab, 0xd9, 0xca, 0x1b, 0xff, 0x6f, - 0x40, 0x4d, 0x42, 0x44, 0x96, 0x5e, 0x7f, 0xcf, 0x59, 0x92, 0x0e, 0x39, 0x4b, 0xc9, 0x0c, 0x1c, - 0x96, 0x67, 0x28, 0x67, 0xe1, 0x89, 0x41, 0x3e, 0x55, 0x0d, 0xef, 0xac, 0x8f, 0xbf, 0x98, 0x01, - 0x0d, 0x81, 0x38, 0xaf, 0x4f, 0xd6, 0xa1, 0xf9, 0x75, 0xe4, 0x87, 0xdb, 0xaa, 0x07, 0x4c, 0xe6, - 0x61, 0xf9, 0x9a, 0xfe, 0x23, 0x30, 0x77, 0x85, 0xc4, 0xff, 0xeb, 0xaa, 0x18, 0x5e, 0xcb, 0xae, - 0xe1, 0x2c, 0x6c, 0xfc, 0x5f, 0x15, 0x6a, 0xdf, 0xf1, 0x24, 0x22, 0x9f, 0x42, 0x43, 0x77, 0x78, - 0x48, 0xa9, 0x93, 0xb3, 0x8c, 0xef, 0xeb, 0x5c, 0xeb, 0x07, 0x57, 0x69, 0xab, 0x08, 0x5d, 0xd4, - 0x86, 0xa4, 0x68, 0x40, 0xbd, 0xb6, 0xa9, 0xa7, 0xd0, 0xee, 0xa5, 0x09, 0x67, 0xe3, 0x92, 0xfa, - 0xac, 0x99, 0x2e, 0x2b, 0x34, 0xd1, 0x5a, 0x0f, 0xc1, 0x54, 0x41, 0x66, 0x6e, 0xc0, 0x7c, 0xcd, - 0x88, 0xca, 0xf7, 0xa0, 0xd9, 0x3b, 0x8d, 0x26, 0x81, 0xd7, 0xe3, 0xc9, 0x39, 0x27, 0xa5, 0x2e, - 0xeb, 0x72, 0xe9, 0xdb, 0x59, 0x20, 0x6b, 0x00, 0xca, 0xcb, 0x8e, 0x7d, 0x4f, 0x90, 0x86, 0x94, - 0x1d, 0x4c, 0xc6, 0x6a, 0xd2, 0x92, 0xfb, 0x29, 0xcd, 0x52, 0xac, 0x79, 0x93, 0xe6, 0xe7, 0xb0, - 0xb8, 0x8d, 0xb1, 0xf8, 0x30, 0xd9, 0x1c, 0x46, 0x49, 0x4a, 0xe6, 0x3b, 0xad, 0xcb, 0xf3, 0x0c, - 0x67, 0x81, 0x3c, 0x01, 0xab, 0x9f, 0x5c, 0x28, 0xfd, 0x77, 0x74, 0x88, 0x2e, 0xd6, 0xbb, 0xe4, - 0x94, 0x1b, 0xff, 0x5d, 0x05, 0xf3, 0xdb, 0x28, 0x39, 0xe3, 0x09, 0x79, 0x00, 0x26, 0x16, 0xf7, - 0x1a, 0x44, 0x79, 0xa1, 0x7f, 0xd9, 0x42, 0x77, 0xc0, 0x46, 0xa3, 0xf4, 0x99, 0x38, 0x53, 0x57, - 0x85, 0x7f, 0xbc, 0x2a, 0xbb, 0xa8, 0xe4, 0x0d, 0xef, 0x75, 0x49, 0x5d, 0x54, 0xde, 0xd0, 0x98, - 0xa9, 0xb8, 0x97, 0x1b, 0xaa, 0x7c, 0xee, 0x39, 0x0b, 0x6b, 0xc6, 0x13, 0x83, 0xdc, 0x87, 0x5a, - 0x4f, 0x9d, 0x54, 0x2a, 0x15, 0x7f, 0x4e, 0x2d, 0x2f, 0x65, 0x8c, 0x7c, 0xe6, 0xc7, 0x60, 0xaa, - 0x97, 0x5d, 0x1d, 0x73, 0x26, 0x31, 0x5d, 0x6e, 0x97, 0x59, 0x7a, 0xc0, 0x7d, 0x30, 0x55, 0xd0, - 0x50, 0x03, 0x66, 0x02, 0x88, 0xda, 0xb5, 0x8a, 0x41, 0x4a, 0x55, 0x79, 0xb9, 0x52, 0x9d, 0xf1, - 0xf8, 0x39, 0xd5, 0x47, 0xd0, 0xa6, 0xdc, 0xe5, 0x7e, 0xe9, 0xcd, 0x27, 0xd9, 0xa1, 0xe6, 0x61, - 0xbb, 0x66, 0x90, 0xa7, 0xb0, 0x38, 0x93, 0x1f, 0x90, 0x0e, 0x1a, 0xfa, 0x92, 0x94, 0x61, 0x7e, - 0xf0, 0x56, 0xfb, 0x97, 0xaf, 0x56, 0x8c, 0x5f, 0xbd, 0x5a, 0x31, 0x7e, 0xf3, 0x6a, 0xc5, 0xf8, - 0xe1, 0xb7, 0x2b, 0x0b, 0x43, 0x13, 0xff, 0xb0, 0xff, 0xfc, 0x4f, 0x01, 0x00, 0x00, 0xff, 0xff, - 0xa0, 0x45, 0x23, 0xbc, 0xf4, 0x1f, 0x00, 0x00, + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4b, 0x73, 0x1b, 0x49, + 0x72, 0x66, 0x37, 0x80, 0x46, 0x77, 0x02, 0xa4, 0x30, 0x25, 0xad, 0x06, 0x43, 0xed, 0x52, 0x9c, + 0x1e, 0x69, 0x44, 0x49, 0x23, 0x4a, 0xc3, 0x19, 0xdb, 0xab, 0x75, 0xf8, 0xc0, 0x07, 0x24, 0x73, + 0x86, 0x2f, 0x17, 0x40, 0xad, 0x3d, 0xe1, 0x30, 0xa2, 0xd0, 0x5d, 0x04, 0x7b, 0xd9, 0xe8, 0x6e, + 0x77, 0x35, 0x18, 0xa0, 0x6e, 0x3e, 0xd8, 0xbf, 0x61, 0x0f, 0x0e, 0xdf, 0x7c, 0xb0, 0x2f, 0xbe, + 0xee, 0xd1, 0x17, 0x47, 0xf8, 0xe8, 0xf0, 0x2f, 0x70, 0xc8, 0xfe, 0x07, 0x8e, 0x70, 0x84, 0x6f, + 0x8e, 0xca, 0xaa, 0x7e, 0x00, 0x22, 0xa5, 0x1d, 0x47, 0xec, 0x09, 0x9d, 0x8f, 0x7a, 0x7d, 0x95, + 0x99, 0x95, 0x99, 0x00, 0x3b, 0x19, 0x6d, 0x26, 0x69, 0x9c, 0xc5, 0xc4, 0x4c, 0x46, 0xab, 0x0e, + 0x4b, 0x02, 0x45, 0xae, 0x3e, 0x1a, 0x07, 0xd9, 0xf9, 0x74, 0xb4, 0xe9, 0xc5, 0x93, 0xe7, 0xfe, + 0x38, 0x65, 0xc9, 0xf9, 0xb3, 0x20, 0x7e, 0x3e, 0x62, 0xfe, 0x98, 0xa7, 0xcf, 0x93, 0xd1, 0xf3, + 0x7c, 0x9c, 0xbb, 0x0a, 0xf5, 0x83, 0x40, 0x64, 0x84, 0x40, 0x7d, 0x1a, 0xf8, 0xa2, 0x6b, 0xac, + 0xd7, 0x36, 0x2c, 0x8a, 0xdf, 0xee, 0x21, 0x38, 0x03, 0x26, 0x2e, 0xde, 0xb0, 0x70, 0xca, 0x49, + 0x07, 0x6a, 0x97, 0x2c, 0xec, 0x1a, 0xeb, 0xc6, 0x46, 0x9b, 0xca, 0x4f, 0xb2, 0x09, 0xf6, 0x25, + 0x0b, 0x87, 0xd9, 0x55, 0xc2, 0xbb, 0xe6, 0xba, 0xb1, 0xb1, 0xb2, 0x75, 0x7b, 0x33, 0x19, 0x6d, + 0x9e, 0xc4, 0x22, 0x0b, 0xa2, 0xf1, 0xe6, 0x1b, 0x16, 0x0e, 0xae, 0x12, 0x4e, 0x9b, 0x97, 0xea, + 0xc3, 0x3d, 0x86, 0x56, 0x3f, 0xf5, 0x5e, 0x4d, 0x23, 0x2f, 0x0b, 0xe2, 0x48, 0xae, 0x18, 0xb1, + 0x09, 0xc7, 0x19, 0x1d, 0x8a, 0xdf, 0x92, 0xc7, 0xd2, 0xb1, 0xe8, 0xd6, 0xd6, 0x6b, 0x92, 0x27, + 0xbf, 0x49, 0x17, 0x9a, 0x81, 0xd8, 0x8d, 0xa7, 0x51, 0xd6, 0xad, 0xaf, 0x1b, 0x1b, 0x36, 0xcd, + 0x49, 0xf7, 0xbf, 0x4d, 0x68, 0xfc, 0xc9, 0x94, 0xa7, 0x57, 0x38, 0x2e, 0xcb, 0xd2, 0x7c, 0x2e, + 0xf9, 0x4d, 0xee, 0x40, 0x23, 0x64, 0xd1, 0x58, 0x74, 0x4d, 0x9c, 0x4c, 0x11, 0xe4, 0x1e, 0x38, + 0xec, 0x2c, 0xe3, 0xe9, 0x70, 0x1a, 0xf8, 0xdd, 0xda, 0xba, 0xb1, 0x61, 0x51, 0x1b, 0x19, 0xa7, + 0x81, 0x4f, 0x3e, 0x03, 0xdb, 0x8f, 0x87, 0x5e, 0x75, 0x2d, 0x3f, 0xc6, 0xb5, 0xc8, 0x17, 0x60, + 0x4f, 0x03, 0x7f, 0x18, 0x06, 0x22, 0xeb, 0x36, 0xd6, 0x8d, 0x8d, 0xd6, 0x96, 0x2d, 0x0f, 0x2b, + 0xb1, 0xa3, 0xcd, 0x69, 0xe0, 0x23, 0x88, 0x4f, 0xc0, 0x16, 0xa9, 0x37, 0x3c, 0x9b, 0x46, 0x5e, + 0xd7, 0x42, 0xa5, 0x5b, 0x52, 0xa9, 0x72, 0x6a, 0xda, 0x14, 0x8a, 0x90, 0xc7, 0x4a, 0xf9, 0x25, + 0x4f, 0x05, 0xef, 0x36, 0xd5, 0x52, 0x9a, 0x24, 0x2f, 0xa0, 0x75, 0xc6, 0x3c, 0x9e, 0x0d, 0x13, + 0x96, 0xb2, 0x49, 0xd7, 0x2e, 0x27, 0x7a, 0x25, 0xd9, 0x27, 0x92, 0x2b, 0x28, 0x9c, 0x15, 0x04, + 0xf9, 0x06, 0x96, 0x91, 0x12, 0xc3, 0xb3, 0x20, 0xcc, 0x78, 0xda, 0x75, 0x70, 0xcc, 0x0a, 0x8e, + 0x41, 0xce, 0x20, 0xe5, 0x9c, 0xb6, 0x95, 0x92, 0xe2, 0x90, 0x9f, 0x01, 0xf0, 0x59, 0xc2, 0x22, + 0x7f, 0xc8, 0xc2, 0xb0, 0x0b, 0xb8, 0x07, 0x47, 0x71, 0xb6, 0xc3, 0x90, 0x7c, 0x2a, 0xf7, 0xc7, + 0xfc, 0x61, 0x26, 0xba, 0xcb, 0xeb, 0xc6, 0x46, 0x9d, 0x5a, 0x92, 0x1c, 0x08, 0x77, 0x0b, 0x1c, + 0xb4, 0x08, 0x3c, 0xf1, 0x43, 0xb0, 0x2e, 0x25, 0xa1, 0x0c, 0xa7, 0xb5, 0xb5, 0x2c, 0x97, 0x2c, + 0x8c, 0x86, 0x6a, 0xa1, 0xbb, 0x06, 0xf6, 0x01, 0x8b, 0xc6, 0xb9, 0xa5, 0xc9, 0xab, 0xc0, 0x01, + 0x0e, 0xc5, 0x6f, 0xf7, 0xd7, 0x26, 0x58, 0x94, 0x8b, 0x69, 0x98, 0x91, 0x47, 0x00, 0x12, 0xe8, + 0x09, 0xcb, 0xd2, 0x60, 0xa6, 0x67, 0x2d, 0xa1, 0x76, 0xa6, 0x81, 0x7f, 0x88, 0x22, 0xf2, 0x02, + 0xda, 0x38, 0x7b, 0xae, 0x6a, 0x96, 0x1b, 0x28, 0xf6, 0x47, 0x5b, 0xa8, 0xa2, 0x47, 0xdc, 0x05, + 0x0b, 0xef, 0x56, 0xd9, 0xd7, 0x32, 0xd5, 0x14, 0x79, 0x08, 0x2b, 0x41, 0x94, 0x49, 0xec, 0xbd, + 0x6c, 0xe8, 0x73, 0x91, 0x5f, 0xfe, 0x72, 0xc1, 0xdd, 0xe3, 0x22, 0x23, 0x5f, 0x83, 0x02, 0x30, + 0x5f, 0xb0, 0x81, 0x0b, 0xae, 0x14, 0x17, 0x23, 0xd4, 0x8a, 0xa8, 0xa3, 0x57, 0x7c, 0x06, 0x2d, + 0x79, 0xbe, 0x7c, 0x84, 0x85, 0x23, 0xda, 0x78, 0x1a, 0x0d, 0x07, 0x05, 0xa9, 0xa0, 0xd5, 0x25, + 0x34, 0xd2, 0xc0, 0x94, 0x41, 0xe0, 0xb7, 0xdb, 0x83, 0xc6, 0x71, 0xea, 0xf3, 0xf4, 0x5a, 0x1b, + 0x27, 0x50, 0xf7, 0xb9, 0xf0, 0xd0, 0xfd, 0x6c, 0x8a, 0xdf, 0xa5, 0xdd, 0xd7, 0x2a, 0x76, 0xef, + 0xfe, 0x9d, 0x01, 0xad, 0x7e, 0x9c, 0x66, 0x87, 0x5c, 0x08, 0x36, 0xe6, 0xe4, 0x3e, 0x34, 0x62, + 0x39, 0xad, 0x46, 0xd8, 0x91, 0x7b, 0xc2, 0x75, 0xa8, 0xe2, 0x2f, 0xdc, 0x83, 0x79, 0xf3, 0x3d, + 0xdc, 0x81, 0x86, 0xf2, 0x18, 0xe9, 0x4d, 0x0d, 0xaa, 0x08, 0x89, 0x75, 0x7c, 0x76, 0x26, 0xb8, + 0xc2, 0xb2, 0x41, 0x35, 0x75, 0xb3, 0x59, 0xfd, 0x1e, 0x80, 0xdc, 0xdf, 0x8f, 0xb4, 0x02, 0xf7, + 0x1c, 0x5a, 0x94, 0x9d, 0x65, 0xbb, 0x71, 0x94, 0xf1, 0x59, 0x46, 0x56, 0xc0, 0x0c, 0x7c, 0x84, + 0xc8, 0xa2, 0x66, 0xe0, 0xcb, 0xcd, 0x8d, 0xd3, 0x78, 0x9a, 0x20, 0x42, 0xcb, 0x54, 0x11, 0x08, + 0xa5, 0xef, 0xa7, 0xb8, 0x63, 0x09, 0xa5, 0xef, 0xa7, 0xe4, 0x3e, 0xb4, 0x44, 0xc4, 0x12, 0x71, + 0x1e, 0x67, 0x72, 0x73, 0x75, 0xdc, 0x1c, 0xe4, 0xac, 0x81, 0x70, 0xff, 0xc5, 0x00, 0xeb, 0x90, + 0x4f, 0x46, 0x3c, 0x7d, 0x6f, 0x95, 0xcf, 0xc0, 0xc6, 0x89, 0x87, 0x81, 0xaf, 0x17, 0x6a, 0x22, + 0xbd, 0xef, 0x5f, 0xbb, 0xd4, 0x5d, 0xb0, 0x42, 0xce, 0x24, 0xf8, 0xca, 0xce, 0x34, 0x25, 0xb1, + 0x61, 0x93, 0xa1, 0xcf, 0x99, 0x8f, 0x21, 0xc6, 0xa6, 0x16, 0x9b, 0xec, 0x71, 0xe6, 0xcb, 0xbd, + 0x85, 0x4c, 0x64, 0xc3, 0x69, 0xe2, 0xb3, 0x8c, 0x63, 0x68, 0xa9, 0x4b, 0xc3, 0x11, 0xd9, 0x29, + 0x72, 0xc8, 0x13, 0xf8, 0xc4, 0x0b, 0xa7, 0x42, 0xc6, 0xb5, 0x20, 0x3a, 0x8b, 0x87, 0x71, 0x14, + 0x5e, 0x21, 0xbe, 0x36, 0xbd, 0xa5, 0x05, 0xfb, 0xd1, 0x59, 0x7c, 0x1c, 0x85, 0x57, 0xee, 0x6f, + 0x4c, 0x68, 0xbc, 0x46, 0x18, 0x5e, 0x40, 0x73, 0x82, 0x07, 0xca, 0xbd, 0xf7, 0xae, 0x44, 0x18, + 0x65, 0x9b, 0xea, 0xa4, 0xa2, 0x17, 0x65, 0xe9, 0x15, 0xcd, 0xd5, 0xe4, 0x88, 0x8c, 0x8d, 0x42, + 0x9e, 0x09, 0x6d, 0x11, 0x95, 0x11, 0x03, 0x25, 0xd0, 0x23, 0xb4, 0xda, 0x22, 0xac, 0xb5, 0x45, + 0x58, 0xc9, 0x2a, 0xd8, 0xde, 0x39, 0xf7, 0x2e, 0xc4, 0x74, 0xa2, 0x41, 0x2f, 0xe8, 0xd5, 0x57, + 0xd0, 0xae, 0xee, 0x43, 0xbe, 0x41, 0x17, 0xfc, 0x0a, 0x81, 0xaf, 0x53, 0xf9, 0x49, 0xd6, 0xa1, + 0x81, 0x1e, 0x8e, 0xb0, 0xb7, 0xb6, 0x40, 0x6e, 0x47, 0x0d, 0xa1, 0x4a, 0xf0, 0x0b, 0xf3, 0xe7, + 0x86, 0x9c, 0xa7, 0xba, 0xbb, 0xea, 0x3c, 0xce, 0xcd, 0xf3, 0xa8, 0x21, 0x95, 0x79, 0xdc, 0xff, + 0x35, 0xa1, 0xfd, 0x03, 0x4f, 0xe3, 0x93, 0x34, 0x4e, 0x62, 0xc1, 0x42, 0xb2, 0x3d, 0x7f, 0x3a, + 0x85, 0xe2, 0xba, 0x1c, 0x5c, 0x55, 0xdb, 0xec, 0x17, 0xc7, 0x55, 0xe8, 0x54, 0xcf, 0xef, 0x82, + 0xa5, 0xd0, 0xbd, 0xe6, 0x08, 0x5a, 0x22, 0x75, 0x14, 0x9e, 0x88, 0xdf, 0xfc, 0xf6, 0xb4, 0x84, + 0xac, 0x01, 0x4c, 0xd8, 0xec, 0x80, 0x33, 0xc1, 0xf7, 0xfd, 0xdc, 0x7c, 0x4b, 0x8e, 0xc4, 0x79, + 0xc2, 0x66, 0x83, 0x59, 0x34, 0x10, 0x68, 0x5d, 0x75, 0x5a, 0xd0, 0xe4, 0xa7, 0xe0, 0x4c, 0xd8, + 0x4c, 0xfa, 0xd1, 0xbe, 0xaf, 0xad, 0xab, 0x64, 0x90, 0xcf, 0xa1, 0x96, 0xcd, 0x22, 0x0c, 0x4a, + 0xf2, 0x1d, 0x92, 0x49, 0xc6, 0x60, 0x16, 0x69, 0x8f, 0xa3, 0x52, 0x96, 0x03, 0x6a, 0x97, 0x80, + 0x76, 0xa0, 0xe6, 0x05, 0x3e, 0x3e, 0x44, 0x0e, 0x95, 0x9f, 0xab, 0x7f, 0x04, 0xb7, 0x16, 0x70, + 0xa8, 0xde, 0xc3, 0xb2, 0x1a, 0x76, 0xa7, 0x7a, 0x0f, 0xf5, 0x2a, 0xf6, 0xbf, 0xa9, 0xc1, 0x2d, + 0x6d, 0x0c, 0xe7, 0x41, 0xd2, 0xcf, 0xa4, 0xd9, 0x77, 0xa1, 0x89, 0xd1, 0x86, 0xa7, 0xda, 0x26, + 0x72, 0x92, 0xfc, 0x01, 0x58, 0xe8, 0x81, 0xb9, 0x9d, 0xde, 0x2f, 0x51, 0x2d, 0x86, 0x2b, 0xbb, + 0xd5, 0x57, 0xa2, 0xd5, 0xc9, 0xb7, 0xd0, 0x78, 0xcb, 0xd3, 0x58, 0x45, 0xcf, 0xd6, 0xd6, 0xda, + 0x75, 0xe3, 0xe4, 0xdd, 0xea, 0x61, 0x4a, 0xf9, 0x77, 0x08, 0xfe, 0x03, 0x19, 0x2f, 0x27, 0xf1, + 0x25, 0xf7, 0xbb, 0x4d, 0xdc, 0x51, 0xd5, 0x3e, 0x72, 0x51, 0x8e, 0xb6, 0x5d, 0xa2, 0xbd, 0x07, + 0xad, 0xca, 0xf1, 0xae, 0x41, 0xfa, 0xfe, 0xbc, 0xc5, 0x3b, 0x85, 0x23, 0x57, 0x1d, 0x67, 0x0f, + 0xa0, 0x3c, 0xec, 0xff, 0xd7, 0xfd, 0xdc, 0xbf, 0x32, 0xe0, 0xd6, 0x6e, 0x1c, 0x45, 0x1c, 0x53, + 0x20, 0x75, 0x75, 0xa5, 0xd9, 0x1b, 0x37, 0x9a, 0xfd, 0x63, 0x68, 0x08, 0xa9, 0xac, 0x67, 0xbf, + 0x7d, 0xcd, 0x5d, 0x50, 0xa5, 0x21, 0xc3, 0xcc, 0x84, 0xcd, 0x86, 0x09, 0x8f, 0xfc, 0x20, 0x1a, + 0xe7, 0x61, 0x66, 0xc2, 0x66, 0x27, 0x8a, 0xe3, 0xfe, 0xb5, 0x01, 0x96, 0xf2, 0x98, 0xb9, 0x68, + 0x6d, 0xcc, 0x47, 0xeb, 0x9f, 0x82, 0x93, 0xa4, 0xdc, 0x0f, 0xbc, 0x7c, 0x55, 0x87, 0x96, 0x0c, + 0x69, 0x9c, 0x67, 0x71, 0xea, 0x71, 0x9c, 0xde, 0xa6, 0x8a, 0x90, 0x5c, 0x91, 0x30, 0x4f, 0xa5, + 0x71, 0x35, 0xaa, 0x08, 0x19, 0xe3, 0xd5, 0xe5, 0xe0, 0xa5, 0xd8, 0x54, 0x53, 0xee, 0x3f, 0x9a, + 0xd0, 0xde, 0x0b, 0x52, 0xee, 0x65, 0xdc, 0xef, 0xf9, 0x63, 0x54, 0xe4, 0x51, 0x16, 0x64, 0x57, + 0xfa, 0x3d, 0xd1, 0x54, 0xf1, 0xdc, 0x9b, 0xf3, 0x29, 0xad, 0x82, 0xbb, 0x86, 0x59, 0xb8, 0x22, + 0xc8, 0x16, 0x80, 0x4a, 0x84, 0x30, 0x13, 0xaf, 0xdf, 0x9c, 0x89, 0x3b, 0xa8, 0x26, 0x3f, 0x25, + 0x06, 0x6a, 0x4c, 0xa0, 0xde, 0x1a, 0x0b, 0xd3, 0xf4, 0xa9, 0xb4, 0x55, 0xcc, 0x1f, 0x46, 0x3c, + 0x44, 0x5b, 0xc4, 0xfc, 0x61, 0xc4, 0xc3, 0x22, 0x6b, 0x6b, 0xaa, 0xed, 0xc8, 0x6f, 0xf2, 0x05, + 0x98, 0x71, 0x82, 0xe7, 0xd3, 0x0b, 0x56, 0x0f, 0xb6, 0x79, 0x9c, 0x50, 0x33, 0x4e, 0xe4, 0x45, + 0xab, 0xb4, 0xb3, 0xeb, 0x68, 0xfb, 0x95, 0x01, 0x04, 0x13, 0x26, 0xaa, 0x25, 0xee, 0x5d, 0x30, + 0x8f, 0x13, 0xd2, 0x84, 0x5a, 0xbf, 0x37, 0xe8, 0x2c, 0xc9, 0x8f, 0xbd, 0xde, 0x41, 0xc7, 0x70, + 0xdf, 0x19, 0xe0, 0x1c, 0x4e, 0x33, 0x26, 0xcd, 0x46, 0x7c, 0xe8, 0xde, 0x3e, 0x03, 0x5b, 0x64, + 0x2c, 0xc5, 0x20, 0xac, 0x22, 0x47, 0x13, 0xe9, 0x81, 0x20, 0x5f, 0x42, 0x83, 0xfb, 0x63, 0x9e, + 0x3b, 0x74, 0x67, 0x71, 0x9f, 0x54, 0x89, 0xc9, 0x06, 0x58, 0xc2, 0x3b, 0xe7, 0x13, 0xd6, 0xad, + 0x97, 0x8a, 0x7d, 0xe4, 0xa8, 0x47, 0x96, 0x6a, 0x39, 0x56, 0x09, 0x69, 0x9c, 0x60, 0xda, 0xdc, + 0xd0, 0x55, 0x42, 0x1a, 0x27, 0x32, 0x69, 0xde, 0x82, 0x9f, 0x04, 0xe3, 0x28, 0x4e, 0xf9, 0x30, + 0x88, 0x7c, 0x3e, 0x1b, 0x7a, 0x71, 0x74, 0x16, 0x06, 0x5e, 0x86, 0x58, 0xda, 0xf4, 0xb6, 0x12, + 0xee, 0x4b, 0xd9, 0xae, 0x16, 0xb9, 0x33, 0xb0, 0xf3, 0xb8, 0x48, 0x1e, 0xcb, 0x80, 0x86, 0x71, + 0x55, 0xbb, 0x05, 0xa6, 0xfd, 0x95, 0x04, 0x87, 0xe6, 0x72, 0x79, 0x4d, 0xb8, 0x46, 0x1e, 0x29, + 0x91, 0xa8, 0xa6, 0x57, 0xb5, 0x6a, 0x7a, 0x85, 0x99, 0x62, 0x1c, 0x71, 0x9d, 0x71, 0xe0, 0xb7, + 0xfb, 0xb7, 0x26, 0xd8, 0xc5, 0x53, 0xf6, 0x14, 0x9c, 0x49, 0x0e, 0xb5, 0x76, 0x38, 0xcc, 0xa5, + 0x0b, 0xfc, 0x69, 0x29, 0x27, 0x77, 0xc1, 0xbc, 0xb8, 0xd4, 0x40, 0x59, 0x52, 0xeb, 0xfb, 0x37, + 0xd4, 0xbc, 0xb8, 0x2c, 0x3d, 0xb6, 0xf1, 0x51, 0x8f, 0x7d, 0x04, 0xb7, 0xbc, 0x90, 0xb3, 0x68, + 0x58, 0x3a, 0x9c, 0x32, 0xb8, 0x15, 0x64, 0x9f, 0x14, 0x5e, 0xa7, 0xa3, 0x4e, 0xb3, 0x7c, 0x5b, + 0x1e, 0x42, 0xc3, 0xe7, 0x61, 0xc6, 0xaa, 0xa5, 0xd1, 0x71, 0xca, 0xbc, 0x90, 0xef, 0x49, 0x36, + 0x55, 0x52, 0xb2, 0x01, 0x76, 0xfe, 0xce, 0xea, 0x82, 0x08, 0x33, 0xef, 0x1c, 0x6c, 0x5a, 0x48, + 0x4b, 0x2c, 0xa1, 0x82, 0xa5, 0xfb, 0x35, 0xd4, 0xbe, 0x7f, 0xd3, 0xd7, 0x67, 0x35, 0xde, 0x3b, + 0x6b, 0x8e, 0xa8, 0x59, 0x41, 0xf4, 0x7f, 0x6a, 0xd0, 0xd4, 0x5e, 0x27, 0xf7, 0x3d, 0x2d, 0xb2, + 0x44, 0xf9, 0x39, 0xff, 0xb8, 0x15, 0xee, 0x5b, 0x2d, 0xa3, 0x6b, 0x1f, 0x2f, 0xa3, 0xc9, 0x2f, + 0xa0, 0x9d, 0x28, 0x59, 0xd5, 0xe1, 0x3f, 0xad, 0x8e, 0xd1, 0xbf, 0x38, 0xae, 0x95, 0x94, 0x84, + 0x34, 0x5d, 0xac, 0x47, 0x32, 0x36, 0xc6, 0x2b, 0x6a, 0xd3, 0xa6, 0xa4, 0x07, 0x6c, 0x7c, 0x83, + 0xdb, 0xff, 0x16, 0xde, 0x2b, 0xb3, 0xe1, 0x38, 0xe9, 0xb6, 0xd1, 0x23, 0xa5, 0xc7, 0x57, 0x9d, + 0x71, 0x79, 0xde, 0x19, 0xef, 0x81, 0xe3, 0xc5, 0x93, 0x49, 0x80, 0xb2, 0x15, 0x9d, 0xed, 0x21, + 0x63, 0x20, 0xdc, 0xbf, 0x31, 0xa0, 0xa9, 0x4f, 0x4b, 0x5a, 0xd0, 0xdc, 0xeb, 0xbd, 0xda, 0x3e, + 0x3d, 0x90, 0xf1, 0x00, 0xc0, 0xda, 0xd9, 0x3f, 0xda, 0xa6, 0x7f, 0xd6, 0x31, 0x64, 0x6c, 0xd8, + 0x3f, 0x1a, 0x74, 0x4c, 0xe2, 0x40, 0xe3, 0xd5, 0xc1, 0xf1, 0xf6, 0xa0, 0x53, 0x23, 0x36, 0xd4, + 0x77, 0x8e, 0x8f, 0x0f, 0x3a, 0x75, 0xd2, 0x06, 0x7b, 0x6f, 0x7b, 0xd0, 0x1b, 0xec, 0x1f, 0xf6, + 0x3a, 0x0d, 0xa9, 0xfb, 0xba, 0x77, 0xdc, 0xb1, 0xe4, 0xc7, 0xe9, 0xfe, 0x5e, 0xa7, 0x29, 0xe5, + 0x27, 0xdb, 0xfd, 0xfe, 0x2f, 0x8f, 0xe9, 0x5e, 0xc7, 0x96, 0xf3, 0xf6, 0x07, 0x74, 0xff, 0xe8, + 0x75, 0xc7, 0x91, 0xdf, 0xc7, 0x3b, 0xdf, 0xf5, 0x76, 0x07, 0x1d, 0x70, 0xbf, 0x86, 0x56, 0x05, + 0x41, 0x39, 0x9a, 0xf6, 0x5e, 0x75, 0x96, 0xe4, 0x92, 0x6f, 0xb6, 0x0f, 0x4e, 0x7b, 0x1d, 0x83, + 0xac, 0x00, 0xe0, 0xe7, 0xf0, 0x60, 0xfb, 0xe8, 0x75, 0xc7, 0x74, 0x7f, 0x1f, 0xec, 0xd3, 0xc0, + 0xdf, 0x09, 0x63, 0xef, 0x42, 0x1a, 0xc6, 0x88, 0x09, 0xae, 0xdf, 0x49, 0xfc, 0x96, 0x51, 0x1e, + 0x8d, 0x52, 0xe8, 0xbb, 0xd7, 0x94, 0x7b, 0x04, 0xcd, 0xd3, 0xc0, 0x3f, 0x61, 0xde, 0x85, 0xac, + 0xc7, 0x47, 0x72, 0xfc, 0x50, 0x04, 0x6f, 0xb9, 0x0e, 0x70, 0x0e, 0x72, 0xfa, 0xc1, 0x5b, 0x4e, + 0x1e, 0x80, 0x85, 0x44, 0x9e, 0xd1, 0xa0, 0x2d, 0xe7, 0x6b, 0x52, 0x2d, 0x73, 0xff, 0xc1, 0x28, + 0xf6, 0x8e, 0xc5, 0xf6, 0x7d, 0xa8, 0x27, 0xcc, 0xbb, 0xd0, 0xd1, 0xa4, 0xa5, 0xc7, 0xc8, 0xf5, + 0x28, 0x0a, 0xc8, 0x23, 0xb0, 0xb5, 0x81, 0xe4, 0x13, 0xb7, 0x2a, 0x96, 0x44, 0x0b, 0xe1, 0xfc, + 0xd5, 0xd5, 0xe6, 0xaf, 0x4e, 0x1e, 0x4f, 0x24, 0x61, 0x80, 0x75, 0x53, 0x4d, 0x46, 0x1d, 0x45, + 0xc9, 0x41, 0xca, 0x14, 0xa6, 0xfa, 0x9d, 0xa9, 0x53, 0x65, 0x1b, 0xa7, 0x81, 0xef, 0x7e, 0x0b, + 0x50, 0x36, 0x34, 0xae, 0xc9, 0xc9, 0xef, 0x40, 0x83, 0x85, 0x81, 0x86, 0xcc, 0xa1, 0x8a, 0x70, + 0x8f, 0xa0, 0x55, 0x69, 0x83, 0x48, 0x63, 0x63, 0x61, 0x38, 0xbc, 0xe0, 0x57, 0x02, 0xc7, 0xda, + 0xb4, 0xc9, 0xc2, 0xf0, 0x7b, 0x7e, 0x25, 0xc8, 0x03, 0x68, 0xa8, 0x0e, 0x8a, 0xb9, 0x50, 0xa8, + 0xe3, 0x50, 0xaa, 0x84, 0xee, 0x57, 0x60, 0xa9, 0xea, 0xbd, 0x62, 0xeb, 0xc6, 0x8d, 0x2f, 0xd5, + 0x4b, 0xbd, 0x67, 0xac, 0xf5, 0xc9, 0x53, 0xdd, 0xa9, 0x11, 0xaa, 0x2f, 0x64, 0x94, 0x09, 0x9a, + 0x52, 0xd2, 0x4d, 0x1a, 0x54, 0x76, 0xf7, 0xc0, 0xfe, 0x60, 0xef, 0x4b, 0x03, 0x60, 0x96, 0x00, + 0x5c, 0xd3, 0x0d, 0x73, 0x7f, 0x05, 0x50, 0x76, 0x74, 0xb4, 0xeb, 0xa9, 0x59, 0xa4, 0xeb, 0x3d, + 0x91, 0xc5, 0x54, 0x10, 0xfa, 0x29, 0x8f, 0xe6, 0x4e, 0x5d, 0xf6, 0x80, 0x0a, 0x39, 0x59, 0x87, + 0x3a, 0x36, 0xaa, 0x6a, 0x65, 0x68, 0x2c, 0xba, 0x54, 0x28, 0x71, 0x47, 0xb0, 0xac, 0x1e, 0x40, + 0xca, 0xff, 0x72, 0xca, 0xc5, 0x07, 0x33, 0xa7, 0x35, 0x80, 0x22, 0x90, 0xe7, 0x2d, 0xb7, 0x0a, + 0x47, 0x5a, 0xc8, 0x59, 0xc0, 0x43, 0x3f, 0x3f, 0x8d, 0xa6, 0xdc, 0x3f, 0x84, 0x76, 0xbe, 0x06, + 0x16, 0xfe, 0x4f, 0x8b, 0x67, 0x58, 0xa1, 0xa9, 0xea, 0x0d, 0xa5, 0x72, 0x14, 0xfb, 0x7c, 0xc7, + 0xec, 0x1a, 0xf9, 0x4b, 0xec, 0xfe, 0x7b, 0x2d, 0x1f, 0xad, 0xeb, 0xe0, 0xb9, 0xfc, 0xcd, 0x58, + 0xcc, 0xdf, 0xe6, 0x13, 0x25, 0xf3, 0xb7, 0x4a, 0x94, 0x7e, 0x0e, 0x8e, 0x8f, 0xd9, 0x42, 0x70, + 0x99, 0x87, 0xe7, 0xd5, 0xc5, 0xcc, 0x40, 0xe7, 0x13, 0xc1, 0x25, 0xa7, 0xa5, 0xb2, 0xdc, 0x4b, + 0x16, 0x5f, 0xf0, 0x28, 0x78, 0x8b, 0x85, 0xbe, 0x3c, 0x74, 0xc9, 0x28, 0xbb, 0x26, 0x2a, 0x83, + 0xd0, 0x5d, 0x93, 0xbc, 0x01, 0x64, 0x95, 0x0d, 0x20, 0x89, 0xdc, 0x34, 0x11, 0x3c, 0xcd, 0xf2, + 0x4c, 0x52, 0x51, 0x45, 0x46, 0xe6, 0x68, 0x5d, 0x99, 0x91, 0x7d, 0x0e, 0xed, 0x28, 0x8e, 0x86, + 0xd1, 0x34, 0x0c, 0x65, 0xae, 0xab, 0xbb, 0x7a, 0xad, 0x28, 0x8e, 0x8e, 0x34, 0x8b, 0x3c, 0x81, + 0x4f, 0xaa, 0x2a, 0xca, 0x72, 0x5b, 0xaa, 0x55, 0x50, 0xd1, 0x43, 0xfb, 0xde, 0x80, 0x4e, 0x3c, + 0xfa, 0x15, 0xf7, 0x32, 0x44, 0x6c, 0x88, 0x26, 0xdb, 0x56, 0x8f, 0xb4, 0xe2, 0x4b, 0x88, 0x8e, + 0xd8, 0x84, 0xbb, 0x2f, 0xc1, 0x29, 0x40, 0x90, 0xf1, 0xf8, 0xe8, 0xf8, 0xa8, 0xa7, 0x22, 0xe6, + 0xfe, 0xd1, 0x5e, 0xef, 0x4f, 0x3b, 0x86, 0x8c, 0xe8, 0xb4, 0xf7, 0xa6, 0x47, 0xfb, 0xbd, 0x8e, + 0x29, 0xa3, 0xed, 0x5e, 0xef, 0xa0, 0x37, 0xe8, 0x75, 0x6a, 0xdf, 0xd5, 0xed, 0x66, 0xc7, 0xa6, + 0x36, 0x9f, 0x25, 0x61, 0xe0, 0x05, 0x99, 0xdb, 0x07, 0x90, 0xd3, 0xea, 0x1b, 0xbd, 0x07, 0x4e, + 0xb9, 0xb6, 0xba, 0x51, 0x3b, 0xd3, 0xab, 0xca, 0x9c, 0x4d, 0x1b, 0x95, 0x79, 0x53, 0xce, 0xa6, + 0xcd, 0xec, 0x14, 0xec, 0x43, 0x96, 0xbc, 0x57, 0xc6, 0xb4, 0x8b, 0x62, 0x75, 0xaa, 0x5b, 0x37, + 0xfa, 0xa9, 0x7e, 0x08, 0x4d, 0x1d, 0xf9, 0xb4, 0x7f, 0xcc, 0x45, 0xc5, 0x5c, 0x26, 0xab, 0x8a, + 0x3b, 0x87, 0xf1, 0x25, 0x2f, 0xb2, 0x95, 0x13, 0x76, 0x15, 0xc6, 0xcc, 0xff, 0x88, 0x21, 0xfe, + 0x0c, 0x40, 0xc4, 0xd3, 0xd4, 0xe3, 0xc3, 0x71, 0xd1, 0x31, 0x72, 0x14, 0xe7, 0xb5, 0x6e, 0x43, + 0x73, 0x91, 0xa1, 0xb0, 0xa6, 0xdc, 0x4c, 0xd2, 0x52, 0xf4, 0x13, 0xb0, 0xb2, 0x59, 0x54, 0x36, + 0xa8, 0x1a, 0x99, 0xac, 0x21, 0xdd, 0x5d, 0x70, 0x06, 0x33, 0xac, 0xac, 0xa6, 0x62, 0xee, 0xfd, + 0x35, 0x3e, 0xf0, 0xfe, 0x9a, 0x0b, 0xef, 0xef, 0x7f, 0x19, 0xd0, 0xaa, 0xa4, 0x51, 0xe4, 0x73, + 0xa8, 0x67, 0xb3, 0x68, 0xbe, 0xb3, 0x9b, 0x2f, 0x42, 0x51, 0x24, 0xed, 0x4d, 0x96, 0x5d, 0x4c, + 0x88, 0x60, 0x1c, 0x71, 0x5f, 0x4f, 0x29, 0x4b, 0xb1, 0x6d, 0xcd, 0x22, 0x07, 0x70, 0x4b, 0xc5, + 0x8c, 0xbc, 0xab, 0x93, 0x67, 0xe2, 0x5f, 0x2c, 0xa4, 0x6d, 0xaa, 0xfa, 0xdc, 0xcd, 0xb5, 0x54, + 0x7d, 0xbd, 0x32, 0x9e, 0x63, 0xae, 0x6e, 0xc3, 0xed, 0x6b, 0xd4, 0x7e, 0x54, 0x23, 0xe1, 0x3e, + 0x2c, 0xcb, 0xc2, 0x3b, 0x98, 0x70, 0x91, 0xb1, 0x49, 0x82, 0xf9, 0x8b, 0x8e, 0xf9, 0x75, 0x6a, + 0x66, 0xc2, 0xfd, 0x12, 0xda, 0x27, 0x9c, 0xa7, 0x94, 0x8b, 0x24, 0x8e, 0xd4, 0xdb, 0x2d, 0xf0, + 0xd0, 0xfa, 0x81, 0xd1, 0x94, 0xfb, 0x17, 0xe0, 0xc8, 0xcc, 0x7c, 0x87, 0x65, 0xde, 0xf9, 0x8f, + 0xc9, 0xdc, 0xbf, 0x84, 0x66, 0xa2, 0xcc, 0x44, 0xe7, 0xd9, 0x6d, 0x8c, 0x71, 0xda, 0x74, 0x68, + 0x2e, 0x74, 0xbf, 0x85, 0xda, 0xd1, 0x74, 0x52, 0xfd, 0xe3, 0xa5, 0xae, 0xfe, 0x78, 0xb9, 0x07, + 0x0e, 0x26, 0xf9, 0xd8, 0xe5, 0x53, 0xe9, 0xa7, 0x2d, 0x19, 0xd8, 0xde, 0xfb, 0x01, 0x5a, 0x39, + 0xf6, 0xfb, 0x3e, 0xfe, 0x7b, 0x82, 0x97, 0xbf, 0xef, 0xcf, 0xd9, 0x82, 0xaa, 0xf3, 0x78, 0xe4, + 0xef, 0xe7, 0x97, 0xa6, 0x88, 0xf9, 0xb9, 0xf5, 0x8b, 0x5d, 0xcc, 0xfd, 0x0a, 0xda, 0x79, 0xf6, + 0x7c, 0xc8, 0x33, 0x86, 0xe6, 0x14, 0x06, 0x3c, 0xaa, 0x98, 0x9a, 0xad, 0x18, 0x03, 0xf1, 0x81, + 0xa6, 0xa8, 0xbb, 0x09, 0x96, 0xb6, 0x55, 0x02, 0x75, 0x2f, 0xf6, 0x95, 0x8b, 0x34, 0x28, 0x7e, + 0xcb, 0x03, 0x4f, 0xc4, 0x38, 0x7f, 0x08, 0x27, 0x62, 0xec, 0xce, 0x60, 0x79, 0x87, 0x79, 0x17, + 0xd3, 0x24, 0x7f, 0x88, 0x2a, 0x65, 0x8e, 0x31, 0x57, 0xe6, 0x7c, 0xa0, 0x13, 0xfb, 0x29, 0x34, + 0xa7, 0x51, 0x30, 0xcb, 0xd3, 0x17, 0x87, 0x5a, 0x92, 0x54, 0x1d, 0xc8, 0x30, 0xf6, 0xb0, 0xb2, + 0x41, 0xaf, 0x72, 0x68, 0x41, 0xbb, 0x7f, 0x0e, 0xcb, 0xbd, 0x59, 0x82, 0x7d, 0xe9, 0x8f, 0x3e, + 0x81, 0x95, 0x4d, 0x99, 0x73, 0x9b, 0x5a, 0x58, 0xb9, 0x96, 0xaf, 0xbc, 0xf5, 0xcf, 0x06, 0xd4, + 0xa5, 0x89, 0xc8, 0xe2, 0xeb, 0x8f, 0x39, 0x4b, 0xb3, 0x11, 0x67, 0x19, 0x99, 0x33, 0x87, 0xd5, + 0x39, 0xca, 0x5d, 0x7a, 0x61, 0x90, 0xaf, 0x54, 0xcb, 0x3b, 0xef, 0xe4, 0x2f, 0xe7, 0x86, 0x86, + 0x86, 0xb8, 0xa8, 0x4f, 0x36, 0xa1, 0xf5, 0x5d, 0x1c, 0x44, 0xbb, 0xaa, 0x0b, 0x4c, 0x16, 0xcd, + 0xf2, 0x3d, 0xfd, 0x67, 0x60, 0xed, 0x0b, 0x69, 0xff, 0xef, 0xab, 0x62, 0x78, 0xad, 0xba, 0x86, + 0xbb, 0xb4, 0xf5, 0x4f, 0x35, 0xa8, 0xff, 0xc0, 0xd3, 0x98, 0x7c, 0x05, 0x4d, 0xdd, 0xe3, 0x21, + 0x95, 0x5e, 0xce, 0x2a, 0xbe, 0xaf, 0x0b, 0xcd, 0x1f, 0x5c, 0xa5, 0xa3, 0x22, 0x74, 0x59, 0x1d, + 0x92, 0xb2, 0x05, 0xf5, 0xde, 0xa6, 0x5e, 0x42, 0xa7, 0x9f, 0xa5, 0x9c, 0x4d, 0x2a, 0xea, 0xf3, + 0x30, 0x5d, 0x57, 0x6a, 0x22, 0x5a, 0x4f, 0xc1, 0x52, 0x41, 0x66, 0x61, 0xc0, 0x62, 0xd5, 0x88, + 0xca, 0x8f, 0xa0, 0xd5, 0x3f, 0x8f, 0xa7, 0xa1, 0xdf, 0xe7, 0xe9, 0x25, 0x27, 0x95, 0x3e, 0xeb, + 0x6a, 0xe5, 0xdb, 0x5d, 0x22, 0x1b, 0x00, 0xca, 0xcb, 0x4e, 0x03, 0x5f, 0x90, 0xa6, 0x94, 0x1d, + 0x4d, 0x27, 0x6a, 0xd2, 0x8a, 0xfb, 0x29, 0xcd, 0x4a, 0xac, 0xf9, 0x90, 0xe6, 0x37, 0xb0, 0xbc, + 0x8b, 0xb1, 0xf8, 0x38, 0xdd, 0x1e, 0xc5, 0x69, 0x46, 0x16, 0x7b, 0xad, 0xab, 0x8b, 0x0c, 0x77, + 0x89, 0xbc, 0x00, 0x7b, 0x90, 0x5e, 0x29, 0xfd, 0x4f, 0x74, 0x88, 0x2e, 0xd7, 0xbb, 0xe6, 0x94, + 0x5b, 0x7f, 0x5f, 0x03, 0xeb, 0x97, 0x71, 0x7a, 0xc1, 0x53, 0xf2, 0x04, 0x2c, 0x2c, 0xef, 0xb5, + 0x11, 0x15, 0xa5, 0xfe, 0x75, 0x0b, 0x3d, 0x00, 0x07, 0x41, 0x19, 0x30, 0x71, 0xa1, 0xae, 0x0a, + 0xff, 0x7a, 0x55, 0xb8, 0xa8, 0xe4, 0x0d, 0xef, 0x75, 0x45, 0x5d, 0x54, 0xd1, 0xd2, 0x98, 0xab, + 0xb9, 0x57, 0x9b, 0xaa, 0x80, 0xee, 0xbb, 0x4b, 0x1b, 0xc6, 0x0b, 0x83, 0x3c, 0x86, 0x7a, 0x5f, + 0x9d, 0x54, 0x2a, 0x95, 0x7f, 0x4f, 0xad, 0xae, 0xe4, 0x8c, 0x62, 0xe6, 0xe7, 0x60, 0xa9, 0x97, + 0x5d, 0x1d, 0x73, 0x2e, 0x31, 0x5d, 0xed, 0x54, 0x59, 0x7a, 0xc0, 0x63, 0xb0, 0x54, 0xd0, 0x50, + 0x03, 0xe6, 0x02, 0x88, 0xda, 0xb5, 0x8a, 0x41, 0x4a, 0x55, 0x79, 0xb9, 0x52, 0x9d, 0xf3, 0xf8, + 0x05, 0xd5, 0x67, 0xd0, 0xa1, 0xdc, 0xe3, 0x41, 0xe5, 0xcd, 0x27, 0xf9, 0xa1, 0x16, 0xcd, 0x76, + 0xc3, 0x20, 0x2f, 0x61, 0x79, 0x2e, 0x3f, 0x20, 0x5d, 0x04, 0xfa, 0x9a, 0x94, 0x61, 0x71, 0xf0, + 0x4e, 0xe7, 0x5f, 0xdf, 0xad, 0x19, 0xff, 0xf6, 0x6e, 0xcd, 0xf8, 0x8f, 0x77, 0x6b, 0xc6, 0xaf, + 0xff, 0x73, 0x6d, 0x69, 0x64, 0xe1, 0x5f, 0xf6, 0xdf, 0xfc, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, + 0xd8, 0x05, 0x85, 0x05, 0xf6, 0x1f, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -6430,10 +6430,10 @@ func (m *PostingList) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintPb(dAtA, i, uint64(m.CommitTs)) } - if len(m.Parts) > 0 { - dAtA23 := make([]byte, len(m.Parts)*10) + if len(m.Splits) > 0 { + dAtA23 := make([]byte, len(m.Splits)*10) var j22 int - for _, num := range m.Parts { + for _, num := range m.Splits { for num >= 1<<7 { dAtA23[j22] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 @@ -8261,9 +8261,9 @@ func (m *PostingList) Size() (n int) { if m.CommitTs != 0 { n += 1 + sovPb(uint64(m.CommitTs)) } - if len(m.Parts) > 0 { + if len(m.Splits) > 0 { l = 0 - for _, e := range m.Parts { + for _, e := range m.Splits { l += sovPb(uint64(e)) } n += 1 + sovPb(uint64(l)) + l @@ -13598,7 +13598,7 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { break } } - m.Parts = append(m.Parts, v) + m.Splits = append(m.Splits, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -13630,8 +13630,8 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.Parts) == 0 { - m.Parts = make([]uint64, 0, elementCount) + if elementCount != 0 && len(m.Splits) == 0 { + m.Splits = make([]uint64, 0, elementCount) } for iNdEx < postIndex { var v uint64 @@ -13649,10 +13649,10 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { break } } - m.Parts = append(m.Parts, v) + m.Splits = append(m.Splits, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field Parts", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Splits", wireType) } case 5: if wireType != 0 { From 74b38892ee865e44122f9104b271ae973ad7782e Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Mon, 11 Mar 2019 16:31:30 -0700 Subject: [PATCH 26/67] Change tests to read and write posting lists to disk. --- posting/index.go | 2 +- posting/list.go | 110 ++++++++++------------ posting/list_test.go | 199 +++++++++++++++++++++------------------ posting/mvcc.go | 2 +- posting/mvcc_test.go | 3 +- worker/draft.go | 2 +- worker/predicate_move.go | 2 +- 7 files changed, 166 insertions(+), 154 deletions(-) diff --git a/posting/index.go b/posting/index.go index e7d2f9583c1..003564a7fbd 100644 --- a/posting/index.go +++ b/posting/index.go @@ -550,7 +550,7 @@ func (r *rebuild) Run(ctx context.Context) error { if le == 0 { continue } - kvs, err := pl.MarshalToKv() + kvs, err := pl.Rollup() if err != nil { return err } diff --git a/posting/list.go b/posting/list.go index 29365c66126..b348c7ee014 100644 --- a/posting/list.go +++ b/posting/list.go @@ -78,10 +78,7 @@ type List struct { pendingTxns int32 // Using atomic for this, to avoid locking in SetForDeletion operation. deleteMe int32 // Using atomic for this, to avoid expensive SetForDeletion operation. - // Rolling up might create new parts (either because there were changes or - // a split occurred). These parts need to be kept in memory until they are - // committed to disk. - uncommittedParts map[uint64]*pb.PostingList + partCache map[uint64]*pb.PostingList } func getNextPartKey(baseKey []byte, nextPartStart uint64) []byte { @@ -103,7 +100,6 @@ func (l *List) maxVersion() uint64 { type PIterator struct { l *List plist *pb.PostingList - opts PItrOpts uidPosting *pb.Posting pidx int // index of postings plen int @@ -111,18 +107,17 @@ type PIterator struct { dec *codec.Decoder uids []uint64 uidx int // Offset into the uids slice -} -type PItrOpts struct { - discardPl bool - afterUid uint64 - startPart int - readTs uint64 + afterUid uint64 + deleteBelow uint64 + splitIdx int } -func (it *PIterator) Init(l *List, opts PItrOpts) error { - if len(l.plist.Splits) > 0 { - plist, err := l.readListPart(l.plist.Splits[opts.startPart]) +func (it *PIterator) Init(l *List, afterUid, deleteBelow uint64) error { + it.l = l + it.splitIdx = 0 + if len(it.l.plist.Splits) > 0 { + plist, err := l.readListPart(it.l.plist.Splits[0]) if err != nil { return err } @@ -131,24 +126,46 @@ func (it *PIterator) Init(l *List, opts PItrOpts) error { it.plist = l.plist } - it.l = l - it.opts = opts + it.afterUid = afterUid + it.deleteBelow = deleteBelow + it.uidPosting = &pb.Posting{} + it.dec = &codec.Decoder{Pack: it.plist.Pack} + it.uids = it.dec.Seek(it.afterUid) + it.uidx = 0 + + it.plen = len(it.plist.Postings) + it.pidx = sort.Search(it.plen, func(idx int) bool { + p := it.plist.Postings[idx] + return it.afterUid < p.Uid + }) + return nil +} + +func (it *PIterator) moveToNextSplit() error { + it.splitIdx++ + plist, err := it.l.readListPart(it.l.plist.Splits[it.splitIdx]) + if err != nil { + return err + } + it.plist = plist + it.uidPosting = &pb.Posting{} it.dec = &codec.Decoder{Pack: it.plist.Pack} - it.uids = it.dec.Seek(opts.afterUid) + it.uids = it.dec.Seek(it.afterUid) it.uidx = 0 it.plen = len(it.plist.Postings) it.pidx = sort.Search(it.plen, func(idx int) bool { p := it.plist.Postings[idx] - return it.opts.afterUid < p.Uid + return it.afterUid < p.Uid }) + return nil } func (it *PIterator) Next() error { - if it.opts.discardPl { + if it.deleteBelow > 0 { return nil } @@ -162,7 +179,7 @@ func (it *PIterator) Next() error { } func (it *PIterator) Valid() bool { - if it.opts.discardPl { + if it.deleteBelow > 0 { return false } @@ -174,9 +191,8 @@ func (it *PIterator) Valid() bool { return false } - for it.opts.startPart+1 < len(it.l.plist.Splits) { - it.opts.startPart++ - it.Init(it.l, it.opts) + for it.splitIdx+1 < len(it.l.plist.Splits) { + it.moveToNextSplit() if len(it.uids) > 0 { return true @@ -585,12 +601,7 @@ func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) e var mp, pp *pb.Posting var pitr PIterator - err := pitr.Init(l, PItrOpts{ - afterUid: afterUid, - discardPl: deleteBelow > 0, - readTs: readTs, - startPart: 0, - }) + err := pitr.Init(l, afterUid, deleteBelow) if err != nil { return err } @@ -682,7 +693,7 @@ func (l *List) Length(readTs, afterUid uint64) int { return l.length(readTs, afterUid) } -func (l *List) MarshalToKv() ([]*bpb.KV, error) { +func (l *List) Rollup() ([]*bpb.KV, error) { l.Lock() defer l.Unlock() if err := l.rollup(math.MaxUint64); err != nil { @@ -716,7 +727,7 @@ func (l *List) MarshalToKv() ([]*bpb.KV, error) { } func marshalPostingList(plist *pb.PostingList) (data []byte, meta byte) { - if plist.Pack == nil || len(plist.Pack.Blocks) == 0 { + if (plist.Pack == nil || len(plist.Pack.Blocks) == 0) && len(plist.Splits) == 0 { return nil, BitEmptyPosting } data, err := plist.Marshal() @@ -726,12 +737,6 @@ func marshalPostingList(plist *pb.PostingList) (data []byte, meta byte) { const blockSize int = 256 -func (l *List) Rollup(readTs uint64) error { - l.Lock() - defer l.Unlock() - return l.rollup(readTs) -} - // Merge all entries in mutation layer with commitTs <= l.commitTs into // immutable layer. Note that readTs can be math.MaxUint64, so do NOT use it // directly. It should only serve as the read timestamp for iteration. @@ -744,20 +749,6 @@ func (l *List) rollup(readTs uint64) error { return nil } - // Delete lists from the uncommittedParts map that are already in disk. - for _, startUid := range l.plist.Splits { - pl, version, err := l.readListPartFromDisk(startUid) - if err != nil || pl == nil { - // Ignore errors since this might be that the list has never - // been committed to disk. - continue - } - - if version >= l.minTs { - delete(l.uncommittedParts, startUid) - } - } - var plist *pb.PostingList var final *pb.PostingList var enc codec.Encoder @@ -799,7 +790,7 @@ func (l *List) rollup(readTs uint64) error { final.Pack = enc.Done() plist.Pack = final.Pack plist.Postings = final.Postings - l.uncommittedParts[plist.StartUid] = plist + l.partCache[plist.StartUid] = plist splitIdx++ init() @@ -816,7 +807,7 @@ func (l *List) rollup(readTs uint64) error { final.Pack = enc.Done() plist.Pack = final.Pack plist.Postings = final.Postings - l.uncommittedParts[plist.StartUid] = plist + l.partCache[plist.StartUid] = plist maxCommitTs := l.minTs { @@ -847,7 +838,7 @@ func (l *List) rollup(readTs uint64) error { // Check if the list (or any of it's parts if it's been previously split) have // become too big. Split the list if that is the case. - if err := l.splitList(readTs); err != nil { + if err := l.splitList(); err != nil { return nil } @@ -1108,7 +1099,7 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs } func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { - if part, ok := l.uncommittedParts[startUid]; ok { + if part, ok := l.partCache[startUid]; ok { return part, nil } part, _, err := l.readListPartFromDisk(startUid) @@ -1133,7 +1124,7 @@ func needsSplit(plist *pb.PostingList) bool { return plist.Size() >= maxListSize && len(plist.Pack.Blocks) > 1 } -func (l *List) splitList(readTs uint64) error { +func (l *List) splitList() error { l.AssertLock() var lists []*pb.PostingList @@ -1154,7 +1145,7 @@ func (l *List) splitList(readTs uint64) error { if needsSplit(list) { splitList := splitPostingList(list) for _, part := range splitList { - l.uncommittedParts[part.StartUid] = part + l.partCache[part.StartUid] = part newLists = append(newLists, part) } } else { @@ -1169,7 +1160,10 @@ func (l *List) splitList(readTs uint64) error { for _, list := range newLists { splits = append(splits, list.StartUid) } - l.plist.Splits = splits + l.plist = &pb.PostingList{ + CommitTs: l.plist.CommitTs, + Splits: splits, + } } return nil } diff --git a/posting/list_test.go b/posting/list_test.go index 56c3654f3ab..42b400e0eb9 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -26,6 +26,7 @@ import ( "testing" "github.com/dgraph-io/badger" + bpb "github.com/dgraph-io/badger/pb" "github.com/stretchr/testify/require" "github.com/dgraph-io/dgraph/protos/pb" @@ -415,6 +416,9 @@ func TestAddMutation_mrjn1(t *testing.T) { } func TestMillion(t *testing.T) { + // Ensure list is stored in a single part. + maxListSize = math.MaxInt32 + key := x.DataKey("bal", 1331) ol, err := getNew(key, ps) require.NoError(t, err) @@ -431,7 +435,8 @@ func TestMillion(t *testing.T) { // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting // list. t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) - require.NoError(t, ol.Rollup(math.MaxUint64)) + _, err := ol.Rollup() + require.NoError(t, err) } commits++ } @@ -852,61 +857,11 @@ func TestAfterUIDCountWithCommit(t *testing.T) { require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) } -var ps *badger.DB - -func TestMain(m *testing.M) { - x.Init() - Config.AllottedMemory = 1024.0 - Config.CommitFraction = 0.10 - - dir, err := ioutil.TempDir("", "storetest_") - x.Check(err) - - opt := badger.DefaultOptions - opt.Dir = dir - opt.ValueDir = dir - ps, err = badger.OpenManaged(opt) - x.Check(err) - Init(ps) - schema.Init(ps) - - r := m.Run() - - os.RemoveAll(dir) - os.Exit(r) -} - -func BenchmarkAddMutations(b *testing.B) { - key := x.DataKey("name", 1) - l, err := getNew(key, ps) - if err != nil { - b.Error(err) - } - b.ResetTimer() - - ctx := context.Background() - for i := 0; i < b.N; i++ { - if err != nil { - b.Error(err) - return - } - edge := &pb.DirectedEdge{ - ValueId: uint64(rand.Intn(b.N) + 1), - Label: "testing", - Op: pb.DirectedEdge_SET, - } - txn := &Txn{StartTs: 1} - if err = l.AddMutation(ctx, txn, edge); err != nil { - b.Error(err) - } - } -} - -func createMultiPartList(t *testing.T, size int) (*List, int) { +func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { // For testing, set the max list size to a lower threshold. - maxListSize = 10000 + maxListSize = 5000 defer func() { - maxListSize = 2000000 + maxListSize = math.MaxInt32 }() key := x.DataKey("bal", 1331) @@ -917,14 +872,19 @@ func createMultiPartList(t *testing.T, size int) (*List, int) { edge := &pb.DirectedEdge{ ValueId: uint64(i), } + if addLabel { + edge.Label = strconv.Itoa(i) + } + txn := Txn{StartTs: uint64(i)} addMutationHelper(t, ol, edge, Set, &txn) require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%1000 == 0 { - // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting - // list. - t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) - require.NoError(t, ol.Rollup(math.MaxUint64)) + if i%2000 == 0 { + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) } commits++ } @@ -932,9 +892,19 @@ func createMultiPartList(t *testing.T, size int) (*List, int) { return ol, commits } -func TestMultiPartList(t *testing.T) { +func writePostingListToDisk(kvs []*bpb.KV) error { + writer := NewTxnWriter(pstore) + for _, kv := range kvs { + if err := writer.SetAt(kv.Key, kv.Value, kv.UserMeta[0], kv.Version); err != nil { + return err + } + } + return writer.Flush() +} + +func TestMultiPartListBasic(t *testing.T) { N := int(1e5) - ol, commits := createMultiPartList(t, N) + ol, commits := createMultiPartList(t, N, false) t.Logf("List parts %v", len(ol.plist.Splits)) opt := ListOptions{ReadTs: uint64(N) + 1} l, err := ol.Uids(opt) @@ -946,37 +916,12 @@ func TestMultiPartList(t *testing.T) { } func TestMultiPartListWithPostings(t *testing.T) { - // For testing, set the max list length to a lower threshold. - maxListSize = 10000 - defer func() { - maxListSize = 2000000 - }() - - key := x.DataKey("bal", 1331) - ol, err := getNew(key, ps) - require.NoError(t, err) - var commits int N := int(1e5) - for i := 2; i <= N; i += 2 { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - Label: strconv.Itoa(i), - } - txn := Txn{StartTs: uint64(i)} - addMutationHelper(t, ol, edge, Set, &txn) - require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%1000 == 0 { - // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting - // list. - t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) - require.NoError(t, ol.Rollup(math.MaxUint64)) - } - commits++ - } + ol, commits := createMultiPartList(t, N, true) t.Logf("List parts %v", len(ol.plist.Splits)) var labels []string - ol.Iterate(uint64(N)+1, 0, func(p *pb.Posting) error { + err := ol.Iterate(uint64(N)+1, 0, func(p *pb.Posting) error { if len(p.Label) > 0 { labels = append(labels, p.Label) } @@ -991,13 +936,15 @@ func TestMultiPartListWithPostings(t *testing.T) { func TestMultiPartListMarshal(t *testing.T) { N := int(1e5) - ol, commits := createMultiPartList(t, N) + ol, commits := createMultiPartList(t, N, false) t.Logf("List parts %v", len(ol.plist.Splits)) - kvs, err := ol.MarshalToKv() + kvs, err := ol.Rollup() require.NoError(t, err) require.Equal(t, len(kvs), len(ol.plist.Splits)+1) + require.NoError(t, writePostingListToDisk(kvs)) + key := x.DataKey("bal", 1331) require.Equal(t, key, kvs[0].Key) @@ -1023,5 +970,77 @@ func TestMultiPartListMarshal(t *testing.T) { for i, uid := range l.Uids { require.Equal(t, uint64(i+1)*2, uid) } +} + +func TestMultiPartListWrite(t *testing.T) { + N := int(1e5) + originalList, commits := createMultiPartList(t, N, false) + + kvs, err := originalList.Rollup() + require.NoError(t, err) + require.Equal(t, len(kvs), len(originalList.plist.Splits)+1) + + require.NoError(t, writePostingListToDisk(kvs)) + newList, err := getNew(kvs[0].Key, ps) + + opt := ListOptions{ReadTs: uint64(N) + 1} + originalUids, err := originalList.Uids(opt) + require.NoError(t, err) + newUids, err := newList.Uids(opt) + require.NoError(t, err) + require.Equal(t, commits, len(originalUids.Uids)) + require.Equal(t, len(originalUids.Uids), len(newUids.Uids)) + for i, _ := range originalUids.Uids { + require.Equal(t, originalUids.Uids[i], newUids.Uids[i]) + } +} + +var ps *badger.DB + +func TestMain(m *testing.M) { + x.Init() + Config.AllottedMemory = 1024.0 + Config.CommitFraction = 0.10 + + dir, err := ioutil.TempDir("", "storetest_") + x.Check(err) + + opt := badger.DefaultOptions + opt.Dir = dir + opt.ValueDir = dir + ps, err = badger.OpenManaged(opt) + x.Check(err) + Init(ps) + schema.Init(ps) + + r := m.Run() + + os.RemoveAll(dir) + os.Exit(r) +} + +func BenchmarkAddMutations(b *testing.B) { + key := x.DataKey("name", 1) + l, err := getNew(key, ps) + if err != nil { + b.Error(err) + } + b.ResetTimer() + ctx := context.Background() + for i := 0; i < b.N; i++ { + if err != nil { + b.Error(err) + return + } + edge := &pb.DirectedEdge{ + ValueId: uint64(rand.Intn(b.N) + 1), + Label: "testing", + Op: pb.DirectedEdge_SET, + } + txn := &Txn{StartTs: 1} + if err = l.AddMutation(ctx, txn, edge); err != nil { + b.Error(err) + } + } } diff --git a/posting/mvcc.go b/posting/mvcc.go index 310b501b2ca..79557bf6ed7 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -185,7 +185,7 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { l := new(List) l.key = key l.mutationMap = make(map[uint64]*pb.PostingList) - l.uncommittedParts = make(map[uint64]*pb.PostingList) + l.partCache = make(map[uint64]*pb.PostingList) l.plist = new(pb.PostingList) // Iterates from highest Ts to lowest Ts diff --git a/posting/mvcc_test.go b/posting/mvcc_test.go index 6a1708b74cf..39f2a3488be 100644 --- a/posting/mvcc_test.go +++ b/posting/mvcc_test.go @@ -55,8 +55,7 @@ func TestRollupTimestamp(t *testing.T) { // Now check that we don't lost the highest version during a rollup operation, despite the STAR // delete marker being the most recent update. - require.NoError(t, nl.Rollup(11)) - kvs, err := nl.MarshalToKv() + kvs, err := nl.Rollup() require.NoError(t, err) require.Equal(t, uint64(10), kvs[0].Version) } diff --git a/worker/draft.go b/worker/draft.go index 5b3679983ed..816c6437a09 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -892,7 +892,7 @@ func (n *node) rollupLists(readTs uint64) error { return nil, err } atomic.AddUint64(&numKeys, 1) - kvs, err := l.MarshalToKv() + kvs, err := l.Rollup() // If there are multiple keys, the posting list was split into multiple // parts. The key of the first part is the right key to use for tablet diff --git a/worker/predicate_move.go b/worker/predicate_move.go index a4eef343da6..2c470856d7a 100644 --- a/worker/predicate_move.go +++ b/worker/predicate_move.go @@ -257,7 +257,7 @@ func movePredicateHelper(ctx context.Context, in *pb.MovePredicatePayload) error if err != nil { return nil, err } - kvs, err := l.MarshalToKv() + kvs, err := l.Rollup() for _, kv := range kvs { // Let's set all of them at this move timestamp. kv.Version = in.TxnTs From db750d604dffb81f50b4e6c00b6eabd040b93189 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Mon, 11 Mar 2019 16:36:50 -0700 Subject: [PATCH 27/67] remove readListPartFromDisk method. --- posting/list.go | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/posting/list.go b/posting/list.go index b348c7ee014..a761c91efd0 100644 --- a/posting/list.go +++ b/posting/list.go @@ -1102,22 +1102,18 @@ func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { if part, ok := l.partCache[startUid]; ok { return part, nil } - part, _, err := l.readListPartFromDisk(startUid) - return part, err -} -func (l *List) readListPartFromDisk(startUid uint64) (*pb.PostingList, uint64, error) { nextKey := getNextPartKey(l.key, startUid) txn := pstore.NewTransactionAt(l.minTs, false) item, err := txn.Get(nextKey) if err != nil { - return nil, 0, err + return nil, err } var part pb.PostingList if err := unmarshalOrCopy(&part, item); err != nil { - return nil, 0, err + return nil, err } - return &part, item.Version(), nil + return &part, nil } func needsSplit(plist *pb.PostingList) bool { From b21038cf845e42e868472cf084c14311e6df55c0 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Mon, 11 Mar 2019 16:42:56 -0700 Subject: [PATCH 28/67] Check error --- posting/list.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/posting/list.go b/posting/list.go index a761c91efd0..ba55e8713c3 100644 --- a/posting/list.go +++ b/posting/list.go @@ -192,7 +192,9 @@ func (it *PIterator) Valid() bool { } for it.splitIdx+1 < len(it.l.plist.Splits) { - it.moveToNextSplit() + if err := it.moveToNextSplit(); err != nil { + return false + } if len(it.uids) > 0 { return true From 86c595155d4d73ec3ce7c52d714778166dfb9290 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Mon, 11 Mar 2019 17:52:01 -0700 Subject: [PATCH 29/67] Store splits in emptylists and fix logic to decide the list metadata. --- posting/list.go | 50 ++++++++++++++++++++++++++++++++++++++++++------- posting/mvcc.go | 7 +------ 2 files changed, 44 insertions(+), 13 deletions(-) diff --git a/posting/list.go b/posting/list.go index ba55e8713c3..c57d903e0fd 100644 --- a/posting/list.go +++ b/posting/list.go @@ -695,6 +695,33 @@ func (l *List) Length(readTs, afterUid uint64) int { return l.length(readTs, afterUid) } +func (l *List) plsAreEmpty() (bool, error) { + if len(l.plist.Splits) == 0 { + if l.plist.Pack == nil || len(l.plist.Pack.Blocks) == 0 { + return true, nil + } else { + return false, nil + } + } + + for _, startUid := range l.plist.Splits { + kv := &bpb.KV{} + kv.Version = l.minTs + kv.Key = getNextPartKey(l.key, startUid) + plist, err := l.readListPart(startUid) + if err != nil { + return false, err + } + + if plist.Pack == nil || len(plist.Pack.Blocks) == 0 { + continue + } else { + return false, nil + } + } + return true, nil +} + func (l *List) Rollup() ([]*bpb.KV, error) { l.Lock() defer l.Unlock() @@ -706,7 +733,10 @@ func (l *List) Rollup() ([]*bpb.KV, error) { kv := &bpb.KV{} kv.Version = l.minTs kv.Key = l.key - val, meta := marshalPostingList(l.plist) + val, meta, err := l.marshalPostingList(l.plist) + if err != nil { + return nil, err + } kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) @@ -719,7 +749,10 @@ func (l *List) Rollup() ([]*bpb.KV, error) { if err != nil { return nil, err } - val, meta := marshalPostingList(plist) + val, meta, err := l.marshalPostingList(plist) + if err != nil { + return nil, err + } kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) @@ -728,13 +761,16 @@ func (l *List) Rollup() ([]*bpb.KV, error) { return kvs, nil } -func marshalPostingList(plist *pb.PostingList) (data []byte, meta byte) { - if (plist.Pack == nil || len(plist.Pack.Blocks) == 0) && len(plist.Splits) == 0 { - return nil, BitEmptyPosting - } +func (l *List) marshalPostingList(plist *pb.PostingList) ([]byte, byte, error) { data, err := plist.Marshal() x.Check(err) - return data, BitCompletePosting + + if empty, err := l.plsAreEmpty(); err != nil { + return nil, BitEmptyPosting, err + } else if empty { + return data, BitEmptyPosting, nil + } + return data, BitCompletePosting, nil } const blockSize int = 256 diff --git a/posting/mvcc.go b/posting/mvcc.go index 79557bf6ed7..1a283e35a57 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -201,16 +201,11 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { } switch item.UserMeta() { - case BitEmptyPosting: - l.minTs = item.Version() - return l, nil - case BitCompletePosting: + case BitEmptyPosting, BitCompletePosting: if err := unmarshalOrCopy(l.plist, item); err != nil { return nil, err } l.minTs = item.Version() - // No need to do Next here. The outer loop can take care of skipping more versions of - // the same key. return l, nil case BitDeltaPosting: err := item.Value(func(val []byte) error { From d6c5dfb1bcfcca3a84b6a2ca484c45d14075d34d Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Tue, 12 Mar 2019 10:20:03 -0700 Subject: [PATCH 30/67] Check error status of init. --- posting/list.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/posting/list.go b/posting/list.go index c57d903e0fd..e7c950096ac 100644 --- a/posting/list.go +++ b/posting/list.go @@ -831,7 +831,9 @@ func (l *List) rollup(readTs uint64) error { l.partCache[plist.StartUid] = plist splitIdx++ - init() + if err := init(); err != nil { + return err + } } enc.Add(p.Uid) From f05f3ed8b2f3f0a43fe84dd77bced6de9a90ae88 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Tue, 12 Mar 2019 11:53:15 -0700 Subject: [PATCH 31/67] Fix marshalPostingList --- posting/list.go | 27 +++++++++++++-------------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/posting/list.go b/posting/list.go index e7c950096ac..659423d6b64 100644 --- a/posting/list.go +++ b/posting/list.go @@ -729,14 +729,18 @@ func (l *List) Rollup() ([]*bpb.KV, error) { return nil, err } + var setEmptyBit bool + if empty, err := l.plsAreEmpty(); err != nil { + return nil, err + } else if empty { + setEmptyBit = true + } + var kvs []*bpb.KV kv := &bpb.KV{} kv.Version = l.minTs kv.Key = l.key - val, meta, err := l.marshalPostingList(l.plist) - if err != nil { - return nil, err - } + val, meta := marshalPostingList(l.plist, setEmptyBit) kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) @@ -749,10 +753,7 @@ func (l *List) Rollup() ([]*bpb.KV, error) { if err != nil { return nil, err } - val, meta, err := l.marshalPostingList(plist) - if err != nil { - return nil, err - } + val, meta := marshalPostingList(plist, setEmptyBit) kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) @@ -761,16 +762,14 @@ func (l *List) Rollup() ([]*bpb.KV, error) { return kvs, nil } -func (l *List) marshalPostingList(plist *pb.PostingList) ([]byte, byte, error) { +func marshalPostingList(plist *pb.PostingList, setEmptyBit bool) ([]byte, byte) { data, err := plist.Marshal() x.Check(err) - if empty, err := l.plsAreEmpty(); err != nil { - return nil, BitEmptyPosting, err - } else if empty { - return data, BitEmptyPosting, nil + if setEmptyBit { + return data, BitEmptyPosting } - return data, BitCompletePosting, nil + return data, BitCompletePosting } const blockSize int = 256 From 240e8987f6730aaec2b406d7fb44dc21d9cd9c5d Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Tue, 12 Mar 2019 15:33:53 -0700 Subject: [PATCH 32/67] Add flag to control max posting list size. --- dgraph/cmd/alpha/run.go | 4 +++- posting/list.go | 3 +-- posting/list_test.go | 2 +- posting/lists.go | 10 +++++++--- worker/worker_test.go | 2 +- 5 files changed, 13 insertions(+), 8 deletions(-) diff --git a/dgraph/cmd/alpha/run.go b/dgraph/cmd/alpha/run.go index 30fe1023f7b..930c925a3fb 100644 --- a/dgraph/cmd/alpha/run.go +++ b/dgraph/cmd/alpha/run.go @@ -152,6 +152,8 @@ they form a Raft group and provide synchronous replication. flag.Uint64("query_edge_limit", 1e6, "Limit for the maximum number of edges that can be returned in a query."+ " This applies to shortest path and recursive queries.") + flag.Int("max_posting_list_size", posting.MaxListSize, + "Limit for the maximum size of a posting list before it is split into multiple parts.") // TLS configurations x.RegisterTLSFlags(flag) @@ -521,7 +523,7 @@ func run() { // Posting will initialize index which requires schema. Hence, initialize // schema before calling posting.Init(). schema.Init(edgraph.State.Pstore) - posting.Init(edgraph.State.Pstore) + posting.Init(edgraph.State.Pstore, Alpha.Conf.GetInt("max_posting_list_size")) defer posting.Cleanup() worker.Init(edgraph.State.Pstore) diff --git a/posting/list.go b/posting/list.go index 659423d6b64..4a684f452c7 100644 --- a/posting/list.go +++ b/posting/list.go @@ -51,7 +51,6 @@ var ( ErrInvalidTxn = fmt.Errorf("Invalid transaction") ErrStopIteration = errors.New("Stop iteration") emptyPosting = &pb.Posting{} - maxListSize = 2000000 ) const ( @@ -1156,7 +1155,7 @@ func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { } func needsSplit(plist *pb.PostingList) bool { - return plist.Size() >= maxListSize && len(plist.Pack.Blocks) > 1 + return plist.Size() >= MaxListSize && len(plist.Pack.Blocks) > 1 } func (l *List) splitList() error { diff --git a/posting/list_test.go b/posting/list_test.go index 42b400e0eb9..b87a1554f22 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -1010,7 +1010,7 @@ func TestMain(m *testing.M) { opt.ValueDir = dir ps, err = badger.OpenManaged(opt) x.Check(err) - Init(ps) + Init(ps, 0) schema.Init(ps) r := m.Run() diff --git a/posting/lists.go b/posting/lists.go index d8000086700..120fd101c3b 100644 --- a/posting/lists.go +++ b/posting/lists.go @@ -145,12 +145,16 @@ func updateMemoryMetrics(lc *y.Closer) { } var ( - pstore *badger.DB - closer *y.Closer + pstore *badger.DB + closer *y.Closer + MaxListSize = 10 * MB ) // Init initializes the posting lists package, the in memory and dirty list hash. -func Init(ps *badger.DB) { +func Init(ps *badger.DB, maxListSize int) { + if maxListSize > 0 { + MaxListSize = maxListSize + } pstore = ps closer = y.NewCloser(1) go updateMemoryMetrics(closer) diff --git a/worker/worker_test.go b/worker/worker_test.go index 776b536f839..bf414d48aeb 100644 --- a/worker/worker_test.go +++ b/worker/worker_test.go @@ -625,7 +625,7 @@ func TestMain(m *testing.M) { ps, err := badger.OpenManaged(opt) x.Check(err) pstore = ps - posting.Init(ps) + posting.Init(ps, 0) Init(ps) os.Exit(m.Run()) } From 08e7601f49fa452c532daa1732ad9358d37fbf2d Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Tue, 12 Mar 2019 13:38:42 -0700 Subject: [PATCH 33/67] Parse startuid in keys.go and add tests. --- x/keys.go | 29 ++++++++++++++++++++++------- x/keys_test.go | 31 +++++++++++++++++++++++++++++++ 2 files changed, 53 insertions(+), 7 deletions(-) diff --git a/x/keys.go b/x/keys.go index fe9ded9f578..472107485b3 100644 --- a/x/keys.go +++ b/x/keys.go @@ -83,7 +83,7 @@ func DataKey(attr string, uid uint64) []byte { return buf } -func DataKeyMultiPart(attr string, uid, startUid uint64) []byte { +func DataKeyWithStartUid(attr string, uid, startUid uint64) []byte { buf := make([]byte, 2+len(attr)+2+8+8) buf[0] = defaultPrefix rest := buf[1:] @@ -145,12 +145,14 @@ func CountKey(attr string, count uint32, reverse bool) []byte { } type ParsedKey struct { - byteType byte - Attr string - Uid uint64 - Term string - Count uint32 - bytePrefix byte + byteType byte + Attr string + Uid uint64 + StartUid uint64 + HasStartUid bool + Term string + Count uint32 + bytePrefix byte } func (p ParsedKey) IsData() bool { @@ -327,6 +329,19 @@ func Parse(key []byte) *ParsedKey { return nil } p.Uid = binary.BigEndian.Uint64(k) + + if len(k) == 8 { + return p + } + k = k[8:] + if len(k) < 8 { + if Config.DebugMode { + fmt.Printf("Error: StartUid length < 8 for key: %q, parsed key: %+v\n", key, p) + } + return nil + } + p.StartUid = binary.BigEndian.Uint64(k) + p.HasStartUid = true case ByteIndex: p.Term = string(k) case ByteCount, ByteCountRev: diff --git a/x/keys_test.go b/x/keys_test.go index d8c366f8461..7ff6ada0a48 100644 --- a/x/keys_test.go +++ b/x/keys_test.go @@ -34,6 +34,8 @@ func TestDataKey(t *testing.T) { require.True(t, pk.IsData()) require.Equal(t, sattr, pk.Attr) require.Equal(t, uid, pk.Uid) + require.Equal(t, uint64(0), pk.StartUid) + require.Equal(t, false, pk.HasStartUid) } keys := make([]string, 0, 1024) @@ -50,6 +52,35 @@ func TestDataKey(t *testing.T) { } } +func TestDataKeyWithStartUid(t *testing.T) { + var uid uint64 + startUid := uint64(1024) + for uid = 0; uid < 1001; uid++ { + sattr := fmt.Sprintf("attr:%d", uid) + key := DataKeyWithStartUid(sattr, uid, startUid) + pk := Parse(key) + + require.True(t, pk.IsData()) + require.Equal(t, sattr, pk.Attr) + require.Equal(t, uid, pk.Uid) + require.Equal(t, startUid, pk.StartUid) + require.Equal(t, true, pk.HasStartUid) + } + + keys := make([]string, 0, 1024) + for uid = 1024; uid >= 1; uid-- { + key := DataKeyWithStartUid("testing.key", uid, startUid) + keys = append(keys, string(key)) + } + // Test that sorting is as expected. + sort.Strings(keys) + require.True(t, sort.StringsAreSorted(keys)) + for i, key := range keys { + exp := DataKeyWithStartUid("testing.key", uint64(i+1), startUid) + require.Equal(t, string(exp), key) + } +} + func TestIndexKey(t *testing.T) { var uid uint64 for uid = 0; uid < 1001; uid++ { From f328c18db77d68428c02ee9c55bf56543dc6b2a8 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Tue, 12 Mar 2019 13:59:06 -0700 Subject: [PATCH 34/67] Update debug tool to understand multi-part lists. --- dgraph/cmd/debug/run.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/dgraph/cmd/debug/run.go b/dgraph/cmd/debug/run.go index 232ae309b67..ece8c246d53 100644 --- a/dgraph/cmd/debug/run.go +++ b/dgraph/cmd/debug/run.go @@ -112,6 +112,11 @@ func uidToVal(itr *badger.Iterator, prefix string) map[uint64]int { if pk.IsSchema() { continue } + if pk.HasStartUid { + // This key is part of a multi-part posting list. Skip it and only read + // the main key, which is the entry point to read the whole list. + continue + } pl, err := posting.ReadPostingList(item.KeyCopy(nil), itr) if err != nil { @@ -501,6 +506,9 @@ func printKeys(db *badger.DB) { if pk.Uid > 0 { fmt.Fprintf(&buf, " uid: %d ", pk.Uid) } + if pk.HasStartUid { + fmt.Fprintf(&buf, " startUid: %d ", pk.StartUid) + } fmt.Fprintf(&buf, " key: %s", hex.EncodeToString(item.Key())) if opt.itemMeta { fmt.Fprintf(&buf, " item: [%d, b%04b]", item.EstimatedSize(), item.UserMeta()) From 3a521625a0e8f571215c29ab11a94e30d6cfb3d4 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Tue, 12 Mar 2019 17:57:06 -0700 Subject: [PATCH 35/67] Restore comment. --- posting/mvcc.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/posting/mvcc.go b/posting/mvcc.go index 1a283e35a57..8d220d84ef6 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -206,6 +206,8 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { return nil, err } l.minTs = item.Version() + // No need to do Next here. The outer loop can take care of skipping + // more versions of the same key. return l, nil case BitDeltaPosting: err := item.Value(func(val []byte) error { From e0d13ac1cf6e38e1710a2032f789340901f7ad3a Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Tue, 12 Mar 2019 18:18:44 -0700 Subject: [PATCH 36/67] Fix posting tests --- posting/list_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/posting/list_test.go b/posting/list_test.go index b87a1554f22..ba97bd3f24a 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -417,7 +417,7 @@ func TestAddMutation_mrjn1(t *testing.T) { func TestMillion(t *testing.T) { // Ensure list is stored in a single part. - maxListSize = math.MaxInt32 + MaxListSize = math.MaxInt32 key := x.DataKey("bal", 1331) ol, err := getNew(key, ps) @@ -859,9 +859,9 @@ func TestAfterUIDCountWithCommit(t *testing.T) { func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { // For testing, set the max list size to a lower threshold. - maxListSize = 5000 + MaxListSize = 5000 defer func() { - maxListSize = math.MaxInt32 + MaxListSize = math.MaxInt32 }() key := x.DataKey("bal", 1331) From 92360187400e8c09fd918953142379bef6956d4b Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Tue, 12 Mar 2019 18:33:32 -0700 Subject: [PATCH 37/67] Add test files. --- posting/docker-compose.yml | 87 +++++++++++++++++++++++++++++++++ posting/multi_part_list_test.go | 79 ++++++++++++++++++++++++++++++ 2 files changed, 166 insertions(+) create mode 100644 posting/docker-compose.yml create mode 100644 posting/multi_part_list_test.go diff --git a/posting/docker-compose.yml b/posting/docker-compose.yml new file mode 100644 index 00000000000..d13f6cc5f67 --- /dev/null +++ b/posting/docker-compose.yml @@ -0,0 +1,87 @@ +# Docker compose file for testing. + +version: "3.5" +services: + zero1: + image: dgraph/dgraph:latest + container_name: bank-dg0.1 + working_dir: /data/dg0.1 + ports: + - 5080:5080 + - 6080:6080 + labels: + cluster: test + service: zero + volumes: + - type: bind + source: $GOPATH/bin + target: /gobin + read_only: true + command: /gobin/dgraph zero -o 0 --my=zero1:5080 --logtostderr -v=2 --bindall --expose_trace --profile_mode block --block_rate 10 + + dg1: + image: dgraph/dgraph:latest + container_name: bank-dg1 + working_dir: /data/dg1 + volumes: + - type: bind + source: $GOPATH/bin + target: /gobin + read_only: true + - type: bind + source: $GOPATH/src/github.com/dgraph-io/dgraph/ee/acl/hmac-secret + target: /dgraph-acl/hmac-secret + read_only: true + ports: + - 8180:8180 + - 9180:9180 + labels: + cluster: test + service: alpha + command: /gobin/dgraph alpha --my=dg1:7180 --lru_mb=1024 --zero=zero1:5080 -o 100 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 --logtostderr -v=2 --whitelist 10.0.0.0/8,172.16.0.0/12,192.168.0.0/16 --acl_secret_file /dgraph-acl/hmac-secret --enterprise_features --acl_access_ttl 10s + + dg2: + image: dgraph/dgraph:latest + container_name: bank-dg2 + working_dir: /data/dg2 + depends_on: + - dg1 + volumes: + - type: bind + source: $GOPATH/bin + target: /gobin + read_only: true + - type: bind + source: $GOPATH/src/github.com/dgraph-io/dgraph/ee/acl/hmac-secret + target: /dgraph-acl/hmac-secret + read_only: true + ports: + - 8182:8182 + - 9182:9182 + labels: + cluster: test + service: alpha + command: /gobin/dgraph alpha --my=dg2:7182 --lru_mb=1024 --zero=zero1:5080 -o 102 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 --logtostderr -v=2 --whitelist 10.0.0.0/8,172.16.0.0/12,192.168.0.0/16 --acl_secret_file /dgraph-acl/hmac-secret --enterprise_features --acl_access_ttl 10s + + dg3: + image: dgraph/dgraph:latest + container_name: bank-dg3 + working_dir: /data/dg3 + depends_on: + - dg2 + volumes: + - type: bind + source: $GOPATH/bin + target: /gobin + read_only: true + - type: bind + source: $GOPATH/src/github.com/dgraph-io/dgraph/ee/acl/hmac-secret + target: /dgraph-acl/hmac-secret + read_only: true + ports: + - 8183:8183 + - 9183:9183 + labels: + cluster: test + service: alpha + command: /gobin/dgraph alpha --my=dg3:7183 --lru_mb=1024 --zero=zero1:5080 -o 103 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 --logtostderr -v=2 --whitelist 10.0.0.0/8,172.16.0.0/12,192.168.0.0/16 --acl_secret_file /dgraph-acl/hmac-secret --enterprise_features --acl_access_ttl 10s diff --git a/posting/multi_part_list_test.go b/posting/multi_part_list_test.go new file mode 100644 index 00000000000..aeb3276b557 --- /dev/null +++ b/posting/multi_part_list_test.go @@ -0,0 +1,79 @@ +/* + * Copyright 2015-2019 Dgraph Labs, Inc. and Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package posting + +import ( + "context" + "encoding/json" + "fmt" + "net/http" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/dgraph-io/dgo" + "github.com/dgraph-io/dgo/protos/api" + "github.com/dgraph-io/dgraph/x" + "google.golang.org/grpc" +) + +var client *dgo.Dgraph + +func assignUids(num uint64) { + _, err := http.Get(fmt.Sprintf("http://localhost:6080/assign?what=uids&num=%d", num)) + if err != nil { + panic(fmt.Sprintf("Could not assign uids. Got error %v", err.Error())) + } +} + +func getNewClient() *dgo.Dgraph { + conn, err := grpc.Dial("localhost:9180", grpc.WithInsecure()) + x.Check(err) + return dgo.NewDgraphClient(api.NewDgraphClient(conn)) +} + +func setSchema(schema string) { + err := client.Alter(context.Background(), &api.Operation{ + Schema: schema, + }) + if err != nil { + panic(fmt.Sprintf("Could not alter schema. Got error %v", err.Error())) + } +} + +func processQuery(t *testing.T, ctx context.Context, query string) (string, error) { + txn := client.NewTxn() + defer txn.Discard(ctx) + + res, err := txn.Query(ctx, query) + if err != nil { + return "", err + } + + response := map[string]interface{}{} + response["data"] = json.RawMessage(string(res.Json)) + + jsonResponse, err := json.Marshal(response) + require.NoError(t, err) + return string(jsonResponse), err +} + +func processQueryNoErr(t *testing.T, query string) string { + res, err := processQuery(t, context.Background(), query) + require.NoError(t, err) + return res +} From becab2d901459c6c693253c86f67b03a9c0309ab Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 13 Mar 2019 11:53:18 -0700 Subject: [PATCH 38/67] Delete unused testing code. --- posting/docker-compose.yml | 87 --------------------------------- posting/multi_part_list_test.go | 79 ------------------------------ 2 files changed, 166 deletions(-) delete mode 100644 posting/docker-compose.yml delete mode 100644 posting/multi_part_list_test.go diff --git a/posting/docker-compose.yml b/posting/docker-compose.yml deleted file mode 100644 index d13f6cc5f67..00000000000 --- a/posting/docker-compose.yml +++ /dev/null @@ -1,87 +0,0 @@ -# Docker compose file for testing. - -version: "3.5" -services: - zero1: - image: dgraph/dgraph:latest - container_name: bank-dg0.1 - working_dir: /data/dg0.1 - ports: - - 5080:5080 - - 6080:6080 - labels: - cluster: test - service: zero - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - command: /gobin/dgraph zero -o 0 --my=zero1:5080 --logtostderr -v=2 --bindall --expose_trace --profile_mode block --block_rate 10 - - dg1: - image: dgraph/dgraph:latest - container_name: bank-dg1 - working_dir: /data/dg1 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - - type: bind - source: $GOPATH/src/github.com/dgraph-io/dgraph/ee/acl/hmac-secret - target: /dgraph-acl/hmac-secret - read_only: true - ports: - - 8180:8180 - - 9180:9180 - labels: - cluster: test - service: alpha - command: /gobin/dgraph alpha --my=dg1:7180 --lru_mb=1024 --zero=zero1:5080 -o 100 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 --logtostderr -v=2 --whitelist 10.0.0.0/8,172.16.0.0/12,192.168.0.0/16 --acl_secret_file /dgraph-acl/hmac-secret --enterprise_features --acl_access_ttl 10s - - dg2: - image: dgraph/dgraph:latest - container_name: bank-dg2 - working_dir: /data/dg2 - depends_on: - - dg1 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - - type: bind - source: $GOPATH/src/github.com/dgraph-io/dgraph/ee/acl/hmac-secret - target: /dgraph-acl/hmac-secret - read_only: true - ports: - - 8182:8182 - - 9182:9182 - labels: - cluster: test - service: alpha - command: /gobin/dgraph alpha --my=dg2:7182 --lru_mb=1024 --zero=zero1:5080 -o 102 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 --logtostderr -v=2 --whitelist 10.0.0.0/8,172.16.0.0/12,192.168.0.0/16 --acl_secret_file /dgraph-acl/hmac-secret --enterprise_features --acl_access_ttl 10s - - dg3: - image: dgraph/dgraph:latest - container_name: bank-dg3 - working_dir: /data/dg3 - depends_on: - - dg2 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - - type: bind - source: $GOPATH/src/github.com/dgraph-io/dgraph/ee/acl/hmac-secret - target: /dgraph-acl/hmac-secret - read_only: true - ports: - - 8183:8183 - - 9183:9183 - labels: - cluster: test - service: alpha - command: /gobin/dgraph alpha --my=dg3:7183 --lru_mb=1024 --zero=zero1:5080 -o 103 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 --logtostderr -v=2 --whitelist 10.0.0.0/8,172.16.0.0/12,192.168.0.0/16 --acl_secret_file /dgraph-acl/hmac-secret --enterprise_features --acl_access_ttl 10s diff --git a/posting/multi_part_list_test.go b/posting/multi_part_list_test.go deleted file mode 100644 index aeb3276b557..00000000000 --- a/posting/multi_part_list_test.go +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Copyright 2015-2019 Dgraph Labs, Inc. and Contributors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package posting - -import ( - "context" - "encoding/json" - "fmt" - "net/http" - "testing" - - "github.com/stretchr/testify/require" - - "github.com/dgraph-io/dgo" - "github.com/dgraph-io/dgo/protos/api" - "github.com/dgraph-io/dgraph/x" - "google.golang.org/grpc" -) - -var client *dgo.Dgraph - -func assignUids(num uint64) { - _, err := http.Get(fmt.Sprintf("http://localhost:6080/assign?what=uids&num=%d", num)) - if err != nil { - panic(fmt.Sprintf("Could not assign uids. Got error %v", err.Error())) - } -} - -func getNewClient() *dgo.Dgraph { - conn, err := grpc.Dial("localhost:9180", grpc.WithInsecure()) - x.Check(err) - return dgo.NewDgraphClient(api.NewDgraphClient(conn)) -} - -func setSchema(schema string) { - err := client.Alter(context.Background(), &api.Operation{ - Schema: schema, - }) - if err != nil { - panic(fmt.Sprintf("Could not alter schema. Got error %v", err.Error())) - } -} - -func processQuery(t *testing.T, ctx context.Context, query string) (string, error) { - txn := client.NewTxn() - defer txn.Discard(ctx) - - res, err := txn.Query(ctx, query) - if err != nil { - return "", err - } - - response := map[string]interface{}{} - response["data"] = json.RawMessage(string(res.Json)) - - jsonResponse, err := json.Marshal(response) - require.NoError(t, err) - return string(jsonResponse), err -} - -func processQueryNoErr(t *testing.T, query string) string { - res, err := processQuery(t, context.Background(), query) - require.NoError(t, err) - return res -} From 294b8d470f21010023aa8aa5ee213b4655baa1e8 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 13 Mar 2019 12:07:58 -0700 Subject: [PATCH 39/67] Remove max_posting_list_size flag --- dgraph/cmd/alpha/run.go | 4 +--- posting/list.go | 3 ++- posting/list_test.go | 8 ++++---- posting/lists.go | 6 +----- worker/worker_test.go | 2 +- 5 files changed, 9 insertions(+), 14 deletions(-) diff --git a/dgraph/cmd/alpha/run.go b/dgraph/cmd/alpha/run.go index 930c925a3fb..30fe1023f7b 100644 --- a/dgraph/cmd/alpha/run.go +++ b/dgraph/cmd/alpha/run.go @@ -152,8 +152,6 @@ they form a Raft group and provide synchronous replication. flag.Uint64("query_edge_limit", 1e6, "Limit for the maximum number of edges that can be returned in a query."+ " This applies to shortest path and recursive queries.") - flag.Int("max_posting_list_size", posting.MaxListSize, - "Limit for the maximum size of a posting list before it is split into multiple parts.") // TLS configurations x.RegisterTLSFlags(flag) @@ -523,7 +521,7 @@ func run() { // Posting will initialize index which requires schema. Hence, initialize // schema before calling posting.Init(). schema.Init(edgraph.State.Pstore) - posting.Init(edgraph.State.Pstore, Alpha.Conf.GetInt("max_posting_list_size")) + posting.Init(edgraph.State.Pstore) defer posting.Cleanup() worker.Init(edgraph.State.Pstore) diff --git a/posting/list.go b/posting/list.go index 4a684f452c7..753f33d82a1 100644 --- a/posting/list.go +++ b/posting/list.go @@ -51,6 +51,7 @@ var ( ErrInvalidTxn = fmt.Errorf("Invalid transaction") ErrStopIteration = errors.New("Stop iteration") emptyPosting = &pb.Posting{} + maxListSize = MB/2 ) const ( @@ -1155,7 +1156,7 @@ func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { } func needsSplit(plist *pb.PostingList) bool { - return plist.Size() >= MaxListSize && len(plist.Pack.Blocks) > 1 + return plist.Size() >= maxListSize && len(plist.Pack.Blocks) > 1 } func (l *List) splitList() error { diff --git a/posting/list_test.go b/posting/list_test.go index ba97bd3f24a..42b400e0eb9 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -417,7 +417,7 @@ func TestAddMutation_mrjn1(t *testing.T) { func TestMillion(t *testing.T) { // Ensure list is stored in a single part. - MaxListSize = math.MaxInt32 + maxListSize = math.MaxInt32 key := x.DataKey("bal", 1331) ol, err := getNew(key, ps) @@ -859,9 +859,9 @@ func TestAfterUIDCountWithCommit(t *testing.T) { func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { // For testing, set the max list size to a lower threshold. - MaxListSize = 5000 + maxListSize = 5000 defer func() { - MaxListSize = math.MaxInt32 + maxListSize = math.MaxInt32 }() key := x.DataKey("bal", 1331) @@ -1010,7 +1010,7 @@ func TestMain(m *testing.M) { opt.ValueDir = dir ps, err = badger.OpenManaged(opt) x.Check(err) - Init(ps, 0) + Init(ps) schema.Init(ps) r := m.Run() diff --git a/posting/lists.go b/posting/lists.go index 120fd101c3b..dfbc323efc1 100644 --- a/posting/lists.go +++ b/posting/lists.go @@ -147,14 +147,10 @@ func updateMemoryMetrics(lc *y.Closer) { var ( pstore *badger.DB closer *y.Closer - MaxListSize = 10 * MB ) // Init initializes the posting lists package, the in memory and dirty list hash. -func Init(ps *badger.DB, maxListSize int) { - if maxListSize > 0 { - MaxListSize = maxListSize - } +func Init(ps *badger.DB) { pstore = ps closer = y.NewCloser(1) go updateMemoryMetrics(closer) diff --git a/worker/worker_test.go b/worker/worker_test.go index bf414d48aeb..776b536f839 100644 --- a/worker/worker_test.go +++ b/worker/worker_test.go @@ -625,7 +625,7 @@ func TestMain(m *testing.M) { ps, err := badger.OpenManaged(opt) x.Check(err) pstore = ps - posting.Init(ps, 0) + posting.Init(ps) Init(ps) os.Exit(m.Run()) } From 38f7e7e92987c928160b8daacde0bcd0dcfcf05c Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 13 Mar 2019 14:40:04 -0700 Subject: [PATCH 40/67] Add extra tests for multi-posting lists. --- posting/list.go | 15 ++++- posting/list_test.go | 132 +++++++++++++++++++++++++++++++++++-------- posting/lists.go | 4 +- 3 files changed, 121 insertions(+), 30 deletions(-) diff --git a/posting/list.go b/posting/list.go index 753f33d82a1..017790d4dec 100644 --- a/posting/list.go +++ b/posting/list.go @@ -51,7 +51,7 @@ var ( ErrInvalidTxn = fmt.Errorf("Invalid transaction") ErrStopIteration = errors.New("Stop iteration") emptyPosting = &pb.Posting{} - maxListSize = MB/2 + maxListSize = MB / 2 ) const ( @@ -115,9 +115,9 @@ type PIterator struct { func (it *PIterator) Init(l *List, afterUid, deleteBelow uint64) error { it.l = l - it.splitIdx = 0 + it.splitIdx = it.selectInitialSplit(afterUid) if len(it.l.plist.Splits) > 0 { - plist, err := l.readListPart(it.l.plist.Splits[0]) + plist, err := l.readListPart(it.l.plist.Splits[it.splitIdx]) if err != nil { return err } @@ -142,6 +142,15 @@ func (it *PIterator) Init(l *List, afterUid, deleteBelow uint64) error { return nil } +func (it *PIterator) selectInitialSplit(afterUid uint64) int { + for i, startUid := range it.l.plist.Splits { + if startUid > afterUid { + return i - 1 + } + } + return len(it.l.plist.Splits) - 1 +} + func (it *PIterator) moveToNextSplit() error { it.splitIdx++ plist, err := it.l.readListPart(it.l.plist.Splits[it.splitIdx]) diff --git a/posting/list_test.go b/posting/list_test.go index 42b400e0eb9..6e45d8f3b8c 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -19,6 +19,7 @@ package posting import ( "context" "io/ioutil" + "log" "math" "math/rand" "os" @@ -231,6 +232,7 @@ func TestAddMutation_DelSet(t *testing.T) { require.EqualValues(t, 1, ol.Length(3, 0)) checkValue(t, ol, "newcars", 3) } + func TestAddMutation_DelRead(t *testing.T) { key := x.DataKey("value", 1543) ol, err := GetNoStore(key) @@ -868,7 +870,7 @@ func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { ol, err := getNew(key, ps) require.NoError(t, err) commits := 0 - for i := 2; i <= size; i += 2 { + for i := 1; i <= size; i++ { edge := &pb.DirectedEdge{ ValueId: uint64(i), } @@ -892,6 +894,57 @@ func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { return ol, commits } +func createAndDeleteMultiPartList(t *testing.T, size int) (*List, int) { + // For testing, set the max list size to a lower threshold. + maxListSize = math.MaxInt32 + defer func() { + maxListSize = math.MaxInt32 + }() + + key := x.DataKey("bal_del", 1331) + ol, err := getNew(key, ps) + require.NoError(t, err) + commits := 0 + for i := 1; i <= size; i++ { + edge := &pb.DirectedEdge{ + ValueId: uint64(i), + } + + txn := Txn{StartTs: uint64(i)} + addMutationHelper(t, ol, edge, Set, &txn) + require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) + if i%2000 == 0 { + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + } + commits++ + } + + // Delete all the previously inserted entries from the list. + baseStartTs := uint64(size) + 1 + for i := 1; i <= size; i++ { + edge := &pb.DirectedEdge{ + ValueId: uint64(i), + } + txn := Txn{StartTs: baseStartTs + uint64(i)} + addMutationHelper(t, ol, edge, Del, &txn) + require.NoError(t, ol.CommitMutation(baseStartTs+uint64(i), baseStartTs+uint64(i)+1)) + if i%2000 == 0 { + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + } + commits++ + } + + return ol, commits +} + func writePostingListToDisk(kvs []*bpb.KV) error { writer := NewTxnWriter(pstore) for _, kv := range kvs { @@ -903,25 +956,41 @@ func writePostingListToDisk(kvs []*bpb.KV) error { } func TestMultiPartListBasic(t *testing.T) { - N := int(1e5) - ol, commits := createMultiPartList(t, N, false) + size := int(1e5) + ol, commits := createMultiPartList(t, size, false) t.Logf("List parts %v", len(ol.plist.Splits)) - opt := ListOptions{ReadTs: uint64(N) + 1} + opt := ListOptions{ReadTs: uint64(size) + 1} l, err := ol.Uids(opt) require.NoError(t, err) require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) for i, uid := range l.Uids { - require.Equal(t, uint64(i+1)*2, uid) + require.Equal(t, uint64(i+1), uid) + } +} + +func TestMultiPartListIterAfterUid(t *testing.T) { + size := int(1e5) + ol, _ := createMultiPartList(t, size, false) + t.Logf("List parts %v", len(ol.plist.Splits)) + + var visitedUids []uint64 + ol.Iterate(uint64(size+1), 50000, func(p *pb.Posting) error { + visitedUids = append(visitedUids, p.Uid) + return nil + }) + require.Equal(t, 50001, len(visitedUids)) + for i, uid := range visitedUids { + require.Equal(t, uint64(50000+i), uid) } } func TestMultiPartListWithPostings(t *testing.T) { - N := int(1e5) - ol, commits := createMultiPartList(t, N, true) + size := int(1e5) + ol, commits := createMultiPartList(t, size, true) t.Logf("List parts %v", len(ol.plist.Splits)) var labels []string - err := ol.Iterate(uint64(N)+1, 0, func(p *pb.Posting) error { + err := ol.Iterate(uint64(size)+1, 0, func(p *pb.Posting) error { if len(p.Label) > 0 { labels = append(labels, p.Label) } @@ -930,19 +999,18 @@ func TestMultiPartListWithPostings(t *testing.T) { require.NoError(t, err) require.Equal(t, commits, len(labels)) for i, label := range labels { - require.Equal(t, label, strconv.Itoa(int(i+1)*2)) + require.Equal(t, label, strconv.Itoa(int(i+1))) } } func TestMultiPartListMarshal(t *testing.T) { - N := int(1e5) - ol, commits := createMultiPartList(t, N, false) + size := int(1e5) + ol, _ := createMultiPartList(t, size, false) t.Logf("List parts %v", len(ol.plist.Splits)) kvs, err := ol.Rollup() require.NoError(t, err) require.Equal(t, len(kvs), len(ol.plist.Splits)+1) - require.NoError(t, writePostingListToDisk(kvs)) key := x.DataKey("bal", 1331) @@ -960,21 +1028,11 @@ func TestMultiPartListMarshal(t *testing.T) { require.Equal(t, ol.minTs, kvs[i+1].Version) } - // Marshalling causes the mutation map to be cleaned up. Verify all changes - // can still be seen even though the posting lists have not been committed to - // disk yet. - opt := ListOptions{ReadTs: uint64(N) + 1} - l, err := ol.Uids(opt) - require.NoError(t, err) - require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) - for i, uid := range l.Uids { - require.Equal(t, uint64(i+1)*2, uid) - } } func TestMultiPartListWrite(t *testing.T) { - N := int(1e5) - originalList, commits := createMultiPartList(t, N, false) + size := int(1e5) + originalList, commits := createMultiPartList(t, size, false) kvs, err := originalList.Rollup() require.NoError(t, err) @@ -983,7 +1041,7 @@ func TestMultiPartListWrite(t *testing.T) { require.NoError(t, writePostingListToDisk(kvs)) newList, err := getNew(kvs[0].Key, ps) - opt := ListOptions{ReadTs: uint64(N) + 1} + opt := ListOptions{ReadTs: uint64(size) + 1} originalUids, err := originalList.Uids(opt) require.NoError(t, err) newUids, err := newList.Uids(opt) @@ -995,6 +1053,30 @@ func TestMultiPartListWrite(t *testing.T) { } } +func TestMultiPartListDelete(t *testing.T) { + size := int(1e4) + ol, commits := createAndDeleteMultiPartList(t, size) + t.Logf("List parts %v", len(ol.plist.Splits)) + require.Equal(t, size*2, commits) + + counter := 0 + ol.Iterate(math.MaxUint64, 0, func(p *pb.Posting) error { + log.Printf("%v", p.Uid) + counter++ + return nil + }) + require.Equal(t, 0, counter) + + kvs, err := ol.Rollup() + require.NoError(t, err) + require.Equal(t, len(kvs), len(ol.plist.Splits)+1) + + for _, kv := range kvs { + require.Equal(t, []byte{BitEmptyPosting}, kv.UserMeta) + require.Equal(t, ol.minTs, kv.Version) + } +} + var ps *badger.DB func TestMain(m *testing.M) { diff --git a/posting/lists.go b/posting/lists.go index dfbc323efc1..d8000086700 100644 --- a/posting/lists.go +++ b/posting/lists.go @@ -145,8 +145,8 @@ func updateMemoryMetrics(lc *y.Closer) { } var ( - pstore *badger.DB - closer *y.Closer + pstore *badger.DB + closer *y.Closer ) // Init initializes the posting lists package, the in memory and dirty list hash. From dd9b9683c9ecd51f73f8dfbe86a6ee9cdc10324a Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 20 Mar 2019 17:08:04 -0700 Subject: [PATCH 41/67] Manish's review. Renamed some funcs, readPartList does not use cache, etc. --- posting/list.go | 32 ++++++++++++++------------------ posting/list_test.go | 2 +- 2 files changed, 15 insertions(+), 19 deletions(-) diff --git a/posting/list.go b/posting/list.go index 017790d4dec..5e3b761569c 100644 --- a/posting/list.go +++ b/posting/list.go @@ -81,13 +81,10 @@ type List struct { partCache map[uint64]*pb.PostingList } -func getNextPartKey(baseKey []byte, nextPartStart uint64) []byte { - keyCopy := make([]byte, len(baseKey)) +func getSplitKey(baseKey []byte, startUid uint64) []byte { + keyCopy := make([]byte, len(baseKey)+8) copy(keyCopy, baseKey) - - encNexStart := make([]byte, 8) - binary.BigEndian.PutUint64(encNexStart, nextPartStart) - keyCopy = append(keyCopy, encNexStart...) + binary.BigEndian.PutUint64(keyCopy[len(baseKey):], startUid) return keyCopy } @@ -716,7 +713,8 @@ func (l *List) plsAreEmpty() (bool, error) { for _, startUid := range l.plist.Splits { kv := &bpb.KV{} kv.Version = l.minTs - kv.Key = getNextPartKey(l.key, startUid) + kv.Key = getSplitKey(l.key, startUid) + // TODO: Can be done purely from the cache? plist, err := l.readListPart(startUid) if err != nil { return false, err @@ -757,7 +755,7 @@ func (l *List) Rollup() ([]*bpb.KV, error) { for _, startUid := range l.plist.Splits { kv := &bpb.KV{} kv.Version = l.minTs - kv.Key = getNextPartKey(l.key, startUid) + kv.Key = getSplitKey(l.key, startUid) plist, err := l.readListPart(startUid) if err != nil { return nil, err @@ -886,7 +884,7 @@ func (l *List) rollup(readTs uint64) error { // Check if the list (or any of it's parts if it's been previously split) have // become too big. Split the list if that is the case. - if err := l.splitList(); err != nil { + if err := l.splitUpList(); err != nil { return nil } @@ -1146,14 +1144,11 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs return facets.CopyFacets(p.Facets, param), nil } +// readListPart does not use any cache. It directly reads from Badger. func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { - if part, ok := l.partCache[startUid]; ok { - return part, nil - } - - nextKey := getNextPartKey(l.key, startUid) + key := getSplitKey(l.key, startUid) txn := pstore.NewTransactionAt(l.minTs, false) - item, err := txn.Get(nextKey) + item, err := txn.Get(key) if err != nil { return nil, err } @@ -1164,11 +1159,12 @@ func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { return &part, nil } -func needsSplit(plist *pb.PostingList) bool { +func tooBig(plist *pb.PostingList) bool { return plist.Size() >= maxListSize && len(plist.Pack.Blocks) > 1 } -func (l *List) splitList() error { +// TODO: This can accept the generated map of start uid -> postinglist. +func (l *List) splitUpList() error { l.AssertLock() var lists []*pb.PostingList @@ -1186,7 +1182,7 @@ func (l *List) splitList() error { var newLists []*pb.PostingList for _, list := range lists { - if needsSplit(list) { + if tooBig(list) { splitList := splitPostingList(list) for _, part := range splitList { l.partCache[part.StartUid] = part diff --git a/posting/list_test.go b/posting/list_test.go index 6e45d8f3b8c..df5a0f6ce82 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -1017,7 +1017,7 @@ func TestMultiPartListMarshal(t *testing.T) { require.Equal(t, key, kvs[0].Key) for i, startUid := range ol.plist.Splits { - partKey := getNextPartKey(key, startUid) + partKey := getSplitKey(key, startUid) require.Equal(t, partKey, kvs[i+1].Key) part, err := ol.readListPart(startUid) require.NoError(t, err) From b1142320b6eedcc12dba2598457fdcba83ea3056 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 20 Mar 2019 17:40:49 -0700 Subject: [PATCH 42/67] Fix keys.go tests --- posting/list.go | 18 +++++++----------- posting/list_test.go | 9 +++++++-- x/keys.go | 27 +++++++++------------------ x/keys_test.go | 18 +++--------------- 4 files changed, 26 insertions(+), 46 deletions(-) diff --git a/posting/list.go b/posting/list.go index 5e3b761569c..91b044ecf1e 100644 --- a/posting/list.go +++ b/posting/list.go @@ -19,7 +19,6 @@ package posting import ( "bytes" "context" - "encoding/binary" "errors" "fmt" "log" @@ -81,13 +80,6 @@ type List struct { partCache map[uint64]*pb.PostingList } -func getSplitKey(baseKey []byte, startUid uint64) []byte { - keyCopy := make([]byte, len(baseKey)+8) - copy(keyCopy, baseKey) - binary.BigEndian.PutUint64(keyCopy[len(baseKey):], startUid) - return keyCopy -} - func (l *List) maxVersion() uint64 { l.RLock() defer l.RUnlock() @@ -713,7 +705,7 @@ func (l *List) plsAreEmpty() (bool, error) { for _, startUid := range l.plist.Splits { kv := &bpb.KV{} kv.Version = l.minTs - kv.Key = getSplitKey(l.key, startUid) + kv.Key = x.GetSplitKey(l.key, startUid) // TODO: Can be done purely from the cache? plist, err := l.readListPart(startUid) if err != nil { @@ -755,7 +747,7 @@ func (l *List) Rollup() ([]*bpb.KV, error) { for _, startUid := range l.plist.Splits { kv := &bpb.KV{} kv.Version = l.minTs - kv.Key = getSplitKey(l.key, startUid) + kv.Key = x.GetSplitKey(l.key, startUid) plist, err := l.readListPart(startUid) if err != nil { return nil, err @@ -1146,7 +1138,11 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs // readListPart does not use any cache. It directly reads from Badger. func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { - key := getSplitKey(l.key, startUid) + if part, ok := l.partCache[startUid]; ok { + return part, nil + } + + key := x.GetSplitKey(l.key, startUid) txn := pstore.NewTransactionAt(l.minTs, false) item, err := txn.Get(key) if err != nil { diff --git a/posting/list_test.go b/posting/list_test.go index df5a0f6ce82..be1b0597ddb 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -891,6 +891,12 @@ func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { commits++ } + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + return ol, commits } @@ -1017,7 +1023,7 @@ func TestMultiPartListMarshal(t *testing.T) { require.Equal(t, key, kvs[0].Key) for i, startUid := range ol.plist.Splits { - partKey := getSplitKey(key, startUid) + partKey := x.GetSplitKey(key, startUid) require.Equal(t, partKey, kvs[i+1].Key) part, err := ol.readListPart(startUid) require.NoError(t, err) @@ -1027,7 +1033,6 @@ func TestMultiPartListMarshal(t *testing.T) { require.Equal(t, []byte{BitCompletePosting}, kvs[i+1].UserMeta) require.Equal(t, ol.minTs, kvs[i+1].Version) } - } func TestMultiPartListWrite(t *testing.T) { diff --git a/x/keys.go b/x/keys.go index 472107485b3..05cee24043f 100644 --- a/x/keys.go +++ b/x/keys.go @@ -83,24 +83,6 @@ func DataKey(attr string, uid uint64) []byte { return buf } -func DataKeyWithStartUid(attr string, uid, startUid uint64) []byte { - buf := make([]byte, 2+len(attr)+2+8+8) - buf[0] = defaultPrefix - rest := buf[1:] - - rest = writeAttr(rest, attr) - rest[0] = ByteData - - rest = rest[1:] - binary.BigEndian.PutUint64(rest, uid) - - // This list is split in multiple parts. startUid represents the first UID - // in the range of UIDs stored by this part of the list. - rest = rest[8:] - binary.BigEndian.PutUint64(rest, startUid) - return buf -} - func ReverseKey(attr string, uid uint64) []byte { buf := make([]byte, 2+len(attr)+2+8) buf[0] = defaultPrefix @@ -299,6 +281,15 @@ func PredicatePrefix(predicate string) []byte { return buf } +// GetSplitKey takes a data key baseKey and generates the key of the list split +// that starts at startUid. +func GetSplitKey(baseKey []byte, startUid uint64) []byte { + keyCopy := make([]byte, len(baseKey)+8) + copy(keyCopy, baseKey) + binary.BigEndian.PutUint64(keyCopy[len(baseKey):], startUid) + return keyCopy +} + // Parse would parse the key. ParsedKey does not reuse the key slice, so the key slice can change // without affecting the contents of ParsedKey. func Parse(key []byte) *ParsedKey { diff --git a/x/keys_test.go b/x/keys_test.go index 7ff6ada0a48..65ff81d2dfc 100644 --- a/x/keys_test.go +++ b/x/keys_test.go @@ -52,12 +52,13 @@ func TestDataKey(t *testing.T) { } } -func TestDataKeyWithStartUid(t *testing.T) { +func TestParseKeysWithStartUid(t *testing.T) { var uid uint64 startUid := uint64(1024) for uid = 0; uid < 1001; uid++ { sattr := fmt.Sprintf("attr:%d", uid) - key := DataKeyWithStartUid(sattr, uid, startUid) + key := DataKey(sattr, uid) + key = GetSplitKey(key, startUid) pk := Parse(key) require.True(t, pk.IsData()) @@ -66,19 +67,6 @@ func TestDataKeyWithStartUid(t *testing.T) { require.Equal(t, startUid, pk.StartUid) require.Equal(t, true, pk.HasStartUid) } - - keys := make([]string, 0, 1024) - for uid = 1024; uid >= 1; uid-- { - key := DataKeyWithStartUid("testing.key", uid, startUid) - keys = append(keys, string(key)) - } - // Test that sorting is as expected. - sort.Strings(keys) - require.True(t, sort.StringsAreSorted(keys)) - for i, key := range keys { - exp := DataKeyWithStartUid("testing.key", uint64(i+1), startUid) - require.Equal(t, string(exp), key) - } } func TestIndexKey(t *testing.T) { From d907e5155cbcbcb539abc0691f4ac740200921e1 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 20 Mar 2019 18:04:13 -0700 Subject: [PATCH 43/67] Rename deleteBelow and add sanity check for it. --- posting/list.go | 39 +++++++++++++++++++++++---------------- posting/list_test.go | 6 ------ 2 files changed, 23 insertions(+), 22 deletions(-) diff --git a/posting/list.go b/posting/list.go index 91b044ecf1e..48d0d72af0d 100644 --- a/posting/list.go +++ b/posting/list.go @@ -97,12 +97,19 @@ type PIterator struct { uids []uint64 uidx int // Offset into the uids slice - afterUid uint64 - deleteBelow uint64 - splitIdx int + afterUid uint64 + splitIdx int + // The timestamp of a delete marker in the mutable layer. If this value is greater + // than zero, then the immutable posting list should not be traversed. + deleteBelowTs uint64 } -func (it *PIterator) Init(l *List, afterUid, deleteBelow uint64) error { +func (it *PIterator) Init(l *List, afterUid, deleteBelowTs uint64) error { + if deleteBelowTs > 0 && deleteBelowTs <= l.minTs { + return fmt.Errorf("deleteBelowTs (%d) must be greater than the minTs in the list (%d)", + deleteBelowTs, l.minTs) + } + it.l = l it.splitIdx = it.selectInitialSplit(afterUid) if len(it.l.plist.Splits) > 0 { @@ -116,7 +123,7 @@ func (it *PIterator) Init(l *List, afterUid, deleteBelow uint64) error { } it.afterUid = afterUid - it.deleteBelow = deleteBelow + it.deleteBelowTs = deleteBelowTs it.uidPosting = &pb.Posting{} it.dec = &codec.Decoder{Pack: it.plist.Pack} @@ -163,7 +170,7 @@ func (it *PIterator) moveToNextSplit() error { } func (it *PIterator) Next() error { - if it.deleteBelow > 0 { + if it.deleteBelowTs > 0 { return nil } @@ -177,7 +184,7 @@ func (it *PIterator) Next() error { } func (it *PIterator) Valid() bool { - if it.deleteBelow > 0 { + if it.deleteBelowTs > 0 { return false } @@ -539,16 +546,16 @@ func (l *List) pickPostings(readTs uint64) (uint64, []*pb.Posting) { } // First pick up the postings. - var deleteBelow uint64 + var deleteBelowTs uint64 var posts []*pb.Posting for startTs, plist := range l.mutationMap { // Pick up the transactions which are either committed, or the one which is ME. effectiveTs := effective(startTs, plist.CommitTs) - if effectiveTs > deleteBelow { - // We're above the deleteBelow marker. We wouldn't reach here if effectiveTs is zero. + if effectiveTs > deleteBelowTs { + // We're above the deleteBelowTs marker. We wouldn't reach here if effectiveTs is zero. for _, mpost := range plist.Postings { if hasDeleteAll(mpost) { - deleteBelow = effectiveTs + deleteBelowTs = effectiveTs continue } posts = append(posts, mpost) @@ -556,12 +563,12 @@ func (l *List) pickPostings(readTs uint64) (uint64, []*pb.Posting) { } } - if deleteBelow > 0 { + if deleteBelowTs > 0 { // There was a delete all marker. So, trim down the list of postings. result := posts[:0] for _, post := range posts { effectiveTs := effective(post.StartTs, post.CommitTs) - if effectiveTs < deleteBelow { // Do pick the posts at effectiveTs == deleteBelow. + if effectiveTs < deleteBelowTs { // Do pick the posts at effectiveTs == deleteBelowTs. continue } result = append(result, post) @@ -580,13 +587,13 @@ func (l *List) pickPostings(readTs uint64) (uint64, []*pb.Posting) { } return pi.Uid < pj.Uid }) - return deleteBelow, posts + return deleteBelowTs, posts } func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) error) error { l.AssertRLock() - deleteBelow, mposts := l.pickPostings(readTs) + deleteBelowTs, mposts := l.pickPostings(readTs) if readTs < l.minTs { return x.Errorf("readTs: %d less than minTs: %d for key: %q", readTs, l.minTs, l.key) } @@ -601,7 +608,7 @@ func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) e var mp, pp *pb.Posting var pitr PIterator - err := pitr.Init(l, afterUid, deleteBelow) + err := pitr.Init(l, afterUid, deleteBelowTs) if err != nil { return err } diff --git a/posting/list_test.go b/posting/list_test.go index be1b0597ddb..ddc5f09771f 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -891,12 +891,6 @@ func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { commits++ } - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - return ol, commits } From 2033a412000ac8da1be0476f270dd697f9e8ccd4 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Wed, 20 Mar 2019 18:12:03 -0700 Subject: [PATCH 44/67] Move iteration logic to Next() --- posting/list.go | 38 +++++++++++++++++--------------------- 1 file changed, 17 insertions(+), 21 deletions(-) diff --git a/posting/list.go b/posting/list.go index 48d0d72af0d..67c365db2e9 100644 --- a/posting/list.go +++ b/posting/list.go @@ -171,6 +171,7 @@ func (it *PIterator) moveToNextSplit() error { func (it *PIterator) Next() error { if it.deleteBelowTs > 0 { + it.uids = nil return nil } @@ -180,33 +181,28 @@ func (it *PIterator) Next() error { } it.uidx = 0 it.uids = it.dec.Next() - return nil -} -func (it *PIterator) Valid() bool { - if it.deleteBelowTs > 0 { - return false - } - - if len(it.uids) > 0 { - return true - } - - if len(it.l.plist.Splits) == 0 { - return false - } - - for it.splitIdx+1 < len(it.l.plist.Splits) { - if err := it.moveToNextSplit(); err != nil { - return false + if len(it.uids) == 0 { + if len(it.l.plist.Splits) == 0 { + return nil } - if len(it.uids) > 0 { - return true + for it.splitIdx+1 < len(it.l.plist.Splits) { + if err := it.moveToNextSplit(); err != nil { + return err + } + + if len(it.uids) > 0 { + return nil + } } } - return false + return nil +} + +func (it *PIterator) Valid() bool { + return len(it.uids) > 0 } func (it *PIterator) Posting() *pb.Posting { From b33444b4e4af853683078600953534571f61abfa Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 21 Mar 2019 11:37:21 -0700 Subject: [PATCH 45/67] Add documentation for pickPostings --- posting/list.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/posting/list.go b/posting/list.go index 67c365db2e9..e50401647c4 100644 --- a/posting/list.go +++ b/posting/list.go @@ -527,6 +527,10 @@ func (l *List) Conflicts(readTs uint64) []uint64 { return conflicts } +// pickPostings goes through the mutable layer and returns the appropriate postings, +// along with the timestamp of the delete marker, if any. If this timestamp is greater +// than zero, it indicates that the immutable layer should be ignored during traversals. +// If greater than zero, this timestamp must thus be greater than l.minTs. func (l *List) pickPostings(readTs uint64) (uint64, []*pb.Posting) { // This function would return zero ts for entries above readTs. effective := func(start, commit uint64) uint64 { From 7dc357e05e25469b7a33ebf02e9de3af238e06bc Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 21 Mar 2019 14:06:40 -0700 Subject: [PATCH 46/67] Make rollup read-only, remove cache. --- posting/list.go | 160 ++++++++++++++++++------------------------- posting/list_test.go | 16 ++++- posting/mvcc.go | 1 - 3 files changed, 81 insertions(+), 96 deletions(-) diff --git a/posting/list.go b/posting/list.go index e50401647c4..a40e24e1dd6 100644 --- a/posting/list.go +++ b/posting/list.go @@ -76,8 +76,6 @@ type List struct { pendingTxns int32 // Using atomic for this, to avoid locking in SetForDeletion operation. deleteMe int32 // Using atomic for this, to avoid expensive SetForDeletion operation. - - partCache map[uint64]*pb.PostingList } func (l *List) maxVersion() uint64 { @@ -700,24 +698,17 @@ func (l *List) Length(readTs, afterUid uint64) int { return l.length(readTs, afterUid) } -func (l *List) plsAreEmpty() (bool, error) { - if len(l.plist.Splits) == 0 { - if l.plist.Pack == nil || len(l.plist.Pack.Blocks) == 0 { +func (out *rollupOutput) plsAreEmpty() (bool, error) { + if len(out.newPlist.Splits) == 0 { + if out.newPlist.Pack == nil || len(out.newPlist.Pack.Blocks) == 0 { return true, nil - } else { - return false, nil } + return false, nil } - for _, startUid := range l.plist.Splits { - kv := &bpb.KV{} - kv.Version = l.minTs - kv.Key = x.GetSplitKey(l.key, startUid) + for _, startUid := range out.newPlist.Splits { // TODO: Can be done purely from the cache? - plist, err := l.readListPart(startUid) - if err != nil { - return false, err - } + plist := out.newSplits[startUid] if plist.Pack == nil || len(plist.Pack.Blocks) == 0 { continue @@ -729,14 +720,18 @@ func (l *List) plsAreEmpty() (bool, error) { } func (l *List) Rollup() ([]*bpb.KV, error) { - l.Lock() - defer l.Unlock() - if err := l.rollup(math.MaxUint64); err != nil { + l.RLock() + defer l.RUnlock() + out, err := l.rollup(math.MaxUint64) + if err != nil { return nil, err } + if out == nil { + return nil, nil + } var setEmptyBit bool - if empty, err := l.plsAreEmpty(); err != nil { + if empty, err := out.plsAreEmpty(); err != nil { return nil, err } else if empty { setEmptyBit = true @@ -744,21 +739,18 @@ func (l *List) Rollup() ([]*bpb.KV, error) { var kvs []*bpb.KV kv := &bpb.KV{} - kv.Version = l.minTs + kv.Version = out.newMinTs kv.Key = l.key - val, meta := marshalPostingList(l.plist, setEmptyBit) + val, meta := marshalPostingList(out.newPlist, setEmptyBit) kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) - for _, startUid := range l.plist.Splits { + for _, startUid := range out.newPlist.Splits { kv := &bpb.KV{} - kv.Version = l.minTs + kv.Version = out.newMinTs kv.Key = x.GetSplitKey(l.key, startUid) - plist, err := l.readListPart(startUid) - if err != nil { - return nil, err - } + plist := out.newSplits[startUid] val, meta := marshalPostingList(plist, setEmptyBit) kv.UserMeta = []byte{meta} kv.Value = val @@ -780,17 +772,28 @@ func marshalPostingList(plist *pb.PostingList, setEmptyBit bool) ([]byte, byte) const blockSize int = 256 +type rollupOutput struct { + newPlist *pb.PostingList + newSplits map[uint64]*pb.PostingList + newMinTs uint64 +} + // Merge all entries in mutation layer with commitTs <= l.commitTs into // immutable layer. Note that readTs can be math.MaxUint64, so do NOT use it // directly. It should only serve as the read timestamp for iteration. -func (l *List) rollup(readTs uint64) error { - l.AssertLock() +func (l *List) rollup(readTs uint64) (*rollupOutput, error) { + l.AssertRLock() // Pick all committed entries if l.minTs > readTs { // If we are already past the readTs, then skip the rollup. - return nil + return nil, nil + } + + newPlist := &pb.PostingList{ + Splits: l.plist.Splits, } + newSplits := make(map[uint64]*pb.PostingList) var plist *pb.PostingList var final *pb.PostingList @@ -799,15 +802,15 @@ func (l *List) rollup(readTs uint64) error { var splitIdx int // Method to properly initialize all the variables described above. - init := func() error { + init := func() { final = new(pb.PostingList) enc = codec.Encoder{BlockSize: blockSize} // If not a multi-part list, all uids go to the same encoder. if len(l.plist.Splits) == 0 { - plist = l.plist + plist = newPlist endUid = math.MaxUint64 - return nil + return } // Otherwise, load the corresponding part and set endUid to correctly @@ -819,26 +822,21 @@ func (l *List) rollup(readTs uint64) error { endUid = l.plist.Splits[splitIdx+1] - 1 } - var err error - plist, err = l.readListPart(startUid) - return err - } - - if err := init(); err != nil { - return err + plist = &pb.PostingList{ + StartUid: startUid, + } } + init() err := l.iterate(readTs, 0, func(p *pb.Posting) error { if p.Uid > endUid { final.Pack = enc.Done() plist.Pack = final.Pack plist.Postings = final.Postings - l.partCache[plist.StartUid] = plist + newSplits[plist.StartUid] = plist splitIdx++ - if err := init(); err != nil { - return err - } + init() } enc.Add(p.Uid) @@ -852,7 +850,9 @@ func (l *List) rollup(readTs uint64) error { final.Pack = enc.Done() plist.Pack = final.Pack plist.Postings = final.Postings - l.partCache[plist.StartUid] = plist + if len(l.plist.Splits) > 0 { + newSplits[plist.StartUid] = plist + } maxCommitTs := l.minTs { @@ -867,27 +867,15 @@ func (l *List) rollup(readTs uint64) error { } } - // Keep all uncommitted Entries or postings with commitTs > l.commitTs - // in mutation map. Discard all else. - // TODO: This could be removed after LRU cache is removed. - for startTs, plist := range l.mutationMap { - cl := plist.CommitTs - if cl == 0 || cl > maxCommitTs { - // Keep this. - } else { - delete(l.mutationMap, startTs) - } - } - - l.minTs = maxCommitTs - // Check if the list (or any of it's parts if it's been previously split) have // become too big. Split the list if that is the case. - if err := l.splitUpList(); err != nil { - return nil + out := &rollupOutput{ + newPlist: newPlist, + newSplits: newSplits, + newMinTs: maxCommitTs, } - - return nil + out.splitUpList() + return out, nil } func (l *List) ApproxLen() int { @@ -1143,12 +1131,7 @@ func (l *List) Facets(readTs uint64, param *pb.FacetParams, langs []string) (fs return facets.CopyFacets(p.Facets, param), nil } -// readListPart does not use any cache. It directly reads from Badger. func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { - if part, ok := l.partCache[startUid]; ok { - return part, nil - } - key := x.GetSplitKey(l.key, startUid) txn := pstore.NewTransactionAt(l.minTs, false) item, err := txn.Get(key) @@ -1166,19 +1149,14 @@ func tooBig(plist *pb.PostingList) bool { return plist.Size() >= maxListSize && len(plist.Pack.Blocks) > 1 } -// TODO: This can accept the generated map of start uid -> postinglist. -func (l *List) splitUpList() error { - l.AssertLock() - +func (out *rollupOutput) splitUpList() { var lists []*pb.PostingList - if len(l.plist.Splits) == 0 { - lists = append(lists, l.plist) + + if len(out.newPlist.Splits) == 0 { + lists = append(lists, out.newPlist) } else { - for _, startUid := range l.plist.Splits { - part, err := l.readListPart(startUid) - if err != nil { - return err - } + for _, startUid := range out.newPlist.Splits { + part := out.newSplits[startUid] lists = append(lists, part) } } @@ -1188,7 +1166,7 @@ func (l *List) splitUpList() error { if tooBig(list) { splitList := splitPostingList(list) for _, part := range splitList { - l.partCache[part.StartUid] = part + out.newSplits[part.StartUid] = part newLists = append(newLists, part) } } else { @@ -1196,19 +1174,17 @@ func (l *List) splitUpList() error { } } - if len(newLists) == 1 || len(newLists) == len(l.plist.Splits) { - return nil - } else { - var splits []uint64 - for _, list := range newLists { - splits = append(splits, list.StartUid) - } - l.plist = &pb.PostingList{ - CommitTs: l.plist.CommitTs, - Splits: splits, - } + if len(newLists) == 1 || len(newLists) == len(out.newPlist.Splits) { + return + } + + var splits []uint64 + for _, list := range newLists { + splits = append(splits, list.StartUid) + } + out.newPlist = &pb.PostingList{ + Splits: splits, } - return nil } func splitPostingList(plist *pb.PostingList) []*pb.PostingList { diff --git a/posting/list_test.go b/posting/list_test.go index ddc5f09771f..a66b9bc516e 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -437,11 +437,15 @@ func TestMillion(t *testing.T) { // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting // list. t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) - _, err := ol.Rollup() + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) require.NoError(t, err) } commits++ } + t.Logf("Completed a million writes.\n") opt := ListOptions{ReadTs: uint64(N) + 1} l, err := ol.Uids(opt) @@ -866,7 +870,7 @@ func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { maxListSize = math.MaxInt32 }() - key := x.DataKey("bal", 1331) + key := x.DataKey("multi-bal", 1331) ol, err := getNew(key, ps) require.NoError(t, err) commits := 0 @@ -891,6 +895,12 @@ func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { commits++ } + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + return ol, commits } @@ -1013,7 +1023,7 @@ func TestMultiPartListMarshal(t *testing.T) { require.Equal(t, len(kvs), len(ol.plist.Splits)+1) require.NoError(t, writePostingListToDisk(kvs)) - key := x.DataKey("bal", 1331) + key := x.DataKey("multi-bal", 1331) require.Equal(t, key, kvs[0].Key) for i, startUid := range ol.plist.Splits { diff --git a/posting/mvcc.go b/posting/mvcc.go index 8d220d84ef6..1efb53fe059 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -185,7 +185,6 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { l := new(List) l.key = key l.mutationMap = make(map[uint64]*pb.PostingList) - l.partCache = make(map[uint64]*pb.PostingList) l.plist = new(pb.PostingList) // Iterates from highest Ts to lowest Ts From 92f872461e2e5ae7c1b4ea40e024eccddf10502a Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 21 Mar 2019 14:25:09 -0700 Subject: [PATCH 47/67] Remove err from plsAreEmpty --- posting/list.go | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/posting/list.go b/posting/list.go index a40e24e1dd6..732431a9a8c 100644 --- a/posting/list.go +++ b/posting/list.go @@ -698,12 +698,12 @@ func (l *List) Length(readTs, afterUid uint64) int { return l.length(readTs, afterUid) } -func (out *rollupOutput) plsAreEmpty() (bool, error) { +func (out *rollupOutput) plsAreEmpty() bool { if len(out.newPlist.Splits) == 0 { if out.newPlist.Pack == nil || len(out.newPlist.Pack.Blocks) == 0 { - return true, nil + return true } - return false, nil + return false } for _, startUid := range out.newPlist.Splits { @@ -713,10 +713,10 @@ func (out *rollupOutput) plsAreEmpty() (bool, error) { if plist.Pack == nil || len(plist.Pack.Blocks) == 0 { continue } else { - return false, nil + return false } } - return true, nil + return true } func (l *List) Rollup() ([]*bpb.KV, error) { @@ -875,6 +875,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { newMinTs: maxCommitTs, } out.splitUpList() + out.cleanUpList() return out, nil } @@ -1220,3 +1221,7 @@ func splitPostingList(plist *pb.PostingList) []*pb.PostingList { return []*pb.PostingList{lowPl, highPl} } + +func (out *rollupOutput) cleanUpList() { + +} From 3dc54ad9b33942fa8e05168de87cac5415112792 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 21 Mar 2019 15:29:51 -0700 Subject: [PATCH 48/67] Fix marshalling and deletion logic. --- posting/list.go | 113 +++++++++++++++++++++++++------------------ posting/list_test.go | 2 +- posting/mvcc.go | 5 +- 3 files changed, 71 insertions(+), 49 deletions(-) diff --git a/posting/list.go b/posting/list.go index 732431a9a8c..8d905ef0ab6 100644 --- a/posting/list.go +++ b/posting/list.go @@ -698,27 +698,6 @@ func (l *List) Length(readTs, afterUid uint64) int { return l.length(readTs, afterUid) } -func (out *rollupOutput) plsAreEmpty() bool { - if len(out.newPlist.Splits) == 0 { - if out.newPlist.Pack == nil || len(out.newPlist.Pack.Blocks) == 0 { - return true - } - return false - } - - for _, startUid := range out.newPlist.Splits { - // TODO: Can be done purely from the cache? - plist := out.newSplits[startUid] - - if plist.Pack == nil || len(plist.Pack.Blocks) == 0 { - continue - } else { - return false - } - } - return true -} - func (l *List) Rollup() ([]*bpb.KV, error) { l.RLock() defer l.RUnlock() @@ -730,52 +709,67 @@ func (l *List) Rollup() ([]*bpb.KV, error) { return nil, nil } - var setEmptyBit bool - if empty, err := out.plsAreEmpty(); err != nil { - return nil, err - } else if empty { - setEmptyBit = true - } - var kvs []*bpb.KV kv := &bpb.KV{} kv.Version = out.newMinTs kv.Key = l.key - val, meta := marshalPostingList(out.newPlist, setEmptyBit) + val, meta := marshalPostingList(out.newPlist) kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) for _, startUid := range out.newPlist.Splits { - kv := &bpb.KV{} - kv.Version = out.newMinTs - kv.Key = x.GetSplitKey(l.key, startUid) plist := out.newSplits[startUid] - val, meta := marshalPostingList(plist, setEmptyBit) - kv.UserMeta = []byte{meta} - kv.Value = val + kv := out.marshalPostingListPart(l.key, startUid, plist) + kvs = append(kvs, kv) + } + + // Sort the startUids in out.splitsToDelete to make the output deterministic. + var sortedSplits []uint64 + for startUid, _ := range out.splitsToDelete { + sortedSplits = append(sortedSplits, startUid) + } + sort.Slice(sortedSplits, func(i, j int) bool { + return sortedSplits[i] < sortedSplits[j] + }) + for _, startUid := range sortedSplits { + plist := out.splitsToDelete[startUid] + kv := out.marshalPostingListPart(l.key, startUid, plist) kvs = append(kvs, kv) } return kvs, nil } -func marshalPostingList(plist *pb.PostingList, setEmptyBit bool) ([]byte, byte) { - data, err := plist.Marshal() - x.Check(err) +func (out *rollupOutput) marshalPostingListPart( + baseKey []byte, startUid uint64, plist *pb.PostingList) *bpb.KV { + kv := &bpb.KV{} + kv.Version = out.newMinTs + kv.Key = x.GetSplitKey(baseKey, startUid) + val, meta := marshalPostingList(plist) + kv.UserMeta = []byte{meta} + kv.Value = val + + return kv +} - if setEmptyBit { - return data, BitEmptyPosting +func marshalPostingList(plist *pb.PostingList) ([]byte, byte) { + if isPlistEmpty(plist) { + return nil, BitEmptyPosting } + + data, err := plist.Marshal() + x.Check(err) return data, BitCompletePosting } const blockSize int = 256 type rollupOutput struct { - newPlist *pb.PostingList - newSplits map[uint64]*pb.PostingList - newMinTs uint64 + newPlist *pb.PostingList + newSplits map[uint64]*pb.PostingList + splitsToDelete map[uint64]*pb.PostingList + newMinTs uint64 } // Merge all entries in mutation layer with commitTs <= l.commitTs into @@ -1146,10 +1140,12 @@ func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { return &part, nil } -func tooBig(plist *pb.PostingList) bool { +// isPlistTooBig returns true if the given plist should be split in two. +func isPlistTooBig(plist *pb.PostingList) bool { return plist.Size() >= maxListSize && len(plist.Pack.Blocks) > 1 } +// splitUpList checks the list and splits it in smaller parts if needed. func (out *rollupOutput) splitUpList() { var lists []*pb.PostingList @@ -1164,7 +1160,7 @@ func (out *rollupOutput) splitUpList() { var newLists []*pb.PostingList for _, list := range lists { - if tooBig(list) { + if isPlistTooBig(list) { splitList := splitPostingList(list) for _, part := range splitList { out.newSplits[part.StartUid] = part @@ -1188,6 +1184,8 @@ func (out *rollupOutput) splitUpList() { } } +// splitPostingList takes the given plist and returns two new plists, each with +// half of the blocks and postings of the original. func splitPostingList(plist *pb.PostingList) []*pb.PostingList { midBlock := len(plist.Pack.Blocks) / 2 midUid := plist.Pack.Blocks[midBlock].GetBase() @@ -1198,7 +1196,6 @@ func splitPostingList(plist *pb.PostingList) []*pb.PostingList { BlockSize: plist.Pack.BlockSize, Blocks: plist.Pack.Blocks[:midBlock], } - lowPl.CommitTs = plist.CommitTs lowPl.StartUid = plist.StartUid // Generate posting list holding the second half of the current list's postings. @@ -1207,7 +1204,6 @@ func splitPostingList(plist *pb.PostingList) []*pb.PostingList { BlockSize: plist.Pack.BlockSize, Blocks: plist.Pack.Blocks[midBlock:], } - highPl.CommitTs = plist.CommitTs highPl.StartUid = midUid // Add elements in plist.Postings to the corresponding list. @@ -1222,6 +1218,29 @@ func splitPostingList(plist *pb.PostingList) []*pb.PostingList { return []*pb.PostingList{lowPl, highPl} } +// cleanUpList marks empty splits for removal and update the split list accordingly. func (out *rollupOutput) cleanUpList() { + var splits []uint64 + for _, startUid := range out.newPlist.Splits { + plist := out.newSplits[startUid] + if isPlistEmpty(plist) { + out.splitsToDelete[startUid] = plist + delete(out.newSplits, startUid) + } else { + splits = append(splits, startUid) + } + } + out.newPlist.Splits = splits +} +// isPlistEmpty returns true if the given plist is empty. Plists with splits are +// considered non-empty. +func isPlistEmpty(plist *pb.PostingList) bool { + if len(plist.Splits) > 0 { + return false + } + if plist.Pack == nil || len(plist.Pack.Blocks) == 0 { + return true + } + return false } diff --git a/posting/list_test.go b/posting/list_test.go index a66b9bc516e..2dff817bdd4 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -1078,7 +1078,7 @@ func TestMultiPartListDelete(t *testing.T) { kvs, err := ol.Rollup() require.NoError(t, err) - require.Equal(t, len(kvs), len(ol.plist.Splits)+1) + require.Equal(t, len(kvs), 1) for _, kv := range kvs { require.Equal(t, []byte{BitEmptyPosting}, kv.UserMeta) diff --git a/posting/mvcc.go b/posting/mvcc.go index 1efb53fe059..62908e9985a 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -200,7 +200,10 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { } switch item.UserMeta() { - case BitEmptyPosting, BitCompletePosting: + case BitEmptyPosting: + l.minTs = item.Version() + return l, nil + case BitCompletePosting: if err := unmarshalOrCopy(l.plist, item); err != nil { return nil, err } From 65538a0fa0f28d53ab96cad2120c240876584e7d Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 21 Mar 2019 16:18:11 -0700 Subject: [PATCH 49/67] Remove start_uid field from proto. --- posting/list.go | 53 +++--- protos/pb.proto | 1 - protos/pb/pb.pb.go | 451 +++++++++++++++++++++------------------------ 3 files changed, 237 insertions(+), 268 deletions(-) diff --git a/posting/list.go b/posting/list.go index 8d905ef0ab6..fe91ed39d2a 100644 --- a/posting/list.go +++ b/posting/list.go @@ -794,6 +794,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { var enc codec.Encoder var endUid uint64 var splitIdx int + var startUid uint64 // Method to properly initialize all the variables described above. init := func() { @@ -809,16 +810,14 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { // Otherwise, load the corresponding part and set endUid to correctly // detect the end of the list. - startUid := l.plist.Splits[splitIdx] + startUid = l.plist.Splits[splitIdx] if splitIdx+1 == len(l.plist.Splits) { endUid = math.MaxUint64 } else { endUid = l.plist.Splits[splitIdx+1] - 1 } - plist = &pb.PostingList{ - StartUid: startUid, - } + plist = &pb.PostingList{} } init() @@ -827,7 +826,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { final.Pack = enc.Done() plist.Pack = final.Pack plist.Postings = final.Postings - newSplits[plist.StartUid] = plist + newSplits[startUid] = plist splitIdx++ init() @@ -845,7 +844,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { plist.Pack = final.Pack plist.Postings = final.Postings if len(l.plist.Splits) > 0 { - newSplits[plist.StartUid] = plist + newSplits[startUid] = plist } maxCommitTs := l.minTs @@ -1149,44 +1148,52 @@ func isPlistTooBig(plist *pb.PostingList) bool { func (out *rollupOutput) splitUpList() { var lists []*pb.PostingList + existingSplits := false if len(out.newPlist.Splits) == 0 { lists = append(lists, out.newPlist) } else { + existingSplits = true for _, startUid := range out.newPlist.Splits { part := out.newSplits[startUid] lists = append(lists, part) } } - var newLists []*pb.PostingList - for _, list := range lists { + var newSplits []uint64 + var newPls []*pb.PostingList + for i, list := range lists { + var startUid uint64 + if existingSplits { + startUid = out.newPlist.Splits[i] + } + if isPlistTooBig(list) { - splitList := splitPostingList(list) - for _, part := range splitList { - out.newSplits[part.StartUid] = part - newLists = append(newLists, part) + startUids, pls := splitPostingList(startUid, list) + for i, startUid := range startUids { + pl := pls[i] + out.newSplits[startUid] = pl + newPls = append(newPls, pl) + newSplits = append(newSplits, startUid) } } else { - newLists = append(newLists, list) + newPls = append(newPls, list) + newSplits = append(newSplits, startUid) } } - if len(newLists) == 1 || len(newLists) == len(out.newPlist.Splits) { + if len(newPls) == 1 || len(newPls) == len(out.newPlist.Splits) { return } - var splits []uint64 - for _, list := range newLists { - splits = append(splits, list.StartUid) - } out.newPlist = &pb.PostingList{ - Splits: splits, + Splits: newSplits, } } // splitPostingList takes the given plist and returns two new plists, each with -// half of the blocks and postings of the original. -func splitPostingList(plist *pb.PostingList) []*pb.PostingList { +// half of the blocks and postings of the original as well as the new startUids +// for each of the new parts. +func splitPostingList(startUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingList) { midBlock := len(plist.Pack.Blocks) / 2 midUid := plist.Pack.Blocks[midBlock].GetBase() @@ -1196,7 +1203,6 @@ func splitPostingList(plist *pb.PostingList) []*pb.PostingList { BlockSize: plist.Pack.BlockSize, Blocks: plist.Pack.Blocks[:midBlock], } - lowPl.StartUid = plist.StartUid // Generate posting list holding the second half of the current list's postings. highPl := new(pb.PostingList) @@ -1204,7 +1210,6 @@ func splitPostingList(plist *pb.PostingList) []*pb.PostingList { BlockSize: plist.Pack.BlockSize, Blocks: plist.Pack.Blocks[midBlock:], } - highPl.StartUid = midUid // Add elements in plist.Postings to the corresponding list. for _, posting := range plist.Postings { @@ -1215,7 +1220,7 @@ func splitPostingList(plist *pb.PostingList) []*pb.PostingList { } } - return []*pb.PostingList{lowPl, highPl} + return []uint64{startUid, midUid}, []*pb.PostingList{lowPl, highPl} } // cleanUpList marks empty splits for removal and update the split list accordingly. diff --git a/protos/pb.proto b/protos/pb.proto index 3aee90e7c74..1cee3bae0c0 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -282,7 +282,6 @@ message PostingList { uint64 commit_ts = 3; // More inclination towards smaller values. repeated uint64 splits = 4; - uint64 start_uid = 5; } message FacetParam { diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index 84b9a9f1621..ca4d70fc242 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -2125,7 +2125,6 @@ type PostingList struct { Postings []*Posting `protobuf:"bytes,2,rep,name=postings,proto3" json:"postings,omitempty"` CommitTs uint64 `protobuf:"varint,3,opt,name=commit_ts,json=commitTs,proto3" json:"commit_ts,omitempty"` Splits []uint64 `protobuf:"varint,4,rep,packed,name=splits,proto3" json:"splits,omitempty"` - StartUid uint64 `protobuf:"varint,5,opt,name=start_uid,json=startUid,proto3" json:"start_uid,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -2192,13 +2191,6 @@ func (m *PostingList) GetSplits() []uint64 { return nil } -func (m *PostingList) GetStartUid() uint64 { - if m != nil { - return m.StartUid - } - return 0 -} - type FacetParam struct { Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` Alias string `protobuf:"bytes,2,opt,name=alias,proto3" json:"alias,omitempty"` @@ -3669,218 +3661,218 @@ func init() { func init() { proto.RegisterFile("pb.proto", fileDescriptor_f80abaa17e25ccc8) } var fileDescriptor_f80abaa17e25ccc8 = []byte{ - // 3374 bytes of a gzipped FileDescriptorProto + // 3362 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x3a, 0xcb, 0x6e, 0x23, 0x57, 0x76, 0x5d, 0x7c, 0x14, 0xab, 0x0e, 0x29, 0x35, 0x7d, 0xed, 0x69, 0xd3, 0x9a, 0x99, 0x6e, 0xb9, 0xfc, 0x68, 0xb9, 0x3d, 0x56, 0xb7, 0x65, 0x27, 0x19, 0x4f, 0x90, 0x85, 0x5a, 0x62, 0x77, 0x64, 0xeb, 0x95, 0x4b, 0xaa, 0x27, 0x31, 0x82, 0x10, 0x57, 0x55, 0x57, 0x54, 0x8d, 0x8a, 0x55, 0x95, - 0xba, 0x45, 0x81, 0xf2, 0x2e, 0x8b, 0x04, 0xc9, 0x1f, 0xcc, 0x22, 0xc8, 0x22, 0x40, 0x16, 0xc9, - 0x26, 0xdb, 0x59, 0x07, 0x08, 0x90, 0x65, 0x90, 0x2f, 0x08, 0x9c, 0x7c, 0x43, 0x80, 0xec, 0x82, - 0x73, 0xee, 0xad, 0x07, 0xd9, 0xea, 0xb6, 0x1d, 0x60, 0x56, 0xaa, 0xf3, 0xb8, 0xaf, 0xf3, 0x3e, - 0x87, 0x02, 0x27, 0x3d, 0xdf, 0x4e, 0xb3, 0x24, 0x4f, 0x58, 0x23, 0x3d, 0xdf, 0x70, 0x45, 0x1a, - 0x6a, 0x70, 0xe3, 0xe1, 0x34, 0xcc, 0x2f, 0xe7, 0xe7, 0xdb, 0x7e, 0x32, 0x7b, 0x1c, 0x4c, 0x33, - 0x91, 0x5e, 0x7e, 0x12, 0x26, 0x8f, 0xcf, 0x45, 0x30, 0x95, 0xd9, 0xe3, 0xf4, 0xfc, 0x71, 0xb1, - 0xce, 0xdb, 0x80, 0xd6, 0x61, 0xa8, 0x72, 0xc6, 0xa0, 0x35, 0x0f, 0x03, 0x35, 0xb0, 0x36, 0x9b, - 0x5b, 0x36, 0xa7, 0x6f, 0xef, 0x08, 0xdc, 0xb1, 0x50, 0x57, 0x2f, 0x44, 0x34, 0x97, 0xac, 0x0f, - 0xcd, 0x6b, 0x11, 0x0d, 0xac, 0x4d, 0x6b, 0xab, 0xc7, 0xf1, 0x93, 0x6d, 0x83, 0x73, 0x2d, 0xa2, - 0x49, 0x7e, 0x93, 0xca, 0x41, 0x63, 0xd3, 0xda, 0x5a, 0xdf, 0x79, 0x73, 0x3b, 0x3d, 0xdf, 0x3e, - 0x4d, 0x54, 0x1e, 0xc6, 0xd3, 0xed, 0x17, 0x22, 0x1a, 0xdf, 0xa4, 0x92, 0x77, 0xae, 0xf5, 0x87, - 0x77, 0x02, 0xdd, 0x51, 0xe6, 0x3f, 0x9b, 0xc7, 0x7e, 0x1e, 0x26, 0x31, 0x9e, 0x18, 0x8b, 0x99, - 0xa4, 0x1d, 0x5d, 0x4e, 0xdf, 0x88, 0x13, 0xd9, 0x54, 0x0d, 0x9a, 0x9b, 0x4d, 0xc4, 0xe1, 0x37, - 0x1b, 0x40, 0x27, 0x54, 0x7b, 0xc9, 0x3c, 0xce, 0x07, 0xad, 0x4d, 0x6b, 0xcb, 0xe1, 0x05, 0xe8, - 0xfd, 0x4d, 0x13, 0xda, 0x7f, 0x34, 0x97, 0xd9, 0x0d, 0xad, 0xcb, 0xf3, 0xac, 0xd8, 0x0b, 0xbf, - 0xd9, 0x5b, 0xd0, 0x8e, 0x44, 0x3c, 0x55, 0x83, 0x06, 0x6d, 0xa6, 0x01, 0xf6, 0x63, 0x70, 0xc5, - 0x45, 0x2e, 0xb3, 0xc9, 0x3c, 0x0c, 0x06, 0xcd, 0x4d, 0x6b, 0xcb, 0xe6, 0x0e, 0x21, 0xce, 0xc2, - 0x80, 0xbd, 0x03, 0x4e, 0x90, 0x4c, 0xfc, 0xfa, 0x59, 0x41, 0x42, 0x67, 0xb1, 0xf7, 0xc0, 0x99, - 0x87, 0xc1, 0x24, 0x0a, 0x55, 0x3e, 0x68, 0x6f, 0x5a, 0x5b, 0xdd, 0x1d, 0x07, 0x1f, 0x8b, 0xb2, - 0xe3, 0x9d, 0x79, 0x18, 0x90, 0x10, 0x1f, 0x81, 0xa3, 0x32, 0x7f, 0x72, 0x31, 0x8f, 0xfd, 0x81, - 0x4d, 0x4c, 0x77, 0x91, 0xa9, 0xf6, 0x6a, 0xde, 0x51, 0x1a, 0xc0, 0x67, 0x65, 0xf2, 0x5a, 0x66, - 0x4a, 0x0e, 0x3a, 0xfa, 0x28, 0x03, 0xb2, 0x27, 0xd0, 0xbd, 0x10, 0xbe, 0xcc, 0x27, 0xa9, 0xc8, - 0xc4, 0x6c, 0xe0, 0x54, 0x1b, 0x3d, 0x43, 0xf4, 0x29, 0x62, 0x15, 0x87, 0x8b, 0x12, 0x60, 0x9f, - 0xc1, 0x1a, 0x41, 0x6a, 0x72, 0x11, 0x46, 0xb9, 0xcc, 0x06, 0x2e, 0xad, 0x59, 0xa7, 0x35, 0x84, - 0x19, 0x67, 0x52, 0xf2, 0x9e, 0x66, 0xd2, 0x18, 0xf6, 0x53, 0x00, 0xb9, 0x48, 0x45, 0x1c, 0x4c, - 0x44, 0x14, 0x0d, 0x80, 0xee, 0xe0, 0x6a, 0xcc, 0x6e, 0x14, 0xb1, 0xb7, 0xf1, 0x7e, 0x22, 0x98, - 0xe4, 0x6a, 0xb0, 0xb6, 0x69, 0x6d, 0xb5, 0xb8, 0x8d, 0xe0, 0x58, 0xa1, 0x5c, 0x7d, 0xe1, 0x5f, - 0xca, 0xc1, 0xfa, 0xa6, 0xb5, 0xd5, 0xe6, 0x1a, 0xf0, 0x76, 0xc0, 0x25, 0x3b, 0x21, 0x39, 0x7c, - 0x00, 0xf6, 0x35, 0x02, 0xda, 0x9c, 0xba, 0x3b, 0x6b, 0x78, 0x91, 0xd2, 0x94, 0xb8, 0x21, 0x7a, - 0xf7, 0xc1, 0x39, 0x14, 0xf1, 0xb4, 0xb0, 0x3f, 0x54, 0x10, 0x2d, 0x70, 0x39, 0x7d, 0x7b, 0xbf, - 0x6e, 0x80, 0xcd, 0xa5, 0x9a, 0x47, 0x39, 0x7b, 0x08, 0x80, 0xe2, 0x9f, 0x89, 0x3c, 0x0b, 0x17, - 0x66, 0xd7, 0x4a, 0x01, 0xee, 0x3c, 0x0c, 0x8e, 0x88, 0xc4, 0x9e, 0x40, 0x8f, 0x76, 0x2f, 0x58, - 0x1b, 0xd5, 0x05, 0xca, 0xfb, 0xf1, 0x2e, 0xb1, 0x98, 0x15, 0xf7, 0xc0, 0x26, 0x8d, 0x6b, 0xab, - 0x5b, 0xe3, 0x06, 0x62, 0x1f, 0xc0, 0x7a, 0x18, 0xe7, 0xa8, 0x11, 0x3f, 0x9f, 0x04, 0x52, 0x15, - 0x26, 0xb1, 0x56, 0x62, 0xf7, 0xa5, 0xca, 0xd9, 0xa7, 0xa0, 0xc5, 0x5a, 0x1c, 0xd8, 0xa6, 0x03, - 0xd7, 0x4b, 0x75, 0x29, 0x7d, 0x22, 0xf1, 0x98, 0x13, 0x3f, 0x81, 0x2e, 0xbe, 0xaf, 0x58, 0x61, - 0xd3, 0x8a, 0x1e, 0xbd, 0xc6, 0x88, 0x83, 0x03, 0x32, 0x18, 0x76, 0x14, 0x0d, 0x9a, 0x9d, 0x36, - 0x13, 0xfa, 0xf6, 0x86, 0xd0, 0x3e, 0xc9, 0x02, 0x99, 0xdd, 0x6a, 0xf9, 0x0c, 0x5a, 0x81, 0x54, - 0x3e, 0x39, 0xa5, 0xc3, 0xe9, 0xbb, 0xf2, 0x86, 0x66, 0xcd, 0x1b, 0xbc, 0xbf, 0xb3, 0xa0, 0x3b, - 0x4a, 0xb2, 0xfc, 0x48, 0x2a, 0x25, 0xa6, 0x92, 0x3d, 0x80, 0x76, 0x82, 0xdb, 0x1a, 0x09, 0xbb, - 0x78, 0x27, 0x3a, 0x87, 0x6b, 0xfc, 0x8a, 0x1e, 0x1a, 0xaf, 0xd6, 0x03, 0x5a, 0x09, 0xf9, 0x51, - 0xd3, 0x58, 0x09, 0x79, 0xd1, 0x3d, 0xb0, 0x93, 0x8b, 0x0b, 0x25, 0xb5, 0x2c, 0xdb, 0xdc, 0x40, - 0xaf, 0x34, 0x36, 0xef, 0x77, 0x00, 0xf0, 0x7e, 0x3f, 0xd0, 0x0a, 0xbc, 0x4b, 0xe8, 0x72, 0x71, - 0x91, 0xef, 0x25, 0x71, 0x2e, 0x17, 0x39, 0x5b, 0x87, 0x46, 0x18, 0x90, 0x88, 0x6c, 0xde, 0x08, - 0x03, 0xbc, 0xdc, 0x34, 0x4b, 0xe6, 0x29, 0x49, 0x68, 0x8d, 0x6b, 0x80, 0x44, 0x19, 0x04, 0x19, - 0xdd, 0x18, 0x45, 0x19, 0x04, 0x19, 0x7b, 0x00, 0x5d, 0x15, 0x8b, 0x54, 0x5d, 0x26, 0x39, 0x5e, - 0xae, 0x45, 0x97, 0x83, 0x02, 0x35, 0x56, 0xde, 0xbf, 0x5a, 0x60, 0x1f, 0xc9, 0xd9, 0xb9, 0xcc, - 0x5e, 0x3a, 0xe5, 0x1d, 0x70, 0x68, 0xe3, 0x49, 0x18, 0x98, 0x83, 0x3a, 0x04, 0x1f, 0x04, 0xb7, - 0x1e, 0x75, 0x0f, 0xec, 0x48, 0x0a, 0x14, 0xbe, 0xb6, 0x33, 0x03, 0xa1, 0x6c, 0xc4, 0x6c, 0x12, - 0x48, 0x11, 0x50, 0xe0, 0x71, 0xb8, 0x2d, 0x66, 0xfb, 0x52, 0x04, 0x78, 0xb7, 0x48, 0xa8, 0x7c, - 0x32, 0x4f, 0x03, 0x91, 0x4b, 0x0a, 0x38, 0x2d, 0x34, 0x1c, 0x95, 0x9f, 0x11, 0x86, 0x3d, 0x82, - 0x37, 0xfc, 0x68, 0xae, 0x30, 0xda, 0x85, 0xf1, 0x45, 0x32, 0x49, 0xe2, 0xe8, 0x86, 0xe4, 0xeb, - 0xf0, 0xbb, 0x86, 0x70, 0x10, 0x5f, 0x24, 0x27, 0x71, 0x74, 0xe3, 0xfd, 0xa6, 0x01, 0xed, 0xe7, - 0x24, 0x86, 0x27, 0xd0, 0x99, 0xd1, 0x83, 0x0a, 0xef, 0xbd, 0x87, 0x12, 0x26, 0xda, 0xb6, 0x7e, - 0xa9, 0x1a, 0xc6, 0x79, 0x76, 0xc3, 0x0b, 0x36, 0x5c, 0x91, 0x8b, 0xf3, 0x48, 0xe6, 0xca, 0x58, - 0x44, 0x6d, 0xc5, 0x58, 0x13, 0xcc, 0x0a, 0xc3, 0xb6, 0x2a, 0xd6, 0xe6, 0xaa, 0x58, 0xd9, 0x06, - 0x38, 0xfe, 0xa5, 0xf4, 0xaf, 0xd4, 0x7c, 0x66, 0x84, 0x5e, 0xc2, 0x1b, 0xcf, 0xa0, 0x57, 0xbf, - 0x07, 0x66, 0xa6, 0x2b, 0x79, 0x43, 0x82, 0x6f, 0x71, 0xfc, 0x64, 0x9b, 0xd0, 0x26, 0x0f, 0x27, - 0xb1, 0x77, 0x77, 0x00, 0xaf, 0xa3, 0x97, 0x70, 0x4d, 0xf8, 0x45, 0xe3, 0xe7, 0x16, 0xee, 0x53, - 0xbf, 0x5d, 0x7d, 0x1f, 0xf7, 0xd5, 0xfb, 0xe8, 0x25, 0xb5, 0x7d, 0xbc, 0xff, 0x6d, 0x40, 0xef, - 0x6b, 0x99, 0x25, 0xa7, 0x59, 0x92, 0x26, 0x4a, 0x44, 0x6c, 0x77, 0xf9, 0x75, 0x5a, 0x8a, 0x9b, - 0xb8, 0xb8, 0xce, 0xb6, 0x3d, 0x2a, 0x9f, 0xab, 0xa5, 0x53, 0x7f, 0xbf, 0x07, 0xb6, 0x96, 0xee, - 0x2d, 0x4f, 0x30, 0x14, 0xe4, 0xd1, 0xf2, 0x24, 0xf9, 0x2d, 0x5f, 0xcf, 0x50, 0xd8, 0x7d, 0x80, - 0x99, 0x58, 0x1c, 0x4a, 0xa1, 0xe4, 0x41, 0x50, 0x98, 0x6f, 0x85, 0x41, 0x39, 0xcf, 0xc4, 0x62, - 0xbc, 0x88, 0xc7, 0x8a, 0xac, 0xab, 0xc5, 0x4b, 0x98, 0xfd, 0x04, 0xdc, 0x99, 0x58, 0xa0, 0x1f, - 0x1d, 0x04, 0xc6, 0xba, 0x2a, 0x04, 0x7b, 0x17, 0x9a, 0xf9, 0x22, 0xa6, 0xa0, 0x84, 0xd9, 0x09, - 0x4b, 0x8f, 0xf1, 0x22, 0x36, 0x1e, 0xc7, 0x91, 0x56, 0x08, 0xd4, 0xa9, 0x04, 0xda, 0x87, 0xa6, - 0x1f, 0x06, 0x94, 0x9e, 0x5c, 0x8e, 0x9f, 0x1b, 0x7f, 0x00, 0x77, 0x57, 0xe4, 0x50, 0xd7, 0xc3, - 0x9a, 0x5e, 0xf6, 0x56, 0x5d, 0x0f, 0xad, 0xba, 0xec, 0x7f, 0xd3, 0x84, 0xbb, 0xc6, 0x18, 0x2e, - 0xc3, 0x74, 0x94, 0xa3, 0xd9, 0x0f, 0xa0, 0x43, 0xd1, 0x46, 0x66, 0xc6, 0x26, 0x0a, 0x90, 0xfd, - 0x1e, 0xd8, 0xe4, 0x81, 0x85, 0x9d, 0x3e, 0xa8, 0xa4, 0x5a, 0x2e, 0xd7, 0x76, 0x6b, 0x54, 0x62, - 0xd8, 0xd9, 0xe7, 0xd0, 0xfe, 0x46, 0x66, 0x89, 0x8e, 0x9e, 0xdd, 0x9d, 0xfb, 0xb7, 0xad, 0x43, - 0xdd, 0x9a, 0x65, 0x9a, 0xf9, 0xb7, 0x28, 0xfc, 0xf7, 0x31, 0x5e, 0xce, 0x92, 0x6b, 0x19, 0x0c, - 0x3a, 0x74, 0xa3, 0xba, 0x7d, 0x14, 0xa4, 0x42, 0xda, 0x4e, 0x25, 0xed, 0x7d, 0xe8, 0xd6, 0x9e, - 0x77, 0x8b, 0xa4, 0x1f, 0x2c, 0x5b, 0xbc, 0x5b, 0x3a, 0x72, 0xdd, 0x71, 0xf6, 0x01, 0xaa, 0xc7, - 0xfe, 0x7f, 0xdd, 0xcf, 0xfb, 0x0b, 0x0b, 0xee, 0xee, 0x25, 0x71, 0x2c, 0xa9, 0x30, 0xd2, 0xaa, - 0xab, 0xcc, 0xde, 0x7a, 0xa5, 0xd9, 0x7f, 0x04, 0x6d, 0x85, 0xcc, 0x66, 0xf7, 0x37, 0x6f, 0xd1, - 0x05, 0xd7, 0x1c, 0x18, 0x66, 0x66, 0x62, 0x31, 0x49, 0x65, 0x1c, 0x84, 0xf1, 0xb4, 0x08, 0x33, - 0x33, 0xb1, 0x38, 0xd5, 0x18, 0xef, 0xef, 0x2d, 0xb0, 0xb5, 0xc7, 0x2c, 0x45, 0x6b, 0x6b, 0x39, - 0x5a, 0xff, 0x04, 0xdc, 0x34, 0x93, 0x41, 0xe8, 0x17, 0xa7, 0xba, 0xbc, 0x42, 0xa0, 0x71, 0x5e, - 0x24, 0x99, 0x2f, 0x69, 0x7b, 0x87, 0x6b, 0x00, 0xb1, 0x2a, 0x15, 0xbe, 0x2e, 0xee, 0x9a, 0x5c, - 0x03, 0x18, 0xe3, 0xb5, 0x72, 0x48, 0x29, 0x0e, 0x37, 0x10, 0x56, 0xa5, 0x94, 0xff, 0x28, 0x42, - 0xbb, 0x44, 0x72, 0x10, 0x41, 0xa1, 0xf9, 0x9f, 0x1a, 0xd0, 0xdb, 0x0f, 0x33, 0xe9, 0xe7, 0x32, - 0x18, 0x06, 0x53, 0xda, 0x45, 0xc6, 0x79, 0x98, 0xdf, 0x98, 0x64, 0x63, 0xa0, 0xb2, 0x16, 0x68, - 0x2c, 0x57, 0xc1, 0x5a, 0x17, 0x4d, 0x2a, 0xdc, 0x35, 0xc0, 0x76, 0x00, 0x74, 0x95, 0x44, 0xc5, - 0x7b, 0xeb, 0xd5, 0xc5, 0xbb, 0x4b, 0x6c, 0xf8, 0x89, 0x02, 0xd2, 0x6b, 0x42, 0x9d, 0x88, 0x6c, - 0xaa, 0xec, 0xe7, 0x68, 0xc8, 0x54, 0x5c, 0x9c, 0xcb, 0x88, 0x0c, 0x95, 0x8a, 0x8b, 0x73, 0x19, - 0x95, 0x25, 0x5d, 0x47, 0x5f, 0x07, 0xbf, 0xd9, 0x7b, 0xd0, 0x48, 0x52, 0x7a, 0xbc, 0x39, 0xb0, - 0xfe, 0xb0, 0xed, 0x93, 0x94, 0x37, 0x92, 0x14, 0xad, 0x40, 0x57, 0xaa, 0x03, 0xd7, 0x18, 0x37, - 0x46, 0x17, 0xaa, 0xa6, 0xb8, 0xa1, 0x78, 0xf7, 0xa0, 0x71, 0x92, 0xb2, 0x0e, 0x34, 0x47, 0xc3, - 0x71, 0xff, 0x0e, 0x7e, 0xec, 0x0f, 0x0f, 0xfb, 0x96, 0xf7, 0xd7, 0x0d, 0x70, 0x8f, 0xe6, 0xb9, - 0x40, 0x9b, 0x52, 0xaf, 0x53, 0xea, 0x3b, 0xe0, 0xa8, 0x5c, 0x64, 0x14, 0xa1, 0x75, 0x58, 0xe9, - 0x10, 0x3c, 0x56, 0xec, 0x43, 0x68, 0xcb, 0x60, 0x2a, 0x0b, 0x6f, 0xef, 0xaf, 0xde, 0x93, 0x6b, - 0x32, 0xdb, 0x02, 0x5b, 0xf9, 0x97, 0x72, 0x26, 0x06, 0xad, 0x8a, 0x71, 0x44, 0x18, 0x9d, 0x81, - 0xb9, 0xa1, 0x53, 0x63, 0x91, 0x25, 0x29, 0x55, 0xda, 0x6d, 0xd3, 0x58, 0x64, 0x49, 0x8a, 0x75, - 0xf6, 0x0e, 0xfc, 0x28, 0x9c, 0xc6, 0x49, 0x26, 0x27, 0x61, 0x1c, 0xc8, 0xc5, 0xc4, 0x4f, 0xe2, - 0x8b, 0x28, 0xf4, 0x73, 0x92, 0xa5, 0xc3, 0xdf, 0xd4, 0xc4, 0x03, 0xa4, 0xed, 0x19, 0x12, 0x7b, - 0x1f, 0xda, 0xa8, 0x38, 0x65, 0x9c, 0x9f, 0x8a, 0x4d, 0xd4, 0x91, 0x39, 0x55, 0x13, 0xbd, 0x05, - 0x38, 0x45, 0x68, 0x65, 0x1f, 0x61, 0x4c, 0xa4, 0xd0, 0x6c, 0x3c, 0x8b, 0xfa, 0x89, 0x5a, 0x8d, - 0xc4, 0x0b, 0x3a, 0x2a, 0x93, 0x6e, 0x52, 0x04, 0x5b, 0x02, 0xea, 0x15, 0x5a, 0x73, 0xa9, 0x1d, - 0xc0, 0x62, 0x33, 0x89, 0xa5, 0x29, 0x5a, 0xe8, 0xdb, 0xfb, 0xdb, 0x06, 0x38, 0x65, 0x36, 0xfc, - 0x18, 0xdc, 0x59, 0xa1, 0x10, 0xe3, 0xb3, 0x54, 0x8e, 0x97, 0x5a, 0xe2, 0x15, 0x9d, 0xdd, 0x83, - 0xc6, 0xd5, 0xb5, 0x11, 0xa7, 0x8d, 0x5c, 0x5f, 0xbd, 0xe0, 0x8d, 0xab, 0xeb, 0xca, 0xe9, 0xdb, - 0xdf, 0xe9, 0xf4, 0x0f, 0xe1, 0xae, 0x1f, 0x49, 0x11, 0x4f, 0x2a, 0x9f, 0xd5, 0x66, 0xb9, 0x4e, - 0xe8, 0xd3, 0xd2, 0x71, 0x4d, 0xe0, 0xea, 0x54, 0xe9, 0xe9, 0x03, 0x68, 0x07, 0x32, 0xca, 0x45, - 0xbd, 0xe7, 0x3a, 0xc9, 0x84, 0x1f, 0xc9, 0x7d, 0x44, 0x73, 0x4d, 0x65, 0x5b, 0xe0, 0x14, 0xa9, - 0xda, 0x74, 0x5a, 0x54, 0xbc, 0x17, 0xc2, 0xe6, 0x25, 0xb5, 0x92, 0x25, 0xd4, 0x64, 0xe9, 0x7d, - 0x0a, 0xcd, 0xaf, 0x5e, 0x8c, 0xcc, 0x5b, 0xad, 0x97, 0xde, 0x5a, 0x48, 0xb4, 0x51, 0x93, 0xe8, - 0xff, 0x34, 0xa1, 0x63, 0x7c, 0x13, 0xef, 0x3d, 0x2f, 0x0b, 0x4d, 0xfc, 0x5c, 0xce, 0x8f, 0xa5, - 0x93, 0xd7, 0xfb, 0xf3, 0xe6, 0x77, 0xf7, 0xe7, 0xec, 0x17, 0xd0, 0x4b, 0x35, 0xad, 0x1e, 0x16, - 0xde, 0xae, 0xaf, 0x31, 0x7f, 0x69, 0x5d, 0x37, 0xad, 0x00, 0x34, 0x70, 0x6a, 0x69, 0x72, 0x31, - 0x25, 0x15, 0xf5, 0x78, 0x07, 0xe1, 0xb1, 0x98, 0xbe, 0x22, 0x38, 0x7c, 0x0f, 0x1f, 0xc7, 0x82, - 0x3a, 0x49, 0x07, 0x3d, 0xf2, 0x5b, 0x8c, 0x0b, 0x75, 0x97, 0x5d, 0x5b, 0x76, 0xd9, 0x1f, 0x83, - 0xeb, 0x27, 0xb3, 0x59, 0x48, 0xb4, 0x75, 0x53, 0x30, 0x12, 0x62, 0xac, 0xbc, 0xbf, 0xb2, 0xa0, - 0x63, 0x5e, 0xcb, 0xba, 0xd0, 0xd9, 0x1f, 0x3e, 0xdb, 0x3d, 0x3b, 0xc4, 0xa8, 0x01, 0x60, 0x3f, - 0x3d, 0x38, 0xde, 0xe5, 0x7f, 0xd2, 0xb7, 0x30, 0x82, 0x1c, 0x1c, 0x8f, 0xfb, 0x0d, 0xe6, 0x42, - 0xfb, 0xd9, 0xe1, 0xc9, 0xee, 0xb8, 0xdf, 0x64, 0x0e, 0xb4, 0x9e, 0x9e, 0x9c, 0x1c, 0xf6, 0x5b, - 0xac, 0x07, 0xce, 0xfe, 0xee, 0x78, 0x38, 0x3e, 0x38, 0x1a, 0xf6, 0xdb, 0xc8, 0xfb, 0x7c, 0x78, - 0xd2, 0xb7, 0xf1, 0xe3, 0xec, 0x60, 0xbf, 0xdf, 0x41, 0xfa, 0xe9, 0xee, 0x68, 0xf4, 0xcb, 0x13, - 0xbe, 0xdf, 0x77, 0x70, 0xdf, 0xd1, 0x98, 0x1f, 0x1c, 0x3f, 0xef, 0xbb, 0xf8, 0x7d, 0xf2, 0xf4, - 0xcb, 0xe1, 0xde, 0xb8, 0x0f, 0xde, 0xa7, 0xd0, 0xad, 0x49, 0x10, 0x57, 0xf3, 0xe1, 0xb3, 0xfe, - 0x1d, 0x3c, 0xf2, 0xc5, 0xee, 0xe1, 0xd9, 0xb0, 0x6f, 0xb1, 0x75, 0x00, 0xfa, 0x9c, 0x1c, 0xee, - 0x1e, 0x3f, 0xef, 0x37, 0xbc, 0xdf, 0x05, 0xe7, 0x2c, 0x0c, 0x9e, 0x46, 0x89, 0x7f, 0x85, 0x86, - 0x71, 0x2e, 0x94, 0x34, 0xa9, 0x96, 0xbe, 0x31, 0x17, 0x90, 0x51, 0x2a, 0xa3, 0x7b, 0x03, 0x79, - 0xc7, 0xd0, 0x39, 0x0b, 0x83, 0x53, 0xe1, 0x5f, 0x61, 0xa3, 0x7f, 0x8e, 0xeb, 0x27, 0x2a, 0xfc, - 0x46, 0x9a, 0x30, 0xe8, 0x12, 0x66, 0x14, 0x7e, 0x23, 0xd9, 0xfb, 0x60, 0x13, 0x50, 0x14, 0x45, - 0x64, 0xcb, 0xc5, 0x99, 0xdc, 0xd0, 0xbc, 0x7f, 0xb4, 0xca, 0xbb, 0x53, 0xbf, 0xfe, 0x00, 0x5a, - 0xa9, 0xf0, 0xaf, 0x4c, 0x34, 0xe9, 0x9a, 0x35, 0x78, 0x1e, 0x27, 0x02, 0x7b, 0x08, 0x8e, 0x31, - 0x90, 0x62, 0xe3, 0x6e, 0xcd, 0x92, 0x78, 0x49, 0x5c, 0x56, 0x5d, 0x73, 0x59, 0x75, 0xf8, 0x3c, - 0x95, 0x46, 0x21, 0xb5, 0x5e, 0x4d, 0x8c, 0x3a, 0x1a, 0xc2, 0x45, 0xda, 0x14, 0xe6, 0x26, 0x1b, - 0xb5, 0xb8, 0xb6, 0x8d, 0xb3, 0x30, 0xf0, 0x3e, 0x07, 0xa8, 0x26, 0x25, 0xb7, 0x94, 0xf5, 0x6f, - 0x41, 0x5b, 0x44, 0xa1, 0x11, 0x99, 0xcb, 0x35, 0xe0, 0x1d, 0x43, 0xb7, 0x36, 0x5f, 0x41, 0x63, - 0x13, 0x51, 0x34, 0xb9, 0x92, 0x37, 0x8a, 0xd6, 0x3a, 0xbc, 0x23, 0xa2, 0xe8, 0x2b, 0x79, 0xa3, - 0x30, 0xfc, 0xea, 0xd1, 0x4c, 0x63, 0xa5, 0xd7, 0xa7, 0xa5, 0x5c, 0x13, 0xbd, 0x9f, 0x81, 0xad, - 0x07, 0x00, 0x35, 0x5b, 0xb7, 0x5e, 0x99, 0xcf, 0xbe, 0x30, 0x77, 0xa6, 0x71, 0x01, 0xfb, 0xd8, - 0x8c, 0x80, 0x94, 0x1e, 0x38, 0x59, 0x55, 0x8d, 0xa7, 0x99, 0xcc, 0xf4, 0x87, 0x98, 0xbd, 0x7d, - 0x70, 0x5e, 0x3b, 0x54, 0x33, 0x02, 0x68, 0x54, 0x02, 0xb8, 0x65, 0xcc, 0xe6, 0xfd, 0x0a, 0xa0, - 0x1a, 0x15, 0x19, 0xd7, 0xd3, 0xbb, 0xa0, 0xeb, 0x3d, 0xc2, 0x7e, 0x2c, 0x8c, 0x82, 0x4c, 0xc6, - 0x4b, 0xaf, 0xae, 0x86, 0x4b, 0x25, 0x9d, 0x6d, 0x42, 0x8b, 0x26, 0x60, 0xcd, 0x2a, 0x34, 0x96, - 0xe3, 0x2f, 0xa2, 0x78, 0x0b, 0x58, 0xd3, 0x69, 0x92, 0xcb, 0x3f, 0x9f, 0x4b, 0xf5, 0xda, 0xe2, - 0xeb, 0x3e, 0x40, 0x19, 0xc8, 0x8b, 0x59, 0x5e, 0x0d, 0x83, 0x16, 0x72, 0x11, 0xca, 0x28, 0x28, - 0x5e, 0x63, 0x20, 0x54, 0xb2, 0xce, 0x91, 0x2d, 0x3d, 0xf0, 0xd0, 0x39, 0xf1, 0xf7, 0xa1, 0x57, - 0x9c, 0x4c, 0x13, 0x85, 0x8f, 0xcb, 0x14, 0xae, 0x65, 0xac, 0x1b, 0x19, 0xcd, 0x72, 0x9c, 0x04, - 0xf2, 0x69, 0x63, 0x60, 0x15, 0x59, 0xdc, 0xfb, 0x8f, 0x66, 0xb1, 0xda, 0x34, 0xd8, 0x4b, 0x85, - 0xa1, 0xb5, 0x5a, 0x18, 0x2e, 0x17, 0x59, 0x8d, 0xef, 0x55, 0x64, 0xfd, 0x1c, 0xdc, 0x80, 0x2a, - 0x8d, 0xf0, 0xba, 0x08, 0xda, 0x1b, 0xab, 0x55, 0x85, 0xa9, 0x45, 0xc2, 0x6b, 0xc9, 0x2b, 0x66, - 0xbc, 0x4b, 0x9e, 0x5c, 0xc9, 0x38, 0xfc, 0x86, 0x26, 0x08, 0xf8, 0xe6, 0x0a, 0x51, 0x8d, 0x63, - 0x74, 0xf5, 0x61, 0xc6, 0x31, 0xc5, 0x64, 0xc9, 0xae, 0x26, 0x4b, 0x28, 0xcf, 0x79, 0xaa, 0x64, - 0x96, 0x17, 0x25, 0xaa, 0x86, 0xca, 0x6a, 0xce, 0x35, 0xbc, 0x58, 0xcd, 0xbd, 0x0b, 0xbd, 0x38, - 0x89, 0x27, 0xf1, 0x3c, 0x8a, 0xb0, 0x88, 0x36, 0x43, 0xc4, 0x6e, 0x9c, 0xc4, 0xc7, 0x06, 0xc5, - 0x1e, 0xc1, 0x1b, 0x75, 0x16, 0x6d, 0xcf, 0x5d, 0x3d, 0x83, 0xa8, 0xf1, 0x91, 0xd5, 0x6f, 0x41, - 0x3f, 0x39, 0xff, 0x95, 0xf4, 0x73, 0x92, 0xd8, 0x84, 0x0c, 0xb9, 0xa7, 0x53, 0xb7, 0xc6, 0xa3, - 0x88, 0x8e, 0xc5, 0x4c, 0x7a, 0x5f, 0x80, 0x5b, 0x0a, 0x01, 0xa3, 0xf4, 0xf1, 0xc9, 0xf1, 0x50, - 0xc7, 0xd1, 0x83, 0xe3, 0xfd, 0xe1, 0x1f, 0xf7, 0x2d, 0x8c, 0xf3, 0x7c, 0xf8, 0x62, 0xc8, 0x47, - 0xc3, 0x7e, 0x03, 0x63, 0xf0, 0xfe, 0xf0, 0x70, 0x38, 0x1e, 0xf6, 0x9b, 0x5f, 0xb6, 0x9c, 0x4e, - 0xdf, 0xe1, 0x8e, 0x5c, 0xa4, 0x51, 0xe8, 0x87, 0xb9, 0x37, 0x02, 0xa8, 0x4a, 0x27, 0x8c, 0x2b, - 0xd5, 0xd9, 0x5a, 0xa3, 0x4e, 0x6e, 0x4e, 0xc5, 0x7a, 0xcf, 0x98, 0x5a, 0xe3, 0x55, 0xf5, 0x9e, - 0xa6, 0x7b, 0x67, 0xe0, 0x1c, 0x89, 0xf4, 0xa5, 0xfe, 0xa8, 0x57, 0x76, 0xc1, 0x73, 0x33, 0x13, - 0x32, 0x09, 0xfc, 0x03, 0xe8, 0x98, 0x78, 0x68, 0xbc, 0x66, 0x29, 0x56, 0x16, 0x34, 0xef, 0x2f, - 0x2d, 0x78, 0xeb, 0x28, 0xb9, 0x96, 0x65, 0x0d, 0x73, 0x2a, 0x6e, 0xa2, 0x44, 0x04, 0xdf, 0x61, - 0x88, 0x3f, 0x05, 0x50, 0xc9, 0x3c, 0xf3, 0xe5, 0x64, 0x5a, 0x8e, 0xa2, 0x5c, 0x8d, 0x79, 0x6e, - 0xa6, 0xde, 0x52, 0xe5, 0x44, 0x6c, 0x6a, 0xe7, 0x43, 0x18, 0x49, 0x3f, 0x02, 0x3b, 0x5f, 0xc4, - 0xd5, 0xe4, 0xab, 0x9d, 0x63, 0x73, 0xea, 0xed, 0x81, 0x3b, 0x5e, 0x50, 0xcb, 0x36, 0x57, 0x4b, - 0x59, 0xd9, 0x7a, 0x4d, 0x56, 0x6e, 0xac, 0x64, 0xe5, 0xff, 0xb6, 0xa0, 0x5b, 0x2b, 0xae, 0xd8, - 0xbb, 0xd0, 0xca, 0x17, 0xf1, 0xf2, 0xc8, 0xb8, 0x38, 0x84, 0x13, 0x09, 0xed, 0x0d, 0xfb, 0x39, - 0xa1, 0x54, 0x38, 0x8d, 0x65, 0x60, 0xb6, 0xc4, 0x1e, 0x6f, 0xd7, 0xa0, 0xd8, 0x21, 0xdc, 0xd5, - 0x91, 0xa4, 0x18, 0x17, 0x15, 0x55, 0xfc, 0x7b, 0x2b, 0xc5, 0x9c, 0x6e, 0x6b, 0xf7, 0x0a, 0x2e, - 0xdd, 0xb8, 0xaf, 0x4f, 0x97, 0x90, 0x1b, 0xbb, 0xf0, 0xe6, 0x2d, 0x6c, 0x3f, 0x68, 0x42, 0xf1, - 0x00, 0xd6, 0xb0, 0xa3, 0x0f, 0x67, 0x52, 0xe5, 0x62, 0x96, 0x52, 0x55, 0x63, 0x32, 0x41, 0x8b, - 0x37, 0x72, 0xe5, 0x7d, 0x08, 0xbd, 0x53, 0x29, 0x33, 0x2e, 0x55, 0x9a, 0xc4, 0x3a, 0xa3, 0x2b, - 0x7a, 0xb4, 0x49, 0x3b, 0x06, 0xf2, 0xfe, 0x0c, 0x5c, 0xac, 0xd7, 0x9f, 0x8a, 0xdc, 0xbf, 0xfc, - 0x21, 0xf5, 0xfc, 0x87, 0xd0, 0x49, 0xb5, 0x99, 0x98, 0xea, 0xbb, 0x47, 0x31, 0xce, 0x98, 0x0e, - 0x2f, 0x88, 0xde, 0xe7, 0xd0, 0x3c, 0x9e, 0xcf, 0xea, 0xbf, 0xf3, 0xb4, 0xf4, 0xef, 0x3c, 0x4b, - 0xcd, 0x69, 0x63, 0xa5, 0x39, 0xfd, 0x1a, 0xba, 0x85, 0xec, 0x0f, 0x02, 0xfa, 0xb1, 0x86, 0x94, - 0x7f, 0x10, 0x2c, 0xd9, 0x82, 0xee, 0x11, 0x65, 0x1c, 0x1c, 0x14, 0x4a, 0xd3, 0xc0, 0xf2, 0xde, - 0x26, 0x8f, 0x97, 0x7b, 0x3f, 0x83, 0x5e, 0x51, 0x53, 0x1f, 0xc9, 0x5c, 0x90, 0x39, 0x45, 0xa1, - 0x8c, 0x6b, 0xa6, 0xe6, 0x68, 0xc4, 0x58, 0xbd, 0x66, 0xda, 0xea, 0x6d, 0x83, 0x6d, 0x6c, 0x95, - 0x41, 0xcb, 0x4f, 0x02, 0xed, 0x22, 0x6d, 0x4e, 0xdf, 0xf8, 0xe0, 0x99, 0x9a, 0x16, 0xe9, 0x71, - 0xa6, 0xa6, 0x98, 0x9e, 0x9e, 0x0a, 0xff, 0x6a, 0x9e, 0x16, 0xe9, 0xa9, 0xd6, 0xfc, 0x58, 0x4b, - 0xcd, 0xcf, 0x6b, 0x46, 0xbc, 0x6f, 0x43, 0x67, 0x1e, 0x87, 0x8b, 0xa2, 0xa8, 0x71, 0xb9, 0x8d, - 0xa0, 0x1e, 0x6d, 0x46, 0x89, 0x4f, 0xfd, 0x0e, 0x79, 0x95, 0xcb, 0x4b, 0xd8, 0xfb, 0x53, 0x58, - 0x1b, 0x2e, 0x52, 0x1a, 0x78, 0x7f, 0x67, 0x62, 0xac, 0x5d, 0xaa, 0xb1, 0x74, 0xa9, 0x95, 0x93, - 0x9b, 0xc5, 0xc9, 0x3b, 0xff, 0x62, 0x41, 0x0b, 0x4d, 0x04, 0x5b, 0xb2, 0x3f, 0x94, 0x22, 0xcb, - 0xcf, 0xa5, 0xc8, 0xd9, 0x92, 0x39, 0x6c, 0x2c, 0x41, 0xde, 0x9d, 0x27, 0x16, 0xdb, 0xd6, 0xb3, - 0xf4, 0xe2, 0x27, 0x82, 0xb5, 0xc2, 0xd0, 0xc8, 0x10, 0x57, 0xf9, 0xb7, 0x88, 0xff, 0xcb, 0x24, - 0x8c, 0xf7, 0xf4, 0x80, 0x99, 0xad, 0x1a, 0xe6, 0xea, 0x0a, 0xf6, 0x09, 0xd8, 0x07, 0x0a, 0x3d, - 0xe0, 0x65, 0x56, 0x0a, 0xb0, 0x75, 0xe7, 0xf0, 0xee, 0xec, 0xfc, 0x73, 0x13, 0x5a, 0x5f, 0xcb, - 0x2c, 0x61, 0x3f, 0x83, 0x8e, 0x19, 0x1f, 0xb1, 0xda, 0x98, 0x68, 0x83, 0x32, 0xec, 0xca, 0x5c, - 0x89, 0x4e, 0xe9, 0xeb, 0x18, 0x5d, 0x75, 0x8d, 0xac, 0x9a, 0x6e, 0xbd, 0x74, 0xa9, 0x2f, 0xa0, - 0x3f, 0xca, 0x33, 0x29, 0x66, 0x35, 0xf6, 0x65, 0x41, 0xdd, 0xd6, 0x82, 0x92, 0xbc, 0x3e, 0x06, - 0x5b, 0x87, 0x99, 0x95, 0x05, 0xab, 0xdd, 0x24, 0x31, 0x3f, 0x84, 0xee, 0xe8, 0x32, 0x99, 0x47, - 0xc1, 0x48, 0x66, 0xd7, 0x92, 0xd5, 0x46, 0xb8, 0x1b, 0xb5, 0x6f, 0xef, 0x0e, 0xdb, 0x02, 0xd0, - 0x7e, 0x76, 0x16, 0x06, 0x8a, 0x75, 0x90, 0x76, 0x3c, 0x9f, 0xe9, 0x4d, 0x6b, 0x0e, 0xa8, 0x39, - 0x6b, 0xd1, 0xe6, 0x75, 0x9c, 0x9f, 0xc1, 0xda, 0x1e, 0x45, 0xe3, 0x93, 0x6c, 0xf7, 0x3c, 0xc9, - 0x72, 0xb6, 0x3a, 0xc6, 0xdd, 0x58, 0x45, 0x78, 0x77, 0xd8, 0x13, 0x70, 0xc6, 0xd9, 0x8d, 0xe6, - 0x7f, 0xc3, 0x04, 0xe9, 0xea, 0xbc, 0x5b, 0x5e, 0xb9, 0xf3, 0x0f, 0x4d, 0xb0, 0x7f, 0x99, 0x64, - 0x57, 0x32, 0x63, 0x8f, 0xc0, 0xa6, 0xb6, 0xdf, 0x98, 0x51, 0x39, 0x02, 0xb8, 0xed, 0xa0, 0xf7, - 0xc1, 0x25, 0xa1, 0x8c, 0x85, 0xba, 0xd2, 0xaa, 0xa2, 0xdf, 0x7a, 0xb5, 0x5c, 0x74, 0xf9, 0x46, - 0x7a, 0x5d, 0xd7, 0x8a, 0x2a, 0x47, 0x1d, 0x4b, 0xbd, 0xf8, 0x46, 0x47, 0x37, 0xd6, 0x23, 0x34, - 0xcd, 0x27, 0x16, 0xfb, 0x08, 0x5a, 0x23, 0xfd, 0x52, 0x64, 0xaa, 0x7e, 0xf9, 0xda, 0x58, 0x2f, - 0x10, 0xe5, 0xce, 0x8f, 0xc1, 0xd6, 0xb9, 0x5d, 0x3f, 0x73, 0xa9, 0x60, 0xdd, 0xe8, 0xd7, 0x51, - 0x66, 0xc1, 0x47, 0x60, 0xeb, 0xb0, 0xa1, 0x17, 0x2c, 0x85, 0x10, 0x7d, 0x6b, 0x1d, 0x85, 0x34, - 0xab, 0xf6, 0x73, 0xcd, 0xba, 0xe4, 0xf3, 0x2b, 0xac, 0x9f, 0x40, 0x9f, 0x4b, 0x5f, 0x86, 0xb5, - 0xac, 0xcf, 0x8a, 0x47, 0xdd, 0xe2, 0x7d, 0x5f, 0xc0, 0xda, 0x52, 0x85, 0xc0, 0x06, 0x24, 0xe8, - 0x5b, 0x8a, 0x86, 0xd5, 0xc5, 0x4f, 0xfb, 0xff, 0xf6, 0xed, 0x7d, 0xeb, 0xdf, 0xbf, 0xbd, 0x6f, - 0xfd, 0xe7, 0xb7, 0xf7, 0xad, 0x5f, 0xff, 0xd7, 0xfd, 0x3b, 0xe7, 0x36, 0xfd, 0x8f, 0xc0, 0x67, - 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0x14, 0x9a, 0x79, 0x72, 0x67, 0x20, 0x00, 0x00, + 0xba, 0x45, 0x81, 0xf2, 0x2e, 0x8b, 0x04, 0x09, 0x90, 0x0f, 0x98, 0x45, 0x90, 0x45, 0x80, 0x6c, + 0xb2, 0xc9, 0x76, 0xd6, 0x01, 0x02, 0x64, 0x19, 0xe4, 0x0b, 0x02, 0x27, 0xdf, 0x10, 0x20, 0xbb, + 0xe0, 0x9c, 0x7b, 0xeb, 0x41, 0xb6, 0xba, 0x6d, 0x07, 0x98, 0x95, 0xea, 0x3c, 0xee, 0xeb, 0xbc, + 0xcf, 0xa1, 0xc0, 0x49, 0xcf, 0xb7, 0xd3, 0x2c, 0xc9, 0x13, 0xd6, 0x48, 0xcf, 0x37, 0x5c, 0x91, + 0x86, 0x1a, 0xdc, 0x78, 0x38, 0x0d, 0xf3, 0xcb, 0xf9, 0xf9, 0xb6, 0x9f, 0xcc, 0x1e, 0x07, 0xd3, + 0x4c, 0xa4, 0x97, 0x9f, 0x84, 0xc9, 0xe3, 0x73, 0x11, 0x4c, 0x65, 0xf6, 0x38, 0x3d, 0x7f, 0x5c, + 0xac, 0xf3, 0x36, 0xa0, 0x75, 0x18, 0xaa, 0x9c, 0x31, 0x68, 0xcd, 0xc3, 0x40, 0x0d, 0xac, 0xcd, + 0xe6, 0x96, 0xcd, 0xe9, 0xdb, 0x3b, 0x02, 0x77, 0x2c, 0xd4, 0xd5, 0x0b, 0x11, 0xcd, 0x25, 0xeb, + 0x43, 0xf3, 0x5a, 0x44, 0x03, 0x6b, 0xd3, 0xda, 0xea, 0x71, 0xfc, 0x64, 0xdb, 0xe0, 0x5c, 0x8b, + 0x68, 0x92, 0xdf, 0xa4, 0x72, 0xd0, 0xd8, 0xb4, 0xb6, 0xd6, 0x77, 0xde, 0xdc, 0x4e, 0xcf, 0xb7, + 0x4f, 0x13, 0x95, 0x87, 0xf1, 0x74, 0xfb, 0x85, 0x88, 0xc6, 0x37, 0xa9, 0xe4, 0x9d, 0x6b, 0xfd, + 0xe1, 0x9d, 0x40, 0x77, 0x94, 0xf9, 0xcf, 0xe6, 0xb1, 0x9f, 0x87, 0x49, 0x8c, 0x27, 0xc6, 0x62, + 0x26, 0x69, 0x47, 0x97, 0xd3, 0x37, 0xe2, 0x44, 0x36, 0x55, 0x83, 0xe6, 0x66, 0x13, 0x71, 0xf8, + 0xcd, 0x06, 0xd0, 0x09, 0xd5, 0x5e, 0x32, 0x8f, 0xf3, 0x41, 0x6b, 0xd3, 0xda, 0x72, 0x78, 0x01, + 0x7a, 0x7f, 0xd3, 0x84, 0xf6, 0x1f, 0xcd, 0x65, 0x76, 0x43, 0xeb, 0xf2, 0x3c, 0x2b, 0xf6, 0xc2, + 0x6f, 0xf6, 0x16, 0xb4, 0x23, 0x11, 0x4f, 0xd5, 0xa0, 0x41, 0x9b, 0x69, 0x80, 0xfd, 0x18, 0x5c, + 0x71, 0x91, 0xcb, 0x6c, 0x32, 0x0f, 0x83, 0x41, 0x73, 0xd3, 0xda, 0xb2, 0xb9, 0x43, 0x88, 0xb3, + 0x30, 0x60, 0xef, 0x80, 0x13, 0x24, 0x13, 0xbf, 0x7e, 0x56, 0x90, 0xd0, 0x59, 0xec, 0x3d, 0x70, + 0xe6, 0x61, 0x30, 0x89, 0x42, 0x95, 0x0f, 0xda, 0x9b, 0xd6, 0x56, 0x77, 0xc7, 0xc1, 0xc7, 0xa2, + 0xec, 0x78, 0x67, 0x1e, 0x06, 0x24, 0xc4, 0x47, 0xe0, 0xa8, 0xcc, 0x9f, 0x5c, 0xcc, 0x63, 0x7f, + 0x60, 0x13, 0xd3, 0x5d, 0x64, 0xaa, 0xbd, 0x9a, 0x77, 0x94, 0x06, 0xf0, 0x59, 0x99, 0xbc, 0x96, + 0x99, 0x92, 0x83, 0x8e, 0x3e, 0xca, 0x80, 0xec, 0x09, 0x74, 0x2f, 0x84, 0x2f, 0xf3, 0x49, 0x2a, + 0x32, 0x31, 0x1b, 0x38, 0xd5, 0x46, 0xcf, 0x10, 0x7d, 0x8a, 0x58, 0xc5, 0xe1, 0xa2, 0x04, 0xd8, + 0x67, 0xb0, 0x46, 0x90, 0x9a, 0x5c, 0x84, 0x51, 0x2e, 0xb3, 0x81, 0x4b, 0x6b, 0xd6, 0x69, 0x0d, + 0x61, 0xc6, 0x99, 0x94, 0xbc, 0xa7, 0x99, 0x34, 0x86, 0xfd, 0x14, 0x40, 0x2e, 0x52, 0x11, 0x07, + 0x13, 0x11, 0x45, 0x03, 0xa0, 0x3b, 0xb8, 0x1a, 0xb3, 0x1b, 0x45, 0xec, 0x6d, 0xbc, 0x9f, 0x08, + 0x26, 0xb9, 0x1a, 0xac, 0x6d, 0x5a, 0x5b, 0x2d, 0x6e, 0x23, 0x38, 0x56, 0x28, 0x57, 0x5f, 0xf8, + 0x97, 0x72, 0xb0, 0xbe, 0x69, 0x6d, 0xb5, 0xb9, 0x06, 0xbc, 0x1d, 0x70, 0xc9, 0x4e, 0x48, 0x0e, + 0x1f, 0x80, 0x7d, 0x8d, 0x80, 0x36, 0xa7, 0xee, 0xce, 0x1a, 0x5e, 0xa4, 0x34, 0x25, 0x6e, 0x88, + 0xde, 0x7d, 0x70, 0x0e, 0x45, 0x3c, 0x2d, 0xec, 0x0f, 0x15, 0x44, 0x0b, 0x5c, 0x4e, 0xdf, 0xde, + 0xaf, 0x1b, 0x60, 0x73, 0xa9, 0xe6, 0x51, 0xce, 0x1e, 0x02, 0xa0, 0xf8, 0x67, 0x22, 0xcf, 0xc2, + 0x85, 0xd9, 0xb5, 0x52, 0x80, 0x3b, 0x0f, 0x83, 0x23, 0x22, 0xb1, 0x27, 0xd0, 0xa3, 0xdd, 0x0b, + 0xd6, 0x46, 0x75, 0x81, 0xf2, 0x7e, 0xbc, 0x4b, 0x2c, 0x66, 0xc5, 0x3d, 0xb0, 0x49, 0xe3, 0xda, + 0xea, 0xd6, 0xb8, 0x81, 0xd8, 0x07, 0xb0, 0x1e, 0xc6, 0x39, 0x6a, 0xc4, 0xcf, 0x27, 0x81, 0x54, + 0x85, 0x49, 0xac, 0x95, 0xd8, 0x7d, 0xa9, 0x72, 0xf6, 0x29, 0x68, 0xb1, 0x16, 0x07, 0xb6, 0xe9, + 0xc0, 0xf5, 0x52, 0x5d, 0x4a, 0x9f, 0x48, 0x3c, 0xe6, 0xc4, 0x4f, 0xa0, 0x8b, 0xef, 0x2b, 0x56, + 0xd8, 0xb4, 0xa2, 0x47, 0xaf, 0x31, 0xe2, 0xe0, 0x80, 0x0c, 0x86, 0x1d, 0x45, 0x83, 0x66, 0xa7, + 0xcd, 0x84, 0xbe, 0xbd, 0x21, 0xb4, 0x4f, 0xb2, 0x40, 0x66, 0xb7, 0x5a, 0x3e, 0x83, 0x56, 0x20, + 0x95, 0x4f, 0x4e, 0xe9, 0x70, 0xfa, 0xae, 0xbc, 0xa1, 0x59, 0xf3, 0x06, 0xef, 0xef, 0x2d, 0xe8, + 0x8e, 0x92, 0x2c, 0x3f, 0x92, 0x4a, 0x89, 0xa9, 0x64, 0x0f, 0xa0, 0x9d, 0xe0, 0xb6, 0x46, 0xc2, + 0x2e, 0xde, 0x89, 0xce, 0xe1, 0x1a, 0xbf, 0xa2, 0x87, 0xc6, 0xab, 0xf5, 0x80, 0x56, 0x42, 0x7e, + 0xd4, 0x34, 0x56, 0x42, 0x5e, 0x74, 0x0f, 0xec, 0xe4, 0xe2, 0x42, 0x49, 0x2d, 0xcb, 0x36, 0x37, + 0xd0, 0x2b, 0x8d, 0xcd, 0xfb, 0x1d, 0x00, 0xbc, 0xdf, 0x0f, 0xb4, 0x02, 0xef, 0x12, 0xba, 0x5c, + 0x5c, 0xe4, 0x7b, 0x49, 0x9c, 0xcb, 0x45, 0xce, 0xd6, 0xa1, 0x11, 0x06, 0x24, 0x22, 0x9b, 0x37, + 0xc2, 0x00, 0x2f, 0x37, 0xcd, 0x92, 0x79, 0x4a, 0x12, 0x5a, 0xe3, 0x1a, 0x20, 0x51, 0x06, 0x41, + 0x46, 0x37, 0x46, 0x51, 0x06, 0x41, 0xc6, 0x1e, 0x40, 0x57, 0xc5, 0x22, 0x55, 0x97, 0x49, 0x8e, + 0x97, 0x6b, 0xd1, 0xe5, 0xa0, 0x40, 0x8d, 0x95, 0xf7, 0xaf, 0x16, 0xd8, 0x47, 0x72, 0x76, 0x2e, + 0xb3, 0x97, 0x4e, 0x79, 0x07, 0x1c, 0xda, 0x78, 0x12, 0x06, 0xe6, 0xa0, 0x0e, 0xc1, 0x07, 0xc1, + 0xad, 0x47, 0xdd, 0x03, 0x3b, 0x92, 0x02, 0x85, 0xaf, 0xed, 0xcc, 0x40, 0x28, 0x1b, 0x31, 0x9b, + 0x04, 0x52, 0x04, 0x14, 0x78, 0x1c, 0x6e, 0x8b, 0xd9, 0xbe, 0x14, 0x01, 0xde, 0x2d, 0x12, 0x2a, + 0x9f, 0xcc, 0xd3, 0x40, 0xe4, 0x92, 0x02, 0x4e, 0x0b, 0x0d, 0x47, 0xe5, 0x67, 0x84, 0x61, 0x8f, + 0xe0, 0x0d, 0x3f, 0x9a, 0x2b, 0x8c, 0x76, 0x61, 0x7c, 0x91, 0x4c, 0x92, 0x38, 0xba, 0x21, 0xf9, + 0x3a, 0xfc, 0xae, 0x21, 0x1c, 0xc4, 0x17, 0xc9, 0x49, 0x1c, 0xdd, 0x78, 0xbf, 0x69, 0x40, 0xfb, + 0x39, 0x89, 0xe1, 0x09, 0x74, 0x66, 0xf4, 0xa0, 0xc2, 0x7b, 0xef, 0xa1, 0x84, 0x89, 0xb6, 0xad, + 0x5f, 0xaa, 0x86, 0x71, 0x9e, 0xdd, 0xf0, 0x82, 0x0d, 0x57, 0xe4, 0xe2, 0x3c, 0x92, 0xb9, 0x32, + 0x16, 0x51, 0x5b, 0x31, 0xd6, 0x04, 0xb3, 0xc2, 0xb0, 0xad, 0x8a, 0xb5, 0xb9, 0x2a, 0x56, 0xb6, + 0x01, 0x8e, 0x7f, 0x29, 0xfd, 0x2b, 0x35, 0x9f, 0x19, 0xa1, 0x97, 0xf0, 0xc6, 0x33, 0xe8, 0xd5, + 0xef, 0x81, 0x99, 0xe9, 0x4a, 0xde, 0x90, 0xe0, 0x5b, 0x1c, 0x3f, 0xd9, 0x26, 0xb4, 0xc9, 0xc3, + 0x49, 0xec, 0xdd, 0x1d, 0xc0, 0xeb, 0xe8, 0x25, 0x5c, 0x13, 0x7e, 0xd1, 0xf8, 0xb9, 0x85, 0xfb, + 0xd4, 0x6f, 0x57, 0xdf, 0xc7, 0x7d, 0xf5, 0x3e, 0x7a, 0x49, 0x6d, 0x1f, 0xef, 0x7f, 0x1b, 0xd0, + 0xfb, 0x5a, 0x66, 0xc9, 0x69, 0x96, 0xa4, 0x89, 0x12, 0x11, 0xdb, 0x5d, 0x7e, 0x9d, 0x96, 0xe2, + 0x26, 0x2e, 0xae, 0xb3, 0x6d, 0x8f, 0xca, 0xe7, 0x6a, 0xe9, 0xd4, 0xdf, 0xef, 0x81, 0xad, 0xa5, + 0x7b, 0xcb, 0x13, 0x0c, 0x05, 0x79, 0xb4, 0x3c, 0x49, 0x7e, 0xcb, 0xd7, 0x33, 0x14, 0x76, 0x1f, + 0x60, 0x26, 0x16, 0x87, 0x52, 0x28, 0x79, 0x10, 0x14, 0xe6, 0x5b, 0x61, 0x50, 0xce, 0x33, 0xb1, + 0x18, 0x2f, 0xe2, 0xb1, 0x22, 0xeb, 0x6a, 0xf1, 0x12, 0x66, 0x3f, 0x01, 0x77, 0x26, 0x16, 0xe8, + 0x47, 0x07, 0x81, 0xb1, 0xae, 0x0a, 0xc1, 0xde, 0x85, 0x66, 0xbe, 0x88, 0x29, 0x28, 0x61, 0x76, + 0xc2, 0xd2, 0x63, 0xbc, 0x88, 0x8d, 0xc7, 0x71, 0xa4, 0x15, 0x02, 0x75, 0x2a, 0x81, 0xf6, 0xa1, + 0xe9, 0x87, 0x01, 0xa5, 0x27, 0x97, 0xe3, 0xe7, 0xc6, 0x1f, 0xc0, 0xdd, 0x15, 0x39, 0xd4, 0xf5, + 0xb0, 0xa6, 0x97, 0xbd, 0x55, 0xd7, 0x43, 0xab, 0x2e, 0xfb, 0xdf, 0x34, 0xe1, 0xae, 0x31, 0x86, + 0xcb, 0x30, 0x1d, 0xe5, 0x68, 0xf6, 0x03, 0xe8, 0x50, 0xb4, 0x91, 0x99, 0xb1, 0x89, 0x02, 0x64, + 0xbf, 0x07, 0x36, 0x79, 0x60, 0x61, 0xa7, 0x0f, 0x2a, 0xa9, 0x96, 0xcb, 0xb5, 0xdd, 0x1a, 0x95, + 0x18, 0x76, 0xf6, 0x39, 0xb4, 0xbf, 0x91, 0x59, 0xa2, 0xa3, 0x67, 0x77, 0xe7, 0xfe, 0x6d, 0xeb, + 0x50, 0xb7, 0x66, 0x99, 0x66, 0xfe, 0x2d, 0x0a, 0xff, 0x7d, 0x8c, 0x97, 0xb3, 0xe4, 0x5a, 0x06, + 0x83, 0x0e, 0xdd, 0xa8, 0x6e, 0x1f, 0x05, 0xa9, 0x90, 0xb6, 0x53, 0x49, 0x7b, 0x1f, 0xba, 0xb5, + 0xe7, 0xdd, 0x22, 0xe9, 0x07, 0xcb, 0x16, 0xef, 0x96, 0x8e, 0x5c, 0x77, 0x9c, 0x7d, 0x80, 0xea, + 0xb1, 0xff, 0x5f, 0xf7, 0xf3, 0xfe, 0xc2, 0x82, 0xbb, 0x7b, 0x49, 0x1c, 0x4b, 0x2a, 0x8c, 0xb4, + 0xea, 0x2a, 0xb3, 0xb7, 0x5e, 0x69, 0xf6, 0x1f, 0x41, 0x5b, 0x21, 0xb3, 0xd9, 0xfd, 0xcd, 0x5b, + 0x74, 0xc1, 0x35, 0x07, 0x86, 0x99, 0x99, 0x58, 0x4c, 0x52, 0x19, 0x07, 0x61, 0x3c, 0x2d, 0xc2, + 0xcc, 0x4c, 0x2c, 0x4e, 0x35, 0xc6, 0xfb, 0x07, 0x0b, 0x6c, 0xed, 0x31, 0x4b, 0xd1, 0xda, 0x5a, + 0x8e, 0xd6, 0x3f, 0x01, 0x37, 0xcd, 0x64, 0x10, 0xfa, 0xc5, 0xa9, 0x2e, 0xaf, 0x10, 0x68, 0x9c, + 0x17, 0x49, 0xe6, 0x4b, 0xda, 0xde, 0xe1, 0x1a, 0x40, 0xac, 0x4a, 0x85, 0xaf, 0x8b, 0xbb, 0x26, + 0xd7, 0x00, 0xc6, 0x78, 0xad, 0x1c, 0x52, 0x8a, 0xc3, 0x0d, 0x84, 0x55, 0x29, 0xe5, 0x3f, 0x8a, + 0xd0, 0x2e, 0x91, 0x1c, 0x44, 0x50, 0x68, 0xfe, 0xa7, 0x06, 0xf4, 0xf6, 0xc3, 0x4c, 0xfa, 0xb9, + 0x0c, 0x86, 0xc1, 0x94, 0x76, 0x91, 0x71, 0x1e, 0xe6, 0x37, 0x26, 0xd9, 0x18, 0xa8, 0xac, 0x05, + 0x1a, 0xcb, 0x55, 0xb0, 0xd6, 0x45, 0x93, 0x0a, 0x77, 0x0d, 0xb0, 0x1d, 0x00, 0x5d, 0x25, 0x51, + 0xf1, 0xde, 0x7a, 0x75, 0xf1, 0xee, 0x12, 0x1b, 0x7e, 0xa2, 0x80, 0xf4, 0x9a, 0x50, 0x27, 0x22, + 0x9b, 0x2a, 0xfb, 0x39, 0x1a, 0x32, 0x15, 0x17, 0xe7, 0x32, 0x22, 0x43, 0xa5, 0xe2, 0xe2, 0x5c, + 0x46, 0x65, 0x49, 0xd7, 0xd1, 0xd7, 0xc1, 0x6f, 0xf6, 0x1e, 0x34, 0x92, 0x94, 0x1e, 0x6f, 0x0e, + 0xac, 0x3f, 0x6c, 0xfb, 0x24, 0xe5, 0x8d, 0x24, 0x45, 0x2b, 0xd0, 0x95, 0xea, 0xc0, 0x35, 0xc6, + 0x8d, 0xd1, 0x85, 0xaa, 0x29, 0x6e, 0x28, 0xde, 0x3d, 0x68, 0x9c, 0xa4, 0xac, 0x03, 0xcd, 0xd1, + 0x70, 0xdc, 0xbf, 0x83, 0x1f, 0xfb, 0xc3, 0xc3, 0xbe, 0xe5, 0xfd, 0x75, 0x03, 0xdc, 0xa3, 0x79, + 0x2e, 0xd0, 0xa6, 0xd4, 0xeb, 0x94, 0xfa, 0x0e, 0x38, 0x2a, 0x17, 0x19, 0x45, 0x68, 0x1d, 0x56, + 0x3a, 0x04, 0x8f, 0x15, 0xfb, 0x10, 0xda, 0x32, 0x98, 0xca, 0xc2, 0xdb, 0xfb, 0xab, 0xf7, 0xe4, + 0x9a, 0xcc, 0xb6, 0xc0, 0x56, 0xfe, 0xa5, 0x9c, 0x89, 0x41, 0xab, 0x62, 0x1c, 0x11, 0x46, 0x67, + 0x60, 0x6e, 0xe8, 0xd4, 0x58, 0x64, 0x49, 0x4a, 0x95, 0x76, 0xdb, 0x34, 0x16, 0x59, 0x92, 0x62, + 0x9d, 0xbd, 0x03, 0x3f, 0x0a, 0xa7, 0x71, 0x92, 0xc9, 0x49, 0x18, 0x07, 0x72, 0x31, 0xf1, 0x93, + 0xf8, 0x22, 0x0a, 0xfd, 0x9c, 0x64, 0xe9, 0xf0, 0x37, 0x35, 0xf1, 0x00, 0x69, 0x7b, 0x86, 0xc4, + 0xde, 0x87, 0x36, 0x2a, 0x4e, 0x19, 0xe7, 0xa7, 0x62, 0x13, 0x75, 0x64, 0x4e, 0xd5, 0x44, 0x6f, + 0x01, 0x4e, 0x11, 0x5a, 0xd9, 0x47, 0x18, 0x13, 0x29, 0x34, 0x1b, 0xcf, 0xa2, 0x7e, 0xa2, 0x56, + 0x23, 0xf1, 0x82, 0x8e, 0xca, 0xa4, 0x9b, 0x14, 0xc1, 0x96, 0x80, 0x7a, 0x85, 0xd6, 0x5c, 0x6a, + 0x07, 0xb0, 0xd8, 0x4c, 0x62, 0x69, 0x8a, 0x16, 0xfa, 0xf6, 0xfe, 0xae, 0x01, 0x4e, 0x99, 0x0d, + 0x3f, 0x06, 0x77, 0x56, 0x28, 0xc4, 0xf8, 0x2c, 0x95, 0xe3, 0xa5, 0x96, 0x78, 0x45, 0x67, 0xf7, + 0xa0, 0x71, 0x75, 0x6d, 0xc4, 0x69, 0x23, 0xd7, 0x57, 0x2f, 0x78, 0xe3, 0xea, 0xba, 0x72, 0xfa, + 0xf6, 0x77, 0x3a, 0xfd, 0x43, 0xb8, 0xeb, 0x47, 0x52, 0xc4, 0x93, 0xca, 0x67, 0xb5, 0x59, 0xae, + 0x13, 0xfa, 0xb4, 0x74, 0x5c, 0x13, 0xb8, 0x3a, 0x55, 0x7a, 0xfa, 0x00, 0xda, 0x81, 0x8c, 0x72, + 0x51, 0xef, 0xb9, 0x4e, 0x32, 0xe1, 0x47, 0x72, 0x1f, 0xd1, 0x5c, 0x53, 0xd9, 0x16, 0x38, 0x45, + 0xaa, 0x36, 0x9d, 0x16, 0x15, 0xef, 0x85, 0xb0, 0x79, 0x49, 0xad, 0x64, 0x09, 0x35, 0x59, 0x7a, + 0x9f, 0x42, 0xf3, 0xab, 0x17, 0x23, 0xf3, 0x56, 0xeb, 0xa5, 0xb7, 0x16, 0x12, 0x6d, 0xd4, 0x24, + 0xfa, 0x3f, 0x4d, 0xe8, 0x18, 0xdf, 0xc4, 0x7b, 0xcf, 0xcb, 0x42, 0x13, 0x3f, 0x97, 0xf3, 0x63, + 0xe9, 0xe4, 0xf5, 0xfe, 0xbc, 0xf9, 0xdd, 0xfd, 0x39, 0xfb, 0x05, 0xf4, 0x52, 0x4d, 0xab, 0x87, + 0x85, 0xb7, 0xeb, 0x6b, 0xcc, 0x5f, 0x5a, 0xd7, 0x4d, 0x2b, 0x00, 0x0d, 0x9c, 0x5a, 0x9a, 0x5c, + 0x4c, 0x49, 0x45, 0x3d, 0xde, 0x41, 0x78, 0x2c, 0xa6, 0xaf, 0x08, 0x0e, 0xdf, 0xc3, 0xc7, 0xb1, + 0xa0, 0x4e, 0xd2, 0x41, 0x8f, 0xfc, 0x16, 0xe3, 0x42, 0xdd, 0x65, 0xd7, 0x96, 0x5d, 0xf6, 0xc7, + 0xe0, 0xfa, 0xc9, 0x6c, 0x16, 0x12, 0x6d, 0xdd, 0x14, 0x8c, 0x84, 0x18, 0x2b, 0xef, 0xaf, 0x2c, + 0xe8, 0x98, 0xd7, 0xb2, 0x2e, 0x74, 0xf6, 0x87, 0xcf, 0x76, 0xcf, 0x0e, 0x31, 0x6a, 0x00, 0xd8, + 0x4f, 0x0f, 0x8e, 0x77, 0xf9, 0x9f, 0xf4, 0x2d, 0x8c, 0x20, 0x07, 0xc7, 0xe3, 0x7e, 0x83, 0xb9, + 0xd0, 0x7e, 0x76, 0x78, 0xb2, 0x3b, 0xee, 0x37, 0x99, 0x03, 0xad, 0xa7, 0x27, 0x27, 0x87, 0xfd, + 0x16, 0xeb, 0x81, 0xb3, 0xbf, 0x3b, 0x1e, 0x8e, 0x0f, 0x8e, 0x86, 0xfd, 0x36, 0xf2, 0x3e, 0x1f, + 0x9e, 0xf4, 0x6d, 0xfc, 0x38, 0x3b, 0xd8, 0xef, 0x77, 0x90, 0x7e, 0xba, 0x3b, 0x1a, 0xfd, 0xf2, + 0x84, 0xef, 0xf7, 0x1d, 0xdc, 0x77, 0x34, 0xe6, 0x07, 0xc7, 0xcf, 0xfb, 0x2e, 0x7e, 0x9f, 0x3c, + 0xfd, 0x72, 0xb8, 0x37, 0xee, 0x83, 0xf7, 0x29, 0x74, 0x6b, 0x12, 0xc4, 0xd5, 0x7c, 0xf8, 0xac, + 0x7f, 0x07, 0x8f, 0x7c, 0xb1, 0x7b, 0x78, 0x36, 0xec, 0x5b, 0x6c, 0x1d, 0x80, 0x3e, 0x27, 0x87, + 0xbb, 0xc7, 0xcf, 0xfb, 0x0d, 0xef, 0x77, 0xc1, 0x39, 0x0b, 0x83, 0xa7, 0x51, 0xe2, 0x5f, 0xa1, + 0x61, 0x9c, 0x0b, 0x25, 0x4d, 0xaa, 0xa5, 0x6f, 0xcc, 0x05, 0x64, 0x94, 0xca, 0xe8, 0xde, 0x40, + 0xde, 0x31, 0x74, 0xce, 0xc2, 0xe0, 0x54, 0xf8, 0x57, 0xd8, 0xe8, 0x9f, 0xe3, 0xfa, 0x89, 0x0a, + 0xbf, 0x91, 0x26, 0x0c, 0xba, 0x84, 0x19, 0x85, 0xdf, 0x48, 0xf6, 0x3e, 0xd8, 0x04, 0x14, 0x45, + 0x11, 0xd9, 0x72, 0x71, 0x26, 0x37, 0x34, 0xef, 0x6f, 0xad, 0xf2, 0xee, 0xd4, 0xaf, 0x3f, 0x80, + 0x56, 0x2a, 0xfc, 0x2b, 0x13, 0x4d, 0xba, 0x66, 0x0d, 0x9e, 0xc7, 0x89, 0xc0, 0x1e, 0x82, 0x63, + 0x0c, 0xa4, 0xd8, 0xb8, 0x5b, 0xb3, 0x24, 0x5e, 0x12, 0x97, 0x55, 0xd7, 0x5c, 0x56, 0x1d, 0x3e, + 0x4f, 0xa5, 0x51, 0x48, 0xad, 0x57, 0x13, 0xa3, 0x8e, 0x86, 0xbc, 0xcf, 0x01, 0xaa, 0x61, 0xc8, + 0x2d, 0x95, 0xfb, 0x5b, 0xd0, 0x16, 0x51, 0x68, 0xa4, 0xe2, 0x72, 0x0d, 0x78, 0xc7, 0xd0, 0xad, + 0x8d, 0x50, 0xd0, 0x9e, 0x44, 0x14, 0x4d, 0xae, 0xe4, 0x8d, 0xa2, 0xb5, 0x0e, 0xef, 0x88, 0x28, + 0xfa, 0x4a, 0xde, 0x28, 0x8c, 0xb0, 0x7a, 0xfa, 0xd2, 0x58, 0x69, 0xe7, 0x69, 0x29, 0xd7, 0x44, + 0xef, 0x67, 0x60, 0xeb, 0x1e, 0xbf, 0x66, 0xce, 0xd6, 0x2b, 0x53, 0xd6, 0x17, 0xe6, 0xce, 0x34, + 0x11, 0x60, 0x1f, 0x9b, 0x29, 0x8f, 0xd2, 0x33, 0x25, 0xab, 0x2a, 0xe3, 0x34, 0x93, 0x19, 0xf0, + 0x10, 0xb3, 0xb7, 0x0f, 0xce, 0x6b, 0xe7, 0x66, 0x46, 0x00, 0x8d, 0x4a, 0x00, 0xb7, 0x4c, 0xd2, + 0xbc, 0x5f, 0x01, 0x54, 0xd3, 0x20, 0xe3, 0x5d, 0x7a, 0x17, 0xf4, 0xae, 0x47, 0xd8, 0x72, 0x85, + 0x51, 0x90, 0xc9, 0x78, 0xe9, 0xd5, 0xd5, 0xfc, 0xa8, 0xa4, 0xb3, 0x4d, 0x68, 0xd1, 0x90, 0xab, + 0x59, 0x45, 0xbf, 0x72, 0xc2, 0x45, 0x14, 0x6f, 0x01, 0x6b, 0x3a, 0x13, 0x72, 0xf9, 0xe7, 0x73, + 0xa9, 0x5e, 0x5b, 0x5f, 0xdd, 0x07, 0x28, 0x63, 0x75, 0x31, 0xae, 0xab, 0x61, 0xd0, 0x08, 0x2e, + 0x42, 0x19, 0x05, 0xc5, 0x6b, 0x0c, 0x84, 0x4a, 0xd6, 0x69, 0xb0, 0xa5, 0x67, 0x1a, 0x3a, 0xed, + 0xfd, 0x3e, 0xf4, 0x8a, 0x93, 0x69, 0x68, 0xf0, 0x71, 0x99, 0xa5, 0xb5, 0x8c, 0x75, 0xaf, 0xa2, + 0x59, 0x8e, 0x93, 0x40, 0x3e, 0x6d, 0x0c, 0xac, 0x22, 0x51, 0x7b, 0xff, 0xd1, 0x2c, 0x56, 0x9b, + 0x1e, 0x7a, 0xa9, 0xf6, 0xb3, 0x56, 0x6b, 0xbf, 0xe5, 0x3a, 0xaa, 0xf1, 0xbd, 0xea, 0xa8, 0x9f, + 0x83, 0x1b, 0x50, 0x31, 0x11, 0x5e, 0x17, 0x71, 0x79, 0x63, 0xb5, 0x70, 0x30, 0xe5, 0x46, 0x78, + 0x2d, 0x79, 0xc5, 0x8c, 0x77, 0xc9, 0x93, 0x2b, 0x19, 0x87, 0xdf, 0xd0, 0x90, 0x00, 0xdf, 0x5c, + 0x21, 0xaa, 0x89, 0x8b, 0x2e, 0x30, 0xcc, 0xc4, 0xa5, 0x18, 0x1e, 0xd9, 0xd5, 0xf0, 0x08, 0xe5, + 0x39, 0x4f, 0x95, 0xcc, 0xf2, 0xa2, 0x0a, 0xd5, 0x50, 0x59, 0xb0, 0xb9, 0x86, 0x17, 0x0b, 0xb6, + 0x77, 0xa1, 0x17, 0x27, 0xf1, 0x24, 0x9e, 0x47, 0x11, 0xd6, 0xc9, 0x66, 0x4e, 0xd8, 0x8d, 0x93, + 0xf8, 0xd8, 0xa0, 0xd8, 0x23, 0x78, 0xa3, 0xce, 0xa2, 0xed, 0xb9, 0xab, 0xc7, 0x0c, 0x35, 0x3e, + 0xb2, 0xfa, 0x2d, 0xe8, 0x27, 0xe7, 0xbf, 0x92, 0x7e, 0x4e, 0x12, 0x9b, 0x90, 0x21, 0xf7, 0x74, + 0x76, 0xd6, 0x78, 0x14, 0xd1, 0xb1, 0x98, 0x49, 0xef, 0x0b, 0x70, 0x4b, 0x21, 0x60, 0x20, 0x3e, + 0x3e, 0x39, 0x1e, 0xea, 0x50, 0x79, 0x70, 0xbc, 0x3f, 0xfc, 0xe3, 0xbe, 0x85, 0xa1, 0x9c, 0x0f, + 0x5f, 0x0c, 0xf9, 0x68, 0xd8, 0x6f, 0x60, 0x98, 0xdd, 0x1f, 0x1e, 0x0e, 0xc7, 0xc3, 0x7e, 0xf3, + 0xcb, 0x96, 0xd3, 0xe9, 0x3b, 0xdc, 0x91, 0x8b, 0x34, 0x0a, 0xfd, 0x30, 0xf7, 0x46, 0x00, 0x55, + 0x75, 0x84, 0xf1, 0xa6, 0x3a, 0x5b, 0x6b, 0xd4, 0xc9, 0xcd, 0xa9, 0x58, 0xd2, 0x19, 0x53, 0x6b, + 0xbc, 0xaa, 0xa4, 0xd3, 0x74, 0xef, 0x0c, 0x9c, 0x23, 0x91, 0xbe, 0xd4, 0x02, 0xf5, 0xca, 0x46, + 0x77, 0x6e, 0xc6, 0x3e, 0x26, 0x47, 0x7f, 0x00, 0x1d, 0x13, 0xf2, 0x8c, 0xd7, 0x2c, 0x85, 0xc3, + 0x82, 0xe6, 0xfd, 0xa5, 0x05, 0x6f, 0x1d, 0x25, 0xd7, 0xb2, 0x2c, 0x53, 0x4e, 0xc5, 0x4d, 0x94, + 0x88, 0xe0, 0x3b, 0x0c, 0xf1, 0xa7, 0x00, 0x2a, 0x99, 0x67, 0xbe, 0x9c, 0x4c, 0xcb, 0x69, 0x93, + 0xab, 0x31, 0xcf, 0xcd, 0x60, 0x5b, 0xaa, 0x9c, 0x88, 0x4d, 0xed, 0x7c, 0x08, 0x23, 0xe9, 0x47, + 0x60, 0xe7, 0x8b, 0xb8, 0x1a, 0x6e, 0xb5, 0x73, 0xec, 0x3f, 0xbd, 0x3d, 0x70, 0xc7, 0x0b, 0xea, + 0xca, 0xe6, 0x6a, 0x29, 0xf1, 0x5a, 0xaf, 0x49, 0xbc, 0x8d, 0x95, 0xc4, 0xfb, 0xdf, 0x16, 0x74, + 0x6b, 0xf5, 0x13, 0x7b, 0x17, 0x5a, 0xf9, 0x22, 0x5e, 0x9e, 0x0a, 0x17, 0x87, 0x70, 0x22, 0xa1, + 0xbd, 0x61, 0xcb, 0x26, 0x94, 0x0a, 0xa7, 0xb1, 0x0c, 0xcc, 0x96, 0xd8, 0xc6, 0xed, 0x1a, 0x14, + 0x3b, 0x84, 0xbb, 0x3a, 0x92, 0x14, 0x13, 0xa1, 0xa2, 0x50, 0x7f, 0x6f, 0xa5, 0x5e, 0xd3, 0x9d, + 0xeb, 0x5e, 0xc1, 0xa5, 0x7b, 0xf3, 0xf5, 0xe9, 0x12, 0x72, 0x63, 0x17, 0xde, 0xbc, 0x85, 0xed, + 0x07, 0x0d, 0x21, 0x1e, 0xc0, 0x1a, 0x36, 0xed, 0xe1, 0x4c, 0xaa, 0x5c, 0xcc, 0x52, 0x2a, 0x5c, + 0x4c, 0x26, 0x68, 0xf1, 0x46, 0xae, 0xbc, 0x0f, 0xa1, 0x77, 0x2a, 0x65, 0xc6, 0xa5, 0x4a, 0x93, + 0x58, 0x27, 0x6d, 0x45, 0x8f, 0x36, 0x69, 0xc7, 0x40, 0xde, 0x9f, 0x81, 0x8b, 0x25, 0xf9, 0x53, + 0x91, 0xfb, 0x97, 0x3f, 0xa4, 0x64, 0xff, 0x10, 0x3a, 0xa9, 0x36, 0x13, 0x53, 0x60, 0xf7, 0x28, + 0xc6, 0x19, 0xd3, 0xe1, 0x05, 0xd1, 0xfb, 0x1c, 0x9a, 0xc7, 0xf3, 0x59, 0xfd, 0xa7, 0x9c, 0x96, + 0xfe, 0x29, 0x67, 0xa9, 0xff, 0x6c, 0xac, 0xf4, 0x9f, 0x5f, 0x43, 0xb7, 0x90, 0xfd, 0x41, 0x40, + 0xbf, 0xc7, 0x90, 0xf2, 0x0f, 0x82, 0x25, 0x5b, 0xd0, 0x6d, 0xa0, 0x8c, 0x83, 0x83, 0x42, 0x69, + 0x1a, 0x58, 0xde, 0xdb, 0x8c, 0x39, 0xca, 0xbd, 0x9f, 0x41, 0xaf, 0x28, 0x9b, 0x8f, 0x64, 0x2e, + 0xc8, 0x9c, 0xa2, 0x50, 0xc6, 0x35, 0x53, 0x73, 0x34, 0x62, 0xac, 0x5e, 0x33, 0x50, 0xf5, 0xb6, + 0xc1, 0x36, 0xb6, 0xca, 0xa0, 0xe5, 0x27, 0x81, 0x76, 0x91, 0x36, 0xa7, 0x6f, 0x7c, 0xf0, 0x4c, + 0x4d, 0x8b, 0xf4, 0x38, 0x53, 0x53, 0x4c, 0x4f, 0x4f, 0x85, 0x7f, 0x35, 0x4f, 0x8b, 0xf4, 0x54, + 0xeb, 0x6f, 0xac, 0xa5, 0xfe, 0xe6, 0x35, 0x53, 0xdc, 0xb7, 0xa1, 0x33, 0x8f, 0xc3, 0x45, 0x51, + 0xb7, 0xb8, 0xdc, 0x46, 0x50, 0x4f, 0x2f, 0xa3, 0xc4, 0xa7, 0x96, 0x86, 0xbc, 0xca, 0xe5, 0x25, + 0xec, 0xfd, 0x29, 0xac, 0x0d, 0x17, 0x29, 0xcd, 0xb4, 0xbf, 0x33, 0x31, 0xd6, 0x2e, 0xd5, 0x58, + 0xba, 0xd4, 0xca, 0xc9, 0xcd, 0xe2, 0xe4, 0x9d, 0x7f, 0xb1, 0xa0, 0x85, 0x26, 0x82, 0x5d, 0xd7, + 0x1f, 0x4a, 0x91, 0xe5, 0xe7, 0x52, 0xe4, 0x6c, 0xc9, 0x1c, 0x36, 0x96, 0x20, 0xef, 0xce, 0x13, + 0x8b, 0x6d, 0xeb, 0x71, 0x79, 0xf1, 0x2b, 0xc0, 0x5a, 0x61, 0x68, 0x64, 0x88, 0xab, 0xfc, 0x5b, + 0xc4, 0xff, 0x65, 0x12, 0xc6, 0x7b, 0x7a, 0x86, 0xcc, 0x56, 0x0d, 0x73, 0x75, 0x05, 0xfb, 0x04, + 0xec, 0x03, 0x85, 0x1e, 0xf0, 0x32, 0x2b, 0x05, 0xd8, 0xba, 0x73, 0x78, 0x77, 0x76, 0xfe, 0xb9, + 0x09, 0xad, 0xaf, 0x65, 0x96, 0xb0, 0x9f, 0x41, 0xc7, 0x4c, 0x88, 0x58, 0x6d, 0x12, 0xb4, 0x41, + 0x19, 0x76, 0x65, 0x74, 0x44, 0xa7, 0xf4, 0x75, 0x8c, 0xae, 0x1a, 0x43, 0x56, 0x0d, 0xb0, 0x5e, + 0xba, 0xd4, 0x17, 0xd0, 0x1f, 0xe5, 0x99, 0x14, 0xb3, 0x1a, 0xfb, 0xb2, 0xa0, 0x6e, 0xeb, 0x32, + 0x49, 0x5e, 0x1f, 0x83, 0xad, 0xc3, 0xcc, 0xca, 0x82, 0xd5, 0x86, 0x91, 0x98, 0x1f, 0x42, 0x77, + 0x74, 0x99, 0xcc, 0xa3, 0x60, 0x24, 0xb3, 0x6b, 0xc9, 0x6a, 0x53, 0xda, 0x8d, 0xda, 0xb7, 0x77, + 0x87, 0x6d, 0x01, 0x68, 0x3f, 0x3b, 0x0b, 0x03, 0xc5, 0x3a, 0x48, 0x3b, 0x9e, 0xcf, 0xf4, 0xa6, + 0x35, 0x07, 0xd4, 0x9c, 0xb5, 0x68, 0xf3, 0x3a, 0xce, 0xcf, 0x60, 0x6d, 0x8f, 0xa2, 0xf1, 0x49, + 0xb6, 0x7b, 0x9e, 0x64, 0x39, 0x5b, 0x9d, 0xd4, 0x6e, 0xac, 0x22, 0xbc, 0x3b, 0xec, 0x09, 0x38, + 0xe3, 0xec, 0x46, 0xf3, 0xbf, 0x61, 0x82, 0x74, 0x75, 0xde, 0x2d, 0xaf, 0xdc, 0xf9, 0xc7, 0x26, + 0xd8, 0xbf, 0x4c, 0xb2, 0x2b, 0x99, 0xb1, 0x47, 0x60, 0x53, 0x67, 0x6f, 0xcc, 0xa8, 0xec, 0xf2, + 0x6f, 0x3b, 0xe8, 0x7d, 0x70, 0x49, 0x28, 0x63, 0xa1, 0xae, 0xb4, 0xaa, 0xe8, 0xe7, 0x5c, 0x2d, + 0x17, 0x5d, 0xbe, 0x91, 0x5e, 0xd7, 0xb5, 0xa2, 0xca, 0x69, 0xc6, 0x52, 0xbb, 0xbd, 0xd1, 0xd1, + 0xbd, 0xf3, 0x08, 0x4d, 0xf3, 0x89, 0xc5, 0x3e, 0x82, 0xd6, 0x48, 0xbf, 0x14, 0x99, 0xaa, 0x1f, + 0xb7, 0x36, 0xd6, 0x0b, 0x44, 0xb9, 0xf3, 0x63, 0xb0, 0x75, 0x6e, 0xd7, 0xcf, 0x5c, 0x2a, 0x58, + 0x37, 0xfa, 0x75, 0x94, 0x59, 0xf0, 0x11, 0xd8, 0x3a, 0x6c, 0xe8, 0x05, 0x4b, 0x21, 0x44, 0xdf, + 0x5a, 0x47, 0x21, 0xcd, 0xaa, 0xfd, 0x5c, 0xb3, 0x2e, 0xf9, 0xfc, 0x0a, 0xeb, 0x27, 0xd0, 0xe7, + 0xd2, 0x97, 0x61, 0x2d, 0xeb, 0xb3, 0xe2, 0x51, 0xb7, 0x78, 0xdf, 0x17, 0xb0, 0xb6, 0x54, 0x21, + 0xb0, 0x01, 0x09, 0xfa, 0x96, 0xa2, 0x61, 0x75, 0xf1, 0xd3, 0xfe, 0xbf, 0x7d, 0x7b, 0xdf, 0xfa, + 0xf7, 0x6f, 0xef, 0x5b, 0xff, 0xf9, 0xed, 0x7d, 0xeb, 0xd7, 0xff, 0x75, 0xff, 0xce, 0xb9, 0x4d, + 0xff, 0x06, 0xf0, 0xd9, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x5d, 0x09, 0x6e, 0x74, 0x4a, 0x20, + 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -6542,11 +6534,6 @@ func (m *PostingList) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintPb(dAtA, i, uint64(j22)) i += copy(dAtA[i:], dAtA23[:j22]) } - if m.StartUid != 0 { - dAtA[i] = 0x28 - i++ - i = encodeVarintPb(dAtA, i, uint64(m.StartUid)) - } if m.XXX_unrecognized != nil { i += copy(dAtA[i:], m.XXX_unrecognized) } @@ -8390,9 +8377,6 @@ func (m *PostingList) Size() (n int) { } n += 1 + sovPb(uint64(l)) + l } - if m.StartUid != 0 { - n += 1 + sovPb(uint64(m.StartUid)) - } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -13852,25 +13836,6 @@ func (m *PostingList) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field Splits", wireType) } - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StartUid", wireType) - } - m.StartUid = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.StartUid |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skipPb(dAtA[iNdEx:]) From 184fc54388ca3eb608778c8f48dbdc12956e9bd1 Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 10 Apr 2019 19:26:04 -0700 Subject: [PATCH 50/67] Some small nitpick changes. --- posting/list.go | 10 ++++++---- posting/mvcc.go | 1 - 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/posting/list.go b/posting/list.go index d05a41eb273..eb74d4aa45c 100644 --- a/posting/list.go +++ b/posting/list.go @@ -185,7 +185,9 @@ func (it *PIterator) Next() error { return nil } - for it.splitIdx+1 < len(it.l.plist.Splits) { + for it.splitIdx <= len(it.l.plist.Splits)-2 { + // moveToNextSplit will increment it.splitIdx. Therefore, the for loop must only + // continue until len(splits) - 2. if err := it.moveToNextSplit(); err != nil { return err } @@ -1132,11 +1134,11 @@ func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { if err != nil { return nil, err } - var part pb.PostingList - if err := unmarshalOrCopy(&part, item); err != nil { + part := &pb.PostingList{} + if err := unmarshalOrCopy(part, item); err != nil { return nil, err } - return &part, nil + return part, nil } // isPlistTooBig returns true if the given plist should be split in two. diff --git a/posting/mvcc.go b/posting/mvcc.go index 62908e9985a..d67326142a5 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -166,7 +166,6 @@ func (txn *Txn) CommitToMemory(commitTs uint64) error { } func unmarshalOrCopy(plist *pb.PostingList, item *badger.Item) error { - // It's delta return item.Value(func(val []byte) error { if len(val) == 0 { // empty pl From dfdf290d7eadb29df6307cbe1d984bef879c6a1a Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Wed, 10 Apr 2019 19:43:54 -0700 Subject: [PATCH 51/67] Remove unnecessary variables to simplify code reading. Also rename some funcs. --- posting/list.go | 47 +++++++++++++++++++---------------------------- 1 file changed, 19 insertions(+), 28 deletions(-) diff --git a/posting/list.go b/posting/list.go index eb74d4aa45c..eaff3ef6a50 100644 --- a/posting/list.go +++ b/posting/list.go @@ -786,26 +786,25 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { return nil, nil } - newPlist := &pb.PostingList{ - Splits: l.plist.Splits, + out := &rollupOutput{ + newPlist: &pb.PostingList{ + Splits: l.plist.Splits, + }, + newSplits: make(map[uint64]*pb.PostingList), } - newSplits := make(map[uint64]*pb.PostingList) var plist *pb.PostingList - var final *pb.PostingList var enc codec.Encoder - var endUid uint64 + var startUid, endUid uint64 var splitIdx int - var startUid uint64 // Method to properly initialize all the variables described above. init := func() { - final = new(pb.PostingList) enc = codec.Encoder{BlockSize: blockSize} // If not a multi-part list, all uids go to the same encoder. if len(l.plist.Splits) == 0 { - plist = newPlist + plist = out.newPlist endUid = math.MaxUint64 return } @@ -825,10 +824,8 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { init() err := l.iterate(readTs, 0, func(p *pb.Posting) error { if p.Uid > endUid { - final.Pack = enc.Done() - plist.Pack = final.Pack - plist.Postings = final.Postings - newSplits[startUid] = plist + plist.Pack = enc.Done() + out.newSplits[startUid] = plist splitIdx++ init() @@ -836,17 +833,15 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { enc.Add(p.Uid) if p.Facets != nil || p.PostingType != pb.Posting_REF || len(p.Label) != 0 { - final.Postings = append(final.Postings, p) + plist.Postings = append(plist.Postings, p) } return nil }) // Finish writing the last part of the list (or the whole list if not a multi-part list). x.Check(err) - final.Pack = enc.Done() - plist.Pack = final.Pack - plist.Postings = final.Postings + plist.Pack = enc.Done() if len(l.plist.Splits) > 0 { - newSplits[startUid] = plist + out.newSplits[startUid] = plist } maxCommitTs := l.minTs @@ -864,11 +859,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { // Check if the list (or any of it's parts if it's been previously split) have // become too big. Split the list if that is the case. - out := &rollupOutput{ - newPlist: newPlist, - newSplits: newSplits, - newMinTs: maxCommitTs, - } + out.newMinTs = maxCommitTs out.splitUpList() out.cleanUpList() return out, nil @@ -1141,8 +1132,8 @@ func (l *List) readListPart(startUid uint64) (*pb.PostingList, error) { return part, nil } -// isPlistTooBig returns true if the given plist should be split in two. -func isPlistTooBig(plist *pb.PostingList) bool { +// shouldSplit returns true if the given plist should be split in two. +func shouldSplit(plist *pb.PostingList) bool { return plist.Size() >= maxListSize && len(plist.Pack.Blocks) > 1 } @@ -1169,8 +1160,8 @@ func (out *rollupOutput) splitUpList() { startUid = out.newPlist.Splits[i] } - if isPlistTooBig(list) { - startUids, pls := splitPostingList(startUid, list) + if shouldSplit(list) { + startUids, pls := binSplit(startUid, list) for i, startUid := range startUids { pl := pls[i] out.newSplits[startUid] = pl @@ -1192,10 +1183,10 @@ func (out *rollupOutput) splitUpList() { } } -// splitPostingList takes the given plist and returns two new plists, each with +// binSplit takes the given plist and returns two new plists, each with // half of the blocks and postings of the original as well as the new startUids // for each of the new parts. -func splitPostingList(startUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingList) { +func binSplit(startUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingList) { midBlock := len(plist.Pack.Blocks) / 2 midUid := plist.Pack.Blocks[midBlock].GetBase() From f4244c0146690dc9ff7e78d05fae91d0cd73f3e7 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 11 Apr 2019 12:28:46 -0700 Subject: [PATCH 52/67] Addresssed review comments. --- ee/acl/acl_curl_test.go | 2 +- posting/list.go | 84 ++++++++++++++++++++++++++--------------- 2 files changed, 54 insertions(+), 32 deletions(-) diff --git a/ee/acl/acl_curl_test.go b/ee/acl/acl_curl_test.go index a6e89616e02..a5bf1ba478a 100644 --- a/ee/acl/acl_curl_test.go +++ b/ee/acl/acl_curl_test.go @@ -22,7 +22,7 @@ import ( "github.com/stretchr/testify/require" ) -var loginEndpoint = "http://"+z.SockAddrHttp +"/login" +var loginEndpoint = "http://" + z.SockAddrHttp + "/login" func TestCurlAuthorization(t *testing.T) { if testing.Short() { diff --git a/posting/list.go b/posting/list.go index eaff3ef6a50..801310f1ca3 100644 --- a/posting/list.go +++ b/posting/list.go @@ -138,10 +138,19 @@ func (it *PIterator) Init(l *List, afterUid, deleteBelowTs uint64) error { func (it *PIterator) selectInitialSplit(afterUid uint64) int { for i, startUid := range it.l.plist.Splits { + // If startUid == afterUid, the current block should be selected. + if startUid == afterUid { + return i + } + // If this split starts at an uid greater than afterUid, there might be + // elements in the previous split that need to be checked. if startUid > afterUid { return i - 1 } } + + // In case no split's startUid is greater or equal than afterUid, start the + // iteration at the start of the last split. return len(it.l.plist.Splits) - 1 } @@ -715,13 +724,13 @@ func (l *List) Rollup() ([]*bpb.KV, error) { kv := &bpb.KV{} kv.Version = out.newMinTs kv.Key = l.key - val, meta := marshalPostingList(out.newPlist) + val, meta := marshalPostingList(out.plist) kv.UserMeta = []byte{meta} kv.Value = val kvs = append(kvs, kv) - for _, startUid := range out.newPlist.Splits { - plist := out.newSplits[startUid] + for _, startUid := range out.plist.Splits { + plist := out.splits[startUid] kv := out.marshalPostingListPart(l.key, startUid, plist) kvs = append(kvs, kv) } @@ -768,8 +777,8 @@ func marshalPostingList(plist *pb.PostingList) ([]byte, byte) { const blockSize int = 256 type rollupOutput struct { - newPlist *pb.PostingList - newSplits map[uint64]*pb.PostingList + plist *pb.PostingList + splits map[uint64]*pb.PostingList splitsToDelete map[uint64]*pb.PostingList newMinTs uint64 } @@ -787,10 +796,11 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { } out := &rollupOutput{ - newPlist: &pb.PostingList{ + plist: &pb.PostingList{ Splits: l.plist.Splits, }, - newSplits: make(map[uint64]*pb.PostingList), + splits: make(map[uint64]*pb.PostingList), + splitsToDelete: make(map[uint64]*pb.PostingList), } var plist *pb.PostingList @@ -804,7 +814,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { // If not a multi-part list, all uids go to the same encoder. if len(l.plist.Splits) == 0 { - plist = out.newPlist + plist = out.plist endUid = math.MaxUint64 return } @@ -825,7 +835,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { err := l.iterate(readTs, 0, func(p *pb.Posting) error { if p.Uid > endUid { plist.Pack = enc.Done() - out.newSplits[startUid] = plist + out.splits[startUid] = plist splitIdx++ init() @@ -841,7 +851,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { x.Check(err) plist.Pack = enc.Done() if len(l.plist.Splits) > 0 { - out.newSplits[startUid] = plist + out.splits[startUid] = plist } maxCommitTs := l.minTs @@ -1139,46 +1149,58 @@ func shouldSplit(plist *pb.PostingList) bool { // splitUpList checks the list and splits it in smaller parts if needed. func (out *rollupOutput) splitUpList() { + // Contains the posting lists that should be split. var lists []*pb.PostingList - existingSplits := false - if len(out.newPlist.Splits) == 0 { - lists = append(lists, out.newPlist) - } else { - existingSplits = true - for _, startUid := range out.newPlist.Splits { - part := out.newSplits[startUid] - lists = append(lists, part) - } + // If list is not split yet, insert the main list. + if len(out.plist.Splits) == 0 { + lists = append(lists, out.plist) + } + + // Insert the split lists if they exist. + for _, startUid := range out.plist.Splits { + part := out.splits[startUid] + lists = append(lists, part) } + // List of startUids for each list part after the splitting process is complete. var newSplits []uint64 - var newPls []*pb.PostingList + for i, list := range lists { var startUid uint64 - if existingSplits { - startUid = out.newPlist.Splits[i] + // If the list is split, select the right startUid for this list. + if len(out.plist.Splits) > 0 { + startUid = out.plist.Splits[i] } if shouldSplit(list) { + // Split the list. Update out.splits with the new lists and add their + // start uids to the list of new splits. startUids, pls := binSplit(startUid, list) for i, startUid := range startUids { pl := pls[i] - out.newSplits[startUid] = pl - newPls = append(newPls, pl) + out.splits[startUid] = pl newSplits = append(newSplits, startUid) } } else { - newPls = append(newPls, list) + // No need to split the list. Add the startUid to the array of new splits. newSplits = append(newSplits, startUid) } } - if len(newPls) == 1 || len(newPls) == len(out.newPlist.Splits) { + // Only one split after the splitting process, which means that the original list + // was not split and did not change. + if len(newSplits) == 1 { + return + } + + // The list was split before but the number of splits did not change. + if len(newSplits) == len(out.plist.Splits) { return } - out.newPlist = &pb.PostingList{ + // The splits changed so update them. + out.plist = &pb.PostingList{ Splits: newSplits, } } @@ -1219,16 +1241,16 @@ func binSplit(startUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingLi // cleanUpList marks empty splits for removal and update the split list accordingly. func (out *rollupOutput) cleanUpList() { var splits []uint64 - for _, startUid := range out.newPlist.Splits { - plist := out.newSplits[startUid] + for _, startUid := range out.plist.Splits { + plist := out.splits[startUid] if isPlistEmpty(plist) { out.splitsToDelete[startUid] = plist - delete(out.newSplits, startUid) + delete(out.splits, startUid) } else { splits = append(splits, startUid) } } - out.newPlist.Splits = splits + out.plist.Splits = splits } // isPlistEmpty returns true if the given plist is empty. Plists with splits are From 514e5e5b750b9377ae462a5abc6277d56569a7af Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 11 Apr 2019 17:42:27 -0700 Subject: [PATCH 53/67] Remove unnecessary check. --- posting/list.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/posting/list.go b/posting/list.go index 801310f1ca3..a072861ea46 100644 --- a/posting/list.go +++ b/posting/list.go @@ -1188,14 +1188,8 @@ func (out *rollupOutput) splitUpList() { } } - // Only one split after the splitting process, which means that the original list - // was not split and did not change. - if len(newSplits) == 1 { - return - } - - // The list was split before but the number of splits did not change. - if len(newSplits) == len(out.plist.Splits) { + // No new lists were created so there's no need to update the list of splits. + if len(newSplits) == len(lists) { return } From 0f13be3de7f634355b39432f0de8d1ca764b330f Mon Sep 17 00:00:00 2001 From: Manish R Jain Date: Thu, 11 Apr 2019 18:11:30 -0700 Subject: [PATCH 54/67] Manish's review. Remove delete splits var in output. Rename some fields and funcs. --- posting/list.go | 57 +++++++++++++++++-------------------------------- x/keys.go | 2 +- 2 files changed, 21 insertions(+), 38 deletions(-) diff --git a/posting/list.go b/posting/list.go index a072861ea46..a02941af00a 100644 --- a/posting/list.go +++ b/posting/list.go @@ -729,22 +729,9 @@ func (l *List) Rollup() ([]*bpb.KV, error) { kv.Value = val kvs = append(kvs, kv) - for _, startUid := range out.plist.Splits { - plist := out.splits[startUid] - kv := out.marshalPostingListPart(l.key, startUid, plist) - kvs = append(kvs, kv) - } - - // Sort the startUids in out.splitsToDelete to make the output deterministic. - var sortedSplits []uint64 - for startUid, _ := range out.splitsToDelete { - sortedSplits = append(sortedSplits, startUid) - } - sort.Slice(sortedSplits, func(i, j int) bool { - return sortedSplits[i] < sortedSplits[j] - }) - for _, startUid := range sortedSplits { - plist := out.splitsToDelete[startUid] + for startUid, plist := range out.parts { + // Any empty posting list would still have BitEmpty set. And the main posting list + // would NOT have that posting list startUid in the splits list. kv := out.marshalPostingListPart(l.key, startUid, plist) kvs = append(kvs, kv) } @@ -777,10 +764,9 @@ func marshalPostingList(plist *pb.PostingList) ([]byte, byte) { const blockSize int = 256 type rollupOutput struct { - plist *pb.PostingList - splits map[uint64]*pb.PostingList - splitsToDelete map[uint64]*pb.PostingList - newMinTs uint64 + plist *pb.PostingList + parts map[uint64]*pb.PostingList + newMinTs uint64 } // Merge all entries in mutation layer with commitTs <= l.commitTs into @@ -799,8 +785,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { plist: &pb.PostingList{ Splits: l.plist.Splits, }, - splits: make(map[uint64]*pb.PostingList), - splitsToDelete: make(map[uint64]*pb.PostingList), + parts: make(map[uint64]*pb.PostingList), } var plist *pb.PostingList @@ -835,7 +820,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { err := l.iterate(readTs, 0, func(p *pb.Posting) error { if p.Uid > endUid { plist.Pack = enc.Done() - out.splits[startUid] = plist + out.parts[startUid] = plist splitIdx++ init() @@ -851,7 +836,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { x.Check(err) plist.Pack = enc.Done() if len(l.plist.Splits) > 0 { - out.splits[startUid] = plist + out.parts[startUid] = plist } maxCommitTs := l.minTs @@ -871,7 +856,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { // become too big. Split the list if that is the case. out.newMinTs = maxCommitTs out.splitUpList() - out.cleanUpList() + out.removeEmptySplits() return out, nil } @@ -1159,7 +1144,7 @@ func (out *rollupOutput) splitUpList() { // Insert the split lists if they exist. for _, startUid := range out.plist.Splits { - part := out.splits[startUid] + part := out.parts[startUid] lists = append(lists, part) } @@ -1167,6 +1152,7 @@ func (out *rollupOutput) splitUpList() { var newSplits []uint64 for i, list := range lists { + // TODO: Make this start from one instead of zero. var startUid uint64 // If the list is split, select the right startUid for this list. if len(out.plist.Splits) > 0 { @@ -1178,8 +1164,7 @@ func (out *rollupOutput) splitUpList() { // start uids to the list of new splits. startUids, pls := binSplit(startUid, list) for i, startUid := range startUids { - pl := pls[i] - out.splits[startUid] = pl + out.parts[startUid] = pls[i] newSplits = append(newSplits, startUid) } } else { @@ -1202,7 +1187,7 @@ func (out *rollupOutput) splitUpList() { // binSplit takes the given plist and returns two new plists, each with // half of the blocks and postings of the original as well as the new startUids // for each of the new parts. -func binSplit(startUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingList) { +func binSplit(lowUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingList) { midBlock := len(plist.Pack.Blocks) / 2 midUid := plist.Pack.Blocks[midBlock].GetBase() @@ -1229,18 +1214,16 @@ func binSplit(startUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingLi } } - return []uint64{startUid, midUid}, []*pb.PostingList{lowPl, highPl} + return []uint64{lowUid, midUid}, []*pb.PostingList{lowPl, highPl} } -// cleanUpList marks empty splits for removal and update the split list accordingly. -func (out *rollupOutput) cleanUpList() { +// removeEmptySplits updates the split list by removing empty posting lists' startUids. +func (out *rollupOutput) removeEmptySplits() { var splits []uint64 + // TODO: Always iterate over out.parts and derive out.plist.Splits. for _, startUid := range out.plist.Splits { - plist := out.splits[startUid] - if isPlistEmpty(plist) { - out.splitsToDelete[startUid] = plist - delete(out.splits, startUid) - } else { + plist := out.parts[startUid] + if !isPlistEmpty(plist) { splits = append(splits, startUid) } } diff --git a/x/keys.go b/x/keys.go index dd32efdacb2..23cccf0140b 100644 --- a/x/keys.go +++ b/x/keys.go @@ -131,7 +131,7 @@ type ParsedKey struct { Attr string Uid uint64 StartUid uint64 - HasStartUid bool + HasStartUid bool // TODO: Remove this, once StartUid must be atleast 1. Term string Count uint32 bytePrefix byte From 2165095e10451c06410fa7254b0784e708a175b7 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 11 Apr 2019 18:22:55 -0700 Subject: [PATCH 55/67] Fix marshal test. --- posting/list_test.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/posting/list_test.go b/posting/list_test.go index 2dff817bdd4..bf3c12675ce 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -23,6 +23,7 @@ import ( "math" "math/rand" "os" + "sort" "strconv" "testing" @@ -1023,6 +1024,10 @@ func TestMultiPartListMarshal(t *testing.T) { require.Equal(t, len(kvs), len(ol.plist.Splits)+1) require.NoError(t, writePostingListToDisk(kvs)) + sort.Slice(kvs, func(i, j int) bool { + return string(kvs[i].Key) < string(kvs[j].Key) + }) + key := x.DataKey("multi-bal", 1331) require.Equal(t, key, kvs[0].Key) From f7debac20458cd9ed1ce790482111e3358d9454e Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 12 Apr 2019 12:30:16 -0700 Subject: [PATCH 56/67] Do not use out.plist.Splits for iteration. --- posting/list.go | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/posting/list.go b/posting/list.go index a02941af00a..53daa779c93 100644 --- a/posting/list.go +++ b/posting/list.go @@ -1143,7 +1143,7 @@ func (out *rollupOutput) splitUpList() { } // Insert the split lists if they exist. - for _, startUid := range out.plist.Splits { + for _, startUid := range out.splits() { part := out.parts[startUid] lists = append(lists, part) } @@ -1220,9 +1220,7 @@ func binSplit(lowUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingList // removeEmptySplits updates the split list by removing empty posting lists' startUids. func (out *rollupOutput) removeEmptySplits() { var splits []uint64 - // TODO: Always iterate over out.parts and derive out.plist.Splits. - for _, startUid := range out.plist.Splits { - plist := out.parts[startUid] + for startUid, plist := range out.parts { if !isPlistEmpty(plist) { splits = append(splits, startUid) } @@ -1230,6 +1228,18 @@ func (out *rollupOutput) removeEmptySplits() { out.plist.Splits = splits } +// Returns the sorted list of start uids based on the keys in out.parts. +// out.parts is considered the source of truth so this method is considered +// safer than using out.plist.Splits directly. +func (out *rollupOutput) splits() []uint64 { + var splits []uint64 + for startUid, _ := range out.parts { + splits = append(splits, startUid) + } + sortSplits(splits) + return splits +} + // isPlistEmpty returns true if the given plist is empty. Plists with splits are // considered non-empty. func isPlistEmpty(plist *pb.PostingList) bool { @@ -1241,3 +1251,9 @@ func isPlistEmpty(plist *pb.PostingList) bool { } return false } + +func sortSplits(splits []uint64) { + sort.Slice(splits, func(i, j int) bool { + return splits[i] < splits[j] + }) +} From 4c1ded916674126b6fa42f114a0f6ebce573ff49 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 12 Apr 2019 13:53:21 -0700 Subject: [PATCH 57/67] Sort splits after removing empty list parts. --- posting/list.go | 1 + 1 file changed, 1 insertion(+) diff --git a/posting/list.go b/posting/list.go index 53daa779c93..bb8ae91fb80 100644 --- a/posting/list.go +++ b/posting/list.go @@ -1226,6 +1226,7 @@ func (out *rollupOutput) removeEmptySplits() { } } out.plist.Splits = splits + sortSplits(splits) } // Returns the sorted list of start uids based on the keys in out.parts. From ba74bcecaeb9744b20d71500bc2c1572b6d2f4fc Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 12 Apr 2019 13:53:52 -0700 Subject: [PATCH 58/67] First split should start at 1 instead of zero. --- posting/list.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/posting/list.go b/posting/list.go index bb8ae91fb80..59c4d95c9a1 100644 --- a/posting/list.go +++ b/posting/list.go @@ -137,6 +137,10 @@ func (it *PIterator) Init(l *List, afterUid, deleteBelowTs uint64) error { } func (it *PIterator) selectInitialSplit(afterUid uint64) int { + if afterUid == 0 { + return 0 + } + for i, startUid := range it.l.plist.Splits { // If startUid == afterUid, the current block should be selected. if startUid == afterUid { @@ -1153,7 +1157,7 @@ func (out *rollupOutput) splitUpList() { for i, list := range lists { // TODO: Make this start from one instead of zero. - var startUid uint64 + startUid := uint64(1) // If the list is split, select the right startUid for this list. if len(out.plist.Splits) > 0 { startUid = out.plist.Splits[i] From b5b6a7f3bd96fcdfc20ccc1bacdfa9d7d3116335 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 12 Apr 2019 14:03:27 -0700 Subject: [PATCH 59/67] Remove HasStartUid field from ParsedKey. --- x/keys.go | 23 ++++++++++++++--------- x/keys_test.go | 2 -- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/x/keys.go b/x/keys.go index 23cccf0140b..32a70ee3c94 100644 --- a/x/keys.go +++ b/x/keys.go @@ -127,14 +127,13 @@ func CountKey(attr string, count uint32, reverse bool) []byte { } type ParsedKey struct { - byteType byte - Attr string - Uid uint64 - StartUid uint64 - HasStartUid bool // TODO: Remove this, once StartUid must be atleast 1. - Term string - Count uint32 - bytePrefix byte + byteType byte + Attr string + Uid uint64 + StartUid uint64 + Term string + Count uint32 + bytePrefix byte } func (p ParsedKey) IsData() bool { @@ -332,7 +331,13 @@ func Parse(key []byte) *ParsedKey { return nil } p.StartUid = binary.BigEndian.Uint64(k) - p.HasStartUid = true + if p.StartUid == 0 { + if Config.DebugMode { + fmt.Printf("Error: StartUid must be greater than 0 for key %q, parsed key: %+v\n", + key, p) + } + return nil + } case ByteIndex: p.Term = string(k) case ByteCount, ByteCountRev: diff --git a/x/keys_test.go b/x/keys_test.go index 65ff81d2dfc..efaeae77621 100644 --- a/x/keys_test.go +++ b/x/keys_test.go @@ -35,7 +35,6 @@ func TestDataKey(t *testing.T) { require.Equal(t, sattr, pk.Attr) require.Equal(t, uid, pk.Uid) require.Equal(t, uint64(0), pk.StartUid) - require.Equal(t, false, pk.HasStartUid) } keys := make([]string, 0, 1024) @@ -65,7 +64,6 @@ func TestParseKeysWithStartUid(t *testing.T) { require.Equal(t, sattr, pk.Attr) require.Equal(t, uid, pk.Uid) require.Equal(t, startUid, pk.StartUid) - require.Equal(t, true, pk.HasStartUid) } } From fb04677828faccdd4f578fc74762144c0d719eda Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 12 Apr 2019 14:06:39 -0700 Subject: [PATCH 60/67] Remove completed todo. --- posting/list.go | 1 - 1 file changed, 1 deletion(-) diff --git a/posting/list.go b/posting/list.go index 59c4d95c9a1..5455d7c59c2 100644 --- a/posting/list.go +++ b/posting/list.go @@ -1156,7 +1156,6 @@ func (out *rollupOutput) splitUpList() { var newSplits []uint64 for i, list := range lists { - // TODO: Make this start from one instead of zero. startUid := uint64(1) // If the list is split, select the right startUid for this list. if len(out.plist.Splits) > 0 { From 30c1799b2386b08c70f7e833efa967960567d6ca Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 12 Apr 2019 16:39:11 -0700 Subject: [PATCH 61/67] First split should not be removed unless entire list is empty. Add test to verify this invariant. --- posting/list.go | 79 +++++++++++++++++++-------- posting/list_test.go | 123 +++++++++++++++++++++++++++++++++++++++++-- 2 files changed, 176 insertions(+), 26 deletions(-) diff --git a/posting/list.go b/posting/list.go index 5455d7c59c2..800ec29ffc9 100644 --- a/posting/list.go +++ b/posting/list.go @@ -158,7 +158,8 @@ func (it *PIterator) selectInitialSplit(afterUid uint64) int { return len(it.l.plist.Splits) - 1 } -func (it *PIterator) moveToNextSplit() error { +// moveToNextPart re-initializes the iterator at the start of the next list part. +func (it *PIterator) moveToNextPart() error { it.splitIdx++ plist, err := it.l.readListPart(it.l.plist.Splits[it.splitIdx]) if err != nil { @@ -180,28 +181,21 @@ func (it *PIterator) moveToNextSplit() error { return nil } -func (it *PIterator) Next() error { - if it.deleteBelowTs > 0 { - it.uids = nil +// moveToNextValidPart moves the iterator to the next part that contains valid data. +// This is used to skip over parts of the list that might not contain postings. +func (it *PIterator) moveToNextValidPart() error { + // Not a multi-part list, the iterator has reached the end of the list. + if len(it.l.plist.Splits) == 0 { return nil } - it.uidx++ - if it.uidx < len(it.uids) { - return nil - } - it.uidx = 0 - it.uids = it.dec.Next() - + // If there are no more uids to iterate over, move to the next part of the + // list that contains valid data. if len(it.uids) == 0 { - if len(it.l.plist.Splits) == 0 { - return nil - } - for it.splitIdx <= len(it.l.plist.Splits)-2 { - // moveToNextSplit will increment it.splitIdx. Therefore, the for loop must only + // moveToNextPart will increment it.splitIdx. Therefore, the for loop must only // continue until len(splits) - 2. - if err := it.moveToNextSplit(); err != nil { + if err := it.moveToNextPart(); err != nil { return err } @@ -210,12 +204,36 @@ func (it *PIterator) Next() error { } } } - return nil } -func (it *PIterator) Valid() bool { - return len(it.uids) > 0 +func (it *PIterator) Next() error { + if it.deleteBelowTs > 0 { + it.uids = nil + return nil + } + + it.uidx++ + if it.uidx < len(it.uids) { + return nil + } + it.uidx = 0 + it.uids = it.dec.Next() + + return it.moveToNextValidPart() +} + +func (it *PIterator) Valid() (bool, error) { + if len(it.uids) > 0 { + return true, nil + } + + if err := it.moveToNextValidPart(); err != nil { + return false, err + } else if len(it.uids) > 0 { + return true, nil + } + return false, nil } func (it *PIterator) Posting() *pb.Posting { @@ -633,7 +651,9 @@ func (l *List) iterate(readTs uint64, afterUid uint64, f func(obj *pb.Posting) e } else { mp = emptyPosting } - if pitr.Valid() { + if valid, err := pitr.Valid(); err != nil { + return err + } else if valid { pp = pitr.Posting() } else { pp = emptyPosting @@ -808,7 +828,7 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { return } - // Otherwise, load the corresponding part and set endUid to correctly + // Otherwise, load the corresponding part and set endUid to correctly // detect the end of the list. startUid = l.plist.Splits[splitIdx] if splitIdx+1 == len(l.plist.Splits) { @@ -1224,12 +1244,27 @@ func binSplit(lowUid uint64, plist *pb.PostingList) ([]uint64, []*pb.PostingList func (out *rollupOutput) removeEmptySplits() { var splits []uint64 for startUid, plist := range out.parts { + // Do not remove the first split for now, as every multi-part list should always + // have a split starting with uid 1. + if startUid == 1 { + splits = append(splits, startUid) + continue + } + if !isPlistEmpty(plist) { splits = append(splits, startUid) } } out.plist.Splits = splits sortSplits(splits) + + if len(out.plist.Splits) == 1 { + // Only the first split remains. If it's also empty, remove it as well. + // This should mark the entire list for deletion. + if isPlistEmpty(out.parts[1]) { + out.plist.Splits = []uint64{} + } + } } // Returns the sorted list of start uids based on the keys in out.parts. diff --git a/posting/list_test.go b/posting/list_test.go index bf3c12675ce..6c09025029f 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -19,7 +19,6 @@ package posting import ( "context" "io/ioutil" - "log" "math" "math/rand" "os" @@ -907,7 +906,7 @@ func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { func createAndDeleteMultiPartList(t *testing.T, size int) (*List, int) { // For testing, set the max list size to a lower threshold. - maxListSize = math.MaxInt32 + maxListSize = 5000 defer func() { maxListSize = math.MaxInt32 }() @@ -966,6 +965,7 @@ func writePostingListToDisk(kvs []*bpb.KV) error { return writer.Flush() } +// Create a multi-part list and verify all the uids are there. func TestMultiPartListBasic(t *testing.T) { size := int(1e5) ol, commits := createMultiPartList(t, size, false) @@ -979,6 +979,7 @@ func TestMultiPartListBasic(t *testing.T) { } } +// Verify that iteration works with an afterUid value greater than zero. func TestMultiPartListIterAfterUid(t *testing.T) { size := int(1e5) ol, _ := createMultiPartList(t, size, false) @@ -995,6 +996,7 @@ func TestMultiPartListIterAfterUid(t *testing.T) { } } +// Verify that postings can be retrieved in multi-part lists. func TestMultiPartListWithPostings(t *testing.T) { size := int(1e5) ol, commits := createMultiPartList(t, size, true) @@ -1014,6 +1016,7 @@ func TestMultiPartListWithPostings(t *testing.T) { } } +// Verify marshaling of multi-part lists. func TestMultiPartListMarshal(t *testing.T) { size := int(1e5) ol, _ := createMultiPartList(t, size, false) @@ -1044,7 +1047,8 @@ func TestMultiPartListMarshal(t *testing.T) { } } -func TestMultiPartListWrite(t *testing.T) { +// Verify that writing a multi-part list to disk works correctly. +func TestMultiPartListWriteToDisk(t *testing.T) { size := int(1e5) originalList, commits := createMultiPartList(t, size, false) @@ -1067,6 +1071,7 @@ func TestMultiPartListWrite(t *testing.T) { } } +// Verify that adding and deleting all the entries returns an empty list. func TestMultiPartListDelete(t *testing.T) { size := int(1e4) ol, commits := createAndDeleteMultiPartList(t, size) @@ -1075,7 +1080,6 @@ func TestMultiPartListDelete(t *testing.T) { counter := 0 ol.Iterate(math.MaxUint64, 0, func(p *pb.Posting) error { - log.Printf("%v", p.Uid) counter++ return nil }) @@ -1091,6 +1095,117 @@ func TestMultiPartListDelete(t *testing.T) { } } +// Verify that the first part of a multi-part list is kept even when all its +// entries have been deleted. Do this by creating a list, deleting the first +// half, and ensuring iteration and mutation still work as expected. +func TestMultiPartListDeleteAndAdd(t *testing.T) { + size := int(1e5) + // For testing, set the max list size to a lower threshold. + maxListSize = 5000 + defer func() { + maxListSize = math.MaxInt32 + }() + + // Add entries to the maps. + key := x.DataKey("del_add", 1331) + ol, err := getNew(key, ps) + require.NoError(t, err) + for i := 1; i <= size; i++ { + edge := &pb.DirectedEdge{ + ValueId: uint64(i), + } + + txn := Txn{StartTs: uint64(i)} + addMutationHelper(t, ol, edge, Set, &txn) + require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) + if i%2000 == 0 { + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + } + } + + // Verify all entries are in the list. + opt := ListOptions{ReadTs: math.MaxUint64} + l, err := ol.Uids(opt) + require.NoError(t, err) + require.Equal(t, size, len(l.Uids), "List of Uids received: %+v", l.Uids) + for i, uid := range l.Uids { + require.Equal(t, uint64(i+1), uid) + } + + // Delete the first half of the previously inserted entries from the list. + baseStartTs := uint64(size) + 1 + for i := 1; i <= 50000; i++ { + edge := &pb.DirectedEdge{ + ValueId: uint64(i), + } + txn := Txn{StartTs: baseStartTs + uint64(i)} + addMutationHelper(t, ol, edge, Del, &txn) + require.NoError(t, ol.CommitMutation(baseStartTs+uint64(i), baseStartTs+uint64(i)+1)) + if i%2000 == 0 { + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + } + } + + // Rollup list at the end of all the deletions. + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + + // Verify that the entries were actually deleted. + opt = ListOptions{ReadTs: math.MaxUint64} + l, err = ol.Uids(opt) + require.NoError(t, err) + require.Equal(t, 50000, len(l.Uids), "List of Uids received: %+v", l.Uids) + for i, uid := range l.Uids { + require.Equal(t, 50000+uint64(i+1), uid) + } + + // Re-add the entries that were just deleted. + baseStartTs = uint64(2*size) + 1 + for i := 1; i <= 50000; i++ { + edge := &pb.DirectedEdge{ + ValueId: uint64(i), + } + txn := Txn{StartTs: baseStartTs + uint64(i)} + addMutationHelper(t, ol, edge, Set, &txn) + require.NoError(t, ol.CommitMutation(baseStartTs+uint64(i), baseStartTs+uint64(i)+1)) + + if i%2000 == 0 { + kvs, err := ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + } + } + + // Rollup list at the end of all the additions + kvs, err = ol.Rollup() + require.NoError(t, err) + require.NoError(t, writePostingListToDisk(kvs)) + ol, err = getNew(key, ps) + require.NoError(t, err) + + // Verify all entries are once again in the list. + opt = ListOptions{ReadTs: math.MaxUint64} + l, err = ol.Uids(opt) + require.NoError(t, err) + require.Equal(t, size, len(l.Uids), "List of Uids received: %+v", l.Uids) + for i, uid := range l.Uids { + require.Equal(t, uint64(i+1), uid) + } +} + var ps *badger.DB func TestMain(m *testing.M) { From c13b4cb2b45988da60d6d2be9e1bb1978009b8a1 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 12 Apr 2019 17:05:58 -0700 Subject: [PATCH 62/67] Add comment to Rollup --- posting/list.go | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/posting/list.go b/posting/list.go index 800ec29ffc9..cfcc7834c32 100644 --- a/posting/list.go +++ b/posting/list.go @@ -733,6 +733,25 @@ func (l *List) Length(readTs, afterUid uint64) int { return l.length(readTs, afterUid) } +// Rollup performs the rollup process, merging the immutable and mutable layers +// and outputting the resulting list so it can be written to disk. +// During this process, the list might be split into multiple lists if the main +// list or any of the existing parts become too big. +// +// A normal list has the following format: +// -> +// +// A multi-part list is stored in multiple keys. The keys for the parts will be generated by +// appending the first uid in the part to the key. The list will have the following format: +// -> +// -> +// -> +// ... +// -> +// +// The first part of a multi-part list always has start uid 1 and will be the last part +// to be deleted, at which point the entire list will be marked for deletion. +// As the list grows, existing parts might be split if they become too big. func (l *List) Rollup() ([]*bpb.KV, error) { l.RLock() defer l.RUnlock() From 99cc5cb186342e17ea791920950ab5b6dbebe35f Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 12 Apr 2019 18:13:12 -0700 Subject: [PATCH 63/67] Fix debug tool. --- dgraph/cmd/debug/run.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dgraph/cmd/debug/run.go b/dgraph/cmd/debug/run.go index 8ec378554a3..0bfb2d17a51 100644 --- a/dgraph/cmd/debug/run.go +++ b/dgraph/cmd/debug/run.go @@ -112,7 +112,7 @@ func uidToVal(itr *badger.Iterator, prefix string) map[uint64]int { if pk.IsSchema() { continue } - if pk.HasStartUid { + if pk.StartUid > 0 { // This key is part of a multi-part posting list. Skip it and only read // the main key, which is the entry point to read the whole list. continue @@ -506,7 +506,7 @@ func printKeys(db *badger.DB) { if pk.Uid > 0 { fmt.Fprintf(&buf, " uid: %d ", pk.Uid) } - if pk.HasStartUid { + if pk.StartUid >0 { fmt.Fprintf(&buf, " startUid: %d ", pk.StartUid) } fmt.Fprintf(&buf, " key: %s", hex.EncodeToString(item.Key())) From 2adc1389b047e73b93528f5ac3cb57be2f6b154e Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Mon, 15 Apr 2019 10:47:04 -0700 Subject: [PATCH 64/67] go fmt. --- dgraph/cmd/debug/run.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dgraph/cmd/debug/run.go b/dgraph/cmd/debug/run.go index 0bfb2d17a51..7af32335ed2 100644 --- a/dgraph/cmd/debug/run.go +++ b/dgraph/cmd/debug/run.go @@ -506,7 +506,7 @@ func printKeys(db *badger.DB) { if pk.Uid > 0 { fmt.Fprintf(&buf, " uid: %d ", pk.Uid) } - if pk.StartUid >0 { + if pk.StartUid > 0 { fmt.Fprintf(&buf, " startUid: %d ", pk.StartUid) } fmt.Fprintf(&buf, " key: %s", hex.EncodeToString(item.Key())) From bca3c05c67bd92eba8bfbb9d10865557d9f80509 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Mon, 15 Apr 2019 11:15:32 -0700 Subject: [PATCH 65/67] Copied comment. --- posting/list.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/posting/list.go b/posting/list.go index c0e45716992..aadc692ed4e 100644 --- a/posting/list.go +++ b/posting/list.go @@ -890,8 +890,10 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { // postings which had deletions to provide a sorted view of the list. Therefore, the safest // way to get the max commit timestamp is to pick all the relevant postings for the given // readTs and calculate the maxCommitTs. - deleteBelow, mposts := l.pickPostings(readTs) - maxCommitTs = x.Max(maxCommitTs, deleteBelow) + // If deleteBelowTs is greater than zero, there was a delete all marker. The list of postings + // has been trimmed down. + deleteBelowTs, mposts := l.pickPostings(readTs) + maxCommitTs = x.Max(maxCommitTs, deleteBelowTs) for _, mp := range mposts { maxCommitTs = x.Max(maxCommitTs, mp.CommitTs) } From 51786ad1a5c0bb58581f1b3685d2d12d6208a76c Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Thu, 18 Apr 2019 10:54:53 -0700 Subject: [PATCH 66/67] Fix comment length. --- posting/list.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/posting/list.go b/posting/list.go index aadc692ed4e..3cac5198a29 100644 --- a/posting/list.go +++ b/posting/list.go @@ -890,8 +890,8 @@ func (l *List) rollup(readTs uint64) (*rollupOutput, error) { // postings which had deletions to provide a sorted view of the list. Therefore, the safest // way to get the max commit timestamp is to pick all the relevant postings for the given // readTs and calculate the maxCommitTs. - // If deleteBelowTs is greater than zero, there was a delete all marker. The list of postings - // has been trimmed down. + // If deleteBelowTs is greater than zero, there was a delete all marker. The list of + // postings has been trimmed down. deleteBelowTs, mposts := l.pickPostings(readTs) maxCommitTs = x.Max(maxCommitTs, deleteBelowTs) for _, mp := range mposts { From 76e1a540b5026c54a5d7356dc7f9e5082108d534 Mon Sep 17 00:00:00 2001 From: Martin Martinez Rivera Date: Fri, 19 Apr 2019 11:35:32 -0700 Subject: [PATCH 67/67] Remove extra file --- list_test_backup.go | 1257 ------------------------------------------- 1 file changed, 1257 deletions(-) delete mode 100644 list_test_backup.go diff --git a/list_test_backup.go b/list_test_backup.go deleted file mode 100644 index 8e4e1e746dc..00000000000 --- a/list_test_backup.go +++ /dev/null @@ -1,1257 +0,0 @@ -/* - * Copyright 2015-2018 Dgraph Labs, Inc. and Contributors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package posting - -import ( - "context" - "io/ioutil" - "math" - "math/rand" - "os" - "sort" - "strconv" - "testing" - - "github.com/dgraph-io/badger" - bpb "github.com/dgraph-io/badger/pb" - "github.com/stretchr/testify/require" - - "github.com/dgraph-io/dgraph/protos/pb" - "github.com/dgraph-io/dgraph/schema" - "github.com/dgraph-io/dgraph/x" -) - -func (l *List) PostingList() *pb.PostingList { - l.RLock() - defer l.RUnlock() - return l.plist -} - -func listToArray(t *testing.T, afterUid uint64, l *List, readTs uint64) []uint64 { - out := make([]uint64, 0, 10) - l.Iterate(readTs, afterUid, func(p *pb.Posting) error { - out = append(out, p.Uid) - return nil - }) - return out -} - -func checkUids(t *testing.T, l *List, uids []uint64, readTs uint64) { - require.Equal(t, uids, listToArray(t, 0, l, readTs)) - if len(uids) >= 3 { - require.Equal(t, uids[1:], listToArray(t, 10, l, readTs), uids[1:]) - require.Equal(t, []uint64{81}, listToArray(t, 80, l, readTs)) - require.Empty(t, listToArray(t, 82, l, readTs)) - } -} - -func addMutationHelper(t *testing.T, l *List, edge *pb.DirectedEdge, op uint32, txn *Txn) { - if op == Del { - edge.Op = pb.DirectedEdge_DEL - } else if op == Set { - edge.Op = pb.DirectedEdge_SET - } else { - x.Fatalf("Unhandled op: %v", op) - } - err := l.AddMutation(context.Background(), txn, edge) - require.NoError(t, err) -} - -func TestAddMutation(t *testing.T) { - key := x.DataKey("name", 2) - - txn := NewTxn(1) - l, err := txn.Get(key) - require.NoError(t, err) - - edge := &pb.DirectedEdge{ - ValueId: 9, - Label: "testing", - } - addMutationHelper(t, l, edge, Set, txn) - - require.Equal(t, listToArray(t, 0, l, 1), []uint64{9}) - - p := getFirst(l, 1) - require.NotNil(t, p, "Unable to retrieve posting") - require.EqualValues(t, p.Label, "testing") - - // Add another edge now. - edge.ValueId = 81 - addMutationHelper(t, l, edge, Set, txn) - require.Equal(t, listToArray(t, 0, l, 1), []uint64{9, 81}) - - // Add another edge, in between the two above. - edge.ValueId = 49 - addMutationHelper(t, l, edge, Set, txn) - require.Equal(t, listToArray(t, 0, l, 1), []uint64{9, 49, 81}) - - checkUids(t, l, []uint64{9, 49, 81}, 1) - - // Delete an edge, add an edge, replace an edge - edge.ValueId = 49 - addMutationHelper(t, l, edge, Del, txn) - - edge.ValueId = 69 - addMutationHelper(t, l, edge, Set, txn) - - edge.ValueId = 9 - edge.Label = "anti-testing" - addMutationHelper(t, l, edge, Set, txn) - l.CommitMutation(1, 2) - - uids := []uint64{9, 69, 81} - checkUids(t, l, uids, 3) - - p = getFirst(l, 3) - require.NotNil(t, p, "Unable to retrieve posting") - require.EqualValues(t, "anti-testing", p.Label) -} - -func getFirst(l *List, readTs uint64) (res pb.Posting) { - l.Iterate(readTs, 0, func(p *pb.Posting) error { - res = *p - return ErrStopIteration - }) - return res -} - -func checkValue(t *testing.T, ol *List, val string, readTs uint64) { - p := getFirst(ol, readTs) - require.Equal(t, uint64(math.MaxUint64), p.Uid) // Cast to prevent overflow. - require.EqualValues(t, val, p.Value) -} - -// TODO(txn): Add tests after lru eviction -func TestAddMutation_Value(t *testing.T) { - key := x.DataKey("value", 10) - ol, err := getNew(key, ps) - require.NoError(t, err) - edge := &pb.DirectedEdge{ - Value: []byte("oh hey there"), - Label: "new-testing", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - checkValue(t, ol, "oh hey there", txn.StartTs) - - // Run the same check after committing. - ol.CommitMutation(txn.StartTs, txn.StartTs+1) - checkValue(t, ol, "oh hey there", uint64(3)) - - // The value made it to the posting list. Changing it now. - edge.Value = []byte(strconv.Itoa(119)) - txn = &Txn{StartTs: 3} - addMutationHelper(t, ol, edge, Set, txn) - checkValue(t, ol, "119", txn.StartTs) -} - -func TestAddMutation_jchiu1(t *testing.T) { - key := x.DataKey("value", 12) - ol, err := GetNoStore(key) - require.NoError(t, err) - - // Set value to cars and merge to BadgerDB. - edge := &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(1, uint64(2)) - - // TODO: Read at commitTimestamp with all committed - require.EqualValues(t, 1, ol.Length(uint64(3), 0)) - checkValue(t, ol, "cars", uint64(3)) - - txn = &Txn{StartTs: 3} - // Set value to newcars, but don't merge yet. - edge = &pb.DirectedEdge{ - Value: []byte("newcars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Set, txn) - require.EqualValues(t, 1, ol.Length(txn.StartTs, 0)) - checkValue(t, ol, "newcars", txn.StartTs) - - // Set value to someothercars, but don't merge yet. - edge = &pb.DirectedEdge{ - Value: []byte("someothercars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Set, txn) - require.EqualValues(t, 1, ol.Length(txn.StartTs, 0)) - checkValue(t, ol, "someothercars", txn.StartTs) - - // Set value back to the committed value cars, but don't merge yet. - edge = &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Set, txn) - require.EqualValues(t, 1, ol.Length(txn.StartTs, 0)) - checkValue(t, ol, "cars", txn.StartTs) -} - -func TestAddMutation_DelSet(t *testing.T) { - key := x.DataKey("value", 1534) - ol, err := GetNoStore(key) - require.NoError(t, err) - - // DO sp*, don't commit - // Del a value cars and but don't merge. - edge := &pb.DirectedEdge{ - Value: []byte(x.Star), - Op: pb.DirectedEdge_DEL, - } - txn := &Txn{StartTs: 1} - err = ol.AddMutation(context.Background(), txn, edge) - require.NoError(t, err) - - // Set value to newcars, commit it - edge = &pb.DirectedEdge{ - Value: []byte("newcars"), - } - txn = &Txn{StartTs: 2} - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(2, uint64(3)) - require.EqualValues(t, 1, ol.Length(3, 0)) - checkValue(t, ol, "newcars", 3) -} - -func TestAddMutation_DelRead(t *testing.T) { - key := x.DataKey("value", 1543) - ol, err := GetNoStore(key) - require.NoError(t, err) - - // Set value to newcars, and commit it - edge := &pb.DirectedEdge{ - Value: []byte("newcars"), - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(1, uint64(2)) - require.EqualValues(t, 1, ol.Length(2, 0)) - checkValue(t, ol, "newcars", 2) - - // DO sp*, don't commit - // Del a value cars and but don't merge. - edge = &pb.DirectedEdge{ - Value: []byte(x.Star), - Op: pb.DirectedEdge_DEL, - } - txn = &Txn{StartTs: 3} - err = ol.AddMutation(context.Background(), txn, edge) - require.NoError(t, err) - - // Part of same transaction as sp*, so should see zero length even - // if not committed yet. - require.EqualValues(t, 0, ol.Length(3, 0)) - - // Commit sp* only in oracle, don't apply to pl yet - ol.CommitMutation(3, 5) - - // This read should ignore sp*, since readts is 4 and it was committed at 5 - require.EqualValues(t, 1, ol.Length(4, 0)) - checkValue(t, ol, "newcars", 4) - - require.EqualValues(t, 0, ol.Length(6, 0)) -} - -func TestAddMutation_jchiu2(t *testing.T) { - key := x.DataKey("value", 15) - ol, err := GetNoStore(key) - require.NoError(t, err) - - // Del a value cars and but don't merge. - edge := &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Del, txn) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 0)) - - // Set value to newcars, but don't merge yet. - edge = &pb.DirectedEdge{ - Value: []byte("newcars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Set, txn) - require.EqualValues(t, 1, ol.Length(txn.StartTs, 0)) - checkValue(t, ol, "newcars", txn.StartTs) -} - -func TestAddMutation_jchiu2_Commit(t *testing.T) { - key := x.DataKey("value", 16) - ol, err := GetNoStore(key) - require.NoError(t, err) - - // Del a value cars and but don't merge. - edge := &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Del, txn) - ol.CommitMutation(1, uint64(2)) - require.EqualValues(t, 0, ol.Length(uint64(3), 0)) - - // Set value to newcars, but don't merge yet. - edge = &pb.DirectedEdge{ - Value: []byte("newcars"), - Label: "jchiu", - } - txn = &Txn{StartTs: 3} - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(3, uint64(4)) - require.EqualValues(t, 1, ol.Length(5, 0)) - checkValue(t, ol, "newcars", 5) -} - -func TestAddMutation_jchiu3(t *testing.T) { - key := x.DataKey("value", 29) - ol, err := GetNoStore(key) - require.NoError(t, err) - - // Set value to cars and merge to BadgerDB. - edge := &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(1, uint64(2)) - require.Equal(t, 1, ol.Length(uint64(3), 0)) - require.EqualValues(t, 1, ol.Length(uint64(3), 0)) - checkValue(t, ol, "cars", uint64(3)) - - // Del a value cars and but don't merge. - edge = &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - txn = &Txn{StartTs: 3} - addMutationHelper(t, ol, edge, Del, txn) - require.Equal(t, 0, ol.Length(txn.StartTs, 0)) - - // Set value to newcars, but don't merge yet. - edge = &pb.DirectedEdge{ - Value: []byte("newcars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Set, txn) - require.EqualValues(t, 1, ol.Length(txn.StartTs, 0)) - checkValue(t, ol, "newcars", txn.StartTs) - - // Del a value newcars and but don't merge. - edge = &pb.DirectedEdge{ - Value: []byte("newcars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Del, txn) - require.Equal(t, 0, ol.Length(txn.StartTs, 0)) -} - -func TestAddMutation_mrjn1(t *testing.T) { - key := x.DataKey("value", 21) - ol, err := GetNoStore(key) - require.NoError(t, err) - - // Set a value cars and merge. - edge := &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(1, uint64(2)) - - // Delete the previously committed value cars. But don't merge. - txn = &Txn{StartTs: 3} - edge = &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Del, txn) - require.Equal(t, 0, ol.Length(txn.StartTs, 0)) - - // Do this again to cover Del, muid == curUid, inPlist test case. - // Delete the previously committed value cars. But don't merge. - edge = &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Del, txn) - require.Equal(t, 0, ol.Length(txn.StartTs, 0)) - - // Set the value again to cover Set, muid == curUid, inPlist test case. - // Set the previously committed value cars. But don't merge. - edge = &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Set, txn) - checkValue(t, ol, "cars", txn.StartTs) - - // Delete it again, just for fun. - edge = &pb.DirectedEdge{ - Value: []byte("cars"), - Label: "jchiu", - } - addMutationHelper(t, ol, edge, Del, txn) - require.Equal(t, 0, ol.Length(txn.StartTs, 0)) -} - -func TestMillion(t *testing.T) { - // Ensure list is stored in a single part. - maxListSize = math.MaxInt32 - - key := x.DataKey("bal", 1331) - ol, err := getNew(key, ps) - require.NoError(t, err) - var commits int - N := int(1e6) - for i := 2; i <= N; i += 2 { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - } - txn := Txn{StartTs: uint64(i)} - addMutationHelper(t, ol, edge, Set, &txn) - require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%10000 == 0 { - // Do a rollup, otherwise, it gets too slow to add a million mutations to one posting - // list. - t.Logf("Start Ts: %d. Rolling up posting list.\n", txn.StartTs) - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - } - commits++ - } - - t.Logf("Completed a million writes.\n") - opt := ListOptions{ReadTs: uint64(N) + 1} - l, err := ol.Uids(opt) - require.NoError(t, err) - require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) - for i, uid := range l.Uids { - require.Equal(t, uint64(i+1)*2, uid) - } -} - -// Test the various mutate, commit and abort sequences. -func TestAddMutation_mrjn2(t *testing.T) { - ctx := context.Background() - key := x.DataKey("bal", 1001) - ol, err := getNew(key, ps) - require.NoError(t, err) - var readTs uint64 - for readTs = 1; readTs < 10; readTs++ { - edge := &pb.DirectedEdge{ - ValueId: readTs, - ValueType: pb.Posting_INT, - } - txn := &Txn{StartTs: readTs} - addMutationHelper(t, ol, edge, Set, txn) - } - for i := 1; i < 10; i++ { - // Each of these txns see their own write. - opt := ListOptions{ReadTs: uint64(i)} - list, err := ol.Uids(opt) - require.NoError(t, err) - require.EqualValues(t, 1, len(list.Uids)) - require.EqualValues(t, uint64(i), list.Uids[0]) - } - require.EqualValues(t, 0, ol.Length(readTs, 0)) - require.NoError(t, ol.CommitMutation(1, 0)) - require.NoError(t, ol.CommitMutation(3, 4)) - require.NoError(t, ol.CommitMutation(6, 10)) - require.NoError(t, ol.CommitMutation(9, 14)) - require.EqualValues(t, 3, ol.Length(15, 0)) // The three commits. - - { - edge := &pb.DirectedEdge{ - Value: []byte(x.Star), - Op: pb.DirectedEdge_DEL, - } - txn := &Txn{StartTs: 7} - err := ol.AddMutation(ctx, txn, edge) - require.NoError(t, err) - - // Add edge just to test that the deletion still happens. - edge = &pb.DirectedEdge{ - ValueId: 7, - ValueType: pb.Posting_INT, - } - err = ol.AddMutation(ctx, txn, edge) - require.NoError(t, err) - - require.EqualValues(t, 3, ol.Length(15, 0)) // The three commits should still be found. - require.NoError(t, ol.CommitMutation(7, 11)) - - require.EqualValues(t, 2, ol.Length(10, 0)) // Two commits should be found. - require.EqualValues(t, 1, ol.Length(12, 0)) // Only one commit should be found. - require.EqualValues(t, 2, ol.Length(15, 0)) // Only one commit should be found. - } - { - edge := &pb.DirectedEdge{ - Value: []byte(x.Star), - Op: pb.DirectedEdge_DEL, - } - txn := &Txn{StartTs: 5} - err := ol.AddMutation(ctx, txn, edge) - require.NoError(t, err) - require.NoError(t, ol.CommitMutation(5, 7)) - - // Commits are: - // 4, 7 (Delete *), 10, 11 (Delete *), 14 - require.EqualValues(t, 1, ol.Length(8, 0)) // Nothing below 8, but consider itself. - require.NoError(t, ol.CommitMutation(8, 0)) - require.EqualValues(t, 0, ol.Length(8, 0)) // Nothing <= 8. - require.EqualValues(t, 1, ol.Length(10, 0)) // Find committed 10. - require.EqualValues(t, 1, ol.Length(12, 0)) // Find committed 11. - require.EqualValues(t, 2, ol.Length(15, 0)) // Find committed 14. - opts := ListOptions{ReadTs: 15} - list, err := ol.Uids(opts) - require.NoError(t, err) - require.EqualValues(t, 7, list.Uids[0]) - require.EqualValues(t, 9, list.Uids[1]) - } -} - -func TestAddMutation_gru(t *testing.T) { - key := x.DataKey("question.tag", 0x01) - ol, err := getNew(key, ps) - require.NoError(t, err) - - { - // Set two tag ids and merge. - edge := &pb.DirectedEdge{ - ValueId: 0x2b693088816b04b7, - Label: "gru", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - edge = &pb.DirectedEdge{ - ValueId: 0x29bf442b48a772e0, - Label: "gru", - } - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(1, uint64(2)) - } - - { - edge := &pb.DirectedEdge{ - ValueId: 0x38dec821d2ac3a79, - Label: "gru", - } - txn := &Txn{StartTs: 3} - addMutationHelper(t, ol, edge, Set, txn) - edge = &pb.DirectedEdge{ - ValueId: 0x2b693088816b04b7, - Label: "gru", - } - addMutationHelper(t, ol, edge, Del, txn) - ol.CommitMutation(3, uint64(4)) - } -} - -func TestAddMutation_gru2(t *testing.T) { - key := x.DataKey("question.tag", 0x100) - ol, err := getNew(key, ps) - require.NoError(t, err) - - { - // Set two tag ids and merge. - edge := &pb.DirectedEdge{ - ValueId: 0x02, - Label: "gru", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - edge = &pb.DirectedEdge{ - ValueId: 0x03, - Label: "gru", - } - txn = &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(1, uint64(2)) - } - - { - // Lets set a new tag and delete the two older ones. - edge := &pb.DirectedEdge{ - ValueId: 0x02, - Label: "gru", - } - txn := &Txn{StartTs: 3} - addMutationHelper(t, ol, edge, Del, txn) - edge = &pb.DirectedEdge{ - ValueId: 0x03, - Label: "gru", - } - addMutationHelper(t, ol, edge, Del, txn) - - edge = &pb.DirectedEdge{ - ValueId: 0x04, - Label: "gru", - } - addMutationHelper(t, ol, edge, Set, txn) - - ol.CommitMutation(3, uint64(4)) - } - - // Posting list should just have the new tag. - uids := []uint64{0x04} - require.Equal(t, uids, listToArray(t, 0, ol, uint64(5))) -} - -func TestAddAndDelMutation(t *testing.T) { - // Ensure each test uses unique key since we don't clear the postings - // after each test - key := x.DataKey("dummy_key", 0x927) - ol, err := getNew(key, ps) - require.NoError(t, err) - - { - edge := &pb.DirectedEdge{ - ValueId: 0x02, - Label: "gru", - } - txn := &Txn{StartTs: 1} - addMutationHelper(t, ol, edge, Set, txn) - ol.CommitMutation(1, uint64(2)) - } - - { - edge := &pb.DirectedEdge{ - ValueId: 0x02, - Label: "gru", - } - txn := &Txn{StartTs: 3} - addMutationHelper(t, ol, edge, Del, txn) - addMutationHelper(t, ol, edge, Del, txn) - ol.CommitMutation(3, uint64(4)) - - checkUids(t, ol, []uint64{}, 5) - } - checkUids(t, ol, []uint64{}, 5) -} - -func TestAfterUIDCount(t *testing.T) { - key := x.DataKey("value", 22) - ol, err := getNew(key, ps) - require.NoError(t, err) - // Set value to cars and merge to BadgerDB. - edge := &pb.DirectedEdge{ - Label: "jchiu", - } - - txn := &Txn{StartTs: 1} - for i := 100; i < 300; i++ { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 200, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 100, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Delete half of the edges. - for i := 100; i < 300; i += 2 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Del, txn) - } - require.EqualValues(t, 100, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 50, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Try to delete half of the edges. Redundant deletes. - for i := 100; i < 300; i += 2 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Del, txn) - } - require.EqualValues(t, 100, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 50, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Delete everything. - for i := 100; i < 300; i++ { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Del, txn) - } - require.EqualValues(t, 0, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Insert 1/4 of the edges. - for i := 100; i < 300; i += 4 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 50, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 25, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Insert 1/4 of the edges. - edge.Label = "somethingelse" - for i := 100; i < 300; i += 4 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 50, ol.Length(txn.StartTs, 0)) // Expect no change. - require.EqualValues(t, 25, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Insert 1/4 of the edges. - for i := 103; i < 300; i += 4 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 100, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 50, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) -} - -func TestAfterUIDCount2(t *testing.T) { - key := x.DataKey("value", 23) - ol, err := getNew(key, ps) - require.NoError(t, err) - - // Set value to cars and merge to BadgerDB. - edge := &pb.DirectedEdge{ - Label: "jchiu", - } - - txn := &Txn{StartTs: 1} - for i := 100; i < 300; i++ { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 200, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 100, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Re-insert 1/4 of the edges. Counts should not change. - edge.Label = "somethingelse" - for i := 100; i < 300; i += 4 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 200, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 100, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) -} - -func TestDelete(t *testing.T) { - key := x.DataKey("value", 25) - ol, err := getNew(key, ps) - require.NoError(t, err) - - // Set value to cars and merge to BadgerDB. - edge := &pb.DirectedEdge{ - Label: "jchiu", - } - - txn := &Txn{StartTs: 1} - for i := 1; i <= 30; i++ { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 30, ol.Length(txn.StartTs, 0)) - edge.Value = []byte(x.Star) - addMutationHelper(t, ol, edge, Del, txn) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 0)) - ol.CommitMutation(txn.StartTs, txn.StartTs+1) - - require.EqualValues(t, 0, ol.Length(txn.StartTs+2, 0)) -} - -func TestAfterUIDCountWithCommit(t *testing.T) { - key := x.DataKey("value", 26) - ol, err := getNew(key, ps) - require.NoError(t, err) - - // Set value to cars and merge to BadgerDB. - edge := &pb.DirectedEdge{ - Label: "jchiu", - } - - txn := &Txn{StartTs: 1} - for i := 100; i < 400; i++ { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 300, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 200, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 400)) - - // Commit to database. - ol.CommitMutation(txn.StartTs, txn.StartTs+1) - - txn = &Txn{StartTs: 3} - // Mutation layer starts afresh from here. - // Delete half of the edges. - for i := 100; i < 400; i += 2 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Del, txn) - } - require.EqualValues(t, 150, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 100, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 400)) - - // Try to delete half of the edges. Redundant deletes. - for i := 100; i < 400; i += 2 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Del, txn) - } - require.EqualValues(t, 150, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 100, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 400)) - - // Delete everything. - for i := 100; i < 400; i++ { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Del, txn) - } - require.EqualValues(t, 0, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 400)) - - // Insert 1/4 of the edges. - for i := 100; i < 300; i += 4 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 50, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 25, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Insert 1/4 of the edges. - edge.Label = "somethingelse" - for i := 100; i < 300; i += 4 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 50, ol.Length(txn.StartTs, 0)) // Expect no change. - require.EqualValues(t, 25, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) - - // Insert 1/4 of the edges. - for i := 103; i < 300; i += 4 { - edge.ValueId = uint64(i) - addMutationHelper(t, ol, edge, Set, txn) - } - require.EqualValues(t, 100, ol.Length(txn.StartTs, 0)) - require.EqualValues(t, 50, ol.Length(txn.StartTs, 199)) - require.EqualValues(t, 0, ol.Length(txn.StartTs, 300)) -} - -func createMultiPartList(t *testing.T, size int, addLabel bool) (*List, int) { - // For testing, set the max list size to a lower threshold. - maxListSize = 5000 - defer func() { - maxListSize = math.MaxInt32 - }() - - key := x.DataKey("multi-bal", 1331) - ol, err := getNew(key, ps) - require.NoError(t, err) - commits := 0 - for i := 1; i <= size; i++ { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - } - if addLabel { - edge.Label = strconv.Itoa(i) - } - - txn := Txn{StartTs: uint64(i)} - addMutationHelper(t, ol, edge, Set, &txn) - require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%2000 == 0 { - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - } - commits++ - } - - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - - return ol, commits -} - -func createAndDeleteMultiPartList(t *testing.T, size int) (*List, int) { - // For testing, set the max list size to a lower threshold. - maxListSize = 5000 - defer func() { - maxListSize = math.MaxInt32 - }() - - key := x.DataKey("bal_del", 1331) - ol, err := getNew(key, ps) - require.NoError(t, err) - commits := 0 - for i := 1; i <= size; i++ { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - } - - txn := Txn{StartTs: uint64(i)} - addMutationHelper(t, ol, edge, Set, &txn) - require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%2000 == 0 { - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - } - commits++ - } - - // Delete all the previously inserted entries from the list. - baseStartTs := uint64(size) + 1 - for i := 1; i <= size; i++ { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - } - txn := Txn{StartTs: baseStartTs + uint64(i)} - addMutationHelper(t, ol, edge, Del, &txn) - require.NoError(t, ol.CommitMutation(baseStartTs+uint64(i), baseStartTs+uint64(i)+1)) - if i%2000 == 0 { - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - } - commits++ - } - - return ol, commits -} - -func writePostingListToDisk(kvs []*bpb.KV) error { - writer := NewTxnWriter(pstore) - for _, kv := range kvs { - if err := writer.SetAt(kv.Key, kv.Value, kv.UserMeta[0], kv.Version); err != nil { - return err - } - } - return writer.Flush() -} - -// Create a multi-part list and verify all the uids are there. -func TestMultiPartListBasic(t *testing.T) { - size := int(1e5) - ol, commits := createMultiPartList(t, size, false) - t.Logf("List parts %v", len(ol.plist.Splits)) - opt := ListOptions{ReadTs: uint64(size) + 1} - l, err := ol.Uids(opt) - require.NoError(t, err) - require.Equal(t, commits, len(l.Uids), "List of Uids received: %+v", l.Uids) - for i, uid := range l.Uids { - require.Equal(t, uint64(i+1), uid) - } -} - -// Verify that iteration works with an afterUid value greater than zero. -func TestMultiPartListIterAfterUid(t *testing.T) { - size := int(1e5) - ol, _ := createMultiPartList(t, size, false) - t.Logf("List parts %v", len(ol.plist.Splits)) - - var visitedUids []uint64 - ol.Iterate(uint64(size+1), 50000, func(p *pb.Posting) error { - visitedUids = append(visitedUids, p.Uid) - return nil - }) - require.Equal(t, 50000, len(visitedUids)) - for i, uid := range visitedUids { - require.Equal(t, uint64(50000+i+1), uid) - } -} - -// Verify that postings can be retrieved in multi-part lists. -func TestMultiPartListWithPostings(t *testing.T) { - size := int(1e5) - ol, commits := createMultiPartList(t, size, true) - t.Logf("List parts %v", len(ol.plist.Splits)) - - var labels []string - err := ol.Iterate(uint64(size)+1, 0, func(p *pb.Posting) error { - if len(p.Label) > 0 { - labels = append(labels, p.Label) - } - return nil - }) - require.NoError(t, err) - require.Equal(t, commits, len(labels)) - for i, label := range labels { - require.Equal(t, label, strconv.Itoa(int(i+1))) - } -} - -// Verify marshaling of multi-part lists. -func TestMultiPartListMarshal(t *testing.T) { - size := int(1e5) - ol, _ := createMultiPartList(t, size, false) - t.Logf("List parts %v", len(ol.plist.Splits)) - - kvs, err := ol.Rollup() - require.NoError(t, err) - require.Equal(t, len(kvs), len(ol.plist.Splits)+1) - require.NoError(t, writePostingListToDisk(kvs)) - - sort.Slice(kvs, func(i, j int) bool { - return string(kvs[i].Key) < string(kvs[j].Key) - }) - - key := x.DataKey("multi-bal", 1331) - require.Equal(t, key, kvs[0].Key) - - for i, startUid := range ol.plist.Splits { - partKey := x.GetSplitKey(key, startUid) - require.Equal(t, partKey, kvs[i+1].Key) - part, err := ol.readListPart(startUid) - require.NoError(t, err) - data, err := part.Marshal() - require.NoError(t, err) - require.Equal(t, data, kvs[i+1].Value) - require.Equal(t, []byte{BitCompletePosting}, kvs[i+1].UserMeta) - require.Equal(t, ol.minTs, kvs[i+1].Version) - } -} - -// Verify that writing a multi-part list to disk works correctly. -func TestMultiPartListWriteToDisk(t *testing.T) { - size := int(1e5) - originalList, commits := createMultiPartList(t, size, false) - - kvs, err := originalList.Rollup() - require.NoError(t, err) - require.Equal(t, len(kvs), len(originalList.plist.Splits)+1) - - require.NoError(t, writePostingListToDisk(kvs)) - newList, err := getNew(kvs[0].Key, ps) - - opt := ListOptions{ReadTs: uint64(size) + 1} - originalUids, err := originalList.Uids(opt) - require.NoError(t, err) - newUids, err := newList.Uids(opt) - require.NoError(t, err) - require.Equal(t, commits, len(originalUids.Uids)) - require.Equal(t, len(originalUids.Uids), len(newUids.Uids)) - for i, _ := range originalUids.Uids { - require.Equal(t, originalUids.Uids[i], newUids.Uids[i]) - } -} - -// Verify that adding and deleting all the entries returns an empty list. -func TestMultiPartListDelete(t *testing.T) { - size := int(1e4) - ol, commits := createAndDeleteMultiPartList(t, size) - t.Logf("List parts %v", len(ol.plist.Splits)) - require.Equal(t, size*2, commits) - - counter := 0 - ol.Iterate(math.MaxUint64, 0, func(p *pb.Posting) error { - counter++ - return nil - }) - require.Equal(t, 0, counter) - - kvs, err := ol.Rollup() - require.NoError(t, err) - require.Equal(t, len(kvs), 1) - - for _, kv := range kvs { - require.Equal(t, []byte{BitEmptyPosting}, kv.UserMeta) - require.Equal(t, ol.minTs, kv.Version) - } -} - -// Verify that the first part of a multi-part list is kept even when all its -// entries have been deleted. Do this by creating a list, deleting the first -// half, and ensuring iteration and mutation still work as expected. -func TestMultiPartListDeleteAndAdd(t *testing.T) { - size := int(1e5) - // For testing, set the max list size to a lower threshold. - maxListSize = 5000 - defer func() { - maxListSize = math.MaxInt32 - }() - - // Add entries to the maps. - key := x.DataKey("del_add", 1331) - ol, err := getNew(key, ps) - require.NoError(t, err) - for i := 1; i <= size; i++ { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - } - - txn := Txn{StartTs: uint64(i)} - addMutationHelper(t, ol, edge, Set, &txn) - require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1)) - if i%2000 == 0 { - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - } - } - - // Verify all entries are in the list. - opt := ListOptions{ReadTs: math.MaxUint64} - l, err := ol.Uids(opt) - require.NoError(t, err) - require.Equal(t, size, len(l.Uids), "List of Uids received: %+v", l.Uids) - for i, uid := range l.Uids { - require.Equal(t, uint64(i+1), uid) - } - - // Delete the first half of the previously inserted entries from the list. - baseStartTs := uint64(size) + 1 - for i := 1; i <= 50000; i++ { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - } - txn := Txn{StartTs: baseStartTs + uint64(i)} - addMutationHelper(t, ol, edge, Del, &txn) - require.NoError(t, ol.CommitMutation(baseStartTs+uint64(i), baseStartTs+uint64(i)+1)) - if i%2000 == 0 { - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - } - } - - // Rollup list at the end of all the deletions. - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - - // Verify that the entries were actually deleted. - opt = ListOptions{ReadTs: math.MaxUint64} - l, err = ol.Uids(opt) - require.NoError(t, err) - require.Equal(t, 50000, len(l.Uids), "List of Uids received: %+v", l.Uids) - for i, uid := range l.Uids { - require.Equal(t, 50000+uint64(i+1), uid) - } - - // Re-add the entries that were just deleted. - baseStartTs = uint64(2*size) + 1 - for i := 1; i <= 50000; i++ { - edge := &pb.DirectedEdge{ - ValueId: uint64(i), - } - txn := Txn{StartTs: baseStartTs + uint64(i)} - addMutationHelper(t, ol, edge, Set, &txn) - require.NoError(t, ol.CommitMutation(baseStartTs+uint64(i), baseStartTs+uint64(i)+1)) - - if i%2000 == 0 { - kvs, err := ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - } - } - - // Rollup list at the end of all the additions - kvs, err = ol.Rollup() - require.NoError(t, err) - require.NoError(t, writePostingListToDisk(kvs)) - ol, err = getNew(key, ps) - require.NoError(t, err) - - // Verify all entries are once again in the list. - opt = ListOptions{ReadTs: math.MaxUint64} - l, err = ol.Uids(opt) - require.NoError(t, err) - require.Equal(t, size, len(l.Uids), "List of Uids received: %+v", l.Uids) - for i, uid := range l.Uids { - require.Equal(t, uint64(i+1), uid) - } -} - -var ps *badger.DB - -func TestMain(m *testing.M) { - x.Init() - Config.AllottedMemory = 1024.0 - Config.CommitFraction = 0.10 - - dir, err := ioutil.TempDir("", "storetest_") - x.Check(err) - - opt := badger.DefaultOptions - opt.Dir = dir - opt.ValueDir = dir - ps, err = badger.OpenManaged(opt) - x.Check(err) - Init(ps) - schema.Init(ps) - - r := m.Run() - - os.RemoveAll(dir) - os.Exit(r) -} - -func BenchmarkAddMutations(b *testing.B) { - key := x.DataKey("name", 1) - l, err := getNew(key, ps) - if err != nil { - b.Error(err) - } - b.ResetTimer() - - ctx := context.Background() - for i := 0; i < b.N; i++ { - if err != nil { - b.Error(err) - return - } - edge := &pb.DirectedEdge{ - ValueId: uint64(rand.Intn(b.N) + 1), - Label: "testing", - Op: pb.DirectedEdge_SET, - } - txn := &Txn{StartTs: 1} - if err = l.AddMutation(ctx, txn, edge); err != nil { - b.Error(err) - } - } -}