From daf42d6b8c87cbd3c0cf831b0cd8f7b97fafd3f5 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 14 Jan 2021 18:12:51 -0500 Subject: [PATCH 1/2] kv: support locking Get requests This commit adds support for Get requests to acquire unreplicated locks in cases where they find a present key-value. This is supported by a new field on GetRequest called KeyLocking, which mirrors the field of the same name found on ScanRequest and ReverseScanRequest. Much of this commit was modeled after a98be1f, which was the original commit that added locking support to ScanRequest and ReverseScanRequest. The intention is to use this functionality to fix #46639. --- pkg/kv/batch.go | 34 +- pkg/kv/db.go | 14 + pkg/kv/db_test.go | 13 + pkg/kv/kvclient/kvcoord/dist_sender_test.go | 12 +- .../kvclient/kvcoord/txn_coord_sender_test.go | 13 +- pkg/kv/kvserver/batcheval/cmd_get.go | 15 +- pkg/kv/kvserver/batcheval/result/intent.go | 11 - pkg/kv/kvserver/replica_evaluate_test.go | 49 +- pkg/kv/txn.go | 16 +- pkg/roachpb/api.go | 29 +- pkg/roachpb/api.pb.go | 1282 +++++++++-------- pkg/roachpb/api.proto | 7 + pkg/roachpb/batch_test.go | 18 +- 13 files changed, 832 insertions(+), 681 deletions(-) diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 407e14f5327c..d64600816e09 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -335,23 +335,39 @@ func (b *Batch) AddRawRequest(reqs ...roachpb.Request) { } } -// Get retrieves the value for a key. A new result will be appended to the -// batch which will contain a single row. -// -// r, err := db.Get("a") -// // string(r.Rows[0].Key) == "a" -// -// key can be either a byte slice or a string. -func (b *Batch) Get(key interface{}) { +func (b *Batch) get(key interface{}, forUpdate bool) { k, err := marshalKey(key) if err != nil { b.initResult(0, 1, notRaw, err) return } - b.appendReqs(roachpb.NewGet(k)) + b.appendReqs(roachpb.NewGet(k, forUpdate)) b.initResult(1, 1, notRaw, nil) } +// Get retrieves the value for a key. A new result will be appended to the batch +// which will contain a single row. +// +// r, err := db.Get("a") +// // string(r.Rows[0].Key) == "a" +// +// key can be either a byte slice or a string. +func (b *Batch) Get(key interface{}) { + b.get(key, false /* forUpdate */) +} + +// GetForUpdate retrieves the value for a key. An unreplicated, exclusive lock +// is acquired on the key, if it exists. A new result will be appended to the +// batch which will contain a single row. +// +// r, err := db.GetForUpdate("a") +// // string(r.Rows[0].Key) == "a" +// +// key can be either a byte slice or a string. +func (b *Batch) GetForUpdate(key interface{}) { + b.get(key, true /* forUpdate */) +} + func (b *Batch) put(key, value interface{}, inline bool) { k, err := marshalKey(key) if err != nil { diff --git a/pkg/kv/db.go b/pkg/kv/db.go index 3794fe5e2752..9a499bd45cd0 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -317,6 +317,20 @@ func (db *DB) Get(ctx context.Context, key interface{}) (KeyValue, error) { return getOneRow(db.Run(ctx, b), b) } +// GetForUpdate retrieves the value for a key, returning the retrieved key/value +// or an error. An unreplicated, exclusive lock is acquired on the key, if it +// exists. It is not considered an error for the key not to exist. +// +// r, err := db.GetForUpdate("a") +// // string(r.Key) == "a" +// +// key can be either a byte slice or a string. +func (db *DB) GetForUpdate(ctx context.Context, key interface{}) (KeyValue, error) { + b := &Batch{} + b.GetForUpdate(key) + return getOneRow(db.Run(ctx, b), b) +} + // GetProto retrieves the value for a key and decodes the result as a proto // message. If the key doesn't exist, the proto will simply be reset. // diff --git a/pkg/kv/db_test.go b/pkg/kv/db_test.go index d39e52106b27..5f54eeb1bfed 100644 --- a/pkg/kv/db_test.go +++ b/pkg/kv/db_test.go @@ -81,6 +81,19 @@ func TestDB_Get(t *testing.T) { checkResult(t, []byte(""), result.ValueBytes()) } +func TestDB_GetForUpdate(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + s, db := setup(t) + defer s.Stopper().Stop(context.Background()) + + result, err := db.GetForUpdate(context.Background(), "aa") + if err != nil { + t.Fatal(err) + } + checkResult(t, []byte(""), result.ValueBytes()) +} + func TestDB_Put(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index 4869167f33b6..247343dc6742 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -887,7 +887,7 @@ func TestDistSenderMovesOnFromReplicaWithStaleLease(t *testing.T) { Lease: cachedLease, }) - get := roachpb.NewGet(roachpb.Key("a")) + get := roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */) _, pErr := kv.SendWrapped(ctx, ds, get) require.Nil(t, pErr) @@ -1768,7 +1768,7 @@ func TestDistSenderDescriptorUpdatesOnSuccessfulRPCs(t *testing.T) { // Send a request that's going to receive a response with a RangeInfo. k := roachpb.Key("a") - get := roachpb.NewGet(k) + get := roachpb.NewGet(k, false /* forUpdate */) var ba roachpb.BatchRequest ba.Add(get) _, pErr := ds.Send(ctx, ba) @@ -1879,7 +1879,7 @@ func TestSendRPCRangeNotFoundError(t *testing.T) { Settings: cluster.MakeTestingClusterSettings(), } ds = NewDistSender(cfg) - get := roachpb.NewGet(roachpb.Key("b")) + get := roachpb.NewGet(roachpb.Key("b"), false /* forUpdate */) _, err := kv.SendWrapped(ctx, ds, get) if err != nil { t.Fatal(err) @@ -3488,19 +3488,19 @@ func TestCanSendToFollower(t *testing.T) { roachpb.Header{ Txn: &roachpb.Transaction{}, }, - roachpb.NewGet(roachpb.Key("a")), + roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */), 1, }, { true, roachpb.Header{}, - roachpb.NewGet(roachpb.Key("a")), + roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */), 1, }, { false, roachpb.Header{}, - roachpb.NewGet(roachpb.Key("a")), + roachpb.NewGet(roachpb.Key("a"), false /* forUpdate */), 2, }, } { diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index 7e266cf2cd72..9f8c5a2d0369 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -963,7 +963,7 @@ func TestTxnCoordSenderNoDuplicateLockSpans(t *testing.T) { db := kv.NewDB(ambient, factory, clock, stopper) txn := kv.NewTxn(ctx, db, 0 /* gatewayNodeID */) - // Acquire locks on a-b, c, u-w before the final batch. + // Acquire locks on a-b, c, m, u-w before the final batch. _, pErr := txn.ReverseScanForUpdate(ctx, roachpb.Key("a"), roachpb.Key("b"), 0) if pErr != nil { t.Fatal(pErr) @@ -972,23 +972,30 @@ func TestTxnCoordSenderNoDuplicateLockSpans(t *testing.T) { if pErr != nil { t.Fatal(pErr) } + _, pErr = txn.GetForUpdate(ctx, roachpb.Key("m")) + if pErr != nil { + t.Fatal(pErr) + } pErr = txn.DelRange(ctx, roachpb.Key("u"), roachpb.Key("w")) if pErr != nil { t.Fatal(pErr) } - // The final batch overwrites key c and overlaps part of the a-b and u-w ranges. + // The final batch overwrites key c, reads key n, and overlaps part of the a-b and u-w ranges. b := txn.NewBatch() b.Put(roachpb.Key("b"), []byte("value")) b.Put(roachpb.Key("c"), []byte("value")) b.Put(roachpb.Key("d"), []byte("value")) + b.GetForUpdate(roachpb.Key("n")) b.ReverseScanForUpdate(roachpb.Key("v"), roachpb.Key("z")) - // The expected locks are a-b, c, and u-z. + // The expected locks are a-b, c, m, n, and u-z. expectedLockSpans = []roachpb.Span{ {Key: roachpb.Key("a"), EndKey: roachpb.Key("b").Next()}, {Key: roachpb.Key("c"), EndKey: nil}, {Key: roachpb.Key("d"), EndKey: nil}, + {Key: roachpb.Key("m"), EndKey: nil}, + {Key: roachpb.Key("n"), EndKey: nil}, {Key: roachpb.Key("u"), EndKey: roachpb.Key("z")}, } diff --git a/pkg/kv/kvserver/batcheval/cmd_get.go b/pkg/kv/kvserver/batcheval/cmd_get.go index ffa57985415f..0c5b8645f6bb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get.go +++ b/pkg/kv/kvserver/batcheval/cmd_get.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -32,8 +33,9 @@ func Get( reply := resp.(*roachpb.GetResponse) val, intent, err := storage.MVCCGet(ctx, reader, args.Key, h.Timestamp, storage.MVCCGetOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, - Txn: h.Txn, + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + FailOnMoreRecent: args.KeyLocking != lock.None, }) if err != nil { return result.Result{}, err @@ -61,5 +63,12 @@ func Get( } } } - return result.FromEncounteredIntents(intents), err + + var res result.Result + if args.KeyLocking != lock.None && h.Txn != nil && val != nil { + acq := roachpb.MakeLockAcquisition(h.Txn, args.Key, lock.Unreplicated) + res.Local.AcquiredLocks = []roachpb.LockAcquisition{acq} + } + res.Local.EncounteredIntents = intents + return res, err } diff --git a/pkg/kv/kvserver/batcheval/result/intent.go b/pkg/kv/kvserver/batcheval/result/intent.go index 1dc98f4a92a8..8fb548be8365 100644 --- a/pkg/kv/kvserver/batcheval/result/intent.go +++ b/pkg/kv/kvserver/batcheval/result/intent.go @@ -15,17 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" ) -// FromEncounteredIntents creates a Result communicating that the intents were -// encountered and should be handled. -func FromEncounteredIntents(intents []roachpb.Intent) Result { - var pd Result - if len(intents) == 0 { - return pd - } - pd.Local.EncounteredIntents = intents - return pd -} - // FromAcquiredLocks creates a Result communicating that the locks were // acquired or re-acquired by the given transaction and should be handled. func FromAcquiredLocks(txn *roachpb.Transaction, keys ...roachpb.Key) Result { diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index 7c7ca676940f..ddab99e42c97 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -254,6 +254,45 @@ func TestEvaluateBatch(t *testing.T) { // // Test suite for KeyLocking. // + { + // Two gets, one of which finds a key, one of which does not. An + // unreplicated lock should be acquired on the key that existed. + name: "gets with key locking", + setup: func(t *testing.T, d *data) { + writeABCDEFAt(t, d, ts.Prev()) + scanA := getArgsString("a") + scanA.KeyLocking = lock.Exclusive + d.ba.Add(scanA) + scanG := getArgsString("g") + scanG.KeyLocking = lock.Exclusive + d.ba.Add(scanG) + d.ba.Txn = &txn + }, + check: func(t *testing.T, r resp) { + verifyScanResult(t, r, []string{"a"}, nil) + verifyAcquiredLocks(t, r, lock.Unreplicated, "a") + verifyAcquiredLocks(t, r, lock.Replicated, []string(nil)...) + }, + }, + { + // Two gets, one of which finds a key, one of which does not. No + // transaction set, so no locks should be acquired. + name: "gets with key locking without txn", + setup: func(t *testing.T, d *data) { + writeABCDEFAt(t, d, ts.Prev()) + scanA := getArgsString("a") + scanA.KeyLocking = lock.Exclusive + d.ba.Add(scanA) + scanG := getArgsString("g") + scanG.KeyLocking = lock.Exclusive + d.ba.Add(scanG) + }, + check: func(t *testing.T, r resp) { + verifyScanResult(t, r, []string{"a"}, nil) + verifyAcquiredLocks(t, r, lock.Unreplicated, []string(nil)...) + verifyAcquiredLocks(t, r, lock.Replicated, []string(nil)...) + }, + }, { // Three scans that observe 3, 1, and 0 keys, respectively. An // unreplicated lock should be acquired on each key that is scanned. @@ -651,10 +690,12 @@ func verifyScanResult(t *testing.T, r resp, keysPerResp ...[]string) { rows = req.Rows case *roachpb.GetResponse: isGet = true - rows = []roachpb.KeyValue{{ - Key: r.d.ba.Requests[i].GetGet().Key, - Value: *req.Value, - }} + if req.Value != nil { + rows = []roachpb.KeyValue{{ + Key: r.d.ba.Requests[i].GetGet().Key, + Value: *req.Value, + }} + } default: } diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 2e4f5b611a5a..071ae667b1bc 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -358,7 +358,7 @@ func (txn *Txn) NewBatch() *Batch { // Get retrieves the value for a key, returning the retrieved key/value or an // error. It is not considered an error for the key to not exist. // -// r, err := db.Get("a") +// r, err := txn.Get("a") // // string(r.Key) == "a" // // key can be either a byte slice or a string. @@ -368,6 +368,20 @@ func (txn *Txn) Get(ctx context.Context, key interface{}) (KeyValue, error) { return getOneRow(txn.Run(ctx, b), b) } +// GetForUpdate retrieves the value for a key, returning the retrieved key/value +// or an error. An unreplicated, exclusive lock is acquired on the key, if it +// exists. It is not considered an error for the key to not exist. +// +// r, err := txn.GetForUpdate("a") +// // string(r.Key) == "a" +// +// key can be either a byte slice or a string. +func (txn *Txn) GetForUpdate(ctx context.Context, key interface{}) (KeyValue, error) { + b := txn.NewBatch() + b.GetForUpdate(key) + return getOneRow(txn.Run(ctx, b), b) +} + // GetProto retrieves the value for a key and decodes the result as a proto // message. If the key doesn't exist, the proto will simply be reset. // diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 688d2b7f6d2e..7ac6f5a545cf 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -984,12 +984,15 @@ func (r *AdminVerifyProtectedTimestampRequest) ShallowCopy() Request { return &shallowCopy } -// NewGet returns a Request initialized to get the value at key. -func NewGet(key Key) Request { +// NewGet returns a Request initialized to get the value at key. If +// forUpdate is true, an unreplicated, exclusive lock is acquired on on +// the key, if it exists. +func NewGet(key Key, forUpdate bool) Request { return &GetRequest{ RequestHeader: RequestHeader{ Key: key, }, + KeyLocking: scanLockStrength(forUpdate), } } @@ -1156,8 +1159,16 @@ func scanLockStrength(forUpdate bool) lock.Strength { return lock.None } -func (*GetRequest) flags() int { - return isRead | isTxn | updatesTSCache | needsRefresh +func flagForLockStrength(l lock.Strength) int { + if l != lock.None { + return isLocking + } + return 0 +} + +func (gr *GetRequest) flags() int { + maybeLocking := flagForLockStrength(gr.KeyLocking) + return isRead | isTxn | maybeLocking | updatesTSCache | needsRefresh } func (*PutRequest) flags() int { @@ -1231,18 +1242,12 @@ func (*ClearRangeRequest) flags() int { return isWrite | isRange | isAlone } func (*RevertRangeRequest) flags() int { return isWrite | isRange } func (sr *ScanRequest) flags() int { - maybeLocking := 0 - if sr.KeyLocking != lock.None { - maybeLocking = isLocking - } + maybeLocking := flagForLockStrength(sr.KeyLocking) return isRead | isRange | isTxn | maybeLocking | updatesTSCache | needsRefresh } func (rsr *ReverseScanRequest) flags() int { - maybeLocking := 0 - if rsr.KeyLocking != lock.None { - maybeLocking = isLocking - } + maybeLocking := flagForLockStrength(rsr.KeyLocking) return isRead | isRange | isReverse | isTxn | maybeLocking | updatesTSCache | needsRefresh } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index d148317cf906..91aed7404b5f 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -74,7 +74,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{0} + return fileDescriptor_api_a0e30b34ee568726, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -102,7 +102,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{1} + return fileDescriptor_api_a0e30b34ee568726, []int{1} } type ChecksumMode int32 @@ -149,7 +149,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{2} + return fileDescriptor_api_a0e30b34ee568726, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -180,7 +180,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{3} + return fileDescriptor_api_a0e30b34ee568726, []int{3} } type ExternalStorageProvider int32 @@ -221,7 +221,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{4} + return fileDescriptor_api_a0e30b34ee568726, []int{4} } type MVCCFilter int32 @@ -244,7 +244,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{5} + return fileDescriptor_api_a0e30b34ee568726, []int{5} } type ResponseHeader_ResumeReason int32 @@ -270,7 +270,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{1, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -312,7 +312,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{25, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -333,7 +333,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{0} + return fileDescriptor_api_a0e30b34ee568726, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -399,7 +399,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{1} + return fileDescriptor_api_a0e30b34ee568726, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -427,13 +427,19 @@ var xxx_messageInfo_ResponseHeader proto.InternalMessageInfo // A GetRequest is the argument for the Get() method. type GetRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + // The desired key-level locking mode used during this get. When set to None + // (the default), no key-level locking mode is used - meaning that the get + // does not acquire a lock. When set to any other strength, a lock of that + // strength is acquired with the Unreplicated durability (i.e. best-effort) + // the key, if it exists. + KeyLocking lock.Strength `protobuf:"varint,2,opt,name=key_locking,json=keyLocking,proto3,enum=cockroach.kv.kvserver.concurrency.lock.Strength" json:"key_locking,omitempty"` } func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{2} + return fileDescriptor_api_a0e30b34ee568726, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -476,7 +482,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{3} + return fileDescriptor_api_a0e30b34ee568726, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -519,7 +525,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{4} + return fileDescriptor_api_a0e30b34ee568726, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -553,7 +559,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{5} + return fileDescriptor_api_a0e30b34ee568726, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -642,7 +648,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{6} + return fileDescriptor_api_a0e30b34ee568726, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -677,7 +683,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{7} + return fileDescriptor_api_a0e30b34ee568726, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -723,7 +729,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{8} + return fileDescriptor_api_a0e30b34ee568726, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -757,7 +763,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{9} + return fileDescriptor_api_a0e30b34ee568726, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -797,7 +803,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{10} + return fileDescriptor_api_a0e30b34ee568726, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -834,7 +840,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{11} + return fileDescriptor_api_a0e30b34ee568726, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -868,7 +874,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{12} + return fileDescriptor_api_a0e30b34ee568726, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -902,7 +908,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{13} + return fileDescriptor_api_a0e30b34ee568726, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -954,7 +960,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{14} + return fileDescriptor_api_a0e30b34ee568726, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -991,7 +997,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{15} + return fileDescriptor_api_a0e30b34ee568726, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1046,7 +1052,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{16} + return fileDescriptor_api_a0e30b34ee568726, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1080,7 +1086,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{17} + return fileDescriptor_api_a0e30b34ee568726, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1122,7 +1128,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{18} + return fileDescriptor_api_a0e30b34ee568726, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1156,7 +1162,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{19} + return fileDescriptor_api_a0e30b34ee568726, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1207,7 +1213,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{20} + return fileDescriptor_api_a0e30b34ee568726, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1259,7 +1265,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{21} + return fileDescriptor_api_a0e30b34ee568726, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1310,7 +1316,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{22} + return fileDescriptor_api_a0e30b34ee568726, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1362,7 +1368,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{23} + return fileDescriptor_api_a0e30b34ee568726, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1415,7 +1421,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{24} + return fileDescriptor_api_a0e30b34ee568726, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1452,7 +1458,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{25} + return fileDescriptor_api_a0e30b34ee568726, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1496,7 +1502,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{25, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1544,7 +1550,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{26} + return fileDescriptor_api_a0e30b34ee568726, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1580,7 +1586,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{27} + return fileDescriptor_api_a0e30b34ee568726, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1691,7 +1697,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{28} + return fileDescriptor_api_a0e30b34ee568726, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1737,7 +1743,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{29} + return fileDescriptor_api_a0e30b34ee568726, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1798,7 +1804,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{30} + return fileDescriptor_api_a0e30b34ee568726, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1833,7 +1839,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{31} + return fileDescriptor_api_a0e30b34ee568726, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1872,7 +1878,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{32} + return fileDescriptor_api_a0e30b34ee568726, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1907,7 +1913,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{33} + return fileDescriptor_api_a0e30b34ee568726, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1950,7 +1956,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{34} + return fileDescriptor_api_a0e30b34ee568726, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1985,7 +1991,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{35} + return fileDescriptor_api_a0e30b34ee568726, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2023,7 +2029,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{36} + return fileDescriptor_api_a0e30b34ee568726, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2056,7 +2062,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{37} + return fileDescriptor_api_a0e30b34ee568726, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2091,7 +2097,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{38} + return fileDescriptor_api_a0e30b34ee568726, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2149,7 +2155,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{39} + return fileDescriptor_api_a0e30b34ee568726, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2184,7 +2190,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{40} + return fileDescriptor_api_a0e30b34ee568726, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2222,7 +2228,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{41} + return fileDescriptor_api_a0e30b34ee568726, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2255,7 +2261,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{42} + return fileDescriptor_api_a0e30b34ee568726, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2297,7 +2303,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{43} + return fileDescriptor_api_a0e30b34ee568726, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2334,7 +2340,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{44} + return fileDescriptor_api_a0e30b34ee568726, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2372,7 +2378,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{45} + return fileDescriptor_api_a0e30b34ee568726, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2406,7 +2412,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{45, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2440,7 +2446,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{46} + return fileDescriptor_api_a0e30b34ee568726, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2509,7 +2515,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{47} + return fileDescriptor_api_a0e30b34ee568726, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2552,7 +2558,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{48} + return fileDescriptor_api_a0e30b34ee568726, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2599,7 +2605,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{49} + return fileDescriptor_api_a0e30b34ee568726, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2635,7 +2641,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{50} + return fileDescriptor_api_a0e30b34ee568726, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2679,7 +2685,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{51} + return fileDescriptor_api_a0e30b34ee568726, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2723,7 +2729,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{52} + return fileDescriptor_api_a0e30b34ee568726, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2783,7 +2789,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{53} + return fileDescriptor_api_a0e30b34ee568726, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2819,7 +2825,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{54} + return fileDescriptor_api_a0e30b34ee568726, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2866,7 +2872,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{55} + return fileDescriptor_api_a0e30b34ee568726, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2901,7 +2907,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{56} + return fileDescriptor_api_a0e30b34ee568726, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2952,7 +2958,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{57} + return fileDescriptor_api_a0e30b34ee568726, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2987,7 +2993,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{58} + return fileDescriptor_api_a0e30b34ee568726, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3024,7 +3030,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{59} + return fileDescriptor_api_a0e30b34ee568726, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3058,7 +3064,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{60} + return fileDescriptor_api_a0e30b34ee568726, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3103,7 +3109,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{61} + return fileDescriptor_api_a0e30b34ee568726, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3137,7 +3143,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{62} + return fileDescriptor_api_a0e30b34ee568726, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3181,7 +3187,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{63} + return fileDescriptor_api_a0e30b34ee568726, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3230,7 +3236,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{64} + return fileDescriptor_api_a0e30b34ee568726, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3267,7 +3273,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{65} + return fileDescriptor_api_a0e30b34ee568726, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3304,7 +3310,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{66} + return fileDescriptor_api_a0e30b34ee568726, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3339,7 +3345,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{67} + return fileDescriptor_api_a0e30b34ee568726, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3394,7 +3400,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{68} + return fileDescriptor_api_a0e30b34ee568726, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3431,7 +3437,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{69} + return fileDescriptor_api_a0e30b34ee568726, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3471,7 +3477,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{70} + return fileDescriptor_api_a0e30b34ee568726, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3505,7 +3511,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3538,7 +3544,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 1} + return fileDescriptor_api_a0e30b34ee568726, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3580,7 +3586,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 2} + return fileDescriptor_api_a0e30b34ee568726, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3619,7 +3625,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 3} + return fileDescriptor_api_a0e30b34ee568726, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3655,7 +3661,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 4} + return fileDescriptor_api_a0e30b34ee568726, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3694,7 +3700,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 5} + return fileDescriptor_api_a0e30b34ee568726, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3736,7 +3742,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 6} + return fileDescriptor_api_a0e30b34ee568726, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3776,7 +3782,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{71} + return fileDescriptor_api_a0e30b34ee568726, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3810,7 +3816,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{72} + return fileDescriptor_api_a0e30b34ee568726, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3846,7 +3852,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{73} + return fileDescriptor_api_a0e30b34ee568726, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3916,7 +3922,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{74} + return fileDescriptor_api_a0e30b34ee568726, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3968,7 +3974,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{75} + return fileDescriptor_api_a0e30b34ee568726, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4004,7 +4010,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{76} + return fileDescriptor_api_a0e30b34ee568726, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4044,7 +4050,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{76, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4095,7 +4101,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{77} + return fileDescriptor_api_a0e30b34ee568726, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4130,7 +4136,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{77, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4166,7 +4172,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{77, 1} + return fileDescriptor_api_a0e30b34ee568726, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4201,7 +4207,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{78} + return fileDescriptor_api_a0e30b34ee568726, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4239,7 +4245,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{79} + return fileDescriptor_api_a0e30b34ee568726, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4276,7 +4282,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{80} + return fileDescriptor_api_a0e30b34ee568726, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4309,7 +4315,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{80, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4354,7 +4360,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{81} + return fileDescriptor_api_a0e30b34ee568726, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4392,7 +4398,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{82} + return fileDescriptor_api_a0e30b34ee568726, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4445,7 +4451,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{83} + return fileDescriptor_api_a0e30b34ee568726, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4479,7 +4485,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{84} + return fileDescriptor_api_a0e30b34ee568726, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4523,7 +4529,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{85} + return fileDescriptor_api_a0e30b34ee568726, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4557,7 +4563,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{86} + return fileDescriptor_api_a0e30b34ee568726, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4596,7 +4602,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{87} + return fileDescriptor_api_a0e30b34ee568726, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4630,7 +4636,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{88} + return fileDescriptor_api_a0e30b34ee568726, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4679,7 +4685,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{89} + return fileDescriptor_api_a0e30b34ee568726, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4728,7 +4734,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{90} + return fileDescriptor_api_a0e30b34ee568726, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4763,7 +4769,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{91} + return fileDescriptor_api_a0e30b34ee568726, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4804,7 +4810,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{92} + return fileDescriptor_api_a0e30b34ee568726, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4847,7 +4853,7 @@ func (m *MigrateRequest) Reset() { *m = MigrateRequest{} } func (m *MigrateRequest) String() string { return proto.CompactTextString(m) } func (*MigrateRequest) ProtoMessage() {} func (*MigrateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{93} + return fileDescriptor_api_a0e30b34ee568726, []int{93} } func (m *MigrateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4881,7 +4887,7 @@ func (m *MigrateResponse) Reset() { *m = MigrateResponse{} } func (m *MigrateResponse) String() string { return proto.CompactTextString(m) } func (*MigrateResponse) ProtoMessage() {} func (*MigrateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{94} + return fileDescriptor_api_a0e30b34ee568726, []int{94} } func (m *MigrateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4965,7 +4971,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{95} + return fileDescriptor_api_a0e30b34ee568726, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6446,7 +6452,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{96} + return fileDescriptor_api_a0e30b34ee568726, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8002,7 +8008,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{97} + return fileDescriptor_api_a0e30b34ee568726, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8040,7 +8046,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{98} + return fileDescriptor_api_a0e30b34ee568726, []int{98} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8076,7 +8082,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{99} + return fileDescriptor_api_a0e30b34ee568726, []int{99} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8113,7 +8119,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{100} + return fileDescriptor_api_a0e30b34ee568726, []int{100} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8189,7 +8195,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{100, 0} + return fileDescriptor_api_a0e30b34ee568726, []int{100, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8227,7 +8233,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{101} + return fileDescriptor_api_a0e30b34ee568726, []int{101} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8265,7 +8271,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{102} + return fileDescriptor_api_a0e30b34ee568726, []int{102} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8304,7 +8310,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{103} + return fileDescriptor_api_a0e30b34ee568726, []int{103} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8345,7 +8351,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{104} + return fileDescriptor_api_a0e30b34ee568726, []int{104} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8386,7 +8392,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{105} + return fileDescriptor_api_a0e30b34ee568726, []int{105} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8423,7 +8429,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{106} + return fileDescriptor_api_a0e30b34ee568726, []int{106} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8460,7 +8466,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{107} + return fileDescriptor_api_a0e30b34ee568726, []int{107} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8504,7 +8510,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{108} + return fileDescriptor_api_a0e30b34ee568726, []int{108} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8536,7 +8542,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{109} + return fileDescriptor_api_a0e30b34ee568726, []int{109} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8577,7 +8583,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{110} + return fileDescriptor_api_a0e30b34ee568726, []int{110} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8617,7 +8623,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{111} + return fileDescriptor_api_a0e30b34ee568726, []int{111} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8653,7 +8659,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{112} + return fileDescriptor_api_a0e30b34ee568726, []int{112} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8692,7 +8698,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{113} + return fileDescriptor_api_a0e30b34ee568726, []int{113} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8732,7 +8738,7 @@ type ContentionEvent struct { func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_cdfee8e6fce2cd19, []int{114} + return fileDescriptor_api_a0e30b34ee568726, []int{114} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -9306,6 +9312,11 @@ func (m *GetRequest) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n3 + if m.KeyLocking != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.KeyLocking)) + } return i, nil } @@ -15974,6 +15985,9 @@ func (m *GetRequest) Size() (n int) { _ = l l = m.RequestHeader.Size() n += 1 + l + sovApi(uint64(l)) + if m.KeyLocking != 0 { + n += 1 + sovApi(uint64(m.KeyLocking)) + } return n } @@ -19692,6 +19706,25 @@ func (m *GetRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeyLocking", wireType) + } + m.KeyLocking = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.KeyLocking |= (lock.Strength(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -39051,16 +39084,16 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_cdfee8e6fce2cd19) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_a0e30b34ee568726) } -var fileDescriptor_api_cdfee8e6fce2cd19 = []byte{ - // 8233 bytes of a gzipped FileDescriptorProto +var fileDescriptor_api_a0e30b34ee568726 = []byte{ + // 8242 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5b, 0x6c, 0x23, 0xc9, 0x7a, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0xa7, 0x78, 0x51, 0x69, 0x2e, 0x1c, 0xee, 0xee, 0x68, 0xa6, - 0xe7, 0x3e, 0x67, 0x57, 0xda, 0x99, 0x39, 0x9b, 0x5d, 0xef, 0xac, 0xf7, 0x58, 0xa2, 0x38, 0x43, - 0x4a, 0x23, 0x8d, 0xa6, 0x49, 0xcd, 0x60, 0xd7, 0xc7, 0xe9, 0xd3, 0xea, 0x2e, 0x51, 0x7d, 0x44, - 0x76, 0x73, 0xba, 0x9b, 0xba, 0x2c, 0x10, 0x20, 0x8e, 0x0d, 0xe7, 0x3c, 0x19, 0xe7, 0xc1, 0x41, - 0x8e, 0xe1, 0x20, 0x3e, 0x8e, 0x83, 0xf8, 0x21, 0x40, 0x12, 0x20, 0x41, 0x6e, 0x48, 0xec, 0x17, + 0xe7, 0x3e, 0xde, 0x95, 0x76, 0x66, 0xce, 0x66, 0xd7, 0x3b, 0xeb, 0x3d, 0x96, 0x28, 0xce, 0x90, + 0xd2, 0x48, 0xa3, 0x69, 0x52, 0x33, 0xd8, 0xf5, 0x71, 0xfa, 0xb4, 0xba, 0x4b, 0x54, 0x1f, 0x91, + 0xdd, 0x9c, 0xee, 0xa6, 0x2e, 0x0b, 0x04, 0x88, 0x63, 0xc3, 0x39, 0x4f, 0xc6, 0x79, 0x70, 0x90, + 0x63, 0x38, 0x88, 0x8f, 0x73, 0x82, 0xf8, 0x21, 0x40, 0x12, 0x20, 0x41, 0x6e, 0x48, 0xec, 0x17, 0x03, 0x39, 0x08, 0x9c, 0xf8, 0xf8, 0x29, 0x46, 0x80, 0x28, 0xb6, 0x4e, 0x5e, 0x02, 0x07, 0x46, 0x10, 0x04, 0x30, 0xb0, 0x0f, 0x41, 0x50, 0x97, 0xbe, 0x91, 0xcd, 0x8b, 0x66, 0xfb, 0x24, 0x0b, 0xf8, 0x45, 0x62, 0xff, 0x55, 0xff, 0xdf, 0x55, 0x7f, 0x55, 0xfd, 0xf5, 0x7f, 0xd5, 0x7f, 0x55, @@ -39078,8 +39111,8 @@ var fileDescriptor_api_cdfee8e6fce2cd19 = []byte{ 0xe7, 0x02, 0xe4, 0x24, 0xfc, 0xba, 0x87, 0x6d, 0xa7, 0x86, 0x15, 0x0d, 0x5b, 0xe8, 0x0a, 0x24, 0x0f, 0xf0, 0x49, 0x29, 0x79, 0x4d, 0xb8, 0x3b, 0xb7, 0x3a, 0xfb, 0xe5, 0xe9, 0x62, 0x72, 0x03, 0x9f, 0x48, 0x84, 0x86, 0xae, 0xc1, 0x2c, 0x36, 0x34, 0x99, 0x24, 0xa7, 0xc2, 0xc9, 0x33, 0xd8, - 0xd0, 0x36, 0xf0, 0x09, 0xfa, 0x36, 0xa4, 0x6d, 0x22, 0xcd, 0x50, 0x71, 0x69, 0xfa, 0x9a, 0x70, - 0x77, 0x7a, 0xf5, 0xe7, 0xbe, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x25, 0xd5, + 0xd0, 0x36, 0xf0, 0x09, 0xfa, 0x16, 0xa4, 0x6d, 0x22, 0xcd, 0x50, 0x71, 0x69, 0xfa, 0x9a, 0x70, + 0x77, 0x7a, 0xf5, 0xe7, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x25, 0xd5, 0xec, 0x2c, 0x7b, 0xed, 0xa4, 0xed, 0xfa, 0xbf, 0x97, 0xbb, 0x07, 0xad, 0xe5, 0x7e, 0x1d, 0x2d, 0x35, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, 0xa5, 0x13, 0xc5, 0xa4, 0xf8, 0x87, 0x09, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xc9, 0xdf, @@ -39092,482 +39125,483 @@ var fileDescriptor_api_cdfee8e6fce2cd19 = []byte{ 0x73, 0x56, 0xe0, 0x09, 0xbd, 0x05, 0x19, 0xf2, 0xbe, 0xdd, 0x13, 0x07, 0xdb, 0xa5, 0x34, 0x7d, 0x21, 0x29, 0xc0, 0x2a, 0x79, 0x16, 0x3f, 0x85, 0xb9, 0x20, 0x2b, 0x42, 0x90, 0x97, 0xaa, 0x8d, 0x9d, 0xcd, 0xaa, 0xbc, 0xb3, 0xb5, 0xb1, 0xf5, 0xfc, 0xd5, 0x56, 0x71, 0x0a, 0x5d, 0x80, 0x22, - 0xa7, 0x6d, 0x54, 0x3f, 0x93, 0x9f, 0xd5, 0x37, 0xeb, 0xcd, 0xa2, 0x50, 0x4e, 0x7d, 0xef, 0xb7, - 0xaf, 0x4e, 0xad, 0xa7, 0xd2, 0x33, 0xc5, 0x59, 0x71, 0x1b, 0xe0, 0x29, 0x76, 0x78, 0x67, 0x40, - 0xab, 0x30, 0xb3, 0x4f, 0x4b, 0x55, 0x12, 0xa8, 0x56, 0xae, 0x45, 0x16, 0x3f, 0xd0, 0x71, 0x56, - 0xd3, 0x3f, 0x3a, 0x5d, 0x9c, 0xfa, 0xf1, 0xe9, 0xa2, 0x20, 0x71, 0x4e, 0xf1, 0xf7, 0x04, 0xc8, - 0x52, 0x91, 0xac, 0x96, 0xa8, 0xd2, 0x27, 0xf3, 0xfa, 0x58, 0x95, 0x0c, 0x0a, 0x45, 0x4b, 0x30, - 0x7d, 0xa8, 0xb4, 0x7b, 0xb8, 0x94, 0xa0, 0x32, 0x4a, 0x11, 0x32, 0x5e, 0x92, 0x74, 0x89, 0x65, - 0x43, 0x8f, 0x61, 0x4e, 0x37, 0x1c, 0x6c, 0x38, 0x32, 0x63, 0x4b, 0x8e, 0x61, 0xcb, 0xb2, 0xdc, - 0xf4, 0x41, 0xfc, 0x67, 0x02, 0xc0, 0x76, 0x2f, 0x4e, 0xa5, 0xa0, 0x6f, 0x4e, 0x58, 0xfe, 0xd5, - 0x14, 0x61, 0x75, 0x6b, 0x71, 0x09, 0x66, 0x74, 0xa3, 0xad, 0x1b, 0xac, 0xfc, 0x69, 0x89, 0x3f, - 0xa1, 0x0b, 0x30, 0xbd, 0xdb, 0xd6, 0x0d, 0x8d, 0xf6, 0xdd, 0xb4, 0xc4, 0x1e, 0x44, 0x09, 0xb2, - 0xb4, 0xd4, 0x31, 0xea, 0x5d, 0x3c, 0x4d, 0xc0, 0xc5, 0x8a, 0x69, 0x68, 0x3a, 0x19, 0x3e, 0x4a, - 0xfb, 0x6b, 0xa1, 0x95, 0x75, 0xb8, 0xa0, 0xe1, 0xae, 0x85, 0x55, 0xc5, 0xc1, 0x9a, 0x8c, 0x8f, - 0xbb, 0x13, 0xb6, 0x31, 0xf2, 0xb9, 0xaa, 0xc7, 0x5d, 0x4a, 0x8b, 0xd6, 0x24, 0xfa, 0x26, 0x5c, - 0x56, 0xda, 0x6d, 0xf3, 0x48, 0xd6, 0xf7, 0x64, 0xcd, 0xc4, 0xb6, 0x6c, 0x98, 0x8e, 0x8c, 0x8f, - 0x75, 0xdb, 0xa1, 0xc3, 0x3e, 0x2d, 0x2d, 0xd0, 0xe4, 0xfa, 0xde, 0x9a, 0x89, 0xed, 0x2d, 0xd3, - 0xa9, 0x92, 0x24, 0x32, 0x5a, 0x49, 0x61, 0xd8, 0x68, 0x9d, 0x21, 0xa6, 0x52, 0x4a, 0xe3, 0xe3, - 0x2e, 0x1d, 0xad, 0x81, 0xa6, 0x9c, 0x0d, 0x36, 0xa5, 0xf8, 0x0b, 0x70, 0xa9, 0x5f, 0xbf, 0x71, - 0xb6, 0xdf, 0x1f, 0x08, 0x90, 0xaf, 0x1b, 0xba, 0xf3, 0xb5, 0x68, 0x38, 0x4f, 0xd9, 0xc9, 0xa0, - 0xb2, 0xef, 0x43, 0x71, 0x4f, 0xd1, 0xdb, 0xcf, 0x8d, 0xa6, 0xd9, 0xd9, 0xb5, 0x1d, 0xd3, 0xc0, - 0x36, 0x6f, 0x8d, 0x01, 0xba, 0xf8, 0x12, 0x0a, 0x5e, 0x6d, 0xe2, 0x54, 0x93, 0x03, 0xc5, 0xba, - 0xa1, 0x5a, 0xb8, 0x83, 0x8d, 0x58, 0xf5, 0xf4, 0x36, 0x64, 0x74, 0x57, 0x2e, 0xd5, 0x55, 0x52, - 0xf2, 0x09, 0x62, 0x0f, 0xe6, 0x03, 0x6f, 0x8d, 0xd3, 0x5c, 0x92, 0x89, 0x03, 0x1f, 0xc9, 0x7e, - 0x1b, 0x91, 0x89, 0x03, 0x1f, 0x31, 0xf3, 0xd6, 0x80, 0xdc, 0x1a, 0x6e, 0x63, 0x07, 0xc7, 0x69, - 0xf5, 0x77, 0x20, 0xef, 0x0a, 0x8d, 0xb3, 0x61, 0xfe, 0xb6, 0x00, 0x88, 0xcb, 0x55, 0x8c, 0x56, - 0x9c, 0x25, 0x46, 0x8b, 0xc4, 0x0d, 0x70, 0x7a, 0x96, 0xc1, 0xe6, 0x73, 0xd6, 0x27, 0x81, 0x91, - 0xe8, 0x94, 0xee, 0x0f, 0xd9, 0x54, 0x70, 0xc8, 0x72, 0x57, 0xe4, 0x08, 0x16, 0x42, 0x05, 0x8b, - 0xb7, 0xf9, 0x52, 0xb4, 0x4c, 0x89, 0x6b, 0xc9, 0xa0, 0xbf, 0x45, 0x89, 0xe2, 0x0f, 0x04, 0x98, - 0xaf, 0xb4, 0xb1, 0x62, 0xc5, 0xae, 0x91, 0x6f, 0x41, 0x5a, 0xc3, 0x8a, 0x46, 0xab, 0xcc, 0x06, - 0xf6, 0x3b, 0x01, 0x29, 0xc4, 0x2b, 0x5d, 0xda, 0x6f, 0xab, 0x4b, 0x4d, 0xd7, 0x5f, 0xe5, 0xa3, - 0xdb, 0x63, 0x12, 0x3f, 0x03, 0x14, 0x2c, 0x59, 0x9c, 0x1d, 0xe1, 0x7f, 0x0b, 0x80, 0x24, 0x7c, - 0x88, 0x2d, 0x27, 0xf6, 0x6a, 0xaf, 0x41, 0xd6, 0x51, 0xac, 0x16, 0x76, 0x64, 0xe2, 0x89, 0x9f, - 0xa7, 0xe6, 0xc0, 0xf8, 0x08, 0x19, 0x35, 0xe1, 0x0e, 0x36, 0x94, 0xdd, 0x36, 0xa6, 0x52, 0xe4, - 0x5d, 0xb3, 0x67, 0x68, 0xb2, 0xee, 0x60, 0x4b, 0x71, 0x4c, 0x4b, 0x36, 0xbb, 0x8e, 0xde, 0xd1, - 0xbf, 0xa0, 0x4e, 0x38, 0xef, 0x6a, 0x37, 0x58, 0x76, 0xc2, 0xbc, 0x4a, 0x32, 0xd7, 0x79, 0xde, - 0xe7, 0x81, 0xac, 0xe2, 0xe7, 0xb0, 0x10, 0xaa, 0x75, 0x9c, 0x2a, 0xfd, 0x9f, 0x02, 0x64, 0x1b, - 0xaa, 0x62, 0xc4, 0xa9, 0xcb, 0x4f, 0x21, 0x6b, 0xab, 0x8a, 0x21, 0xef, 0x99, 0x56, 0x47, 0x71, - 0xe8, 0xc0, 0xc9, 0x87, 0x74, 0xe9, 0xf9, 0xd6, 0xaa, 0x62, 0x3c, 0xa1, 0x99, 0x24, 0xb0, 0xbd, - 0xdf, 0xe8, 0x05, 0x64, 0x0f, 0xf0, 0x89, 0xcc, 0x71, 0x1a, 0x9d, 0x6d, 0xf3, 0x0f, 0xdf, 0x0f, - 0xf0, 0x1f, 0x1c, 0x2e, 0xb9, 0xf0, 0x6e, 0x29, 0x00, 0xef, 0x96, 0x08, 0xc7, 0x52, 0xc3, 0xb1, - 0xb0, 0xd1, 0x72, 0xf6, 0x25, 0x38, 0xc0, 0x27, 0xcf, 0x98, 0x0c, 0x36, 0x5c, 0xd7, 0x53, 0xe9, - 0x64, 0x31, 0x25, 0xfe, 0x85, 0x00, 0x73, 0xac, 0xca, 0x71, 0x0e, 0xd7, 0x0f, 0x20, 0x65, 0x99, - 0x47, 0x6c, 0xb8, 0x66, 0x1f, 0xbe, 0x15, 0x21, 0x62, 0x03, 0x9f, 0x04, 0xe7, 0x43, 0x9a, 0x1d, - 0xad, 0x02, 0xf7, 0x3a, 0x65, 0xca, 0x9d, 0x9c, 0x94, 0x1b, 0x18, 0x97, 0x44, 0x64, 0xdc, 0x81, - 0xc2, 0xae, 0xe2, 0xa8, 0xfb, 0xb2, 0xc5, 0x0b, 0x49, 0xe6, 0xce, 0xe4, 0xdd, 0x39, 0x29, 0x4f, - 0xc9, 0x6e, 0xd1, 0x6d, 0x52, 0x73, 0x36, 0x7e, 0x6c, 0xfc, 0x97, 0xac, 0xcd, 0xff, 0x8f, 0xc0, - 0xc7, 0x90, 0x5b, 0xf3, 0xbf, 0x6c, 0x4d, 0xff, 0xeb, 0x09, 0xb8, 0x5c, 0xd9, 0xc7, 0xea, 0x41, - 0xc5, 0x34, 0x6c, 0xdd, 0x76, 0x88, 0xee, 0xe2, 0x6c, 0xff, 0xb7, 0x20, 0x73, 0xa4, 0x3b, 0xfb, - 0xb2, 0xa6, 0xef, 0xed, 0x51, 0xeb, 0x99, 0x96, 0xd2, 0x84, 0xb0, 0xa6, 0xef, 0xed, 0xa1, 0x47, - 0x90, 0xea, 0x98, 0x1a, 0x73, 0xce, 0xf3, 0x0f, 0x17, 0x23, 0xc4, 0xd3, 0xa2, 0xd9, 0xbd, 0xce, - 0xa6, 0xa9, 0x61, 0x89, 0x66, 0x46, 0x57, 0x01, 0x54, 0x42, 0xed, 0x9a, 0xba, 0xe1, 0xf0, 0xd9, - 0x37, 0x40, 0x41, 0x35, 0xc8, 0x38, 0xd8, 0xea, 0xe8, 0x86, 0xe2, 0xe0, 0xd2, 0x34, 0x55, 0xde, - 0xcd, 0xc8, 0x82, 0x77, 0xdb, 0xba, 0xaa, 0xac, 0x61, 0x5b, 0xb5, 0xf4, 0xae, 0x63, 0x5a, 0x5c, - 0x8b, 0x3e, 0xb3, 0xf8, 0xab, 0x29, 0x28, 0x0d, 0xea, 0x26, 0xce, 0x1e, 0xb2, 0x0d, 0x33, 0x04, - 0xd3, 0xb7, 0x1d, 0xde, 0x47, 0x1e, 0x0e, 0x53, 0x41, 0x44, 0x09, 0xe8, 0xda, 0x40, 0xdb, 0xe1, - 0xc5, 0xe6, 0x72, 0xca, 0xff, 0x46, 0x80, 0x19, 0x96, 0x80, 0x1e, 0x40, 0xda, 0x22, 0x13, 0x83, - 0xac, 0x6b, 0xb4, 0x8c, 0xc9, 0xd5, 0x4b, 0x67, 0xa7, 0x8b, 0xb3, 0x74, 0xb2, 0xa8, 0xaf, 0x7d, - 0xe9, 0xff, 0x94, 0x66, 0x69, 0xbe, 0xba, 0x46, 0x5a, 0xcb, 0x76, 0x14, 0xcb, 0xa1, 0x0b, 0x3a, - 0x09, 0x86, 0x52, 0x28, 0x61, 0x03, 0x9f, 0xa0, 0x75, 0x98, 0xb1, 0x1d, 0xc5, 0xe9, 0xd9, 0xbc, - 0xbd, 0xce, 0x55, 0xd8, 0x06, 0xe5, 0x94, 0xb8, 0x04, 0xe2, 0x3e, 0x69, 0xd8, 0x51, 0xf4, 0x36, - 0x6d, 0xc0, 0x8c, 0xc4, 0x9f, 0xc4, 0xdf, 0x10, 0x60, 0x86, 0x65, 0x45, 0x97, 0x61, 0x41, 0x5a, - 0xd9, 0x7a, 0x5a, 0x95, 0xeb, 0x5b, 0x6b, 0xd5, 0x66, 0x55, 0xda, 0xac, 0x6f, 0xad, 0x34, 0xab, - 0xc5, 0x29, 0x74, 0x09, 0x90, 0x9b, 0x50, 0x79, 0xbe, 0xd5, 0xa8, 0x37, 0x9a, 0xd5, 0xad, 0x66, - 0x51, 0xa0, 0xeb, 0x19, 0x94, 0x1e, 0xa0, 0x26, 0xd0, 0x4d, 0xb8, 0xd6, 0x4f, 0x95, 0x1b, 0xcd, - 0x95, 0x66, 0x43, 0xae, 0x36, 0x9a, 0xf5, 0xcd, 0x95, 0x66, 0x75, 0xad, 0x98, 0x1c, 0x91, 0x8b, - 0xbc, 0x44, 0x92, 0xaa, 0x95, 0x66, 0x31, 0x25, 0x3a, 0x70, 0x51, 0xc2, 0xaa, 0xd9, 0xe9, 0xf6, - 0x1c, 0x4c, 0x4a, 0x69, 0xc7, 0x39, 0x52, 0x2e, 0xc3, 0xac, 0x66, 0x9d, 0xc8, 0x56, 0xcf, 0xe0, - 0xe3, 0x64, 0x46, 0xb3, 0x4e, 0xa4, 0x9e, 0x21, 0xfe, 0x13, 0x01, 0x2e, 0xf5, 0xbf, 0x36, 0xce, - 0x4e, 0xf8, 0x02, 0xb2, 0x8a, 0xa6, 0x61, 0x4d, 0xd6, 0x70, 0xdb, 0x51, 0xb8, 0x8b, 0x73, 0x3f, - 0x20, 0x89, 0x2f, 0xc3, 0x2d, 0x79, 0xcb, 0x70, 0x9b, 0x2f, 0x2b, 0x15, 0x5a, 0x90, 0x35, 0xc2, - 0xe1, 0x9a, 0x1f, 0x2a, 0x84, 0x52, 0xc4, 0xff, 0x91, 0x82, 0x5c, 0xd5, 0xd0, 0x9a, 0xc7, 0xb1, - 0xce, 0x25, 0x97, 0x60, 0x46, 0x35, 0x3b, 0x1d, 0xdd, 0x71, 0x15, 0xc4, 0x9e, 0xd0, 0xcf, 0x04, - 0x5c, 0xd3, 0xe4, 0x04, 0x0e, 0x9a, 0xef, 0x94, 0xa2, 0xef, 0xc0, 0x65, 0x62, 0x35, 0x2d, 0x43, - 0x69, 0xcb, 0x4c, 0x9a, 0xec, 0x58, 0x7a, 0xab, 0x85, 0x2d, 0xbe, 0xf4, 0x77, 0x37, 0xa2, 0x9c, - 0x75, 0xce, 0x51, 0xa1, 0x0c, 0x4d, 0x96, 0x5f, 0xba, 0xa8, 0x47, 0x91, 0xd1, 0x27, 0x00, 0x64, - 0x2a, 0xa2, 0xcb, 0x89, 0x36, 0xb7, 0x47, 0xc3, 0xd6, 0x13, 0x5d, 0x13, 0x44, 0x18, 0xc8, 0xb3, - 0x8d, 0x96, 0x09, 0x0e, 0x79, 0xdd, 0xd3, 0x2d, 0x2c, 0x3f, 0xe8, 0xaa, 0x74, 0xe1, 0x20, 0xbd, - 0x9a, 0x3f, 0x3b, 0x5d, 0x04, 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x2e, 0x61, 0xbf, 0xbb, 0x2a, - 0x7a, 0x05, 0xf7, 0x02, 0xeb, 0x1f, 0x64, 0xe6, 0xe5, 0xd5, 0x52, 0x1c, 0x79, 0x5f, 0x6f, 0xed, - 0x63, 0x4b, 0xf6, 0x96, 0x94, 0xe9, 0xaa, 0x61, 0x5a, 0xba, 0xe9, 0x33, 0x54, 0x14, 0x83, 0x95, - 0x7e, 0xc5, 0xa9, 0xd1, 0xcc, 0x9e, 0xce, 0x88, 0xf2, 0xbb, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xc3, - 0x94, 0xcf, 0x9e, 0xd0, 0x3d, 0x28, 0x3a, 0xc7, 0x86, 0xbc, 0x8f, 0x15, 0xcb, 0xd9, 0xc5, 0x8a, - 0x43, 0x66, 0x69, 0xa0, 0x39, 0x0a, 0xce, 0xb1, 0x51, 0x0b, 0x90, 0xd1, 0x0b, 0x28, 0xea, 0x86, - 0xbc, 0xd7, 0xd6, 0x5b, 0xfb, 0x8e, 0x7c, 0x64, 0xe9, 0x0e, 0xb6, 0x4b, 0xf3, 0x54, 0x21, 0x51, - 0xfd, 0xb6, 0xc1, 0xd7, 0x78, 0xb5, 0x57, 0x24, 0x27, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, 0x7e, - 0x4a, 0xb4, 0xd7, 0x53, 0xe9, 0xd9, 0x62, 0x5a, 0xfc, 0x2f, 0x02, 0xe4, 0xdd, 0xee, 0x16, 0xe7, - 0xc8, 0xb8, 0x0b, 0x45, 0xd3, 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x9f, 0x70, 0xf2, - 0xa6, 0x81, 0xb7, 0x09, 0x99, 0xa9, 0x0b, 0x6d, 0xc3, 0xbc, 0xed, 0x28, 0x2d, 0xdd, 0x68, 0x05, - 0xd4, 0x3b, 0x3d, 0x39, 0x58, 0x28, 0x72, 0x6e, 0x8f, 0x1e, 0xf2, 0x52, 0xfe, 0x48, 0x80, 0xf9, - 0x15, 0xad, 0xa3, 0x1b, 0x8d, 0x6e, 0x5b, 0x8f, 0x75, 0x0d, 0xe2, 0x26, 0x64, 0x6c, 0x22, 0xd3, - 0x37, 0xf8, 0x3e, 0xa2, 0x4c, 0xd3, 0x14, 0x62, 0xf9, 0x9f, 0x41, 0x01, 0x1f, 0x77, 0x75, 0xf6, - 0x99, 0x80, 0x01, 0xa1, 0xd4, 0xe4, 0x75, 0xcb, 0xfb, 0xbc, 0x24, 0x89, 0xd7, 0xe9, 0x33, 0x40, - 0xc1, 0x2a, 0xc5, 0x89, 0x5d, 0x3e, 0x83, 0x05, 0x2a, 0x7a, 0xc7, 0xb0, 0x63, 0xd6, 0x97, 0xf8, - 0xf3, 0x70, 0x21, 0x2c, 0x3a, 0xce, 0x72, 0xbf, 0xe2, 0xad, 0xbc, 0x89, 0xad, 0x58, 0x41, 0xac, - 0xa7, 0x6b, 0x2e, 0x38, 0xce, 0x32, 0xff, 0xb2, 0x00, 0x57, 0xa8, 0x6c, 0xfa, 0x25, 0x65, 0x0f, - 0x5b, 0xcf, 0xb0, 0x62, 0xc7, 0x8a, 0xc0, 0x6f, 0xc0, 0x0c, 0x43, 0xd2, 0xb4, 0x7f, 0x4e, 0xaf, - 0x66, 0x89, 0xe7, 0xd2, 0x70, 0x4c, 0x8b, 0x78, 0x2e, 0x3c, 0x49, 0x54, 0xa0, 0x1c, 0x55, 0x8a, - 0x38, 0x6b, 0xfa, 0x77, 0x05, 0x98, 0xe7, 0x4e, 0x23, 0xe9, 0xca, 0x95, 0x7d, 0xe2, 0x33, 0xa1, - 0x2a, 0x64, 0x55, 0xfa, 0x4b, 0x76, 0x4e, 0xba, 0x98, 0xca, 0xcf, 0x8f, 0xf2, 0x37, 0x19, 0x5b, - 0xf3, 0xa4, 0x8b, 0x89, 0xd3, 0xea, 0xfe, 0x26, 0x8a, 0x0a, 0x54, 0x72, 0xa4, 0xc7, 0x4a, 0xc7, - 0x11, 0xcd, 0xeb, 0xba, 0x7e, 0x5c, 0x07, 0xff, 0x34, 0xc9, 0x95, 0xc0, 0xde, 0xc1, 0xb3, 0xc7, - 0xea, 0xa3, 0x7c, 0x0e, 0x97, 0x82, 0xb3, 0x4b, 0xa0, 0xe2, 0x89, 0x73, 0x54, 0x3c, 0xb0, 0x42, - 0xef, 0x53, 0xd1, 0x67, 0x10, 0x58, 0x83, 0x97, 0x59, 0x9d, 0x5c, 0xf4, 0x73, 0x1e, 0x75, 0xcc, - 0xfb, 0x52, 0x18, 0xdd, 0x46, 0x15, 0x48, 0xe3, 0xe3, 0xae, 0xac, 0x61, 0x5b, 0xe5, 0x86, 0x4b, - 0x8c, 0x12, 0x48, 0x8a, 0x32, 0x80, 0x07, 0x66, 0xf1, 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x99, 0xbd, - 0x5c, 0x57, 0x81, 0x16, 0xdb, 0x1e, 0x0f, 0x2f, 0xfc, 0x9e, 0xc2, 0xc5, 0x15, 0x3c, 0x2f, 0x81, - 0x89, 0x10, 0x7f, 0x28, 0xc0, 0x5b, 0x91, 0xad, 0x16, 0xe7, 0x44, 0xf6, 0x09, 0xa4, 0x68, 0xe5, - 0x13, 0xe7, 0xac, 0x3c, 0xe5, 0x12, 0xbf, 0x97, 0xe0, 0x63, 0x5c, 0xc2, 0x6d, 0x93, 0x28, 0x36, - 0xf6, 0x55, 0xb6, 0xe7, 0x90, 0x3b, 0x34, 0x1d, 0xe2, 0x9b, 0xf0, 0x66, 0x4f, 0x9c, 0xbb, 0xd9, - 0xe7, 0xa8, 0x00, 0xb7, 0xc5, 0x5f, 0xc2, 0xbc, 0x61, 0x1a, 0x72, 0x58, 0xe8, 0xf9, 0xfb, 0x52, - 0xc1, 0x30, 0x8d, 0x97, 0x01, 0xb9, 0x9e, 0x9d, 0xe9, 0xd3, 0x44, 0x9c, 0x76, 0xe6, 0xfb, 0x02, - 0x2c, 0x78, 0x6e, 0x53, 0xcc, 0x1e, 0xf4, 0x07, 0x90, 0x34, 0xcc, 0xa3, 0xf3, 0xac, 0x62, 0x92, - 0xfc, 0x64, 0xd6, 0x0b, 0x97, 0x28, 0xce, 0xfa, 0xfe, 0xdb, 0x04, 0x64, 0x9e, 0x56, 0xe2, 0xac, - 0xe5, 0x27, 0x7c, 0x85, 0x9c, 0xb5, 0x77, 0x54, 0x6f, 0xf7, 0xde, 0xb7, 0xf4, 0xb4, 0xb2, 0x81, - 0x4f, 0xdc, 0xde, 0x4e, 0xb8, 0xd0, 0x0a, 0x64, 0x9c, 0x7d, 0x0b, 0xdb, 0xfb, 0x66, 0x5b, 0x3b, - 0x8f, 0x9b, 0xe3, 0x73, 0x95, 0x31, 0x4c, 0x53, 0xb9, 0x6e, 0xe4, 0x84, 0x10, 0x11, 0x39, 0x41, - 0x5e, 0xe3, 0x79, 0x8a, 0x89, 0xf3, 0xbc, 0x26, 0xe0, 0x22, 0x4e, 0x17, 0x67, 0xc4, 0x17, 0x00, - 0xa4, 0x3a, 0x71, 0x36, 0xc9, 0xaf, 0x24, 0x21, 0xbf, 0xdd, 0xb3, 0xf7, 0x63, 0xee, 0x7d, 0x15, - 0x80, 0x6e, 0xcf, 0xa6, 0x10, 0xe4, 0xd8, 0xe0, 0x75, 0x1e, 0x13, 0x94, 0xe1, 0x56, 0x9a, 0xf1, - 0x35, 0x8f, 0x0d, 0x54, 0xe3, 0x42, 0xb0, 0xec, 0x47, 0x76, 0xdc, 0x18, 0x05, 0x56, 0x9b, 0xc7, - 0xc6, 0x26, 0xf6, 0x50, 0x2a, 0x93, 0x84, 0x89, 0xa4, 0x4f, 0x60, 0x96, 0x3c, 0xc8, 0x8e, 0x79, - 0x9e, 0x66, 0x9e, 0x21, 0x3c, 0x4d, 0x13, 0x3d, 0x86, 0x0c, 0xe3, 0x26, 0xb3, 0xdf, 0x0c, 0x9d, - 0xfd, 0xa2, 0xea, 0xc2, 0xd5, 0x48, 0xe7, 0xbd, 0x34, 0x65, 0x25, 0x73, 0xdd, 0x05, 0x98, 0xde, - 0x33, 0x2d, 0xd5, 0xfd, 0xde, 0xcb, 0x1e, 0x58, 0x7b, 0xae, 0xa7, 0xd2, 0xe9, 0x62, 0x66, 0x3d, - 0x95, 0xce, 0x14, 0x41, 0xfc, 0x0d, 0x01, 0x0a, 0x5e, 0x43, 0xc4, 0x39, 0x21, 0x54, 0x42, 0x5a, - 0x3c, 0x7f, 0x53, 0x10, 0x05, 0x8a, 0xff, 0x8e, 0x7a, 0x44, 0xaa, 0x79, 0x48, 0x5b, 0x26, 0xce, - 0x9e, 0xf2, 0x98, 0xc5, 0xed, 0x24, 0xce, 0xdb, 0xba, 0x34, 0x84, 0xe7, 0x01, 0x5c, 0xd0, 0x3b, - 0xc4, 0x9e, 0xeb, 0x4e, 0xfb, 0x84, 0xc3, 0x36, 0x07, 0xbb, 0x1f, 0x96, 0x17, 0xfc, 0xb4, 0x8a, - 0x9b, 0x24, 0xfe, 0x7d, 0xba, 0x00, 0xee, 0xd7, 0x24, 0x4e, 0x55, 0xd7, 0x21, 0x67, 0x31, 0xd1, - 0xc4, 0xad, 0x39, 0xa7, 0xb6, 0xe7, 0x3c, 0x56, 0xa2, 0xf0, 0xdf, 0x4a, 0x40, 0xe1, 0x45, 0x0f, - 0x5b, 0x27, 0x5f, 0x27, 0x75, 0xdf, 0x86, 0xc2, 0x91, 0xa2, 0x3b, 0xf2, 0x9e, 0x69, 0xc9, 0xbd, - 0xae, 0xa6, 0x38, 0x6e, 0x40, 0x4a, 0x8e, 0x90, 0x9f, 0x98, 0xd6, 0x0e, 0x25, 0x22, 0x0c, 0xe8, - 0xc0, 0x30, 0x8f, 0x0c, 0x99, 0x90, 0x29, 0x50, 0x3e, 0x36, 0xf8, 0xaa, 0xf4, 0xea, 0x87, 0xff, - 0xf9, 0x74, 0xf1, 0xd1, 0x44, 0x21, 0x61, 0x34, 0xfc, 0xad, 0xd7, 0xd3, 0xb5, 0xa5, 0x9d, 0x9d, - 0xfa, 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x41, 0x02, 0x8a, - 0xbe, 0x8e, 0xe2, 0x6c, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x33, 0x02, 0x67, - 0x24, 0x66, 0xe7, 0x73, 0x98, 0x0b, 0x69, 0x20, 0xf9, 0xd5, 0x34, 0x90, 0x3d, 0xf2, 0x2b, 0x8f, - 0xee, 0xc3, 0xbc, 0x73, 0x6c, 0xc8, 0x2c, 0x18, 0x90, 0x05, 0xa5, 0xb8, 0xf1, 0x12, 0x05, 0x87, - 0xe8, 0x83, 0xd0, 0x69, 0x40, 0x8a, 0x2d, 0xfe, 0x9e, 0x00, 0x88, 0x2a, 0xaa, 0xce, 0x3e, 0x1b, - 0x7c, 0x5d, 0xfa, 0xd3, 0x5d, 0x28, 0xd2, 0xf0, 0x4a, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, - 0x46, 0x8b, 0x77, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0x6b, 0xb0, 0x10, 0xaa, - 0x40, 0x9c, 0x8d, 0x7d, 0x1d, 0xe6, 0xf6, 0xd8, 0x57, 0x5a, 0x2a, 0x9c, 0xaf, 0x38, 0x66, 0x29, - 0x8d, 0xbd, 0x4f, 0xfc, 0xb3, 0x04, 0x5c, 0x90, 0xb0, 0x6d, 0xb6, 0x0f, 0x71, 0xfc, 0x2a, 0xac, - 0x01, 0xff, 0x9c, 0x23, 0xbf, 0x91, 0x26, 0x33, 0x8c, 0x99, 0x4d, 0x73, 0xe1, 0x65, 0xfb, 0x9b, - 0xa3, 0x7b, 0xec, 0xe0, 0x42, 0x3d, 0x5f, 0xf6, 0x4b, 0x85, 0x96, 0xfd, 0x4c, 0x28, 0xe8, 0x2d, - 0xc3, 0x24, 0x36, 0xcd, 0xc6, 0xaf, 0x8d, 0x5e, 0xc7, 0x05, 0x43, 0x4b, 0xa3, 0x0a, 0x59, 0x67, - 0x2c, 0x0d, 0xfc, 0x7a, 0xab, 0xd7, 0xa1, 0xbe, 0xf3, 0xea, 0x25, 0x52, 0xde, 0xb3, 0xd3, 0xc5, - 0x7c, 0x28, 0xcd, 0x96, 0xf2, 0xba, 0xf7, 0x4c, 0xa4, 0x8b, 0xdf, 0x86, 0x8b, 0x7d, 0xca, 0x8e, - 0xd3, 0xe3, 0xf9, 0xd7, 0x49, 0xb8, 0x12, 0x16, 0x1f, 0x37, 0xc4, 0xf9, 0xba, 0x37, 0x68, 0x0d, - 0x72, 0x1d, 0xdd, 0x78, 0xb3, 0xd5, 0xcb, 0xb9, 0x8e, 0x6e, 0xf8, 0x2b, 0xc5, 0x11, 0x5d, 0x63, - 0xe6, 0xa7, 0xda, 0x35, 0x14, 0x28, 0x47, 0xb5, 0x5d, 0x9c, 0xfd, 0xe3, 0x7b, 0x02, 0xcc, 0xc5, - 0xbd, 0x2c, 0xf7, 0x66, 0x81, 0x72, 0x62, 0x13, 0x72, 0x3f, 0x85, 0x75, 0xbc, 0xdf, 0x12, 0x00, - 0x35, 0xad, 0x9e, 0x41, 0x40, 0xed, 0x33, 0xb3, 0x15, 0x67, 0x35, 0x2f, 0xc0, 0xb4, 0x6e, 0x68, - 0xf8, 0x98, 0x56, 0x33, 0x25, 0xb1, 0x87, 0xd0, 0xd7, 0xc9, 0xe4, 0x44, 0x5f, 0x27, 0xc5, 0xcf, - 0x61, 0x21, 0x54, 0xc4, 0x38, 0xeb, 0xff, 0xdf, 0x13, 0xb0, 0xc0, 0x2b, 0x12, 0xfb, 0x0a, 0xe6, - 0x37, 0x61, 0xba, 0x4d, 0x64, 0x8e, 0x68, 0x67, 0xfa, 0x4e, 0xb7, 0x9d, 0x69, 0x66, 0xf4, 0xb3, - 0x00, 0x5d, 0x0b, 0x1f, 0xca, 0x8c, 0x35, 0x39, 0x11, 0x6b, 0x86, 0x70, 0x50, 0x02, 0xfa, 0x81, - 0x00, 0x05, 0x32, 0xa0, 0xbb, 0x96, 0xd9, 0x35, 0x6d, 0xe2, 0xb3, 0xd8, 0x93, 0xc1, 0x9c, 0x17, - 0x67, 0xa7, 0x8b, 0xb9, 0x4d, 0xdd, 0xd8, 0xe6, 0x8c, 0xcd, 0xc6, 0xc4, 0xf1, 0xfa, 0xee, 0xae, - 0x85, 0xa5, 0x4a, 0xdb, 0x54, 0x0f, 0xfc, 0xef, 0x6d, 0xc4, 0xb2, 0x78, 0xe2, 0x6c, 0xf1, 0x0f, - 0x05, 0xb8, 0xf0, 0x53, 0x5b, 0x2e, 0xfe, 0xff, 0xa1, 0x6c, 0xf1, 0x25, 0x14, 0xe9, 0x8f, 0xba, - 0xb1, 0x67, 0xc6, 0xb9, 0x70, 0xff, 0xab, 0x02, 0xcc, 0x07, 0x04, 0xc7, 0xe9, 0xe0, 0xbc, 0x91, - 0x9e, 0xc4, 0x9f, 0x27, 0x2e, 0x4f, 0x70, 0x94, 0xc4, 0x39, 0x06, 0xff, 0x45, 0x02, 0x2e, 0x55, - 0xd8, 0x67, 0x6e, 0x37, 0xee, 0x23, 0xce, 0x9e, 0x51, 0x82, 0xd9, 0x43, 0x6c, 0xd9, 0xba, 0xc9, - 0xa6, 0xdf, 0x9c, 0xe4, 0x3e, 0xa2, 0x32, 0xa4, 0x6d, 0x43, 0xe9, 0xda, 0xfb, 0xa6, 0xfb, 0x7d, - 0xcf, 0x7b, 0xf6, 0x62, 0x54, 0xa6, 0xdf, 0x3c, 0x46, 0x65, 0x66, 0x74, 0x8c, 0xca, 0xec, 0x57, - 0x88, 0x51, 0xe1, 0x1f, 0xd3, 0xfe, 0xbd, 0x00, 0x97, 0x07, 0x34, 0x17, 0x67, 0x6f, 0xf9, 0x2e, - 0x64, 0x55, 0x2e, 0x98, 0x18, 0x6c, 0xf6, 0xa5, 0xb0, 0x4e, 0xb2, 0xbd, 0x21, 0x6e, 0x39, 0x3b, - 0x5d, 0x04, 0xb7, 0xa8, 0xf5, 0x35, 0xae, 0x1c, 0xf2, 0x5b, 0x13, 0x7f, 0x29, 0x07, 0x85, 0xea, - 0x31, 0x5b, 0x25, 0x6f, 0x30, 0x37, 0x01, 0x3d, 0x81, 0x74, 0xd7, 0x32, 0x0f, 0x75, 0xb7, 0x1a, - 0xf9, 0x50, 0x80, 0x82, 0x5b, 0x8d, 0x3e, 0xae, 0x6d, 0xce, 0x21, 0x79, 0xbc, 0xa8, 0x09, 0x99, - 0x67, 0xa6, 0xaa, 0xb4, 0x9f, 0xe8, 0x6d, 0xb7, 0xe7, 0xbf, 0x3f, 0x5e, 0xd0, 0x92, 0xc7, 0xb3, - 0xad, 0x38, 0xfb, 0x6e, 0x23, 0x78, 0x44, 0x54, 0x87, 0x74, 0xcd, 0x71, 0xba, 0x24, 0x91, 0xdb, - 0x8e, 0x3b, 0x13, 0x08, 0x25, 0x2c, 0x6e, 0x94, 0xac, 0xcb, 0x8e, 0x9a, 0x30, 0xff, 0x94, 0xee, - 0xcf, 0xaa, 0xb4, 0xcd, 0x9e, 0x56, 0x31, 0x8d, 0x3d, 0xbd, 0xc5, 0xed, 0xf6, 0xed, 0x09, 0x64, - 0x3e, 0xad, 0x34, 0xa4, 0x41, 0x01, 0x68, 0x05, 0xd2, 0x8d, 0x47, 0x5c, 0x18, 0xf3, 0xeb, 0x6e, - 0x4d, 0x20, 0xac, 0xf1, 0x48, 0xf2, 0xd8, 0xd0, 0x3a, 0x64, 0x57, 0xbe, 0xe8, 0x59, 0x98, 0x4b, - 0x99, 0x19, 0x1a, 0x1d, 0xd1, 0x2f, 0x85, 0x72, 0x49, 0x41, 0x66, 0xd4, 0x80, 0xfc, 0x2b, 0xd3, - 0x3a, 0x68, 0x9b, 0x8a, 0x5b, 0xc3, 0x59, 0x2a, 0xee, 0x1b, 0x13, 0x88, 0x73, 0x19, 0xa5, 0x3e, - 0x11, 0xe8, 0xdb, 0x50, 0x20, 0x8d, 0xd1, 0x54, 0x76, 0xdb, 0x6e, 0x21, 0xd3, 0x54, 0xea, 0xbb, - 0x13, 0x48, 0xf5, 0x38, 0xdd, 0x85, 0xff, 0x3e, 0x51, 0x65, 0x09, 0x72, 0xa1, 0x4e, 0x80, 0x10, - 0xa4, 0xba, 0xa4, 0xbd, 0x05, 0x1a, 0xbf, 0x44, 0x7f, 0xa3, 0xf7, 0x60, 0xd6, 0x30, 0x35, 0xec, - 0x8e, 0x90, 0xdc, 0xea, 0x85, 0xb3, 0xd3, 0xc5, 0x99, 0x2d, 0x53, 0x63, 0x1e, 0x0d, 0xff, 0x25, - 0xcd, 0x90, 0x4c, 0x75, 0xad, 0x7c, 0x0d, 0x52, 0xa4, 0xdd, 0x89, 0x61, 0xda, 0x55, 0x6c, 0xbc, - 0x63, 0xe9, 0x5c, 0x9a, 0xfb, 0x58, 0xfe, 0xc7, 0x09, 0x48, 0x34, 0x1e, 0x11, 0x9f, 0x7d, 0xb7, - 0xa7, 0x1e, 0x60, 0x87, 0xa7, 0xf3, 0x27, 0xea, 0xcb, 0x5b, 0x78, 0x4f, 0x67, 0xae, 0x55, 0x46, - 0xe2, 0x4f, 0xe8, 0x1d, 0x00, 0x45, 0x55, 0xb1, 0x6d, 0xcb, 0xee, 0xbe, 0xbd, 0x8c, 0x94, 0x61, - 0x94, 0x0d, 0x7c, 0x42, 0xd8, 0x6c, 0xac, 0x5a, 0xd8, 0x71, 0x83, 0xaf, 0xd8, 0x13, 0x61, 0x73, - 0x70, 0xa7, 0x2b, 0x3b, 0xe6, 0x01, 0x36, 0x68, 0x3f, 0xc9, 0x10, 0x53, 0xd3, 0xe9, 0x36, 0x09, - 0x81, 0x58, 0x49, 0x6c, 0x68, 0xbe, 0x49, 0xcb, 0x48, 0xde, 0x33, 0x11, 0x69, 0xe1, 0x96, 0xce, - 0xb7, 0xb6, 0x65, 0x24, 0xfe, 0x44, 0xb4, 0xa4, 0xf4, 0x9c, 0x7d, 0xda, 0x12, 0x19, 0x89, 0xfe, - 0x46, 0xb7, 0xa1, 0xc0, 0xe2, 0x35, 0x65, 0x6c, 0xa8, 0x32, 0x35, 0xae, 0x19, 0x9a, 0x9c, 0x63, - 0xe4, 0xaa, 0xa1, 0x12, 0x53, 0x8a, 0x1e, 0x01, 0x27, 0xc8, 0x07, 0x1d, 0x9b, 0xe8, 0x14, 0x48, - 0xae, 0xd5, 0xc2, 0xd9, 0xe9, 0x62, 0xb6, 0x41, 0x13, 0x36, 0x36, 0x1b, 0xf5, 0x35, 0x29, 0xcb, - 0x72, 0x6d, 0x74, 0xec, 0xba, 0x56, 0xfe, 0x35, 0x01, 0x92, 0x4f, 0x2b, 0x8d, 0x73, 0xab, 0xcc, - 0x2d, 0x68, 0x32, 0x50, 0xd0, 0x3b, 0x50, 0xd8, 0xd5, 0xdb, 0x6d, 0xdd, 0x68, 0x11, 0x2f, 0xea, - 0xbb, 0x58, 0x75, 0x15, 0x96, 0xe7, 0xe4, 0x6d, 0x46, 0x45, 0xd7, 0x20, 0xab, 0x5a, 0x58, 0xc3, - 0x86, 0xa3, 0x2b, 0x6d, 0x9b, 0x6b, 0x2e, 0x48, 0x2a, 0xff, 0xa2, 0x00, 0xd3, 0x74, 0x04, 0xa0, - 0xb7, 0x21, 0xa3, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0x37, 0x65, 0x19, 0xc9, 0x27, 0x0c, 0x2d, 0xde, - 0x75, 0x98, 0x53, 0x54, 0xd5, 0xec, 0x19, 0x8e, 0x6c, 0x28, 0x1d, 0xcc, 0x8b, 0x99, 0xe5, 0xb4, - 0x2d, 0xa5, 0x83, 0xd1, 0x22, 0xb8, 0x8f, 0xde, 0x76, 0xcc, 0x8c, 0x04, 0x9c, 0xb4, 0x81, 0x4f, - 0xca, 0xbf, 0x2f, 0x40, 0xda, 0x1d, 0x33, 0xa4, 0x18, 0x2d, 0x6c, 0xb0, 0xa0, 0x72, 0xb7, 0x18, - 0x1e, 0xa1, 0x7f, 0xaa, 0xcc, 0xf8, 0x53, 0xe5, 0x05, 0x98, 0x76, 0xc8, 0xb0, 0xe0, 0x25, 0x60, - 0x0f, 0x74, 0x3d, 0xbb, 0xad, 0xb4, 0xd8, 0x72, 0x5e, 0x46, 0x62, 0x0f, 0xa4, 0x32, 0x3c, 0xec, - 0x97, 0x69, 0x84, 0x3f, 0x91, 0x92, 0xb2, 0xe0, 0xd4, 0x5d, 0xdc, 0xd2, 0x0d, 0xda, 0x97, 0x92, - 0x12, 0x50, 0xd2, 0x2a, 0xa1, 0xa0, 0xb7, 0x20, 0xc3, 0x32, 0x60, 0x43, 0xa3, 0x1d, 0x2a, 0x29, - 0xa5, 0x29, 0xa1, 0x6a, 0x68, 0x65, 0x0c, 0x19, 0x6f, 0x70, 0x92, 0x66, 0xeb, 0xd9, 0x9e, 0x22, - 0xe9, 0x6f, 0xf4, 0x3e, 0x5c, 0x78, 0xdd, 0x53, 0xda, 0xfa, 0x1e, 0x5d, 0xa9, 0xa3, 0x51, 0xf7, - 0x54, 0x67, 0xac, 0x26, 0xc8, 0x4b, 0xa3, 0x12, 0xa8, 0xea, 0xdc, 0xb1, 0x9c, 0xf4, 0xc7, 0xb2, - 0xf8, 0x3b, 0x02, 0xcc, 0xb3, 0xc8, 0x23, 0x16, 0x30, 0x1b, 0x9f, 0x1f, 0xf2, 0x31, 0x64, 0x34, - 0xc5, 0x51, 0xd8, 0x06, 0xd3, 0xc4, 0xc8, 0x0d, 0xa6, 0xde, 0x26, 0x0a, 0xc5, 0x51, 0xe8, 0x26, - 0x53, 0x04, 0x29, 0xf2, 0x9b, 0xed, 0xc5, 0x95, 0xe8, 0x6f, 0xf1, 0x33, 0x40, 0xc1, 0x82, 0xc6, - 0xe9, 0x91, 0xdd, 0x83, 0x8b, 0x44, 0xd7, 0x55, 0x43, 0xb5, 0x4e, 0xba, 0x8e, 0x6e, 0x1a, 0xcf, - 0xe9, 0x5f, 0x1b, 0x15, 0x03, 0x1f, 0xb6, 0xe8, 0xf7, 0x2c, 0xf1, 0x77, 0x67, 0x20, 0x57, 0x3d, - 0xee, 0x9a, 0x56, 0xac, 0xcb, 0x60, 0xab, 0x30, 0xcb, 0x57, 0x0a, 0x46, 0x7c, 0xbb, 0xee, 0x33, - 0xe6, 0xee, 0x87, 0x7b, 0xce, 0x88, 0x56, 0x01, 0x58, 0x50, 0x2b, 0x0d, 0x5c, 0x4a, 0x9e, 0xe3, - 0x53, 0x1b, 0x65, 0xa3, 0x1b, 0x38, 0xb6, 0x20, 0xdb, 0x39, 0x54, 0x55, 0x79, 0x4f, 0x6f, 0x3b, - 0x3c, 0x36, 0x30, 0x3a, 0x8c, 0x7d, 0xf3, 0x65, 0xa5, 0xf2, 0x84, 0x66, 0x62, 0x61, 0x7a, 0xfe, - 0xb3, 0x04, 0x44, 0x02, 0xfb, 0x8d, 0xde, 0x05, 0xbe, 0x99, 0x48, 0xb6, 0xdd, 0x7d, 0x83, 0xab, - 0xb9, 0xb3, 0xd3, 0xc5, 0x8c, 0x44, 0xa9, 0x8d, 0x46, 0x53, 0xca, 0xb0, 0x0c, 0x0d, 0xdb, 0x41, - 0x37, 0x20, 0x67, 0x76, 0x74, 0x47, 0x76, 0x9d, 0x24, 0xee, 0x51, 0xce, 0x11, 0xa2, 0xeb, 0x44, - 0x9d, 0x67, 0x8f, 0xc9, 0xec, 0xc4, 0x7b, 0x4c, 0xd0, 0xdf, 0x14, 0xe0, 0x12, 0x57, 0xa4, 0xbc, - 0x4b, 0xe3, 0xf0, 0x95, 0xb6, 0xee, 0x9c, 0xc8, 0x07, 0x87, 0xa5, 0x34, 0xf5, 0x5b, 0x7f, 0x26, - 0xb2, 0x41, 0x02, 0xfd, 0x60, 0xc9, 0x6d, 0x96, 0x93, 0x67, 0x9c, 0x79, 0xe3, 0xb0, 0x6a, 0x38, - 0xd6, 0xc9, 0xea, 0xe5, 0xb3, 0xd3, 0xc5, 0x85, 0xc1, 0xd4, 0x97, 0xd2, 0x82, 0x3d, 0xc8, 0x82, - 0x6a, 0x00, 0xd8, 0xeb, 0x87, 0x74, 0xc6, 0x88, 0xf6, 0x3f, 0x22, 0x3b, 0xac, 0x14, 0xe0, 0x45, - 0x77, 0xa1, 0xc8, 0xf7, 0xf4, 0xec, 0xe9, 0x6d, 0x2c, 0xdb, 0xfa, 0x17, 0x98, 0xce, 0x2d, 0x49, - 0x29, 0xcf, 0xe8, 0x44, 0x44, 0x43, 0xff, 0x02, 0x97, 0xbf, 0x0b, 0xa5, 0x61, 0xa5, 0x0f, 0x0e, - 0x81, 0x0c, 0xfb, 0xa4, 0xfb, 0x51, 0x78, 0x3d, 0x67, 0x82, 0xae, 0xca, 0xd7, 0x74, 0x3e, 0x4e, - 0x7c, 0x24, 0x88, 0xff, 0x30, 0x01, 0xb9, 0xd5, 0x5e, 0xfb, 0xe0, 0x79, 0xb7, 0xd1, 0xeb, 0x74, - 0x14, 0xeb, 0x84, 0x98, 0x41, 0x66, 0x28, 0x48, 0x01, 0x05, 0x66, 0x06, 0xa9, 0x25, 0xd0, 0xbf, - 0xc0, 0x64, 0x72, 0x0a, 0x84, 0xcb, 0xf0, 0x7d, 0x06, 0xb4, 0x0e, 0x3e, 0x99, 0x6e, 0x05, 0xf8, - 0x08, 0x4a, 0x81, 0x8c, 0x74, 0xf1, 0x45, 0xc6, 0x86, 0x63, 0xe9, 0x98, 0x2d, 0x20, 0x26, 0xa5, - 0x40, 0x4c, 0x4f, 0x9d, 0x24, 0x57, 0x59, 0x2a, 0x6a, 0xc2, 0x1c, 0xc9, 0x78, 0x22, 0xd3, 0x29, - 0xc4, 0x5d, 0xe0, 0x7d, 0x10, 0x51, 0xad, 0x50, 0xb9, 0x97, 0xa8, 0x7e, 0x2a, 0x94, 0x87, 0xfe, - 0x94, 0xb2, 0xd8, 0xa7, 0x94, 0x3f, 0x85, 0x62, 0x7f, 0x86, 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0x17, - 0x82, 0xba, 0x4c, 0x06, 0xf4, 0xb4, 0x9e, 0x4a, 0xa7, 0x8a, 0xd3, 0xe2, 0x9f, 0x26, 0x21, 0xef, - 0x76, 0xb3, 0x38, 0x81, 0xce, 0x2a, 0x4c, 0x93, 0x4e, 0xe1, 0x46, 0xa0, 0xdc, 0x1e, 0xd1, 0xbb, - 0x79, 0x64, 0x3b, 0xe9, 0x2c, 0x2e, 0x48, 0xa6, 0xac, 0x71, 0x18, 0x9c, 0xf2, 0x2f, 0x26, 0x20, - 0x45, 0xb1, 0xc5, 0x03, 0x48, 0xd1, 0x89, 0x42, 0x98, 0x64, 0xa2, 0xa0, 0x59, 0xbd, 0xe9, 0x2c, - 0x11, 0x70, 0x4d, 0x89, 0xcf, 0xb7, 0xaf, 0x7c, 0xf0, 0xe0, 0x21, 0x35, 0x36, 0x73, 0x12, 0x7f, - 0x42, 0xab, 0x34, 0x34, 0xca, 0xb4, 0x1c, 0xac, 0x71, 0x9f, 0xfe, 0xda, 0xb8, 0xf6, 0x75, 0x27, - 0x25, 0x97, 0x0f, 0x5d, 0x81, 0x24, 0xb1, 0x62, 0xb3, 0x2c, 0xca, 0xe1, 0xec, 0x74, 0x31, 0x49, - 0xec, 0x17, 0xa1, 0xa1, 0x65, 0xc8, 0x86, 0x4d, 0x06, 0xf1, 0xe0, 0xa8, 0x61, 0x0c, 0x0c, 0x77, - 0x68, 0x7b, 0x43, 0x8b, 0xe1, 0x59, 0xde, 0xc6, 0x7f, 0x9e, 0x82, 0x5c, 0xbd, 0x13, 0xf7, 0x94, - 0xb2, 0x12, 0x6e, 0xe1, 0x28, 0x20, 0x14, 0x7a, 0x69, 0x44, 0x03, 0x87, 0x66, 0xf0, 0xe4, 0xf9, - 0x66, 0xf0, 0x3a, 0xf1, 0x94, 0xf9, 0x21, 0x11, 0xc9, 0x21, 0x98, 0x27, 0xfc, 0x7e, 0xea, 0xa7, - 0x48, 0x84, 0xc7, 0xdf, 0xeb, 0x41, 0x23, 0x55, 0x3e, 0xa5, 0x0e, 0x39, 0xeb, 0x65, 0x33, 0x93, - 0xf7, 0xb2, 0x59, 0x6c, 0x68, 0x74, 0x52, 0x0b, 0x5b, 0xd4, 0xd9, 0x37, 0xb7, 0xa8, 0x65, 0x87, - 0x77, 0xd6, 0x8f, 0x21, 0xa9, 0xe9, 0x6e, 0xe3, 0x4c, 0x3e, 0x55, 0x13, 0xa6, 0x31, 0xbd, 0x36, - 0x15, 0xec, 0xb5, 0xac, 0x97, 0x94, 0xeb, 0x00, 0xbe, 0x6e, 0xd0, 0x35, 0x98, 0x31, 0xdb, 0x9a, - 0xbb, 0xd9, 0x25, 0xb7, 0x9a, 0x39, 0x3b, 0x5d, 0x9c, 0x7e, 0xde, 0xd6, 0xea, 0x6b, 0xd2, 0xb4, - 0xd9, 0xd6, 0xea, 0x1a, 0x3d, 0xa1, 0x03, 0x1f, 0xc9, 0x5e, 0x24, 0xdc, 0x9c, 0x34, 0x6b, 0xe0, - 0xa3, 0x35, 0x6c, 0xab, 0xbc, 0xc3, 0xfd, 0xa6, 0x00, 0x79, 0x57, 0xf7, 0xf1, 0x1a, 0x95, 0xb4, - 0xde, 0xe1, 0x83, 0x2c, 0x79, 0xbe, 0x41, 0xe6, 0xf2, 0xf1, 0x8d, 0xc5, 0xbf, 0x2c, 0xf0, 0xd8, - 0xe6, 0x86, 0xaa, 0x38, 0xc4, 0xa9, 0x88, 0x71, 0x60, 0xdc, 0x83, 0xa2, 0xa5, 0x18, 0x9a, 0xd9, - 0xd1, 0xbf, 0xc0, 0x6c, 0x21, 0xd4, 0xe6, 0x9f, 0x3d, 0x0b, 0x1e, 0x9d, 0xae, 0xfa, 0xd9, 0xe2, - 0xef, 0x27, 0x78, 0x1c, 0xb4, 0x57, 0x8c, 0x38, 0xd5, 0xf5, 0x1d, 0x98, 0x0f, 0x4e, 0x6d, 0x2c, - 0xd4, 0x92, 0x8d, 0xd6, 0xf7, 0x22, 0xe4, 0x45, 0x15, 0x84, 0xc5, 0x33, 0xba, 0xc1, 0xf5, 0x81, - 0x19, 0x91, 0x0a, 0x43, 0x15, 0xc8, 0xf2, 0x8f, 0x0f, 0xc6, 0x9e, 0xe9, 0x06, 0x8a, 0xbd, 0x3d, - 0x2c, 0x2c, 0xb2, 0x6e, 0xec, 0x99, 0xee, 0x87, 0x7c, 0xcb, 0x25, 0xd8, 0xe5, 0x9f, 0x83, 0x69, - 0x9a, 0xfc, 0x06, 0x26, 0x9a, 0xb7, 0xe6, 0x9f, 0x24, 0xe0, 0x26, 0x2d, 0xfd, 0x4b, 0x6c, 0xe9, - 0x7b, 0x27, 0xdb, 0x96, 0xe9, 0x60, 0xd5, 0xc1, 0x9a, 0xbf, 0x72, 0x1e, 0xab, 0xdd, 0xcb, 0x74, - 0xdd, 0x17, 0x9c, 0x2b, 0xe0, 0xcc, 0xe3, 0x42, 0x1b, 0x50, 0xe0, 0xa1, 0x05, 0x4a, 0x5b, 0x3f, - 0xc4, 0xb2, 0xe2, 0x9c, 0x67, 0x76, 0xcb, 0x31, 0xde, 0x15, 0xc2, 0xba, 0xe2, 0x20, 0x0d, 0x32, - 0x5c, 0x98, 0xae, 0xf1, 0xc3, 0x83, 0x9e, 0x7e, 0xb5, 0x05, 0xc5, 0x34, 0x8b, 0x6f, 0xa8, 0xaf, - 0x49, 0x69, 0x26, 0xb9, 0xae, 0x89, 0xff, 0x49, 0x80, 0x5b, 0x63, 0x54, 0x1c, 0x67, 0xd7, 0x2d, - 0x43, 0xfa, 0x90, 0xbc, 0x48, 0xe7, 0x3a, 0x4e, 0x4b, 0xde, 0x33, 0xda, 0x84, 0xdc, 0x9e, 0xa2, - 0xb7, 0xfd, 0x2e, 0x3d, 0x3c, 0x3e, 0x31, 0x3a, 0x1a, 0x77, 0x8e, 0xb1, 0xb3, 0x3e, 0x2c, 0xfe, - 0x7a, 0x02, 0xe6, 0x57, 0x34, 0xad, 0xd1, 0xe0, 0x36, 0x30, 0xbe, 0x9e, 0xe2, 0x82, 0xcc, 0x84, - 0x0f, 0x32, 0xd1, 0x7b, 0x80, 0x34, 0xdd, 0x66, 0x07, 0x9f, 0xd8, 0xfb, 0x8a, 0x66, 0x1e, 0xf9, - 0x71, 0x19, 0xf3, 0x6e, 0x4a, 0xc3, 0x4d, 0x40, 0x0d, 0xa0, 0x68, 0x47, 0xb6, 0x1d, 0xc5, 0xfb, - 0xee, 0x74, 0x6b, 0xa2, 0x2d, 0x65, 0x0c, 0x06, 0x79, 0x8f, 0x52, 0x86, 0xc8, 0xa1, 0x3f, 0x89, - 0xdf, 0xae, 0x93, 0xaa, 0x3b, 0xb2, 0x62, 0xbb, 0xfb, 0x87, 0xd8, 0x91, 0x2b, 0x79, 0x46, 0x5f, - 0xb1, 0xd9, 0xb6, 0x20, 0xb6, 0xe1, 0xc1, 0x57, 0x4d, 0x9c, 0x90, 0xf8, 0xef, 0x09, 0x90, 0x97, - 0xf0, 0x9e, 0x85, 0xed, 0x58, 0x17, 0x05, 0x9e, 0xc0, 0x9c, 0xc5, 0xa4, 0xca, 0x7b, 0x96, 0xd9, - 0x39, 0xcf, 0xb8, 0xca, 0x72, 0xc6, 0x27, 0x96, 0xd9, 0xe1, 0x86, 0xe5, 0x25, 0x14, 0xbc, 0x32, - 0xc6, 0x59, 0xf9, 0xdf, 0xa1, 0xdb, 0xa5, 0x99, 0xe0, 0xb8, 0x03, 0x24, 0xe2, 0xd5, 0x00, 0xfd, - 0x50, 0x15, 0x2c, 0x68, 0x9c, 0x6a, 0xf8, 0x6f, 0x02, 0xe4, 0x1b, 0xbd, 0x5d, 0x76, 0xbe, 0x56, - 0x7c, 0x1a, 0xa8, 0x42, 0xa6, 0x8d, 0xf7, 0x1c, 0xf9, 0x8d, 0x42, 0xf5, 0xd3, 0x84, 0x95, 0x6e, - 0x54, 0x78, 0x0a, 0x60, 0xd1, 0x2d, 0x76, 0x54, 0x4e, 0xf2, 0x9c, 0x72, 0x32, 0x94, 0x97, 0x90, - 0xc5, 0x3f, 0x4f, 0x40, 0xc1, 0xab, 0x66, 0x9c, 0x56, 0xf2, 0x55, 0xc8, 0x3a, 0x24, 0xcf, 0x63, - 0x1d, 0xe6, 0x79, 0x4c, 0x48, 0xb4, 0x85, 0x58, 0x82, 0x05, 0xea, 0xb8, 0xc8, 0x4a, 0xb7, 0xdb, - 0xd6, 0x5d, 0xb8, 0x4b, 0xed, 0x4f, 0x4a, 0x9a, 0xa7, 0x49, 0x2b, 0x2c, 0x85, 0x02, 0x5d, 0xf4, - 0x2b, 0x02, 0xcc, 0xed, 0x59, 0x18, 0x7f, 0x81, 0x65, 0x0a, 0xbd, 0x26, 0x0b, 0x7a, 0x59, 0x23, - 0x65, 0xf8, 0xca, 0x1f, 0xc5, 0xb3, 0xec, 0xc5, 0x0d, 0xf2, 0x5e, 0xde, 0x69, 0x5f, 0xc1, 0x3c, - 0x6d, 0x94, 0xb8, 0xf7, 0x16, 0x8b, 0xbf, 0x9d, 0x00, 0x14, 0x94, 0xfc, 0xd3, 0x6b, 0xcc, 0x44, - 0x7c, 0x8d, 0xf9, 0x2e, 0x20, 0x16, 0x36, 0x69, 0xcb, 0x5d, 0x6c, 0xc9, 0x36, 0x56, 0x4d, 0x7e, - 0x3c, 0x94, 0x20, 0x15, 0x79, 0xca, 0x36, 0xb6, 0x1a, 0x94, 0x8e, 0x56, 0x00, 0x7c, 0x97, 0x8e, - 0xcf, 0x38, 0x93, 0x78, 0x74, 0x19, 0xcf, 0xa3, 0x13, 0xbf, 0x2f, 0x40, 0x7e, 0x53, 0x6f, 0x59, - 0x4a, 0xac, 0xa7, 0x1f, 0xa1, 0x8f, 0xc3, 0x8b, 0xe9, 0xd9, 0x87, 0xe5, 0xa8, 0x40, 0x1f, 0x96, - 0xc3, 0x85, 0x68, 0x9c, 0x81, 0x18, 0x72, 0xaf, 0x44, 0x71, 0x5a, 0xb0, 0xff, 0x50, 0x86, 0x39, - 0x5e, 0xee, 0x1d, 0x43, 0x37, 0x0d, 0xf4, 0x00, 0x92, 0x2d, 0xfe, 0xb1, 0x24, 0x1b, 0xb9, 0xb0, - 0xe9, 0x1f, 0x04, 0x58, 0x9b, 0x92, 0x48, 0x5e, 0xc2, 0xd2, 0xed, 0x39, 0x11, 0xae, 0xa4, 0x1f, - 0xfb, 0x1e, 0x64, 0xe9, 0xf6, 0x1c, 0xd4, 0x80, 0x82, 0xea, 0x1f, 0x68, 0x26, 0x13, 0xf6, 0xe4, - 0x50, 0xd8, 0x19, 0x79, 0xb4, 0x5c, 0x6d, 0x4a, 0xca, 0xab, 0xa1, 0x04, 0x54, 0x09, 0x9e, 0xa3, - 0x95, 0x1a, 0x08, 0xac, 0xf3, 0x77, 0x6d, 0x87, 0xcf, 0xf0, 0xaa, 0x4d, 0x05, 0x8e, 0xdb, 0x42, - 0x1f, 0xc3, 0x8c, 0x46, 0x4f, 0x6c, 0xe2, 0x26, 0x20, 0xaa, 0xa1, 0x43, 0x07, 0x63, 0xd5, 0xa6, - 0x24, 0xce, 0x81, 0xd6, 0x61, 0x8e, 0xfd, 0x62, 0x8e, 0x1d, 0xc7, 0xe2, 0xb7, 0x86, 0x4b, 0x08, - 0x4c, 0x9d, 0xb5, 0x29, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0xa2, 0xf1, - 0xab, 0x43, 0x0e, 0x4a, 0xf1, 0x99, 0x69, 0x6e, 0xf4, 0x98, 0x1d, 0xc3, 0xe9, 0x1c, 0xbb, 0x0b, - 0xa3, 0x51, 0xc5, 0x0f, 0x6d, 0xc8, 0x27, 0xc5, 0xc7, 0x94, 0x80, 0x9e, 0x42, 0x56, 0x21, 0x1e, - 0xb2, 0x4c, 0xb7, 0xb4, 0xd2, 0x95, 0xd0, 0xe8, 0x70, 0x84, 0x81, 0x2d, 0xc8, 0x35, 0xba, 0xeb, - 0xdf, 0x25, 0xfa, 0x82, 0x3a, 0xd8, 0x6a, 0xe1, 0x52, 0x76, 0xb4, 0xa0, 0x60, 0x38, 0x9d, 0x27, - 0x88, 0x12, 0x89, 0xa7, 0xec, 0xed, 0x27, 0xa7, 0x95, 0x9a, 0x1b, 0xfa, 0x01, 0x3c, 0x62, 0xa7, - 0x54, 0x6d, 0x4a, 0x9a, 0xdb, 0x0f, 0x90, 0xd1, 0x12, 0x24, 0x5a, 0x6a, 0x29, 0x37, 0xd4, 0x24, - 0x78, 0xbb, 0x81, 0x6a, 0x53, 0x52, 0xa2, 0xa5, 0xa2, 0x4f, 0x21, 0xcd, 0xb6, 0x76, 0x1c, 0x1b, - 0xa5, 0xfc, 0xd0, 0x31, 0x16, 0xde, 0x20, 0x53, 0x9b, 0x92, 0xe8, 0x6e, 0x12, 0xf2, 0xbe, 0x6d, - 0xc8, 0x5b, 0x2c, 0x1e, 0xd1, 0x8d, 0x1e, 0x2e, 0x0e, 0x0d, 0x0a, 0x88, 0x0a, 0x20, 0xae, 0x51, - 0xac, 0x14, 0xa0, 0xa3, 0xef, 0xc0, 0x85, 0xb0, 0x44, 0xde, 0xd3, 0xe6, 0x87, 0x7e, 0xe0, 0x1e, - 0x1a, 0xcc, 0x5a, 0x9b, 0x92, 0x90, 0x35, 0x90, 0x88, 0x3e, 0x84, 0x69, 0xd6, 0x6a, 0x88, 0x8a, - 0x8c, 0x8a, 0x73, 0xe9, 0x6b, 0x30, 0x96, 0x9f, 0x74, 0x7e, 0x87, 0x07, 0xe5, 0xc9, 0x6d, 0xb3, - 0x55, 0x5a, 0x18, 0xda, 0xf9, 0x07, 0xc3, 0x0b, 0x49, 0xe7, 0x77, 0x7c, 0x2a, 0x69, 0x77, 0x8b, - 0xa5, 0xf0, 0x40, 0xac, 0x0b, 0x43, 0xdb, 0x3d, 0x22, 0x56, 0xaf, 0x46, 0xb7, 0x4b, 0xf8, 0x64, - 0x52, 0x34, 0x8b, 0x9d, 0xed, 0x23, 0xd3, 0x31, 0x75, 0x71, 0x68, 0xd1, 0x06, 0x0f, 0x3f, 0xaa, - 0x51, 0xaf, 0xd2, 0xa3, 0xa2, 0x97, 0x50, 0xe4, 0x27, 0x70, 0xf8, 0x5f, 0x61, 0x2e, 0x51, 0x79, - 0xf7, 0x22, 0x4d, 0x57, 0x54, 0x14, 0x53, 0x6d, 0x4a, 0x2a, 0xa8, 0xe1, 0x14, 0xf4, 0x19, 0xcc, - 0x53, 0x79, 0xb2, 0xea, 0x1f, 0x9a, 0x52, 0x2a, 0x0d, 0x1c, 0xc1, 0x31, 0xfc, 0x7c, 0x15, 0x57, - 0x72, 0x51, 0xed, 0x4b, 0x22, 0xdd, 0x58, 0x37, 0x74, 0x87, 0x5a, 0xd9, 0xf2, 0xd0, 0x6e, 0x1c, - 0x3e, 0x00, 0x92, 0x74, 0x63, 0x9d, 0x51, 0x48, 0x37, 0x76, 0x78, 0x94, 0x1e, 0x6f, 0x8e, 0xb7, - 0x87, 0x76, 0xe3, 0xa8, 0x70, 0x3e, 0xd2, 0x8d, 0x9d, 0x20, 0x9d, 0x74, 0x63, 0x66, 0x20, 0xfa, - 0xe4, 0xbe, 0x33, 0xb4, 0x1b, 0x0f, 0xdd, 0x5a, 0x4e, 0xba, 0xb1, 0x32, 0x90, 0x88, 0xd6, 0x00, - 0x98, 0x03, 0x48, 0xbd, 0x80, 0xab, 0x43, 0x27, 0x83, 0xfe, 0x68, 0x3d, 0x32, 0x19, 0xb4, 0x5d, - 0x1a, 0x31, 0x64, 0x14, 0x5e, 0xca, 0xf4, 0xa3, 0x73, 0x69, 0x71, 0xa8, 0x21, 0x1b, 0xf8, 0x40, - 0x4c, 0x0c, 0xd9, 0x91, 0x47, 0x24, 0xb3, 0x0a, 0x5b, 0x25, 0x2f, 0x5d, 0x1b, 0x6e, 0x96, 0x83, - 0x1f, 0xcb, 0xa8, 0x59, 0xa6, 0x04, 0xb4, 0x02, 0x19, 0xe2, 0xe4, 0x9c, 0x50, 0x33, 0x74, 0x7d, - 0xa8, 0x17, 0xdf, 0xb7, 0x1f, 0xa8, 0x36, 0x25, 0xa5, 0x5f, 0x73, 0x12, 0x79, 0x3d, 0x5b, 0x3f, - 0x2c, 0x89, 0x43, 0x5f, 0x1f, 0x5a, 0x6b, 0x26, 0xaf, 0x67, 0x1c, 0x48, 0x85, 0x8b, 0xac, 0xad, - 0xf8, 0x2e, 0x6f, 0x8b, 0x6f, 0x4c, 0x2e, 0xdd, 0xa0, 0xa2, 0x86, 0x2e, 0xc4, 0x45, 0x6e, 0x3e, - 0xaf, 0x4d, 0x49, 0x0b, 0xca, 0x60, 0x2a, 0x19, 0xf0, 0x7c, 0xea, 0x61, 0xcb, 0x77, 0xa5, 0x9b, - 0x43, 0x07, 0x7c, 0xc4, 0xaa, 0x27, 0x19, 0xf0, 0x4a, 0x80, 0xcc, 0x26, 0x20, 0x4d, 0xb6, 0x6d, - 0x16, 0xa2, 0x70, 0x6b, 0xc4, 0x04, 0xd4, 0xb7, 0x6e, 0xc2, 0x26, 0x20, 0xad, 0xc1, 0x38, 0x89, - 0x20, 0xb5, 0x8d, 0x15, 0x8b, 0x9b, 0xd9, 0xdb, 0x43, 0x05, 0x0d, 0x9c, 0xb5, 0x48, 0x04, 0xa9, - 0x1e, 0x91, 0x38, 0x3c, 0x96, 0x7b, 0x74, 0x0f, 0xf7, 0x90, 0xef, 0x0c, 0x75, 0x78, 0x22, 0xcf, - 0x16, 0x22, 0x0e, 0x8f, 0x15, 0x4a, 0x40, 0x3f, 0x0b, 0xb3, 0x1c, 0xf0, 0x96, 0xee, 0x8e, 0x70, - 0x01, 0x83, 0x6b, 0x14, 0x64, 0x5c, 0x73, 0x1e, 0x66, 0x65, 0x19, 0xd0, 0x66, 0xd5, 0xbb, 0x37, - 0xc2, 0xca, 0x0e, 0x60, 0x7d, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, 0xac, 0x9f, 0xf2, 0xb9, 0xee, - 0xfe, 0x50, 0x2b, 0x3b, 0xb8, 0xdb, 0x88, 0x58, 0xd9, 0xd7, 0x3e, 0x95, 0xd4, 0xcc, 0x66, 0x80, - 0xb3, 0xf4, 0x8d, 0xa1, 0x35, 0x0b, 0x23, 0x6f, 0x52, 0x33, 0xce, 0x43, 0x9a, 0x8d, 0x61, 0x00, - 0xa6, 0xe9, 0x77, 0x87, 0x1f, 0xa5, 0xd0, 0x8f, 0xb2, 0x6a, 0xee, 0xd2, 0x2e, 0xd3, 0xb0, 0x67, - 0xa8, 0x2c, 0xbe, 0xcf, 0x9b, 0x6b, 0xea, 0xbd, 0xd1, 0x86, 0x2a, 0x6a, 0x7f, 0xbc, 0x67, 0xa8, - 0x42, 0x89, 0xb4, 0xa8, 0x6c, 0x6b, 0x1f, 0x1d, 0xdf, 0x4b, 0x23, 0x4e, 0x7d, 0xe8, 0xdb, 0x60, - 0x49, 0x8b, 0xea, 0x11, 0xfd, 0x21, 0xd4, 0x63, 0x47, 0x92, 0x94, 0x96, 0x47, 0x0f, 0xa1, 0xf0, - 0xa1, 0x28, 0xde, 0x10, 0xe2, 0x64, 0x6f, 0xce, 0x74, 0x3d, 0x8c, 0xf7, 0x47, 0xcf, 0x99, 0xfd, - 0xae, 0x05, 0x9b, 0x33, 0xb9, 0x4f, 0xf1, 0x37, 0x04, 0xb8, 0xc6, 0xca, 0x46, 0xd7, 0x40, 0x4f, - 0x64, 0x6f, 0x25, 0x39, 0xb0, 0xcd, 0xe4, 0x01, 0x7d, 0xc1, 0x87, 0xc3, 0x8a, 0x3b, 0x66, 0x65, - 0xbc, 0x36, 0x25, 0xbd, 0xa3, 0x8c, 0xca, 0x47, 0xba, 0x54, 0x87, 0x21, 0xa8, 0xd2, 0xc3, 0xa1, - 0x5d, 0x2a, 0x8c, 0xfa, 0x48, 0x97, 0xe2, 0x3c, 0xab, 0xb3, 0xfc, 0x3b, 0xb4, 0xb7, 0x2d, 0xb7, - 0x50, 0x2c, 0xae, 0xa7, 0xd2, 0x97, 0x8b, 0xa5, 0xf5, 0x54, 0xfa, 0x4a, 0xb1, 0xbc, 0x9e, 0x4a, - 0xbf, 0x55, 0x7c, 0x5b, 0xfc, 0x47, 0x65, 0xc8, 0xb9, 0xa0, 0x8b, 0x01, 0xaa, 0x87, 0x41, 0x40, - 0x75, 0x75, 0x18, 0xa0, 0xe2, 0x30, 0x8d, 0x23, 0xaa, 0x87, 0x41, 0x44, 0x75, 0x75, 0x18, 0xa2, - 0xf2, 0x79, 0x08, 0xa4, 0x6a, 0x0e, 0x83, 0x54, 0xf7, 0x26, 0x80, 0x54, 0x9e, 0xa8, 0x7e, 0x4c, - 0xb5, 0x36, 0x88, 0xa9, 0x6e, 0x8e, 0xc6, 0x54, 0x9e, 0xa8, 0x00, 0xa8, 0x7a, 0xdc, 0x07, 0xaa, - 0xae, 0x8f, 0x00, 0x55, 0x1e, 0xbf, 0x8b, 0xaa, 0x36, 0x22, 0x51, 0xd5, 0xed, 0x71, 0xa8, 0xca, - 0x93, 0x13, 0x82, 0x55, 0x1f, 0x84, 0x60, 0xd5, 0xe2, 0x50, 0x58, 0xe5, 0x71, 0x33, 0x5c, 0xf5, - 0x49, 0x3f, 0xae, 0xba, 0x3e, 0x02, 0x57, 0xf9, 0x35, 0xe0, 0xc0, 0xaa, 0x16, 0x05, 0xac, 0x6e, - 0x8d, 0x01, 0x56, 0x9e, 0x94, 0x20, 0xb2, 0xaa, 0x45, 0x21, 0xab, 0x5b, 0x63, 0x90, 0x55, 0x9f, - 0x24, 0x06, 0xad, 0xb6, 0xa2, 0xa1, 0xd5, 0x9d, 0xb1, 0xd0, 0xca, 0x93, 0x16, 0xc6, 0x56, 0xcb, - 0x01, 0x6c, 0xf5, 0xce, 0x10, 0x6c, 0xe5, 0xb1, 0x12, 0x70, 0xf5, 0xad, 0x01, 0x70, 0x25, 0x8e, - 0x02, 0x57, 0x1e, 0xaf, 0x87, 0xae, 0x5e, 0x0c, 0x41, 0x57, 0x77, 0xc7, 0xa3, 0x2b, 0x4f, 0x58, - 0x1f, 0xbc, 0x52, 0x46, 0xc2, 0xab, 0xf7, 0x26, 0x84, 0x57, 0x9e, 0xf4, 0x28, 0x7c, 0xf5, 0x51, - 0x18, 0x5f, 0x5d, 0x1b, 0x8e, 0xaf, 0x3c, 0x31, 0x1c, 0x60, 0x6d, 0x44, 0x02, 0xac, 0xdb, 0xe3, - 0x00, 0x96, 0x3f, 0x0e, 0x82, 0x08, 0x6b, 0x2b, 0x1a, 0x61, 0xdd, 0x19, 0x8b, 0xb0, 0xfc, 0xe6, - 0x0f, 0x41, 0xac, 0x8d, 0x48, 0x88, 0x75, 0x7b, 0x1c, 0xc4, 0xf2, 0x0b, 0x17, 0xc4, 0x58, 0xaf, - 0x86, 0x62, 0xac, 0xfb, 0x93, 0x60, 0x2c, 0x4f, 0xe8, 0x00, 0xc8, 0xfa, 0x7c, 0x38, 0xc8, 0xfa, - 0xc6, 0x39, 0x0e, 0xb1, 0x8c, 0x44, 0x59, 0xdf, 0x1a, 0x40, 0x59, 0xe2, 0x28, 0x94, 0xe5, 0xf7, - 0x67, 0x17, 0x66, 0x29, 0x23, 0x41, 0xd1, 0x7b, 0x13, 0x82, 0x22, 0xbf, 0xf3, 0x45, 0xa0, 0xa2, - 0x6a, 0x04, 0x2a, 0xba, 0x39, 0x1a, 0x15, 0xf9, 0xe6, 0xdc, 0x87, 0x45, 0xb5, 0x28, 0x58, 0x74, - 0x6b, 0x0c, 0x2c, 0xf2, 0xad, 0x50, 0x00, 0x17, 0x3d, 0xee, 0xc3, 0x45, 0xd7, 0xc7, 0x86, 0x59, - 0x05, 0x80, 0xd1, 0xea, 0x20, 0x30, 0xba, 0x31, 0x12, 0x18, 0x79, 0x12, 0x7c, 0x64, 0xf4, 0xb8, - 0x0f, 0x19, 0x5d, 0x1f, 0x81, 0x8c, 0xfc, 0x02, 0x70, 0x68, 0xa4, 0x8d, 0x86, 0x46, 0x4b, 0x93, - 0x42, 0x23, 0x4f, 0x70, 0x24, 0x36, 0xda, 0x8a, 0xc6, 0x46, 0x77, 0x26, 0x8c, 0x80, 0x18, 0x00, - 0x47, 0xb5, 0x28, 0x70, 0x74, 0x6b, 0x0c, 0x38, 0x0a, 0xce, 0x21, 0x1e, 0x3a, 0xaa, 0x45, 0xa1, - 0xa3, 0x5b, 0x63, 0xd0, 0x91, 0x2f, 0x29, 0x00, 0x8f, 0x9a, 0xc3, 0xe0, 0xd1, 0xbd, 0x09, 0xe0, - 0x91, 0xef, 0xbc, 0xf4, 0xe1, 0xa3, 0x4f, 0xfb, 0xf1, 0x91, 0x38, 0x0a, 0x1f, 0xf9, 0x23, 0xd2, - 0x05, 0x48, 0x5b, 0xd1, 0x00, 0xe9, 0xce, 0x58, 0x80, 0x14, 0x34, 0x92, 0x01, 0x84, 0xb4, 0x11, - 0x89, 0x90, 0x6e, 0x8f, 0x43, 0x48, 0xbe, 0x91, 0x0c, 0x42, 0xa4, 0x4f, 0xfb, 0x21, 0x92, 0x38, - 0x0a, 0x22, 0xf9, 0x95, 0x73, 0x31, 0x52, 0x2d, 0x0a, 0x23, 0xdd, 0x1a, 0x83, 0x91, 0xfc, 0xc6, - 0x0b, 0x80, 0x24, 0x65, 0x24, 0x48, 0x7a, 0x6f, 0x42, 0x90, 0xd4, 0x67, 0xb8, 0xc2, 0x28, 0xa9, - 0x16, 0x85, 0x92, 0x6e, 0x8d, 0x41, 0x49, 0x81, 0xc2, 0xfa, 0x30, 0x69, 0x2b, 0x1a, 0x26, 0xdd, - 0x19, 0x0b, 0x93, 0xfa, 0x46, 0x93, 0x8b, 0x93, 0x36, 0x22, 0x71, 0xd2, 0xed, 0x71, 0x38, 0xa9, - 0x6f, 0xe2, 0xe3, 0xce, 0xc1, 0x2f, 0x4d, 0x0e, 0x94, 0x3e, 0x3a, 0x3f, 0x50, 0xf2, 0xde, 0x39, - 0x06, 0x29, 0x7d, 0xda, 0x8f, 0x94, 0xc4, 0x51, 0x48, 0xc9, 0xef, 0x59, 0xe7, 0x83, 0x4a, 0xeb, - 0xa9, 0xf4, 0xdb, 0xc5, 0x77, 0xc4, 0x3f, 0x9b, 0x81, 0x99, 0x9a, 0x17, 0x98, 0xe4, 0xd7, 0x52, - 0x78, 0x93, 0x93, 0xb0, 0xd0, 0x1a, 0x19, 0xf1, 0xd4, 0x6e, 0x8e, 0x3f, 0x3f, 0x71, 0xf0, 0x84, - 0x3f, 0xce, 0xfa, 0x06, 0x5b, 0xd2, 0xd1, 0x07, 0x90, 0xeb, 0xd9, 0xd8, 0x92, 0xbb, 0x96, 0x6e, - 0x5a, 0xba, 0xc3, 0xb6, 0xdd, 0x08, 0xab, 0xc5, 0x2f, 0x4f, 0x17, 0xe7, 0x76, 0x6c, 0x6c, 0x6d, - 0x73, 0xba, 0x34, 0xd7, 0x0b, 0x3c, 0xb9, 0xf7, 0x92, 0x4d, 0x4f, 0x7e, 0x2f, 0xd9, 0x0b, 0x28, - 0x5a, 0x58, 0xd1, 0x42, 0x1e, 0x0c, 0x3b, 0x75, 0x2a, 0xba, 0xcf, 0xd1, 0x6d, 0x71, 0x6e, 0x4e, - 0x7a, 0xfa, 0x54, 0xc1, 0x0a, 0x13, 0xd1, 0x03, 0xb8, 0xd8, 0x51, 0x8e, 0x69, 0x10, 0xab, 0xec, - 0x3a, 0x85, 0x34, 0x30, 0x95, 0x5d, 0x26, 0x86, 0x3a, 0xca, 0x31, 0xbd, 0xe4, 0x8c, 0x25, 0xd1, - 0x5b, 0x4f, 0x6e, 0x41, 0x5e, 0xd3, 0x6d, 0x47, 0x37, 0x54, 0x87, 0x1f, 0x68, 0xcc, 0x0e, 0x03, - 0xce, 0xb9, 0x54, 0x76, 0x6a, 0x71, 0x05, 0x0a, 0x2d, 0xc5, 0xc1, 0x47, 0xca, 0x89, 0xec, 0xee, - 0x87, 0xcb, 0xd2, 0xb3, 0x3b, 0xdf, 0x3a, 0x3b, 0x5d, 0xcc, 0x3d, 0x65, 0x49, 0x03, 0xdb, 0xe2, - 0x72, 0xad, 0x40, 0x82, 0x86, 0xee, 0x40, 0x41, 0xb1, 0x4f, 0x0c, 0x95, 0x56, 0x19, 0x1b, 0x76, - 0xcf, 0xa6, 0x30, 0x23, 0x2d, 0xe5, 0x29, 0xb9, 0xe2, 0x52, 0xd1, 0x75, 0x98, 0xe3, 0xe1, 0xfc, - 0xec, 0x76, 0xa5, 0x02, 0x2d, 0x3e, 0xbf, 0xb6, 0x83, 0x5d, 0xb0, 0xf4, 0x18, 0xca, 0xfc, 0x12, - 0x82, 0x23, 0xc5, 0xd2, 0x64, 0xaa, 0x49, 0xbf, 0xcf, 0x15, 0xa9, 0xd8, 0xcb, 0xec, 0xd2, 0x01, - 0x92, 0x81, 0xa8, 0xcf, 0x1f, 0x17, 0x4d, 0x98, 0x57, 0xdb, 0xba, 0x87, 0x0a, 0x98, 0x13, 0x35, - 0x3f, 0x74, 0x84, 0x54, 0x68, 0xde, 0xfe, 0xcf, 0xcc, 0x05, 0x35, 0x4c, 0x46, 0x0d, 0xa0, 0x27, - 0xf7, 0xc8, 0x5d, 0xb3, 0xad, 0xab, 0x27, 0x14, 0x16, 0x84, 0x8f, 0x54, 0x1f, 0x79, 0xb1, 0xc1, - 0x2b, 0x45, 0x77, 0xb6, 0x29, 0xa7, 0x04, 0x47, 0xde, 0x6f, 0x76, 0x1c, 0xf2, 0x7a, 0x2a, 0x0d, - 0xc5, 0xec, 0x7a, 0x2a, 0x3d, 0x57, 0xcc, 0xad, 0xa7, 0xd2, 0xf9, 0x62, 0x41, 0xfc, 0x5b, 0x02, - 0x14, 0xfa, 0xca, 0x85, 0x6a, 0x70, 0x51, 0xf3, 0x86, 0x82, 0xcc, 0xf7, 0x7e, 0xe9, 0xa6, 0xc1, - 0xcf, 0x8a, 0x5f, 0xf8, 0xf2, 0x74, 0xb1, 0x40, 0x73, 0x3f, 0xf5, 0x92, 0xa4, 0x0b, 0x3e, 0x87, - 0x4f, 0x45, 0x1f, 0x41, 0x9e, 0xb9, 0x97, 0xde, 0x45, 0x7f, 0x34, 0x68, 0x7f, 0x75, 0xfe, 0xcb, - 0xd3, 0xc5, 0x1c, 0xf5, 0x29, 0xdd, 0xb3, 0x9c, 0xa5, 0x5c, 0x3b, 0xf8, 0x28, 0xfe, 0x9a, 0x00, - 0x73, 0xa1, 0xfd, 0x55, 0x8f, 0xfb, 0x3e, 0x6e, 0x5f, 0x89, 0xc6, 0xa5, 0xc3, 0x22, 0x1c, 0xd3, - 0xbc, 0x1f, 0xbb, 0xe1, 0xa2, 0x8b, 0xc3, 0x71, 0x0d, 0x5d, 0xa5, 0x71, 0x63, 0x67, 0x5c, 0xb6, - 0x8f, 0x53, 0x3f, 0xf8, 0xe1, 0xe2, 0x94, 0xf8, 0x17, 0x29, 0xc8, 0x85, 0x77, 0x53, 0xd5, 0xfb, - 0xca, 0x15, 0x35, 0x6f, 0x84, 0x38, 0x96, 0x46, 0xdc, 0x28, 0x93, 0xf1, 0x2f, 0x65, 0x60, 0xc5, - 0xbc, 0x36, 0xe2, 0x13, 0x7e, 0xb0, 0x9c, 0x3e, 0x63, 0xf9, 0x3f, 0x26, 0x3d, 0xfb, 0xb9, 0x04, - 0xd3, 0xf4, 0x60, 0x24, 0x5e, 0xb4, 0x52, 0x7f, 0x1f, 0x22, 0xbe, 0x34, 0x49, 0x97, 0x58, 0x36, - 0x62, 0x6f, 0x9b, 0x6f, 0x74, 0xf2, 0xa0, 0x3f, 0x24, 0xce, 0x7f, 0xaf, 0x62, 0x8f, 0x9d, 0x3c, - 0xf9, 0xff, 0x30, 0xbe, 0x86, 0xbc, 0x0f, 0xfd, 0x02, 0x14, 0x54, 0xb3, 0xdd, 0x66, 0x73, 0x29, - 0xb3, 0x58, 0x83, 0x67, 0xd1, 0xd0, 0x22, 0xf0, 0xab, 0x34, 0x97, 0xbc, 0x2b, 0x35, 0x97, 0x24, - 0x7e, 0xa5, 0x66, 0x20, 0x78, 0x37, 0xef, 0x09, 0x73, 0x0d, 0x5d, 0x28, 0x8e, 0x78, 0xf6, 0x4d, - 0xe2, 0x88, 0x59, 0xec, 0x39, 0xef, 0x79, 0x7f, 0x24, 0xf0, 0x40, 0x9d, 0x67, 0xa6, 0x79, 0xd0, - 0xf3, 0xe2, 0x7f, 0xcb, 0xc1, 0x73, 0x24, 0xd3, 0x5f, 0x9e, 0x2e, 0xa6, 0x24, 0xef, 0x20, 0xc9, - 0xa8, 0x99, 0x21, 0xf1, 0xd5, 0x66, 0x86, 0xeb, 0x30, 0xd7, 0xb5, 0xf0, 0x1e, 0x76, 0xd4, 0x7d, - 0xd9, 0xe8, 0x75, 0xf8, 0x36, 0xa1, 0xac, 0x4b, 0xdb, 0xea, 0x75, 0xd0, 0x3d, 0x28, 0x7a, 0x59, - 0x38, 0x86, 0x77, 0x0f, 0x1a, 0x73, 0xe9, 0x1c, 0xf1, 0x8b, 0xff, 0x4b, 0x80, 0x85, 0x50, 0x9d, - 0xf8, 0x98, 0x5a, 0x87, 0xac, 0x6f, 0x4e, 0xec, 0x92, 0x70, 0xce, 0x68, 0xd8, 0x20, 0x33, 0x92, - 0xe1, 0x92, 0xfb, 0x5a, 0x7a, 0x91, 0x81, 0x2f, 0x36, 0x71, 0x4e, 0xb1, 0x17, 0x7d, 0x39, 0x6b, - 0x81, 0x17, 0x78, 0x83, 0x2c, 0x39, 0xd1, 0x20, 0x13, 0x7f, 0x53, 0x80, 0x22, 0x7d, 0xc1, 0x13, - 0x8c, 0xb5, 0x58, 0xac, 0x9b, 0x1b, 0x65, 0x9e, 0x98, 0x7c, 0x23, 0x50, 0xe8, 0xf2, 0x95, 0x64, - 0xf8, 0xf2, 0x15, 0xf1, 0x87, 0x02, 0xe4, 0xbd, 0x12, 0xb2, 0x0b, 0x0f, 0x47, 0x1c, 0x57, 0xfa, - 0x66, 0x97, 0xfa, 0xb9, 0xc7, 0xaa, 0x4c, 0x74, 0x07, 0x63, 0xf0, 0x58, 0x15, 0x76, 0x19, 0xdd, - 0xdf, 0x71, 0x7b, 0x0e, 0x29, 0x62, 0xc5, 0x3f, 0x3e, 0xe3, 0x0d, 0xf6, 0x44, 0x49, 0xf4, 0x5e, - 0x57, 0xb3, 0x7d, 0xc8, 0x4e, 0xc2, 0x99, 0xc8, 0xec, 0x21, 0x1e, 0x9e, 0x06, 0x7c, 0xb5, 0x4f, - 0x6b, 0x36, 0xe8, 0x8d, 0xaf, 0xec, 0xb7, 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x36, 0x3e, 0xd1, 0xd2, - 0x44, 0xa6, 0xd8, 0xd5, 0x12, 0xeb, 0x2b, 0x7f, 0x10, 0x6c, 0x89, 0xea, 0x21, 0x41, 0x79, 0x8f, - 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x45, 0x78, 0x85, 0x5a, 0x4e, 0x22, 0xb9, 0xd1, 0x93, 0xd0, 0xa9, - 0x23, 0x89, 0xe1, 0x88, 0x64, 0x50, 0xa5, 0xa1, 0xd3, 0x49, 0x3e, 0x0c, 0xf7, 0xf5, 0x91, 0xaf, - 0x0f, 0x76, 0xfa, 0x8f, 0x53, 0x3f, 0xfa, 0xe1, 0xa2, 0x20, 0x7e, 0x02, 0x48, 0xc2, 0x36, 0x76, - 0x5e, 0xf4, 0x4c, 0xcb, 0x3f, 0xc1, 0xe5, 0x76, 0xdf, 0x6d, 0x34, 0xd3, 0xab, 0xd9, 0xb3, 0xa8, - 0x43, 0x9e, 0x2e, 0xc2, 0x42, 0x88, 0x9b, 0x19, 0x0b, 0xf1, 0x43, 0xb8, 0xf2, 0xd4, 0xb4, 0x6d, - 0xbd, 0x4b, 0xa0, 0x2d, 0x1d, 0x95, 0x64, 0x6a, 0xf1, 0xcc, 0x63, 0xba, 0x4b, 0x17, 0x39, 0x0c, - 0x66, 0x46, 0x32, 0x92, 0xf7, 0x2c, 0xfe, 0xae, 0x00, 0x97, 0x07, 0x39, 0x99, 0x96, 0xa3, 0xb6, - 0x70, 0xce, 0xaa, 0xa6, 0x7f, 0xe2, 0xdf, 0xf8, 0xde, 0xea, 0x66, 0x27, 0xee, 0x2a, 0x7f, 0xa7, - 0xdc, 0x51, 0xa8, 0xf9, 0xe0, 0xdb, 0xc9, 0xf3, 0x9c, 0xbc, 0xc9, 0xa8, 0xbe, 0x25, 0x49, 0x4d, - 0x66, 0x49, 0x9a, 0x50, 0x58, 0x37, 0x75, 0x83, 0x78, 0xc5, 0x6e, 0x7d, 0x57, 0x20, 0xbf, 0xab, - 0x1b, 0x8a, 0x75, 0x22, 0xbb, 0x81, 0x85, 0xc2, 0xb8, 0xc0, 0x42, 0x29, 0xc7, 0x38, 0xf8, 0xa3, - 0xf8, 0x63, 0x01, 0x8a, 0xbe, 0x58, 0x6e, 0x91, 0xdf, 0x05, 0x50, 0xdb, 0x3d, 0xdb, 0xc1, 0x96, - 0xdb, 0x4a, 0x73, 0x2c, 0x1c, 0xbf, 0xc2, 0xa8, 0xf5, 0x35, 0x29, 0xc3, 0x33, 0xd4, 0x35, 0x74, - 0x23, 0x7c, 0xda, 0xc5, 0xf4, 0x2a, 0x9c, 0x0d, 0x9c, 0x71, 0x41, 0x9a, 0xdd, 0x76, 0x4c, 0xcb, - 0xc3, 0x54, 0xbc, 0xd9, 0xdd, 0xf3, 0xfb, 0xe9, 0x26, 0x6d, 0x92, 0x6f, 0x05, 0xf2, 0xc4, 0x5d, - 0x38, 0xc4, 0x5e, 0x95, 0x52, 0xe3, 0xab, 0xc4, 0x38, 0xdc, 0x2a, 0xfd, 0x2b, 0xe2, 0xe4, 0xb2, - 0xd6, 0xf0, 0x5a, 0x78, 0x84, 0x45, 0x5b, 0x83, 0xb4, 0x73, 0x6c, 0xc8, 0x1d, 0xec, 0xdd, 0x79, - 0x73, 0x8e, 0xe3, 0xf8, 0x66, 0x1d, 0xf6, 0x48, 0xaf, 0x45, 0xe4, 0xf7, 0x67, 0xf3, 0xe1, 0x72, - 0x65, 0x89, 0x5d, 0xb0, 0xbd, 0xe4, 0x5e, 0xb0, 0xbd, 0xb4, 0xc6, 0x33, 0x30, 0xa3, 0xfe, 0x83, - 0xff, 0xba, 0x28, 0x48, 0x1e, 0x13, 0x9b, 0xf7, 0xef, 0x37, 0x48, 0xaf, 0x1f, 0x98, 0x99, 0x51, - 0x1e, 0x20, 0x70, 0x99, 0x11, 0xbf, 0xb2, 0x79, 0x65, 0x4d, 0xde, 0xd9, 0xaa, 0x3c, 0xdf, 0xdc, - 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x17, 0xba, 0x0a, 0x29, 0xc1, 0x2e, 0x71, - 0xbe, 0xff, 0x57, 0x00, 0xfc, 0x5b, 0xd5, 0x88, 0xac, 0x8d, 0xea, 0x67, 0xf2, 0xcb, 0x95, 0x67, - 0x3b, 0xd5, 0x46, 0x71, 0x0a, 0x21, 0xc8, 0xaf, 0xae, 0x34, 0x2b, 0x35, 0x59, 0xaa, 0x36, 0xb6, - 0x9f, 0x6f, 0x35, 0xaa, 0xee, 0xe5, 0xcf, 0xf7, 0xd7, 0x60, 0x2e, 0x78, 0xa6, 0x11, 0x5a, 0x80, - 0x42, 0xa5, 0x56, 0xad, 0x6c, 0xc8, 0x2f, 0xeb, 0x2b, 0xf2, 0x8b, 0x9d, 0xea, 0x4e, 0xb5, 0x38, - 0x45, 0x8b, 0x46, 0x89, 0x4f, 0x76, 0x9e, 0x3d, 0x2b, 0x0a, 0xa8, 0x00, 0x59, 0xf6, 0x4c, 0xaf, - 0x4d, 0x2a, 0x26, 0xee, 0x6f, 0x42, 0x36, 0x70, 0xf8, 0x31, 0x79, 0xdd, 0xf6, 0x4e, 0xa3, 0x26, - 0x37, 0xeb, 0x9b, 0xd5, 0x46, 0x73, 0x65, 0x73, 0x9b, 0xc9, 0xa0, 0xb4, 0x95, 0xd5, 0xe7, 0x52, - 0xb3, 0x28, 0x78, 0xcf, 0xcd, 0xe7, 0x3b, 0x95, 0x9a, 0x5b, 0x0d, 0x31, 0x95, 0x4e, 0x16, 0x93, - 0xf7, 0xff, 0xba, 0x00, 0x97, 0x87, 0x9c, 0xef, 0x83, 0xb2, 0x30, 0xbb, 0x63, 0xd0, 0x93, 0x60, - 0x8b, 0x53, 0x28, 0x17, 0x38, 0xe2, 0xa7, 0x28, 0xa0, 0x34, 0x3b, 0x64, 0xa5, 0x98, 0x40, 0x33, - 0x90, 0x68, 0x3c, 0x2a, 0x26, 0x49, 0x49, 0x03, 0x27, 0xe4, 0x14, 0x53, 0x28, 0xc3, 0xcf, 0xe6, - 0x28, 0x4e, 0xa3, 0x39, 0xff, 0x88, 0x8c, 0xe2, 0x0c, 0x11, 0xe5, 0x1d, 0x35, 0x51, 0x9c, 0xbd, - 0x7f, 0x1d, 0x02, 0xdb, 0xf9, 0x11, 0xc0, 0xcc, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, 0x42, 0xb3, - 0x90, 0x5c, 0x69, 0xb7, 0x8b, 0xc2, 0xc3, 0x7f, 0x99, 0x82, 0xb4, 0x7b, 0x3d, 0x10, 0x7a, 0x06, - 0xd3, 0x6c, 0xb9, 0x7b, 0x71, 0x38, 0x5a, 0xa0, 0x03, 0xba, 0x7c, 0x6d, 0x1c, 0x9c, 0x10, 0xa7, - 0xd0, 0x5f, 0x85, 0x6c, 0xc0, 0x8b, 0x42, 0x43, 0x97, 0xec, 0x42, 0x9e, 0x63, 0xf9, 0xf6, 0xb8, - 0x6c, 0x9e, 0xfc, 0x57, 0x90, 0xf1, 0xac, 0x3a, 0xba, 0x31, 0xca, 0xe6, 0xbb, 0xb2, 0x47, 0x4f, - 0x0c, 0x64, 0xfc, 0x89, 0x53, 0xef, 0x0b, 0xc8, 0x02, 0x34, 0x68, 0x80, 0x51, 0x54, 0x10, 0xc5, - 0x50, 0x0b, 0x5f, 0xbe, 0x3f, 0x51, 0x6e, 0xff, 0x9d, 0x44, 0x59, 0xfe, 0x2c, 0x12, 0xad, 0xac, - 0x81, 0x39, 0x2a, 0x5a, 0x59, 0x11, 0x93, 0xd1, 0x14, 0x7a, 0x01, 0x29, 0x62, 0x3d, 0x51, 0x94, - 0x5f, 0xd9, 0x67, 0xad, 0xcb, 0x37, 0x46, 0xe6, 0x71, 0x45, 0xae, 0xde, 0xfb, 0xd1, 0x9f, 0x5e, - 0x9d, 0xfa, 0xd1, 0xd9, 0x55, 0xe1, 0xc7, 0x67, 0x57, 0x85, 0x3f, 0x3e, 0xbb, 0x2a, 0xfc, 0xc9, - 0xd9, 0x55, 0xe1, 0xfb, 0x3f, 0xb9, 0x3a, 0xf5, 0xe3, 0x9f, 0x5c, 0x9d, 0xfa, 0xe3, 0x9f, 0x5c, - 0x9d, 0xfa, 0x7c, 0x96, 0x73, 0xef, 0xce, 0x50, 0xd3, 0xf2, 0xe8, 0xff, 0x06, 0x00, 0x00, 0xff, - 0xff, 0xa7, 0x2a, 0x1d, 0x85, 0xf6, 0x80, 0x00, 0x00, + 0xa7, 0x6d, 0x54, 0x3f, 0x93, 0x9f, 0xd5, 0x37, 0xeb, 0xcd, 0xa2, 0x50, 0x4e, 0x7d, 0xf7, 0x87, + 0x57, 0xa7, 0xd6, 0x53, 0xe9, 0x99, 0xe2, 0xac, 0xf8, 0x43, 0x01, 0xe0, 0x29, 0x76, 0x78, 0x6f, + 0x40, 0xab, 0x30, 0xb3, 0x4f, 0x8b, 0x55, 0x12, 0xa8, 0x5a, 0xae, 0x45, 0x96, 0x3f, 0xd0, 0x73, + 0x56, 0xd3, 0x3f, 0x3a, 0x5d, 0x9c, 0xfa, 0xf1, 0xe9, 0xa2, 0x20, 0x71, 0x4e, 0xf4, 0x02, 0xb2, + 0x07, 0xf8, 0x44, 0xe6, 0x63, 0xad, 0x94, 0xa0, 0x8a, 0x78, 0x3f, 0x20, 0xe8, 0xe0, 0x70, 0xc9, + 0x1d, 0xa2, 0x4b, 0x81, 0x21, 0xba, 0x44, 0x38, 0x96, 0x1a, 0x8e, 0x85, 0x8d, 0x96, 0xb3, 0x2f, + 0xc1, 0x01, 0x3e, 0x79, 0xc6, 0x64, 0x88, 0xbf, 0x27, 0x40, 0x96, 0x96, 0x92, 0x69, 0x0e, 0x55, + 0xfa, 0x8a, 0x79, 0x7d, 0xac, 0x9a, 0x23, 0xca, 0xb9, 0x04, 0xd3, 0x87, 0x4a, 0xbb, 0x87, 0x69, + 0x09, 0xb3, 0x0f, 0x4b, 0x11, 0x32, 0x5e, 0x92, 0x74, 0x89, 0x65, 0x43, 0x8f, 0x61, 0x4e, 0x37, + 0x1c, 0x6c, 0x38, 0x32, 0x63, 0x4b, 0x8e, 0x61, 0xcb, 0xb2, 0xdc, 0xf4, 0x41, 0xfc, 0x67, 0x02, + 0xc0, 0x76, 0x2f, 0x56, 0x3d, 0x7f, 0x63, 0xc2, 0xf2, 0xaf, 0xa6, 0x08, 0xab, 0x5b, 0x8b, 0x4b, + 0x30, 0xa3, 0x1b, 0x6d, 0xdd, 0x60, 0xe5, 0x4f, 0x4b, 0xfc, 0x09, 0x5d, 0x80, 0xe9, 0xdd, 0xb6, + 0x6e, 0x68, 0x74, 0x3c, 0xa4, 0x25, 0xf6, 0x20, 0x4a, 0x90, 0xa5, 0xa5, 0x8e, 0x51, 0xef, 0xe2, + 0x69, 0x02, 0x2e, 0x56, 0x4c, 0x43, 0xd3, 0xc9, 0x90, 0x54, 0xda, 0x5f, 0x0b, 0xad, 0xac, 0xc3, + 0x05, 0x0d, 0x77, 0x2d, 0xac, 0x2a, 0x0e, 0xd6, 0x64, 0x7c, 0xdc, 0x9d, 0xb0, 0x8d, 0x91, 0xcf, + 0x55, 0x3d, 0xee, 0x52, 0x5a, 0xb4, 0x26, 0xd1, 0x37, 0xe0, 0xb2, 0xd2, 0x6e, 0x9b, 0x47, 0xb2, + 0xbe, 0x27, 0x6b, 0x26, 0xb6, 0x65, 0xc3, 0x74, 0x64, 0x7c, 0xac, 0xdb, 0x0e, 0x35, 0x25, 0x69, + 0x69, 0x81, 0x26, 0xd7, 0xf7, 0xd6, 0x4c, 0x6c, 0x6f, 0x99, 0x4e, 0x95, 0x24, 0x11, 0x0b, 0x40, + 0x0a, 0xc3, 0x2c, 0xc0, 0x0c, 0x31, 0xbf, 0x52, 0x1a, 0x1f, 0x77, 0xa9, 0x05, 0x08, 0x34, 0xe5, + 0x6c, 0xb0, 0x29, 0xc5, 0x5f, 0x84, 0x4b, 0xfd, 0xfa, 0x8d, 0xb3, 0xfd, 0xfe, 0x40, 0x80, 0x7c, + 0xdd, 0xd0, 0x9d, 0xaf, 0x45, 0xc3, 0x79, 0xca, 0x4e, 0x06, 0x95, 0x7d, 0x1f, 0x8a, 0x7b, 0x8a, + 0xde, 0x7e, 0x6e, 0x34, 0xcd, 0xce, 0xae, 0xed, 0x98, 0x06, 0xb6, 0x79, 0x6b, 0x0c, 0xd0, 0xc5, + 0x97, 0x50, 0xf0, 0x6a, 0x13, 0xa7, 0x9a, 0x1c, 0x28, 0xd6, 0x0d, 0xd5, 0xc2, 0x1d, 0x6c, 0xc4, + 0xaa, 0xa7, 0xb7, 0x21, 0xa3, 0xbb, 0x72, 0xa9, 0xae, 0x92, 0x92, 0x4f, 0x10, 0x7b, 0x30, 0x1f, + 0x78, 0x6b, 0x9c, 0xe6, 0x92, 0x4c, 0x46, 0xf8, 0x48, 0xf6, 0xdb, 0x88, 0x4c, 0x46, 0xf8, 0x88, + 0x99, 0xb7, 0x06, 0xe4, 0xd6, 0x70, 0x1b, 0x3b, 0x38, 0xc6, 0x9a, 0x8a, 0x3b, 0x90, 0x77, 0x85, + 0xc6, 0xd9, 0x30, 0x7f, 0x5b, 0x00, 0xc4, 0xe5, 0x2a, 0x46, 0x2b, 0xce, 0x12, 0xa3, 0x45, 0xe2, + 0x5a, 0x38, 0x3d, 0xcb, 0x60, 0x3e, 0x02, 0xeb, 0x93, 0xc0, 0x48, 0xd4, 0x4d, 0xf0, 0x87, 0x6c, + 0x2a, 0x38, 0x64, 0xb9, 0x7b, 0x73, 0x04, 0x0b, 0xa1, 0x82, 0xc5, 0xdb, 0x7c, 0x29, 0x5a, 0xa6, + 0xc4, 0xb5, 0x64, 0xd0, 0x87, 0xa3, 0x44, 0xf1, 0xfb, 0x02, 0xcc, 0x57, 0xda, 0x58, 0xb1, 0x62, + 0xd7, 0xc8, 0x37, 0x21, 0xad, 0x61, 0x45, 0xa3, 0x55, 0x66, 0x03, 0xfb, 0x9d, 0x80, 0x14, 0xe2, + 0xe9, 0x2e, 0xed, 0xb7, 0xd5, 0xa5, 0xa6, 0xeb, 0x03, 0xf3, 0xd1, 0xed, 0x31, 0x89, 0x9f, 0x01, + 0x0a, 0x96, 0x2c, 0xce, 0x8e, 0xf0, 0xbf, 0x05, 0x40, 0x12, 0x3e, 0xc4, 0x96, 0x13, 0x7b, 0xb5, + 0xd7, 0x20, 0xeb, 0x28, 0x56, 0x0b, 0x3b, 0x32, 0xf1, 0xee, 0xcf, 0x53, 0x73, 0x60, 0x7c, 0x84, + 0x8c, 0x9a, 0x70, 0x07, 0x1b, 0xca, 0x6e, 0x1b, 0x53, 0x29, 0xf2, 0xae, 0xd9, 0x33, 0x34, 0x59, + 0x77, 0xb0, 0xa5, 0x38, 0xa6, 0x25, 0x9b, 0x5d, 0x47, 0xef, 0xe8, 0x5f, 0x50, 0xc7, 0x9e, 0x77, + 0xb5, 0x1b, 0x2c, 0x3b, 0x61, 0x5e, 0x25, 0x99, 0xeb, 0x3c, 0xef, 0xf3, 0x40, 0x56, 0xf1, 0x73, + 0x58, 0x08, 0xd5, 0x3a, 0x4e, 0x95, 0xfe, 0x4f, 0x01, 0xb2, 0x0d, 0x55, 0x31, 0xe2, 0xd4, 0xe5, + 0xa7, 0x90, 0xb5, 0x55, 0xc5, 0x90, 0xf7, 0x4c, 0xab, 0xa3, 0x38, 0x74, 0xe0, 0xe4, 0x43, 0xba, + 0xf4, 0xfc, 0x75, 0x55, 0x31, 0x9e, 0xd0, 0x4c, 0x12, 0xd8, 0xde, 0xef, 0x7e, 0x7f, 0x74, 0xfa, + 0xab, 0xfb, 0xa3, 0x6c, 0xb8, 0xae, 0xa7, 0xd2, 0xc9, 0x62, 0x4a, 0xfc, 0x0b, 0x01, 0xe6, 0x58, + 0x95, 0xe3, 0x1c, 0xae, 0x1f, 0x40, 0xca, 0x32, 0x8f, 0xd8, 0x70, 0xcd, 0x3e, 0x7c, 0x2b, 0x42, + 0xc4, 0x06, 0x3e, 0x09, 0xce, 0x87, 0x34, 0x3b, 0x5a, 0x05, 0xee, 0x75, 0xca, 0x94, 0x3b, 0x39, + 0x29, 0x37, 0x30, 0x2e, 0x89, 0xc8, 0xb8, 0x03, 0x85, 0x5d, 0xc5, 0x51, 0xf7, 0x65, 0x8b, 0x17, + 0x92, 0xcc, 0x9d, 0xc9, 0xbb, 0x73, 0x52, 0x9e, 0x92, 0xdd, 0xa2, 0xdb, 0xa4, 0xe6, 0x6c, 0xfc, + 0xd8, 0xf8, 0x2f, 0x59, 0x9b, 0xff, 0x1f, 0x81, 0x8f, 0x21, 0xb7, 0xe6, 0x7f, 0xd9, 0x9a, 0xfe, + 0x37, 0x12, 0x70, 0xb9, 0xb2, 0x8f, 0xd5, 0x83, 0x8a, 0x69, 0xd8, 0xba, 0xed, 0x10, 0xdd, 0xc5, + 0xd9, 0xfe, 0x6f, 0x41, 0xe6, 0x48, 0x77, 0xf6, 0x65, 0x4d, 0xdf, 0xdb, 0xa3, 0xd6, 0x33, 0x2d, + 0xa5, 0x09, 0x61, 0x4d, 0xdf, 0xdb, 0x43, 0x8f, 0x20, 0xd5, 0x31, 0x35, 0xe6, 0x9c, 0xe7, 0x1f, + 0x2e, 0x46, 0x88, 0xa7, 0x45, 0xb3, 0x7b, 0x9d, 0x4d, 0x53, 0xc3, 0x12, 0xcd, 0x8c, 0xae, 0x02, + 0xa8, 0x84, 0xda, 0x35, 0x75, 0xc3, 0xe1, 0xb3, 0x6f, 0x80, 0x82, 0x6a, 0x90, 0x71, 0xb0, 0xd5, + 0xd1, 0x0d, 0xc5, 0xc1, 0xa5, 0x69, 0xaa, 0xbc, 0x9b, 0x91, 0x05, 0xef, 0xb6, 0x75, 0x55, 0x59, + 0xc3, 0xb6, 0x6a, 0xe9, 0x5d, 0xc7, 0xb4, 0xb8, 0x16, 0x7d, 0x66, 0xf1, 0xd7, 0x52, 0x50, 0x1a, + 0xd4, 0x4d, 0x9c, 0x3d, 0x64, 0x1b, 0x66, 0x2c, 0x6c, 0xf7, 0xda, 0x0e, 0xef, 0x23, 0x0f, 0x87, + 0xa9, 0x20, 0xa2, 0x04, 0x74, 0xbd, 0xa1, 0xed, 0xf0, 0x62, 0x73, 0x39, 0xe5, 0x7f, 0x23, 0xc0, + 0x0c, 0x4b, 0x40, 0x0f, 0x20, 0x6d, 0x91, 0x89, 0x41, 0xd6, 0x35, 0x5a, 0xc6, 0xe4, 0xea, 0xa5, + 0xb3, 0xd3, 0xc5, 0x59, 0x3a, 0x59, 0xd4, 0xd7, 0xbe, 0xf4, 0x7f, 0x4a, 0xb3, 0x34, 0x5f, 0x5d, + 0x23, 0xad, 0x65, 0x3b, 0x8a, 0xe5, 0xd0, 0x45, 0xa2, 0x04, 0x43, 0x29, 0x94, 0xb0, 0x81, 0x4f, + 0xd0, 0x3a, 0xcc, 0xd8, 0x8e, 0xe2, 0xf4, 0x6c, 0xde, 0x5e, 0xe7, 0x2a, 0x6c, 0x83, 0x72, 0x4a, + 0x5c, 0x02, 0x71, 0x9f, 0x34, 0xec, 0x28, 0x7a, 0x9b, 0x36, 0x60, 0x46, 0xe2, 0x4f, 0xe2, 0x6f, + 0x0a, 0x30, 0xc3, 0xb2, 0xa2, 0xcb, 0xb0, 0x20, 0xad, 0x6c, 0x3d, 0xad, 0xca, 0xf5, 0xad, 0xb5, + 0x6a, 0xb3, 0x2a, 0x6d, 0xd6, 0xb7, 0x56, 0x9a, 0xd5, 0xe2, 0x14, 0xba, 0x04, 0xc8, 0x4d, 0xa8, + 0x3c, 0xdf, 0x6a, 0xd4, 0x1b, 0xcd, 0xea, 0x56, 0xb3, 0x28, 0xd0, 0x35, 0x12, 0x4a, 0x0f, 0x50, + 0x13, 0xe8, 0x26, 0x5c, 0xeb, 0xa7, 0xca, 0x8d, 0xe6, 0x4a, 0xb3, 0x21, 0x57, 0x1b, 0xcd, 0xfa, + 0xe6, 0x4a, 0xb3, 0xba, 0x56, 0x4c, 0x8e, 0xc8, 0x45, 0x5e, 0x22, 0x49, 0xd5, 0x4a, 0xb3, 0x98, + 0x12, 0x1d, 0xb8, 0x28, 0x61, 0xd5, 0xec, 0x74, 0x7b, 0x0e, 0x26, 0xa5, 0xb4, 0xe3, 0x1c, 0x29, + 0x97, 0x61, 0x56, 0xb3, 0x4e, 0x64, 0xab, 0x67, 0xf0, 0x71, 0x32, 0xa3, 0x59, 0x27, 0x52, 0xcf, + 0x10, 0xff, 0x89, 0x00, 0x97, 0xfa, 0x5f, 0x1b, 0x67, 0x27, 0x7c, 0x01, 0x59, 0x45, 0xd3, 0xb0, + 0x26, 0x6b, 0xb8, 0xed, 0x28, 0xdc, 0xc5, 0xb9, 0x1f, 0x90, 0xc4, 0x97, 0xf6, 0x96, 0xbc, 0xa5, + 0xbd, 0xcd, 0x97, 0x95, 0x0a, 0x2d, 0xc8, 0x1a, 0xe1, 0x70, 0xcd, 0x0f, 0x15, 0x42, 0x29, 0xe2, + 0xff, 0x48, 0x41, 0xae, 0x6a, 0x68, 0xcd, 0xe3, 0x58, 0xe7, 0x92, 0x4b, 0x30, 0xa3, 0x9a, 0x9d, + 0x8e, 0xee, 0xb8, 0x0a, 0x62, 0x4f, 0xe8, 0x67, 0x03, 0xae, 0x69, 0x72, 0x02, 0x07, 0xcd, 0x77, + 0x4a, 0xd1, 0xb7, 0xe1, 0x32, 0xb1, 0x9a, 0x96, 0xa1, 0xb4, 0x65, 0x26, 0x4d, 0x76, 0x2c, 0xbd, + 0xd5, 0xc2, 0x16, 0x5f, 0x4e, 0xbc, 0x1b, 0x51, 0xce, 0x3a, 0xe7, 0xa8, 0x50, 0x86, 0x26, 0xcb, + 0x2f, 0x5d, 0xd4, 0xa3, 0xc8, 0xe8, 0x13, 0x00, 0x32, 0x15, 0xd1, 0x25, 0x4a, 0x9b, 0xdb, 0xa3, + 0x61, 0x6b, 0x94, 0xae, 0x09, 0x22, 0x0c, 0xe4, 0xd9, 0x46, 0xcb, 0x04, 0x87, 0xbc, 0xee, 0xe9, + 0x16, 0x96, 0x1f, 0x74, 0x55, 0xba, 0x70, 0x90, 0x5e, 0xcd, 0x9f, 0x9d, 0x2e, 0x82, 0xc4, 0xc8, + 0x0f, 0xb6, 0x2b, 0x04, 0x97, 0xb0, 0xdf, 0x5d, 0x15, 0xbd, 0x82, 0x7b, 0x81, 0xf5, 0x0f, 0x32, + 0xf3, 0xf2, 0x6a, 0x29, 0x8e, 0xbc, 0xaf, 0xb7, 0xf6, 0xb1, 0x25, 0x7b, 0xcb, 0xd4, 0x74, 0x25, + 0x32, 0x2d, 0xdd, 0xf4, 0x19, 0x2a, 0x8a, 0xc1, 0x4a, 0xbf, 0xe2, 0xd4, 0x68, 0x66, 0x4f, 0x67, + 0x44, 0xf9, 0x5d, 0x53, 0xb7, 0x4d, 0xa3, 0x94, 0x61, 0xca, 0x67, 0x4f, 0xe8, 0x1e, 0x14, 0x9d, + 0x63, 0x43, 0xde, 0xc7, 0x8a, 0xe5, 0xec, 0x62, 0xc5, 0x21, 0xb3, 0x34, 0xd0, 0x1c, 0x05, 0xe7, + 0xd8, 0xa8, 0x05, 0xc8, 0xe8, 0x05, 0x14, 0x75, 0x43, 0xde, 0x6b, 0xeb, 0xad, 0x7d, 0x47, 0x3e, + 0xb2, 0x74, 0x07, 0xdb, 0xa5, 0x79, 0xaa, 0x90, 0xa8, 0x7e, 0xdb, 0xe0, 0xeb, 0xc6, 0xda, 0x2b, + 0x92, 0x93, 0xab, 0x26, 0xaf, 0x1b, 0x4f, 0x28, 0x3f, 0x25, 0xda, 0xeb, 0xa9, 0xf4, 0x6c, 0x31, + 0x2d, 0xfe, 0x17, 0x01, 0xf2, 0x6e, 0x77, 0x8b, 0x73, 0x64, 0xdc, 0x85, 0xa2, 0x69, 0x60, 0xb9, + 0xbb, 0xaf, 0xd8, 0x98, 0xeb, 0x91, 0x4f, 0x38, 0x79, 0xd3, 0xc0, 0xdb, 0x84, 0xcc, 0xd4, 0x85, + 0xb6, 0x61, 0xde, 0x76, 0x94, 0x96, 0x6e, 0xb4, 0x02, 0xea, 0x9d, 0x9e, 0x1c, 0x2c, 0x14, 0x39, + 0xb7, 0x47, 0x0f, 0x79, 0x29, 0x7f, 0x24, 0xc0, 0xfc, 0x8a, 0xd6, 0xd1, 0x8d, 0x46, 0xb7, 0xad, + 0xc7, 0xba, 0x06, 0x71, 0x13, 0x32, 0x36, 0x91, 0xe9, 0x1b, 0x7c, 0x1f, 0x51, 0xa6, 0x69, 0x0a, + 0xb1, 0xfc, 0xcf, 0xa0, 0x80, 0x8f, 0xbb, 0x3a, 0xfb, 0xf4, 0xc0, 0x80, 0x50, 0x6a, 0xf2, 0xba, + 0xe5, 0x7d, 0x5e, 0x92, 0xc4, 0xeb, 0xf4, 0x19, 0xa0, 0x60, 0x95, 0xe2, 0xc4, 0x2e, 0x9f, 0xc1, + 0x02, 0x15, 0xbd, 0x63, 0xd8, 0x31, 0xeb, 0x4b, 0xfc, 0x05, 0xb8, 0x10, 0x16, 0x1d, 0x67, 0xb9, + 0x5f, 0xf1, 0x56, 0xde, 0xc4, 0x56, 0xac, 0x20, 0xd6, 0xd3, 0x35, 0x17, 0x1c, 0x67, 0x99, 0x7f, + 0x45, 0x80, 0x2b, 0x54, 0x36, 0xfd, 0x3a, 0xb3, 0x87, 0xad, 0x67, 0x58, 0xb1, 0x63, 0x45, 0xe0, + 0x37, 0x60, 0x86, 0x21, 0x69, 0xda, 0x3f, 0xa7, 0x57, 0xb3, 0xc4, 0x73, 0x69, 0x38, 0xa6, 0x45, + 0x3c, 0x17, 0x9e, 0x24, 0x2a, 0x50, 0x8e, 0x2a, 0x45, 0x9c, 0x35, 0xfd, 0xbb, 0x02, 0xcc, 0x73, + 0xa7, 0x91, 0x74, 0xe5, 0xca, 0x3e, 0xf1, 0x99, 0x50, 0x15, 0xb2, 0x2a, 0xfd, 0x25, 0x3b, 0x27, + 0x5d, 0x4c, 0xe5, 0xe7, 0x47, 0xf9, 0x9b, 0x8c, 0xad, 0x79, 0xd2, 0xc5, 0xc4, 0x69, 0x75, 0x7f, + 0x13, 0x45, 0x05, 0x2a, 0x39, 0xd2, 0x63, 0xa5, 0xe3, 0x88, 0xe6, 0x75, 0x5d, 0x3f, 0xae, 0x83, + 0x7f, 0x9a, 0xe4, 0x4a, 0x60, 0xef, 0xe0, 0xd9, 0x63, 0xf5, 0x51, 0x3e, 0x87, 0x4b, 0xc1, 0xd9, + 0x25, 0x50, 0xf1, 0xc4, 0x39, 0x2a, 0x1e, 0x58, 0xa1, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xac, 0xc1, + 0xcb, 0xac, 0x4e, 0x2e, 0xfa, 0x39, 0x8f, 0x3a, 0xe6, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, + 0xf1, 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x46, 0x09, 0x24, 0x45, 0x19, 0xc0, 0x03, + 0xb3, 0xf8, 0xb8, 0x4b, 0x88, 0x68, 0x87, 0xcc, 0x5e, 0xae, 0xab, 0x40, 0x8b, 0x6d, 0x8f, 0x87, + 0x17, 0x7e, 0x4f, 0xe1, 0xe2, 0x0a, 0x9e, 0x97, 0xc0, 0x44, 0x88, 0x3f, 0x10, 0xe0, 0xad, 0xc8, + 0x56, 0x8b, 0x73, 0x22, 0xfb, 0x04, 0x52, 0xb4, 0xf2, 0x89, 0x73, 0x56, 0x9e, 0x72, 0x89, 0xdf, + 0x4d, 0xf0, 0x31, 0x2e, 0xe1, 0xb6, 0x49, 0x14, 0x1b, 0xfb, 0x2a, 0xdb, 0x73, 0xc8, 0x1d, 0x9a, + 0x0e, 0xf1, 0x4d, 0x78, 0xb3, 0x27, 0xce, 0xdd, 0xec, 0x73, 0x54, 0x80, 0xdb, 0xe2, 0x2f, 0x61, + 0xde, 0x30, 0x0d, 0x39, 0x2c, 0xf4, 0xfc, 0x7d, 0xa9, 0x60, 0x98, 0xc6, 0xcb, 0x80, 0x5c, 0xcf, + 0xce, 0xf4, 0x69, 0x22, 0x4e, 0x3b, 0xf3, 0x3d, 0x01, 0x16, 0x3c, 0xb7, 0x29, 0x66, 0x0f, 0xfa, + 0x03, 0x48, 0x1a, 0xe6, 0xd1, 0x79, 0x56, 0x31, 0x49, 0x7e, 0x32, 0xeb, 0x85, 0x4b, 0x14, 0x67, + 0x7d, 0xff, 0x6d, 0x02, 0x32, 0x4f, 0x2b, 0x71, 0xd6, 0xf2, 0x13, 0xbe, 0x42, 0xce, 0xda, 0x3b, + 0xaa, 0xb7, 0x7b, 0xef, 0x5b, 0x7a, 0x5a, 0xd9, 0xc0, 0x27, 0x6e, 0x6f, 0x27, 0x5c, 0x68, 0x05, + 0x32, 0xce, 0xbe, 0x85, 0xed, 0x7d, 0xb3, 0xad, 0x9d, 0xc7, 0xcd, 0xf1, 0xb9, 0xca, 0x18, 0xa6, + 0xa9, 0x5c, 0x37, 0x1a, 0x43, 0x88, 0x88, 0xc6, 0x20, 0xaf, 0xf1, 0x3c, 0xc5, 0xc4, 0x79, 0x5e, + 0x13, 0x70, 0x11, 0xa7, 0x8b, 0x33, 0xe2, 0x0b, 0x00, 0x52, 0x9d, 0x38, 0x9b, 0xe4, 0x57, 0x93, + 0x90, 0xdf, 0xee, 0xd9, 0xfb, 0x31, 0xf7, 0xbe, 0x0a, 0x40, 0xb7, 0x67, 0x53, 0x08, 0x72, 0x6c, + 0xf0, 0x3a, 0x8f, 0x09, 0xf4, 0x70, 0x2b, 0xcd, 0xf8, 0x9a, 0xc7, 0x06, 0xaa, 0x71, 0x21, 0x58, + 0xf6, 0xa3, 0x45, 0x6e, 0x8c, 0x02, 0xab, 0xcd, 0x63, 0x63, 0x13, 0x7b, 0x28, 0x95, 0x49, 0xc2, + 0x44, 0xd2, 0x27, 0x30, 0x4b, 0x1e, 0x64, 0xc7, 0x3c, 0x4f, 0x33, 0xcf, 0x10, 0x9e, 0xa6, 0x89, + 0x1e, 0x43, 0x86, 0x71, 0x93, 0xd9, 0x6f, 0x86, 0xce, 0x7e, 0x51, 0x75, 0xe1, 0x6a, 0xa4, 0xf3, + 0x5e, 0x9a, 0xb2, 0x92, 0xb9, 0xee, 0x02, 0x4c, 0xef, 0x99, 0x96, 0xea, 0x7e, 0xef, 0x65, 0x0f, + 0xac, 0x3d, 0xd7, 0x53, 0xe9, 0x74, 0x31, 0xb3, 0x9e, 0x4a, 0x67, 0x8a, 0x20, 0xfe, 0xa6, 0x00, + 0x05, 0xaf, 0x21, 0xe2, 0x9c, 0x10, 0x2a, 0x21, 0x2d, 0x9e, 0xbf, 0x29, 0x88, 0x02, 0xc5, 0x7f, + 0x47, 0x3d, 0x22, 0xd5, 0x3c, 0xa4, 0x2d, 0x13, 0x67, 0x4f, 0x79, 0xcc, 0x62, 0x81, 0x12, 0xe7, + 0x6d, 0x5d, 0x1a, 0x16, 0xf4, 0x00, 0x2e, 0xe8, 0x1d, 0x62, 0xcf, 0x75, 0xa7, 0x7d, 0xc2, 0x61, + 0x9b, 0x83, 0xdd, 0x0f, 0xcb, 0x0b, 0x7e, 0x5a, 0xc5, 0x4d, 0x12, 0xff, 0x3e, 0x5d, 0x00, 0xf7, + 0x6b, 0x12, 0xa7, 0xaa, 0xeb, 0x90, 0xb3, 0x98, 0x68, 0xe2, 0xd6, 0x9c, 0x53, 0xdb, 0x73, 0x1e, + 0x2b, 0x51, 0xf8, 0x6f, 0x27, 0xa0, 0xf0, 0xa2, 0x87, 0xad, 0x93, 0xaf, 0x93, 0xba, 0x6f, 0x43, + 0xe1, 0x48, 0xd1, 0x1d, 0x79, 0xcf, 0xb4, 0xe4, 0x5e, 0x57, 0x53, 0x1c, 0x37, 0x20, 0x25, 0x47, + 0xc8, 0x4f, 0x4c, 0x6b, 0x87, 0x12, 0x11, 0x06, 0x74, 0x60, 0x98, 0x47, 0x86, 0x4c, 0xc8, 0x14, + 0x28, 0x1f, 0x1b, 0x7c, 0x55, 0x7a, 0xf5, 0xc3, 0xff, 0x7c, 0xba, 0xf8, 0x68, 0xa2, 0x30, 0x33, + 0x1a, 0x52, 0xd7, 0xeb, 0xe9, 0xda, 0xd2, 0xce, 0x4e, 0x7d, 0x4d, 0x2a, 0x52, 0x91, 0xaf, 0x98, + 0xc4, 0xe6, 0xb1, 0x61, 0x8b, 0xff, 0x20, 0x01, 0x45, 0x5f, 0x47, 0x71, 0x36, 0x64, 0x15, 0xb2, + 0xaf, 0x7b, 0xd8, 0xd2, 0xdf, 0xa0, 0x19, 0x81, 0x33, 0x12, 0xb3, 0xf3, 0x39, 0xcc, 0x85, 0x34, + 0x90, 0xfc, 0x6a, 0x1a, 0xc8, 0x1e, 0xf9, 0x95, 0x47, 0xf7, 0x61, 0xde, 0x39, 0x36, 0x64, 0x16, + 0x60, 0xc8, 0x82, 0x52, 0xdc, 0x78, 0x89, 0x82, 0x43, 0xf4, 0x41, 0xe8, 0x34, 0x20, 0xc5, 0x16, + 0x7f, 0x4f, 0x00, 0x44, 0x15, 0x55, 0x67, 0x9f, 0x0d, 0xbe, 0x2e, 0xfd, 0xe9, 0x2e, 0x14, 0x69, + 0xc8, 0xa6, 0xac, 0xef, 0xc9, 0x1d, 0xdd, 0xb6, 0x75, 0xa3, 0xc5, 0x3b, 0x54, 0x9e, 0xd2, 0xeb, + 0x7b, 0x9b, 0x8c, 0x2a, 0xfe, 0x35, 0x58, 0x08, 0x55, 0x20, 0xce, 0xc6, 0xbe, 0x0e, 0x73, 0x7b, + 0xec, 0x2b, 0x2d, 0x15, 0xce, 0x57, 0x1c, 0xb3, 0x94, 0xc6, 0xde, 0x27, 0xfe, 0x59, 0x02, 0x2e, + 0x48, 0xd8, 0x36, 0xdb, 0x87, 0x38, 0x7e, 0x15, 0xd6, 0x80, 0x7f, 0xce, 0x91, 0xdf, 0x48, 0x93, + 0x19, 0xc6, 0xcc, 0xa6, 0xb9, 0xf0, 0xb2, 0xfd, 0xcd, 0xd1, 0x3d, 0x76, 0x70, 0xa1, 0x9e, 0x2f, + 0xfb, 0xa5, 0x42, 0xcb, 0x7e, 0x26, 0x14, 0xf4, 0x96, 0x61, 0x12, 0x9b, 0x66, 0xe3, 0xd7, 0x46, + 0xaf, 0xe3, 0x82, 0xa1, 0xa5, 0x51, 0x85, 0xac, 0x33, 0x96, 0x06, 0x7e, 0xbd, 0xd5, 0xeb, 0x50, + 0xdf, 0x79, 0xf5, 0x12, 0x29, 0xef, 0xd9, 0xe9, 0x62, 0x3e, 0x94, 0x66, 0x4b, 0x79, 0xdd, 0x7b, + 0x26, 0xd2, 0xc5, 0x6f, 0xc1, 0xc5, 0x3e, 0x65, 0xc7, 0xe9, 0xf1, 0xfc, 0xeb, 0x24, 0x5c, 0x09, + 0x8b, 0x8f, 0x1b, 0xe2, 0x7c, 0xdd, 0x1b, 0xb4, 0x06, 0xb9, 0x8e, 0x6e, 0xbc, 0xd9, 0xea, 0xe5, + 0x5c, 0x47, 0x37, 0xfc, 0x95, 0xe2, 0x88, 0xae, 0x31, 0xf3, 0x53, 0xed, 0x1a, 0x0a, 0x94, 0xa3, + 0xda, 0x2e, 0xce, 0xfe, 0xf1, 0x5d, 0x01, 0xe6, 0xe2, 0x5e, 0x96, 0x7b, 0xb3, 0x40, 0x39, 0xb1, + 0x09, 0xb9, 0x9f, 0xc2, 0x3a, 0xde, 0x6f, 0x0b, 0x80, 0x9a, 0x56, 0xcf, 0x20, 0xa0, 0xf6, 0x99, + 0xd9, 0x8a, 0xb3, 0x9a, 0x17, 0x60, 0x5a, 0x37, 0x34, 0x7c, 0x4c, 0xab, 0x99, 0x92, 0xd8, 0x43, + 0xe8, 0xeb, 0x64, 0x72, 0xa2, 0xaf, 0x93, 0xe2, 0xe7, 0xb0, 0x10, 0x2a, 0x62, 0x9c, 0xf5, 0xff, + 0xef, 0x09, 0x58, 0xe0, 0x15, 0x89, 0x7d, 0x05, 0xf3, 0x1b, 0x30, 0xdd, 0x26, 0x32, 0x47, 0xb4, + 0x33, 0x7d, 0xa7, 0xdb, 0xce, 0x34, 0x33, 0xfa, 0x39, 0x80, 0xae, 0x85, 0x0f, 0x65, 0xc6, 0x9a, + 0x9c, 0x88, 0x35, 0x43, 0x38, 0x28, 0x01, 0x7d, 0x5f, 0x80, 0x02, 0x19, 0xd0, 0x5d, 0xcb, 0xec, + 0x9a, 0x36, 0xf1, 0x59, 0xec, 0xc9, 0x60, 0xce, 0x8b, 0xb3, 0xd3, 0xc5, 0xdc, 0xa6, 0x6e, 0x6c, + 0x73, 0xc6, 0x66, 0x63, 0xe2, 0x3d, 0x00, 0xee, 0x4e, 0x88, 0xa5, 0x4a, 0xdb, 0x54, 0x0f, 0xfc, + 0xef, 0x6d, 0xc4, 0xb2, 0x78, 0xe2, 0x6c, 0xf1, 0x0f, 0x05, 0xb8, 0xf0, 0x53, 0x5b, 0x2e, 0xfe, + 0xff, 0xa1, 0x6c, 0xf1, 0x25, 0x14, 0xe9, 0x8f, 0xba, 0xb1, 0x67, 0xc6, 0xb9, 0x70, 0xff, 0x6b, + 0x02, 0xcc, 0x07, 0x04, 0xc7, 0xe9, 0xe0, 0xbc, 0x91, 0x9e, 0xc4, 0x5f, 0x20, 0x2e, 0x4f, 0x70, + 0x94, 0xc4, 0x39, 0x06, 0xff, 0x45, 0x02, 0x2e, 0x55, 0xd8, 0x67, 0x6e, 0x37, 0xee, 0x23, 0xce, + 0x9e, 0x51, 0x82, 0xd9, 0x43, 0x6c, 0xd9, 0xba, 0xc9, 0xa6, 0xdf, 0x9c, 0xe4, 0x3e, 0xa2, 0x32, + 0xa4, 0x6d, 0x43, 0xe9, 0xda, 0xfb, 0xa6, 0xfb, 0x7d, 0xcf, 0x7b, 0xf6, 0x62, 0x54, 0xa6, 0xdf, + 0x3c, 0x46, 0x65, 0x66, 0x74, 0x8c, 0xca, 0xec, 0x57, 0x88, 0x51, 0xe1, 0x1f, 0xd3, 0xfe, 0xbd, + 0x00, 0x97, 0x07, 0x34, 0x17, 0x67, 0x6f, 0xf9, 0x0e, 0x64, 0x55, 0x2e, 0x98, 0x18, 0x6c, 0xf6, + 0xa5, 0xb0, 0x4e, 0xb2, 0xbd, 0x21, 0x6e, 0x39, 0x3b, 0x5d, 0x04, 0xb7, 0xa8, 0xf5, 0x35, 0xae, + 0x1c, 0xf2, 0x5b, 0x13, 0x7f, 0x39, 0x07, 0x85, 0xea, 0x31, 0x5b, 0x25, 0x6f, 0x30, 0x37, 0x01, + 0x3d, 0x81, 0x74, 0xd7, 0x32, 0x0f, 0x75, 0xb7, 0x1a, 0xf9, 0x50, 0x80, 0x82, 0x5b, 0x8d, 0x3e, + 0xae, 0x6d, 0xce, 0x21, 0x79, 0xbc, 0xa8, 0x09, 0x99, 0x67, 0xa6, 0xaa, 0xb4, 0x9f, 0xe8, 0x6d, + 0xb7, 0xe7, 0xbf, 0x3f, 0x5e, 0xd0, 0x92, 0xc7, 0xb3, 0xad, 0x38, 0xfb, 0x6e, 0x23, 0x78, 0x44, + 0x54, 0x87, 0x74, 0xcd, 0x71, 0xba, 0x24, 0x91, 0xdb, 0x8e, 0x3b, 0x13, 0x08, 0x25, 0x2c, 0x6e, + 0x94, 0xac, 0xcb, 0x8e, 0x9a, 0x30, 0xff, 0x94, 0xee, 0xf9, 0xaa, 0xb4, 0xcd, 0x9e, 0x56, 0x31, + 0x8d, 0x3d, 0xbd, 0xc5, 0xed, 0xf6, 0xed, 0x09, 0x64, 0x3e, 0xad, 0x34, 0xa4, 0x41, 0x01, 0x68, + 0x05, 0xd2, 0x8d, 0x47, 0x5c, 0x18, 0xf3, 0xeb, 0x6e, 0x4d, 0x20, 0xac, 0xf1, 0x48, 0xf2, 0xd8, + 0xd0, 0x3a, 0x64, 0x57, 0xbe, 0xe8, 0x59, 0x98, 0x4b, 0x99, 0x19, 0x1a, 0x1d, 0xd1, 0x2f, 0x85, + 0x72, 0x49, 0x41, 0x66, 0xd4, 0x80, 0xfc, 0x2b, 0xd3, 0x3a, 0x68, 0x9b, 0x8a, 0x5b, 0xc3, 0x59, + 0x2a, 0xee, 0x67, 0x26, 0x10, 0xe7, 0x32, 0x4a, 0x7d, 0x22, 0xd0, 0xb7, 0xa0, 0x40, 0x1a, 0xa3, + 0xa9, 0xec, 0xb6, 0xdd, 0x42, 0xa6, 0xa9, 0xd4, 0x77, 0x27, 0x90, 0xea, 0x71, 0xba, 0x0b, 0xff, + 0x7d, 0xa2, 0xca, 0x12, 0xe4, 0x42, 0x9d, 0x00, 0x21, 0x48, 0x75, 0x49, 0x7b, 0x0b, 0x34, 0x7e, + 0x89, 0xfe, 0x46, 0xef, 0xc1, 0xac, 0x61, 0x6a, 0xd8, 0x1d, 0x21, 0xb9, 0xd5, 0x0b, 0x67, 0xa7, + 0x8b, 0x33, 0x5b, 0xa6, 0xc6, 0x3c, 0x1a, 0xfe, 0x4b, 0x9a, 0x21, 0x99, 0xea, 0x5a, 0xf9, 0x1a, + 0xa4, 0x48, 0xbb, 0x13, 0xc3, 0xb4, 0xab, 0xd8, 0x78, 0xc7, 0xd2, 0xb9, 0x34, 0xf7, 0xb1, 0xfc, + 0x8f, 0x13, 0x90, 0x68, 0x3c, 0x22, 0x3e, 0xfb, 0x6e, 0x4f, 0x3d, 0xc0, 0x0e, 0x4f, 0xe7, 0x4f, + 0xd4, 0x97, 0xb7, 0xf0, 0x9e, 0xce, 0x5c, 0xab, 0x8c, 0xc4, 0x9f, 0xd0, 0x3b, 0x00, 0x8a, 0xaa, + 0x62, 0xdb, 0x96, 0xdd, 0xbd, 0x80, 0x19, 0x29, 0xc3, 0x28, 0x1b, 0xf8, 0x84, 0xb0, 0xd9, 0x58, + 0xb5, 0xb0, 0xe3, 0x06, 0x5f, 0xb1, 0x27, 0xc2, 0xe6, 0xe0, 0x4e, 0x57, 0x76, 0xcc, 0x03, 0x6c, + 0xd0, 0x7e, 0x92, 0x21, 0xa6, 0xa6, 0xd3, 0x6d, 0x12, 0x02, 0xb1, 0x92, 0xd8, 0xd0, 0x7c, 0x93, + 0x96, 0x91, 0xbc, 0x67, 0x22, 0xd2, 0xc2, 0x2d, 0x9d, 0x6f, 0x97, 0xcb, 0x48, 0xfc, 0x89, 0x68, + 0x49, 0xe9, 0x39, 0xfb, 0xb4, 0x25, 0x32, 0x12, 0xfd, 0x8d, 0x6e, 0x43, 0x81, 0xc5, 0x6b, 0xca, + 0xd8, 0x50, 0x65, 0x6a, 0x5c, 0x33, 0x34, 0x39, 0xc7, 0xc8, 0x55, 0x43, 0x25, 0xa6, 0x14, 0x3d, + 0x02, 0x4e, 0x90, 0x0f, 0x3a, 0x36, 0xd1, 0x29, 0x90, 0x5c, 0xab, 0x85, 0xb3, 0xd3, 0xc5, 0x6c, + 0x83, 0x26, 0x6c, 0x6c, 0x36, 0xea, 0x6b, 0x52, 0x96, 0xe5, 0xda, 0xe8, 0xd8, 0x75, 0xad, 0xfc, + 0xeb, 0x02, 0x24, 0x9f, 0x56, 0x1a, 0xe7, 0x56, 0x99, 0x5b, 0xd0, 0x64, 0xa0, 0xa0, 0x77, 0xa0, + 0xb0, 0xab, 0xb7, 0xdb, 0xba, 0xd1, 0x22, 0x5e, 0xd4, 0x77, 0xb0, 0xea, 0x2a, 0x2c, 0xcf, 0xc9, + 0xdb, 0x8c, 0x8a, 0xae, 0x41, 0x56, 0xb5, 0xb0, 0x86, 0x0d, 0x47, 0x57, 0xda, 0x36, 0xd7, 0x5c, + 0x90, 0x54, 0xfe, 0x25, 0x01, 0xa6, 0xe9, 0x08, 0x40, 0x6f, 0x43, 0x46, 0x35, 0x0d, 0x47, 0xd1, + 0x0d, 0x6e, 0xca, 0x32, 0x92, 0x4f, 0x18, 0x5a, 0xbc, 0xeb, 0x30, 0xa7, 0xa8, 0xaa, 0xd9, 0x33, + 0x1c, 0xd9, 0x50, 0x3a, 0x98, 0x17, 0x33, 0xcb, 0x69, 0x5b, 0x4a, 0x07, 0xa3, 0x45, 0x70, 0x1f, + 0xbd, 0x2d, 0x9e, 0x19, 0x09, 0x38, 0x69, 0x03, 0x9f, 0x94, 0x7f, 0x5f, 0x80, 0xb4, 0x3b, 0x66, + 0x48, 0x31, 0x5a, 0xd8, 0x60, 0x41, 0xe5, 0x6e, 0x31, 0x3c, 0x42, 0xff, 0x54, 0x99, 0xf1, 0xa7, + 0xca, 0x0b, 0x30, 0xed, 0x90, 0x61, 0xc1, 0x4b, 0xc0, 0x1e, 0xe8, 0x7a, 0x76, 0x5b, 0x69, 0xb1, + 0xe5, 0xbc, 0x8c, 0xc4, 0x1e, 0x48, 0x65, 0x78, 0xd8, 0x2f, 0xd3, 0x08, 0x7f, 0x22, 0x25, 0x65, + 0xc1, 0xa9, 0xbb, 0xb8, 0xa5, 0x1b, 0xb4, 0x2f, 0x25, 0x25, 0xa0, 0xa4, 0x55, 0x42, 0x41, 0x6f, + 0x41, 0x86, 0x65, 0xc0, 0x86, 0x46, 0x3b, 0x54, 0x52, 0x4a, 0x53, 0x42, 0xd5, 0xd0, 0xca, 0x18, + 0x32, 0xde, 0xe0, 0x24, 0xcd, 0xd6, 0xb3, 0x3d, 0x45, 0xd2, 0xdf, 0xe8, 0x7d, 0xb8, 0xf0, 0xba, + 0xa7, 0xb4, 0xf5, 0x3d, 0xba, 0x52, 0x47, 0xa3, 0xee, 0xa9, 0xce, 0x58, 0x4d, 0x90, 0x97, 0x46, + 0x25, 0x50, 0xd5, 0xb9, 0x63, 0x39, 0xe9, 0x8f, 0x65, 0xf1, 0x77, 0x04, 0x98, 0x67, 0x91, 0x47, + 0x2c, 0x60, 0x36, 0x3e, 0x3f, 0xe4, 0x63, 0xc8, 0x68, 0x8a, 0xa3, 0xb0, 0x4d, 0xab, 0x89, 0x91, + 0x9b, 0x56, 0xbd, 0x4d, 0x14, 0x8a, 0xa3, 0xd0, 0x8d, 0xab, 0x08, 0x52, 0xe4, 0x37, 0xdb, 0xdf, + 0x2b, 0xd1, 0xdf, 0xe2, 0x67, 0x80, 0x82, 0x05, 0x8d, 0xd3, 0x23, 0xbb, 0x07, 0x17, 0x89, 0xae, + 0xab, 0x86, 0x6a, 0x9d, 0x74, 0x1d, 0xdd, 0x34, 0x9e, 0xd3, 0xbf, 0x36, 0x2a, 0x06, 0x3e, 0x6c, + 0xd1, 0xef, 0x59, 0xe2, 0xef, 0xce, 0x40, 0xae, 0x7a, 0xdc, 0x35, 0xad, 0x58, 0x97, 0xc1, 0x56, + 0x61, 0x96, 0xaf, 0x14, 0x8c, 0xf8, 0x76, 0xdd, 0x67, 0xcc, 0xdd, 0x0f, 0xf7, 0x9c, 0x11, 0xad, + 0x02, 0xb0, 0xa0, 0x56, 0x1a, 0xb8, 0x94, 0x3c, 0xc7, 0xa7, 0x36, 0xca, 0x46, 0x37, 0x70, 0x6c, + 0x41, 0xb6, 0x73, 0xa8, 0xaa, 0xf2, 0x9e, 0xde, 0x76, 0x78, 0x6c, 0x60, 0x74, 0x18, 0xfb, 0xe6, + 0xcb, 0x4a, 0xe5, 0x09, 0xcd, 0xc4, 0xc2, 0xf4, 0xfc, 0x67, 0x09, 0x88, 0x04, 0xf6, 0x1b, 0xbd, + 0x0b, 0x7c, 0x33, 0x91, 0x6c, 0xbb, 0xfb, 0x06, 0x57, 0x73, 0x67, 0xa7, 0x8b, 0x19, 0x89, 0x52, + 0x1b, 0x8d, 0xa6, 0x94, 0x61, 0x19, 0x1a, 0xb6, 0x83, 0x6e, 0x40, 0xce, 0xec, 0xe8, 0x8e, 0xec, + 0x3a, 0x49, 0xdc, 0xa3, 0x9c, 0x23, 0x44, 0xd7, 0x89, 0x3a, 0xcf, 0x1e, 0x93, 0xd9, 0x89, 0xf7, + 0x98, 0xa0, 0xbf, 0x29, 0xc0, 0x25, 0xae, 0x48, 0x79, 0x97, 0xc6, 0xe1, 0x2b, 0x6d, 0xdd, 0x39, + 0x91, 0x0f, 0x0e, 0x4b, 0x69, 0xea, 0xb7, 0xfe, 0x6c, 0x64, 0x83, 0x04, 0xfa, 0xc1, 0x92, 0xdb, + 0x2c, 0x27, 0xcf, 0x38, 0xf3, 0xc6, 0x61, 0xd5, 0x70, 0xac, 0x93, 0xd5, 0xcb, 0x67, 0xa7, 0x8b, + 0x0b, 0x83, 0xa9, 0x2f, 0xa5, 0x05, 0x7b, 0x90, 0x05, 0xd5, 0x00, 0xb0, 0xd7, 0x0f, 0xe9, 0x8c, + 0x11, 0xed, 0x7f, 0x44, 0x76, 0x58, 0x29, 0xc0, 0x8b, 0xee, 0x42, 0x91, 0xef, 0xe9, 0xd9, 0xd3, + 0xdb, 0x58, 0xb6, 0xf5, 0x2f, 0x30, 0x9d, 0x5b, 0x92, 0x52, 0x9e, 0xd1, 0x89, 0x88, 0x86, 0xfe, + 0x05, 0x2e, 0x7f, 0x07, 0x4a, 0xc3, 0x4a, 0x1f, 0x1c, 0x02, 0x19, 0xf6, 0x49, 0xf7, 0xa3, 0xf0, + 0x7a, 0xce, 0x04, 0x5d, 0x95, 0xaf, 0xe9, 0x7c, 0x9c, 0xf8, 0x48, 0x10, 0xff, 0x61, 0x02, 0x72, + 0xab, 0xbd, 0xf6, 0xc1, 0xf3, 0x6e, 0xa3, 0xd7, 0xe9, 0x28, 0xd6, 0x09, 0x31, 0x83, 0xcc, 0x50, + 0x90, 0x02, 0x0a, 0xcc, 0x0c, 0x52, 0x4b, 0xa0, 0x7f, 0x81, 0xc9, 0xe4, 0x14, 0x08, 0x97, 0xe1, + 0xfb, 0x0c, 0x68, 0x1d, 0x7c, 0x32, 0xdd, 0x0a, 0xf0, 0x11, 0x94, 0x02, 0x19, 0xe9, 0xe2, 0x8b, + 0x8c, 0x0d, 0xc7, 0xd2, 0x31, 0x5b, 0x40, 0x4c, 0x4a, 0x81, 0x98, 0x9e, 0x3a, 0x49, 0xae, 0xb2, + 0x54, 0xd4, 0x84, 0x39, 0x92, 0xf1, 0x44, 0xa6, 0x53, 0x88, 0xbb, 0xc0, 0xfb, 0x20, 0xa2, 0x5a, + 0xa1, 0x72, 0x2f, 0x51, 0xfd, 0x54, 0x28, 0x0f, 0xfd, 0x29, 0x65, 0xb1, 0x4f, 0x29, 0x7f, 0x0a, + 0xc5, 0xfe, 0x0c, 0x41, 0x5d, 0xa6, 0x98, 0x2e, 0x2f, 0x04, 0x75, 0x99, 0x0c, 0xe8, 0x69, 0x3d, + 0x95, 0x4e, 0x15, 0xa7, 0xc5, 0x3f, 0x4d, 0x42, 0xde, 0xed, 0x66, 0x71, 0x02, 0x9d, 0x55, 0x98, + 0x26, 0x9d, 0xc2, 0x8d, 0x40, 0xb9, 0x3d, 0xa2, 0x77, 0xf3, 0xc8, 0x76, 0xd2, 0x59, 0x5c, 0x90, + 0x4c, 0x59, 0xe3, 0x30, 0x38, 0xe5, 0x5f, 0x4a, 0x40, 0x8a, 0x62, 0x8b, 0x07, 0x90, 0xa2, 0x13, + 0x85, 0x30, 0xc9, 0x44, 0x41, 0xb3, 0x7a, 0xd3, 0x59, 0x22, 0xe0, 0x9a, 0x12, 0x9f, 0x6f, 0x5f, + 0xf9, 0xe0, 0xc1, 0x43, 0x6a, 0x6c, 0xe6, 0x24, 0xfe, 0x84, 0x56, 0x69, 0x68, 0x94, 0x69, 0x39, + 0x58, 0xe3, 0x3e, 0xfd, 0xb5, 0x71, 0xed, 0xeb, 0x4e, 0x4a, 0x2e, 0x1f, 0xba, 0x02, 0x49, 0x62, + 0xc5, 0x66, 0x59, 0x94, 0xc3, 0xd9, 0xe9, 0x62, 0x92, 0xd8, 0x2f, 0x42, 0x43, 0xcb, 0x90, 0x0d, + 0x9b, 0x0c, 0xe2, 0xc1, 0x51, 0xc3, 0x18, 0x18, 0xee, 0xd0, 0xf6, 0x86, 0x16, 0xc3, 0xb3, 0xbc, + 0x8d, 0xff, 0x3c, 0x05, 0xb9, 0x7a, 0x27, 0xee, 0x29, 0x65, 0x25, 0xdc, 0xc2, 0x51, 0x40, 0x28, + 0xf4, 0xd2, 0x88, 0x06, 0x0e, 0xcd, 0xe0, 0xc9, 0xf3, 0xcd, 0xe0, 0x75, 0xe2, 0x29, 0xf3, 0x83, + 0x27, 0x92, 0x43, 0x30, 0x4f, 0xf8, 0xfd, 0xd4, 0x4f, 0x91, 0x08, 0x8f, 0xbf, 0xd7, 0x83, 0x46, + 0xaa, 0x7c, 0x4a, 0x1d, 0x72, 0xd6, 0xcb, 0x66, 0x26, 0xef, 0x65, 0xb3, 0xd8, 0xd0, 0xe8, 0xa4, + 0x16, 0xb6, 0xa8, 0xb3, 0x6f, 0x6e, 0x51, 0xcb, 0x0e, 0xef, 0xac, 0x1f, 0x43, 0x52, 0xd3, 0xdd, + 0xc6, 0x99, 0x7c, 0xaa, 0x26, 0x4c, 0x63, 0x7a, 0x6d, 0x2a, 0xd8, 0x6b, 0x59, 0x2f, 0x29, 0xd7, + 0x01, 0x7c, 0xdd, 0xa0, 0x6b, 0x30, 0x63, 0xb6, 0x35, 0x77, 0xb3, 0x4b, 0x6e, 0x35, 0x73, 0x76, + 0xba, 0x38, 0xfd, 0xbc, 0xad, 0xd5, 0xd7, 0xa4, 0x69, 0xb3, 0xad, 0xd5, 0x35, 0x7a, 0xea, 0x07, + 0x3e, 0x92, 0xbd, 0x48, 0xb8, 0x39, 0x69, 0xd6, 0xc0, 0x47, 0x6b, 0xd8, 0x56, 0x79, 0x87, 0xfb, + 0x2d, 0x01, 0xf2, 0xae, 0xee, 0xe3, 0x35, 0x2a, 0x69, 0xbd, 0xc3, 0x07, 0x59, 0xf2, 0x7c, 0x83, + 0xcc, 0xe5, 0xe3, 0x1b, 0x8b, 0x7f, 0x45, 0xe0, 0xb1, 0xcd, 0x0d, 0x55, 0x71, 0x88, 0x53, 0x11, + 0xe3, 0xc0, 0xb8, 0x07, 0x45, 0x4b, 0x31, 0x34, 0xb3, 0xa3, 0x7f, 0x81, 0xd9, 0x42, 0xa8, 0xcd, + 0x3f, 0x7b, 0x16, 0x3c, 0x3a, 0x5d, 0xf5, 0xb3, 0xc5, 0xdf, 0x4f, 0xf0, 0x38, 0x68, 0xaf, 0x18, + 0x71, 0xaa, 0xeb, 0xdb, 0x30, 0x1f, 0x9c, 0xda, 0x58, 0xa8, 0x25, 0x1b, 0xad, 0xef, 0x45, 0xc8, + 0x8b, 0x2a, 0x08, 0x8b, 0x67, 0x74, 0x83, 0xeb, 0x03, 0x33, 0x22, 0x15, 0x86, 0x2a, 0x90, 0xe5, + 0x1f, 0x1f, 0x8c, 0x3d, 0xd3, 0x0d, 0x14, 0x7b, 0x7b, 0x58, 0x58, 0x64, 0xdd, 0xd8, 0x33, 0xdd, + 0x0f, 0xf9, 0x96, 0x4b, 0xb0, 0xcb, 0x3f, 0x0f, 0xd3, 0x34, 0xf9, 0x0d, 0x4c, 0x34, 0x6f, 0xcd, + 0x3f, 0x49, 0xc0, 0x4d, 0x5a, 0xfa, 0x97, 0xd8, 0xd2, 0xf7, 0x4e, 0xb6, 0x2d, 0xd3, 0xc1, 0xaa, + 0x83, 0x35, 0x7f, 0xe5, 0x3c, 0x56, 0xbb, 0x97, 0xe9, 0xba, 0x2f, 0x38, 0x57, 0xc0, 0x99, 0xc7, + 0x85, 0x36, 0xa0, 0xc0, 0x43, 0x0b, 0x94, 0xb6, 0x7e, 0x88, 0x65, 0xc5, 0x39, 0xcf, 0xec, 0x96, + 0x63, 0xbc, 0x2b, 0x84, 0x75, 0xc5, 0x41, 0x1a, 0x64, 0xb8, 0x30, 0x5d, 0xe3, 0x07, 0x12, 0x3d, + 0xfd, 0x6a, 0x0b, 0x8a, 0x69, 0x16, 0xdf, 0x50, 0x5f, 0x93, 0xd2, 0x4c, 0x72, 0x5d, 0x13, 0xff, + 0x93, 0x00, 0xb7, 0xc6, 0xa8, 0x38, 0xce, 0xae, 0x5b, 0x86, 0xf4, 0x21, 0x79, 0x91, 0xce, 0x75, + 0x9c, 0x96, 0xbc, 0x67, 0xb4, 0x09, 0xb9, 0x3d, 0x45, 0x6f, 0xfb, 0x5d, 0x7a, 0x78, 0x7c, 0x62, + 0x74, 0x34, 0xee, 0x1c, 0x63, 0x67, 0x7d, 0x58, 0xfc, 0x8d, 0x04, 0xcc, 0xaf, 0x68, 0x5a, 0xa3, + 0xc1, 0x6d, 0x60, 0x7c, 0x3d, 0xc5, 0x05, 0x99, 0x09, 0x1f, 0x64, 0xa2, 0xf7, 0x00, 0x69, 0xba, + 0xcd, 0x0e, 0x3e, 0xb1, 0xf7, 0x15, 0xcd, 0x3c, 0xf2, 0xe3, 0x32, 0xe6, 0xdd, 0x94, 0x86, 0x9b, + 0x80, 0x1a, 0x40, 0xd1, 0x8e, 0x6c, 0x3b, 0x8a, 0xf7, 0xdd, 0xe9, 0xd6, 0x44, 0x5b, 0xca, 0x18, + 0x0c, 0xf2, 0x1e, 0xa5, 0x0c, 0x91, 0x43, 0x7f, 0x12, 0xbf, 0x5d, 0x27, 0x55, 0x77, 0x64, 0xc5, + 0x76, 0xf7, 0x0f, 0xb1, 0x23, 0x57, 0xf2, 0x8c, 0xbe, 0x62, 0xb3, 0x6d, 0x41, 0x6c, 0xc3, 0x83, + 0xaf, 0x9a, 0x38, 0x21, 0xf1, 0xdf, 0x13, 0x20, 0x2f, 0xe1, 0x3d, 0x0b, 0xdb, 0xb1, 0x2e, 0x0a, + 0x3c, 0x81, 0x39, 0x8b, 0x49, 0x95, 0xf7, 0x2c, 0xb3, 0x73, 0x9e, 0x71, 0x95, 0xe5, 0x8c, 0x4f, + 0x2c, 0xb3, 0xc3, 0x0d, 0xcb, 0x4b, 0x28, 0x78, 0x65, 0x8c, 0xb3, 0xf2, 0xbf, 0x43, 0xb7, 0x4b, + 0x33, 0xc1, 0x71, 0x07, 0x48, 0xc4, 0xab, 0x01, 0xfa, 0xa1, 0x2a, 0x58, 0xd0, 0x38, 0xd5, 0xf0, + 0xdf, 0x04, 0xc8, 0x37, 0x7a, 0xbb, 0xec, 0xcc, 0xae, 0xf8, 0x34, 0x50, 0x85, 0x4c, 0x1b, 0xef, + 0x39, 0xf2, 0x1b, 0x85, 0xea, 0xa7, 0x09, 0x2b, 0xdd, 0xa8, 0xf0, 0x14, 0xc0, 0xa2, 0x5b, 0xec, + 0xa8, 0x9c, 0xe4, 0x39, 0xe5, 0x64, 0x28, 0x2f, 0x21, 0x8b, 0x7f, 0x9e, 0x80, 0x82, 0x57, 0xcd, + 0x38, 0xad, 0xe4, 0xab, 0x90, 0x75, 0x48, 0x9e, 0xc7, 0x3a, 0xcc, 0xf3, 0x98, 0x90, 0x68, 0x0b, + 0xb1, 0x04, 0x0b, 0xd4, 0x71, 0x91, 0x95, 0x6e, 0xb7, 0xad, 0xbb, 0x70, 0x97, 0xda, 0x9f, 0x94, + 0x34, 0x4f, 0x93, 0x56, 0x58, 0x0a, 0x05, 0xba, 0xe8, 0x57, 0x05, 0x98, 0xdb, 0xb3, 0x30, 0xfe, + 0x02, 0xcb, 0x14, 0x7a, 0x4d, 0x16, 0xf4, 0xb2, 0x46, 0xca, 0xf0, 0x95, 0x3f, 0x8a, 0x67, 0xd9, + 0x8b, 0x1b, 0xe4, 0xbd, 0xbc, 0xd3, 0xbe, 0x82, 0x79, 0xda, 0x28, 0x71, 0xef, 0x2d, 0x16, 0x7f, + 0x98, 0x00, 0x14, 0x94, 0xfc, 0xd3, 0x6b, 0xcc, 0x44, 0x7c, 0x8d, 0xf9, 0x2e, 0x20, 0x16, 0x36, + 0x69, 0xcb, 0x5d, 0x6c, 0xc9, 0x36, 0x56, 0x4d, 0x7e, 0x3c, 0x94, 0x20, 0x15, 0x79, 0xca, 0x36, + 0xb6, 0x1a, 0x94, 0x8e, 0x56, 0x00, 0x7c, 0x97, 0x8e, 0xcf, 0x38, 0x93, 0x78, 0x74, 0x19, 0xcf, + 0xa3, 0x13, 0xbf, 0x27, 0x40, 0x7e, 0x53, 0x6f, 0x59, 0x4a, 0xac, 0xa7, 0x1f, 0xa1, 0x8f, 0xc3, + 0x8b, 0xe9, 0xd9, 0x87, 0xe5, 0xa8, 0x40, 0x1f, 0x96, 0xc3, 0x85, 0x68, 0x9c, 0x81, 0x18, 0x72, + 0xaf, 0x44, 0x71, 0x5a, 0xb0, 0xff, 0x50, 0x86, 0x39, 0x5e, 0xee, 0x1d, 0x43, 0x37, 0x0d, 0xf4, + 0x00, 0x92, 0x2d, 0xfe, 0xb1, 0x24, 0x1b, 0xb9, 0xb0, 0xe9, 0x9f, 0x2d, 0x58, 0x9b, 0x92, 0x48, + 0x5e, 0xc2, 0xd2, 0xed, 0x39, 0x11, 0xae, 0xa4, 0x1f, 0xfb, 0x1e, 0x64, 0xe9, 0xf6, 0x1c, 0xd4, + 0x80, 0x82, 0xea, 0x1f, 0x68, 0x26, 0x13, 0xf6, 0xe4, 0x50, 0xd8, 0x19, 0x79, 0xb4, 0x5c, 0x6d, + 0x4a, 0xca, 0xab, 0xa1, 0x04, 0x54, 0x09, 0x9e, 0xa3, 0x95, 0x1a, 0x08, 0xac, 0xf3, 0x77, 0x6d, + 0x87, 0xcf, 0xf0, 0xaa, 0x4d, 0x05, 0x8e, 0xdb, 0x42, 0x1f, 0xc3, 0x8c, 0x46, 0x4f, 0x6c, 0xe2, + 0x26, 0x20, 0xaa, 0xa1, 0x43, 0x07, 0x63, 0xd5, 0xa6, 0x24, 0xce, 0x81, 0xd6, 0x61, 0x8e, 0xfd, + 0x62, 0x8e, 0x1d, 0xc7, 0xe2, 0xb7, 0x86, 0x4b, 0x08, 0x4c, 0x9d, 0xb5, 0x29, 0x29, 0xab, 0xf9, + 0x54, 0xf4, 0x0d, 0x48, 0xd9, 0xaa, 0xe2, 0xa2, 0xf1, 0xab, 0x43, 0x0e, 0x4a, 0xf1, 0x99, 0x69, + 0x6e, 0xf4, 0x98, 0x1d, 0xed, 0xe9, 0x1c, 0xbb, 0x0b, 0xa3, 0x51, 0xc5, 0x0f, 0x6d, 0xc8, 0x27, + 0xc5, 0xc7, 0x94, 0x80, 0x9e, 0x42, 0x56, 0x21, 0x1e, 0xb2, 0x4c, 0xb7, 0xb4, 0xd2, 0x95, 0xd0, + 0xe8, 0x70, 0x84, 0x81, 0x2d, 0xc8, 0x35, 0xba, 0xeb, 0xdf, 0x25, 0xfa, 0x82, 0x3a, 0xd8, 0x6a, + 0xe1, 0x52, 0x76, 0xb4, 0xa0, 0x60, 0x38, 0x9d, 0x27, 0x88, 0x12, 0x89, 0xa7, 0xec, 0xed, 0x27, + 0xa7, 0x95, 0x9a, 0x1b, 0xfa, 0x01, 0x3c, 0x62, 0xa7, 0x54, 0x6d, 0x4a, 0x9a, 0xdb, 0x0f, 0x90, + 0xd1, 0x12, 0x24, 0x5a, 0x6a, 0x29, 0x37, 0xd4, 0x24, 0x78, 0xbb, 0x81, 0x6a, 0x53, 0x52, 0xa2, + 0xa5, 0xa2, 0x4f, 0x21, 0xcd, 0xb6, 0x76, 0x1c, 0x1b, 0xa5, 0xfc, 0xd0, 0x31, 0x16, 0xde, 0x20, + 0x53, 0x9b, 0x92, 0xe8, 0x6e, 0x12, 0xf2, 0xbe, 0x6d, 0xc8, 0x5b, 0x2c, 0x1e, 0xd1, 0x8d, 0x1e, + 0x2e, 0x0e, 0x0d, 0x0a, 0x88, 0x0a, 0x20, 0xae, 0x51, 0xac, 0x14, 0xa0, 0xa3, 0x6f, 0xc3, 0x85, + 0xb0, 0x44, 0xde, 0xd3, 0xe6, 0x87, 0x7e, 0xe0, 0x1e, 0x1a, 0xcc, 0x5a, 0x9b, 0x92, 0x90, 0x35, + 0x90, 0x88, 0x3e, 0x84, 0x69, 0xd6, 0x6a, 0x88, 0x8a, 0x8c, 0x8a, 0x73, 0xe9, 0x6b, 0x30, 0x96, + 0x9f, 0x74, 0x7e, 0x87, 0x07, 0xe5, 0xc9, 0x6d, 0xb3, 0x55, 0x5a, 0x18, 0xda, 0xf9, 0x07, 0xc3, + 0x0b, 0x49, 0xe7, 0x77, 0x7c, 0x2a, 0x69, 0x77, 0x8b, 0xa5, 0xf0, 0x40, 0xac, 0x0b, 0x43, 0xdb, + 0x3d, 0x22, 0x56, 0xaf, 0x46, 0xb7, 0x4b, 0xf8, 0x64, 0x52, 0x34, 0x8b, 0x9d, 0xed, 0x23, 0xd3, + 0x31, 0x75, 0x71, 0x68, 0xd1, 0x06, 0x0f, 0x3f, 0xaa, 0x51, 0xaf, 0xd2, 0xa3, 0xa2, 0x97, 0x50, + 0xe4, 0x27, 0x70, 0xf8, 0x5f, 0x61, 0x2e, 0x51, 0x79, 0xf7, 0x22, 0x4d, 0x57, 0x54, 0x14, 0x53, + 0x6d, 0x4a, 0x2a, 0xa8, 0xe1, 0x14, 0xf4, 0x19, 0xcc, 0x53, 0x79, 0xb2, 0xea, 0x1f, 0x9a, 0x52, + 0x2a, 0x0d, 0x1c, 0xc1, 0x31, 0xfc, 0x7c, 0x15, 0x57, 0x72, 0x51, 0xed, 0x4b, 0x22, 0xdd, 0x58, + 0x37, 0x74, 0x87, 0x5a, 0xd9, 0xf2, 0xd0, 0x6e, 0x1c, 0x3e, 0x00, 0x92, 0x74, 0x63, 0x9d, 0x51, + 0x48, 0x37, 0x76, 0x78, 0x94, 0x1e, 0x6f, 0x8e, 0xb7, 0x87, 0x76, 0xe3, 0xa8, 0x70, 0x3e, 0xd2, + 0x8d, 0x9d, 0x20, 0x9d, 0x74, 0x63, 0x66, 0x20, 0xfa, 0xe4, 0xbe, 0x33, 0xb4, 0x1b, 0x0f, 0xdd, + 0x5a, 0x4e, 0xba, 0xb1, 0x32, 0x90, 0x88, 0xd6, 0x00, 0x98, 0x03, 0x48, 0xbd, 0x80, 0xab, 0x43, + 0x27, 0x83, 0xfe, 0x68, 0x3d, 0x32, 0x19, 0xb4, 0x5d, 0x1a, 0x31, 0x64, 0x14, 0x5e, 0xca, 0xf4, + 0xa3, 0x73, 0x69, 0x71, 0xa8, 0x21, 0x1b, 0xf8, 0x40, 0x4c, 0x0c, 0xd9, 0x91, 0x47, 0x24, 0xb3, + 0x0a, 0x5b, 0x25, 0x2f, 0x5d, 0x1b, 0x6e, 0x96, 0x83, 0x1f, 0xcb, 0xa8, 0x59, 0xa6, 0x04, 0xb4, + 0x02, 0x19, 0xe2, 0xe4, 0x9c, 0x50, 0x33, 0x74, 0x7d, 0xa8, 0x17, 0xdf, 0xb7, 0x1f, 0xa8, 0x36, + 0x25, 0xa5, 0x5f, 0x73, 0x12, 0x79, 0x3d, 0x5b, 0x3f, 0x2c, 0x89, 0x43, 0x5f, 0x1f, 0x5a, 0x6b, + 0x26, 0xaf, 0x67, 0x1c, 0x48, 0x85, 0x8b, 0xac, 0xad, 0xf8, 0x2e, 0x6f, 0x8b, 0x6f, 0x4c, 0x2e, + 0xdd, 0xa0, 0xa2, 0x86, 0x2e, 0xc4, 0x45, 0x6e, 0x3e, 0xaf, 0x4d, 0x49, 0x0b, 0xca, 0x60, 0x2a, + 0x19, 0xf0, 0x7c, 0xea, 0x61, 0xcb, 0x77, 0xa5, 0x9b, 0x43, 0x07, 0x7c, 0xc4, 0xaa, 0x27, 0x19, + 0xf0, 0x4a, 0x80, 0xcc, 0x26, 0x20, 0x4d, 0xb6, 0x6d, 0x16, 0xa2, 0x70, 0x6b, 0xc4, 0x04, 0xd4, + 0xb7, 0x6e, 0xc2, 0x26, 0x20, 0xad, 0xc1, 0x38, 0x89, 0x20, 0xb5, 0x8d, 0x15, 0x8b, 0x9b, 0xd9, + 0xdb, 0x43, 0x05, 0x0d, 0x9c, 0xb5, 0x48, 0x04, 0xa9, 0x1e, 0x91, 0x38, 0x3c, 0x96, 0x7b, 0x74, + 0x0f, 0xf7, 0x90, 0xef, 0x0c, 0x75, 0x78, 0x22, 0xcf, 0x16, 0x22, 0x0e, 0x8f, 0x15, 0x4a, 0x40, + 0x3f, 0x07, 0xb3, 0x1c, 0xf0, 0x96, 0xee, 0x8e, 0x70, 0x01, 0x83, 0x6b, 0x14, 0x64, 0x5c, 0x73, + 0x1e, 0x66, 0x65, 0x19, 0xd0, 0x66, 0xd5, 0xbb, 0x37, 0xc2, 0xca, 0x0e, 0x60, 0x7d, 0x66, 0x65, + 0x7d, 0x32, 0xb1, 0xb2, 0xac, 0x9f, 0xf2, 0xb9, 0xee, 0xfe, 0x50, 0x2b, 0x3b, 0xb8, 0xdb, 0x88, + 0x58, 0xd9, 0xd7, 0x3e, 0x95, 0xd4, 0xcc, 0x66, 0x80, 0xb3, 0xf4, 0x33, 0x43, 0x6b, 0x16, 0x46, + 0xde, 0xa4, 0x66, 0x9c, 0x87, 0x34, 0x1b, 0xc3, 0x00, 0x4c, 0xd3, 0xef, 0x0e, 0x3f, 0x4a, 0xa1, + 0x1f, 0x65, 0xd5, 0xdc, 0xa5, 0x5d, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0x7c, 0x9f, 0x37, 0xd7, 0xd4, + 0x7b, 0xa3, 0x0d, 0x55, 0xd4, 0xfe, 0x78, 0xcf, 0x50, 0x85, 0x12, 0x69, 0x51, 0xd9, 0xd6, 0x3e, + 0x3a, 0xbe, 0x97, 0x46, 0x9c, 0xfa, 0xd0, 0xb7, 0xc1, 0x92, 0x16, 0xd5, 0x23, 0xfa, 0x43, 0xa8, + 0xc7, 0x8e, 0x24, 0x29, 0x2d, 0x8f, 0x1e, 0x42, 0xe1, 0x43, 0x51, 0xbc, 0x21, 0xc4, 0xc9, 0xde, + 0x9c, 0xe9, 0x7a, 0x18, 0xef, 0x8f, 0x9e, 0x33, 0xfb, 0x5d, 0x0b, 0x36, 0x67, 0x72, 0x9f, 0xe2, + 0x6f, 0x08, 0x70, 0x8d, 0x95, 0x8d, 0xae, 0x81, 0x9e, 0xc8, 0xde, 0x4a, 0x72, 0x60, 0x9b, 0xc9, + 0x03, 0xfa, 0x82, 0x0f, 0x87, 0x15, 0x77, 0xcc, 0xca, 0x78, 0x6d, 0x4a, 0x7a, 0x47, 0x19, 0x95, + 0x8f, 0x74, 0xa9, 0x0e, 0x43, 0x50, 0xa5, 0x87, 0x43, 0xbb, 0x54, 0x18, 0xf5, 0x91, 0x2e, 0xc5, + 0x79, 0x56, 0x67, 0xf9, 0x77, 0x68, 0x6f, 0x5b, 0x6e, 0xa1, 0x58, 0x5c, 0x4f, 0xa5, 0x2f, 0x17, + 0x4b, 0xeb, 0xa9, 0xf4, 0x95, 0x62, 0x79, 0x3d, 0x95, 0x7e, 0xab, 0xf8, 0xb6, 0xf8, 0x8f, 0xca, + 0x90, 0x73, 0x41, 0x17, 0x03, 0x54, 0x0f, 0x83, 0x80, 0xea, 0xea, 0x30, 0x40, 0xc5, 0x61, 0x1a, + 0x47, 0x54, 0x0f, 0x83, 0x88, 0xea, 0xea, 0x30, 0x44, 0xe5, 0xf3, 0x10, 0x48, 0xd5, 0x1c, 0x06, + 0xa9, 0xee, 0x4d, 0x00, 0xa9, 0x3c, 0x51, 0xfd, 0x98, 0x6a, 0x6d, 0x10, 0x53, 0xdd, 0x1c, 0x8d, + 0xa9, 0x3c, 0x51, 0x01, 0x50, 0xf5, 0xb8, 0x0f, 0x54, 0x5d, 0x1f, 0x01, 0xaa, 0x3c, 0x7e, 0x17, + 0x55, 0x6d, 0x44, 0xa2, 0xaa, 0xdb, 0xe3, 0x50, 0x95, 0x27, 0x27, 0x04, 0xab, 0x3e, 0x08, 0xc1, + 0xaa, 0xc5, 0xa1, 0xb0, 0xca, 0xe3, 0x66, 0xb8, 0xea, 0x93, 0x7e, 0x5c, 0x75, 0x7d, 0x04, 0xae, + 0xf2, 0x6b, 0xc0, 0x81, 0x55, 0x2d, 0x0a, 0x58, 0xdd, 0x1a, 0x03, 0xac, 0x3c, 0x29, 0x41, 0x64, + 0x55, 0x8b, 0x42, 0x56, 0xb7, 0xc6, 0x20, 0xab, 0x3e, 0x49, 0x0c, 0x5a, 0x6d, 0x45, 0x43, 0xab, + 0x3b, 0x63, 0xa1, 0x95, 0x27, 0x2d, 0x8c, 0xad, 0x96, 0x03, 0xd8, 0xea, 0x9d, 0x21, 0xd8, 0xca, + 0x63, 0x25, 0xe0, 0xea, 0x9b, 0x03, 0xe0, 0x4a, 0x1c, 0x05, 0xae, 0x3c, 0x5e, 0x0f, 0x5d, 0xbd, + 0x18, 0x82, 0xae, 0xee, 0x8e, 0x47, 0x57, 0x9e, 0xb0, 0x3e, 0x78, 0xa5, 0x8c, 0x84, 0x57, 0xef, + 0x4d, 0x08, 0xaf, 0x3c, 0xe9, 0x51, 0xf8, 0xea, 0xa3, 0x30, 0xbe, 0xba, 0x36, 0x1c, 0x5f, 0x79, + 0x62, 0x38, 0xc0, 0xda, 0x88, 0x04, 0x58, 0xb7, 0xc7, 0x01, 0x2c, 0x7f, 0x1c, 0x04, 0x11, 0xd6, + 0x56, 0x34, 0xc2, 0xba, 0x33, 0x16, 0x61, 0xf9, 0xcd, 0x1f, 0x82, 0x58, 0x1b, 0x91, 0x10, 0xeb, + 0xf6, 0x38, 0x88, 0xe5, 0x17, 0x2e, 0x88, 0xb1, 0x5e, 0x0d, 0xc5, 0x58, 0xf7, 0x27, 0xc1, 0x58, + 0x9e, 0xd0, 0x01, 0x90, 0xf5, 0xf9, 0x70, 0x90, 0xf5, 0x33, 0xe7, 0x38, 0xc4, 0x32, 0x12, 0x65, + 0x7d, 0x73, 0x00, 0x65, 0x89, 0xa3, 0x50, 0x96, 0xdf, 0x9f, 0x5d, 0x98, 0xa5, 0x8c, 0x04, 0x45, + 0xef, 0x4d, 0x08, 0x8a, 0xfc, 0xce, 0x17, 0x81, 0x8a, 0xaa, 0x11, 0xa8, 0xe8, 0xe6, 0x68, 0x54, + 0xe4, 0x9b, 0x73, 0x1f, 0x16, 0xd5, 0xa2, 0x60, 0xd1, 0xad, 0x31, 0xb0, 0xc8, 0xb7, 0x42, 0x01, + 0x5c, 0xf4, 0xb8, 0x0f, 0x17, 0x5d, 0x1f, 0x1b, 0x66, 0x15, 0x00, 0x46, 0xab, 0x83, 0xc0, 0xe8, + 0xc6, 0x48, 0x60, 0xe4, 0x49, 0xf0, 0x91, 0xd1, 0xe3, 0x3e, 0x64, 0x74, 0x7d, 0x04, 0x32, 0xf2, + 0x0b, 0xc0, 0xa1, 0x91, 0x36, 0x1a, 0x1a, 0x2d, 0x4d, 0x0a, 0x8d, 0x3c, 0xc1, 0x91, 0xd8, 0x68, + 0x2b, 0x1a, 0x1b, 0xdd, 0x99, 0x30, 0x02, 0x62, 0x00, 0x1c, 0xd5, 0xa2, 0xc0, 0xd1, 0xad, 0x31, + 0xe0, 0x28, 0x38, 0x87, 0x78, 0xe8, 0xa8, 0x16, 0x85, 0x8e, 0x6e, 0x8d, 0x41, 0x47, 0xbe, 0xa4, + 0x00, 0x3c, 0x6a, 0x0e, 0x83, 0x47, 0xf7, 0x26, 0x80, 0x47, 0xbe, 0xf3, 0xd2, 0x87, 0x8f, 0x3e, + 0xed, 0xc7, 0x47, 0xe2, 0x28, 0x7c, 0xe4, 0x8f, 0x48, 0x17, 0x20, 0x6d, 0x45, 0x03, 0xa4, 0x3b, + 0x63, 0x01, 0x52, 0xd0, 0x48, 0x06, 0x10, 0xd2, 0x46, 0x24, 0x42, 0xba, 0x3d, 0x0e, 0x21, 0xf9, + 0x46, 0x32, 0x08, 0x91, 0x3e, 0xed, 0x87, 0x48, 0xe2, 0x28, 0x88, 0xe4, 0x57, 0xce, 0xc5, 0x48, + 0xb5, 0x28, 0x8c, 0x74, 0x6b, 0x0c, 0x46, 0xf2, 0x1b, 0x2f, 0x00, 0x92, 0x94, 0x91, 0x20, 0xe9, + 0xbd, 0x09, 0x41, 0x52, 0x9f, 0xe1, 0x0a, 0xa3, 0xa4, 0x5a, 0x14, 0x4a, 0xba, 0x35, 0x06, 0x25, + 0x05, 0x0a, 0xeb, 0xc3, 0xa4, 0xad, 0x68, 0x98, 0x74, 0x67, 0x2c, 0x4c, 0xea, 0x1b, 0x4d, 0x2e, + 0x4e, 0xda, 0x88, 0xc4, 0x49, 0xb7, 0xc7, 0xe1, 0xa4, 0xbe, 0x89, 0x8f, 0x3b, 0x07, 0xbf, 0x3c, + 0x39, 0x50, 0xfa, 0xe8, 0xfc, 0x40, 0xc9, 0x7b, 0xe7, 0x18, 0xa4, 0xf4, 0x69, 0x3f, 0x52, 0x12, + 0x47, 0x21, 0x25, 0xbf, 0x67, 0x9d, 0x0f, 0x2a, 0xad, 0xa7, 0xd2, 0x6f, 0x17, 0xdf, 0x11, 0xff, + 0x6c, 0x06, 0x66, 0x6a, 0x5e, 0x60, 0x92, 0x5f, 0x4b, 0xe1, 0x4d, 0x4e, 0xc2, 0x42, 0x6b, 0x64, + 0xc4, 0x53, 0xbb, 0x39, 0xfe, 0xfc, 0xc4, 0xc1, 0x13, 0xfe, 0x38, 0xeb, 0x1b, 0x6c, 0x49, 0x47, + 0x1f, 0x40, 0xae, 0x67, 0x63, 0x4b, 0xee, 0x5a, 0xba, 0x69, 0xe9, 0x0e, 0xdb, 0x76, 0x23, 0xac, + 0x16, 0xbf, 0x3c, 0x5d, 0x9c, 0xdb, 0xb1, 0xb1, 0xb5, 0xcd, 0xe9, 0xd2, 0x5c, 0x2f, 0xf0, 0xe4, + 0xde, 0x75, 0x36, 0x3d, 0xf9, 0x5d, 0x67, 0x2f, 0xa0, 0x68, 0x61, 0x45, 0x0b, 0x79, 0x30, 0xec, + 0xd4, 0xa9, 0xe8, 0x3e, 0x47, 0xb7, 0xc5, 0xb9, 0x39, 0xe9, 0xe9, 0x53, 0x05, 0x2b, 0x4c, 0x44, + 0x0f, 0xe0, 0x62, 0x47, 0x39, 0xa6, 0x41, 0xac, 0xb2, 0xeb, 0x14, 0xd2, 0xc0, 0x54, 0x76, 0x41, + 0x19, 0xea, 0x28, 0xc7, 0xf4, 0xe2, 0x34, 0x96, 0x44, 0x6f, 0x3d, 0xb9, 0x05, 0x79, 0x4d, 0xb7, + 0x1d, 0xdd, 0x50, 0x1d, 0x7e, 0xa0, 0x31, 0x3b, 0x0c, 0x38, 0xe7, 0x52, 0xd9, 0xa9, 0xc5, 0x15, + 0x28, 0xb4, 0x14, 0x07, 0x1f, 0x29, 0x27, 0xb2, 0xbb, 0x1f, 0x2e, 0x4b, 0xcf, 0xee, 0x7c, 0xeb, + 0xec, 0x74, 0x31, 0xf7, 0x94, 0x25, 0x0d, 0x6c, 0x8b, 0xcb, 0xb5, 0x02, 0x09, 0x1a, 0xba, 0x03, + 0x05, 0xc5, 0x3e, 0x31, 0x54, 0x5a, 0x65, 0x6c, 0xd8, 0x3d, 0x9b, 0xc2, 0x8c, 0xb4, 0x94, 0xa7, + 0xe4, 0x8a, 0x4b, 0x45, 0xd7, 0x61, 0x8e, 0x87, 0xf3, 0xb3, 0xdb, 0x95, 0x0a, 0xb4, 0xf8, 0xfc, + 0xda, 0x0e, 0x76, 0xc1, 0xd2, 0x63, 0x28, 0xf3, 0x4b, 0x08, 0x8e, 0x14, 0x4b, 0x93, 0xa9, 0x26, + 0xfd, 0x3e, 0x57, 0xa4, 0x62, 0x2f, 0xb3, 0x4b, 0x07, 0x48, 0x06, 0xa2, 0x3e, 0x7f, 0x5c, 0x34, + 0x61, 0x5e, 0x6d, 0xeb, 0x1e, 0x2a, 0x60, 0x4e, 0xd4, 0xfc, 0xd0, 0x11, 0x52, 0xa1, 0x79, 0xfb, + 0x3f, 0x33, 0x17, 0xd4, 0x30, 0x19, 0x35, 0x80, 0x9e, 0xdc, 0x23, 0x77, 0xcd, 0xb6, 0xae, 0x9e, + 0x50, 0x58, 0x10, 0x3e, 0x52, 0x7d, 0xe4, 0xc5, 0x06, 0xaf, 0x14, 0xdd, 0xd9, 0xa6, 0x9c, 0x12, + 0x1c, 0x79, 0xbf, 0xd9, 0x71, 0xc8, 0xeb, 0xa9, 0x34, 0x14, 0xb3, 0xeb, 0xa9, 0xf4, 0x5c, 0x31, + 0xb7, 0x9e, 0x4a, 0xe7, 0x8b, 0x05, 0xf1, 0x6f, 0x09, 0x50, 0xe8, 0x2b, 0x17, 0xaa, 0xc1, 0x45, + 0xcd, 0x1b, 0x0a, 0x32, 0xdf, 0xfb, 0xa5, 0x9b, 0x06, 0x3f, 0x2b, 0x7e, 0xe1, 0xcb, 0xd3, 0xc5, + 0x02, 0xcd, 0xfd, 0xd4, 0x4b, 0x92, 0x2e, 0xf8, 0x1c, 0x3e, 0x15, 0x7d, 0x04, 0x79, 0xe6, 0x5e, + 0x7a, 0x97, 0x07, 0xd2, 0xa0, 0xfd, 0xd5, 0xf9, 0x2f, 0x4f, 0x17, 0x73, 0xd4, 0xa7, 0x74, 0xcf, + 0x72, 0x96, 0x72, 0xed, 0xe0, 0xa3, 0xf8, 0xeb, 0x02, 0xcc, 0x85, 0xf6, 0x57, 0x3d, 0xee, 0xfb, + 0xb8, 0x7d, 0x25, 0x1a, 0x97, 0x0e, 0x8b, 0x70, 0x4c, 0xf3, 0x7e, 0xec, 0x86, 0x8b, 0x2e, 0x0e, + 0xc7, 0x35, 0x74, 0x95, 0xc6, 0x8d, 0x9d, 0x71, 0xd9, 0x3e, 0x4e, 0x7d, 0xff, 0x07, 0x8b, 0x53, + 0xe2, 0x5f, 0xa4, 0x20, 0x17, 0xde, 0x4d, 0x55, 0xef, 0x2b, 0x57, 0xd4, 0xbc, 0x11, 0xe2, 0x58, + 0x1a, 0x71, 0xa3, 0x4c, 0xc6, 0xbf, 0x94, 0x81, 0x15, 0xf3, 0xda, 0x88, 0x4f, 0xf8, 0xc1, 0x72, + 0xfa, 0x8c, 0xe5, 0xff, 0x98, 0xf4, 0xec, 0xe7, 0x12, 0x4c, 0xd3, 0x83, 0x91, 0x78, 0xd1, 0x4a, + 0xfd, 0x7d, 0x88, 0xf8, 0xd2, 0x24, 0x5d, 0x62, 0xd9, 0x88, 0xbd, 0x6d, 0xbe, 0xd1, 0xc9, 0x83, + 0xfe, 0x90, 0x38, 0xff, 0x5d, 0x8d, 0x3d, 0x76, 0xf2, 0xe4, 0xff, 0xc3, 0xf8, 0x1a, 0xf2, 0x3e, + 0xf4, 0x8b, 0x50, 0x50, 0xcd, 0x76, 0x9b, 0xcd, 0xa5, 0xcc, 0x62, 0x0d, 0x9e, 0x45, 0x43, 0x8b, + 0xc0, 0xaf, 0xe7, 0x5c, 0xf2, 0xae, 0xe9, 0x5c, 0x92, 0xf8, 0x35, 0x9d, 0x81, 0xe0, 0xdd, 0xbc, + 0x27, 0xcc, 0x35, 0x74, 0xa1, 0x38, 0xe2, 0xd9, 0x37, 0x89, 0x23, 0x66, 0xb1, 0xe7, 0xbc, 0xe7, + 0xfd, 0x91, 0xc0, 0x03, 0x75, 0x9e, 0x99, 0xe6, 0x41, 0xcf, 0x8b, 0xff, 0x2d, 0x07, 0xcf, 0x91, + 0x4c, 0x7f, 0x79, 0xba, 0x98, 0x92, 0xbc, 0x83, 0x24, 0xa3, 0x66, 0x86, 0xc4, 0x57, 0x9b, 0x19, + 0xae, 0xc3, 0x5c, 0xd7, 0xc2, 0x7b, 0xd8, 0x51, 0xf7, 0x65, 0xa3, 0xd7, 0xe1, 0xdb, 0x84, 0xb2, + 0x2e, 0x6d, 0xab, 0xd7, 0x41, 0xf7, 0xa0, 0xe8, 0x65, 0xe1, 0x18, 0xde, 0x3d, 0x68, 0xcc, 0xa5, + 0x73, 0xc4, 0x2f, 0xfe, 0x2f, 0x01, 0x16, 0x42, 0x75, 0xe2, 0x63, 0x6a, 0x1d, 0xb2, 0xbe, 0x39, + 0xb1, 0x4b, 0xc2, 0x39, 0xa3, 0x61, 0x83, 0xcc, 0x48, 0x86, 0x4b, 0xee, 0x6b, 0xe9, 0x45, 0x06, + 0xbe, 0xd8, 0xc4, 0x39, 0xc5, 0x5e, 0xf4, 0xe5, 0xac, 0x05, 0x5e, 0xe0, 0x0d, 0xb2, 0xe4, 0x44, + 0x83, 0x4c, 0xfc, 0x2d, 0x01, 0x8a, 0xf4, 0x05, 0x4f, 0x30, 0xd6, 0x62, 0xb1, 0x6e, 0x6e, 0x94, + 0x79, 0x62, 0xf2, 0x8d, 0x40, 0xa1, 0xcb, 0x57, 0x92, 0xe1, 0xcb, 0x57, 0xc4, 0x1f, 0x08, 0x90, + 0xf7, 0x4a, 0xc8, 0x2e, 0x3c, 0x1c, 0x71, 0x5c, 0xe9, 0x9b, 0x5d, 0xea, 0xe7, 0x1e, 0xab, 0x32, + 0xd1, 0x1d, 0x8c, 0xc1, 0x63, 0x55, 0xd8, 0x65, 0x74, 0x7f, 0xc7, 0xed, 0x39, 0xa4, 0x88, 0x15, + 0xff, 0xf8, 0x8c, 0x37, 0xd8, 0x13, 0x25, 0xd1, 0xbb, 0x62, 0xcd, 0xf6, 0x21, 0x3b, 0x09, 0x67, + 0x22, 0xb3, 0x87, 0x78, 0x78, 0x1a, 0xf0, 0xd5, 0x3e, 0xad, 0xd9, 0xa0, 0xb7, 0xc8, 0xb2, 0xdf, + 0xb6, 0xf8, 0x24, 0xa0, 0x40, 0xda, 0xf8, 0x44, 0x4b, 0x13, 0x99, 0x62, 0x57, 0x4b, 0xac, 0xaf, + 0xfc, 0x41, 0xb0, 0x25, 0xaa, 0x87, 0x04, 0xe5, 0x3d, 0x82, 0xe4, 0xa1, 0xd2, 0x1e, 0x15, 0xe1, + 0x15, 0x6a, 0x39, 0x89, 0xe4, 0x46, 0x4f, 0x42, 0xa7, 0x8e, 0x24, 0x86, 0x23, 0x92, 0x41, 0x95, + 0x86, 0x4e, 0x27, 0xf9, 0x30, 0xdc, 0xd7, 0x47, 0xbe, 0x3e, 0xd8, 0xe9, 0x3f, 0x4e, 0xfd, 0xe8, + 0x07, 0x8b, 0x82, 0xf8, 0x09, 0x20, 0x09, 0xdb, 0xd8, 0x79, 0xd1, 0x33, 0x2d, 0xff, 0x04, 0x97, + 0xdb, 0x7d, 0xb7, 0xd1, 0x4c, 0xaf, 0x66, 0xcf, 0xa2, 0x0e, 0x79, 0xba, 0x08, 0x0b, 0x21, 0x6e, + 0x66, 0x2c, 0xc4, 0x0f, 0xe1, 0xca, 0x53, 0xd3, 0xb6, 0xf5, 0x2e, 0x81, 0xb6, 0x74, 0x54, 0x92, + 0xa9, 0xc5, 0x33, 0x8f, 0xe9, 0x2e, 0x5d, 0xe4, 0x30, 0x98, 0x19, 0xc9, 0x48, 0xde, 0xb3, 0xf8, + 0xbb, 0x02, 0x5c, 0x1e, 0xe4, 0x64, 0x5a, 0x8e, 0xda, 0xc2, 0x39, 0xab, 0x9a, 0xfe, 0x89, 0x7f, + 0xe3, 0x7b, 0xab, 0x9b, 0x9d, 0xb8, 0xab, 0xfc, 0x9d, 0x72, 0x47, 0xa1, 0xe6, 0x83, 0x6f, 0x27, + 0xcf, 0x73, 0xf2, 0x26, 0xa3, 0xfa, 0x96, 0x24, 0x35, 0x99, 0x25, 0x69, 0x42, 0x61, 0xdd, 0xd4, + 0x0d, 0xe2, 0x15, 0xbb, 0xf5, 0x5d, 0x81, 0xfc, 0xae, 0x6e, 0x28, 0xd6, 0x89, 0xec, 0x06, 0x16, + 0x0a, 0xe3, 0x02, 0x0b, 0xa5, 0x1c, 0xe3, 0xe0, 0x8f, 0xe2, 0x8f, 0x05, 0x28, 0xfa, 0x62, 0xb9, + 0x45, 0x7e, 0x17, 0x40, 0x6d, 0xf7, 0x6c, 0x07, 0x5b, 0x6e, 0x2b, 0xcd, 0xb1, 0x70, 0xfc, 0x0a, + 0xa3, 0xd6, 0xd7, 0xa4, 0x0c, 0xcf, 0x50, 0xd7, 0xd0, 0x8d, 0xf0, 0x69, 0x17, 0xd3, 0xab, 0x70, + 0x36, 0x70, 0xc6, 0x05, 0x69, 0x76, 0xdb, 0x31, 0x2d, 0x0f, 0x53, 0xf1, 0x66, 0x77, 0xcf, 0xef, + 0xa7, 0x9b, 0xb4, 0x49, 0xbe, 0x15, 0xc8, 0x13, 0x77, 0xe1, 0x10, 0x7b, 0x55, 0x4a, 0x8d, 0xaf, + 0x12, 0xe3, 0x70, 0xab, 0xf4, 0xaf, 0x88, 0x93, 0xcb, 0x5a, 0xc3, 0x6b, 0xe1, 0x11, 0x16, 0x6d, + 0x0d, 0xd2, 0xce, 0xb1, 0x21, 0x77, 0xb0, 0x77, 0xe7, 0xcd, 0x39, 0x8e, 0xe3, 0x9b, 0x75, 0xd8, + 0x23, 0xbd, 0x16, 0x91, 0xdf, 0xc9, 0xcd, 0x87, 0xcb, 0x95, 0x25, 0x76, 0x69, 0xf7, 0x92, 0x7b, + 0x69, 0xf7, 0xd2, 0x1a, 0xcf, 0xc0, 0x8c, 0xfa, 0xf7, 0xff, 0xeb, 0xa2, 0x20, 0x79, 0x4c, 0x6c, + 0xde, 0xbf, 0xdf, 0x20, 0xbd, 0x7e, 0x60, 0x66, 0x46, 0x79, 0x80, 0xc0, 0x65, 0x46, 0xfc, 0x1a, + 0xe8, 0x95, 0x35, 0x79, 0x67, 0xab, 0xf2, 0x7c, 0x73, 0xb3, 0xde, 0x6c, 0x56, 0xd7, 0x8a, 0x02, + 0x2a, 0xc2, 0x5c, 0xe8, 0x2a, 0xa4, 0x04, 0xbb, 0x18, 0xfa, 0xfe, 0x5f, 0x01, 0xf0, 0x6f, 0x55, + 0x23, 0xb2, 0x36, 0xaa, 0x9f, 0xc9, 0x2f, 0x57, 0x9e, 0xed, 0x54, 0x1b, 0xc5, 0x29, 0x84, 0x20, + 0xbf, 0xba, 0xd2, 0xac, 0xd4, 0x64, 0xa9, 0xda, 0xd8, 0x7e, 0xbe, 0xd5, 0xa8, 0xba, 0x17, 0x4a, + 0xdf, 0x5f, 0x83, 0xb9, 0xe0, 0x99, 0x46, 0x68, 0x01, 0x0a, 0x95, 0x5a, 0xb5, 0xb2, 0x21, 0xbf, + 0xac, 0xaf, 0xc8, 0x2f, 0x76, 0xaa, 0x3b, 0xd5, 0xe2, 0x14, 0x2d, 0x1a, 0x25, 0x3e, 0xd9, 0x79, + 0xf6, 0xac, 0x28, 0xa0, 0x02, 0x64, 0xd9, 0x33, 0xbd, 0x36, 0xa9, 0x98, 0xb8, 0xbf, 0x09, 0xd9, + 0xc0, 0xe1, 0xc7, 0xe4, 0x75, 0xdb, 0x3b, 0x8d, 0x9a, 0xdc, 0xac, 0x6f, 0x56, 0x1b, 0xcd, 0x95, + 0xcd, 0x6d, 0x26, 0x83, 0xd2, 0x56, 0x56, 0x9f, 0x4b, 0xcd, 0xa2, 0xe0, 0x3d, 0x37, 0x9f, 0xef, + 0x54, 0x6a, 0x6e, 0x35, 0xc4, 0x54, 0x3a, 0x59, 0x4c, 0xde, 0xff, 0xeb, 0x02, 0x5c, 0x1e, 0x72, + 0xbe, 0x0f, 0xca, 0xc2, 0xec, 0x8e, 0x41, 0x4f, 0x82, 0x2d, 0x4e, 0xa1, 0x5c, 0xe0, 0x88, 0x9f, + 0xa2, 0x80, 0xd2, 0xec, 0x90, 0x95, 0x62, 0x02, 0xcd, 0x40, 0xa2, 0xf1, 0xa8, 0x98, 0x24, 0x25, + 0x0d, 0x9c, 0x90, 0x53, 0x4c, 0xa1, 0x0c, 0x3f, 0x9b, 0xa3, 0x38, 0x8d, 0xe6, 0xfc, 0x23, 0x32, + 0x8a, 0x33, 0x44, 0x94, 0x77, 0xd4, 0x44, 0x71, 0xf6, 0xfe, 0x75, 0x08, 0x6c, 0xe7, 0x47, 0x00, + 0x33, 0xcf, 0x14, 0x07, 0xdb, 0x4e, 0x71, 0x0a, 0xcd, 0x42, 0x72, 0xa5, 0xdd, 0x2e, 0x0a, 0x0f, + 0xff, 0x65, 0x0a, 0xd2, 0xee, 0xf5, 0x40, 0xe8, 0x19, 0x4c, 0xb3, 0xe5, 0xee, 0xc5, 0xe1, 0x68, + 0x81, 0x0e, 0xe8, 0xf2, 0xb5, 0x71, 0x70, 0x42, 0x9c, 0x42, 0x7f, 0x15, 0xb2, 0x01, 0x2f, 0x0a, + 0x0d, 0x5d, 0xb2, 0x0b, 0x79, 0x8e, 0xe5, 0xdb, 0xe3, 0xb2, 0x79, 0xf2, 0x5f, 0x41, 0xc6, 0xb3, + 0xea, 0xe8, 0xc6, 0x28, 0x9b, 0xef, 0xca, 0x1e, 0x3d, 0x31, 0x90, 0xf1, 0x27, 0x4e, 0xbd, 0x2f, + 0x20, 0x0b, 0xd0, 0xa0, 0x01, 0x46, 0x51, 0x41, 0x14, 0x43, 0x2d, 0x7c, 0xf9, 0xfe, 0x44, 0xb9, + 0xfd, 0x77, 0x12, 0x65, 0xf9, 0xb3, 0x48, 0xb4, 0xb2, 0x06, 0xe6, 0xa8, 0x68, 0x65, 0x45, 0x4c, + 0x46, 0x53, 0xe8, 0x05, 0xa4, 0x88, 0xf5, 0x44, 0x51, 0x7e, 0x65, 0x9f, 0xb5, 0x2e, 0xdf, 0x18, + 0x99, 0xc7, 0x15, 0xb9, 0x7a, 0xef, 0x47, 0x7f, 0x7a, 0x75, 0xea, 0x47, 0x67, 0x57, 0x85, 0x1f, + 0x9f, 0x5d, 0x15, 0xfe, 0xf8, 0xec, 0xaa, 0xf0, 0x27, 0x67, 0x57, 0x85, 0xef, 0xfd, 0xe4, 0xea, + 0xd4, 0x8f, 0x7f, 0x72, 0x75, 0xea, 0x8f, 0x7f, 0x72, 0x75, 0xea, 0xf3, 0x59, 0xce, 0xbd, 0x3b, + 0x43, 0x4d, 0xcb, 0xa3, 0xff, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x25, 0xa4, 0x42, 0x40, 0x4a, 0x81, + 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index f3d011c9646b..0796b7792238 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -110,6 +110,13 @@ message ResponseHeader { // A GetRequest is the argument for the Get() method. message GetRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + + // The desired key-level locking mode used during this get. When set to None + // (the default), no key-level locking mode is used - meaning that the get + // does not acquire a lock. When set to any other strength, a lock of that + // strength is acquired with the Unreplicated durability (i.e. best-effort) + // the key, if it exists. + kv.kvserver.concurrency.lock.Strength key_locking = 2; } // A GetResponse is the return value from the Get() method. diff --git a/pkg/roachpb/batch_test.go b/pkg/roachpb/batch_test.go index 0d44a64c61b2..02b7c6d03196 100644 --- a/pkg/roachpb/batch_test.go +++ b/pkg/roachpb/batch_test.go @@ -225,12 +225,14 @@ func TestLockSpanIterate(t *testing.T) { resume Span } testReqs := []testReq{ - {&ScanRequest{}, &ScanResponse{}, sp("a", "c"), sp("b", "c")}, - {&ReverseScanRequest{}, &ReverseScanResponse{}, sp("d", "f"), sp("d", "e")}, - {&PutRequest{}, &PutResponse{}, sp("m", ""), sp("", "")}, - {&DeleteRangeRequest{}, &DeleteRangeResponse{}, sp("n", "p"), sp("o", "p")}, - {&ScanRequest{KeyLocking: lock.Exclusive}, &ScanResponse{}, sp("g", "i"), sp("h", "i")}, - {&ReverseScanRequest{KeyLocking: lock.Exclusive}, &ReverseScanResponse{}, sp("j", "l"), sp("k", "l")}, + {&GetRequest{}, &GetResponse{}, sp("a", ""), sp("", "")}, + {&ScanRequest{}, &ScanResponse{}, sp("b", "d"), sp("c", "d")}, + {&ReverseScanRequest{}, &ReverseScanResponse{}, sp("e", "g"), sp("f", "g")}, + {&PutRequest{}, &PutResponse{}, sp("h", ""), sp("", "")}, + {&DeleteRangeRequest{}, &DeleteRangeResponse{}, sp("i", "k"), sp("j", "k")}, + {&GetRequest{KeyLocking: lock.Exclusive}, &GetResponse{}, sp("l", ""), sp("", "")}, + {&ScanRequest{KeyLocking: lock.Exclusive}, &ScanResponse{}, sp("m", "o"), sp("n", "o")}, + {&ReverseScanRequest{KeyLocking: lock.Exclusive}, &ReverseScanResponse{}, sp("p", "r"), sp("q", "r")}, } // NB: can't import testutils for RunTrueAndFalse. @@ -267,10 +269,10 @@ func TestLockSpanIterate(t *testing.T) { } // The intent writes are replicated locking request. - require.Equal(t, toExpSpans(testReqs[2], testReqs[3]), spans[lock.Replicated]) + require.Equal(t, toExpSpans(testReqs[3], testReqs[4]), spans[lock.Replicated]) // The scans with KeyLocking are unreplicated locking requests. - require.Equal(t, toExpSpans(testReqs[4], testReqs[5]), spans[lock.Unreplicated]) + require.Equal(t, toExpSpans(testReqs[5], testReqs[6], testReqs[7]), spans[lock.Unreplicated]) }) } } From c955e882ef8181b260a72fe1f08152863ecb1c67 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 14 Jan 2021 18:36:50 -0500 Subject: [PATCH 2/2] kv/kvnemesis: support locking Get requests This commit extends kvnemesis to issue locking GetRequests. Much of this is modeled after a98be1f, which added support to kvnemesis for locking ScanRequests. --- pkg/kv/kvnemesis/applier.go | 13 +- pkg/kv/kvnemesis/applier_test.go | 6 +- pkg/kv/kvnemesis/generator.go | 39 +++++- pkg/kv/kvnemesis/generator_test.go | 12 +- pkg/kv/kvnemesis/operations.go | 6 +- pkg/kv/kvnemesis/operations.pb.go | 205 +++++++++++++++++------------ pkg/kv/kvnemesis/operations.proto | 3 +- 7 files changed, 184 insertions(+), 100 deletions(-) diff --git a/pkg/kv/kvnemesis/applier.go b/pkg/kv/kvnemesis/applier.go index fdd9e8fde798..d31b508c55f3 100644 --- a/pkg/kv/kvnemesis/applier.go +++ b/pkg/kv/kvnemesis/applier.go @@ -125,6 +125,7 @@ func applyOp(ctx context.Context, db *kv.DB, op *Operation) { type clientI interface { Get(context.Context, interface{}) (kv.KeyValue, error) + GetForUpdate(context.Context, interface{}) (kv.KeyValue, error) Put(context.Context, interface{}, interface{}) error Scan(context.Context, interface{}, interface{}, int64) ([]kv.KeyValue, error) ScanForUpdate(context.Context, interface{}, interface{}, int64) ([]kv.KeyValue, error) @@ -134,7 +135,11 @@ type clientI interface { func applyClientOp(ctx context.Context, db clientI, op *Operation) { switch o := op.GetValue().(type) { case *GetOperation: - kv, err := db.Get(ctx, o.Key) + fn := db.Get + if o.ForUpdate { + fn = db.GetForUpdate + } + kv, err := fn(ctx, o.Key) if err != nil { o.Result = resultError(ctx, err) } else { @@ -178,7 +183,11 @@ func applyBatchOp( for i := range o.Ops { switch subO := o.Ops[i].GetValue().(type) { case *GetOperation: - b.Get(subO.Key) + if subO.ForUpdate { + b.GetForUpdate(subO.Key) + } else { + b.Get(subO.Key) + } case *PutOperation: b.Put(subO.Key, subO.Value) case *ScanOperation: diff --git a/pkg/kv/kvnemesis/applier_test.go b/pkg/kv/kvnemesis/applier_test.go index 5cb05491a5a2..fae6ebee14ab 100644 --- a/pkg/kv/kvnemesis/applier_test.go +++ b/pkg/kv/kvnemesis/applier_test.go @@ -61,7 +61,7 @@ func TestApplier(t *testing.T) { check(t, step(scan(`a`, `c`)), `db1.Scan(ctx, "a", "c", 0) // ([], nil)`) check(t, step(put(`a`, `1`)), `db0.Put(ctx, "a", 1) // nil`) - check(t, step(get(`a`)), `db1.Get(ctx, "a") // ("1", nil)`) + check(t, step(getForUpdate(`a`)), `db1.GetForUpdate(ctx, "a") // ("1", nil)`) check(t, step(scanForUpdate(`a`, `c`)), `db0.ScanForUpdate(ctx, "a", "c", 0) // (["a":"1"], nil)`) check(t, step(put(`b`, `2`)), `db1.Put(ctx, "b", 2) // nil`) @@ -82,11 +82,11 @@ func TestApplier(t *testing.T) { db1.Run(ctx, b) // nil } `) - checkErr(t, step(batch(put(`b`, `2`), get(`a`), scanForUpdate(`a`, `c`))), ` + checkErr(t, step(batch(put(`b`, `2`), getForUpdate(`a`), scanForUpdate(`a`, `c`))), ` { b := &Batch{} b.Put(ctx, "b", 2) // context canceled - b.Get(ctx, "a") // (nil, context canceled) + b.GetForUpdate(ctx, "a") // (nil, context canceled) b.ScanForUpdate(ctx, "a", "c") // (nil, context canceled) db0.Run(ctx, b) // context canceled } diff --git a/pkg/kv/kvnemesis/generator.go b/pkg/kv/kvnemesis/generator.go index 4e56bda84573..0b4bbfe0d6d4 100644 --- a/pkg/kv/kvnemesis/generator.go +++ b/pkg/kv/kvnemesis/generator.go @@ -73,8 +73,14 @@ type ClosureTxnConfig struct { type ClientOperationConfig struct { // GetMissing is an operation that Gets a key that definitely doesn't exist. GetMissing int + // GetMissingForUpdate is an operation that Gets a key that definitely + // doesn't exist using a locking read. + GetMissingForUpdate int // GetExisting is an operation that Gets a key that likely exists. GetExisting int + // GetExistingForUpdate is an operation that Gets a key that likely exists + // using a locking read. + GetExistingForUpdate int // PutMissing is an operation that Puts a key that definitely doesn't exist. PutMissing int // PutExisting is an operation that Puts a key that likely exists. @@ -143,12 +149,14 @@ type ChangeLeaseConfig struct { // yet pass (for example, if the new operation finds a kv bug or edge case). func newAllOperationsConfig() GeneratorConfig { clientOpConfig := ClientOperationConfig{ - GetMissing: 1, - GetExisting: 1, - PutMissing: 1, - PutExisting: 1, - Scan: 1, - ScanForUpdate: 1, + GetMissing: 1, + GetMissingForUpdate: 1, + GetExisting: 1, + GetExistingForUpdate: 1, + PutMissing: 1, + PutExisting: 1, + Scan: 1, + ScanForUpdate: 1, } batchOpConfig := BatchOperationConfig{ Batch: 4, @@ -370,9 +378,11 @@ func (g *generator) selectOp(rng *rand.Rand, contextuallyValid []opGen) Operatio func (g *generator) registerClientOps(allowed *[]opGen, c *ClientOperationConfig) { addOpGen(allowed, randGetMissing, c.GetMissing) + addOpGen(allowed, randGetMissingForUpdate, c.GetMissingForUpdate) addOpGen(allowed, randPutMissing, c.PutMissing) if len(g.keys) > 0 { addOpGen(allowed, randGetExisting, c.GetExisting) + addOpGen(allowed, randGetExistingForUpdate, c.GetExistingForUpdate) addOpGen(allowed, randPutExisting, c.PutExisting) } addOpGen(allowed, randScan, c.Scan) @@ -387,11 +397,24 @@ func randGetMissing(_ *generator, rng *rand.Rand) Operation { return get(randKey(rng)) } +func randGetMissingForUpdate(_ *generator, rng *rand.Rand) Operation { + op := get(randKey(rng)) + op.Get.ForUpdate = true + return op +} + func randGetExisting(g *generator, rng *rand.Rand) Operation { key := randMapKey(rng, g.keys) return get(key) } +func randGetExistingForUpdate(g *generator, rng *rand.Rand) Operation { + key := randMapKey(rng, g.keys) + op := get(key) + op.Get.ForUpdate = true + return op +} + func randPutMissing(g *generator, rng *rand.Rand) Operation { value := g.getNextValue() key := randKey(rng) @@ -597,6 +620,10 @@ func get(key string) Operation { return Operation{Get: &GetOperation{Key: []byte(key)}} } +func getForUpdate(key string) Operation { + return Operation{Get: &GetOperation{Key: []byte(key), ForUpdate: true}} +} + func put(key, value string) Operation { return Operation{Put: &PutOperation{Key: []byte(key), Value: []byte(value)}} } diff --git a/pkg/kv/kvnemesis/generator_test.go b/pkg/kv/kvnemesis/generator_test.go index 99ab65d7dd65..34d7d90c6af7 100644 --- a/pkg/kv/kvnemesis/generator_test.go +++ b/pkg/kv/kvnemesis/generator_test.go @@ -100,9 +100,17 @@ func TestRandStep(t *testing.T) { switch o := op.GetValue().(type) { case *GetOperation: if _, ok := keys[string(o.Key)]; ok { - client.GetExisting++ + if o.ForUpdate { + client.GetExistingForUpdate++ + } else { + client.GetExisting++ + } } else { - client.GetMissing++ + if o.ForUpdate { + client.GetMissingForUpdate++ + } else { + client.GetMissing++ + } } case *PutOperation: if _, ok := keys[string(o.Key)]; ok { diff --git a/pkg/kv/kvnemesis/operations.go b/pkg/kv/kvnemesis/operations.go index 52ac5de1f474..945cd99857aa 100644 --- a/pkg/kv/kvnemesis/operations.go +++ b/pkg/kv/kvnemesis/operations.go @@ -165,7 +165,11 @@ func (op Operation) format(w *strings.Builder, fctx formatCtx) { } func (op GetOperation) format(w *strings.Builder, fctx formatCtx) { - fmt.Fprintf(w, `%s.Get(ctx, %s)`, fctx.receiver, roachpb.Key(op.Key)) + methodName := `Get` + if op.ForUpdate { + methodName = `GetForUpdate` + } + fmt.Fprintf(w, `%s.%s(ctx, %s)`, fctx.receiver, methodName, roachpb.Key(op.Key)) switch op.Result.Type { case ResultType_Error: err := errors.DecodeError(context.TODO(), *op.Result.Err) diff --git a/pkg/kv/kvnemesis/operations.pb.go b/pkg/kv/kvnemesis/operations.pb.go index ba7c7314a9c9..5dd0cdb4fc61 100644 --- a/pkg/kv/kvnemesis/operations.pb.go +++ b/pkg/kv/kvnemesis/operations.pb.go @@ -45,7 +45,7 @@ func (x ClosureTxnType) String() string { return proto.EnumName(ClosureTxnType_name, int32(x)) } func (ClosureTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{0} + return fileDescriptor_operations_c3246c3d639e73ee, []int{0} } type ResultType int32 @@ -77,7 +77,7 @@ func (x ResultType) String() string { return proto.EnumName(ResultType_name, int32(x)) } func (ResultType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{1} + return fileDescriptor_operations_c3246c3d639e73ee, []int{1} } type BatchOperation struct { @@ -89,7 +89,7 @@ func (m *BatchOperation) Reset() { *m = BatchOperation{} } func (m *BatchOperation) String() string { return proto.CompactTextString(m) } func (*BatchOperation) ProtoMessage() {} func (*BatchOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{0} + return fileDescriptor_operations_c3246c3d639e73ee, []int{0} } func (m *BatchOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -127,7 +127,7 @@ func (m *ClosureTxnOperation) Reset() { *m = ClosureTxnOperation{} } func (m *ClosureTxnOperation) String() string { return proto.CompactTextString(m) } func (*ClosureTxnOperation) ProtoMessage() {} func (*ClosureTxnOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{1} + return fileDescriptor_operations_c3246c3d639e73ee, []int{1} } func (m *ClosureTxnOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -153,15 +153,16 @@ func (m *ClosureTxnOperation) XXX_DiscardUnknown() { var xxx_messageInfo_ClosureTxnOperation proto.InternalMessageInfo type GetOperation struct { - Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` - Result Result `protobuf:"bytes,2,opt,name=result,proto3" json:"result"` + Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + ForUpdate bool `protobuf:"varint,2,opt,name=for_update,json=forUpdate,proto3" json:"for_update,omitempty"` + Result Result `protobuf:"bytes,3,opt,name=result,proto3" json:"result"` } func (m *GetOperation) Reset() { *m = GetOperation{} } func (m *GetOperation) String() string { return proto.CompactTextString(m) } func (*GetOperation) ProtoMessage() {} func (*GetOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{2} + return fileDescriptor_operations_c3246c3d639e73ee, []int{2} } func (m *GetOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -197,7 +198,7 @@ func (m *ScanOperation) Reset() { *m = ScanOperation{} } func (m *ScanOperation) String() string { return proto.CompactTextString(m) } func (*ScanOperation) ProtoMessage() {} func (*ScanOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{3} + return fileDescriptor_operations_c3246c3d639e73ee, []int{3} } func (m *ScanOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -232,7 +233,7 @@ func (m *PutOperation) Reset() { *m = PutOperation{} } func (m *PutOperation) String() string { return proto.CompactTextString(m) } func (*PutOperation) ProtoMessage() {} func (*PutOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{4} + return fileDescriptor_operations_c3246c3d639e73ee, []int{4} } func (m *PutOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -266,7 +267,7 @@ func (m *SplitOperation) Reset() { *m = SplitOperation{} } func (m *SplitOperation) String() string { return proto.CompactTextString(m) } func (*SplitOperation) ProtoMessage() {} func (*SplitOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{5} + return fileDescriptor_operations_c3246c3d639e73ee, []int{5} } func (m *SplitOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -300,7 +301,7 @@ func (m *MergeOperation) Reset() { *m = MergeOperation{} } func (m *MergeOperation) String() string { return proto.CompactTextString(m) } func (*MergeOperation) ProtoMessage() {} func (*MergeOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{6} + return fileDescriptor_operations_c3246c3d639e73ee, []int{6} } func (m *MergeOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -335,7 +336,7 @@ func (m *ChangeReplicasOperation) Reset() { *m = ChangeReplicasOperation func (m *ChangeReplicasOperation) String() string { return proto.CompactTextString(m) } func (*ChangeReplicasOperation) ProtoMessage() {} func (*ChangeReplicasOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{7} + return fileDescriptor_operations_c3246c3d639e73ee, []int{7} } func (m *ChangeReplicasOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -370,7 +371,7 @@ func (m *TransferLeaseOperation) Reset() { *m = TransferLeaseOperation{} func (m *TransferLeaseOperation) String() string { return proto.CompactTextString(m) } func (*TransferLeaseOperation) ProtoMessage() {} func (*TransferLeaseOperation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{8} + return fileDescriptor_operations_c3246c3d639e73ee, []int{8} } func (m *TransferLeaseOperation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -414,7 +415,7 @@ type Operation struct { func (m *Operation) Reset() { *m = Operation{} } func (*Operation) ProtoMessage() {} func (*Operation) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{9} + return fileDescriptor_operations_c3246c3d639e73ee, []int{9} } func (m *Operation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -449,7 +450,7 @@ func (m *KeyValue) Reset() { *m = KeyValue{} } func (m *KeyValue) String() string { return proto.CompactTextString(m) } func (*KeyValue) ProtoMessage() {} func (*KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{10} + return fileDescriptor_operations_c3246c3d639e73ee, []int{10} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -488,7 +489,7 @@ func (m *Result) Reset() { *m = Result{} } func (m *Result) String() string { return proto.CompactTextString(m) } func (*Result) ProtoMessage() {} func (*Result) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{11} + return fileDescriptor_operations_c3246c3d639e73ee, []int{11} } func (m *Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -526,7 +527,7 @@ type Step struct { func (m *Step) Reset() { *m = Step{} } func (*Step) ProtoMessage() {} func (*Step) Descriptor() ([]byte, []int) { - return fileDescriptor_operations_7efc7ec20a8915d9, []int{12} + return fileDescriptor_operations_c3246c3d639e73ee, []int{12} } func (m *Step) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -696,7 +697,17 @@ func (m *GetOperation) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintOperations(dAtA, i, uint64(len(m.Key))) i += copy(dAtA[i:], m.Key) } - dAtA[i] = 0x12 + if m.ForUpdate { + dAtA[i] = 0x10 + i++ + if m.ForUpdate { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + dAtA[i] = 0x1a i++ i = encodeVarintOperations(dAtA, i, uint64(m.Result.Size())) n5, err := m.Result.MarshalTo(dAtA[i:]) @@ -1248,6 +1259,9 @@ func (m *GetOperation) Size() (n int) { if l > 0 { n += 1 + l + sovOperations(uint64(l)) } + if m.ForUpdate { + n += 2 + } l = m.Result.Size() n += 1 + l + sovOperations(uint64(l)) return n @@ -1939,6 +1953,26 @@ func (m *GetOperation) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ForUpdate", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowOperations + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.ForUpdate = bool(v != 0) + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } @@ -3704,74 +3738,75 @@ var ( ) func init() { - proto.RegisterFile("kv/kvnemesis/operations.proto", fileDescriptor_operations_7efc7ec20a8915d9) + proto.RegisterFile("kv/kvnemesis/operations.proto", fileDescriptor_operations_c3246c3d639e73ee) } -var fileDescriptor_operations_7efc7ec20a8915d9 = []byte{ - // 1040 bytes of a gzipped FileDescriptorProto +var fileDescriptor_operations_c3246c3d639e73ee = []byte{ + // 1045 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0x41, 0x6f, 0x1b, 0x45, - 0x14, 0xf6, 0xda, 0x6b, 0xc7, 0x7e, 0x49, 0x5c, 0x33, 0x94, 0x76, 0x55, 0x29, 0xb6, 0xb1, 0x0a, - 0x8a, 0x5a, 0x69, 0x17, 0x05, 0x54, 0x48, 0xa8, 0x38, 0x38, 0xa9, 0xc0, 0x6a, 0x29, 0xd5, 0x24, - 0x41, 0x88, 0x03, 0xee, 0x78, 0x77, 0x62, 0xaf, 0xbc, 0xde, 0x59, 0xcd, 0x8e, 0x43, 0xf2, 0x13, - 0xb8, 0x71, 0x83, 0x63, 0xff, 0x01, 0x77, 0x7e, 0x41, 0xc4, 0xa9, 0xc7, 0x9e, 0x22, 0x70, 0xee, - 0x1c, 0xb8, 0xc1, 0x09, 0xcd, 0xcc, 0x6e, 0xbc, 0x16, 0xb1, 0x49, 0x13, 0xf5, 0xf6, 0xe6, 0xed, - 0xfb, 0xde, 0x7b, 0xf3, 0xcd, 0x7b, 0x6f, 0x1f, 0xac, 0x0d, 0x0f, 0x9d, 0xe1, 0x61, 0x48, 0x47, - 0x34, 0xf6, 0x63, 0x87, 0x45, 0x94, 0x13, 0xe1, 0xb3, 0x30, 0xb6, 0x23, 0xce, 0x04, 0x43, 0xb7, - 0x5c, 0xe6, 0x0e, 0x39, 0x23, 0xee, 0xc0, 0x1e, 0x1e, 0xda, 0xe7, 0x86, 0x77, 0xde, 0xa1, 0x9c, - 0x33, 0x1e, 0x47, 0x3d, 0x47, 0x0b, 0xda, 0xfc, 0xce, 0xcd, 0x3e, 0xeb, 0x33, 0x25, 0x3a, 0x52, - 0x4a, 0xb4, 0x6f, 0x29, 0x07, 0x51, 0xcf, 0x21, 0x91, 0x9f, 0xa8, 0x50, 0xaa, 0xf2, 0x88, 0x20, - 0x89, 0xce, 0x1a, 0x0b, 0x3f, 0x70, 0x06, 0x81, 0xeb, 0x08, 0x7f, 0x44, 0x63, 0x41, 0x46, 0x91, - 0xfe, 0xd2, 0xfa, 0xc1, 0x80, 0x6a, 0x9b, 0x08, 0x77, 0xf0, 0x55, 0x9a, 0x1f, 0xda, 0x84, 0x02, - 0x8b, 0x62, 0xcb, 0x68, 0x16, 0xd6, 0x97, 0x37, 0xde, 0xb5, 0x2f, 0x4e, 0xd3, 0x3e, 0xb7, 0x6f, - 0x9b, 0x27, 0xa7, 0x8d, 0x1c, 0x96, 0x18, 0xf4, 0x10, 0x4a, 0x9c, 0xc6, 0xe3, 0x40, 0x58, 0xf9, - 0xa6, 0xb1, 0xbe, 0xbc, 0x51, 0x9f, 0x87, 0xc6, 0xca, 0x2a, 0x81, 0x26, 0x98, 0xd6, 0x9f, 0x79, - 0x78, 0x7b, 0x3b, 0x60, 0xf1, 0x98, 0xd3, 0xbd, 0xa3, 0x70, 0x9a, 0x50, 0x13, 0x4a, 0xe2, 0x28, - 0xec, 0xfa, 0x9e, 0x65, 0x34, 0x8d, 0xf5, 0x4a, 0xbb, 0x32, 0x39, 0x6d, 0x14, 0xf7, 0x8e, 0xc2, - 0xce, 0x0e, 0x2e, 0x8a, 0xa3, 0xb0, 0xe3, 0xa5, 0x29, 0xe7, 0xaf, 0x90, 0xf2, 0x53, 0xb8, 0xe1, - 0xb2, 0xd1, 0xc8, 0x17, 0x5d, 0x3f, 0xec, 0xf6, 0x24, 0x13, 0x56, 0x41, 0xe5, 0xfe, 0xfe, 0x3c, - 0x37, 0xb3, 0x74, 0xe1, 0x55, 0x0d, 0xef, 0x84, 0x4a, 0x8f, 0xb6, 0xc0, 0x14, 0xc7, 0x11, 0xb5, - 0xcc, 0xa6, 0xb1, 0x5e, 0x9d, 0xef, 0x64, 0x7a, 0xcf, 0xbd, 0xe3, 0x88, 0x62, 0x85, 0xc9, 0xd0, - 0x57, 0x7c, 0x7d, 0xfa, 0xd0, 0x07, 0x50, 0x10, 0x47, 0xa1, 0x55, 0xfa, 0x0f, 0x34, 0x29, 0x08, - 0x7b, 0x8f, 0x93, 0x30, 0x26, 0xae, 0xca, 0x5a, 0x9a, 0xb6, 0xbe, 0x83, 0x95, 0xcf, 0xa9, 0x98, - 0x12, 0x5d, 0x83, 0xc2, 0x90, 0x1e, 0x2b, 0x96, 0x57, 0xb0, 0x14, 0xaf, 0xf9, 0xa0, 0x3f, 0x19, - 0xb0, 0xba, 0xeb, 0x92, 0x70, 0x51, 0x84, 0xdb, 0xb0, 0x44, 0x43, 0xaf, 0x2b, 0xb5, 0x79, 0xa5, - 0x2d, 0xd1, 0xd0, 0x7b, 0x4c, 0x8f, 0xd1, 0x1a, 0xc0, 0x01, 0xe3, 0xdd, 0x71, 0xe4, 0x11, 0x41, - 0xd5, 0x9b, 0x94, 0x71, 0xe5, 0x80, 0xf1, 0x7d, 0xa5, 0xc8, 0x64, 0x66, 0x5e, 0x21, 0x33, 0x01, - 0x2b, 0xcf, 0xc6, 0x0b, 0x6f, 0x7e, 0x13, 0x8a, 0x87, 0x24, 0x18, 0xd3, 0x24, 0x2b, 0x7d, 0xc8, - 0x44, 0x2d, 0x5c, 0x21, 0xea, 0x73, 0xa8, 0xee, 0x46, 0x81, 0xff, 0x06, 0x19, 0x7f, 0x0e, 0xd5, - 0x2f, 0x29, 0xef, 0xd3, 0x37, 0x17, 0xe1, 0x17, 0x03, 0x6e, 0x6f, 0x0f, 0x48, 0xd8, 0xa7, 0x98, - 0x46, 0x81, 0xef, 0x92, 0x78, 0x51, 0xac, 0x1d, 0x58, 0x72, 0x95, 0x71, 0xda, 0x9c, 0x77, 0x2f, - 0xa8, 0xcb, 0xc4, 0x91, 0x74, 0xa1, 0x3d, 0x27, 0x21, 0x53, 0xe8, 0x35, 0x59, 0xff, 0xd5, 0x80, - 0x5b, 0xaa, 0xf4, 0x0f, 0x28, 0x7f, 0x42, 0x49, 0xbc, 0x90, 0x9c, 0x67, 0x50, 0x12, 0x84, 0xf7, - 0xa9, 0x26, 0xa7, 0xd8, 0xfe, 0xe4, 0x9f, 0xd3, 0xc6, 0x47, 0x7d, 0x5f, 0x0c, 0xc6, 0x3d, 0xdb, - 0x65, 0x23, 0xe7, 0x3c, 0xb0, 0xd7, 0x9b, 0xca, 0x4e, 0x34, 0xec, 0x3b, 0xe9, 0x6d, 0x76, 0x05, - 0xe3, 0xb4, 0xb3, 0x83, 0x13, 0x3f, 0xd7, 0x4c, 0xfe, 0x6f, 0x13, 0x2a, 0xd3, 0x7c, 0x1f, 0x42, - 0x51, 0x8f, 0x28, 0xe3, 0xb5, 0x46, 0x94, 0x06, 0xa1, 0x27, 0xb0, 0xec, 0xea, 0xb1, 0xd3, 0x95, - 0x83, 0x42, 0xbf, 0xfe, 0xfd, 0xff, 0x9f, 0x50, 0x53, 0x47, 0xe0, 0x9e, 0x2b, 0xd1, 0x03, 0x28, - 0x48, 0x9a, 0x96, 0x94, 0x97, 0xbb, 0xf3, 0xbc, 0x64, 0xe7, 0x0b, 0x96, 0x00, 0x89, 0x8b, 0xc6, - 0xc2, 0x2a, 0x2f, 0xc6, 0x65, 0xbb, 0x13, 0x4b, 0x00, 0xda, 0x04, 0x33, 0x76, 0x49, 0x68, 0x55, - 0x14, 0xf0, 0xbd, 0x79, 0xc0, 0x99, 0x79, 0x83, 0x15, 0x44, 0xd2, 0x16, 0xcb, 0xbe, 0xb3, 0x60, - 0x31, 0x6d, 0xb3, 0xcd, 0x89, 0x35, 0x48, 0xa2, 0x47, 0xb2, 0xa7, 0xac, 0xe5, 0xc5, 0xe8, 0xd9, - 0xc6, 0xc3, 0x1a, 0x84, 0xbe, 0x81, 0x1b, 0xba, 0x8c, 0xbb, 0x3c, 0xe9, 0x17, 0x6b, 0x45, 0xf9, - 0x71, 0xe6, 0x12, 0x7f, 0x71, 0x77, 0xe1, 0xaa, 0x3b, 0xf3, 0x01, 0xed, 0x43, 0x55, 0x24, 0x65, - 0xdd, 0x0d, 0x64, 0x5d, 0x5b, 0xab, 0xca, 0xb1, 0x3d, 0xcf, 0xf1, 0xc5, 0x4d, 0x80, 0x57, 0x45, - 0x56, 0xbf, 0x55, 0xfe, 0xf9, 0x45, 0x23, 0x77, 0xf2, 0xa2, 0x61, 0xb4, 0x36, 0xa0, 0xfc, 0x98, - 0x1e, 0x7f, 0xad, 0x06, 0xdf, 0x25, 0x07, 0x64, 0xeb, 0x37, 0x03, 0x4a, 0xba, 0x90, 0xd1, 0x83, - 0xe4, 0x4f, 0x68, 0xa8, 0x3f, 0x61, 0x6b, 0x71, 0xd9, 0x67, 0xfe, 0x82, 0x1b, 0x50, 0xa0, 0x9c, - 0x27, 0xe5, 0xd9, 0xcc, 0xc0, 0xd2, 0xc5, 0xc8, 0x7e, 0x14, 0xba, 0xcc, 0xa3, 0xde, 0x23, 0x79, - 0xc6, 0xd2, 0x78, 0x9a, 0x4c, 0x21, 0x3b, 0xad, 0x3f, 0x83, 0x92, 0x12, 0x62, 0xcb, 0x54, 0xc3, - 0xa7, 0x39, 0x2f, 0x87, 0xf4, 0x9a, 0x69, 0xf3, 0x69, 0x54, 0xeb, 0x2f, 0x03, 0xcc, 0x5d, 0x41, - 0x23, 0xf4, 0x31, 0xe4, 0x59, 0x94, 0x34, 0xdd, 0xa5, 0xd7, 0x8b, 0x3c, 0x8b, 0xd0, 0xa7, 0x50, - 0xea, 0xd1, 0x03, 0xc6, 0x69, 0x72, 0x9d, 0xb5, 0x0c, 0x58, 0xee, 0x64, 0xf6, 0x20, 0x70, 0xed, - 0xbd, 0x74, 0x27, 0x4b, 0xc3, 0x6b, 0x08, 0xda, 0x84, 0x22, 0x39, 0x10, 0x94, 0x27, 0x83, 0xe3, - 0x52, 0x58, 0x8d, 0x40, 0x6b, 0x50, 0xf4, 0x7a, 0x72, 0x63, 0x32, 0xd5, 0x14, 0x2b, 0x4f, 0x4e, - 0x1b, 0xe6, 0x4e, 0xbb, 0xb3, 0x83, 0x4d, 0xaf, 0xd7, 0xf1, 0x24, 0x5d, 0x82, 0x13, 0x97, 0xaa, - 0x3d, 0xa3, 0x82, 0xf5, 0x61, 0xcb, 0x94, 0x2f, 0x7f, 0xef, 0x1e, 0x54, 0x67, 0x97, 0x13, 0x04, - 0x50, 0xda, 0x56, 0x3b, 0x4e, 0x2d, 0x87, 0x56, 0xa0, 0x8c, 0x59, 0x10, 0xf4, 0x88, 0x3b, 0xac, - 0x19, 0xf7, 0xbe, 0x00, 0x98, 0x3e, 0x1f, 0x5a, 0x86, 0xa5, 0xfd, 0x70, 0x18, 0xb2, 0xef, 0xc3, - 0x5a, 0x4e, 0x1e, 0x9e, 0x32, 0xf5, 0x42, 0x35, 0x03, 0x55, 0xa0, 0xa8, 0xc5, 0xbc, 0x14, 0x15, - 0xd5, 0xb5, 0x82, 0xf4, 0xab, 0xc4, 0xb8, 0x66, 0xb6, 0xef, 0x9f, 0xfc, 0x51, 0xcf, 0x9d, 0x4c, - 0xea, 0xc6, 0xcb, 0x49, 0xdd, 0x78, 0x35, 0xa9, 0x1b, 0xbf, 0x4f, 0xea, 0xc6, 0x8f, 0x67, 0xf5, - 0xdc, 0xcb, 0xb3, 0x7a, 0xee, 0xd5, 0x59, 0x3d, 0xf7, 0x6d, 0xe5, 0x9c, 0xe9, 0x5e, 0x49, 0xed, - 0xae, 0x1f, 0xfe, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x62, 0x08, 0xb0, 0xfa, 0x62, 0x0b, 0x00, 0x00, + 0x14, 0xf6, 0x7a, 0xd7, 0x8e, 0xfd, 0x92, 0xb8, 0x66, 0x28, 0xed, 0xaa, 0x52, 0x6c, 0x63, 0x15, + 0x14, 0xb5, 0xd2, 0x2e, 0x0a, 0xa8, 0x90, 0x50, 0x71, 0x70, 0x52, 0x81, 0xd5, 0x52, 0xaa, 0x49, + 0x82, 0x10, 0x17, 0x77, 0xbc, 0x3b, 0xb1, 0x57, 0x5e, 0xef, 0xac, 0x66, 0xc7, 0x21, 0x39, 0xf0, + 0x03, 0xb8, 0x71, 0x83, 0x63, 0xff, 0x01, 0x77, 0x7e, 0x41, 0xc4, 0xa9, 0xc7, 0x9e, 0x22, 0x70, + 0xee, 0x1c, 0xb8, 0xc1, 0x09, 0xcd, 0xcc, 0x6e, 0x6c, 0xab, 0xb1, 0x49, 0x13, 0x71, 0x7b, 0xf3, + 0x76, 0xbe, 0x37, 0xdf, 0x7c, 0xf3, 0xde, 0xdb, 0x07, 0x6b, 0x83, 0x43, 0x77, 0x70, 0x18, 0xd1, + 0x21, 0x4d, 0x82, 0xc4, 0x65, 0x31, 0xe5, 0x44, 0x04, 0x2c, 0x4a, 0x9c, 0x98, 0x33, 0xc1, 0xd0, + 0x2d, 0x8f, 0x79, 0x03, 0xce, 0x88, 0xd7, 0x77, 0x06, 0x87, 0xce, 0xf9, 0xc6, 0x3b, 0xef, 0x50, + 0xce, 0x19, 0x4f, 0xe2, 0xae, 0xab, 0x0d, 0xbd, 0xfd, 0xce, 0xcd, 0x1e, 0xeb, 0x31, 0x65, 0xba, + 0xd2, 0x4a, 0xbd, 0x6f, 0xa9, 0x00, 0x71, 0xd7, 0x25, 0x71, 0x90, 0xba, 0x50, 0xe6, 0xf2, 0x89, + 0x20, 0xa9, 0xcf, 0x1e, 0x89, 0x20, 0x74, 0xfb, 0xa1, 0xe7, 0x8a, 0x60, 0x48, 0x13, 0x41, 0x86, + 0xb1, 0xfe, 0xd2, 0xfc, 0xc1, 0x80, 0x4a, 0x8b, 0x08, 0xaf, 0xff, 0x55, 0xc6, 0x0f, 0x6d, 0x82, + 0xc9, 0xe2, 0xc4, 0x36, 0x1a, 0xe6, 0xfa, 0xf2, 0xc6, 0xbb, 0xce, 0xc5, 0x34, 0x9d, 0xf3, 0xfd, + 0x2d, 0xeb, 0xe4, 0xb4, 0x9e, 0xc3, 0x12, 0x83, 0x1e, 0x42, 0x91, 0xd3, 0x64, 0x14, 0x0a, 0x3b, + 0xdf, 0x30, 0xd6, 0x97, 0x37, 0x6a, 0xf3, 0xd0, 0x58, 0xed, 0x4a, 0xa1, 0x29, 0xa6, 0xf9, 0x67, + 0x1e, 0xde, 0xde, 0x0e, 0x59, 0x32, 0xe2, 0x74, 0xef, 0x28, 0x9a, 0x10, 0x6a, 0x40, 0x51, 0x1c, + 0x45, 0x9d, 0xc0, 0xb7, 0x8d, 0x86, 0xb1, 0x5e, 0x6e, 0x95, 0xc7, 0xa7, 0xf5, 0xc2, 0xde, 0x51, + 0xd4, 0xde, 0xc1, 0x05, 0x71, 0x14, 0xb5, 0xfd, 0x8c, 0x72, 0xfe, 0x0a, 0x94, 0x9f, 0xc2, 0x0d, + 0x8f, 0x0d, 0x87, 0x81, 0xe8, 0x04, 0x51, 0xa7, 0x2b, 0x95, 0xb0, 0x4d, 0xc5, 0xfd, 0xfd, 0x79, + 0x61, 0x66, 0xe5, 0xc2, 0xab, 0x1a, 0xde, 0x8e, 0x94, 0x1f, 0x6d, 0x81, 0x25, 0x8e, 0x63, 0x6a, + 0x5b, 0x0d, 0x63, 0xbd, 0x32, 0x3f, 0xc8, 0xe4, 0x9e, 0x7b, 0xc7, 0x31, 0xc5, 0x0a, 0x33, 0x25, + 0x5f, 0xe1, 0xcd, 0xe5, 0x43, 0x1f, 0x80, 0x29, 0x8e, 0x22, 0xbb, 0xf8, 0x1a, 0x34, 0x4d, 0x08, + 0x67, 0x8f, 0x93, 0x28, 0x21, 0x9e, 0x62, 0x2d, 0xb7, 0x36, 0xbf, 0x87, 0x95, 0xcf, 0xa9, 0x98, + 0x08, 0x5d, 0x05, 0x73, 0x40, 0x8f, 0x95, 0xca, 0x2b, 0x58, 0x9a, 0x68, 0x0d, 0xe0, 0x80, 0xf1, + 0xce, 0x28, 0xf6, 0x89, 0xa0, 0xea, 0x51, 0x4b, 0xb8, 0x7c, 0xc0, 0xf8, 0xbe, 0x72, 0x4c, 0x11, + 0x36, 0xaf, 0xf0, 0xde, 0x3f, 0x19, 0xb0, 0xba, 0xeb, 0x91, 0x68, 0x11, 0x81, 0xdb, 0xb0, 0x44, + 0x23, 0xbf, 0x23, 0xbd, 0x79, 0xe5, 0x2d, 0xd2, 0xc8, 0x7f, 0xfc, 0x1a, 0x33, 0x73, 0x3e, 0x33, + 0xeb, 0x0a, 0xcc, 0x04, 0xac, 0x3c, 0x1b, 0x2d, 0x14, 0xe6, 0x26, 0x14, 0x0e, 0x49, 0x38, 0xa2, + 0x29, 0x2b, 0xbd, 0xb8, 0xa6, 0x1e, 0xcf, 0xa1, 0xb2, 0x1b, 0x87, 0xc1, 0xc2, 0x73, 0xaf, 0x57, + 0x61, 0xcf, 0xa1, 0xf2, 0x25, 0xe5, 0x3d, 0xfa, 0xff, 0x9d, 0xf0, 0x8b, 0x01, 0xb7, 0xb7, 0xfb, + 0x24, 0xea, 0x51, 0x4c, 0xe3, 0x30, 0xf0, 0x48, 0xb2, 0xe8, 0xac, 0x1d, 0x58, 0xf2, 0xd4, 0xe6, + 0xac, 0x76, 0xef, 0x5e, 0x90, 0xb6, 0x69, 0x20, 0x19, 0x42, 0x47, 0x4e, 0x8f, 0xcc, 0xa0, 0xd7, + 0x54, 0xfd, 0x57, 0x03, 0x6e, 0xa9, 0xca, 0x38, 0xa0, 0xfc, 0x09, 0x25, 0xc9, 0x42, 0x71, 0x9e, + 0x41, 0x51, 0x10, 0xde, 0xa3, 0x5a, 0x9c, 0x42, 0xeb, 0x93, 0x7f, 0x4e, 0xeb, 0x1f, 0xf5, 0x02, + 0xd1, 0x1f, 0x75, 0x1d, 0x8f, 0x0d, 0xdd, 0xf3, 0x83, 0xfd, 0xee, 0xc4, 0x76, 0xe3, 0x41, 0xcf, + 0xcd, 0x6e, 0xb3, 0x2b, 0x18, 0xa7, 0xed, 0x1d, 0x9c, 0xc6, 0xb9, 0x26, 0xf9, 0xbf, 0x2d, 0x28, + 0x4f, 0xf8, 0x3e, 0x84, 0x82, 0xee, 0x60, 0xc6, 0x1b, 0x75, 0x30, 0x0d, 0x42, 0x4f, 0x60, 0xd9, + 0xd3, 0x5d, 0xa9, 0x23, 0xfb, 0x88, 0x7e, 0xfd, 0xfb, 0xff, 0xdd, 0xc0, 0x26, 0x81, 0xc0, 0x3b, + 0x77, 0xa2, 0x07, 0x60, 0x4a, 0x99, 0x96, 0x54, 0x94, 0xbb, 0xf3, 0xa2, 0x4c, 0xb7, 0x1f, 0x2c, + 0x01, 0x12, 0x17, 0x8f, 0x84, 0x5d, 0x5a, 0x8c, 0x9b, 0xae, 0x4e, 0x2c, 0x01, 0x68, 0x13, 0xac, + 0xc4, 0x23, 0x91, 0x5d, 0x56, 0xc0, 0xf7, 0xe6, 0x01, 0x67, 0xfa, 0x0d, 0x56, 0x10, 0x29, 0x5b, + 0x22, 0xeb, 0xce, 0x86, 0xc5, 0xb2, 0xcd, 0x16, 0x27, 0xd6, 0x20, 0x89, 0x1e, 0xca, 0x9a, 0xb2, + 0x97, 0x17, 0xa3, 0x67, 0x0b, 0x0f, 0x6b, 0x10, 0xfa, 0x06, 0x6e, 0xe8, 0x34, 0xee, 0xf0, 0xb4, + 0x5e, 0xec, 0x15, 0x15, 0xc7, 0x9d, 0x2b, 0xfc, 0xc5, 0xd5, 0x85, 0x2b, 0xde, 0xcc, 0x07, 0xb4, + 0x0f, 0x15, 0x91, 0xa6, 0x75, 0x27, 0x94, 0x79, 0x6d, 0xaf, 0xaa, 0xc0, 0xce, 0xbc, 0xc0, 0x17, + 0x17, 0x01, 0x5e, 0x15, 0xd3, 0xfe, 0xad, 0xd2, 0xcf, 0x2f, 0xea, 0xb9, 0x93, 0x17, 0x75, 0xa3, + 0xb9, 0x01, 0xa5, 0xc7, 0xf4, 0xf8, 0x6b, 0xd5, 0xf8, 0x2e, 0xd9, 0x20, 0x9b, 0xbf, 0x19, 0x50, + 0xd4, 0x89, 0x8c, 0x1e, 0xa4, 0x3f, 0x4a, 0x43, 0xfd, 0x28, 0x9b, 0x8b, 0xd3, 0x7e, 0xea, 0x27, + 0xb9, 0x01, 0x26, 0xe5, 0x3c, 0x4d, 0xcf, 0xc6, 0x14, 0x2c, 0x9b, 0x9b, 0x9c, 0x47, 0x91, 0xc7, + 0x7c, 0xea, 0x3f, 0x92, 0x6b, 0x2c, 0x37, 0x4f, 0xc8, 0x98, 0xd3, 0xdd, 0xfa, 0x33, 0x28, 0x2a, + 0x23, 0xb1, 0x2d, 0xd5, 0x7c, 0x1a, 0xf3, 0x38, 0x64, 0xd7, 0xcc, 0x8a, 0x4f, 0xa3, 0x9a, 0x7f, + 0x19, 0x60, 0xed, 0x0a, 0x1a, 0xa3, 0x8f, 0x21, 0xcf, 0xe2, 0xb4, 0xe8, 0x2e, 0x3d, 0x7d, 0xe4, + 0x59, 0x8c, 0x3e, 0x85, 0x62, 0x97, 0x1e, 0x30, 0x4e, 0xd3, 0xeb, 0xac, 0x4d, 0x81, 0xe5, 0xc8, + 0xe6, 0xf4, 0x43, 0xcf, 0xd9, 0xcb, 0x46, 0xb6, 0xec, 0x78, 0x0d, 0x41, 0x9b, 0x50, 0x20, 0x07, + 0x82, 0xf2, 0xb4, 0x71, 0x5c, 0x0a, 0xab, 0x11, 0x68, 0x0d, 0x0a, 0x7e, 0x57, 0x0e, 0x54, 0x96, + 0xea, 0x62, 0xa5, 0xf1, 0x69, 0xdd, 0xda, 0x69, 0xb5, 0x77, 0xb0, 0xe5, 0x77, 0xdb, 0xbe, 0x94, + 0x4b, 0x70, 0xe2, 0x51, 0x35, 0x86, 0x94, 0xb1, 0x5e, 0x6c, 0x59, 0xf2, 0xe5, 0xef, 0xdd, 0x83, + 0xca, 0xec, 0xec, 0x82, 0x00, 0x8a, 0xdb, 0x6a, 0x04, 0xaa, 0xe6, 0xd0, 0x0a, 0x94, 0x30, 0x0b, + 0xc3, 0x2e, 0xf1, 0x06, 0x55, 0xe3, 0xde, 0x17, 0x00, 0x93, 0xe7, 0x43, 0xcb, 0xb0, 0xb4, 0x1f, + 0x0d, 0x22, 0xf6, 0x5d, 0x54, 0xcd, 0xc9, 0xc5, 0x53, 0xa6, 0x5e, 0xa8, 0x6a, 0xa0, 0x32, 0x14, + 0xb4, 0x99, 0x97, 0xa6, 0x92, 0xba, 0x6a, 0xca, 0xb8, 0xca, 0x4c, 0xaa, 0x56, 0xeb, 0xfe, 0xc9, + 0x1f, 0xb5, 0xdc, 0xc9, 0xb8, 0x66, 0xbc, 0x1c, 0xd7, 0x8c, 0x57, 0xe3, 0x9a, 0xf1, 0xfb, 0xb8, + 0x66, 0xfc, 0x78, 0x56, 0xcb, 0xbd, 0x3c, 0xab, 0xe5, 0x5e, 0x9d, 0xd5, 0x72, 0xdf, 0x96, 0xcf, + 0x95, 0xee, 0x16, 0xd5, 0x68, 0xfb, 0xe1, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xca, 0xa2, 0x33, + 0x28, 0x81, 0x0b, 0x00, 0x00, } diff --git a/pkg/kv/kvnemesis/operations.proto b/pkg/kv/kvnemesis/operations.proto index 0e843c39bb21..328f775d901a 100644 --- a/pkg/kv/kvnemesis/operations.proto +++ b/pkg/kv/kvnemesis/operations.proto @@ -40,7 +40,8 @@ message ClosureTxnOperation { message GetOperation { bytes key = 1; - Result result = 2 [(gogoproto.nullable) = false]; + bool for_update = 2; + Result result = 3 [(gogoproto.nullable) = false]; } message ScanOperation {