From c3c4c667824a8af27c13db595b78b4dd3e4893cd Mon Sep 17 00:00:00 2001 From: Sam Huang Date: Wed, 4 Nov 2020 16:17:35 -0500 Subject: [PATCH] server: allow healing ranges whose replicas are all lost Introduces and implements an `ResetQuorumRequest` RPC. `ResetQuorumRequest` takes in the range id of a range that that is unavailable due to lost quorum and makes it available again, at the cost of losing all of the data in that range. Any existing replica, even one residing on the target node, will irrevocably be removed. ResetQuorumRequest first uses meta2 to identify the range descriptor. Then, it removes all replicas from the range descriptor and adds a store from the target node as the one designated survivor replica. This change is then written to meta2 and sent as a snapshot to a store local to the target node in order to use crdb internal upreplication and rebalancing mechanisms to create further replicas from this fresh snapshot. This RPC is meant to be called by the user directly. It will not work for ranges that have not lost quorum or for a meta range. Release note: None --- pkg/ccl/kvccl/kvtenantccl/connector_test.go | 6 + pkg/kv/kvclient/kvcoord/send_test.go | 6 + pkg/kv/kvclient/kvcoord/transport_test.go | 6 + pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/reset_quorum_test.go | 156 ++ pkg/kv/kvserver/store_raft.go | 18 + pkg/kv/kvserver/store_snapshot.go | 136 ++ pkg/roachpb/api.pb.go | 1579 +++++++++++-------- pkg/roachpb/api.proto | 19 + pkg/rpc/context.go | 6 + pkg/rpc/context_test.go | 6 + pkg/server/node.go | 101 ++ 12 files changed, 1398 insertions(+), 642 deletions(-) create mode 100644 pkg/kv/kvserver/reset_quorum_test.go diff --git a/pkg/ccl/kvccl/kvtenantccl/connector_test.go b/pkg/ccl/kvccl/kvtenantccl/connector_test.go index aec146c2459f..d1d3485d92aa 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector_test.go @@ -58,6 +58,12 @@ func (m *mockServer) GossipSubscription( return m.gossipSubFn(req, stream) } +func (*mockServer) ResetQuorum( + context.Context, *roachpb.ResetQuorumRequest, +) (*roachpb.ResetQuorumResponse, error) { + panic("unimplemented") +} + func (*mockServer) Batch(context.Context, *roachpb.BatchRequest) (*roachpb.BatchResponse, error) { panic("unimplemented") } diff --git a/pkg/kv/kvclient/kvcoord/send_test.go b/pkg/kv/kvclient/kvcoord/send_test.go index 724d259fe82c..3bde66fa0fab 100644 --- a/pkg/kv/kvclient/kvcoord/send_test.go +++ b/pkg/kv/kvclient/kvcoord/send_test.go @@ -67,6 +67,12 @@ func (n Node) Join(context.Context, *roachpb.JoinNodeRequest) (*roachpb.JoinNode panic("unimplemented") } +func (n Node) ResetQuorum( + context.Context, *roachpb.ResetQuorumRequest, +) (*roachpb.ResetQuorumResponse, error) { + panic("unimplemented") +} + // TestSendToOneClient verifies that Send correctly sends a request // to one server using the heartbeat RPC. func TestSendToOneClient(t *testing.T) { diff --git a/pkg/kv/kvclient/kvcoord/transport_test.go b/pkg/kv/kvclient/kvcoord/transport_test.go index ff049afdaa25..41ee621c079f 100644 --- a/pkg/kv/kvclient/kvcoord/transport_test.go +++ b/pkg/kv/kvclient/kvcoord/transport_test.go @@ -140,6 +140,12 @@ type mockInternalClient struct { var _ roachpb.InternalClient = &mockInternalClient{} +func (*mockInternalClient) ResetQuorum( + context.Context, *roachpb.ResetQuorumRequest, ...grpc.CallOption, +) (*roachpb.ResetQuorumResponse, error) { + panic("unimplemented") +} + // Batch is part of the roachpb.InternalClient interface. func (m *mockInternalClient) Batch( ctx context.Context, in *roachpb.BatchRequest, opts ...grpc.CallOption, diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 0a1be25eb498..b13e9e1d94da 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -257,6 +257,7 @@ go_test( "replica_test.go", "replicate_queue_test.go", "replicate_test.go", + "reset_quorum_test.go", "scanner_test.go", "scheduler_test.go", "single_key_test.go", diff --git a/pkg/kv/kvserver/reset_quorum_test.go b/pkg/kv/kvserver/reset_quorum_test.go new file mode 100644 index 000000000000..4396faa68712 --- /dev/null +++ b/pkg/kv/kvserver/reset_quorum_test.go @@ -0,0 +1,156 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package kvserver_test + +import ( + "context" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +// TestResetQuorum tests the ResetQuorum method to restore quorum for a +// range whose replicas are all lost. It starts a cluster with two nodes, +// n1 and n2, with a range isolated to n2. Then, it stops n2 and checks +// that the range is unavailable. Finally, it uses ResetQuorum to make the +// range accessible again and checks that the range is available again. +// +// TODO(thesamhuang): Add additional testing to cover two cases: +// 1. if there's an existing replica and we send the recovery snapshot to +// that replica, we still get the quorum reset and end up with an empty +// range. +// 2. if there's an existing replica and we send the recovery snapshot to +// another node that doesn't have that replica, we also still get the +// reset, and replicaGC removes the original survivor when triggered. +func TestResetQuorum(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + + livenessDuration := 3000 * time.Millisecond + + clusterArgs := base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + NodeLiveness: kvserver.NodeLivenessTestingKnobs{ + LivenessDuration: livenessDuration, // set duration so n2 liveness expires shortly after stopping + RenewalDuration: 1500 * time.Millisecond, + }, + }, + }, + } + tc := testcluster.StartTestCluster(t, 2, clusterArgs) + defer tc.Stopper().Stop(ctx) + + n1, n2 := 0, 1 + + // Set up a scratch range isolated to n2. + k := tc.ScratchRange(t) + desc, err := tc.AddVoters(k, tc.Target(n2)) + require.NoError(t, err) + require.NoError(t, tc.TransferRangeLease(desc, tc.Target(n2))) + desc, err = tc.RemoveVoters(k, tc.Target(n1)) + require.NoError(t, err) + require.Len(t, desc.Replicas().All(), 1) + + srv := tc.Server(n1) + + require.NoError(t, srv.DB().Put(ctx, k, "bar")) + + metaKey := keys.RangeMetaKey(desc.EndKey).AsRawKey() + // Read the meta2 which ResetQuorum will be updating while n2 + // is still available. This avoids an intent from the previous + // replication change to linger; such an intent would be anchored + // on the scratch range, which is unavailable when n2 goes down. + _, err = srv.DB().Get(ctx, metaKey) + require.NoError(t, err) + tc.StopServer(n2) + // Wait for n2 liveness to expire. + time.Sleep(livenessDuration) + + // Sanity check that requests to the ScratchRange fail. + func() { + cCtx, cancel := context.WithTimeout(ctx, 50*time.Millisecond) + defer cancel() + err := srv.DB().Put(cCtx, k, "baz") + // NB: we don't assert on the exact error since our RPC layer + // tries to return a better error than DeadlineExceeded (at + // the time of writing, we get a connection failure to n2), + // and we don't wait for the context to even time out. + // We're probably checking in the RPC layer whether we've + // retried with an up-to-date desc and fail fast if so. + require.Error(t, err) + }() + + // Get the store on the designated survivor n1. + var store *kvserver.Store + require.NoError(t, srv.GetStores().(*kvserver.Stores).VisitStores(func(inner *kvserver.Store) error { + store = inner + return nil + })) + if store == nil { + t.Fatal("no store found on n1") + } + + // Call ResetQuorum to reset quorum on the unhealthy range. + t.Logf("resetting quorum on node id: %v, store id: %v", store.NodeID(), store.StoreID()) + _, err = srv.Node().(*server.Node).ResetQuorum( + ctx, + &roachpb.ResetQuorumRequest{ + RangeID: int32(desc.RangeID), + }, + ) + require.NoError(t, err) + + t.Logf("resetting quorum complete") + + require.NoError(t, srv.DB().Put(ctx, k, "baz")) + + // Get range descriptor from meta2 and verify it was updated correctly. + var updatedDesc roachpb.RangeDescriptor + require.NoError(t, store.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + kvs, err := sql.ScanMetaKVs(ctx, txn, roachpb.Span{ + Key: roachpb.KeyMin, + EndKey: roachpb.KeyMax, + }) + if err != nil { + return err + } + + for i := range kvs { + if err := kvs[i].Value.GetProto(&updatedDesc); err != nil { + return err + } + if updatedDesc.RangeID == desc.RangeID { + return nil + } + } + return errors.Errorf("range id %v not found after resetting quorum", desc.RangeID) + })) + if len(updatedDesc.Replicas().All()) != 1 { + t.Fatalf("found %v replicas found after resetting quorum, expected 1", len(updatedDesc.Replicas().All())) + } + if updatedDesc.Replicas().All()[0].NodeID != srv.NodeID() { + t.Fatalf("replica found after resetting quorum is on node id %v, expected node id %v", updatedDesc.Replicas().All()[0].NodeID, srv.NodeID()) + } + +} diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index 7bb88c0e73cc..bf4b4e4a3005 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -331,12 +331,30 @@ func (s *Store) processRaftSnapshotRequest( } }() } + + if snapHeader.RaftMessageRequest.Message.From == snapHeader.RaftMessageRequest.Message.To { + // This is a special case exercised during recovery from loss of quorum. + // In this case, a forged snapshot will be sent to the replica and will + // hit this code path (if we make up a non-existent follower, Raft will + // drop the message, hence we are forced to make the receiver the sender). + // + // Unfortunately, at the time of writing, Raft assumes that a snapshot + // is always received from the leader (of the given term), which plays + // poorly with these forged snapshots. However, a zero sender works just + // fine as the value zero represents "no known leader". + // + // We prefer not to introduce a zero origin of the message as throughout + // our code we rely on it being present. Instead, we reset the origin + // that raft looks at just before handing the message off. + snapHeader.RaftMessageRequest.Message.From = 0 + } // NB: we cannot get errRemoved here because we're promised by // withReplicaForRequest that this replica is not currently being removed // and we've been holding the raftMu the entire time. if err := r.stepRaftGroup(&snapHeader.RaftMessageRequest); err != nil { return roachpb.NewError(err) } + _, expl, err := r.handleRaftReadyRaftMuLocked(ctx, inSnap) maybeFatalOnRaftReadyErr(ctx, expl, err) removePlaceholder = false diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 10b6caa03eb4..4d93249e8d2e 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -16,12 +16,17 @@ import ( "io" "time" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftentry" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "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" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -30,6 +35,7 @@ import ( "github.com/cockroachdb/errors" "go.etcd.io/etcd/raft/v3/raftpb" "golang.org/x/time/rate" + "google.golang.org/grpc" ) const ( @@ -898,6 +904,136 @@ func (e *errMustRetrySnapshotDueToTruncation) Error() string { ) } +// SendEmptySnapshot creates an OutgoingSnapshot for the input range +// descriptor and seeds it with an empty range. Then, it sends this +// snapshot to the replica specified in the input. +func SendEmptySnapshot( + ctx context.Context, + st *cluster.Settings, + cc *grpc.ClientConn, + now hlc.Timestamp, + desc roachpb.RangeDescriptor, + to roachpb.ReplicaDescriptor, +) error { + // Create an engine to use as a buffer for the empty snapshot. + eng := storage.NewDefaultInMem() + defer eng.Close() + + var ms enginepb.MVCCStats + // Seed an empty range into the new engine. + if err := storage.MVCCPutProto( + ctx, eng, &ms, keys.RangeDescriptorKey(desc.StartKey), now, nil /* txn */, &desc, + ); err != nil { + return err + } + ms, err := stateloader.WriteInitialReplicaState( + ctx, + eng, + ms, + desc, + roachpb.Lease{}, + hlc.Timestamp{}, // gcThreshold + stateloader.TruncatedStateUnreplicated, + ) + if err != nil { + return err + } + + // Use stateloader to load state out of memory from the previously created engine. + sl := stateloader.Make(desc.RangeID) + state, err := sl.Load(ctx, eng, &desc) + if err != nil { + return err + } + hs, err := sl.LoadHardState(ctx, eng) + if err != nil { + return err + } + + snapUUID, err := uuid.NewV4() + if err != nil { + return err + } + + // Create an OutgoingSnapshot to send. + outgoingSnap, err := snapshot( + ctx, + snapUUID, + sl, + // TODO(tbg): We may want a separate SnapshotRequest type + // for recovery that always goes through by bypassing all throttling + // so they cannot be declined. We don't want our operation to be held + // up behind a long running snapshot. We want this to go through + // quickly. + SnapshotRequest_VIA_SNAPSHOT_QUEUE, + eng, + desc.RangeID, + raftentry.NewCache(1), // cache is not used + func(func(SideloadStorage) error) error { return nil }, // this is used for sstables, not needed here as there are no logs + desc.StartKey, + ) + if err != nil { + return err + } + defer outgoingSnap.Close() + + // From and to replica descriptors are the same because we have + // to send the snapshot from a member of the range descriptor. + // Sending it from the current replica ensures that. Otherwise, + // it would be a malformed request if it came from a non-member. + from := to + req := RaftMessageRequest{ + RangeID: desc.RangeID, + FromReplica: from, + ToReplica: to, + Message: raftpb.Message{ + Type: raftpb.MsgSnap, + To: uint64(to.ReplicaID), + From: uint64(from.ReplicaID), + Term: hs.Term, + Snapshot: outgoingSnap.RaftSnap, + }, + } + + header := SnapshotRequest_Header{ + State: state, + RaftMessageRequest: req, + RangeSize: ms.Total(), + CanDecline: false, + Priority: SnapshotRequest_RECOVERY, + Strategy: SnapshotRequest_KV_BATCH, + Type: SnapshotRequest_VIA_SNAPSHOT_QUEUE, + UnreplicatedTruncatedState: true, + } + + stream, err := NewMultiRaftClient(cc).RaftSnapshot(ctx) + if err != nil { + return err + } + + defer func() { + if err := stream.CloseSend(); err != nil { + log.Warningf(ctx, "failed to close snapshot stream: %+v", err) + } + }() + + return sendSnapshot( + ctx, + st, + stream, + noopStorePool{}, + header, + &outgoingSnap, + eng.NewBatch, + func() {}, + ) +} + +// noopStorePool is a hollowed out StorePool that does not throttle. It's used in recovery scenarios. +type noopStorePool struct{} + +func (n noopStorePool) throttle(throttleReason, string, roachpb.StoreID) {} + // sendSnapshot sends an outgoing snapshot via a pre-opened GRPC stream. func sendSnapshot( ctx context.Context, diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 5f308d9932b3..8806b751f919 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -73,7 +73,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{0} + return fileDescriptor_api_290a2f113c3e6c17, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -101,7 +101,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{1} + return fileDescriptor_api_290a2f113c3e6c17, []int{1} } type ChecksumMode int32 @@ -148,7 +148,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{2} + return fileDescriptor_api_290a2f113c3e6c17, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -179,7 +179,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{3} + return fileDescriptor_api_290a2f113c3e6c17, []int{3} } type ExternalStorageProvider int32 @@ -220,7 +220,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{4} + return fileDescriptor_api_290a2f113c3e6c17, []int{4} } type MVCCFilter int32 @@ -243,7 +243,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{5} + return fileDescriptor_api_290a2f113c3e6c17, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +269,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{1, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +311,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{25, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -332,7 +332,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_92f269cadb1337bc, []int{0} + return fileDescriptor_api_290a2f113c3e6c17, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -403,7 +403,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_92f269cadb1337bc, []int{1} + return fileDescriptor_api_290a2f113c3e6c17, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +437,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_92f269cadb1337bc, []int{2} + return fileDescriptor_api_290a2f113c3e6c17, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +480,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_92f269cadb1337bc, []int{3} + return fileDescriptor_api_290a2f113c3e6c17, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,7 +523,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_92f269cadb1337bc, []int{4} + return fileDescriptor_api_290a2f113c3e6c17, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -557,7 +557,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_92f269cadb1337bc, []int{5} + return fileDescriptor_api_290a2f113c3e6c17, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -639,7 +639,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_92f269cadb1337bc, []int{6} + return fileDescriptor_api_290a2f113c3e6c17, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -674,7 +674,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_92f269cadb1337bc, []int{7} + return fileDescriptor_api_290a2f113c3e6c17, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +720,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_92f269cadb1337bc, []int{8} + return fileDescriptor_api_290a2f113c3e6c17, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +754,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_92f269cadb1337bc, []int{9} + return fileDescriptor_api_290a2f113c3e6c17, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -794,7 +794,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_92f269cadb1337bc, []int{10} + return fileDescriptor_api_290a2f113c3e6c17, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -831,7 +831,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_92f269cadb1337bc, []int{11} + return fileDescriptor_api_290a2f113c3e6c17, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -865,7 +865,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_92f269cadb1337bc, []int{12} + return fileDescriptor_api_290a2f113c3e6c17, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -899,7 +899,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_92f269cadb1337bc, []int{13} + return fileDescriptor_api_290a2f113c3e6c17, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -951,7 +951,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_92f269cadb1337bc, []int{14} + return fileDescriptor_api_290a2f113c3e6c17, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -988,7 +988,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_92f269cadb1337bc, []int{15} + return fileDescriptor_api_290a2f113c3e6c17, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1043,7 +1043,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_92f269cadb1337bc, []int{16} + return fileDescriptor_api_290a2f113c3e6c17, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1077,7 +1077,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_92f269cadb1337bc, []int{17} + return fileDescriptor_api_290a2f113c3e6c17, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1118,7 +1118,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_92f269cadb1337bc, []int{18} + return fileDescriptor_api_290a2f113c3e6c17, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1152,7 +1152,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_92f269cadb1337bc, []int{19} + return fileDescriptor_api_290a2f113c3e6c17, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1203,7 +1203,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{20} + return fileDescriptor_api_290a2f113c3e6c17, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1255,7 +1255,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_92f269cadb1337bc, []int{21} + return fileDescriptor_api_290a2f113c3e6c17, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1306,7 +1306,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_92f269cadb1337bc, []int{22} + return fileDescriptor_api_290a2f113c3e6c17, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1358,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_92f269cadb1337bc, []int{23} + return fileDescriptor_api_290a2f113c3e6c17, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1411,7 +1411,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_92f269cadb1337bc, []int{24} + return fileDescriptor_api_290a2f113c3e6c17, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1448,7 +1448,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_92f269cadb1337bc, []int{25} + return fileDescriptor_api_290a2f113c3e6c17, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1492,7 +1492,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_92f269cadb1337bc, []int{25, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1540,7 +1540,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_92f269cadb1337bc, []int{26} + return fileDescriptor_api_290a2f113c3e6c17, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1576,7 +1576,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_92f269cadb1337bc, []int{27} + return fileDescriptor_api_290a2f113c3e6c17, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1687,7 +1687,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_92f269cadb1337bc, []int{28} + return fileDescriptor_api_290a2f113c3e6c17, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1733,7 +1733,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_92f269cadb1337bc, []int{29} + return fileDescriptor_api_290a2f113c3e6c17, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1794,7 +1794,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_92f269cadb1337bc, []int{30} + return fileDescriptor_api_290a2f113c3e6c17, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1829,7 +1829,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_92f269cadb1337bc, []int{31} + return fileDescriptor_api_290a2f113c3e6c17, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1868,7 +1868,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_92f269cadb1337bc, []int{32} + return fileDescriptor_api_290a2f113c3e6c17, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1903,7 +1903,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_92f269cadb1337bc, []int{33} + return fileDescriptor_api_290a2f113c3e6c17, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1946,7 +1946,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_92f269cadb1337bc, []int{34} + return fileDescriptor_api_290a2f113c3e6c17, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1981,7 +1981,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_92f269cadb1337bc, []int{35} + return fileDescriptor_api_290a2f113c3e6c17, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2019,7 +2019,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_92f269cadb1337bc, []int{36} + return fileDescriptor_api_290a2f113c3e6c17, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2052,7 +2052,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_92f269cadb1337bc, []int{37} + return fileDescriptor_api_290a2f113c3e6c17, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2087,7 +2087,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_92f269cadb1337bc, []int{38} + return fileDescriptor_api_290a2f113c3e6c17, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2145,7 +2145,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_92f269cadb1337bc, []int{39} + return fileDescriptor_api_290a2f113c3e6c17, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2180,7 +2180,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_92f269cadb1337bc, []int{40} + return fileDescriptor_api_290a2f113c3e6c17, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2217,7 +2217,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_92f269cadb1337bc, []int{41} + return fileDescriptor_api_290a2f113c3e6c17, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2250,7 +2250,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_92f269cadb1337bc, []int{42} + return fileDescriptor_api_290a2f113c3e6c17, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2289,7 +2289,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_92f269cadb1337bc, []int{43} + return fileDescriptor_api_290a2f113c3e6c17, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2326,7 +2326,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_92f269cadb1337bc, []int{44} + return fileDescriptor_api_290a2f113c3e6c17, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2364,7 +2364,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_92f269cadb1337bc, []int{45} + return fileDescriptor_api_290a2f113c3e6c17, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2398,7 +2398,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_92f269cadb1337bc, []int{45, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2432,7 +2432,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_92f269cadb1337bc, []int{46} + return fileDescriptor_api_290a2f113c3e6c17, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2501,7 +2501,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_92f269cadb1337bc, []int{47} + return fileDescriptor_api_290a2f113c3e6c17, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2544,7 +2544,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_92f269cadb1337bc, []int{48} + return fileDescriptor_api_290a2f113c3e6c17, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2591,7 +2591,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_92f269cadb1337bc, []int{49} + return fileDescriptor_api_290a2f113c3e6c17, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2627,7 +2627,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_92f269cadb1337bc, []int{50} + return fileDescriptor_api_290a2f113c3e6c17, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2671,7 +2671,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_92f269cadb1337bc, []int{51} + return fileDescriptor_api_290a2f113c3e6c17, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2715,7 +2715,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_92f269cadb1337bc, []int{52} + return fileDescriptor_api_290a2f113c3e6c17, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2775,7 +2775,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_92f269cadb1337bc, []int{53} + return fileDescriptor_api_290a2f113c3e6c17, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2811,7 +2811,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_92f269cadb1337bc, []int{54} + return fileDescriptor_api_290a2f113c3e6c17, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2858,7 +2858,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_92f269cadb1337bc, []int{55} + return fileDescriptor_api_290a2f113c3e6c17, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2893,7 +2893,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_92f269cadb1337bc, []int{56} + return fileDescriptor_api_290a2f113c3e6c17, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2944,7 +2944,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_92f269cadb1337bc, []int{57} + return fileDescriptor_api_290a2f113c3e6c17, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2979,7 +2979,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_92f269cadb1337bc, []int{58} + return fileDescriptor_api_290a2f113c3e6c17, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3016,7 +3016,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_92f269cadb1337bc, []int{59} + return fileDescriptor_api_290a2f113c3e6c17, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3050,7 +3050,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_92f269cadb1337bc, []int{60} + return fileDescriptor_api_290a2f113c3e6c17, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3095,7 +3095,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_92f269cadb1337bc, []int{61} + return fileDescriptor_api_290a2f113c3e6c17, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3129,7 +3129,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_92f269cadb1337bc, []int{62} + return fileDescriptor_api_290a2f113c3e6c17, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3173,7 +3173,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_92f269cadb1337bc, []int{63} + return fileDescriptor_api_290a2f113c3e6c17, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3222,7 +3222,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_92f269cadb1337bc, []int{64} + return fileDescriptor_api_290a2f113c3e6c17, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3259,7 +3259,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_92f269cadb1337bc, []int{65} + return fileDescriptor_api_290a2f113c3e6c17, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3296,7 +3296,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_92f269cadb1337bc, []int{66} + return fileDescriptor_api_290a2f113c3e6c17, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3331,7 +3331,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_92f269cadb1337bc, []int{67} + return fileDescriptor_api_290a2f113c3e6c17, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3386,7 +3386,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_92f269cadb1337bc, []int{68} + return fileDescriptor_api_290a2f113c3e6c17, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3423,7 +3423,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_92f269cadb1337bc, []int{69} + return fileDescriptor_api_290a2f113c3e6c17, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3463,7 +3463,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_92f269cadb1337bc, []int{70} + return fileDescriptor_api_290a2f113c3e6c17, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3497,7 +3497,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_92f269cadb1337bc, []int{70, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3530,7 +3530,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_92f269cadb1337bc, []int{70, 1} + return fileDescriptor_api_290a2f113c3e6c17, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3572,7 +3572,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_92f269cadb1337bc, []int{70, 2} + return fileDescriptor_api_290a2f113c3e6c17, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3611,7 +3611,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_92f269cadb1337bc, []int{70, 3} + return fileDescriptor_api_290a2f113c3e6c17, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3647,7 +3647,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_92f269cadb1337bc, []int{70, 4} + return fileDescriptor_api_290a2f113c3e6c17, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3686,7 +3686,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_92f269cadb1337bc, []int{70, 5} + return fileDescriptor_api_290a2f113c3e6c17, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3728,7 +3728,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{70, 6} + return fileDescriptor_api_290a2f113c3e6c17, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3768,7 +3768,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_92f269cadb1337bc, []int{71} + return fileDescriptor_api_290a2f113c3e6c17, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3802,7 +3802,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_92f269cadb1337bc, []int{72} + return fileDescriptor_api_290a2f113c3e6c17, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3838,7 +3838,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{73} + return fileDescriptor_api_290a2f113c3e6c17, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3908,7 +3908,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_92f269cadb1337bc, []int{74} + return fileDescriptor_api_290a2f113c3e6c17, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3960,7 +3960,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_92f269cadb1337bc, []int{75} + return fileDescriptor_api_290a2f113c3e6c17, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3996,7 +3996,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_92f269cadb1337bc, []int{76} + return fileDescriptor_api_290a2f113c3e6c17, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4036,7 +4036,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_92f269cadb1337bc, []int{76, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4087,7 +4087,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_92f269cadb1337bc, []int{77} + return fileDescriptor_api_290a2f113c3e6c17, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4122,7 +4122,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_92f269cadb1337bc, []int{77, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4158,7 +4158,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_92f269cadb1337bc, []int{77, 1} + return fileDescriptor_api_290a2f113c3e6c17, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4193,7 +4193,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_92f269cadb1337bc, []int{78} + return fileDescriptor_api_290a2f113c3e6c17, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4231,7 +4231,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_92f269cadb1337bc, []int{79} + return fileDescriptor_api_290a2f113c3e6c17, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4268,7 +4268,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_92f269cadb1337bc, []int{80} + return fileDescriptor_api_290a2f113c3e6c17, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4301,7 +4301,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_92f269cadb1337bc, []int{80, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4346,7 +4346,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_92f269cadb1337bc, []int{81} + return fileDescriptor_api_290a2f113c3e6c17, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4384,7 +4384,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_92f269cadb1337bc, []int{82} + return fileDescriptor_api_290a2f113c3e6c17, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4437,7 +4437,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_92f269cadb1337bc, []int{83} + return fileDescriptor_api_290a2f113c3e6c17, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4471,7 +4471,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_92f269cadb1337bc, []int{84} + return fileDescriptor_api_290a2f113c3e6c17, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4515,7 +4515,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_92f269cadb1337bc, []int{85} + return fileDescriptor_api_290a2f113c3e6c17, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4549,7 +4549,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_92f269cadb1337bc, []int{86} + return fileDescriptor_api_290a2f113c3e6c17, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4588,7 +4588,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_92f269cadb1337bc, []int{87} + return fileDescriptor_api_290a2f113c3e6c17, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4622,7 +4622,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_92f269cadb1337bc, []int{88} + return fileDescriptor_api_290a2f113c3e6c17, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4671,7 +4671,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_92f269cadb1337bc, []int{89} + return fileDescriptor_api_290a2f113c3e6c17, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4720,7 +4720,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_92f269cadb1337bc, []int{90} + return fileDescriptor_api_290a2f113c3e6c17, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4755,7 +4755,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_92f269cadb1337bc, []int{91} + return fileDescriptor_api_290a2f113c3e6c17, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4797,7 +4797,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_92f269cadb1337bc, []int{92} + return fileDescriptor_api_290a2f113c3e6c17, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4880,7 +4880,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_92f269cadb1337bc, []int{93} + return fileDescriptor_api_290a2f113c3e6c17, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6330,7 +6330,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_92f269cadb1337bc, []int{94} + return fileDescriptor_api_290a2f113c3e6c17, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7865,7 +7865,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_92f269cadb1337bc, []int{95} + return fileDescriptor_api_290a2f113c3e6c17, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7903,7 +7903,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{96} + return fileDescriptor_api_290a2f113c3e6c17, []int{96} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7939,7 +7939,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{97} + return fileDescriptor_api_290a2f113c3e6c17, []int{97} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7976,7 +7976,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{98} + return fileDescriptor_api_290a2f113c3e6c17, []int{98} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8055,7 +8055,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_92f269cadb1337bc, []int{98, 0} + return fileDescriptor_api_290a2f113c3e6c17, []int{98, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8093,7 +8093,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{99} + return fileDescriptor_api_290a2f113c3e6c17, []int{99} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8131,7 +8131,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{100} + return fileDescriptor_api_290a2f113c3e6c17, []int{100} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8170,7 +8170,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_92f269cadb1337bc, []int{101} + return fileDescriptor_api_290a2f113c3e6c17, []int{101} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8211,7 +8211,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_92f269cadb1337bc, []int{102} + return fileDescriptor_api_290a2f113c3e6c17, []int{102} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8252,7 +8252,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_92f269cadb1337bc, []int{103} + return fileDescriptor_api_290a2f113c3e6c17, []int{103} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8289,7 +8289,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_92f269cadb1337bc, []int{104} + return fileDescriptor_api_290a2f113c3e6c17, []int{104} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8326,7 +8326,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_92f269cadb1337bc, []int{105} + return fileDescriptor_api_290a2f113c3e6c17, []int{105} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8351,6 +8351,82 @@ func (m *RangeFeedEvent) XXX_DiscardUnknown() { var xxx_messageInfo_RangeFeedEvent proto.InternalMessageInfo +// ResetQuorumRequest makes a range that is unavailable due to lost quorum +// available again, at the cost of losing all of the data in the range. Any +// existing replica, even one residing on the target node, will irrevocably +// be removed. ResetQuorumRequest first uses meta2 to identify the range +// descriptor. Then, it removes all replicas from the range descriptor and +// adds a store from the target node as the one designated survivor replica. +// This change is then written to meta2 and sent as a snapshot to a store +// local to the target node. +// +// This RPC is called by the user directly and will not work for ranges +// that have not lost quorum or for a meta range. +type ResetQuorumRequest struct { + RangeID int32 `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3" json:"range_id,omitempty"` +} + +func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } +func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } +func (*ResetQuorumRequest) ProtoMessage() {} +func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_290a2f113c3e6c17, []int{106} +} +func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResetQuorumRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResetQuorumRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResetQuorumRequest.Merge(dst, src) +} +func (m *ResetQuorumRequest) XXX_Size() int { + return m.Size() +} +func (m *ResetQuorumRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ResetQuorumRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ResetQuorumRequest proto.InternalMessageInfo + +type ResetQuorumResponse struct { +} + +func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } +func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } +func (*ResetQuorumResponse) ProtoMessage() {} +func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_290a2f113c3e6c17, []int{107} +} +func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResetQuorumResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResetQuorumResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResetQuorumResponse.Merge(dst, src) +} +func (m *ResetQuorumResponse) XXX_Size() int { + return m.Size() +} +func (m *ResetQuorumResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ResetQuorumResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ResetQuorumResponse proto.InternalMessageInfo + // GossipSubscriptionRequest initiates a game of telephone. It establishes an // indefinite stream that proxies gossip information overheard by the recipient // node back to the caller. Gossip information is filtered down to just those @@ -8367,7 +8443,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{106} + return fileDescriptor_api_290a2f113c3e6c17, []int{108} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8407,7 +8483,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{107} + return fileDescriptor_api_290a2f113c3e6c17, []int{109} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8443,7 +8519,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{108} + return fileDescriptor_api_290a2f113c3e6c17, []int{110} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8482,7 +8558,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{109} + return fileDescriptor_api_290a2f113c3e6c17, []int{111} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8525,7 +8601,7 @@ func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (m *ContentionEvent) String() string { return proto.CompactTextString(m) } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_92f269cadb1337bc, []int{110} + return fileDescriptor_api_290a2f113c3e6c17, []int{112} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8673,6 +8749,8 @@ func init() { proto.RegisterType((*RangeFeedCheckpoint)(nil), "cockroach.roachpb.RangeFeedCheckpoint") proto.RegisterType((*RangeFeedError)(nil), "cockroach.roachpb.RangeFeedError") proto.RegisterType((*RangeFeedEvent)(nil), "cockroach.roachpb.RangeFeedEvent") + proto.RegisterType((*ResetQuorumRequest)(nil), "cockroach.roachpb.ResetQuorumRequest") + proto.RegisterType((*ResetQuorumResponse)(nil), "cockroach.roachpb.ResetQuorumResponse") proto.RegisterType((*GossipSubscriptionRequest)(nil), "cockroach.roachpb.GossipSubscriptionRequest") proto.RegisterType((*GossipSubscriptionEvent)(nil), "cockroach.roachpb.GossipSubscriptionEvent") proto.RegisterType((*JoinNodeRequest)(nil), "cockroach.roachpb.JoinNodeRequest") @@ -8704,6 +8782,7 @@ type InternalClient interface { RangeLookup(ctx context.Context, in *RangeLookupRequest, opts ...grpc.CallOption) (*RangeLookupResponse, error) RangeFeed(ctx context.Context, in *RangeFeedRequest, opts ...grpc.CallOption) (Internal_RangeFeedClient, error) GossipSubscription(ctx context.Context, in *GossipSubscriptionRequest, opts ...grpc.CallOption) (Internal_GossipSubscriptionClient, error) + ResetQuorum(ctx context.Context, in *ResetQuorumRequest, opts ...grpc.CallOption) (*ResetQuorumResponse, error) // Join a bootstrapped cluster. If the target node is itself not part of a // bootstrapped cluster, an appropriate error is returned. Join(ctx context.Context, in *JoinNodeRequest, opts ...grpc.CallOption) (*JoinNodeResponse, error) @@ -8799,6 +8878,15 @@ func (x *internalGossipSubscriptionClient) Recv() (*GossipSubscriptionEvent, err return m, nil } +func (c *internalClient) ResetQuorum(ctx context.Context, in *ResetQuorumRequest, opts ...grpc.CallOption) (*ResetQuorumResponse, error) { + out := new(ResetQuorumResponse) + err := c.cc.Invoke(ctx, "/cockroach.roachpb.Internal/ResetQuorum", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *internalClient) Join(ctx context.Context, in *JoinNodeRequest, opts ...grpc.CallOption) (*JoinNodeResponse, error) { out := new(JoinNodeResponse) err := c.cc.Invoke(ctx, "/cockroach.roachpb.Internal/Join", in, out, opts...) @@ -8814,6 +8902,7 @@ type InternalServer interface { RangeLookup(context.Context, *RangeLookupRequest) (*RangeLookupResponse, error) RangeFeed(*RangeFeedRequest, Internal_RangeFeedServer) error GossipSubscription(*GossipSubscriptionRequest, Internal_GossipSubscriptionServer) error + ResetQuorum(context.Context, *ResetQuorumRequest) (*ResetQuorumResponse, error) // Join a bootstrapped cluster. If the target node is itself not part of a // bootstrapped cluster, an appropriate error is returned. Join(context.Context, *JoinNodeRequest) (*JoinNodeResponse, error) @@ -8901,6 +8990,24 @@ func (x *internalGossipSubscriptionServer) Send(m *GossipSubscriptionEvent) erro return x.ServerStream.SendMsg(m) } +func _Internal_ResetQuorum_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResetQuorumRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(InternalServer).ResetQuorum(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.roachpb.Internal/ResetQuorum", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(InternalServer).ResetQuorum(ctx, req.(*ResetQuorumRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Internal_Join_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(JoinNodeRequest) if err := dec(in); err != nil { @@ -8931,6 +9038,10 @@ var _Internal_serviceDesc = grpc.ServiceDesc{ MethodName: "RangeLookup", Handler: _Internal_RangeLookup_Handler, }, + { + MethodName: "ResetQuorum", + Handler: _Internal_ResetQuorum_Handler, + }, { MethodName: "Join", Handler: _Internal_Join_Handler, @@ -15335,6 +15446,47 @@ func (m *RangeFeedEvent) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *ResetQuorumRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ResetQuorumRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.RangeID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.RangeID)) + } + return i, nil +} + +func (m *ResetQuorumResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ResetQuorumResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + func (m *GossipSubscriptionRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -18748,6 +18900,27 @@ func (m *RangeFeedEvent) Size() (n int) { return n } +func (m *ResetQuorumRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RangeID != 0 { + n += 1 + sovApi(uint64(m.RangeID)) + } + return n +} + +func (m *ResetQuorumResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func (m *GossipSubscriptionRequest) Size() (n int) { if m == nil { return 0 @@ -37482,6 +37655,125 @@ func (m *RangeFeedEvent) Unmarshal(dAtA []byte) error { } return nil } +func (m *ResetQuorumRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ResetQuorumRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ResetQuorumRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RangeID", wireType) + } + m.RangeID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RangeID |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ResetQuorumResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ResetQuorumResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ResetQuorumResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *GossipSubscriptionRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -38213,513 +38505,516 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_92f269cadb1337bc) } - -var fileDescriptor_api_92f269cadb1337bc = []byte{ - // 8079 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x6f, 0x6c, 0x23, 0x49, - 0x76, 0x9f, 0x9a, 0xa4, 0x24, 0xf2, 0x51, 0xfc, 0xa3, 0x92, 0x66, 0x86, 0xa3, 0xdd, 0x1d, 0x69, - 0x7a, 0xfe, 0xcf, 0xed, 0x4a, 0x3b, 0x33, 0xb7, 0xde, 0xf5, 0xce, 0x7a, 0xcf, 0x22, 0xc5, 0x19, - 0x52, 0x1a, 0x69, 0x34, 0x4d, 0x6a, 0xc6, 0xbb, 0x3e, 0xa7, 0xaf, 0xd5, 0x5d, 0xa2, 0xfa, 0x44, - 0x76, 0x73, 0xba, 0x9b, 0xfa, 0xb3, 0x40, 0x80, 0x38, 0x36, 0x12, 0x03, 0x01, 0x0e, 0xfe, 0xe0, - 0x20, 0x67, 0x38, 0x89, 0xcf, 0x71, 0x1c, 0x07, 0x08, 0x82, 0x04, 0x48, 0x10, 0x03, 0x41, 0x62, - 0x7f, 0x31, 0x92, 0x43, 0x60, 0x20, 0xe7, 0x4f, 0x39, 0x04, 0x88, 0x62, 0xeb, 0xf2, 0x2d, 0x81, - 0x11, 0xe4, 0x4b, 0x80, 0xfd, 0x10, 0x04, 0xf5, 0xa7, 0xff, 0x91, 0x4d, 0x8a, 0x9a, 0xeb, 0x85, - 0x17, 0xf0, 0x17, 0x89, 0xfd, 0xaa, 0xde, 0xeb, 0xaa, 0x57, 0x55, 0xaf, 0xde, 0xaf, 0xea, 0x55, - 0x35, 0xcc, 0x5a, 0xa6, 0xa2, 0xee, 0x77, 0x77, 0x57, 0x94, 0xae, 0xbe, 0xdc, 0xb5, 0x4c, 0xc7, - 0x44, 0xb3, 0xaa, 0xa9, 0x1e, 0x50, 0xf2, 0x32, 0x4f, 0x5c, 0xb8, 0x7f, 0x70, 0xb8, 0x72, 0x70, - 0x68, 0x63, 0xeb, 0x10, 0x5b, 0x2b, 0xaa, 0x69, 0xa8, 0x3d, 0xcb, 0xc2, 0x86, 0x7a, 0xb2, 0xd2, - 0x36, 0xd5, 0x03, 0xfa, 0x47, 0x37, 0x5a, 0x8c, 0x7d, 0x01, 0xb9, 0x12, 0x35, 0xc5, 0x51, 0x38, - 0x6d, 0xde, 0xa5, 0x61, 0xcb, 0x32, 0x2d, 0x9b, 0x53, 0x2f, 0xbb, 0xd4, 0x0e, 0x76, 0x94, 0x40, - 0xee, 0xb7, 0x6c, 0xc7, 0xb4, 0x94, 0x16, 0x5e, 0xc1, 0x46, 0x4b, 0x37, 0x30, 0xc9, 0x70, 0xa8, - 0xaa, 0x3c, 0xf1, 0xed, 0xc8, 0xc4, 0x47, 0x3c, 0xb5, 0xd4, 0x73, 0xf4, 0xf6, 0xca, 0x7e, 0x5b, - 0x5d, 0x71, 0xf4, 0x0e, 0xb6, 0x1d, 0xa5, 0xd3, 0xe5, 0x29, 0xf7, 0x69, 0x8a, 0x63, 0x29, 0xaa, - 0x6e, 0xb4, 0xdc, 0xff, 0xdd, 0xdd, 0x15, 0x0b, 0xab, 0xa6, 0xa5, 0x61, 0x4d, 0xb6, 0xbb, 0x8a, - 0xe1, 0x16, 0xb7, 0x65, 0xb6, 0x4c, 0xfa, 0x73, 0x85, 0xfc, 0xe2, 0xd4, 0x6b, 0x2d, 0xd3, 0x6c, - 0xb5, 0xf1, 0x0a, 0x7d, 0xda, 0xed, 0xed, 0xad, 0x68, 0x3d, 0x4b, 0x71, 0x74, 0x93, 0x73, 0x89, - 0xff, 0x46, 0x80, 0x9c, 0x84, 0x5f, 0xf7, 0xb0, 0xed, 0xd4, 0xb0, 0xa2, 0x61, 0x0b, 0x5d, 0x85, - 0xe4, 0x01, 0x3e, 0x29, 0x25, 0x97, 0x84, 0xbb, 0x33, 0xe5, 0xe9, 0x2f, 0x4f, 0x17, 0x93, 0x1b, - 0xf8, 0x44, 0x22, 0x34, 0xb4, 0x04, 0xd3, 0xd8, 0xd0, 0x64, 0x92, 0x9c, 0x0a, 0x27, 0x4f, 0x61, - 0x43, 0xdb, 0xc0, 0x27, 0xe8, 0xdb, 0x90, 0xb6, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x25, 0xe1, - 0xee, 0x64, 0xf9, 0xe7, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x65, 0xd5, - 0xec, 0xac, 0x78, 0xed, 0xa4, 0xed, 0xfa, 0xbf, 0x57, 0xba, 0x07, 0xad, 0x95, 0x7e, 0x1d, 0x2d, - 0x37, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, 0xa5, - 0x13, 0xc5, 0xa4, 0xf8, 0xbb, 0x49, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xc9, 0xdf, - 0x87, 0xa4, 0x73, 0x6c, 0xd0, 0x92, 0x67, 0x1f, 0x5e, 0x5b, 0x1e, 0xe8, 0x11, 0xcb, 0x4d, 0x4b, - 0x31, 0x6c, 0x45, 0x25, 0xd5, 0x97, 0x48, 0x56, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0xf7, 0x3a, 0x98, - 0x2a, 0x92, 0x56, 0x2a, 0xfb, 0xf0, 0x4a, 0x04, 0x67, 0xa3, 0xab, 0x18, 0x12, 0xb0, 0xbc, 0xe4, - 0x37, 0xba, 0x0a, 0x69, 0xa3, 0xd7, 0x21, 0xaa, 0xb0, 0x69, 0x45, 0x93, 0xd2, 0xb4, 0xd1, 0xeb, - 0x6c, 0xe0, 0x13, 0x1b, 0xfd, 0x02, 0x5c, 0xd6, 0x70, 0xd7, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, 0xb6, - 0x14, 0xa3, 0x85, 0x65, 0xdd, 0xd8, 0x33, 0xed, 0xd2, 0xd4, 0x52, 0xf2, 0x6e, 0xf6, 0xe1, 0xdb, - 0x11, 0xf2, 0x25, 0x92, 0xab, 0x6e, 0xec, 0x99, 0xe5, 0xd4, 0x0f, 0x4f, 0x17, 0x27, 0xa4, 0x79, - 0x5f, 0x82, 0x97, 0x64, 0xa3, 0x06, 0xe4, 0x78, 0x71, 0x2d, 0xac, 0xd8, 0xa6, 0x51, 0x9a, 0x5e, - 0x12, 0xee, 0xe6, 0x1f, 0x2e, 0x47, 0x09, 0x0c, 0xa9, 0x86, 0x3c, 0xf6, 0x3a, 0x58, 0xa2, 0x5c, - 0xd2, 0x8c, 0x15, 0x78, 0x42, 0x6f, 0x41, 0x86, 0xd4, 0x64, 0xf7, 0xc4, 0xc1, 0x76, 0x29, 0x4d, - 0xab, 0x42, 0xaa, 0x56, 0x26, 0xcf, 0xe2, 0xa7, 0x30, 0x13, 0x64, 0x45, 0x08, 0xf2, 0x52, 0xb5, - 0xb1, 0xb3, 0x59, 0x95, 0x77, 0xb6, 0x36, 0xb6, 0x9e, 0xbf, 0xda, 0x2a, 0x4e, 0xa0, 0x79, 0x28, - 0x72, 0xda, 0x46, 0xf5, 0x33, 0xf9, 0x59, 0x7d, 0xb3, 0xde, 0x2c, 0x0a, 0x0b, 0xa9, 0x5f, 0xfb, - 0xdd, 0x6b, 0x13, 0xe2, 0x36, 0xc0, 0x53, 0xec, 0xf0, 0x0e, 0x86, 0xca, 0x30, 0xb5, 0x4f, 0xcb, - 0x53, 0x12, 0xa8, 0xa6, 0x97, 0x22, 0x0b, 0x1e, 0xe8, 0x8c, 0xe5, 0x34, 0xd1, 0xc6, 0x8f, 0x4e, - 0x17, 0x05, 0x89, 0x73, 0x8a, 0x7f, 0x24, 0x40, 0x96, 0x8a, 0x64, 0xf5, 0x43, 0x95, 0x3e, 0x99, - 0xd7, 0xcf, 0x55, 0xc6, 0xa0, 0x50, 0xb4, 0x0c, 0x93, 0x87, 0x4a, 0xbb, 0x87, 0x4b, 0x09, 0x2a, - 0xa3, 0x14, 0x21, 0xe3, 0x25, 0x49, 0x97, 0x58, 0x36, 0xf4, 0x18, 0x66, 0x74, 0xc3, 0xc1, 0x86, - 0x23, 0x33, 0xb6, 0xe4, 0x39, 0x6c, 0x59, 0x96, 0x9b, 0x3e, 0x88, 0xff, 0x5a, 0x00, 0xd8, 0xee, - 0xc5, 0xa9, 0x14, 0xf4, 0xcd, 0x31, 0xcb, 0xcf, 0x7b, 0x17, 0xaf, 0xc5, 0x65, 0x98, 0xd2, 0x8d, - 0xb6, 0x6e, 0xb0, 0xf2, 0xa7, 0x25, 0xfe, 0x84, 0xe6, 0x61, 0x72, 0xb7, 0xad, 0x1b, 0x1a, 0x1d, - 0x0f, 0x69, 0x89, 0x3d, 0x88, 0x12, 0x64, 0x69, 0xa9, 0x63, 0xd4, 0xbb, 0xf8, 0x1f, 0x12, 0x70, - 0xa9, 0x62, 0x1a, 0x9a, 0x4e, 0x86, 0xa4, 0xd2, 0xfe, 0x5a, 0x68, 0x65, 0x1d, 0x02, 0x83, 0x4f, - 0xc6, 0xc7, 0xdd, 0x31, 0xdb, 0x18, 0xf9, 0x5c, 0xd5, 0xe3, 0x2e, 0xa5, 0x45, 0x6b, 0x12, 0x7d, - 0x13, 0xae, 0x28, 0xed, 0xb6, 0x79, 0x24, 0xeb, 0x7b, 0xb2, 0x66, 0x62, 0x5b, 0x36, 0x4c, 0x47, - 0xc6, 0xc7, 0xba, 0xed, 0x50, 0x53, 0x92, 0x96, 0xe6, 0x68, 0x72, 0x7d, 0x6f, 0xcd, 0xc4, 0xf6, - 0x96, 0xe9, 0x54, 0x49, 0x12, 0x19, 0xa7, 0xa4, 0x30, 0x6c, 0x9c, 0x4e, 0x11, 0xf3, 0x2b, 0xa5, - 0xf1, 0x71, 0x97, 0x8d, 0xd3, 0x5f, 0x82, 0xcb, 0xfd, 0x7a, 0x8c, 0xb3, 0x9d, 0xfe, 0x44, 0x80, - 0x7c, 0xdd, 0xd0, 0x9d, 0xaf, 0x45, 0x03, 0x79, 0x4a, 0x4d, 0x06, 0x95, 0x7a, 0x1f, 0x8a, 0x7b, - 0x8a, 0xde, 0x7e, 0x6e, 0x34, 0xcd, 0xce, 0xae, 0xed, 0x98, 0x06, 0xb6, 0xb9, 0xd6, 0x07, 0xe8, - 0xe2, 0x4b, 0x28, 0x78, 0xb5, 0x89, 0x53, 0x4d, 0x0e, 0x14, 0xeb, 0x86, 0x6a, 0xe1, 0x0e, 0x36, - 0x62, 0xd5, 0xd3, 0xdb, 0x90, 0xd1, 0x5d, 0xb9, 0x54, 0x57, 0x49, 0xc9, 0x27, 0x88, 0x3d, 0x98, - 0x0d, 0xbc, 0x35, 0x4e, 0xb3, 0x48, 0xa6, 0x06, 0x7c, 0x24, 0xfb, 0x6d, 0x44, 0xa6, 0x06, 0x7c, - 0xc4, 0xcc, 0x58, 0x03, 0x72, 0x6b, 0xb8, 0x8d, 0x1d, 0x1c, 0xa7, 0x75, 0xdf, 0x81, 0xbc, 0x2b, - 0x34, 0xce, 0x86, 0xf9, 0x7b, 0x02, 0x20, 0x2e, 0x97, 0xcc, 0xa6, 0x71, 0xb6, 0xcd, 0x22, 0x71, - 0x21, 0x9c, 0x9e, 0x65, 0x30, 0x5f, 0x80, 0xf5, 0x49, 0x60, 0x24, 0xea, 0x0e, 0xf8, 0x56, 0x36, - 0x15, 0xb4, 0xb2, 0xdc, 0x8d, 0x39, 0x82, 0xb9, 0x50, 0xc1, 0xe2, 0x6d, 0xbe, 0x14, 0x2d, 0x53, - 0x62, 0x29, 0x19, 0xf4, 0xd5, 0x28, 0x51, 0xfc, 0xbe, 0x00, 0xb3, 0x95, 0x36, 0x56, 0xac, 0xd8, - 0x35, 0xf2, 0x2d, 0x48, 0x6b, 0x58, 0xd1, 0x68, 0x95, 0xd9, 0xc0, 0x7e, 0x27, 0x20, 0x85, 0x78, - 0xb4, 0xcb, 0xfb, 0x6d, 0x75, 0xb9, 0xe9, 0xfa, 0xba, 0x7c, 0x74, 0x7b, 0x4c, 0xe2, 0x67, 0x80, - 0x82, 0x25, 0x8b, 0xb3, 0x23, 0xfc, 0x43, 0x01, 0x90, 0x84, 0x0f, 0xb1, 0xe5, 0xc4, 0x5e, 0xed, - 0x35, 0xc8, 0x3a, 0x8a, 0xd5, 0xc2, 0x8e, 0x4c, 0xbc, 0xf8, 0x8b, 0xd4, 0x1c, 0x18, 0x1f, 0x21, - 0x8b, 0x9f, 0xc3, 0x5c, 0xa8, 0x7c, 0x71, 0x56, 0xfe, 0x7f, 0x0b, 0x90, 0x6d, 0xa8, 0x8a, 0x11, - 0x67, 0xad, 0x3f, 0x85, 0xac, 0xad, 0x2a, 0x86, 0xbc, 0x67, 0x5a, 0x1d, 0xc5, 0xa1, 0x5d, 0x3c, - 0x1f, 0xaa, 0xb5, 0xe7, 0x41, 0xab, 0x8a, 0xf1, 0x84, 0x66, 0x92, 0xc0, 0xf6, 0x7e, 0xa3, 0x17, - 0x90, 0x3d, 0xc0, 0x27, 0x32, 0x47, 0x63, 0x74, 0xfe, 0xcb, 0x3f, 0x7c, 0x3f, 0xc0, 0x7f, 0x70, - 0xb8, 0xec, 0x82, 0xb8, 0xe5, 0x00, 0x88, 0x5b, 0x26, 0x1c, 0xcb, 0x0d, 0xc7, 0xc2, 0x46, 0xcb, - 0xd9, 0x97, 0xe0, 0x00, 0x9f, 0x3c, 0x63, 0x32, 0xd8, 0xc0, 0x5a, 0x4f, 0xa5, 0x93, 0xc5, 0x94, - 0xf8, 0x7f, 0x05, 0x98, 0x61, 0x55, 0x8e, 0x73, 0x60, 0x7d, 0x00, 0x29, 0xcb, 0x3c, 0x62, 0x03, - 0x2b, 0xfb, 0xf0, 0xad, 0x08, 0x11, 0x1b, 0xf8, 0x24, 0x38, 0x73, 0xd1, 0xec, 0xa8, 0x0c, 0xdc, - 0x0f, 0x94, 0x29, 0x77, 0x72, 0x5c, 0x6e, 0x60, 0x5c, 0x12, 0x91, 0x71, 0x07, 0x0a, 0xbb, 0x8a, - 0xa3, 0xee, 0xcb, 0x16, 0x2f, 0x24, 0x99, 0xe5, 0x92, 0x77, 0x67, 0xa4, 0x3c, 0x25, 0xbb, 0x45, - 0xb7, 0x49, 0xcd, 0x59, 0x4f, 0xb7, 0xf1, 0x5f, 0xb1, 0x36, 0xff, 0x7f, 0x02, 0x1f, 0x43, 0x6e, - 0xcd, 0xff, 0xaa, 0x35, 0xfd, 0x6f, 0x26, 0xe0, 0x4a, 0x65, 0x1f, 0xab, 0x07, 0x15, 0xd3, 0xb0, - 0x75, 0xdb, 0x21, 0xba, 0x8b, 0xb3, 0xfd, 0xdf, 0x82, 0xcc, 0x91, 0xee, 0xec, 0xcb, 0x9a, 0xbe, - 0xb7, 0x47, 0xed, 0x5c, 0x5a, 0x4a, 0x13, 0xc2, 0x9a, 0xbe, 0xb7, 0x87, 0x1e, 0x41, 0xaa, 0x63, - 0x6a, 0xcc, 0x5d, 0xce, 0x3f, 0x5c, 0x8c, 0x10, 0x4f, 0x8b, 0x66, 0xf7, 0x3a, 0x9b, 0xa6, 0x86, - 0x25, 0x9a, 0x19, 0x5d, 0x03, 0x50, 0x09, 0xb5, 0x6b, 0xea, 0x86, 0xc3, 0xe7, 0xc9, 0x00, 0x05, - 0xd5, 0x20, 0xe3, 0x60, 0xab, 0xa3, 0x1b, 0x8a, 0x83, 0x4b, 0x93, 0x54, 0x79, 0x37, 0x23, 0x0b, - 0xde, 0x6d, 0xeb, 0xaa, 0xb2, 0x86, 0x6d, 0xd5, 0xd2, 0xbb, 0x8e, 0x69, 0x71, 0x2d, 0xfa, 0xcc, - 0xe2, 0xf7, 0x52, 0x50, 0x1a, 0xd4, 0x4d, 0x9c, 0x3d, 0x64, 0x1b, 0xa6, 0x08, 0xbe, 0x6e, 0x3b, - 0xbc, 0x8f, 0x3c, 0x1c, 0xa6, 0x82, 0x88, 0x12, 0x50, 0x9c, 0xde, 0x76, 0x78, 0xb1, 0xb9, 0x9c, - 0x85, 0x7f, 0x27, 0xc0, 0x14, 0x4b, 0x40, 0x0f, 0x20, 0xcd, 0x17, 0x14, 0x34, 0x5a, 0xc6, 0x64, - 0xf9, 0xf2, 0xd9, 0xe9, 0xe2, 0x34, 0x5b, 0x23, 0x58, 0xfb, 0xd2, 0xff, 0x29, 0x4d, 0xd3, 0x7c, - 0x75, 0x8d, 0xb4, 0x96, 0xed, 0x28, 0x96, 0x43, 0x97, 0x6d, 0x12, 0x0c, 0x37, 0x50, 0xc2, 0x06, - 0x3e, 0x41, 0xeb, 0x30, 0x65, 0x3b, 0x8a, 0xd3, 0xb3, 0x79, 0x7b, 0x5d, 0xa8, 0xb0, 0x0d, 0xca, - 0x29, 0x71, 0x09, 0xc4, 0xd1, 0xd1, 0xb0, 0xa3, 0xe8, 0x6d, 0xda, 0x80, 0x19, 0x89, 0x3f, 0x89, - 0xbf, 0x25, 0xc0, 0x14, 0xcb, 0x8a, 0xae, 0xc0, 0x9c, 0xb4, 0xba, 0xf5, 0xb4, 0x2a, 0xd7, 0xb7, - 0xd6, 0xaa, 0xcd, 0xaa, 0xb4, 0x59, 0xdf, 0x5a, 0x6d, 0x56, 0x8b, 0x13, 0xe8, 0x32, 0x20, 0x37, - 0xa1, 0xf2, 0x7c, 0xab, 0x51, 0x6f, 0x34, 0xab, 0x5b, 0xcd, 0xa2, 0x40, 0xd7, 0x16, 0x28, 0x3d, - 0x40, 0x4d, 0xa0, 0x9b, 0xb0, 0xd4, 0x4f, 0x95, 0x1b, 0xcd, 0xd5, 0x66, 0x43, 0xae, 0x36, 0x9a, - 0xf5, 0xcd, 0xd5, 0x66, 0x75, 0xad, 0x98, 0x1c, 0x91, 0x8b, 0xbc, 0x44, 0x92, 0xaa, 0x95, 0x66, - 0x31, 0x25, 0x3a, 0x70, 0x49, 0xc2, 0xaa, 0xd9, 0xe9, 0xf6, 0x1c, 0x4c, 0x4a, 0x69, 0xc7, 0x39, - 0x52, 0xae, 0xc0, 0xb4, 0x66, 0x9d, 0xc8, 0x56, 0xcf, 0xe0, 0xe3, 0x64, 0x4a, 0xb3, 0x4e, 0xa4, - 0x9e, 0x21, 0xfe, 0x4b, 0x01, 0x2e, 0xf7, 0xbf, 0x36, 0xce, 0x4e, 0xf8, 0x02, 0xb2, 0x8a, 0xa6, - 0x61, 0x4d, 0xd6, 0x70, 0xdb, 0x51, 0xb8, 0x33, 0x72, 0x3f, 0x20, 0x89, 0x2f, 0xb6, 0x2d, 0x7b, - 0x8b, 0x6d, 0x9b, 0x2f, 0x2b, 0x15, 0x5a, 0x90, 0x35, 0xc2, 0xe1, 0x9a, 0x1f, 0x2a, 0x84, 0x52, - 0xc4, 0xff, 0x95, 0x82, 0x5c, 0xd5, 0xd0, 0x9a, 0xc7, 0xb1, 0xce, 0x25, 0x97, 0x61, 0x4a, 0x35, - 0x3b, 0x1d, 0xdd, 0x71, 0x15, 0xc4, 0x9e, 0xd0, 0xcf, 0x06, 0x9c, 0xc8, 0xe4, 0x18, 0xae, 0x94, - 0xef, 0x3e, 0xa2, 0xef, 0xc0, 0x15, 0x62, 0x35, 0x2d, 0x43, 0x69, 0xcb, 0x4c, 0x9a, 0xec, 0x58, - 0x7a, 0xab, 0x85, 0x2d, 0xbe, 0xc0, 0x77, 0x37, 0xa2, 0x9c, 0x75, 0xce, 0x51, 0xa1, 0x0c, 0x4d, - 0x96, 0x5f, 0xba, 0xa4, 0x47, 0x91, 0xd1, 0x27, 0x00, 0x64, 0x2a, 0xa2, 0x8b, 0x86, 0x36, 0xb7, - 0x47, 0xc3, 0x56, 0x0d, 0x5d, 0x13, 0x44, 0x18, 0xc8, 0xb3, 0x8d, 0x56, 0x08, 0x62, 0x78, 0xdd, - 0xd3, 0x2d, 0x2c, 0x3f, 0xe8, 0xaa, 0x14, 0xca, 0xa7, 0xcb, 0xf9, 0xb3, 0xd3, 0x45, 0x90, 0x18, - 0xf9, 0xc1, 0x76, 0x85, 0x20, 0x08, 0xf6, 0xbb, 0xab, 0xa2, 0x57, 0x70, 0x2f, 0xb0, 0x22, 0x41, - 0x66, 0x5e, 0x5e, 0x2d, 0xc5, 0x91, 0xf7, 0xf5, 0xd6, 0x3e, 0xb6, 0x64, 0x6f, 0xe1, 0x98, 0xae, - 0xe0, 0xa5, 0xa5, 0x9b, 0x3e, 0x43, 0x45, 0x31, 0x58, 0xe9, 0x57, 0x9d, 0x1a, 0xcd, 0xec, 0xe9, - 0x8c, 0x28, 0xbf, 0x6b, 0xea, 0xb6, 0x69, 0x94, 0x32, 0x4c, 0xf9, 0xec, 0x09, 0xdd, 0x83, 0xa2, - 0x73, 0x6c, 0xc8, 0xfb, 0x58, 0xb1, 0x9c, 0x5d, 0xac, 0x38, 0x64, 0x96, 0x06, 0x9a, 0xa3, 0xe0, - 0x1c, 0x1b, 0xb5, 0x00, 0x19, 0xbd, 0x80, 0xa2, 0x6e, 0xc8, 0x7b, 0x6d, 0xbd, 0xb5, 0xef, 0xc8, - 0x47, 0x96, 0xee, 0x60, 0xbb, 0x34, 0x4b, 0x15, 0x12, 0xd5, 0x6f, 0x1b, 0x7c, 0x25, 0x57, 0x7b, - 0x45, 0x72, 0x72, 0xd5, 0xe4, 0x75, 0xe3, 0x09, 0xe5, 0xa7, 0x44, 0x7b, 0x3d, 0x95, 0x9e, 0x2e, - 0xa6, 0xc5, 0xff, 0x26, 0x40, 0xde, 0xed, 0x6e, 0x71, 0x8e, 0x8c, 0xbb, 0x50, 0x34, 0x0d, 0x2c, - 0x77, 0xf7, 0x15, 0x1b, 0x73, 0x3d, 0xf2, 0x09, 0x27, 0x6f, 0x1a, 0x78, 0x9b, 0x90, 0x99, 0xba, - 0xd0, 0x36, 0xcc, 0xda, 0x8e, 0xd2, 0xd2, 0x8d, 0x56, 0x40, 0xbd, 0x93, 0xe3, 0xbb, 0xf5, 0x45, - 0xce, 0xed, 0xd1, 0x43, 0x5e, 0xca, 0x9f, 0x0a, 0x30, 0xbb, 0xaa, 0x75, 0x74, 0xa3, 0xd1, 0x6d, - 0xeb, 0xb1, 0xae, 0x16, 0xdc, 0x84, 0x8c, 0x4d, 0x64, 0xfa, 0x06, 0xdf, 0xc7, 0x7e, 0x69, 0x9a, - 0x42, 0x2c, 0xff, 0x33, 0x28, 0xe0, 0xe3, 0xae, 0xce, 0x36, 0x03, 0x18, 0x64, 0x49, 0x8d, 0x5f, - 0xb7, 0xbc, 0xcf, 0x4b, 0x92, 0x78, 0x9d, 0x3e, 0x03, 0x14, 0xac, 0x52, 0x9c, 0xd8, 0xe5, 0x33, - 0x98, 0xa3, 0xa2, 0x77, 0x0c, 0x3b, 0x66, 0x7d, 0x89, 0xbf, 0x08, 0xf3, 0x61, 0xd1, 0x71, 0x96, - 0xfb, 0x15, 0x6f, 0xe5, 0x4d, 0x6c, 0xc5, 0x0a, 0x37, 0x3d, 0x5d, 0x73, 0xc1, 0x71, 0x96, 0xf9, - 0x57, 0x05, 0xb8, 0x4a, 0x65, 0xd3, 0xfd, 0x92, 0x3d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, 0x2b, 0x56, - 0xbe, 0x01, 0x53, 0x0c, 0xf3, 0xd2, 0xfe, 0x39, 0x59, 0xce, 0x12, 0xcf, 0xa5, 0xe1, 0x98, 0x16, - 0xf1, 0x5c, 0x78, 0x92, 0xa8, 0xc0, 0x42, 0x54, 0x29, 0x62, 0x5e, 0x0e, 0x98, 0xe5, 0x4e, 0x23, - 0xe9, 0xca, 0x95, 0x7d, 0xe2, 0x33, 0xa1, 0x2a, 0x64, 0x55, 0xfa, 0x4b, 0x76, 0x4e, 0xba, 0x98, - 0xca, 0xcf, 0x8f, 0xf2, 0x37, 0x19, 0x5b, 0xf3, 0xa4, 0x8b, 0x89, 0xd3, 0xea, 0xfe, 0x26, 0x8a, - 0x0a, 0x54, 0x72, 0xa4, 0xc7, 0x4a, 0xc7, 0x11, 0xcd, 0xeb, 0xba, 0x7e, 0x5c, 0x07, 0xff, 0x2a, - 0xc9, 0x95, 0xc0, 0xde, 0xc1, 0xb3, 0xc7, 0xea, 0xa3, 0x7c, 0x1e, 0xda, 0xae, 0x0a, 0x56, 0x3c, - 0x71, 0x81, 0x8a, 0x07, 0xd6, 0xcc, 0x7d, 0x2a, 0xfa, 0x0c, 0x02, 0xab, 0xe2, 0x32, 0xab, 0x93, - 0x8b, 0x7e, 0x2e, 0xa2, 0x8e, 0x59, 0x5f, 0x0a, 0xa3, 0xdb, 0xa8, 0x02, 0x69, 0x7c, 0xdc, 0x95, - 0x35, 0x6c, 0xab, 0xdc, 0x70, 0x89, 0xc3, 0xf6, 0xd5, 0x06, 0xf0, 0xc0, 0x34, 0x3e, 0xee, 0x12, - 0x22, 0xda, 0x21, 0xb3, 0x97, 0xeb, 0x2a, 0xd0, 0x62, 0xdb, 0xe7, 0xc3, 0x0b, 0xbf, 0xa7, 0x70, - 0x71, 0x05, 0xcf, 0x4b, 0x60, 0x22, 0xc4, 0x1f, 0x08, 0xf0, 0x56, 0x64, 0xab, 0xc5, 0x39, 0x91, - 0x7d, 0x02, 0x29, 0x5a, 0xf9, 0xc4, 0x05, 0x2b, 0x4f, 0xb9, 0xc4, 0xdf, 0x73, 0xc7, 0xb8, 0x84, - 0xdb, 0x26, 0x51, 0xec, 0x57, 0xb0, 0x1e, 0x36, 0xed, 0x36, 0x78, 0xe2, 0xc2, 0x0d, 0xee, 0xb2, - 0x7a, 0x46, 0xa0, 0xaf, 0x98, 0x71, 0x1a, 0x81, 0x5f, 0x17, 0x60, 0xce, 0xf3, 0x69, 0x62, 0x76, - 0x6f, 0x3f, 0x80, 0xa4, 0x61, 0x1e, 0x5d, 0x64, 0x31, 0x90, 0xe4, 0x27, 0x53, 0x52, 0xb8, 0x44, - 0x71, 0xd6, 0xf7, 0xdf, 0x27, 0x20, 0xf3, 0xb4, 0x12, 0x67, 0x2d, 0x3f, 0xe1, 0x0b, 0xcd, 0x6c, - 0x60, 0x47, 0x75, 0x45, 0xef, 0x7d, 0xcb, 0x4f, 0x2b, 0x1b, 0xf8, 0xc4, 0xed, 0x8a, 0x84, 0x0b, - 0xad, 0x42, 0xc6, 0xd9, 0xb7, 0xb0, 0xbd, 0x6f, 0xb6, 0xb5, 0x8b, 0xf8, 0x20, 0x3e, 0xd7, 0x02, - 0x86, 0x49, 0x2a, 0xd7, 0x0d, 0x5e, 0x10, 0x22, 0x82, 0x17, 0xc8, 0x6b, 0x3c, 0x37, 0x2e, 0x71, - 0x91, 0xd7, 0x04, 0xfc, 0xb7, 0xc9, 0xe2, 0x94, 0xf8, 0x02, 0x80, 0x54, 0x27, 0xce, 0x26, 0xf9, - 0x5b, 0x49, 0xc8, 0x6f, 0xf7, 0xec, 0xfd, 0x98, 0x7b, 0x5f, 0x05, 0xa0, 0xdb, 0xb3, 0x29, 0x3e, - 0x38, 0x36, 0x78, 0x9d, 0xcf, 0x89, 0x8b, 0x70, 0x2b, 0xcd, 0xf8, 0x9a, 0xc7, 0x06, 0xaa, 0x71, - 0x21, 0x58, 0xf6, 0x83, 0x2b, 0x6e, 0x8c, 0x42, 0x92, 0xcd, 0x63, 0x63, 0x13, 0x7b, 0x10, 0x92, - 0x49, 0xc2, 0x44, 0xd2, 0x27, 0x30, 0x4d, 0x1e, 0x64, 0xc7, 0xbc, 0x48, 0x33, 0x4f, 0x11, 0x9e, - 0xa6, 0x89, 0x1e, 0x43, 0x86, 0x71, 0x93, 0xa9, 0x69, 0x8a, 0x4e, 0x4d, 0x51, 0x75, 0xe1, 0x6a, - 0xa4, 0x93, 0x52, 0x9a, 0xb2, 0x92, 0x89, 0x68, 0x1e, 0x26, 0xf7, 0x4c, 0x4b, 0xc5, 0x34, 0x62, - 0x22, 0x2d, 0xb1, 0x07, 0xd6, 0x9e, 0xeb, 0xa9, 0x74, 0xba, 0x98, 0x59, 0x4f, 0xa5, 0x33, 0x45, - 0x10, 0x7f, 0x4b, 0x80, 0x82, 0xd7, 0x10, 0x71, 0x5a, 0xeb, 0x4a, 0x48, 0x8b, 0x17, 0x6f, 0x0a, - 0xa2, 0x40, 0xf1, 0x3f, 0x52, 0x77, 0x45, 0x35, 0x0f, 0x69, 0xcb, 0xc4, 0xd9, 0x53, 0x1e, 0xb3, - 0xd0, 0x99, 0xc4, 0x45, 0x5b, 0x97, 0x46, 0xd1, 0x3c, 0x80, 0x79, 0xbd, 0x43, 0xec, 0xb8, 0xee, - 0xb4, 0x4f, 0x38, 0xa6, 0x72, 0xb0, 0xbb, 0x3f, 0x3b, 0xe7, 0xa7, 0x55, 0xdc, 0x24, 0xf1, 0x1f, - 0xd3, 0xd5, 0x69, 0xbf, 0x26, 0x71, 0xaa, 0xba, 0x0e, 0x39, 0x8b, 0x89, 0x26, 0x3e, 0xc7, 0x05, - 0xb5, 0x3d, 0xe3, 0xb1, 0x12, 0x85, 0xff, 0x4e, 0x02, 0x0a, 0x2f, 0x7a, 0xd8, 0x3a, 0xf9, 0x3a, - 0xa9, 0xfb, 0x36, 0x14, 0x8e, 0x14, 0xdd, 0x91, 0xf7, 0x4c, 0x4b, 0xee, 0x75, 0x35, 0xc5, 0x71, - 0xe3, 0x37, 0x72, 0x84, 0xfc, 0xc4, 0xb4, 0x76, 0x28, 0x11, 0x61, 0x40, 0x07, 0x86, 0x79, 0x64, - 0xc8, 0x84, 0x4c, 0x51, 0xec, 0xb1, 0xc1, 0x97, 0x8c, 0xcb, 0x1f, 0xfe, 0xd7, 0xd3, 0xc5, 0x47, - 0x63, 0x45, 0x65, 0xd1, 0x08, 0xb4, 0x5e, 0x4f, 0xd7, 0x96, 0x77, 0x76, 0xea, 0x6b, 0x52, 0x91, - 0x8a, 0x7c, 0xc5, 0x24, 0x36, 0x8f, 0x0d, 0x5b, 0xfc, 0x27, 0x09, 0x28, 0xfa, 0x3a, 0x8a, 0xb3, - 0x21, 0xab, 0x90, 0x7d, 0xdd, 0xc3, 0x96, 0xfe, 0x06, 0xcd, 0x08, 0x9c, 0x91, 0x98, 0x9d, 0xcf, - 0x61, 0x26, 0xa4, 0x81, 0xe4, 0x4f, 0xa7, 0x81, 0xec, 0x91, 0x5f, 0x79, 0x74, 0x1f, 0x66, 0x9d, - 0x63, 0x43, 0x66, 0xf1, 0x78, 0x2c, 0x86, 0xc3, 0x0d, 0x3b, 0x28, 0x38, 0x44, 0x1f, 0x84, 0x4e, - 0xe3, 0x37, 0x6c, 0xf1, 0x8f, 0x04, 0x40, 0x54, 0x51, 0x75, 0xb6, 0xa6, 0xff, 0x75, 0xe9, 0x4f, - 0x77, 0xa1, 0x48, 0x23, 0x1c, 0x65, 0x7d, 0x4f, 0xee, 0xe8, 0xb6, 0xad, 0x1b, 0x2d, 0xde, 0xa1, - 0xf2, 0x94, 0x5e, 0xdf, 0xdb, 0x64, 0x54, 0xf1, 0xaf, 0xc3, 0x5c, 0xa8, 0x02, 0x71, 0x36, 0xf6, - 0x75, 0x98, 0xd9, 0x33, 0x7b, 0x86, 0x26, 0xb3, 0x1d, 0x0f, 0xbe, 0x1c, 0x98, 0xa5, 0x34, 0xf6, - 0x3e, 0xf1, 0x7f, 0x26, 0x60, 0x5e, 0xc2, 0xb6, 0xd9, 0x3e, 0xc4, 0xf1, 0xab, 0xb0, 0x06, 0x7c, - 0xaf, 0x45, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x9b, 0xe6, 0xc2, 0x6b, 0xea, 0x37, 0x47, 0xf7, - 0xd8, 0xc1, 0x55, 0x74, 0xbe, 0x26, 0x97, 0x0a, 0xad, 0xc9, 0x99, 0x50, 0xd0, 0x5b, 0x86, 0x49, - 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x8b, 0x54, 0x96, 0x47, 0x15, 0xb2, 0xce, 0x58, 0x1a, - 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xf2, 0x65, 0x52, 0xde, 0xb3, 0xd3, 0xc5, 0x7c, 0x28, - 0xcd, 0x96, 0xf2, 0xba, 0xf7, 0x4c, 0xa4, 0x8b, 0xdf, 0x86, 0x4b, 0x7d, 0xca, 0x8e, 0xd3, 0xe3, - 0xf9, 0xb7, 0x49, 0xb8, 0x1a, 0x16, 0x1f, 0x37, 0xfe, 0xf8, 0xba, 0x37, 0x68, 0x0d, 0x72, 0x1d, - 0xdd, 0x78, 0xb3, 0xa5, 0xc5, 0x99, 0x8e, 0x6e, 0xf8, 0xcb, 0xb8, 0x11, 0x5d, 0x63, 0xea, 0x2b, - 0xed, 0x1a, 0x0a, 0x2c, 0x44, 0xb5, 0x5d, 0x9c, 0xfd, 0xe3, 0xd7, 0x04, 0x98, 0x89, 0x7b, 0xcd, - 0xec, 0xcd, 0xe2, 0xcd, 0xc4, 0x26, 0xe4, 0xbe, 0x82, 0x45, 0xb6, 0xdf, 0x11, 0x00, 0x35, 0xad, - 0x9e, 0x41, 0x40, 0xed, 0x33, 0xb3, 0x15, 0x67, 0x35, 0xe7, 0x61, 0x52, 0x37, 0x34, 0x7c, 0x4c, - 0xab, 0x99, 0x92, 0xd8, 0x43, 0x68, 0xeb, 0x30, 0x39, 0xd6, 0xd6, 0xa1, 0xf8, 0x39, 0xcc, 0x85, - 0x8a, 0x18, 0x67, 0xfd, 0xff, 0x69, 0x02, 0xe6, 0x78, 0x45, 0x62, 0x5f, 0x5e, 0xfc, 0x26, 0x4c, - 0xb6, 0x89, 0xcc, 0x11, 0xed, 0x4c, 0xdf, 0xe9, 0xb6, 0x33, 0xcd, 0x8c, 0x7e, 0x0e, 0xa0, 0x6b, - 0xe1, 0x43, 0x99, 0xb1, 0x26, 0xc7, 0x62, 0xcd, 0x10, 0x0e, 0x4a, 0x40, 0xbf, 0x00, 0x05, 0x32, - 0x9e, 0xbb, 0x96, 0xd9, 0x35, 0x6d, 0xe2, 0xb2, 0xd8, 0xe3, 0xa1, 0x9c, 0xd9, 0xb3, 0xd3, 0xc5, - 0xdc, 0xa6, 0x6e, 0x6c, 0x73, 0xc6, 0x66, 0x43, 0x22, 0x86, 0xc1, 0x7b, 0xb4, 0xc5, 0xff, 0x2c, - 0xc0, 0xfc, 0x57, 0xb6, 0x14, 0xfb, 0x97, 0xa1, 0x2b, 0xf1, 0x25, 0x14, 0xe9, 0x8f, 0xba, 0xb1, - 0x67, 0xc6, 0xb9, 0x28, 0xfe, 0x3d, 0x01, 0x66, 0x03, 0x82, 0xe3, 0xf4, 0x4f, 0xde, 0x48, 0x4f, - 0xe2, 0x2f, 0x12, 0x8f, 0x25, 0xd8, 0xc9, 0xe3, 0x1c, 0x42, 0x7f, 0x90, 0x80, 0xcb, 0x15, 0xb6, - 0x85, 0xec, 0xc6, 0x54, 0xc4, 0xd9, 0x33, 0x4a, 0x30, 0x7d, 0x88, 0x2d, 0x5b, 0x37, 0xd9, 0xec, - 0x99, 0x93, 0xdc, 0x47, 0xb4, 0x00, 0x69, 0xdb, 0x50, 0xba, 0xf6, 0xbe, 0xe9, 0xee, 0x9d, 0x79, - 0xcf, 0x5e, 0xfc, 0xc7, 0xe4, 0x9b, 0xc7, 0x7f, 0x4c, 0x8d, 0x8e, 0xff, 0x98, 0xfe, 0x29, 0xe2, - 0x3f, 0xf8, 0x46, 0xd5, 0x7f, 0x12, 0xe0, 0xca, 0x80, 0xe6, 0xe2, 0xec, 0x2d, 0xdf, 0x85, 0xac, - 0xca, 0x05, 0x13, 0x7b, 0xcb, 0x76, 0xe1, 0xea, 0x24, 0xdb, 0x1b, 0xc2, 0x8e, 0xb3, 0xd3, 0x45, - 0x70, 0x8b, 0x5a, 0x5f, 0xe3, 0xca, 0x21, 0xbf, 0x35, 0xf1, 0x57, 0x72, 0x50, 0xa8, 0x1e, 0xb3, - 0x15, 0xe8, 0x06, 0x9b, 0xe5, 0xd1, 0x13, 0x48, 0x77, 0x2d, 0xf3, 0x50, 0x77, 0xab, 0x91, 0x0f, - 0x6d, 0xfe, 0xbb, 0xd5, 0xe8, 0xe3, 0xda, 0xe6, 0x1c, 0x92, 0xc7, 0x8b, 0x9a, 0x90, 0x79, 0x66, - 0xaa, 0x4a, 0xfb, 0x89, 0xde, 0x76, 0x7b, 0xfe, 0xfb, 0xe7, 0x0b, 0x5a, 0xf6, 0x78, 0xb6, 0x15, - 0x67, 0xdf, 0x6d, 0x04, 0x8f, 0x88, 0xea, 0x90, 0xae, 0x39, 0x4e, 0x97, 0x24, 0x72, 0xdb, 0x71, - 0x67, 0x0c, 0xa1, 0x84, 0xc5, 0x8d, 0x15, 0x75, 0xd9, 0x51, 0x13, 0x66, 0x9f, 0xd2, 0x13, 0x4e, - 0x95, 0xb6, 0xd9, 0xd3, 0x2a, 0xa6, 0xb1, 0xa7, 0xb7, 0xb8, 0xdd, 0xbd, 0x3d, 0x86, 0xcc, 0xa7, - 0x95, 0x86, 0x34, 0x28, 0x00, 0xad, 0x42, 0xba, 0xf1, 0x88, 0x0b, 0x63, 0x6e, 0xd9, 0xad, 0x31, - 0x84, 0x35, 0x1e, 0x49, 0x1e, 0x1b, 0x5a, 0x87, 0xec, 0xea, 0x17, 0x3d, 0x0b, 0x73, 0x29, 0x53, - 0x43, 0x23, 0x0f, 0xfa, 0xa5, 0x50, 0x2e, 0x29, 0xc8, 0x8c, 0x1a, 0x90, 0x7f, 0x65, 0x5a, 0x07, - 0x6d, 0x53, 0x71, 0x6b, 0x38, 0x4d, 0xc5, 0x7d, 0x63, 0x0c, 0x71, 0x2e, 0xa3, 0xd4, 0x27, 0x02, - 0x7d, 0x1b, 0x0a, 0xa4, 0x31, 0x9a, 0xca, 0x6e, 0xdb, 0x2d, 0x64, 0x9a, 0x4a, 0x7d, 0x77, 0x0c, - 0xa9, 0x1e, 0xa7, 0xbb, 0x05, 0xd2, 0x27, 0x6a, 0x41, 0x82, 0x5c, 0xa8, 0x13, 0x20, 0x04, 0xa9, - 0x2e, 0x69, 0x6f, 0x81, 0xc6, 0x06, 0xd1, 0xdf, 0xe8, 0x3d, 0x98, 0x36, 0x4c, 0x0d, 0xbb, 0x23, - 0x24, 0x57, 0x9e, 0x3f, 0x3b, 0x5d, 0x9c, 0xda, 0x32, 0x35, 0xe6, 0x90, 0xf0, 0x5f, 0xd2, 0x14, - 0xc9, 0x54, 0xd7, 0x16, 0x96, 0x20, 0x45, 0xda, 0x9d, 0x18, 0xa6, 0x5d, 0xc5, 0xc6, 0x3b, 0x96, - 0xce, 0xa5, 0xb9, 0x8f, 0x0b, 0xff, 0x22, 0x01, 0x89, 0xc6, 0x23, 0xe2, 0x72, 0xef, 0xf6, 0xd4, - 0x03, 0xec, 0xf0, 0x74, 0xfe, 0x44, 0x5d, 0x71, 0x0b, 0xef, 0xe9, 0xcc, 0x33, 0xca, 0x48, 0xfc, - 0x09, 0xbd, 0x03, 0xa0, 0xa8, 0x2a, 0xb6, 0x6d, 0xd9, 0x3d, 0xf9, 0x96, 0x91, 0x32, 0x8c, 0xb2, - 0x81, 0x4f, 0x08, 0x9b, 0x8d, 0x55, 0x0b, 0x3b, 0x6e, 0x60, 0x13, 0x7b, 0x22, 0x6c, 0x0e, 0xee, - 0x74, 0x65, 0xc7, 0x3c, 0xc0, 0x06, 0xed, 0x27, 0x19, 0x62, 0x6a, 0x3a, 0xdd, 0x26, 0x21, 0x10, - 0x2b, 0x89, 0x0d, 0xcd, 0x37, 0x69, 0x19, 0xc9, 0x7b, 0x26, 0x22, 0x2d, 0xdc, 0xd2, 0xf9, 0x11, - 0xae, 0x8c, 0xc4, 0x9f, 0x88, 0x96, 0x94, 0x9e, 0xb3, 0x4f, 0x5b, 0x22, 0x23, 0xd1, 0xdf, 0xe8, - 0x36, 0x14, 0x58, 0x2c, 0xa4, 0x8c, 0x0d, 0x55, 0xa6, 0xc6, 0x35, 0x43, 0x93, 0x73, 0x8c, 0x5c, - 0x35, 0x54, 0x62, 0x4a, 0xd1, 0x23, 0xe0, 0x04, 0xf9, 0xa0, 0x63, 0x13, 0x9d, 0x02, 0xc9, 0x55, - 0x2e, 0x9c, 0x9d, 0x2e, 0x66, 0x1b, 0x34, 0x61, 0x63, 0xb3, 0x51, 0x5f, 0x93, 0xb2, 0x2c, 0xd7, - 0x46, 0xc7, 0xae, 0x6b, 0x0b, 0xbf, 0x21, 0x40, 0xf2, 0x69, 0xa5, 0x71, 0x61, 0x95, 0xb9, 0x05, - 0x4d, 0x06, 0x0a, 0x7a, 0x07, 0x0a, 0xbb, 0x7a, 0xbb, 0xad, 0x1b, 0x2d, 0xe2, 0x05, 0x7d, 0x17, - 0xab, 0xae, 0xc2, 0xf2, 0x9c, 0xbc, 0xcd, 0xa8, 0x68, 0x09, 0xb2, 0xaa, 0x85, 0x35, 0x6c, 0x38, - 0xba, 0xd2, 0xb6, 0xb9, 0xe6, 0x82, 0xa4, 0x85, 0x5f, 0x16, 0x60, 0x92, 0x8e, 0x00, 0xf4, 0x36, - 0x64, 0x54, 0xd3, 0x70, 0x14, 0xdd, 0xe0, 0xa6, 0x2c, 0x23, 0xf9, 0x84, 0xa1, 0xc5, 0xbb, 0x0e, - 0x33, 0x8a, 0xaa, 0x9a, 0x3d, 0xc3, 0x91, 0x0d, 0xa5, 0x83, 0x79, 0x31, 0xb3, 0x9c, 0xb6, 0xa5, - 0x74, 0x30, 0x5a, 0x04, 0xf7, 0xd1, 0x3b, 0xd0, 0x98, 0x91, 0x80, 0x93, 0x36, 0xf0, 0xc9, 0xc2, - 0x1f, 0x0b, 0x90, 0x76, 0xc7, 0x0c, 0x29, 0x46, 0x0b, 0x1b, 0xd8, 0x52, 0x1c, 0xd3, 0x2b, 0x86, - 0x47, 0xe8, 0x9f, 0x2a, 0x33, 0xfe, 0x54, 0x39, 0x0f, 0x93, 0x0e, 0x19, 0x16, 0xbc, 0x04, 0xec, - 0x81, 0x2e, 0x47, 0xb7, 0x95, 0x16, 0x5b, 0x8d, 0xcb, 0x48, 0xec, 0x81, 0x54, 0x86, 0x87, 0xd4, - 0x32, 0x8d, 0xf0, 0x27, 0x52, 0x52, 0x16, 0xf8, 0xb9, 0x8b, 0x5b, 0xba, 0x41, 0xfb, 0x52, 0x52, - 0x02, 0x4a, 0x2a, 0x13, 0x0a, 0x7a, 0x0b, 0x32, 0x2c, 0x03, 0x36, 0x34, 0xda, 0xa1, 0x92, 0x52, - 0x9a, 0x12, 0xaa, 0x86, 0xb6, 0x80, 0x21, 0xe3, 0x0d, 0x4e, 0xd2, 0x6c, 0x3d, 0xdb, 0x53, 0x24, - 0xfd, 0x8d, 0xde, 0x87, 0xf9, 0xd7, 0x3d, 0xa5, 0xad, 0xef, 0xd1, 0x85, 0x36, 0x92, 0x8d, 0xe9, - 0x8c, 0xd5, 0x04, 0x79, 0x69, 0x54, 0x02, 0x55, 0x9d, 0x3b, 0x96, 0x93, 0xfe, 0x58, 0x16, 0x7f, - 0x5f, 0x80, 0x59, 0x16, 0xd5, 0xc3, 0x82, 0x51, 0xe3, 0xf3, 0x43, 0x3e, 0x86, 0x8c, 0xa6, 0x38, - 0x0a, 0x3b, 0xa2, 0x99, 0x18, 0x79, 0x44, 0xd3, 0x3b, 0x4a, 0xa0, 0x38, 0x0a, 0x3d, 0xa6, 0x89, - 0x20, 0x45, 0x7e, 0xb3, 0xd3, 0xac, 0x12, 0xfd, 0x2d, 0x7e, 0x06, 0x28, 0x58, 0xd0, 0x38, 0x3d, - 0xb2, 0x7b, 0x70, 0x89, 0xe8, 0xba, 0x6a, 0xa8, 0xd6, 0x49, 0xd7, 0xd1, 0x4d, 0xe3, 0x39, 0xfd, - 0x6b, 0xa3, 0x62, 0x60, 0x5f, 0x8a, 0x6e, 0x47, 0x89, 0x7f, 0x38, 0x05, 0xb9, 0xea, 0x71, 0xd7, - 0xb4, 0x62, 0x5d, 0xc5, 0x2a, 0xc3, 0x34, 0x07, 0xfa, 0x23, 0xf6, 0x85, 0xfb, 0x8c, 0xb9, 0xbb, - 0xe5, 0xca, 0x19, 0x51, 0x19, 0x80, 0x05, 0x8c, 0xd2, 0xa0, 0xa0, 0xe4, 0x05, 0x76, 0xca, 0x28, - 0x1b, 0xa1, 0xa2, 0x2d, 0xc8, 0x76, 0x0e, 0x55, 0x55, 0xde, 0xd3, 0xdb, 0x0e, 0x8f, 0xbb, 0x8b, - 0x0e, 0x11, 0xdf, 0x7c, 0x59, 0xa9, 0x3c, 0xa1, 0x99, 0x58, 0x08, 0x9c, 0xff, 0x2c, 0x01, 0x91, - 0xc0, 0x7e, 0xa3, 0x77, 0x81, 0x1f, 0xa9, 0x91, 0x6d, 0xf7, 0x94, 0x5c, 0x39, 0x77, 0x76, 0xba, - 0x98, 0x91, 0x28, 0xb5, 0xd1, 0x68, 0x4a, 0x19, 0x96, 0xa1, 0x61, 0x3b, 0xe8, 0x06, 0xe4, 0xcc, - 0x8e, 0xee, 0xc8, 0xae, 0x93, 0xc4, 0x3d, 0xca, 0x19, 0x42, 0x74, 0x9d, 0x28, 0xd4, 0x84, 0x3b, - 0xd8, 0xa0, 0xbd, 0x9d, 0xd4, 0x53, 0xde, 0x65, 0x8b, 0x8f, 0x0e, 0x1b, 0xd1, 0xb2, 0xd9, 0x75, - 0xf4, 0x8e, 0xfe, 0x05, 0xdd, 0x99, 0xe6, 0x9b, 0x46, 0x37, 0x58, 0x76, 0x52, 0xbf, 0x32, 0x5d, - 0x95, 0xe4, 0x79, 0x9f, 0x07, 0xb2, 0xa2, 0xbf, 0x2d, 0xc0, 0x65, 0xae, 0x48, 0x79, 0x97, 0xc6, - 0xb8, 0x2b, 0x6d, 0xdd, 0x39, 0x91, 0x0f, 0x0e, 0x4b, 0x69, 0xea, 0xb7, 0xfe, 0x6c, 0x64, 0x83, - 0x04, 0xfa, 0xc1, 0xb2, 0xdb, 0x2c, 0x27, 0xcf, 0x38, 0xf3, 0xc6, 0x61, 0xd5, 0x70, 0xac, 0x93, - 0xf2, 0x95, 0xb3, 0xd3, 0xc5, 0xb9, 0xc1, 0xd4, 0x97, 0xd2, 0x9c, 0x3d, 0xc8, 0x82, 0x6a, 0x00, - 0xd8, 0xeb, 0x87, 0x74, 0xc6, 0x88, 0xf6, 0x3f, 0x22, 0x3b, 0xac, 0x14, 0xe0, 0x45, 0x77, 0xa1, - 0xc8, 0x4f, 0xb6, 0xec, 0xe9, 0x6d, 0x2c, 0xdb, 0xfa, 0x17, 0x98, 0xce, 0x2d, 0x49, 0x29, 0xcf, - 0xe8, 0x44, 0x44, 0x43, 0xff, 0x02, 0x2f, 0x7c, 0x17, 0x4a, 0xc3, 0x4a, 0x1f, 0x1c, 0x02, 0x19, - 0xb6, 0x23, 0xfb, 0x51, 0x78, 0x39, 0x66, 0x8c, 0xae, 0xca, 0x97, 0x64, 0x3e, 0x4e, 0x7c, 0x24, - 0x88, 0xff, 0x2c, 0x01, 0xb9, 0x72, 0xaf, 0x7d, 0xf0, 0xbc, 0xdb, 0xe8, 0x75, 0x3a, 0x8a, 0x75, - 0x42, 0xcc, 0x20, 0x33, 0x14, 0xa4, 0x80, 0x02, 0x33, 0x83, 0xd4, 0x12, 0xe8, 0x5f, 0x60, 0x32, - 0x39, 0x05, 0x4f, 0x65, 0xb3, 0x18, 0x7e, 0x5a, 0x87, 0xc0, 0x51, 0x6b, 0xf3, 0xc8, 0x46, 0x1f, - 0x41, 0x29, 0x90, 0x91, 0xae, 0x9d, 0xc8, 0xd8, 0x70, 0x2c, 0x1d, 0xb3, 0xf5, 0xbf, 0xa4, 0x14, - 0x88, 0x97, 0xa9, 0x93, 0xe4, 0x2a, 0x4b, 0x45, 0x4d, 0x98, 0x21, 0x19, 0x4f, 0x64, 0x3a, 0x85, - 0xb8, 0xeb, 0xb3, 0x0f, 0x22, 0xaa, 0x15, 0x2a, 0xf7, 0x32, 0xd5, 0x4f, 0x85, 0xf2, 0xd0, 0x9f, - 0x52, 0x16, 0xfb, 0x94, 0x85, 0x4f, 0xa1, 0xd8, 0x9f, 0x21, 0xa8, 0xcb, 0x14, 0xd3, 0xe5, 0x7c, - 0x50, 0x97, 0xc9, 0x80, 0x9e, 0xd6, 0x53, 0xe9, 0x54, 0x71, 0x52, 0xfc, 0xf3, 0x24, 0xe4, 0xdd, - 0x6e, 0x16, 0x27, 0xd0, 0x29, 0xc3, 0x24, 0xe9, 0x14, 0x6e, 0x8c, 0xc7, 0xed, 0x11, 0xbd, 0x9b, - 0x47, 0x8d, 0x93, 0xce, 0xe2, 0x82, 0x64, 0xca, 0x1a, 0x87, 0xc1, 0x59, 0xf8, 0xe5, 0x04, 0xa4, - 0x28, 0xb6, 0x78, 0x00, 0x29, 0x3a, 0x51, 0x08, 0xe3, 0x4c, 0x14, 0x34, 0xab, 0x37, 0x9d, 0x25, - 0x02, 0xae, 0x29, 0xf1, 0xf9, 0xf6, 0x95, 0x0f, 0x1e, 0x3c, 0xa4, 0xc6, 0x66, 0x46, 0xe2, 0x4f, - 0xa8, 0x4c, 0xc3, 0x8e, 0x4c, 0xcb, 0xc1, 0x1a, 0xf7, 0xe9, 0x97, 0xce, 0x6b, 0x5f, 0x77, 0x52, - 0x72, 0xf9, 0xd0, 0x55, 0x48, 0x12, 0x2b, 0x36, 0xcd, 0x82, 0x14, 0xce, 0x4e, 0x17, 0x93, 0xc4, - 0x7e, 0x11, 0x1a, 0x5a, 0x81, 0x6c, 0xd8, 0x64, 0x10, 0x0f, 0x8e, 0x1a, 0xc6, 0xc0, 0x70, 0x87, - 0xb6, 0x37, 0xb4, 0x18, 0x9e, 0xe5, 0x6d, 0xfc, 0x17, 0x29, 0xc8, 0xd5, 0x3b, 0x71, 0x4f, 0x29, - 0xab, 0xe1, 0x16, 0x8e, 0x02, 0x42, 0xa1, 0x97, 0x46, 0x34, 0x70, 0x68, 0x06, 0x4f, 0x5e, 0x6c, - 0x06, 0xaf, 0x13, 0x4f, 0x99, 0x5f, 0xb3, 0x90, 0x1c, 0x82, 0x79, 0xc2, 0xef, 0xa7, 0x7e, 0x8a, - 0x44, 0x78, 0xfc, 0x73, 0x14, 0x34, 0xd0, 0xe4, 0x53, 0xea, 0x90, 0xb3, 0x5e, 0x36, 0x35, 0x7e, - 0x2f, 0x9b, 0xc6, 0x86, 0x46, 0x27, 0xb5, 0xb0, 0x45, 0x9d, 0x7e, 0x73, 0x8b, 0xba, 0xe0, 0xf0, - 0xce, 0xfa, 0x31, 0x24, 0x35, 0xdd, 0x6d, 0x9c, 0xf1, 0xa7, 0x6a, 0xc2, 0x74, 0x4e, 0xaf, 0x4d, - 0x05, 0x7b, 0x2d, 0xeb, 0x25, 0x0b, 0x75, 0x00, 0x5f, 0x37, 0x68, 0x09, 0xa6, 0xcc, 0xb6, 0xe6, - 0x1e, 0x24, 0xc9, 0x95, 0x33, 0x67, 0xa7, 0x8b, 0x93, 0xcf, 0xdb, 0x5a, 0x7d, 0x4d, 0x9a, 0x34, - 0xdb, 0x5a, 0x5d, 0xa3, 0x77, 0x5c, 0xe0, 0x23, 0xd9, 0x8b, 0x32, 0x9b, 0x91, 0xa6, 0x0d, 0x7c, - 0xb4, 0x86, 0x6d, 0x95, 0x77, 0xb8, 0xdf, 0x16, 0x20, 0xef, 0xea, 0x3e, 0x5e, 0xa3, 0x92, 0xd6, - 0x3b, 0x7c, 0x90, 0x25, 0x2f, 0x36, 0xc8, 0x5c, 0x3e, 0x7e, 0xbc, 0xf6, 0x57, 0x05, 0x1e, 0x37, - 0xdc, 0x50, 0x15, 0x87, 0x38, 0x15, 0x31, 0x0e, 0x8c, 0x7b, 0x50, 0xb4, 0x14, 0x43, 0x33, 0x3b, - 0xfa, 0x17, 0x98, 0x2d, 0x84, 0xda, 0x7c, 0xd7, 0xb2, 0xe0, 0xd1, 0xe9, 0xaa, 0x9f, 0x2d, 0xfe, - 0x71, 0x82, 0xc7, 0x18, 0x7b, 0xc5, 0x88, 0x53, 0x5d, 0xdf, 0x81, 0xd9, 0xfe, 0x0b, 0x47, 0xdc, - 0xd1, 0xfa, 0x5e, 0x84, 0xbc, 0xa8, 0x82, 0xb0, 0x58, 0x41, 0x37, 0x70, 0xbd, 0xef, 0xf2, 0x11, - 0x1b, 0x55, 0x20, 0x1b, 0xbc, 0xc7, 0x24, 0x39, 0xf6, 0x3d, 0x26, 0x60, 0x79, 0xb7, 0x97, 0x2c, - 0xfc, 0x3c, 0x4c, 0xd2, 0xe4, 0x37, 0x30, 0xd1, 0xbc, 0x35, 0xff, 0x2c, 0x01, 0x37, 0x69, 0xe9, - 0x5f, 0x62, 0x4b, 0xdf, 0x3b, 0xd9, 0xb6, 0x4c, 0x07, 0xab, 0x0e, 0xd6, 0xfc, 0x53, 0x20, 0xb1, - 0xda, 0xbd, 0x4c, 0xd7, 0x7d, 0xc1, 0x85, 0xe2, 0xc5, 0x3c, 0x2e, 0xb4, 0x01, 0x05, 0x1e, 0x19, - 0xa0, 0xb4, 0xf5, 0x43, 0x2c, 0x2b, 0xce, 0x45, 0x66, 0xb7, 0x1c, 0xe3, 0x5d, 0x25, 0xac, 0xab, - 0x0e, 0xd2, 0x20, 0xc3, 0x85, 0xe9, 0x1a, 0xbf, 0x7e, 0xe7, 0xe9, 0x4f, 0xb7, 0xa0, 0x98, 0x66, - 0xe1, 0x09, 0xf5, 0x35, 0x29, 0xcd, 0x24, 0xd7, 0x35, 0xf1, 0xbf, 0x08, 0x70, 0xeb, 0x1c, 0x15, - 0xc7, 0xd9, 0x75, 0x17, 0x20, 0x7d, 0x48, 0x5e, 0xa4, 0x73, 0x1d, 0xa7, 0x25, 0xef, 0x19, 0x6d, - 0x42, 0x6e, 0x4f, 0xd1, 0xdb, 0x7e, 0x97, 0x1e, 0x1e, 0x5e, 0x18, 0x1d, 0xe9, 0x3a, 0xc3, 0xd8, - 0x59, 0x1f, 0x16, 0x7f, 0x33, 0x01, 0xb3, 0xab, 0x9a, 0xd6, 0x68, 0x70, 0x1b, 0x18, 0x5f, 0x4f, - 0x71, 0x41, 0x66, 0xc2, 0x07, 0x99, 0xe8, 0x3d, 0x40, 0x9a, 0x6e, 0xb3, 0x6b, 0x3e, 0xec, 0x7d, - 0x45, 0x33, 0x8f, 0xfc, 0xb0, 0x8a, 0x59, 0x37, 0xa5, 0xe1, 0x26, 0xa0, 0x06, 0x50, 0xb4, 0x23, - 0xdb, 0x8e, 0xe2, 0xed, 0x1b, 0xdd, 0x1a, 0xeb, 0xb8, 0x16, 0x83, 0x41, 0xde, 0xa3, 0x94, 0x21, - 0x72, 0xe8, 0x4f, 0xe2, 0xb7, 0xeb, 0xa4, 0xea, 0x8e, 0xac, 0xd8, 0xee, 0xd9, 0x1c, 0x76, 0xc1, - 0x48, 0x9e, 0xd1, 0x57, 0x6d, 0x76, 0xe4, 0x86, 0x1d, 0x26, 0xf0, 0x55, 0x13, 0x27, 0x24, 0xfe, - 0x47, 0x02, 0xe4, 0x25, 0xbc, 0x67, 0x61, 0x3b, 0xd6, 0x45, 0x81, 0x27, 0x30, 0x63, 0x31, 0xa9, - 0xf2, 0x9e, 0x65, 0x76, 0x2e, 0x32, 0xae, 0xb2, 0x9c, 0xf1, 0x89, 0x65, 0x76, 0xb8, 0x61, 0x79, - 0x09, 0x05, 0xaf, 0x8c, 0x71, 0x56, 0xfe, 0xf7, 0xe9, 0x51, 0x64, 0x26, 0x38, 0xee, 0xf8, 0x86, - 0x78, 0x35, 0x40, 0x37, 0xaa, 0x82, 0x05, 0x8d, 0x53, 0x0d, 0xff, 0x43, 0x80, 0x7c, 0xa3, 0xb7, - 0xcb, 0xee, 0x91, 0x8a, 0x4f, 0x03, 0x55, 0xc8, 0xb4, 0xf1, 0x9e, 0x23, 0xbf, 0x51, 0x18, 0x7c, - 0x9a, 0xb0, 0xd2, 0x43, 0x00, 0x4f, 0x01, 0x2c, 0x7a, 0x7c, 0x8d, 0xca, 0x49, 0x5e, 0x50, 0x4e, - 0x86, 0xf2, 0x12, 0xb2, 0xf8, 0xcf, 0x13, 0x50, 0xf0, 0xaa, 0x19, 0xa7, 0x95, 0x7c, 0x15, 0xb2, - 0x0e, 0xc9, 0x8b, 0x58, 0x87, 0x59, 0x1e, 0xd2, 0x11, 0x6d, 0x21, 0x96, 0x61, 0x8e, 0x3a, 0x2e, - 0xb2, 0xd2, 0xed, 0xb6, 0x75, 0x17, 0xee, 0x52, 0xfb, 0x93, 0x92, 0x66, 0x69, 0xd2, 0x2a, 0x4b, - 0xa1, 0x40, 0x97, 0xf4, 0xb9, 0x3d, 0x0b, 0xe3, 0x2f, 0xb0, 0x4c, 0x91, 0xd7, 0x45, 0x42, 0x56, - 0xb2, 0x8c, 0xb1, 0x41, 0xf8, 0x78, 0x9f, 0x7b, 0x05, 0xb3, 0x54, 0xa7, 0x71, 0x1f, 0xbb, 0x15, - 0xff, 0x41, 0x02, 0x50, 0x50, 0xf2, 0x57, 0xd7, 0x16, 0x89, 0xf8, 0xda, 0xe2, 0x5d, 0x40, 0x2c, - 0x68, 0xd1, 0x96, 0xbb, 0xd8, 0x92, 0x6d, 0xac, 0x9a, 0xfc, 0x8e, 0x23, 0x41, 0x2a, 0xf2, 0x94, - 0x6d, 0x6c, 0x35, 0x28, 0x1d, 0x3d, 0x06, 0xf0, 0x3d, 0x32, 0x3e, 0x61, 0x8c, 0x74, 0xc8, 0xa4, - 0x8c, 0xe7, 0x8a, 0x89, 0xdf, 0x5b, 0x80, 0x19, 0xae, 0xc3, 0x1d, 0x43, 0x37, 0x0d, 0xf4, 0x00, - 0x92, 0x2d, 0xbe, 0xf4, 0x9f, 0x8d, 0x5c, 0xa6, 0xf3, 0x2f, 0x71, 0xab, 0x4d, 0x48, 0x24, 0x2f, - 0x61, 0xe9, 0xf6, 0x9c, 0x08, 0xc7, 0xc8, 0x0f, 0xc4, 0x0e, 0xb2, 0x74, 0x7b, 0x0e, 0x6a, 0x40, - 0x41, 0xf5, 0x2f, 0xa9, 0x92, 0x09, 0x7b, 0x72, 0x28, 0x88, 0x8a, 0xbc, 0x16, 0xac, 0x36, 0x21, - 0xe5, 0xd5, 0x50, 0x02, 0xaa, 0x04, 0xef, 0x46, 0x4a, 0x0d, 0x44, 0x79, 0xf9, 0xe7, 0x7b, 0xc3, - 0xf7, 0x32, 0xd5, 0x26, 0x02, 0x57, 0x28, 0xa1, 0x8f, 0x61, 0x4a, 0xa3, 0xb7, 0xf0, 0xf0, 0x1e, - 0x1d, 0xd5, 0xe9, 0x42, 0x97, 0x1d, 0xd5, 0x26, 0x24, 0xce, 0x81, 0xd6, 0x61, 0x86, 0xfd, 0x62, - 0x6e, 0x0a, 0x47, 0x96, 0xb7, 0x86, 0x4b, 0x08, 0x4c, 0x04, 0xb5, 0x09, 0x29, 0xab, 0xf9, 0x54, - 0xf4, 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0x62, 0xcb, 0x6b, 0x43, 0xae, 0xd4, 0xf0, 0x99, 0x69, 0x6e, - 0xf4, 0x98, 0x5d, 0xcb, 0xe8, 0x1c, 0xbb, 0xcb, 0x7c, 0x51, 0xc5, 0x0f, 0x1d, 0xdd, 0x26, 0xc5, - 0xc7, 0x94, 0x80, 0x9e, 0x42, 0x56, 0x21, 0xfe, 0x9e, 0x4c, 0x0f, 0x3f, 0xd2, 0x75, 0xbd, 0xe8, - 0xcd, 0xf5, 0x81, 0xc3, 0xaa, 0x35, 0x7a, 0x3e, 0xdc, 0x25, 0xfa, 0x82, 0x3a, 0xd8, 0x6a, 0xe1, - 0x52, 0x76, 0xb4, 0xa0, 0x60, 0x6c, 0x97, 0x27, 0x88, 0x12, 0x89, 0xdf, 0xe7, 0x9d, 0x3c, 0xa6, - 0x95, 0x9a, 0x19, 0xba, 0x9d, 0x1b, 0x71, 0x6c, 0xa7, 0x36, 0x21, 0xcd, 0xec, 0x07, 0xc8, 0x68, - 0x19, 0x12, 0x2d, 0xb5, 0x94, 0x1b, 0x3a, 0x42, 0xbc, 0xa3, 0x29, 0xb5, 0x09, 0x29, 0xd1, 0x52, - 0xd1, 0xa7, 0x90, 0x66, 0xe7, 0x0c, 0x8e, 0x8d, 0x52, 0x7e, 0xa8, 0x9d, 0x08, 0x9f, 0xd6, 0xa8, - 0x4d, 0x48, 0xf4, 0x68, 0x03, 0x79, 0xdf, 0x36, 0xe4, 0x2d, 0x16, 0x1c, 0xe7, 0x86, 0xb2, 0x16, - 0x87, 0x6e, 0x71, 0x47, 0x45, 0xb3, 0xd6, 0xa8, 0xe7, 0x1f, 0xa0, 0xa3, 0xef, 0xc0, 0x7c, 0x58, - 0x22, 0xef, 0x69, 0xb3, 0x43, 0xb7, 0x6b, 0x87, 0x46, 0x56, 0xd6, 0x26, 0x24, 0x64, 0x0d, 0x24, - 0xa2, 0x0f, 0x61, 0x92, 0xb5, 0x1a, 0xa2, 0x22, 0xa3, 0xa2, 0x36, 0xfa, 0x1a, 0x8c, 0xe5, 0x27, - 0x9d, 0xdf, 0xe1, 0x11, 0x62, 0x72, 0xdb, 0x6c, 0x95, 0xe6, 0x86, 0x76, 0xfe, 0xc1, 0x58, 0x37, - 0xd2, 0xf9, 0x1d, 0x9f, 0x4a, 0xda, 0xdd, 0x62, 0x29, 0x3c, 0xac, 0x68, 0x7e, 0x68, 0xbb, 0x47, - 0x04, 0x8e, 0xd5, 0x68, 0xec, 0xbe, 0x4f, 0x26, 0x45, 0xb3, 0xd8, 0x2d, 0x30, 0x32, 0x1d, 0x53, - 0x97, 0x86, 0x16, 0x6d, 0xf0, 0x9a, 0x9c, 0x1a, 0xf5, 0x91, 0x3c, 0x2a, 0x7a, 0x09, 0x45, 0x7e, - 0x57, 0x83, 0xbf, 0xa7, 0x70, 0x99, 0xca, 0xbb, 0x17, 0x69, 0xba, 0xa2, 0x62, 0x72, 0x6a, 0x13, - 0x52, 0x41, 0x0d, 0xa7, 0xa0, 0xcf, 0x60, 0x96, 0xca, 0x93, 0x55, 0xff, 0x7a, 0x8d, 0x52, 0x69, - 0xe0, 0xb2, 0x86, 0xe1, 0x37, 0x71, 0xb8, 0x92, 0x8b, 0x6a, 0x5f, 0x12, 0xe9, 0xc6, 0xba, 0xa1, - 0x3b, 0xd4, 0xca, 0x2e, 0x0c, 0xed, 0xc6, 0xe1, 0x4b, 0xfd, 0x48, 0x37, 0xd6, 0x19, 0x85, 0x74, - 0x63, 0x87, 0xc7, 0x9c, 0xf1, 0xe6, 0x78, 0x7b, 0x68, 0x37, 0x8e, 0x0a, 0x4e, 0x23, 0xdd, 0xd8, - 0x09, 0xd2, 0x49, 0x37, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, 0xbb, 0xf1, 0xd0, 0x43, 0xc8, - 0xa4, 0x1b, 0x2b, 0x03, 0x89, 0x68, 0x0d, 0x80, 0xb9, 0x33, 0x74, 0x52, 0xbc, 0x36, 0x74, 0x32, - 0xe8, 0x8f, 0x3d, 0x23, 0x93, 0x41, 0xdb, 0xa5, 0x11, 0x43, 0x46, 0xc1, 0x92, 0x4c, 0xb7, 0x50, - 0x4b, 0x8b, 0x43, 0x0d, 0xd9, 0xc0, 0x76, 0x27, 0x31, 0x64, 0x47, 0x1e, 0x91, 0xcc, 0x2a, 0x6c, - 0xcd, 0xb7, 0xb4, 0x34, 0xdc, 0x2c, 0x07, 0xb7, 0x7e, 0xa8, 0x59, 0xa6, 0x04, 0xb4, 0x0a, 0x19, - 0x32, 0xe7, 0x9f, 0x50, 0x33, 0x74, 0x7d, 0xa8, 0x4f, 0xda, 0x77, 0x38, 0xa5, 0x36, 0x21, 0xa5, - 0x5f, 0x73, 0x12, 0x79, 0x3d, 0x5b, 0x0d, 0x2b, 0x89, 0x43, 0x5f, 0x1f, 0x5a, 0x39, 0x25, 0xaf, - 0x67, 0x1c, 0x48, 0x85, 0x4b, 0xac, 0xad, 0xf8, 0x79, 0x60, 0x8b, 0x1f, 0x61, 0x2d, 0xdd, 0xa0, - 0xa2, 0x86, 0x2e, 0x2b, 0x45, 0x1e, 0x53, 0xae, 0x4d, 0x48, 0x73, 0xca, 0x60, 0x2a, 0x19, 0xf0, - 0x7c, 0xea, 0x61, 0x8b, 0x51, 0xa5, 0x9b, 0x43, 0x07, 0x7c, 0xc4, 0x1a, 0x1e, 0x19, 0xf0, 0x4a, - 0x80, 0xcc, 0x26, 0x20, 0x4d, 0xb6, 0x6d, 0xb6, 0xe1, 0x7e, 0x6b, 0xc4, 0x04, 0xd4, 0xb7, 0x0a, - 0xc0, 0x26, 0x20, 0xad, 0xc1, 0x38, 0x89, 0x20, 0xb5, 0x8d, 0x15, 0x8b, 0x9b, 0xd9, 0xdb, 0x43, - 0x05, 0x0d, 0xdc, 0x9f, 0x47, 0x04, 0xa9, 0x1e, 0x91, 0x38, 0x3c, 0x96, 0x7b, 0xc9, 0x0b, 0x77, - 0x18, 0xef, 0x0c, 0x75, 0x78, 0x22, 0x6f, 0xa1, 0x21, 0x0e, 0x8f, 0x15, 0x4a, 0x40, 0x3f, 0x07, - 0xd3, 0x1c, 0xbe, 0x95, 0xee, 0x8e, 0x70, 0x63, 0x83, 0x88, 0x9b, 0x8c, 0x6b, 0xce, 0xc3, 0xac, - 0x2c, 0x83, 0x8d, 0xac, 0x7a, 0xf7, 0x46, 0x58, 0xd9, 0x01, 0xe4, 0xca, 0xac, 0xac, 0x4f, 0x26, - 0x56, 0x96, 0xf5, 0x53, 0x3e, 0xd7, 0xdd, 0x1f, 0x6a, 0x65, 0x07, 0x8f, 0xbe, 0x10, 0x2b, 0xfb, - 0xda, 0xa7, 0x92, 0x9a, 0xd9, 0x0c, 0x3e, 0x95, 0xbe, 0x31, 0xb4, 0x66, 0x61, 0x1c, 0x49, 0x6a, - 0xc6, 0x79, 0x48, 0xb3, 0x31, 0x97, 0x98, 0x69, 0xfa, 0xdd, 0xe1, 0x87, 0xee, 0xfb, 0x41, 0x47, - 0xcd, 0x5d, 0xa8, 0x64, 0x1a, 0xf6, 0x0c, 0x95, 0xc5, 0x0f, 0x1d, 0x73, 0x4d, 0xbd, 0x37, 0xda, - 0x50, 0x45, 0x9d, 0xa4, 0xf6, 0x0c, 0x55, 0x28, 0x91, 0x16, 0x95, 0x9d, 0x33, 0xa3, 0xe3, 0x7b, - 0x79, 0xc4, 0xfd, 0x00, 0x7d, 0xa7, 0xfd, 0x68, 0x51, 0x3d, 0xa2, 0x3f, 0x84, 0x7a, 0xec, 0xf2, - 0x8a, 0xd2, 0xca, 0xe8, 0x21, 0x14, 0xbe, 0x3e, 0xc3, 0x1b, 0x42, 0x9c, 0xec, 0xcd, 0x99, 0xae, - 0x87, 0xf1, 0xfe, 0xe8, 0x39, 0xb3, 0xdf, 0xb5, 0x60, 0x73, 0x26, 0xf7, 0x29, 0xfe, 0xa6, 0x00, - 0x4b, 0xac, 0x6c, 0x74, 0x45, 0xef, 0x44, 0xf6, 0xd6, 0x45, 0x03, 0x67, 0x1e, 0x1e, 0xd0, 0x17, - 0x7c, 0x38, 0xac, 0xb8, 0xe7, 0xac, 0xf3, 0xd6, 0x26, 0xa4, 0x77, 0x94, 0x51, 0xf9, 0xca, 0xd3, - 0x7c, 0x5b, 0xd4, 0x3b, 0xe4, 0x59, 0x28, 0x16, 0xd7, 0x53, 0xe9, 0x2b, 0xc5, 0xd2, 0x7a, 0x2a, - 0x7d, 0xb5, 0xb8, 0xb0, 0x9e, 0x4a, 0xbf, 0x55, 0x7c, 0x5b, 0xfc, 0x8b, 0xab, 0x90, 0x73, 0x91, - 0x1f, 0x43, 0x44, 0x0f, 0x83, 0x88, 0xe8, 0xda, 0x30, 0x44, 0xc4, 0xb1, 0x22, 0x87, 0x44, 0x0f, - 0x83, 0x90, 0xe8, 0xda, 0x30, 0x48, 0xe4, 0xf3, 0x10, 0x4c, 0xd4, 0x1c, 0x86, 0x89, 0xee, 0x8d, - 0x81, 0x89, 0x3c, 0x51, 0xfd, 0xa0, 0x68, 0x6d, 0x10, 0x14, 0xdd, 0x1c, 0x0d, 0x8a, 0x3c, 0x51, - 0x01, 0x54, 0xf4, 0xb8, 0x0f, 0x15, 0x5d, 0x1f, 0x81, 0x8a, 0x3c, 0x7e, 0x17, 0x16, 0x6d, 0x44, - 0xc2, 0xa2, 0xdb, 0xe7, 0xc1, 0x22, 0x4f, 0x4e, 0x08, 0x17, 0x7d, 0x10, 0xc2, 0x45, 0x8b, 0x43, - 0x71, 0x91, 0xc7, 0xcd, 0x80, 0xd1, 0x27, 0xfd, 0xc0, 0xe8, 0xfa, 0x08, 0x60, 0xe4, 0xd7, 0x80, - 0x23, 0xa3, 0x5a, 0x14, 0x32, 0xba, 0x75, 0x0e, 0x32, 0xf2, 0xa4, 0x04, 0xa1, 0x51, 0x2d, 0x0a, - 0x1a, 0xdd, 0x3a, 0x07, 0x1a, 0xf5, 0x49, 0x62, 0xd8, 0x68, 0x2b, 0x1a, 0x1b, 0xdd, 0x39, 0x17, - 0x1b, 0x79, 0xd2, 0xc2, 0xe0, 0x68, 0x25, 0x00, 0x8e, 0xde, 0x19, 0x02, 0x8e, 0x3c, 0x56, 0x82, - 0x8e, 0xbe, 0x35, 0x80, 0x8e, 0xc4, 0x51, 0xe8, 0xc8, 0xe3, 0xf5, 0xe0, 0xd1, 0x8b, 0x21, 0xf0, - 0xe8, 0xee, 0xf9, 0xf0, 0xc8, 0x13, 0xd6, 0x87, 0x8f, 0x94, 0x91, 0xf8, 0xe8, 0xbd, 0x31, 0xf1, - 0x91, 0x27, 0x3d, 0x0a, 0x20, 0x7d, 0x14, 0x06, 0x48, 0x4b, 0xc3, 0x01, 0x92, 0x27, 0x86, 0x23, - 0xa4, 0x8d, 0x48, 0x84, 0x74, 0xfb, 0x3c, 0x84, 0xe4, 0x8f, 0x83, 0x20, 0x44, 0xda, 0x8a, 0x86, - 0x48, 0x77, 0xce, 0x85, 0x48, 0x7e, 0xf3, 0x87, 0x30, 0xd2, 0x46, 0x24, 0x46, 0xba, 0x7d, 0x1e, - 0x46, 0xf2, 0x0b, 0x17, 0x04, 0x49, 0xaf, 0x86, 0x82, 0xa4, 0xfb, 0xe3, 0x80, 0x24, 0x4f, 0xe8, - 0x00, 0x4a, 0xfa, 0x7c, 0x38, 0x4a, 0xfa, 0xc6, 0x05, 0xee, 0x2b, 0x8c, 0x84, 0x49, 0xdf, 0x1a, - 0x80, 0x49, 0xe2, 0x28, 0x98, 0xe4, 0xf7, 0x67, 0x17, 0x27, 0x29, 0x23, 0x51, 0xcd, 0x7b, 0x63, - 0xa2, 0x1a, 0xbf, 0xf3, 0x45, 0xc0, 0x9a, 0x6a, 0x04, 0xac, 0xb9, 0x39, 0x1a, 0xd6, 0xf8, 0xe6, - 0xdc, 0xc7, 0x35, 0xb5, 0x28, 0x5c, 0x73, 0xeb, 0x1c, 0x5c, 0xe3, 0x5b, 0xa1, 0x00, 0xb0, 0x79, - 0xdc, 0x07, 0x6c, 0xae, 0x9f, 0x1b, 0xf5, 0x13, 0x40, 0x36, 0xe5, 0x41, 0x64, 0x73, 0x63, 0x24, - 0xb2, 0xf1, 0x24, 0xf8, 0xd0, 0xe6, 0x71, 0x1f, 0xb4, 0xb9, 0x3e, 0x02, 0xda, 0xf8, 0x05, 0xe0, - 0xd8, 0x46, 0x1b, 0x8d, 0x6d, 0x96, 0xc7, 0xc5, 0x36, 0x9e, 0xe0, 0x48, 0x70, 0xb3, 0x15, 0x0d, - 0x6e, 0xee, 0x8c, 0xb9, 0x21, 0x3f, 0x80, 0x6e, 0x6a, 0x51, 0xe8, 0xe6, 0xd6, 0x39, 0xe8, 0x26, - 0x38, 0x87, 0x78, 0xf0, 0xa6, 0x16, 0x05, 0x6f, 0x6e, 0x9d, 0x03, 0x6f, 0x7c, 0x49, 0x01, 0x7c, - 0xd3, 0x1c, 0x86, 0x6f, 0xee, 0x8d, 0x81, 0x6f, 0x7c, 0xe7, 0xa5, 0x0f, 0xe0, 0x7c, 0xda, 0x0f, - 0x70, 0xc4, 0x51, 0x00, 0xc7, 0x1f, 0x91, 0x2e, 0xc2, 0xd9, 0x8a, 0x46, 0x38, 0x77, 0xce, 0x45, - 0x38, 0x41, 0x23, 0x19, 0x80, 0x38, 0x1b, 0x91, 0x10, 0xe7, 0xf6, 0x79, 0x10, 0xc7, 0x37, 0x92, - 0x41, 0x8c, 0xf3, 0x69, 0x3f, 0xc6, 0x11, 0x47, 0x61, 0x1c, 0xbf, 0x72, 0x2e, 0xc8, 0xa9, 0x45, - 0x81, 0x9c, 0x5b, 0xe7, 0x80, 0x1c, 0xbf, 0xf1, 0x02, 0x28, 0x47, 0x19, 0x89, 0x72, 0xde, 0x1b, - 0x13, 0xe5, 0xf4, 0x19, 0xae, 0x30, 0xcc, 0xa9, 0x45, 0xc1, 0x9c, 0x5b, 0xe7, 0xc0, 0x9c, 0x40, - 0x61, 0x7d, 0x9c, 0xb3, 0x15, 0x8d, 0x73, 0xee, 0x9c, 0x8b, 0x73, 0xfa, 0x46, 0x93, 0x0b, 0x74, - 0x36, 0x22, 0x81, 0xce, 0xed, 0xf3, 0x80, 0x4e, 0xdf, 0xc4, 0xc7, 0x9d, 0x83, 0x5f, 0x19, 0x1f, - 0xe9, 0x7c, 0x74, 0x71, 0xa4, 0xe3, 0xbd, 0x33, 0x16, 0xa8, 0xb3, 0x9e, 0x4a, 0xbf, 0x5d, 0x7c, - 0x47, 0xfc, 0x3b, 0xd3, 0x30, 0x55, 0xf3, 0xe2, 0x5c, 0xfc, 0x52, 0x0a, 0x6f, 0x72, 0x2f, 0x12, - 0x5a, 0x23, 0x23, 0x96, 0xda, 0xbd, 0xf3, 0xaf, 0xba, 0x1b, 0xbc, 0x8c, 0x8d, 0xb3, 0xbe, 0xc1, - 0x01, 0x65, 0xf4, 0x01, 0xe4, 0x7a, 0x36, 0xb6, 0xe4, 0xae, 0xa5, 0x9b, 0x96, 0xee, 0xb0, 0x53, - 0x1c, 0x42, 0xb9, 0xf8, 0xe5, 0xe9, 0xe2, 0xcc, 0x8e, 0x8d, 0xad, 0x6d, 0x4e, 0x97, 0x66, 0x7a, - 0x81, 0x27, 0xf7, 0x43, 0x51, 0x93, 0xe3, 0x7f, 0x28, 0xea, 0x05, 0x14, 0x2d, 0xac, 0x68, 0x21, - 0x0f, 0x84, 0xdd, 0x41, 0x14, 0xdd, 0x67, 0xe8, 0x29, 0x2b, 0x37, 0x27, 0xbd, 0x8b, 0xa8, 0x60, - 0x85, 0x89, 0xe8, 0x01, 0x5c, 0xea, 0x28, 0xc7, 0x34, 0x26, 0x52, 0x76, 0x9d, 0x3a, 0x1a, 0xe7, - 0xc8, 0xbe, 0xc1, 0x84, 0x3a, 0xca, 0x31, 0xfd, 0xea, 0x14, 0x4b, 0xa2, 0x9f, 0x92, 0xb8, 0x05, - 0x79, 0x4d, 0xb7, 0x1d, 0xdd, 0x50, 0x1d, 0x7e, 0xf7, 0x2c, 0xbb, 0xb7, 0x35, 0xe7, 0x52, 0xd9, - 0x05, 0xb3, 0xf7, 0x61, 0x96, 0x07, 0xcb, 0x07, 0xb6, 0x08, 0xf9, 0xfd, 0xad, 0x2c, 0xc1, 0xdb, - 0x15, 0x44, 0x15, 0x28, 0xb4, 0x14, 0x07, 0x1f, 0x29, 0x27, 0xb2, 0x7b, 0x14, 0x2b, 0x4b, 0xaf, - 0x64, 0x7c, 0xeb, 0xec, 0x74, 0x31, 0xf7, 0x94, 0x25, 0x0d, 0x9c, 0xc8, 0xca, 0xb5, 0x02, 0x09, - 0x1a, 0xba, 0x03, 0x05, 0xc5, 0x3e, 0x31, 0x54, 0xaa, 0x1e, 0x6c, 0xd8, 0x3d, 0x9b, 0x42, 0x8a, - 0xb4, 0x94, 0xa7, 0xe4, 0x8a, 0x4b, 0x45, 0xd7, 0x61, 0x86, 0x47, 0x92, 0xb3, 0xcf, 0xd8, 0x14, - 0x68, 0x55, 0xf9, 0x77, 0x13, 0xe8, 0x97, 0x6c, 0xd0, 0x63, 0x58, 0xe0, 0x77, 0xcb, 0x1f, 0x29, - 0x96, 0x26, 0x53, 0xad, 0xfb, 0xfd, 0xb3, 0x48, 0xc5, 0x5e, 0x61, 0x77, 0xc9, 0x93, 0x0c, 0x44, - 0xd5, 0xfe, 0x4d, 0x08, 0x5b, 0x30, 0xab, 0xb6, 0x75, 0x0f, 0x01, 0xb0, 0x9a, 0xcf, 0x0e, 0xb5, - 0xb3, 0x15, 0x9a, 0xd7, 0xdf, 0x22, 0x2d, 0xa8, 0x61, 0x02, 0x6a, 0x00, 0xbd, 0xed, 0x45, 0xee, - 0x9a, 0x6d, 0x5d, 0x3d, 0xa1, 0xce, 0x7f, 0xf8, 0x8e, 0xec, 0x91, 0x37, 0xd5, 0xbf, 0x52, 0x74, - 0x67, 0x9b, 0x72, 0x4a, 0x70, 0xe4, 0xfd, 0x66, 0xf7, 0xdb, 0xae, 0xa7, 0xd2, 0x33, 0xc5, 0xdc, - 0x7a, 0x2a, 0x9d, 0x2f, 0x16, 0xc4, 0xbf, 0x2b, 0x40, 0xa1, 0xaf, 0x2c, 0xa8, 0x06, 0x97, 0x34, - 0x6f, 0xa8, 0xc8, 0xfc, 0xa8, 0x91, 0x6e, 0x1a, 0xfc, 0xda, 0xef, 0xb9, 0x2f, 0x4f, 0x17, 0x0b, - 0x34, 0xf7, 0x53, 0x2f, 0x49, 0x9a, 0xf7, 0x39, 0x7c, 0x2a, 0xfa, 0x08, 0xf2, 0xcc, 0x7d, 0xf4, - 0xbe, 0xcc, 0x46, 0x63, 0xc4, 0xcb, 0xb3, 0x5f, 0x9e, 0x2e, 0xe6, 0xa8, 0xcf, 0xe8, 0x5e, 0xcb, - 0x2b, 0xe5, 0xda, 0xc1, 0x47, 0xf1, 0x37, 0x04, 0x98, 0x09, 0x1d, 0xe7, 0x79, 0xdc, 0xb7, 0x83, - 0x7e, 0x35, 0x1a, 0x77, 0x0e, 0x0b, 0xa8, 0x4b, 0xf3, 0x7e, 0xee, 0x46, 0x27, 0x2e, 0x0e, 0xc7, - 0x2d, 0x74, 0x15, 0xc6, 0x0d, 0xd5, 0x70, 0xd9, 0x3e, 0x4e, 0x7d, 0xff, 0x07, 0x8b, 0x13, 0xe2, - 0x1f, 0xa4, 0x20, 0x17, 0x3e, 0xbc, 0x53, 0xef, 0x2b, 0x57, 0xd4, 0xbc, 0x10, 0xe2, 0x58, 0x1e, - 0x71, 0x6d, 0x61, 0xc6, 0xbf, 0x5f, 0x9f, 0x15, 0x73, 0x69, 0x44, 0x9c, 0x40, 0xb0, 0x9c, 0x3e, - 0xe3, 0xc2, 0xf7, 0x92, 0x9e, 0x7d, 0x5d, 0x86, 0x49, 0x7a, 0x8d, 0x0e, 0x2f, 0x5a, 0xa9, 0xbf, - 0xf7, 0x10, 0x5f, 0x99, 0xa4, 0x4b, 0x2c, 0x1b, 0xb1, 0xc7, 0xcd, 0x37, 0xba, 0xa7, 0xce, 0x1f, - 0x06, 0x17, 0xff, 0x10, 0x1e, 0xbf, 0xa7, 0x70, 0xf2, 0x62, 0xf7, 0x14, 0xa2, 0x5f, 0x82, 0x82, - 0x6a, 0xb6, 0xdb, 0x6c, 0xae, 0x63, 0x16, 0x69, 0xf0, 0xe6, 0x11, 0x2a, 0x82, 0x7f, 0xbb, 0x70, - 0xd9, 0xfb, 0x86, 0xe1, 0xb2, 0xc4, 0xbf, 0x61, 0x18, 0x88, 0xf5, 0xcc, 0x7b, 0xc2, 0x98, 0x21, - 0xeb, 0x0b, 0x3b, 0x9d, 0x7e, 0x93, 0xb0, 0x53, 0x16, 0xaa, 0xcc, 0x7b, 0xce, 0x9f, 0x0a, 0x3c, - 0x30, 0xe4, 0x99, 0x69, 0x1e, 0xf4, 0xbc, 0x70, 0xd1, 0x85, 0xe0, 0xad, 0x81, 0xe9, 0x2f, 0x4f, - 0x17, 0x53, 0x92, 0x77, 0x6d, 0x60, 0x94, 0xe5, 0x4f, 0xfc, 0x74, 0x96, 0xff, 0x3a, 0xcc, 0x74, - 0x2d, 0xbc, 0x87, 0x1d, 0x75, 0x5f, 0x36, 0x7a, 0x1d, 0x7e, 0xaa, 0x24, 0xeb, 0xd2, 0xb6, 0x7a, - 0x1d, 0x74, 0x0f, 0x8a, 0x5e, 0x16, 0x8e, 0xb1, 0xdd, 0x6b, 0xa5, 0x5c, 0x3a, 0x47, 0xe4, 0xe2, - 0xff, 0x11, 0x60, 0x2e, 0x54, 0x27, 0x3e, 0x26, 0xd6, 0x21, 0xeb, 0x9b, 0x03, 0xbb, 0x24, 0x5c, - 0x30, 0x78, 0x32, 0xc8, 0x8c, 0x64, 0xb8, 0xec, 0xbe, 0x96, 0xde, 0x29, 0xef, 0x8b, 0x4d, 0x5c, - 0x50, 0xec, 0x25, 0x5f, 0xce, 0x5a, 0xe0, 0x05, 0xde, 0x20, 0x49, 0x8e, 0x35, 0x48, 0xc4, 0xdf, - 0x16, 0xa0, 0x48, 0x5f, 0xf0, 0x04, 0x63, 0x2d, 0x16, 0xeb, 0xe4, 0x06, 0x25, 0x27, 0xc6, 0x3f, - 0x37, 0x12, 0xfa, 0x0e, 0x46, 0x32, 0xfc, 0x1d, 0x0c, 0xf1, 0x07, 0x02, 0xe4, 0xbd, 0x12, 0xb2, - 0xaf, 0xc1, 0x8d, 0xb8, 0x9c, 0xf2, 0xcd, 0xbe, 0x84, 0xe6, 0xde, 0xc2, 0x31, 0xd6, 0x07, 0xea, - 0x82, 0xb7, 0x70, 0xb0, 0x2f, 0x78, 0xfd, 0x7d, 0xb7, 0xe7, 0x90, 0x22, 0x56, 0xfc, 0xdb, 0x16, - 0xde, 0xe0, 0x08, 0x8d, 0x44, 0x3f, 0xa4, 0x69, 0xb6, 0x0f, 0xd9, 0xc5, 0x27, 0x63, 0x99, 0x2d, - 0xc4, 0xc3, 0xa1, 0x80, 0xaf, 0xc6, 0x69, 0xcd, 0x06, 0xfd, 0xc4, 0x26, 0xfb, 0x6d, 0x8b, 0x4f, - 0x02, 0x0a, 0xa4, 0x8d, 0x4f, 0xb4, 0x34, 0x96, 0x29, 0x75, 0xb5, 0xc4, 0xfa, 0xca, 0x9f, 0x04, - 0x5b, 0xa2, 0x7a, 0x48, 0x50, 0xd8, 0x23, 0x48, 0x1e, 0x2a, 0xed, 0x51, 0x61, 0x60, 0xa1, 0x96, - 0x93, 0x48, 0x6e, 0xf4, 0x24, 0x74, 0x49, 0x45, 0x62, 0x38, 0x62, 0x18, 0x54, 0x69, 0xe8, 0x32, - 0x8b, 0x0f, 0xc3, 0x7d, 0x7d, 0xe4, 0xeb, 0x83, 0x9d, 0xfe, 0xe3, 0xd4, 0x0f, 0x7f, 0xb0, 0x28, - 0x88, 0x1f, 0xc2, 0xd5, 0xa7, 0xa6, 0x6d, 0xeb, 0x5d, 0x82, 0x12, 0xe9, 0x00, 0x22, 0x56, 0xdc, - 0xb3, 0x64, 0xe9, 0x2e, 0x5d, 0x2f, 0x30, 0xd8, 0x88, 0xcf, 0x48, 0xde, 0xb3, 0xf8, 0x87, 0x02, - 0x5c, 0x19, 0xe4, 0x64, 0x0a, 0x89, 0x3a, 0x9c, 0x37, 0xad, 0x9a, 0xfe, 0x55, 0x6c, 0xe7, 0x77, - 0x2c, 0x37, 0x3b, 0xf1, 0x06, 0xf9, 0x3b, 0xe5, 0x8e, 0x42, 0x47, 0x3a, 0x3f, 0x28, 0x9c, 0xe7, - 0xe4, 0x4d, 0x46, 0xf5, 0x07, 0x7d, 0x6a, 0xbc, 0x41, 0xdf, 0x84, 0xc2, 0xba, 0xa9, 0x1b, 0xc4, - 0xe9, 0x74, 0xeb, 0xbb, 0x0a, 0xf9, 0x5d, 0xdd, 0x50, 0xac, 0x13, 0xd9, 0x3d, 0x7f, 0xcd, 0xda, - 0x74, 0x21, 0xaa, 0xb0, 0x2c, 0x87, 0x94, 0x63, 0x1c, 0xfc, 0x51, 0xfc, 0x91, 0x00, 0x45, 0x5f, - 0x2c, 0x37, 0x9e, 0xef, 0x02, 0xa8, 0xed, 0x9e, 0xed, 0x60, 0xcb, 0x3d, 0x20, 0x33, 0xc3, 0x02, - 0xad, 0x2b, 0x8c, 0x5a, 0x5f, 0x93, 0x32, 0x3c, 0x43, 0x5d, 0x43, 0x37, 0xc2, 0xf7, 0x18, 0x4c, - 0x96, 0xe1, 0x6c, 0xe0, 0xf6, 0x02, 0x74, 0x1b, 0xd2, 0xb6, 0x63, 0x5a, 0x1e, 0xbc, 0x99, 0x2c, - 0x67, 0xcf, 0x02, 0xb7, 0x9e, 0xd3, 0xe3, 0xb7, 0x24, 0xdf, 0x2a, 0xe4, 0xc9, 0xcc, 0x7c, 0x88, - 0xbd, 0x2a, 0xa5, 0xce, 0xaf, 0x12, 0xe3, 0x70, 0xab, 0xf4, 0x7b, 0xc4, 0x9f, 0x64, 0xad, 0xe1, - 0xb5, 0xf0, 0x08, 0xe3, 0xf3, 0x33, 0xc1, 0xdb, 0x03, 0xc7, 0xbb, 0x5f, 0x91, 0x3a, 0x0d, 0xdf, - 0x82, 0xb4, 0xfb, 0x35, 0x61, 0xde, 0x97, 0xaf, 0x2e, 0xb3, 0xcf, 0x0d, 0x2f, 0xbb, 0x9f, 0x1b, - 0x5e, 0x5e, 0xe3, 0x19, 0x98, 0xc5, 0xfd, 0xfe, 0x7f, 0x5f, 0x14, 0x24, 0x8f, 0xe9, 0x7e, 0x03, - 0xe6, 0x22, 0x26, 0x4c, 0x94, 0x07, 0x08, 0x7c, 0xee, 0x85, 0x7f, 0x60, 0x76, 0x75, 0x4d, 0xde, - 0xd9, 0xaa, 0x3c, 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x13, 0xfa, - 0x58, 0x4c, 0x82, 0x7d, 0x72, 0xf6, 0xfe, 0xcf, 0x00, 0xf8, 0xdf, 0x9d, 0x22, 0xb2, 0x36, 0xaa, - 0x9f, 0xc9, 0x2f, 0x57, 0x9f, 0xed, 0x54, 0x1b, 0xc5, 0x09, 0x84, 0x20, 0x5f, 0x5e, 0x6d, 0x56, - 0x6a, 0xb2, 0x54, 0x6d, 0x6c, 0x3f, 0xdf, 0x6a, 0x54, 0xdd, 0x4f, 0xd5, 0xde, 0x5f, 0x83, 0x99, - 0xe0, 0xcd, 0x34, 0x68, 0x0e, 0x0a, 0x95, 0x5a, 0xb5, 0xb2, 0x21, 0xbf, 0xac, 0xaf, 0xca, 0x2f, - 0x76, 0xaa, 0x3b, 0xd5, 0xe2, 0x04, 0x2d, 0x1a, 0x25, 0x3e, 0xd9, 0x79, 0xf6, 0xac, 0x28, 0xa0, - 0x02, 0x64, 0xd9, 0x33, 0xfd, 0xb0, 0x4c, 0x31, 0x71, 0x7f, 0x13, 0xb2, 0x81, 0x1b, 0x68, 0xc9, - 0xeb, 0xb6, 0x77, 0x1a, 0x35, 0xb9, 0x59, 0xdf, 0xac, 0x36, 0x9a, 0xab, 0x9b, 0xdb, 0x4c, 0x06, - 0xa5, 0xad, 0x96, 0x9f, 0x4b, 0xcd, 0xa2, 0xe0, 0x3d, 0x37, 0x9f, 0xef, 0x54, 0x6a, 0x6e, 0x35, - 0xc4, 0x54, 0x3a, 0x59, 0x4c, 0xde, 0xff, 0x1b, 0x02, 0x5c, 0x19, 0x72, 0x4b, 0x0b, 0xca, 0xc2, - 0xf4, 0x8e, 0x41, 0xaf, 0xe3, 0x2c, 0x4e, 0xa0, 0x5c, 0xe0, 0xa2, 0x96, 0xa2, 0x80, 0xd2, 0xec, - 0xaa, 0x8c, 0x62, 0x02, 0x4d, 0x41, 0xa2, 0xf1, 0xa8, 0x98, 0x24, 0x25, 0x0d, 0xdc, 0x73, 0x52, - 0x4c, 0xa1, 0x0c, 0xbf, 0x61, 0xa1, 0x38, 0x89, 0x66, 0xfc, 0x8b, 0x0e, 0x8a, 0x53, 0x44, 0x94, - 0x77, 0x61, 0x40, 0x71, 0xfa, 0xfe, 0x75, 0x08, 0x1c, 0xca, 0x46, 0x00, 0x53, 0xcf, 0x14, 0x07, - 0xdb, 0x4e, 0x71, 0x02, 0x4d, 0x43, 0x72, 0xb5, 0xdd, 0x2e, 0x0a, 0x0f, 0x7f, 0x9c, 0x84, 0xb4, - 0xfb, 0x01, 0x15, 0xf4, 0x0c, 0x26, 0xd9, 0x2a, 0xf1, 0xe2, 0x70, 0x27, 0x9c, 0x0e, 0xde, 0x85, - 0xa5, 0xf3, 0xbc, 0x74, 0x71, 0x02, 0xfd, 0x35, 0xc8, 0x06, 0x9c, 0x1b, 0x34, 0x74, 0xa5, 0x2b, - 0xe4, 0xd0, 0x2d, 0xdc, 0x3e, 0x2f, 0x9b, 0x27, 0xff, 0x15, 0x64, 0x3c, 0x63, 0x8b, 0x6e, 0x8c, - 0x32, 0xc5, 0xae, 0xec, 0xd1, 0xf6, 0x9a, 0x8c, 0x35, 0x71, 0xe2, 0x7d, 0x01, 0x59, 0x80, 0x06, - 0x8d, 0x2d, 0x8a, 0x0a, 0x1e, 0x18, 0x6a, 0xcd, 0x17, 0xee, 0x8f, 0x95, 0xdb, 0x7f, 0xe7, 0x0b, - 0x48, 0x11, 0x4b, 0x86, 0xa2, 0xdc, 0xb1, 0x3e, 0xcb, 0xb9, 0x70, 0x63, 0x64, 0x1e, 0x57, 0x3f, - 0xe5, 0x7b, 0x3f, 0xfc, 0xf3, 0x6b, 0x13, 0x3f, 0x3c, 0xbb, 0x26, 0xfc, 0xe8, 0xec, 0x9a, 0xf0, - 0xe3, 0xb3, 0x6b, 0xc2, 0x9f, 0x9d, 0x5d, 0x13, 0x7e, 0xfd, 0x27, 0xd7, 0x26, 0x7e, 0xf4, 0x93, - 0x6b, 0x13, 0x3f, 0xfe, 0xc9, 0xb5, 0x89, 0xcf, 0xa7, 0x39, 0xf7, 0xee, 0x14, 0x1d, 0xf4, 0x8f, - 0xfe, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa2, 0x06, 0x52, 0xd6, 0x9e, 0x7d, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_290a2f113c3e6c17) } + +var fileDescriptor_api_290a2f113c3e6c17 = []byte{ + // 8124 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x59, + 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0xd1, 0x95, 0xba, 0x9b, 0xad, 0x99, 0x69, 0xa9, + 0xab, 0xff, 0x7b, 0x67, 0xa4, 0xe9, 0xee, 0x1d, 0xcf, 0x78, 0x7a, 0x3c, 0x6b, 0x91, 0x62, 0x37, + 0x29, 0xb5, 0xd4, 0xea, 0x22, 0xd5, 0xed, 0x19, 0xaf, 0x53, 0x5b, 0xaa, 0xba, 0xa2, 0x6a, 0x45, + 0x56, 0xb1, 0xab, 0x8a, 0xfa, 0x19, 0x20, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x0b, 0x3f, 0x38, + 0xc8, 0x1a, 0x4e, 0xe2, 0x75, 0x1c, 0xc7, 0x01, 0x82, 0x20, 0x01, 0x12, 0xc4, 0x41, 0x90, 0xd8, + 0x2f, 0x46, 0xb2, 0x08, 0x0c, 0x64, 0xfd, 0x14, 0x23, 0x40, 0x14, 0x5b, 0x9b, 0xb7, 0x04, 0x46, + 0x90, 0x97, 0x00, 0xf3, 0x10, 0x04, 0xf7, 0xa7, 0xfe, 0xc8, 0x22, 0x45, 0xf5, 0xd6, 0x20, 0x03, + 0xec, 0x8b, 0xc4, 0x3a, 0x75, 0xcf, 0xa9, 0x7b, 0xcf, 0xbd, 0xf7, 0xdc, 0xf3, 0xdd, 0x3a, 0xf7, + 0x14, 0xcc, 0x5a, 0xa6, 0xa2, 0xee, 0x77, 0x77, 0x57, 0x94, 0xae, 0xbe, 0xdc, 0xb5, 0x4c, 0xc7, + 0x44, 0xb3, 0xaa, 0xa9, 0x1e, 0x50, 0xf2, 0x32, 0xbf, 0xb9, 0x70, 0xff, 0xe0, 0x70, 0xe5, 0xe0, + 0xd0, 0xc6, 0xd6, 0x21, 0xb6, 0x56, 0x54, 0xd3, 0x50, 0x7b, 0x96, 0x85, 0x0d, 0xf5, 0x64, 0xa5, + 0x6d, 0xaa, 0x07, 0xf4, 0x8f, 0x6e, 0xb4, 0x18, 0xfb, 0x02, 0x72, 0x25, 0x6a, 0x8a, 0xa3, 0x70, + 0xda, 0xbc, 0x4b, 0xc3, 0x96, 0x65, 0x5a, 0x36, 0xa7, 0x5e, 0x76, 0xa9, 0x1d, 0xec, 0x28, 0x81, + 0xd2, 0x6f, 0xd9, 0x8e, 0x69, 0x29, 0x2d, 0xbc, 0x82, 0x8d, 0x96, 0x6e, 0x60, 0x52, 0xe0, 0x50, + 0x55, 0xf9, 0xcd, 0xb7, 0x23, 0x6f, 0x3e, 0xe2, 0x77, 0x4b, 0x3d, 0x47, 0x6f, 0xaf, 0xec, 0xb7, + 0xd5, 0x15, 0x47, 0xef, 0x60, 0xdb, 0x51, 0x3a, 0x5d, 0x7e, 0xe7, 0x3e, 0xbd, 0xe3, 0x58, 0x8a, + 0xaa, 0x1b, 0x2d, 0xf7, 0x7f, 0x77, 0x77, 0xc5, 0xc2, 0xaa, 0x69, 0x69, 0x58, 0x93, 0xed, 0xae, + 0x62, 0xb8, 0xd5, 0x6d, 0x99, 0x2d, 0x93, 0xfe, 0x5c, 0x21, 0xbf, 0x38, 0xf5, 0x5a, 0xcb, 0x34, + 0x5b, 0x6d, 0xbc, 0x42, 0xaf, 0x76, 0x7b, 0x7b, 0x2b, 0x5a, 0xcf, 0x52, 0x1c, 0xdd, 0xe4, 0x5c, + 0xe2, 0xbf, 0x12, 0x20, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac, 0x68, 0xd8, 0x42, 0x57, + 0x21, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0x25, 0xe1, 0xee, 0x4c, 0x79, 0xfa, 0xcb, 0xd3, 0xc5, 0xe4, + 0x06, 0x3e, 0x91, 0x08, 0x0d, 0x2d, 0xc1, 0x34, 0x36, 0x34, 0x99, 0xdc, 0x4e, 0x85, 0x6f, 0x4f, + 0x61, 0x43, 0xdb, 0xc0, 0x27, 0xe8, 0xdb, 0x90, 0xb6, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x25, + 0xe1, 0xee, 0x64, 0xf9, 0xe7, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x65, + 0xd5, 0xec, 0xac, 0x78, 0xfd, 0xa4, 0xed, 0xfa, 0xbf, 0x57, 0xba, 0x07, 0xad, 0x95, 0x7e, 0x1d, + 0x2d, 0x37, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, + 0xa5, 0x13, 0xc5, 0xa4, 0xf8, 0xbb, 0x49, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xcd, + 0xdf, 0x87, 0xa4, 0x73, 0x6c, 0xd0, 0x9a, 0x67, 0x1f, 0x5e, 0x5b, 0x1e, 0x18, 0x11, 0xcb, 0x4d, + 0x4b, 0x31, 0x6c, 0x45, 0x25, 0xcd, 0x97, 0x48, 0x51, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0xf7, 0x3a, + 0x98, 0x2a, 0x92, 0x36, 0x2a, 0xfb, 0xf0, 0x4a, 0x04, 0x67, 0xa3, 0xab, 0x18, 0x12, 0xb0, 0xb2, + 0xe4, 0x37, 0xba, 0x0a, 0x69, 0xa3, 0xd7, 0x21, 0xaa, 0xb0, 0x69, 0x43, 0x93, 0xd2, 0xb4, 0xd1, + 0xeb, 0x6c, 0xe0, 0x13, 0x1b, 0xfd, 0x02, 0x5c, 0xd6, 0x70, 0xd7, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, + 0xb6, 0x14, 0xa3, 0x85, 0x65, 0xdd, 0xd8, 0x33, 0xed, 0xd2, 0xd4, 0x52, 0xf2, 0x6e, 0xf6, 0xe1, + 0xdb, 0x11, 0xf2, 0x25, 0x52, 0xaa, 0x6e, 0xec, 0x99, 0xe5, 0xd4, 0x0f, 0x4f, 0x17, 0x27, 0xa4, + 0x79, 0x5f, 0x82, 0x77, 0xcb, 0x46, 0x0d, 0xc8, 0xf1, 0xea, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x34, + 0xbd, 0x24, 0xdc, 0xcd, 0x3f, 0x5c, 0x8e, 0x12, 0x18, 0x52, 0x0d, 0xb9, 0xec, 0x75, 0xb0, 0x44, + 0xb9, 0xa4, 0x19, 0x2b, 0x70, 0x85, 0xde, 0x82, 0x0c, 0x69, 0xc9, 0xee, 0x89, 0x83, 0xed, 0x52, + 0x9a, 0x36, 0x85, 0x34, 0xad, 0x4c, 0xae, 0xc5, 0x4f, 0x61, 0x26, 0xc8, 0x8a, 0x10, 0xe4, 0xa5, + 0x6a, 0x63, 0x67, 0xb3, 0x2a, 0xef, 0x6c, 0x6d, 0x6c, 0x3d, 0x7f, 0xb5, 0x55, 0x9c, 0x40, 0xf3, + 0x50, 0xe4, 0xb4, 0x8d, 0xea, 0x67, 0xf2, 0xb3, 0xfa, 0x66, 0xbd, 0x59, 0x14, 0x16, 0x52, 0xbf, + 0xf6, 0xbb, 0xd7, 0x26, 0xc4, 0x6d, 0x80, 0xa7, 0xd8, 0xe1, 0x03, 0x0c, 0x95, 0x61, 0x6a, 0x9f, + 0xd6, 0xa7, 0x24, 0x50, 0x4d, 0x2f, 0x45, 0x56, 0x3c, 0x30, 0x18, 0xcb, 0x69, 0xa2, 0x8d, 0x1f, + 0x9d, 0x2e, 0x0a, 0x12, 0xe7, 0x14, 0xff, 0x48, 0x80, 0x2c, 0x15, 0xc9, 0xda, 0x87, 0x2a, 0x7d, + 0x32, 0xaf, 0x9f, 0xab, 0x8c, 0x41, 0xa1, 0x68, 0x19, 0x26, 0x0f, 0x95, 0x76, 0x0f, 0x97, 0x12, + 0x54, 0x46, 0x29, 0x42, 0xc6, 0x4b, 0x72, 0x5f, 0x62, 0xc5, 0xd0, 0x63, 0x98, 0xd1, 0x0d, 0x07, + 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0xe7, 0xb0, 0x65, 0x59, 0x69, 0x7a, 0x21, 0xfe, 0x4b, 0x01, 0x60, + 0xbb, 0x17, 0xa7, 0x52, 0xd0, 0x37, 0xc7, 0xac, 0x3f, 0x1f, 0x5d, 0xbc, 0x15, 0x97, 0x61, 0x4a, + 0x37, 0xda, 0xba, 0xc1, 0xea, 0x9f, 0x96, 0xf8, 0x15, 0x9a, 0x87, 0xc9, 0xdd, 0xb6, 0x6e, 0x68, + 0x74, 0x3e, 0xa4, 0x25, 0x76, 0x21, 0x4a, 0x90, 0xa5, 0xb5, 0x8e, 0x51, 0xef, 0xe2, 0xbf, 0x4f, + 0xc0, 0xa5, 0x8a, 0x69, 0x68, 0x3a, 0x99, 0x92, 0x4a, 0xfb, 0x6b, 0xa1, 0x95, 0x75, 0x08, 0x4c, + 0x3e, 0x19, 0x1f, 0x77, 0xc7, 0xec, 0x63, 0xe4, 0x73, 0x55, 0x8f, 0xbb, 0x94, 0x16, 0xad, 0x49, + 0xf4, 0x4d, 0xb8, 0xa2, 0xb4, 0xdb, 0xe6, 0x91, 0xac, 0xef, 0xc9, 0x9a, 0x89, 0x6d, 0xd9, 0x30, + 0x1d, 0x19, 0x1f, 0xeb, 0xb6, 0x43, 0x4d, 0x49, 0x5a, 0x9a, 0xa3, 0xb7, 0xeb, 0x7b, 0x6b, 0x26, + 0xb6, 0xb7, 0x4c, 0xa7, 0x4a, 0x6e, 0x91, 0x79, 0x4a, 0x2a, 0xc3, 0xe6, 0xe9, 0x14, 0x31, 0xbf, + 0x52, 0x1a, 0x1f, 0x77, 0xd9, 0x3c, 0xfd, 0x25, 0xb8, 0xdc, 0xaf, 0xc7, 0x38, 0xfb, 0xe9, 0x4f, + 0x04, 0xc8, 0xd7, 0x0d, 0xdd, 0xf9, 0x5a, 0x74, 0x90, 0xa7, 0xd4, 0x64, 0x50, 0xa9, 0xf7, 0xa1, + 0xb8, 0xa7, 0xe8, 0xed, 0xe7, 0x46, 0xd3, 0xec, 0xec, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0x6b, 0x7d, + 0x80, 0x2e, 0xbe, 0x84, 0x82, 0xd7, 0x9a, 0x38, 0xd5, 0xe4, 0x40, 0xb1, 0x6e, 0xa8, 0x16, 0xee, + 0x60, 0x23, 0x56, 0x3d, 0xbd, 0x0d, 0x19, 0xdd, 0x95, 0x4b, 0x75, 0x95, 0x94, 0x7c, 0x82, 0xd8, + 0x83, 0xd9, 0xc0, 0x53, 0xe3, 0x34, 0x8b, 0x64, 0x69, 0xc0, 0x47, 0xb2, 0xdf, 0x47, 0x64, 0x69, + 0xc0, 0x47, 0xcc, 0x8c, 0x35, 0x20, 0xb7, 0x86, 0xdb, 0xd8, 0xc1, 0x71, 0x5a, 0xf7, 0x1d, 0xc8, + 0xbb, 0x42, 0xe3, 0xec, 0x98, 0xbf, 0x23, 0x00, 0xe2, 0x72, 0xc9, 0x6a, 0x1a, 0x67, 0xdf, 0x2c, + 0x12, 0x17, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x05, 0xd8, 0x98, 0x04, 0x46, 0xa2, 0xee, 0x80, 0x6f, + 0x65, 0x53, 0x41, 0x2b, 0xcb, 0xdd, 0x98, 0x23, 0x98, 0x0b, 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, + 0x3a, 0x25, 0x96, 0x92, 0x41, 0x5f, 0x8d, 0x12, 0xc5, 0xef, 0x0b, 0x30, 0x5b, 0x69, 0x63, 0xc5, + 0x8a, 0x5d, 0x23, 0xdf, 0x82, 0xb4, 0x86, 0x15, 0x8d, 0x36, 0x99, 0x4d, 0xec, 0x77, 0x02, 0x52, + 0x88, 0x47, 0xbb, 0xbc, 0xdf, 0x56, 0x97, 0x9b, 0xae, 0xaf, 0xcb, 0x67, 0xb7, 0xc7, 0x24, 0x7e, + 0x06, 0x28, 0x58, 0xb3, 0x38, 0x07, 0xc2, 0xdf, 0x17, 0x00, 0x49, 0xf8, 0x10, 0x5b, 0x4e, 0xec, + 0xcd, 0x5e, 0x83, 0xac, 0xa3, 0x58, 0x2d, 0xec, 0xc8, 0xc4, 0x8b, 0xbf, 0x48, 0xcb, 0x81, 0xf1, + 0x11, 0xb2, 0xf8, 0x39, 0xcc, 0x85, 0xea, 0x17, 0x67, 0xe3, 0xff, 0x97, 0x00, 0xd9, 0x86, 0xaa, + 0x18, 0x71, 0xb6, 0xfa, 0x53, 0xc8, 0xda, 0xaa, 0x62, 0xc8, 0x7b, 0xa6, 0xd5, 0x51, 0x1c, 0x3a, + 0xc4, 0xf3, 0xa1, 0x56, 0x7b, 0x1e, 0xb4, 0xaa, 0x18, 0x4f, 0x68, 0x21, 0x09, 0x6c, 0xef, 0x37, + 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0x34, 0x46, 0xd7, 0xbf, 0xfc, 0xc3, 0xf7, 0x03, 0xfc, + 0x07, 0x87, 0xcb, 0x2e, 0x88, 0x5b, 0x0e, 0x80, 0xb8, 0x65, 0xc2, 0xb1, 0xdc, 0x70, 0x2c, 0x6c, + 0xb4, 0x9c, 0x7d, 0x09, 0x0e, 0xf0, 0xc9, 0x33, 0x26, 0x83, 0x4d, 0xac, 0xf5, 0x54, 0x3a, 0x59, + 0x4c, 0x89, 0xff, 0x47, 0x80, 0x19, 0xd6, 0xe4, 0x38, 0x27, 0xd6, 0x07, 0x90, 0xb2, 0xcc, 0x23, + 0x36, 0xb1, 0xb2, 0x0f, 0xdf, 0x8a, 0x10, 0xb1, 0x81, 0x4f, 0x82, 0x2b, 0x17, 0x2d, 0x8e, 0xca, + 0xc0, 0xfd, 0x40, 0x99, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0x77, 0xa0, 0xb0, + 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x4a, 0x92, 0x55, 0x2e, 0x79, 0x77, 0x46, 0xca, 0x53, 0xb2, + 0x5b, 0x75, 0x9b, 0xb4, 0x9c, 0x8d, 0x74, 0x1b, 0xff, 0x94, 0xf5, 0xf9, 0xff, 0x15, 0xf8, 0x1c, + 0x72, 0x5b, 0xfe, 0xd3, 0xd6, 0xf5, 0xbf, 0x99, 0x80, 0x2b, 0x95, 0x7d, 0xac, 0x1e, 0x54, 0x4c, + 0xc3, 0xd6, 0x6d, 0x87, 0xe8, 0x2e, 0xce, 0xfe, 0x7f, 0x0b, 0x32, 0x47, 0xba, 0xb3, 0x2f, 0x6b, + 0xfa, 0xde, 0x1e, 0xb5, 0x73, 0x69, 0x29, 0x4d, 0x08, 0x6b, 0xfa, 0xde, 0x1e, 0x7a, 0x04, 0xa9, + 0x8e, 0xa9, 0x31, 0x77, 0x39, 0xff, 0x70, 0x31, 0x42, 0x3c, 0xad, 0x9a, 0xdd, 0xeb, 0x6c, 0x9a, + 0x1a, 0x96, 0x68, 0x61, 0x74, 0x0d, 0x40, 0x25, 0xd4, 0xae, 0xa9, 0x1b, 0x0e, 0x5f, 0x27, 0x03, + 0x14, 0x54, 0x83, 0x8c, 0x83, 0xad, 0x8e, 0x6e, 0x28, 0x0e, 0x2e, 0x4d, 0x52, 0xe5, 0xdd, 0x8c, + 0xac, 0x78, 0xb7, 0xad, 0xab, 0xca, 0x1a, 0xb6, 0x55, 0x4b, 0xef, 0x3a, 0xa6, 0xc5, 0xb5, 0xe8, + 0x33, 0x8b, 0xdf, 0x4b, 0x41, 0x69, 0x50, 0x37, 0x71, 0x8e, 0x90, 0x6d, 0x98, 0x22, 0xf8, 0xba, + 0xed, 0xf0, 0x31, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x6a, 0x40, 0x71, 0x7a, 0xdb, 0xe1, 0xd5, 0xe6, + 0x72, 0x16, 0xfe, 0xad, 0x00, 0x53, 0xec, 0x06, 0x7a, 0x00, 0x69, 0xbe, 0xa1, 0xa0, 0xd1, 0x3a, + 0x26, 0xcb, 0x97, 0xcf, 0x4e, 0x17, 0xa7, 0xd9, 0x1e, 0xc1, 0xda, 0x97, 0xfe, 0x4f, 0x69, 0x9a, + 0x96, 0xab, 0x6b, 0xa4, 0xb7, 0x6c, 0x47, 0xb1, 0x1c, 0xba, 0x6d, 0x93, 0x60, 0xb8, 0x81, 0x12, + 0x36, 0xf0, 0x09, 0x5a, 0x87, 0x29, 0xdb, 0x51, 0x9c, 0x9e, 0xcd, 0xfb, 0xeb, 0x42, 0x95, 0x6d, + 0x50, 0x4e, 0x89, 0x4b, 0x20, 0x8e, 0x8e, 0x86, 0x1d, 0x45, 0x6f, 0xd3, 0x0e, 0xcc, 0x48, 0xfc, + 0x4a, 0xfc, 0x2d, 0x01, 0xa6, 0x58, 0x51, 0x74, 0x05, 0xe6, 0xa4, 0xd5, 0xad, 0xa7, 0x55, 0xb9, + 0xbe, 0xb5, 0x56, 0x6d, 0x56, 0xa5, 0xcd, 0xfa, 0xd6, 0x6a, 0xb3, 0x5a, 0x9c, 0x40, 0x97, 0x01, + 0xb9, 0x37, 0x2a, 0xcf, 0xb7, 0x1a, 0xf5, 0x46, 0xb3, 0xba, 0xd5, 0x2c, 0x0a, 0x74, 0x6f, 0x81, + 0xd2, 0x03, 0xd4, 0x04, 0xba, 0x09, 0x4b, 0xfd, 0x54, 0xb9, 0xd1, 0x5c, 0x6d, 0x36, 0xe4, 0x6a, + 0xa3, 0x59, 0xdf, 0x5c, 0x6d, 0x56, 0xd7, 0x8a, 0xc9, 0x11, 0xa5, 0xc8, 0x43, 0x24, 0xa9, 0x5a, + 0x69, 0x16, 0x53, 0xa2, 0x03, 0x97, 0x24, 0xac, 0x9a, 0x9d, 0x6e, 0xcf, 0xc1, 0xa4, 0x96, 0x76, + 0x9c, 0x33, 0xe5, 0x0a, 0x4c, 0x6b, 0xd6, 0x89, 0x6c, 0xf5, 0x0c, 0x3e, 0x4f, 0xa6, 0x34, 0xeb, + 0x44, 0xea, 0x19, 0xe2, 0x3f, 0x17, 0xe0, 0x72, 0xff, 0x63, 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, + 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0xb7, 0x1d, 0x85, 0x3b, 0x23, 0xf7, 0x03, 0x92, 0xf8, 0x66, 0xdb, + 0xb2, 0xb7, 0xd9, 0xb6, 0xf9, 0xb2, 0x52, 0xa1, 0x15, 0x59, 0x23, 0x1c, 0xae, 0xf9, 0xa1, 0x42, + 0x28, 0x45, 0xfc, 0x9f, 0x29, 0xc8, 0x55, 0x0d, 0xad, 0x79, 0x1c, 0xeb, 0x5a, 0x72, 0x19, 0xa6, + 0x54, 0xb3, 0xd3, 0xd1, 0x1d, 0x57, 0x41, 0xec, 0x0a, 0xfd, 0x6c, 0xc0, 0x89, 0x4c, 0x8e, 0xe1, + 0x4a, 0xf9, 0xee, 0x23, 0xfa, 0x0e, 0x5c, 0x21, 0x56, 0xd3, 0x32, 0x94, 0xb6, 0xcc, 0xa4, 0xc9, + 0x8e, 0xa5, 0xb7, 0x5a, 0xd8, 0xe2, 0x1b, 0x7c, 0x77, 0x23, 0xea, 0x59, 0xe7, 0x1c, 0x15, 0xca, + 0xd0, 0x64, 0xe5, 0xa5, 0x4b, 0x7a, 0x14, 0x19, 0x7d, 0x02, 0x40, 0x96, 0x22, 0xba, 0x69, 0x68, + 0x73, 0x7b, 0x34, 0x6c, 0xd7, 0xd0, 0x35, 0x41, 0x84, 0x81, 0x5c, 0xdb, 0x68, 0x85, 0x20, 0x86, + 0xd7, 0x3d, 0xdd, 0xc2, 0xf2, 0x83, 0xae, 0x4a, 0xa1, 0x7c, 0xba, 0x9c, 0x3f, 0x3b, 0x5d, 0x04, + 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x82, 0x60, 0xbf, 0xbb, 0x2a, 0x7a, 0x05, 0xf7, 0x02, 0x3b, + 0x12, 0x64, 0xe5, 0xe5, 0xcd, 0x52, 0x1c, 0x79, 0x5f, 0x6f, 0xed, 0x63, 0x4b, 0xf6, 0x36, 0x8e, + 0xe9, 0x0e, 0x5e, 0x5a, 0xba, 0xe9, 0x33, 0x54, 0x14, 0x83, 0xd5, 0x7e, 0xd5, 0xa9, 0xd1, 0xc2, + 0x9e, 0xce, 0x88, 0xf2, 0xbb, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xc3, 0x94, 0xcf, 0xae, 0xd0, 0x3d, + 0x28, 0x3a, 0xc7, 0x86, 0xbc, 0x8f, 0x15, 0xcb, 0xd9, 0xc5, 0x8a, 0x43, 0x56, 0x69, 0xa0, 0x25, + 0x0a, 0xce, 0xb1, 0x51, 0x0b, 0x90, 0xd1, 0x0b, 0x28, 0xea, 0x86, 0xbc, 0xd7, 0xd6, 0x5b, 0xfb, + 0x8e, 0x7c, 0x64, 0xe9, 0x0e, 0xb6, 0x4b, 0xb3, 0x54, 0x21, 0x51, 0xe3, 0xb6, 0xc1, 0x77, 0x72, + 0xb5, 0x57, 0xa4, 0x24, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, 0x7e, 0x4a, 0xb4, 0xd7, 0x53, 0xe9, + 0xe9, 0x62, 0x5a, 0xfc, 0xaf, 0x02, 0xe4, 0xdd, 0xe1, 0x16, 0xe7, 0xcc, 0xb8, 0x0b, 0x45, 0xd3, + 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x5f, 0x70, 0xf2, 0xa6, 0x81, 0xb7, 0x09, 0x99, + 0xa9, 0x0b, 0x6d, 0xc3, 0xac, 0xed, 0x28, 0x2d, 0xdd, 0x68, 0x05, 0xd4, 0x3b, 0x39, 0xbe, 0x5b, + 0x5f, 0xe4, 0xdc, 0x1e, 0x3d, 0xe4, 0xa5, 0xfc, 0xa9, 0x00, 0xb3, 0xab, 0x5a, 0x47, 0x37, 0x1a, + 0xdd, 0xb6, 0x1e, 0xeb, 0x6e, 0xc1, 0x4d, 0xc8, 0xd8, 0x44, 0xa6, 0x6f, 0xf0, 0x7d, 0xec, 0x97, + 0xa6, 0x77, 0x88, 0xe5, 0x7f, 0x06, 0x05, 0x7c, 0xdc, 0xd5, 0xd9, 0xcb, 0x00, 0x06, 0x59, 0x52, + 0xe3, 0xb7, 0x2d, 0xef, 0xf3, 0x92, 0x5b, 0xbc, 0x4d, 0x9f, 0x01, 0x0a, 0x36, 0x29, 0x4e, 0xec, + 0xf2, 0x19, 0xcc, 0x51, 0xd1, 0x3b, 0x86, 0x1d, 0xb3, 0xbe, 0xc4, 0x5f, 0x84, 0xf9, 0xb0, 0xe8, + 0x38, 0xeb, 0xfd, 0x8a, 0xf7, 0xf2, 0x26, 0xb6, 0x62, 0x85, 0x9b, 0x9e, 0xae, 0xb9, 0xe0, 0x38, + 0xeb, 0xfc, 0xab, 0x02, 0x5c, 0xa5, 0xb2, 0xe9, 0xfb, 0x92, 0x3d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, + 0x2b, 0x56, 0xbe, 0x01, 0x53, 0x0c, 0xf3, 0xd2, 0xf1, 0x39, 0x59, 0xce, 0x12, 0xcf, 0xa5, 0xe1, + 0x98, 0x16, 0xf1, 0x5c, 0xf8, 0x2d, 0x51, 0x81, 0x85, 0xa8, 0x5a, 0xc4, 0xbc, 0x1d, 0x30, 0xcb, + 0x9d, 0x46, 0x32, 0x94, 0x2b, 0xfb, 0xc4, 0x67, 0x42, 0x55, 0xc8, 0xaa, 0xf4, 0x97, 0xec, 0x9c, + 0x74, 0x31, 0x95, 0x9f, 0x1f, 0xe5, 0x6f, 0x32, 0xb6, 0xe6, 0x49, 0x17, 0x13, 0xa7, 0xd5, 0xfd, + 0x4d, 0x14, 0x15, 0x68, 0xe4, 0x48, 0x8f, 0x95, 0xce, 0x23, 0x5a, 0xd6, 0x75, 0xfd, 0xb8, 0x0e, + 0xfe, 0x45, 0x92, 0x2b, 0x81, 0x3d, 0x83, 0x17, 0x8f, 0xd5, 0x47, 0xf9, 0x3c, 0xf4, 0xba, 0x2a, + 0xd8, 0xf0, 0xc4, 0x05, 0x1a, 0x1e, 0xd8, 0x33, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xec, 0x8a, 0xcb, + 0xac, 0x4d, 0x2e, 0xfa, 0xb9, 0x88, 0x3a, 0x66, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, + 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, 0x7b, 0xaf, 0x36, 0x80, 0x07, 0xa6, 0xf1, + 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x59, 0xbd, 0x5c, 0x57, 0x81, 0x56, 0xdb, 0x3e, 0x1f, 0x5e, 0xf8, + 0x23, 0x85, 0x8b, 0x2b, 0x78, 0x5e, 0x02, 0x13, 0x21, 0xfe, 0x40, 0x80, 0xb7, 0x22, 0x7b, 0x2d, + 0xce, 0x85, 0xec, 0x13, 0x48, 0xd1, 0xc6, 0x27, 0x2e, 0xd8, 0x78, 0xca, 0x25, 0xfe, 0x9e, 0x3b, + 0xc7, 0x25, 0xdc, 0x36, 0x89, 0x62, 0xbf, 0x82, 0xfd, 0xb0, 0x69, 0xb7, 0xc3, 0x13, 0x17, 0xee, + 0x70, 0x97, 0xd5, 0x33, 0x02, 0x7d, 0xd5, 0x8c, 0xd3, 0x08, 0xfc, 0xba, 0x00, 0x73, 0x9e, 0x4f, + 0x13, 0xb3, 0x7b, 0xfb, 0x01, 0x24, 0x0d, 0xf3, 0xe8, 0x22, 0x9b, 0x81, 0xa4, 0x3c, 0x59, 0x92, + 0xc2, 0x35, 0x8a, 0xb3, 0xbd, 0xff, 0x2e, 0x01, 0x99, 0xa7, 0x95, 0x38, 0x5b, 0xf9, 0x09, 0xdf, + 0x68, 0x66, 0x13, 0x3b, 0x6a, 0x28, 0x7a, 0xcf, 0x5b, 0x7e, 0x5a, 0xd9, 0xc0, 0x27, 0xee, 0x50, + 0x24, 0x5c, 0x68, 0x15, 0x32, 0xce, 0xbe, 0x85, 0xed, 0x7d, 0xb3, 0xad, 0x5d, 0xc4, 0x07, 0xf1, + 0xb9, 0x16, 0x30, 0x4c, 0x52, 0xb9, 0x6e, 0xf0, 0x82, 0x10, 0x11, 0xbc, 0x40, 0x1e, 0xe3, 0xb9, + 0x71, 0x89, 0x8b, 0x3c, 0x26, 0xe0, 0xbf, 0x4d, 0x16, 0xa7, 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, + 0x76, 0xc9, 0xdf, 0x48, 0x42, 0x7e, 0xbb, 0x67, 0xef, 0xc7, 0x3c, 0xfa, 0x2a, 0x00, 0xdd, 0x9e, + 0x4d, 0xf1, 0xc1, 0xb1, 0xc1, 0xdb, 0x7c, 0x4e, 0x5c, 0x84, 0xdb, 0x68, 0xc6, 0xd7, 0x3c, 0x36, + 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x5c, 0x71, 0x63, 0x14, 0x92, 0x6c, 0x1e, 0x1b, 0x9b, 0xd8, + 0x83, 0x90, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x69, 0x72, 0x21, 0x3b, 0xe6, 0x45, 0xba, 0x79, + 0x8a, 0xf0, 0x34, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x2c, 0x4d, 0x53, 0x74, 0x69, 0x8a, 0x6a, + 0x0b, 0x57, 0x23, 0x5d, 0x94, 0xd2, 0x94, 0x95, 0x2c, 0x44, 0xf3, 0x30, 0xb9, 0x67, 0x5a, 0x2a, + 0xa6, 0x11, 0x13, 0x69, 0x89, 0x5d, 0xb0, 0xfe, 0x5c, 0x4f, 0xa5, 0xd3, 0xc5, 0xcc, 0x7a, 0x2a, + 0x9d, 0x29, 0x82, 0xf8, 0x5b, 0x02, 0x14, 0xbc, 0x8e, 0x88, 0xd3, 0x5a, 0x57, 0x42, 0x5a, 0xbc, + 0x78, 0x57, 0x10, 0x05, 0x8a, 0xff, 0x81, 0xba, 0x2b, 0xaa, 0x79, 0x48, 0x7b, 0x26, 0xce, 0x91, + 0xf2, 0x98, 0x85, 0xce, 0x24, 0x2e, 0xda, 0xbb, 0x34, 0x8a, 0xe6, 0x01, 0xcc, 0xeb, 0x1d, 0x62, + 0xc7, 0x75, 0xa7, 0x7d, 0xc2, 0x31, 0x95, 0x83, 0xdd, 0xf7, 0xb3, 0x73, 0xfe, 0xbd, 0x8a, 0x7b, + 0x4b, 0xfc, 0x87, 0x74, 0x77, 0xda, 0x6f, 0x49, 0x9c, 0xaa, 0xae, 0x43, 0xce, 0x62, 0xa2, 0x89, + 0xcf, 0x71, 0x41, 0x6d, 0xcf, 0x78, 0xac, 0x44, 0xe1, 0xbf, 0x93, 0x80, 0xc2, 0x8b, 0x1e, 0xb6, + 0x4e, 0xbe, 0x4e, 0xea, 0xbe, 0x0d, 0x85, 0x23, 0x45, 0x77, 0xe4, 0x3d, 0xd3, 0x92, 0x7b, 0x5d, + 0x4d, 0x71, 0xdc, 0xf8, 0x8d, 0x1c, 0x21, 0x3f, 0x31, 0xad, 0x1d, 0x4a, 0x44, 0x18, 0xd0, 0x81, + 0x61, 0x1e, 0x19, 0x32, 0x21, 0x53, 0x14, 0x7b, 0x6c, 0xf0, 0x2d, 0xe3, 0xf2, 0x87, 0xff, 0xe5, + 0x74, 0xf1, 0xd1, 0x58, 0x51, 0x59, 0x34, 0x02, 0xad, 0xd7, 0xd3, 0xb5, 0xe5, 0x9d, 0x9d, 0xfa, + 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x51, 0x02, 0x8a, 0xbe, + 0x8e, 0xe2, 0xec, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x37, 0x02, 0x67, 0x24, + 0x66, 0xe7, 0x73, 0x98, 0x09, 0x69, 0x20, 0xf9, 0x93, 0x69, 0x20, 0x7b, 0xe4, 0x37, 0x1e, 0xdd, + 0x87, 0x59, 0xe7, 0xd8, 0x90, 0x59, 0x3c, 0x1e, 0x8b, 0xe1, 0x70, 0xc3, 0x0e, 0x0a, 0x0e, 0xd1, + 0x07, 0xa1, 0xd3, 0xf8, 0x0d, 0x5b, 0xfc, 0x23, 0x01, 0x10, 0x55, 0x54, 0x9d, 0xed, 0xe9, 0x7f, + 0x5d, 0xc6, 0xd3, 0x5d, 0x28, 0xd2, 0x08, 0x47, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, 0x46, + 0x8b, 0x0f, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0xab, 0x30, 0x17, 0x6a, 0x40, + 0x9c, 0x9d, 0x7d, 0x1d, 0x66, 0xf6, 0xcc, 0x9e, 0xa1, 0xc9, 0xec, 0x8d, 0x07, 0xdf, 0x0e, 0xcc, + 0x52, 0x1a, 0x7b, 0x9e, 0xf8, 0x3f, 0x12, 0x30, 0x2f, 0x61, 0xdb, 0x6c, 0x1f, 0xe2, 0xf8, 0x55, + 0x58, 0x03, 0xfe, 0xae, 0x45, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x5b, 0xe6, 0xc2, 0x7b, 0xea, + 0x37, 0x47, 0x8f, 0xd8, 0xc1, 0x5d, 0x74, 0xbe, 0x27, 0x97, 0x0a, 0xed, 0xc9, 0x99, 0x50, 0xd0, + 0x5b, 0x86, 0x49, 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x8b, 0x54, 0x96, 0x47, 0x55, 0xb2, + 0xce, 0x58, 0x1a, 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xf2, 0x65, 0x52, 0xdf, 0xb3, 0xd3, + 0xc5, 0x7c, 0xe8, 0x9e, 0x2d, 0xe5, 0x75, 0xef, 0x9a, 0x48, 0x17, 0xbf, 0x0d, 0x97, 0xfa, 0x94, + 0x1d, 0xa7, 0xc7, 0xf3, 0x6f, 0x92, 0x70, 0x35, 0x2c, 0x3e, 0x6e, 0xfc, 0xf1, 0x75, 0xef, 0xd0, + 0x1a, 0xe4, 0x3a, 0xba, 0xf1, 0x66, 0x5b, 0x8b, 0x33, 0x1d, 0xdd, 0xf0, 0xb7, 0x71, 0x23, 0x86, + 0xc6, 0xd4, 0x57, 0x3a, 0x34, 0x14, 0x58, 0x88, 0xea, 0xbb, 0x38, 0xc7, 0xc7, 0xaf, 0x09, 0x30, + 0x13, 0xf7, 0x9e, 0xd9, 0x9b, 0xc5, 0x9b, 0x89, 0x4d, 0xc8, 0x7d, 0x05, 0x9b, 0x6c, 0xbf, 0x23, + 0x00, 0x6a, 0x5a, 0x3d, 0x83, 0x80, 0xda, 0x67, 0x66, 0x2b, 0xce, 0x66, 0xce, 0xc3, 0xa4, 0x6e, + 0x68, 0xf8, 0x98, 0x36, 0x33, 0x25, 0xb1, 0x8b, 0xd0, 0xab, 0xc3, 0xe4, 0x58, 0xaf, 0x0e, 0xc5, + 0xcf, 0x61, 0x2e, 0x54, 0xc5, 0x38, 0xdb, 0xff, 0x8f, 0x13, 0x30, 0xc7, 0x1b, 0x12, 0xfb, 0xf6, + 0xe2, 0x37, 0x61, 0xb2, 0x4d, 0x64, 0x8e, 0xe8, 0x67, 0xfa, 0x4c, 0xb7, 0x9f, 0x69, 0x61, 0xf4, + 0x73, 0x00, 0x5d, 0x0b, 0x1f, 0xca, 0x8c, 0x35, 0x39, 0x16, 0x6b, 0x86, 0x70, 0x50, 0x02, 0xfa, + 0x05, 0x28, 0x90, 0xf9, 0xdc, 0xb5, 0xcc, 0xae, 0x69, 0x13, 0x97, 0xc5, 0x1e, 0x0f, 0xe5, 0xcc, + 0x9e, 0x9d, 0x2e, 0xe6, 0x36, 0x75, 0x63, 0x9b, 0x33, 0x36, 0x1b, 0x12, 0x31, 0x0c, 0xde, 0xa5, + 0x2d, 0xfe, 0x27, 0x01, 0xe6, 0xbf, 0xb2, 0xad, 0xd8, 0xff, 0x1f, 0xba, 0x12, 0x5f, 0x42, 0x91, + 0xfe, 0xa8, 0x1b, 0x7b, 0x66, 0x9c, 0x9b, 0xe2, 0xdf, 0x13, 0x60, 0x36, 0x20, 0x38, 0x4e, 0xff, + 0xe4, 0x8d, 0xf4, 0x24, 0xfe, 0x22, 0xf1, 0x58, 0x82, 0x83, 0x3c, 0xce, 0x29, 0xf4, 0x07, 0x09, + 0xb8, 0x5c, 0x61, 0xaf, 0x90, 0xdd, 0x98, 0x8a, 0x38, 0x47, 0x46, 0x09, 0xa6, 0x0f, 0xb1, 0x65, + 0xeb, 0x26, 0x5b, 0x3d, 0x73, 0x92, 0x7b, 0x89, 0x16, 0x20, 0x6d, 0x1b, 0x4a, 0xd7, 0xde, 0x37, + 0xdd, 0x77, 0x67, 0xde, 0xb5, 0x17, 0xff, 0x31, 0xf9, 0xe6, 0xf1, 0x1f, 0x53, 0xa3, 0xe3, 0x3f, + 0xa6, 0x7f, 0x82, 0xf8, 0x0f, 0xfe, 0xa2, 0xea, 0x3f, 0x0a, 0x70, 0x65, 0x40, 0x73, 0x71, 0x8e, + 0x96, 0xef, 0x42, 0x56, 0xe5, 0x82, 0x89, 0xbd, 0x65, 0x6f, 0xe1, 0xea, 0xa4, 0xd8, 0x1b, 0xc2, + 0x8e, 0xb3, 0xd3, 0x45, 0x70, 0xab, 0x5a, 0x5f, 0xe3, 0xca, 0x21, 0xbf, 0x35, 0xf1, 0x57, 0x72, + 0x50, 0xa8, 0x1e, 0xb3, 0x1d, 0xe8, 0x06, 0x5b, 0xe5, 0xd1, 0x13, 0x48, 0x77, 0x2d, 0xf3, 0x50, + 0x77, 0x9b, 0x91, 0x0f, 0xbd, 0xfc, 0x77, 0x9b, 0xd1, 0xc7, 0xb5, 0xcd, 0x39, 0x24, 0x8f, 0x17, + 0x35, 0x21, 0xf3, 0xcc, 0x54, 0x95, 0xf6, 0x13, 0xbd, 0xed, 0x8e, 0xfc, 0xf7, 0xcf, 0x17, 0xb4, + 0xec, 0xf1, 0x6c, 0x2b, 0xce, 0xbe, 0xdb, 0x09, 0x1e, 0x11, 0xd5, 0x21, 0x5d, 0x73, 0x9c, 0x2e, + 0xb9, 0xc9, 0x6d, 0xc7, 0x9d, 0x31, 0x84, 0x12, 0x16, 0x37, 0x56, 0xd4, 0x65, 0x47, 0x4d, 0x98, + 0x7d, 0x4a, 0x4f, 0x38, 0x55, 0xda, 0x66, 0x4f, 0xab, 0x98, 0xc6, 0x9e, 0xde, 0xe2, 0x76, 0xf7, + 0xf6, 0x18, 0x32, 0x9f, 0x56, 0x1a, 0xd2, 0xa0, 0x00, 0xb4, 0x0a, 0xe9, 0xc6, 0x23, 0x2e, 0x8c, + 0xb9, 0x65, 0xb7, 0xc6, 0x10, 0xd6, 0x78, 0x24, 0x79, 0x6c, 0x68, 0x1d, 0xb2, 0xab, 0x5f, 0xf4, + 0x2c, 0xcc, 0xa5, 0x4c, 0x0d, 0x8d, 0x3c, 0xe8, 0x97, 0x42, 0xb9, 0xa4, 0x20, 0x33, 0x6a, 0x40, + 0xfe, 0x95, 0x69, 0x1d, 0xb4, 0x4d, 0xc5, 0x6d, 0xe1, 0x34, 0x15, 0xf7, 0x8d, 0x31, 0xc4, 0xb9, + 0x8c, 0x52, 0x9f, 0x08, 0xf4, 0x6d, 0x28, 0x90, 0xce, 0x68, 0x2a, 0xbb, 0x6d, 0xb7, 0x92, 0x69, + 0x2a, 0xf5, 0xdd, 0x31, 0xa4, 0x7a, 0x9c, 0xee, 0x2b, 0x90, 0x3e, 0x51, 0x0b, 0x12, 0xe4, 0x42, + 0x83, 0x00, 0x21, 0x48, 0x75, 0x49, 0x7f, 0x0b, 0x34, 0x36, 0x88, 0xfe, 0x46, 0xef, 0xc1, 0xb4, + 0x61, 0x6a, 0xd8, 0x9d, 0x21, 0xb9, 0xf2, 0xfc, 0xd9, 0xe9, 0xe2, 0xd4, 0x96, 0xa9, 0x31, 0x87, + 0x84, 0xff, 0x92, 0xa6, 0x48, 0xa1, 0xba, 0xb6, 0xb0, 0x04, 0x29, 0xd2, 0xef, 0xc4, 0x30, 0xed, + 0x2a, 0x36, 0xde, 0xb1, 0x74, 0x2e, 0xcd, 0xbd, 0x5c, 0xf8, 0x67, 0x09, 0x48, 0x34, 0x1e, 0x11, + 0x97, 0x7b, 0xb7, 0xa7, 0x1e, 0x60, 0x87, 0xdf, 0xe7, 0x57, 0xd4, 0x15, 0xb7, 0xf0, 0x9e, 0xce, + 0x3c, 0xa3, 0x8c, 0xc4, 0xaf, 0xd0, 0x3b, 0x00, 0x8a, 0xaa, 0x62, 0xdb, 0x96, 0xdd, 0x93, 0x6f, + 0x19, 0x29, 0xc3, 0x28, 0x1b, 0xf8, 0x84, 0xb0, 0xd9, 0x58, 0xb5, 0xb0, 0xe3, 0x06, 0x36, 0xb1, + 0x2b, 0xc2, 0xe6, 0xe0, 0x4e, 0x57, 0x76, 0xcc, 0x03, 0x6c, 0xd0, 0x71, 0x92, 0x21, 0xa6, 0xa6, + 0xd3, 0x6d, 0x12, 0x02, 0xb1, 0x92, 0xd8, 0xd0, 0x7c, 0x93, 0x96, 0x91, 0xbc, 0x6b, 0x22, 0xd2, + 0xc2, 0x2d, 0x9d, 0x1f, 0xe1, 0xca, 0x48, 0xfc, 0x8a, 0x68, 0x49, 0xe9, 0x39, 0xfb, 0xb4, 0x27, + 0x32, 0x12, 0xfd, 0x8d, 0x6e, 0x43, 0x81, 0xc5, 0x42, 0xca, 0xd8, 0x50, 0x65, 0x6a, 0x5c, 0x33, + 0xf4, 0x76, 0x8e, 0x91, 0xab, 0x86, 0x4a, 0x4c, 0x29, 0x7a, 0x04, 0x9c, 0x20, 0x1f, 0x74, 0x6c, + 0xa2, 0x53, 0x20, 0xa5, 0xca, 0x85, 0xb3, 0xd3, 0xc5, 0x6c, 0x83, 0xde, 0xd8, 0xd8, 0x6c, 0xd4, + 0xd7, 0xa4, 0x2c, 0x2b, 0xb5, 0xd1, 0xb1, 0xeb, 0xda, 0xc2, 0x6f, 0x08, 0x90, 0x7c, 0x5a, 0x69, + 0x5c, 0x58, 0x65, 0x6e, 0x45, 0x93, 0x81, 0x8a, 0xde, 0x81, 0xc2, 0xae, 0xde, 0x6e, 0xeb, 0x46, + 0x8b, 0x78, 0x41, 0xdf, 0xc5, 0xaa, 0xab, 0xb0, 0x3c, 0x27, 0x6f, 0x33, 0x2a, 0x5a, 0x82, 0xac, + 0x6a, 0x61, 0x0d, 0x1b, 0x8e, 0xae, 0xb4, 0x6d, 0xae, 0xb9, 0x20, 0x69, 0xe1, 0x97, 0x05, 0x98, + 0xa4, 0x33, 0x00, 0xbd, 0x0d, 0x19, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x29, 0xcb, 0x48, 0x3e, + 0x61, 0x68, 0xf5, 0xae, 0xc3, 0x8c, 0xa2, 0xaa, 0x66, 0xcf, 0x70, 0x64, 0x43, 0xe9, 0x60, 0x5e, + 0xcd, 0x2c, 0xa7, 0x6d, 0x29, 0x1d, 0x8c, 0x16, 0xc1, 0xbd, 0xf4, 0x0e, 0x34, 0x66, 0x24, 0xe0, + 0xa4, 0x0d, 0x7c, 0xb2, 0xf0, 0xc7, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, 0xd1, 0xc2, 0x06, 0xb6, + 0x14, 0xc7, 0xf4, 0xaa, 0xe1, 0x11, 0xfa, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0xce, 0xc3, 0xa4, 0x43, + 0xa6, 0x05, 0xaf, 0x01, 0xbb, 0xa0, 0xdb, 0xd1, 0x6d, 0xa5, 0xc5, 0x76, 0xe3, 0x32, 0x12, 0xbb, + 0x20, 0x8d, 0xe1, 0x21, 0xb5, 0x4c, 0x23, 0xfc, 0x8a, 0xd4, 0x94, 0x05, 0x7e, 0xee, 0xe2, 0x96, + 0x6e, 0xd0, 0xb1, 0x94, 0x94, 0x80, 0x92, 0xca, 0x84, 0x82, 0xde, 0x82, 0x0c, 0x2b, 0x80, 0x0d, + 0x8d, 0x0e, 0xa8, 0xa4, 0x94, 0xa6, 0x84, 0xaa, 0xa1, 0x2d, 0x60, 0xc8, 0x78, 0x93, 0x93, 0x74, + 0x5b, 0xcf, 0xf6, 0x14, 0x49, 0x7f, 0xa3, 0xf7, 0x61, 0xfe, 0x75, 0x4f, 0x69, 0xeb, 0x7b, 0x74, + 0xa3, 0x8d, 0x14, 0x63, 0x3a, 0x63, 0x2d, 0x41, 0xde, 0x3d, 0x2a, 0x81, 0xaa, 0xce, 0x9d, 0xcb, + 0x49, 0x7f, 0x2e, 0x8b, 0xbf, 0x2f, 0xc0, 0x2c, 0x8b, 0xea, 0x61, 0xc1, 0xa8, 0xf1, 0xf9, 0x21, + 0x1f, 0x43, 0x46, 0x53, 0x1c, 0x85, 0x1d, 0xd1, 0x4c, 0x8c, 0x3c, 0xa2, 0xe9, 0x1d, 0x25, 0x50, + 0x1c, 0x85, 0x1e, 0xd3, 0x44, 0x90, 0x22, 0xbf, 0xd9, 0x69, 0x56, 0x89, 0xfe, 0x16, 0x3f, 0x03, + 0x14, 0xac, 0x68, 0x9c, 0x1e, 0xd9, 0x3d, 0xb8, 0x44, 0x74, 0x5d, 0x35, 0x54, 0xeb, 0xa4, 0xeb, + 0xe8, 0xa6, 0xf1, 0x9c, 0xfe, 0xb5, 0x51, 0x31, 0xf0, 0x5e, 0x8a, 0xbe, 0x8e, 0x12, 0xff, 0x70, + 0x0a, 0x72, 0xd5, 0xe3, 0xae, 0x69, 0xc5, 0xba, 0x8b, 0x55, 0x86, 0x69, 0x0e, 0xf4, 0x47, 0xbc, + 0x17, 0xee, 0x33, 0xe6, 0xee, 0x2b, 0x57, 0xce, 0x88, 0xca, 0x00, 0x2c, 0x60, 0x94, 0x06, 0x05, + 0x25, 0x2f, 0xf0, 0xa6, 0x8c, 0xb2, 0x11, 0x2a, 0xda, 0x82, 0x6c, 0xe7, 0x50, 0x55, 0xe5, 0x3d, + 0xbd, 0xed, 0xf0, 0xb8, 0xbb, 0xe8, 0x10, 0xf1, 0xcd, 0x97, 0x95, 0xca, 0x13, 0x5a, 0x88, 0x85, + 0xc0, 0xf9, 0xd7, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x17, 0xf8, 0x91, 0x1a, 0xd9, 0x76, 0x4f, + 0xc9, 0x95, 0x73, 0x67, 0xa7, 0x8b, 0x19, 0x89, 0x52, 0x1b, 0x8d, 0xa6, 0x94, 0x61, 0x05, 0x1a, + 0xb6, 0x83, 0x6e, 0x40, 0xce, 0xec, 0xe8, 0x8e, 0xec, 0x3a, 0x49, 0xdc, 0xa3, 0x9c, 0x21, 0x44, + 0xd7, 0x89, 0x42, 0x4d, 0xb8, 0x83, 0x0d, 0x3a, 0xda, 0x49, 0x3b, 0xe5, 0x5d, 0xb6, 0xf9, 0xe8, + 0xb0, 0x19, 0x2d, 0x9b, 0x5d, 0x47, 0xef, 0xe8, 0x5f, 0xd0, 0x37, 0xd3, 0xfc, 0xa5, 0xd1, 0x0d, + 0x56, 0x9c, 0xb4, 0xaf, 0x4c, 0x77, 0x25, 0x79, 0xd9, 0xe7, 0x81, 0xa2, 0xe8, 0x6f, 0x0a, 0x70, + 0x99, 0x2b, 0x52, 0xde, 0xa5, 0x31, 0xee, 0x4a, 0x5b, 0x77, 0x4e, 0xe4, 0x83, 0xc3, 0x52, 0x9a, + 0xfa, 0xad, 0x3f, 0x1b, 0xd9, 0x21, 0x81, 0x71, 0xb0, 0xec, 0x76, 0xcb, 0xc9, 0x33, 0xce, 0xbc, + 0x71, 0x58, 0x35, 0x1c, 0xeb, 0xa4, 0x7c, 0xe5, 0xec, 0x74, 0x71, 0x6e, 0xf0, 0xee, 0x4b, 0x69, + 0xce, 0x1e, 0x64, 0x41, 0x35, 0x00, 0xec, 0x8d, 0x43, 0xba, 0x62, 0x44, 0xfb, 0x1f, 0x91, 0x03, + 0x56, 0x0a, 0xf0, 0xa2, 0xbb, 0x50, 0xe4, 0x27, 0x5b, 0xf6, 0xf4, 0x36, 0x96, 0x6d, 0xfd, 0x0b, + 0x4c, 0xd7, 0x96, 0xa4, 0x94, 0x67, 0x74, 0x22, 0xa2, 0xa1, 0x7f, 0x81, 0x17, 0xbe, 0x0b, 0xa5, + 0x61, 0xb5, 0x0f, 0x4e, 0x81, 0x0c, 0x7b, 0x23, 0xfb, 0x51, 0x78, 0x3b, 0x66, 0x8c, 0xa1, 0xca, + 0xb7, 0x64, 0x3e, 0x4e, 0x7c, 0x24, 0x88, 0xff, 0x24, 0x01, 0xb9, 0x72, 0xaf, 0x7d, 0xf0, 0xbc, + 0xdb, 0xe8, 0x75, 0x3a, 0x8a, 0x75, 0x42, 0xcc, 0x20, 0x33, 0x14, 0xa4, 0x82, 0x02, 0x33, 0x83, + 0xd4, 0x12, 0xe8, 0x5f, 0x60, 0xb2, 0x38, 0x05, 0x4f, 0x65, 0xb3, 0x18, 0x7e, 0xda, 0x86, 0xc0, + 0x51, 0x6b, 0xf3, 0xc8, 0x46, 0x1f, 0x41, 0x29, 0x50, 0x90, 0xee, 0x9d, 0xc8, 0xd8, 0x70, 0x2c, + 0x1d, 0xb3, 0xfd, 0xbf, 0xa4, 0x14, 0x88, 0x97, 0xa9, 0x93, 0xdb, 0x55, 0x76, 0x17, 0x35, 0x61, + 0x86, 0x14, 0x3c, 0x91, 0xe9, 0x12, 0xe2, 0xee, 0xcf, 0x3e, 0x88, 0x68, 0x56, 0xa8, 0xde, 0xcb, + 0x54, 0x3f, 0x15, 0xca, 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0x16, 0x3e, 0x85, 0x62, 0x7f, 0x81, + 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0xf3, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4f, 0xa5, 0x53, 0xc5, + 0x49, 0xf1, 0x2f, 0x92, 0x90, 0x77, 0x87, 0x59, 0x9c, 0x40, 0xa7, 0x0c, 0x93, 0x64, 0x50, 0xb8, + 0x31, 0x1e, 0xb7, 0x47, 0x8c, 0x6e, 0x1e, 0x35, 0x4e, 0x06, 0x8b, 0x0b, 0x92, 0x29, 0x6b, 0x1c, + 0x06, 0x67, 0xe1, 0x97, 0x13, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x42, 0x21, 0x8c, 0xb3, + 0x50, 0xd0, 0xa2, 0xde, 0x72, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x57, 0x3e, 0x78, 0xf0, + 0x90, 0x1a, 0x9b, 0x19, 0x89, 0x5f, 0xa1, 0x32, 0x0d, 0x3b, 0x32, 0x2d, 0x07, 0x6b, 0xdc, 0xa7, + 0x5f, 0x3a, 0xaf, 0x7f, 0xdd, 0x45, 0xc9, 0xe5, 0x43, 0x57, 0x21, 0x49, 0xac, 0xd8, 0x34, 0x0b, + 0x52, 0x38, 0x3b, 0x5d, 0x4c, 0x12, 0xfb, 0x45, 0x68, 0x68, 0x05, 0xb2, 0x61, 0x93, 0x41, 0x3c, + 0x38, 0x6a, 0x18, 0x03, 0xd3, 0x1d, 0xda, 0xde, 0xd4, 0x62, 0x78, 0x96, 0xf7, 0xf1, 0x5f, 0xa6, + 0x20, 0x57, 0xef, 0xc4, 0xbd, 0xa4, 0xac, 0x86, 0x7b, 0x38, 0x0a, 0x08, 0x85, 0x1e, 0x1a, 0xd1, + 0xc1, 0xa1, 0x15, 0x3c, 0x79, 0xb1, 0x15, 0xbc, 0x4e, 0x3c, 0x65, 0x9e, 0x66, 0x21, 0x39, 0x04, + 0xf3, 0x84, 0x9f, 0x4f, 0xfd, 0x14, 0x89, 0xf0, 0xf8, 0xe7, 0x28, 0x68, 0xa0, 0xc9, 0xa7, 0xd4, + 0x21, 0x67, 0xa3, 0x6c, 0x6a, 0xfc, 0x51, 0x36, 0x8d, 0x0d, 0x8d, 0x2e, 0x6a, 0x61, 0x8b, 0x3a, + 0xfd, 0xe6, 0x16, 0x75, 0xc1, 0xe1, 0x83, 0xf5, 0x63, 0x48, 0x6a, 0xba, 0xdb, 0x39, 0xe3, 0x2f, + 0xd5, 0x84, 0xe9, 0x9c, 0x51, 0x9b, 0x0a, 0x8e, 0x5a, 0x36, 0x4a, 0x16, 0xea, 0x00, 0xbe, 0x6e, + 0xd0, 0x12, 0x4c, 0x99, 0x6d, 0xcd, 0x3d, 0x48, 0x92, 0x2b, 0x67, 0xce, 0x4e, 0x17, 0x27, 0x9f, + 0xb7, 0xb5, 0xfa, 0x9a, 0x34, 0x69, 0xb6, 0xb5, 0xba, 0x46, 0x73, 0x5c, 0xe0, 0x23, 0xd9, 0x8b, + 0x32, 0x9b, 0x91, 0xa6, 0x0d, 0x7c, 0xb4, 0x86, 0x6d, 0x95, 0x0f, 0xb8, 0xdf, 0x16, 0x20, 0xef, + 0xea, 0x3e, 0x5e, 0xa3, 0x92, 0xd6, 0x3b, 0x7c, 0x92, 0x25, 0x2f, 0x36, 0xc9, 0x5c, 0x3e, 0x7e, + 0xbc, 0xf6, 0x57, 0x05, 0x1e, 0x37, 0xdc, 0x50, 0x15, 0x87, 0x38, 0x15, 0x31, 0x4e, 0x8c, 0x7b, + 0x50, 0xb4, 0x14, 0x43, 0x33, 0x3b, 0xfa, 0x17, 0x98, 0x6d, 0x84, 0xda, 0xfc, 0xad, 0x65, 0xc1, + 0xa3, 0xd3, 0x5d, 0x3f, 0x5b, 0xfc, 0xe3, 0x04, 0x8f, 0x31, 0xf6, 0xaa, 0x11, 0xa7, 0xba, 0xbe, + 0x03, 0xb3, 0xfd, 0x09, 0x47, 0xdc, 0xd9, 0xfa, 0x5e, 0x84, 0xbc, 0xa8, 0x8a, 0xb0, 0x58, 0x41, + 0x37, 0x70, 0xbd, 0x2f, 0xf9, 0x88, 0x8d, 0x2a, 0x90, 0x0d, 0xe6, 0x31, 0x49, 0x8e, 0x9d, 0xc7, + 0x04, 0x2c, 0x2f, 0x7b, 0xc9, 0xc2, 0xcf, 0xc3, 0x24, 0xbd, 0xfd, 0x06, 0x26, 0x9a, 0xf7, 0xe6, + 0x9f, 0x27, 0xe0, 0x26, 0xad, 0xfd, 0x4b, 0x6c, 0xe9, 0x7b, 0x27, 0xdb, 0x96, 0xe9, 0x60, 0xd5, + 0xc1, 0x9a, 0x7f, 0x0a, 0x24, 0x56, 0xbb, 0x97, 0xe9, 0xba, 0x0f, 0xb8, 0x50, 0xbc, 0x98, 0xc7, + 0x85, 0x36, 0xa0, 0xc0, 0x23, 0x03, 0x94, 0xb6, 0x7e, 0x88, 0x65, 0xc5, 0xb9, 0xc8, 0xea, 0x96, + 0x63, 0xbc, 0xab, 0x84, 0x75, 0xd5, 0x41, 0x1a, 0x64, 0xb8, 0x30, 0x5d, 0xe3, 0xe9, 0x77, 0x9e, + 0xfe, 0x64, 0x1b, 0x8a, 0x69, 0x16, 0x9e, 0x50, 0x5f, 0x93, 0xd2, 0x4c, 0x72, 0x5d, 0x13, 0xff, + 0xb3, 0x00, 0xb7, 0xce, 0x51, 0x71, 0x9c, 0x43, 0x77, 0x01, 0xd2, 0x87, 0xe4, 0x41, 0x3a, 0xd7, + 0x71, 0x5a, 0xf2, 0xae, 0xd1, 0x26, 0xe4, 0xf6, 0x14, 0xbd, 0xed, 0x0f, 0xe9, 0xe1, 0xe1, 0x85, + 0xd1, 0x91, 0xae, 0x33, 0x8c, 0x9d, 0x8d, 0x61, 0xf1, 0x37, 0x13, 0x30, 0xbb, 0xaa, 0x69, 0x8d, + 0x06, 0xb7, 0x81, 0xf1, 0x8d, 0x14, 0x17, 0x64, 0x26, 0x7c, 0x90, 0x89, 0xde, 0x03, 0xa4, 0xe9, + 0x36, 0x4b, 0xf3, 0x61, 0xef, 0x2b, 0x9a, 0x79, 0xe4, 0x87, 0x55, 0xcc, 0xba, 0x77, 0x1a, 0xee, + 0x0d, 0xd4, 0x00, 0x8a, 0x76, 0x64, 0xdb, 0x51, 0xbc, 0xf7, 0x46, 0xb7, 0xc6, 0x3a, 0xae, 0xc5, + 0x60, 0x90, 0x77, 0x29, 0x65, 0x88, 0x1c, 0xfa, 0x93, 0xf8, 0xed, 0x3a, 0x69, 0xba, 0x23, 0x2b, + 0xb6, 0x7b, 0x36, 0x87, 0x25, 0x18, 0xc9, 0x33, 0xfa, 0xaa, 0xcd, 0x8e, 0xdc, 0xb0, 0xc3, 0x04, + 0xbe, 0x6a, 0xe2, 0x84, 0xc4, 0xff, 0x40, 0x80, 0xbc, 0x84, 0xf7, 0x2c, 0x6c, 0xc7, 0xba, 0x29, + 0xf0, 0x04, 0x66, 0x2c, 0x26, 0x55, 0xde, 0xb3, 0xcc, 0xce, 0x45, 0xe6, 0x55, 0x96, 0x33, 0x3e, + 0xb1, 0xcc, 0x0e, 0x37, 0x2c, 0x2f, 0xa1, 0xe0, 0xd5, 0x31, 0xce, 0xc6, 0xff, 0x3e, 0x3d, 0x8a, + 0xcc, 0x04, 0xc7, 0x1d, 0xdf, 0x10, 0xaf, 0x06, 0xe8, 0x8b, 0xaa, 0x60, 0x45, 0xe3, 0x54, 0xc3, + 0x7f, 0x17, 0x20, 0xdf, 0xe8, 0xed, 0xb2, 0x3c, 0x52, 0xf1, 0x69, 0xa0, 0x0a, 0x99, 0x36, 0xde, + 0x73, 0xe4, 0x37, 0x0a, 0x83, 0x4f, 0x13, 0x56, 0x7a, 0x08, 0xe0, 0x29, 0x80, 0x45, 0x8f, 0xaf, + 0x51, 0x39, 0xc9, 0x0b, 0xca, 0xc9, 0x50, 0x5e, 0x42, 0x16, 0xff, 0x69, 0x02, 0x0a, 0x5e, 0x33, + 0xe3, 0xb4, 0x92, 0xaf, 0x42, 0xd6, 0x21, 0x79, 0x11, 0xeb, 0x30, 0xcb, 0x43, 0x3a, 0xa2, 0x2d, + 0xc4, 0x32, 0xcc, 0x51, 0xc7, 0x45, 0x56, 0xba, 0xdd, 0xb6, 0xee, 0xc2, 0x5d, 0x6a, 0x7f, 0x52, + 0xd2, 0x2c, 0xbd, 0xb5, 0xca, 0xee, 0x50, 0xa0, 0x4b, 0xc6, 0xdc, 0x9e, 0x85, 0xf1, 0x17, 0x58, + 0xa6, 0xc8, 0xeb, 0x22, 0x21, 0x2b, 0x59, 0xc6, 0xd8, 0x20, 0x7c, 0x7c, 0xcc, 0xbd, 0x82, 0x59, + 0xaa, 0xd3, 0xb8, 0x8f, 0xdd, 0x8a, 0x7f, 0x2f, 0x01, 0x28, 0x28, 0xf9, 0xab, 0xeb, 0x8b, 0x44, + 0x7c, 0x7d, 0xf1, 0x2e, 0x20, 0x16, 0xb4, 0x68, 0xcb, 0x5d, 0x6c, 0xc9, 0x36, 0x56, 0x4d, 0x9e, + 0xe3, 0x48, 0x90, 0x8a, 0xfc, 0xce, 0x36, 0xb6, 0x1a, 0x94, 0x8e, 0x1e, 0x03, 0xf8, 0x1e, 0x19, + 0x5f, 0x30, 0x46, 0x3a, 0x64, 0x52, 0xc6, 0x73, 0xc5, 0xc4, 0xef, 0x2d, 0xc0, 0x0c, 0xd7, 0xe1, + 0x8e, 0xa1, 0x9b, 0x06, 0x7a, 0x00, 0xc9, 0x16, 0xdf, 0xfa, 0xcf, 0x46, 0x6e, 0xd3, 0xf9, 0x49, + 0xdc, 0x6a, 0x13, 0x12, 0x29, 0x4b, 0x58, 0xba, 0x3d, 0x27, 0xc2, 0x31, 0xf2, 0x03, 0xb1, 0x83, + 0x2c, 0xdd, 0x9e, 0x83, 0x1a, 0x50, 0x50, 0xfd, 0x24, 0x55, 0x32, 0x61, 0x4f, 0x0e, 0x05, 0x51, + 0x91, 0x69, 0xc1, 0x6a, 0x13, 0x52, 0x5e, 0x0d, 0xdd, 0x40, 0x95, 0x60, 0x6e, 0xa4, 0xd4, 0x40, + 0x94, 0x97, 0x7f, 0xbe, 0x37, 0x9c, 0x97, 0xa9, 0x36, 0x11, 0x48, 0xa1, 0x84, 0x3e, 0x86, 0x29, + 0x8d, 0x66, 0xe1, 0xe1, 0x23, 0x3a, 0x6a, 0xd0, 0x85, 0x92, 0x1d, 0xd5, 0x26, 0x24, 0xce, 0x81, + 0xd6, 0x61, 0x86, 0xfd, 0x62, 0x6e, 0x0a, 0x47, 0x96, 0xb7, 0x86, 0x4b, 0x08, 0x2c, 0x04, 0xb5, + 0x09, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0x62, 0xcb, 0x6b, 0x43, 0x52, + 0x6a, 0xf8, 0xcc, 0xb4, 0x34, 0x7a, 0xcc, 0xd2, 0x32, 0x3a, 0xc7, 0xee, 0x36, 0x5f, 0x54, 0xf5, + 0x43, 0x47, 0xb7, 0x49, 0xf5, 0x31, 0x25, 0xa0, 0xa7, 0x90, 0x55, 0x88, 0xbf, 0x27, 0xd3, 0xc3, + 0x8f, 0x74, 0x5f, 0x2f, 0xfa, 0xe5, 0xfa, 0xc0, 0x61, 0xd5, 0x1a, 0x3d, 0x1f, 0xee, 0x12, 0x7d, + 0x41, 0x1d, 0x6c, 0xb5, 0x70, 0x29, 0x3b, 0x5a, 0x50, 0x30, 0xb6, 0xcb, 0x13, 0x44, 0x89, 0xc4, + 0xef, 0xf3, 0x4e, 0x1e, 0xd3, 0x46, 0xcd, 0x0c, 0x7d, 0x9d, 0x1b, 0x71, 0x6c, 0xa7, 0x36, 0x21, + 0xcd, 0xec, 0x07, 0xc8, 0x68, 0x19, 0x12, 0x2d, 0xb5, 0x94, 0x1b, 0x3a, 0x43, 0xbc, 0xa3, 0x29, + 0xb5, 0x09, 0x29, 0xd1, 0x52, 0xd1, 0xa7, 0x90, 0x66, 0xe7, 0x0c, 0x8e, 0x8d, 0x52, 0x7e, 0xa8, + 0x9d, 0x08, 0x9f, 0xd6, 0xa8, 0x4d, 0x48, 0xf4, 0x68, 0x03, 0x79, 0xde, 0x36, 0xe4, 0x2d, 0x16, + 0x1c, 0xe7, 0x86, 0xb2, 0x16, 0x87, 0xbe, 0xe2, 0x8e, 0x8a, 0x66, 0xad, 0x51, 0xcf, 0x3f, 0x40, + 0x47, 0xdf, 0x81, 0xf9, 0xb0, 0x44, 0x3e, 0xd2, 0x66, 0x87, 0xbe, 0xae, 0x1d, 0x1a, 0x59, 0x59, + 0x9b, 0x90, 0x90, 0x35, 0x70, 0x13, 0x7d, 0x08, 0x93, 0xac, 0xd7, 0x10, 0x15, 0x19, 0x15, 0xb5, + 0xd1, 0xd7, 0x61, 0xac, 0x3c, 0x19, 0xfc, 0x0e, 0x8f, 0x10, 0x93, 0xdb, 0x66, 0xab, 0x34, 0x37, + 0x74, 0xf0, 0x0f, 0xc6, 0xba, 0x91, 0xc1, 0xef, 0xf8, 0x54, 0xd2, 0xef, 0x16, 0xbb, 0xc3, 0xc3, + 0x8a, 0xe6, 0x87, 0xf6, 0x7b, 0x44, 0xe0, 0x58, 0x8d, 0xc6, 0xee, 0xfb, 0x64, 0x52, 0x35, 0x8b, + 0x65, 0x81, 0x91, 0xe9, 0x9c, 0xba, 0x34, 0xb4, 0x6a, 0x83, 0x69, 0x72, 0x6a, 0xd4, 0x47, 0xf2, + 0xa8, 0xe8, 0x25, 0x14, 0x79, 0xae, 0x06, 0xff, 0x9d, 0xc2, 0x65, 0x2a, 0xef, 0x5e, 0xa4, 0xe9, + 0x8a, 0x8a, 0xc9, 0xa9, 0x4d, 0x48, 0x05, 0x35, 0x7c, 0x07, 0x7d, 0x06, 0xb3, 0x54, 0x9e, 0xac, + 0xfa, 0xe9, 0x35, 0x4a, 0xa5, 0x81, 0x64, 0x0d, 0xc3, 0x33, 0x71, 0xb8, 0x92, 0x8b, 0x6a, 0xdf, + 0x2d, 0x32, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x95, 0x5d, 0x18, 0x3a, 0x8c, 0xc3, 0x49, 0xfd, 0xc8, + 0x30, 0xd6, 0x19, 0x85, 0x0c, 0x63, 0x87, 0xc7, 0x9c, 0xf1, 0xee, 0x78, 0x7b, 0xe8, 0x30, 0x8e, + 0x0a, 0x4e, 0x23, 0xc3, 0xd8, 0x09, 0xd2, 0xc9, 0x30, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, + 0x87, 0xf1, 0xd0, 0x43, 0xc8, 0x64, 0x18, 0x2b, 0x03, 0x37, 0xd1, 0x1a, 0x00, 0x73, 0x67, 0xe8, + 0xa2, 0x78, 0x6d, 0xe8, 0x62, 0xd0, 0x1f, 0x7b, 0x46, 0x16, 0x83, 0xb6, 0x4b, 0x23, 0x86, 0x8c, + 0x82, 0x25, 0x99, 0xbe, 0x42, 0x2d, 0x2d, 0x0e, 0x35, 0x64, 0x03, 0xaf, 0x3b, 0x89, 0x21, 0x3b, + 0xf2, 0x88, 0x64, 0x55, 0x61, 0x7b, 0xbe, 0xa5, 0xa5, 0xe1, 0x66, 0x39, 0xf8, 0xea, 0x87, 0x9a, + 0x65, 0x4a, 0x40, 0xab, 0x90, 0x21, 0x6b, 0xfe, 0x09, 0x35, 0x43, 0xd7, 0x87, 0xfa, 0xa4, 0x7d, + 0x87, 0x53, 0x6a, 0x13, 0x52, 0xfa, 0x35, 0x27, 0x91, 0xc7, 0xb3, 0xdd, 0xb0, 0x92, 0x38, 0xf4, + 0xf1, 0xa1, 0x9d, 0x53, 0xf2, 0x78, 0xc6, 0x81, 0x54, 0xb8, 0xc4, 0xfa, 0x8a, 0x9f, 0x07, 0xb6, + 0xf8, 0x11, 0xd6, 0xd2, 0x0d, 0x2a, 0x6a, 0xe8, 0xb6, 0x52, 0xe4, 0x31, 0xe5, 0xda, 0x84, 0x34, + 0xa7, 0x0c, 0xde, 0x25, 0x13, 0x9e, 0x2f, 0x3d, 0x6c, 0x33, 0xaa, 0x74, 0x73, 0xe8, 0x84, 0x8f, + 0xd8, 0xc3, 0x23, 0x13, 0x5e, 0x09, 0x90, 0xd9, 0x02, 0xa4, 0xc9, 0xb6, 0xcd, 0x5e, 0xb8, 0xdf, + 0x1a, 0xb1, 0x00, 0xf5, 0xed, 0x02, 0xb0, 0x05, 0x48, 0x6b, 0x30, 0x4e, 0x22, 0x48, 0x6d, 0x63, + 0xc5, 0xe2, 0x66, 0xf6, 0xf6, 0x50, 0x41, 0x03, 0xf9, 0xf3, 0x88, 0x20, 0xd5, 0x23, 0x12, 0x87, + 0xc7, 0x72, 0x93, 0xbc, 0x70, 0x87, 0xf1, 0xce, 0x50, 0x87, 0x27, 0x32, 0x0b, 0x0d, 0x71, 0x78, + 0xac, 0xd0, 0x0d, 0xf4, 0x73, 0x30, 0xcd, 0xe1, 0x5b, 0xe9, 0xee, 0x08, 0x37, 0x36, 0x88, 0xb8, + 0xc9, 0xbc, 0xe6, 0x3c, 0xcc, 0xca, 0x32, 0xd8, 0xc8, 0x9a, 0x77, 0x6f, 0x84, 0x95, 0x1d, 0x40, + 0xae, 0xcc, 0xca, 0xfa, 0x64, 0x62, 0x65, 0xd9, 0x38, 0xe5, 0x6b, 0xdd, 0xfd, 0xa1, 0x56, 0x76, + 0xf0, 0xe8, 0x0b, 0xb1, 0xb2, 0xaf, 0x7d, 0x2a, 0x69, 0x99, 0xcd, 0xe0, 0x53, 0xe9, 0x1b, 0x43, + 0x5b, 0x16, 0xc6, 0x91, 0xa4, 0x65, 0x9c, 0x87, 0x74, 0x1b, 0x73, 0x89, 0x99, 0xa6, 0xdf, 0x1d, + 0x7e, 0xe8, 0xbe, 0x1f, 0x74, 0xd4, 0xdc, 0x8d, 0x4a, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0xfc, 0xd0, + 0x31, 0xd7, 0xd4, 0x7b, 0xa3, 0x0d, 0x55, 0xd4, 0x49, 0x6a, 0xcf, 0x50, 0x85, 0x6e, 0xd2, 0xaa, + 0xb2, 0x73, 0x66, 0x74, 0x7e, 0x2f, 0x8f, 0xc8, 0x0f, 0xd0, 0x77, 0xda, 0x8f, 0x56, 0xd5, 0x23, + 0xfa, 0x53, 0xa8, 0xc7, 0x92, 0x57, 0x94, 0x56, 0x46, 0x4f, 0xa1, 0x70, 0xfa, 0x0c, 0x6f, 0x0a, + 0x71, 0xb2, 0xb7, 0x66, 0xba, 0x1e, 0xc6, 0xfb, 0xa3, 0xd7, 0xcc, 0x7e, 0xd7, 0x82, 0xad, 0x99, + 0xdc, 0xa7, 0xf8, 0xeb, 0x02, 0x2c, 0xb1, 0xba, 0xd1, 0x1d, 0xbd, 0x13, 0xd9, 0xdb, 0x17, 0x0d, + 0x9c, 0x79, 0x78, 0x40, 0x1f, 0xf0, 0xe1, 0xb0, 0xea, 0x9e, 0xb3, 0xcf, 0x5b, 0x9b, 0x90, 0xde, + 0x51, 0x46, 0x95, 0x2b, 0x4f, 0xf3, 0xd7, 0xa2, 0xde, 0x21, 0xcf, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, + 0x5f, 0x29, 0x96, 0xd6, 0x53, 0xe9, 0xab, 0xc5, 0x85, 0xf5, 0x54, 0xfa, 0xad, 0xe2, 0xdb, 0xe2, + 0x5f, 0x5e, 0x85, 0x9c, 0x8b, 0xfc, 0x18, 0x22, 0x7a, 0x18, 0x44, 0x44, 0xd7, 0x86, 0x21, 0x22, + 0x8e, 0x15, 0x39, 0x24, 0x7a, 0x18, 0x84, 0x44, 0xd7, 0x86, 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, + 0xe6, 0x30, 0x4c, 0x74, 0x6f, 0x0c, 0x4c, 0xe4, 0x89, 0xea, 0x07, 0x45, 0x6b, 0x83, 0xa0, 0xe8, + 0xe6, 0x68, 0x50, 0xe4, 0x89, 0x0a, 0xa0, 0xa2, 0xc7, 0x7d, 0xa8, 0xe8, 0xfa, 0x08, 0x54, 0xe4, + 0xf1, 0xbb, 0xb0, 0x68, 0x23, 0x12, 0x16, 0xdd, 0x3e, 0x0f, 0x16, 0x79, 0x72, 0x42, 0xb8, 0xe8, + 0x83, 0x10, 0x2e, 0x5a, 0x1c, 0x8a, 0x8b, 0x3c, 0x6e, 0x06, 0x8c, 0x3e, 0xe9, 0x07, 0x46, 0xd7, + 0x47, 0x00, 0x23, 0xbf, 0x05, 0x1c, 0x19, 0xd5, 0xa2, 0x90, 0xd1, 0xad, 0x73, 0x90, 0x91, 0x27, + 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xd6, 0x39, 0xd0, 0xa8, 0x4f, 0x12, 0xc3, 0x46, 0x5b, + 0xd1, 0xd8, 0xe8, 0xce, 0xb9, 0xd8, 0xc8, 0x93, 0x16, 0x06, 0x47, 0x2b, 0x01, 0x70, 0xf4, 0xce, + 0x10, 0x70, 0xe4, 0xb1, 0x12, 0x74, 0xf4, 0xad, 0x01, 0x74, 0x24, 0x8e, 0x42, 0x47, 0x1e, 0xaf, + 0x07, 0x8f, 0x5e, 0x0c, 0x81, 0x47, 0x77, 0xcf, 0x87, 0x47, 0x9e, 0xb0, 0x3e, 0x7c, 0xa4, 0x8c, + 0xc4, 0x47, 0xef, 0x8d, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, 0xe9, 0xa3, 0x30, 0x40, 0x5a, 0x1a, + 0x0e, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0x6d, 0x44, 0x22, 0xa4, 0xdb, 0xe7, 0x21, 0x24, 0x7f, 0x1e, + 0x04, 0x21, 0xd2, 0x56, 0x34, 0x44, 0xba, 0x73, 0x2e, 0x44, 0xf2, 0xbb, 0x3f, 0x84, 0x91, 0x36, + 0x22, 0x31, 0xd2, 0xed, 0xf3, 0x30, 0x92, 0x5f, 0xb9, 0x20, 0x48, 0x7a, 0x35, 0x14, 0x24, 0xdd, + 0x1f, 0x07, 0x24, 0x79, 0x42, 0x07, 0x50, 0xd2, 0xe7, 0xc3, 0x51, 0xd2, 0x37, 0x2e, 0x90, 0xaf, + 0x30, 0x12, 0x26, 0x7d, 0x6b, 0x00, 0x26, 0x89, 0xa3, 0x60, 0x92, 0x3f, 0x9e, 0x5d, 0x9c, 0xa4, + 0x8c, 0x44, 0x35, 0xef, 0x8d, 0x89, 0x6a, 0xfc, 0xc1, 0x17, 0x01, 0x6b, 0xaa, 0x11, 0xb0, 0xe6, + 0xe6, 0x68, 0x58, 0xe3, 0x9b, 0x73, 0x1f, 0xd7, 0xd4, 0xa2, 0x70, 0xcd, 0xad, 0x73, 0x70, 0x8d, + 0x6f, 0x85, 0x02, 0xc0, 0xe6, 0x71, 0x1f, 0xb0, 0xb9, 0x7e, 0x6e, 0xd4, 0x4f, 0x00, 0xd9, 0x94, + 0x07, 0x91, 0xcd, 0x8d, 0x91, 0xc8, 0xc6, 0x93, 0xe0, 0x43, 0x9b, 0xc7, 0x7d, 0xd0, 0xe6, 0xfa, + 0x08, 0x68, 0xe3, 0x57, 0x80, 0x63, 0x1b, 0x6d, 0x34, 0xb6, 0x59, 0x1e, 0x17, 0xdb, 0x78, 0x82, + 0x23, 0xc1, 0xcd, 0x56, 0x34, 0xb8, 0xb9, 0x33, 0xe6, 0x0b, 0xf9, 0x01, 0x74, 0x53, 0x8b, 0x42, + 0x37, 0xb7, 0xce, 0x41, 0x37, 0xc1, 0x35, 0xc4, 0x83, 0x37, 0xb5, 0x28, 0x78, 0x73, 0xeb, 0x1c, + 0x78, 0xe3, 0x4b, 0x0a, 0xe0, 0x9b, 0xe6, 0x30, 0x7c, 0x73, 0x6f, 0x0c, 0x7c, 0xe3, 0x3b, 0x2f, + 0x7d, 0x00, 0xe7, 0xd3, 0x7e, 0x80, 0x23, 0x8e, 0x02, 0x38, 0xfe, 0x8c, 0x74, 0x11, 0xce, 0x56, + 0x34, 0xc2, 0xb9, 0x73, 0x2e, 0xc2, 0x09, 0x1a, 0xc9, 0x00, 0xc4, 0xd9, 0x88, 0x84, 0x38, 0xb7, + 0xcf, 0x83, 0x38, 0xbe, 0x91, 0x0c, 0x62, 0x9c, 0x4f, 0xfb, 0x31, 0x8e, 0x38, 0x0a, 0xe3, 0xf8, + 0x8d, 0x73, 0x41, 0x4e, 0x2d, 0x0a, 0xe4, 0xdc, 0x3a, 0x07, 0xe4, 0xf8, 0x9d, 0x17, 0x40, 0x39, + 0xca, 0x48, 0x94, 0xf3, 0xde, 0x98, 0x28, 0xa7, 0xcf, 0x70, 0x85, 0x61, 0x4e, 0x2d, 0x0a, 0xe6, + 0xdc, 0x3a, 0x07, 0xe6, 0x04, 0x2a, 0xeb, 0xe3, 0x9c, 0xad, 0x68, 0x9c, 0x73, 0xe7, 0x5c, 0x9c, + 0xd3, 0x37, 0x9b, 0x5c, 0xa0, 0xb3, 0x11, 0x09, 0x74, 0x6e, 0x9f, 0x07, 0x74, 0xfa, 0x16, 0x3e, + 0xee, 0x1c, 0xfc, 0xca, 0xf8, 0x48, 0xe7, 0xa3, 0x8b, 0x23, 0x1d, 0xef, 0x99, 0xb1, 0x40, 0x9d, + 0xf5, 0x54, 0xfa, 0xed, 0xe2, 0x3b, 0xe2, 0xdf, 0x9a, 0x86, 0xa9, 0x9a, 0x17, 0xe7, 0xe2, 0xd7, + 0x52, 0x78, 0x93, 0xbc, 0x48, 0x68, 0x8d, 0xcc, 0x58, 0x6a, 0xf7, 0xce, 0x4f, 0x75, 0x37, 0x98, + 0x8c, 0x8d, 0xb3, 0xbe, 0xc1, 0x01, 0x65, 0xf4, 0x01, 0xe4, 0x7a, 0x36, 0xb6, 0xe4, 0xae, 0xa5, + 0x9b, 0x96, 0xee, 0xb0, 0x53, 0x1c, 0x42, 0xb9, 0xf8, 0xe5, 0xe9, 0xe2, 0xcc, 0x8e, 0x8d, 0xad, + 0x6d, 0x4e, 0x97, 0x66, 0x7a, 0x81, 0x2b, 0xf7, 0x43, 0x51, 0x93, 0xe3, 0x7f, 0x28, 0xea, 0x05, + 0x14, 0x2d, 0xac, 0x68, 0x21, 0x0f, 0x84, 0xe5, 0x20, 0x8a, 0x1e, 0x33, 0xf4, 0x94, 0x95, 0x5b, + 0x92, 0xe6, 0x22, 0x2a, 0x58, 0x61, 0x22, 0x7a, 0x00, 0x97, 0x3a, 0xca, 0x31, 0x8d, 0x89, 0x94, + 0x5d, 0xa7, 0x8e, 0xc6, 0x39, 0xb2, 0x6f, 0x30, 0xa1, 0x8e, 0x72, 0x4c, 0xbf, 0x3a, 0xc5, 0x6e, + 0xd1, 0x4f, 0x49, 0xdc, 0x82, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0xcf, 0x3d, 0xcb, 0xf2, + 0xb6, 0xe6, 0x5c, 0x2a, 0x4b, 0x30, 0x7b, 0x1f, 0x66, 0x79, 0xb0, 0x7c, 0xe0, 0x15, 0x21, 0xcf, + 0xdf, 0xca, 0x6e, 0x78, 0x6f, 0x05, 0x51, 0x05, 0x0a, 0x2d, 0xc5, 0xc1, 0x47, 0xca, 0x89, 0xec, + 0x1e, 0xc5, 0xca, 0xd2, 0x94, 0x8c, 0x6f, 0x9d, 0x9d, 0x2e, 0xe6, 0x9e, 0xb2, 0x5b, 0x03, 0x27, + 0xb2, 0x72, 0xad, 0xc0, 0x0d, 0x0d, 0xdd, 0x81, 0x82, 0x62, 0x9f, 0x18, 0x2a, 0x55, 0x0f, 0x36, + 0xec, 0x9e, 0x4d, 0x21, 0x45, 0x5a, 0xca, 0x53, 0x72, 0xc5, 0xa5, 0xa2, 0xeb, 0x30, 0xc3, 0x23, + 0xc9, 0xd9, 0x67, 0x6c, 0x0a, 0xb4, 0xa9, 0xfc, 0xbb, 0x09, 0xf4, 0x4b, 0x36, 0xe8, 0x31, 0x2c, + 0xf0, 0xdc, 0xf2, 0x47, 0x8a, 0xa5, 0xc9, 0x54, 0xeb, 0xfe, 0xf8, 0x2c, 0x52, 0xb1, 0x57, 0x58, + 0x2e, 0x79, 0x52, 0x80, 0xa8, 0xda, 0xcf, 0x84, 0xb0, 0x05, 0xb3, 0x6a, 0x5b, 0xf7, 0x10, 0x00, + 0x6b, 0xf9, 0xec, 0x50, 0x3b, 0x5b, 0xa1, 0x65, 0xfd, 0x57, 0xa4, 0x05, 0x35, 0x4c, 0x40, 0x0d, + 0xa0, 0xd9, 0x5e, 0xe4, 0xae, 0xd9, 0xd6, 0xd5, 0x13, 0xea, 0xfc, 0x87, 0x73, 0x64, 0x8f, 0xcc, + 0x54, 0xff, 0x4a, 0xd1, 0x9d, 0x6d, 0xca, 0x29, 0xc1, 0x91, 0xf7, 0x9b, 0xe5, 0xb7, 0x5d, 0x4f, + 0xa5, 0x67, 0x8a, 0xb9, 0xf5, 0x54, 0x3a, 0x5f, 0x2c, 0x88, 0x7f, 0x5b, 0x80, 0x42, 0x5f, 0x5d, + 0x50, 0x0d, 0x2e, 0x69, 0xde, 0x54, 0x91, 0xf9, 0x51, 0x23, 0xdd, 0x34, 0x78, 0xda, 0xef, 0xb9, + 0x2f, 0x4f, 0x17, 0x0b, 0xb4, 0xf4, 0x53, 0xef, 0x96, 0x34, 0xef, 0x73, 0xf8, 0x54, 0xf4, 0x11, + 0xe4, 0x99, 0xfb, 0xe8, 0x7d, 0x99, 0x8d, 0xc6, 0x88, 0x97, 0x67, 0xbf, 0x3c, 0x5d, 0xcc, 0x51, + 0x9f, 0xd1, 0x4d, 0xcb, 0x2b, 0xe5, 0xda, 0xc1, 0x4b, 0xf1, 0x37, 0x04, 0x98, 0x09, 0x1d, 0xe7, + 0x79, 0xdc, 0xf7, 0x06, 0xfd, 0x6a, 0x34, 0xee, 0x1c, 0x16, 0x50, 0x97, 0xe6, 0xe3, 0xdc, 0x8d, + 0x4e, 0x5c, 0x1c, 0x8e, 0x5b, 0xe8, 0x2e, 0x8c, 0x1b, 0xaa, 0xe1, 0xb2, 0x7d, 0x9c, 0xfa, 0xfe, + 0x0f, 0x16, 0x27, 0xc4, 0x3f, 0x48, 0x41, 0x2e, 0x7c, 0x78, 0xa7, 0xde, 0x57, 0xaf, 0xa8, 0x75, + 0x21, 0xc4, 0xb1, 0x3c, 0x22, 0x6d, 0x61, 0xc6, 0xcf, 0xaf, 0xcf, 0xaa, 0xb9, 0x34, 0x22, 0x4e, + 0x20, 0x58, 0x4f, 0x9f, 0x71, 0xe1, 0x7b, 0x49, 0xcf, 0xbe, 0x2e, 0xc3, 0x24, 0x4d, 0xa3, 0xc3, + 0xab, 0x56, 0xea, 0x1f, 0x3d, 0xc4, 0x57, 0x26, 0xf7, 0x25, 0x56, 0x8c, 0xd8, 0xe3, 0xe6, 0x1b, + 0xe5, 0xa9, 0xf3, 0xa7, 0xc1, 0xc5, 0x3f, 0x84, 0xc7, 0xf3, 0x14, 0x4e, 0x5e, 0x2c, 0x4f, 0x21, + 0xfa, 0x25, 0x28, 0xa8, 0x66, 0xbb, 0xcd, 0xd6, 0x3a, 0x66, 0x91, 0x06, 0x33, 0x8f, 0x50, 0x11, + 0xfc, 0xdb, 0x85, 0xcb, 0xde, 0x37, 0x0c, 0x97, 0x25, 0xfe, 0x0d, 0xc3, 0x40, 0xac, 0x67, 0xde, + 0x13, 0xc6, 0x0c, 0x59, 0x5f, 0xd8, 0xe9, 0xf4, 0x9b, 0x84, 0x9d, 0xb2, 0x50, 0x65, 0x3e, 0x72, + 0xfe, 0x54, 0xe0, 0x81, 0x21, 0xcf, 0x4c, 0xf3, 0xa0, 0xe7, 0x85, 0x8b, 0x2e, 0x04, 0xb3, 0x06, + 0xa6, 0xbf, 0x3c, 0x5d, 0x4c, 0x49, 0x5e, 0xda, 0xc0, 0x28, 0xcb, 0x9f, 0xf8, 0xc9, 0x2c, 0xff, + 0x75, 0x98, 0xe9, 0x5a, 0x78, 0x0f, 0x3b, 0xea, 0xbe, 0x6c, 0xf4, 0x3a, 0xfc, 0x54, 0x49, 0xd6, + 0xa5, 0x6d, 0xf5, 0x3a, 0xe8, 0x1e, 0x14, 0xbd, 0x22, 0x1c, 0x63, 0xbb, 0x69, 0xa5, 0x5c, 0x3a, + 0x47, 0xe4, 0xe2, 0xff, 0x16, 0x60, 0x2e, 0xd4, 0x26, 0x3e, 0x27, 0xd6, 0x21, 0xeb, 0x9b, 0x03, + 0xbb, 0x24, 0x5c, 0x30, 0x78, 0x32, 0xc8, 0x8c, 0x64, 0xb8, 0xec, 0x3e, 0x96, 0xe6, 0x94, 0xf7, + 0xc5, 0x26, 0x2e, 0x28, 0xf6, 0x92, 0x2f, 0x67, 0x2d, 0xf0, 0x00, 0x6f, 0x92, 0x24, 0xc7, 0x9a, + 0x24, 0xe2, 0x6f, 0x0b, 0x50, 0xa4, 0x0f, 0x78, 0x82, 0xb1, 0x16, 0x8b, 0x75, 0x72, 0x83, 0x92, + 0x13, 0xe3, 0x9f, 0x1b, 0x09, 0x7d, 0x07, 0x23, 0x19, 0xfe, 0x0e, 0x86, 0xf8, 0x03, 0x01, 0xf2, + 0x5e, 0x0d, 0xd9, 0xd7, 0xe0, 0x46, 0x24, 0xa7, 0x7c, 0xb3, 0x2f, 0xa1, 0xb9, 0x59, 0x38, 0xc6, + 0xfa, 0x40, 0x5d, 0x30, 0x0b, 0x07, 0xfb, 0x82, 0xd7, 0xdf, 0x75, 0x47, 0x0e, 0xa9, 0x62, 0xc5, + 0xcf, 0xb6, 0xf0, 0x06, 0x47, 0x68, 0x24, 0xfa, 0x21, 0x4d, 0xb3, 0x7d, 0xc8, 0x12, 0x9f, 0x8c, + 0x65, 0xb6, 0x10, 0x0f, 0x87, 0x02, 0xbe, 0x1b, 0xa7, 0x35, 0x1b, 0xf4, 0x13, 0x9b, 0xec, 0xb7, + 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x76, 0x3e, 0xd1, 0xd2, 0x58, 0xa6, 0xd4, 0xd5, 0x12, 0x1b, 0x2b, + 0x7f, 0x12, 0xec, 0x89, 0xea, 0x21, 0x41, 0x61, 0x8f, 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x85, 0x81, + 0x85, 0x7a, 0x4e, 0x22, 0xa5, 0xd1, 0x93, 0x50, 0x92, 0x8a, 0xc4, 0x70, 0xc4, 0x30, 0xa8, 0xd2, + 0x50, 0x32, 0x8b, 0x0f, 0xc3, 0x63, 0x7d, 0xe4, 0xe3, 0x83, 0x83, 0xfe, 0xe3, 0xd4, 0x0f, 0x7f, + 0xb0, 0x28, 0x88, 0x9f, 0x00, 0x92, 0xb0, 0x8d, 0x9d, 0x17, 0x3d, 0xd3, 0xf2, 0x13, 0x7e, 0xdc, + 0xee, 0xfb, 0x30, 0xc8, 0x64, 0x39, 0x7b, 0x16, 0x95, 0xd2, 0xe7, 0x12, 0xcc, 0x85, 0xb8, 0x99, + 0xb1, 0x10, 0x3f, 0x84, 0xab, 0x4f, 0x4d, 0xdb, 0xd6, 0xbb, 0x04, 0x7a, 0xd2, 0x59, 0x49, 0x96, + 0x06, 0xcf, 0x3c, 0xa6, 0xbb, 0x74, 0x13, 0xc2, 0x60, 0x66, 0x24, 0x23, 0x79, 0xd7, 0xe2, 0x1f, + 0x0a, 0x70, 0x65, 0x90, 0x93, 0x69, 0x39, 0xea, 0xc4, 0xdf, 0xb4, 0x6a, 0xfa, 0xf9, 0xdd, 0xce, + 0x1f, 0xad, 0x6e, 0x71, 0xe2, 0x62, 0xf2, 0x67, 0xca, 0x1d, 0x85, 0x9a, 0x0f, 0x7e, 0xfa, 0x38, + 0xcf, 0xc9, 0x9b, 0x8c, 0xea, 0x5b, 0x92, 0xd4, 0x78, 0x96, 0xa4, 0x09, 0x85, 0x75, 0x53, 0x37, + 0x88, 0x27, 0xeb, 0xb6, 0x77, 0x15, 0xf2, 0xbb, 0xba, 0xa1, 0x58, 0x27, 0xb2, 0x7b, 0xa8, 0x9b, + 0x0d, 0x94, 0x85, 0xa8, 0xca, 0xb2, 0x12, 0x52, 0x8e, 0x71, 0xf0, 0x4b, 0xf1, 0x47, 0x02, 0x14, + 0x7d, 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, 0xb6, 0x7b, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0x66, 0x58, + 0xf4, 0x76, 0x85, 0x51, 0xeb, 0x6b, 0x52, 0x86, 0x17, 0xa8, 0x6b, 0xe8, 0x46, 0x38, 0x39, 0xc2, + 0x64, 0x19, 0xce, 0x06, 0x52, 0x22, 0x90, 0x6e, 0xb7, 0x1d, 0xd3, 0xf2, 0x30, 0x13, 0xef, 0x76, + 0x37, 0x95, 0x3a, 0x3d, 0xd3, 0x4b, 0xca, 0xad, 0x42, 0x9e, 0x2c, 0xf7, 0x87, 0xd8, 0x6b, 0x52, + 0xea, 0xfc, 0x26, 0x31, 0x0e, 0xb7, 0x49, 0xbf, 0x47, 0x9c, 0x54, 0xd6, 0x1b, 0x5e, 0x0f, 0x8f, + 0xb0, 0x68, 0x3f, 0x13, 0x4c, 0x49, 0x38, 0x5e, 0xd2, 0x46, 0xea, 0x89, 0x7c, 0x0b, 0xd2, 0xee, + 0x27, 0x8a, 0xf9, 0x04, 0xb9, 0xba, 0xcc, 0xbe, 0x61, 0xbc, 0xec, 0x7e, 0xc3, 0x78, 0x79, 0x8d, + 0x17, 0x60, 0x66, 0xfc, 0xfb, 0xff, 0x6d, 0x51, 0x90, 0x3c, 0xa6, 0xfb, 0x0d, 0x32, 0xc2, 0x07, + 0x56, 0x61, 0x94, 0x07, 0x08, 0x7c, 0x43, 0x86, 0x7f, 0xb5, 0x76, 0x75, 0x4d, 0xde, 0xd9, 0xaa, + 0x3c, 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x13, 0xfa, 0x02, 0x4d, + 0x82, 0x7d, 0xc7, 0xf6, 0xfe, 0xcf, 0x00, 0xf8, 0x1f, 0xb3, 0x22, 0xb2, 0x36, 0xaa, 0x9f, 0xc9, + 0x2f, 0x57, 0x9f, 0xed, 0x54, 0x1b, 0xc5, 0x09, 0x84, 0x20, 0x5f, 0x5e, 0x6d, 0x56, 0x6a, 0xb2, + 0x54, 0x6d, 0x6c, 0x3f, 0xdf, 0x6a, 0x54, 0xdd, 0xef, 0xdf, 0xde, 0x5f, 0x83, 0x99, 0x60, 0xba, + 0x1b, 0x34, 0x07, 0x85, 0x4a, 0xad, 0x5a, 0xd9, 0x90, 0x5f, 0xd6, 0x57, 0xe5, 0x17, 0x3b, 0xd5, + 0x9d, 0x6a, 0x71, 0x82, 0x56, 0x8d, 0x12, 0x9f, 0xec, 0x3c, 0x7b, 0x56, 0x14, 0x50, 0x01, 0xb2, + 0xec, 0x9a, 0x7e, 0xad, 0xa6, 0x98, 0xb8, 0xbf, 0x09, 0xd9, 0x40, 0x5a, 0x5b, 0xf2, 0xb8, 0xed, + 0x9d, 0x46, 0x4d, 0x6e, 0xd6, 0x37, 0xab, 0x8d, 0xe6, 0xea, 0xe6, 0x36, 0x93, 0x41, 0x69, 0xab, + 0xe5, 0xe7, 0x52, 0xb3, 0x28, 0x78, 0xd7, 0xcd, 0xe7, 0x3b, 0x95, 0x9a, 0xdb, 0x0c, 0x31, 0x95, + 0x4e, 0x16, 0x93, 0xf7, 0xff, 0x9a, 0x00, 0x57, 0x86, 0xa4, 0x7e, 0x41, 0x59, 0x98, 0xde, 0x31, + 0x68, 0x8e, 0xcf, 0xe2, 0x04, 0xca, 0x05, 0xb2, 0xbf, 0x14, 0x05, 0x94, 0x66, 0xf9, 0x37, 0x8a, + 0x09, 0x34, 0x05, 0x89, 0xc6, 0xa3, 0x62, 0x92, 0xd4, 0x34, 0x90, 0x3c, 0xa5, 0x98, 0x42, 0x19, + 0x9e, 0xb6, 0xa1, 0x38, 0x89, 0x66, 0xfc, 0xec, 0x09, 0xc5, 0x29, 0x22, 0xca, 0xcb, 0x42, 0x50, + 0x9c, 0xbe, 0x7f, 0x1d, 0x02, 0x27, 0xbd, 0x11, 0xc0, 0xd4, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, + 0x40, 0xd3, 0x90, 0x5c, 0x6d, 0xb7, 0x8b, 0xc2, 0xc3, 0x7f, 0x9d, 0x82, 0xb4, 0xfb, 0x55, 0x16, + 0xf4, 0x0c, 0x26, 0xd9, 0xd6, 0xf3, 0xe2, 0x70, 0xcf, 0x9e, 0x4e, 0xde, 0x85, 0xa5, 0xf3, 0x5c, + 0x7f, 0x71, 0x02, 0xfd, 0x15, 0xc8, 0x06, 0x3c, 0x26, 0x34, 0x74, 0xfb, 0x2c, 0xe4, 0x25, 0x2e, + 0xdc, 0x3e, 0xaf, 0x98, 0x27, 0xff, 0x15, 0x64, 0x3c, 0x0b, 0x8e, 0x6e, 0x8c, 0xb2, 0xef, 0xae, + 0xec, 0xd1, 0x8b, 0x00, 0x99, 0x6b, 0xe2, 0xc4, 0xfb, 0x02, 0xb2, 0x00, 0x0d, 0x1a, 0x5b, 0x14, + 0x15, 0x91, 0x30, 0xd4, 0x9a, 0x2f, 0xdc, 0x1f, 0xab, 0xb4, 0xff, 0x4c, 0xa2, 0x2c, 0x7f, 0xc5, + 0x88, 0x56, 0xd6, 0xc0, 0x7a, 0x14, 0xad, 0xac, 0x88, 0x85, 0x67, 0x02, 0xbd, 0x80, 0x14, 0xb1, + 0x94, 0x28, 0xca, 0x87, 0xec, 0xb3, 0xcc, 0x0b, 0x37, 0x46, 0x96, 0x71, 0x45, 0x96, 0xef, 0xfd, + 0xf0, 0x2f, 0xae, 0x4d, 0xfc, 0xf0, 0xec, 0x9a, 0xf0, 0xa3, 0xb3, 0x6b, 0xc2, 0x9f, 0x9d, 0x5d, + 0x13, 0xfe, 0xfc, 0xec, 0x9a, 0xf0, 0xeb, 0x3f, 0xbe, 0x36, 0xf1, 0xa3, 0x1f, 0x5f, 0x9b, 0xf8, + 0xb3, 0x1f, 0x5f, 0x9b, 0xf8, 0x7c, 0x9a, 0x73, 0xef, 0x4e, 0x51, 0xa3, 0xf2, 0xe8, 0xff, 0x05, + 0x00, 0x00, 0xff, 0xff, 0x2a, 0xe5, 0xf1, 0xd1, 0x53, 0x7e, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 6895eaa44b50..ab86df7a67b8 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -2091,6 +2091,24 @@ message RangeFeedEvent { RangeFeedError error = 3; } + +// ResetQuorumRequest makes a range that is unavailable due to lost quorum +// available again, at the cost of losing all of the data in the range. Any +// existing replica, even one residing on the target node, will irrevocably +// be removed. ResetQuorumRequest first uses meta2 to identify the range +// descriptor. Then, it removes all replicas from the range descriptor and +// adds a store from the target node as the one designated survivor replica. +// This change is then written to meta2 and sent as a snapshot to a store +// local to the target node. +// +// This RPC is called by the user directly and will not work for ranges +// that have not lost quorum or for a meta range. +message ResetQuorumRequest { + int32 range_id = 1 [(gogoproto.customname) = "RangeID"]; +} + +message ResetQuorumResponse {} + // GossipSubscriptionRequest initiates a game of telephone. It establishes an // indefinite stream that proxies gossip information overheard by the recipient // node back to the caller. Gossip information is filtered down to just those @@ -2137,6 +2155,7 @@ service Internal { rpc RangeLookup (RangeLookupRequest) returns (RangeLookupResponse) {} rpc RangeFeed (RangeFeedRequest) returns (stream RangeFeedEvent) {} rpc GossipSubscription (GossipSubscriptionRequest) returns (stream GossipSubscriptionEvent) {} + rpc ResetQuorum (ResetQuorumRequest) returns (ResetQuorumResponse) {} // Join a bootstrapped cluster. If the target node is itself not part of a // bootstrapped cluster, an appropriate error is returned. diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 26afb9b2f3c2..87ec44e35f08 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -476,6 +476,12 @@ func (a internalClientAdapter) Join( return a.InternalServer.Join(ctx, req) } +func (a internalClientAdapter) ResetQuorum( + ctx context.Context, req *roachpb.ResetQuorumRequest, _ ...grpc.CallOption, +) (*roachpb.ResetQuorumResponse, error) { + return a.InternalServer.ResetQuorum(ctx, req) +} + type respStreamClientAdapter struct { ctx context.Context respC chan interface{} diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index c11dfd467e4a..3397fc7f13ea 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -244,6 +244,12 @@ func (*internalServer) GossipSubscription( panic("unimplemented") } +func (*internalServer) ResetQuorum( + context.Context, *roachpb.ResetQuorumRequest, +) (*roachpb.ResetQuorumResponse, error) { + panic("unimplemented") +} + func (*internalServer) Join( context.Context, *roachpb.JoinNodeRequest, ) (*roachpb.JoinNodeResponse, error) { diff --git a/pkg/server/node.go b/pkg/server/node.go index 40b1f838727a..3564def10584 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -11,6 +11,7 @@ package server import ( + "bytes" "context" "fmt" "net" @@ -29,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvtenant" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server/status" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -995,6 +997,105 @@ func (n *Node) RangeFeed( return nil } +// ResetQuorum implements the roachpb.InternalServer interface. +func (n *Node) ResetQuorum( + ctx context.Context, req *roachpb.ResetQuorumRequest, +) (_ *roachpb.ResetQuorumResponse, rErr error) { + // Get range descriptor and save original value of the descriptor for the input range id. + var desc roachpb.RangeDescriptor + var expValue roachpb.Value + if err := n.storeCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + kvs, err := sql.ScanMetaKVs(ctx, txn, roachpb.Span{ + Key: roachpb.KeyMin, + EndKey: roachpb.KeyMax, + }) + if err != nil { + return err + } + + for i := range kvs { + if err := kvs[i].Value.GetProto(&desc); err != nil { + return err + } + if desc.RangeID == roachpb.RangeID(req.RangeID) { + expValue = *kvs[i].Value + return nil + } + } + return errors.Errorf("r%d not found", req.RangeID) + }); err != nil { + return nil, err + } + + // Check that we've actually lost quorum. + livenessMap := n.storeCfg.NodeLiveness.GetIsLiveMap() + available := desc.Replicas().CanMakeProgress(func(rDesc roachpb.ReplicaDescriptor) bool { + return livenessMap[rDesc.NodeID].IsLive + }) + if available { + return nil, errors.Errorf("targeted range to recover has not lost quorum.") + } + // Check that we're not a metaX range. + if bytes.HasPrefix(desc.StartKey, keys.Meta1Prefix) || bytes.HasPrefix(desc.StartKey, keys.Meta2Prefix) { + return nil, errors.Errorf("targeted range to recover is a meta1 or meta2 range.") + } + + // Update the range descriptor and update meta ranges for the descriptor, removing all replicas. + deadReplicas := append([]roachpb.ReplicaDescriptor(nil), desc.Replicas().All()...) + for _, rd := range deadReplicas { + desc.RemoveReplica(rd.NodeID, rd.StoreID) + } + // Pick any store on the current node to send the snapshot to. + var storeID roachpb.StoreID + if err := n.stores.VisitStores(func(s *kvserver.Store) error { + if storeID == 0 { + storeID = s.StoreID() + } + return nil + }); err != nil { + return nil, err + } + if storeID == 0 { + return nil, errors.New("no store found") + } + // Add current node as new replica. + toReplicaDescriptor := desc.AddReplica(n.Descriptor.NodeID, storeID, roachpb.VOTER_FULL) + // Increment the generation so that the various caches will recognize this descriptor as newer. + desc.IncrementGeneration() + + log.Infof(ctx, "starting recovery using desc %+v", desc) + // Update the meta2 entry. Note that we're intentionally + // eschewing updateRangeAddressing since the copy of the + // descriptor that resides on the range itself has lost quorum. + metaKey := keys.RangeMetaKey(desc.EndKey).AsRawKey() + + if err := n.storeCfg.DB.CPut(ctx, metaKey, &desc, expValue.TagAndDataBytes()); err != nil { + return nil, err + } + + // Set up connection to self. Use rpc.SystemClass to avoid throttling. + conn, err := n.storeCfg.NodeDialer.Dial(ctx, n.Descriptor.NodeID, rpc.SystemClass) + if err != nil { + return nil, err + } + + // Initialize and send an empty snapshot to self in order to use crdb + // internal upreplication and rebalancing mechanisms to create further + // replicas from this fresh snapshot. + if err := kvserver.SendEmptySnapshot( + ctx, + n.storeCfg.Settings, + conn, + n.storeCfg.Clock.Now(), + desc, + toReplicaDescriptor, + ); err != nil { + return nil, err + } + + return &roachpb.ResetQuorumResponse{}, nil +} + // GossipSubscription implements the roachpb.InternalServer interface. func (n *Node) GossipSubscription( args *roachpb.GossipSubscriptionRequest, stream roachpb.Internal_GossipSubscriptionServer,