diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index fa79cb4e6dea..12ef97af0d53 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -158,14 +158,9 @@ const ( // NoOriginFKIndexes allows for foreign keys to no longer need indexes on // the origin side of the relationship. NoOriginFKIndexes - // ClientRangeInfosOnBatchResponse moves the response RangeInfos from - // individual response headers to the batch header. - ClientRangeInfosOnBatchResponse // NodeMembershipStatus gates the usage of the MembershipStatus enum in the // Liveness proto. See comment on proto definition for more details. NodeMembershipStatus - // RangeStatsRespHasDesc adds the RangeStatsResponse.RangeInfo field. - RangeStatsRespHasDesc // MinPasswordLength adds the server.user_login.min_password_length setting. MinPasswordLength // AbortSpanBytes adds a field to MVCCStats @@ -286,18 +281,10 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: NoOriginFKIndexes, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 9}, }, - { - Key: ClientRangeInfosOnBatchResponse, - Version: roachpb.Version{Major: 20, Minor: 1, Internal: 10}, - }, { Key: NodeMembershipStatus, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 11}, }, - { - Key: RangeStatsRespHasDesc, - Version: roachpb.Version{Major: 20, Minor: 1, Internal: 12}, - }, { Key: MinPasswordLength, Version: roachpb.Version{Major: 20, Minor: 1, Internal: 13}, diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 397d417996fd..552c79047387 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -18,31 +18,29 @@ func _() { _ = x[AddScheduledJobsTable-7] _ = x[UserDefinedSchemas-8] _ = x[NoOriginFKIndexes-9] - _ = x[ClientRangeInfosOnBatchResponse-10] - _ = x[NodeMembershipStatus-11] - _ = x[RangeStatsRespHasDesc-12] - _ = x[MinPasswordLength-13] - _ = x[AbortSpanBytes-14] - _ = x[AlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-15] - _ = x[MaterializedViews-16] - _ = x[Box2DType-17] - _ = x[UpdateScheduledJobsSchema-18] - _ = x[CreateLoginPrivilege-19] - _ = x[HBAForNonTLS-20] - _ = x[V20_2-21] - _ = x[Start21_1-22] - _ = x[EmptyArraysInInvertedIndexes-23] - _ = x[UniqueWithoutIndexConstraints-24] - _ = x[VirtualComputedColumns-25] - _ = x[CPutInline-26] - _ = x[ReplicaVersions-27] - _ = x[TruncatedAndRangeAppliedStateMigration-28] - _ = x[PostTruncatedAndRangeAppliedStateMigration-29] + _ = x[NodeMembershipStatus-10] + _ = x[MinPasswordLength-11] + _ = x[AbortSpanBytes-12] + _ = x[AlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTable-13] + _ = x[MaterializedViews-14] + _ = x[Box2DType-15] + _ = x[UpdateScheduledJobsSchema-16] + _ = x[CreateLoginPrivilege-17] + _ = x[HBAForNonTLS-18] + _ = x[V20_2-19] + _ = x[Start21_1-20] + _ = x[EmptyArraysInInvertedIndexes-21] + _ = x[UniqueWithoutIndexConstraints-22] + _ = x[VirtualComputedColumns-23] + _ = x[CPutInline-24] + _ = x[ReplicaVersions-25] + _ = x[TruncatedAndRangeAppliedStateMigration-26] + _ = x[PostTruncatedAndRangeAppliedStateMigration-27] } -const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesClientRangeInfosOnBatchResponseNodeMembershipStatusRangeStatsRespHasDescMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigration" +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigration" -var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 211, 231, 252, 269, 283, 347, 364, 373, 398, 418, 430, 435, 444, 472, 501, 523, 533, 548, 586, 628} +var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 200, 217, 231, 295, 312, 321, 346, 366, 378, 383, 392, 420, 449, 471, 481, 496, 534, 576} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/kv/kvclient/kvcoord/BUILD.bazel b/pkg/kv/kvclient/kvcoord/BUILD.bazel index 8733adaf4ca7..67205200ab45 100644 --- a/pkg/kv/kvclient/kvcoord/BUILD.bazel +++ b/pkg/kv/kvclient/kvcoord/BUILD.bazel @@ -34,7 +34,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", - "//pkg/clusterversion", "//pkg/gossip", "//pkg/keys", "//pkg/kv", diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 81908c9594d9..8635f4fda8f8 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -20,7 +20,6 @@ import ( "unsafe" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -1554,17 +1553,6 @@ func (ds *DistSender) sendPartialBatch( // If sending succeeded, return immediately. if reply.Error == nil { - // 20.1 nodes return RangeInfos in individual responses. Let's move it to - // the br. - if ba.ReturnRangeInfo && - len(reply.RangeInfos) == 0 && - !ds.st.Version.IsActive(ctx, clusterversion.ClientRangeInfosOnBatchResponse) { - // All the responses have the same RangeInfos in them, so just look at the - // first one. - firstRes := reply.Responses[0].GetInner() - reply.RangeInfos = append(reply.RangeInfos, firstRes.Header().DeprecatedRangeInfos...) - } - return response{reply: reply, positions: positions} } @@ -1870,7 +1858,7 @@ func (ds *DistSender) sendToReplicas( prevReplica = curReplica // Communicate to the server the information our cache has about the range. // If it's stale, the serve will return an update. - ba.ClientRangeInfo = &roachpb.ClientRangeInfo{ + ba.ClientRangeInfo = roachpb.ClientRangeInfo{ // Note that DescriptorGeneration will be 0 if the cached descriptor is // "speculative" (see DescSpeculative()). Even if the speculation is // correct, we want the serve to return an update, at which point the @@ -1973,12 +1961,12 @@ func (ds *DistSender) sendToReplicas( if br.Error == nil { // If the server gave us updated range info, lets update our cache with it. - // - // TODO(andreimatei): shouldn't we do this unconditionally? Our cache knows how - // to disregard stale information. if len(br.RangeInfos) > 0 { log.VEventf(ctx, 2, "received updated range info: %s", br.RangeInfos) routing.EvictAndReplace(ctx, br.RangeInfos...) + // The field is cleared by the DistSender because it refers + // routing information not exposed by the KV API. + br.RangeInfos = nil } return br, nil } diff --git a/pkg/kv/kvserver/batcheval/cmd_range_stats.go b/pkg/kv/kvserver/batcheval/cmd_range_stats.go index aaa3be66d2c6..1fc7621e3c30 100644 --- a/pkg/kv/kvserver/batcheval/cmd_range_stats.go +++ b/pkg/kv/kvserver/batcheval/cmd_range_stats.go @@ -44,6 +44,6 @@ func RangeStats( reply.MVCCStats = cArgs.EvalCtx.GetMVCCStats() reply.QueriesPerSecond = cArgs.EvalCtx.GetSplitQPS() desc, lease := cArgs.EvalCtx.GetDescAndLease(ctx) - reply.RangeInfo = &roachpb.RangeInfo{Desc: desc, Lease: lease} + reply.RangeInfo = roachpb.RangeInfo{Desc: desc, Lease: lease} return result.Result{}, nil } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index fcb609e068ba..19a4e69bc793 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -1633,176 +1633,6 @@ func TestErrorHandlingForNonKVCommand(t *testing.T) { } } -func TestRangeInfo(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - }) - defer tc.Stopper().Stop(ctx) - - // Split the key space at key "a". - minKey := tc.ScratchRange(t) - splitKey := minKey.Next().Next() - tc.SplitRangeOrFatal(t, splitKey) - tc.AddVotersOrFatal(t, minKey, tc.Target(1)) - tc.AddVotersOrFatal(t, splitKey, tc.Target(1)) - - // Get the replicas for each side of the split. This is done within - // a SucceedsSoon loop to ensure the split completes. - var lhsReplica0, lhsReplica1, rhsReplica0, rhsReplica1 *kvserver.Replica - testutils.SucceedsSoon(t, func() error { - lhsReplica0 = tc.GetFirstStoreFromServer(t, 0).LookupReplica(roachpb.RKey(minKey)) - lhsReplica1 = tc.GetFirstStoreFromServer(t, 1).LookupReplica(roachpb.RKey(minKey)) - rhsReplica0 = tc.GetFirstStoreFromServer(t, 0).LookupReplica(roachpb.RKey(splitKey)) - rhsReplica1 = tc.GetFirstStoreFromServer(t, 1).LookupReplica(roachpb.RKey(splitKey)) - if lhsReplica0 == rhsReplica0 || lhsReplica1 == rhsReplica1 { - return errors.Errorf("replicas not post-split %v, %v, %v, %v", - lhsReplica0, rhsReplica0, rhsReplica0, rhsReplica1) - } - return nil - }) - lhsLease, _ := lhsReplica0.GetLease() - rhsDesc, rhsLease := rhsReplica0.GetDescAndLease(ctx) - - send := func(args roachpb.Request, returnRangeInfo bool, txn *roachpb.Transaction) *roachpb.BatchResponse { - ba := roachpb.BatchRequest{ - Header: roachpb.Header{ - ReturnRangeInfo: returnRangeInfo, - Txn: txn, - }, - } - ba.Add(args) - - br, pErr := tc.Servers[0].DistSender().Send(ctx, ba) - if pErr != nil { - t.Fatal(pErr) - } - return br - } - - // Populate the range cache so that the request will be sent to the right - // leaseholder, and it will have the up-to-date ClientRangeInfo populated. - tc.Servers[0].DistSender().RangeDescriptorCache().Insert(ctx, - roachpb.RangeInfo{Desc: rhsDesc, Lease: rhsLease}) - - // Verify range info is not set if the request is sent with up-to-date - // ClientRangeInfo. - getArgs := getArgs(splitKey) - br := send(getArgs, false /* returnRangeInfo */, nil /* txn */) - if len(br.RangeInfos) > 0 { - t.Fatalf("expected empty range infos if unrequested; got %v", br.RangeInfos) - } - - // Verify range info on a get request. - br = send(getArgs, true /* returnRangeInfo */, nil /* txn */) - expRangeInfos := []roachpb.RangeInfo{ - { - Desc: *rhsReplica0.Desc(), - Lease: rhsLease, - }, - } - if !reflect.DeepEqual(br.RangeInfos, expRangeInfos) { - t.Errorf("on get reply, expected %+v; got %+v", expRangeInfos, br.RangeInfos) - } - - // Verify range info on a put request. - br = send(putArgs(splitKey, []byte("foo")), true /* returnRangeInfo */, nil /* txn */) - if !reflect.DeepEqual(br.RangeInfos, expRangeInfos) { - t.Errorf("on put reply, expected %+v; got %+v", expRangeInfos, br.RangeInfos) - } - - // Verify range info on an admin request. - adminArgs := &roachpb.AdminTransferLeaseRequest{ - RequestHeader: roachpb.RequestHeader{ - Key: splitKey, - }, - Target: rhsLease.Replica.StoreID, - } - br = send(adminArgs, true /* returnRangeInfo */, nil /* txn */) - if !reflect.DeepEqual(br.RangeInfos, expRangeInfos) { - t.Errorf("on admin reply, expected %+v; got %+v", expRangeInfos, br.RangeInfos) - } - - // Verify multiple range infos on a scan request. - scanArgs := &roachpb.ScanRequest{ - RequestHeader: roachpb.RequestHeader{ - Key: minKey, - EndKey: roachpb.KeyMax, - }, - } - txn := roachpb.MakeTransaction("test", minKey, 1, tc.Servers[0].Clock().Now(), 0) - br = send(scanArgs, true /* returnRangeInfo */, &txn) - expRangeInfos = []roachpb.RangeInfo{ - { - Desc: *lhsReplica0.Desc(), - Lease: lhsLease, - }, - { - Desc: *rhsReplica0.Desc(), - Lease: rhsLease, - }, - } - if !reflect.DeepEqual(br.RangeInfos, expRangeInfos) { - t.Errorf("on scan reply, expected %+v; got %+v", expRangeInfos, br.RangeInfos) - } - - // Verify multiple range infos and order on a reverse scan request. - revScanArgs := &roachpb.ReverseScanRequest{ - RequestHeader: roachpb.RequestHeader{ - Key: minKey, - EndKey: roachpb.KeyMax, - }, - } - br = send(revScanArgs, true /* returnRangeInfo */, &txn) - expRangeInfos = []roachpb.RangeInfo{ - { - Desc: *lhsReplica0.Desc(), - Lease: lhsLease, - }, - { - Desc: *rhsReplica0.Desc(), - Lease: rhsLease, - }, - } - if !reflect.DeepEqual(br.RangeInfos, expRangeInfos) { - t.Errorf("on reverse scan reply, expected %+v; got %+v", expRangeInfos, br.RangeInfos) - } - - // Change lease holders for both ranges and re-scan. - for _, r := range []*kvserver.Replica{lhsReplica1, rhsReplica1} { - replDesc, err := r.GetReplicaDescriptor() - if err != nil { - t.Fatal(err) - } - if err = tc.GetFirstStoreFromServer(t, 0).DB().AdminTransferLease(context.Background(), - r.Desc().StartKey.AsRawKey(), replDesc.StoreID); err != nil { - t.Fatalf("unable to transfer lease to replica %s: %+v", r, err) - } - } - br = send(scanArgs, true /* returnRangeInfo */, &txn) - // Read the expected lease from replica0 rather than replica1 as it may serve - // a follower read which will contain the new lease information before - // replica1 has applied the lease transfer. - lhsLease, _ = lhsReplica0.GetLease() - rhsLease, _ = rhsReplica0.GetLease() - expRangeInfos = []roachpb.RangeInfo{ - { - Desc: *lhsReplica1.Desc(), - Lease: lhsLease, - }, - { - Desc: *rhsReplica1.Desc(), - Lease: rhsLease, - }, - } - if !reflect.DeepEqual(br.RangeInfos, expRangeInfos) { - t.Errorf("on scan reply, expected %+v; got %+v", expRangeInfos, br.RangeInfos) - } -} - // Test that, if a client makes a request to a range that has recently split and // the client indicates that it has pre-split info, the serve replies with // updated info on both sides of the split. The server has a heuristic for @@ -1854,7 +1684,7 @@ func TestRangeInfoAfterSplit(t *testing.T) { ba := roachpb.BatchRequest{ Header: roachpb.Header{ RangeID: tc.rangeID, - ClientRangeInfo: &roachpb.ClientRangeInfo{ + ClientRangeInfo: roachpb.ClientRangeInfo{ DescriptorGeneration: preSplitDesc.Generation, }, }, @@ -3560,20 +3390,20 @@ func TestDiscoverIntentAcrossLeaseTransferAwayAndBack(t *testing.T) { require.NoError(t, <-err4C) } -// getRangeInfo retreives range info by performing a get against the provided -// key and setting the ReturnRangeInfo flag to true. +// getRangeInfo retrieves range info by performing a RangeStatsRequest against +// the provided key. func getRangeInfo( ctx context.Context, db *kv.DB, key roachpb.Key, ) (ri *roachpb.RangeInfo, err error) { err = db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { b := txn.NewBatch() - b.Header.ReturnRangeInfo = true - b.AddRawRequest(roachpb.NewGet(key)) + b.AddRawRequest(&roachpb.RangeStatsRequest{ + RequestHeader: roachpb.RequestHeader{Key: key}, + }) if err = db.Run(ctx, b); err != nil { return err } - resp := b.RawResponse() - ri = &resp.RangeInfos[0] + ri = &b.RawResponse().Responses[0].GetRangeStats().RangeInfo return nil }) return ri, err diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index feb5885b6d04..c03d47a8ab0a 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -16,7 +16,6 @@ import ( "math" "time" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/kv" @@ -181,28 +180,12 @@ func (mq *mergeQueue) requestRangeStats( RequestHeader: roachpb.RequestHeader{Key: key}, }) - if !mq.store.ClusterSettings().Version.IsActive(ctx, clusterversion.RangeStatsRespHasDesc) { - ba.Header.ReturnRangeInfo = true - } - br, pErr := mq.db.NonTransactionalSender().Send(ctx, ba) if pErr != nil { return nil, enginepb.MVCCStats{}, 0, pErr.GoError() } res := br.Responses[0].GetInner().(*roachpb.RangeStatsResponse) - - var desc *roachpb.RangeDescriptor - if res.RangeInfo != nil { - desc = &res.RangeInfo.Desc - } else { - if len(br.RangeInfos) != 1 { - return nil, enginepb.MVCCStats{}, 0, errors.AssertionFailedf( - "mergeQueue.requestRangeStats: response had %d range infos but exactly one was expected", - len(br.RangeInfos)) - } - desc = &br.RangeInfos[0].Desc - } - return desc, res.MVCCStats, res.QueriesPerSecond, nil + return &res.RangeInfo.Desc, res.MVCCStats, res.QueriesPerSecond, nil } func (mq *mergeQueue) process( diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 3d45f57a0f5b..57d2715cb8d0 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -14,7 +14,6 @@ import ( "context" "reflect" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -126,39 +125,17 @@ func (r *Replica) sendWithRangeID( return br, pErr } +// maybeAddRangeInfoToResponse populates br.RangeInfo if the client doesn't +// have up-to-date info about the range's descriptor and lease. func (r *Replica) maybeAddRangeInfoToResponse( ctx context.Context, ba *roachpb.BatchRequest, br *roachpb.BatchResponse, ) { - if ba.ReturnRangeInfo { - desc, lease := r.GetDescAndLease(ctx) - br.RangeInfos = []roachpb.RangeInfo{{Desc: desc, Lease: lease}} - - if !r.ClusterSettings().Version.IsActive(ctx, clusterversion.ClientRangeInfosOnBatchResponse) { - // Also set the RangeInfo on the individual responses, for compatibility - // with 20.1. - for _, r := range br.Responses { - reply := r.GetInner() - header := reply.Header() - header.DeprecatedRangeInfos = br.RangeInfos - reply.SetHeader(header) - } - } - } else if ba.ClientRangeInfo != nil { - returnRangeInfoIfClientStale(ctx, br, r, *ba.ClientRangeInfo) - } -} - -// returnRangeInfoIfClientStale populates br.RangeInfos if the client doesn't -// have up-to-date info about the range's descriptor and lease. -func returnRangeInfoIfClientStale( - ctx context.Context, br *roachpb.BatchResponse, r *Replica, cinfo roachpb.ClientRangeInfo, -) { - desc, lease := r.GetDescAndLease(ctx) // Compare the client's info with the replica's info to detect if the client // has stale knowledge. Note that the client can have more recent knowledge // than the replica in case this is a follower. - needInfo := (cinfo.LeaseSequence < lease.Sequence) || - (cinfo.DescriptorGeneration < desc.Generation) + cinfo := &ba.ClientRangeInfo + desc, lease := r.GetDescAndLease(ctx) + needInfo := (cinfo.LeaseSequence < lease.Sequence) || (cinfo.DescriptorGeneration < desc.Generation) if !needInfo { return } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 76cb22905576..bb7d3adf9aaf 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -12997,28 +12997,24 @@ func TestRangeInfoReturned(t *testing.T) { staleLeaseSeq := lease.Sequence - 1 for _, test := range []struct { - req *roachpb.ClientRangeInfo + req roachpb.ClientRangeInfo exp *roachpb.RangeInfo }{ - { - req: nil, - exp: nil, - }, { // Empty client info. This case shouldn't happen. - req: &roachpb.ClientRangeInfo{}, + req: roachpb.ClientRangeInfo{}, exp: ri, }, { // Correct descriptor, missing lease. - req: &roachpb.ClientRangeInfo{ + req: roachpb.ClientRangeInfo{ DescriptorGeneration: ri.Desc.Generation, }, exp: ri, }, { // Correct descriptor, stale lease. - req: &roachpb.ClientRangeInfo{ + req: roachpb.ClientRangeInfo{ DescriptorGeneration: ri.Desc.Generation, LeaseSequence: staleLeaseSeq, }, @@ -13026,7 +13022,7 @@ func TestRangeInfoReturned(t *testing.T) { }, { // Correct descriptor, correct lease. - req: &roachpb.ClientRangeInfo{ + req: roachpb.ClientRangeInfo{ DescriptorGeneration: ri.Desc.Generation, LeaseSequence: ri.Lease.Sequence, }, @@ -13034,14 +13030,14 @@ func TestRangeInfoReturned(t *testing.T) { }, { // Stale descriptor, no lease. - req: &roachpb.ClientRangeInfo{ + req: roachpb.ClientRangeInfo{ DescriptorGeneration: staleDescGen, }, exp: ri, }, { // Stale descriptor, stale lease. - req: &roachpb.ClientRangeInfo{ + req: roachpb.ClientRangeInfo{ DescriptorGeneration: staleDescGen, LeaseSequence: staleLeaseSeq, }, @@ -13049,7 +13045,7 @@ func TestRangeInfoReturned(t *testing.T) { }, { // Stale desc, good lease. This case shouldn't happen. - req: &roachpb.ClientRangeInfo{ + req: roachpb.ClientRangeInfo{ DescriptorGeneration: staleDescGen, LeaseSequence: staleLeaseSeq, }, diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 08cd781a8d1f..688d2b7f6d2e 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -12,7 +12,6 @@ package roachpb import ( "fmt" - "sort" "github.com/aws/aws-sdk-go/aws" "github.com/aws/aws-sdk-go/aws/credentials" @@ -329,7 +328,6 @@ func (rh *ResponseHeader) combine(otherRH ResponseHeader) error { rh.ResumeReason = otherRH.ResumeReason rh.NumKeys += otherRH.NumKeys rh.NumBytes += otherRH.NumBytes - rh.DeprecatedRangeInfos = append(rh.DeprecatedRangeInfos, otherRH.DeprecatedRangeInfos...) return nil } @@ -533,23 +531,6 @@ func (h *BatchResponse_Header) combine(o BatchResponse_Header) error { } h.Now.Forward(o.Now) h.CollectedSpans = append(h.CollectedSpans, o.CollectedSpans...) - // Deduplicate the RangeInfos and maintain them in sorted order. - // - // TODO(andrei): stop merging RangeInfos once everybody but the DistSender - // stops using them. - for _, ri := range o.RangeInfos { - id := ri.Desc.RangeID - i := sort.Search(len(h.RangeInfos), func(i int) bool { - return h.RangeInfos[i].Desc.RangeID >= id - }) - if i < len(h.RangeInfos) && h.RangeInfos[i].Desc.RangeID == id { - continue - } - // Insert ri in the middle. - h.RangeInfos = append(h.RangeInfos, RangeInfo{}) - copy(h.RangeInfos[i+1:], h.RangeInfos[i:]) - h.RangeInfos[i] = ri - } return nil } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 5dbc03ed243e..d148317cf906 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -74,7 +74,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -102,7 +102,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{1} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{1} } type ChecksumMode int32 @@ -149,7 +149,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{2} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -180,7 +180,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{3} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{3} } type ExternalStorageProvider int32 @@ -221,7 +221,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{4} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{4} } type MVCCFilter int32 @@ -244,7 +244,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{5} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{5} } type ResponseHeader_ResumeReason int32 @@ -270,7 +270,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{1, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -312,7 +312,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{25, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -333,7 +333,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -393,19 +393,13 @@ type ResponseHeader struct { // measures the bytes in the timestamps, keys, and values of the returned // rows. NumBytes int64 `protobuf:"varint,8,opt,name=num_bytes,json=numBytes,proto3" json:"num_bytes,omitempty"` - // Range or list of ranges used to execute the request. Multiple - // ranges may be returned for Scan, ReverseScan or DeleteRange. - // - // This field is deprecated in 20.2 in favor of - // BatchResponse.Header.range_infos. - DeprecatedRangeInfos []RangeInfo `protobuf:"bytes,6,rep,name=deprecated_range_infos,json=deprecatedRangeInfos,proto3" json:"deprecated_range_infos"` } 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_e9d594aefb977c8a, []int{1} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -439,7 +433,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_e9d594aefb977c8a, []int{2} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -482,7 +476,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_e9d594aefb977c8a, []int{3} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -525,7 +519,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_e9d594aefb977c8a, []int{4} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,7 +553,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_e9d594aefb977c8a, []int{5} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -648,7 +642,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_e9d594aefb977c8a, []int{6} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -683,7 +677,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_e9d594aefb977c8a, []int{7} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -729,7 +723,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_e9d594aefb977c8a, []int{8} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -763,7 +757,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_e9d594aefb977c8a, []int{9} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -803,7 +797,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_e9d594aefb977c8a, []int{10} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -840,7 +834,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_e9d594aefb977c8a, []int{11} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -874,7 +868,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_e9d594aefb977c8a, []int{12} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -908,7 +902,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_e9d594aefb977c8a, []int{13} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -960,7 +954,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_e9d594aefb977c8a, []int{14} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -997,7 +991,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_e9d594aefb977c8a, []int{15} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1052,7 +1046,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_e9d594aefb977c8a, []int{16} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1086,7 +1080,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_e9d594aefb977c8a, []int{17} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1128,7 +1122,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_e9d594aefb977c8a, []int{18} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1162,7 +1156,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_e9d594aefb977c8a, []int{19} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1213,7 +1207,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_e9d594aefb977c8a, []int{20} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1265,7 +1259,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_e9d594aefb977c8a, []int{21} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1316,7 +1310,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_e9d594aefb977c8a, []int{22} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1368,7 +1362,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_e9d594aefb977c8a, []int{23} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1421,7 +1415,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_e9d594aefb977c8a, []int{24} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1452,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_e9d594aefb977c8a, []int{25} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1502,7 +1496,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_e9d594aefb977c8a, []int{25, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1550,7 +1544,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_e9d594aefb977c8a, []int{26} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1586,7 +1580,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_e9d594aefb977c8a, []int{27} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1697,7 +1691,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_e9d594aefb977c8a, []int{28} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1743,7 +1737,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_e9d594aefb977c8a, []int{29} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1804,7 +1798,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_e9d594aefb977c8a, []int{30} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1839,7 +1833,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_e9d594aefb977c8a, []int{31} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1878,7 +1872,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_e9d594aefb977c8a, []int{32} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1913,7 +1907,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_e9d594aefb977c8a, []int{33} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1956,7 +1950,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_e9d594aefb977c8a, []int{34} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1991,7 +1985,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_e9d594aefb977c8a, []int{35} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2029,7 +2023,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_e9d594aefb977c8a, []int{36} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2062,7 +2056,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_e9d594aefb977c8a, []int{37} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2097,7 +2091,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_e9d594aefb977c8a, []int{38} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2155,7 +2149,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_e9d594aefb977c8a, []int{39} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2190,7 +2184,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_e9d594aefb977c8a, []int{40} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2228,7 +2222,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_e9d594aefb977c8a, []int{41} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2261,7 +2255,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_e9d594aefb977c8a, []int{42} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2303,7 +2297,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_e9d594aefb977c8a, []int{43} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2340,7 +2334,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_e9d594aefb977c8a, []int{44} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2378,7 +2372,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_e9d594aefb977c8a, []int{45} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2412,7 +2406,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_e9d594aefb977c8a, []int{45, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2446,7 +2440,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_e9d594aefb977c8a, []int{46} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2515,7 +2509,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_e9d594aefb977c8a, []int{47} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2558,7 +2552,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_e9d594aefb977c8a, []int{48} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2605,7 +2599,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_e9d594aefb977c8a, []int{49} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2641,7 +2635,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_e9d594aefb977c8a, []int{50} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2685,7 +2679,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_e9d594aefb977c8a, []int{51} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2729,7 +2723,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_e9d594aefb977c8a, []int{52} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2789,7 +2783,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_e9d594aefb977c8a, []int{53} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2825,7 +2819,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_e9d594aefb977c8a, []int{54} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2872,7 +2866,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_e9d594aefb977c8a, []int{55} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2907,7 +2901,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_e9d594aefb977c8a, []int{56} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2958,7 +2952,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_e9d594aefb977c8a, []int{57} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2993,7 +2987,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_e9d594aefb977c8a, []int{58} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3030,7 +3024,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_e9d594aefb977c8a, []int{59} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3064,7 +3058,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_e9d594aefb977c8a, []int{60} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3109,7 +3103,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_e9d594aefb977c8a, []int{61} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3143,7 +3137,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_e9d594aefb977c8a, []int{62} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3187,7 +3181,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_e9d594aefb977c8a, []int{63} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3236,7 +3230,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_e9d594aefb977c8a, []int{64} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3273,7 +3267,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_e9d594aefb977c8a, []int{65} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3310,7 +3304,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_e9d594aefb977c8a, []int{66} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3345,7 +3339,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_e9d594aefb977c8a, []int{67} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3400,7 +3394,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_e9d594aefb977c8a, []int{68} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3437,7 +3431,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_e9d594aefb977c8a, []int{69} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3477,7 +3471,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_e9d594aefb977c8a, []int{70} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3511,7 +3505,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_e9d594aefb977c8a, []int{70, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3544,7 +3538,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_e9d594aefb977c8a, []int{70, 1} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3586,7 +3580,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_e9d594aefb977c8a, []int{70, 2} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3625,7 +3619,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_e9d594aefb977c8a, []int{70, 3} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3661,7 +3655,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_e9d594aefb977c8a, []int{70, 4} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3700,7 +3694,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_e9d594aefb977c8a, []int{70, 5} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3742,7 +3736,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_e9d594aefb977c8a, []int{70, 6} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3782,7 +3776,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_e9d594aefb977c8a, []int{71} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3816,7 +3810,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_e9d594aefb977c8a, []int{72} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3852,7 +3846,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_e9d594aefb977c8a, []int{73} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3922,7 +3916,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_e9d594aefb977c8a, []int{74} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3974,7 +3968,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_e9d594aefb977c8a, []int{75} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4010,7 +4004,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_e9d594aefb977c8a, []int{76} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4050,7 +4044,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_e9d594aefb977c8a, []int{76, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4101,7 +4095,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_e9d594aefb977c8a, []int{77} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4136,7 +4130,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_e9d594aefb977c8a, []int{77, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4172,7 +4166,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_e9d594aefb977c8a, []int{77, 1} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4207,7 +4201,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_e9d594aefb977c8a, []int{78} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4245,7 +4239,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_e9d594aefb977c8a, []int{79} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4282,7 +4276,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_e9d594aefb977c8a, []int{80} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4315,7 +4309,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_e9d594aefb977c8a, []int{80, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4360,7 +4354,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_e9d594aefb977c8a, []int{81} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4398,7 +4392,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_e9d594aefb977c8a, []int{82} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4451,7 +4445,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_e9d594aefb977c8a, []int{83} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4485,7 +4479,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_e9d594aefb977c8a, []int{84} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4529,7 +4523,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_e9d594aefb977c8a, []int{85} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4563,7 +4557,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_e9d594aefb977c8a, []int{86} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4602,7 +4596,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_e9d594aefb977c8a, []int{87} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4636,7 +4630,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_e9d594aefb977c8a, []int{88} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4685,7 +4679,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_e9d594aefb977c8a, []int{89} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4734,7 +4728,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_e9d594aefb977c8a, []int{90} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4769,7 +4763,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_e9d594aefb977c8a, []int{91} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4802,16 +4796,15 @@ type RangeStatsResponse struct { MVCCStats enginepb.MVCCStats `protobuf:"bytes,2,opt,name=mvcc_stats,json=mvccStats,proto3" json:"mvcc_stats"` // QueriesPerSecond is the rate of request/s or QPS for the range. QueriesPerSecond float64 `protobuf:"fixed64,3,opt,name=queries_per_second,json=queriesPerSecond,proto3" json:"queries_per_second,omitempty"` - // range_info contains descriptor and lease information. Added in 20.2. - // TODO(andrei): Make non-nullable in 21.1. - RangeInfo *RangeInfo `protobuf:"bytes,4,opt,name=range_info,json=rangeInfo,proto3" json:"range_info,omitempty"` + // range_info contains descriptor and lease information. + RangeInfo RangeInfo `protobuf:"bytes,4,opt,name=range_info,json=rangeInfo,proto3" json:"range_info"` } 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_e9d594aefb977c8a, []int{92} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4854,7 +4847,7 @@ func (m *MigrateRequest) Reset() { *m = MigrateRequest{} } func (m *MigrateRequest) String() string { return proto.CompactTextString(m) } func (*MigrateRequest) ProtoMessage() {} func (*MigrateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{93} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{93} } func (m *MigrateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4888,7 +4881,7 @@ func (m *MigrateResponse) Reset() { *m = MigrateResponse{} } func (m *MigrateResponse) String() string { return proto.CompactTextString(m) } func (*MigrateResponse) ProtoMessage() {} func (*MigrateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{94} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{94} } func (m *MigrateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4972,7 +4965,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_e9d594aefb977c8a, []int{95} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6453,7 +6446,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_e9d594aefb977c8a, []int{96} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7975,20 +7968,11 @@ type Header struct { // EndTxnRequest. Currently set conservatively: a request might be // composed of distinct spans yet have this field set to false. DistinctSpans bool `protobuf:"varint,9,opt,name=distinct_spans,json=distinctSpans,proto3" json:"distinct_spans,omitempty"` - // If set, return_range_info causes RangeInfo details to be returned with - // each ResponseHeader. - // TODO(andrei): Remove straggler users of this field, then delete the field - // in 21.1 now that we have client_range_info. - ReturnRangeInfo bool `protobuf:"varint,10,opt,name=return_range_info,json=returnRangeInfo,proto3" json:"return_range_info,omitempty"` // client_range_info represents the kvclient's knowledge about the state of // the range (i.e. of the range descriptor and lease). The kvserver checks // whether the client's info is up to date and, if it isn't, it will return a - // RangeInfo with up-to-date information. This field supersedes return_range_info. - // - // The field is nil for RPCs coming from 20.1 clients. For 20.2+ clients, at - // least the descriptor info is filled in. - // TODO(andrei): Make the field non-nullable in 21.1. - ClientRangeInfo *ClientRangeInfo `protobuf:"bytes,17,opt,name=client_range_info,json=clientRangeInfo,proto3" json:"client_range_info,omitempty"` + // RangeInfo with up-to-date information. + ClientRangeInfo ClientRangeInfo `protobuf:"bytes,17,opt,name=client_range_info,json=clientRangeInfo,proto3" json:"client_range_info"` // gateway_node_id is the ID of the gateway node where the request originated. GatewayNodeID NodeID `protobuf:"varint,11,opt,name=gateway_node_id,json=gatewayNodeId,proto3,casttype=NodeID" json:"gateway_node_id,omitempty"` // If set, the request will return to the client before proposing the @@ -8018,7 +8002,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_e9d594aefb977c8a, []int{97} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8056,7 +8040,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_e9d594aefb977c8a, []int{98} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{98} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8092,7 +8076,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{99} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{99} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8129,7 +8113,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{100} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{100} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8186,21 +8170,18 @@ type BatchResponse_Header struct { // collected_spans stores trace spans recorded during the execution of this // request. CollectedSpans []tracingpb.RecordedSpan `protobuf:"bytes,6,rep,name=collected_spans,json=collectedSpans,proto3" json:"collected_spans"` - // Range or list of ranges used to execute the request. The server only - // populates this if return_range_info is set on the request, or if the - // server detects the client's client_range_info to be stale. + // Range used to execute the request. The server only populates this if the + // server detects the client's client_range_info to be stale. Otherwise, it + // is left empty. Not set when Error is set. // - // For one batch, the kvserver returns at most one element. The DistSender - // will combine multiple responses though when it reassembles a split batch; - // in this case the RangeInfos are maintained in sorted order by range id. + // The server may also include additional RangeInfo objects if it suspects + // that the client may be interested in them. This is currently the case + // immediately after a Range split, where a stale client_range_info from + // before the split is taken to be an indication that the client may be + // interested in information about both the LHS and RHS post-split ranges. // - // Not set when Error is set. - // - // TODO(andrei): In 21.1 there should be no users of this field left above - // the DistSender. At that point, the DistSender will no longer need to - // combine these - in fact it should clear the field. And then the field - // doesn't need to be `repeated` any more - the proto encoding - // allows us to change it to non-repeated. + // The field is cleared by the DistSender because it refers routing + // information not exposed by the KV API. RangeInfos []RangeInfo `protobuf:"bytes,7,rep,name=range_infos,json=rangeInfos,proto3" json:"range_infos"` } @@ -8208,7 +8189,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_e9d594aefb977c8a, []int{100, 0} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{100, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8246,7 +8227,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_e9d594aefb977c8a, []int{101} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{101} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8284,7 +8265,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_e9d594aefb977c8a, []int{102} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{102} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8323,7 +8304,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_e9d594aefb977c8a, []int{103} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{103} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8364,7 +8345,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_e9d594aefb977c8a, []int{104} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{104} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8405,7 +8386,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_e9d594aefb977c8a, []int{105} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{105} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8442,7 +8423,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_e9d594aefb977c8a, []int{106} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{106} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8479,7 +8460,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_e9d594aefb977c8a, []int{107} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{107} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8523,7 +8504,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{108} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{108} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8555,7 +8536,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{109} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{109} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8596,7 +8577,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_e9d594aefb977c8a, []int{110} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{110} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8636,7 +8617,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_e9d594aefb977c8a, []int{111} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{111} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8672,7 +8653,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_e9d594aefb977c8a, []int{112} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{112} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8711,7 +8692,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_e9d594aefb977c8a, []int{113} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{113} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8751,7 +8732,7 @@ type ContentionEvent struct { func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_e9d594aefb977c8a, []int{114} + return fileDescriptor_api_cdfee8e6fce2cd19, []int{114} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -9289,18 +9270,6 @@ func (m *ResponseHeader) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintApi(dAtA, i, uint64(m.NumKeys)) } - if len(m.DeprecatedRangeInfos) > 0 { - for _, msg := range m.DeprecatedRangeInfos { - dAtA[i] = 0x32 - i++ - i = encodeVarintApi(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } - } if m.ResumeReason != 0 { dAtA[i] = 0x38 i++ @@ -13621,16 +13590,14 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.QueriesPerSecond)))) i += 8 } - if m.RangeInfo != nil { - dAtA[i] = 0x22 - i++ - i = encodeVarintApi(dAtA, i, uint64(m.RangeInfo.Size())) - n161, err := m.RangeInfo.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n161 + dAtA[i] = 0x22 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.RangeInfo.Size())) + n161, err := m.RangeInfo.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } + i += n161 return i, nil } @@ -15188,16 +15155,6 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { } i++ } - if m.ReturnRangeInfo { - dAtA[i] = 0x50 - i++ - if m.ReturnRangeInfo { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i++ - } if m.GatewayNodeID != 0 { dAtA[i] = 0x58 i++ @@ -15230,18 +15187,16 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { } i++ } - if m.ClientRangeInfo != nil { - dAtA[i] = 0x8a - i++ - dAtA[i] = 0x1 - i++ - i = encodeVarintApi(dAtA, i, uint64(m.ClientRangeInfo.Size())) - n259, err := m.ClientRangeInfo.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n259 + dAtA[i] = 0x8a + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.ClientRangeInfo.Size())) + n259, err := m.ClientRangeInfo.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } + i += n259 if m.WaitPolicy != 0 { dAtA[i] = 0x90 i++ @@ -16002,12 +15957,6 @@ func (m *ResponseHeader) Size() (n int) { if m.NumKeys != 0 { n += 1 + sovApi(uint64(m.NumKeys)) } - if len(m.DeprecatedRangeInfos) > 0 { - for _, e := range m.DeprecatedRangeInfos { - l = e.Size() - n += 1 + l + sovApi(uint64(l)) - } - } if m.ResumeReason != 0 { n += 1 + sovApi(uint64(m.ResumeReason)) } @@ -17852,10 +17801,8 @@ func (m *RangeStatsResponse) Size() (n int) { if m.QueriesPerSecond != 0 { n += 9 } - if m.RangeInfo != nil { - l = m.RangeInfo.Size() - n += 1 + l + sovApi(uint64(l)) - } + l = m.RangeInfo.Size() + n += 1 + l + sovApi(uint64(l)) return n } @@ -19004,9 +18951,6 @@ func (m *Header) Size() (n int) { if m.DistinctSpans { n += 2 } - if m.ReturnRangeInfo { - n += 2 - } if m.GatewayNodeID != 0 { n += 1 + sovApi(uint64(m.GatewayNodeID)) } @@ -19019,10 +18963,8 @@ func (m *Header) Size() (n int) { if m.CanForwardReadTimestamp { n += 3 } - if m.ClientRangeInfo != nil { - l = m.ClientRangeInfo.Size() - n += 2 + l + sovApi(uint64(l)) - } + l = m.ClientRangeInfo.Size() + n += 2 + l + sovApi(uint64(l)) if m.WaitPolicy != 0 { n += 2 + sovApi(uint64(m.WaitPolicy)) } @@ -19632,37 +19574,6 @@ func (m *ResponseHeader) Unmarshal(dAtA []byte) error { break } } - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedRangeInfos", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.DeprecatedRangeInfos = append(m.DeprecatedRangeInfos, RangeInfo{}) - if err := m.DeprecatedRangeInfos[len(m.DeprecatedRangeInfos)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 7: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field ResumeReason", wireType) @@ -33327,9 +33238,6 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RangeInfo == nil { - m.RangeInfo = &RangeInfo{} - } if err := m.RangeInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -36703,26 +36611,6 @@ func (m *Header) Unmarshal(dAtA []byte) error { } } m.DistinctSpans = bool(v != 0) - case 10: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ReturnRangeInfo", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.ReturnRangeInfo = bool(v != 0) case 11: if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field GatewayNodeID", wireType) @@ -36827,9 +36715,6 @@ func (m *Header) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ClientRangeInfo == nil { - m.ClientRangeInfo = &ClientRangeInfo{} - } if err := m.ClientRangeInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -39166,215 +39051,213 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_e9d594aefb977c8a) } - -var fileDescriptor_api_e9d594aefb977c8a = []byte{ - // 8267 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0xbd, 0x6b, 0x6c, 0x23, 0xc9, - 0x76, 0x1f, 0xae, 0x26, 0x29, 0x89, 0x3c, 0x14, 0x1f, 0x2a, 0xcd, 0x83, 0xc3, 0xdd, 0x1d, 0xcd, - 0xf4, 0xbc, 0xe7, 0xee, 0x4a, 0x3b, 0x33, 0x77, 0xff, 0xbb, 0xde, 0x59, 0xef, 0xb5, 0x44, 0x71, - 0x86, 0x94, 0x46, 0x1a, 0x4d, 0x93, 0x9a, 0xf9, 0xef, 0xfa, 0x3a, 0x7d, 0x5b, 0xdd, 0x25, 0xaa, - 0xaf, 0xc8, 0x6e, 0x4e, 0x77, 0x53, 0x8f, 0x05, 0x02, 0xc4, 0xb1, 0xe1, 0x5c, 0x20, 0x80, 0x71, - 0x3f, 0x38, 0xc8, 0x35, 0x9c, 0xc4, 0xd7, 0x71, 0x10, 0x7f, 0x08, 0x90, 0x04, 0x48, 0x90, 0x17, - 0x12, 0xfb, 0x8b, 0x81, 0x5c, 0x04, 0x4e, 0x72, 0xfd, 0x29, 0x46, 0x80, 0x28, 0xb6, 0x6e, 0xbe, - 0x04, 0x09, 0x8c, 0x20, 0x08, 0x60, 0x60, 0x3f, 0x04, 0x41, 0x3d, 0xfa, 0x45, 0x36, 0x1f, 0x9a, - 0xed, 0x9b, 0x2c, 0xe0, 0x2f, 0x12, 0xfb, 0x54, 0x9d, 0xd3, 0x55, 0xa7, 0xaa, 0x4e, 0x9d, 0x5f, - 0xf5, 0xa9, 0x2a, 0x98, 0xb7, 0x4c, 0x45, 0xdd, 0xef, 0xee, 0x2e, 0x2b, 0x5d, 0x7d, 0xa9, 0x6b, - 0x99, 0x8e, 0x89, 0xe6, 0x55, 0x53, 0x3d, 0xa0, 0xe4, 0x25, 0x9e, 0x58, 0xbe, 0x7f, 0x70, 0xb8, - 0x7c, 0x70, 0x68, 0x63, 0xeb, 0x10, 0x5b, 0xcb, 0xaa, 0x69, 0xa8, 0x3d, 0xcb, 0xc2, 0x86, 0x7a, - 0xb2, 0xdc, 0x36, 0xd5, 0x03, 0xfa, 0x47, 0x37, 0x5a, 0x8c, 0xbd, 0x8c, 0x5c, 0x89, 0x9a, 0xe2, - 0x28, 0x9c, 0x76, 0xc1, 0xa5, 0x61, 0xcb, 0x32, 0x2d, 0x9b, 0x53, 0x2f, 0xb9, 0xd4, 0x0e, 0x76, - 0x94, 0x40, 0xee, 0xb7, 0x6c, 0xc7, 0xb4, 0x94, 0x16, 0x5e, 0xc6, 0x46, 0x4b, 0x37, 0x30, 0xc9, - 0x70, 0xa8, 0xaa, 0x3c, 0xf1, 0xed, 0xc8, 0xc4, 0x47, 0x3c, 0xb5, 0xd4, 0x73, 0xf4, 0xf6, 0xf2, - 0x7e, 0x5b, 0x5d, 0x76, 0xf4, 0x0e, 0xb6, 0x1d, 0xa5, 0xd3, 0xe5, 0x29, 0xf7, 0x69, 0x8a, 0x63, - 0x29, 0xaa, 0x6e, 0xb4, 0xdc, 0xff, 0xdd, 0xdd, 0x65, 0x0b, 0xab, 0xa6, 0xa5, 0x61, 0x4d, 0xb6, - 0xbb, 0x8a, 0xe1, 0x16, 0xb7, 0x65, 0xb6, 0x4c, 0xfa, 0x73, 0x99, 0xfc, 0xe2, 0xd4, 0xab, 0x2d, - 0xd3, 0x6c, 0xb5, 0xf1, 0x32, 0x7d, 0xda, 0xed, 0xed, 0x2d, 0x6b, 0x3d, 0x4b, 0x71, 0x74, 0x93, - 0x73, 0x89, 0xff, 0x44, 0x80, 0x9c, 0x84, 0x5f, 0xf7, 0xb0, 0xed, 0xd4, 0xb0, 0xa2, 0x61, 0x0b, - 0x5d, 0x81, 0xe4, 0x01, 0x3e, 0x29, 0x25, 0xaf, 0x09, 0x77, 0xe7, 0x56, 0x67, 0xbf, 0x3c, 0x5d, - 0x4c, 0x6e, 0xe0, 0x13, 0x89, 0xd0, 0xd0, 0x35, 0x98, 0xc5, 0x86, 0x26, 0x93, 0xe4, 0x54, 0x38, - 0x79, 0x06, 0x1b, 0xda, 0x06, 0x3e, 0x41, 0xdf, 0x86, 0xb4, 0x4d, 0xa4, 0x19, 0x2a, 0x2e, 0x4d, - 0x5f, 0x13, 0xee, 0x4e, 0xaf, 0xfe, 0xdc, 0x97, 0xa7, 0x8b, 0x9f, 0xb4, 0x74, 0x67, 0xbf, 0xb7, - 0xbb, 0xa4, 0x9a, 0x9d, 0x65, 0xaf, 0x9d, 0xb4, 0x5d, 0xff, 0xf7, 0x72, 0xf7, 0xa0, 0xb5, 0xdc, - 0xaf, 0xa3, 0xa5, 0xe6, 0xb1, 0xd1, 0xc0, 0xaf, 0x25, 0x4f, 0xe2, 0x7a, 0x2a, 0x2d, 0x14, 0x13, - 0xeb, 0xa9, 0x74, 0xa2, 0x98, 0x14, 0x7f, 0x3b, 0x09, 0x79, 0x09, 0xdb, 0x5d, 0xd3, 0xb0, 0x31, - 0x2f, 0xf9, 0xfb, 0x90, 0x74, 0x8e, 0x0d, 0x5a, 0xf2, 0xec, 0xc3, 0xab, 0x4b, 0x03, 0x3d, 0x62, - 0xa9, 0x69, 0x29, 0x86, 0xad, 0xa8, 0xa4, 0xfa, 0x12, 0xc9, 0x8a, 0x3e, 0x82, 0xac, 0x85, 0xed, - 0x5e, 0x07, 0x53, 0x45, 0xd2, 0x4a, 0x65, 0x1f, 0x5e, 0x8e, 0xe0, 0x6c, 0x74, 0x15, 0x43, 0x02, - 0x96, 0x97, 0xfc, 0x46, 0x57, 0x20, 0x6d, 0xf4, 0x3a, 0x44, 0x15, 0x36, 0xad, 0x68, 0x52, 0x9a, - 0x35, 0x7a, 0x9d, 0x0d, 0x7c, 0x62, 0xa3, 0xff, 0x1f, 0x2e, 0x69, 0xb8, 0x6b, 0x61, 0x55, 0x71, - 0xb0, 0x26, 0x5b, 0x8a, 0xd1, 0xc2, 0xb2, 0x6e, 0xec, 0x99, 0x76, 0x69, 0xe6, 0x5a, 0xf2, 0x6e, - 0xf6, 0xe1, 0xdb, 0x11, 0xf2, 0x25, 0x92, 0xab, 0x6e, 0xec, 0x99, 0xab, 0xa9, 0x1f, 0x9d, 0x2e, - 0x4e, 0x49, 0x17, 0x7c, 0x09, 0x5e, 0x92, 0x8d, 0x1a, 0x90, 0xe3, 0xc5, 0xb5, 0xb0, 0x62, 0x9b, - 0x46, 0x69, 0xf6, 0x9a, 0x70, 0x37, 0xff, 0x70, 0x29, 0x4a, 0x60, 0x48, 0x35, 0xe4, 0xb1, 0xd7, - 0xc1, 0x12, 0xe5, 0x92, 0xe6, 0xac, 0xc0, 0x13, 0x7a, 0x0b, 0x32, 0xa4, 0x26, 0xbb, 0x27, 0x0e, - 0xb6, 0x4b, 0x69, 0x5a, 0x15, 0x52, 0xb5, 0x55, 0xf2, 0x2c, 0x7e, 0x0a, 0x73, 0x41, 0x56, 0x84, - 0x20, 0x2f, 0x55, 0x1b, 0x3b, 0x9b, 0x55, 0x79, 0x67, 0x6b, 0x63, 0xeb, 0xf9, 0xab, 0xad, 0xe2, - 0x14, 0xba, 0x00, 0x45, 0x4e, 0xdb, 0xa8, 0x7e, 0x26, 0x3f, 0xab, 0x6f, 0xd6, 0x9b, 0x45, 0xa1, - 0x9c, 0xfa, 0xde, 0x6f, 0x5f, 0x9d, 0x12, 0xb7, 0x01, 0x9e, 0x62, 0x87, 0x77, 0x30, 0xb4, 0x0a, - 0x33, 0xfb, 0xb4, 0x3c, 0x25, 0x81, 0x6a, 0xfa, 0x5a, 0x64, 0xc1, 0x03, 0x9d, 0x71, 0x35, 0x4d, - 0xb4, 0xf1, 0xe3, 0xd3, 0x45, 0x41, 0xe2, 0x9c, 0xe2, 0xef, 0x09, 0x90, 0xa5, 0x22, 0x59, 0xfd, - 0x50, 0xa5, 0x4f, 0xe6, 0xf5, 0xb1, 0xca, 0x18, 0x14, 0x8a, 0x96, 0x60, 0xfa, 0x50, 0x69, 0xf7, - 0x70, 0x29, 0x41, 0x65, 0x94, 0x22, 0x64, 0xbc, 0x24, 0xe9, 0x12, 0xcb, 0x86, 0x1e, 0xc3, 0x9c, - 0x6e, 0x38, 0xd8, 0x70, 0x64, 0xc6, 0x96, 0x1c, 0xc3, 0x96, 0x65, 0xb9, 0xe9, 0x83, 0xf8, 0x8f, - 0x05, 0x80, 0xed, 0x5e, 0x9c, 0x4a, 0x41, 0xdf, 0x9c, 0xb0, 0xfc, 0xbc, 0x77, 0xf1, 0x5a, 0x5c, - 0x82, 0x19, 0xdd, 0x68, 0xeb, 0x06, 0x2b, 0x7f, 0x5a, 0xe2, 0x4f, 0xe8, 0x02, 0x4c, 0xef, 0xb6, - 0x75, 0x43, 0xa3, 0xe3, 0x21, 0x2d, 0xb1, 0x07, 0x51, 0x82, 0x2c, 0x2d, 0x75, 0x8c, 0x7a, 0x17, - 0x4f, 0x13, 0x70, 0xb1, 0x62, 0x1a, 0x9a, 0x4e, 0x86, 0xa4, 0xd2, 0xfe, 0x5a, 0x68, 0x65, 0x1d, - 0x02, 0x83, 0x4f, 0xc6, 0xc7, 0xdd, 0x09, 0xdb, 0x18, 0xf9, 0x5c, 0xd5, 0xe3, 0x2e, 0xa5, 0x45, - 0x6b, 0x12, 0x7d, 0x13, 0x2e, 0x2b, 0xed, 0xb6, 0x79, 0x24, 0xeb, 0x7b, 0xb2, 0x66, 0x62, 0x5b, - 0x36, 0x4c, 0x47, 0xc6, 0xc7, 0xba, 0xed, 0x50, 0x53, 0x92, 0x96, 0x16, 0x68, 0x72, 0x7d, 0x6f, - 0xcd, 0xc4, 0xf6, 0x96, 0xe9, 0x54, 0x49, 0x12, 0x19, 0xa7, 0xa4, 0x30, 0x6c, 0x9c, 0xce, 0x10, - 0xf3, 0x2b, 0xa5, 0xf1, 0x71, 0x97, 0x8e, 0xd3, 0x40, 0x53, 0xce, 0x06, 0x9b, 0x52, 0xfc, 0x05, - 0xb8, 0xd4, 0xaf, 0xdf, 0x38, 0xdb, 0xef, 0x0f, 0x04, 0xc8, 0xd7, 0x0d, 0xdd, 0xf9, 0x5a, 0x34, - 0x9c, 0xa7, 0xec, 0x64, 0x50, 0xd9, 0xf7, 0xa1, 0xb8, 0xa7, 0xe8, 0xed, 0xe7, 0x46, 0xd3, 0xec, - 0xec, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0xb7, 0xc6, 0x00, 0x5d, 0x7c, 0x09, 0x05, 0xaf, 0x36, 0x71, - 0xaa, 0xc9, 0x81, 0x62, 0xdd, 0x50, 0x2d, 0xdc, 0xc1, 0x46, 0xac, 0x7a, 0x7a, 0x1b, 0x32, 0xba, - 0x2b, 0x97, 0xea, 0x2a, 0x29, 0xf9, 0x04, 0xb1, 0x07, 0xf3, 0x81, 0xb7, 0xc6, 0x69, 0x2e, 0xc9, - 0x94, 0x81, 0x8f, 0x64, 0xbf, 0x8d, 0xc8, 0x94, 0x81, 0x8f, 0x98, 0x79, 0x6b, 0x40, 0x6e, 0x0d, - 0xb7, 0xb1, 0x83, 0xe3, 0xb4, 0xfa, 0x3b, 0x90, 0x77, 0x85, 0xc6, 0xd9, 0x30, 0x7f, 0x5d, 0x00, - 0xc4, 0xe5, 0x92, 0x59, 0x36, 0xce, 0xb6, 0x59, 0x24, 0xae, 0x85, 0xd3, 0xb3, 0x0c, 0xe6, 0x23, - 0xb0, 0x3e, 0x09, 0x8c, 0x44, 0xdd, 0x04, 0x7f, 0xc8, 0xa6, 0x82, 0x43, 0x96, 0xbb, 0x37, 0x47, - 0xb0, 0x10, 0x2a, 0x58, 0xbc, 0xcd, 0x97, 0xa2, 0x65, 0x4a, 0x5c, 0x4b, 0x06, 0x7d, 0x38, 0x4a, - 0x14, 0x7f, 0x20, 0xc0, 0x7c, 0xa5, 0x8d, 0x15, 0x2b, 0x76, 0x8d, 0x7c, 0x0b, 0xd2, 0x1a, 0x56, - 0x34, 0x5a, 0x65, 0x36, 0xb0, 0xdf, 0x09, 0x48, 0x21, 0x9e, 0xee, 0xd2, 0x7e, 0x5b, 0x5d, 0x6a, - 0xba, 0x3e, 0x30, 0x1f, 0xdd, 0x1e, 0x93, 0xf8, 0x19, 0xa0, 0x60, 0xc9, 0xe2, 0xec, 0x08, 0xff, - 0x4b, 0x00, 0x24, 0xe1, 0x43, 0x6c, 0x39, 0xb1, 0x57, 0x7b, 0x0d, 0xb2, 0x8e, 0x62, 0xb5, 0xb0, - 0x23, 0x13, 0xef, 0xfe, 0x3c, 0x35, 0x07, 0xc6, 0x47, 0xc8, 0xa8, 0x09, 0x77, 0xb0, 0xa1, 0xec, - 0xb6, 0x31, 0x95, 0x22, 0xef, 0x9a, 0x3d, 0x43, 0x93, 0x75, 0x07, 0x5b, 0x8a, 0x63, 0x5a, 0xb2, - 0xd9, 0x75, 0xf4, 0x8e, 0xfe, 0x05, 0x75, 0xec, 0x79, 0x57, 0xbb, 0xc1, 0xb2, 0x13, 0xe6, 0x55, - 0x92, 0xb9, 0xce, 0xf3, 0x3e, 0x0f, 0x64, 0x15, 0x3f, 0x87, 0x85, 0x50, 0xad, 0xe3, 0x54, 0xe9, - 0xff, 0x10, 0x20, 0xdb, 0x50, 0x15, 0x23, 0x4e, 0x5d, 0x7e, 0x0a, 0x59, 0x5b, 0x55, 0x0c, 0x79, - 0xcf, 0xb4, 0x3a, 0x8a, 0x43, 0x07, 0x4e, 0x3e, 0xa4, 0x4b, 0xcf, 0x5f, 0x57, 0x15, 0xe3, 0x09, - 0xcd, 0x24, 0x81, 0xed, 0xfd, 0x46, 0x2f, 0x20, 0x7b, 0x80, 0x4f, 0x64, 0x8e, 0xfd, 0xe8, 0x6c, - 0x9b, 0x7f, 0xf8, 0x7e, 0x80, 0xff, 0xe0, 0x70, 0xc9, 0x85, 0x8c, 0x4b, 0x01, 0xc8, 0xb8, 0x44, - 0x38, 0x96, 0x1a, 0x8e, 0x85, 0x8d, 0x96, 0xb3, 0x2f, 0xc1, 0x01, 0x3e, 0x79, 0xc6, 0x64, 0xb0, - 0xe1, 0xba, 0x9e, 0x4a, 0x27, 0x8b, 0x29, 0xf1, 0xcf, 0x04, 0x98, 0x63, 0x55, 0x8e, 0x73, 0xb8, - 0x7e, 0x00, 0x29, 0xcb, 0x3c, 0x62, 0xc3, 0x35, 0xfb, 0xf0, 0xad, 0x08, 0x11, 0x1b, 0xf8, 0x24, - 0x38, 0x1f, 0xd2, 0xec, 0x68, 0x15, 0xb8, 0xd7, 0x29, 0x53, 0xee, 0xe4, 0xa4, 0xdc, 0xc0, 0xb8, - 0x24, 0x22, 0xe3, 0x0e, 0x14, 0x76, 0x15, 0x47, 0xdd, 0x97, 0x2d, 0x5e, 0x48, 0x32, 0x77, 0x26, - 0xef, 0xce, 0x49, 0x79, 0x4a, 0x76, 0x8b, 0x6e, 0x93, 0x9a, 0xb3, 0xf1, 0x63, 0xe3, 0x3f, 0x67, - 0x6d, 0xfe, 0xbf, 0x05, 0x3e, 0x86, 0xdc, 0x9a, 0xff, 0x79, 0x6b, 0xfa, 0x5f, 0x4f, 0xc0, 0xe5, - 0xca, 0x3e, 0x56, 0x0f, 0x2a, 0xa6, 0x61, 0xeb, 0xb6, 0x43, 0x74, 0x17, 0x67, 0xfb, 0xbf, 0x05, - 0x99, 0x23, 0xdd, 0xd9, 0x97, 0x35, 0x7d, 0x6f, 0x8f, 0x5a, 0xcf, 0xb4, 0x94, 0x26, 0x84, 0x35, - 0x7d, 0x6f, 0x0f, 0x3d, 0x82, 0x54, 0xc7, 0xd4, 0x98, 0x73, 0x9e, 0x7f, 0xb8, 0x18, 0x21, 0x9e, - 0x16, 0xcd, 0xee, 0x75, 0x36, 0x4d, 0x0d, 0x4b, 0x34, 0x33, 0xba, 0x0a, 0xa0, 0x12, 0x6a, 0xd7, - 0xd4, 0x0d, 0x87, 0xcf, 0xbe, 0x01, 0x0a, 0xaa, 0x41, 0xc6, 0xc1, 0x56, 0x47, 0x37, 0x14, 0x07, - 0x97, 0xa6, 0xa9, 0xf2, 0x6e, 0x46, 0x16, 0xbc, 0xdb, 0xd6, 0x55, 0x65, 0x0d, 0xdb, 0xaa, 0xa5, - 0x77, 0x1d, 0xd3, 0xe2, 0x5a, 0xf4, 0x99, 0xc5, 0x5f, 0x4d, 0x41, 0x69, 0x50, 0x37, 0x71, 0xf6, - 0x90, 0x6d, 0x98, 0x21, 0x68, 0xbe, 0xed, 0xf0, 0x3e, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x4a, 0x40, - 0x57, 0x05, 0xda, 0x0e, 0x2f, 0x36, 0x97, 0x53, 0xfe, 0x97, 0x02, 0xcc, 0xb0, 0x04, 0xf4, 0x00, - 0xd2, 0x7c, 0xf9, 0x42, 0xa3, 0x65, 0x4c, 0xae, 0x5e, 0x3a, 0x3b, 0x5d, 0x9c, 0x65, 0x2b, 0x12, - 0x6b, 0x5f, 0xfa, 0x3f, 0xa5, 0x59, 0x9a, 0xaf, 0xae, 0x91, 0xd6, 0xb2, 0x1d, 0xc5, 0x72, 0xe8, - 0x22, 0x51, 0x82, 0xa1, 0x14, 0x4a, 0xd8, 0xc0, 0x27, 0x68, 0x1d, 0x66, 0x6c, 0x47, 0x71, 0x7a, - 0x36, 0x6f, 0xaf, 0x73, 0x15, 0xb6, 0x41, 0x39, 0x25, 0x2e, 0x81, 0xb8, 0x4f, 0x1a, 0x76, 0x14, - 0xbd, 0x4d, 0x1b, 0x30, 0x23, 0xf1, 0x27, 0xf1, 0x37, 0x04, 0x98, 0x61, 0x59, 0xd1, 0x65, 0x58, - 0x90, 0x56, 0xb6, 0x9e, 0x56, 0xe5, 0xfa, 0xd6, 0x5a, 0xb5, 0x59, 0x95, 0x36, 0xeb, 0x5b, 0x2b, - 0xcd, 0x6a, 0x71, 0x0a, 0x5d, 0x02, 0xe4, 0x26, 0x54, 0x9e, 0x6f, 0x35, 0xea, 0x8d, 0x66, 0x75, - 0xab, 0x59, 0x14, 0xe8, 0x4a, 0x06, 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x13, 0xae, 0xf5, 0x53, 0xe5, - 0x46, 0x73, 0xa5, 0xd9, 0x90, 0xab, 0x8d, 0x66, 0x7d, 0x73, 0xa5, 0x59, 0x5d, 0x2b, 0x26, 0x47, - 0xe4, 0x22, 0x2f, 0x91, 0xa4, 0x6a, 0xa5, 0x59, 0x4c, 0x89, 0x0e, 0x5c, 0x94, 0xb0, 0x6a, 0x76, - 0xba, 0x3d, 0x07, 0x93, 0x52, 0xda, 0x71, 0x8e, 0x94, 0xcb, 0x30, 0xab, 0x59, 0x27, 0xb2, 0xd5, - 0x33, 0xf8, 0x38, 0x99, 0xd1, 0xac, 0x13, 0xa9, 0x67, 0x88, 0xff, 0x50, 0x80, 0x4b, 0xfd, 0xaf, - 0x8d, 0xb3, 0x13, 0xbe, 0x80, 0xac, 0xa2, 0x69, 0x58, 0x93, 0x35, 0xdc, 0x76, 0x14, 0xee, 0xe2, - 0xdc, 0x0f, 0x48, 0xe2, 0x4b, 0x7b, 0x4b, 0xde, 0xd2, 0xde, 0xe6, 0xcb, 0x4a, 0x85, 0x16, 0x64, - 0x8d, 0x70, 0xb8, 0xe6, 0x87, 0x0a, 0xa1, 0x14, 0xf1, 0xbf, 0xa7, 0x20, 0x57, 0x35, 0xb4, 0xe6, - 0x71, 0xac, 0x73, 0xc9, 0x25, 0x98, 0x51, 0xcd, 0x4e, 0x47, 0x77, 0x5c, 0x05, 0xb1, 0x27, 0xf4, - 0x33, 0x01, 0xd7, 0x34, 0x39, 0x81, 0x83, 0xe6, 0x3b, 0xa5, 0xe8, 0x3b, 0x70, 0x99, 0x58, 0x4d, - 0xcb, 0x50, 0xda, 0x32, 0x93, 0x26, 0x3b, 0x96, 0xde, 0x6a, 0x61, 0x8b, 0x2f, 0x27, 0xde, 0x8d, - 0x28, 0x67, 0x9d, 0x73, 0x54, 0x28, 0x43, 0x93, 0xe5, 0x97, 0x2e, 0xea, 0x51, 0x64, 0xf4, 0x09, - 0x00, 0x99, 0x8a, 0xe8, 0x12, 0xa5, 0xcd, 0xed, 0xd1, 0xb0, 0x35, 0x4a, 0xd7, 0x04, 0x11, 0x06, - 0xf2, 0x6c, 0xa3, 0x65, 0x82, 0x43, 0x5e, 0xf7, 0x74, 0x0b, 0xcb, 0x0f, 0xba, 0x2a, 0x5d, 0x38, - 0x48, 0xaf, 0xe6, 0xcf, 0x4e, 0x17, 0x41, 0x62, 0xe4, 0x07, 0xdb, 0x15, 0x82, 0x4b, 0xd8, 0xef, - 0xae, 0x8a, 0x5e, 0xc1, 0xbd, 0xc0, 0xfa, 0x07, 0x99, 0x79, 0x79, 0xb5, 0x14, 0x47, 0xde, 0xd7, - 0x5b, 0xfb, 0xd8, 0x92, 0xbd, 0x65, 0x6a, 0xba, 0x5e, 0x98, 0x96, 0x6e, 0xfa, 0x0c, 0x15, 0xc5, - 0x60, 0xa5, 0x5f, 0x71, 0x6a, 0x34, 0xb3, 0xa7, 0x33, 0xa2, 0xfc, 0xae, 0xa9, 0xdb, 0xa6, 0x51, - 0xca, 0x30, 0xe5, 0xb3, 0x27, 0x74, 0x0f, 0x8a, 0xce, 0xb1, 0x21, 0xef, 0x63, 0xc5, 0x72, 0x76, - 0xb1, 0xe2, 0x90, 0x59, 0x1a, 0x68, 0x8e, 0x82, 0x73, 0x6c, 0xd4, 0x02, 0x64, 0xf4, 0x02, 0x8a, - 0xba, 0x21, 0xef, 0xb5, 0xf5, 0xd6, 0xbe, 0x23, 0x1f, 0x59, 0xba, 0x83, 0xed, 0xd2, 0x3c, 0x55, - 0x48, 0x54, 0xbf, 0x6d, 0xf0, 0x75, 0x63, 0xed, 0x15, 0xc9, 0xc9, 0x55, 0x93, 0xd7, 0x8d, 0x27, - 0x94, 0x9f, 0x12, 0xed, 0xf5, 0x54, 0x7a, 0xb6, 0x98, 0x16, 0xff, 0x93, 0x00, 0x79, 0xb7, 0xbb, - 0xc5, 0x39, 0x32, 0xee, 0x42, 0xd1, 0x34, 0xb0, 0xdc, 0xdd, 0x57, 0x6c, 0xcc, 0xf5, 0xc8, 0x27, - 0x9c, 0xbc, 0x69, 0xe0, 0x6d, 0x42, 0x66, 0xea, 0x42, 0xdb, 0x30, 0x6f, 0x3b, 0x4a, 0x4b, 0x37, - 0x5a, 0x01, 0xf5, 0x4e, 0x4f, 0x0e, 0x16, 0x8a, 0x9c, 0xdb, 0xa3, 0x87, 0xbc, 0x94, 0x3f, 0x14, - 0x60, 0x7e, 0x45, 0xeb, 0xe8, 0x46, 0xa3, 0xdb, 0xd6, 0x63, 0x5d, 0x83, 0xb8, 0x09, 0x19, 0x9b, - 0xc8, 0xf4, 0x0d, 0xbe, 0x8f, 0x28, 0xd3, 0x34, 0x85, 0x58, 0xfe, 0x67, 0x50, 0xc0, 0xc7, 0x5d, - 0x9d, 0x7d, 0x7a, 0x60, 0x40, 0x28, 0x35, 0x79, 0xdd, 0xf2, 0x3e, 0x2f, 0x49, 0xe2, 0x75, 0xfa, - 0x0c, 0x50, 0xb0, 0x4a, 0x71, 0x62, 0x97, 0xcf, 0x60, 0x81, 0x8a, 0xde, 0x31, 0xec, 0x98, 0xf5, - 0x25, 0xfe, 0x3c, 0x5c, 0x08, 0x8b, 0x8e, 0xb3, 0xdc, 0xaf, 0x78, 0x2b, 0x6f, 0x62, 0x2b, 0x56, - 0x10, 0xeb, 0xe9, 0x9a, 0x0b, 0x8e, 0xb3, 0xcc, 0xbf, 0x2c, 0xc0, 0x15, 0x2a, 0x9b, 0x7e, 0x9d, - 0xd9, 0xc3, 0xd6, 0x33, 0xac, 0xd8, 0xb1, 0x22, 0xf0, 0x1b, 0x30, 0xc3, 0x90, 0x34, 0xed, 0x9f, - 0xd3, 0xab, 0x59, 0xe2, 0xb9, 0x34, 0x1c, 0xd3, 0x22, 0x9e, 0x0b, 0x4f, 0x12, 0x15, 0x28, 0x47, - 0x95, 0x22, 0xce, 0x9a, 0xfe, 0x2d, 0x01, 0xe6, 0xb9, 0xd3, 0x48, 0xba, 0x72, 0x65, 0x9f, 0xf8, - 0x4c, 0xa8, 0x0a, 0x59, 0x95, 0xfe, 0x92, 0x9d, 0x93, 0x2e, 0xa6, 0xf2, 0xf3, 0xa3, 0xfc, 0x4d, - 0xc6, 0xd6, 0x3c, 0xe9, 0x62, 0xe2, 0xb4, 0xba, 0xbf, 0x89, 0xa2, 0x02, 0x95, 0x1c, 0xe9, 0xb1, - 0xd2, 0x71, 0x44, 0xf3, 0xba, 0xae, 0x1f, 0xd7, 0xc1, 0x3f, 0x4a, 0x72, 0x25, 0xb0, 0x77, 0xf0, - 0xec, 0xb1, 0xfa, 0x28, 0x9f, 0x87, 0x3e, 0x8e, 0x05, 0x2b, 0x9e, 0x38, 0x47, 0xc5, 0x03, 0x2b, - 0xf4, 0x3e, 0x15, 0x7d, 0x06, 0x81, 0x35, 0x78, 0x99, 0xd5, 0xc9, 0x45, 0x3f, 0xe7, 0x51, 0xc7, - 0xbc, 0x2f, 0x85, 0xd1, 0x6d, 0x54, 0x81, 0x34, 0x3e, 0xee, 0xca, 0x1a, 0xb6, 0x55, 0x6e, 0xb8, - 0xc4, 0x61, 0x5f, 0xf1, 0x06, 0xf0, 0xc0, 0x2c, 0x3e, 0xee, 0x12, 0x22, 0xda, 0x21, 0xb3, 0x97, - 0xeb, 0x2a, 0xd0, 0x62, 0xdb, 0xe3, 0xe1, 0x85, 0xdf, 0x53, 0xb8, 0xb8, 0x82, 0xe7, 0x25, 0x30, - 0x11, 0xe2, 0x0f, 0x05, 0x78, 0x2b, 0xb2, 0xd5, 0xe2, 0x9c, 0xc8, 0x3e, 0x81, 0x14, 0xad, 0x7c, - 0xe2, 0x9c, 0x95, 0xa7, 0x5c, 0xe2, 0xf7, 0x12, 0x7c, 0x8c, 0x4b, 0xb8, 0x6d, 0x12, 0xc5, 0xc6, - 0xbe, 0xca, 0xf6, 0x1c, 0x72, 0x87, 0xa6, 0x43, 0x7c, 0x13, 0xde, 0xec, 0x89, 0x73, 0x37, 0xfb, - 0x1c, 0x15, 0xe0, 0xb6, 0xf8, 0x4b, 0x98, 0x37, 0x4c, 0x43, 0x0e, 0x0b, 0x3d, 0x7f, 0x5f, 0x2a, - 0x18, 0xa6, 0xf1, 0x32, 0x20, 0xd7, 0xb3, 0x33, 0x7d, 0x9a, 0x88, 0xd3, 0xce, 0x7c, 0x5f, 0x80, - 0x05, 0xcf, 0x6d, 0x8a, 0xd9, 0x83, 0xfe, 0x00, 0x92, 0x86, 0x79, 0x74, 0x9e, 0x55, 0x4c, 0x92, - 0x9f, 0xcc, 0x7a, 0xe1, 0x12, 0xc5, 0x59, 0xdf, 0x7f, 0x95, 0x80, 0xcc, 0xd3, 0x4a, 0x9c, 0xb5, - 0xfc, 0x84, 0xaf, 0x90, 0xb3, 0xf6, 0x8e, 0xea, 0xed, 0xde, 0xfb, 0x96, 0x9e, 0x56, 0x36, 0xf0, - 0x89, 0xdb, 0xdb, 0x09, 0x17, 0x5a, 0x81, 0x8c, 0xb3, 0x6f, 0x61, 0x7b, 0xdf, 0x6c, 0x6b, 0xe7, - 0x71, 0x73, 0x7c, 0xae, 0x32, 0x86, 0x69, 0x2a, 0xd7, 0x8d, 0xc6, 0x10, 0x22, 0xa2, 0x31, 0xc8, - 0x6b, 0x3c, 0x4f, 0x31, 0x71, 0x9e, 0xd7, 0x04, 0x5c, 0xc4, 0xe9, 0xe2, 0x8c, 0xf8, 0x02, 0x80, - 0x54, 0x27, 0xce, 0x26, 0xf9, 0x95, 0x24, 0xe4, 0xb7, 0x7b, 0xf6, 0x7e, 0xcc, 0xbd, 0xaf, 0x02, - 0xd0, 0xed, 0xd9, 0x14, 0x82, 0x1c, 0x1b, 0xbc, 0xce, 0x63, 0x02, 0x3d, 0xdc, 0x4a, 0x33, 0xbe, - 0xe6, 0xb1, 0x81, 0x6a, 0x5c, 0x08, 0x96, 0xfd, 0x68, 0x91, 0x1b, 0xa3, 0xc0, 0x6a, 0xf3, 0xd8, - 0xd8, 0xc4, 0x1e, 0x4a, 0x65, 0x92, 0x30, 0x91, 0xf4, 0x09, 0xcc, 0x92, 0x07, 0xd9, 0x31, 0xcf, - 0xd3, 0xcc, 0x33, 0x84, 0xa7, 0x69, 0xa2, 0xc7, 0x90, 0x61, 0xdc, 0x64, 0xf6, 0x9b, 0xa1, 0xb3, - 0x5f, 0x54, 0x5d, 0xb8, 0x1a, 0xe9, 0xbc, 0x97, 0xa6, 0xac, 0x64, 0xae, 0xbb, 0x00, 0xd3, 0x7b, - 0xa6, 0xa5, 0xba, 0xdf, 0x7b, 0xd9, 0x03, 0x6b, 0xcf, 0xf5, 0x54, 0x3a, 0x5d, 0xcc, 0xac, 0xa7, - 0xd2, 0x99, 0x22, 0x88, 0xbf, 0x21, 0x40, 0xc1, 0x6b, 0x88, 0x38, 0x27, 0x84, 0x4a, 0x48, 0x8b, - 0xe7, 0x6f, 0x0a, 0xa2, 0x40, 0xf1, 0x5f, 0x53, 0x8f, 0x48, 0x35, 0x0f, 0x69, 0xcb, 0xc4, 0xd9, - 0x53, 0x1e, 0xb3, 0x58, 0xa0, 0xc4, 0x79, 0x5b, 0x97, 0x86, 0x05, 0x3d, 0x80, 0x0b, 0x7a, 0x87, - 0xd8, 0x73, 0xdd, 0x69, 0x9f, 0x70, 0xd8, 0xe6, 0x60, 0xf7, 0xc3, 0xf2, 0x82, 0x9f, 0x56, 0x71, - 0x93, 0xc4, 0xbf, 0x43, 0x17, 0xc0, 0xfd, 0x9a, 0xc4, 0xa9, 0xea, 0x3a, 0xe4, 0x2c, 0x26, 0x9a, - 0xb8, 0x35, 0xe7, 0xd4, 0xf6, 0x9c, 0xc7, 0x4a, 0x14, 0xfe, 0x5b, 0x09, 0x28, 0xbc, 0xe8, 0x61, - 0xeb, 0xe4, 0xeb, 0xa4, 0xee, 0xdb, 0x50, 0x38, 0x52, 0x74, 0x47, 0xde, 0x33, 0x2d, 0xb9, 0xd7, - 0xd5, 0x14, 0xc7, 0x0d, 0x48, 0xc9, 0x11, 0xf2, 0x13, 0xd3, 0xda, 0xa1, 0x44, 0x84, 0x01, 0x1d, - 0x18, 0xe6, 0x91, 0x21, 0x13, 0x32, 0x05, 0xca, 0xc7, 0x06, 0x5f, 0x95, 0x5e, 0xfd, 0xf0, 0x3f, - 0x9e, 0x2e, 0x3e, 0x9a, 0x28, 0xcc, 0x8c, 0x86, 0xd4, 0xf5, 0x7a, 0xba, 0xb6, 0xb4, 0xb3, 0x53, - 0x5f, 0x93, 0x8a, 0x54, 0xe4, 0x2b, 0x26, 0xb1, 0x79, 0x6c, 0xd8, 0xe2, 0xdf, 0x4d, 0x40, 0xd1, - 0xd7, 0x51, 0x9c, 0x0d, 0x59, 0x85, 0xec, 0xeb, 0x1e, 0xb6, 0xf4, 0x37, 0x68, 0x46, 0xe0, 0x8c, - 0xc4, 0xec, 0x7c, 0x0e, 0x73, 0x21, 0x0d, 0x24, 0xbf, 0x9a, 0x06, 0xb2, 0x47, 0x7e, 0xe5, 0xd1, - 0x7d, 0x98, 0x77, 0x8e, 0x0d, 0x99, 0x05, 0x18, 0xb2, 0xa0, 0x14, 0x37, 0x5e, 0xa2, 0xe0, 0x10, - 0x7d, 0x10, 0x3a, 0x0d, 0x48, 0xb1, 0xc5, 0xdf, 0x13, 0x00, 0x51, 0x45, 0xd5, 0xd9, 0x67, 0x83, - 0xaf, 0x4b, 0x7f, 0xba, 0x0b, 0x45, 0x1a, 0xb2, 0x29, 0xeb, 0x7b, 0x72, 0x47, 0xb7, 0x6d, 0xdd, - 0x68, 0xf1, 0x0e, 0x95, 0xa7, 0xf4, 0xfa, 0xde, 0x26, 0xa3, 0x8a, 0x7f, 0x11, 0x16, 0x42, 0x15, - 0x88, 0xb3, 0xb1, 0xaf, 0xc3, 0xdc, 0x1e, 0xfb, 0x4a, 0x4b, 0x85, 0xf3, 0x15, 0xc7, 0x2c, 0xa5, - 0xb1, 0xf7, 0x89, 0xff, 0x2d, 0x01, 0x17, 0x24, 0x6c, 0x9b, 0xed, 0x43, 0x1c, 0xbf, 0x0a, 0x6b, - 0xc0, 0x3f, 0xe7, 0xc8, 0x6f, 0xa4, 0xc9, 0x0c, 0x63, 0x66, 0xd3, 0x5c, 0x78, 0xd9, 0xfe, 0xe6, - 0xe8, 0x1e, 0x3b, 0xb8, 0x50, 0xcf, 0x97, 0xfd, 0x52, 0xa1, 0x65, 0x3f, 0x13, 0x0a, 0x7a, 0xcb, - 0x30, 0x89, 0x4d, 0xb3, 0xf1, 0x6b, 0xa3, 0xd7, 0x71, 0xc1, 0xd0, 0xd2, 0xa8, 0x42, 0xd6, 0x19, - 0x4b, 0x03, 0xbf, 0xde, 0xea, 0x75, 0xa8, 0xef, 0xbc, 0x7a, 0x89, 0x94, 0xf7, 0xec, 0x74, 0x31, - 0x1f, 0x4a, 0xb3, 0xa5, 0xbc, 0xee, 0x3d, 0x13, 0xe9, 0xe2, 0xb7, 0xe1, 0x62, 0x9f, 0xb2, 0xe3, - 0xf4, 0x78, 0xfe, 0x45, 0x12, 0xae, 0x84, 0xc5, 0xc7, 0x0d, 0x71, 0xbe, 0xee, 0x0d, 0x5a, 0x83, - 0x5c, 0x47, 0x37, 0xde, 0x6c, 0xf5, 0x72, 0xae, 0xa3, 0x1b, 0xfe, 0x4a, 0x71, 0x44, 0xd7, 0x98, - 0xf9, 0xa9, 0x76, 0x0d, 0x05, 0xca, 0x51, 0x6d, 0x17, 0x67, 0xff, 0xf8, 0x9e, 0x00, 0x73, 0x71, - 0x2f, 0xcb, 0xbd, 0x59, 0xa0, 0x9c, 0xd8, 0x84, 0xdc, 0x4f, 0x61, 0x1d, 0xef, 0xb7, 0x04, 0x40, - 0x4d, 0xab, 0x67, 0x10, 0x50, 0xfb, 0xcc, 0x6c, 0xc5, 0x59, 0xcd, 0x0b, 0x30, 0xad, 0x1b, 0x1a, - 0x3e, 0xa6, 0xd5, 0x4c, 0x49, 0xec, 0x21, 0xf4, 0x75, 0x32, 0x39, 0xd1, 0xd7, 0x49, 0xf1, 0x73, - 0x58, 0x08, 0x15, 0x31, 0xce, 0xfa, 0xff, 0xd7, 0x04, 0x2c, 0xf0, 0x8a, 0xc4, 0xbe, 0x82, 0xf9, - 0x4d, 0x98, 0x6e, 0x13, 0x99, 0x23, 0xda, 0x99, 0xbe, 0xd3, 0x6d, 0x67, 0x9a, 0x19, 0xfd, 0x2c, - 0x40, 0xd7, 0xc2, 0x87, 0x32, 0x63, 0x4d, 0x4e, 0xc4, 0x9a, 0x21, 0x1c, 0x94, 0x80, 0x7e, 0x20, - 0x40, 0x81, 0x0c, 0xe8, 0xae, 0x65, 0x76, 0x4d, 0x9b, 0xf8, 0x2c, 0xf6, 0x64, 0x30, 0xe7, 0xc5, - 0xd9, 0xe9, 0x62, 0x6e, 0x53, 0x37, 0xb6, 0x39, 0x63, 0xb3, 0x31, 0xf1, 0x1e, 0x00, 0x77, 0x27, - 0xc4, 0x52, 0xa5, 0x6d, 0xaa, 0x07, 0xfe, 0xf7, 0x36, 0x62, 0x59, 0x3c, 0x71, 0xb6, 0xf8, 0xef, - 0x05, 0xb8, 0xf0, 0x53, 0x5b, 0x2e, 0xfe, 0x7f, 0xa1, 0x6c, 0xf1, 0x25, 0x14, 0xe9, 0x8f, 0xba, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_cdfee8e6fce2cd19) } + +var fileDescriptor_api_cdfee8e6fce2cd19 = []byte{ + // 8233 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5b, 0x6c, 0x23, 0xc9, + 0x7a, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0xa7, 0x78, 0x51, 0x69, 0x2e, 0x1c, 0xee, 0xee, 0x68, 0xa6, + 0xe7, 0x3e, 0x67, 0x57, 0xda, 0x99, 0x39, 0x9b, 0x5d, 0xef, 0xac, 0xf7, 0x58, 0xa2, 0x38, 0x43, + 0x4a, 0x23, 0x8d, 0xa6, 0x49, 0xcd, 0x60, 0xd7, 0xc7, 0xe9, 0xd3, 0xea, 0x2e, 0x51, 0x7d, 0x44, + 0x76, 0x73, 0xba, 0x9b, 0xba, 0x2c, 0x10, 0x20, 0x8e, 0x0d, 0xe7, 0x3c, 0x19, 0xe7, 0xc1, 0x41, + 0x8e, 0xe1, 0x20, 0x3e, 0x8e, 0x83, 0xf8, 0x21, 0x40, 0x12, 0x20, 0x41, 0x6e, 0x48, 0xec, 0x17, + 0x03, 0x39, 0x08, 0x9c, 0xf8, 0xf8, 0x29, 0x46, 0x80, 0x28, 0xb6, 0x4e, 0x5e, 0x02, 0x07, 0x46, + 0x10, 0x04, 0x30, 0xb0, 0x0f, 0x41, 0x50, 0x97, 0xbe, 0x91, 0xcd, 0x8b, 0x66, 0xfb, 0x24, 0x0b, + 0xf8, 0x45, 0x62, 0xff, 0x55, 0xff, 0xdf, 0x55, 0x7f, 0x55, 0xfd, 0xf5, 0x7f, 0xd5, 0x7f, 0x55, + 0xc1, 0xbc, 0x65, 0x2a, 0xea, 0x7e, 0x77, 0x77, 0x59, 0xe9, 0xea, 0x4b, 0x5d, 0xcb, 0x74, 0x4c, + 0x34, 0xaf, 0x9a, 0xea, 0x01, 0x25, 0x2f, 0xf1, 0xc4, 0xf2, 0xfd, 0x83, 0xc3, 0xe5, 0x83, 0x43, + 0x1b, 0x5b, 0x87, 0xd8, 0x5a, 0x56, 0x4d, 0x43, 0xed, 0x59, 0x16, 0x36, 0xd4, 0x93, 0xe5, 0xb6, + 0xa9, 0x1e, 0xd0, 0x3f, 0xba, 0xd1, 0x62, 0xec, 0x65, 0xe4, 0x4a, 0xd4, 0x14, 0x47, 0xe1, 0xb4, + 0x0b, 0x2e, 0x0d, 0x5b, 0x96, 0x69, 0xd9, 0x9c, 0x7a, 0xc9, 0xa5, 0x76, 0xb0, 0xa3, 0x04, 0x72, + 0xbf, 0x65, 0x3b, 0xa6, 0xa5, 0xb4, 0xf0, 0x32, 0x36, 0x5a, 0xba, 0x81, 0x49, 0x86, 0x43, 0x55, + 0xe5, 0x89, 0x6f, 0x47, 0x26, 0x3e, 0xe2, 0xa9, 0xa5, 0x9e, 0xa3, 0xb7, 0x97, 0xf7, 0xdb, 0xea, + 0xb2, 0xa3, 0x77, 0xb0, 0xed, 0x28, 0x9d, 0x2e, 0x4f, 0xb9, 0x4f, 0x53, 0x1c, 0x4b, 0x51, 0x75, + 0xa3, 0xe5, 0xfe, 0xef, 0xee, 0x2e, 0x5b, 0x58, 0x35, 0x2d, 0x0d, 0x6b, 0xb2, 0xdd, 0x55, 0x0c, + 0xb7, 0xb8, 0x2d, 0xb3, 0x65, 0xd2, 0x9f, 0xcb, 0xe4, 0x17, 0xa7, 0x5e, 0x6d, 0x99, 0x66, 0xab, + 0x8d, 0x97, 0xe9, 0xd3, 0x6e, 0x6f, 0x6f, 0x59, 0xeb, 0x59, 0x8a, 0xa3, 0x9b, 0x9c, 0x4b, 0xfc, + 0xe7, 0x02, 0xe4, 0x24, 0xfc, 0xba, 0x87, 0x6d, 0xa7, 0x86, 0x15, 0x0d, 0x5b, 0xe8, 0x0a, 0x24, + 0x0f, 0xf0, 0x49, 0x29, 0x79, 0x4d, 0xb8, 0x3b, 0xb7, 0x3a, 0xfb, 0xe5, 0xe9, 0x62, 0x72, 0x03, + 0x9f, 0x48, 0x84, 0x86, 0xae, 0xc1, 0x2c, 0x36, 0x34, 0x99, 0x24, 0xa7, 0xc2, 0xc9, 0x33, 0xd8, + 0xd0, 0x36, 0xf0, 0x09, 0xfa, 0x36, 0xa4, 0x6d, 0x22, 0xcd, 0x50, 0x71, 0x69, 0xfa, 0x9a, 0x70, + 0x77, 0x7a, 0xf5, 0xe7, 0xbe, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x25, 0xd5, + 0xec, 0x2c, 0x7b, 0xed, 0xa4, 0xed, 0xfa, 0xbf, 0x97, 0xbb, 0x07, 0xad, 0xe5, 0x7e, 0x1d, 0x2d, + 0x35, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, 0xa5, + 0x13, 0xc5, 0xa4, 0xf8, 0x87, 0x09, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xc9, 0xdf, + 0x87, 0xa4, 0x73, 0x6c, 0xd0, 0x92, 0x67, 0x1f, 0x5e, 0x5d, 0x1a, 0xe8, 0x11, 0x4b, 0x4d, 0x4b, + 0x31, 0x6c, 0x45, 0x25, 0xd5, 0x97, 0x48, 0x56, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0xf7, 0x3a, 0x98, + 0x2a, 0x92, 0x56, 0x2a, 0xfb, 0xf0, 0x72, 0x04, 0x67, 0xa3, 0xab, 0x18, 0x12, 0xb0, 0xbc, 0xe4, + 0x37, 0xba, 0x02, 0x69, 0xa3, 0xd7, 0x21, 0xaa, 0xb0, 0x69, 0x45, 0x93, 0xd2, 0xac, 0xd1, 0xeb, + 0x6c, 0xe0, 0x13, 0x1b, 0x35, 0x20, 0xc7, 0x85, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x34, 0x7b, 0x4d, + 0xb8, 0x9b, 0x7f, 0xb8, 0x14, 0x21, 0x36, 0x5c, 0x01, 0xf2, 0xd8, 0xeb, 0x60, 0x89, 0x72, 0x49, + 0x73, 0x56, 0xe0, 0x09, 0xbd, 0x05, 0x19, 0xf2, 0xbe, 0xdd, 0x13, 0x07, 0xdb, 0xa5, 0x34, 0x7d, + 0x21, 0x29, 0xc0, 0x2a, 0x79, 0x16, 0x3f, 0x85, 0xb9, 0x20, 0x2b, 0x42, 0x90, 0x97, 0xaa, 0x8d, + 0x9d, 0xcd, 0xaa, 0xbc, 0xb3, 0xb5, 0xb1, 0xf5, 0xfc, 0xd5, 0x56, 0x71, 0x0a, 0x5d, 0x80, 0x22, + 0xa7, 0x6d, 0x54, 0x3f, 0x93, 0x9f, 0xd5, 0x37, 0xeb, 0xcd, 0xa2, 0x50, 0x4e, 0x7d, 0xef, 0xb7, + 0xaf, 0x4e, 0xad, 0xa7, 0xd2, 0x33, 0xc5, 0x59, 0x71, 0x1b, 0xe0, 0x29, 0x76, 0x78, 0x67, 0x40, + 0xab, 0x30, 0xb3, 0x4f, 0x4b, 0x55, 0x12, 0xa8, 0x56, 0xae, 0x45, 0x16, 0x3f, 0xd0, 0x71, 0x56, + 0xd3, 0x3f, 0x3a, 0x5d, 0x9c, 0xfa, 0xf1, 0xe9, 0xa2, 0x20, 0x71, 0x4e, 0xf1, 0xf7, 0x04, 0xc8, + 0x52, 0x91, 0xac, 0x96, 0xa8, 0xd2, 0x27, 0xf3, 0xfa, 0x58, 0x95, 0x0c, 0x0a, 0x45, 0x4b, 0x30, + 0x7d, 0xa8, 0xb4, 0x7b, 0xb8, 0x94, 0xa0, 0x32, 0x4a, 0x11, 0x32, 0x5e, 0x92, 0x74, 0x89, 0x65, + 0x43, 0x8f, 0x61, 0x4e, 0x37, 0x1c, 0x6c, 0x38, 0x32, 0x63, 0x4b, 0x8e, 0x61, 0xcb, 0xb2, 0xdc, + 0xf4, 0x41, 0xfc, 0x67, 0x02, 0xc0, 0x76, 0x2f, 0x4e, 0xa5, 0xa0, 0x6f, 0x4e, 0x58, 0xfe, 0xd5, + 0x14, 0x61, 0x75, 0x6b, 0x71, 0x09, 0x66, 0x74, 0xa3, 0xad, 0x1b, 0xac, 0xfc, 0x69, 0x89, 0x3f, + 0xa1, 0x0b, 0x30, 0xbd, 0xdb, 0xd6, 0x0d, 0x8d, 0xf6, 0xdd, 0xb4, 0xc4, 0x1e, 0x44, 0x09, 0xb2, + 0xb4, 0xd4, 0x31, 0xea, 0x5d, 0x3c, 0x4d, 0xc0, 0xc5, 0x8a, 0x69, 0x68, 0x3a, 0x19, 0x3e, 0x4a, + 0xfb, 0x6b, 0xa1, 0x95, 0x75, 0xb8, 0xa0, 0xe1, 0xae, 0x85, 0x55, 0xc5, 0xc1, 0x9a, 0x8c, 0x8f, + 0xbb, 0x13, 0xb6, 0x31, 0xf2, 0xb9, 0xaa, 0xc7, 0x5d, 0x4a, 0x8b, 0xd6, 0x24, 0xfa, 0x26, 0x5c, + 0x56, 0xda, 0x6d, 0xf3, 0x48, 0xd6, 0xf7, 0x64, 0xcd, 0xc4, 0xb6, 0x6c, 0x98, 0x8e, 0x8c, 0x8f, + 0x75, 0xdb, 0xa1, 0xc3, 0x3e, 0x2d, 0x2d, 0xd0, 0xe4, 0xfa, 0xde, 0x9a, 0x89, 0xed, 0x2d, 0xd3, + 0xa9, 0x92, 0x24, 0x32, 0x5a, 0x49, 0x61, 0xd8, 0x68, 0x9d, 0x21, 0xa6, 0x52, 0x4a, 0xe3, 0xe3, + 0x2e, 0x1d, 0xad, 0x81, 0xa6, 0x9c, 0x0d, 0x36, 0xa5, 0xf8, 0x0b, 0x70, 0xa9, 0x5f, 0xbf, 0x71, + 0xb6, 0xdf, 0x1f, 0x08, 0x90, 0xaf, 0x1b, 0xba, 0xf3, 0xb5, 0x68, 0x38, 0x4f, 0xd9, 0xc9, 0xa0, + 0xb2, 0xef, 0x43, 0x71, 0x4f, 0xd1, 0xdb, 0xcf, 0x8d, 0xa6, 0xd9, 0xd9, 0xb5, 0x1d, 0xd3, 0xc0, + 0x36, 0x6f, 0x8d, 0x01, 0xba, 0xf8, 0x12, 0x0a, 0x5e, 0x6d, 0xe2, 0x54, 0x93, 0x03, 0xc5, 0xba, + 0xa1, 0x5a, 0xb8, 0x83, 0x8d, 0x58, 0xf5, 0xf4, 0x36, 0x64, 0x74, 0x57, 0x2e, 0xd5, 0x55, 0x52, + 0xf2, 0x09, 0x62, 0x0f, 0xe6, 0x03, 0x6f, 0x8d, 0xd3, 0x5c, 0x92, 0x89, 0x03, 0x1f, 0xc9, 0x7e, + 0x1b, 0x91, 0x89, 0x03, 0x1f, 0x31, 0xf3, 0xd6, 0x80, 0xdc, 0x1a, 0x6e, 0x63, 0x07, 0xc7, 0x69, + 0xf5, 0x77, 0x20, 0xef, 0x0a, 0x8d, 0xb3, 0x61, 0xfe, 0xb6, 0x00, 0x88, 0xcb, 0x55, 0x8c, 0x56, + 0x9c, 0x25, 0x46, 0x8b, 0xc4, 0x0d, 0x70, 0x7a, 0x96, 0xc1, 0xe6, 0x73, 0xd6, 0x27, 0x81, 0x91, + 0xe8, 0x94, 0xee, 0x0f, 0xd9, 0x54, 0x70, 0xc8, 0x72, 0x57, 0xe4, 0x08, 0x16, 0x42, 0x05, 0x8b, + 0xb7, 0xf9, 0x52, 0xb4, 0x4c, 0x89, 0x6b, 0xc9, 0xa0, 0xbf, 0x45, 0x89, 0xe2, 0x0f, 0x04, 0x98, + 0xaf, 0xb4, 0xb1, 0x62, 0xc5, 0xae, 0x91, 0x6f, 0x41, 0x5a, 0xc3, 0x8a, 0x46, 0xab, 0xcc, 0x06, + 0xf6, 0x3b, 0x01, 0x29, 0xc4, 0x2b, 0x5d, 0xda, 0x6f, 0xab, 0x4b, 0x4d, 0xd7, 0x5f, 0xe5, 0xa3, + 0xdb, 0x63, 0x12, 0x3f, 0x03, 0x14, 0x2c, 0x59, 0x9c, 0x1d, 0xe1, 0x7f, 0x0b, 0x80, 0x24, 0x7c, + 0x88, 0x2d, 0x27, 0xf6, 0x6a, 0xaf, 0x41, 0xd6, 0x51, 0xac, 0x16, 0x76, 0x64, 0xe2, 0x89, 0x9f, + 0xa7, 0xe6, 0xc0, 0xf8, 0x08, 0x19, 0x35, 0xe1, 0x0e, 0x36, 0x94, 0xdd, 0x36, 0xa6, 0x52, 0xe4, + 0x5d, 0xb3, 0x67, 0x68, 0xb2, 0xee, 0x60, 0x4b, 0x71, 0x4c, 0x4b, 0x36, 0xbb, 0x8e, 0xde, 0xd1, + 0xbf, 0xa0, 0x4e, 0x38, 0xef, 0x6a, 0x37, 0x58, 0x76, 0xc2, 0xbc, 0x4a, 0x32, 0xd7, 0x79, 0xde, + 0xe7, 0x81, 0xac, 0xe2, 0xe7, 0xb0, 0x10, 0xaa, 0x75, 0x9c, 0x2a, 0xfd, 0x9f, 0x02, 0x64, 0x1b, + 0xaa, 0x62, 0xc4, 0xa9, 0xcb, 0x4f, 0x21, 0x6b, 0xab, 0x8a, 0x21, 0xef, 0x99, 0x56, 0x47, 0x71, + 0xe8, 0xc0, 0xc9, 0x87, 0x74, 0xe9, 0xf9, 0xd6, 0xaa, 0x62, 0x3c, 0xa1, 0x99, 0x24, 0xb0, 0xbd, + 0xdf, 0xe8, 0x05, 0x64, 0x0f, 0xf0, 0x89, 0xcc, 0x71, 0x1a, 0x9d, 0x6d, 0xf3, 0x0f, 0xdf, 0x0f, + 0xf0, 0x1f, 0x1c, 0x2e, 0xb9, 0xf0, 0x6e, 0x29, 0x00, 0xef, 0x96, 0x08, 0xc7, 0x52, 0xc3, 0xb1, + 0xb0, 0xd1, 0x72, 0xf6, 0x25, 0x38, 0xc0, 0x27, 0xcf, 0x98, 0x0c, 0x36, 0x5c, 0xd7, 0x53, 0xe9, + 0x64, 0x31, 0x25, 0xfe, 0x85, 0x00, 0x73, 0xac, 0xca, 0x71, 0x0e, 0xd7, 0x0f, 0x20, 0x65, 0x99, + 0x47, 0x6c, 0xb8, 0x66, 0x1f, 0xbe, 0x15, 0x21, 0x62, 0x03, 0x9f, 0x04, 0xe7, 0x43, 0x9a, 0x1d, + 0xad, 0x02, 0xf7, 0x3a, 0x65, 0xca, 0x9d, 0x9c, 0x94, 0x1b, 0x18, 0x97, 0x44, 0x64, 0xdc, 0x81, + 0xc2, 0xae, 0xe2, 0xa8, 0xfb, 0xb2, 0xc5, 0x0b, 0x49, 0xe6, 0xce, 0xe4, 0xdd, 0x39, 0x29, 0x4f, + 0xc9, 0x6e, 0xd1, 0x6d, 0x52, 0x73, 0x36, 0x7e, 0x6c, 0xfc, 0x97, 0xac, 0xcd, 0xff, 0x8f, 0xc0, + 0xc7, 0x90, 0x5b, 0xf3, 0xbf, 0x6c, 0x4d, 0xff, 0xeb, 0x09, 0xb8, 0x5c, 0xd9, 0xc7, 0xea, 0x41, + 0xc5, 0x34, 0x6c, 0xdd, 0x76, 0x88, 0xee, 0xe2, 0x6c, 0xff, 0xb7, 0x20, 0x73, 0xa4, 0x3b, 0xfb, + 0xb2, 0xa6, 0xef, 0xed, 0x51, 0xeb, 0x99, 0x96, 0xd2, 0x84, 0xb0, 0xa6, 0xef, 0xed, 0xa1, 0x47, + 0x90, 0xea, 0x98, 0x1a, 0x73, 0xce, 0xf3, 0x0f, 0x17, 0x23, 0xc4, 0xd3, 0xa2, 0xd9, 0xbd, 0xce, + 0xa6, 0xa9, 0x61, 0x89, 0x66, 0x46, 0x57, 0x01, 0x54, 0x42, 0xed, 0x9a, 0xba, 0xe1, 0xf0, 0xd9, + 0x37, 0x40, 0x41, 0x35, 0xc8, 0x38, 0xd8, 0xea, 0xe8, 0x86, 0xe2, 0xe0, 0xd2, 0x34, 0x55, 0xde, + 0xcd, 0xc8, 0x82, 0x77, 0xdb, 0xba, 0xaa, 0xac, 0x61, 0x5b, 0xb5, 0xf4, 0xae, 0x63, 0x5a, 0x5c, + 0x8b, 0x3e, 0xb3, 0xf8, 0xab, 0x29, 0x28, 0x0d, 0xea, 0x26, 0xce, 0x1e, 0xb2, 0x0d, 0x33, 0x04, + 0xd3, 0xb7, 0x1d, 0xde, 0x47, 0x1e, 0x0e, 0x53, 0x41, 0x44, 0x09, 0xe8, 0xda, 0x40, 0xdb, 0xe1, + 0xc5, 0xe6, 0x72, 0xca, 0xff, 0x46, 0x80, 0x19, 0x96, 0x80, 0x1e, 0x40, 0xda, 0x22, 0x13, 0x83, + 0xac, 0x6b, 0xb4, 0x8c, 0xc9, 0xd5, 0x4b, 0x67, 0xa7, 0x8b, 0xb3, 0x74, 0xb2, 0xa8, 0xaf, 0x7d, + 0xe9, 0xff, 0x94, 0x66, 0x69, 0xbe, 0xba, 0x46, 0x5a, 0xcb, 0x76, 0x14, 0xcb, 0xa1, 0x0b, 0x3a, + 0x09, 0x86, 0x52, 0x28, 0x61, 0x03, 0x9f, 0xa0, 0x75, 0x98, 0xb1, 0x1d, 0xc5, 0xe9, 0xd9, 0xbc, + 0xbd, 0xce, 0x55, 0xd8, 0x06, 0xe5, 0x94, 0xb8, 0x04, 0xe2, 0x3e, 0x69, 0xd8, 0x51, 0xf4, 0x36, + 0x6d, 0xc0, 0x8c, 0xc4, 0x9f, 0xc4, 0xdf, 0x10, 0x60, 0x86, 0x65, 0x45, 0x97, 0x61, 0x41, 0x5a, + 0xd9, 0x7a, 0x5a, 0x95, 0xeb, 0x5b, 0x6b, 0xd5, 0x66, 0x55, 0xda, 0xac, 0x6f, 0xad, 0x34, 0xab, + 0xc5, 0x29, 0x74, 0x09, 0x90, 0x9b, 0x50, 0x79, 0xbe, 0xd5, 0xa8, 0x37, 0x9a, 0xd5, 0xad, 0x66, + 0x51, 0xa0, 0xeb, 0x19, 0x94, 0x1e, 0xa0, 0x26, 0xd0, 0x4d, 0xb8, 0xd6, 0x4f, 0x95, 0x1b, 0xcd, + 0x95, 0x66, 0x43, 0xae, 0x36, 0x9a, 0xf5, 0xcd, 0x95, 0x66, 0x75, 0xad, 0x98, 0x1c, 0x91, 0x8b, + 0xbc, 0x44, 0x92, 0xaa, 0x95, 0x66, 0x31, 0x25, 0x3a, 0x70, 0x51, 0xc2, 0xaa, 0xd9, 0xe9, 0xf6, + 0x1c, 0x4c, 0x4a, 0x69, 0xc7, 0x39, 0x52, 0x2e, 0xc3, 0xac, 0x66, 0x9d, 0xc8, 0x56, 0xcf, 0xe0, + 0xe3, 0x64, 0x46, 0xb3, 0x4e, 0xa4, 0x9e, 0x21, 0xfe, 0x13, 0x01, 0x2e, 0xf5, 0xbf, 0x36, 0xce, + 0x4e, 0xf8, 0x02, 0xb2, 0x8a, 0xa6, 0x61, 0x4d, 0xd6, 0x70, 0xdb, 0x51, 0xb8, 0x8b, 0x73, 0x3f, + 0x20, 0x89, 0x2f, 0xc3, 0x2d, 0x79, 0xcb, 0x70, 0x9b, 0x2f, 0x2b, 0x15, 0x5a, 0x90, 0x35, 0xc2, + 0xe1, 0x9a, 0x1f, 0x2a, 0x84, 0x52, 0xc4, 0xff, 0x91, 0x82, 0x5c, 0xd5, 0xd0, 0x9a, 0xc7, 0xb1, + 0xce, 0x25, 0x97, 0x60, 0x46, 0x35, 0x3b, 0x1d, 0xdd, 0x71, 0x15, 0xc4, 0x9e, 0xd0, 0xcf, 0x04, + 0x5c, 0xd3, 0xe4, 0x04, 0x0e, 0x9a, 0xef, 0x94, 0xa2, 0xef, 0xc0, 0x65, 0x62, 0x35, 0x2d, 0x43, + 0x69, 0xcb, 0x4c, 0x9a, 0xec, 0x58, 0x7a, 0xab, 0x85, 0x2d, 0xbe, 0xf4, 0x77, 0x37, 0xa2, 0x9c, + 0x75, 0xce, 0x51, 0xa1, 0x0c, 0x4d, 0x96, 0x5f, 0xba, 0xa8, 0x47, 0x91, 0xd1, 0x27, 0x00, 0x64, + 0x2a, 0xa2, 0xcb, 0x89, 0x36, 0xb7, 0x47, 0xc3, 0xd6, 0x13, 0x5d, 0x13, 0x44, 0x18, 0xc8, 0xb3, + 0x8d, 0x96, 0x09, 0x0e, 0x79, 0xdd, 0xd3, 0x2d, 0x2c, 0x3f, 0xe8, 0xaa, 0x74, 0xe1, 0x20, 0xbd, + 0x9a, 0x3f, 0x3b, 0x5d, 0x04, 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x2e, 0x61, 0xbf, 0xbb, 0x2a, + 0x7a, 0x05, 0xf7, 0x02, 0xeb, 0x1f, 0x64, 0xe6, 0xe5, 0xd5, 0x52, 0x1c, 0x79, 0x5f, 0x6f, 0xed, + 0x63, 0x4b, 0xf6, 0x96, 0x94, 0xe9, 0xaa, 0x61, 0x5a, 0xba, 0xe9, 0x33, 0x54, 0x14, 0x83, 0x95, + 0x7e, 0xc5, 0xa9, 0xd1, 0xcc, 0x9e, 0xce, 0x88, 0xf2, 0xbb, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xc3, + 0x94, 0xcf, 0x9e, 0xd0, 0x3d, 0x28, 0x3a, 0xc7, 0x86, 0xbc, 0x8f, 0x15, 0xcb, 0xd9, 0xc5, 0x8a, + 0x43, 0x66, 0x69, 0xa0, 0x39, 0x0a, 0xce, 0xb1, 0x51, 0x0b, 0x90, 0xd1, 0x0b, 0x28, 0xea, 0x86, + 0xbc, 0xd7, 0xd6, 0x5b, 0xfb, 0x8e, 0x7c, 0x64, 0xe9, 0x0e, 0xb6, 0x4b, 0xf3, 0x54, 0x21, 0x51, + 0xfd, 0xb6, 0xc1, 0xd7, 0x78, 0xb5, 0x57, 0x24, 0x27, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, 0x7e, + 0x4a, 0xb4, 0xd7, 0x53, 0xe9, 0xd9, 0x62, 0x5a, 0xfc, 0x2f, 0x02, 0xe4, 0xdd, 0xee, 0x16, 0xe7, + 0xc8, 0xb8, 0x0b, 0x45, 0xd3, 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x9f, 0x70, 0xf2, + 0xa6, 0x81, 0xb7, 0x09, 0x99, 0xa9, 0x0b, 0x6d, 0xc3, 0xbc, 0xed, 0x28, 0x2d, 0xdd, 0x68, 0x05, + 0xd4, 0x3b, 0x3d, 0x39, 0x58, 0x28, 0x72, 0x6e, 0x8f, 0x1e, 0xf2, 0x52, 0xfe, 0x48, 0x80, 0xf9, + 0x15, 0xad, 0xa3, 0x1b, 0x8d, 0x6e, 0x5b, 0x8f, 0x75, 0x0d, 0xe2, 0x26, 0x64, 0x6c, 0x22, 0xd3, + 0x37, 0xf8, 0x3e, 0xa2, 0x4c, 0xd3, 0x14, 0x62, 0xf9, 0x9f, 0x41, 0x01, 0x1f, 0x77, 0x75, 0xf6, + 0x99, 0x80, 0x01, 0xa1, 0xd4, 0xe4, 0x75, 0xcb, 0xfb, 0xbc, 0x24, 0x89, 0xd7, 0xe9, 0x33, 0x40, + 0xc1, 0x2a, 0xc5, 0x89, 0x5d, 0x3e, 0x83, 0x05, 0x2a, 0x7a, 0xc7, 0xb0, 0x63, 0xd6, 0x97, 0xf8, + 0xf3, 0x70, 0x21, 0x2c, 0x3a, 0xce, 0x72, 0xbf, 0xe2, 0xad, 0xbc, 0x89, 0xad, 0x58, 0x41, 0xac, + 0xa7, 0x6b, 0x2e, 0x38, 0xce, 0x32, 0xff, 0xb2, 0x00, 0x57, 0xa8, 0x6c, 0xfa, 0x25, 0x65, 0x0f, + 0x5b, 0xcf, 0xb0, 0x62, 0xc7, 0x8a, 0xc0, 0x6f, 0xc0, 0x0c, 0x43, 0xd2, 0xb4, 0x7f, 0x4e, 0xaf, + 0x66, 0x89, 0xe7, 0xd2, 0x70, 0x4c, 0x8b, 0x78, 0x2e, 0x3c, 0x49, 0x54, 0xa0, 0x1c, 0x55, 0x8a, + 0x38, 0x6b, 0xfa, 0x77, 0x05, 0x98, 0xe7, 0x4e, 0x23, 0xe9, 0xca, 0x95, 0x7d, 0xe2, 0x33, 0xa1, + 0x2a, 0x64, 0x55, 0xfa, 0x4b, 0x76, 0x4e, 0xba, 0x98, 0xca, 0xcf, 0x8f, 0xf2, 0x37, 0x19, 0x5b, + 0xf3, 0xa4, 0x8b, 0x89, 0xd3, 0xea, 0xfe, 0x26, 0x8a, 0x0a, 0x54, 0x72, 0xa4, 0xc7, 0x4a, 0xc7, + 0x11, 0xcd, 0xeb, 0xba, 0x7e, 0x5c, 0x07, 0xff, 0x34, 0xc9, 0x95, 0xc0, 0xde, 0xc1, 0xb3, 0xc7, + 0xea, 0xa3, 0x7c, 0x0e, 0x97, 0x82, 0xb3, 0x4b, 0xa0, 0xe2, 0x89, 0x73, 0x54, 0x3c, 0xb0, 0x42, + 0xef, 0x53, 0xd1, 0x67, 0x10, 0x58, 0x83, 0x97, 0x59, 0x9d, 0x5c, 0xf4, 0x73, 0x1e, 0x75, 0xcc, + 0xfb, 0x52, 0x18, 0xdd, 0x46, 0x15, 0x48, 0xe3, 0xe3, 0xae, 0xac, 0x61, 0x5b, 0xe5, 0x86, 0x4b, + 0x8c, 0x12, 0x48, 0x8a, 0x32, 0x80, 0x07, 0x66, 0xf1, 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x99, 0xbd, + 0x5c, 0x57, 0x81, 0x16, 0xdb, 0x1e, 0x0f, 0x2f, 0xfc, 0x9e, 0xc2, 0xc5, 0x15, 0x3c, 0x2f, 0x81, + 0x89, 0x10, 0x7f, 0x28, 0xc0, 0x5b, 0x91, 0xad, 0x16, 0xe7, 0x44, 0xf6, 0x09, 0xa4, 0x68, 0xe5, + 0x13, 0xe7, 0xac, 0x3c, 0xe5, 0x12, 0xbf, 0x97, 0xe0, 0x63, 0x5c, 0xc2, 0x6d, 0x93, 0x28, 0x36, + 0xf6, 0x55, 0xb6, 0xe7, 0x90, 0x3b, 0x34, 0x1d, 0xe2, 0x9b, 0xf0, 0x66, 0x4f, 0x9c, 0xbb, 0xd9, + 0xe7, 0xa8, 0x00, 0xb7, 0xc5, 0x5f, 0xc2, 0xbc, 0x61, 0x1a, 0x72, 0x58, 0xe8, 0xf9, 0xfb, 0x52, + 0xc1, 0x30, 0x8d, 0x97, 0x01, 0xb9, 0x9e, 0x9d, 0xe9, 0xd3, 0x44, 0x9c, 0x76, 0xe6, 0xfb, 0x02, + 0x2c, 0x78, 0x6e, 0x53, 0xcc, 0x1e, 0xf4, 0x07, 0x90, 0x34, 0xcc, 0xa3, 0xf3, 0xac, 0x62, 0x92, + 0xfc, 0x64, 0xd6, 0x0b, 0x97, 0x28, 0xce, 0xfa, 0xfe, 0xdb, 0x04, 0x64, 0x9e, 0x56, 0xe2, 0xac, + 0xe5, 0x27, 0x7c, 0x85, 0x9c, 0xb5, 0x77, 0x54, 0x6f, 0xf7, 0xde, 0xb7, 0xf4, 0xb4, 0xb2, 0x81, + 0x4f, 0xdc, 0xde, 0x4e, 0xb8, 0xd0, 0x0a, 0x64, 0x9c, 0x7d, 0x0b, 0xdb, 0xfb, 0x66, 0x5b, 0x3b, + 0x8f, 0x9b, 0xe3, 0x73, 0x95, 0x31, 0x4c, 0x53, 0xb9, 0x6e, 0xe4, 0x84, 0x10, 0x11, 0x39, 0x41, + 0x5e, 0xe3, 0x79, 0x8a, 0x89, 0xf3, 0xbc, 0x26, 0xe0, 0x22, 0x4e, 0x17, 0x67, 0xc4, 0x17, 0x00, + 0xa4, 0x3a, 0x71, 0x36, 0xc9, 0xaf, 0x24, 0x21, 0xbf, 0xdd, 0xb3, 0xf7, 0x63, 0xee, 0x7d, 0x15, + 0x80, 0x6e, 0xcf, 0xa6, 0x10, 0xe4, 0xd8, 0xe0, 0x75, 0x1e, 0x13, 0x94, 0xe1, 0x56, 0x9a, 0xf1, + 0x35, 0x8f, 0x0d, 0x54, 0xe3, 0x42, 0xb0, 0xec, 0x47, 0x76, 0xdc, 0x18, 0x05, 0x56, 0x9b, 0xc7, + 0xc6, 0x26, 0xf6, 0x50, 0x2a, 0x93, 0x84, 0x89, 0xa4, 0x4f, 0x60, 0x96, 0x3c, 0xc8, 0x8e, 0x79, + 0x9e, 0x66, 0x9e, 0x21, 0x3c, 0x4d, 0x13, 0x3d, 0x86, 0x0c, 0xe3, 0x26, 0xb3, 0xdf, 0x0c, 0x9d, + 0xfd, 0xa2, 0xea, 0xc2, 0xd5, 0x48, 0xe7, 0xbd, 0x34, 0x65, 0x25, 0x73, 0xdd, 0x05, 0x98, 0xde, + 0x33, 0x2d, 0xd5, 0xfd, 0xde, 0xcb, 0x1e, 0x58, 0x7b, 0xae, 0xa7, 0xd2, 0xe9, 0x62, 0x66, 0x3d, + 0x95, 0xce, 0x14, 0x41, 0xfc, 0x0d, 0x01, 0x0a, 0x5e, 0x43, 0xc4, 0x39, 0x21, 0x54, 0x42, 0x5a, + 0x3c, 0x7f, 0x53, 0x10, 0x05, 0x8a, 0xff, 0x8e, 0x7a, 0x44, 0xaa, 0x79, 0x48, 0x5b, 0x26, 0xce, + 0x9e, 0xf2, 0x98, 0xc5, 0xed, 0x24, 0xce, 0xdb, 0xba, 0x34, 0x84, 0xe7, 0x01, 0x5c, 0xd0, 0x3b, + 0xc4, 0x9e, 0xeb, 0x4e, 0xfb, 0x84, 0xc3, 0x36, 0x07, 0xbb, 0x1f, 0x96, 0x17, 0xfc, 0xb4, 0x8a, + 0x9b, 0x24, 0xfe, 0x7d, 0xba, 0x00, 0xee, 0xd7, 0x24, 0x4e, 0x55, 0xd7, 0x21, 0x67, 0x31, 0xd1, + 0xc4, 0xad, 0x39, 0xa7, 0xb6, 0xe7, 0x3c, 0x56, 0xa2, 0xf0, 0xdf, 0x4a, 0x40, 0xe1, 0x45, 0x0f, + 0x5b, 0x27, 0x5f, 0x27, 0x75, 0xdf, 0x86, 0xc2, 0x91, 0xa2, 0x3b, 0xf2, 0x9e, 0x69, 0xc9, 0xbd, + 0xae, 0xa6, 0x38, 0x6e, 0x40, 0x4a, 0x8e, 0x90, 0x9f, 0x98, 0xd6, 0x0e, 0x25, 0x22, 0x0c, 0xe8, + 0xc0, 0x30, 0x8f, 0x0c, 0x99, 0x90, 0x29, 0x50, 0x3e, 0x36, 0xf8, 0xaa, 0xf4, 0xea, 0x87, 0xff, + 0xf9, 0x74, 0xf1, 0xd1, 0x44, 0x21, 0x61, 0x34, 0xfc, 0xad, 0xd7, 0xd3, 0xb5, 0xa5, 0x9d, 0x9d, + 0xfa, 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x41, 0x02, 0x8a, + 0xbe, 0x8e, 0xe2, 0x6c, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x33, 0x02, 0x67, + 0x24, 0x66, 0xe7, 0x73, 0x98, 0x0b, 0x69, 0x20, 0xf9, 0xd5, 0x34, 0x90, 0x3d, 0xf2, 0x2b, 0x8f, + 0xee, 0xc3, 0xbc, 0x73, 0x6c, 0xc8, 0x2c, 0x18, 0x90, 0x05, 0xa5, 0xb8, 0xf1, 0x12, 0x05, 0x87, + 0xe8, 0x83, 0xd0, 0x69, 0x40, 0x8a, 0x2d, 0xfe, 0x9e, 0x00, 0x88, 0x2a, 0xaa, 0xce, 0x3e, 0x1b, + 0x7c, 0x5d, 0xfa, 0xd3, 0x5d, 0x28, 0xd2, 0xf0, 0x4a, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, + 0x46, 0x8b, 0x77, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0x6b, 0xb0, 0x10, 0xaa, + 0x40, 0x9c, 0x8d, 0x7d, 0x1d, 0xe6, 0xf6, 0xd8, 0x57, 0x5a, 0x2a, 0x9c, 0xaf, 0x38, 0x66, 0x29, + 0x8d, 0xbd, 0x4f, 0xfc, 0xb3, 0x04, 0x5c, 0x90, 0xb0, 0x6d, 0xb6, 0x0f, 0x71, 0xfc, 0x2a, 0xac, + 0x01, 0xff, 0x9c, 0x23, 0xbf, 0x91, 0x26, 0x33, 0x8c, 0x99, 0x4d, 0x73, 0xe1, 0x65, 0xfb, 0x9b, + 0xa3, 0x7b, 0xec, 0xe0, 0x42, 0x3d, 0x5f, 0xf6, 0x4b, 0x85, 0x96, 0xfd, 0x4c, 0x28, 0xe8, 0x2d, + 0xc3, 0x24, 0x36, 0xcd, 0xc6, 0xaf, 0x8d, 0x5e, 0xc7, 0x05, 0x43, 0x4b, 0xa3, 0x0a, 0x59, 0x67, + 0x2c, 0x0d, 0xfc, 0x7a, 0xab, 0xd7, 0xa1, 0xbe, 0xf3, 0xea, 0x25, 0x52, 0xde, 0xb3, 0xd3, 0xc5, + 0x7c, 0x28, 0xcd, 0x96, 0xf2, 0xba, 0xf7, 0x4c, 0xa4, 0x8b, 0xdf, 0x86, 0x8b, 0x7d, 0xca, 0x8e, + 0xd3, 0xe3, 0xf9, 0xd7, 0x49, 0xb8, 0x12, 0x16, 0x1f, 0x37, 0xc4, 0xf9, 0xba, 0x37, 0x68, 0x0d, + 0x72, 0x1d, 0xdd, 0x78, 0xb3, 0xd5, 0xcb, 0xb9, 0x8e, 0x6e, 0xf8, 0x2b, 0xc5, 0x11, 0x5d, 0x63, + 0xe6, 0xa7, 0xda, 0x35, 0x14, 0x28, 0x47, 0xb5, 0x5d, 0x9c, 0xfd, 0xe3, 0x7b, 0x02, 0xcc, 0xc5, + 0xbd, 0x2c, 0xf7, 0x66, 0x81, 0x72, 0x62, 0x13, 0x72, 0x3f, 0x85, 0x75, 0xbc, 0xdf, 0x12, 0x00, + 0x35, 0xad, 0x9e, 0x41, 0x40, 0xed, 0x33, 0xb3, 0x15, 0x67, 0x35, 0x2f, 0xc0, 0xb4, 0x6e, 0x68, + 0xf8, 0x98, 0x56, 0x33, 0x25, 0xb1, 0x87, 0xd0, 0xd7, 0xc9, 0xe4, 0x44, 0x5f, 0x27, 0xc5, 0xcf, + 0x61, 0x21, 0x54, 0xc4, 0x38, 0xeb, 0xff, 0xdf, 0x13, 0xb0, 0xc0, 0x2b, 0x12, 0xfb, 0x0a, 0xe6, + 0x37, 0x61, 0xba, 0x4d, 0x64, 0x8e, 0x68, 0x67, 0xfa, 0x4e, 0xb7, 0x9d, 0x69, 0x66, 0xf4, 0xb3, + 0x00, 0x5d, 0x0b, 0x1f, 0xca, 0x8c, 0x35, 0x39, 0x11, 0x6b, 0x86, 0x70, 0x50, 0x02, 0xfa, 0x81, + 0x00, 0x05, 0x32, 0xa0, 0xbb, 0x96, 0xd9, 0x35, 0x6d, 0xe2, 0xb3, 0xd8, 0x93, 0xc1, 0x9c, 0x17, + 0x67, 0xa7, 0x8b, 0xb9, 0x4d, 0xdd, 0xd8, 0xe6, 0x8c, 0xcd, 0xc6, 0xc4, 0xf1, 0xfa, 0xee, 0xae, + 0x85, 0xa5, 0x4a, 0xdb, 0x54, 0x0f, 0xfc, 0xef, 0x6d, 0xc4, 0xb2, 0x78, 0xe2, 0x6c, 0xf1, 0x0f, + 0x05, 0xb8, 0xf0, 0x53, 0x5b, 0x2e, 0xfe, 0xff, 0xa1, 0x6c, 0xf1, 0x25, 0x14, 0xe9, 0x8f, 0xba, 0xb1, 0x67, 0xc6, 0xb9, 0x70, 0xff, 0xab, 0x02, 0xcc, 0x07, 0x04, 0xc7, 0xe9, 0xe0, 0xbc, 0x91, - 0x9e, 0xc4, 0x9f, 0x27, 0x2e, 0x4f, 0x70, 0x94, 0xc4, 0x39, 0x06, 0xff, 0x69, 0x02, 0x2e, 0x55, + 0x9e, 0xc4, 0x9f, 0x27, 0x2e, 0x4f, 0x70, 0x94, 0xc4, 0x39, 0x06, 0xff, 0x45, 0x02, 0x2e, 0x55, 0xd8, 0x67, 0x6e, 0x37, 0xee, 0x23, 0xce, 0x9e, 0x51, 0x82, 0xd9, 0x43, 0x6c, 0xd9, 0xba, 0xc9, 0xa6, 0xdf, 0x9c, 0xe4, 0x3e, 0xa2, 0x32, 0xa4, 0x6d, 0x43, 0xe9, 0xda, 0xfb, 0xa6, 0xfb, 0x7d, 0xcf, 0x7b, 0xf6, 0x62, 0x54, 0xa6, 0xdf, 0x3c, 0x46, 0x65, 0x66, 0x74, 0x8c, 0xca, 0xec, 0x57, - 0x88, 0x51, 0xe1, 0x1f, 0xd3, 0xfe, 0x8d, 0x00, 0x97, 0x07, 0x34, 0x17, 0x67, 0x6f, 0xf9, 0x2e, + 0x88, 0x51, 0xe1, 0x1f, 0xd3, 0xfe, 0xbd, 0x00, 0x97, 0x07, 0x34, 0x17, 0x67, 0x6f, 0xf9, 0x2e, 0x64, 0x55, 0x2e, 0x98, 0x18, 0x6c, 0xf6, 0xa5, 0xb0, 0x4e, 0xb2, 0xbd, 0x21, 0x6e, 0x39, 0x3b, 0x5d, 0x04, 0xb7, 0xa8, 0xf5, 0x35, 0xae, 0x1c, 0xf2, 0x5b, 0x13, 0x7f, 0x29, 0x07, 0x85, 0xea, 0x31, 0x5b, 0x25, 0x6f, 0x30, 0x37, 0x01, 0x3d, 0x81, 0x74, 0xd7, 0x32, 0x0f, 0x75, 0xb7, 0x1a, @@ -39382,7 +39265,7 @@ var fileDescriptor_api_e9d594aefb977c8a = []byte{ 0x67, 0xa6, 0xaa, 0xb4, 0x9f, 0xe8, 0x6d, 0xb7, 0xe7, 0xbf, 0x3f, 0x5e, 0xd0, 0x92, 0xc7, 0xb3, 0xad, 0x38, 0xfb, 0x6e, 0x23, 0x78, 0x44, 0x54, 0x87, 0x74, 0xcd, 0x71, 0xba, 0x24, 0x91, 0xdb, 0x8e, 0x3b, 0x13, 0x08, 0x25, 0x2c, 0x6e, 0x94, 0xac, 0xcb, 0x8e, 0x9a, 0x30, 0xff, 0x94, 0xee, - 0xf9, 0xaa, 0xb4, 0xcd, 0x9e, 0x56, 0x31, 0x8d, 0x3d, 0xbd, 0xc5, 0xed, 0xf6, 0xed, 0x09, 0x64, + 0xcf, 0xaa, 0xb4, 0xcd, 0x9e, 0x56, 0x31, 0x8d, 0x3d, 0xbd, 0xc5, 0xed, 0xf6, 0xed, 0x09, 0x64, 0x3e, 0xad, 0x34, 0xa4, 0x41, 0x01, 0x68, 0x05, 0xd2, 0x8d, 0x47, 0x5c, 0x18, 0xf3, 0xeb, 0x6e, 0x4d, 0x20, 0xac, 0xf1, 0x48, 0xf2, 0xd8, 0xd0, 0x3a, 0x64, 0x57, 0xbe, 0xe8, 0x59, 0x98, 0x4b, 0x99, 0x19, 0x1a, 0x1d, 0xd1, 0x2f, 0x85, 0x72, 0x49, 0x41, 0x66, 0xd4, 0x80, 0xfc, 0x2b, 0xd3, @@ -39392,13 +39275,13 @@ var fileDescriptor_api_e9d594aefb977c8a = []byte{ 0xa4, 0xba, 0xa4, 0xbd, 0x05, 0x1a, 0xbf, 0x44, 0x7f, 0xa3, 0xf7, 0x60, 0xd6, 0x30, 0x35, 0xec, 0x8e, 0x90, 0xdc, 0xea, 0x85, 0xb3, 0xd3, 0xc5, 0x99, 0x2d, 0x53, 0x63, 0x1e, 0x0d, 0xff, 0x25, 0xcd, 0x90, 0x4c, 0x75, 0xad, 0x7c, 0x0d, 0x52, 0xa4, 0xdd, 0x89, 0x61, 0xda, 0x55, 0x6c, 0xbc, - 0x63, 0xe9, 0x5c, 0x9a, 0xfb, 0x58, 0xfe, 0x07, 0x09, 0x48, 0x34, 0x1e, 0x11, 0x9f, 0x7d, 0xb7, + 0x63, 0xe9, 0x5c, 0x9a, 0xfb, 0x58, 0xfe, 0xc7, 0x09, 0x48, 0x34, 0x1e, 0x11, 0x9f, 0x7d, 0xb7, 0xa7, 0x1e, 0x60, 0x87, 0xa7, 0xf3, 0x27, 0xea, 0xcb, 0x5b, 0x78, 0x4f, 0x67, 0xae, 0x55, 0x46, - 0xe2, 0x4f, 0xe8, 0x1d, 0x00, 0x45, 0x55, 0xb1, 0x6d, 0xcb, 0xee, 0x5e, 0xc0, 0x8c, 0x94, 0x61, + 0xe2, 0x4f, 0xe8, 0x1d, 0x00, 0x45, 0x55, 0xb1, 0x6d, 0xcb, 0xee, 0xbe, 0xbd, 0x8c, 0x94, 0x61, 0x94, 0x0d, 0x7c, 0x42, 0xd8, 0x6c, 0xac, 0x5a, 0xd8, 0x71, 0x83, 0xaf, 0xd8, 0x13, 0x61, 0x73, 0x70, 0xa7, 0x2b, 0x3b, 0xe6, 0x01, 0x36, 0x68, 0x3f, 0xc9, 0x10, 0x53, 0xd3, 0xe9, 0x36, 0x09, 0x81, 0x58, 0x49, 0x6c, 0x68, 0xbe, 0x49, 0xcb, 0x48, 0xde, 0x33, 0x11, 0x69, 0xe1, 0x96, 0xce, - 0x37, 0xb5, 0x65, 0x24, 0xfe, 0x44, 0xb4, 0xa4, 0xf4, 0x9c, 0x7d, 0xda, 0x12, 0x19, 0x89, 0xfe, + 0xb7, 0xb6, 0x65, 0x24, 0xfe, 0x44, 0xb4, 0xa4, 0xf4, 0x9c, 0x7d, 0xda, 0x12, 0x19, 0x89, 0xfe, 0x46, 0xb7, 0xa1, 0xc0, 0xe2, 0x35, 0x65, 0x6c, 0xa8, 0x32, 0x35, 0xae, 0x19, 0x9a, 0x9c, 0x63, 0xe4, 0xaa, 0xa1, 0x12, 0x53, 0x8a, 0x1e, 0x01, 0x27, 0xc8, 0x07, 0x1d, 0x9b, 0xe8, 0x14, 0x48, 0xae, 0xd5, 0xc2, 0xd9, 0xe9, 0x62, 0xb6, 0x41, 0x13, 0x36, 0x36, 0x1b, 0xf5, 0x35, 0x29, 0xcb, @@ -39408,7 +39291,7 @@ var fileDescriptor_api_e9d594aefb977c8a = []byte{ 0x86, 0xa3, 0x2b, 0x6d, 0x9b, 0x6b, 0x2e, 0x48, 0x2a, 0xff, 0xa2, 0x00, 0xd3, 0x74, 0x04, 0xa0, 0xb7, 0x21, 0xa3, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0x37, 0x65, 0x19, 0xc9, 0x27, 0x0c, 0x2d, 0xde, 0x75, 0x98, 0x53, 0x54, 0xd5, 0xec, 0x19, 0x8e, 0x6c, 0x28, 0x1d, 0xcc, 0x8b, 0x99, 0xe5, 0xb4, - 0x2d, 0xa5, 0x83, 0xd1, 0x22, 0xb8, 0x8f, 0xde, 0x16, 0xcf, 0x8c, 0x04, 0x9c, 0xb4, 0x81, 0x4f, + 0x2d, 0xa5, 0x83, 0xd1, 0x22, 0xb8, 0x8f, 0xde, 0x76, 0xcc, 0x8c, 0x04, 0x9c, 0xb4, 0x81, 0x4f, 0xca, 0xbf, 0x2f, 0x40, 0xda, 0x1d, 0x33, 0xa4, 0x18, 0x2d, 0x6c, 0xb0, 0xa0, 0x72, 0xb7, 0x18, 0x1e, 0xa1, 0x7f, 0xaa, 0xcc, 0xf8, 0x53, 0xe5, 0x05, 0x98, 0x76, 0xc8, 0xb0, 0xe0, 0x25, 0x60, 0x0f, 0x74, 0x3d, 0xbb, 0xad, 0xb4, 0xd8, 0x72, 0x5e, 0x46, 0x62, 0x0f, 0xa4, 0x32, 0x3c, 0xec, @@ -39418,8 +39301,8 @@ var fileDescriptor_api_e9d594aefb977c8a = []byte{ 0xe9, 0x6f, 0xf4, 0x3e, 0x5c, 0x78, 0xdd, 0x53, 0xda, 0xfa, 0x1e, 0x5d, 0xa9, 0xa3, 0x51, 0xf7, 0x54, 0x67, 0xac, 0x26, 0xc8, 0x4b, 0xa3, 0x12, 0xa8, 0xea, 0xdc, 0xb1, 0x9c, 0xf4, 0xc7, 0xb2, 0xf8, 0x3b, 0x02, 0xcc, 0xb3, 0xc8, 0x23, 0x16, 0x30, 0x1b, 0x9f, 0x1f, 0xf2, 0x31, 0x64, 0x34, - 0xc5, 0x51, 0xd8, 0xa6, 0xd5, 0xc4, 0xc8, 0x4d, 0xab, 0xde, 0x26, 0x0a, 0xc5, 0x51, 0xe8, 0xc6, - 0x55, 0x04, 0x29, 0xf2, 0x9b, 0xed, 0xef, 0x95, 0xe8, 0x6f, 0xf1, 0x33, 0x40, 0xc1, 0x82, 0xc6, + 0xc5, 0x51, 0xd8, 0x06, 0xd3, 0xc4, 0xc8, 0x0d, 0xa6, 0xde, 0x26, 0x0a, 0xc5, 0x51, 0xe8, 0x26, + 0x53, 0x04, 0x29, 0xf2, 0x9b, 0xed, 0xc5, 0x95, 0xe8, 0x6f, 0xf1, 0x33, 0x40, 0xc1, 0x82, 0xc6, 0xe9, 0x91, 0xdd, 0x83, 0x8b, 0x44, 0xd7, 0x55, 0x43, 0xb5, 0x4e, 0xba, 0x8e, 0x6e, 0x1a, 0xcf, 0xe9, 0x5f, 0x1b, 0x15, 0x03, 0x1f, 0xb6, 0xe8, 0xf7, 0x2c, 0xf1, 0x77, 0x67, 0x20, 0x57, 0x3d, 0xee, 0x9a, 0x56, 0xac, 0xcb, 0x60, 0xab, 0x30, 0xcb, 0x57, 0x0a, 0x46, 0x7c, 0xbb, 0xee, 0x33, @@ -39429,7 +39312,7 @@ var fileDescriptor_api_e9d594aefb977c8a = []byte{ 0xb3, 0x04, 0x44, 0x02, 0xfb, 0x8d, 0xde, 0x05, 0xbe, 0x99, 0x48, 0xb6, 0xdd, 0x7d, 0x83, 0xab, 0xb9, 0xb3, 0xd3, 0xc5, 0x8c, 0x44, 0xa9, 0x8d, 0x46, 0x53, 0xca, 0xb0, 0x0c, 0x0d, 0xdb, 0x41, 0x37, 0x20, 0x67, 0x76, 0x74, 0x47, 0x76, 0x9d, 0x24, 0xee, 0x51, 0xce, 0x11, 0xa2, 0xeb, 0x44, - 0x9d, 0x67, 0x8f, 0xc9, 0xec, 0xc4, 0x7b, 0x4c, 0xd0, 0x5f, 0x11, 0xe0, 0x12, 0x57, 0xa4, 0xbc, + 0x9d, 0x67, 0x8f, 0xc9, 0xec, 0xc4, 0x7b, 0x4c, 0xd0, 0xdf, 0x14, 0xe0, 0x12, 0x57, 0xa4, 0xbc, 0x4b, 0xe3, 0xf0, 0x95, 0xb6, 0xee, 0x9c, 0xc8, 0x07, 0x87, 0xa5, 0x34, 0xf5, 0x5b, 0x7f, 0x26, 0xb2, 0x41, 0x02, 0xfd, 0x60, 0xc9, 0x6d, 0x96, 0x93, 0x67, 0x9c, 0x79, 0xe3, 0xb0, 0x6a, 0x38, 0xd6, 0xc9, 0xea, 0xe5, 0xb3, 0xd3, 0xc5, 0x85, 0xc1, 0xd4, 0x97, 0xd2, 0x82, 0x3d, 0xc8, 0x82, @@ -39437,254 +39320,254 @@ var fileDescriptor_api_e9d594aefb977c8a = []byte{ 0x77, 0xa1, 0xc8, 0xf7, 0xf4, 0xec, 0xe9, 0x6d, 0x2c, 0xdb, 0xfa, 0x17, 0x98, 0xce, 0x2d, 0x49, 0x29, 0xcf, 0xe8, 0x44, 0x44, 0x43, 0xff, 0x02, 0x97, 0xbf, 0x0b, 0xa5, 0x61, 0xa5, 0x0f, 0x0e, 0x81, 0x0c, 0xfb, 0xa4, 0xfb, 0x51, 0x78, 0x3d, 0x67, 0x82, 0xae, 0xca, 0xd7, 0x74, 0x3e, 0x4e, - 0x7c, 0x24, 0x88, 0x7f, 0x2f, 0x01, 0xb9, 0xd5, 0x5e, 0xfb, 0xe0, 0x79, 0xb7, 0xd1, 0xeb, 0x74, + 0x7c, 0x24, 0x88, 0xff, 0x30, 0x01, 0xb9, 0xd5, 0x5e, 0xfb, 0xe0, 0x79, 0xb7, 0xd1, 0xeb, 0x74, 0x14, 0xeb, 0x84, 0x98, 0x41, 0x66, 0x28, 0x48, 0x01, 0x05, 0x66, 0x06, 0xa9, 0x25, 0xd0, 0xbf, - 0xc0, 0x64, 0x72, 0x0a, 0xee, 0x53, 0x67, 0xfb, 0x0c, 0x68, 0x1d, 0x02, 0x9b, 0xcf, 0xcd, 0x23, - 0x1b, 0x7d, 0x04, 0xa5, 0x40, 0x46, 0xba, 0xf8, 0x22, 0x63, 0xc3, 0xb1, 0x74, 0xcc, 0x16, 0x10, - 0x93, 0x52, 0x20, 0xa6, 0xa7, 0x4e, 0x92, 0xab, 0x2c, 0x15, 0x35, 0x61, 0x8e, 0x64, 0x3c, 0x91, - 0xe9, 0x14, 0xe2, 0x2e, 0xf0, 0x3e, 0x88, 0xa8, 0x56, 0xa8, 0xdc, 0x4b, 0x54, 0x3f, 0x15, 0xca, - 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0xca, 0x9f, 0x42, 0xb1, 0x3f, 0x43, 0x50, 0x97, 0x29, 0xa6, - 0xcb, 0x0b, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4f, 0xa5, 0x53, 0xc5, 0x69, 0xf1, 0x4f, 0x92, - 0x90, 0x77, 0xbb, 0x59, 0x9c, 0x40, 0x67, 0x15, 0xa6, 0x49, 0xa7, 0x70, 0x23, 0x50, 0x6e, 0x8f, - 0xe8, 0xdd, 0x3c, 0xb2, 0x9d, 0x74, 0x16, 0x17, 0x24, 0x53, 0xd6, 0x38, 0x0c, 0x4e, 0xf9, 0x17, - 0x13, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x44, 0x21, 0x4c, 0x32, 0x51, 0xd0, 0xac, 0xde, - 0x74, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x57, 0x3e, 0x78, 0xf0, 0x90, 0x1a, 0x9b, 0x39, - 0x89, 0x3f, 0xa1, 0x55, 0x1a, 0x1a, 0x65, 0x5a, 0x0e, 0xd6, 0xb8, 0x4f, 0x7f, 0x6d, 0x5c, 0xfb, - 0xba, 0x93, 0x92, 0xcb, 0x87, 0xae, 0x40, 0x92, 0x58, 0xb1, 0x59, 0x16, 0xe5, 0x70, 0x76, 0xba, - 0x98, 0x24, 0xf6, 0x8b, 0xd0, 0xd0, 0x32, 0x64, 0xc3, 0x26, 0x83, 0x78, 0x70, 0xd4, 0x30, 0x06, - 0x86, 0x3b, 0xb4, 0xbd, 0xa1, 0xc5, 0xf0, 0x2c, 0x6f, 0xe3, 0x3f, 0x4d, 0x41, 0xae, 0xde, 0x89, - 0x7b, 0x4a, 0x59, 0x09, 0xb7, 0x70, 0x14, 0x10, 0x0a, 0xbd, 0x34, 0xa2, 0x81, 0x43, 0x33, 0x78, - 0xf2, 0x7c, 0x33, 0x78, 0x9d, 0x78, 0xca, 0xfc, 0xe0, 0x89, 0xe4, 0x10, 0xcc, 0x13, 0x7e, 0x3f, - 0xf5, 0x53, 0x24, 0xc2, 0xe3, 0xef, 0xf5, 0xa0, 0x91, 0x2a, 0x9f, 0x52, 0x87, 0x9c, 0xf5, 0xb2, - 0x99, 0xc9, 0x7b, 0xd9, 0x2c, 0x36, 0x34, 0x3a, 0xa9, 0x85, 0x2d, 0xea, 0xec, 0x9b, 0x5b, 0xd4, - 0xb2, 0xc3, 0x3b, 0xeb, 0xc7, 0x90, 0xd4, 0x74, 0xb7, 0x71, 0x26, 0x9f, 0xaa, 0x09, 0xd3, 0x98, - 0x5e, 0x9b, 0x0a, 0xf6, 0x5a, 0xd6, 0x4b, 0xca, 0x75, 0x00, 0x5f, 0x37, 0xe8, 0x1a, 0xcc, 0x98, - 0x6d, 0xcd, 0xdd, 0xec, 0x92, 0x5b, 0xcd, 0x9c, 0x9d, 0x2e, 0x4e, 0x3f, 0x6f, 0x6b, 0xf5, 0x35, - 0x69, 0xda, 0x6c, 0x6b, 0x75, 0x8d, 0x9e, 0xfa, 0x81, 0x8f, 0x64, 0x2f, 0x12, 0x6e, 0x4e, 0x9a, - 0x35, 0xf0, 0xd1, 0x1a, 0xb6, 0x55, 0xde, 0xe1, 0x7e, 0x53, 0x80, 0xbc, 0xab, 0xfb, 0x78, 0x8d, - 0x4a, 0x5a, 0xef, 0xf0, 0x41, 0x96, 0x3c, 0xdf, 0x20, 0x73, 0xf9, 0xf8, 0xc6, 0xe2, 0x5f, 0x16, - 0x78, 0x6c, 0x73, 0x43, 0x55, 0x1c, 0xe2, 0x54, 0xc4, 0x38, 0x30, 0xee, 0x41, 0xd1, 0x52, 0x0c, - 0xcd, 0xec, 0xe8, 0x5f, 0x60, 0xb6, 0x10, 0x6a, 0xf3, 0xcf, 0x9e, 0x05, 0x8f, 0x4e, 0x57, 0xfd, - 0x6c, 0xf1, 0xf7, 0x13, 0x3c, 0x0e, 0xda, 0x2b, 0x46, 0x9c, 0xea, 0xfa, 0x0e, 0xcc, 0xf7, 0x1f, - 0xc1, 0xe2, 0x8e, 0xd6, 0xf7, 0x22, 0xe4, 0x45, 0x15, 0x84, 0xc5, 0x33, 0xba, 0xc1, 0xf5, 0x7d, - 0xc7, 0xb1, 0xd8, 0xa8, 0x02, 0xd9, 0xe0, 0xc9, 0x2e, 0xc9, 0x89, 0x4f, 0x76, 0x01, 0xcb, 0x3b, - 0xcf, 0xa5, 0xfc, 0x73, 0x30, 0x4d, 0x93, 0xdf, 0xc0, 0x44, 0xf3, 0xd6, 0xfc, 0xe3, 0x04, 0xdc, - 0xa4, 0xa5, 0x7f, 0x89, 0x2d, 0x7d, 0xef, 0x64, 0xdb, 0x32, 0x1d, 0xac, 0x3a, 0x58, 0xf3, 0x57, - 0xce, 0x63, 0xb5, 0x7b, 0x99, 0xae, 0xfb, 0x82, 0x73, 0x05, 0x9c, 0x79, 0x5c, 0x68, 0x03, 0x0a, - 0x3c, 0xb4, 0x40, 0x69, 0xeb, 0x87, 0x58, 0x56, 0x9c, 0xf3, 0xcc, 0x6e, 0x39, 0xc6, 0xbb, 0x42, - 0x58, 0x57, 0x1c, 0xa4, 0x41, 0x86, 0x0b, 0xd3, 0x35, 0x7e, 0x20, 0xd1, 0xd3, 0xaf, 0xb6, 0xa0, - 0x98, 0x66, 0xf1, 0x0d, 0xf5, 0x35, 0x29, 0xcd, 0x24, 0xd7, 0x35, 0xf1, 0x3f, 0x08, 0x70, 0x6b, - 0x8c, 0x8a, 0xe3, 0xec, 0xba, 0x65, 0x48, 0x1f, 0x92, 0x17, 0xe9, 0x5c, 0xc7, 0x69, 0xc9, 0x7b, - 0x46, 0x9b, 0x90, 0xdb, 0x53, 0xf4, 0xb6, 0xdf, 0xa5, 0x87, 0xc7, 0x27, 0x46, 0x47, 0xe3, 0xce, - 0x31, 0x76, 0xd6, 0x87, 0xc5, 0x5f, 0x4f, 0xc0, 0xfc, 0x8a, 0xa6, 0x35, 0x1a, 0xdc, 0x06, 0xc6, - 0xd7, 0x53, 0x5c, 0x90, 0x99, 0xf0, 0x41, 0x26, 0x7a, 0x0f, 0x90, 0xa6, 0xdb, 0xec, 0xe0, 0x13, - 0x7b, 0x5f, 0xd1, 0xcc, 0x23, 0x3f, 0x2e, 0x63, 0xde, 0x4d, 0x69, 0xb8, 0x09, 0xa8, 0x01, 0x14, - 0xed, 0xc8, 0xb6, 0xa3, 0x78, 0xdf, 0x9d, 0x6e, 0x4d, 0xb4, 0xa5, 0x8c, 0xc1, 0x20, 0xef, 0x51, - 0xca, 0x10, 0x39, 0xf4, 0x27, 0xf1, 0xdb, 0x75, 0x52, 0x75, 0x47, 0x56, 0x6c, 0x77, 0xff, 0x10, - 0x3b, 0x72, 0x25, 0xcf, 0xe8, 0x2b, 0x36, 0xdb, 0x16, 0xc4, 0x36, 0x3c, 0xf8, 0xaa, 0x89, 0x13, - 0x12, 0xff, 0x6d, 0x01, 0xf2, 0x12, 0xde, 0xb3, 0xb0, 0x1d, 0xeb, 0xa2, 0xc0, 0x13, 0x98, 0xb3, - 0x98, 0x54, 0x79, 0xcf, 0x32, 0x3b, 0xe7, 0x19, 0x57, 0x59, 0xce, 0xf8, 0xc4, 0x32, 0x3b, 0xdc, - 0xb0, 0xbc, 0x84, 0x82, 0x57, 0xc6, 0x38, 0x2b, 0xff, 0x3b, 0x74, 0xbb, 0x34, 0x13, 0x1c, 0x77, - 0x80, 0x44, 0xbc, 0x1a, 0xa0, 0x1f, 0xaa, 0x82, 0x05, 0x8d, 0x53, 0x0d, 0xff, 0x45, 0x80, 0x7c, - 0xa3, 0xb7, 0xcb, 0x4e, 0xd6, 0x8a, 0x4f, 0x03, 0x55, 0xc8, 0xb4, 0xf1, 0x9e, 0x23, 0xbf, 0x51, - 0xa8, 0x7e, 0x9a, 0xb0, 0xd2, 0x8d, 0x0a, 0x4f, 0x01, 0x2c, 0xba, 0xc5, 0x8e, 0xca, 0x49, 0x9e, - 0x53, 0x4e, 0x86, 0xf2, 0x12, 0xb2, 0xf8, 0xa7, 0x09, 0x28, 0x78, 0xd5, 0x8c, 0xd3, 0x4a, 0xbe, - 0x0a, 0x59, 0x87, 0xe4, 0x79, 0xac, 0xc3, 0x3c, 0x8f, 0x09, 0x89, 0xb6, 0x10, 0x4b, 0xb0, 0x40, - 0x1d, 0x17, 0x59, 0xe9, 0x76, 0xdb, 0xba, 0x0b, 0x77, 0xa9, 0xfd, 0x49, 0x49, 0xf3, 0x34, 0x69, - 0x85, 0xa5, 0x50, 0xa0, 0x8b, 0x7e, 0x45, 0x80, 0xb9, 0x3d, 0x0b, 0xe3, 0x2f, 0xb0, 0x4c, 0xa1, - 0xd7, 0x64, 0x41, 0x2f, 0x6b, 0xa4, 0x0c, 0x5f, 0xf9, 0xa3, 0x78, 0x96, 0xbd, 0xb8, 0x41, 0xde, - 0xcb, 0x3b, 0xed, 0x2b, 0x98, 0xa7, 0x8d, 0x12, 0xf7, 0xde, 0x62, 0xf1, 0x6f, 0x26, 0x00, 0x05, - 0x25, 0xff, 0xf4, 0x1a, 0x33, 0x11, 0x5f, 0x63, 0xbe, 0x0b, 0x88, 0x85, 0x4d, 0xda, 0x72, 0x17, - 0x5b, 0xb2, 0x8d, 0x55, 0x93, 0x1f, 0x0f, 0x25, 0x48, 0x45, 0x9e, 0xb2, 0x8d, 0xad, 0x06, 0xa5, - 0xa3, 0xc7, 0x00, 0xbe, 0x4b, 0xc7, 0x67, 0x9c, 0x91, 0x1e, 0x9d, 0x94, 0xf1, 0x7c, 0x39, 0xf1, - 0xfb, 0x02, 0xe4, 0x37, 0xf5, 0x96, 0xa5, 0xc4, 0x7a, 0xee, 0x11, 0xfa, 0x38, 0xbc, 0x8c, 0x9e, - 0x7d, 0x58, 0x8e, 0x0a, 0xf1, 0x61, 0x39, 0x5c, 0x70, 0xc6, 0x19, 0x88, 0x09, 0xf7, 0x4a, 0x14, - 0xa7, 0xed, 0xfa, 0xb7, 0x65, 0x98, 0xe3, 0xe5, 0xde, 0x31, 0x74, 0xd3, 0x40, 0x0f, 0x20, 0xd9, - 0xe2, 0x9f, 0x49, 0xb2, 0x91, 0x4b, 0x9a, 0xfe, 0x11, 0x80, 0xb5, 0x29, 0x89, 0xe4, 0x25, 0x2c, - 0xdd, 0x9e, 0x13, 0xe1, 0x44, 0xfa, 0x51, 0xef, 0x41, 0x96, 0x6e, 0xcf, 0x41, 0x0d, 0x28, 0xa8, - 0xfe, 0x51, 0x66, 0x32, 0x61, 0x4f, 0x0e, 0x05, 0x9c, 0x91, 0x87, 0xca, 0xd5, 0xa6, 0xa4, 0xbc, - 0x1a, 0x4a, 0x40, 0x95, 0xe0, 0x09, 0x5a, 0xa9, 0x81, 0x90, 0x3a, 0x7f, 0xbf, 0x76, 0xf8, 0xf4, - 0xae, 0xda, 0x54, 0xe0, 0xa0, 0x2d, 0xf4, 0x31, 0xcc, 0x68, 0xf4, 0xac, 0x26, 0x3e, 0xf8, 0xa3, - 0x1a, 0x3a, 0x74, 0x24, 0x56, 0x6d, 0x4a, 0xe2, 0x1c, 0x68, 0x1d, 0xe6, 0xd8, 0x2f, 0xe6, 0xd2, - 0x71, 0x14, 0x7e, 0x6b, 0xb8, 0x84, 0xc0, 0xa4, 0x59, 0x9b, 0x92, 0xb2, 0x9a, 0x4f, 0x45, 0xdf, - 0x84, 0x94, 0xad, 0x2a, 0x2e, 0x0e, 0xbf, 0x3a, 0xe4, 0x88, 0x14, 0x9f, 0x99, 0xe6, 0x46, 0x8f, - 0xd9, 0xa1, 0x9e, 0xce, 0xb1, 0xbb, 0x24, 0x1a, 0x55, 0xfc, 0xd0, 0x56, 0x7c, 0x52, 0x7c, 0x4c, - 0x09, 0xe8, 0x29, 0x64, 0x15, 0xe2, 0x1b, 0xcb, 0x74, 0x33, 0x2b, 0x5d, 0x03, 0x8d, 0x0e, 0x44, - 0x18, 0xd8, 0x7c, 0x5c, 0xa3, 0xfb, 0xfd, 0x5d, 0xa2, 0x2f, 0xa8, 0x83, 0xad, 0x16, 0x2e, 0x65, - 0x47, 0x0b, 0x0a, 0x06, 0xd2, 0x79, 0x82, 0x28, 0x91, 0xf8, 0xc8, 0xde, 0x4e, 0x72, 0x5a, 0xa9, - 0xb9, 0xa1, 0x9f, 0xbe, 0x23, 0xf6, 0x48, 0xd5, 0xa6, 0xa4, 0xb9, 0xfd, 0x00, 0x19, 0x2d, 0x41, - 0xa2, 0xa5, 0x96, 0x72, 0x43, 0x8d, 0x81, 0xb7, 0x0f, 0xa8, 0x36, 0x25, 0x25, 0x5a, 0x2a, 0xfa, - 0x14, 0xd2, 0x6c, 0x53, 0xc7, 0xb1, 0x51, 0xca, 0x0f, 0x1d, 0x63, 0xe1, 0xad, 0x31, 0xb5, 0x29, - 0x89, 0xee, 0x23, 0x21, 0xef, 0xdb, 0x86, 0xbc, 0xc5, 0x22, 0x11, 0xdd, 0xb8, 0xe1, 0xe2, 0xd0, - 0x70, 0x80, 0xa8, 0xd0, 0xe1, 0x1a, 0x45, 0x49, 0x01, 0x3a, 0xfa, 0x0e, 0x5c, 0x08, 0x4b, 0xe4, - 0x3d, 0x6d, 0x7e, 0xe8, 0xa7, 0xed, 0xa1, 0x61, 0xac, 0xb5, 0x29, 0x09, 0x59, 0x03, 0x89, 0xe8, - 0x43, 0x98, 0x66, 0xad, 0x86, 0xa8, 0xc8, 0xa8, 0x08, 0x97, 0xbe, 0x06, 0x63, 0xf9, 0x49, 0xe7, - 0x77, 0x78, 0x38, 0x9e, 0xdc, 0x36, 0x5b, 0xa5, 0x85, 0xa1, 0x9d, 0x7f, 0x30, 0xb0, 0x90, 0x74, - 0x7e, 0xc7, 0xa7, 0x92, 0x76, 0xb7, 0x58, 0x0a, 0x0f, 0xc1, 0xba, 0x30, 0xb4, 0xdd, 0x23, 0xa2, - 0xf4, 0x6a, 0x74, 0xa3, 0x84, 0x4f, 0x26, 0x45, 0xb3, 0xd8, 0xa9, 0x3e, 0x32, 0x1d, 0x53, 0x17, - 0x87, 0x16, 0x6d, 0xf0, 0xd8, 0xa3, 0x1a, 0xf5, 0x27, 0x3d, 0x2a, 0x7a, 0x09, 0x45, 0x7e, 0xf6, - 0x86, 0xff, 0xfd, 0xe5, 0x12, 0x95, 0x77, 0x2f, 0xd2, 0x74, 0x45, 0xc5, 0x2f, 0xd5, 0xa6, 0xa4, - 0x82, 0x1a, 0x4e, 0x41, 0x9f, 0xc1, 0x3c, 0x95, 0x27, 0xab, 0xfe, 0x71, 0x29, 0xa5, 0xd2, 0xc0, - 0xe1, 0x1b, 0xc3, 0x4f, 0x56, 0x71, 0x25, 0x17, 0xd5, 0xbe, 0x24, 0xd2, 0x8d, 0x75, 0x43, 0x77, - 0xa8, 0x95, 0x2d, 0x0f, 0xed, 0xc6, 0xe1, 0xa3, 0x1f, 0x49, 0x37, 0xd6, 0x19, 0x85, 0x74, 0x63, - 0x87, 0xc7, 0xe7, 0xf1, 0xe6, 0x78, 0x7b, 0x68, 0x37, 0x8e, 0x0a, 0xe4, 0x23, 0xdd, 0xd8, 0x09, - 0xd2, 0x49, 0x37, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, 0xbb, 0xf1, 0xd0, 0x4d, 0xe5, 0xa4, - 0x1b, 0x2b, 0x03, 0x89, 0x68, 0x0d, 0x80, 0xb9, 0x7e, 0x74, 0xfe, 0xbf, 0x3a, 0x74, 0x32, 0xe8, - 0x8f, 0xd3, 0x23, 0x93, 0x41, 0xdb, 0xa5, 0x11, 0x43, 0x46, 0x81, 0xa5, 0x4c, 0x3f, 0x37, 0x97, - 0x16, 0x87, 0x1a, 0xb2, 0x81, 0x4f, 0xc3, 0xc4, 0x90, 0x1d, 0x79, 0x44, 0x32, 0xab, 0xb0, 0xf5, - 0xf1, 0xd2, 0xb5, 0xe1, 0x66, 0x39, 0xf8, 0x99, 0x8c, 0x9a, 0x65, 0x4a, 0x40, 0x2b, 0x90, 0x21, - 0xee, 0xcd, 0x09, 0x35, 0x43, 0xd7, 0x87, 0xfa, 0xef, 0x7d, 0x3b, 0x81, 0x6a, 0x53, 0x52, 0xfa, - 0x35, 0x27, 0x91, 0xd7, 0xb3, 0x95, 0xc3, 0x92, 0x38, 0xf4, 0xf5, 0xa1, 0x55, 0x66, 0xf2, 0x7a, - 0xc6, 0x81, 0x54, 0xb8, 0xc8, 0xda, 0x8a, 0xef, 0xef, 0xb6, 0xf8, 0x96, 0xe4, 0xd2, 0x0d, 0x2a, - 0x6a, 0xe8, 0x12, 0x5c, 0xe4, 0xb6, 0xf3, 0xda, 0x94, 0xb4, 0xa0, 0x0c, 0xa6, 0x92, 0x01, 0xcf, - 0xa7, 0x1e, 0xb6, 0x70, 0x57, 0xba, 0x39, 0x74, 0xc0, 0x47, 0xac, 0x77, 0x92, 0x01, 0xaf, 0x04, - 0xc8, 0x6c, 0x02, 0xd2, 0x64, 0xdb, 0x66, 0xc1, 0x09, 0xb7, 0x46, 0x4c, 0x40, 0x7d, 0x2b, 0x26, - 0x6c, 0x02, 0xd2, 0x1a, 0x8c, 0x93, 0x08, 0x52, 0xdb, 0x58, 0xb1, 0xb8, 0x99, 0xbd, 0x3d, 0x54, - 0xd0, 0xc0, 0x29, 0x8b, 0x44, 0x90, 0xea, 0x11, 0x89, 0xc3, 0x63, 0xb9, 0x87, 0xf6, 0x70, 0xdf, - 0xf8, 0xce, 0x50, 0x87, 0x27, 0xf2, 0x54, 0x21, 0xe2, 0xf0, 0x58, 0xa1, 0x04, 0xf4, 0xb3, 0x30, - 0xcb, 0xa1, 0x6e, 0xe9, 0xee, 0x08, 0x17, 0x30, 0xb8, 0x3a, 0x41, 0xc6, 0x35, 0xe7, 0x61, 0x56, - 0x96, 0x41, 0x6c, 0x56, 0xbd, 0x7b, 0x23, 0xac, 0xec, 0x00, 0xca, 0x67, 0x56, 0xd6, 0x27, 0x13, - 0x2b, 0xcb, 0xfa, 0x29, 0x9f, 0xeb, 0xee, 0x0f, 0xb5, 0xb2, 0x83, 0xfb, 0x8c, 0x88, 0x95, 0x7d, - 0xed, 0x53, 0x49, 0xcd, 0x6c, 0x06, 0x35, 0x4b, 0xdf, 0x18, 0x5a, 0xb3, 0x30, 0xe6, 0x26, 0x35, - 0xe3, 0x3c, 0xa4, 0xd9, 0x98, 0xf7, 0xcf, 0x34, 0xfd, 0xee, 0xf0, 0x43, 0x14, 0xfa, 0xf1, 0x55, - 0xcd, 0x5d, 0xd4, 0x65, 0x1a, 0xf6, 0x0c, 0x95, 0xc5, 0x77, 0x78, 0x73, 0x4d, 0xbd, 0x37, 0xda, - 0x50, 0x45, 0xed, 0x8c, 0xf7, 0x0c, 0x55, 0x28, 0x91, 0x16, 0x95, 0x6d, 0xea, 0xa3, 0xe3, 0x7b, - 0x69, 0xc4, 0x79, 0x0f, 0x7d, 0x5b, 0x2b, 0x69, 0x51, 0x3d, 0xa2, 0x3f, 0x84, 0x7a, 0xec, 0x30, - 0x92, 0xd2, 0xf2, 0xe8, 0x21, 0x14, 0x3e, 0x0e, 0xc5, 0x1b, 0x42, 0x9c, 0xec, 0xcd, 0x99, 0xae, - 0x87, 0xf1, 0xfe, 0xe8, 0x39, 0xb3, 0xdf, 0xb5, 0x60, 0x73, 0x26, 0xf7, 0x29, 0xfe, 0xb2, 0x00, - 0xd7, 0x58, 0xd9, 0xe8, 0xea, 0xe7, 0x89, 0xec, 0xad, 0x21, 0x07, 0x36, 0x98, 0x3c, 0xa0, 0x2f, - 0xf8, 0x70, 0x58, 0x71, 0xc7, 0xac, 0x89, 0xd7, 0xa6, 0xa4, 0x77, 0x94, 0x51, 0xf9, 0x48, 0x97, - 0xea, 0x30, 0x04, 0x55, 0x7a, 0x38, 0xb4, 0x4b, 0x85, 0x51, 0x1f, 0xe9, 0x52, 0x9c, 0x67, 0x75, - 0x96, 0x7f, 0x81, 0xf6, 0x36, 0xe4, 0x16, 0x8a, 0xc5, 0xf5, 0x54, 0xfa, 0x72, 0xb1, 0xb4, 0x9e, - 0x4a, 0x5f, 0x29, 0x96, 0xd7, 0x53, 0xe9, 0xb7, 0x8a, 0x6f, 0x8b, 0x7f, 0xbf, 0x0c, 0x39, 0x17, - 0x74, 0x31, 0x40, 0xf5, 0x30, 0x08, 0xa8, 0xae, 0x0e, 0x03, 0x54, 0x1c, 0xa6, 0x71, 0x44, 0xf5, - 0x30, 0x88, 0xa8, 0xae, 0x0e, 0x43, 0x54, 0x3e, 0x0f, 0x81, 0x54, 0xcd, 0x61, 0x90, 0xea, 0xde, - 0x04, 0x90, 0xca, 0x13, 0xd5, 0x8f, 0xa9, 0xd6, 0x06, 0x31, 0xd5, 0xcd, 0xd1, 0x98, 0xca, 0x13, - 0x15, 0x00, 0x55, 0x8f, 0xfb, 0x40, 0xd5, 0xf5, 0x11, 0xa0, 0xca, 0xe3, 0x77, 0x51, 0xd5, 0x46, - 0x24, 0xaa, 0xba, 0x3d, 0x0e, 0x55, 0x79, 0x72, 0x42, 0xb0, 0xea, 0x83, 0x10, 0xac, 0x5a, 0x1c, - 0x0a, 0xab, 0x3c, 0x6e, 0x86, 0xab, 0x3e, 0xe9, 0xc7, 0x55, 0xd7, 0x47, 0xe0, 0x2a, 0xbf, 0x06, - 0x1c, 0x58, 0xd5, 0xa2, 0x80, 0xd5, 0xad, 0x31, 0xc0, 0xca, 0x93, 0x12, 0x44, 0x56, 0xb5, 0x28, - 0x64, 0x75, 0x6b, 0x0c, 0xb2, 0xea, 0x93, 0xc4, 0xa0, 0xd5, 0x56, 0x34, 0xb4, 0xba, 0x33, 0x16, - 0x5a, 0x79, 0xd2, 0xc2, 0xd8, 0x6a, 0x39, 0x80, 0xad, 0xde, 0x19, 0x82, 0xad, 0x3c, 0x56, 0x02, - 0xae, 0xbe, 0x35, 0x00, 0xae, 0xc4, 0x51, 0xe0, 0xca, 0xe3, 0xf5, 0xd0, 0xd5, 0x8b, 0x21, 0xe8, - 0xea, 0xee, 0x78, 0x74, 0xe5, 0x09, 0xeb, 0x83, 0x57, 0xca, 0x48, 0x78, 0xf5, 0xde, 0x84, 0xf0, - 0xca, 0x93, 0x1e, 0x85, 0xaf, 0x3e, 0x0a, 0xe3, 0xab, 0x6b, 0xc3, 0xf1, 0x95, 0x27, 0x86, 0x03, - 0xac, 0x8d, 0x48, 0x80, 0x75, 0x7b, 0x1c, 0xc0, 0xf2, 0xc7, 0x41, 0x10, 0x61, 0x6d, 0x45, 0x23, - 0xac, 0x3b, 0x63, 0x11, 0x96, 0xdf, 0xfc, 0x21, 0x88, 0xb5, 0x11, 0x09, 0xb1, 0x6e, 0x8f, 0x83, - 0x58, 0x7e, 0xe1, 0x82, 0x18, 0xeb, 0xd5, 0x50, 0x8c, 0x75, 0x7f, 0x12, 0x8c, 0xe5, 0x09, 0x1d, - 0x00, 0x59, 0x9f, 0x0f, 0x07, 0x59, 0xdf, 0x38, 0xc7, 0xf1, 0x95, 0x91, 0x28, 0xeb, 0x5b, 0x03, - 0x28, 0x4b, 0x1c, 0x85, 0xb2, 0xfc, 0xfe, 0xec, 0xc2, 0x2c, 0x65, 0x24, 0x28, 0x7a, 0x6f, 0x42, - 0x50, 0xe4, 0x77, 0xbe, 0x08, 0x54, 0x54, 0x8d, 0x40, 0x45, 0x37, 0x47, 0xa3, 0x22, 0xdf, 0x9c, - 0xfb, 0xb0, 0xa8, 0x16, 0x05, 0x8b, 0x6e, 0x8d, 0x81, 0x45, 0xbe, 0x15, 0x0a, 0xe0, 0xa2, 0xc7, - 0x7d, 0xb8, 0xe8, 0xfa, 0xd8, 0x00, 0xab, 0x00, 0x30, 0x5a, 0x1d, 0x04, 0x46, 0x37, 0x46, 0x02, - 0x23, 0x4f, 0x82, 0x8f, 0x8c, 0x1e, 0xf7, 0x21, 0xa3, 0xeb, 0x23, 0x90, 0x91, 0x5f, 0x00, 0x0e, - 0x8d, 0xb4, 0xd1, 0xd0, 0x68, 0x69, 0x52, 0x68, 0xe4, 0x09, 0x8e, 0xc4, 0x46, 0x5b, 0xd1, 0xd8, - 0xe8, 0xce, 0x84, 0xb1, 0x0f, 0x03, 0xe0, 0xa8, 0x16, 0x05, 0x8e, 0x6e, 0x8d, 0x01, 0x47, 0xc1, - 0x39, 0xc4, 0x43, 0x47, 0xb5, 0x28, 0x74, 0x74, 0x6b, 0x0c, 0x3a, 0xf2, 0x25, 0x05, 0xe0, 0x51, - 0x73, 0x18, 0x3c, 0xba, 0x37, 0x01, 0x3c, 0xf2, 0x9d, 0x97, 0x3e, 0x7c, 0xf4, 0x69, 0x3f, 0x3e, - 0x12, 0x47, 0xe1, 0x23, 0x7f, 0x44, 0xba, 0x00, 0x69, 0x2b, 0x1a, 0x20, 0xdd, 0x19, 0x0b, 0x90, - 0x82, 0x46, 0x32, 0x80, 0x90, 0x36, 0x22, 0x11, 0xd2, 0xed, 0x71, 0x08, 0xc9, 0x37, 0x92, 0x41, - 0x88, 0xf4, 0x69, 0x3f, 0x44, 0x12, 0x47, 0x41, 0x24, 0xbf, 0x72, 0x2e, 0x46, 0xaa, 0x45, 0x61, - 0xa4, 0x5b, 0x63, 0x30, 0x92, 0xdf, 0x78, 0x01, 0x90, 0xa4, 0x8c, 0x04, 0x49, 0xef, 0x4d, 0x08, - 0x92, 0xfa, 0x0c, 0x57, 0x18, 0x25, 0xd5, 0xa2, 0x50, 0xd2, 0xad, 0x31, 0x28, 0x29, 0x50, 0x58, - 0x1f, 0x26, 0x6d, 0x45, 0xc3, 0xa4, 0x3b, 0x63, 0x61, 0x52, 0xdf, 0x68, 0x72, 0x71, 0xd2, 0x46, - 0x24, 0x4e, 0xba, 0x3d, 0x0e, 0x27, 0xf5, 0x4d, 0x7c, 0xdc, 0x39, 0xf8, 0xa5, 0xc9, 0x81, 0xd2, - 0x47, 0xe7, 0x07, 0x4a, 0xde, 0x3b, 0xc7, 0x20, 0xa5, 0x4f, 0xfb, 0x91, 0x92, 0x38, 0x0a, 0x29, - 0xf9, 0x3d, 0xeb, 0x7c, 0x50, 0x69, 0x3d, 0x95, 0x7e, 0xbb, 0xf8, 0x8e, 0xf8, 0x57, 0x67, 0x61, - 0xa6, 0xe6, 0x85, 0x24, 0xf9, 0xb5, 0x14, 0xde, 0xe4, 0x0c, 0x2c, 0xb4, 0x46, 0x46, 0x3c, 0xb5, - 0x9b, 0xe3, 0x4f, 0x4e, 0x1c, 0x3c, 0xdb, 0x8f, 0xb3, 0xbe, 0xc1, 0x66, 0x74, 0xf4, 0x01, 0xe4, - 0x7a, 0x36, 0xb6, 0xe4, 0xae, 0xa5, 0x9b, 0x96, 0xee, 0xb0, 0x0d, 0x37, 0xc2, 0x6a, 0xf1, 0xcb, - 0xd3, 0xc5, 0xb9, 0x1d, 0x1b, 0x5b, 0xdb, 0x9c, 0x2e, 0xcd, 0xf5, 0x02, 0x4f, 0xee, 0x2d, 0x67, - 0xd3, 0x93, 0xdf, 0x72, 0xf6, 0x02, 0x8a, 0x16, 0x56, 0xb4, 0x90, 0x07, 0xc3, 0xce, 0x9b, 0x8a, - 0xee, 0x73, 0x74, 0x43, 0x9c, 0x9b, 0x93, 0x9e, 0x3b, 0x55, 0xb0, 0xc2, 0x44, 0xf4, 0x00, 0x2e, - 0x76, 0x94, 0x63, 0x1a, 0xbe, 0x2a, 0xbb, 0x4e, 0x21, 0x0d, 0x49, 0x65, 0x17, 0x88, 0xa1, 0x8e, - 0x72, 0x4c, 0xaf, 0x4c, 0x63, 0x49, 0xf4, 0xbe, 0x93, 0x5b, 0x90, 0xd7, 0x74, 0xdb, 0xd1, 0x0d, - 0xd5, 0xe1, 0x47, 0x19, 0xb3, 0x63, 0x80, 0x73, 0x2e, 0x95, 0x9d, 0x57, 0x7c, 0x1f, 0xe6, 0xf9, - 0xbe, 0x86, 0xc0, 0xc7, 0x58, 0x7e, 0x1c, 0x30, 0x4b, 0xf0, 0xbe, 0xbf, 0xa2, 0x0a, 0x14, 0x5a, - 0x8a, 0x83, 0x8f, 0x94, 0x13, 0xd9, 0xdd, 0x35, 0x97, 0xa5, 0x27, 0x7c, 0xbe, 0x75, 0x76, 0xba, - 0x98, 0x7b, 0xca, 0x92, 0x06, 0x36, 0xcf, 0xe5, 0x5a, 0x81, 0x04, 0x0d, 0xdd, 0x81, 0x82, 0x62, - 0x9f, 0x18, 0x2a, 0x55, 0x0f, 0x36, 0xec, 0x9e, 0x4d, 0x21, 0x49, 0x5a, 0xca, 0x53, 0x72, 0xc5, - 0xa5, 0xa2, 0xeb, 0x30, 0xc7, 0x83, 0xfe, 0xd9, 0x1d, 0x4c, 0x05, 0x5a, 0x55, 0x7e, 0xb9, 0x07, - 0xbb, 0x86, 0xe9, 0x31, 0x94, 0xf9, 0x55, 0x05, 0x47, 0x8a, 0xa5, 0xc9, 0x54, 0xeb, 0x7e, 0xff, - 0x2c, 0x52, 0xb1, 0x97, 0xd9, 0xd5, 0x04, 0x24, 0x03, 0x51, 0xb5, 0x3f, 0x86, 0xb6, 0x60, 0x5e, - 0x6d, 0xeb, 0x1e, 0x82, 0x60, 0x35, 0x9f, 0x1f, 0x3a, 0x9a, 0x2a, 0x34, 0xaf, 0xff, 0x31, 0xba, - 0xa0, 0x86, 0x09, 0xa8, 0x01, 0xf4, 0x64, 0x1f, 0xb9, 0x6b, 0xb6, 0x75, 0xf5, 0x84, 0x82, 0x87, - 0xf0, 0x91, 0xeb, 0x23, 0x2f, 0x3e, 0x78, 0xa5, 0xe8, 0xce, 0x36, 0xe5, 0x94, 0xe0, 0xc8, 0xfb, - 0xcd, 0x8e, 0x4b, 0x5e, 0x4f, 0xa5, 0xe7, 0x8a, 0xb9, 0xf5, 0x54, 0x3a, 0x5f, 0x2c, 0x88, 0x7f, - 0x4d, 0x80, 0x42, 0x5f, 0x59, 0x50, 0x0d, 0x2e, 0x6a, 0xde, 0x50, 0x91, 0xf9, 0xae, 0x30, 0xdd, - 0x34, 0xf8, 0x29, 0xf2, 0x0b, 0x5f, 0x9e, 0x2e, 0x16, 0x68, 0xee, 0xa7, 0x5e, 0x92, 0x74, 0xc1, - 0xe7, 0xf0, 0xa9, 0xe8, 0x23, 0xc8, 0x33, 0xf7, 0xd3, 0xbb, 0x56, 0x90, 0x86, 0xf3, 0xaf, 0xce, - 0x7f, 0x79, 0xba, 0x98, 0xa3, 0x3e, 0xa7, 0x7b, 0xca, 0xb3, 0x94, 0x6b, 0x07, 0x1f, 0xc5, 0x5f, - 0x13, 0x60, 0x2e, 0xb4, 0xf3, 0xea, 0x71, 0xdf, 0xc7, 0xef, 0x2b, 0xd1, 0xb8, 0x75, 0x58, 0xec, - 0x63, 0x9a, 0xf7, 0x73, 0x37, 0x90, 0x74, 0x71, 0x38, 0xee, 0xa1, 0xab, 0x38, 0x6e, 0x54, 0x8d, - 0xcb, 0xf6, 0x71, 0xea, 0x07, 0x3f, 0x5c, 0x9c, 0x12, 0xff, 0x2c, 0x05, 0xb9, 0xf0, 0x3e, 0xab, - 0x7a, 0x5f, 0xb9, 0xa2, 0xe6, 0x95, 0x10, 0xc7, 0xd2, 0x88, 0xbb, 0x66, 0x32, 0xfe, 0x75, 0x0d, - 0xac, 0x98, 0xd7, 0x46, 0x7c, 0xe2, 0x0f, 0x96, 0xd3, 0x67, 0x2c, 0xff, 0xbb, 0xa4, 0x67, 0x5f, - 0x97, 0x60, 0x9a, 0x1e, 0x99, 0xc4, 0x8b, 0x56, 0xea, 0xef, 0x3d, 0xc4, 0xd7, 0x26, 0xe9, 0x12, - 0xcb, 0x46, 0xec, 0x71, 0xf3, 0x8d, 0xce, 0x24, 0xf4, 0x87, 0xc1, 0xf9, 0x6f, 0x71, 0xec, 0xb1, - 0x33, 0x29, 0xff, 0x2f, 0x46, 0xde, 0x90, 0xf7, 0xa1, 0x5f, 0x80, 0x82, 0x6a, 0xb6, 0xdb, 0x6c, - 0xae, 0x65, 0x16, 0x6d, 0xf0, 0x94, 0x1a, 0x5a, 0x04, 0x7e, 0x71, 0xe7, 0x92, 0x77, 0x81, 0xe7, - 0x92, 0xc4, 0x2f, 0xf0, 0x0c, 0x84, 0xf5, 0xe6, 0x3d, 0x61, 0xcc, 0x10, 0xf6, 0x45, 0x18, 0xcf, - 0xbe, 0x49, 0x84, 0x31, 0x8b, 0x4a, 0xe7, 0x3d, 0xef, 0x0f, 0x05, 0x1e, 0xc2, 0xf3, 0xcc, 0x34, - 0x0f, 0x7a, 0x5e, 0x64, 0x70, 0x39, 0x78, 0xc2, 0x64, 0xfa, 0xcb, 0xd3, 0xc5, 0x94, 0xe4, 0x1d, - 0x31, 0x19, 0x35, 0x73, 0x24, 0xbe, 0xda, 0xcc, 0x71, 0x1d, 0xe6, 0xba, 0x16, 0xde, 0xc3, 0x8e, - 0xba, 0x2f, 0x1b, 0xbd, 0x0e, 0xdf, 0x40, 0x94, 0x75, 0x69, 0x5b, 0xbd, 0x0e, 0xba, 0x07, 0x45, - 0x2f, 0x0b, 0xc7, 0xf8, 0xee, 0x11, 0x64, 0x2e, 0x9d, 0xaf, 0x08, 0x88, 0xff, 0x53, 0x80, 0x85, - 0x50, 0x9d, 0xf8, 0x98, 0x5a, 0x87, 0xac, 0x6f, 0x4e, 0xec, 0x92, 0x70, 0xce, 0x38, 0xd9, 0x20, - 0x33, 0x92, 0xe1, 0x92, 0xfb, 0x5a, 0x7a, 0xc5, 0x81, 0x2f, 0x36, 0x71, 0x4e, 0xb1, 0x17, 0x7d, - 0x39, 0x6b, 0x81, 0x17, 0x78, 0x83, 0x2c, 0x39, 0xd1, 0x20, 0x13, 0x7f, 0x53, 0x80, 0x22, 0x7d, - 0xc1, 0x13, 0x8c, 0xb5, 0x58, 0xac, 0x9b, 0x1b, 0x7f, 0x9e, 0x98, 0x7c, 0x8b, 0x50, 0xe8, 0x5a, - 0x96, 0x64, 0xf8, 0x5a, 0x16, 0xf1, 0x87, 0x02, 0xe4, 0xbd, 0x12, 0xb2, 0xab, 0x10, 0x47, 0x1c, - 0x64, 0xfa, 0x66, 0xd7, 0xfd, 0xb9, 0x07, 0xae, 0x4c, 0x74, 0x3b, 0x63, 0xf0, 0xc0, 0x15, 0x76, - 0x4d, 0xdd, 0xdf, 0x70, 0x7b, 0x0e, 0x29, 0x62, 0xc5, 0x3f, 0x58, 0xe3, 0x0d, 0x76, 0x4b, 0x49, - 0xf4, 0x16, 0x59, 0xb3, 0x7d, 0xc8, 0xce, 0xc8, 0x99, 0xc8, 0xec, 0x21, 0x1e, 0xb8, 0x06, 0x7c, - 0x35, 0x50, 0x6b, 0x36, 0xe8, 0xfd, 0xb2, 0xec, 0xb7, 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x36, 0x3e, - 0xd1, 0xd2, 0x44, 0xa6, 0xd8, 0xd5, 0x12, 0xeb, 0x2b, 0x7f, 0x10, 0x6c, 0x89, 0xea, 0x21, 0x41, - 0x81, 0x8f, 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x45, 0x80, 0x85, 0x5a, 0x4e, 0x22, 0xb9, 0xd1, 0x93, - 0xd0, 0x79, 0x24, 0x89, 0xe1, 0x88, 0x65, 0x50, 0xa5, 0xa1, 0x73, 0x4b, 0x3e, 0x0c, 0xf7, 0xf5, - 0x91, 0xaf, 0x0f, 0x76, 0xfa, 0x8f, 0x53, 0x3f, 0xfa, 0xe1, 0xa2, 0x20, 0x7e, 0x02, 0x48, 0xc2, - 0x36, 0x76, 0x5e, 0xf4, 0x4c, 0xcb, 0x3f, 0xdb, 0xe5, 0x76, 0xdf, 0x3d, 0x35, 0xd3, 0xab, 0xd9, - 0xb3, 0xa8, 0xe3, 0x9f, 0x2e, 0xc2, 0x42, 0x88, 0x9b, 0x19, 0x0b, 0xf1, 0x43, 0xb8, 0xf2, 0xd4, - 0xb4, 0x6d, 0xbd, 0x4b, 0xa0, 0x2f, 0x1d, 0x95, 0x64, 0x6a, 0xf1, 0xcc, 0x63, 0xba, 0x4b, 0x17, - 0x41, 0x0c, 0x66, 0x46, 0x32, 0x92, 0xf7, 0x2c, 0xfe, 0xae, 0x00, 0x97, 0x07, 0x39, 0x99, 0x96, - 0xa3, 0x36, 0x77, 0xce, 0xaa, 0xa6, 0x7f, 0x16, 0xe0, 0xf8, 0xde, 0xea, 0x66, 0x27, 0x2e, 0x2a, - 0x7f, 0xa7, 0xdc, 0x51, 0xa8, 0xf9, 0xe0, 0x1b, 0xcd, 0xf3, 0x9c, 0xbc, 0xc9, 0xa8, 0xbe, 0x25, - 0x49, 0x4d, 0x66, 0x49, 0x9a, 0x50, 0x58, 0x37, 0x75, 0x83, 0x78, 0xc2, 0x6e, 0x7d, 0x57, 0x20, - 0xbf, 0xab, 0x1b, 0x8a, 0x75, 0x22, 0xbb, 0x81, 0x87, 0xc2, 0xb8, 0xc0, 0x43, 0x29, 0xc7, 0x38, - 0xf8, 0xa3, 0xf8, 0x63, 0x01, 0x8a, 0xbe, 0x58, 0x6e, 0x91, 0xdf, 0x05, 0x50, 0xdb, 0x3d, 0xdb, - 0xc1, 0x96, 0xdb, 0x4a, 0x73, 0x2c, 0x50, 0xbf, 0xc2, 0xa8, 0xf5, 0x35, 0x29, 0xc3, 0x33, 0xd4, - 0x35, 0x74, 0x23, 0x7c, 0x0e, 0xc6, 0xf4, 0x2a, 0x9c, 0x0d, 0x9c, 0x7e, 0x41, 0x9a, 0xdd, 0x76, - 0x4c, 0xcb, 0xc3, 0x5c, 0xbc, 0xd9, 0xdd, 0x93, 0xfd, 0xe9, 0xf6, 0x6d, 0x92, 0x6f, 0x05, 0xf2, - 0xc4, 0x5d, 0x38, 0xc4, 0x5e, 0x95, 0x52, 0xe3, 0xab, 0xc4, 0x38, 0xdc, 0x2a, 0xfd, 0x73, 0xe2, - 0xe4, 0xb2, 0xd6, 0xf0, 0x5a, 0x78, 0x84, 0x45, 0x5b, 0x83, 0xb4, 0x73, 0x6c, 0xc8, 0x1d, 0xec, - 0xdd, 0x86, 0x73, 0x8e, 0x83, 0xfa, 0x66, 0x1d, 0xf6, 0x48, 0x2f, 0x4c, 0xe4, 0xb7, 0x75, 0xf3, - 0xe1, 0x72, 0x65, 0x89, 0x5d, 0xe7, 0xbd, 0xe4, 0x5e, 0xe7, 0xbd, 0xb4, 0xc6, 0x33, 0x30, 0xa3, - 0xfe, 0x83, 0xff, 0xbc, 0x28, 0x48, 0x1e, 0x13, 0x9b, 0xf7, 0xef, 0x37, 0x48, 0xaf, 0x1f, 0x98, - 0x99, 0x51, 0x1e, 0x20, 0x70, 0xcd, 0x11, 0xbf, 0xc6, 0x79, 0x65, 0x4d, 0xde, 0xd9, 0xaa, 0x3c, - 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x17, 0xba, 0x24, 0x29, 0xc1, - 0x2e, 0x76, 0xbe, 0xff, 0xff, 0x01, 0xf8, 0xf7, 0xad, 0x11, 0x59, 0x1b, 0xd5, 0xcf, 0xe4, 0x97, - 0x2b, 0xcf, 0x76, 0xaa, 0x8d, 0xe2, 0x14, 0x42, 0x90, 0x5f, 0x5d, 0x69, 0x56, 0x6a, 0xb2, 0x54, - 0x6d, 0x6c, 0x3f, 0xdf, 0x6a, 0x54, 0xdd, 0x0b, 0xa1, 0xef, 0xaf, 0xc1, 0x5c, 0xf0, 0xb4, 0x23, - 0xb4, 0x00, 0x85, 0x4a, 0xad, 0x5a, 0xd9, 0x90, 0x5f, 0xd6, 0x57, 0xe4, 0x17, 0x3b, 0xd5, 0x9d, - 0x6a, 0x71, 0x8a, 0x16, 0x8d, 0x12, 0x9f, 0xec, 0x3c, 0x7b, 0x56, 0x14, 0x50, 0x01, 0xb2, 0xec, - 0x99, 0x5e, 0xa8, 0x54, 0x4c, 0xdc, 0xdf, 0x84, 0x6c, 0xe0, 0x58, 0x64, 0xf2, 0xba, 0xed, 0x9d, - 0x46, 0x4d, 0x6e, 0xd6, 0x37, 0xab, 0x8d, 0xe6, 0xca, 0xe6, 0x36, 0x93, 0x41, 0x69, 0x2b, 0xab, - 0xcf, 0xa5, 0x66, 0x51, 0xf0, 0x9e, 0x9b, 0xcf, 0x77, 0x2a, 0x35, 0xb7, 0x1a, 0x62, 0x2a, 0x9d, - 0x2c, 0x26, 0xef, 0xff, 0x25, 0x01, 0x2e, 0x0f, 0x39, 0xf9, 0x07, 0x65, 0x61, 0x76, 0xc7, 0xa0, - 0x67, 0xc4, 0x16, 0xa7, 0x50, 0x2e, 0x70, 0xf8, 0x4f, 0x51, 0x40, 0x69, 0x76, 0xfc, 0x4a, 0x31, - 0x81, 0x66, 0x20, 0xd1, 0x78, 0x54, 0x4c, 0x92, 0x92, 0x06, 0xce, 0xce, 0x29, 0xa6, 0x50, 0x86, - 0x9f, 0xda, 0x51, 0x9c, 0x46, 0x73, 0xfe, 0xe1, 0x19, 0xc5, 0x19, 0x22, 0xca, 0x3b, 0x84, 0xa2, - 0x38, 0x7b, 0xff, 0x3a, 0x04, 0x36, 0xfa, 0x23, 0x80, 0x99, 0x67, 0x8a, 0x83, 0x6d, 0xa7, 0x38, - 0x85, 0x66, 0x21, 0xb9, 0xd2, 0x6e, 0x17, 0x85, 0x87, 0xff, 0x2c, 0x05, 0x69, 0xf7, 0xe2, 0x20, - 0xf4, 0x0c, 0xa6, 0xd9, 0x72, 0xf8, 0xe2, 0x70, 0xb4, 0x40, 0x07, 0x74, 0xf9, 0xda, 0x38, 0x38, - 0x21, 0x4e, 0xa1, 0xbf, 0x00, 0xd9, 0x80, 0x17, 0x85, 0x86, 0x2e, 0xe9, 0x85, 0x3c, 0xc7, 0xf2, - 0xed, 0x71, 0xd9, 0x3c, 0xf9, 0xaf, 0x20, 0xe3, 0x59, 0x75, 0x74, 0x63, 0x94, 0xcd, 0x77, 0x65, - 0x8f, 0x9e, 0x18, 0xc8, 0xf8, 0x13, 0xa7, 0xde, 0x17, 0x90, 0x05, 0x68, 0xd0, 0x00, 0xa3, 0xa8, - 0x20, 0x8b, 0xa1, 0x16, 0xbe, 0x7c, 0x7f, 0xa2, 0xdc, 0xfe, 0x3b, 0x89, 0xb2, 0xfc, 0x59, 0x24, - 0x5a, 0x59, 0x03, 0x73, 0x54, 0xb4, 0xb2, 0x22, 0x26, 0xa3, 0x29, 0xf4, 0x02, 0x52, 0xc4, 0x7a, - 0xa2, 0x28, 0xbf, 0xb2, 0xcf, 0x5a, 0x97, 0x6f, 0x8c, 0xcc, 0xe3, 0x8a, 0x5c, 0xbd, 0xf7, 0xa3, - 0x3f, 0xb9, 0x3a, 0xf5, 0xa3, 0xb3, 0xab, 0xc2, 0x8f, 0xcf, 0xae, 0x0a, 0x7f, 0x74, 0x76, 0x55, - 0xf8, 0xe3, 0xb3, 0xab, 0xc2, 0xf7, 0x7f, 0x72, 0x75, 0xea, 0xc7, 0x3f, 0xb9, 0x3a, 0xf5, 0x47, - 0x3f, 0xb9, 0x3a, 0xf5, 0xf9, 0x2c, 0xe7, 0xde, 0x9d, 0xa1, 0xa6, 0xe5, 0xd1, 0xff, 0x09, 0x00, - 0x00, 0xff, 0xff, 0xaf, 0x6d, 0x62, 0x22, 0x64, 0x81, 0x00, 0x00, + 0xc0, 0x64, 0x72, 0x0a, 0x84, 0xcb, 0xf0, 0x7d, 0x06, 0xb4, 0x0e, 0x3e, 0x99, 0x6e, 0x05, 0xf8, + 0x08, 0x4a, 0x81, 0x8c, 0x74, 0xf1, 0x45, 0xc6, 0x86, 0x63, 0xe9, 0x98, 0x2d, 0x20, 0x26, 0xa5, + 0x40, 0x4c, 0x4f, 0x9d, 0x24, 0x57, 0x59, 0x2a, 0x6a, 0xc2, 0x1c, 0xc9, 0x78, 0x22, 0xd3, 0x29, + 0xc4, 0x5d, 0xe0, 0x7d, 0x10, 0x51, 0xad, 0x50, 0xb9, 0x97, 0xa8, 0x7e, 0x2a, 0x94, 0x87, 0xfe, + 0x94, 0xb2, 0xd8, 0xa7, 0x94, 0x3f, 0x85, 0x62, 0x7f, 0x86, 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0x17, + 0x82, 0xba, 0x4c, 0x06, 0xf4, 0xb4, 0x9e, 0x4a, 0xa7, 0x8a, 0xd3, 0xe2, 0x9f, 0x26, 0x21, 0xef, + 0x76, 0xb3, 0x38, 0x81, 0xce, 0x2a, 0x4c, 0x93, 0x4e, 0xe1, 0x46, 0xa0, 0xdc, 0x1e, 0xd1, 0xbb, + 0x79, 0x64, 0x3b, 0xe9, 0x2c, 0x2e, 0x48, 0xa6, 0xac, 0x71, 0x18, 0x9c, 0xf2, 0x2f, 0x26, 0x20, + 0x45, 0xb1, 0xc5, 0x03, 0x48, 0xd1, 0x89, 0x42, 0x98, 0x64, 0xa2, 0xa0, 0x59, 0xbd, 0xe9, 0x2c, + 0x11, 0x70, 0x4d, 0x89, 0xcf, 0xb7, 0xaf, 0x7c, 0xf0, 0xe0, 0x21, 0x35, 0x36, 0x73, 0x12, 0x7f, + 0x42, 0xab, 0x34, 0x34, 0xca, 0xb4, 0x1c, 0xac, 0x71, 0x9f, 0xfe, 0xda, 0xb8, 0xf6, 0x75, 0x27, + 0x25, 0x97, 0x0f, 0x5d, 0x81, 0x24, 0xb1, 0x62, 0xb3, 0x2c, 0xca, 0xe1, 0xec, 0x74, 0x31, 0x49, + 0xec, 0x17, 0xa1, 0xa1, 0x65, 0xc8, 0x86, 0x4d, 0x06, 0xf1, 0xe0, 0xa8, 0x61, 0x0c, 0x0c, 0x77, + 0x68, 0x7b, 0x43, 0x8b, 0xe1, 0x59, 0xde, 0xc6, 0x7f, 0x9e, 0x82, 0x5c, 0xbd, 0x13, 0xf7, 0x94, + 0xb2, 0x12, 0x6e, 0xe1, 0x28, 0x20, 0x14, 0x7a, 0x69, 0x44, 0x03, 0x87, 0x66, 0xf0, 0xe4, 0xf9, + 0x66, 0xf0, 0x3a, 0xf1, 0x94, 0xf9, 0x21, 0x11, 0xc9, 0x21, 0x98, 0x27, 0xfc, 0x7e, 0xea, 0xa7, + 0x48, 0x84, 0xc7, 0xdf, 0xeb, 0x41, 0x23, 0x55, 0x3e, 0xa5, 0x0e, 0x39, 0xeb, 0x65, 0x33, 0x93, + 0xf7, 0xb2, 0x59, 0x6c, 0x68, 0x74, 0x52, 0x0b, 0x5b, 0xd4, 0xd9, 0x37, 0xb7, 0xa8, 0x65, 0x87, + 0x77, 0xd6, 0x8f, 0x21, 0xa9, 0xe9, 0x6e, 0xe3, 0x4c, 0x3e, 0x55, 0x13, 0xa6, 0x31, 0xbd, 0x36, + 0x15, 0xec, 0xb5, 0xac, 0x97, 0x94, 0xeb, 0x00, 0xbe, 0x6e, 0xd0, 0x35, 0x98, 0x31, 0xdb, 0x9a, + 0xbb, 0xd9, 0x25, 0xb7, 0x9a, 0x39, 0x3b, 0x5d, 0x9c, 0x7e, 0xde, 0xd6, 0xea, 0x6b, 0xd2, 0xb4, + 0xd9, 0xd6, 0xea, 0x1a, 0x3d, 0xa1, 0x03, 0x1f, 0xc9, 0x5e, 0x24, 0xdc, 0x9c, 0x34, 0x6b, 0xe0, + 0xa3, 0x35, 0x6c, 0xab, 0xbc, 0xc3, 0xfd, 0xa6, 0x00, 0x79, 0x57, 0xf7, 0xf1, 0x1a, 0x95, 0xb4, + 0xde, 0xe1, 0x83, 0x2c, 0x79, 0xbe, 0x41, 0xe6, 0xf2, 0xf1, 0x8d, 0xc5, 0xbf, 0x2c, 0xf0, 0xd8, + 0xe6, 0x86, 0xaa, 0x38, 0xc4, 0xa9, 0x88, 0x71, 0x60, 0xdc, 0x83, 0xa2, 0xa5, 0x18, 0x9a, 0xd9, + 0xd1, 0xbf, 0xc0, 0x6c, 0x21, 0xd4, 0xe6, 0x9f, 0x3d, 0x0b, 0x1e, 0x9d, 0xae, 0xfa, 0xd9, 0xe2, + 0xef, 0x27, 0x78, 0x1c, 0xb4, 0x57, 0x8c, 0x38, 0xd5, 0xf5, 0x1d, 0x98, 0x0f, 0x4e, 0x6d, 0x2c, + 0xd4, 0x92, 0x8d, 0xd6, 0xf7, 0x22, 0xe4, 0x45, 0x15, 0x84, 0xc5, 0x33, 0xba, 0xc1, 0xf5, 0x81, + 0x19, 0x91, 0x0a, 0x43, 0x15, 0xc8, 0xf2, 0x8f, 0x0f, 0xc6, 0x9e, 0xe9, 0x06, 0x8a, 0xbd, 0x3d, + 0x2c, 0x2c, 0xb2, 0x6e, 0xec, 0x99, 0xee, 0x87, 0x7c, 0xcb, 0x25, 0xd8, 0xe5, 0x9f, 0x83, 0x69, + 0x9a, 0xfc, 0x06, 0x26, 0x9a, 0xb7, 0xe6, 0x9f, 0x24, 0xe0, 0x26, 0x2d, 0xfd, 0x4b, 0x6c, 0xe9, + 0x7b, 0x27, 0xdb, 0x96, 0xe9, 0x60, 0xd5, 0xc1, 0x9a, 0xbf, 0x72, 0x1e, 0xab, 0xdd, 0xcb, 0x74, + 0xdd, 0x17, 0x9c, 0x2b, 0xe0, 0xcc, 0xe3, 0x42, 0x1b, 0x50, 0xe0, 0xa1, 0x05, 0x4a, 0x5b, 0x3f, + 0xc4, 0xb2, 0xe2, 0x9c, 0x67, 0x76, 0xcb, 0x31, 0xde, 0x15, 0xc2, 0xba, 0xe2, 0x20, 0x0d, 0x32, + 0x5c, 0x98, 0xae, 0xf1, 0xc3, 0x83, 0x9e, 0x7e, 0xb5, 0x05, 0xc5, 0x34, 0x8b, 0x6f, 0xa8, 0xaf, + 0x49, 0x69, 0x26, 0xb9, 0xae, 0x89, 0xff, 0x49, 0x80, 0x5b, 0x63, 0x54, 0x1c, 0x67, 0xd7, 0x2d, + 0x43, 0xfa, 0x90, 0xbc, 0x48, 0xe7, 0x3a, 0x4e, 0x4b, 0xde, 0x33, 0xda, 0x84, 0xdc, 0x9e, 0xa2, + 0xb7, 0xfd, 0x2e, 0x3d, 0x3c, 0x3e, 0x31, 0x3a, 0x1a, 0x77, 0x8e, 0xb1, 0xb3, 0x3e, 0x2c, 0xfe, + 0x7a, 0x02, 0xe6, 0x57, 0x34, 0xad, 0xd1, 0xe0, 0x36, 0x30, 0xbe, 0x9e, 0xe2, 0x82, 0xcc, 0x84, + 0x0f, 0x32, 0xd1, 0x7b, 0x80, 0x34, 0xdd, 0x66, 0x07, 0x9f, 0xd8, 0xfb, 0x8a, 0x66, 0x1e, 0xf9, + 0x71, 0x19, 0xf3, 0x6e, 0x4a, 0xc3, 0x4d, 0x40, 0x0d, 0xa0, 0x68, 0x47, 0xb6, 0x1d, 0xc5, 0xfb, + 0xee, 0x74, 0x6b, 0xa2, 0x2d, 0x65, 0x0c, 0x06, 0x79, 0x8f, 0x52, 0x86, 0xc8, 0xa1, 0x3f, 0x89, + 0xdf, 0xae, 0x93, 0xaa, 0x3b, 0xb2, 0x62, 0xbb, 0xfb, 0x87, 0xd8, 0x91, 0x2b, 0x79, 0x46, 0x5f, + 0xb1, 0xd9, 0xb6, 0x20, 0xb6, 0xe1, 0xc1, 0x57, 0x4d, 0x9c, 0x90, 0xf8, 0xef, 0x09, 0x90, 0x97, + 0xf0, 0x9e, 0x85, 0xed, 0x58, 0x17, 0x05, 0x9e, 0xc0, 0x9c, 0xc5, 0xa4, 0xca, 0x7b, 0x96, 0xd9, + 0x39, 0xcf, 0xb8, 0xca, 0x72, 0xc6, 0x27, 0x96, 0xd9, 0xe1, 0x86, 0xe5, 0x25, 0x14, 0xbc, 0x32, + 0xc6, 0x59, 0xf9, 0xdf, 0xa1, 0xdb, 0xa5, 0x99, 0xe0, 0xb8, 0x03, 0x24, 0xe2, 0xd5, 0x00, 0xfd, + 0x50, 0x15, 0x2c, 0x68, 0x9c, 0x6a, 0xf8, 0x6f, 0x02, 0xe4, 0x1b, 0xbd, 0x5d, 0x76, 0xbe, 0x56, + 0x7c, 0x1a, 0xa8, 0x42, 0xa6, 0x8d, 0xf7, 0x1c, 0xf9, 0x8d, 0x42, 0xf5, 0xd3, 0x84, 0x95, 0x6e, + 0x54, 0x78, 0x0a, 0x60, 0xd1, 0x2d, 0x76, 0x54, 0x4e, 0xf2, 0x9c, 0x72, 0x32, 0x94, 0x97, 0x90, + 0xc5, 0x3f, 0x4f, 0x40, 0xc1, 0xab, 0x66, 0x9c, 0x56, 0xf2, 0x55, 0xc8, 0x3a, 0x24, 0xcf, 0x63, + 0x1d, 0xe6, 0x79, 0x4c, 0x48, 0xb4, 0x85, 0x58, 0x82, 0x05, 0xea, 0xb8, 0xc8, 0x4a, 0xb7, 0xdb, + 0xd6, 0x5d, 0xb8, 0x4b, 0xed, 0x4f, 0x4a, 0x9a, 0xa7, 0x49, 0x2b, 0x2c, 0x85, 0x02, 0x5d, 0xf4, + 0x2b, 0x02, 0xcc, 0xed, 0x59, 0x18, 0x7f, 0x81, 0x65, 0x0a, 0xbd, 0x26, 0x0b, 0x7a, 0x59, 0x23, + 0x65, 0xf8, 0xca, 0x1f, 0xc5, 0xb3, 0xec, 0xc5, 0x0d, 0xf2, 0x5e, 0xde, 0x69, 0x5f, 0xc1, 0x3c, + 0x6d, 0x94, 0xb8, 0xf7, 0x16, 0x8b, 0xbf, 0x9d, 0x00, 0x14, 0x94, 0xfc, 0xd3, 0x6b, 0xcc, 0x44, + 0x7c, 0x8d, 0xf9, 0x2e, 0x20, 0x16, 0x36, 0x69, 0xcb, 0x5d, 0x6c, 0xc9, 0x36, 0x56, 0x4d, 0x7e, + 0x3c, 0x94, 0x20, 0x15, 0x79, 0xca, 0x36, 0xb6, 0x1a, 0x94, 0x8e, 0x56, 0x00, 0x7c, 0x97, 0x8e, + 0xcf, 0x38, 0x93, 0x78, 0x74, 0x19, 0xcf, 0xa3, 0x13, 0xbf, 0x2f, 0x40, 0x7e, 0x53, 0x6f, 0x59, + 0x4a, 0xac, 0xa7, 0x1f, 0xa1, 0x8f, 0xc3, 0x8b, 0xe9, 0xd9, 0x87, 0xe5, 0xa8, 0x40, 0x1f, 0x96, + 0xc3, 0x85, 0x68, 0x9c, 0x81, 0x18, 0x72, 0xaf, 0x44, 0x71, 0x5a, 0xb0, 0xff, 0x50, 0x86, 0x39, + 0x5e, 0xee, 0x1d, 0x43, 0x37, 0x0d, 0xf4, 0x00, 0x92, 0x2d, 0xfe, 0xb1, 0x24, 0x1b, 0xb9, 0xb0, + 0xe9, 0x1f, 0x04, 0x58, 0x9b, 0x92, 0x48, 0x5e, 0xc2, 0xd2, 0xed, 0x39, 0x11, 0xae, 0xa4, 0x1f, + 0xfb, 0x1e, 0x64, 0xe9, 0xf6, 0x1c, 0xd4, 0x80, 0x82, 0xea, 0x1f, 0x68, 0x26, 0x13, 0xf6, 0xe4, + 0x50, 0xd8, 0x19, 0x79, 0xb4, 0x5c, 0x6d, 0x4a, 0xca, 0xab, 0xa1, 0x04, 0x54, 0x09, 0x9e, 0xa3, + 0x95, 0x1a, 0x08, 0xac, 0xf3, 0x77, 0x6d, 0x87, 0xcf, 0xf0, 0xaa, 0x4d, 0x05, 0x8e, 0xdb, 0x42, + 0x1f, 0xc3, 0x8c, 0x46, 0x4f, 0x6c, 0xe2, 0x26, 0x20, 0xaa, 0xa1, 0x43, 0x07, 0x63, 0xd5, 0xa6, + 0x24, 0xce, 0x81, 0xd6, 0x61, 0x8e, 0xfd, 0x62, 0x8e, 0x1d, 0xc7, 0xe2, 0xb7, 0x86, 0x4b, 0x08, + 0x4c, 0x9d, 0xb5, 0x29, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0xa2, 0xf1, + 0xab, 0x43, 0x0e, 0x4a, 0xf1, 0x99, 0x69, 0x6e, 0xf4, 0x98, 0x1d, 0xc3, 0xe9, 0x1c, 0xbb, 0x0b, + 0xa3, 0x51, 0xc5, 0x0f, 0x6d, 0xc8, 0x27, 0xc5, 0xc7, 0x94, 0x80, 0x9e, 0x42, 0x56, 0x21, 0x1e, + 0xb2, 0x4c, 0xb7, 0xb4, 0xd2, 0x95, 0xd0, 0xe8, 0x70, 0x84, 0x81, 0x2d, 0xc8, 0x35, 0xba, 0xeb, + 0xdf, 0x25, 0xfa, 0x82, 0x3a, 0xd8, 0x6a, 0xe1, 0x52, 0x76, 0xb4, 0xa0, 0x60, 0x38, 0x9d, 0x27, + 0x88, 0x12, 0x89, 0xa7, 0xec, 0xed, 0x27, 0xa7, 0x95, 0x9a, 0x1b, 0xfa, 0x01, 0x3c, 0x62, 0xa7, + 0x54, 0x6d, 0x4a, 0x9a, 0xdb, 0x0f, 0x90, 0xd1, 0x12, 0x24, 0x5a, 0x6a, 0x29, 0x37, 0xd4, 0x24, + 0x78, 0xbb, 0x81, 0x6a, 0x53, 0x52, 0xa2, 0xa5, 0xa2, 0x4f, 0x21, 0xcd, 0xb6, 0x76, 0x1c, 0x1b, + 0xa5, 0xfc, 0xd0, 0x31, 0x16, 0xde, 0x20, 0x53, 0x9b, 0x92, 0xe8, 0x6e, 0x12, 0xf2, 0xbe, 0x6d, + 0xc8, 0x5b, 0x2c, 0x1e, 0xd1, 0x8d, 0x1e, 0x2e, 0x0e, 0x0d, 0x0a, 0x88, 0x0a, 0x20, 0xae, 0x51, + 0xac, 0x14, 0xa0, 0xa3, 0xef, 0xc0, 0x85, 0xb0, 0x44, 0xde, 0xd3, 0xe6, 0x87, 0x7e, 0xe0, 0x1e, + 0x1a, 0xcc, 0x5a, 0x9b, 0x92, 0x90, 0x35, 0x90, 0x88, 0x3e, 0x84, 0x69, 0xd6, 0x6a, 0x88, 0x8a, + 0x8c, 0x8a, 0x73, 0xe9, 0x6b, 0x30, 0x96, 0x9f, 0x74, 0x7e, 0x87, 0x07, 0xe5, 0xc9, 0x6d, 0xb3, + 0x55, 0x5a, 0x18, 0xda, 0xf9, 0x07, 0xc3, 0x0b, 0x49, 0xe7, 0x77, 0x7c, 0x2a, 0x69, 0x77, 0x8b, + 0xa5, 0xf0, 0x40, 0xac, 0x0b, 0x43, 0xdb, 0x3d, 0x22, 0x56, 0xaf, 0x46, 0xb7, 0x4b, 0xf8, 0x64, + 0x52, 0x34, 0x8b, 0x9d, 0xed, 0x23, 0xd3, 0x31, 0x75, 0x71, 0x68, 0xd1, 0x06, 0x0f, 0x3f, 0xaa, + 0x51, 0xaf, 0xd2, 0xa3, 0xa2, 0x97, 0x50, 0xe4, 0x27, 0x70, 0xf8, 0x5f, 0x61, 0x2e, 0x51, 0x79, + 0xf7, 0x22, 0x4d, 0x57, 0x54, 0x14, 0x53, 0x6d, 0x4a, 0x2a, 0xa8, 0xe1, 0x14, 0xf4, 0x19, 0xcc, + 0x53, 0x79, 0xb2, 0xea, 0x1f, 0x9a, 0x52, 0x2a, 0x0d, 0x1c, 0xc1, 0x31, 0xfc, 0x7c, 0x15, 0x57, + 0x72, 0x51, 0xed, 0x4b, 0x22, 0xdd, 0x58, 0x37, 0x74, 0x87, 0x5a, 0xd9, 0xf2, 0xd0, 0x6e, 0x1c, + 0x3e, 0x00, 0x92, 0x74, 0x63, 0x9d, 0x51, 0x48, 0x37, 0x76, 0x78, 0x94, 0x1e, 0x6f, 0x8e, 0xb7, + 0x87, 0x76, 0xe3, 0xa8, 0x70, 0x3e, 0xd2, 0x8d, 0x9d, 0x20, 0x9d, 0x74, 0x63, 0x66, 0x20, 0xfa, + 0xe4, 0xbe, 0x33, 0xb4, 0x1b, 0x0f, 0xdd, 0x5a, 0x4e, 0xba, 0xb1, 0x32, 0x90, 0x88, 0xd6, 0x00, + 0x98, 0x03, 0x48, 0xbd, 0x80, 0xab, 0x43, 0x27, 0x83, 0xfe, 0x68, 0x3d, 0x32, 0x19, 0xb4, 0x5d, + 0x1a, 0x31, 0x64, 0x14, 0x5e, 0xca, 0xf4, 0xa3, 0x73, 0x69, 0x71, 0xa8, 0x21, 0x1b, 0xf8, 0x40, + 0x4c, 0x0c, 0xd9, 0x91, 0x47, 0x24, 0xb3, 0x0a, 0x5b, 0x25, 0x2f, 0x5d, 0x1b, 0x6e, 0x96, 0x83, + 0x1f, 0xcb, 0xa8, 0x59, 0xa6, 0x04, 0xb4, 0x02, 0x19, 0xe2, 0xe4, 0x9c, 0x50, 0x33, 0x74, 0x7d, + 0xa8, 0x17, 0xdf, 0xb7, 0x1f, 0xa8, 0x36, 0x25, 0xa5, 0x5f, 0x73, 0x12, 0x79, 0x3d, 0x5b, 0x3f, + 0x2c, 0x89, 0x43, 0x5f, 0x1f, 0x5a, 0x6b, 0x26, 0xaf, 0x67, 0x1c, 0x48, 0x85, 0x8b, 0xac, 0xad, + 0xf8, 0x2e, 0x6f, 0x8b, 0x6f, 0x4c, 0x2e, 0xdd, 0xa0, 0xa2, 0x86, 0x2e, 0xc4, 0x45, 0x6e, 0x3e, + 0xaf, 0x4d, 0x49, 0x0b, 0xca, 0x60, 0x2a, 0x19, 0xf0, 0x7c, 0xea, 0x61, 0xcb, 0x77, 0xa5, 0x9b, + 0x43, 0x07, 0x7c, 0xc4, 0xaa, 0x27, 0x19, 0xf0, 0x4a, 0x80, 0xcc, 0x26, 0x20, 0x4d, 0xb6, 0x6d, + 0x16, 0xa2, 0x70, 0x6b, 0xc4, 0x04, 0xd4, 0xb7, 0x6e, 0xc2, 0x26, 0x20, 0xad, 0xc1, 0x38, 0x89, + 0x20, 0xb5, 0x8d, 0x15, 0x8b, 0x9b, 0xd9, 0xdb, 0x43, 0x05, 0x0d, 0x9c, 0xb5, 0x48, 0x04, 0xa9, + 0x1e, 0x91, 0x38, 0x3c, 0x96, 0x7b, 0x74, 0x0f, 0xf7, 0x90, 0xef, 0x0c, 0x75, 0x78, 0x22, 0xcf, + 0x16, 0x22, 0x0e, 0x8f, 0x15, 0x4a, 0x40, 0x3f, 0x0b, 0xb3, 0x1c, 0xf0, 0x96, 0xee, 0x8e, 0x70, + 0x01, 0x83, 0x6b, 0x14, 0x64, 0x5c, 0x73, 0x1e, 0x66, 0x65, 0x19, 0xd0, 0x66, 0xd5, 0xbb, 0x37, + 0xc2, 0xca, 0x0e, 0x60, 0x7d, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, 0xac, 0x9f, 0xf2, 0xb9, 0xee, + 0xfe, 0x50, 0x2b, 0x3b, 0xb8, 0xdb, 0x88, 0x58, 0xd9, 0xd7, 0x3e, 0x95, 0xd4, 0xcc, 0x66, 0x80, + 0xb3, 0xf4, 0x8d, 0xa1, 0x35, 0x0b, 0x23, 0x6f, 0x52, 0x33, 0xce, 0x43, 0x9a, 0x8d, 0x61, 0x00, + 0xa6, 0xe9, 0x77, 0x87, 0x1f, 0xa5, 0xd0, 0x8f, 0xb2, 0x6a, 0xee, 0xd2, 0x2e, 0xd3, 0xb0, 0x67, + 0xa8, 0x2c, 0xbe, 0xcf, 0x9b, 0x6b, 0xea, 0xbd, 0xd1, 0x86, 0x2a, 0x6a, 0x7f, 0xbc, 0x67, 0xa8, + 0x42, 0x89, 0xb4, 0xa8, 0x6c, 0x6b, 0x1f, 0x1d, 0xdf, 0x4b, 0x23, 0x4e, 0x7d, 0xe8, 0xdb, 0x60, + 0x49, 0x8b, 0xea, 0x11, 0xfd, 0x21, 0xd4, 0x63, 0x47, 0x92, 0x94, 0x96, 0x47, 0x0f, 0xa1, 0xf0, + 0xa1, 0x28, 0xde, 0x10, 0xe2, 0x64, 0x6f, 0xce, 0x74, 0x3d, 0x8c, 0xf7, 0x47, 0xcf, 0x99, 0xfd, + 0xae, 0x05, 0x9b, 0x33, 0xb9, 0x4f, 0xf1, 0x37, 0x04, 0xb8, 0xc6, 0xca, 0x46, 0xd7, 0x40, 0x4f, + 0x64, 0x6f, 0x25, 0x39, 0xb0, 0xcd, 0xe4, 0x01, 0x7d, 0xc1, 0x87, 0xc3, 0x8a, 0x3b, 0x66, 0x65, + 0xbc, 0x36, 0x25, 0xbd, 0xa3, 0x8c, 0xca, 0x47, 0xba, 0x54, 0x87, 0x21, 0xa8, 0xd2, 0xc3, 0xa1, + 0x5d, 0x2a, 0x8c, 0xfa, 0x48, 0x97, 0xe2, 0x3c, 0xab, 0xb3, 0xfc, 0x3b, 0xb4, 0xb7, 0x2d, 0xb7, + 0x50, 0x2c, 0xae, 0xa7, 0xd2, 0x97, 0x8b, 0xa5, 0xf5, 0x54, 0xfa, 0x4a, 0xb1, 0xbc, 0x9e, 0x4a, + 0xbf, 0x55, 0x7c, 0x5b, 0xfc, 0x47, 0x65, 0xc8, 0xb9, 0xa0, 0x8b, 0x01, 0xaa, 0x87, 0x41, 0x40, + 0x75, 0x75, 0x18, 0xa0, 0xe2, 0x30, 0x8d, 0x23, 0xaa, 0x87, 0x41, 0x44, 0x75, 0x75, 0x18, 0xa2, + 0xf2, 0x79, 0x08, 0xa4, 0x6a, 0x0e, 0x83, 0x54, 0xf7, 0x26, 0x80, 0x54, 0x9e, 0xa8, 0x7e, 0x4c, + 0xb5, 0x36, 0x88, 0xa9, 0x6e, 0x8e, 0xc6, 0x54, 0x9e, 0xa8, 0x00, 0xa8, 0x7a, 0xdc, 0x07, 0xaa, + 0xae, 0x8f, 0x00, 0x55, 0x1e, 0xbf, 0x8b, 0xaa, 0x36, 0x22, 0x51, 0xd5, 0xed, 0x71, 0xa8, 0xca, + 0x93, 0x13, 0x82, 0x55, 0x1f, 0x84, 0x60, 0xd5, 0xe2, 0x50, 0x58, 0xe5, 0x71, 0x33, 0x5c, 0xf5, + 0x49, 0x3f, 0xae, 0xba, 0x3e, 0x02, 0x57, 0xf9, 0x35, 0xe0, 0xc0, 0xaa, 0x16, 0x05, 0xac, 0x6e, + 0x8d, 0x01, 0x56, 0x9e, 0x94, 0x20, 0xb2, 0xaa, 0x45, 0x21, 0xab, 0x5b, 0x63, 0x90, 0x55, 0x9f, + 0x24, 0x06, 0xad, 0xb6, 0xa2, 0xa1, 0xd5, 0x9d, 0xb1, 0xd0, 0xca, 0x93, 0x16, 0xc6, 0x56, 0xcb, + 0x01, 0x6c, 0xf5, 0xce, 0x10, 0x6c, 0xe5, 0xb1, 0x12, 0x70, 0xf5, 0xad, 0x01, 0x70, 0x25, 0x8e, + 0x02, 0x57, 0x1e, 0xaf, 0x87, 0xae, 0x5e, 0x0c, 0x41, 0x57, 0x77, 0xc7, 0xa3, 0x2b, 0x4f, 0x58, + 0x1f, 0xbc, 0x52, 0x46, 0xc2, 0xab, 0xf7, 0x26, 0x84, 0x57, 0x9e, 0xf4, 0x28, 0x7c, 0xf5, 0x51, + 0x18, 0x5f, 0x5d, 0x1b, 0x8e, 0xaf, 0x3c, 0x31, 0x1c, 0x60, 0x6d, 0x44, 0x02, 0xac, 0xdb, 0xe3, + 0x00, 0x96, 0x3f, 0x0e, 0x82, 0x08, 0x6b, 0x2b, 0x1a, 0x61, 0xdd, 0x19, 0x8b, 0xb0, 0xfc, 0xe6, + 0x0f, 0x41, 0xac, 0x8d, 0x48, 0x88, 0x75, 0x7b, 0x1c, 0xc4, 0xf2, 0x0b, 0x17, 0xc4, 0x58, 0xaf, + 0x86, 0x62, 0xac, 0xfb, 0x93, 0x60, 0x2c, 0x4f, 0xe8, 0x00, 0xc8, 0xfa, 0x7c, 0x38, 0xc8, 0xfa, + 0xc6, 0x39, 0x0e, 0xb1, 0x8c, 0x44, 0x59, 0xdf, 0x1a, 0x40, 0x59, 0xe2, 0x28, 0x94, 0xe5, 0xf7, + 0x67, 0x17, 0x66, 0x29, 0x23, 0x41, 0xd1, 0x7b, 0x13, 0x82, 0x22, 0xbf, 0xf3, 0x45, 0xa0, 0xa2, + 0x6a, 0x04, 0x2a, 0xba, 0x39, 0x1a, 0x15, 0xf9, 0xe6, 0xdc, 0x87, 0x45, 0xb5, 0x28, 0x58, 0x74, + 0x6b, 0x0c, 0x2c, 0xf2, 0xad, 0x50, 0x00, 0x17, 0x3d, 0xee, 0xc3, 0x45, 0xd7, 0xc7, 0x86, 0x59, + 0x05, 0x80, 0xd1, 0xea, 0x20, 0x30, 0xba, 0x31, 0x12, 0x18, 0x79, 0x12, 0x7c, 0x64, 0xf4, 0xb8, + 0x0f, 0x19, 0x5d, 0x1f, 0x81, 0x8c, 0xfc, 0x02, 0x70, 0x68, 0xa4, 0x8d, 0x86, 0x46, 0x4b, 0x93, + 0x42, 0x23, 0x4f, 0x70, 0x24, 0x36, 0xda, 0x8a, 0xc6, 0x46, 0x77, 0x26, 0x8c, 0x80, 0x18, 0x00, + 0x47, 0xb5, 0x28, 0x70, 0x74, 0x6b, 0x0c, 0x38, 0x0a, 0xce, 0x21, 0x1e, 0x3a, 0xaa, 0x45, 0xa1, + 0xa3, 0x5b, 0x63, 0xd0, 0x91, 0x2f, 0x29, 0x00, 0x8f, 0x9a, 0xc3, 0xe0, 0xd1, 0xbd, 0x09, 0xe0, + 0x91, 0xef, 0xbc, 0xf4, 0xe1, 0xa3, 0x4f, 0xfb, 0xf1, 0x91, 0x38, 0x0a, 0x1f, 0xf9, 0x23, 0xd2, + 0x05, 0x48, 0x5b, 0xd1, 0x00, 0xe9, 0xce, 0x58, 0x80, 0x14, 0x34, 0x92, 0x01, 0x84, 0xb4, 0x11, + 0x89, 0x90, 0x6e, 0x8f, 0x43, 0x48, 0xbe, 0x91, 0x0c, 0x42, 0xa4, 0x4f, 0xfb, 0x21, 0x92, 0x38, + 0x0a, 0x22, 0xf9, 0x95, 0x73, 0x31, 0x52, 0x2d, 0x0a, 0x23, 0xdd, 0x1a, 0x83, 0x91, 0xfc, 0xc6, + 0x0b, 0x80, 0x24, 0x65, 0x24, 0x48, 0x7a, 0x6f, 0x42, 0x90, 0xd4, 0x67, 0xb8, 0xc2, 0x28, 0xa9, + 0x16, 0x85, 0x92, 0x6e, 0x8d, 0x41, 0x49, 0x81, 0xc2, 0xfa, 0x30, 0x69, 0x2b, 0x1a, 0x26, 0xdd, + 0x19, 0x0b, 0x93, 0xfa, 0x46, 0x93, 0x8b, 0x93, 0x36, 0x22, 0x71, 0xd2, 0xed, 0x71, 0x38, 0xa9, + 0x6f, 0xe2, 0xe3, 0xce, 0xc1, 0x2f, 0x4d, 0x0e, 0x94, 0x3e, 0x3a, 0x3f, 0x50, 0xf2, 0xde, 0x39, + 0x06, 0x29, 0x7d, 0xda, 0x8f, 0x94, 0xc4, 0x51, 0x48, 0xc9, 0xef, 0x59, 0xe7, 0x83, 0x4a, 0xeb, + 0xa9, 0xf4, 0xdb, 0xc5, 0x77, 0xc4, 0x3f, 0x9b, 0x81, 0x99, 0x9a, 0x17, 0x98, 0xe4, 0xd7, 0x52, + 0x78, 0x93, 0x93, 0xb0, 0xd0, 0x1a, 0x19, 0xf1, 0xd4, 0x6e, 0x8e, 0x3f, 0x3f, 0x71, 0xf0, 0x84, + 0x3f, 0xce, 0xfa, 0x06, 0x5b, 0xd2, 0xd1, 0x07, 0x90, 0xeb, 0xd9, 0xd8, 0x92, 0xbb, 0x96, 0x6e, + 0x5a, 0xba, 0xc3, 0xb6, 0xdd, 0x08, 0xab, 0xc5, 0x2f, 0x4f, 0x17, 0xe7, 0x76, 0x6c, 0x6c, 0x6d, + 0x73, 0xba, 0x34, 0xd7, 0x0b, 0x3c, 0xb9, 0xf7, 0x92, 0x4d, 0x4f, 0x7e, 0x2f, 0xd9, 0x0b, 0x28, + 0x5a, 0x58, 0xd1, 0x42, 0x1e, 0x0c, 0x3b, 0x75, 0x2a, 0xba, 0xcf, 0xd1, 0x6d, 0x71, 0x6e, 0x4e, + 0x7a, 0xfa, 0x54, 0xc1, 0x0a, 0x13, 0xd1, 0x03, 0xb8, 0xd8, 0x51, 0x8e, 0x69, 0x10, 0xab, 0xec, + 0x3a, 0x85, 0x34, 0x30, 0x95, 0x5d, 0x26, 0x86, 0x3a, 0xca, 0x31, 0xbd, 0xe4, 0x8c, 0x25, 0xd1, + 0x5b, 0x4f, 0x6e, 0x41, 0x5e, 0xd3, 0x6d, 0x47, 0x37, 0x54, 0x87, 0x1f, 0x68, 0xcc, 0x0e, 0x03, + 0xce, 0xb9, 0x54, 0x76, 0x6a, 0x71, 0x05, 0x0a, 0x2d, 0xc5, 0xc1, 0x47, 0xca, 0x89, 0xec, 0xee, + 0x87, 0xcb, 0xd2, 0xb3, 0x3b, 0xdf, 0x3a, 0x3b, 0x5d, 0xcc, 0x3d, 0x65, 0x49, 0x03, 0xdb, 0xe2, + 0x72, 0xad, 0x40, 0x82, 0x86, 0xee, 0x40, 0x41, 0xb1, 0x4f, 0x0c, 0x95, 0x56, 0x19, 0x1b, 0x76, + 0xcf, 0xa6, 0x30, 0x23, 0x2d, 0xe5, 0x29, 0xb9, 0xe2, 0x52, 0xd1, 0x75, 0x98, 0xe3, 0xe1, 0xfc, + 0xec, 0x76, 0xa5, 0x02, 0x2d, 0x3e, 0xbf, 0xb6, 0x83, 0x5d, 0xb0, 0xf4, 0x18, 0xca, 0xfc, 0x12, + 0x82, 0x23, 0xc5, 0xd2, 0x64, 0xaa, 0x49, 0xbf, 0xcf, 0x15, 0xa9, 0xd8, 0xcb, 0xec, 0xd2, 0x01, + 0x92, 0x81, 0xa8, 0xcf, 0x1f, 0x17, 0x4d, 0x98, 0x57, 0xdb, 0xba, 0x87, 0x0a, 0x98, 0x13, 0x35, + 0x3f, 0x74, 0x84, 0x54, 0x68, 0xde, 0xfe, 0xcf, 0xcc, 0x05, 0x35, 0x4c, 0x46, 0x0d, 0xa0, 0x27, + 0xf7, 0xc8, 0x5d, 0xb3, 0xad, 0xab, 0x27, 0x14, 0x16, 0x84, 0x8f, 0x54, 0x1f, 0x79, 0xb1, 0xc1, + 0x2b, 0x45, 0x77, 0xb6, 0x29, 0xa7, 0x04, 0x47, 0xde, 0x6f, 0x76, 0x1c, 0xf2, 0x7a, 0x2a, 0x0d, + 0xc5, 0xec, 0x7a, 0x2a, 0x3d, 0x57, 0xcc, 0xad, 0xa7, 0xd2, 0xf9, 0x62, 0x41, 0xfc, 0x5b, 0x02, + 0x14, 0xfa, 0xca, 0x85, 0x6a, 0x70, 0x51, 0xf3, 0x86, 0x82, 0xcc, 0xf7, 0x7e, 0xe9, 0xa6, 0xc1, + 0xcf, 0x8a, 0x5f, 0xf8, 0xf2, 0x74, 0xb1, 0x40, 0x73, 0x3f, 0xf5, 0x92, 0xa4, 0x0b, 0x3e, 0x87, + 0x4f, 0x45, 0x1f, 0x41, 0x9e, 0xb9, 0x97, 0xde, 0x45, 0x7f, 0x34, 0x68, 0x7f, 0x75, 0xfe, 0xcb, + 0xd3, 0xc5, 0x1c, 0xf5, 0x29, 0xdd, 0xb3, 0x9c, 0xa5, 0x5c, 0x3b, 0xf8, 0x28, 0xfe, 0x9a, 0x00, + 0x73, 0xa1, 0xfd, 0x55, 0x8f, 0xfb, 0x3e, 0x6e, 0x5f, 0x89, 0xc6, 0xa5, 0xc3, 0x22, 0x1c, 0xd3, + 0xbc, 0x1f, 0xbb, 0xe1, 0xa2, 0x8b, 0xc3, 0x71, 0x0d, 0x5d, 0xa5, 0x71, 0x63, 0x67, 0x5c, 0xb6, + 0x8f, 0x53, 0x3f, 0xf8, 0xe1, 0xe2, 0x94, 0xf8, 0x17, 0x29, 0xc8, 0x85, 0x77, 0x53, 0xd5, 0xfb, + 0xca, 0x15, 0x35, 0x6f, 0x84, 0x38, 0x96, 0x46, 0xdc, 0x28, 0x93, 0xf1, 0x2f, 0x65, 0x60, 0xc5, + 0xbc, 0x36, 0xe2, 0x13, 0x7e, 0xb0, 0x9c, 0x3e, 0x63, 0xf9, 0x3f, 0x26, 0x3d, 0xfb, 0xb9, 0x04, + 0xd3, 0xf4, 0x60, 0x24, 0x5e, 0xb4, 0x52, 0x7f, 0x1f, 0x22, 0xbe, 0x34, 0x49, 0x97, 0x58, 0x36, + 0x62, 0x6f, 0x9b, 0x6f, 0x74, 0xf2, 0xa0, 0x3f, 0x24, 0xce, 0x7f, 0xaf, 0x62, 0x8f, 0x9d, 0x3c, + 0xf9, 0xff, 0x30, 0xbe, 0x86, 0xbc, 0x0f, 0xfd, 0x02, 0x14, 0x54, 0xb3, 0xdd, 0x66, 0x73, 0x29, + 0xb3, 0x58, 0x83, 0x67, 0xd1, 0xd0, 0x22, 0xf0, 0xab, 0x34, 0x97, 0xbc, 0x2b, 0x35, 0x97, 0x24, + 0x7e, 0xa5, 0x66, 0x20, 0x78, 0x37, 0xef, 0x09, 0x73, 0x0d, 0x5d, 0x28, 0x8e, 0x78, 0xf6, 0x4d, + 0xe2, 0x88, 0x59, 0xec, 0x39, 0xef, 0x79, 0x7f, 0x24, 0xf0, 0x40, 0x9d, 0x67, 0xa6, 0x79, 0xd0, + 0xf3, 0xe2, 0x7f, 0xcb, 0xc1, 0x73, 0x24, 0xd3, 0x5f, 0x9e, 0x2e, 0xa6, 0x24, 0xef, 0x20, 0xc9, + 0xa8, 0x99, 0x21, 0xf1, 0xd5, 0x66, 0x86, 0xeb, 0x30, 0xd7, 0xb5, 0xf0, 0x1e, 0x76, 0xd4, 0x7d, + 0xd9, 0xe8, 0x75, 0xf8, 0x36, 0xa1, 0xac, 0x4b, 0xdb, 0xea, 0x75, 0xd0, 0x3d, 0x28, 0x7a, 0x59, + 0x38, 0x86, 0x77, 0x0f, 0x1a, 0x73, 0xe9, 0x1c, 0xf1, 0x8b, 0xff, 0x4b, 0x80, 0x85, 0x50, 0x9d, + 0xf8, 0x98, 0x5a, 0x87, 0xac, 0x6f, 0x4e, 0xec, 0x92, 0x70, 0xce, 0x68, 0xd8, 0x20, 0x33, 0x92, + 0xe1, 0x92, 0xfb, 0x5a, 0x7a, 0x91, 0x81, 0x2f, 0x36, 0x71, 0x4e, 0xb1, 0x17, 0x7d, 0x39, 0x6b, + 0x81, 0x17, 0x78, 0x83, 0x2c, 0x39, 0xd1, 0x20, 0x13, 0x7f, 0x53, 0x80, 0x22, 0x7d, 0xc1, 0x13, + 0x8c, 0xb5, 0x58, 0xac, 0x9b, 0x1b, 0x65, 0x9e, 0x98, 0x7c, 0x23, 0x50, 0xe8, 0xf2, 0x95, 0x64, + 0xf8, 0xf2, 0x15, 0xf1, 0x87, 0x02, 0xe4, 0xbd, 0x12, 0xb2, 0x0b, 0x0f, 0x47, 0x1c, 0x57, 0xfa, + 0x66, 0x97, 0xfa, 0xb9, 0xc7, 0xaa, 0x4c, 0x74, 0x07, 0x63, 0xf0, 0x58, 0x15, 0x76, 0x19, 0xdd, + 0xdf, 0x71, 0x7b, 0x0e, 0x29, 0x62, 0xc5, 0x3f, 0x3e, 0xe3, 0x0d, 0xf6, 0x44, 0x49, 0xf4, 0x5e, + 0x57, 0xb3, 0x7d, 0xc8, 0x4e, 0xc2, 0x99, 0xc8, 0xec, 0x21, 0x1e, 0x9e, 0x06, 0x7c, 0xb5, 0x4f, + 0x6b, 0x36, 0xe8, 0x8d, 0xaf, 0xec, 0xb7, 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x36, 0x3e, 0xd1, 0xd2, + 0x44, 0xa6, 0xd8, 0xd5, 0x12, 0xeb, 0x2b, 0x7f, 0x10, 0x6c, 0x89, 0xea, 0x21, 0x41, 0x79, 0x8f, + 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x45, 0x78, 0x85, 0x5a, 0x4e, 0x22, 0xb9, 0xd1, 0x93, 0xd0, 0xa9, + 0x23, 0x89, 0xe1, 0x88, 0x64, 0x50, 0xa5, 0xa1, 0xd3, 0x49, 0x3e, 0x0c, 0xf7, 0xf5, 0x91, 0xaf, + 0x0f, 0x76, 0xfa, 0x8f, 0x53, 0x3f, 0xfa, 0xe1, 0xa2, 0x20, 0x7e, 0x02, 0x48, 0xc2, 0x36, 0x76, + 0x5e, 0xf4, 0x4c, 0xcb, 0x3f, 0xc1, 0xe5, 0x76, 0xdf, 0x6d, 0x34, 0xd3, 0xab, 0xd9, 0xb3, 0xa8, + 0x43, 0x9e, 0x2e, 0xc2, 0x42, 0x88, 0x9b, 0x19, 0x0b, 0xf1, 0x43, 0xb8, 0xf2, 0xd4, 0xb4, 0x6d, + 0xbd, 0x4b, 0xa0, 0x2d, 0x1d, 0x95, 0x64, 0x6a, 0xf1, 0xcc, 0x63, 0xba, 0x4b, 0x17, 0x39, 0x0c, + 0x66, 0x46, 0x32, 0x92, 0xf7, 0x2c, 0xfe, 0xae, 0x00, 0x97, 0x07, 0x39, 0x99, 0x96, 0xa3, 0xb6, + 0x70, 0xce, 0xaa, 0xa6, 0x7f, 0xe2, 0xdf, 0xf8, 0xde, 0xea, 0x66, 0x27, 0xee, 0x2a, 0x7f, 0xa7, + 0xdc, 0x51, 0xa8, 0xf9, 0xe0, 0xdb, 0xc9, 0xf3, 0x9c, 0xbc, 0xc9, 0xa8, 0xbe, 0x25, 0x49, 0x4d, + 0x66, 0x49, 0x9a, 0x50, 0x58, 0x37, 0x75, 0x83, 0x78, 0xc5, 0x6e, 0x7d, 0x57, 0x20, 0xbf, 0xab, + 0x1b, 0x8a, 0x75, 0x22, 0xbb, 0x81, 0x85, 0xc2, 0xb8, 0xc0, 0x42, 0x29, 0xc7, 0x38, 0xf8, 0xa3, + 0xf8, 0x63, 0x01, 0x8a, 0xbe, 0x58, 0x6e, 0x91, 0xdf, 0x05, 0x50, 0xdb, 0x3d, 0xdb, 0xc1, 0x96, + 0xdb, 0x4a, 0x73, 0x2c, 0x1c, 0xbf, 0xc2, 0xa8, 0xf5, 0x35, 0x29, 0xc3, 0x33, 0xd4, 0x35, 0x74, + 0x23, 0x7c, 0xda, 0xc5, 0xf4, 0x2a, 0x9c, 0x0d, 0x9c, 0x71, 0x41, 0x9a, 0xdd, 0x76, 0x4c, 0xcb, + 0xc3, 0x54, 0xbc, 0xd9, 0xdd, 0xf3, 0xfb, 0xe9, 0x26, 0x6d, 0x92, 0x6f, 0x05, 0xf2, 0xc4, 0x5d, + 0x38, 0xc4, 0x5e, 0x95, 0x52, 0xe3, 0xab, 0xc4, 0x38, 0xdc, 0x2a, 0xfd, 0x2b, 0xe2, 0xe4, 0xb2, + 0xd6, 0xf0, 0x5a, 0x78, 0x84, 0x45, 0x5b, 0x83, 0xb4, 0x73, 0x6c, 0xc8, 0x1d, 0xec, 0xdd, 0x79, + 0x73, 0x8e, 0xe3, 0xf8, 0x66, 0x1d, 0xf6, 0x48, 0xaf, 0x45, 0xe4, 0xf7, 0x67, 0xf3, 0xe1, 0x72, + 0x65, 0x89, 0x5d, 0xb0, 0xbd, 0xe4, 0x5e, 0xb0, 0xbd, 0xb4, 0xc6, 0x33, 0x30, 0xa3, 0xfe, 0x83, + 0xff, 0xba, 0x28, 0x48, 0x1e, 0x13, 0x9b, 0xf7, 0xef, 0x37, 0x48, 0xaf, 0x1f, 0x98, 0x99, 0x51, + 0x1e, 0x20, 0x70, 0x99, 0x11, 0xbf, 0xb2, 0x79, 0x65, 0x4d, 0xde, 0xd9, 0xaa, 0x3c, 0xdf, 0xdc, + 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x17, 0xba, 0x0a, 0x29, 0xc1, 0x2e, 0x71, + 0xbe, 0xff, 0x57, 0x00, 0xfc, 0x5b, 0xd5, 0x88, 0xac, 0x8d, 0xea, 0x67, 0xf2, 0xcb, 0x95, 0x67, + 0x3b, 0xd5, 0x46, 0x71, 0x0a, 0x21, 0xc8, 0xaf, 0xae, 0x34, 0x2b, 0x35, 0x59, 0xaa, 0x36, 0xb6, + 0x9f, 0x6f, 0x35, 0xaa, 0xee, 0xe5, 0xcf, 0xf7, 0xd7, 0x60, 0x2e, 0x78, 0xa6, 0x11, 0x5a, 0x80, + 0x42, 0xa5, 0x56, 0xad, 0x6c, 0xc8, 0x2f, 0xeb, 0x2b, 0xf2, 0x8b, 0x9d, 0xea, 0x4e, 0xb5, 0x38, + 0x45, 0x8b, 0x46, 0x89, 0x4f, 0x76, 0x9e, 0x3d, 0x2b, 0x0a, 0xa8, 0x00, 0x59, 0xf6, 0x4c, 0xaf, + 0x4d, 0x2a, 0x26, 0xee, 0x6f, 0x42, 0x36, 0x70, 0xf8, 0x31, 0x79, 0xdd, 0xf6, 0x4e, 0xa3, 0x26, + 0x37, 0xeb, 0x9b, 0xd5, 0x46, 0x73, 0x65, 0x73, 0x9b, 0xc9, 0xa0, 0xb4, 0x95, 0xd5, 0xe7, 0x52, + 0xb3, 0x28, 0x78, 0xcf, 0xcd, 0xe7, 0x3b, 0x95, 0x9a, 0x5b, 0x0d, 0x31, 0x95, 0x4e, 0x16, 0x93, + 0xf7, 0xff, 0xba, 0x00, 0x97, 0x87, 0x9c, 0xef, 0x83, 0xb2, 0x30, 0xbb, 0x63, 0xd0, 0x93, 0x60, + 0x8b, 0x53, 0x28, 0x17, 0x38, 0xe2, 0xa7, 0x28, 0xa0, 0x34, 0x3b, 0x64, 0xa5, 0x98, 0x40, 0x33, + 0x90, 0x68, 0x3c, 0x2a, 0x26, 0x49, 0x49, 0x03, 0x27, 0xe4, 0x14, 0x53, 0x28, 0xc3, 0xcf, 0xe6, + 0x28, 0x4e, 0xa3, 0x39, 0xff, 0x88, 0x8c, 0xe2, 0x0c, 0x11, 0xe5, 0x1d, 0x35, 0x51, 0x9c, 0xbd, + 0x7f, 0x1d, 0x02, 0xdb, 0xf9, 0x11, 0xc0, 0xcc, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, 0x42, 0xb3, + 0x90, 0x5c, 0x69, 0xb7, 0x8b, 0xc2, 0xc3, 0x7f, 0x99, 0x82, 0xb4, 0x7b, 0x3d, 0x10, 0x7a, 0x06, + 0xd3, 0x6c, 0xb9, 0x7b, 0x71, 0x38, 0x5a, 0xa0, 0x03, 0xba, 0x7c, 0x6d, 0x1c, 0x9c, 0x10, 0xa7, + 0xd0, 0x5f, 0x85, 0x6c, 0xc0, 0x8b, 0x42, 0x43, 0x97, 0xec, 0x42, 0x9e, 0x63, 0xf9, 0xf6, 0xb8, + 0x6c, 0x9e, 0xfc, 0x57, 0x90, 0xf1, 0xac, 0x3a, 0xba, 0x31, 0xca, 0xe6, 0xbb, 0xb2, 0x47, 0x4f, + 0x0c, 0x64, 0xfc, 0x89, 0x53, 0xef, 0x0b, 0xc8, 0x02, 0x34, 0x68, 0x80, 0x51, 0x54, 0x10, 0xc5, + 0x50, 0x0b, 0x5f, 0xbe, 0x3f, 0x51, 0x6e, 0xff, 0x9d, 0x44, 0x59, 0xfe, 0x2c, 0x12, 0xad, 0xac, + 0x81, 0x39, 0x2a, 0x5a, 0x59, 0x11, 0x93, 0xd1, 0x14, 0x7a, 0x01, 0x29, 0x62, 0x3d, 0x51, 0x94, + 0x5f, 0xd9, 0x67, 0xad, 0xcb, 0x37, 0x46, 0xe6, 0x71, 0x45, 0xae, 0xde, 0xfb, 0xd1, 0x9f, 0x5e, + 0x9d, 0xfa, 0xd1, 0xd9, 0x55, 0xe1, 0xc7, 0x67, 0x57, 0x85, 0x3f, 0x3e, 0xbb, 0x2a, 0xfc, 0xc9, + 0xd9, 0x55, 0xe1, 0xfb, 0x3f, 0xb9, 0x3a, 0xf5, 0xe3, 0x9f, 0x5c, 0x9d, 0xfa, 0xe3, 0x9f, 0x5c, + 0x9d, 0xfa, 0x7c, 0x96, 0x73, 0xef, 0xce, 0x50, 0xd3, 0xf2, 0xe8, 0xff, 0x06, 0x00, 0x00, 0xff, + 0xff, 0xa7, 0x2a, 0x1d, 0x85, 0xf6, 0x80, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 63421d4fdf7e..f3d011c9646b 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -104,12 +104,7 @@ message ResponseHeader { // measures the bytes in the timestamps, keys, and values of the returned // rows. int64 num_bytes = 8; - // Range or list of ranges used to execute the request. Multiple - // ranges may be returned for Scan, ReverseScan or DeleteRange. - // - // This field is deprecated in 20.2 in favor of - // BatchResponse.Header.range_infos. - repeated RangeInfo deprecated_range_infos = 6 [(gogoproto.nullable) = false]; + reserved 6; } // A GetRequest is the argument for the Get() method. @@ -1677,9 +1672,8 @@ message RangeStatsResponse { // QueriesPerSecond is the rate of request/s or QPS for the range. double queries_per_second = 3; - // range_info contains descriptor and lease information. Added in 20.2. - // TODO(andrei): Make non-nullable in 21.1. - RangeInfo range_info = 4; + // range_info contains descriptor and lease information. + RangeInfo range_info = 4 [(gogoproto.nullable) = false]; } // MigrateRequest is used instruct all ranges overlapping with it to exercise @@ -1936,20 +1930,11 @@ message Header { // EndTxnRequest. Currently set conservatively: a request might be // composed of distinct spans yet have this field set to false. bool distinct_spans = 9; - // If set, return_range_info causes RangeInfo details to be returned with - // each ResponseHeader. - // TODO(andrei): Remove straggler users of this field, then delete the field - // in 21.1 now that we have client_range_info. - bool return_range_info = 10; // client_range_info represents the kvclient's knowledge about the state of // the range (i.e. of the range descriptor and lease). The kvserver checks // whether the client's info is up to date and, if it isn't, it will return a - // RangeInfo with up-to-date information. This field supersedes return_range_info. - // - // The field is nil for RPCs coming from 20.1 clients. For 20.2+ clients, at - // least the descriptor info is filled in. - // TODO(andrei): Make the field non-nullable in 21.1. - ClientRangeInfo client_range_info = 17; + // RangeInfo with up-to-date information. + ClientRangeInfo client_range_info = 17 [(gogoproto.nullable) = false]; // gateway_node_id is the ID of the gateway node where the request originated. int32 gateway_node_id = 11 [(gogoproto.customname) = "GatewayNodeID", (gogoproto.casttype) = "NodeID"]; // If set, the request will return to the client before proposing the @@ -1973,7 +1958,7 @@ message Header { // That flag should be deprecated in favor of this one. // TODO(nvanbenschoten): perform this migration. bool can_forward_read_timestamp = 16; - reserved 7, 12, 14; + reserved 7, 10, 12, 14; } // ClientRangeInfo represents the kvclient's knowledge about the state of the @@ -2037,21 +2022,18 @@ message BatchResponse { // collected_spans stores trace spans recorded during the execution of this // request. repeated util.tracing.tracingpb.RecordedSpan collected_spans = 6 [(gogoproto.nullable) = false]; - // Range or list of ranges used to execute the request. The server only - // populates this if return_range_info is set on the request, or if the - // server detects the client's client_range_info to be stale. - // - // For one batch, the kvserver returns at most one element. The DistSender - // will combine multiple responses though when it reassembles a split batch; - // in this case the RangeInfos are maintained in sorted order by range id. + // Range used to execute the request. The server only populates this if the + // server detects the client's client_range_info to be stale. Otherwise, it + // is left empty. Not set when Error is set. // - // Not set when Error is set. + // The server may also include additional RangeInfo objects if it suspects + // that the client may be interested in them. This is currently the case + // immediately after a Range split, where a stale client_range_info from + // before the split is taken to be an indication that the client may be + // interested in information about both the LHS and RHS post-split ranges. // - // TODO(andrei): In 21.1 there should be no users of this field left above - // the DistSender. At that point, the DistSender will no longer need to - // combine these - in fact it should clear the field. And then the field - // doesn't need to be `repeated` any more - the proto encoding - // allows us to change it to non-repeated. + // The field is cleared by the DistSender because it refers routing + // information not exposed by the KV API. repeated RangeInfo range_infos = 7 [(gogoproto.nullable) = false]; // NB: if you add a field here, don't forget to update combine(). }