From 4a8e8dcc27f427f5009319122d1829ce2db5880c Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 6 Jan 2020 18:14:05 -0500 Subject: [PATCH] [DNM] sql/storage/workload: prototype SELECT FOR UPDATE This commit hacks in upgrade locking into ScanRequests and uses it in YCSB. --- c-deps/libroach/include/libroach.h | 3 +- c-deps/libroach/mvcc.cc | 8 +- c-deps/libroach/mvcc.h | 17 +- c-deps/libroach/protos/roachpb/api.pb.cc | 41 +- c-deps/libroach/protos/roachpb/api.pb.h | 21 + pkg/kv/txn_interceptor_heartbeater.go | 3 + pkg/kv/txn_interceptor_pipeliner.go | 4 + pkg/roachpb/api.pb.go | 1116 +++++++++--------- pkg/roachpb/api.proto | 2 + pkg/roachpb/batch.go | 15 +- pkg/sql/distsql_physical_planner.go | 4 + pkg/sql/row/kv_batch_fetcher.go | 4 +- pkg/storage/batcheval/cmd_end_transaction.go | 2 +- pkg/storage/batcheval/cmd_scan.go | 63 +- pkg/storage/concurrency/lock_table.go | 47 +- pkg/storage/engine/mvcc.go | 30 +- pkg/storage/engine/rocksdb.go | 43 + pkg/workload/tpcc/new_order.go | 44 +- pkg/workload/ycsb/ycsb.go | 18 +- 19 files changed, 868 insertions(+), 617 deletions(-) diff --git a/c-deps/libroach/include/libroach.h b/c-deps/libroach/include/libroach.h index ad916cac3893..4a72789bd5c2 100644 --- a/c-deps/libroach/include/libroach.h +++ b/c-deps/libroach/include/libroach.h @@ -359,6 +359,7 @@ typedef struct { DBChunkedBuffer data; DBSlice intents; DBTimestamp uncertainty_timestamp; + DBTimestamp write_too_old; DBSlice resume_key; } DBScanResults; @@ -366,7 +367,7 @@ DBScanResults MVCCGet(DBIterator* iter, DBSlice key, DBTimestamp timestamp, DBTx bool inconsistent, bool tombstones); DBScanResults MVCCScan(DBIterator* iter, DBSlice start, DBSlice end, DBTimestamp timestamp, int64_t max_keys, DBTxn txn, bool inconsistent, bool reverse, - bool tombstones); + bool tombstones, bool write_too_old); // DBStatsResult contains various runtime stats for RocksDB. typedef struct { diff --git a/c-deps/libroach/mvcc.cc b/c-deps/libroach/mvcc.cc index f96cd5d3983b..492571c85f8d 100644 --- a/c-deps/libroach/mvcc.cc +++ b/c-deps/libroach/mvcc.cc @@ -276,19 +276,19 @@ DBScanResults MVCCGet(DBIterator* iter, DBSlice key, DBTimestamp timestamp, DBTx const DBSlice end = {0, 0}; ScopedStats scoped_iter(iter); mvccForwardScanner scanner(iter, key, end, timestamp, 1 /* max_keys */, txn, inconsistent, - tombstones); + tombstones, false /* write_too_old */); return scanner.get(); } DBScanResults MVCCScan(DBIterator* iter, DBSlice start, DBSlice end, DBTimestamp timestamp, int64_t max_keys, DBTxn txn, bool inconsistent, bool reverse, - bool tombstones) { + bool tombstones, bool write_too_old) { ScopedStats scoped_iter(iter); if (reverse) { - mvccReverseScanner scanner(iter, end, start, timestamp, max_keys, txn, inconsistent, tombstones); + mvccReverseScanner scanner(iter, end, start, timestamp, max_keys, txn, inconsistent, tombstones, write_too_old); return scanner.scan(); } else { - mvccForwardScanner scanner(iter, start, end, timestamp, max_keys, txn, inconsistent, tombstones); + mvccForwardScanner scanner(iter, start, end, timestamp, max_keys, txn, inconsistent, tombstones, write_too_old); return scanner.scan(); } } diff --git a/c-deps/libroach/mvcc.h b/c-deps/libroach/mvcc.h index ba634dffbf04..6d136fa65152 100644 --- a/c-deps/libroach/mvcc.h +++ b/c-deps/libroach/mvcc.h @@ -48,7 +48,7 @@ static const int kMaxItersBeforeSeek = 10; template class mvccScanner { public: mvccScanner(DBIterator* iter, DBSlice start, DBSlice end, DBTimestamp timestamp, int64_t max_keys, - DBTxn txn, bool inconsistent, bool tombstones) + DBTxn txn, bool inconsistent, bool tombstones, bool write_too_old) : iter_(iter), iter_rep_(iter->rep.get()), start_key_(ToSlice(start)), @@ -62,6 +62,7 @@ template class mvccScanner { txn_ignored_seqnums_(txn.ignored_seqnums), inconsistent_(inconsistent), tombstones_(tombstones), + write_too_old_(write_too_old), check_uncertainty_(timestamp < txn.max_timestamp), kvs_(new chunkedBuffer), intents_(new rocksdb::WriteBatch), @@ -261,6 +262,13 @@ template class mvccScanner { return false; } + bool writeTooOld(DBTimestamp ts) { + results_.write_too_old = ts; + kvs_->Clear(); + intents_->Clear(); + return false; + } + bool setStatus(const DBStatus& status) { results_.status = status; return false; @@ -276,6 +284,10 @@ template class mvccScanner { return addAndAdvance(cur_value_); } + if (write_too_old_) { + return writeTooOld(cur_timestamp_); + } + if (check_uncertainty_) { // 2. Our txn's read timestamp is less than the max timestamp // seen by the txn. We need to check for clock uncertainty @@ -326,7 +338,7 @@ template class mvccScanner { // Intents for other transactions are visible at or below: // max(txn.max_timestamp, read_timestamp) const DBTimestamp max_visible_timestamp = check_uncertainty_ ? txn_max_timestamp_ : timestamp_; - if (max_visible_timestamp < meta_timestamp && !own_intent) { + if (max_visible_timestamp < meta_timestamp && !own_intent && !write_too_old_) { // 5. The key contains an intent, but we're reading before the // intent. Seek to the desired version. Note that if we own the // intent (i.e. we're reading transactionally) we want to read @@ -729,6 +741,7 @@ template class mvccScanner { const DBIgnoredSeqNums txn_ignored_seqnums_; const bool inconsistent_; const bool tombstones_; + const bool write_too_old_; const bool check_uncertainty_; DBScanResults results_; std::unique_ptr kvs_; diff --git a/c-deps/libroach/protos/roachpb/api.pb.cc b/c-deps/libroach/protos/roachpb/api.pb.cc index b8ff326ba127..57ebe2735241 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.cc +++ b/c-deps/libroach/protos/roachpb/api.pb.cc @@ -7891,6 +7891,7 @@ void ScanRequest::InitAsDefaultInstance() { #if !defined(_MSC_VER) || _MSC_VER >= 1900 const int ScanRequest::kHeaderFieldNumber; const int ScanRequest::kScanFormatFieldNumber; +const int ScanRequest::kSelectForUpdateFieldNumber; #endif // !defined(_MSC_VER) || _MSC_VER >= 1900 ScanRequest::ScanRequest() @@ -7909,14 +7910,16 @@ ScanRequest::ScanRequest(const ScanRequest& from) } else { header_ = NULL; } - scan_format_ = from.scan_format_; + ::memcpy(&scan_format_, &from.scan_format_, + static_cast(reinterpret_cast(&select_for_update_) - + reinterpret_cast(&scan_format_)) + sizeof(select_for_update_)); // @@protoc_insertion_point(copy_constructor:cockroach.roachpb.ScanRequest) } void ScanRequest::SharedCtor() { ::memset(&header_, 0, static_cast( - reinterpret_cast(&scan_format_) - - reinterpret_cast(&header_)) + sizeof(scan_format_)); + reinterpret_cast(&select_for_update_) - + reinterpret_cast(&header_)) + sizeof(select_for_update_)); } ScanRequest::~ScanRequest() { @@ -7947,7 +7950,9 @@ void ScanRequest::Clear() { delete header_; } header_ = NULL; - scan_format_ = 0; + ::memset(&scan_format_, 0, static_cast( + reinterpret_cast(&select_for_update_) - + reinterpret_cast(&scan_format_)) + sizeof(select_for_update_)); _internal_metadata_.Clear(); } @@ -7993,6 +7998,20 @@ bool ScanRequest::MergePartialFromCodedStream( break; } + // bool select_for_update = 5; + case 5: { + if (static_cast< ::google::protobuf::uint8>(tag) == + static_cast< ::google::protobuf::uint8>(40u /* 40 & 0xFF */)) { + + DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< + bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>( + input, &select_for_update_))); + } else { + goto handle_unusual; + } + break; + } + default: { handle_unusual: if (tag == 0) { @@ -8030,6 +8049,11 @@ void ScanRequest::SerializeWithCachedSizes( 4, this->scan_format(), output); } + // bool select_for_update = 5; + if (this->select_for_update() != 0) { + ::google::protobuf::internal::WireFormatLite::WriteBool(5, this->select_for_update(), output); + } + output->WriteRaw((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).data(), static_cast((::google::protobuf::internal::GetProto3PreserveUnknownsDefault() ? _internal_metadata_.unknown_fields() : _internal_metadata_.default_instance()).size())); // @@protoc_insertion_point(serialize_end:cockroach.roachpb.ScanRequest) @@ -8053,6 +8077,11 @@ size_t ScanRequest::ByteSizeLong() const { ::google::protobuf::internal::WireFormatLite::EnumSize(this->scan_format()); } + // bool select_for_update = 5; + if (this->select_for_update() != 0) { + total_size += 1 + 1; + } + int cached_size = ::google::protobuf::internal::ToCachedSize(total_size); SetCachedSize(cached_size); return total_size; @@ -8076,6 +8105,9 @@ void ScanRequest::MergeFrom(const ScanRequest& from) { if (from.scan_format() != 0) { set_scan_format(from.scan_format()); } + if (from.select_for_update() != 0) { + set_select_for_update(from.select_for_update()); + } } void ScanRequest::CopyFrom(const ScanRequest& from) { @@ -8097,6 +8129,7 @@ void ScanRequest::InternalSwap(ScanRequest* other) { using std::swap; swap(header_, other->header_); swap(scan_format_, other->scan_format_); + swap(select_for_update_, other->select_for_update_); _internal_metadata_.Swap(&other->_internal_metadata_); } diff --git a/c-deps/libroach/protos/roachpb/api.pb.h b/c-deps/libroach/protos/roachpb/api.pb.h index 6aaccf69fb2c..a1162ab97e05 100644 --- a/c-deps/libroach/protos/roachpb/api.pb.h +++ b/c-deps/libroach/protos/roachpb/api.pb.h @@ -3233,12 +3233,19 @@ class ScanRequest : public ::google::protobuf::MessageLite /* @@protoc_insertion ::cockroach::roachpb::ScanFormat scan_format() const; void set_scan_format(::cockroach::roachpb::ScanFormat value); + // bool select_for_update = 5; + void clear_select_for_update(); + static const int kSelectForUpdateFieldNumber = 5; + bool select_for_update() const; + void set_select_for_update(bool value); + // @@protoc_insertion_point(class_scope:cockroach.roachpb.ScanRequest) private: ::google::protobuf::internal::InternalMetadataWithArenaLite _internal_metadata_; ::cockroach::roachpb::RequestHeader* header_; int scan_format_; + bool select_for_update_; mutable ::google::protobuf::internal::CachedSize _cached_size_; friend struct ::protobuf_roachpb_2fapi_2eproto::TableStruct; }; @@ -18622,6 +18629,20 @@ inline void ScanRequest::set_scan_format(::cockroach::roachpb::ScanFormat value) // @@protoc_insertion_point(field_set:cockroach.roachpb.ScanRequest.scan_format) } +// bool select_for_update = 5; +inline void ScanRequest::clear_select_for_update() { + select_for_update_ = false; +} +inline bool ScanRequest::select_for_update() const { + // @@protoc_insertion_point(field_get:cockroach.roachpb.ScanRequest.select_for_update) + return select_for_update_; +} +inline void ScanRequest::set_select_for_update(bool value) { + + select_for_update_ = value; + // @@protoc_insertion_point(field_set:cockroach.roachpb.ScanRequest.select_for_update) +} + // ------------------------------------------------------------------- // ScanResponse diff --git a/pkg/kv/txn_interceptor_heartbeater.go b/pkg/kv/txn_interceptor_heartbeater.go index 739bd61b4735..b6cfc1e5a1fa 100644 --- a/pkg/kv/txn_interceptor_heartbeater.go +++ b/pkg/kv/txn_interceptor_heartbeater.go @@ -407,6 +407,9 @@ func firstWriteIndex(ba *roachpb.BatchRequest) (int, *roachpb.Error) { if roachpb.IsTransactionWrite(args) { return i, nil } + if t, ok := args.(*roachpb.ScanRequest); ok && t.SelectForUpdate { + return i, nil + } } return -1, nil } diff --git a/pkg/kv/txn_interceptor_pipeliner.go b/pkg/kv/txn_interceptor_pipeliner.go index 2c4db968214b..64e859109d9a 100644 --- a/pkg/kv/txn_interceptor_pipeliner.go +++ b/pkg/kv/txn_interceptor_pipeliner.go @@ -503,6 +503,10 @@ func (tp *txnPipeliner) updateWriteTracking( tp.footprint.insert(sp) } } + } else if scan, ok := req.(*roachpb.ScanRequest); ok && scan.SelectForUpdate { + if sp, ok := roachpb.ActualSpan(req, resp); ok { + tp.footprint.insert(sp) + } } } } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index dd5504eefa55..256d09fc775e 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -71,7 +71,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -99,7 +99,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{1} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{1} } type ChecksumMode int32 @@ -146,7 +146,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{2} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -177,7 +177,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{3} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{3} } type ExternalStorageProvider int32 @@ -215,7 +215,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{4} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{4} } type MVCCFilter int32 @@ -238,7 +238,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{5} + return fileDescriptor_api_b867aaa08bf9d0c5, []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_d2c1ab7675455c27, []int{2, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{2, 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_d2c1ab7675455c27, []int{26, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{26, 0} } // RangeInfo describes a range which executed a request. It contains @@ -326,7 +326,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -369,7 +369,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_d2c1ab7675455c27, []int{1} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -431,7 +431,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_d2c1ab7675455c27, []int{2} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -465,7 +465,7 @@ 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_d2c1ab7675455c27, []int{3} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -503,7 +503,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_d2c1ab7675455c27, []int{4} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -546,7 +546,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_d2c1ab7675455c27, []int{5} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -580,7 +580,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_d2c1ab7675455c27, []int{6} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -634,7 +634,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_d2c1ab7675455c27, []int{7} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -669,7 +669,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_d2c1ab7675455c27, []int{8} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -715,7 +715,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_d2c1ab7675455c27, []int{9} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -749,7 +749,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_d2c1ab7675455c27, []int{10} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -789,7 +789,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_d2c1ab7675455c27, []int{11} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -826,7 +826,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_d2c1ab7675455c27, []int{12} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -860,7 +860,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_d2c1ab7675455c27, []int{13} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -894,7 +894,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_d2c1ab7675455c27, []int{14} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -946,7 +946,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_d2c1ab7675455c27, []int{15} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -983,7 +983,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_d2c1ab7675455c27, []int{16} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1030,7 +1030,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_d2c1ab7675455c27, []int{17} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1064,7 +1064,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_d2c1ab7675455c27, []int{18} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1105,7 +1105,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_d2c1ab7675455c27, []int{19} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{19} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1139,7 +1139,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_d2c1ab7675455c27, []int{20} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{20} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1172,14 +1172,15 @@ type ScanRequest struct { // The desired format for the response. If set to BATCH_RESPONSE, the server // will set the batch_responses field in the ScanResponse instead of the rows // field. - ScanFormat ScanFormat `protobuf:"varint,4,opt,name=scan_format,json=scanFormat,proto3,enum=cockroach.roachpb.ScanFormat" json:"scan_format,omitempty"` + ScanFormat ScanFormat `protobuf:"varint,4,opt,name=scan_format,json=scanFormat,proto3,enum=cockroach.roachpb.ScanFormat" json:"scan_format,omitempty"` + SelectForUpdate bool `protobuf:"varint,5,opt,name=select_for_update,json=selectForUpdate,proto3" json:"select_for_update,omitempty"` } 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_d2c1ab7675455c27, []int{21} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{21} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1226,7 +1227,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_d2c1ab7675455c27, []int{22} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{22} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1266,7 +1267,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_d2c1ab7675455c27, []int{23} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{23} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1313,7 +1314,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_d2c1ab7675455c27, []int{24} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{24} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1366,7 +1367,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_d2c1ab7675455c27, []int{25} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{25} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1403,7 +1404,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_d2c1ab7675455c27, []int{26} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{26} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1447,7 +1448,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_d2c1ab7675455c27, []int{26, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{26, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1495,7 +1496,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_d2c1ab7675455c27, []int{27} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{27} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1531,7 +1532,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_d2c1ab7675455c27, []int{28} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{28} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1619,7 +1620,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_d2c1ab7675455c27, []int{29} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{29} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1665,7 +1666,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_d2c1ab7675455c27, []int{30} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{30} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1726,7 +1727,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_d2c1ab7675455c27, []int{31} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{31} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1761,7 +1762,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_d2c1ab7675455c27, []int{32} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{32} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1800,7 +1801,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_d2c1ab7675455c27, []int{33} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{33} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1835,7 +1836,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_d2c1ab7675455c27, []int{34} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{34} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1878,7 +1879,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_d2c1ab7675455c27, []int{35} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{35} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1913,7 +1914,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_d2c1ab7675455c27, []int{36} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{36} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1951,7 +1952,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_d2c1ab7675455c27, []int{37} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{37} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1984,7 +1985,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_d2c1ab7675455c27, []int{38} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{38} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2019,7 +2020,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_d2c1ab7675455c27, []int{39} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{39} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2077,7 +2078,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_d2c1ab7675455c27, []int{40} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{40} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2112,7 +2113,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_d2c1ab7675455c27, []int{41} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{41} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2149,7 +2150,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_d2c1ab7675455c27, []int{42} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{42} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2182,7 +2183,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_d2c1ab7675455c27, []int{43} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{43} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2221,7 +2222,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_d2c1ab7675455c27, []int{44} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{44} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2258,7 +2259,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_d2c1ab7675455c27, []int{45} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{45} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2296,7 +2297,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_d2c1ab7675455c27, []int{46} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{46} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2330,7 +2331,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_d2c1ab7675455c27, []int{46, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{46, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2364,7 +2365,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_d2c1ab7675455c27, []int{47} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{47} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2433,7 +2434,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_d2c1ab7675455c27, []int{48} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{48} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2476,7 +2477,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_d2c1ab7675455c27, []int{49} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{49} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2523,7 +2524,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_d2c1ab7675455c27, []int{50} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{50} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2559,7 +2560,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_d2c1ab7675455c27, []int{51} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{51} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2603,7 +2604,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_d2c1ab7675455c27, []int{52} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{52} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2642,7 +2643,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_d2c1ab7675455c27, []int{53} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{53} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2702,7 +2703,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_d2c1ab7675455c27, []int{54} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{54} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2738,7 +2739,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_d2c1ab7675455c27, []int{55} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{55} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2784,7 +2785,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_d2c1ab7675455c27, []int{56} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{56} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2819,7 +2820,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_d2c1ab7675455c27, []int{57} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{57} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2869,7 +2870,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_d2c1ab7675455c27, []int{58} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{58} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2904,7 +2905,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_d2c1ab7675455c27, []int{59} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{59} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2941,7 +2942,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_d2c1ab7675455c27, []int{60} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{60} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2975,7 +2976,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_d2c1ab7675455c27, []int{61} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{61} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3020,7 +3021,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_d2c1ab7675455c27, []int{62} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{62} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3054,7 +3055,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_d2c1ab7675455c27, []int{63} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{63} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3098,7 +3099,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_d2c1ab7675455c27, []int{64} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{64} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3147,7 +3148,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_d2c1ab7675455c27, []int{65} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{65} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3184,7 +3185,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_d2c1ab7675455c27, []int{66} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{66} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3221,7 +3222,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_d2c1ab7675455c27, []int{67} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{67} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3256,7 +3257,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_d2c1ab7675455c27, []int{68} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{68} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3311,7 +3312,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_d2c1ab7675455c27, []int{69} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{69} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3348,7 +3349,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_d2c1ab7675455c27, []int{70} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{70} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3387,7 +3388,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_d2c1ab7675455c27, []int{71} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3421,7 +3422,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_d2c1ab7675455c27, []int{71, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3454,7 +3455,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_d2c1ab7675455c27, []int{71, 1} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3494,7 +3495,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_d2c1ab7675455c27, []int{71, 2} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3533,7 +3534,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_d2c1ab7675455c27, []int{71, 3} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3569,7 +3570,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_d2c1ab7675455c27, []int{71, 4} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3608,7 +3609,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_d2c1ab7675455c27, []int{71, 5} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{71, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3648,7 +3649,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_d2c1ab7675455c27, []int{72} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{72} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3682,7 +3683,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_d2c1ab7675455c27, []int{73} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{73} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3744,7 +3745,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_d2c1ab7675455c27, []int{74} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3780,7 +3781,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_d2c1ab7675455c27, []int{75} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3816,7 +3817,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_d2c1ab7675455c27, []int{76} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3856,7 +3857,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_d2c1ab7675455c27, []int{76, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3906,7 +3907,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_d2c1ab7675455c27, []int{77} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3941,7 +3942,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_d2c1ab7675455c27, []int{77, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3977,7 +3978,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_d2c1ab7675455c27, []int{77, 1} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4012,7 +4013,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_d2c1ab7675455c27, []int{78} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4050,7 +4051,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_d2c1ab7675455c27, []int{79} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4085,7 +4086,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_d2c1ab7675455c27, []int{80} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4118,7 +4119,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_d2c1ab7675455c27, []int{80, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4163,7 +4164,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_d2c1ab7675455c27, []int{81} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4201,7 +4202,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_d2c1ab7675455c27, []int{82} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4254,7 +4255,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_d2c1ab7675455c27, []int{83} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4288,7 +4289,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_d2c1ab7675455c27, []int{84} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4332,7 +4333,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_d2c1ab7675455c27, []int{85} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4366,7 +4367,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_d2c1ab7675455c27, []int{86} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4405,7 +4406,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_d2c1ab7675455c27, []int{87} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4439,7 +4440,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_d2c1ab7675455c27, []int{88} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4488,7 +4489,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_d2c1ab7675455c27, []int{89} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4537,7 +4538,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_d2c1ab7675455c27, []int{90} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4572,7 +4573,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_d2c1ab7675455c27, []int{91} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4611,7 +4612,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_d2c1ab7675455c27, []int{92} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4694,7 +4695,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_d2c1ab7675455c27, []int{93} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6144,7 +6145,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_d2c1ab7675455c27, []int{94} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7591,7 +7592,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_d2c1ab7675455c27, []int{95} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7627,7 +7628,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{96} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{96} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7664,7 +7665,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_d2c1ab7675455c27, []int{97} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{97} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7723,7 +7724,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_d2c1ab7675455c27, []int{97, 0} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{97, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7762,7 +7763,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_d2c1ab7675455c27, []int{98} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{98} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7803,7 +7804,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_d2c1ab7675455c27, []int{99} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{99} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7844,7 +7845,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_d2c1ab7675455c27, []int{100} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{100} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7881,7 +7882,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_d2c1ab7675455c27, []int{101} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{101} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7918,7 +7919,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_d2c1ab7675455c27, []int{102} + return fileDescriptor_api_b867aaa08bf9d0c5, []int{102} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8383,6 +8384,9 @@ func (this *ScanRequest) Equal(that interface{}) bool { if this.ScanFormat != that1.ScanFormat { return false } + if this.SelectForUpdate != that1.SelectForUpdate { + return false + } return true } func (this *ReverseScanRequest) Equal(that interface{}) bool { @@ -10807,6 +10811,16 @@ func (m *ScanRequest) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintApi(dAtA, i, uint64(m.ScanFormat)) } + if m.SelectForUpdate { + dAtA[i] = 0x28 + i++ + if m.SelectForUpdate { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -16423,6 +16437,9 @@ func (m *ScanRequest) Size() (n int) { if m.ScanFormat != 0 { n += 1 + sovApi(uint64(m.ScanFormat)) } + if m.SelectForUpdate { + n += 2 + } return n } @@ -21580,6 +21597,26 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { break } } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SelectForUpdate", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SelectForUpdate = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -36845,17 +36882,17 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_d2c1ab7675455c27) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_b867aaa08bf9d0c5) } -var fileDescriptor_api_d2c1ab7675455c27 = []byte{ - // 7159 bytes of a gzipped FileDescriptorProto +var fileDescriptor_api_b867aaa08bf9d0c5 = []byte{ + // 7176 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x24, 0xc7, 0x75, 0x2e, 0x7b, 0x66, 0x38, 0x9c, 0x39, 0xf3, 0xc3, 0x66, 0x71, 0x7f, 0x66, 0x29, 0x69, 0xc9, 0x9d, 0xfd, 0xd5, 0x5a, 0xe2, 0x6a, 0x77, 0xad, 0x2b, 0x59, 0x92, 0x65, 0x93, 0xc3, 0xd9, 0x1d, 0x2e, 0x97, 0x5c, 0xaa, 0x67, 0xb8, 0xf2, 0xca, 0x57, 0xb7, 0xdd, 0xdb, 0x5d, 0x1c, 0xb6, 0x77, 0xa6, 0x7b, 0xb6, 0xbb, 0x87, 0x3f, 0x0b, 0x5c, 0x5c, 0xdc, 0x7b, 0x1f, 0x7c, 0xe1, 0x6b, 0x08, - 0x01, 0x12, 0x04, 0x81, 0xed, 0xc0, 0x02, 0x1c, 0x24, 0x41, 0x02, 0x1b, 0x08, 0x10, 0x24, 0x88, - 0x91, 0xc0, 0x0f, 0x79, 0x51, 0x0c, 0x23, 0x10, 0x82, 0x04, 0x36, 0x02, 0x84, 0x88, 0xd7, 0x40, + 0x01, 0x12, 0x04, 0x81, 0xed, 0xc0, 0x02, 0x12, 0x24, 0x41, 0x02, 0x1b, 0x08, 0x10, 0x24, 0x88, + 0x91, 0xc0, 0x0f, 0x79, 0x51, 0x0c, 0x23, 0x11, 0x82, 0x04, 0x36, 0x02, 0x84, 0x88, 0xd7, 0x40, 0x62, 0xf8, 0x21, 0xaf, 0x01, 0xf4, 0x90, 0x04, 0xf5, 0xd3, 0x7f, 0x33, 0x3d, 0x3f, 0xa4, 0x5a, 0x89, 0x82, 0x3c, 0x91, 0x7d, 0xba, 0xce, 0xe9, 0xaa, 0x53, 0xa7, 0x4e, 0x9d, 0xaf, 0xea, 0x54, 0x0d, 0xcc, 0x58, 0xa6, 0xa2, 0xee, 0x74, 0x1e, 0x5e, 0x53, 0x3a, 0xfa, 0x62, 0xc7, 0x32, 0x1d, @@ -36870,13 +36907,13 @@ var fileDescriptor_api_d2c1ab7675455c27 = []byte{ 0x05, 0xe1, 0x4a, 0xee, 0x46, 0x79, 0xb1, 0x4f, 0x17, 0x8b, 0xb4, 0xec, 0x0a, 0xb6, 0x55, 0x4b, 0xef, 0x38, 0xa6, 0xb5, 0x9c, 0xfa, 0xe0, 0x70, 0x7e, 0x42, 0xa2, 0x5c, 0xe8, 0xb3, 0x30, 0xd9, 0xc2, 0x8a, 0x8d, 0x4b, 0x09, 0xca, 0x5e, 0x8a, 0x60, 0xbf, 0x4b, 0xde, 0x73, 0x26, 0x56, 0xb8, - 0xfc, 0xe7, 0x02, 0x14, 0x24, 0xfc, 0xb8, 0x8b, 0x6d, 0xa7, 0x86, 0x15, 0x0d, 0x5b, 0xe8, 0x0c, + 0xfc, 0x67, 0x02, 0x14, 0x24, 0xfc, 0xb8, 0x8b, 0x6d, 0xa7, 0x86, 0x15, 0x0d, 0x5b, 0xe8, 0x0c, 0x24, 0x1f, 0xe1, 0x83, 0x52, 0x72, 0x41, 0xb8, 0x92, 0x5f, 0x9e, 0xfa, 0xe8, 0x70, 0x3e, 0xb9, 0x86, 0x0f, 0x24, 0x42, 0x43, 0x0b, 0x30, 0x85, 0x0d, 0x4d, 0x26, 0xaf, 0x53, 0xe1, 0xd7, 0x69, 0x6c, 0x68, 0x6b, 0xf8, 0x00, 0xa9, 0x90, 0xb1, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x05, 0xe1, 0xca, 0xe4, 0xf2, 0xed, 0x8f, 0x0e, 0xe7, 0x2b, 0x4d, 0xdd, 0xd9, 0xe9, 0x3e, 0x5c, 0x54, 0xcd, 0xf6, 0x35, 0xaf, 0x56, 0xda, 0x43, 0xff, 0xff, 0x6b, 0x9d, 0x47, 0xcd, 0x6b, 0x03, 0x7a, 0x60, - 0xb1, 0xb1, 0x6f, 0xd4, 0xf1, 0x63, 0xc9, 0x13, 0xfc, 0x5a, 0xea, 0x17, 0xef, 0xcf, 0x0b, 0x77, + 0xb1, 0xb1, 0x6f, 0xd4, 0xf1, 0x63, 0xc9, 0x13, 0xfc, 0x5a, 0xea, 0xe7, 0xef, 0xcf, 0x0b, 0x77, 0x52, 0x19, 0x41, 0x4c, 0xdc, 0x49, 0x65, 0x12, 0x62, 0xb2, 0xfc, 0x8d, 0x24, 0x14, 0x25, 0x6c, 0x77, 0x4c, 0xc3, 0xc6, 0xbc, 0x19, 0x2f, 0x41, 0xd2, 0xd9, 0x37, 0x68, 0x33, 0x72, 0x37, 0xce, 0x46, 0x28, 0xa3, 0x61, 0x29, 0x86, 0xad, 0xa8, 0x8e, 0x6e, 0x1a, 0x12, 0x29, 0x8a, 0x5e, 0x85, @@ -36890,411 +36927,412 @@ var fileDescriptor_api_d2c1ab7675455c27 = []byte{ 0xeb, 0x5b, 0xeb, 0x55, 0x79, 0x6b, 0x63, 0x6d, 0xe3, 0xde, 0xdb, 0x1b, 0xe2, 0x04, 0x3a, 0x01, 0x22, 0xa7, 0xad, 0x55, 0x1f, 0xc8, 0x77, 0x57, 0xd7, 0x57, 0x1b, 0xa2, 0x80, 0xce, 0xc0, 0x49, 0x4e, 0x95, 0x96, 0x36, 0x6e, 0x57, 0xe5, 0xe5, 0x7b, 0x5b, 0x1b, 0x2b, 0x4b, 0xd2, 0x03, 0x31, - 0x31, 0x97, 0xfa, 0x7f, 0xdf, 0x3d, 0x3b, 0x51, 0xbe, 0x0f, 0x70, 0x1b, 0x3b, 0xdc, 0xac, 0xd0, - 0x32, 0xa4, 0x77, 0x68, 0x6d, 0xb8, 0x61, 0x2f, 0x44, 0x56, 0x3b, 0x60, 0x82, 0xcb, 0x19, 0xa2, - 0x81, 0x0f, 0x0f, 0xe7, 0x05, 0x89, 0x73, 0xb2, 0x2e, 0x2f, 0xff, 0x50, 0x80, 0x1c, 0x15, 0xcc, - 0xda, 0x88, 0x2a, 0x3d, 0x92, 0xcf, 0x8d, 0x54, 0x48, 0xbf, 0x68, 0xb4, 0x08, 0x93, 0xbb, 0x4a, - 0xab, 0x3b, 0x6c, 0xdc, 0xdc, 0x27, 0xef, 0x25, 0x56, 0x0c, 0xbd, 0x0e, 0x79, 0xdd, 0x70, 0xb0, - 0xe1, 0xc8, 0x8c, 0x2d, 0x39, 0x82, 0x2d, 0xc7, 0x4a, 0xd3, 0x87, 0xf2, 0x0f, 0x04, 0x80, 0xcd, - 0x6e, 0x9c, 0xaa, 0x21, 0xe3, 0x7e, 0xac, 0xfa, 0xbb, 0xe3, 0x9e, 0xb5, 0xe2, 0x14, 0xa4, 0x75, - 0xa3, 0xa5, 0x1b, 0xac, 0xfe, 0x19, 0x89, 0x3f, 0xa1, 0x13, 0x30, 0xf9, 0xb0, 0xa5, 0x1b, 0x1a, - 0x35, 0xff, 0x8c, 0xc4, 0x1e, 0xb8, 0xfa, 0x25, 0xc8, 0xd1, 0xba, 0xc7, 0xa8, 0xfd, 0xf2, 0x37, - 0x13, 0x70, 0xb2, 0x62, 0x1a, 0x9a, 0x4e, 0xc6, 0xa1, 0xd2, 0xfa, 0x54, 0xe8, 0xe6, 0x65, 0xc8, - 0xe2, 0xfd, 0xce, 0x98, 0xdd, 0x9b, 0xc1, 0xfb, 0x1d, 0xfa, 0x5f, 0xb4, 0xea, 0xd0, 0x67, 0xe1, - 0xb4, 0xd2, 0x6a, 0x99, 0x7b, 0xb2, 0xbe, 0x2d, 0x6b, 0x26, 0xb6, 0x65, 0xc3, 0x74, 0x64, 0xbc, - 0xaf, 0xdb, 0x0e, 0x75, 0x15, 0x19, 0x69, 0x96, 0xbe, 0x5e, 0xdd, 0x5e, 0x31, 0xb1, 0xbd, 0x61, - 0x3a, 0x55, 0xf2, 0x8a, 0x2b, 0xfc, 0x5d, 0x38, 0xd5, 0xab, 0x9b, 0x38, 0x75, 0xff, 0x57, 0x02, - 0x14, 0x57, 0x0d, 0xdd, 0xf9, 0x54, 0x28, 0xdd, 0xd3, 0x5e, 0x32, 0xa8, 0xbd, 0xab, 0x20, 0x6e, - 0x2b, 0x7a, 0xeb, 0x9e, 0xd1, 0x30, 0xdb, 0x0f, 0x6d, 0xc7, 0x34, 0xb0, 0xcd, 0xd5, 0xdb, 0x47, - 0xe7, 0x3a, 0xbb, 0x0f, 0xd3, 0x5e, 0x9b, 0xe2, 0x54, 0xd6, 0x13, 0x10, 0x57, 0x0d, 0xd5, 0xc2, - 0x6d, 0x6c, 0xc4, 0xaa, 0xad, 0x67, 0x21, 0xab, 0xbb, 0x72, 0xa9, 0xc6, 0x92, 0x92, 0x4f, 0xe0, - 0x6d, 0xea, 0xc2, 0x4c, 0xe0, 0xdb, 0x71, 0x3a, 0xbf, 0x67, 0x20, 0x6b, 0xe0, 0x3d, 0xd9, 0xef, - 0xaf, 0xa4, 0x94, 0x31, 0xf0, 0x1e, 0x73, 0x56, 0x0f, 0xa0, 0xb0, 0x82, 0x5b, 0xd8, 0xc1, 0xf1, - 0x7b, 0xf2, 0x2d, 0x28, 0xba, 0xa2, 0xe3, 0xec, 0xa4, 0xdf, 0x14, 0x00, 0x71, 0xb9, 0x64, 0xf6, - 0x8c, 0xb3, 0x9f, 0xe6, 0x49, 0x74, 0xe0, 0x74, 0x2d, 0x83, 0x4d, 0xf3, 0xcc, 0x4a, 0x81, 0x91, - 0xe8, 0x4c, 0xef, 0x7b, 0xd4, 0x54, 0xd0, 0xa3, 0x7a, 0xd1, 0x0a, 0x89, 0x53, 0xf6, 0x60, 0x36, - 0x54, 0xbd, 0x78, 0xbb, 0x32, 0x45, 0x6b, 0x96, 0x58, 0x48, 0x06, 0x23, 0x33, 0x4a, 0x2c, 0xbf, - 0x0b, 0x33, 0x95, 0x16, 0x56, 0xac, 0xb8, 0xd5, 0xc2, 0xbb, 0xf3, 0x01, 0xa0, 0xa0, 0xf8, 0x38, - 0xbb, 0xf4, 0xb7, 0x04, 0x40, 0x12, 0xde, 0xc5, 0x96, 0x13, 0x7b, 0x97, 0xae, 0x40, 0xce, 0x51, - 0xac, 0x26, 0x76, 0x64, 0x12, 0xce, 0x73, 0x77, 0xf5, 0x5c, 0x40, 0x10, 0x09, 0xea, 0x17, 0x77, - 0x5a, 0xea, 0x62, 0xc3, 0x0d, 0xf7, 0xdd, 0xe0, 0x8c, 0xf1, 0x11, 0x32, 0xd7, 0xc0, 0x3b, 0x30, - 0x1b, 0xaa, 0x65, 0x9c, 0x2a, 0xf8, 0xae, 0x00, 0xb9, 0xba, 0xaa, 0x18, 0x71, 0xb6, 0xfd, 0x4d, - 0xc8, 0xd9, 0xaa, 0x62, 0xc8, 0xdb, 0xa6, 0xd5, 0x56, 0x1c, 0x6a, 0xb2, 0xc5, 0x50, 0xdb, 0xbd, - 0x60, 0x57, 0x55, 0x8c, 0x5b, 0xb4, 0x90, 0x04, 0xb6, 0xf7, 0x7f, 0xd0, 0xaa, 0xef, 0xa4, 0x32, - 0x49, 0x31, 0x55, 0xfe, 0x67, 0x01, 0xf2, 0xac, 0x96, 0x71, 0x5a, 0xf5, 0xcb, 0x90, 0xb2, 0xcc, - 0x3d, 0x66, 0xd5, 0xb9, 0x1b, 0xcf, 0x44, 0x88, 0x58, 0xc3, 0x07, 0xc1, 0xe9, 0x84, 0x16, 0x47, - 0xcb, 0xc0, 0xc3, 0x2e, 0x99, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0x97, 0x61, - 0xfa, 0xa1, 0xe2, 0xa8, 0x3b, 0xb2, 0xc5, 0x2b, 0x49, 0xa6, 0x9e, 0xe4, 0x95, 0xbc, 0x54, 0xa4, - 0x64, 0xb7, 0xea, 0x76, 0xf9, 0xf7, 0x5c, 0x13, 0xb5, 0xf1, 0xa7, 0xbf, 0x9b, 0xfe, 0x45, 0xe0, - 0x96, 0xea, 0x56, 0xf6, 0xbf, 0x5a, 0x6f, 0x7d, 0x27, 0x01, 0xa7, 0x2b, 0x3b, 0x58, 0x7d, 0x54, - 0x31, 0x0d, 0x5b, 0xb7, 0x1d, 0x6c, 0xa8, 0x07, 0x71, 0x76, 0xd9, 0x33, 0x90, 0xdd, 0xd3, 0x9d, - 0x1d, 0x59, 0xd3, 0xb7, 0xb7, 0xa9, 0x4f, 0xc9, 0x48, 0x19, 0x42, 0x58, 0xd1, 0xb7, 0xb7, 0xd1, - 0x4d, 0x48, 0xb5, 0x4d, 0x8d, 0x45, 0x95, 0xc5, 0x1b, 0xf3, 0x11, 0xe2, 0x69, 0xd5, 0xec, 0x6e, - 0x7b, 0xdd, 0xd4, 0xb0, 0x44, 0x0b, 0xa3, 0xb3, 0x00, 0x2a, 0xa1, 0x76, 0x4c, 0xdd, 0x70, 0xf8, - 0xec, 0x12, 0xa0, 0xa0, 0x1a, 0x64, 0x1d, 0x6c, 0xb5, 0x75, 0x43, 0x71, 0x08, 0xea, 0x26, 0xca, - 0xbb, 0x10, 0x59, 0xf1, 0x4e, 0x4b, 0x57, 0x95, 0xbe, 0xe5, 0x03, 0x9f, 0x99, 0xfb, 0xb2, 0xf7, - 0x52, 0x50, 0xea, 0xd7, 0x50, 0x9c, 0x76, 0xb2, 0x09, 0x69, 0x82, 0x45, 0x5b, 0x0e, 0xb7, 0x94, - 0x1b, 0x83, 0x14, 0x11, 0x51, 0x03, 0x8a, 0x69, 0x5b, 0x0e, 0xaf, 0x3c, 0x97, 0x33, 0xf7, 0xa7, - 0x02, 0xa4, 0xd9, 0x0b, 0x74, 0x1d, 0x32, 0x1c, 0x72, 0x6b, 0xb4, 0x8e, 0xc9, 0xe5, 0x53, 0x4f, - 0x0f, 0xe7, 0xa7, 0x18, 0xc0, 0x5e, 0xf9, 0xc8, 0xff, 0x57, 0x9a, 0x62, 0x18, 0x5b, 0x23, 0x7d, - 0x66, 0x3b, 0x8a, 0xe5, 0xd0, 0xa5, 0x0d, 0xd2, 0x67, 0x79, 0x29, 0x43, 0x09, 0x6b, 0xf8, 0x00, - 0xdd, 0x81, 0xb4, 0xed, 0x28, 0x4e, 0xd7, 0xe6, 0xbd, 0x76, 0xa4, 0xca, 0xd6, 0x29, 0xa7, 0xc4, - 0x25, 0x90, 0x20, 0x41, 0xc3, 0x8e, 0xa2, 0xb7, 0x68, 0x37, 0x66, 0x25, 0xfe, 0x54, 0xfe, 0x96, - 0x00, 0x69, 0x56, 0x14, 0x9d, 0x86, 0x59, 0x06, 0xab, 0x57, 0x37, 0x56, 0xaa, 0x8d, 0xaa, 0xb4, - 0xbe, 0xba, 0xb1, 0xd4, 0xa8, 0x8a, 0x13, 0xe8, 0x14, 0x20, 0xf7, 0x45, 0xe5, 0xde, 0x46, 0x7d, - 0xb5, 0xde, 0xa8, 0x6e, 0x10, 0x38, 0x4e, 0x40, 0x3a, 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x01, 0x16, - 0x7a, 0xa9, 0x72, 0xbd, 0xb1, 0xd4, 0xa8, 0xcb, 0xd5, 0x7a, 0x63, 0x75, 0x7d, 0xa9, 0x51, 0x5d, - 0x11, 0x93, 0x43, 0x4a, 0x91, 0x8f, 0x48, 0x52, 0xb5, 0xd2, 0x10, 0x53, 0xe5, 0x27, 0x70, 0x52, - 0xc2, 0xaa, 0xd9, 0xee, 0x74, 0x1d, 0x4c, 0x6a, 0x69, 0xc7, 0x39, 0x5e, 0x4e, 0xc3, 0x94, 0x66, - 0x1d, 0xc8, 0x56, 0xd7, 0xe0, 0xa3, 0x25, 0xad, 0x59, 0x07, 0x52, 0xd7, 0xe0, 0xc6, 0xf8, 0xc7, - 0x02, 0x9c, 0xea, 0xfd, 0x78, 0x9c, 0xa6, 0xf8, 0x25, 0xc8, 0x29, 0x9a, 0x86, 0x35, 0x59, 0xc3, - 0x2d, 0x47, 0xe1, 0x41, 0xc0, 0xf5, 0x80, 0x24, 0xbe, 0x2c, 0xb5, 0xc8, 0xd6, 0xa3, 0x16, 0xbd, - 0x65, 0xa9, 0xf5, 0xfb, 0x95, 0x0a, 0xad, 0xcf, 0x0a, 0x61, 0x74, 0x3d, 0x12, 0x95, 0x45, 0x29, - 0xe5, 0x6f, 0xa7, 0xa0, 0x50, 0x35, 0xb4, 0xc6, 0x7e, 0xac, 0x33, 0xc2, 0x29, 0x48, 0xab, 0x66, - 0xbb, 0xad, 0x3b, 0xae, 0xb6, 0xd8, 0x13, 0xfa, 0x1c, 0x64, 0x34, 0xac, 0x68, 0x1e, 0xa4, 0x1f, - 0x15, 0xc9, 0x48, 0x5e, 0x71, 0xf4, 0x15, 0x38, 0x4d, 0x1c, 0xa9, 0x65, 0x28, 0x2d, 0x99, 0x49, - 0x93, 0x1d, 0x4b, 0x6f, 0x36, 0xb1, 0xc5, 0x17, 0xc1, 0xae, 0x44, 0xd4, 0x73, 0x95, 0x73, 0x54, - 0x28, 0x43, 0x83, 0x95, 0x97, 0x4e, 0xea, 0x51, 0x64, 0xf4, 0x45, 0x6f, 0xcd, 0xc4, 0xee, 0x28, - 0x86, 0xcd, 0x9d, 0xd4, 0xa0, 0xb5, 0x35, 0xae, 0x4b, 0x3e, 0x27, 0x10, 0x8a, 0x8d, 0xae, 0x91, - 0xf0, 0xfb, 0x71, 0x57, 0xb7, 0xb0, 0x7c, 0xbd, 0xa3, 0x96, 0xd2, 0xa4, 0xed, 0xcb, 0xc5, 0xa7, - 0x87, 0xf3, 0x20, 0x31, 0xf2, 0xf5, 0xcd, 0x0a, 0x09, 0xc7, 0xd9, 0xff, 0x1d, 0x15, 0x5d, 0x01, - 0xd1, 0x30, 0x65, 0x0b, 0x6f, 0x5b, 0xd8, 0xde, 0xe1, 0x9f, 0xcd, 0x50, 0x8d, 0x15, 0x0d, 0x53, - 0x62, 0x64, 0x26, 0xfa, 0x14, 0xa4, 0x3b, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xcb, 0x34, 0xca, 0x9e, - 0xd0, 0x5b, 0x20, 0xea, 0x86, 0xbc, 0xdd, 0xd2, 0x9b, 0x3b, 0x8e, 0xbc, 0x67, 0xe9, 0x0e, 0xb6, - 0x4b, 0x33, 0xb4, 0xe2, 0x51, 0x86, 0x56, 0xe7, 0xab, 0x93, 0xda, 0xdb, 0xa4, 0x24, 0x6f, 0x42, - 0x51, 0x37, 0x6e, 0x51, 0x7e, 0x4a, 0xb4, 0xbd, 0xe9, 0x78, 0x4a, 0xcc, 0x94, 0xff, 0x4e, 0x80, - 0xa2, 0x6b, 0x1e, 0x71, 0x1a, 0xf4, 0x15, 0x10, 0x4d, 0x03, 0xcb, 0x9d, 0x1d, 0xc5, 0xc6, 0xbc, - 0x3b, 0xf9, 0x9c, 0x51, 0x34, 0x0d, 0xbc, 0x49, 0xc8, 0xac, 0x73, 0xd0, 0x26, 0xcc, 0xd8, 0x8e, - 0xd2, 0xd4, 0x8d, 0xa6, 0xec, 0xad, 0x67, 0xd3, 0x45, 0x89, 0x31, 0xa3, 0x60, 0x91, 0x73, 0x7b, - 0xf4, 0x50, 0xa0, 0xf1, 0x13, 0x01, 0x66, 0x96, 0xb4, 0xb6, 0x6e, 0xd4, 0x3b, 0x2d, 0x3d, 0x56, - 0xc8, 0x7c, 0x01, 0xb2, 0x36, 0x91, 0xe9, 0x7b, 0x6b, 0x1f, 0xee, 0x64, 0xe8, 0x1b, 0xe2, 0xb6, - 0xef, 0xc2, 0x34, 0xde, 0xef, 0xe8, 0x96, 0xe2, 0xe8, 0xa6, 0xc1, 0x22, 0xfc, 0xd4, 0xf8, 0x6d, - 0x2b, 0xfa, 0xbc, 0x7e, 0x94, 0xcf, 0x5b, 0xf6, 0x00, 0x50, 0xb0, 0x61, 0x71, 0x86, 0xfa, 0x32, - 0xcc, 0x52, 0xd1, 0x5b, 0x86, 0x1d, 0xb3, 0xd6, 0xb8, 0x3b, 0xfd, 0x32, 0x9c, 0x08, 0x7f, 0x20, - 0xce, 0xda, 0xbf, 0xcb, 0x7b, 0x7c, 0x1d, 0x5b, 0x9f, 0x10, 0xca, 0x0c, 0x8a, 0x8f, 0xb3, 0xe6, - 0x5f, 0x17, 0xe0, 0x0c, 0x95, 0x4d, 0x77, 0x05, 0xb6, 0xb1, 0x45, 0xb7, 0x4a, 0xe2, 0x34, 0xda, - 0xf3, 0x90, 0x66, 0xa0, 0x91, 0x5a, 0xec, 0xe4, 0x72, 0x8e, 0x04, 0x22, 0x75, 0xc7, 0xb4, 0x48, - 0x20, 0xc2, 0x5f, 0xf1, 0x76, 0x2a, 0x30, 0x17, 0x55, 0x97, 0x98, 0x51, 0xf5, 0x0c, 0x8f, 0x07, - 0x89, 0x89, 0x57, 0x76, 0x48, 0x20, 0x84, 0xaa, 0x90, 0x53, 0xe9, 0x7f, 0xb2, 0x73, 0xd0, 0xc1, - 0x54, 0x7e, 0x71, 0x58, 0x28, 0xc9, 0xd8, 0x1a, 0x07, 0x1d, 0x4c, 0xe2, 0x51, 0xf7, 0x7f, 0xa2, - 0xae, 0x40, 0x53, 0x87, 0x06, 0xa3, 0x74, 0x7c, 0xd1, 0xb2, 0x6e, 0x3c, 0x17, 0xd2, 0xc4, 0x9f, - 0x24, 0xb9, 0x2a, 0xd8, 0x97, 0x38, 0x53, 0xac, 0xe1, 0xc7, 0x3b, 0x70, 0x4a, 0xc3, 0x1d, 0x0b, - 0xab, 0x8a, 0x83, 0x35, 0x39, 0xd8, 0xfc, 0xc4, 0x11, 0x9a, 0x7f, 0xc2, 0x97, 0xe1, 0x53, 0xd1, - 0x03, 0x40, 0x01, 0xd9, 0xac, 0x65, 0x2e, 0xbc, 0x39, 0x8a, 0x52, 0x66, 0x7c, 0x29, 0x8c, 0x6e, - 0xa3, 0x0a, 0x64, 0xf0, 0x7e, 0x47, 0xa6, 0xfb, 0x85, 0xa9, 0x23, 0xee, 0x17, 0x4e, 0xe1, 0xfd, - 0x0e, 0x21, 0xa2, 0x2d, 0x32, 0xc3, 0xb9, 0x13, 0x3f, 0xad, 0xb6, 0x3d, 0x1a, 0x3f, 0xf8, 0xf6, - 0xc2, 0xc5, 0x4d, 0x7b, 0x73, 0x3e, 0x13, 0xc1, 0xfb, 0xee, 0x7d, 0x01, 0x9e, 0x89, 0xec, 0xbb, - 0x38, 0x27, 0x3b, 0x77, 0xcb, 0x34, 0x71, 0x9c, 0x2d, 0xd3, 0xf2, 0xef, 0xbb, 0xa3, 0x5e, 0xc2, - 0x2d, 0x93, 0xa8, 0xf7, 0x13, 0x58, 0x62, 0x9a, 0x72, 0xbb, 0x3d, 0x71, 0xe4, 0x6e, 0x77, 0x59, - 0x7b, 0xdc, 0x42, 0x4f, 0x65, 0xe3, 0x74, 0x0b, 0xbf, 0x2e, 0xc0, 0x6c, 0x0d, 0x2b, 0x96, 0xf3, - 0x10, 0x2b, 0x4e, 0xcc, 0x81, 0xeb, 0xcb, 0x90, 0x34, 0xcc, 0xbd, 0xa3, 0xac, 0xb2, 0x91, 0xf2, - 0xfe, 0xb4, 0x15, 0xae, 0x57, 0x9c, 0xad, 0xfe, 0x8b, 0x04, 0x64, 0x6f, 0x57, 0xe2, 0x6c, 0xeb, - 0x1b, 0x7c, 0x2d, 0x96, 0x0d, 0xf5, 0x28, 0xb3, 0xf4, 0xbe, 0xb7, 0x78, 0xbb, 0xb2, 0x86, 0x0f, - 0x5c, 0xb3, 0x24, 0x5c, 0x68, 0x09, 0xb2, 0xce, 0x0e, 0x89, 0x4f, 0xcd, 0x96, 0x76, 0x94, 0x98, - 0xc5, 0xe7, 0x9a, 0x7b, 0x04, 0x93, 0x54, 0xae, 0xbb, 0x9b, 0x2f, 0x44, 0xec, 0xe6, 0x93, 0xcf, - 0x78, 0x61, 0x5f, 0xe2, 0x28, 0x9f, 0x71, 0x09, 0xac, 0x73, 0xbc, 0xd8, 0x68, 0x52, 0x4c, 0x97, - 0xdf, 0x02, 0x20, 0x4d, 0x8b, 0xb3, 0x7b, 0x7e, 0x35, 0x09, 0xc5, 0xcd, 0xae, 0xbd, 0x13, 0xb3, - 0x3d, 0x56, 0x00, 0x3a, 0x5d, 0x7b, 0x07, 0x5b, 0xb2, 0xb3, 0x6f, 0xf0, 0xf6, 0x8f, 0xc8, 0x13, - 0x70, 0x15, 0xc0, 0xf8, 0x1a, 0xfb, 0x06, 0xba, 0xc7, 0x85, 0x60, 0xd9, 0x4f, 0x36, 0xb8, 0x3a, - 0x06, 0x78, 0x6c, 0xec, 0x1b, 0xeb, 0xd8, 0x43, 0x8d, 0x4c, 0x20, 0x26, 0x02, 0xdf, 0x80, 0x29, - 0xf2, 0x20, 0x3b, 0xe6, 0x51, 0x7a, 0x3e, 0x4d, 0x78, 0x1a, 0x26, 0x7a, 0x1d, 0xb2, 0x8c, 0x9b, - 0xcc, 0x5f, 0x69, 0x3a, 0x7f, 0x45, 0x35, 0x89, 0x6b, 0x93, 0xce, 0x5c, 0x19, 0xca, 0x4a, 0x66, - 0xab, 0x13, 0x30, 0xb9, 0x6d, 0x5a, 0x2a, 0xa6, 0xd9, 0x05, 0x19, 0x89, 0x3d, 0x04, 0x3b, 0xf7, - 0x4e, 0x2a, 0x93, 0x11, 0xb3, 0x77, 0x52, 0x99, 0xac, 0x08, 0xe5, 0x6f, 0x09, 0x30, 0xed, 0xf5, - 0x4a, 0x9c, 0x2e, 0xbd, 0x12, 0x52, 0xe9, 0xd1, 0xfb, 0x85, 0xa8, 0xb1, 0xfc, 0xd7, 0x34, 0xbe, - 0x51, 0xcd, 0x5d, 0xda, 0x4d, 0x71, 0x9a, 0xcd, 0x32, 0xcb, 0x2b, 0x49, 0x1c, 0xb3, 0xab, 0x69, - 0xa6, 0xc9, 0x75, 0x38, 0xa1, 0xb7, 0x89, 0xcf, 0xd7, 0x9d, 0xd6, 0x01, 0xc7, 0x68, 0x0e, 0x76, - 0xb7, 0x3e, 0x67, 0xfd, 0x77, 0x15, 0xf7, 0x15, 0x77, 0x83, 0x6c, 0x33, 0xc4, 0x6f, 0x56, 0x9c, - 0x7a, 0x5f, 0x85, 0x82, 0xc5, 0x44, 0x93, 0x58, 0xe5, 0x88, 0xaa, 0xcf, 0x7b, 0xac, 0x44, 0xfb, - 0xdf, 0x4f, 0xc0, 0xf4, 0x5b, 0x5d, 0x6c, 0x1d, 0x7c, 0x0a, 0x75, 0x7f, 0x09, 0xa6, 0xf7, 0x14, - 0xdd, 0x91, 0xb7, 0x4d, 0x4b, 0xee, 0x76, 0x34, 0xc5, 0x71, 0x33, 0x20, 0x0a, 0x84, 0x7c, 0xcb, - 0xb4, 0xb6, 0x28, 0x11, 0x61, 0x40, 0x8f, 0x0c, 0x73, 0xcf, 0x90, 0x09, 0x99, 0x42, 0xe4, 0x7d, - 0x83, 0x2f, 0x29, 0x2f, 0xbf, 0xf2, 0xb7, 0x87, 0xf3, 0x37, 0xc7, 0xca, 0x69, 0xa2, 0x69, 0x61, - 0xdd, 0xae, 0xae, 0x2d, 0x6e, 0x6d, 0xad, 0xae, 0x48, 0x22, 0x15, 0xf9, 0x36, 0x93, 0xd8, 0xd8, - 0x37, 0xdc, 0xa9, 0xfd, 0x23, 0x01, 0x44, 0x5f, 0x61, 0x71, 0xf6, 0x6a, 0x15, 0x72, 0x8f, 0xbb, - 0xd8, 0xd2, 0x8f, 0xd1, 0xa7, 0xc0, 0x19, 0x89, 0x5b, 0x7a, 0x07, 0xf2, 0x21, 0x3d, 0x24, 0x3f, - 0x9e, 0x1e, 0x72, 0x7b, 0xbe, 0x0a, 0xca, 0x3f, 0x16, 0x00, 0xd1, 0xc6, 0xaf, 0xb2, 0xd5, 0xfc, - 0x4f, 0x99, 0xc1, 0x5c, 0x01, 0x91, 0x66, 0x19, 0xca, 0xfa, 0xb6, 0xdc, 0xd6, 0x6d, 0x5b, 0x37, - 0x9a, 0xdc, 0x62, 0x8a, 0x94, 0xbe, 0xba, 0xbd, 0xce, 0xa8, 0xbc, 0x2f, 0xff, 0x27, 0xcc, 0x86, - 0x5a, 0x13, 0x67, 0x6f, 0x9e, 0x83, 0xfc, 0xb6, 0xd9, 0x35, 0x34, 0x99, 0x2d, 0x8d, 0xf1, 0x25, - 0xc0, 0x1c, 0xa5, 0xb1, 0xef, 0x95, 0xff, 0x35, 0x01, 0x27, 0x24, 0x6c, 0x9b, 0xad, 0x5d, 0x1c, - 0xbf, 0x3e, 0xef, 0x01, 0xdf, 0x72, 0x91, 0x3f, 0x8e, 0x5a, 0xb3, 0x4c, 0x06, 0x9b, 0xee, 0xc2, - 0x6b, 0xeb, 0x17, 0x86, 0x5b, 0x66, 0xff, 0x6a, 0x3a, 0x5f, 0xb9, 0x4b, 0x85, 0x56, 0xee, 0x76, - 0x61, 0x5a, 0x6f, 0x1a, 0x26, 0x71, 0x64, 0x36, 0x7e, 0x6c, 0x74, 0xdb, 0x2e, 0xac, 0x79, 0x79, - 0x8c, 0xba, 0xae, 0x32, 0xce, 0x3a, 0x7e, 0xbc, 0xd1, 0x6d, 0xd3, 0x18, 0x7b, 0xf9, 0x14, 0xa9, - 0xf6, 0xd3, 0xc3, 0xf9, 0x62, 0xe8, 0x9d, 0x2d, 0x15, 0x75, 0xef, 0x99, 0x7c, 0x84, 0x1b, 0xc0, - 0x7f, 0x87, 0x93, 0x3d, 0x1d, 0x10, 0x67, 0x34, 0xf4, 0x97, 0x49, 0x38, 0x13, 0x16, 0x1f, 0x37, - 0x66, 0xf9, 0x4f, 0xd2, 0xc9, 0x35, 0x28, 0xb4, 0x75, 0xe3, 0x78, 0x2b, 0x97, 0xf9, 0xb6, 0x6e, - 0x78, 0xb4, 0x28, 0x73, 0x49, 0xff, 0xfb, 0x99, 0x8b, 0x02, 0x73, 0x51, 0xfd, 0x19, 0xa7, 0xcd, - 0xbc, 0x27, 0x40, 0x3e, 0xee, 0x35, 0xb9, 0xe3, 0xa5, 0x79, 0xf1, 0x36, 0x37, 0xa0, 0xf0, 0x09, - 0x2c, 0xe2, 0xfd, 0x8e, 0x00, 0xa8, 0x61, 0x75, 0x0d, 0x02, 0x8e, 0xef, 0x9a, 0xcd, 0x38, 0x1b, - 0x7b, 0x02, 0x26, 0x75, 0x43, 0xc3, 0xfb, 0xb4, 0xb1, 0x29, 0x89, 0x3d, 0x84, 0x76, 0x1a, 0x93, - 0x63, 0xed, 0x34, 0xfa, 0xd9, 0x22, 0xa1, 0x8a, 0xc6, 0xa9, 0x85, 0xef, 0x27, 0x60, 0x96, 0x37, - 0x27, 0xf6, 0x45, 0xcc, 0x63, 0xe5, 0x98, 0xa3, 0xcf, 0x03, 0x74, 0x2c, 0xbc, 0x2b, 0x33, 0xd6, - 0xe4, 0x58, 0xac, 0x59, 0xc2, 0x41, 0x09, 0xe8, 0x4b, 0x30, 0x4d, 0x06, 0x7a, 0xc7, 0x32, 0x3b, - 0xa6, 0x4d, 0xe2, 0x18, 0x7b, 0x3c, 0x68, 0x34, 0xf3, 0xf4, 0x70, 0xbe, 0xb0, 0xae, 0x1b, 0x9b, - 0x9c, 0xb1, 0x51, 0x97, 0x88, 0xc7, 0xf0, 0x1e, 0xdd, 0x01, 0xf8, 0x37, 0x02, 0x9c, 0xf8, 0xc4, - 0x96, 0x7d, 0xff, 0x23, 0x34, 0xe6, 0xcd, 0x43, 0x22, 0x7d, 0x5c, 0x35, 0xb6, 0xcd, 0xf8, 0x17, - 0xe3, 0xdf, 0x13, 0x60, 0x26, 0x20, 0x3e, 0xce, 0x28, 0xe7, 0x78, 0x27, 0x19, 0xbe, 0x4c, 0xe2, - 0x9e, 0xa0, 0xd9, 0xc7, 0x39, 0xa8, 0xfe, 0x2c, 0x01, 0xa7, 0x2a, 0x6c, 0x0f, 0xda, 0x4d, 0xd0, - 0x88, 0xd3, 0x4a, 0x4a, 0x30, 0xb5, 0x8b, 0x2d, 0x5b, 0x37, 0xd9, 0x7c, 0x5b, 0x90, 0xdc, 0x47, - 0x34, 0x07, 0x19, 0xdb, 0x50, 0x3a, 0xf6, 0x8e, 0xe9, 0xee, 0xe2, 0x79, 0xcf, 0x5e, 0x32, 0xc9, - 0xe4, 0xf1, 0x93, 0x49, 0xd2, 0xc3, 0x93, 0x49, 0xa6, 0x3e, 0x76, 0x32, 0x09, 0xdf, 0x32, 0xfb, - 0x91, 0x00, 0xa7, 0xfb, 0xf4, 0x17, 0xa7, 0xcd, 0x7c, 0x15, 0x72, 0x2a, 0x17, 0x4c, 0xbc, 0x31, - 0xdb, 0x15, 0x5c, 0x25, 0xc5, 0x8e, 0x89, 0x51, 0x9e, 0x1e, 0xce, 0x83, 0x5b, 0xd5, 0xd5, 0x15, - 0xae, 0x22, 0xf2, 0xbf, 0x56, 0xfe, 0xff, 0x39, 0x98, 0xae, 0xee, 0xb3, 0x35, 0xef, 0x3a, 0x0b, - 0x0b, 0xd0, 0x2d, 0xc8, 0x74, 0x2c, 0x73, 0x57, 0x77, 0x9b, 0x51, 0x0c, 0x85, 0x4c, 0x6e, 0x33, - 0x7a, 0xb8, 0x36, 0x39, 0x87, 0xe4, 0xf1, 0xa2, 0x06, 0x64, 0xef, 0x9a, 0xaa, 0xd2, 0xba, 0xa5, - 0xb7, 0x5c, 0xfb, 0x7f, 0x69, 0xb4, 0xa0, 0x45, 0x8f, 0x67, 0x53, 0x71, 0x76, 0xdc, 0xae, 0xf0, - 0x88, 0x68, 0x15, 0x32, 0x35, 0xc7, 0xe9, 0x90, 0x97, 0xdc, 0x9b, 0x5c, 0x1e, 0x43, 0x28, 0x61, - 0xe1, 0xb2, 0x3c, 0x76, 0xd4, 0x80, 0x99, 0xdb, 0xa6, 0xd9, 0x6c, 0xe1, 0x4a, 0xcb, 0xec, 0x6a, - 0x15, 0xd3, 0xd8, 0xd6, 0x9b, 0xdc, 0x1f, 0x5f, 0x1a, 0x43, 0xe6, 0xed, 0x4a, 0x5d, 0xea, 0x17, - 0x80, 0x96, 0x20, 0x53, 0xbf, 0xc9, 0x85, 0xb1, 0x38, 0xee, 0xe2, 0x18, 0xc2, 0xea, 0x37, 0x25, - 0x8f, 0x0d, 0xdd, 0x81, 0xdc, 0xd2, 0x93, 0xae, 0x85, 0xb9, 0x94, 0xf4, 0xc0, 0xcc, 0x85, 0x5e, - 0x29, 0x94, 0x4b, 0x0a, 0x32, 0xa3, 0x3a, 0x14, 0xdf, 0x36, 0xad, 0x47, 0x2d, 0x53, 0x71, 0x5b, - 0x38, 0x45, 0xc5, 0x7d, 0x66, 0x0c, 0x71, 0x2e, 0xa3, 0xd4, 0x23, 0x62, 0xee, 0x4b, 0x50, 0x08, - 0x75, 0x13, 0x42, 0x90, 0xea, 0x90, 0x1e, 0x11, 0x68, 0x2a, 0x10, 0xfd, 0x1f, 0xbd, 0x08, 0x53, - 0x86, 0xa9, 0x61, 0xd7, 0x86, 0x0b, 0xcb, 0x27, 0x9e, 0x1e, 0xce, 0xa7, 0x37, 0x4c, 0x8d, 0x05, - 0x14, 0xfc, 0x3f, 0x29, 0x4d, 0x0a, 0xb9, 0xe1, 0xc4, 0xdc, 0x25, 0x48, 0x91, 0xfe, 0x21, 0x6e, - 0xe4, 0xa1, 0x62, 0xe3, 0x2d, 0x4b, 0xe7, 0x32, 0xdd, 0x47, 0x5e, 0xee, 0xa7, 0x02, 0x24, 0xea, - 0x37, 0x49, 0x44, 0xfd, 0xb0, 0xab, 0x3e, 0xc2, 0x0e, 0x2f, 0xc5, 0x9f, 0x68, 0xa4, 0x6d, 0xe1, - 0x6d, 0x9d, 0x45, 0x39, 0x59, 0x89, 0x3f, 0xa1, 0xe7, 0x00, 0x14, 0x55, 0xc5, 0xb6, 0x2d, 0xbb, - 0x07, 0xc3, 0xb2, 0x52, 0x96, 0x51, 0xd6, 0xf0, 0x01, 0x61, 0xb3, 0xb1, 0x6a, 0x61, 0xc7, 0xcd, - 0x69, 0x62, 0x4f, 0x84, 0xcd, 0xc1, 0xed, 0x8e, 0xec, 0x98, 0x8f, 0xb0, 0x41, 0x7b, 0x35, 0x4b, - 0xdc, 0x43, 0xbb, 0xd3, 0x20, 0x04, 0xe2, 0xd9, 0xb0, 0xa1, 0xf9, 0x6e, 0x28, 0x2b, 0x79, 0xcf, - 0x44, 0xa4, 0x85, 0x9b, 0x3a, 0x3f, 0xe9, 0x94, 0x95, 0xf8, 0x13, 0xd1, 0x98, 0xd2, 0x75, 0x76, - 0x68, 0x22, 0x47, 0x56, 0xa2, 0xff, 0xf3, 0xa6, 0x7d, 0x53, 0x80, 0xe4, 0xed, 0x4a, 0xfd, 0xc8, - 0x6d, 0x73, 0x25, 0x26, 0x7d, 0x89, 0x34, 0x95, 0x50, 0x6f, 0xb5, 0x74, 0xa3, 0x49, 0x82, 0x8e, - 0xaf, 0x62, 0xd5, 0x6d, 0x59, 0x91, 0x93, 0x37, 0x19, 0x15, 0x2d, 0x40, 0x4e, 0xb5, 0xb0, 0x86, - 0x0d, 0x47, 0x57, 0x5a, 0x36, 0x6f, 0x62, 0x90, 0xc4, 0x2b, 0xf7, 0x35, 0x01, 0x26, 0xa9, 0x79, - 0xa1, 0x67, 0x21, 0xab, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0xf7, 0x13, 0x59, 0xc9, 0x27, 0x0c, 0xac, - 0xe4, 0x39, 0xc8, 0x2b, 0xaa, 0x6a, 0x76, 0x0d, 0x47, 0x36, 0x94, 0x36, 0xe6, 0x95, 0xcd, 0x71, - 0xda, 0x86, 0xd2, 0xc6, 0x68, 0x1e, 0xdc, 0x47, 0xef, 0x78, 0x5e, 0x56, 0x02, 0x4e, 0x5a, 0xc3, - 0x07, 0xbc, 0x26, 0x3f, 0x12, 0x20, 0xe3, 0x9a, 0x25, 0xa9, 0x4c, 0x13, 0x1b, 0xd8, 0x52, 0x1c, - 0xd3, 0xab, 0x8c, 0x47, 0xe8, 0x9d, 0x93, 0xb2, 0xfe, 0x9c, 0x74, 0x02, 0x26, 0x1d, 0xe5, 0x61, - 0xcb, 0xad, 0x07, 0x7b, 0xa0, 0xcb, 0xc7, 0x2d, 0xa5, 0xc9, 0xd6, 0xc8, 0xb2, 0x12, 0x7b, 0x20, - 0x4d, 0xe2, 0xb9, 0xab, 0x4c, 0x3b, 0xfc, 0x89, 0xd4, 0x97, 0xa5, 0x6b, 0x3e, 0xc4, 0x4d, 0xdd, - 0xa0, 0x06, 0x90, 0x94, 0x80, 0x92, 0x96, 0x09, 0x05, 0x3d, 0x03, 0x59, 0x56, 0x00, 0x1b, 0x1a, - 0xb5, 0x82, 0xa4, 0x94, 0xa1, 0x84, 0xaa, 0x7b, 0x1e, 0x89, 0x07, 0x22, 0xdf, 0x13, 0x60, 0x86, - 0xe5, 0xdc, 0xb0, 0x3c, 0xcf, 0xf8, 0x66, 0xe5, 0xd7, 0x20, 0xab, 0x29, 0x8e, 0xc2, 0x8e, 0x03, - 0x26, 0x86, 0x1e, 0x07, 0x74, 0xdd, 0x24, 0x29, 0x4f, 0x8f, 0x04, 0x22, 0x48, 0x91, 0xff, 0xd9, - 0x31, 0x4a, 0x89, 0xfe, 0xef, 0x67, 0x31, 0x04, 0xab, 0x1b, 0x67, 0x94, 0xf2, 0xe1, 0x24, 0x14, - 0xaa, 0xfb, 0x1d, 0xd3, 0x8a, 0x79, 0x11, 0x6d, 0x8a, 0x83, 0xdf, 0x21, 0xdb, 0xac, 0x3d, 0x5e, - 0xd0, 0xdd, 0xc1, 0xe4, 0x8c, 0x68, 0x19, 0x80, 0x25, 0x58, 0xd2, 0x3c, 0x9c, 0xe4, 0x11, 0x36, - 0x9b, 0x28, 0x1b, 0xa1, 0xa2, 0x0d, 0xc8, 0xb5, 0x77, 0x55, 0x55, 0xde, 0xd6, 0x5b, 0x0e, 0x4f, - 0x4d, 0x8b, 0xce, 0x85, 0x5e, 0xbf, 0x5f, 0xa9, 0xdc, 0xa2, 0x85, 0x58, 0x86, 0x98, 0xff, 0x2c, - 0x01, 0x91, 0xc0, 0xfe, 0x47, 0x2f, 0x00, 0x3f, 0xbe, 0x21, 0xdb, 0xee, 0x61, 0xac, 0xe5, 0xc2, - 0xd3, 0xc3, 0xf9, 0xac, 0x44, 0xa9, 0xf5, 0x7a, 0x43, 0xca, 0xb2, 0x02, 0x75, 0xdb, 0x41, 0xe7, - 0xa1, 0x60, 0xb6, 0x75, 0x47, 0x76, 0xe3, 0x00, 0x1e, 0x3a, 0xe5, 0x09, 0xd1, 0x8d, 0x13, 0x50, - 0x03, 0x2e, 0x63, 0x83, 0x8c, 0x06, 0xda, 0x4e, 0xf9, 0x21, 0x5b, 0xab, 0x73, 0xd8, 0x88, 0x92, - 0xcd, 0x8e, 0xa3, 0xb7, 0xf5, 0x27, 0x74, 0xa3, 0x97, 0x6f, 0xb2, 0x9c, 0x67, 0xc5, 0x49, 0xfb, - 0x96, 0xe9, 0x22, 0x1e, 0x2f, 0x7b, 0x2f, 0x50, 0x14, 0x7d, 0x4d, 0x80, 0x53, 0x5c, 0x91, 0xf2, - 0xc3, 0x03, 0xb9, 0x45, 0x26, 0x11, 0xdd, 0x39, 0x90, 0x1f, 0xed, 0x96, 0x32, 0x34, 0x40, 0xfb, - 0x5c, 0x64, 0x87, 0x04, 0xec, 0x60, 0xd1, 0xed, 0x96, 0x83, 0xbb, 0x9c, 0x79, 0x6d, 0xb7, 0x6a, - 0x38, 0xd6, 0xc1, 0xf2, 0xe9, 0xa7, 0x87, 0xf3, 0xb3, 0xfd, 0x6f, 0xef, 0x4b, 0xb3, 0x76, 0x3f, - 0xcb, 0xdc, 0x57, 0xa1, 0x34, 0x48, 0x12, 0x12, 0xfd, 0x9d, 0xc6, 0x2c, 0xdb, 0x60, 0x7c, 0x35, - 0xbc, 0x42, 0x30, 0x86, 0xd9, 0xb8, 0xab, 0x04, 0x89, 0x57, 0xdd, 0xd1, 0xfd, 0x0d, 0x01, 0x0a, - 0xcb, 0xdd, 0xd6, 0xa3, 0x7b, 0x9d, 0x7a, 0xb7, 0xdd, 0x56, 0xac, 0x03, 0xe2, 0x18, 0xd8, 0xa8, - 0xd4, 0x9f, 0xb0, 0x14, 0x95, 0x24, 0x1f, 0x76, 0xfa, 0x13, 0x4c, 0x86, 0x1d, 0x4f, 0x40, 0x27, - 0x74, 0x96, 0x5d, 0x7e, 0x1e, 0x0a, 0x14, 0xae, 0xcb, 0xd8, 0x70, 0x2c, 0x1d, 0xb3, 0x55, 0xa8, - 0xa4, 0x94, 0xa7, 0xc4, 0x2a, 0xa3, 0xa1, 0x8b, 0x50, 0xb4, 0x0f, 0x6c, 0x07, 0xb7, 0x65, 0x76, - 0x78, 0x9b, 0x61, 0xcc, 0xa4, 0x54, 0x60, 0x54, 0x89, 0x11, 0xcb, 0x3f, 0x4b, 0x42, 0xd1, 0xd5, - 0x6c, 0x9c, 0xe1, 0xeb, 0x32, 0x4c, 0x6e, 0xeb, 0x2d, 0xec, 0x66, 0x09, 0x5c, 0x1a, 0xd2, 0xa1, - 0x3c, 0xb1, 0x98, 0x44, 0x11, 0x2e, 0x00, 0xa2, 0xac, 0x71, 0x8c, 0xb1, 0xb9, 0xff, 0x9d, 0x80, - 0x14, 0x8d, 0x18, 0xaf, 0x43, 0x8a, 0xba, 0x3d, 0x61, 0x1c, 0xb7, 0x47, 0x8b, 0x7a, 0xe1, 0x4c, - 0x22, 0x10, 0xce, 0x90, 0xd8, 0x60, 0x47, 0x79, 0xf9, 0xfa, 0x0d, 0x3a, 0xbe, 0xf2, 0x12, 0x7f, - 0x42, 0xcb, 0x34, 0x7d, 0xc5, 0xb4, 0x1c, 0xac, 0xf1, 0x48, 0x2d, 0xca, 0x33, 0x85, 0x3a, 0xde, - 0x75, 0xb1, 0x2e, 0x1f, 0x3a, 0x03, 0x49, 0x32, 0x70, 0xa7, 0xd8, 0xd6, 0xf6, 0xd3, 0xc3, 0xf9, - 0x24, 0x19, 0xb2, 0x84, 0x86, 0xae, 0x41, 0x2e, 0x3c, 0x4a, 0x84, 0x2b, 0x59, 0xe6, 0x0b, 0x02, - 0x16, 0x0e, 0x2d, 0xcf, 0x82, 0x19, 0x4a, 0xb9, 0x93, 0xca, 0xa4, 0xc4, 0xc9, 0xf2, 0x1f, 0xa5, - 0xa0, 0xb0, 0xda, 0x8e, 0xdb, 0x8b, 0x2e, 0x85, 0x7b, 0x38, 0x2a, 0xbc, 0x0d, 0x7d, 0x34, 0xa2, - 0x83, 0x43, 0xf3, 0x51, 0xf2, 0x68, 0xf3, 0xd1, 0x2a, 0x89, 0xa8, 0xf8, 0x01, 0xf5, 0xe4, 0x80, - 0x48, 0x36, 0xfc, 0xfd, 0x06, 0x71, 0x4c, 0x12, 0xe1, 0xf1, 0x53, 0xed, 0x69, 0x7a, 0xc2, 0x9b, - 0x34, 0x70, 0x63, 0x56, 0x96, 0x1e, 0xdf, 0xca, 0xa6, 0xb0, 0xa1, 0x51, 0x1b, 0x7b, 0xc2, 0x4d, - 0xec, 0x35, 0x48, 0x6a, 0xba, 0x35, 0xe4, 0xb6, 0x83, 0xe8, 0x39, 0x85, 0x30, 0x8d, 0xb0, 0xb5, - 0x54, 0xd0, 0xd6, 0x82, 0x38, 0x74, 0xee, 0x1e, 0x80, 0xdf, 0x2e, 0xb4, 0x00, 0x69, 0xb3, 0xa5, - 0xb9, 0xe7, 0x04, 0x0a, 0xcb, 0xd9, 0xa7, 0x87, 0xf3, 0x93, 0xf7, 0x5a, 0xda, 0xea, 0x8a, 0x34, - 0x69, 0xb6, 0xb4, 0x55, 0x8d, 0x9e, 0xec, 0xc7, 0x7b, 0xb2, 0x97, 0x63, 0x94, 0x97, 0xa6, 0x0c, - 0xbc, 0x47, 0x50, 0x6f, 0x4f, 0xee, 0x03, 0x31, 0x9c, 0xef, 0x08, 0x50, 0x74, 0x75, 0x18, 0xaf, - 0x73, 0xc8, 0xe8, 0x6d, 0x3e, 0x58, 0x92, 0x47, 0x1b, 0x2c, 0x2e, 0x1f, 0x3f, 0x7f, 0xf8, 0x75, - 0x81, 0xe7, 0x97, 0xd6, 0x55, 0xc5, 0x21, 0xf3, 0x61, 0x8c, 0x06, 0xfe, 0x3c, 0x88, 0x96, 0x62, - 0x68, 0x66, 0x5b, 0x7f, 0x82, 0xd9, 0xc2, 0x95, 0xcd, 0xf7, 0xa7, 0xa6, 0x3d, 0x3a, 0x5d, 0x99, - 0x71, 0xd7, 0xdd, 0x7e, 0x29, 0xf0, 0x5c, 0x54, 0xaf, 0x32, 0x71, 0x2a, 0x6d, 0x0d, 0xd2, 0x16, - 0xcb, 0x68, 0x63, 0x03, 0xee, 0xc5, 0x08, 0x21, 0x51, 0x5f, 0x67, 0x09, 0x63, 0x9e, 0xc9, 0x53, - 0x11, 0x73, 0x5f, 0x84, 0x49, 0x4a, 0x3e, 0x86, 0x5b, 0xe4, 0x9a, 0xff, 0x87, 0x04, 0x5c, 0xa0, - 0x9f, 0xbb, 0x8f, 0x2d, 0x7d, 0xfb, 0x60, 0xd3, 0x32, 0x1d, 0xac, 0x3a, 0x58, 0xf3, 0xf3, 0xf1, - 0x63, 0xf5, 0x35, 0xd9, 0x8e, 0xfb, 0x81, 0x23, 0x65, 0xf6, 0x78, 0x5c, 0x68, 0x0d, 0xa6, 0xd9, - 0x44, 0x28, 0x2b, 0x2d, 0x7d, 0x17, 0xcb, 0x8a, 0x73, 0x94, 0x19, 0xa5, 0xc0, 0x78, 0x97, 0x08, - 0xeb, 0x92, 0x83, 0x34, 0xc8, 0x72, 0x61, 0xba, 0xc6, 0x6f, 0x0e, 0xb9, 0xfd, 0xf1, 0x96, 0x66, - 0x32, 0x6c, 0x42, 0x5e, 0x5d, 0x91, 0x32, 0x4c, 0xb2, 0xb7, 0xb4, 0xfe, 0x13, 0x01, 0x2e, 0x8e, - 0x50, 0x74, 0x9c, 0x66, 0x36, 0x07, 0x99, 0x5d, 0xf2, 0x21, 0x9d, 0x6b, 0x3a, 0x23, 0x79, 0xcf, - 0x68, 0x1d, 0x0a, 0xdb, 0x8a, 0xde, 0xc2, 0x9a, 0xcc, 0x2d, 0x71, 0x70, 0x3a, 0x58, 0x74, 0x96, - 0x62, 0x9e, 0xb1, 0xd3, 0x97, 0x76, 0xf9, 0xb7, 0x13, 0x30, 0xb3, 0xa4, 0x69, 0xf5, 0x3a, 0xf7, - 0x60, 0xf1, 0xd9, 0x8b, 0x0b, 0x56, 0x12, 0x3e, 0x58, 0x41, 0x2f, 0x02, 0xd2, 0x74, 0x9b, 0x5d, - 0x5d, 0x60, 0xef, 0x28, 0x9a, 0xb9, 0xe7, 0x6f, 0x7c, 0xcf, 0xb8, 0x6f, 0xea, 0xee, 0x0b, 0xf4, - 0x0e, 0xd0, 0xd0, 0x5a, 0xb6, 0x1d, 0xc5, 0x5b, 0x9f, 0x7f, 0xe1, 0x28, 0xa7, 0x68, 0x58, 0xe8, - 0xed, 0x3d, 0x4a, 0x59, 0x22, 0x8e, 0xfe, 0x8b, 0xae, 0x80, 0xa8, 0x13, 0x0d, 0x38, 0xb2, 0x62, - 0xbb, 0x07, 0x31, 0xd8, 0xdd, 0x09, 0x45, 0x46, 0x5f, 0xb2, 0x83, 0xe7, 0x2b, 0x58, 0x9e, 0xb8, - 0xaf, 0xa7, 0x38, 0x11, 0xd6, 0xef, 0x0a, 0x50, 0xe4, 0x87, 0x47, 0xe2, 0xec, 0x80, 0x5b, 0x90, - 0x77, 0x4f, 0xaa, 0x6c, 0x5b, 0x66, 0xfb, 0x28, 0x43, 0x2d, 0xc7, 0x19, 0x6f, 0x59, 0x66, 0x3b, - 0x74, 0xd6, 0xfc, 0x3e, 0x4c, 0x7b, 0x35, 0x8d, 0x53, 0x05, 0xdf, 0xa3, 0x07, 0x48, 0x99, 0xe0, - 0xb8, 0xb7, 0x9e, 0x3f, 0x09, 0x3d, 0xd0, 0x7d, 0x81, 0x60, 0x75, 0xe3, 0x54, 0xc6, 0x2f, 0x05, - 0x28, 0xd6, 0xbb, 0x0f, 0xd9, 0x35, 0x3a, 0xf1, 0xe9, 0xa1, 0x0a, 0xd9, 0x16, 0xde, 0x76, 0xe4, - 0x63, 0xe5, 0x36, 0x67, 0x08, 0x2b, 0xcd, 0xef, 0xbe, 0x0d, 0x60, 0xd1, 0xd3, 0x4b, 0x54, 0x4e, - 0xf2, 0x88, 0x72, 0xb2, 0x94, 0xd7, 0x8f, 0x75, 0xca, 0x7f, 0x98, 0x80, 0x69, 0xaf, 0xb1, 0x71, - 0x3a, 0xd1, 0xff, 0x11, 0x72, 0x1e, 0xc9, 0x63, 0x38, 0x8f, 0x19, 0xbe, 0xe5, 0x1e, 0xed, 0x40, - 0x16, 0x61, 0x96, 0x06, 0x24, 0xb2, 0xd2, 0xe9, 0xb4, 0x74, 0xac, 0xc9, 0x6c, 0x2f, 0x37, 0x45, - 0xf7, 0x72, 0x67, 0xe8, 0xab, 0x25, 0xf6, 0x66, 0x95, 0xee, 0xeb, 0xde, 0x82, 0xfc, 0xb6, 0x85, - 0xf1, 0x13, 0x2c, 0x53, 0x64, 0x74, 0x94, 0xcc, 0x82, 0x1c, 0x63, 0xac, 0x13, 0x3e, 0x6e, 0x80, - 0xef, 0xc2, 0x0c, 0x55, 0x70, 0xdc, 0x27, 0x27, 0x79, 0xaf, 0xfc, 0xa3, 0x00, 0x28, 0x28, 0xff, - 0x93, 0xeb, 0x98, 0x44, 0xec, 0x1d, 0xf3, 0x02, 0x20, 0x96, 0x64, 0x66, 0xcb, 0x1d, 0x6c, 0xc9, - 0x36, 0x56, 0x4d, 0x7e, 0x03, 0x8c, 0x20, 0x89, 0xfc, 0xcd, 0x26, 0xb6, 0xea, 0x94, 0x5e, 0x7e, - 0x6f, 0x0e, 0xf2, 0x5c, 0x27, 0x5b, 0x86, 0x6e, 0x1a, 0xe8, 0x3a, 0x24, 0x9b, 0x7c, 0xa5, 0x37, - 0x17, 0xb9, 0x12, 0xe4, 0xdf, 0x42, 0x55, 0x9b, 0x90, 0x48, 0x59, 0xc2, 0xd2, 0xe9, 0x3a, 0x11, - 0x41, 0x91, 0x9f, 0x1b, 0x1b, 0x64, 0xe9, 0x74, 0x1d, 0x54, 0x87, 0x69, 0xd5, 0xbf, 0x85, 0x47, - 0x26, 0xec, 0xc9, 0x81, 0x9b, 0x0b, 0x91, 0x77, 0x19, 0xd5, 0x26, 0xa4, 0xa2, 0x1a, 0x7a, 0x81, - 0x2a, 0xc1, 0x6b, 0x5f, 0xd8, 0x74, 0x79, 0x3e, 0xf2, 0x94, 0x65, 0xf8, 0xca, 0x99, 0xda, 0x44, - 0xe0, 0x76, 0x18, 0xf4, 0x1a, 0xa4, 0x35, 0x7a, 0x9d, 0x08, 0xb7, 0xd0, 0x28, 0x23, 0x0a, 0xdd, - 0xe0, 0x52, 0x9b, 0x90, 0x38, 0x07, 0xba, 0x03, 0x79, 0xf6, 0x1f, 0x0b, 0x4e, 0x38, 0x92, 0xbb, - 0x38, 0x58, 0x42, 0xc0, 0xd7, 0xd7, 0x26, 0xa4, 0x9c, 0xe6, 0x53, 0xd1, 0x67, 0x21, 0x65, 0xab, - 0x8a, 0xc1, 0x37, 0x49, 0xce, 0x0e, 0xb8, 0x9e, 0xc0, 0x67, 0xa6, 0xa5, 0xd1, 0xeb, 0xec, 0x36, - 0x39, 0x67, 0x9f, 0x1d, 0xbc, 0x8c, 0xae, 0x7e, 0xe8, 0x00, 0x2d, 0xa9, 0x3e, 0xa6, 0x04, 0x74, - 0x1b, 0x72, 0x0a, 0x89, 0xf2, 0x64, 0x7a, 0x8c, 0xad, 0x04, 0x03, 0x0f, 0x1a, 0xf5, 0x1d, 0x41, - 0xac, 0xd1, 0x53, 0xba, 0x2e, 0xd1, 0x17, 0xd4, 0xc6, 0x56, 0x13, 0x97, 0x72, 0xc3, 0x05, 0x05, - 0xb3, 0x68, 0x3c, 0x41, 0x94, 0x48, 0xa2, 0xbd, 0x1d, 0xf7, 0x88, 0x02, 0x6d, 0x54, 0x7e, 0xe0, - 0xa6, 0x58, 0xc4, 0x11, 0x8b, 0xda, 0x84, 0x94, 0xdf, 0x09, 0x90, 0xd1, 0x22, 0x24, 0x9a, 0x6a, - 0xa9, 0x40, 0x65, 0x3c, 0x3b, 0xec, 0x00, 0x41, 0x6d, 0x42, 0x4a, 0x34, 0x55, 0x82, 0xca, 0x59, - 0xea, 0xf7, 0xbe, 0x51, 0x2a, 0x0e, 0x1c, 0xf1, 0xe1, 0x3c, 0xfa, 0xda, 0x84, 0x44, 0xb3, 0xcd, - 0xc9, 0xf7, 0x36, 0xa1, 0x68, 0xb1, 0x34, 0x24, 0x37, 0xb9, 0x50, 0x1c, 0xb8, 0x51, 0x18, 0x95, - 0x5f, 0x58, 0xa3, 0x51, 0x7f, 0x80, 0x8e, 0xbe, 0x02, 0x27, 0xc2, 0x12, 0xb9, 0xa5, 0xcd, 0xf4, - 0xb9, 0x91, 0x68, 0xb9, 0x61, 0x83, 0x43, 0x56, 0xdf, 0x4b, 0xf4, 0x0a, 0x4c, 0xb2, 0x5e, 0x43, - 0x54, 0x64, 0xd4, 0x0e, 0x78, 0x4f, 0x87, 0xb1, 0xf2, 0xc4, 0xf8, 0x1d, 0x9e, 0x7f, 0x23, 0xb7, - 0xcc, 0x66, 0x69, 0x76, 0xa0, 0xf1, 0xf7, 0xe7, 0x13, 0x11, 0xe3, 0x77, 0x7c, 0x2a, 0xe9, 0x77, - 0x8b, 0xbd, 0xe1, 0xe9, 0x1a, 0x27, 0x06, 0xf6, 0x7b, 0x44, 0x5a, 0x4e, 0x8d, 0xe6, 0x4e, 0xfb, - 0x64, 0x52, 0x35, 0x8b, 0x5d, 0xcf, 0x21, 0xd3, 0x31, 0x75, 0x72, 0x60, 0xd5, 0xfa, 0xaf, 0x1c, - 0xa9, 0xd1, 0x30, 0xc8, 0xa3, 0xa2, 0xfb, 0x20, 0xf2, 0x83, 0xf3, 0xfe, 0xb2, 0xf5, 0x29, 0x2a, - 0xef, 0xf9, 0x48, 0xd7, 0x15, 0x95, 0xdf, 0x50, 0x9b, 0x90, 0xa6, 0xd5, 0xf0, 0x1b, 0xf4, 0x00, - 0x66, 0xa8, 0x3c, 0x59, 0xf5, 0x6f, 0x3c, 0x28, 0x95, 0xfa, 0x12, 0x05, 0x07, 0x5f, 0x8e, 0xe0, - 0x4a, 0x16, 0xd5, 0x9e, 0x57, 0xc4, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x97, 0x9d, 0x1b, 0x68, 0xc6, - 0xe1, 0x5b, 0xcb, 0x88, 0x19, 0xeb, 0x8c, 0x42, 0xcc, 0xd8, 0xe1, 0xb9, 0x3c, 0xbc, 0x3b, 0x9e, - 0x1d, 0x68, 0xc6, 0x51, 0x49, 0x3f, 0xc4, 0x8c, 0x9d, 0x20, 0x9d, 0x98, 0x31, 0x73, 0x10, 0x3d, - 0x72, 0x9f, 0x1b, 0x68, 0xc6, 0x03, 0x0f, 0x92, 0x12, 0x33, 0x56, 0xfa, 0x5e, 0xa2, 0x15, 0x00, - 0x16, 0x9e, 0xe8, 0xc6, 0xb6, 0x59, 0x3a, 0x3b, 0x70, 0x32, 0xe8, 0xcd, 0xe6, 0x21, 0x93, 0x41, - 0xcb, 0xa5, 0x11, 0x47, 0x46, 0xb1, 0x91, 0x4c, 0x77, 0xc9, 0x4a, 0xf3, 0x03, 0x1d, 0x59, 0xdf, - 0x66, 0x19, 0x71, 0x64, 0x7b, 0x1e, 0x91, 0xcc, 0x2a, 0x6c, 0x8d, 0xb5, 0xb4, 0x30, 0xd8, 0x2d, - 0x07, 0x77, 0x17, 0xa8, 0x5b, 0xa6, 0x04, 0xb4, 0x04, 0x59, 0x32, 0x6d, 0x1f, 0x50, 0x37, 0x74, - 0x6e, 0x60, 0xc0, 0xd9, 0x73, 0x38, 0xa0, 0x36, 0x21, 0x65, 0x1e, 0x73, 0x12, 0xf9, 0x3c, 0x5b, - 0xb5, 0x2a, 0x95, 0x07, 0x7e, 0x3e, 0xb4, 0x52, 0x49, 0x3e, 0xcf, 0x38, 0x90, 0x0a, 0x27, 0x59, - 0x5f, 0xf1, 0x73, 0x9c, 0x16, 0x3f, 0x74, 0x58, 0x3a, 0x4f, 0x45, 0x0d, 0x5c, 0x03, 0x8a, 0x3c, - 0x5e, 0x5a, 0x9b, 0x90, 0x66, 0x95, 0xfe, 0xb7, 0x64, 0xc0, 0xf3, 0xa9, 0x87, 0xad, 0x1c, 0x95, - 0x2e, 0x0c, 0x1c, 0xf0, 0x11, 0x6b, 0x6d, 0x64, 0xc0, 0x2b, 0x01, 0x32, 0x9b, 0x80, 0x34, 0xd9, - 0xb6, 0xd9, 0x9e, 0xea, 0xc5, 0x21, 0x13, 0x50, 0x0f, 0xf6, 0x67, 0x13, 0x90, 0x56, 0x67, 0x9c, - 0x44, 0x90, 0xda, 0xc2, 0x8a, 0xc5, 0xdd, 0xec, 0xa5, 0x81, 0x82, 0xfa, 0x6e, 0x02, 0x23, 0x82, - 0x54, 0x8f, 0x48, 0x02, 0x1e, 0xcb, 0xbd, 0x71, 0x83, 0x87, 0x7e, 0x97, 0x07, 0x06, 0x3c, 0x91, - 0x17, 0x83, 0x90, 0x80, 0xc7, 0x0a, 0xbd, 0x40, 0x9f, 0x87, 0x29, 0x8e, 0xd0, 0x4a, 0x57, 0x86, - 0x04, 0xa4, 0x41, 0x68, 0x4d, 0xc6, 0x35, 0xe7, 0x61, 0x5e, 0x96, 0x21, 0x43, 0xd6, 0xbc, 0xe7, - 0x87, 0x78, 0xd9, 0x3e, 0x70, 0xca, 0xbc, 0xac, 0x4f, 0x26, 0x5e, 0x96, 0xd9, 0x29, 0x9f, 0xeb, - 0xae, 0x0e, 0xf4, 0xb2, 0xfd, 0x27, 0x13, 0x88, 0x97, 0x7d, 0xec, 0x53, 0x49, 0xcb, 0x6c, 0x86, - 0x8a, 0x4a, 0x9f, 0x19, 0xd8, 0xb2, 0x30, 0x48, 0x24, 0x2d, 0xe3, 0x3c, 0xa4, 0xdb, 0x58, 0x12, - 0x29, 0xd3, 0xf4, 0x0b, 0x83, 0x8f, 0x4c, 0xf7, 0x82, 0x88, 0x9a, 0x7b, 0x4b, 0x2c, 0xd3, 0xb0, - 0xe7, 0xa8, 0x2c, 0x7e, 0x40, 0x94, 0x6b, 0xea, 0xc5, 0xe1, 0x8e, 0x2a, 0xea, 0xec, 0xab, 0xe7, - 0xa8, 0x42, 0x2f, 0x69, 0x55, 0xd9, 0x39, 0x1f, 0x3a, 0xbe, 0x17, 0x87, 0x9c, 0xee, 0xee, 0x39, - 0x7a, 0x45, 0xab, 0xea, 0x11, 0xfd, 0x21, 0xd4, 0x65, 0xd7, 0x10, 0x94, 0xae, 0x0d, 0x1f, 0x42, - 0xe1, 0xeb, 0x10, 0xbc, 0x21, 0xc4, 0xc9, 0xde, 0x9c, 0xe9, 0x46, 0x18, 0x2f, 0x0d, 0x9f, 0x33, - 0x7b, 0x43, 0x0b, 0x36, 0x67, 0xf2, 0x98, 0xe2, 0xff, 0x08, 0xb0, 0xc0, 0xea, 0x46, 0xd7, 0xf1, - 0x0e, 0x64, 0x6f, 0x4d, 0x34, 0x90, 0x6a, 0x7e, 0x9d, 0x7e, 0xe0, 0x95, 0x41, 0xd5, 0x1d, 0xb1, - 0xc6, 0x5b, 0x9b, 0x90, 0x9e, 0x53, 0x86, 0x95, 0x5b, 0x9e, 0xe2, 0x3b, 0xa7, 0xde, 0x89, 0xbb, - 0x69, 0x51, 0xbc, 0x93, 0xca, 0x9c, 0x16, 0x4b, 0x77, 0x52, 0x99, 0x33, 0xe2, 0xdc, 0x9d, 0x54, - 0xe6, 0x19, 0xf1, 0xd9, 0xf2, 0x3f, 0x9d, 0x81, 0x82, 0x8b, 0xe1, 0x18, 0x22, 0xba, 0x11, 0x44, - 0x44, 0x67, 0x07, 0x21, 0x22, 0x8e, 0xfa, 0x38, 0x24, 0xba, 0x11, 0x84, 0x44, 0x67, 0x07, 0x41, - 0x22, 0x9f, 0x87, 0x60, 0xa2, 0xc6, 0x20, 0x4c, 0xf4, 0xfc, 0x18, 0x98, 0xc8, 0x13, 0xd5, 0x0b, - 0x8a, 0x56, 0xfa, 0x41, 0xd1, 0x85, 0xe1, 0xa0, 0xc8, 0x13, 0x15, 0x40, 0x45, 0xaf, 0xf7, 0xa0, - 0xa2, 0x73, 0x43, 0x50, 0x91, 0xc7, 0xef, 0xc2, 0xa2, 0xb5, 0x48, 0x58, 0x74, 0x69, 0x14, 0x2c, - 0xf2, 0xe4, 0x84, 0x70, 0xd1, 0xcb, 0x21, 0x5c, 0x34, 0x3f, 0x10, 0x17, 0x79, 0xdc, 0x0c, 0x18, - 0xbd, 0xd1, 0x0b, 0x8c, 0xce, 0x0d, 0x01, 0x46, 0x7e, 0x0b, 0x38, 0x32, 0xaa, 0x45, 0x21, 0xa3, - 0x8b, 0x23, 0x90, 0x91, 0x27, 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xe2, 0x08, 0x68, 0xd4, - 0x23, 0x89, 0x61, 0xa3, 0x8d, 0x68, 0x6c, 0x74, 0x79, 0x24, 0x36, 0xf2, 0xa4, 0x85, 0xc1, 0xd1, - 0xb5, 0x00, 0x38, 0x7a, 0x6e, 0x00, 0x38, 0xf2, 0x58, 0x09, 0x3a, 0xfa, 0x42, 0x1f, 0x3a, 0x2a, - 0x0f, 0x43, 0x47, 0x1e, 0xaf, 0x07, 0x8f, 0xde, 0x1a, 0x00, 0x8f, 0xae, 0x8c, 0x86, 0x47, 0x9e, - 0xb0, 0x1e, 0x7c, 0xa4, 0x0c, 0xc5, 0x47, 0x2f, 0x8e, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, 0xe9, - 0xd5, 0x30, 0x40, 0x5a, 0x18, 0x0c, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0xad, 0x45, 0x22, 0xa4, 0x4b, - 0xa3, 0x10, 0x92, 0x3f, 0x0e, 0x82, 0x10, 0x69, 0x23, 0x1a, 0x22, 0x5d, 0x1e, 0x09, 0x91, 0xfc, - 0xee, 0x0f, 0x61, 0xa4, 0xb5, 0x48, 0x8c, 0x74, 0x69, 0x14, 0x46, 0xf2, 0x2b, 0x17, 0x04, 0x49, - 0x6f, 0x0f, 0x04, 0x49, 0x57, 0xc7, 0x01, 0x49, 0x9e, 0xd0, 0x3e, 0x94, 0xf4, 0xce, 0x60, 0x94, - 0xf4, 0x99, 0x23, 0x5c, 0x21, 0x17, 0x09, 0x93, 0xbe, 0xd0, 0x07, 0x93, 0xca, 0xc3, 0x60, 0x92, - 0x6f, 0xcf, 0x2e, 0x4e, 0x52, 0x86, 0xa2, 0x9a, 0x17, 0xc7, 0x44, 0x35, 0xbe, 0xf1, 0x45, 0xc0, - 0x9a, 0x6a, 0x04, 0xac, 0xb9, 0x30, 0x1c, 0xd6, 0xf8, 0xee, 0xdc, 0xc7, 0x35, 0xb5, 0x28, 0x5c, - 0x73, 0x71, 0x04, 0xae, 0xf1, 0xbd, 0x50, 0x00, 0xd8, 0xbc, 0xde, 0x03, 0x6c, 0xce, 0x8d, 0xcc, - 0xb2, 0x09, 0x20, 0x9b, 0xe5, 0x7e, 0x64, 0x73, 0x7e, 0x28, 0xb2, 0xf1, 0x24, 0xf8, 0xd0, 0xe6, - 0xf5, 0x1e, 0x68, 0x73, 0x6e, 0x08, 0xb4, 0xf1, 0x2b, 0xc0, 0xb1, 0x8d, 0x36, 0x1c, 0xdb, 0x2c, - 0x8e, 0x8b, 0x6d, 0x3c, 0xc1, 0x91, 0xe0, 0x66, 0x23, 0x1a, 0xdc, 0x5c, 0x1e, 0x73, 0xf7, 0xbc, - 0x0f, 0xdd, 0xd4, 0xa2, 0xd0, 0xcd, 0xc5, 0x11, 0xe8, 0x26, 0x38, 0x87, 0x78, 0xf0, 0xa6, 0x16, - 0x05, 0x6f, 0x2e, 0x8e, 0x80, 0x37, 0xbe, 0xa4, 0x00, 0xbe, 0x69, 0x0c, 0xc2, 0x37, 0xcf, 0x8f, - 0x81, 0x6f, 0xfc, 0xe0, 0xa5, 0x07, 0xe0, 0xbc, 0xd9, 0x0b, 0x70, 0xca, 0xc3, 0x00, 0x8e, 0x3f, - 0x22, 0x5d, 0x84, 0xb3, 0x11, 0x8d, 0x70, 0x2e, 0x8f, 0x44, 0x38, 0x41, 0x27, 0x19, 0x80, 0x38, - 0x6b, 0x91, 0x10, 0xe7, 0xd2, 0x28, 0x88, 0xe3, 0x3b, 0xc9, 0x20, 0xc6, 0x79, 0xb3, 0x17, 0xe3, - 0x94, 0x87, 0x61, 0x1c, 0xbf, 0x71, 0x2e, 0xc8, 0xa9, 0x45, 0x81, 0x9c, 0x8b, 0x23, 0x40, 0x8e, - 0xdf, 0x79, 0x01, 0x94, 0xa3, 0x0c, 0x45, 0x39, 0x2f, 0x8e, 0x89, 0x72, 0x7a, 0x1c, 0x57, 0x18, - 0xe6, 0xd4, 0xa2, 0x60, 0xce, 0xc5, 0x11, 0x30, 0x27, 0x50, 0x59, 0x1f, 0xe7, 0x6c, 0x44, 0xe3, - 0x9c, 0xcb, 0x23, 0x71, 0x4e, 0xcf, 0x68, 0x72, 0x81, 0xce, 0x5a, 0x24, 0xd0, 0xb9, 0x34, 0x0a, - 0xe8, 0xf4, 0x4c, 0x7c, 0x3c, 0x38, 0xf8, 0xbf, 0xe3, 0x23, 0x9d, 0x57, 0x8f, 0x8e, 0x74, 0xbc, - 0x6f, 0xc6, 0x02, 0x75, 0xee, 0xa4, 0x32, 0xcf, 0x8a, 0xcf, 0x95, 0x7f, 0x30, 0x09, 0xe9, 0x9a, - 0x97, 0xe3, 0xe2, 0xd7, 0x52, 0x38, 0xce, 0xed, 0x35, 0x68, 0x85, 0x8c, 0x58, 0xea, 0xf7, 0x46, - 0x5f, 0x54, 0xd6, 0x7f, 0x89, 0x16, 0x67, 0x3d, 0xc6, 0x21, 0x50, 0xf4, 0x32, 0x14, 0xba, 0x36, - 0xb6, 0xe4, 0x8e, 0xa5, 0x9b, 0x96, 0xee, 0xb0, 0x74, 0x7d, 0x61, 0x59, 0xfc, 0xe8, 0x70, 0x3e, - 0xbf, 0x65, 0x63, 0x6b, 0x93, 0xd3, 0xa5, 0x7c, 0x37, 0xf0, 0xe4, 0xfe, 0xa4, 0xcd, 0xe4, 0xf8, - 0x3f, 0x69, 0xf3, 0x16, 0x88, 0x16, 0x56, 0xb4, 0x50, 0x04, 0xc2, 0xae, 0x85, 0x89, 0xb6, 0x19, - 0x7a, 0x56, 0xc5, 0x2d, 0x49, 0xaf, 0x87, 0x99, 0xb6, 0xc2, 0x44, 0x74, 0x1d, 0x4e, 0xb6, 0x95, - 0x7d, 0x9a, 0x83, 0x28, 0xbb, 0x41, 0x1d, 0xcd, 0x2b, 0xcc, 0xd0, 0x7c, 0x59, 0xd4, 0x56, 0xf6, - 0xe9, 0xef, 0xe3, 0xb0, 0x57, 0xf4, 0x66, 0xfc, 0x8b, 0x50, 0xd4, 0x74, 0xdb, 0xd1, 0x0d, 0xd5, - 0xbd, 0xff, 0x93, 0x5d, 0xb4, 0x59, 0x70, 0xa9, 0xec, 0x1e, 0xce, 0xab, 0x30, 0xc3, 0xf3, 0xb1, - 0xfd, 0x5f, 0xcc, 0xa1, 0xf0, 0x25, 0x43, 0x6a, 0x41, 0x5e, 0xf8, 0x3f, 0x95, 0x54, 0x81, 0xe9, - 0xa6, 0xe2, 0xe0, 0x3d, 0xe5, 0x40, 0x76, 0x8f, 0xcb, 0xe4, 0xe8, 0xb5, 0x7a, 0xcf, 0x3c, 0x3d, - 0x9c, 0x2f, 0xdc, 0x66, 0xaf, 0xfa, 0x4e, 0xcd, 0x14, 0x9a, 0x81, 0x17, 0x1a, 0xba, 0x0c, 0xd3, - 0x8a, 0x7d, 0x60, 0xa8, 0x54, 0x3d, 0xd8, 0xb0, 0xbb, 0x36, 0x85, 0x14, 0x19, 0xa9, 0x48, 0xc9, - 0x15, 0x97, 0x8a, 0x5e, 0x85, 0x33, 0x1a, 0x26, 0x31, 0x12, 0x0b, 0x44, 0x1c, 0xd3, 0x94, 0xcd, - 0x96, 0x26, 0xd3, 0xeb, 0x1f, 0x28, 0x9c, 0xc8, 0x48, 0x27, 0x69, 0x01, 0x1a, 0x82, 0x34, 0x4c, - 0xf3, 0x5e, 0x4b, 0xab, 0x92, 0x97, 0xec, 0xaa, 0xcf, 0x3b, 0xa9, 0x4c, 0x5e, 0x2c, 0x94, 0x7f, - 0x4d, 0x80, 0x7c, 0xe8, 0x8c, 0xc2, 0xeb, 0x3d, 0x5b, 0xb4, 0x67, 0xa2, 0xe1, 0xd0, 0xa0, 0x1c, - 0xaf, 0x0c, 0x57, 0xbf, 0x9b, 0xe1, 0x36, 0x3f, 0x38, 0x9c, 0xa6, 0x8b, 0x03, 0x6e, 0x7a, 0x80, - 0xcb, 0xf6, 0x5a, 0xea, 0x37, 0xde, 0x9f, 0x9f, 0x28, 0xff, 0x22, 0x09, 0x85, 0xf0, 0x59, 0x84, - 0xd5, 0x9e, 0x7a, 0x45, 0xb9, 0xab, 0x10, 0xc7, 0xe2, 0x90, 0xfb, 0xcf, 0xb2, 0xfe, 0x7d, 0xdc, - 0xac, 0x9a, 0x0b, 0x43, 0x36, 0xa2, 0x83, 0xf5, 0xf4, 0x19, 0xe7, 0x7e, 0x98, 0xf0, 0x86, 0xfd, - 0x22, 0x4c, 0x32, 0xb5, 0x0b, 0x03, 0xcf, 0x86, 0x52, 0xcd, 0x4b, 0xac, 0x18, 0x71, 0x13, 0x8d, - 0x63, 0x5d, 0x72, 0xe5, 0x5f, 0x0f, 0x70, 0xf4, 0x5f, 0x92, 0xe2, 0x57, 0x9d, 0x4d, 0x1e, 0xed, - 0xaa, 0x33, 0xb6, 0xd1, 0xdc, 0x6a, 0x31, 0x17, 0xcc, 0x06, 0x4a, 0xba, 0xef, 0x00, 0x26, 0x15, - 0xc1, 0x7f, 0x68, 0x6c, 0x51, 0xe2, 0x3f, 0x34, 0x16, 0x48, 0x3a, 0x2c, 0x7a, 0x22, 0xe8, 0xa8, - 0x62, 0xa9, 0xa9, 0xbc, 0xab, 0xbf, 0x23, 0x80, 0x48, 0xc7, 0xd0, 0x2d, 0x8c, 0xb5, 0x58, 0xac, - 0xd0, 0xcd, 0x87, 0x4c, 0x8c, 0x9f, 0x26, 0x1e, 0xba, 0x1f, 0x3d, 0x19, 0xbe, 0x1f, 0xbd, 0xfc, - 0xbe, 0x00, 0x45, 0xaf, 0x86, 0xec, 0x67, 0x75, 0x86, 0xdc, 0x60, 0x76, 0xbc, 0x5f, 0x9a, 0x71, - 0x0f, 0x57, 0x8f, 0xf5, 0xfb, 0x3e, 0xc1, 0xc3, 0xd5, 0xec, 0x57, 0x51, 0xbe, 0x2d, 0xc0, 0xac, - 0x57, 0xc5, 0x8a, 0x7f, 0x70, 0xf6, 0x18, 0x19, 0xf3, 0x12, 0xfd, 0xc5, 0x31, 0x02, 0xda, 0xe9, - 0xa9, 0xf6, 0xb1, 0xcc, 0x13, 0xf1, 0x84, 0x0a, 0xe0, 0x8b, 0x01, 0x5a, 0xa3, 0x4e, 0x7f, 0x8b, - 0x8c, 0xfd, 0x6f, 0x97, 0x6f, 0x05, 0x14, 0x48, 0x47, 0x02, 0xd1, 0xd2, 0x58, 0x43, 0xc6, 0xd5, - 0x12, 0x2d, 0x5c, 0xfe, 0x71, 0xb0, 0x27, 0xaa, 0xbb, 0x24, 0x08, 0xbc, 0x09, 0xc9, 0x5d, 0xa5, - 0x35, 0x2c, 0x9f, 0x24, 0xd4, 0x73, 0x12, 0x29, 0x8d, 0x6e, 0x85, 0xce, 0x1b, 0x27, 0x06, 0x07, - 0x2c, 0xfd, 0x2a, 0x0d, 0x9d, 0x4b, 0x7e, 0xc5, 0x6d, 0x45, 0x72, 0xf4, 0xe7, 0x83, 0x1e, 0xe0, - 0xb5, 0xd4, 0x07, 0xef, 0xcf, 0x0b, 0x57, 0xeb, 0x30, 0x1b, 0x31, 0xbd, 0xa1, 0x22, 0x40, 0xe0, - 0xd6, 0x74, 0xfe, 0x83, 0x67, 0x4b, 0x2b, 0xf2, 0xd6, 0x46, 0xe5, 0xde, 0xfa, 0xfa, 0x6a, 0xa3, - 0x51, 0x5d, 0x11, 0x05, 0x24, 0x42, 0x3e, 0x74, 0xe7, 0x3a, 0xff, 0x9d, 0xb3, 0xab, 0xff, 0x0d, - 0xc0, 0xff, 0xdd, 0x05, 0x22, 0x6b, 0xad, 0xfa, 0x40, 0xbe, 0xbf, 0x74, 0x77, 0xab, 0x5a, 0x17, - 0x27, 0x10, 0x82, 0xe2, 0xf2, 0x52, 0xa3, 0x52, 0x93, 0xa5, 0x6a, 0x7d, 0xf3, 0xde, 0x46, 0xbd, - 0x2a, 0x0a, 0x9c, 0x6f, 0x05, 0xf2, 0xc1, 0x93, 0xd9, 0x68, 0x16, 0xa6, 0x2b, 0xb5, 0x6a, 0x65, - 0x4d, 0xbe, 0xbf, 0xba, 0x24, 0xbf, 0xb5, 0x55, 0xdd, 0xaa, 0x8a, 0x13, 0xb4, 0x6a, 0x94, 0x78, - 0x6b, 0xeb, 0xee, 0x5d, 0x51, 0x40, 0xd3, 0x90, 0x63, 0xcf, 0xf4, 0x7e, 0x76, 0x31, 0x71, 0x75, - 0x1d, 0x72, 0x81, 0x8b, 0xdc, 0xc8, 0xe7, 0x36, 0xb7, 0xea, 0x35, 0xb9, 0xb1, 0xba, 0x5e, 0xad, - 0x37, 0x96, 0xd6, 0x37, 0x99, 0x0c, 0x4a, 0x5b, 0x5a, 0xbe, 0x27, 0x35, 0x44, 0xc1, 0x7b, 0x6e, - 0xdc, 0xdb, 0xaa, 0xd4, 0xbc, 0x9f, 0x6b, 0x4b, 0x65, 0x92, 0x62, 0xf2, 0xea, 0x63, 0x38, 0x3d, - 0xe0, 0x78, 0x32, 0xca, 0xc1, 0xd4, 0x96, 0x41, 0xaf, 0xae, 0x12, 0x27, 0x50, 0x21, 0x70, 0x42, - 0x59, 0x14, 0x50, 0x86, 0x9d, 0x3d, 0x15, 0x13, 0x28, 0x0d, 0x89, 0xfa, 0x4d, 0x31, 0x49, 0x2a, - 0x1a, 0x38, 0xe0, 0x2b, 0xa6, 0x50, 0x96, 0x9f, 0x7e, 0x14, 0x27, 0x51, 0xde, 0x3f, 0x7e, 0x28, - 0xa6, 0xaf, 0x9e, 0x83, 0xc0, 0xd9, 0x2c, 0x04, 0x90, 0xbe, 0xab, 0x38, 0xd8, 0x76, 0xc4, 0x09, - 0x34, 0x05, 0xc9, 0xa5, 0x56, 0x4b, 0x14, 0x6e, 0xfc, 0x81, 0x00, 0x19, 0xf7, 0xaa, 0x71, 0x74, - 0x17, 0x26, 0x19, 0x92, 0x9f, 0x1f, 0x3c, 0x23, 0x51, 0xa7, 0x36, 0xb7, 0x30, 0x6a, 0xca, 0x2a, - 0x4f, 0xa0, 0xb7, 0xf9, 0x6f, 0x2f, 0x12, 0x8b, 0x41, 0xe7, 0x87, 0xd9, 0x93, 0x2b, 0x75, 0xb8, - 0xd1, 0x91, 0x31, 0x52, 0x9e, 0x78, 0x49, 0x58, 0x7e, 0xfe, 0x83, 0x9f, 0x9d, 0x9d, 0xf8, 0xe0, - 0xe9, 0x59, 0xe1, 0xc3, 0xa7, 0x67, 0x85, 0x9f, 0x3e, 0x3d, 0x2b, 0xfc, 0xfd, 0xd3, 0xb3, 0xc2, - 0xaf, 0xfc, 0xfc, 0xec, 0xc4, 0x87, 0x3f, 0x3f, 0x3b, 0xf1, 0xd3, 0x9f, 0x9f, 0x9d, 0x78, 0x67, - 0x8a, 0x73, 0x3f, 0x4c, 0xd3, 0x9f, 0x81, 0xbc, 0xf9, 0x6f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x95, - 0xb2, 0x7d, 0xda, 0x0b, 0x73, 0x00, 0x00, + 0x31, 0x97, 0xfa, 0x7f, 0xbf, 0x71, 0x76, 0xa2, 0x7c, 0x1f, 0xe0, 0x36, 0x76, 0xb8, 0x59, 0xa1, + 0x65, 0x48, 0xef, 0xd0, 0xda, 0x70, 0xc3, 0x5e, 0x88, 0xac, 0x76, 0xc0, 0x04, 0x97, 0x33, 0x44, + 0x03, 0x1f, 0x1e, 0xce, 0x0b, 0x12, 0xe7, 0x64, 0x5d, 0x5e, 0xfe, 0x81, 0x00, 0x39, 0x2a, 0x98, + 0xb5, 0x11, 0x55, 0x7a, 0x24, 0x9f, 0x1b, 0xa9, 0x90, 0x7e, 0xd1, 0x68, 0x11, 0x26, 0x77, 0x95, + 0x56, 0x77, 0xd8, 0xb8, 0xb9, 0x4f, 0xde, 0x4b, 0xac, 0x18, 0x7a, 0x1d, 0xf2, 0xba, 0xe1, 0x60, + 0xc3, 0x91, 0x19, 0x5b, 0x72, 0x04, 0x5b, 0x8e, 0x95, 0xa6, 0x0f, 0xe5, 0xef, 0x0b, 0x00, 0x9b, + 0xdd, 0x38, 0x55, 0x43, 0xc6, 0xfd, 0x58, 0xf5, 0x77, 0xc7, 0x3d, 0x6b, 0xc5, 0x29, 0x48, 0xeb, + 0x46, 0x4b, 0x37, 0x58, 0xfd, 0x33, 0x12, 0x7f, 0x42, 0x27, 0x60, 0xf2, 0x61, 0x4b, 0x37, 0x34, + 0x6a, 0xfe, 0x19, 0x89, 0x3d, 0x70, 0xf5, 0x4b, 0x90, 0xa3, 0x75, 0x8f, 0x51, 0xfb, 0xe5, 0x6f, + 0x26, 0xe0, 0x64, 0xc5, 0x34, 0x34, 0x9d, 0x8c, 0x43, 0xa5, 0xf5, 0xa9, 0xd0, 0xcd, 0xcb, 0x90, + 0xc5, 0xfb, 0x9d, 0x31, 0xbb, 0x37, 0x83, 0xf7, 0x3b, 0xf4, 0xbf, 0x68, 0xd5, 0xa1, 0xcf, 0xc2, + 0x69, 0xa5, 0xd5, 0x32, 0xf7, 0x64, 0x7d, 0x5b, 0xd6, 0x4c, 0x6c, 0xcb, 0x86, 0xe9, 0xc8, 0x78, + 0x5f, 0xb7, 0x1d, 0xea, 0x2a, 0x32, 0xd2, 0x2c, 0x7d, 0xbd, 0xba, 0xbd, 0x62, 0x62, 0x7b, 0xc3, + 0x74, 0xaa, 0xe4, 0x15, 0x57, 0xf8, 0xbb, 0x70, 0xaa, 0x57, 0x37, 0x71, 0xea, 0xfe, 0xaf, 0x04, + 0x28, 0xae, 0x1a, 0xba, 0xf3, 0xa9, 0x50, 0xba, 0xa7, 0xbd, 0x64, 0x50, 0x7b, 0x57, 0x41, 0xdc, + 0x56, 0xf4, 0xd6, 0x3d, 0xa3, 0x61, 0xb6, 0x1f, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0xab, 0xb7, 0x8f, + 0xce, 0x75, 0x76, 0x1f, 0xa6, 0xbd, 0x36, 0xc5, 0xa9, 0xac, 0x27, 0x20, 0xae, 0x1a, 0xaa, 0x85, + 0xdb, 0xd8, 0x88, 0x55, 0x5b, 0xcf, 0x42, 0x56, 0x77, 0xe5, 0x52, 0x8d, 0x25, 0x25, 0x9f, 0xc0, + 0xdb, 0xd4, 0x85, 0x99, 0xc0, 0xb7, 0xe3, 0x74, 0x7e, 0xcf, 0x40, 0xd6, 0xc0, 0x7b, 0xb2, 0xdf, + 0x5f, 0x49, 0x29, 0x63, 0xe0, 0x3d, 0xe6, 0xac, 0x1e, 0x40, 0x61, 0x05, 0xb7, 0xb0, 0x83, 0xe3, + 0xf7, 0xe4, 0x5b, 0x50, 0x74, 0x45, 0xc7, 0xd9, 0x49, 0xbf, 0x2e, 0x00, 0xe2, 0x72, 0xc9, 0xec, + 0x19, 0x67, 0x3f, 0xcd, 0x93, 0xe8, 0xc0, 0xe9, 0x5a, 0x06, 0x9b, 0xe6, 0x99, 0x95, 0x02, 0x23, + 0xd1, 0x99, 0xde, 0xf7, 0xa8, 0xa9, 0xa0, 0x47, 0xf5, 0xa2, 0x15, 0x12, 0xa7, 0xec, 0xc1, 0x6c, + 0xa8, 0x7a, 0xf1, 0x76, 0x65, 0x8a, 0xd6, 0x2c, 0xb1, 0x90, 0x0c, 0x46, 0x66, 0x94, 0x58, 0x7e, + 0x17, 0x66, 0x2a, 0x2d, 0xac, 0x58, 0x71, 0xab, 0x85, 0x77, 0xe7, 0x03, 0x40, 0x41, 0xf1, 0x71, + 0x76, 0xe9, 0x6f, 0x0a, 0x80, 0x24, 0xbc, 0x8b, 0x2d, 0x27, 0xf6, 0x2e, 0x5d, 0x81, 0x9c, 0xa3, + 0x58, 0x4d, 0xec, 0xc8, 0x24, 0x9c, 0xe7, 0xee, 0xea, 0xb9, 0x80, 0x20, 0x12, 0xd4, 0x2f, 0xee, + 0xb4, 0xd4, 0xc5, 0x86, 0x1b, 0xee, 0xbb, 0xc1, 0x19, 0xe3, 0x23, 0x64, 0xae, 0x81, 0x77, 0x60, + 0x36, 0x54, 0xcb, 0x38, 0x55, 0xf0, 0x97, 0x02, 0xe4, 0xea, 0xaa, 0x62, 0xc4, 0xd9, 0xf6, 0x37, + 0x21, 0x67, 0xab, 0x8a, 0x21, 0x6f, 0x9b, 0x56, 0x5b, 0x71, 0xa8, 0xc9, 0x16, 0x43, 0x6d, 0xf7, + 0x82, 0x5d, 0x55, 0x31, 0x6e, 0xd1, 0x42, 0x12, 0xd8, 0xde, 0xff, 0xe8, 0x2a, 0xcc, 0xd8, 0xb8, + 0x85, 0x55, 0x87, 0x48, 0x90, 0xbb, 0x1d, 0x4d, 0x71, 0x30, 0x9f, 0xd0, 0xa6, 0xd9, 0x8b, 0x5b, + 0xa6, 0xb5, 0x45, 0xc9, 0xc1, 0x11, 0x70, 0x27, 0x95, 0x49, 0x8a, 0xa9, 0xf2, 0x3f, 0x0b, 0x90, + 0x67, 0x2d, 0x8a, 0x73, 0x04, 0xbc, 0x0c, 0x29, 0xcb, 0xdc, 0x63, 0x23, 0x20, 0x77, 0xe3, 0x99, + 0x08, 0x11, 0x6b, 0xf8, 0x20, 0x38, 0xf5, 0xd0, 0xe2, 0x68, 0x19, 0x78, 0x88, 0x26, 0x53, 0xee, + 0xe4, 0xb8, 0xdc, 0xc0, 0xb8, 0x24, 0x22, 0xe3, 0x32, 0x4c, 0x3f, 0x54, 0x1c, 0x75, 0x47, 0xb6, + 0x78, 0x25, 0xc9, 0x34, 0x95, 0xbc, 0x92, 0x97, 0x8a, 0x94, 0xec, 0x56, 0xdd, 0x2e, 0xff, 0xae, + 0x6b, 0xce, 0x36, 0xfe, 0x94, 0x75, 0x69, 0x44, 0x37, 0xfd, 0x8b, 0xc0, 0xad, 0xda, 0xad, 0xec, + 0x7f, 0xb5, 0xde, 0xfa, 0x4e, 0x02, 0x4e, 0x57, 0x76, 0xb0, 0xfa, 0xa8, 0x62, 0x1a, 0xb6, 0x6e, + 0x3b, 0xd8, 0x50, 0x0f, 0xe2, 0xec, 0xb2, 0x67, 0x20, 0xbb, 0xa7, 0x3b, 0x3b, 0xb2, 0xa6, 0x6f, + 0x6f, 0x53, 0xff, 0x93, 0x91, 0x32, 0x84, 0xb0, 0xa2, 0x6f, 0x6f, 0xa3, 0x9b, 0x90, 0x6a, 0x9b, + 0x1a, 0x8b, 0x40, 0x8b, 0x37, 0xe6, 0x23, 0xc4, 0xd3, 0xaa, 0xd9, 0xdd, 0xf6, 0xba, 0xa9, 0x61, + 0x89, 0x16, 0x46, 0x67, 0x01, 0x54, 0x42, 0xed, 0x98, 0xba, 0xe1, 0xf0, 0x99, 0x28, 0x40, 0x41, + 0x35, 0xc8, 0x3a, 0xd8, 0x6a, 0xeb, 0x06, 0x1b, 0xaf, 0x44, 0x79, 0x17, 0x22, 0x2b, 0xde, 0x69, + 0xe9, 0xaa, 0xd2, 0xb7, 0xd4, 0xe0, 0x33, 0x73, 0xbf, 0xf7, 0x5e, 0x0a, 0x4a, 0xfd, 0x1a, 0x8a, + 0xd3, 0x4e, 0x36, 0x21, 0x4d, 0x70, 0x6b, 0xcb, 0xe1, 0x96, 0x72, 0x63, 0x90, 0x22, 0x22, 0x6a, + 0x40, 0xf1, 0x6f, 0xcb, 0xe1, 0x95, 0xe7, 0x72, 0xe6, 0xfe, 0x44, 0x80, 0x34, 0x7b, 0x81, 0xae, + 0x43, 0x86, 0xc3, 0x73, 0x8d, 0xd6, 0x31, 0xb9, 0x7c, 0xea, 0xe9, 0xe1, 0xfc, 0x14, 0x03, 0xe3, + 0x2b, 0x1f, 0xf9, 0xff, 0x4a, 0x53, 0x0c, 0x8f, 0x6b, 0xa4, 0xcf, 0x6c, 0x47, 0xb1, 0x1c, 0xba, + 0x0c, 0x42, 0xfa, 0x2c, 0x2f, 0x65, 0x28, 0x61, 0x0d, 0x1f, 0xa0, 0x3b, 0x90, 0xb6, 0x1d, 0xc5, + 0xe9, 0xda, 0xbc, 0xd7, 0x8e, 0x54, 0xd9, 0x3a, 0xe5, 0x94, 0xb8, 0x04, 0x12, 0x50, 0x68, 0xd8, + 0x51, 0xf4, 0x16, 0xed, 0xc6, 0xac, 0xc4, 0x9f, 0xca, 0xdf, 0x12, 0x20, 0xcd, 0x8a, 0xa2, 0xd3, + 0x30, 0xcb, 0x20, 0xf8, 0xea, 0xc6, 0x4a, 0xb5, 0x51, 0x95, 0xd6, 0x57, 0x37, 0x96, 0x1a, 0x55, + 0x71, 0x02, 0x9d, 0x02, 0xe4, 0xbe, 0xa8, 0xdc, 0xdb, 0xa8, 0xaf, 0xd6, 0x1b, 0xd5, 0x0d, 0x02, + 0xdd, 0x09, 0xa0, 0xa7, 0xf4, 0x00, 0x35, 0x81, 0x2e, 0xc0, 0x42, 0x2f, 0x55, 0xae, 0x37, 0x96, + 0x1a, 0x75, 0xb9, 0x5a, 0x6f, 0xac, 0xae, 0x2f, 0x35, 0xaa, 0x2b, 0x62, 0x72, 0x48, 0x29, 0xf2, + 0x11, 0x49, 0xaa, 0x56, 0x1a, 0x62, 0xaa, 0xfc, 0x04, 0x4e, 0x4a, 0x58, 0x35, 0xdb, 0x9d, 0xae, + 0x83, 0x49, 0x2d, 0xed, 0x38, 0xc7, 0xcb, 0x69, 0x98, 0xd2, 0xac, 0x03, 0xd9, 0xea, 0x1a, 0x7c, + 0xb4, 0xa4, 0x35, 0xeb, 0x40, 0xea, 0x1a, 0xdc, 0x18, 0xff, 0x48, 0x80, 0x53, 0xbd, 0x1f, 0x8f, + 0xd3, 0x14, 0xbf, 0x04, 0x39, 0x45, 0xd3, 0xb0, 0x26, 0x6b, 0xb8, 0xe5, 0x28, 0x3c, 0x60, 0xb8, + 0x1e, 0x90, 0xc4, 0x97, 0xb0, 0x16, 0xd9, 0xda, 0xd5, 0xa2, 0xb7, 0x84, 0xb5, 0x7e, 0xbf, 0x52, + 0xa1, 0xf5, 0x59, 0x21, 0x8c, 0xae, 0x47, 0xa2, 0xb2, 0x28, 0xa5, 0xfc, 0xed, 0x14, 0x14, 0xaa, + 0x86, 0xd6, 0xd8, 0x8f, 0x75, 0x46, 0x38, 0x05, 0x69, 0xd5, 0x6c, 0xb7, 0x75, 0xc7, 0xd5, 0x16, + 0x7b, 0x42, 0x9f, 0x83, 0x8c, 0x86, 0x15, 0xcd, 0x83, 0xff, 0xa3, 0xa2, 0x1e, 0xc9, 0x2b, 0x8e, + 0xbe, 0x02, 0xa7, 0x89, 0x23, 0xb5, 0x0c, 0xa5, 0x25, 0x33, 0x69, 0xb2, 0x63, 0xe9, 0xcd, 0x26, + 0xb6, 0xf8, 0x82, 0xd9, 0x95, 0x88, 0x7a, 0xae, 0x72, 0x8e, 0x0a, 0x65, 0x68, 0xb0, 0xf2, 0xd2, + 0x49, 0x3d, 0x8a, 0x8c, 0xbe, 0xe8, 0xad, 0xaf, 0xd8, 0x1d, 0xc5, 0xb0, 0xb9, 0x93, 0x1a, 0xb4, + 0x0e, 0xc7, 0x75, 0xc9, 0xe7, 0x04, 0x42, 0xb1, 0xd1, 0x35, 0x12, 0xaa, 0x3f, 0xee, 0xea, 0x16, + 0x96, 0xaf, 0x77, 0xd4, 0x52, 0x9a, 0xb4, 0x7d, 0xb9, 0xf8, 0xf4, 0x70, 0x1e, 0x24, 0x46, 0xbe, + 0xbe, 0x59, 0x21, 0xa1, 0x3b, 0xfb, 0xbf, 0xa3, 0xa2, 0x2b, 0x20, 0x1a, 0xa6, 0x6c, 0xe1, 0x6d, + 0x0b, 0xdb, 0x3b, 0xfc, 0xb3, 0x19, 0xaa, 0xb1, 0xa2, 0x61, 0x4a, 0x8c, 0xcc, 0x44, 0x9f, 0x82, + 0x74, 0xc7, 0xd4, 0x6d, 0xd3, 0x28, 0x65, 0x99, 0x46, 0xd9, 0x13, 0x7a, 0x0b, 0x44, 0xdd, 0x90, + 0xb7, 0x5b, 0x7a, 0x73, 0xc7, 0x91, 0xf7, 0x2c, 0xdd, 0xc1, 0x76, 0x69, 0x86, 0x56, 0x3c, 0xca, + 0xd0, 0xea, 0x7c, 0x25, 0x53, 0x7b, 0x9b, 0x94, 0xe4, 0x4d, 0x28, 0xea, 0xc6, 0x2d, 0xca, 0x4f, + 0x89, 0xb6, 0x37, 0x1d, 0x4f, 0x89, 0x99, 0xf2, 0xdf, 0x09, 0x50, 0x74, 0xcd, 0x23, 0x4e, 0x83, + 0xbe, 0x02, 0xa2, 0x69, 0x60, 0xb9, 0xb3, 0xa3, 0xd8, 0x98, 0x77, 0x27, 0x9f, 0x33, 0x8a, 0xa6, + 0x81, 0x37, 0x09, 0x99, 0x75, 0x0e, 0xda, 0x84, 0x19, 0xdb, 0x51, 0x9a, 0xba, 0xd1, 0x94, 0xbd, + 0xb5, 0x6f, 0x1a, 0xef, 0x8d, 0x19, 0x31, 0x8b, 0x9c, 0xdb, 0xa3, 0x87, 0x02, 0x8d, 0x1f, 0x0b, + 0x30, 0xb3, 0xa4, 0xb5, 0x75, 0xa3, 0xde, 0x69, 0xe9, 0xb1, 0xc2, 0xeb, 0x0b, 0x90, 0xb5, 0x89, + 0x4c, 0xdf, 0x5b, 0xfb, 0xd0, 0x28, 0x43, 0xdf, 0x10, 0xb7, 0x7d, 0x17, 0xa6, 0xf1, 0x7e, 0x47, + 0xb7, 0x14, 0x47, 0x37, 0x0d, 0x86, 0x06, 0x52, 0xe3, 0xb7, 0xad, 0xe8, 0xf3, 0xfa, 0x88, 0x80, + 0xb7, 0xec, 0x01, 0xa0, 0x60, 0xc3, 0xe2, 0x84, 0x05, 0x32, 0xcc, 0x52, 0xd1, 0x5b, 0x86, 0x1d, + 0xb3, 0xd6, 0xb8, 0x3b, 0xfd, 0x32, 0x9c, 0x08, 0x7f, 0x20, 0xce, 0xda, 0xbf, 0xcb, 0x7b, 0x7c, + 0x1d, 0x5b, 0x9f, 0x10, 0x22, 0x0d, 0x8a, 0x8f, 0xb3, 0xe6, 0x5f, 0x17, 0xe0, 0x0c, 0x95, 0x4d, + 0x77, 0x10, 0xb6, 0xb1, 0x45, 0xb7, 0x55, 0xe2, 0x34, 0xda, 0xf3, 0x90, 0x66, 0x00, 0x93, 0x5a, + 0xec, 0xe4, 0x72, 0x8e, 0x04, 0x22, 0x75, 0xc7, 0xb4, 0x48, 0x20, 0xc2, 0x5f, 0xf1, 0x76, 0x2a, + 0x30, 0x17, 0x55, 0x97, 0x98, 0x11, 0xf8, 0x0c, 0x8f, 0x07, 0x89, 0x89, 0x57, 0x76, 0x48, 0x20, + 0x84, 0xaa, 0x90, 0x53, 0xe9, 0x7f, 0xb2, 0x73, 0xd0, 0xc1, 0x54, 0x7e, 0x71, 0x58, 0x28, 0xc9, + 0xd8, 0x1a, 0x07, 0x1d, 0x4c, 0xe2, 0x51, 0xf7, 0x7f, 0xa2, 0xae, 0x40, 0x53, 0x87, 0x06, 0xa3, + 0x74, 0x7c, 0xd1, 0xb2, 0x6e, 0x3c, 0x17, 0xd2, 0xc4, 0x1f, 0x27, 0xb9, 0x2a, 0xd8, 0x97, 0x38, + 0x53, 0xac, 0xe1, 0xc7, 0x3b, 0x70, 0x4a, 0xc3, 0x1d, 0x0b, 0xab, 0x8a, 0x83, 0x35, 0x39, 0xd8, + 0xfc, 0xc4, 0x11, 0x9a, 0x7f, 0xc2, 0x97, 0xe1, 0x53, 0xd1, 0x03, 0x40, 0x01, 0xd9, 0xac, 0x65, + 0x2e, 0xbc, 0x39, 0x8a, 0x52, 0x66, 0x7c, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0x64, 0xf0, 0x7e, 0x47, + 0xa6, 0x7b, 0x8b, 0xa9, 0x23, 0xee, 0x2d, 0x4e, 0xe1, 0xfd, 0x0e, 0x21, 0xa2, 0x2d, 0x32, 0xc3, + 0xb9, 0x13, 0x3f, 0xad, 0xb6, 0x3d, 0x1a, 0x3f, 0xf8, 0xf6, 0xc2, 0xc5, 0x4d, 0x7b, 0x73, 0x3e, + 0x13, 0xc1, 0xfb, 0xee, 0x7d, 0x01, 0x9e, 0x89, 0xec, 0xbb, 0x38, 0x27, 0x3b, 0x77, 0x7b, 0x35, + 0x71, 0x9c, 0xed, 0xd5, 0xf2, 0xef, 0xb9, 0xa3, 0x5e, 0xc2, 0x2d, 0x93, 0xa8, 0xf7, 0x13, 0x58, + 0x8e, 0x9a, 0x72, 0xbb, 0x3d, 0x71, 0xe4, 0x6e, 0x77, 0x59, 0x7b, 0xdc, 0x42, 0x4f, 0x65, 0xe3, + 0x74, 0x0b, 0xbf, 0x2a, 0xc0, 0x6c, 0x0d, 0x2b, 0x96, 0xf3, 0x10, 0x2b, 0x4e, 0xcc, 0x81, 0xeb, + 0xcb, 0x90, 0x34, 0xcc, 0xbd, 0xa3, 0xac, 0xc8, 0x91, 0xf2, 0xfe, 0xb4, 0x15, 0xae, 0x57, 0x9c, + 0xad, 0xfe, 0xf3, 0x04, 0x64, 0x6f, 0x57, 0xe2, 0x6c, 0xeb, 0x1b, 0x7c, 0xdd, 0x96, 0x0d, 0xf5, + 0x28, 0xb3, 0xf4, 0xbe, 0xb7, 0x78, 0xbb, 0xb2, 0x86, 0x0f, 0x5c, 0xb3, 0x24, 0x5c, 0x68, 0x09, + 0xb2, 0xce, 0x0e, 0x89, 0x4f, 0xcd, 0x96, 0x76, 0x94, 0x98, 0xc5, 0xe7, 0x9a, 0x7b, 0x04, 0x93, + 0x54, 0xae, 0xbb, 0xf3, 0x2f, 0x44, 0xec, 0xfc, 0x93, 0xcf, 0x78, 0x61, 0x5f, 0xe2, 0x28, 0x9f, + 0x71, 0x09, 0xac, 0x73, 0xbc, 0xd8, 0x68, 0x52, 0x4c, 0x97, 0xdf, 0x02, 0x20, 0x4d, 0x8b, 0xb3, + 0x7b, 0x7e, 0x39, 0x09, 0xc5, 0xcd, 0xae, 0xbd, 0x13, 0xb3, 0x3d, 0x56, 0x00, 0x3a, 0x5d, 0x7b, + 0x07, 0x5b, 0xb2, 0xb3, 0x6f, 0xf0, 0xf6, 0x8f, 0xc8, 0x29, 0x70, 0x15, 0xc0, 0xf8, 0x1a, 0xfb, + 0x06, 0xba, 0xc7, 0x85, 0x60, 0xd9, 0x4f, 0x4c, 0xb8, 0x3a, 0x06, 0x78, 0x6c, 0xec, 0x1b, 0xeb, + 0xd8, 0x43, 0x8d, 0x4c, 0x20, 0x26, 0x02, 0xdf, 0x80, 0x29, 0xf2, 0x20, 0x3b, 0xe6, 0x51, 0x7a, + 0x3e, 0x4d, 0x78, 0x1a, 0x26, 0x7a, 0x1d, 0xb2, 0x8c, 0x9b, 0xcc, 0x5f, 0x69, 0x3a, 0x7f, 0x45, + 0x35, 0x89, 0x6b, 0x93, 0xce, 0x5c, 0x19, 0xca, 0x4a, 0x66, 0xab, 0x13, 0x30, 0xb9, 0x6d, 0x5a, + 0x2a, 0xa6, 0x99, 0x08, 0x19, 0x89, 0x3d, 0x04, 0x3b, 0xf7, 0x4e, 0x2a, 0x93, 0x11, 0xb3, 0x77, + 0x52, 0x99, 0xac, 0x08, 0xe5, 0x6f, 0x09, 0x30, 0xed, 0xf5, 0x4a, 0x9c, 0x2e, 0xbd, 0x12, 0x52, + 0xe9, 0xd1, 0xfb, 0x85, 0xa8, 0xb1, 0xfc, 0xd7, 0x34, 0xbe, 0x51, 0xcd, 0x5d, 0xda, 0x4d, 0x71, + 0x9a, 0xcd, 0x32, 0xcb, 0x41, 0x49, 0x1c, 0xb3, 0xab, 0x69, 0x56, 0xca, 0x75, 0x38, 0xa1, 0xb7, + 0x89, 0xcf, 0xd7, 0x9d, 0xd6, 0x01, 0xc7, 0x68, 0x0e, 0x76, 0xb7, 0x49, 0x67, 0xfd, 0x77, 0x15, + 0xf7, 0x15, 0x77, 0x83, 0x6c, 0xe3, 0xc4, 0x6f, 0x56, 0x9c, 0x7a, 0x5f, 0x85, 0x82, 0xc5, 0x44, + 0x93, 0x58, 0xe5, 0x88, 0xaa, 0xcf, 0x7b, 0xac, 0x44, 0xfb, 0xdf, 0x4b, 0xc0, 0xf4, 0x5b, 0x5d, + 0x6c, 0x1d, 0x7c, 0x0a, 0x75, 0x7f, 0x09, 0xa6, 0xf7, 0x14, 0x3d, 0xb4, 0xc5, 0xc1, 0xd4, 0x5e, + 0x20, 0x64, 0x6f, 0x83, 0x03, 0x61, 0x40, 0x8f, 0x0c, 0x73, 0xcf, 0x90, 0x09, 0x99, 0x42, 0xe4, + 0x7d, 0x83, 0x2f, 0x29, 0x2f, 0xbf, 0xf2, 0xb7, 0x87, 0xf3, 0x37, 0xc7, 0xca, 0x7f, 0xa2, 0x29, + 0x64, 0xdd, 0xae, 0xae, 0x2d, 0x6e, 0x6d, 0xad, 0xae, 0x48, 0x22, 0x15, 0xf9, 0x36, 0x93, 0xd8, + 0xd8, 0x37, 0xdc, 0xa9, 0xfd, 0x23, 0x01, 0x44, 0x5f, 0x61, 0x71, 0xf6, 0x6a, 0x15, 0x72, 0x8f, + 0xbb, 0xd8, 0xd2, 0x8f, 0xd1, 0xa7, 0xc0, 0x19, 0x89, 0x5b, 0x7a, 0x07, 0xf2, 0x21, 0x3d, 0x24, + 0x3f, 0x9e, 0x1e, 0x72, 0x7b, 0xbe, 0x0a, 0xca, 0x3f, 0x12, 0x00, 0xd1, 0xc6, 0xaf, 0xb2, 0xd5, + 0xfc, 0x4f, 0x99, 0xc1, 0x5c, 0x01, 0x91, 0x66, 0x24, 0xca, 0xfa, 0xb6, 0xdc, 0xd6, 0x6d, 0x5b, + 0x37, 0x9a, 0xdc, 0x62, 0x8a, 0x94, 0xbe, 0xba, 0xbd, 0xce, 0xa8, 0xbc, 0x2f, 0xff, 0x27, 0xcc, + 0x86, 0x5a, 0x13, 0x67, 0x6f, 0x9e, 0x83, 0xfc, 0xb6, 0xd9, 0x35, 0x34, 0x99, 0x2d, 0x8d, 0xf1, + 0x25, 0xc0, 0x1c, 0xa5, 0xb1, 0xef, 0x95, 0xff, 0x35, 0x01, 0x27, 0x24, 0x6c, 0x9b, 0xad, 0x5d, + 0x1c, 0xbf, 0x3e, 0xef, 0x01, 0xdf, 0x72, 0x91, 0x3f, 0x8e, 0x5a, 0xb3, 0x4c, 0x06, 0x9b, 0xee, + 0xc2, 0x6b, 0xeb, 0x17, 0x86, 0x5b, 0x66, 0xff, 0x6a, 0x3a, 0x5f, 0xb9, 0x4b, 0x85, 0x56, 0xee, + 0x76, 0x61, 0x5a, 0x6f, 0x1a, 0x26, 0x71, 0x64, 0x36, 0x7e, 0x6c, 0x74, 0xdb, 0x2e, 0xac, 0x79, + 0x79, 0x8c, 0xba, 0xae, 0x32, 0xce, 0x3a, 0x7e, 0xbc, 0xd1, 0x6d, 0xd3, 0x18, 0x7b, 0xf9, 0x14, + 0xa9, 0xf6, 0xd3, 0xc3, 0xf9, 0x62, 0xe8, 0x9d, 0x2d, 0x15, 0x75, 0xef, 0x99, 0x7c, 0x84, 0x1b, + 0xc0, 0x7f, 0x87, 0x93, 0x3d, 0x1d, 0x10, 0x67, 0x34, 0xf4, 0x17, 0x49, 0x38, 0x13, 0x16, 0x1f, + 0x37, 0x66, 0xf9, 0x4f, 0xd2, 0xc9, 0x35, 0x28, 0xb4, 0x75, 0xe3, 0x78, 0x2b, 0x97, 0xf9, 0xb6, + 0x6e, 0x78, 0xb4, 0x28, 0x73, 0x49, 0xff, 0xfb, 0x99, 0x8b, 0x02, 0x73, 0x51, 0xfd, 0x19, 0xa7, + 0xcd, 0xbc, 0x27, 0x40, 0x3e, 0xee, 0x35, 0xb9, 0xe3, 0xa5, 0x84, 0xf1, 0x36, 0x37, 0xa0, 0xf0, + 0x09, 0x2c, 0xe2, 0xfd, 0xb6, 0x00, 0xa8, 0x61, 0x75, 0x0d, 0x02, 0x8e, 0xef, 0x9a, 0xcd, 0x38, + 0x1b, 0x7b, 0x02, 0x26, 0x75, 0x43, 0xc3, 0xfb, 0xb4, 0xb1, 0x29, 0x89, 0x3d, 0x84, 0x76, 0x1a, + 0x93, 0x63, 0xed, 0x34, 0xfa, 0x99, 0x25, 0xa1, 0x8a, 0xc6, 0xa9, 0x85, 0xef, 0x25, 0x60, 0x96, + 0x37, 0x27, 0xf6, 0x45, 0xcc, 0x63, 0xe5, 0xa3, 0xa3, 0xcf, 0x03, 0x74, 0x2c, 0xbc, 0x2b, 0x33, + 0xd6, 0xe4, 0x58, 0xac, 0x59, 0xc2, 0x41, 0x09, 0xe8, 0x4b, 0x30, 0x4d, 0x06, 0x7a, 0xc7, 0x32, + 0x3b, 0xa6, 0x4d, 0xe2, 0x18, 0x7b, 0x3c, 0x68, 0x34, 0xf3, 0xf4, 0x70, 0xbe, 0xb0, 0xae, 0x1b, + 0x9b, 0x9c, 0xb1, 0x51, 0x97, 0x88, 0xc7, 0xf0, 0x1e, 0xdd, 0x01, 0xf8, 0x37, 0x02, 0x9c, 0xf8, + 0xc4, 0x96, 0x7d, 0xff, 0x23, 0x34, 0xe6, 0xcd, 0x43, 0x22, 0x7d, 0x5c, 0x35, 0xb6, 0xcd, 0xf8, + 0x17, 0xe3, 0xdf, 0x13, 0x60, 0x26, 0x20, 0x3e, 0xce, 0x28, 0xe7, 0x78, 0xa7, 0x1e, 0xbe, 0x4c, + 0xe2, 0x9e, 0xa0, 0xd9, 0xc7, 0x39, 0xa8, 0xfe, 0x34, 0x01, 0xa7, 0x2a, 0x6c, 0x0f, 0xda, 0x4d, + 0xd0, 0x88, 0xd3, 0x4a, 0x4a, 0x30, 0xb5, 0x8b, 0x2d, 0x5b, 0x37, 0xd9, 0x7c, 0x5b, 0x90, 0xdc, + 0x47, 0x34, 0x07, 0x19, 0xdb, 0x50, 0x3a, 0xf6, 0x8e, 0xe9, 0xee, 0xe2, 0x79, 0xcf, 0x5e, 0x32, + 0xc9, 0xe4, 0xf1, 0x93, 0x49, 0xd2, 0xc3, 0x93, 0x49, 0xa6, 0x3e, 0x76, 0x32, 0x09, 0xdf, 0x32, + 0xfb, 0xa1, 0x00, 0xa7, 0xfb, 0xf4, 0x17, 0xa7, 0xcd, 0x7c, 0x15, 0x72, 0x2a, 0x17, 0x4c, 0xbc, + 0x31, 0xdb, 0x15, 0x5c, 0x25, 0xc5, 0x8e, 0x89, 0x51, 0x9e, 0x1e, 0xce, 0x83, 0x5b, 0xd5, 0xd5, + 0x15, 0xae, 0x22, 0xf2, 0xbf, 0x56, 0xfe, 0xff, 0x39, 0x98, 0xae, 0xee, 0xb3, 0x35, 0xef, 0x3a, + 0x0b, 0x0b, 0xd0, 0x2d, 0xc8, 0x74, 0x2c, 0x73, 0x57, 0x77, 0x9b, 0x51, 0x0c, 0x85, 0x4c, 0x6e, + 0x33, 0x7a, 0xb8, 0x36, 0x39, 0x87, 0xe4, 0xf1, 0xa2, 0x06, 0x64, 0xef, 0x9a, 0xaa, 0xd2, 0xba, + 0xa5, 0xb7, 0x5c, 0xfb, 0x7f, 0x69, 0xb4, 0xa0, 0x45, 0x8f, 0x67, 0x53, 0x71, 0x76, 0xdc, 0xae, + 0xf0, 0x88, 0x68, 0x15, 0x32, 0x35, 0xc7, 0xe9, 0x90, 0x97, 0xdc, 0x9b, 0x5c, 0x1e, 0x43, 0x28, + 0x61, 0xe1, 0xb2, 0x3c, 0x76, 0xd4, 0x80, 0x99, 0xdb, 0xa6, 0xd9, 0x6c, 0xe1, 0x4a, 0xcb, 0xec, + 0x6a, 0x15, 0xd3, 0xd8, 0xd6, 0x9b, 0xdc, 0x1f, 0x5f, 0x1a, 0x43, 0xe6, 0xed, 0x4a, 0x5d, 0xea, + 0x17, 0x80, 0x96, 0x20, 0x53, 0xbf, 0xc9, 0x85, 0xb1, 0x38, 0xee, 0xe2, 0x18, 0xc2, 0xea, 0x37, + 0x25, 0x8f, 0x0d, 0xdd, 0x81, 0xdc, 0xd2, 0x93, 0xae, 0x85, 0xb9, 0x94, 0xf4, 0xc0, 0xcc, 0x85, + 0x5e, 0x29, 0x94, 0x4b, 0x0a, 0x32, 0xa3, 0x3a, 0x14, 0xdf, 0x36, 0xad, 0x47, 0x2d, 0x53, 0x71, + 0x5b, 0x38, 0x45, 0xc5, 0x7d, 0x66, 0x0c, 0x71, 0x2e, 0xa3, 0xd4, 0x23, 0x62, 0xee, 0x4b, 0x50, + 0x08, 0x75, 0x13, 0x42, 0x90, 0xea, 0x90, 0x1e, 0x11, 0x68, 0x2a, 0x10, 0xfd, 0x1f, 0xbd, 0x08, + 0x53, 0x86, 0xa9, 0x61, 0xd7, 0x86, 0x0b, 0xcb, 0x27, 0x9e, 0x1e, 0xce, 0xa7, 0x37, 0x4c, 0x8d, + 0x05, 0x14, 0xfc, 0x3f, 0x29, 0x4d, 0x0a, 0xb9, 0xe1, 0xc4, 0xdc, 0x25, 0x48, 0x91, 0xfe, 0x21, + 0x6e, 0xe4, 0xa1, 0x62, 0xe3, 0x2d, 0x4b, 0xe7, 0x32, 0xdd, 0x47, 0x5e, 0xee, 0x27, 0x02, 0x24, + 0xea, 0x37, 0x49, 0x44, 0xfd, 0xb0, 0xab, 0x3e, 0xc2, 0x0e, 0x2f, 0xc5, 0x9f, 0x68, 0xa4, 0x6d, + 0xe1, 0x6d, 0x9d, 0x45, 0x39, 0x59, 0x89, 0x3f, 0xa1, 0xe7, 0x00, 0x14, 0x55, 0xc5, 0xb6, 0x2d, + 0xbb, 0x87, 0xc8, 0xb2, 0x52, 0x96, 0x51, 0xd6, 0xf0, 0x01, 0x61, 0xb3, 0xb1, 0x6a, 0x61, 0xc7, + 0xcd, 0x69, 0x62, 0x4f, 0x84, 0xcd, 0xc1, 0xed, 0x8e, 0xec, 0x98, 0x8f, 0xb0, 0x41, 0x7b, 0x35, + 0x4b, 0xdc, 0x43, 0xbb, 0xd3, 0x20, 0x04, 0xe2, 0xd9, 0xb0, 0xa1, 0xf9, 0x6e, 0x28, 0x2b, 0x79, + 0xcf, 0x44, 0xa4, 0x85, 0x9b, 0x3a, 0x3f, 0x15, 0x95, 0x95, 0xf8, 0x13, 0xd1, 0x98, 0xd2, 0x75, + 0x76, 0x68, 0x22, 0x47, 0x56, 0xa2, 0xff, 0xf3, 0xa6, 0x7d, 0x53, 0x80, 0xe4, 0xed, 0x4a, 0xfd, + 0xc8, 0x6d, 0x73, 0x25, 0x26, 0x7d, 0x89, 0x34, 0x95, 0x50, 0x6f, 0xb5, 0x74, 0xa3, 0x49, 0x82, + 0x8e, 0xaf, 0x62, 0xd5, 0x6d, 0x59, 0x91, 0x93, 0x37, 0x19, 0x15, 0x2d, 0x40, 0x4e, 0xb5, 0xb0, + 0x86, 0x0d, 0x47, 0x57, 0x5a, 0x36, 0x6f, 0x62, 0x90, 0xc4, 0x2b, 0xf7, 0x35, 0x01, 0x26, 0xa9, + 0x79, 0xa1, 0x67, 0x21, 0xab, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0xf7, 0x13, 0x59, 0xc9, 0x27, 0x0c, + 0xac, 0xe4, 0x39, 0xc8, 0x2b, 0xaa, 0x6a, 0x76, 0x0d, 0x47, 0x36, 0x94, 0x36, 0xe6, 0x95, 0xcd, + 0x71, 0xda, 0x86, 0xd2, 0xc6, 0x68, 0x1e, 0xdc, 0x47, 0xef, 0x28, 0x5f, 0x56, 0x02, 0x4e, 0x5a, + 0xc3, 0x07, 0xbc, 0x26, 0x3f, 0x14, 0x20, 0xe3, 0x9a, 0x25, 0xa9, 0x4c, 0x13, 0x1b, 0xd8, 0x52, + 0x1c, 0xd3, 0xab, 0x8c, 0x47, 0xe8, 0x9d, 0x93, 0xb2, 0xfe, 0x9c, 0x74, 0x02, 0x26, 0x1d, 0xe5, + 0x61, 0xcb, 0xad, 0x07, 0x7b, 0xa0, 0xcb, 0xc7, 0x2d, 0xa5, 0xc9, 0xd6, 0xc8, 0xb2, 0x12, 0x7b, + 0x20, 0x4d, 0xe2, 0xb9, 0xab, 0x4c, 0x3b, 0xfc, 0x89, 0xd4, 0x97, 0xa5, 0x6b, 0x3e, 0xc4, 0x4d, + 0xdd, 0xa0, 0x06, 0x90, 0x94, 0x80, 0x92, 0x96, 0x09, 0x05, 0x3d, 0x03, 0x59, 0x56, 0x00, 0x1b, + 0x1a, 0xb5, 0x82, 0xa4, 0x94, 0xa1, 0x84, 0xaa, 0x7b, 0x76, 0x89, 0x07, 0x22, 0xdf, 0x15, 0x60, + 0x86, 0xe5, 0xdc, 0xb0, 0x3c, 0xcf, 0xf8, 0x66, 0xe5, 0xd7, 0x20, 0xab, 0x29, 0x8e, 0xc2, 0x8e, + 0x0e, 0x26, 0x86, 0x1e, 0x1d, 0x74, 0xdd, 0x24, 0x29, 0x4f, 0x8f, 0x0f, 0x22, 0x48, 0x91, 0xff, + 0xd9, 0x91, 0x4b, 0x89, 0xfe, 0xef, 0x67, 0x31, 0x04, 0xab, 0x1b, 0x67, 0x94, 0xf2, 0xe1, 0x24, + 0x14, 0xaa, 0xfb, 0x1d, 0xd3, 0x8a, 0x79, 0x11, 0x6d, 0x8a, 0x83, 0xdf, 0x21, 0xdb, 0xac, 0x3d, + 0x5e, 0xd0, 0xdd, 0xc1, 0xe4, 0x8c, 0x68, 0x19, 0x80, 0x25, 0x58, 0xd2, 0x3c, 0x9c, 0xe4, 0x11, + 0x36, 0x9b, 0x28, 0x1b, 0xa1, 0xa2, 0x0d, 0xc8, 0xb5, 0x77, 0x55, 0x55, 0xde, 0xd6, 0x5b, 0x0e, + 0x4f, 0x4d, 0x8b, 0xce, 0x85, 0x5e, 0xbf, 0x5f, 0xa9, 0xdc, 0xa2, 0x85, 0x58, 0x86, 0x98, 0xff, + 0x2c, 0x01, 0x91, 0xc0, 0xfe, 0x47, 0x2f, 0x00, 0x3f, 0xea, 0x21, 0xdb, 0xee, 0xc1, 0xad, 0xe5, + 0xc2, 0xd3, 0xc3, 0xf9, 0xac, 0x44, 0xa9, 0xf5, 0x7a, 0x43, 0xca, 0xb2, 0x02, 0x75, 0xdb, 0x41, + 0xe7, 0xa1, 0x60, 0xb6, 0x75, 0x47, 0x76, 0xe3, 0x00, 0x1e, 0x3a, 0xe5, 0x09, 0xd1, 0x8d, 0x13, + 0x50, 0x03, 0x2e, 0x63, 0x83, 0x8c, 0x06, 0xda, 0x4e, 0xf9, 0x21, 0x5b, 0xab, 0x73, 0xd8, 0x88, + 0x92, 0xcd, 0x8e, 0xa3, 0xb7, 0xf5, 0x27, 0x74, 0xa3, 0x97, 0x6f, 0xb2, 0x9c, 0x67, 0xc5, 0x49, + 0xfb, 0x96, 0xe9, 0x22, 0x1e, 0x2f, 0x7b, 0x2f, 0x50, 0x14, 0x7d, 0x4d, 0x80, 0x53, 0x5c, 0x91, + 0xf2, 0xc3, 0x03, 0xb9, 0x45, 0x26, 0x11, 0xdd, 0x39, 0x90, 0x1f, 0xed, 0x96, 0x32, 0x34, 0x40, + 0xfb, 0x5c, 0x64, 0x87, 0x04, 0xec, 0x60, 0xd1, 0xed, 0x96, 0x83, 0xbb, 0x9c, 0x79, 0x6d, 0xb7, + 0x6a, 0x38, 0xd6, 0xc1, 0xf2, 0xe9, 0xa7, 0x87, 0xf3, 0xb3, 0xfd, 0x6f, 0xef, 0x4b, 0xb3, 0x76, + 0x3f, 0xcb, 0xdc, 0x57, 0xa1, 0x34, 0x48, 0x12, 0x12, 0xfd, 0x9d, 0xc6, 0x2c, 0xdb, 0x60, 0x7c, + 0x35, 0xbc, 0x42, 0x30, 0x86, 0xd9, 0xb8, 0xab, 0x04, 0x89, 0x57, 0xdd, 0xd1, 0xfd, 0x0d, 0x01, + 0x0a, 0xcb, 0xdd, 0xd6, 0xa3, 0x7b, 0x9d, 0x7a, 0xb7, 0xdd, 0x56, 0xac, 0x03, 0xe2, 0x18, 0xd8, + 0xa8, 0xd4, 0x9f, 0xb0, 0x14, 0x95, 0x24, 0x1f, 0x76, 0xfa, 0x13, 0x4c, 0x86, 0x1d, 0x4f, 0x40, + 0x27, 0x74, 0x96, 0x5d, 0x7e, 0x1e, 0x0a, 0x14, 0xae, 0xcb, 0xd8, 0x70, 0x2c, 0x1d, 0xb3, 0x55, + 0xa8, 0xa4, 0x94, 0xa7, 0xc4, 0x2a, 0xa3, 0xa1, 0x8b, 0x50, 0xb4, 0x0f, 0x6c, 0x07, 0xb7, 0x65, + 0x76, 0xd0, 0x9b, 0x61, 0xcc, 0xa4, 0x54, 0x60, 0x54, 0x89, 0x11, 0xcb, 0x3f, 0x4d, 0x42, 0xd1, + 0xd5, 0x6c, 0x9c, 0xe1, 0xeb, 0x32, 0x4c, 0x6e, 0xeb, 0x2d, 0xec, 0x66, 0x09, 0x5c, 0x1a, 0xd2, + 0xa1, 0x3c, 0xb1, 0x98, 0x44, 0x11, 0x2e, 0x00, 0xa2, 0xac, 0x71, 0x8c, 0xb1, 0xb9, 0xff, 0x9d, + 0x80, 0x14, 0x8d, 0x18, 0xaf, 0x43, 0x8a, 0xba, 0x3d, 0x61, 0x1c, 0xb7, 0x47, 0x8b, 0x7a, 0xe1, + 0x4c, 0x22, 0x10, 0xce, 0x90, 0xd8, 0x60, 0x47, 0x79, 0xf9, 0xfa, 0x0d, 0x3a, 0xbe, 0xf2, 0x12, + 0x7f, 0x42, 0xcb, 0x34, 0x7d, 0xc5, 0xb4, 0x1c, 0xac, 0xf1, 0x48, 0x2d, 0xca, 0x33, 0x85, 0x3a, + 0xde, 0x75, 0xb1, 0x2e, 0x1f, 0x3a, 0x03, 0x49, 0x32, 0x70, 0xa7, 0xd8, 0xd6, 0xf6, 0xd3, 0xc3, + 0xf9, 0x24, 0x19, 0xb2, 0x84, 0x86, 0xae, 0x41, 0x2e, 0x3c, 0x4a, 0x84, 0x2b, 0x59, 0xe6, 0x0b, + 0x02, 0x16, 0x0e, 0x2d, 0xcf, 0x82, 0x19, 0x4a, 0xb9, 0x93, 0xca, 0xa4, 0xc4, 0xc9, 0xf2, 0x1f, + 0xa6, 0xa0, 0xb0, 0xda, 0x8e, 0xdb, 0x8b, 0x2e, 0x85, 0x7b, 0x38, 0x2a, 0xbc, 0x0d, 0x7d, 0x34, + 0xa2, 0x83, 0x43, 0xf3, 0x51, 0xf2, 0x68, 0xf3, 0xd1, 0x2a, 0x89, 0xa8, 0xf8, 0x61, 0xf6, 0xe4, + 0x80, 0x48, 0x36, 0xfc, 0xfd, 0x06, 0x71, 0x4c, 0x12, 0xe1, 0xf1, 0x53, 0xed, 0x69, 0x7a, 0xc2, + 0x9b, 0x34, 0x70, 0x63, 0x56, 0x96, 0x1e, 0xdf, 0xca, 0xa6, 0xb0, 0xa1, 0x51, 0x1b, 0x7b, 0xc2, + 0x4d, 0xec, 0x35, 0x48, 0x6a, 0xba, 0x35, 0xe4, 0x66, 0x84, 0xe8, 0x39, 0x85, 0x30, 0x8d, 0xb0, + 0xb5, 0x54, 0xd0, 0xd6, 0x82, 0x38, 0x74, 0xee, 0x1e, 0x80, 0xdf, 0x2e, 0xb4, 0x00, 0x69, 0xb3, + 0xa5, 0xb9, 0xe7, 0x04, 0x0a, 0xcb, 0xd9, 0xa7, 0x87, 0xf3, 0x93, 0xf7, 0x5a, 0xda, 0xea, 0x8a, + 0x34, 0x69, 0xb6, 0xb4, 0x55, 0x8d, 0xde, 0x02, 0x80, 0xf7, 0x64, 0x2f, 0xc7, 0x28, 0x2f, 0x4d, + 0x19, 0x78, 0x8f, 0xa0, 0xde, 0x9e, 0xdc, 0x07, 0x62, 0x38, 0xdf, 0x11, 0xa0, 0xe8, 0xea, 0x30, + 0x5e, 0xe7, 0x90, 0xd1, 0xdb, 0x7c, 0xb0, 0x24, 0x8f, 0x36, 0x58, 0x5c, 0x3e, 0x7e, 0x56, 0xf1, + 0xeb, 0x02, 0xcf, 0x2f, 0xad, 0xab, 0x8a, 0x43, 0xe6, 0xc3, 0x18, 0x0d, 0xfc, 0x79, 0x10, 0x2d, + 0xc5, 0xd0, 0xcc, 0xb6, 0xfe, 0x04, 0xb3, 0x85, 0x2b, 0x9b, 0xef, 0x4f, 0x4d, 0x7b, 0x74, 0xba, + 0x32, 0xe3, 0xae, 0xbb, 0xfd, 0x42, 0xe0, 0xb9, 0xa8, 0x5e, 0x65, 0xe2, 0x54, 0xda, 0x1a, 0xa4, + 0x2d, 0x96, 0xd1, 0xc6, 0x06, 0xdc, 0x8b, 0x11, 0x42, 0xa2, 0xbe, 0xce, 0x12, 0xc6, 0x3c, 0x93, + 0xa7, 0x22, 0xe6, 0xbe, 0x08, 0x93, 0x94, 0x7c, 0x0c, 0xb7, 0xc8, 0x35, 0xff, 0x0f, 0x09, 0xb8, + 0x40, 0x3f, 0x77, 0x1f, 0x5b, 0xfa, 0xf6, 0xc1, 0xa6, 0x65, 0x3a, 0x58, 0x75, 0xb0, 0xe6, 0xe7, + 0xe3, 0xc7, 0xea, 0x6b, 0xb2, 0x1d, 0xf7, 0x03, 0x47, 0xca, 0xec, 0xf1, 0xb8, 0xd0, 0x1a, 0x4c, + 0xb3, 0x89, 0x50, 0x56, 0x5a, 0xfa, 0x2e, 0x96, 0x15, 0xe7, 0x28, 0x33, 0x4a, 0x81, 0xf1, 0x2e, + 0x11, 0xd6, 0x25, 0x07, 0x69, 0x90, 0xe5, 0xc2, 0x74, 0x8d, 0xdf, 0x32, 0x72, 0xfb, 0xe3, 0x2d, + 0xcd, 0x64, 0xd8, 0x84, 0xbc, 0xba, 0x22, 0x65, 0x98, 0x64, 0x6f, 0x69, 0xfd, 0xc7, 0x02, 0x5c, + 0x1c, 0xa1, 0xe8, 0x38, 0xcd, 0x6c, 0x0e, 0x32, 0xbb, 0xe4, 0x43, 0x3a, 0xd7, 0x74, 0x46, 0xf2, + 0x9e, 0xd1, 0x3a, 0x14, 0xb6, 0x15, 0xbd, 0x85, 0x35, 0x99, 0x5b, 0xe2, 0xe0, 0x74, 0xb0, 0xe8, + 0x2c, 0xc5, 0x3c, 0x63, 0xa7, 0x2f, 0xed, 0xf2, 0x6f, 0x25, 0x60, 0x66, 0x49, 0xd3, 0xea, 0x75, + 0xee, 0xc1, 0xe2, 0xb3, 0x17, 0x17, 0xac, 0x24, 0x7c, 0xb0, 0x82, 0x5e, 0x04, 0xa4, 0xe9, 0x36, + 0xbb, 0xe6, 0xc0, 0xde, 0x51, 0x34, 0x73, 0xcf, 0xdf, 0xf8, 0x9e, 0x71, 0xdf, 0xd4, 0xdd, 0x17, + 0xe8, 0x1d, 0xa0, 0xa1, 0xb5, 0x6c, 0x3b, 0x8a, 0xb7, 0x3e, 0xff, 0xc2, 0x51, 0x4e, 0xd1, 0xb0, + 0xd0, 0xdb, 0x7b, 0x94, 0xb2, 0x44, 0x1c, 0xfd, 0x17, 0x5d, 0x01, 0x51, 0x27, 0x1a, 0x70, 0x64, + 0xc5, 0x76, 0x0f, 0x62, 0xb0, 0x63, 0xa9, 0x45, 0x46, 0x5f, 0xb2, 0x83, 0xe7, 0x2b, 0x58, 0x9e, + 0xb8, 0xaf, 0xa7, 0x38, 0x11, 0xd6, 0xef, 0x08, 0x50, 0xe4, 0x87, 0x47, 0xe2, 0xec, 0x80, 0x5b, + 0x90, 0x77, 0x4f, 0xaa, 0x6c, 0x5b, 0x66, 0xfb, 0x28, 0x43, 0x2d, 0xc7, 0x19, 0x6f, 0x59, 0x66, + 0x3b, 0x74, 0x2e, 0xfd, 0x3e, 0x4c, 0x7b, 0x35, 0x8d, 0x53, 0x05, 0xdf, 0xa5, 0x07, 0x48, 0x99, + 0xe0, 0xb8, 0xb7, 0x9e, 0x3f, 0x09, 0x3d, 0xd0, 0x7d, 0x81, 0x60, 0x75, 0xe3, 0x54, 0xc6, 0x2f, + 0x04, 0x28, 0xd6, 0xbb, 0x0f, 0xd9, 0x95, 0x3b, 0xf1, 0xe9, 0xa1, 0x0a, 0xd9, 0x16, 0xde, 0x76, + 0xe4, 0x63, 0xe5, 0x36, 0x67, 0x08, 0x2b, 0xcd, 0xef, 0xbe, 0x0d, 0x60, 0xd1, 0xd3, 0x4b, 0x54, + 0x4e, 0xf2, 0x88, 0x72, 0xb2, 0x94, 0xd7, 0x8f, 0x75, 0xca, 0x7f, 0x90, 0x80, 0x69, 0xaf, 0xb1, + 0x71, 0x3a, 0xd1, 0xff, 0x11, 0x72, 0x1e, 0xc9, 0x63, 0x38, 0x8f, 0x19, 0xbe, 0xe5, 0x1e, 0xed, + 0x40, 0x16, 0x61, 0x96, 0x06, 0x24, 0xb2, 0xd2, 0xe9, 0xb4, 0x74, 0xac, 0xc9, 0x6c, 0x2f, 0x37, + 0x45, 0xf7, 0x72, 0x67, 0xe8, 0xab, 0x25, 0xf6, 0x66, 0x95, 0xee, 0xeb, 0xde, 0x82, 0xfc, 0xb6, + 0x85, 0xf1, 0x13, 0x2c, 0x53, 0x64, 0x74, 0x94, 0xcc, 0x82, 0x1c, 0x63, 0xac, 0x13, 0x3e, 0x6e, + 0x80, 0xef, 0xc2, 0x0c, 0x55, 0x70, 0xdc, 0x27, 0x27, 0x79, 0xaf, 0xfc, 0xa3, 0x00, 0x28, 0x28, + 0xff, 0x93, 0xeb, 0x98, 0x44, 0xec, 0x1d, 0xf3, 0x02, 0x20, 0x96, 0x64, 0x66, 0xcb, 0x1d, 0x6c, + 0xc9, 0x36, 0x56, 0x4d, 0x7e, 0x5b, 0x8c, 0x20, 0x89, 0xfc, 0xcd, 0x26, 0xb6, 0xea, 0x94, 0x5e, + 0x7e, 0x6f, 0x0e, 0xf2, 0x5c, 0x27, 0x5b, 0x86, 0x6e, 0x1a, 0xe8, 0x3a, 0x24, 0x9b, 0x7c, 0xa5, + 0x37, 0x17, 0xb9, 0x12, 0xe4, 0xdf, 0x58, 0x55, 0x9b, 0x90, 0x48, 0x59, 0xc2, 0xd2, 0xe9, 0x3a, + 0x11, 0x41, 0x91, 0x9f, 0x1b, 0x1b, 0x64, 0xe9, 0x74, 0x1d, 0x54, 0x87, 0x69, 0xd5, 0xbf, 0xb1, + 0x47, 0x26, 0xec, 0xc9, 0x81, 0x9b, 0x0b, 0x91, 0xf7, 0x1e, 0xd5, 0x26, 0xa4, 0xa2, 0x1a, 0x7a, + 0x81, 0x2a, 0xc1, 0x2b, 0x62, 0xd8, 0x74, 0x79, 0x3e, 0xf2, 0x94, 0x65, 0xf8, 0x7a, 0x9a, 0xda, + 0x44, 0xe0, 0x26, 0x19, 0xf4, 0x1a, 0xa4, 0x35, 0x7a, 0xf5, 0x08, 0xb7, 0xd0, 0x28, 0x23, 0x0a, + 0xdd, 0xf6, 0x52, 0x9b, 0x90, 0x38, 0x07, 0xba, 0x03, 0x79, 0xf6, 0x1f, 0x0b, 0x4e, 0x38, 0x92, + 0xbb, 0x38, 0x58, 0x42, 0xc0, 0xd7, 0xd7, 0x26, 0xa4, 0x9c, 0xe6, 0x53, 0xd1, 0x67, 0x21, 0x65, + 0xab, 0x8a, 0xc1, 0x37, 0x49, 0xce, 0x0e, 0xb8, 0x9e, 0xc0, 0x67, 0xa6, 0xa5, 0xd1, 0xeb, 0xec, + 0xe6, 0x39, 0x67, 0x9f, 0x1d, 0xbc, 0x8c, 0xae, 0x7e, 0xe8, 0x00, 0x2d, 0xa9, 0x3e, 0xa6, 0x04, + 0x74, 0x1b, 0x72, 0x0a, 0x89, 0xf2, 0x64, 0x7a, 0x8c, 0xad, 0x04, 0x03, 0x0f, 0x1a, 0xf5, 0x1d, + 0x41, 0xac, 0xd1, 0x53, 0xba, 0x2e, 0xd1, 0x17, 0xd4, 0xc6, 0x56, 0x13, 0x97, 0x72, 0xc3, 0x05, + 0x05, 0xb3, 0x68, 0x3c, 0x41, 0x94, 0x48, 0xa2, 0xbd, 0x1d, 0xf7, 0x88, 0x02, 0x6d, 0x54, 0x7e, + 0xe0, 0xa6, 0x58, 0xc4, 0x11, 0x8b, 0xda, 0x84, 0x94, 0xdf, 0x09, 0x90, 0xd1, 0x22, 0x24, 0x9a, + 0x6a, 0xa9, 0x40, 0x65, 0x3c, 0x3b, 0xec, 0x00, 0x41, 0x6d, 0x42, 0x4a, 0x34, 0x55, 0x82, 0xca, + 0x59, 0xea, 0xf7, 0xbe, 0x51, 0x2a, 0x0e, 0x1c, 0xf1, 0xe1, 0x3c, 0xfa, 0xda, 0x84, 0x44, 0xb3, + 0xcd, 0xc9, 0xf7, 0x36, 0xa1, 0x68, 0xb1, 0x34, 0x24, 0x37, 0xb9, 0x50, 0x1c, 0xb8, 0x51, 0x18, + 0x95, 0x5f, 0x58, 0xa3, 0x51, 0x7f, 0x80, 0x8e, 0xbe, 0x02, 0x27, 0xc2, 0x12, 0xb9, 0xa5, 0xcd, + 0xf4, 0xb9, 0x91, 0x68, 0xb9, 0x61, 0x83, 0x43, 0x56, 0xdf, 0x4b, 0xf4, 0x0a, 0x4c, 0xb2, 0x5e, + 0x43, 0x54, 0x64, 0xd4, 0x0e, 0x78, 0x4f, 0x87, 0xb1, 0xf2, 0xc4, 0xf8, 0x1d, 0x9e, 0x7f, 0x23, + 0xb7, 0xcc, 0x66, 0x69, 0x76, 0xa0, 0xf1, 0xf7, 0xe7, 0x13, 0x11, 0xe3, 0x77, 0x7c, 0x2a, 0xe9, + 0x77, 0x8b, 0xbd, 0xe1, 0xe9, 0x1a, 0x27, 0x06, 0xf6, 0x7b, 0x44, 0x5a, 0x4e, 0x8d, 0xe6, 0x4e, + 0xfb, 0x64, 0x52, 0x35, 0x8b, 0x5d, 0xcf, 0x21, 0xd3, 0x31, 0x75, 0x72, 0x60, 0xd5, 0xfa, 0xaf, + 0x1c, 0xa9, 0xd1, 0x30, 0xc8, 0xa3, 0xa2, 0xfb, 0x20, 0xf2, 0x83, 0xf3, 0xfe, 0xb2, 0xf5, 0x29, + 0x2a, 0xef, 0xf9, 0x48, 0xd7, 0x15, 0x95, 0xdf, 0x50, 0x9b, 0x90, 0xa6, 0xd5, 0xf0, 0x1b, 0xf4, + 0x00, 0x66, 0xa8, 0x3c, 0x59, 0xf5, 0x6f, 0x3c, 0x28, 0x95, 0xfa, 0x12, 0x05, 0x07, 0x5f, 0x8e, + 0xe0, 0x4a, 0x16, 0xd5, 0x9e, 0x57, 0xc4, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x97, 0x9d, 0x1b, 0x68, + 0xc6, 0xe1, 0x1b, 0xce, 0x88, 0x19, 0xeb, 0x8c, 0x42, 0xcc, 0xd8, 0xe1, 0xb9, 0x3c, 0xbc, 0x3b, + 0x9e, 0x1d, 0x68, 0xc6, 0x51, 0x49, 0x3f, 0xc4, 0x8c, 0x9d, 0x20, 0x9d, 0x98, 0x31, 0x73, 0x10, + 0x3d, 0x72, 0x9f, 0x1b, 0x68, 0xc6, 0x03, 0x0f, 0x92, 0x12, 0x33, 0x56, 0xfa, 0x5e, 0xa2, 0x15, + 0x00, 0x16, 0x9e, 0xe8, 0xc6, 0xb6, 0x59, 0x3a, 0x3b, 0x70, 0x32, 0xe8, 0xcd, 0xe6, 0x21, 0x93, + 0x41, 0xcb, 0xa5, 0x11, 0x47, 0x46, 0xb1, 0x91, 0x4c, 0x77, 0xc9, 0x4a, 0xf3, 0x03, 0x1d, 0x59, + 0xdf, 0x66, 0x19, 0x71, 0x64, 0x7b, 0x1e, 0x91, 0xcc, 0x2a, 0x6c, 0x8d, 0xb5, 0xb4, 0x30, 0xd8, + 0x2d, 0x07, 0x77, 0x17, 0xa8, 0x5b, 0xa6, 0x04, 0xb4, 0x04, 0x59, 0x32, 0x6d, 0x1f, 0x50, 0x37, + 0x74, 0x6e, 0x60, 0xc0, 0xd9, 0x73, 0x38, 0xa0, 0x36, 0x21, 0x65, 0x1e, 0x73, 0x12, 0xf9, 0x3c, + 0x5b, 0xb5, 0x2a, 0x95, 0x07, 0x7e, 0x3e, 0xb4, 0x52, 0x49, 0x3e, 0xcf, 0x38, 0x90, 0x0a, 0x27, + 0x59, 0x5f, 0xf1, 0x73, 0x9c, 0x16, 0x3f, 0x74, 0x58, 0x3a, 0x4f, 0x45, 0x0d, 0x5c, 0x03, 0x8a, + 0x3c, 0x5e, 0x5a, 0x9b, 0x90, 0x66, 0x95, 0xfe, 0xb7, 0x64, 0xc0, 0xf3, 0xa9, 0x87, 0xad, 0x1c, + 0x95, 0x2e, 0x0c, 0x1c, 0xf0, 0x11, 0x6b, 0x6d, 0x64, 0xc0, 0x2b, 0x01, 0x32, 0x9b, 0x80, 0x34, + 0xd9, 0xb6, 0xd9, 0x9e, 0xea, 0xc5, 0x21, 0x13, 0x50, 0x0f, 0xf6, 0x67, 0x13, 0x90, 0x56, 0x67, + 0x9c, 0x44, 0x90, 0xda, 0xc2, 0x8a, 0xc5, 0xdd, 0xec, 0xa5, 0x81, 0x82, 0xfa, 0x6e, 0x0d, 0x23, + 0x82, 0x54, 0x8f, 0x48, 0x02, 0x1e, 0xcb, 0xbd, 0x71, 0x83, 0x87, 0x7e, 0x97, 0x07, 0x06, 0x3c, + 0x91, 0x17, 0x83, 0x90, 0x80, 0xc7, 0x0a, 0xbd, 0x40, 0x9f, 0x87, 0x29, 0x8e, 0xd0, 0x4a, 0x57, + 0x86, 0x04, 0xa4, 0x41, 0x68, 0x4d, 0xc6, 0x35, 0xe7, 0x61, 0x5e, 0x96, 0x21, 0x43, 0xd6, 0xbc, + 0xe7, 0x87, 0x78, 0xd9, 0x3e, 0x70, 0xca, 0xbc, 0xac, 0x4f, 0x26, 0x5e, 0x96, 0xd9, 0x29, 0x9f, + 0xeb, 0xae, 0x0e, 0xf4, 0xb2, 0xfd, 0x27, 0x13, 0x88, 0x97, 0x7d, 0xec, 0x53, 0x49, 0xcb, 0x6c, + 0x86, 0x8a, 0x4a, 0x9f, 0x19, 0xd8, 0xb2, 0x30, 0x48, 0x24, 0x2d, 0xe3, 0x3c, 0xa4, 0xdb, 0x58, + 0x12, 0x29, 0xd3, 0xf4, 0x0b, 0x83, 0x8f, 0x4c, 0xf7, 0x82, 0x88, 0x9a, 0x7b, 0xa3, 0x2c, 0xd3, + 0xb0, 0xe7, 0xa8, 0x2c, 0x7e, 0x40, 0x94, 0x6b, 0xea, 0xc5, 0xe1, 0x8e, 0x2a, 0xea, 0xec, 0xab, + 0xe7, 0xa8, 0x42, 0x2f, 0x69, 0x55, 0xd9, 0x39, 0x1f, 0x3a, 0xbe, 0x17, 0x87, 0x9c, 0xee, 0xee, + 0x39, 0x7a, 0x45, 0xab, 0xea, 0x11, 0xfd, 0x21, 0xd4, 0x65, 0xd7, 0x10, 0x94, 0xae, 0x0d, 0x1f, + 0x42, 0xe1, 0xeb, 0x10, 0xbc, 0x21, 0xc4, 0xc9, 0xde, 0x9c, 0xe9, 0x46, 0x18, 0x2f, 0x0d, 0x9f, + 0x33, 0x7b, 0x43, 0x0b, 0x36, 0x67, 0xf2, 0x98, 0xe2, 0xff, 0x08, 0xb0, 0xc0, 0xea, 0x46, 0xd7, + 0xf1, 0x0e, 0x64, 0x6f, 0x4d, 0x34, 0x90, 0x6a, 0x7e, 0x9d, 0x7e, 0xe0, 0x95, 0x41, 0xd5, 0x1d, + 0xb1, 0xc6, 0x5b, 0x9b, 0x90, 0x9e, 0x53, 0x86, 0x95, 0x5b, 0x9e, 0xe2, 0x3b, 0xa7, 0xde, 0x89, + 0xbb, 0x69, 0x51, 0xbc, 0x93, 0xca, 0x9c, 0x16, 0x4b, 0x77, 0x52, 0x99, 0x33, 0xe2, 0xdc, 0x9d, + 0x54, 0xe6, 0x19, 0xf1, 0xd9, 0xf2, 0x3f, 0x9d, 0x81, 0x82, 0x8b, 0xe1, 0x18, 0x22, 0xba, 0x11, + 0x44, 0x44, 0x67, 0x07, 0x21, 0x22, 0x8e, 0xfa, 0x38, 0x24, 0xba, 0x11, 0x84, 0x44, 0x67, 0x07, + 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, 0xc6, 0x20, 0x4c, 0xf4, 0xfc, 0x18, 0x98, 0xc8, 0x13, 0xd5, + 0x0b, 0x8a, 0x56, 0xfa, 0x41, 0xd1, 0x85, 0xe1, 0xa0, 0xc8, 0x13, 0x15, 0x40, 0x45, 0xaf, 0xf7, + 0xa0, 0xa2, 0x73, 0x43, 0x50, 0x91, 0xc7, 0xef, 0xc2, 0xa2, 0xb5, 0x48, 0x58, 0x74, 0x69, 0x14, + 0x2c, 0xf2, 0xe4, 0x84, 0x70, 0xd1, 0xcb, 0x21, 0x5c, 0x34, 0x3f, 0x10, 0x17, 0x79, 0xdc, 0x0c, + 0x18, 0xbd, 0xd1, 0x0b, 0x8c, 0xce, 0x0d, 0x01, 0x46, 0x7e, 0x0b, 0x38, 0x32, 0xaa, 0x45, 0x21, + 0xa3, 0x8b, 0x23, 0x90, 0x91, 0x27, 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xe2, 0x08, 0x68, + 0xd4, 0x23, 0x89, 0x61, 0xa3, 0x8d, 0x68, 0x6c, 0x74, 0x79, 0x24, 0x36, 0xf2, 0xa4, 0x85, 0xc1, + 0xd1, 0xb5, 0x00, 0x38, 0x7a, 0x6e, 0x00, 0x38, 0xf2, 0x58, 0x09, 0x3a, 0xfa, 0x42, 0x1f, 0x3a, + 0x2a, 0x0f, 0x43, 0x47, 0x1e, 0xaf, 0x07, 0x8f, 0xde, 0x1a, 0x00, 0x8f, 0xae, 0x8c, 0x86, 0x47, + 0x9e, 0xb0, 0x1e, 0x7c, 0xa4, 0x0c, 0xc5, 0x47, 0x2f, 0x8e, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, + 0xe9, 0xd5, 0x30, 0x40, 0x5a, 0x18, 0x0c, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0xad, 0x45, 0x22, 0xa4, + 0x4b, 0xa3, 0x10, 0x92, 0x3f, 0x0e, 0x82, 0x10, 0x69, 0x23, 0x1a, 0x22, 0x5d, 0x1e, 0x09, 0x91, + 0xfc, 0xee, 0x0f, 0x61, 0xa4, 0xb5, 0x48, 0x8c, 0x74, 0x69, 0x14, 0x46, 0xf2, 0x2b, 0x17, 0x04, + 0x49, 0x6f, 0x0f, 0x04, 0x49, 0x57, 0xc7, 0x01, 0x49, 0x9e, 0xd0, 0x3e, 0x94, 0xf4, 0xce, 0x60, + 0x94, 0xf4, 0x99, 0x23, 0x5c, 0x21, 0x17, 0x09, 0x93, 0xbe, 0xd0, 0x07, 0x93, 0xca, 0xc3, 0x60, + 0x92, 0x6f, 0xcf, 0x2e, 0x4e, 0x52, 0x86, 0xa2, 0x9a, 0x17, 0xc7, 0x44, 0x35, 0xbe, 0xf1, 0x45, + 0xc0, 0x9a, 0x6a, 0x04, 0xac, 0xb9, 0x30, 0x1c, 0xd6, 0xf8, 0xee, 0xdc, 0xc7, 0x35, 0xb5, 0x28, + 0x5c, 0x73, 0x71, 0x04, 0xae, 0xf1, 0xbd, 0x50, 0x00, 0xd8, 0xbc, 0xde, 0x03, 0x6c, 0xce, 0x8d, + 0xcc, 0xb2, 0x09, 0x20, 0x9b, 0xe5, 0x7e, 0x64, 0x73, 0x7e, 0x28, 0xb2, 0xf1, 0x24, 0xf8, 0xd0, + 0xe6, 0xf5, 0x1e, 0x68, 0x73, 0x6e, 0x08, 0xb4, 0xf1, 0x2b, 0xc0, 0xb1, 0x8d, 0x36, 0x1c, 0xdb, + 0x2c, 0x8e, 0x8b, 0x6d, 0x3c, 0xc1, 0x91, 0xe0, 0x66, 0x23, 0x1a, 0xdc, 0x5c, 0x1e, 0x73, 0xf7, + 0xbc, 0x0f, 0xdd, 0xd4, 0xa2, 0xd0, 0xcd, 0xc5, 0x11, 0xe8, 0x26, 0x38, 0x87, 0x78, 0xf0, 0xa6, + 0x16, 0x05, 0x6f, 0x2e, 0x8e, 0x80, 0x37, 0xbe, 0xa4, 0x00, 0xbe, 0x69, 0x0c, 0xc2, 0x37, 0xcf, + 0x8f, 0x81, 0x6f, 0xfc, 0xe0, 0xa5, 0x07, 0xe0, 0xbc, 0xd9, 0x0b, 0x70, 0xca, 0xc3, 0x00, 0x8e, + 0x3f, 0x22, 0x5d, 0x84, 0xb3, 0x11, 0x8d, 0x70, 0x2e, 0x8f, 0x44, 0x38, 0x41, 0x27, 0x19, 0x80, + 0x38, 0x6b, 0x91, 0x10, 0xe7, 0xd2, 0x28, 0x88, 0xe3, 0x3b, 0xc9, 0x20, 0xc6, 0x79, 0xb3, 0x17, + 0xe3, 0x94, 0x87, 0x61, 0x1c, 0xbf, 0x71, 0x2e, 0xc8, 0xa9, 0x45, 0x81, 0x9c, 0x8b, 0x23, 0x40, + 0x8e, 0xdf, 0x79, 0x01, 0x94, 0xa3, 0x0c, 0x45, 0x39, 0x2f, 0x8e, 0x89, 0x72, 0x7a, 0x1c, 0x57, + 0x18, 0xe6, 0xd4, 0xa2, 0x60, 0xce, 0xc5, 0x11, 0x30, 0x27, 0x50, 0x59, 0x1f, 0xe7, 0x6c, 0x44, + 0xe3, 0x9c, 0xcb, 0x23, 0x71, 0x4e, 0xcf, 0x68, 0x72, 0x81, 0xce, 0x5a, 0x24, 0xd0, 0xb9, 0x34, + 0x0a, 0xe8, 0xf4, 0x4c, 0x7c, 0x3c, 0x38, 0xf8, 0xbf, 0xe3, 0x23, 0x9d, 0x57, 0x8f, 0x8e, 0x74, + 0xbc, 0x6f, 0xc6, 0x02, 0x75, 0xee, 0xa4, 0x32, 0xcf, 0x8a, 0xcf, 0x95, 0xbf, 0x3f, 0x09, 0xe9, + 0x9a, 0x97, 0xe3, 0xe2, 0xd7, 0x52, 0x38, 0xce, 0xed, 0x35, 0x68, 0x85, 0x8c, 0x58, 0xea, 0xf7, + 0x46, 0x5f, 0x54, 0xd6, 0x7f, 0x89, 0x16, 0x67, 0x3d, 0xc6, 0x21, 0x50, 0xf4, 0x32, 0x14, 0xba, + 0x36, 0xb6, 0xe4, 0x8e, 0xa5, 0x9b, 0x96, 0xee, 0xb0, 0x74, 0x7d, 0x61, 0x59, 0xfc, 0xe8, 0x70, + 0x3e, 0xbf, 0x65, 0x63, 0x6b, 0x93, 0xd3, 0xa5, 0x7c, 0x37, 0xf0, 0xe4, 0xfe, 0xfc, 0xcd, 0xe4, + 0xf8, 0x3f, 0x7f, 0xf3, 0x16, 0x88, 0x16, 0x56, 0xb4, 0x50, 0x04, 0xc2, 0xae, 0x85, 0x89, 0xb6, + 0x19, 0x7a, 0x56, 0xc5, 0x2d, 0x49, 0xaf, 0x87, 0x99, 0xb6, 0xc2, 0x44, 0x74, 0x1d, 0x4e, 0xb6, + 0x95, 0x7d, 0x9a, 0x83, 0x28, 0xbb, 0x41, 0x1d, 0xcd, 0x2b, 0xcc, 0xd0, 0x7c, 0x59, 0xd4, 0x56, + 0xf6, 0xe9, 0x6f, 0xe9, 0xb0, 0x57, 0xf4, 0x16, 0xfd, 0x8b, 0x50, 0xd4, 0x74, 0xdb, 0xd1, 0x0d, + 0xd5, 0xbd, 0xff, 0x93, 0x5d, 0xb4, 0x59, 0x70, 0xa9, 0xec, 0x1e, 0xce, 0xab, 0x30, 0xc3, 0xf3, + 0xb1, 0xfd, 0x5f, 0xd7, 0xa1, 0xf0, 0x25, 0x43, 0x6a, 0x41, 0x5e, 0xf8, 0x3f, 0xab, 0x54, 0x81, + 0xe9, 0xa6, 0xe2, 0xe0, 0x3d, 0xe5, 0x40, 0x76, 0x8f, 0xcb, 0xe4, 0xe8, 0xb5, 0x7a, 0xcf, 0x3c, + 0x3d, 0x9c, 0x2f, 0xdc, 0x66, 0xaf, 0xfa, 0x4e, 0xcd, 0x14, 0x9a, 0x81, 0x17, 0x1a, 0xba, 0x0c, + 0xd3, 0x8a, 0x7d, 0x60, 0xa8, 0x54, 0x3d, 0xd8, 0xb0, 0xbb, 0x36, 0x85, 0x14, 0x19, 0xa9, 0x48, + 0xc9, 0x15, 0x97, 0x8a, 0x5e, 0x85, 0x33, 0x1a, 0x26, 0x31, 0x12, 0x0b, 0x44, 0x1c, 0xd3, 0x94, + 0xcd, 0x96, 0x26, 0xd3, 0xeb, 0x1f, 0x28, 0x9c, 0xc8, 0x48, 0x27, 0x69, 0x01, 0x1a, 0x82, 0x34, + 0x4c, 0xf3, 0x5e, 0x4b, 0xab, 0x92, 0x97, 0xec, 0xaa, 0xcf, 0x3b, 0xa9, 0x4c, 0x5e, 0x2c, 0x94, + 0x7f, 0x45, 0x80, 0x7c, 0xe8, 0x8c, 0xc2, 0xeb, 0x3d, 0x5b, 0xb4, 0x67, 0xa2, 0xe1, 0xd0, 0xa0, + 0x1c, 0xaf, 0x0c, 0x57, 0xbf, 0x9b, 0xe1, 0x36, 0x3f, 0x38, 0x9c, 0xa6, 0x8b, 0x03, 0x6e, 0x7a, + 0x80, 0xcb, 0xf6, 0x5a, 0xea, 0xd7, 0xde, 0x9f, 0x9f, 0x28, 0xff, 0x3c, 0x09, 0x85, 0xf0, 0x59, + 0x84, 0xd5, 0x9e, 0x7a, 0x45, 0xb9, 0xab, 0x10, 0xc7, 0xe2, 0x90, 0xfb, 0xcf, 0xb2, 0xfe, 0x7d, + 0xdc, 0xac, 0x9a, 0x0b, 0x43, 0x36, 0xa2, 0x83, 0xf5, 0xf4, 0x19, 0xe7, 0x7e, 0x90, 0xf0, 0x86, + 0xfd, 0x22, 0x4c, 0x32, 0xb5, 0x0b, 0x03, 0xcf, 0x86, 0x52, 0xcd, 0x4b, 0xac, 0x18, 0x71, 0x13, + 0x8d, 0x63, 0x5d, 0x72, 0xe5, 0x5f, 0x0f, 0x70, 0xf4, 0x5f, 0x9d, 0xe2, 0x57, 0x9d, 0x4d, 0x1e, + 0xed, 0xaa, 0x33, 0xb6, 0xd1, 0xdc, 0x6a, 0x31, 0x17, 0xcc, 0x06, 0x4a, 0xba, 0xef, 0x00, 0x26, + 0x15, 0xc1, 0x7f, 0x94, 0x6c, 0x51, 0xe2, 0x3f, 0x4a, 0x16, 0x48, 0x3a, 0x2c, 0x7a, 0x22, 0xe8, + 0xa8, 0x62, 0xa9, 0xa9, 0xbc, 0xab, 0xbf, 0x23, 0x80, 0x48, 0xc7, 0xd0, 0x2d, 0x8c, 0xb5, 0x58, + 0xac, 0xd0, 0xcd, 0x87, 0x4c, 0x8c, 0x9f, 0x26, 0x1e, 0xba, 0x1f, 0x3d, 0x19, 0xbe, 0x1f, 0xbd, + 0xfc, 0xbe, 0x00, 0x45, 0xaf, 0x86, 0xec, 0x27, 0x78, 0x86, 0xdc, 0x60, 0x76, 0xbc, 0x5f, 0xa5, + 0x71, 0x0f, 0x57, 0x8f, 0xf5, 0x5b, 0x40, 0xc1, 0xc3, 0xd5, 0xec, 0x17, 0x54, 0xbe, 0x2d, 0xc0, + 0xac, 0x57, 0xc5, 0x8a, 0x7f, 0x70, 0xf6, 0x18, 0x19, 0xf3, 0x12, 0xfd, 0x75, 0x32, 0x02, 0xda, + 0xe9, 0xa9, 0xf6, 0xb1, 0xcc, 0x13, 0xf1, 0x84, 0x0a, 0xe0, 0x8b, 0x01, 0x5a, 0xa3, 0x4e, 0x7f, + 0xb7, 0x8c, 0xfd, 0x6f, 0x97, 0x6f, 0x05, 0x14, 0x48, 0x47, 0x02, 0xd1, 0xd2, 0x58, 0x43, 0xc6, + 0xd5, 0x12, 0x2d, 0x5c, 0xfe, 0x51, 0xb0, 0x27, 0xaa, 0xbb, 0x24, 0x08, 0xbc, 0x09, 0xc9, 0x5d, + 0xa5, 0x35, 0x2c, 0x9f, 0x24, 0xd4, 0x73, 0x12, 0x29, 0x8d, 0x6e, 0x85, 0xce, 0x1b, 0x27, 0x06, + 0x07, 0x2c, 0xfd, 0x2a, 0x0d, 0x9d, 0x4b, 0x7e, 0xc5, 0x6d, 0x45, 0x72, 0xf4, 0xe7, 0x83, 0x1e, + 0xe0, 0xb5, 0xd4, 0x07, 0xef, 0xcf, 0x0b, 0x57, 0xeb, 0x30, 0x1b, 0x31, 0xbd, 0xa1, 0x22, 0x40, + 0xe0, 0xd6, 0x74, 0xfe, 0xe3, 0x68, 0x4b, 0x2b, 0xf2, 0xd6, 0x46, 0xe5, 0xde, 0xfa, 0xfa, 0x6a, + 0xa3, 0x51, 0x5d, 0x11, 0x05, 0x24, 0x42, 0x3e, 0x74, 0xe7, 0x3a, 0xff, 0x4d, 0xb4, 0xab, 0xff, + 0x0d, 0xc0, 0xff, 0xdd, 0x05, 0x22, 0x6b, 0xad, 0xfa, 0x40, 0xbe, 0xbf, 0x74, 0x77, 0xab, 0x5a, + 0x17, 0x27, 0x10, 0x82, 0xe2, 0xf2, 0x52, 0xa3, 0x52, 0x93, 0xa5, 0x6a, 0x7d, 0xf3, 0xde, 0x46, + 0xbd, 0x2a, 0x0a, 0x9c, 0x6f, 0x05, 0xf2, 0xc1, 0x93, 0xd9, 0x68, 0x16, 0xa6, 0x2b, 0xb5, 0x6a, + 0x65, 0x4d, 0xbe, 0xbf, 0xba, 0x24, 0xbf, 0xb5, 0x55, 0xdd, 0xaa, 0x8a, 0x13, 0xb4, 0x6a, 0x94, + 0x78, 0x6b, 0xeb, 0xee, 0x5d, 0x51, 0x40, 0xd3, 0x90, 0x63, 0xcf, 0xf4, 0x7e, 0x76, 0x31, 0x71, + 0x75, 0x1d, 0x72, 0x81, 0x8b, 0xdc, 0xc8, 0xe7, 0x36, 0xb7, 0xea, 0x35, 0xb9, 0xb1, 0xba, 0x5e, + 0xad, 0x37, 0x96, 0xd6, 0x37, 0x99, 0x0c, 0x4a, 0x5b, 0x5a, 0xbe, 0x27, 0x35, 0x44, 0xc1, 0x7b, + 0x6e, 0xdc, 0xdb, 0xaa, 0xd4, 0xbc, 0x9f, 0x76, 0x4b, 0x65, 0x92, 0x62, 0xf2, 0xea, 0x63, 0x38, + 0x3d, 0xe0, 0x78, 0x32, 0xca, 0xc1, 0xd4, 0x96, 0x41, 0xaf, 0xae, 0x12, 0x27, 0x50, 0x21, 0x70, + 0x42, 0x59, 0x14, 0x50, 0x86, 0x9d, 0x3d, 0x15, 0x13, 0x28, 0x0d, 0x89, 0xfa, 0x4d, 0x31, 0x49, + 0x2a, 0x1a, 0x38, 0xe0, 0x2b, 0xa6, 0x50, 0x96, 0x9f, 0x7e, 0x14, 0x27, 0x51, 0xde, 0x3f, 0x7e, + 0x28, 0xa6, 0xaf, 0x9e, 0x83, 0xc0, 0xd9, 0x2c, 0x04, 0x90, 0xbe, 0xab, 0x38, 0xd8, 0x76, 0xc4, + 0x09, 0x34, 0x05, 0xc9, 0xa5, 0x56, 0x4b, 0x14, 0x6e, 0xfc, 0xbe, 0x00, 0x19, 0xf7, 0xaa, 0x71, + 0x74, 0x17, 0x26, 0x19, 0x92, 0x9f, 0x1f, 0x3c, 0x23, 0x51, 0xa7, 0x36, 0xb7, 0x30, 0x6a, 0xca, + 0x2a, 0x4f, 0xa0, 0xb7, 0xf9, 0xef, 0x34, 0x12, 0x8b, 0x41, 0xe7, 0x87, 0xd9, 0x93, 0x2b, 0x75, + 0xb8, 0xd1, 0x91, 0x31, 0x52, 0x9e, 0x78, 0x49, 0x58, 0x7e, 0xfe, 0x83, 0x9f, 0x9e, 0x9d, 0xf8, + 0xe0, 0xe9, 0x59, 0xe1, 0xc3, 0xa7, 0x67, 0x85, 0x9f, 0x3c, 0x3d, 0x2b, 0xfc, 0xfd, 0xd3, 0xb3, + 0xc2, 0x2f, 0xfd, 0xec, 0xec, 0xc4, 0x87, 0x3f, 0x3b, 0x3b, 0xf1, 0x93, 0x9f, 0x9d, 0x9d, 0x78, + 0x67, 0x8a, 0x73, 0x3f, 0x4c, 0xd3, 0x9f, 0x8c, 0xbc, 0xf9, 0x6f, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x5e, 0xd6, 0x84, 0x9e, 0x37, 0x73, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 35fa87c95adc..ed95c4303440 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -348,6 +348,8 @@ message ScanRequest { // will set the batch_responses field in the ScanResponse instead of the rows // field. ScanFormat scan_format = 4; + + bool select_for_update = 5; } // A ScanResponse is the return value from the Scan() method. diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index e7549f4a947d..ff36901c1629 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -99,6 +99,17 @@ func (ba *BatchRequest) IsReadOnly() bool { return len(ba.Requests) > 0 && !ba.hasFlag(isWrite|isAdmin) } +func (ba *BatchRequest) HasSelectForUpdate() bool { + for _, union := range ba.Requests { + if scan, ok := union.GetInner().(*ScanRequest); ok { + if scan.SelectForUpdate { + return true + } + } + } + return false +} + // RequiresLeaseHolder returns true if the request can only be served by the // leaseholders of the ranges it addresses. func (ba *BatchRequest) RequiresLeaseHolder() bool { @@ -359,7 +370,9 @@ func (ba *BatchRequest) IntentSpanIterate(br *BatchResponse, fn func(Span)) { for i, arg := range ba.Requests { req := arg.GetInner() if !IsTransactionWrite(req) { - continue + if scan, ok := req.(*ScanRequest); !ok || !scan.SelectForUpdate { + continue + } } var resp Response if br != nil { diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index adad6712b545..0e7380d384d5 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -413,6 +413,10 @@ func (dsp *DistSQLPlanner) checkSupportForNode(node planNode) (distRecommendatio return dsp.checkSupportForNode(n.source.plan) case *scanNode: + if n.lockingStrength != sqlbase.ScanLockingStrength_FOR_NONE { + return cannotDistribute, newQueryNotSupportedErrorf("can't distribute locking scans") + } + rec := canDistribute if n.softLimit != 0 { // We don't yet recommend distributing plans where soft limits propagate diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index 42a9e1de7bfd..01725c9aee2a 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -246,7 +246,9 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { for i := range f.spans { scans[i].ScanFormat = roachpb.BATCH_RESPONSE scans[i].SetSpan(f.spans[i]) - // TODO(nvanbenschoten): use f.lockStr here. + if f.lockStr != sqlbase.ScanLockingStrength_FOR_NONE { + scans[i].SelectForUpdate = true + } ba.Requests[i].MustSetInner(&scans[i]) } } diff --git a/pkg/storage/batcheval/cmd_end_transaction.go b/pkg/storage/batcheval/cmd_end_transaction.go index 4e9be72c4efa..85af1ac6e3bb 100644 --- a/pkg/storage/batcheval/cmd_end_transaction.go +++ b/pkg/storage/batcheval/cmd_end_transaction.go @@ -494,8 +494,8 @@ func resolveLocalIntents( ok, err := engine.MVCCResolveWriteIntentUsingIter(ctx, readWriter, iterAndBuf, resolveMS, intent) if ok { resolveAllowance-- - resolvedIntents = append(resolvedIntents, intent) } + resolvedIntents = append(resolvedIntents, intent) return err } // For intent ranges, cut into parts inside and outside our key diff --git a/pkg/storage/batcheval/cmd_scan.go b/pkg/storage/batcheval/cmd_scan.go index 3beb1b018b0c..948d55fd9451 100644 --- a/pkg/storage/batcheval/cmd_scan.go +++ b/pkg/storage/batcheval/cmd_scan.go @@ -14,13 +14,33 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/batcheval/result" "github.com/cockroachdb/cockroach/pkg/storage/engine" + "github.com/cockroachdb/cockroach/pkg/storage/spanset" ) func init() { - RegisterReadOnlyCommand(roachpb.Scan, DefaultDeclareKeys, Scan) + RegisterReadOnlyCommand(roachpb.Scan, scanDeclareKeys, Scan) +} + +func scanDeclareKeys( + desc *roachpb.RangeDescriptor, header roachpb.Header, req roachpb.Request, spans *spanset.SpanSet, +) { + scan := req.(*roachpb.ScanRequest) + var access spanset.SpanAccess + if scan.SelectForUpdate { + access = spanset.SpanReadWrite + } else { + access = spanset.SpanReadOnly + } + + if keys.IsLocal(scan.Key) { + spans.AddNonMVCC(access, scan.Span()) + } else { + spans.AddMVCC(access, scan.Span(), header.Timestamp) + } } // Scan scans the key range specified by start key through end key @@ -34,9 +54,17 @@ func Scan( h := cArgs.Header reply := resp.(*roachpb.ScanResponse) + var writeTooOldOnWriteInFuture bool + if args.SelectForUpdate { + writeTooOldOnWriteInFuture = true + } else { + writeTooOldOnWriteInFuture = false + } + var err error var intents []roachpb.Intent var resumeSpan *roachpb.Span + var res result.Result switch args.ScanFormat { case roachpb.BATCH_RESPONSE: @@ -45,26 +73,48 @@ func Scan( kvData, numKvs, resumeSpan, intents, err = engine.MVCCScanToBytes( ctx, reader, args.Key, args.EndKey, cArgs.MaxKeys, h.Timestamp, engine.MVCCScanOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, - Txn: h.Txn, + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + WriteTooOldOnWriteInFuture: writeTooOldOnWriteInFuture, }) if err != nil { return result.Result{}, err } reply.NumKeys = numKvs reply.BatchResponses = kvData + if args.SelectForUpdate && h.Txn != nil { + res.Local.UpdatedIntents = make([]roachpb.Intent, numKvs) + var i int + if err := engine.MVCCScanDecodeKeyValues(kvData, func(key engine.MVCCKey, _ []byte) error { + res.Local.UpdatedIntents[i] = roachpb.Intent{ + Span: roachpb.Span{Key: key.Key}, Txn: h.Txn.TxnMeta, Status: roachpb.PENDING, + } + return nil + }); err != nil { + return result.Result{}, err + } + } case roachpb.KEY_VALUES: var rows []roachpb.KeyValue rows, resumeSpan, intents, err = engine.MVCCScan( ctx, reader, args.Key, args.EndKey, cArgs.MaxKeys, h.Timestamp, engine.MVCCScanOptions{ - Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, - Txn: h.Txn, + Inconsistent: h.ReadConsistency != roachpb.CONSISTENT, + Txn: h.Txn, + WriteTooOldOnWriteInFuture: writeTooOldOnWriteInFuture, }) if err != nil { return result.Result{}, err } reply.NumKeys = int64(len(rows)) reply.Rows = rows + if args.SelectForUpdate && h.Txn != nil { + res.Local.UpdatedIntents = make([]roachpb.Intent, len(rows)) + for i, row := range rows { + res.Local.UpdatedIntents[i] = roachpb.Intent{ + Span: roachpb.Span{Key: row.Key}, Txn: h.Txn.TxnMeta, Status: roachpb.PENDING, + } + } + } default: panic(fmt.Sprintf("Unknown scanFormat %d", args.ScanFormat)) } @@ -77,5 +127,6 @@ func Scan( if h.ReadConsistency == roachpb.READ_UNCOMMITTED { reply.IntentRows, err = CollectIntentRows(ctx, reader, cArgs, intents) } - return result.FromEncounteredIntents(intents), err + res.Local.EncounteredIntents = intents + return res, err } diff --git a/pkg/storage/concurrency/lock_table.go b/pkg/storage/concurrency/lock_table.go index d9fc5069e8a9..1617e80155ed 100644 --- a/pkg/storage/concurrency/lock_table.go +++ b/pkg/storage/concurrency/lock_table.go @@ -81,25 +81,38 @@ func (lt *lockTableImpl) AcquireLock(in roachpb.Intent) { func (lt *lockTableImpl) ReleaseLock(in roachpb.Intent) { lt.mu.Lock() defer lt.mu.Unlock() - lt.tmp1.in = in - wqI := lt.qs.Get(<.tmp1) - if wqI == nil { - return - } - wq := wqI.(*perKeyWaitQueue) - if !wq.held || wq.in.Txn.ID != in.Txn.ID { - return + + var toDelete []*perKeyWaitQueue + maybeRelease := func(i btree.Item) bool { + wq := i.(*perKeyWaitQueue) + if !wq.held || wq.in.Txn.ID != in.Txn.ID { + return true + } + if wq.ll.Len() == 0 { + toDelete = append(toDelete, wq) + return true + } + wq.in.Txn = enginepb.TxnMeta{} + wq.held = false + front := wq.ll.Front().Value.(*perKeyWaitQueueElem) + if !front.closed { + close(front.done) + front.closed = true + } + return true } - if wq.ll.Len() == 0 { - lt.qs.Delete(wq) - return + + lt.tmp1.in.Key = in.Key + if in.EndKey == nil { + if i := lt.qs.Get(<.tmp1); i != nil { + maybeRelease(i) + } + } else { + lt.tmp2.in.Key = in.EndKey + lt.qs.AscendRange(<.tmp1, <.tmp2, maybeRelease) } - wq.in.Txn = enginepb.TxnMeta{} - wq.held = false - front := wq.ll.Front().Value.(*perKeyWaitQueueElem) - if !front.closed { - close(front.done) - front.closed = true + for _, d := range toDelete { + lt.qs.Delete(d) } } diff --git a/pkg/storage/engine/mvcc.go b/pkg/storage/engine/mvcc.go index f419cd7cda3c..c7b9a51dd1e5 100644 --- a/pkg/storage/engine/mvcc.go +++ b/pkg/storage/engine/mvcc.go @@ -2356,20 +2356,15 @@ func mvccScanToKvs( return nil, nil, nil, err } kvs := make([]roachpb.KeyValue, numKVs) - var k MVCCKey - var rawBytes []byte var i int - for _, data := range kvData { - for len(data) > 0 { - k, rawBytes, data, err = MVCCScanDecodeKeyValue(data) - if err != nil { - return nil, nil, nil, err - } - kvs[i].Key = k.Key - kvs[i].Value.RawBytes = rawBytes - kvs[i].Value.Timestamp = k.Timestamp - i++ - } + if err := MVCCScanDecodeKeyValues(kvData, func(key MVCCKey, rawBytes []byte) error { + kvs[i].Key = key.Key + kvs[i].Value.RawBytes = rawBytes + kvs[i].Value.Timestamp = key.Timestamp + i++ + return nil + }); err != nil { + return nil, nil, nil, err } return kvs, resumeSpan, intents, err } @@ -2411,10 +2406,11 @@ type MVCCScanOptions struct { // to return no results. // See the documentation for MVCCScan for information on these parameters. - Inconsistent bool - Tombstones bool - Reverse bool - Txn *roachpb.Transaction + Inconsistent bool + Tombstones bool + Reverse bool + Txn *roachpb.Transaction + WriteTooOldOnWriteInFuture bool } // MVCCScan scans the key range [key, endKey) in the provided reader up to some diff --git a/pkg/storage/engine/rocksdb.go b/pkg/storage/engine/rocksdb.go index 87266e0f1093..8200f41d2a2b 100644 --- a/pkg/storage/engine/rocksdb.go +++ b/pkg/storage/engine/rocksdb.go @@ -2484,6 +2484,7 @@ func (r *rocksDBIterator) MVCCScan( goToCTimestamp(timestamp), C.int64_t(max), goToCTxn(opts.Txn), C.bool(opts.Inconsistent), C.bool(opts.Reverse), C.bool(opts.Tombstones), + C.bool(opts.WriteTooOldOnWriteInFuture), ) if err := statusToError(state.status); err != nil { @@ -2514,6 +2515,9 @@ func (r *rocksDBIterator) MVCCScan( return nil, 0, resumeSpan, nil, &roachpb.WriteIntentError{Intents: intents} } + if err := writeTooOldToError(timestamp, state.write_too_old, opts.Txn); err != nil { + return nil, 0, nil, nil, err + } return kvData, numKVs, resumeSpan, intents, nil } @@ -2759,6 +2763,23 @@ func uncertaintyToError( return nil } +func writeTooOldToError( + readTS hlc.Timestamp, wtoTS C.DBTimestamp, txn *roachpb.Transaction, +) error { + if wtoTS.wall_time != 0 || wtoTS.logical != 0 { + wtoTimestamp := hlc.Timestamp{ + WallTime: int64(wtoTS.wall_time), + Logical: int32(wtoTS.logical), + } + writeTimestamp := txn.WriteTimestamp + writeTimestamp.Forward(wtoTimestamp.Next()) + return &roachpb.WriteTooOldError{ + Timestamp: readTS, ActualTimestamp: writeTimestamp, + } + } + return nil +} + // goMerge takes existing and update byte slices that are expected to // be marshaled roachpb.Values and merges the two values returning a // marshaled roachpb.Value or an error. @@ -3295,6 +3316,28 @@ func MVCCScanDecodeKeyValue(repr []byte) (key MVCCKey, value []byte, orepr []byt return MVCCKey{k, ts}, value, orepr, err } +// MVCCScanDecodeKeyValues decodes all key/value pairs returned in one or more +// MVCCScan "batches" (this is not the RocksDB batch repr format). The provided +// function is called for each key/value pair. +func MVCCScanDecodeKeyValues(repr [][]byte, fn func(key MVCCKey, rawBytes []byte) error) error { + var k MVCCKey + var rawBytes []byte + var err error + for _, data := range repr { + for len(data) > 0 { + k, rawBytes, data, err = MVCCScanDecodeKeyValue(data) + if err != nil { + return err + } + err = fn(k, rawBytes) + if err != nil { + return err + } + } + } + return nil +} + func notFoundErrOrDefault(err error) error { errStr := err.Error() if strings.Contains(errStr, "No such") || diff --git a/pkg/workload/tpcc/new_order.go b/pkg/workload/tpcc/new_order.go index c120f9ecee9c..9c147cf17771 100644 --- a/pkg/workload/tpcc/new_order.go +++ b/pkg/workload/tpcc/new_order.go @@ -213,15 +213,6 @@ func (n *newOrder) run(ctx context.Context, wID int) (interface{}, error) { err = crdb.ExecuteInTx( ctx, (*workload.PgxTx)(tx), func() error { - // Select the district tax rate and next available order number, bumping it. - var dNextOID int - if err := n.updateDistrict.QueryRowTx( - ctx, tx, d.wID, d.dID, - ).Scan(&d.dTax, &dNextOID); err != nil { - return err - } - d.oID = dNextOID - 1 - // Select the warehouse tax rate. if err := n.selectWarehouseTax.QueryRowTx( ctx, tx, wID, @@ -362,19 +353,6 @@ func (n *newOrder) run(ctx context.Context, wID int) (interface{}, error) { } rows.Close() - // Insert row into the orders and new orders table. - if _, err := n.insertOrder.ExecTx( - ctx, tx, - d.oID, d.dID, d.wID, d.cID, d.oEntryD.Format("2006-01-02 15:04:05"), d.oOlCnt, allLocal, - ); err != nil { - return err - } - if _, err := n.insertNewOrder.ExecTx( - ctx, tx, d.oID, d.dID, d.wID, - ); err != nil { - return err - } - // Update the stock table for each item. if _, err := tx.ExecEx( ctx, @@ -397,6 +375,28 @@ func (n *newOrder) run(ctx context.Context, wID int) (interface{}, error) { return err } + // Select the district tax rate and next available order number, bumping it. + var dNextOID int + if err := n.updateDistrict.QueryRowTx( + ctx, tx, d.wID, d.dID, + ).Scan(&d.dTax, &dNextOID); err != nil { + return err + } + d.oID = dNextOID - 1 + + // Insert row into the orders and new orders table. + if _, err := n.insertOrder.ExecTx( + ctx, tx, + d.oID, d.dID, d.wID, d.cID, d.oEntryD.Format("2006-01-02 15:04:05"), d.oOlCnt, allLocal, + ); err != nil { + return err + } + if _, err := n.insertNewOrder.ExecTx( + ctx, tx, d.oID, d.dID, d.wID, + ); err != nil { + return err + } + // Insert a new order line for each item in the order. olValsStrings := make([]string, d.oOlCnt) for i := range d.items { diff --git a/pkg/workload/ycsb/ycsb.go b/pkg/workload/ycsb/ycsb.go index f02142c370ff..e5dad0c0eea3 100644 --- a/pkg/workload/ycsb/ycsb.go +++ b/pkg/workload/ycsb/ycsb.go @@ -39,6 +39,7 @@ const ( usertableSchemaRelational = `( ycsb_key VARCHAR(255) PRIMARY KEY NOT NULL, + OTHER BOOL, FIELD0 TEXT, FIELD1 TEXT, FIELD2 TEXT, @@ -52,6 +53,7 @@ const ( )` usertableSchemaRelationalWithFamilies = `( ycsb_key VARCHAR(255) PRIMARY KEY NOT NULL, + OTHER BOOL, FIELD0 TEXT, FIELD1 TEXT, FIELD2 TEXT, @@ -63,6 +65,7 @@ const ( FIELD8 TEXT, FIELD9 TEXT, FAMILY (ycsb_key), + FAMILY (OTHER), FAMILY (FIELD0), FAMILY (FIELD1), FAMILY (FIELD2), @@ -91,6 +94,7 @@ type ycsb struct { json bool families bool splits int + singleRow bool workload string requestDistribution string @@ -121,6 +125,7 @@ var ycsbMeta = workload.Meta{ g.flags.BoolVar(&g.json, `json`, false, `Use JSONB rather than relational data`) g.flags.BoolVar(&g.families, `families`, true, `Place each column in its own column family`) g.flags.IntVar(&g.splits, `splits`, 0, `Number of splits to perform before starting normal operations`) + g.flags.BoolVar(&g.singleRow, `single-row`, false, `Read and write to a single row`) g.flags.StringVar(&g.workload, `workload`, `B`, `Workload type. Choose from A-F.`) g.flags.StringVar(&g.requestDistribution, `request-distribution`, ``, `Distribution for request key generation [zipfian, uniform, latest]. The default for workloads A, B, C, E, and F is zipfian, and the default for workload D is latest.`) g.flags.StringVar(&g.scanLengthDistribution, `scan-length-distribution`, `uniform`, `Distribution for scan length generation [zipfian, uniform]. Primarily used for workload E.`) @@ -170,6 +175,9 @@ func (g *ycsb) Hooks() workload.Hooks { g.readFreq = 0.5 g.readModifyWriteFreq = 0.5 g.requestDistribution = "zipfian" + case "U", "u": + g.updateFreq = 1.0 + g.requestDistribution = "zipfian" default: return errors.Errorf("Unknown workload: %q", g.workload) } @@ -210,7 +218,10 @@ func (g *ycsb) Tables() []workload.Table { if g.json { return []interface{}{key, "{}"} } - return []interface{}{key, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil} + if g.singleRow && rowIdx == 0 { + key = keyNameFromHash(0) + } + return []interface{}{key, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil} } if g.json { usertable.Schema = usertableSchemaJSON @@ -283,7 +294,7 @@ func (g *ycsb) Ops(urls []string, reg *histogram.Registry) (workload.QueryLoad, ))`) } else { insertStmt, err = db.Prepare(`INSERT INTO usertable VALUES ( - $1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11 + $1, FALSE, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11 )`) } if err != nil { @@ -484,6 +495,9 @@ func (yw *ycsbWorker) nextReadKey() string { // distribution, so it might be worthwhile to exactly emulate what they're // doing. rowIndex := yw.requestGen.Uint64() % rowCount + if yw.config.singleRow { + return keyNameFromHash(0) + } return yw.buildKeyName(rowIndex) }