From c6f9b9270ba35b613b6f088929930a3c55e6504a Mon Sep 17 00:00:00 2001 From: Spencer Kimball Date: Thu, 1 Feb 2018 17:24:54 -0500 Subject: [PATCH] storage: minimize retries with 1PC commits Introduce a new `NoRefreshSpans` field to the `EndTransactionRequest` arguments. This specifies that a serializable isolation transaction has encountered no refresh spans. On a 1PC commit, this can be used to avoid serializable restarts by re-executing the 1PC transaction at an appropriately higher timestamp in the event of the timestamp being forwarded by the timestamp cache or because of write-too-old errors. When evaluating a write batch, we now allow a local retry for write too old errors for non-transactional batches, and for serializable 1PC txns where `NoRefreshSpans` is true. Release note: None --- pkg/kv/dist_sender_server_test.go | 79 ++++- pkg/kv/txn_coord_sender.go | 7 +- pkg/roachpb/api.pb.go | 555 ++++++++++++++++-------------- pkg/roachpb/api.proto | 8 +- pkg/roachpb/data.go | 10 +- pkg/storage/replica.go | 177 +++++++--- pkg/storage/replica_test.go | 203 ++++++++++- 7 files changed, 708 insertions(+), 331 deletions(-) diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index 7147421044e4..7a9d638d0d2d 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1852,8 +1852,10 @@ func TestTxnCoordSenderRetries(t *testing.T) { afterTxnStart func(context.Context, *client.DB) error // called after the txn chooses a timestamp retryable func(context.Context, *client.Txn) error // called during the txn; may be retried filter func(storagebase.FilterArgs) *roachpb.Error - txnCoordRetry bool - expFailure string // regexp pattern to match on error if not empty + // If both of these are false, no retries. + txnCoordRetry bool + clientRetry bool + expFailure string // regexp pattern to match on error if not empty }{ { name: "forwarded timestamp with get and put", @@ -1897,7 +1899,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { } return nil }, - txnCoordRetry: false, + clientRetry: true, }, { name: "forwarded timestamp with get and cput", @@ -1922,7 +1924,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { txnCoordRetry: true, }, { - name: "forwarded timestamp with batch commit", + name: "forwarded timestamp with put in batch commit", afterTxnStart: func(ctx context.Context, db *client.DB) error { _, err := db.Get(ctx, "a") // set ts cache return err @@ -1930,9 +1932,25 @@ func TestTxnCoordSenderRetries(t *testing.T) { retryable: func(ctx context.Context, txn *client.Txn) error { b := txn.NewBatch() b.Put("a", "put") - return txn.CommitInBatch(ctx, b) // will be a 1PC, but will get an auto retry + return txn.CommitInBatch(ctx, b) }, - txnCoordRetry: true, + // No retries, 1pc commit. + }, + { + name: "forwarded timestamp with cput in batch commit", + beforeTxnStart: func(ctx context.Context, db *client.DB) error { + return db.Put(ctx, "a", "orig") + }, + afterTxnStart: func(ctx context.Context, db *client.DB) error { + _, err := db.Get(ctx, "a") // set ts cache + return err + }, + retryable: func(ctx context.Context, txn *client.Txn) error { + b := txn.NewBatch() + b.CPut("a", "cput", "orig") + return txn.CommitInBatch(ctx, b) + }, + // No retries, 1pc commit. }, { name: "write too old with put", @@ -1942,7 +1960,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { retryable: func(ctx context.Context, txn *client.Txn) error { return txn.Put(ctx, "a", "put") }, - txnCoordRetry: false, + clientRetry: true, }, { name: "write too old with cput matching newer value", @@ -2007,6 +2025,33 @@ func TestTxnCoordSenderRetries(t *testing.T) { }, txnCoordRetry: true, }, + { + name: "write too old with put in batch commit", + afterTxnStart: func(ctx context.Context, db *client.DB) error { + return db.Put(ctx, "a", "put") + }, + retryable: func(ctx context.Context, txn *client.Txn) error { + b := txn.NewBatch() + b.Put("a", "new-put") + return txn.CommitInBatch(ctx, b) // will be a 1PC, won't get auto retry + }, + // No retries, 1pc commit. + }, + { + name: "write too old with cput in batch commit", + beforeTxnStart: func(ctx context.Context, db *client.DB) error { + return db.Put(ctx, "a", "orig") + }, + afterTxnStart: func(ctx context.Context, db *client.DB) error { + return db.Put(ctx, "a", "put") + }, + retryable: func(ctx context.Context, txn *client.Txn) error { + b := txn.NewBatch() + b.CPut("a", "cput", "put") + return txn.CommitInBatch(ctx, b) // will be a 1PC, won't get auto retry + }, + // No retries, 1pc commit. + }, { name: "multi-range batch with forwarded timestamp", afterTxnStart: func(ctx context.Context, db *client.DB) error { @@ -2065,7 +2110,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { b.Put("c", "put") return txn.CommitInBatch(ctx, b) }, - txnCoordRetry: false, // cput with write too old requires restart + clientRetry: true, // cput with write too old requires restart }, { name: "cput within uncertainty interval", @@ -2118,8 +2163,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { } return nil }, - filter: newUncertaintyFilter(roachpb.Key([]byte("ac"))), - txnCoordRetry: false, // note this txn is read-only but still restarts + filter: newUncertaintyFilter(roachpb.Key([]byte("ac"))), + clientRetry: true, // note this txn is read-only but still restarts }, { name: "multi range batch with uncertainty interval error", @@ -2153,8 +2198,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { b.CPut("c", "cput", "value") return txn.CommitInBatch(ctx, b) }, - filter: newUncertaintyFilter(roachpb.Key([]byte("c"))), - txnCoordRetry: false, // will fail because of write too old on cput + filter: newUncertaintyFilter(roachpb.Key([]byte("c"))), + clientRetry: true, // will fail because of write too old on cput }, { name: "multi range batch with uncertainty interval error and mixed success", @@ -2167,8 +2212,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { b.CPut("c", "cput", "value") return txn.CommitInBatch(ctx, b) }, - filter: newUncertaintyFilter(roachpb.Key([]byte("c"))), - txnCoordRetry: false, // client-side retry required as this will be an mixed success + filter: newUncertaintyFilter(roachpb.Key([]byte("c"))), + clientRetry: true, // client-side retry required as this will be an mixed success }, { name: "multi range scan with uncertainty interval error", @@ -2184,8 +2229,8 @@ func TestTxnCoordSenderRetries(t *testing.T) { retryable: func(ctx context.Context, txn *client.Txn) error { return txn.DelRange(ctx, "a", "d") }, - filter: newUncertaintyFilter(roachpb.Key([]byte("c"))), - txnCoordRetry: false, // can't restart because of mixed success and write batch + filter: newUncertaintyFilter(roachpb.Key([]byte("c"))), + clientRetry: true, // can't restart because of mixed success and write batch }, } @@ -2207,7 +2252,7 @@ func TestTxnCoordSenderRetries(t *testing.T) { epoch := 0 if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { if epoch > 0 { - if tc.txnCoordRetry { + if !tc.clientRetry { t.Fatal("expected txn coord sender to retry, but got client-side retry") } // We expected a new epoch and got it; return success. diff --git a/pkg/kv/txn_coord_sender.go b/pkg/kv/txn_coord_sender.go index f506e4923ffe..d0b3f359b0f8 100644 --- a/pkg/kv/txn_coord_sender.go +++ b/pkg/kv/txn_coord_sender.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -462,6 +461,10 @@ func (tc *TxnCoordSender) Send( tc.mu.meta.Intents = et.IntentSpans tc.mu.intentsSizeBytes = intentsSizeBytes + if tc.mu.meta.Txn.IsSerializable() && tc.mu.meta.RefreshValid && + len(tc.mu.meta.RefreshReads) == 0 && len(tc.mu.meta.RefreshWrites) == 0 { + et.NoRefreshSpans = true + } return nil }(); pErr != nil { return nil, pErr @@ -487,7 +490,7 @@ func (tc *TxnCoordSender) Send( // qualified by possible resume spans in the responses, if the txn // has serializable isolation and we haven't yet exceeded the max // read key bytes. - if pErr == nil && ba.Txn != nil && ba.Txn.Isolation == enginepb.SERIALIZABLE { + if pErr == nil && ba.Txn.IsSerializable() { tc.mu.Lock() if tc.mu.meta.RefreshValid { if !tc.appendRefreshSpansLocked(ctx, ba, br) { diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 9d9dc3600bf3..08f64ea5ada0 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -862,6 +862,10 @@ type EndTransactionRequest struct { // guarantees that all writes are to the same range and that no // intents are left in the event of an error. Require1PC bool `protobuf:"varint,6,opt,name=require_1pc,json=require1pc,proto3" json:"require_1pc,omitempty"` + // Set to true if this transaction is serializable isolation but has + // accummulated no refresh spans. This allows the executing server + // to retry it locally on the fast path. + NoRefreshSpans bool `protobuf:"varint,7,opt,name=no_refresh_spans,json=noRefreshSpans,proto3" json:"no_refresh_spans,omitempty"` } func (m *EndTransactionRequest) Reset() { *m = EndTransactionRequest{} } @@ -2013,8 +2017,8 @@ type BatchResponse_Header struct { // error is non-nil if an error occurred. Error *Error `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` // timestamp is set only for non-transactional responses and denotes the - // highest timestamp at which a command from the batch executed. At the - // time of writing, it is used solely for informational purposes and tests. + // timestamp at which the batch executed. The timestamp cache is updated + // at this timestamp. Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=Timestamp" json:"Timestamp"` // txn is non-nil if the request specified a non-nil // transaction. The transaction timestamp and/or priority may have @@ -2659,6 +2663,9 @@ func (this *EndTransactionRequest) Equal(that interface{}) bool { if this.Require1PC != that1.Require1PC { return false } + if this.NoRefreshSpans != that1.NoRefreshSpans { + return false + } return true } func (this *AdminSplitRequest) Equal(that interface{}) bool { @@ -5095,6 +5102,16 @@ func (m *EndTransactionRequest) MarshalTo(dAtA []byte) (int, error) { } i++ } + if m.NoRefreshSpans { + dAtA[i] = 0x38 + i++ + if m.NoRefreshSpans { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -8951,6 +8968,9 @@ func (m *EndTransactionRequest) Size() (n int) { if m.Require1PC { n += 2 } + if m.NoRefreshSpans { + n += 2 + } return n } @@ -13867,6 +13887,26 @@ func (m *EndTransactionRequest) Unmarshal(dAtA []byte) error { } } m.Require1PC = bool(v != 0) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NoRefreshSpans", 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.NoRefreshSpans = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -24830,13 +24870,13 @@ var ( func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptorApi) } var fileDescriptorApi = []byte{ - // 5290 bytes of a gzipped FileDescriptorProto + // 5309 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x7c, 0x4d, 0x8c, 0x23, 0x49, 0x56, 0x7f, 0x67, 0xd9, 0x2e, 0xdb, 0xcf, 0x1f, 0xed, 0x8a, 0xfe, 0x72, 0x57, 0x7f, 0xb8, 0xda, 0xfd, 0xdd, 0xbb, 0x53, 0x3d, 0xdd, 0xf5, 0xef, 0x3f, 0xf3, 0xc9, 0x4e, 0xd9, 0x55, 0xdd, 0xed, 0xe9, 0xea, 0xaa, 0x9e, 0xb0, 0x6b, 0xbe, 0x16, 0x36, 0xc9, 0xca, 0x8c, 0xb2, 0x93, 0xb6, 0x33, - 0xdd, 0x99, 0xe9, 0xae, 0xaa, 0xe1, 0x00, 0x8b, 0x84, 0xe0, 0xb0, 0x20, 0x24, 0xe6, 0x80, 0x84, - 0x04, 0x2b, 0x01, 0x12, 0x12, 0x27, 0x0e, 0x1c, 0x38, 0x71, 0xe1, 0x30, 0x68, 0x17, 0x34, 0x07, + 0xdd, 0x99, 0xe9, 0xae, 0xaa, 0xe1, 0x80, 0x16, 0x09, 0xc1, 0x61, 0x41, 0x48, 0xcc, 0x01, 0x09, + 0x09, 0x56, 0x02, 0x24, 0x04, 0x27, 0x0e, 0x1c, 0x38, 0x71, 0xe1, 0x30, 0x68, 0x17, 0x34, 0x07, 0x90, 0x10, 0x48, 0x25, 0x28, 0x2e, 0xab, 0x15, 0x9c, 0xb8, 0x0d, 0x12, 0x42, 0xf1, 0x65, 0xa7, 0xed, 0x4c, 0xdb, 0xd5, 0x93, 0x68, 0x04, 0x27, 0x3b, 0x5f, 0xc6, 0x7b, 0x11, 0xf1, 0xe2, 0xc5, 0x8b, 0x5f, 0xc4, 0x7b, 0x91, 0xb0, 0xe0, 0xd8, 0x9a, 0xde, 0xea, 0xee, 0xdc, 0xd5, 0xba, 0xe6, @@ -24863,10 +24903,10 @@ var fileDescriptorApi = []byte{ 0x81, 0xac, 0xff, 0x2d, 0x42, 0x90, 0xc7, 0xeb, 0xf5, 0xed, 0xa7, 0xeb, 0xea, 0xf6, 0xe6, 0x93, 0xcd, 0xad, 0x8f, 0x36, 0x0b, 0x27, 0xd0, 0x69, 0x28, 0x08, 0xda, 0x93, 0xf5, 0x4f, 0xd4, 0x8d, 0xda, 0xd3, 0x5a, 0xa3, 0xa0, 0xa0, 0xf3, 0x70, 0x46, 0x50, 0xf1, 0xea, 0xe6, 0xa3, 0x75, 0xb5, - 0xb2, 0xb5, 0xbd, 0xb9, 0xb6, 0x8a, 0x3f, 0x29, 0xcc, 0x2d, 0xc6, 0x7f, 0xe3, 0x0f, 0x2f, 0x9f, + 0xb2, 0xb5, 0xbd, 0xb9, 0xb6, 0x8a, 0x3f, 0x29, 0xcc, 0x2d, 0xc6, 0x7f, 0xfd, 0x0f, 0x2e, 0x9f, 0x28, 0x3f, 0x05, 0x78, 0x44, 0x3c, 0x4c, 0x5e, 0xf4, 0x88, 0xeb, 0xa1, 0x37, 0x61, 0xbe, 0xc5, 0x5a, 0x23, 0x4c, 0x22, 0x4c, 0xa5, 0x95, 0x14, 0xed, 0xf8, 0x97, 0x87, 0x25, 0x05, 0x0b, 0x86, - 0xb7, 0xe2, 0x3f, 0xf9, 0x61, 0x49, 0x29, 0xff, 0xaa, 0x02, 0x19, 0x26, 0x8f, 0x77, 0x0d, 0x55, + 0xb7, 0xe2, 0x3f, 0xf9, 0x61, 0x49, 0x29, 0xff, 0x8a, 0x02, 0x19, 0x26, 0x8f, 0x77, 0x0d, 0x55, 0x47, 0x04, 0x5e, 0x99, 0xaa, 0x87, 0x71, 0xd1, 0x68, 0x19, 0x12, 0x2f, 0xb5, 0x76, 0x6f, 0x92, 0xa1, 0x7d, 0x48, 0xdf, 0x63, 0x5e, 0xac, 0xfc, 0x67, 0x0a, 0xc0, 0xb3, 0x5e, 0x04, 0x9d, 0xa2, 0x26, 0x3e, 0x53, 0xcd, 0xd2, 0xc4, 0x59, 0x61, 0x74, 0x16, 0xe6, 0x4d, 0xab, 0x6d, 0x5a, 0x84, @@ -24905,7 +24945,7 @@ var fileDescriptorApi = []byte{ 0xf1, 0x49, 0x12, 0x5b, 0xdf, 0x65, 0xbe, 0xe7, 0x5d, 0x96, 0x5b, 0xdf, 0xe5, 0xa7, 0x1f, 0x56, 0xab, 0x9b, 0xc4, 0xdb, 0xb3, 0x9d, 0xe7, 0xac, 0x59, 0xd2, 0x10, 0x98, 0xb4, 0x35, 0x2a, 0xac, 0xfc, 0x29, 0x9c, 0xab, 0x90, 0xa6, 0x69, 0xf9, 0xb7, 0x84, 0x51, 0x79, 0x75, 0x15, 0x8a, 0xe3, - 0xb2, 0xa3, 0x1c, 0xeb, 0x5f, 0x89, 0xc1, 0x99, 0x75, 0xcb, 0x88, 0xb4, 0xed, 0x74, 0xf9, 0xd5, + 0xb2, 0xa3, 0x1c, 0xeb, 0x3f, 0x89, 0xc1, 0x99, 0x75, 0xcb, 0x88, 0xb4, 0xed, 0x74, 0xf9, 0xd5, 0xed, 0x4e, 0xc7, 0xf4, 0xe4, 0x58, 0xf3, 0x27, 0xf4, 0x26, 0xa4, 0x0c, 0xa2, 0x19, 0xfd, 0x7d, 0x48, 0xe6, 0xfe, 0x25, 0x9f, 0xd0, 0x9e, 0x67, 0xb6, 0x97, 0x5b, 0x6d, 0x7d, 0xb9, 0x21, 0x0f, 0x16, 0x70, 0xbf, 0x38, 0xfa, 0x05, 0x38, 0x47, 0xe7, 0x9e, 0x63, 0x69, 0x6d, 0x95, 0x4b, 0x53, @@ -24913,253 +24953,254 @@ var fileDescriptorApi = []byte{ 0xd0, 0xe0, 0xe5, 0xf1, 0x19, 0x33, 0x88, 0x8c, 0xde, 0x83, 0xac, 0xf0, 0x09, 0x74, 0x03, 0x4f, 0xb7, 0xe2, 0xb1, 0x49, 0xbd, 0xe6, 0x76, 0x20, 0xdc, 0x08, 0xa5, 0xb8, 0xe8, 0x2e, 0x85, 0x25, 0x2f, 0x7a, 0xa6, 0x43, 0xd4, 0x7b, 0x5d, 0xbd, 0x38, 0x4f, 0xfb, 0x5e, 0xc9, 0x1f, 0x1d, 0x96, - 0x00, 0x73, 0xf2, 0xbd, 0x67, 0x55, 0x0a, 0x53, 0xf8, 0xff, 0xae, 0x2e, 0xc6, 0xf8, 0x77, 0x14, - 0x38, 0x3b, 0x3a, 0x04, 0x51, 0xda, 0xfe, 0x2d, 0x28, 0xd8, 0x16, 0x51, 0xbb, 0x2d, 0xcd, 0x25, - 0x42, 0x77, 0x02, 0x16, 0xe5, 0x6d, 0x8b, 0x3c, 0xa3, 0x64, 0xae, 0x09, 0xee, 0xb0, 0xdf, 0x8f, - 0xa7, 0x62, 0x85, 0x78, 0xf9, 0x33, 0x58, 0x58, 0x35, 0x3a, 0xa6, 0x55, 0xef, 0xb6, 0xcd, 0x28, - 0x90, 0xf4, 0x35, 0x48, 0xbb, 0x54, 0x14, 0x85, 0x6c, 0xcc, 0x2c, 0x7c, 0xb8, 0x28, 0xc5, 0xde, - 0x3c, 0x21, 0x07, 0x03, 0x2c, 0xe3, 0xaf, 0x3b, 0x4a, 0x7b, 0x6f, 0x88, 0x6e, 0x3d, 0x25, 0x4e, - 0xb4, 0xe0, 0xcb, 0x2f, 0x35, 0xca, 0x06, 0x7f, 0x5f, 0x81, 0xf3, 0x4c, 0x36, 0xb3, 0x8f, 0x5d, - 0xe2, 0xb0, 0x03, 0xac, 0x08, 0x06, 0xe4, 0x2a, 0xcc, 0x7b, 0x9a, 0xd3, 0x24, 0x7c, 0x92, 0x26, - 0x2a, 0x99, 0xaf, 0x0e, 0x4b, 0xc9, 0xba, 0x67, 0x3b, 0xa4, 0xb6, 0x86, 0xc5, 0x2b, 0xd1, 0x3d, - 0x0d, 0x16, 0x83, 0x9a, 0x10, 0x65, 0x37, 0x7f, 0xaa, 0x88, 0x3a, 0xaa, 0x2d, 0x0e, 0x60, 0xbb, - 0x6d, 0x53, 0xd7, 0xa2, 0x58, 0x79, 0xd6, 0x21, 0xa3, 0x33, 0x99, 0xaa, 0x77, 0xd0, 0xe5, 0xdb, - 0xa8, 0xfc, 0xfd, 0x6b, 0x81, 0x6d, 0x64, 0x75, 0xf2, 0x06, 0x34, 0x0e, 0xba, 0x04, 0x83, 0xde, - 0xff, 0x8f, 0xd6, 0x20, 0xc9, 0x75, 0x22, 0xe1, 0xc2, 0x04, 0x11, 0x74, 0x0e, 0x37, 0x58, 0x61, - 0xe1, 0x26, 0x24, 0xab, 0xd0, 0xe7, 0x0e, 0x5c, 0x08, 0xec, 0x6b, 0xd4, 0xbb, 0x2e, 0x06, 0xc1, - 0x37, 0x6c, 0xfb, 0x79, 0xaf, 0x1b, 0x81, 0x22, 0x2f, 0x01, 0x74, 0xb4, 0x7d, 0x8e, 0xf9, 0x39, - 0x82, 0x4f, 0xe0, 0x74, 0x47, 0xdb, 0x67, 0xb5, 0xb8, 0xa8, 0x08, 0x49, 0x87, 0xc3, 0x3c, 0xe1, - 0x5d, 0xe4, 0x63, 0x1f, 0x0d, 0x52, 0xb7, 0xf2, 0x1f, 0x14, 0x0d, 0xfa, 0x9b, 0x15, 0xa5, 0xa7, - 0x7b, 0x0f, 0xe6, 0xfb, 0xad, 0x8b, 0x1d, 0xeb, 0xf8, 0x58, 0xf0, 0xa1, 0x6d, 0x58, 0xe8, 0x3a, - 0x64, 0x97, 0x78, 0x7a, 0x8b, 0x18, 0xb2, 0xab, 0xb1, 0x63, 0x0a, 0x2b, 0x0c, 0x44, 0x70, 0xdd, - 0x94, 0x7f, 0x53, 0x81, 0x53, 0x8f, 0x89, 0xe6, 0x78, 0x3b, 0x44, 0xf3, 0x1a, 0xfb, 0x51, 0xac, - 0xb1, 0x0f, 0x20, 0x66, 0xd9, 0x7b, 0x02, 0xc9, 0x4c, 0x5e, 0x46, 0x45, 0xb3, 0x68, 0x79, 0x61, - 0x80, 0xdf, 0x85, 0xd3, 0xc3, 0xcd, 0x89, 0xd2, 0xf2, 0xfe, 0x24, 0x06, 0xe9, 0x47, 0xd5, 0x08, - 0xba, 0xf8, 0x8e, 0xd8, 0x45, 0x87, 0xeb, 0xbf, 0x5f, 0xcd, 0xf2, 0xa3, 0xea, 0x13, 0x72, 0x20, - 0x31, 0x3e, 0xe5, 0x42, 0xab, 0x90, 0xf6, 0x5a, 0x0e, 0x71, 0x5b, 0x76, 0xdb, 0x10, 0x18, 0x61, - 0x26, 0x35, 0x0d, 0xb8, 0x50, 0x1b, 0xce, 0x78, 0xfb, 0x16, 0xc3, 0x03, 0x6a, 0x53, 0x57, 0x07, - 0xe2, 0x12, 0xb3, 0x88, 0x5b, 0xa4, 0xe2, 0x8e, 0x0e, 0x4b, 0xa8, 0xb1, 0x6f, 0xd1, 0x1e, 0x3e, - 0xaa, 0x36, 0xa4, 0x00, 0x8c, 0x3c, 0x41, 0xd3, 0xfb, 0xb4, 0xc5, 0xe7, 0x90, 0x60, 0xbd, 0x40, - 0xe7, 0x21, 0x46, 0x17, 0x49, 0x65, 0x78, 0x91, 0xa4, 0x34, 0xd6, 0x29, 0x59, 0xc1, 0x71, 0xc6, - 0x7e, 0xc0, 0xc5, 0x2d, 0x40, 0xd8, 0xc1, 0x07, 0x00, 0x54, 0x85, 0x51, 0x8e, 0xfe, 0x9f, 0xc7, - 0x20, 0xff, 0xac, 0xe7, 0xb6, 0xa2, 0xb1, 0xf2, 0x2a, 0x40, 0xb7, 0xe7, 0xb6, 0x88, 0xa3, 0x7a, - 0xfb, 0x96, 0xe8, 0xf0, 0x94, 0x70, 0x8c, 0xec, 0x31, 0xe7, 0x6b, 0xec, 0x5b, 0x68, 0x4b, 0x08, - 0x21, 0xea, 0x20, 0xa6, 0x73, 0x67, 0x06, 0xec, 0xdf, 0xd8, 0xb7, 0x9e, 0x12, 0x4f, 0x1b, 0x12, - 0x48, 0xa8, 0xc0, 0x77, 0x20, 0x49, 0x1f, 0x54, 0xcf, 0x3e, 0x8e, 0x61, 0xcd, 0x53, 0x9e, 0x86, - 0x2d, 0x67, 0x6e, 0xe2, 0x78, 0x33, 0x17, 0xbd, 0x0d, 0x69, 0x5e, 0x29, 0x5d, 0xc5, 0xe6, 0xd9, - 0x2a, 0x16, 0xa4, 0x09, 0xa1, 0x7b, 0xb6, 0x7e, 0xa5, 0x58, 0x8d, 0x74, 0xf5, 0x3a, 0x0d, 0x89, - 0x5d, 0xdb, 0xd1, 0x09, 0x8b, 0xfd, 0xa4, 0x30, 0x7f, 0xe8, 0x3b, 0xe6, 0x54, 0x21, 0x5d, 0xfe, - 0x3d, 0x05, 0x4e, 0xf6, 0xc7, 0x2d, 0x4a, 0xa7, 0x5c, 0x1d, 0xd2, 0xfe, 0xf1, 0x87, 0x90, 0x6a, - 0xbc, 0xfc, 0xc7, 0x73, 0x70, 0xf2, 0x83, 0x1e, 0x71, 0x0e, 0xa2, 0x31, 0xab, 0x0a, 0x0f, 0xef, - 0xcd, 0xbd, 0xa2, 0x29, 0xb0, 0x80, 0xdf, 0x0d, 0x38, 0xb9, 0xa7, 0x99, 0x9e, 0xba, 0x6b, 0x3b, - 0x6a, 0xaf, 0x6b, 0x68, 0x9e, 0x8c, 0xad, 0xe4, 0x28, 0xf9, 0xa1, 0xed, 0x6c, 0x33, 0x22, 0x22, - 0x80, 0x9e, 0x5b, 0xf6, 0x9e, 0xa5, 0x52, 0xb2, 0x69, 0x35, 0xa9, 0x1a, 0xdc, 0x62, 0x9c, 0x9d, - 0x0c, 0xfe, 0xcc, 0x3f, 0x1e, 0x96, 0x56, 0x9a, 0xa6, 0xd7, 0xea, 0xed, 0x2c, 0xeb, 0x76, 0xe7, - 0x6e, 0xbf, 0x21, 0xc6, 0xce, 0xe0, 0xff, 0xdd, 0xee, 0xf3, 0xe6, 0x5d, 0x16, 0x5b, 0xed, 0xf5, - 0x4c, 0x63, 0x79, 0x7b, 0xbb, 0xb6, 0x86, 0x0b, 0x4c, 0xe4, 0x47, 0x5c, 0x62, 0x63, 0xdf, 0x92, - 0xc8, 0xe2, 0x2b, 0x05, 0x0a, 0x03, 0x3d, 0x45, 0x39, 0x8c, 0xeb, 0x90, 0x79, 0xd1, 0x23, 0x8e, - 0x49, 0x8c, 0x63, 0x8f, 0x23, 0x08, 0x46, 0x3a, 0x75, 0x3e, 0x85, 0xec, 0x90, 0x1e, 0x62, 0x5f, - 0x4f, 0x0f, 0x99, 0xbd, 0x81, 0x0a, 0xca, 0xff, 0xa5, 0xc0, 0x69, 0x4c, 0x5c, 0xbb, 0xfd, 0x92, - 0xf0, 0x83, 0xa8, 0x08, 0x2c, 0x65, 0x0b, 0xc4, 0x99, 0x8f, 0xfa, 0x75, 0x0c, 0x26, 0xcd, 0x65, - 0x70, 0xdf, 0x31, 0xef, 0x7a, 0x9a, 0xd7, 0xe3, 0x27, 0x67, 0xc1, 0x48, 0xd4, 0xa7, 0xc2, 0x3a, - 0x2b, 0x8b, 0x05, 0x0f, 0xdd, 0x59, 0x77, 0x6d, 0xd3, 0xb5, 0x2d, 0x79, 0xb0, 0xcd, 0x9f, 0xc4, - 0xe8, 0xff, 0x1c, 0x9c, 0x19, 0xe9, 0x7f, 0x94, 0x9e, 0xfd, 0x6f, 0xe6, 0xe0, 0xfc, 0xb0, 0xf8, - 0x88, 0x8e, 0xf3, 0xff, 0x77, 0xe8, 0x18, 0x3d, 0x86, 0x5c, 0xc7, 0xb4, 0xd4, 0xc1, 0xfa, 0x7b, - 0x0c, 0x0f, 0x9e, 0xa5, 0xbb, 0xa7, 0xe1, 0x25, 0xb8, 0x9c, 0x87, 0xec, 0xa6, 0x6d, 0xf7, 0x21, - 0x70, 0x39, 0x07, 0x19, 0xfe, 0xcc, 0x14, 0x4a, 0x37, 0x5d, 0x41, 0xda, 0x8e, 0x72, 0x44, 0x7f, - 0x4d, 0x81, 0x6c, 0x44, 0x1b, 0xe1, 0x57, 0x0b, 0x27, 0x0a, 0x4d, 0x34, 0x20, 0xf7, 0x3f, 0xb0, - 0x73, 0xfe, 0x7d, 0x05, 0x50, 0xc3, 0xe9, 0x59, 0xba, 0xe6, 0x91, 0x0d, 0xbb, 0x19, 0x41, 0x1f, - 0x4f, 0x43, 0xc2, 0xb4, 0x0c, 0xb2, 0xcf, 0xfa, 0x18, 0xc7, 0xfc, 0x01, 0xdd, 0x83, 0x94, 0x48, - 0xd2, 0xe0, 0x51, 0xd1, 0x58, 0xe5, 0xec, 0xd1, 0x61, 0x29, 0xc9, 0x53, 0x32, 0xd6, 0xbe, 0x1a, - 0xfc, 0xc5, 0x49, 0x9e, 0x95, 0x21, 0xc3, 0xc5, 0x9f, 0xc2, 0xa9, 0xa1, 0xf6, 0x45, 0xd9, 0xf9, - 0x1f, 0xb1, 0x53, 0x77, 0xd6, 0xe3, 0xa8, 0x0e, 0x0c, 0x5e, 0x29, 0xb9, 0x06, 0xbd, 0x0b, 0xd0, - 0x75, 0xc8, 0x4b, 0x95, 0xb3, 0xc6, 0x66, 0x62, 0x4d, 0x53, 0x0e, 0x46, 0x10, 0x9a, 0xfa, 0xb1, - 0x02, 0xa7, 0xa3, 0x3e, 0xff, 0xf8, 0x06, 0xbb, 0x53, 0x87, 0x02, 0x7b, 0xac, 0x59, 0xbb, 0x76, - 0x64, 0x67, 0x50, 0xbf, 0xa5, 0xc0, 0x82, 0x4f, 0x6a, 0x94, 0x6b, 0xff, 0xab, 0xa5, 0x55, 0x7d, - 0x97, 0xae, 0xc6, 0x7e, 0x0b, 0x8c, 0xd2, 0xbe, 0xff, 0x53, 0x81, 0xb3, 0x55, 0x1e, 0x2e, 0x60, - 0xc1, 0x10, 0xb7, 0xd7, 0x89, 0xc0, 0x26, 0x8a, 0x90, 0x7c, 0x49, 0x1c, 0xd7, 0xb4, 0xf9, 0x32, - 0x94, 0xc3, 0xf2, 0x11, 0xfd, 0x22, 0x64, 0x74, 0x51, 0x8f, 0x9c, 0xe7, 0xd9, 0x4a, 0x8d, 0x0a, - 0x78, 0x45, 0xe8, 0x72, 0x74, 0x58, 0x02, 0xd9, 0xf2, 0xda, 0x1a, 0x06, 0x29, 0xbd, 0x66, 0xa0, - 0x45, 0x48, 0xb9, 0x96, 0xd6, 0x75, 0x5b, 0xb6, 0x3c, 0xa8, 0xed, 0x3f, 0x8b, 0xb1, 0xfe, 0x1e, - 0x9c, 0x1b, 0xeb, 0x7c, 0x94, 0xda, 0xdd, 0x81, 0xd2, 0x1a, 0xe9, 0x3a, 0x84, 0xba, 0x26, 0xe3, - 0x43, 0xe2, 0x98, 0xbb, 0x07, 0xd1, 0x69, 0x59, 0xf4, 0xa1, 0x09, 0x4b, 0xe1, 0x75, 0x44, 0xd9, - 0x99, 0x1f, 0x27, 0x21, 0xb7, 0xbe, 0xdf, 0xb5, 0x1d, 0xaf, 0xce, 0x81, 0x04, 0x5a, 0x83, 0x54, - 0xd7, 0xb1, 0x5f, 0x9a, 0x52, 0x70, 0x3e, 0x30, 0x7a, 0x30, 0xc4, 0xf3, 0x4c, 0x94, 0xc7, 0x7d, - 0x4e, 0x84, 0x21, 0xbd, 0x61, 0xeb, 0x5a, 0xfb, 0xa1, 0xd9, 0x96, 0x33, 0x63, 0x79, 0x9a, 0x98, - 0xe5, 0x3e, 0xc7, 0x33, 0xcd, 0x6b, 0x49, 0xff, 0xd0, 0x27, 0xa2, 0x47, 0x90, 0x7a, 0xec, 0x79, - 0x5d, 0xfa, 0x52, 0x38, 0x97, 0xeb, 0x53, 0x45, 0x52, 0x06, 0x21, 0xa9, 0xcf, 0x8c, 0x30, 0x2c, - 0x3c, 0xb2, 0xed, 0x66, 0x9b, 0x54, 0xdb, 0x76, 0xcf, 0xa8, 0xda, 0xd6, 0xae, 0xd9, 0x14, 0x9b, - 0xd5, 0x6b, 0x53, 0x25, 0x3e, 0xaa, 0xd6, 0xf1, 0x38, 0x3b, 0xfa, 0x0e, 0xa4, 0xea, 0x2b, 0x42, - 0x14, 0xc7, 0x3e, 0x57, 0xa7, 0x8a, 0xaa, 0xaf, 0xe0, 0x3e, 0x13, 0x7a, 0x0c, 0x99, 0xd5, 0xcf, - 0x7a, 0x0e, 0x11, 0x32, 0xe6, 0x99, 0x8c, 0x1b, 0x53, 0x65, 0x30, 0x1e, 0xec, 0x67, 0x5d, 0xbc, - 0x0d, 0xb9, 0x21, 0x4d, 0x22, 0x04, 0xf1, 0x2e, 0x55, 0x1a, 0x1d, 0xce, 0x34, 0x66, 0xff, 0xb9, - 0x9d, 0x2d, 0xde, 0x80, 0x38, 0xd5, 0x0a, 0x9d, 0xdb, 0x3b, 0x9a, 0x4b, 0xb6, 0x1d, 0x53, 0x14, - 0x92, 0x8f, 0xa2, 0xdc, 0x5f, 0x29, 0x30, 0x57, 0x5f, 0xa1, 0xe8, 0x6f, 0xa7, 0xa7, 0x3f, 0x27, - 0x9e, 0x28, 0x25, 0x9e, 0x18, 0x2a, 0x74, 0xc8, 0xae, 0xc9, 0x17, 0xff, 0x34, 0x16, 0x4f, 0xe8, - 0x12, 0x80, 0xa6, 0xeb, 0xc4, 0x75, 0x59, 0x60, 0x23, 0xc6, 0xde, 0xa5, 0x39, 0xe5, 0x09, 0x39, - 0xa0, 0x6c, 0x2e, 0xd1, 0x1d, 0xc2, 0x67, 0x72, 0x1a, 0x8b, 0x27, 0xca, 0xe6, 0x91, 0x4e, 0x57, - 0xf5, 0xec, 0xe7, 0xc4, 0x62, 0xda, 0x4c, 0xe3, 0x34, 0xa5, 0x34, 0x28, 0x81, 0xba, 0x00, 0x62, - 0x19, 0x5d, 0xdb, 0xb4, 0x3c, 0xa6, 0xa6, 0x34, 0xee, 0x3f, 0x53, 0x91, 0x0e, 0x69, 0x9a, 0x22, - 0x91, 0x33, 0x8d, 0xc5, 0x93, 0xe8, 0xc6, 0x16, 0xc4, 0x1e, 0x55, 0xeb, 0xc7, 0xee, 0x06, 0x82, - 0xb8, 0xd6, 0x13, 0x46, 0x97, 0xc6, 0xec, 0xbf, 0x10, 0xf8, 0xeb, 0x0a, 0x24, 0x98, 0xea, 0xd1, - 0x45, 0x48, 0xeb, 0xb6, 0xe5, 0x69, 0xa6, 0x25, 0xe6, 0x4d, 0x1a, 0x0f, 0x08, 0xa1, 0x92, 0xaf, - 0x40, 0x56, 0xd3, 0x75, 0xbb, 0x67, 0x79, 0xaa, 0xa5, 0x75, 0x88, 0xa8, 0x21, 0x23, 0x68, 0x9b, - 0x5a, 0x87, 0xa0, 0x12, 0xc8, 0x47, 0xa6, 0x44, 0xae, 0x29, 0x10, 0xa4, 0x7e, 0x58, 0x48, 0xf8, - 0x8d, 0x3f, 0x52, 0x60, 0xe1, 0x23, 0xc7, 0xf4, 0x48, 0x45, 0xf3, 0xf4, 0x56, 0x04, 0x4e, 0xff, - 0x2d, 0x48, 0x1b, 0x9a, 0xa7, 0xf1, 0xbc, 0xdd, 0xb9, 0xc9, 0xdc, 0x62, 0x9a, 0xd1, 0xf2, 0x2c, - 0x77, 0x17, 0x41, 0x9c, 0xfe, 0xe7, 0xeb, 0x01, 0x66, 0xff, 0x07, 0x21, 0x21, 0x7f, 0x2b, 0xa3, - 0x74, 0x68, 0x7f, 0x37, 0x27, 0x1d, 0x5a, 0x04, 0xbd, 0x7f, 0x0f, 0x92, 0x62, 0x7f, 0x25, 0xfa, - 0xbe, 0x34, 0x6d, 0x3e, 0xca, 0x98, 0x86, 0x60, 0x43, 0x15, 0x00, 0xd7, 0xd3, 0x1c, 0x8f, 0xed, - 0x8c, 0x66, 0x8a, 0xeb, 0x4a, 0xbf, 0xc7, 0xd8, 0x28, 0x15, 0x6d, 0x42, 0xa6, 0xf3, 0x52, 0xd7, - 0xd5, 0x5d, 0xb3, 0xed, 0x89, 0x90, 0x6e, 0x7e, 0x48, 0x88, 0x6c, 0xc9, 0xd3, 0x0f, 0xab, 0xd5, - 0x87, 0xac, 0x10, 0x8f, 0xac, 0x0e, 0x9e, 0x31, 0x50, 0x09, 0xfc, 0x3f, 0xfa, 0x36, 0x88, 0x74, - 0x30, 0xd5, 0x75, 0x3d, 0x36, 0xbd, 0x52, 0x95, 0xdc, 0xd1, 0x61, 0x29, 0x8d, 0x19, 0xb5, 0x5e, - 0x6f, 0xe0, 0x34, 0x2f, 0x50, 0x77, 0xe5, 0xa2, 0xfa, 0x03, 0x05, 0x72, 0x95, 0x5e, 0xfb, 0xf9, - 0x56, 0xb7, 0xde, 0xeb, 0x74, 0x34, 0xe7, 0x00, 0x5d, 0x90, 0x96, 0x61, 0x7e, 0x46, 0x98, 0x66, - 0x63, 0x62, 0xe8, 0xcd, 0xcf, 0x08, 0x1d, 0x7a, 0x91, 0x7a, 0x42, 0xe9, 0x3c, 0xaf, 0xe4, 0x2a, - 0xe4, 0xd8, 0x9e, 0x40, 0x25, 0x96, 0xe7, 0x98, 0x84, 0xef, 0x3f, 0x63, 0x38, 0xcb, 0x88, 0xeb, - 0x9c, 0x86, 0xae, 0x43, 0xde, 0x3d, 0x70, 0x3d, 0xd2, 0x51, 0x79, 0x8e, 0x3c, 0x4f, 0x95, 0x8c, - 0xe1, 0x1c, 0xa7, 0x62, 0x4e, 0x2c, 0xff, 0xdb, 0x1c, 0xe4, 0xe5, 0x28, 0x47, 0x09, 0xe6, 0x2a, - 0x90, 0xd8, 0x35, 0xdb, 0xfd, 0x18, 0x49, 0xb8, 0xfb, 0x95, 0x92, 0x96, 0xa9, 0x93, 0x95, 0xd0, - 0x8e, 0xb1, 0x2e, 0x7e, 0xa9, 0x40, 0x9c, 0xad, 0x57, 0xf7, 0x20, 0xce, 0xa6, 0x8d, 0x32, 0xcb, - 0xb4, 0x61, 0x45, 0xfb, 0x9e, 0x7a, 0x6e, 0xe0, 0xa9, 0x99, 0x97, 0x6c, 0x69, 0x0f, 0xee, 0xdd, - 0x67, 0x43, 0x95, 0xc5, 0xe2, 0x09, 0x55, 0x20, 0x45, 0x58, 0x5b, 0x88, 0x21, 0x56, 0x8b, 0x20, - 0xeb, 0x1c, 0x1a, 0x34, 0x39, 0x45, 0x25, 0x1f, 0x3a, 0x0f, 0x31, 0x6a, 0x03, 0x49, 0x7e, 0x9a, - 0x7e, 0x74, 0x58, 0x8a, 0xd1, 0xd1, 0xa7, 0x34, 0x1e, 0x92, 0x7a, 0x3f, 0x9e, 0x8a, 0x17, 0x12, - 0xe5, 0x3f, 0x8d, 0x43, 0xae, 0xd6, 0x89, 0x68, 0x52, 0xad, 0x0e, 0xeb, 0x38, 0x68, 0x0d, 0x1f, - 0xaa, 0x6b, 0x5c, 0xc5, 0xc3, 0x5e, 0x29, 0x76, 0x3c, 0xaf, 0x54, 0xa3, 0x2b, 0x84, 0xb8, 0x4f, - 0x40, 0xeb, 0xff, 0xd6, 0xd4, 0xfa, 0x1b, 0xda, 0x4e, 0x9b, 0x60, 0xca, 0xd3, 0x0f, 0x88, 0x31, - 0x01, 0xe8, 0x67, 0xd9, 0x42, 0xc4, 0xa7, 0xf6, 0xfc, 0xec, 0x53, 0x3b, 0x49, 0x2c, 0x83, 0xd2, - 0x16, 0xf7, 0x85, 0xa1, 0xbc, 0x01, 0x31, 0xc3, 0x94, 0x9a, 0x9c, 0xd5, 0xc5, 0x50, 0x96, 0x29, - 0xf6, 0x12, 0xf7, 0xdb, 0x8b, 0x3f, 0xd2, 0xb8, 0xb8, 0x05, 0x30, 0xe8, 0x15, 0x5a, 0x82, 0x79, - 0xbb, 0x6d, 0x50, 0xe8, 0x4e, 0x9b, 0x90, 0xab, 0xa4, 0x8f, 0x0e, 0x4b, 0x89, 0xad, 0xb6, 0x51, - 0x5b, 0xc3, 0x09, 0xbb, 0x6d, 0xd4, 0x0c, 0x76, 0x0d, 0x83, 0xec, 0xa9, 0xec, 0xf6, 0x09, 0xcb, - 0x4f, 0xc0, 0x49, 0x8b, 0xec, 0xad, 0x11, 0x57, 0xf7, 0x2f, 0x3f, 0xc2, 0x5a, 0xfe, 0x40, 0x81, - 0xbc, 0xd4, 0x60, 0xb4, 0x93, 0x33, 0x65, 0x76, 0x84, 0xc1, 0xc7, 0x8e, 0x67, 0xf0, 0x92, 0x4f, - 0xa4, 0xdd, 0x7d, 0x5f, 0x81, 0x53, 0x3c, 0x89, 0x42, 0xd7, 0x3c, 0xea, 0x1e, 0xbf, 0xbe, 0x55, - 0xdf, 0x86, 0x82, 0xa3, 0x59, 0x86, 0xdd, 0x31, 0x3f, 0x23, 0x7c, 0x03, 0xec, 0x8a, 0x04, 0x9f, - 0x93, 0x7d, 0x3a, 0xdb, 0xea, 0xc9, 0xd3, 0xe8, 0x9f, 0x2a, 0x70, 0x7a, 0xb8, 0x0d, 0x51, 0xea, - 0xea, 0xc9, 0x48, 0xb4, 0xf7, 0xb5, 0x00, 0x21, 0x41, 0xb5, 0xf3, 0xa8, 0xed, 0x70, 0xe0, 0x77, - 0xf1, 0x3d, 0x48, 0x30, 0xf2, 0x2b, 0x78, 0x34, 0xa1, 0xf0, 0x16, 0x2c, 0xac, 0x1a, 0x46, 0xbd, - 0x2e, 0x8c, 0xee, 0x6b, 0x6b, 0x5b, 0x42, 0x8b, 0xb9, 0x20, 0x68, 0xe1, 0xaf, 0x29, 0x4a, 0x68, - 0xd1, 0x84, 0x3c, 0x26, 0xbb, 0x0e, 0x71, 0x5b, 0xd1, 0x1c, 0x97, 0xed, 0x51, 0x08, 0x24, 0x8c, - 0x84, 0x3f, 0x0c, 0xf2, 0xff, 0xfb, 0x15, 0x45, 0xd9, 0x81, 0x36, 0x9c, 0x92, 0x72, 0x23, 0x3a, - 0x9d, 0x9e, 0xd4, 0x0b, 0x76, 0xc4, 0xe1, 0xaf, 0x2d, 0xca, 0xae, 0xfc, 0xfb, 0x59, 0xc8, 0x8a, - 0xf6, 0x6f, 0x5b, 0xa6, 0x6d, 0xa1, 0xbb, 0x10, 0x6b, 0x0a, 0x40, 0x9f, 0x09, 0x04, 0x47, 0x83, - 0x6b, 0x53, 0x98, 0x96, 0xa4, 0x0c, 0xdd, 0x9e, 0x17, 0x10, 0x27, 0x1e, 0x04, 0x0b, 0x07, 0x0c, - 0xdd, 0x9e, 0x87, 0x3e, 0x80, 0x93, 0xfa, 0xe0, 0x02, 0x8b, 0x4a, 0x99, 0x63, 0xa1, 0xd9, 0x75, - 0x81, 0xf7, 0x78, 0x70, 0x5e, 0x1f, 0x22, 0xa3, 0x55, 0xff, 0x0d, 0x8a, 0x78, 0xe8, 0xae, 0x71, - 0xf4, 0xd2, 0x86, 0xef, 0x9a, 0x05, 0x7a, 0x03, 0xe6, 0x0d, 0x96, 0xa9, 0x2f, 0x76, 0x9d, 0x41, - 0x2e, 0x71, 0xe8, 0x2a, 0x04, 0x16, 0xe5, 0xd1, 0x63, 0xc8, 0xf2, 0x7f, 0x3c, 0x95, 0x43, 0xac, - 0x60, 0xd7, 0xc3, 0xf9, 0x7d, 0x36, 0x83, 0x33, 0xc6, 0x80, 0x86, 0xee, 0x43, 0xdc, 0xd5, 0x35, - 0xbe, 0xe5, 0x0a, 0x8e, 0x7b, 0xf9, 0x32, 0xac, 0x31, 0x2b, 0x8b, 0x3e, 0x82, 0x85, 0x1d, 0xd2, - 0x34, 0x2d, 0xd5, 0x1b, 0xc4, 0x1a, 0x8a, 0xa9, 0xb1, 0xf0, 0x46, 0xdf, 0xab, 0x07, 0x27, 0x91, - 0xe2, 0xc2, 0xce, 0xc8, 0x0b, 0x3a, 0x4c, 0x6c, 0x51, 0xf6, 0x89, 0x4d, 0x87, 0x0e, 0x53, 0x60, - 0x76, 0x27, 0xce, 0x93, 0x21, 0x32, 0x5a, 0x87, 0x8c, 0x46, 0x7d, 0xa5, 0xca, 0x52, 0xf1, 0x8a, - 0x10, 0x7a, 0x52, 0x30, 0x96, 0x14, 0x88, 0x41, 0xeb, 0x93, 0x06, 0x62, 0x3a, 0xc4, 0x69, 0x92, - 0x62, 0x66, 0xb2, 0x18, 0x7f, 0xec, 0x41, 0x88, 0x61, 0x24, 0xf4, 0x04, 0x72, 0x2d, 0x99, 0x9f, - 0xc2, 0x82, 0x42, 0xd9, 0xd0, 0xa3, 0x82, 0x80, 0xb4, 0x1a, 0x9c, 0x6d, 0xf9, 0x88, 0xe8, 0xdb, - 0x30, 0xd7, 0xd4, 0x8b, 0x39, 0x26, 0xe1, 0xe2, 0xa4, 0x24, 0x12, 0x3c, 0xd7, 0xd4, 0xd1, 0x3b, - 0x90, 0xe2, 0x61, 0xf6, 0x7d, 0xab, 0x98, 0x0f, 0x9d, 0xbc, 0xc3, 0x19, 0x0e, 0x98, 0xa5, 0x03, - 0xd0, 0xba, 0x1e, 0x43, 0x96, 0xc7, 0x02, 0xda, 0x2c, 0xbd, 0xa9, 0x78, 0x32, 0xd4, 0xe0, 0xc6, - 0x73, 0xb3, 0x30, 0xbf, 0xeb, 0xc9, 0x69, 0x68, 0x13, 0xf2, 0x0e, 0x0f, 0xff, 0x88, 0x54, 0xfb, - 0x62, 0x81, 0xc9, 0xba, 0x19, 0xec, 0x4a, 0xc6, 0x82, 0x9e, 0x38, 0xe7, 0xf8, 0xa9, 0xe8, 0x7b, - 0x70, 0x7a, 0x58, 0x9e, 0x98, 0x12, 0x0b, 0x4c, 0xea, 0xb7, 0xa7, 0x4a, 0xf5, 0xcf, 0x0c, 0xe4, - 0x8c, 0xbd, 0x42, 0x0f, 0x20, 0xc1, 0xc7, 0x1c, 0x31, 0x81, 0xa5, 0xa0, 0xbd, 0x9b, 0x7f, 0xb8, - 0x79, 0x69, 0xaa, 0x30, 0x4f, 0xc4, 0x40, 0xd4, 0xb6, 0xdd, 0x2c, 0x9e, 0x0a, 0x55, 0xd8, 0x78, - 0x28, 0x07, 0x67, 0xbc, 0x01, 0x8d, 0xda, 0x8c, 0xc3, 0xe9, 0xe2, 0x70, 0xfe, 0x74, 0xa8, 0xcd, - 0x04, 0x04, 0x46, 0x70, 0xd6, 0xf1, 0x11, 0xd9, 0x38, 0xf2, 0xec, 0x35, 0x95, 0x4d, 0xfb, 0x33, - 0xe1, 0xe3, 0x38, 0x76, 0xbf, 0x02, 0x67, 0x9c, 0x01, 0x0d, 0x35, 0xa0, 0x20, 0xd2, 0xda, 0x55, - 0x79, 0xc4, 0x5b, 0x3c, 0xcb, 0xa4, 0xdd, 0x0e, 0xf4, 0xa9, 0x41, 0x47, 0xda, 0xf8, 0xa4, 0x3e, - 0x4c, 0x47, 0x5d, 0x58, 0x34, 0xfa, 0x87, 0xa7, 0xea, 0x4b, 0x76, 0x7a, 0x3a, 0x90, 0x7f, 0x8e, - 0xc9, 0xbf, 0x1f, 0xe8, 0xe6, 0x26, 0x9e, 0xea, 0xe2, 0xa2, 0x11, 0x52, 0x80, 0x3a, 0x33, 0x26, - 0x5f, 0xd5, 0x07, 0xd7, 0x0e, 0x8a, 0xc5, 0x50, 0x67, 0x16, 0x72, 0x35, 0x02, 0x17, 0xf4, 0x91, - 0x17, 0xd4, 0xb3, 0x5a, 0xb6, 0xdd, 0x2d, 0x9e, 0x0f, 0xf5, 0xac, 0xbe, 0xa8, 0x28, 0x66, 0x65, - 0xe9, 0x24, 0x35, 0x2d, 0xd3, 0x63, 0x0b, 0xd4, 0x62, 0xe8, 0x24, 0x1d, 0xbe, 0x2b, 0x89, 0x93, - 0x26, 0x7f, 0xa6, 0x53, 0xcb, 0x13, 0xc1, 0x24, 0x61, 0x2a, 0x17, 0x43, 0xa7, 0x56, 0x50, 0xd4, - 0x09, 0xe7, 0x3c, 0x3f, 0x95, 0x4e, 0x2d, 0xee, 0xf4, 0x46, 0xa4, 0x5e, 0x0a, 0x9d, 0x5a, 0xa1, - 0x09, 0xbd, 0x18, 0x69, 0x63, 0xaf, 0x50, 0x05, 0x80, 0x09, 0x64, 0xb7, 0xc0, 0x8b, 0x97, 0x43, - 0xd7, 0xd0, 0xd1, 0x98, 0x12, 0x4e, 0xb7, 0x25, 0x85, 0x3a, 0x66, 0x06, 0x5c, 0xd4, 0x1d, 0xcd, - 0xd3, 0x5b, 0xc5, 0x52, 0xa8, 0x63, 0x1e, 0x3b, 0x5a, 0xc3, 0xb0, 0xd7, 0x27, 0xd1, 0xa5, 0x98, - 0x6f, 0xac, 0x8b, 0x4b, 0x53, 0x76, 0x72, 0xfd, 0xa5, 0x98, 0x97, 0x47, 0xdf, 0x81, 0xf4, 0x8b, - 0x1e, 0x71, 0x0e, 0x98, 0x63, 0xbd, 0x12, 0x7a, 0xbb, 0x7f, 0x24, 0xc9, 0x07, 0xa7, 0x5e, 0x08, - 0x02, 0xad, 0x9a, 0x6f, 0x71, 0x8a, 0xe5, 0xd0, 0xaa, 0x87, 0x36, 0xb5, 0x58, 0x94, 0x47, 0x1a, - 0x9c, 0xe1, 0xe3, 0x23, 0xf2, 0x80, 0x1d, 0x91, 0x70, 0x5b, 0xbc, 0xca, 0x04, 0x85, 0xee, 0x1b, - 0x02, 0x53, 0x91, 0xf1, 0x29, 0x6d, 0xfc, 0x1d, 0x75, 0x3e, 0x62, 0xf9, 0xe4, 0x7b, 0x8d, 0xe2, - 0xb5, 0x50, 0xe7, 0x13, 0xb0, 0x29, 0xc3, 0x59, 0xcd, 0x47, 0xe4, 0x8b, 0xa8, 0xa1, 0xba, 0xae, - 0x47, 0x01, 0x7e, 0xf1, 0xfa, 0x84, 0x45, 0x74, 0x64, 0xbf, 0xc1, 0xee, 0xa5, 0xd4, 0x39, 0x1f, - 0x4b, 0x7c, 0x6e, 0x13, 0xcd, 0x11, 0x8e, 0xfe, 0x46, 0xa8, 0x98, 0xb1, 0xdb, 0x88, 0x18, 0xf4, - 0x3e, 0x89, 0x82, 0x0d, 0x47, 0xde, 0xcc, 0x51, 0x5d, 0x4f, 0xf3, 0xdc, 0xe2, 0xcd, 0x50, 0xb0, - 0x11, 0x78, 0x6f, 0x08, 0xe7, 0x9d, 0x21, 0x32, 0x7a, 0x1b, 0x92, 0x0e, 0x87, 0xcd, 0xc5, 0x5b, - 0x13, 0xf0, 0xb1, 0x7f, 0x1f, 0x82, 0x25, 0x07, 0xf7, 0xf3, 0xec, 0xaf, 0xe8, 0xd8, 0xed, 0x09, - 0x7e, 0x7e, 0x6c, 0x27, 0x40, 0xfd, 0xfc, 0x80, 0xf8, 0x56, 0xfc, 0x0b, 0xbe, 0xa7, 0xbf, 0x50, - 0xb8, 0x58, 0xfe, 0xd1, 0x59, 0xc8, 0x49, 0x50, 0xce, 0x01, 0xf7, 0xeb, 0x7e, 0xc0, 0x7d, 0x39, - 0x0c, 0x70, 0x73, 0x0e, 0x8e, 0xb8, 0x5f, 0xf7, 0x23, 0xee, 0xcb, 0x61, 0x88, 0x5b, 0x72, 0x50, - 0xc8, 0x8d, 0xc3, 0x20, 0xf7, 0xed, 0x19, 0x20, 0xb7, 0x10, 0x34, 0x8a, 0xb9, 0x2b, 0xe3, 0x98, - 0xfb, 0xda, 0x64, 0xcc, 0x2d, 0x04, 0xf9, 0x40, 0xf7, 0x9b, 0x23, 0xa0, 0xfb, 0xca, 0x04, 0xd0, - 0x2d, 0xb8, 0x25, 0xea, 0xae, 0x05, 0xa2, 0xee, 0x1b, 0xd3, 0x50, 0xb7, 0x90, 0x32, 0x04, 0xbb, - 0x57, 0x86, 0x60, 0x77, 0x29, 0x14, 0x76, 0x0b, 0x5e, 0x8e, 0xbb, 0x3f, 0x0e, 0xc7, 0xdd, 0xdf, - 0x9a, 0x09, 0x77, 0x0b, 0x69, 0xe3, 0xc0, 0x1b, 0x87, 0x01, 0xef, 0xdb, 0x33, 0x00, 0x6f, 0x39, - 0x58, 0x23, 0xc8, 0xfb, 0x61, 0x10, 0xf2, 0xbe, 0x3e, 0x05, 0x79, 0x0b, 0x59, 0x7e, 0xe8, 0xfd, - 0x30, 0x08, 0x7a, 0x5f, 0x9f, 0x02, 0xbd, 0x87, 0xe4, 0x70, 0xec, 0xbd, 0x11, 0x8c, 0xbd, 0x6f, - 0x4e, 0xc5, 0xde, 0x42, 0xd6, 0x30, 0xf8, 0x7e, 0xcd, 0x07, 0xbe, 0x2f, 0x85, 0x80, 0x6f, 0xc1, - 0x48, 0xd1, 0xf7, 0xbb, 0x63, 0xe8, 0xbb, 0x3c, 0x09, 0x7d, 0x0b, 0xce, 0x3e, 0xfc, 0xae, 0x05, - 0xc2, 0xef, 0x1b, 0xd3, 0xe0, 0xb7, 0xb4, 0x3c, 0x3f, 0xfe, 0xde, 0x0a, 0xc1, 0xdf, 0xb7, 0xa6, - 0xe3, 0x6f, 0x21, 0x6e, 0x04, 0x80, 0xab, 0x13, 0x01, 0xf8, 0x6b, 0x33, 0x02, 0x70, 0x21, 0x3b, - 0x08, 0x81, 0xff, 0xff, 0x61, 0x04, 0xbe, 0x14, 0x8e, 0xc0, 0x85, 0x10, 0x01, 0xc1, 0x6b, 0x81, - 0x10, 0xfc, 0xc6, 0x34, 0x08, 0x2e, 0x95, 0xe6, 0xc7, 0xe0, 0x1b, 0xc1, 0x18, 0xfc, 0xe6, 0x54, - 0x0c, 0x2e, 0x6d, 0x67, 0x08, 0x84, 0xd7, 0x02, 0x41, 0xf8, 0x8d, 0x69, 0x20, 0xbc, 0x3f, 0x9a, - 0x3e, 0x14, 0xbe, 0x1d, 0x8a, 0xc2, 0xef, 0xcc, 0x82, 0xc2, 0x85, 0xc8, 0x31, 0x18, 0xfe, 0x62, - 0x06, 0x18, 0xbe, 0x72, 0x2c, 0x18, 0x2e, 0x6a, 0x0a, 0xc7, 0xe1, 0x1f, 0x87, 0xe3, 0xf0, 0x6f, - 0xcd, 0x84, 0xc3, 0xa5, 0x73, 0x1b, 0x03, 0xe2, 0x2b, 0x43, 0x40, 0xbc, 0x14, 0x0a, 0xc4, 0xa5, - 0xaf, 0x65, 0x48, 0xfc, 0xdd, 0x31, 0x24, 0x5e, 0x9e, 0x84, 0xc4, 0xe5, 0x84, 0x95, 0x50, 0x5c, - 0x9d, 0x08, 0x9d, 0x5f, 0x9b, 0x11, 0x3a, 0xcb, 0x49, 0x11, 0x80, 0x9d, 0xab, 0x01, 0xd8, 0xf9, - 0xda, 0x64, 0xec, 0x2c, 0xd7, 0xc2, 0x01, 0x78, 0x7e, 0x18, 0x04, 0x9e, 0xaf, 0x4f, 0x01, 0xcf, - 0xd2, 0xb5, 0xfa, 0xd0, 0xf3, 0x9b, 0x23, 0xe8, 0xf9, 0xca, 0xd4, 0xd8, 0x5b, 0x1f, 0x3e, 0xbf, - 0x37, 0x0e, 0x9f, 0xaf, 0x4e, 0x84, 0xcf, 0x82, 0x7f, 0x80, 0x9f, 0xdf, 0x1c, 0xc1, 0xcf, 0x57, - 0x26, 0xe0, 0x67, 0x59, 0xb9, 0x00, 0xd0, 0x3b, 0x93, 0x01, 0xf4, 0xf2, 0xac, 0x00, 0x5a, 0x88, - 0x0d, 0x44, 0xd0, 0x1b, 0xc1, 0x08, 0xfa, 0xe6, 0x8c, 0x87, 0xfa, 0x23, 0x10, 0xfa, 0x61, 0x10, - 0x84, 0xbe, 0x3e, 0x05, 0x42, 0x0f, 0x16, 0xc3, 0x3e, 0x86, 0x7e, 0x18, 0x84, 0xa1, 0xaf, 0x4f, - 0xc1, 0xd0, 0x52, 0x8e, 0x0f, 0x44, 0xe3, 0x30, 0x10, 0x7d, 0x7b, 0x06, 0x10, 0x2d, 0x81, 0xc3, - 0x08, 0x8a, 0x7e, 0x67, 0x14, 0x45, 0x97, 0x27, 0xa1, 0x68, 0x39, 0xf3, 0x24, 0x8c, 0xde, 0x08, - 0x86, 0xd1, 0x37, 0xa7, 0xc2, 0xe8, 0x81, 0xab, 0x0e, 0xc6, 0xd1, 0xef, 0xc7, 0x53, 0x17, 0x0b, - 0x97, 0xca, 0x9f, 0x27, 0x60, 0xfe, 0xb1, 0x8c, 0x86, 0xfa, 0x2e, 0x2d, 0x29, 0xaf, 0x72, 0x69, - 0x09, 0xad, 0xd1, 0xbe, 0x32, 0x4b, 0x11, 0xd8, 0x7a, 0xc2, 0xed, 0xcb, 0xb1, 0xfb, 0x78, 0x92, - 0xf5, 0x15, 0x32, 0x75, 0xd1, 0x03, 0xc8, 0xf5, 0x5c, 0xe2, 0xa8, 0x5d, 0xc7, 0xb4, 0x1d, 0xd3, - 0xe3, 0xc9, 0x29, 0x4a, 0xa5, 0xf0, 0xd5, 0x61, 0x29, 0xbb, 0xed, 0x12, 0xe7, 0x99, 0xa0, 0xe3, - 0x6c, 0xcf, 0xf7, 0x24, 0xbf, 0x1f, 0x97, 0x98, 0xfd, 0xfb, 0x71, 0x1f, 0x40, 0xc1, 0x21, 0x9a, - 0x31, 0xe4, 0xd6, 0xf9, 0x2d, 0x9f, 0xe0, 0x05, 0x4f, 0x33, 0x7c, 0xbe, 0x9b, 0xdd, 0xf6, 0x39, - 0xe9, 0x0c, 0x13, 0xd1, 0x3d, 0x38, 0xd3, 0xd1, 0xf6, 0xf9, 0xf5, 0x35, 0xb9, 0x2c, 0xb3, 0xa8, - 0x70, 0x8a, 0xe5, 0x1b, 0xa0, 0x8e, 0xb6, 0xcf, 0x3e, 0x46, 0xc7, 0x5f, 0xb1, 0x0f, 0xe7, 0x5c, - 0x87, 0xbc, 0x61, 0xba, 0x9e, 0x69, 0xe9, 0xf2, 0x1a, 0x7c, 0x9a, 0xdf, 0x69, 0x91, 0x54, 0x7e, - 0xd3, 0xfd, 0x0e, 0x2c, 0x88, 0xf4, 0x8a, 0xc1, 0xe7, 0xe9, 0x18, 0x72, 0x4d, 0xd1, 0x56, 0xd0, - 0x17, 0x83, 0x2f, 0xfa, 0x55, 0xe1, 0x64, 0x53, 0xf3, 0xc8, 0x9e, 0x76, 0xa0, 0x5a, 0xb6, 0xc1, - 0x74, 0x9f, 0x61, 0x17, 0x8e, 0x2f, 0x1c, 0x1d, 0x96, 0x72, 0x8f, 0xf8, 0xab, 0x4d, 0xdb, 0xe0, - 0x23, 0x30, 0xcf, 0xff, 0xe1, 0x5c, 0xd3, 0xf7, 0xc2, 0x40, 0xab, 0x90, 0xa5, 0x10, 0x40, 0xb5, - 0xf9, 0x37, 0x68, 0x04, 0x26, 0x0d, 0x3b, 0x86, 0x17, 0x5f, 0xaa, 0xc1, 0x19, 0x77, 0xf0, 0xf0, - 0x7e, 0x3c, 0x95, 0x2c, 0xa4, 0xca, 0x9f, 0x2b, 0x90, 0x1d, 0x4a, 0x1c, 0x7a, 0x7b, 0x24, 0x54, - 0x73, 0x3e, 0x18, 0xe7, 0x06, 0x87, 0x20, 0x57, 0x21, 0x25, 0x14, 0x2b, 0x83, 0x90, 0xa5, 0x70, - 0xa8, 0xc3, 0xf6, 0x94, 0x32, 0x5a, 0x2b, 0xd9, 0xde, 0x8a, 0xff, 0xee, 0x0f, 0x4b, 0x27, 0xca, - 0x3f, 0x89, 0x41, 0x6e, 0x38, 0x53, 0xa8, 0x36, 0xd2, 0xae, 0xa0, 0x89, 0x39, 0xc4, 0x11, 0xde, - 0xca, 0x35, 0x48, 0x3b, 0xa2, 0x90, 0x6c, 0xe6, 0xd2, 0x84, 0x80, 0x94, 0xbf, 0x9d, 0x03, 0xc6, - 0xc5, 0xbf, 0x9c, 0xeb, 0x4f, 0xe8, 0x65, 0x48, 0xb0, 0x0f, 0x4a, 0x8a, 0xa6, 0x05, 0xe5, 0x03, - 0xaf, 0xd3, 0xf7, 0x98, 0x17, 0xa3, 0x0e, 0xa0, 0xf1, 0x4a, 0xb7, 0x16, 0xfb, 0x84, 0x57, 0xf8, - 0x20, 0xe3, 0x2b, 0x5e, 0xb3, 0xab, 0xd3, 0xfd, 0x78, 0xbb, 0x4d, 0x74, 0x4f, 0x7c, 0xbb, 0x52, - 0x7e, 0x76, 0xf1, 0xda, 0xa8, 0x08, 0xf1, 0xa5, 0xcb, 0x65, 0x2c, 0xbe, 0x74, 0xe9, 0x8b, 0x0b, - 0xe7, 0xfb, 0x22, 0xd8, 0x7c, 0xe1, 0x49, 0x03, 0x7c, 0xa8, 0xef, 0x6c, 0xc0, 0xa9, 0x80, 0xd9, - 0x8b, 0xf2, 0x00, 0xd5, 0xad, 0xcd, 0x7a, 0xad, 0xde, 0x58, 0xdf, 0x6c, 0x14, 0x4e, 0xa0, 0x1c, - 0xa4, 0xe9, 0xf3, 0xfa, 0x66, 0x7d, 0xbb, 0x5e, 0x50, 0x50, 0x01, 0xb2, 0xb5, 0x4d, 0x5f, 0x01, - 0xf1, 0xb1, 0xc4, 0x3b, 0x1f, 0x41, 0xc6, 0x77, 0xe3, 0x0f, 0x21, 0xc8, 0x3f, 0xdb, 0xae, 0x3f, - 0x56, 0x1b, 0xb5, 0xa7, 0xeb, 0xf5, 0xc6, 0xea, 0xd3, 0x67, 0x85, 0x13, 0x54, 0x32, 0xa3, 0xad, - 0x56, 0xb6, 0x70, 0xa3, 0xa0, 0xf4, 0x9f, 0x1b, 0x5b, 0xdb, 0xd5, 0xc7, 0x85, 0xb9, 0xfe, 0xf3, - 0x07, 0xdb, 0xeb, 0xf8, 0x93, 0x42, 0x4c, 0x08, 0xd6, 0xe0, 0x4c, 0x60, 0x02, 0x2c, 0xca, 0x40, - 0x72, 0xdb, 0x62, 0xd7, 0xcf, 0x78, 0x2b, 0xfb, 0x79, 0x98, 0x05, 0x05, 0xa5, 0x78, 0xae, 0x65, - 0x61, 0x0e, 0xcd, 0xc3, 0x5c, 0x7d, 0xa5, 0x10, 0x43, 0x27, 0x21, 0xe3, 0x4b, 0x24, 0x2d, 0xc4, - 0x51, 0x5a, 0x64, 0x13, 0x16, 0x12, 0x77, 0xae, 0x80, 0x2f, 0x7d, 0x0b, 0x01, 0xcc, 0x6f, 0x68, - 0x1e, 0x71, 0xbd, 0xc2, 0x09, 0x94, 0x84, 0xd8, 0x6a, 0xbb, 0x5d, 0x50, 0xee, 0x7f, 0x0c, 0x29, - 0xf9, 0x11, 0x0f, 0xb4, 0x01, 0x09, 0x0e, 0xa3, 0x4a, 0xe1, 0x53, 0x81, 0x4d, 0xaa, 0xc5, 0xa5, - 0x69, 0x73, 0xa5, 0x7c, 0xa2, 0x72, 0xe5, 0x8b, 0x7f, 0xb9, 0x7c, 0xe2, 0x8b, 0xa3, 0xcb, 0xca, - 0x97, 0x47, 0x97, 0x95, 0x7f, 0x38, 0xba, 0xac, 0xfc, 0xf3, 0xd1, 0x65, 0xe5, 0xb7, 0xff, 0xf5, - 0xf2, 0x89, 0x4f, 0x93, 0x82, 0x65, 0x67, 0x9e, 0x7d, 0x9f, 0x74, 0xe5, 0xbf, 0x03, 0x00, 0x00, - 0xff, 0xff, 0x05, 0x9e, 0x24, 0x7c, 0x80, 0x55, 0x00, 0x00, + 0x00, 0x73, 0xf2, 0xbd, 0x67, 0x55, 0x0a, 0x53, 0xf8, 0xff, 0xae, 0x8e, 0x6e, 0x41, 0xc1, 0xb2, + 0x55, 0x87, 0xec, 0x3a, 0xc4, 0x6d, 0x89, 0x6a, 0x93, 0x4c, 0x63, 0x79, 0xcb, 0xc6, 0x9c, 0xcc, + 0x44, 0x0b, 0x6b, 0xf8, 0x6d, 0x05, 0xce, 0x8e, 0x0e, 0x56, 0x94, 0xb3, 0xe4, 0x16, 0x14, 0x6c, + 0x8b, 0xa8, 0xdd, 0x96, 0xe6, 0x12, 0xa1, 0x65, 0x01, 0xa0, 0xf2, 0xb6, 0x45, 0x9e, 0x51, 0x32, + 0xd7, 0x19, 0x77, 0xed, 0xef, 0xc7, 0x53, 0xb1, 0x42, 0xbc, 0xfc, 0x19, 0x2c, 0xac, 0x1a, 0x1d, + 0xd3, 0xaa, 0x77, 0xdb, 0x66, 0x14, 0x98, 0xfb, 0x1a, 0xa4, 0x5d, 0x2a, 0x8a, 0x82, 0x3b, 0x66, + 0x40, 0x3e, 0x04, 0x95, 0x62, 0x6f, 0x9e, 0x90, 0x83, 0x01, 0xea, 0xf1, 0xd7, 0x1d, 0xe5, 0xcc, + 0x68, 0x88, 0x6e, 0x3d, 0x25, 0x4e, 0xb4, 0x30, 0xcd, 0x2f, 0x35, 0xca, 0x06, 0x7f, 0x5f, 0x81, + 0xf3, 0x4c, 0x36, 0xb3, 0x8f, 0x5d, 0xe2, 0xb0, 0xa3, 0xae, 0x08, 0x06, 0xe4, 0x2a, 0xcc, 0x7b, + 0x9a, 0xd3, 0x24, 0x7c, 0x3a, 0x27, 0x2a, 0x99, 0xaf, 0x0e, 0x4b, 0xc9, 0xba, 0x67, 0x3b, 0xa4, + 0xb6, 0x86, 0xc5, 0x2b, 0xd1, 0x3d, 0x0d, 0x16, 0x83, 0x9a, 0x10, 0x65, 0x37, 0x7f, 0xaa, 0x88, + 0x3a, 0xaa, 0x2d, 0x0e, 0x75, 0xbb, 0x6d, 0x53, 0xd7, 0xa2, 0x58, 0xa3, 0xd6, 0x21, 0xa3, 0x33, + 0x99, 0xaa, 0x77, 0xd0, 0xe5, 0x1b, 0xae, 0xfc, 0xfd, 0x6b, 0x81, 0x6d, 0x64, 0x75, 0xf2, 0x06, + 0x34, 0x0e, 0xba, 0x04, 0x83, 0xde, 0xff, 0x8f, 0xd6, 0x20, 0xc9, 0x75, 0x22, 0x81, 0xc5, 0x04, + 0x11, 0x74, 0x0e, 0x37, 0x58, 0x61, 0xe1, 0x50, 0x24, 0xab, 0xd0, 0xe7, 0x0e, 0x5c, 0x08, 0xec, + 0x6b, 0xd4, 0xfb, 0x33, 0x06, 0xd6, 0x37, 0x6c, 0xfb, 0x79, 0xaf, 0x1b, 0x81, 0x22, 0x2f, 0x01, + 0x74, 0xb4, 0x7d, 0xbe, 0x3b, 0xe0, 0x58, 0x3f, 0x81, 0xd3, 0x1d, 0x6d, 0x9f, 0xd5, 0xe2, 0xa2, + 0x22, 0x24, 0x1d, 0x0e, 0x08, 0x85, 0x77, 0x91, 0x8f, 0x7d, 0xdc, 0x48, 0xdd, 0xca, 0x7f, 0x50, + 0xdc, 0xe8, 0x6f, 0x56, 0x94, 0x9e, 0xee, 0x3d, 0x98, 0xef, 0xb7, 0x2e, 0x76, 0xac, 0x83, 0x66, + 0xc1, 0x87, 0xb6, 0x61, 0xa1, 0xeb, 0x90, 0x5d, 0xe2, 0xe9, 0x2d, 0x62, 0xc8, 0xae, 0xc6, 0x8e, + 0x29, 0xac, 0x30, 0x10, 0xc1, 0x75, 0x53, 0xfe, 0x0d, 0x05, 0x4e, 0x3d, 0x26, 0x9a, 0xe3, 0xed, + 0x10, 0xcd, 0x6b, 0xec, 0x47, 0xb1, 0x1a, 0x3f, 0x80, 0x98, 0x65, 0xef, 0x09, 0xcc, 0x33, 0x79, + 0xc1, 0x15, 0xcd, 0xa2, 0xe5, 0x85, 0x01, 0x7e, 0x17, 0x4e, 0x0f, 0x37, 0x27, 0x4a, 0xcb, 0xfb, + 0xe3, 0x18, 0xa4, 0x1f, 0x55, 0x23, 0xe8, 0xe2, 0x3b, 0x62, 0xbf, 0x1d, 0xae, 0xff, 0x7e, 0x35, + 0xcb, 0x8f, 0xaa, 0x4f, 0xc8, 0x81, 0xdc, 0x0d, 0x50, 0x2e, 0xb4, 0x0a, 0x69, 0xaf, 0x45, 0xd7, + 0x5a, 0xbb, 0x6d, 0x08, 0x34, 0x31, 0x93, 0x9a, 0x06, 0x5c, 0xa8, 0x0d, 0x67, 0xbc, 0x7d, 0x8b, + 0x2d, 0xe1, 0x6a, 0x53, 0x57, 0x07, 0xe2, 0x12, 0xb3, 0x88, 0x5b, 0xa4, 0xe2, 0x8e, 0x0e, 0x4b, + 0xa8, 0xb1, 0x6f, 0xd1, 0x1e, 0x3e, 0xaa, 0x36, 0xa4, 0x00, 0x8c, 0x3c, 0x41, 0xd3, 0xfb, 0xb4, + 0xc5, 0xe7, 0x90, 0x60, 0xbd, 0x40, 0xe7, 0x21, 0x46, 0x17, 0x49, 0x65, 0x78, 0x91, 0xa4, 0x34, + 0xd6, 0x29, 0x59, 0xc1, 0x71, 0xc6, 0x7e, 0xc0, 0xc5, 0x2d, 0x40, 0xd8, 0xc1, 0x07, 0x00, 0x54, + 0x85, 0x51, 0x8e, 0xfe, 0x9f, 0xc7, 0x20, 0xff, 0xac, 0xe7, 0xb6, 0xa2, 0xb1, 0xf2, 0x2a, 0x40, + 0xb7, 0xe7, 0xb6, 0x88, 0xa3, 0x7a, 0xfb, 0x96, 0xe8, 0xf0, 0x94, 0xc0, 0x8d, 0xec, 0x31, 0xe7, + 0x6b, 0xec, 0x5b, 0x68, 0x4b, 0x08, 0x21, 0xea, 0x20, 0xfa, 0x73, 0x67, 0x86, 0x5d, 0x42, 0x63, + 0xdf, 0x7a, 0x4a, 0x3c, 0x6d, 0x48, 0x20, 0xa1, 0x02, 0xdf, 0x81, 0x24, 0x7d, 0x50, 0x3d, 0xfb, + 0x38, 0x86, 0x35, 0x4f, 0x79, 0x1a, 0xb6, 0x9c, 0xb9, 0x89, 0xe3, 0xcd, 0x5c, 0xf4, 0x36, 0xa4, + 0x79, 0xa5, 0x74, 0x15, 0x9b, 0x67, 0xab, 0x58, 0x90, 0x26, 0x84, 0xee, 0xd9, 0xfa, 0x95, 0x62, + 0x35, 0xd2, 0xd5, 0xeb, 0x34, 0x24, 0x76, 0x6d, 0x47, 0x27, 0x02, 0x88, 0xf2, 0x87, 0xbe, 0x63, + 0x4e, 0x15, 0xd2, 0xe5, 0xdf, 0x55, 0xe0, 0x64, 0x7f, 0xdc, 0xa2, 0x74, 0xca, 0xd5, 0x21, 0xed, + 0x1f, 0x7f, 0x08, 0xa9, 0xc6, 0xcb, 0x7f, 0x34, 0x07, 0x27, 0x3f, 0xe8, 0x11, 0xe7, 0x20, 0x1a, + 0xb3, 0xaa, 0xf0, 0x40, 0xe0, 0xdc, 0x2b, 0x9a, 0x02, 0x0b, 0x0d, 0xde, 0x80, 0x93, 0x7b, 0x9a, + 0xe9, 0xa9, 0xbb, 0xb6, 0xa3, 0xf6, 0xba, 0x86, 0xe6, 0xc9, 0x28, 0x4c, 0x8e, 0x92, 0x1f, 0xda, + 0xce, 0x36, 0x23, 0x22, 0x02, 0xe8, 0xb9, 0x65, 0xef, 0x59, 0x2a, 0x25, 0x9b, 0x56, 0x93, 0xaa, + 0xc1, 0x2d, 0xc6, 0xd9, 0x19, 0xe2, 0xcf, 0xfc, 0xe3, 0x61, 0x69, 0xa5, 0x69, 0x7a, 0xad, 0xde, + 0xce, 0xb2, 0x6e, 0x77, 0xee, 0xf6, 0x1b, 0x62, 0xec, 0x0c, 0xfe, 0xdf, 0xed, 0x3e, 0x6f, 0xde, + 0x65, 0x51, 0xd8, 0x5e, 0xcf, 0x34, 0x96, 0xb7, 0xb7, 0x6b, 0x6b, 0xb8, 0xc0, 0x44, 0x7e, 0xc4, + 0x25, 0x36, 0xf6, 0xfb, 0x7b, 0x89, 0xaf, 0x14, 0x28, 0x0c, 0xf4, 0x14, 0xe5, 0x30, 0xae, 0x43, + 0xe6, 0x45, 0x8f, 0x38, 0x26, 0x31, 0x8e, 0x3d, 0x8e, 0x20, 0x18, 0xe9, 0xd4, 0xf9, 0x14, 0xb2, + 0x43, 0x7a, 0x88, 0x7d, 0x3d, 0x3d, 0x64, 0xf6, 0x06, 0x2a, 0x28, 0xff, 0x97, 0x02, 0xa7, 0x31, + 0x71, 0xed, 0xf6, 0x4b, 0xc2, 0x8f, 0xac, 0x22, 0xb0, 0x94, 0x2d, 0x10, 0xa7, 0x43, 0xea, 0xd7, + 0x31, 0x98, 0x34, 0x97, 0xc1, 0x7d, 0xc7, 0xbc, 0xeb, 0x69, 0x5e, 0x8f, 0x9f, 0xb1, 0x05, 0x23, + 0x51, 0x9f, 0x0a, 0xeb, 0xac, 0x2c, 0x16, 0x3c, 0x74, 0x0f, 0xde, 0xb5, 0x4d, 0xd7, 0xb6, 0xe4, + 0x11, 0x38, 0x7f, 0x12, 0xa3, 0xff, 0x73, 0x70, 0x66, 0xa4, 0xff, 0x51, 0x7a, 0xf6, 0xbf, 0x99, + 0x83, 0xf3, 0xc3, 0xe2, 0x23, 0x3a, 0xf8, 0xff, 0xdf, 0xa1, 0x63, 0xf4, 0x18, 0x72, 0x1d, 0xd3, + 0x52, 0x07, 0xeb, 0xef, 0x31, 0x3c, 0x78, 0x96, 0xee, 0x9e, 0x86, 0x97, 0xe0, 0x72, 0x1e, 0xb2, + 0x9b, 0xb6, 0xdd, 0x87, 0xc0, 0xe5, 0x1c, 0x64, 0xf8, 0x33, 0x53, 0x28, 0xdd, 0x74, 0x05, 0x69, + 0x3b, 0xca, 0x11, 0xfd, 0x55, 0x05, 0xb2, 0x11, 0x6d, 0x84, 0x5f, 0x2d, 0xf0, 0x28, 0x34, 0xd1, + 0x80, 0xdc, 0xff, 0xc0, 0xce, 0xf9, 0xf7, 0x14, 0x40, 0x0d, 0xa7, 0x67, 0xe9, 0x9a, 0x47, 0x36, + 0xec, 0x66, 0x04, 0x7d, 0x3c, 0x0d, 0x09, 0xd3, 0x32, 0xc8, 0x3e, 0xeb, 0x63, 0x1c, 0xf3, 0x07, + 0x74, 0x0f, 0x52, 0x22, 0x9d, 0x83, 0xc7, 0x4f, 0x63, 0x95, 0xb3, 0x47, 0x87, 0xa5, 0x24, 0x4f, + 0xde, 0x58, 0xfb, 0x6a, 0xf0, 0x17, 0x27, 0x79, 0xfe, 0x86, 0x0c, 0x2c, 0x7f, 0x0a, 0xa7, 0x86, + 0xda, 0x17, 0x65, 0xe7, 0x7f, 0xc4, 0xce, 0xe7, 0x59, 0x8f, 0xa3, 0x3a, 0x30, 0x78, 0xa5, 0x34, + 0x1c, 0xf4, 0x2e, 0x40, 0xd7, 0x21, 0x2f, 0x55, 0xce, 0x1a, 0x9b, 0x89, 0x35, 0x4d, 0x39, 0x18, + 0x41, 0x68, 0xea, 0xc7, 0x0a, 0x9c, 0x8e, 0xfa, 0xfc, 0xe3, 0x1b, 0xec, 0x4e, 0x1d, 0x0a, 0xec, + 0xb1, 0x66, 0xed, 0xda, 0x91, 0x9d, 0x41, 0xfd, 0xa6, 0x02, 0x0b, 0x3e, 0xa9, 0x51, 0xae, 0xfd, + 0xaf, 0x96, 0x80, 0xf5, 0x5d, 0xba, 0x1a, 0xfb, 0x2d, 0x30, 0x4a, 0xfb, 0xfe, 0x4f, 0x05, 0xce, + 0x56, 0x79, 0x60, 0x81, 0x85, 0x4d, 0xdc, 0x5e, 0x27, 0x02, 0x9b, 0x28, 0x42, 0xf2, 0x25, 0x71, + 0x5c, 0xd3, 0xe6, 0xcb, 0x50, 0x0e, 0xcb, 0x47, 0xf4, 0x8b, 0x90, 0xd1, 0x45, 0x3d, 0x72, 0x9e, + 0x67, 0x2b, 0x35, 0x2a, 0xe0, 0x15, 0xa1, 0xcb, 0xd1, 0x61, 0x09, 0x64, 0xcb, 0x6b, 0x6b, 0x18, + 0xa4, 0xf4, 0x9a, 0x81, 0x16, 0x21, 0xe5, 0x5a, 0x5a, 0xd7, 0x6d, 0xd9, 0xf2, 0xa0, 0xb6, 0xff, + 0x2c, 0xc6, 0xfa, 0x7b, 0x70, 0x6e, 0xac, 0xf3, 0x51, 0x6a, 0x77, 0x07, 0x4a, 0x6b, 0xa4, 0xeb, + 0x10, 0xea, 0x9a, 0x8c, 0x0f, 0x89, 0x63, 0xee, 0x1e, 0x44, 0xa7, 0x65, 0xd1, 0x87, 0x26, 0x2c, + 0x85, 0xd7, 0x11, 0x65, 0x67, 0x7e, 0x9c, 0x84, 0xdc, 0xfa, 0x7e, 0xd7, 0x76, 0xbc, 0x3a, 0x07, + 0x12, 0x68, 0x0d, 0x52, 0x5d, 0xc7, 0x7e, 0x69, 0x4a, 0xc1, 0xf9, 0xc0, 0x38, 0xc3, 0x10, 0xcf, + 0x33, 0x51, 0x1e, 0xf7, 0x39, 0x11, 0x86, 0xf4, 0x86, 0xad, 0x6b, 0xed, 0x87, 0x66, 0x5b, 0xce, + 0x8c, 0xe5, 0x69, 0x62, 0x96, 0xfb, 0x1c, 0xcf, 0x34, 0xaf, 0x25, 0xfd, 0x43, 0x9f, 0x88, 0x1e, + 0x41, 0xea, 0xb1, 0xe7, 0x75, 0xe9, 0x4b, 0xe1, 0x5c, 0xae, 0x4f, 0x15, 0x49, 0x19, 0x84, 0xa4, + 0x3e, 0x33, 0xc2, 0xb0, 0xf0, 0xc8, 0xb6, 0x9b, 0x6d, 0x52, 0x6d, 0xdb, 0x3d, 0xa3, 0x6a, 0x5b, + 0xbb, 0x66, 0x53, 0x6c, 0x56, 0xaf, 0x4d, 0x95, 0xf8, 0xa8, 0x5a, 0xc7, 0xe3, 0xec, 0xe8, 0x3b, + 0x90, 0xaa, 0xaf, 0x08, 0x51, 0x1c, 0xfb, 0x5c, 0x9d, 0x2a, 0xaa, 0xbe, 0x82, 0xfb, 0x4c, 0xe8, + 0x31, 0x64, 0x56, 0x3f, 0xeb, 0x39, 0x44, 0xc8, 0x98, 0x67, 0x32, 0x6e, 0x4c, 0x95, 0xc1, 0x78, + 0xb0, 0x9f, 0x75, 0xf1, 0x36, 0xe4, 0x86, 0x34, 0x89, 0x10, 0xc4, 0xbb, 0x54, 0x69, 0x74, 0x38, + 0xd3, 0x98, 0xfd, 0xe7, 0x76, 0xb6, 0x78, 0x03, 0xe2, 0x54, 0x2b, 0x74, 0x6e, 0xef, 0x68, 0x2e, + 0xd9, 0x76, 0x4c, 0x51, 0x48, 0x3e, 0x8a, 0x72, 0x7f, 0xa5, 0xc0, 0x5c, 0x7d, 0x85, 0xa2, 0xbf, + 0x9d, 0x9e, 0xfe, 0x9c, 0x78, 0xa2, 0x94, 0x78, 0x62, 0xa8, 0xd0, 0x21, 0xbb, 0x26, 0x5f, 0xfc, + 0xd3, 0x58, 0x3c, 0xa1, 0x4b, 0x00, 0x9a, 0xae, 0x13, 0xd7, 0x65, 0x81, 0x8d, 0x18, 0x7b, 0x97, + 0xe6, 0x94, 0x27, 0xe4, 0x80, 0xb2, 0xb9, 0x44, 0x77, 0x08, 0x9f, 0xc9, 0x69, 0x2c, 0x9e, 0x28, + 0x9b, 0x47, 0x3a, 0x5d, 0xd5, 0xb3, 0x9f, 0x13, 0x8b, 0x69, 0x33, 0x8d, 0xd3, 0x94, 0xd2, 0xa0, + 0x04, 0xea, 0x02, 0x88, 0x65, 0x74, 0x6d, 0xd3, 0xf2, 0x98, 0x9a, 0xd2, 0xb8, 0xff, 0x4c, 0x45, + 0x3a, 0xa4, 0x69, 0x8a, 0x94, 0xcf, 0x34, 0x16, 0x4f, 0xa2, 0x1b, 0x5b, 0x10, 0x7b, 0x54, 0xad, + 0x1f, 0xbb, 0x1b, 0x08, 0xe2, 0x5a, 0x4f, 0x18, 0x5d, 0x1a, 0xb3, 0xff, 0x42, 0xe0, 0xaf, 0x29, + 0x90, 0x60, 0xaa, 0x47, 0x17, 0x21, 0xad, 0xdb, 0x96, 0xa7, 0x99, 0x96, 0x98, 0x37, 0x69, 0x3c, + 0x20, 0x84, 0x4a, 0xbe, 0x02, 0x59, 0x4d, 0xd7, 0xed, 0x9e, 0xe5, 0xa9, 0x96, 0xd6, 0x21, 0xa2, + 0x86, 0x8c, 0xa0, 0x6d, 0x6a, 0x1d, 0x82, 0x4a, 0x20, 0x1f, 0x99, 0x12, 0xb9, 0xa6, 0x40, 0x90, + 0xfa, 0x61, 0x21, 0xe1, 0x37, 0xfe, 0x50, 0x81, 0x85, 0x8f, 0x1c, 0xd3, 0x23, 0x15, 0xcd, 0xd3, + 0x5b, 0x11, 0x38, 0xfd, 0xb7, 0x20, 0x6d, 0x68, 0x9e, 0xc6, 0x33, 0x7c, 0xe7, 0x26, 0x73, 0x8b, + 0x69, 0x46, 0xcb, 0xb3, 0x2c, 0x5f, 0x04, 0x71, 0xfa, 0x9f, 0xaf, 0x07, 0x98, 0xfd, 0x1f, 0x84, + 0x84, 0xfc, 0xad, 0x8c, 0xd2, 0xa1, 0xfd, 0xdd, 0x9c, 0x74, 0x68, 0x11, 0xf4, 0xfe, 0x3d, 0x48, + 0x8a, 0xfd, 0x95, 0xe8, 0xfb, 0xd2, 0xb4, 0xf9, 0x28, 0x63, 0x1a, 0x82, 0x0d, 0x55, 0x00, 0x5c, + 0x4f, 0x73, 0x3c, 0xb6, 0x33, 0x9a, 0x29, 0x02, 0x2c, 0xfd, 0x1e, 0x63, 0xa3, 0x54, 0xb4, 0x09, + 0x99, 0xce, 0x4b, 0x5d, 0x57, 0x77, 0xcd, 0xb6, 0x27, 0x82, 0xbf, 0xf9, 0x21, 0x21, 0xb2, 0x25, + 0x4f, 0x3f, 0xac, 0x56, 0x1f, 0xb2, 0x42, 0x3c, 0x06, 0x3b, 0x78, 0xc6, 0x40, 0x25, 0xf0, 0xff, + 0xe8, 0xdb, 0x20, 0x12, 0xc7, 0x54, 0xd7, 0xf5, 0xd8, 0xf4, 0x4a, 0x55, 0x72, 0x47, 0x87, 0xa5, + 0x34, 0x66, 0xd4, 0x7a, 0xbd, 0x81, 0xd3, 0xbc, 0x40, 0xdd, 0x95, 0x8b, 0xea, 0x0f, 0x14, 0xc8, + 0x55, 0x7a, 0xed, 0xe7, 0x5b, 0xdd, 0x7a, 0xaf, 0xd3, 0xd1, 0x9c, 0x03, 0x74, 0x41, 0x5a, 0x86, + 0xf9, 0x19, 0x61, 0x9a, 0x8d, 0x89, 0xa1, 0x37, 0x3f, 0x23, 0x74, 0xe8, 0x45, 0x92, 0x0a, 0xa5, + 0xf3, 0x0c, 0x94, 0xab, 0x90, 0x63, 0x7b, 0x02, 0x95, 0x58, 0x9e, 0x63, 0x12, 0xbe, 0xff, 0x8c, + 0xe1, 0x2c, 0x23, 0xae, 0x73, 0x1a, 0xba, 0x0e, 0x79, 0xf7, 0xc0, 0xf5, 0x48, 0x47, 0xe5, 0xd9, + 0xf4, 0x3c, 0xa9, 0x32, 0x86, 0x73, 0x9c, 0x8a, 0x39, 0xb1, 0xfc, 0x6f, 0x73, 0x90, 0x97, 0xa3, + 0x1c, 0x25, 0x98, 0xab, 0x40, 0x62, 0xd7, 0x6c, 0xf7, 0x63, 0x24, 0xe1, 0xee, 0x57, 0x4a, 0x5a, + 0xa6, 0x4e, 0x56, 0x42, 0x3b, 0xc6, 0xba, 0xf8, 0xa5, 0x02, 0x71, 0xb6, 0x5e, 0xdd, 0x83, 0x38, + 0x9b, 0x36, 0xca, 0x2c, 0xd3, 0x86, 0x15, 0xed, 0x7b, 0xea, 0xb9, 0x81, 0xa7, 0x66, 0x5e, 0xb2, + 0xa5, 0x3d, 0xb8, 0x77, 0x9f, 0x0d, 0x55, 0x16, 0x8b, 0x27, 0x54, 0x81, 0x14, 0x61, 0x6d, 0x21, + 0x86, 0x58, 0x2d, 0x82, 0xac, 0x73, 0x68, 0xd0, 0xe4, 0x14, 0x95, 0x7c, 0xe8, 0x3c, 0xc4, 0xa8, + 0x0d, 0x24, 0xf9, 0x69, 0xfa, 0xd1, 0x61, 0x29, 0x46, 0x47, 0x9f, 0xd2, 0x78, 0x48, 0xea, 0xfd, + 0x78, 0x2a, 0x5e, 0x48, 0x94, 0xff, 0x34, 0x0e, 0xb9, 0x5a, 0x27, 0xa2, 0x49, 0xb5, 0x3a, 0xac, + 0xe3, 0xa0, 0x35, 0x7c, 0xa8, 0xae, 0x71, 0x15, 0x0f, 0x7b, 0xa5, 0xd8, 0xf1, 0xbc, 0x52, 0x8d, + 0xae, 0x10, 0xe2, 0xe6, 0x01, 0xad, 0xff, 0x5b, 0x53, 0xeb, 0x6f, 0x68, 0x3b, 0x6d, 0x82, 0x29, + 0x4f, 0x3f, 0x20, 0xc6, 0x04, 0xa0, 0x9f, 0x65, 0x0b, 0x11, 0x9f, 0xda, 0xf3, 0xb3, 0x4f, 0xed, + 0x24, 0xb1, 0x0c, 0x4a, 0x5b, 0xdc, 0x17, 0x86, 0xf2, 0x06, 0xc4, 0x0c, 0x53, 0x6a, 0x72, 0x56, + 0x17, 0x43, 0x59, 0xa6, 0xd8, 0x4b, 0xdc, 0x6f, 0x2f, 0xfe, 0x48, 0xe3, 0xe2, 0x16, 0xc0, 0xa0, + 0x57, 0x68, 0x09, 0xe6, 0xed, 0xb6, 0x41, 0xa1, 0x3b, 0x6d, 0x42, 0xae, 0x92, 0x3e, 0x3a, 0x2c, + 0x25, 0xb6, 0xda, 0x46, 0x6d, 0x0d, 0x27, 0xec, 0xb6, 0x51, 0x33, 0xd8, 0x85, 0x0d, 0xb2, 0xa7, + 0xb2, 0x7b, 0x2a, 0x2c, 0x3f, 0x01, 0x27, 0x2d, 0xb2, 0xb7, 0x46, 0x5c, 0xdd, 0xbf, 0xfc, 0x08, + 0x6b, 0xf9, 0x7d, 0x05, 0xf2, 0x52, 0x83, 0xd1, 0x4e, 0xce, 0x94, 0xd9, 0x11, 0x06, 0x1f, 0x3b, + 0x9e, 0xc1, 0x4b, 0x3e, 0x91, 0xa0, 0xf7, 0x7d, 0x05, 0x4e, 0xf1, 0x24, 0x0a, 0x5d, 0xf3, 0xa8, + 0x7b, 0xfc, 0xfa, 0x56, 0x7d, 0x1b, 0x0a, 0x8e, 0x66, 0x19, 0x76, 0xc7, 0xfc, 0x8c, 0xf0, 0x0d, + 0xb0, 0x2b, 0x52, 0x81, 0x4e, 0xf6, 0xe9, 0x6c, 0xab, 0x27, 0x4f, 0xa3, 0x7f, 0xaa, 0xc0, 0xe9, + 0xe1, 0x36, 0x44, 0xa9, 0xab, 0x27, 0x23, 0xd1, 0xde, 0xd7, 0x02, 0x84, 0x04, 0xd5, 0xce, 0xa3, + 0xb6, 0xc3, 0x81, 0xdf, 0xc5, 0xf7, 0x20, 0xc1, 0xc8, 0xaf, 0xe0, 0xd1, 0x84, 0xc2, 0x5b, 0xb0, + 0xb0, 0x6a, 0x18, 0xf5, 0xba, 0x30, 0xba, 0xaf, 0xad, 0x6d, 0x09, 0x2d, 0xe6, 0x82, 0xa0, 0x85, + 0xbf, 0xa6, 0x28, 0xa1, 0x45, 0x13, 0xf2, 0x22, 0x43, 0x29, 0x9a, 0xe3, 0xb2, 0x3d, 0x0a, 0x81, + 0x84, 0x91, 0xf0, 0x87, 0xc1, 0x4d, 0x81, 0x7e, 0x45, 0x51, 0x76, 0xa0, 0x0d, 0xa7, 0xa4, 0xdc, + 0x88, 0x4e, 0xa7, 0x27, 0xf5, 0x82, 0x1d, 0x71, 0xf8, 0x6b, 0x8b, 0xb2, 0x2b, 0xff, 0x7e, 0x16, + 0xb2, 0xa2, 0xfd, 0xdb, 0x96, 0x69, 0x5b, 0xe8, 0x2e, 0xc4, 0x9a, 0x02, 0xd0, 0x67, 0x02, 0xc1, + 0xd1, 0xe0, 0x82, 0x15, 0xa6, 0x25, 0x29, 0x43, 0xb7, 0xe7, 0x05, 0xc4, 0x89, 0x07, 0xc1, 0xc2, + 0x01, 0x43, 0xb7, 0xe7, 0xa1, 0x0f, 0xe0, 0xa4, 0x3e, 0xb8, 0xea, 0xa2, 0x52, 0xe6, 0x58, 0x68, + 0x1e, 0x5e, 0xe0, 0x8d, 0x1f, 0x9c, 0xd7, 0x87, 0xc8, 0x68, 0xd5, 0x7f, 0xd7, 0x22, 0x1e, 0xba, + 0x6b, 0x1c, 0xbd, 0xde, 0xe1, 0xbb, 0x90, 0x81, 0xde, 0x80, 0x79, 0x83, 0xe5, 0xf4, 0x8b, 0x5d, + 0x67, 0x90, 0x4b, 0x1c, 0xba, 0x34, 0x81, 0x45, 0x79, 0xf4, 0x18, 0xb2, 0xfc, 0x1f, 0x4f, 0xe5, + 0x10, 0x2b, 0xd8, 0xf5, 0x70, 0x7e, 0x9f, 0xcd, 0xe0, 0x8c, 0x31, 0xa0, 0xa1, 0xfb, 0x10, 0x77, + 0x75, 0x8d, 0x6f, 0xb9, 0x82, 0xe3, 0x5e, 0xbe, 0x5c, 0x6c, 0xcc, 0xca, 0xa2, 0x8f, 0x60, 0x61, + 0x87, 0x34, 0x4d, 0x4b, 0xf5, 0x06, 0xb1, 0x86, 0x62, 0x6a, 0x2c, 0xbc, 0xd1, 0xf7, 0xea, 0xc1, + 0xe9, 0xa6, 0xb8, 0xb0, 0x33, 0xf2, 0x82, 0x0e, 0x13, 0x5b, 0x94, 0x7d, 0x62, 0xd3, 0xa1, 0xc3, + 0x14, 0x98, 0x07, 0x8a, 0xf3, 0x64, 0x88, 0x8c, 0xd6, 0x21, 0xa3, 0x51, 0x5f, 0xa9, 0xb2, 0x54, + 0xbc, 0x22, 0x84, 0x9e, 0x14, 0x8c, 0x25, 0x05, 0x62, 0xd0, 0xfa, 0xa4, 0x81, 0x98, 0x0e, 0x71, + 0x9a, 0xa4, 0x98, 0x99, 0x2c, 0xc6, 0x1f, 0x7b, 0x10, 0x62, 0x18, 0x09, 0x3d, 0x81, 0x5c, 0x4b, + 0xe6, 0xa7, 0xb0, 0xa0, 0x50, 0x36, 0xf4, 0xa8, 0x20, 0x20, 0xad, 0x06, 0x67, 0x5b, 0x3e, 0x22, + 0xfa, 0x36, 0xcc, 0x35, 0xf5, 0x62, 0x8e, 0x49, 0xb8, 0x38, 0x29, 0x89, 0x04, 0xcf, 0x35, 0x75, + 0xf4, 0x0e, 0xa4, 0x78, 0x98, 0x7d, 0xdf, 0x2a, 0xe6, 0x43, 0x27, 0xef, 0x70, 0x86, 0x03, 0x66, + 0xe9, 0x00, 0xb4, 0xae, 0xc7, 0x90, 0xe5, 0xb1, 0x80, 0x36, 0x4b, 0x6f, 0x2a, 0x9e, 0x0c, 0x35, + 0xb8, 0xf1, 0xdc, 0x2c, 0xcc, 0x6f, 0x85, 0x72, 0x1a, 0xda, 0x84, 0xbc, 0xc3, 0xc3, 0x3f, 0x22, + 0x29, 0xbf, 0x58, 0x60, 0xb2, 0x6e, 0x06, 0xbb, 0x92, 0xb1, 0xa0, 0x27, 0xce, 0x39, 0x7e, 0x2a, + 0xfa, 0x1e, 0x9c, 0x1e, 0x96, 0x27, 0xa6, 0xc4, 0x02, 0x93, 0xfa, 0xed, 0xa9, 0x52, 0xfd, 0x33, + 0x03, 0x39, 0x63, 0xaf, 0xd0, 0x03, 0x48, 0xf0, 0x31, 0x47, 0x4c, 0x60, 0x29, 0x68, 0xef, 0xe6, + 0x1f, 0x6e, 0x5e, 0x9a, 0x2a, 0xcc, 0x13, 0x31, 0x10, 0xb5, 0x6d, 0x37, 0x8b, 0xa7, 0x42, 0x15, + 0x36, 0x1e, 0xca, 0xc1, 0x19, 0x6f, 0x40, 0xa3, 0x36, 0xe3, 0x70, 0xba, 0x38, 0x9c, 0x3f, 0x1d, + 0x6a, 0x33, 0x01, 0x81, 0x11, 0x9c, 0x75, 0x7c, 0x44, 0x36, 0x8e, 0x3c, 0x7b, 0x4d, 0x65, 0xd3, + 0xfe, 0x4c, 0xf8, 0x38, 0x8e, 0xdd, 0xc4, 0xc0, 0x19, 0x67, 0x40, 0x43, 0x0d, 0x28, 0x88, 0x04, + 0x78, 0x55, 0x1e, 0xf1, 0x16, 0xcf, 0x32, 0x69, 0xb7, 0x03, 0x7d, 0x6a, 0xd0, 0x91, 0x36, 0x3e, + 0xa9, 0x0f, 0xd3, 0x51, 0x17, 0x16, 0x8d, 0xfe, 0xe1, 0xa9, 0xfa, 0x92, 0x9d, 0x9e, 0x0e, 0xe4, + 0x9f, 0x63, 0xf2, 0xef, 0x07, 0xba, 0xb9, 0x89, 0xa7, 0xba, 0xb8, 0x68, 0x84, 0x14, 0xa0, 0xce, + 0x8c, 0xc9, 0x57, 0xf5, 0xc1, 0x05, 0x85, 0x62, 0x31, 0xd4, 0x99, 0x85, 0x5c, 0xa2, 0xc0, 0x05, + 0x7d, 0xe4, 0x05, 0xf5, 0xac, 0x96, 0x6d, 0x77, 0x8b, 0xe7, 0x43, 0x3d, 0xab, 0x2f, 0x2a, 0x8a, + 0x59, 0x59, 0x3a, 0x49, 0x4d, 0xcb, 0xf4, 0xd8, 0x02, 0xb5, 0x18, 0x3a, 0x49, 0x87, 0x6f, 0x55, + 0xe2, 0xa4, 0xc9, 0x9f, 0xe9, 0xd4, 0xf2, 0x44, 0x30, 0x49, 0x98, 0xca, 0xc5, 0xd0, 0xa9, 0x15, + 0x14, 0x75, 0xc2, 0x39, 0xcf, 0x4f, 0xa5, 0x53, 0x8b, 0x3b, 0xbd, 0x11, 0xa9, 0x97, 0x42, 0xa7, + 0x56, 0x68, 0x42, 0x2f, 0x46, 0xda, 0xd8, 0x2b, 0x54, 0x01, 0x60, 0x02, 0xd9, 0x7d, 0xf1, 0xe2, + 0xe5, 0xd0, 0x35, 0x74, 0x34, 0xa6, 0x84, 0xd3, 0x6d, 0x49, 0xa1, 0x8e, 0x99, 0x01, 0x17, 0x75, + 0x47, 0xf3, 0xf4, 0x56, 0xb1, 0x14, 0xea, 0x98, 0xc7, 0x8e, 0xd6, 0x30, 0xec, 0xf5, 0x49, 0x74, + 0x29, 0xe6, 0x1b, 0xeb, 0xe2, 0xd2, 0x94, 0x9d, 0x5c, 0x7f, 0x29, 0xe6, 0xe5, 0xd1, 0x77, 0x20, + 0xfd, 0xa2, 0x47, 0x9c, 0x03, 0xe6, 0x58, 0xaf, 0x84, 0x7e, 0x07, 0x60, 0x24, 0xc9, 0x07, 0xa7, + 0x5e, 0x08, 0x02, 0xad, 0x9a, 0x6f, 0x71, 0x8a, 0xe5, 0xd0, 0xaa, 0x87, 0x36, 0xb5, 0x58, 0x94, + 0x47, 0x1a, 0x9c, 0xe1, 0xe3, 0x23, 0xf2, 0x80, 0x1d, 0x91, 0x70, 0x5b, 0xbc, 0xca, 0x04, 0x85, + 0xee, 0x1b, 0x02, 0x53, 0x91, 0xf1, 0x29, 0x6d, 0xfc, 0x1d, 0x75, 0x3e, 0x62, 0xf9, 0xe4, 0x7b, + 0x8d, 0xe2, 0xb5, 0x50, 0xe7, 0x13, 0xb0, 0x29, 0xc3, 0x59, 0xcd, 0x47, 0xe4, 0x8b, 0xa8, 0xa1, + 0xba, 0xae, 0x47, 0x01, 0x7e, 0xf1, 0xfa, 0x84, 0x45, 0x74, 0x64, 0xbf, 0xc1, 0x6e, 0xb0, 0xd4, + 0x39, 0x1f, 0x4b, 0x7c, 0x6e, 0x13, 0xcd, 0x11, 0x8e, 0xfe, 0x46, 0xa8, 0x98, 0xb1, 0x7b, 0x8b, + 0x18, 0xf4, 0x3e, 0x89, 0x82, 0x0d, 0x47, 0xde, 0xe1, 0x51, 0x5d, 0x4f, 0xf3, 0xdc, 0xe2, 0xcd, + 0x50, 0xb0, 0x11, 0x78, 0xc3, 0x08, 0xe7, 0x9d, 0x21, 0x32, 0x7a, 0x1b, 0x92, 0xe2, 0x7a, 0x44, + 0xf1, 0xd6, 0x04, 0x7c, 0xec, 0xdf, 0x87, 0x60, 0xc9, 0xc1, 0xfd, 0x3c, 0xbf, 0x5b, 0xc1, 0x3b, + 0x76, 0x7b, 0x82, 0x9f, 0x1f, 0xdb, 0x09, 0x50, 0x3f, 0x3f, 0x20, 0xbe, 0x15, 0xff, 0x82, 0xef, + 0xe9, 0x2f, 0x14, 0x2e, 0x96, 0x7f, 0x74, 0x16, 0x72, 0x12, 0x94, 0x73, 0xc0, 0xfd, 0xba, 0x1f, + 0x70, 0x5f, 0x0e, 0x03, 0xdc, 0x9c, 0x83, 0x23, 0xee, 0xd7, 0xfd, 0x88, 0xfb, 0x72, 0x18, 0xe2, + 0x96, 0x1c, 0x14, 0x72, 0xe3, 0x30, 0xc8, 0x7d, 0x7b, 0x06, 0xc8, 0x2d, 0x04, 0x8d, 0x62, 0xee, + 0xca, 0x38, 0xe6, 0xbe, 0x36, 0x19, 0x73, 0x0b, 0x41, 0x3e, 0xd0, 0xfd, 0xe6, 0x08, 0xe8, 0xbe, + 0x32, 0x01, 0x74, 0x0b, 0x6e, 0x89, 0xba, 0x6b, 0x81, 0xa8, 0xfb, 0xc6, 0x34, 0xd4, 0x2d, 0xa4, + 0x0c, 0xc1, 0xee, 0x95, 0x21, 0xd8, 0x5d, 0x0a, 0x85, 0xdd, 0x82, 0x97, 0xe3, 0xee, 0x8f, 0xc3, + 0x71, 0xf7, 0xb7, 0x66, 0xc2, 0xdd, 0x42, 0xda, 0x38, 0xf0, 0xc6, 0x61, 0xc0, 0xfb, 0xf6, 0x0c, + 0xc0, 0x5b, 0x0e, 0xd6, 0x08, 0xf2, 0x7e, 0x18, 0x84, 0xbc, 0xaf, 0x4f, 0x41, 0xde, 0x42, 0x96, + 0x1f, 0x7a, 0x3f, 0x0c, 0x82, 0xde, 0xd7, 0xa7, 0x40, 0xef, 0x21, 0x39, 0x1c, 0x7b, 0x6f, 0x04, + 0x63, 0xef, 0x9b, 0x53, 0xb1, 0xb7, 0x90, 0x35, 0x0c, 0xbe, 0x5f, 0xf3, 0x81, 0xef, 0x4b, 0x21, + 0xe0, 0x5b, 0x30, 0x52, 0xf4, 0xfd, 0xee, 0x18, 0xfa, 0x2e, 0x4f, 0x42, 0xdf, 0x82, 0xb3, 0x0f, + 0xbf, 0x6b, 0x81, 0xf0, 0xfb, 0xc6, 0x34, 0xf8, 0x2d, 0x2d, 0xcf, 0x8f, 0xbf, 0xb7, 0x42, 0xf0, + 0xf7, 0xad, 0xe9, 0xf8, 0x5b, 0x88, 0x1b, 0x01, 0xe0, 0xea, 0x44, 0x00, 0xfe, 0xda, 0x8c, 0x00, + 0x5c, 0xc8, 0x0e, 0x42, 0xe0, 0xff, 0x7f, 0x18, 0x81, 0x2f, 0x85, 0x23, 0x70, 0x21, 0x44, 0x40, + 0xf0, 0x5a, 0x20, 0x04, 0xbf, 0x31, 0x0d, 0x82, 0x4b, 0xa5, 0xf9, 0x31, 0xf8, 0x46, 0x30, 0x06, + 0xbf, 0x39, 0x15, 0x83, 0x4b, 0xdb, 0x19, 0x02, 0xe1, 0xb5, 0x40, 0x10, 0x7e, 0x63, 0x1a, 0x08, + 0xef, 0x8f, 0xa6, 0x0f, 0x85, 0x6f, 0x87, 0xa2, 0xf0, 0x3b, 0xb3, 0xa0, 0x70, 0x21, 0x72, 0x0c, + 0x86, 0xbf, 0x98, 0x01, 0x86, 0xaf, 0x1c, 0x0b, 0x86, 0x8b, 0x9a, 0xc2, 0x71, 0xf8, 0xc7, 0xe1, + 0x38, 0xfc, 0x5b, 0x33, 0xe1, 0x70, 0xe9, 0xdc, 0xc6, 0x80, 0xf8, 0xca, 0x10, 0x10, 0x2f, 0x85, + 0x02, 0x71, 0xe9, 0x6b, 0x19, 0x12, 0x7f, 0x77, 0x0c, 0x89, 0x97, 0x27, 0x21, 0x71, 0x39, 0x61, + 0x25, 0x14, 0x57, 0x27, 0x42, 0xe7, 0xd7, 0x66, 0x84, 0xce, 0x72, 0x52, 0x04, 0x60, 0xe7, 0x6a, + 0x00, 0x76, 0xbe, 0x36, 0x19, 0x3b, 0xcb, 0xb5, 0x70, 0x00, 0x9e, 0x1f, 0x06, 0x81, 0xe7, 0xeb, + 0x53, 0xc0, 0xb3, 0x74, 0xad, 0x3e, 0xf4, 0xfc, 0xe6, 0x08, 0x7a, 0xbe, 0x32, 0x35, 0xf6, 0xd6, + 0x87, 0xcf, 0xef, 0x8d, 0xc3, 0xe7, 0xab, 0x13, 0xe1, 0xb3, 0xe0, 0x1f, 0xe0, 0xe7, 0x37, 0x47, + 0xf0, 0xf3, 0x95, 0x09, 0xf8, 0x59, 0x56, 0x2e, 0x00, 0xf4, 0xce, 0x64, 0x00, 0xbd, 0x3c, 0x2b, + 0x80, 0x16, 0x62, 0x03, 0x11, 0xf4, 0x46, 0x30, 0x82, 0xbe, 0x39, 0xe3, 0xa1, 0xfe, 0x08, 0x84, + 0x7e, 0x18, 0x04, 0xa1, 0xaf, 0x4f, 0x81, 0xd0, 0x83, 0xc5, 0xb0, 0x8f, 0xa1, 0x1f, 0x06, 0x61, + 0xe8, 0xeb, 0x53, 0x30, 0xb4, 0x94, 0xe3, 0x03, 0xd1, 0x38, 0x0c, 0x44, 0xdf, 0x9e, 0x01, 0x44, + 0x4b, 0xe0, 0x30, 0x82, 0xa2, 0xdf, 0x19, 0x45, 0xd1, 0xe5, 0x49, 0x28, 0x5a, 0xce, 0x3c, 0x09, + 0xa3, 0x37, 0x82, 0x61, 0xf4, 0xcd, 0xa9, 0x30, 0x7a, 0xe0, 0xaa, 0x83, 0x71, 0xf4, 0xfb, 0xf1, + 0xd4, 0xc5, 0xc2, 0xa5, 0xf2, 0xe7, 0x09, 0x98, 0x7f, 0x2c, 0xa3, 0xa1, 0xbe, 0x4b, 0x4b, 0xca, + 0xab, 0x5c, 0x5a, 0x42, 0x6b, 0xb4, 0xaf, 0xcc, 0x52, 0x04, 0xb6, 0x9e, 0x70, 0xfb, 0x72, 0xec, + 0x3e, 0x9e, 0x64, 0x7d, 0x85, 0x4c, 0x5d, 0xf4, 0x00, 0x72, 0x3d, 0x97, 0x38, 0x6a, 0xd7, 0x31, + 0x6d, 0xc7, 0xf4, 0x78, 0x72, 0x8a, 0x52, 0x29, 0x7c, 0x75, 0x58, 0xca, 0x6e, 0xbb, 0xc4, 0x79, + 0x26, 0xe8, 0x38, 0xdb, 0xf3, 0x3d, 0xc9, 0x2f, 0xcd, 0x25, 0x66, 0xff, 0xd2, 0xdc, 0x07, 0x50, + 0x70, 0x88, 0x66, 0x0c, 0xb9, 0x75, 0x7e, 0xcb, 0x27, 0x78, 0xc1, 0xd3, 0x0c, 0x9f, 0xef, 0x66, + 0xb7, 0x7d, 0x4e, 0x3a, 0xc3, 0x44, 0x74, 0x0f, 0xce, 0x74, 0xb4, 0x7d, 0x7e, 0x7d, 0x4d, 0x2e, + 0xcb, 0x2c, 0x2a, 0x9c, 0x62, 0xf9, 0x06, 0xa8, 0xa3, 0xed, 0xb3, 0xcf, 0xd6, 0xf1, 0x57, 0xec, + 0x13, 0x3b, 0xd7, 0x21, 0x6f, 0x98, 0xae, 0x67, 0x5a, 0xba, 0xbc, 0x30, 0x9f, 0xe6, 0x77, 0x5a, + 0x24, 0x95, 0xdf, 0x89, 0xbf, 0x03, 0x0b, 0x22, 0xbd, 0x62, 0xf0, 0x21, 0x3b, 0x86, 0x5c, 0x53, + 0xb4, 0x15, 0xf4, 0xc5, 0xe0, 0xdb, 0x7f, 0x55, 0x38, 0xd9, 0xd4, 0x3c, 0xb2, 0xa7, 0x1d, 0xa8, + 0x96, 0x6d, 0x30, 0xdd, 0x67, 0xd8, 0x85, 0xe3, 0x0b, 0x47, 0x87, 0xa5, 0xdc, 0x23, 0xfe, 0x6a, + 0xd3, 0x36, 0xf8, 0x08, 0xcc, 0xf3, 0x7f, 0x38, 0xd7, 0xf4, 0xbd, 0x30, 0xd0, 0x2a, 0x64, 0x29, + 0x04, 0x50, 0x6d, 0xfe, 0xb5, 0x1a, 0x81, 0x49, 0xc3, 0x8e, 0xe1, 0xc5, 0x37, 0x6d, 0x70, 0xc6, + 0x1d, 0x3c, 0xbc, 0x1f, 0x4f, 0x25, 0x0b, 0xa9, 0xf2, 0xe7, 0x0a, 0x64, 0x87, 0x12, 0x87, 0xde, + 0x1e, 0x09, 0xd5, 0x9c, 0x0f, 0xc6, 0xb9, 0xc1, 0x21, 0xc8, 0x55, 0x48, 0x09, 0xc5, 0xca, 0x20, + 0x64, 0x29, 0x1c, 0xea, 0xb0, 0x3d, 0xa5, 0x8c, 0xd6, 0x4a, 0xb6, 0xb7, 0xe2, 0xbf, 0xf3, 0xc3, + 0xd2, 0x89, 0xf2, 0x4f, 0x62, 0x90, 0x1b, 0xce, 0x14, 0xaa, 0x8d, 0xb4, 0x2b, 0x68, 0x62, 0x0e, + 0x71, 0x84, 0xb7, 0x72, 0x0d, 0xd2, 0x8e, 0x28, 0x24, 0x9b, 0xb9, 0x34, 0x21, 0x20, 0xe5, 0x6f, + 0xe7, 0x80, 0x71, 0xf1, 0x2f, 0xe7, 0xfa, 0x13, 0x7a, 0x19, 0x12, 0xec, 0xd3, 0x93, 0xa2, 0x69, + 0x41, 0xf9, 0xc0, 0xeb, 0xf4, 0x3d, 0xe6, 0xc5, 0xa8, 0x03, 0x68, 0xbc, 0xd2, 0xad, 0xc5, 0x3e, + 0xe1, 0x15, 0x3e, 0xdd, 0xf8, 0x8a, 0xd7, 0xec, 0xea, 0x74, 0x3f, 0xde, 0x6e, 0x13, 0xdd, 0x13, + 0x5f, 0xb9, 0x94, 0x1f, 0x68, 0xbc, 0x36, 0x2a, 0x42, 0x7c, 0x13, 0x73, 0x19, 0x8b, 0x6f, 0x62, + 0xfa, 0xe2, 0xc2, 0xf9, 0xbe, 0x08, 0x36, 0x5f, 0x78, 0xd2, 0x00, 0x1f, 0xea, 0x3b, 0x1b, 0x70, + 0x2a, 0x60, 0xf6, 0xa2, 0x3c, 0x40, 0x75, 0x6b, 0xb3, 0x5e, 0xab, 0x37, 0xd6, 0x37, 0x1b, 0x85, + 0x13, 0x28, 0x07, 0x69, 0xfa, 0xbc, 0xbe, 0x59, 0xdf, 0xae, 0x17, 0x14, 0x54, 0x80, 0x6c, 0x6d, + 0xd3, 0x57, 0x40, 0x7c, 0x56, 0xf1, 0xce, 0x47, 0x90, 0xf1, 0xdd, 0xf8, 0x43, 0x08, 0xf2, 0xcf, + 0xb6, 0xeb, 0x8f, 0xd5, 0x46, 0xed, 0xe9, 0x7a, 0xbd, 0xb1, 0xfa, 0xf4, 0x59, 0xe1, 0x04, 0x95, + 0xcc, 0x68, 0xab, 0x95, 0x2d, 0xdc, 0x28, 0x28, 0xfd, 0xe7, 0xc6, 0xd6, 0x76, 0xf5, 0x71, 0x61, + 0xae, 0xff, 0xfc, 0xc1, 0xf6, 0x3a, 0xfe, 0xa4, 0x10, 0x13, 0x82, 0x35, 0x38, 0x13, 0x98, 0x00, + 0x8b, 0x32, 0x90, 0xdc, 0xb6, 0xd8, 0xf5, 0x33, 0xde, 0xca, 0x7e, 0x1e, 0x66, 0x41, 0x41, 0x29, + 0x9e, 0x6b, 0x59, 0x98, 0x43, 0xf3, 0x30, 0x57, 0x5f, 0x29, 0xc4, 0xd0, 0x49, 0xc8, 0xf8, 0x12, + 0x49, 0x0b, 0x71, 0x94, 0x16, 0xd9, 0x84, 0x85, 0xc4, 0x9d, 0x2b, 0xe0, 0x4b, 0xdf, 0x42, 0x00, + 0xf3, 0x1b, 0x9a, 0x47, 0x5c, 0xaf, 0x70, 0x02, 0x25, 0x21, 0xb6, 0xda, 0x6e, 0x17, 0x94, 0xfb, + 0x1f, 0x43, 0x4a, 0x7e, 0xee, 0x03, 0x6d, 0x40, 0x82, 0xc3, 0xa8, 0x52, 0xf8, 0x54, 0x60, 0x93, + 0x6a, 0x71, 0x69, 0xda, 0x5c, 0x29, 0x9f, 0xa8, 0x5c, 0xf9, 0xe2, 0x5f, 0x2e, 0x9f, 0xf8, 0xe2, + 0xe8, 0xb2, 0xf2, 0xe5, 0xd1, 0x65, 0xe5, 0x1f, 0x8e, 0x2e, 0x2b, 0xff, 0x7c, 0x74, 0x59, 0xf9, + 0xad, 0x7f, 0xbd, 0x7c, 0xe2, 0xd3, 0xa4, 0x60, 0xd9, 0x99, 0x67, 0x5f, 0x32, 0x5d, 0xf9, 0xef, + 0x00, 0x00, 0x00, 0xff, 0xff, 0xef, 0x48, 0xf0, 0xce, 0xaa, 0x55, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 39db14a9fbc0..f0ffed6c28ca 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -455,6 +455,10 @@ message EndTransactionRequest { // guarantees that all writes are to the same range and that no // intents are left in the event of an error. bool require_1pc = 6 [(gogoproto.customname) = "Require1PC"]; + // Set to true if this transaction is serializable isolation but has + // accummulated no refresh spans. This allows the executing server + // to retry it locally on the fast path. + bool no_refresh_spans = 7; } // An EndTransactionResponse is the return value from the @@ -1371,8 +1375,8 @@ message BatchResponse { // error is non-nil if an error occurred. Error error = 1; // timestamp is set only for non-transactional responses and denotes the - // highest timestamp at which a command from the batch executed. At the - // time of writing, it is used solely for informational purposes and tests. + // timestamp at which the batch executed. The timestamp cache is updated + // at this timestamp. util.hlc.Timestamp Timestamp = 2 [(gogoproto.nullable) = false]; // txn is non-nil if the request specified a non-nil // transaction. The transaction timestamp and/or priority may have diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 3ae60cafe548..1fc5f90e6abe 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -956,6 +956,12 @@ func (t *Transaction) UpgradePriority(minPriority int32) { } } +// IsSerializable returns whether this transaction uses serializable +// isolation. +func (t *Transaction) IsSerializable() bool { + return t != nil && t.Isolation == enginepb.SERIALIZABLE +} + // String formats transaction into human readable string. func (t Transaction) String() string { var buf bytes.Buffer @@ -1096,7 +1102,7 @@ func CanTransactionRetryAtRefreshedTimestamp( ctx context.Context, pErr *Error, ) (bool, *Transaction) { txn := pErr.GetTxn() - if txn == nil || txn.Isolation != enginepb.SERIALIZABLE { + if !txn.IsSerializable() { return false, nil } timestamp := txn.Timestamp @@ -1136,7 +1142,7 @@ func readWithinUncertaintyIntervalRetryTimestamp( origin, err, txn, txn.ObservedTimestamps) } // Also forward by the existing timestamp. - ts.Forward(err.ExistingTimestamp) + ts.Forward(err.ExistingTimestamp.Next()) return ts } diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 4922437c6884..1bd786ef68dd 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -1995,7 +1995,13 @@ func (r *Replica) updateTimestampCache( } tc := r.store.tsCache + // Update the timestamp cache using the timestamp at which the batch + // was executed. Note this may have moved forward from ba.Timestamp, + // as when the request is retried locally on WriteTooOldErrors. ts := ba.Timestamp + if br != nil { + ts = br.Timestamp + } var txnID uuid.UUID if ba.Txn != nil { txnID = ba.Txn.ID @@ -5041,7 +5047,7 @@ func (r *Replica) evaluateProposalInner( var pErr *roachpb.Error var ms enginepb.MVCCStats var br *roachpb.BatchResponse - batch, ms, br, result, pErr = r.evaluateTxnWriteBatch(ctx, idKey, ba, spans) + batch, ms, br, result, pErr = r.evaluateWriteBatch(ctx, idKey, ba, spans) if r.store.cfg.Settings.Version.IsActive(cluster.VersionMVCCNetworkStats) { result.Replicated.Delta = ms.ToNetworkStats() } else { @@ -5112,18 +5118,17 @@ func checkIfTxnAborted( return nil } -// evaluateTxnWriteBatch attempts to execute transactional batches on -// the 1-phase-commit path as just an atomic, non-transactional batch -// of write commands. One phase commit batches contain transactional -// writes sandwiched by BeginTransaction and EndTransaction requests. +// evaluateWriteBatch evaluates the supplied batch. // -// If the batch is transactional, and there's nothing to suggest that -// the transaction will require retry or restart, the batch's txn is -// stripped and it's executed as a normal batch write. If the writes -// cannot all be completed at the intended timestamp, the batch's txn -// is restored and it's re-executed as transactional. This allows it -// to lay down intents and return an appropriate retryable error. -func (r *Replica) evaluateTxnWriteBatch( +// If the batch is transactional and has all the hallmarks of a 1PC +// commit (i.e. includes BeginTransaction & EndTransaction, and +// there's nothing to suggest that the transaction will require retry +// or restart), the batch's txn is stripped and it's executed as an +// atomic batch write. If the writes cannot all be completed at the +// intended timestamp, the batch's txn is restored and it's +// re-executed in full. This allows it to lay down intents and return +// an appropriate retryable error. +func (r *Replica) evaluateWriteBatch( ctx context.Context, idKey storagebase.CmdIDKey, ba roachpb.BatchRequest, spans *spanset.SpanSet, ) (engine.Batch, enginepb.MVCCStats, *roachpb.BatchResponse, result.Result, *roachpb.Error) { ms := enginepb.MVCCStats{} @@ -5140,17 +5145,24 @@ func (r *Replica) evaluateTxnWriteBatch( strippedBa.Txn = nil strippedBa.Requests = ba.Requests[1 : len(ba.Requests)-1] // strip begin/end txn reqs + // If there were no refreshable spans earlier in a serializable + // txn (e.g. earlier gets or scans), then the batch can be retried + // locally in the event of write too old errors. + retryLocally := ba.Txn.IsSerializable() && etArg.NoRefreshSpans + // If all writes occurred at the intended timestamp, we've succeeded on the fast path. - batch := r.store.Engine().NewBatch() - if util.RaceEnabled && spans != nil { - batch = spanset.NewBatch(batch, spans) - } rec := NewReplicaEvalContext(r, spans) - br, res, pErr := evaluateBatch(ctx, idKey, batch, rec, &ms, strippedBa) - if pErr == nil && ba.Timestamp == br.Timestamp { + batch, br, res, pErr := r.evaluateWriteBatchWithLocalRetries( + ctx, idKey, rec, &ms, strippedBa, spans, retryLocally, + ) + if pErr == nil && (ba.Timestamp == br.Timestamp || retryLocally) { clonedTxn := ba.Txn.Clone() clonedTxn.Writing = true clonedTxn.Status = roachpb.COMMITTED + // Make sure the returned txn has the actual commit + // timestamp. This can be different if the stripped batch was + // executed at the server's hlc now timestamp. + clonedTxn.Timestamp = br.Timestamp // If the end transaction is not committed, clear the batch and mark the status aborted. if !etArg.Commit { @@ -5193,13 +5205,47 @@ func (r *Replica) evaluateTxnWriteBatch( log.VEventf(ctx, 2, "1PC execution failed, reverting to regular execution for batch") } - batch := r.store.Engine().NewBatch() - if util.RaceEnabled && spans != nil { - batch = spanset.NewBatch(batch, spans) - } rec := NewReplicaEvalContext(r, spans) - br, result, pErr := evaluateBatch(ctx, idKey, batch, rec, &ms, ba) - return batch, ms, br, result, pErr + // We can retry locally if this is a non-transactional request. + canRetry := ba.Txn == nil + batch, br, res, pErr := r.evaluateWriteBatchWithLocalRetries(ctx, idKey, rec, &ms, ba, spans, canRetry) + return batch, ms, br, res, pErr +} + +// evaluateWriteBatchWithLocalRetries invokes evaluateBatch and +// retries in the event of a WriteTooOldError at a higher timestamp if +// canRetry is true. +func (r *Replica) evaluateWriteBatchWithLocalRetries( + ctx context.Context, + idKey storagebase.CmdIDKey, + rec batcheval.EvalContext, + ms *enginepb.MVCCStats, + ba roachpb.BatchRequest, + spans *spanset.SpanSet, + canRetry bool, +) (batch engine.Batch, br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { + for retries := 0; ; retries++ { + if batch != nil { + batch.Close() + } + batch = r.store.Engine().NewBatch() + if util.RaceEnabled && spans != nil { + batch = spanset.NewBatch(batch, spans) + } + br, res, pErr = evaluateBatch(ctx, idKey, batch, rec, ms, ba) + // If we can retry, set a higher batch timestamp and continue. + if wtoErr, ok := pErr.GetDetail().(*roachpb.WriteTooOldError); ok && canRetry { + // Allow one retry only; a non-txn batch containing overlapping + // spans will always experience WriteTooOldError. + if retries == 1 { + break + } + ba.Timestamp = wtoErr.ActualTimestamp + continue + } + break + } + return } // isOnePhaseCommit returns true iff the BatchRequest contains all commands in @@ -5213,9 +5259,6 @@ func isOnePhaseCommit(ba roachpb.BatchRequest, knobs *StoreTestingKnobs) bool { if ba.Txn == nil { return false } - if retry, _ := isEndTransactionTriggeringRetryError(ba.Txn); retry { - return false - } if _, hasBegin := ba.GetArg(roachpb.BeginTransaction); !hasBegin { return false } @@ -5227,6 +5270,16 @@ func isOnePhaseCommit(ba roachpb.BatchRequest, knobs *StoreTestingKnobs) bool { if isEndTransactionExceedingDeadline(ba.Header.Timestamp, *etArg) { return false } + if retry, reason := isEndTransactionTriggeringRetryError(ba.Txn); retry { + // We can still attempt the one phase commit on serializable + // retries (caused by our timestamp getting pushed forward due to + // the timestamp cache) if the end transaction request specifies + // there are no refresh spans. + canIgnoreSerializableRetry := ba.Txn.IsSerializable() && etArg.NoRefreshSpans + if reason != roachpb.RETRY_SERIALIZABLE || !canIgnoreSerializableRetry { + return false + } + } return !knobs.DisableOptional1PC || etArg.Require1PC } @@ -5387,6 +5440,9 @@ func evaluateBatch( } var result result.Result + var writeTooOldErr *roachpb.Error + returnWriteTooOldErr := true + for index, union := range ba.Requests { // Execute the command. args := union.GetInner() @@ -5414,16 +5470,32 @@ func evaluateBatch( switch tErr := pErr.GetDetail().(type) { case *roachpb.WriteTooOldError: - // We got a WriteTooOldError. In case this is a transactional request, - // we want to run the other commands in the batch and let them lay - // intents so that other concurrent overlapping transactions are forced - // through intent resolution and the chances of this batch succeeding - // when it will be retried are increased. - if ba.Txn == nil { - return nil, result, pErr + // We got a WriteTooOldError. We continue on to run all + // commands in the batch in order to determine the highest + // timestamp for more efficient retries. If the batch is + // transactional, we continue to lay down intents so that + // other concurrent overlapping transactions are forced + // through intent resolution and the chances of this batch + // succeeding when it will be retried are increased. + if writeTooOldErr != nil { + writeTooOldErr.GetDetail().(*roachpb.WriteTooOldError).ActualTimestamp.Forward(tErr.ActualTimestamp) + } else { + writeTooOldErr = pErr + // For transaction, we want to swallow the write too old error + // and just move the transaction timestamp forward and set the + // WriteTooOld flag. See below for exceptions. + if ba.Txn != nil { + returnWriteTooOldErr = false + } } - // Return error early on some requests for serializable isolation. - if ba.Txn != nil && ba.Txn.Isolation == enginepb.SERIALIZABLE { + // Set the flag to return a WriteTooOldError with the max timestamp + // encountered evaluating the entire batch on cput and inc requests, + // with serializable isolation. Because both of these requests must + // have their keys refreshed on commit with Transaction.WriteTooOld + // is true, and that refresh will fail, we'd be otherwise guaranteed + // to do a client-side retry. Returning an error allows a + // txn-coord-side retry. + if ba.Txn.IsSerializable() { switch args.(type) { case *roachpb.ConditionalPutRequest: // Conditional puts are an exception. Here, it makes less sense to @@ -5434,23 +5506,22 @@ func evaluateBatch( // already during the transaction, and then, if the cput results in a // condition failed error, report that back to the client instead of a // retryable error. - return nil, result, pErr + returnWriteTooOldErr = true case *roachpb.IncrementRequest: // Increments are an exception for similar reasons. If we wait until // commit, we'll need a client-side retry, so we return immediately // to see if we can do a txn coord sender retry instead. - return nil, result, pErr + returnWriteTooOldErr = true } } - // On WriteTooOldError, we've written a new value or an intent - // at a too-high timestamp and we must forward the batch txn or - // timestamp as appropriate so that it's returned. - ba.Txn.Timestamp.Forward(tErr.ActualTimestamp) - ba.Txn.WriteTooOld = true - // Clear the WriteTooOldError; we're done processing it by having - // moved the batch or txn timestamps forward and set WriteTooOld - // if this is a transactional write. The EndTransaction will detect this - // pushed timestamp and return a TransactionRetryError. + if ba.Txn != nil { + ba.Txn.Timestamp.Forward(tErr.ActualTimestamp) + ba.Txn.WriteTooOld = true + } + // Clear pErr; we're done processing it by having moved the + // batch or txn timestamps forward and set WriteTooOld if this + // is a transactional write. The EndTransaction will detect + // this pushed timestamp and return a TransactionRetryError. pErr = nil default: return nil, result, pErr @@ -5476,13 +5547,19 @@ func evaluateBatch( } } + // If there's a write too old error, return now that we've found + // the high water timestamp for retries. + if writeTooOldErr != nil && returnWriteTooOldErr { + return nil, result, writeTooOldErr + } + if ba.Txn != nil { // If transactional, send out the final transaction entry with the reply. br.Txn = ba.Txn - } else { - // When non-transactional, use the timestamp field. - br.Timestamp.Forward(ba.Timestamp) } + // Always update the batch response timestamp field to the timestamp at + // which the batch executed. + br.Timestamp.Forward(ba.Timestamp) return br, result, nil } diff --git a/pkg/storage/replica_test.go b/pkg/storage/replica_test.go index a34885bd4d30..1fe217ffd7df 100644 --- a/pkg/storage/replica_test.go +++ b/pkg/storage/replica_test.go @@ -8362,7 +8362,7 @@ func TestReplicaEvaluationNotTxnMutation(t *testing.T) { ba.Add(&txnPut) ba.Add(&txnPut) - batch, _, _, _, pErr := tc.repl.evaluateTxnWriteBatch(ctx, makeIDKey(), ba, &allSpans) + batch, _, _, _, pErr := tc.repl.evaluateWriteBatch(ctx, makeIDKey(), ba, &allSpans) defer batch.Close() if pErr != nil { t.Fatal(pErr) @@ -9062,3 +9062,204 @@ func TestReplicaRecomputeStats(t *testing.T) { runTest(testCase{"noopagain", sKey, enginepb.MVCCStats{}, ""}) } } + +// TestReplicaLocalRetries verifies local retry logic for transactional +// and non transactional batches. Verifies the timestamp cache is updated +// to reflect the timestamp at which retried batches are executed. +func TestReplicaLocalRetries(t *testing.T) { + defer leaktest.AfterTest(t)() + tc := testContext{} + stopper := stop.NewStopper() + defer stopper.Stop(context.TODO()) + tc.Start(t, stopper) + + newTxn := func(key string, ts hlc.Timestamp) *roachpb.Transaction { + txn := roachpb.MakeTransaction( + "test", roachpb.Key(key), roachpb.NormalUserPriority, enginepb.SERIALIZABLE, ts, 0, + ) + return &txn + } + send := func(ba roachpb.BatchRequest) (hlc.Timestamp, error) { + br, pErr := tc.Sender().Send(context.Background(), ba) + if pErr != nil { + return hlc.Timestamp{}, pErr.GetDetail() + } + return br.Timestamp, nil + } + put := func(key, val string) (hlc.Timestamp, error) { + var ba roachpb.BatchRequest + put := putArgs(roachpb.Key(key), []byte(val)) + ba.Add(&put) + return send(ba) + } + + testCases := []struct { + name string + setupFn func() (hlc.Timestamp, error) // returns expected batch execution timestamp + batchFn func(hlc.Timestamp) (roachpb.BatchRequest, hlc.Timestamp) + expErr string + expTSCUpdateKeys []string + }{ + { + name: "local retry of write too old on put", + setupFn: func() (hlc.Timestamp, error) { + return put("a", "put") + }, + batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) { + ba.Timestamp = ts.Prev() + expTS = ts.Next() + put := putArgs(roachpb.Key("a"), []byte("put2")) + ba.Add(&put) + return + }, + }, + { + name: "local retry of write too old on cput", + setupFn: func() (hlc.Timestamp, error) { + // Note there are two different version of the value, but a + // non-txnal cput will evaluate the most recent version and + // avoid a condition failed error. + _, _ = put("b", "put1") + return put("b", "put2") + }, + batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) { + ba.Timestamp = ts.Prev() + expTS = ts.Next() + cput := cPutArgs(roachpb.Key("b"), []byte("cput"), []byte("put2")) + ba.Add(&cput) + return + }, + expTSCUpdateKeys: []string{"b"}, + }, + // Non-1PC serializable txn cput will fail with write too old error. + { + name: "no local retry of write too old on non-1PC txn", + setupFn: func() (hlc.Timestamp, error) { + _, _ = put("c", "put") + return put("c", "put") + }, + batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) { + ba.Txn = newTxn("c", ts.Prev()) + cput := cPutArgs(roachpb.Key("c"), []byte("cput"), []byte("put")) + ba.Add(&cput) + return + }, + expErr: "write at timestamp .* too old", + }, + // 1PC serializable transaction will fail instead of retrying if + // EndTransactionRequest.NoRefreshSpans is not true. + { + name: "no local retry of write too old on 1PC txn and refresh spans", + setupFn: func() (hlc.Timestamp, error) { + _, _ = put("d", "put") + return put("d", "put") + }, + batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) { + ba.Txn = newTxn("d", ts.Prev()) + bt, _ := beginTxnArgs(ba.Txn.Key, ba.Txn) + ba.Add(&bt) + cput := cPutArgs(ba.Txn.Key, []byte("cput"), []byte("put")) + ba.Add(&cput) + et, _ := endTxnArgs(ba.Txn, true /* commit */) + ba.Add(&et) + return + }, + expErr: "RETRY_WRITE_TOO_OLD", + }, + // 1PC serializable transaction will retry locally. + { + name: "local retry of write too old on 1PC txn", + setupFn: func() (hlc.Timestamp, error) { + _, _ = put("e", "put") + return put("e", "put") + }, + batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) { + ba.Txn = newTxn("e", ts.Prev()) + expTS = ts.Next() + bt, _ := beginTxnArgs(ba.Txn.Key, ba.Txn) + ba.Add(&bt) + cput := cPutArgs(ba.Txn.Key, []byte("cput"), []byte("put")) + ba.Add(&cput) + et, _ := endTxnArgs(ba.Txn, true /* commit */) + et.NoRefreshSpans = true // necessary to indicate local retry is possible + ba.Add(&et) + return + }, + expTSCUpdateKeys: []string{"e"}, + }, + // Handle multiple write too old errors. + { + name: "local retry with multiple write too old errors", + setupFn: func() (hlc.Timestamp, error) { + if _, err := put("f1", "put"); err != nil { + return hlc.Timestamp{}, err + } + if _, err := put("f2", "put"); err != nil { + return hlc.Timestamp{}, err + } + return put("f3", "put") + }, + batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) { + ba.Timestamp = ts.Prev() + expTS = ts.Next() + for i := 1; i <= 3; i++ { + cput := cPutArgs(roachpb.Key(fmt.Sprintf("f%d", i)), []byte("cput"), []byte("put")) + ba.Add(&cput) + } + return + }, + expTSCUpdateKeys: []string{"f1", "f2", "f3"}, + }, + // Handle multiple write too old errors in 1PC transaction. + { + name: "local retry with multiple write too old errors", + setupFn: func() (hlc.Timestamp, error) { + if _, err := put("g1", "put"); err != nil { + return hlc.Timestamp{}, err + } + if _, err := put("g2", "put"); err != nil { + return hlc.Timestamp{}, err + } + return put("g3", "put") + }, + batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) { + ba.Txn = newTxn("g1", ts.Prev()) + expTS = ts.Next() + bt, _ := beginTxnArgs(ba.Txn.Key, ba.Txn) + ba.Add(&bt) + for i := 1; i <= 3; i++ { + cput := cPutArgs(roachpb.Key(fmt.Sprintf("g%d", i)), []byte("cput"), []byte("put")) + ba.Add(&cput) + } + et, _ := endTxnArgs(ba.Txn, true /* commit */) + et.NoRefreshSpans = true // necessary to indicate local retry is possible + ba.Add(&et) + return + }, + expTSCUpdateKeys: []string{"g1", "g2", "g3"}, + }, + } + + for _, test := range testCases { + t.Run(test.name, func(t *testing.T) { + ts, err := test.setupFn() + if err != nil { + t.Fatal(err) + } + ba, expTS := test.batchFn(ts) + actualTS, err := send(ba) + if !testutils.IsError(err, test.expErr) { + t.Fatalf("expected error %q; got \"%v\"", test.expErr, err) + } + if actualTS != expTS { + t.Fatalf("expected ts=%s; got %s", expTS, actualTS) + } + for _, k := range test.expTSCUpdateKeys { + rTS, _ := tc.repl.store.tsCache.GetMaxRead(roachpb.Key(k), nil) + if rTS != expTS { + t.Fatalf("expected timestamp cache update for %s to %s; got %s", k, expTS, rTS) + } + } + }) + } +}