diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index f38c81110a28..423d997b083f 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -99,4 +99,4 @@ timeseries.storage.resolution_30m.ttl duration 2160h0m0s the maximum age of time trace.debug.enable boolean false if set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.token string if set, traces go to Lightstep using this token trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -version version 20.2-42 set the active cluster version in the format '.' +version version 20.2-44 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index fed2b0fe760a..d3050343ebda 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -101,6 +101,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versionversion20.2-42set the active cluster version in the format '.' +versionversion20.2-44set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 9ad9ebceee4d..3a987c97dc0c 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -266,6 +266,10 @@ const ( // ForeignKeyRepresentationMigration is used to ensure that all no table // descriptors use the pre-19.2 foreign key migration. ForeignKeyRepresentationMigration + // PriorReadSummaries introduces support for the use of read summary objects + // to ship information about reads on a range through lease changes and + // range merges. + PriorReadSummaries // Step (1): Add new versions here. ) @@ -452,11 +456,14 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: NamespaceTableWithSchemasMigration, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 40}, }, - { Key: ForeignKeyRepresentationMigration, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 42}, }, + { + Key: PriorReadSummaries, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 44}, + }, // Step (2): Add new versions here. }) diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 0edb624b41de..6afb398956ab 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -49,11 +49,12 @@ func _() { _ = x[ChangefeedsSupportPrimaryIndexChanges-38] _ = x[NamespaceTableWithSchemasMigration-39] _ = x[ForeignKeyRepresentationMigration-40] + _ = x[PriorReadSummaries-41] } -const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsSequencesRegclassImplicitColumnPartitioningMultiRegionFeaturesClosedTimestampsRaftTransportChangefeedsSupportPrimaryIndexChangesNamespaceTableWithSchemasMigrationForeignKeyRepresentationMigration" +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationSeparatedIntentsTracingVerbosityIndependentSemanticsSequencesRegclassImplicitColumnPartitioningMultiRegionFeaturesClosedTimestampsRaftTransportChangefeedsSupportPrimaryIndexChangesNamespaceTableWithSchemasMigrationForeignKeyRepresentationMigrationPriorReadSummaries" -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, 542, 592, 608, 629, 667, 709, 725, 761, 778, 804, 823, 852, 889, 923, 956} +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, 542, 592, 608, 629, 667, 709, 725, 761, 778, 804, 823, 852, 889, 923, 956, 974} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 428fd0a3d466..203bc3621a6e 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -93,6 +93,9 @@ var ( // LocalLeaseAppliedIndexLegacySuffix is the suffix for the applied lease // index. LocalLeaseAppliedIndexLegacySuffix = []byte("rlla") + // LocalRangePriorReadSummarySuffix is the suffix for a range's prior read + // summary. + LocalRangePriorReadSummarySuffix = []byte("rprs") // LocalRangeVersionSuffix is the suffix for the range version. LocalRangeVersionSuffix = []byte("rver") // LocalRangeStatsLegacySuffix is the suffix for range statistics. diff --git a/pkg/keys/doc.go b/pkg/keys/doc.go index c2a94ba441d2..48bae63d796e 100644 --- a/pkg/keys/doc.go +++ b/pkg/keys/doc.go @@ -192,6 +192,7 @@ var _ = [...]interface{}{ RaftTruncatedStateLegacyKey, // "rftt" RangeLeaseKey, // "rll-" LeaseAppliedIndexLegacyKey, // "rlla" + RangePriorReadSummaryKey, // "rprs" RangeVersionKey, // "rver" RangeStatsLegacyKey, // "stat" diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index a329c09c366e..a0a1b421968e 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -271,6 +271,12 @@ func RangeLeaseKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RangeLeaseKey() } +// RangePriorReadSummaryKey returns a system-local key for a range's prior read +// summary. +func RangePriorReadSummaryKey(rangeID roachpb.RangeID) roachpb.Key { + return MakeRangeIDPrefixBuf(rangeID).RangePriorReadSummaryKey() +} + // RangeStatsLegacyKey returns the key for accessing the MVCCStats struct for // the specified Range ID. The key is no longer written to. Its responsibility // has been subsumed by the RangeAppliedStateKey. @@ -952,6 +958,12 @@ func (b RangeIDPrefixBuf) RangeLeaseKey() roachpb.Key { return append(b.replicatedPrefix(), LocalRangeLeaseSuffix...) } +// RangePriorReadSummaryKey returns a system-local key for a range's prior read +// summary. +func (b RangeIDPrefixBuf) RangePriorReadSummaryKey() roachpb.Key { + return append(b.replicatedPrefix(), LocalRangePriorReadSummarySuffix...) +} + // RangeStatsLegacyKey returns the key for accessing the MVCCStats struct // for the specified Range ID. // See comment on RangeStatsLegacyKey function. diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index 2ac5c3b8bdfb..5d97ca551dae 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -169,6 +169,7 @@ var ( {name: "RaftTruncatedState", suffix: LocalRaftTruncatedStateLegacySuffix}, {name: "RangeLastReplicaGCTimestamp", suffix: LocalRangeLastReplicaGCTimestampSuffix}, {name: "RangeLease", suffix: LocalRangeLeaseSuffix}, + {name: "RangePriorReadSummary", suffix: LocalRangePriorReadSummarySuffix}, {name: "RangeStats", suffix: LocalRangeStatsLegacySuffix}, {name: "RangeLastGC", suffix: LocalRangeLastGCSuffix}, {name: "RangeVersion", suffix: LocalRangeVersionSuffix}, diff --git a/pkg/keys/printer_test.go b/pkg/keys/printer_test.go index 655b857f0f8b..8fc686ed6212 100644 --- a/pkg/keys/printer_test.go +++ b/pkg/keys/printer_test.go @@ -74,6 +74,7 @@ func TestPrettyPrint(t *testing.T) { {keys.RaftTruncatedStateLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RaftTruncatedState", revertSupportUnknown}, {keys.RaftTruncatedStateKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RaftTruncatedState", revertSupportUnknown}, {keys.RangeLeaseKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeLease", revertSupportUnknown}, + {keys.RangePriorReadSummaryKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangePriorReadSummary", revertSupportUnknown}, {keys.RangeStatsLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeStats", revertSupportUnknown}, {keys.RangeLastGCKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeLastGC", revertSupportUnknown}, {keys.RangeVersionKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeVersion", revertSupportUnknown}, diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index ca306e3d1fc1..24e3da6833b5 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -133,6 +133,8 @@ go_library( "//pkg/kv/kvserver/raftentry", "//pkg/kv/kvserver/rangefeed", "//pkg/kv/kvserver/rditer", + "//pkg/kv/kvserver/readsummary", + "//pkg/kv/kvserver/readsummary/rspb", "//pkg/kv/kvserver/spanset", "//pkg/kv/kvserver/split", "//pkg/kv/kvserver/stateloader", diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index abb56d4691be..2e5563b79daa 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -60,6 +60,8 @@ go_library( "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/rditer", + "//pkg/kv/kvserver/readsummary", + "//pkg/kv/kvserver/readsummary/rspb", "//pkg/kv/kvserver/spanset", "//pkg/kv/kvserver/stateloader", "//pkg/kv/kvserver/txnwait", @@ -106,11 +108,14 @@ go_test( "//pkg/kv", "//pkg/kv/kvserver", "//pkg/kv/kvserver/abortspan", + "//pkg/kv/kvserver/readsummary", + "//pkg/kv/kvserver/readsummary/rspb", "//pkg/kv/kvserver/spanset", "//pkg/roachpb", "//pkg/security", "//pkg/security/securitytest", "//pkg/server", + "//pkg/settings/cluster", "//pkg/storage", "//pkg/storage/enginepb", "//pkg/testutils", diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index b1b75dc34454..d43bcd90d607 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -159,6 +160,13 @@ func declareKeysEndTxn( Key: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID), EndKey: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID).PrefixEnd(), }) + // Merges incorporate the prior read summary from the RHS into + // the LHS, which ensures that the current and all future + // leaseholders on the joint range respect reads served on the + // RHS. + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{ + Key: keys.RangePriorReadSummaryKey(mt.LeftDesc.RangeID), + }) } } } @@ -1103,6 +1111,38 @@ func mergeTrigger( return result.Result{}, err } + // If we collected a read summary from the right-hand side when freezing it, + // merge that summary into the left-hand side's prior read summary. In the + // usual case, the RightReadSummary in the MergeTrigger will be used to + // update the left-hand side's leaseholder's timestamp cache when applying + // the merge trigger's Raft log entry. However, if the left-hand side's + // leaseholder hears about the merge through a Raft snapshot, the merge + // trigger will not be available, so it will need to use the range's prior + // read summary to update its timestamp cache to ensure that it does not + // serve any writes that invalidate previous reads served on the right-hand + // side range. See TestStoreRangeMergeTimestampCache for an example of where + // this behavior is necessary. + // + // This communication from the RHS to the LHS is handled differently from + // how we copy over the abortspan. In this case, the read summary is passed + // through the SubsumeResponse and into the MergeTrigger. In the abortspan's + // case, we read from local RHS replica (which may not be the leaseholder) + // directly in this method. The primary reason why these are different is + // because the RHS's persistent read summary may not be up-to-date, as it is + // not updated by the SubsumeRequest. + readSumActive := rec.ClusterSettings().Version.IsActive(ctx, clusterversion.PriorReadSummaries) + if merge.RightReadSummary != nil && readSumActive { + mergedSum := merge.RightReadSummary.Clone() + if priorSum, err := readsummary.Load(ctx, batch, rec.GetRangeID()); err != nil { + return result.Result{}, err + } else if priorSum != nil { + mergedSum.Merge(*priorSum) + } + if err := readsummary.Set(ctx, batch, rec.GetRangeID(), ms, mergedSum); err != nil { + return result.Result{}, err + } + } + // The stats for the merged range are the sum of the LHS and RHS stats, less // the RHS's replicated range ID stats. The only replicated range ID keys we // copy from the RHS are the keys in the abort span, and we've already diff --git a/pkg/kv/kvserver/batcheval/cmd_lease.go b/pkg/kv/kvserver/batcheval/cmd_lease.go index f2787a509ce6..68f9e2648cef 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease.go @@ -14,8 +14,11 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -52,6 +55,7 @@ func evalNewLease( ms *enginepb.MVCCStats, lease roachpb.Lease, prevLease roachpb.Lease, + priorReadSum *rspb.ReadSummary, isExtension bool, isTransfer bool, ) (result.Result, error) { @@ -125,6 +129,21 @@ func evalNewLease( } pd.Replicated.PrevLeaseProposal = prevLease.ProposedTS + // If we're setting a new prior read summary, store it to disk & in-memory. + // We elide this step in mixed-version clusters as old nodes would ignore + // the PriorReadSummary field (they don't know about it). It's possible that + // in this particular case we could get away with it (as the in-mem field + // only ever updates in-mem state) but it's easy to get things wrong (in + // which case they could easily take a catastrophic turn) and the benefit is + // low. + readSumActive := rec.ClusterSettings().Version.IsActive(ctx, clusterversion.PriorReadSummaries) + if priorReadSum != nil && readSumActive { + if err := readsummary.Set(ctx, readWriter, rec.GetRangeID(), ms, priorReadSum); err != nil { + return newFailedLeaseTrigger(isTransfer), err + } + pd.Replicated.PriorReadSummary = priorReadSum + } + pd.Local.Metrics = new(result.Metrics) if isTransfer { pd.Local.Metrics.LeaseTransferSuccess = 1 diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_request.go b/pkg/kv/kvserver/batcheval/cmd_lease_request.go index 297f44a5a59d..13332bdc498f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_request.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_request.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -33,6 +34,7 @@ func declareKeysRequestLease( // (see concurrency.shouldAcquireLatches). However, we continue to // declare the keys in order to appease SpanSet assertions under race. latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())}) + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangePriorReadSummaryKey(rs.GetRangeID())}) latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())}) } @@ -134,6 +136,22 @@ func RequestLease( return newFailedLeaseTrigger(false /* isTransfer */), rErr } newLease.Start = effectiveStart + + var priorReadSum *rspb.ReadSummary + if !prevLease.Equivalent(newLease) { + // If the new lease is not equivalent to the old lease (i.e. either the + // lease is changing hands or the leaseholder restarted), construct a + // read summary to instruct the new leaseholder on how to update its + // timestamp cache. Since we are not the leaseholder ourselves, we must + // pessimistically assume that prior leaseholders served reads all the + // way up to the start of the new lease. + // + // NB: this is equivalent to the leaseChangingHands condition in + // leasePostApplyLocked. + worstCaseSum := rspb.FromTimestamp(newLease.Start.ToTimestamp()) + priorReadSum = &worstCaseSum + } + return evalNewLease(ctx, cArgs.EvalCtx, readWriter, cArgs.Stats, - newLease, prevLease, isExtension, false /* isTransfer */) + newLease, prevLease, priorReadSum, isExtension, false /* isTransfer */) } diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_test.go b/pkg/kv/kvserver/batcheval/cmd_lease_test.go index 65446388ad51..1e4bfdc7eaaf 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_test.go @@ -16,7 +16,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -171,53 +174,79 @@ func TestLeaseTransferForwardsStartTime(t *testing.T) { defer log.Scope(t).Close(t) testutils.RunTrueAndFalse(t, "epoch", func(t *testing.T, epoch bool) { - ctx := context.Background() - db := storage.NewDefaultInMemForTesting() - defer db.Close() - batch := db.NewBatch() - defer batch.Close() + testutils.RunTrueAndFalse(t, "served-future-reads", func(t *testing.T, servedFutureReads bool) { + ctx := context.Background() + db := storage.NewDefaultInMemForTesting() + defer db.Close() + batch := db.NewBatch() + defer batch.Close() - replicas := []roachpb.ReplicaDescriptor{ - {NodeID: 1, StoreID: 1, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 1}, - {NodeID: 2, StoreID: 2, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 2}, - } - desc := roachpb.RangeDescriptor{} - desc.SetReplicas(roachpb.MakeReplicaSet(replicas)) - manual := hlc.NewManualClock(123) - clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) + replicas := []roachpb.ReplicaDescriptor{ + {NodeID: 1, StoreID: 1, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 1}, + {NodeID: 2, StoreID: 2, Type: roachpb.ReplicaTypeVoterFull(), ReplicaID: 2}, + } + desc := roachpb.RangeDescriptor{} + desc.SetReplicas(roachpb.MakeReplicaSet(replicas)) + manual := hlc.NewManualClock(123) + clock := hlc.NewClock(manual.UnixNano, time.Nanosecond) - nextLease := roachpb.Lease{ - Replica: replicas[1], - Start: clock.NowAsClockTimestamp(), - } - if epoch { - nextLease.Epoch = 1 - } else { - exp := nextLease.Start.ToTimestamp().Add(9*time.Second.Nanoseconds(), 0) - nextLease.Expiration = &exp - } - cArgs := CommandArgs{ - EvalCtx: (&MockEvalCtx{ - StoreID: 1, - Desc: &desc, - Clock: clock, - }).EvalContext(), - Args: &roachpb.TransferLeaseRequest{ - Lease: nextLease, - }, - } + nextLease := roachpb.Lease{ + Replica: replicas[1], + Start: clock.NowAsClockTimestamp(), + } + if epoch { + nextLease.Epoch = 1 + } else { + exp := nextLease.Start.ToTimestamp().Add(9*time.Second.Nanoseconds(), 0) + nextLease.Expiration = &exp + } + + var maxPriorReadTS hlc.Timestamp + if servedFutureReads { + maxPriorReadTS = nextLease.Start.ToTimestamp().Add(1*time.Second.Nanoseconds(), 0) + } else { + maxPriorReadTS = nextLease.Start.ToTimestamp().Add(-2*time.Second.Nanoseconds(), 0) + } + currentReadSummary := rspb.FromTimestamp(maxPriorReadTS) + + cArgs := CommandArgs{ + EvalCtx: (&MockEvalCtx{ + ClusterSettings: cluster.MakeTestingClusterSettings(), + StoreID: 1, + Desc: &desc, + Clock: clock, + CurrentReadSummary: currentReadSummary, + }).EvalContext(), + Args: &roachpb.TransferLeaseRequest{ + Lease: nextLease, + }, + } + + manual.Increment(1000) + beforeEval := clock.NowAsClockTimestamp() - manual.Increment(1000) - beforeEval := clock.NowAsClockTimestamp() + res, err := TransferLease(ctx, batch, cArgs, nil) + require.NoError(t, err) - res, err := TransferLease(ctx, batch, cArgs, nil) - require.NoError(t, err) + // The proposed lease start time should be assigned at eval time. + propLease := res.Replicated.State.Lease + require.NotNil(t, propLease) + require.True(t, nextLease.Start.Less(propLease.Start)) + require.True(t, beforeEval.Less(propLease.Start)) - // The proposed lease start time should be assigned at eval time. - propLease := res.Replicated.State.Lease - require.NotNil(t, propLease) - require.True(t, nextLease.Start.Less(propLease.Start)) - require.True(t, beforeEval.Less(propLease.Start)) + // The prior read summary should reflect the maximum read times + // served under the current leaseholder. + propReadSum, err := readsummary.Load(ctx, batch, desc.RangeID) + require.NoError(t, err) + require.NotNil(t, propReadSum, "should write prior read summary") + if servedFutureReads { + require.Equal(t, maxPriorReadTS, propReadSum.Local.LowWater) + require.Equal(t, maxPriorReadTS, propReadSum.Global.LowWater) + } else { + require.Equal(t, propLease.Start.ToTimestamp(), propReadSum.Local.LowWater) + require.Equal(t, propLease.Start.ToTimestamp(), propReadSum.Global.LowWater) + } + }) }) } diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go b/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go index 0c6d7c3a7d7d..badf400f007d 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_transfer.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -76,6 +77,23 @@ func TransferLease( newLease.Start.Forward(cArgs.EvalCtx.Clock().NowAsClockTimestamp()) args.Lease = roachpb.Lease{} // prevent accidental use below + // Collect a read summary from the outgoing leaseholder to ship to the + // incoming leaseholder. This is used to instruct the new leaseholder on how + // to update its timestamp cache to ensure that no future writes are allowed + // to invalidate prior reads. + priorReadSum := cArgs.EvalCtx.GetCurrentReadSummary() + // For now, forward this summary to the proposed lease's start time. This + // may appear to undermine the benefit of the read summary, but it doesn't + // entirely. Until we ship higher-resolution read summaries, the read + // summary doesn't provide much value in avoiding transaction retries, but + // it is necessary for correctness if the outgoing leaseholder has served + // reads at future times above the proposed lease start time. + // + // We can remove this in the future when we increase the resolution of read + // summaries and have a per-range closed timestamp system that is easier to + // think about. + priorReadSum.Merge(rspb.FromTimestamp(newLease.Start.ToTimestamp())) + // If this check is removed at some point, the filtering of learners on the // sending side would have to be removed as well. if err := roachpb.CheckCanReceiveLease(newLease.Replica, cArgs.EvalCtx.Desc()); err != nil { @@ -84,5 +102,5 @@ func TransferLease( log.VEventf(ctx, 2, "lease transfer: prev lease: %+v, new lease: %+v", prevLease, newLease) return evalNewLease(ctx, cArgs.EvalCtx, readWriter, cArgs.Stats, - newLease, prevLease, false /* isExtension */, true /* isTransfer */) + newLease, prevLease, &priorReadSum, false /* isExtension */, true /* isTransfer */) } diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume.go b/pkg/kv/kvserver/batcheval/cmd_subsume.go index fad63a0f2b55..61610efa99ce 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume.go +++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -162,7 +163,24 @@ func Subsume( // ts cache if LHS/RHS leases are not collocated. The case when the leases are // collocated also works out because then the closed timestamp (according to // the old mechanism) is the same for both ranges being merged. - reply.ClosedTimestamp = cArgs.EvalCtx.FrozenClosedTimestamp(ctx) + reply.ClosedTimestamp = cArgs.EvalCtx.GetFrozenClosedTimestamp() + // Collect a read summary from the RHS leaseholder to ship to the LHS + // leaseholder. This is used to instruct the LHS on how to update its + // timestamp cache to ensure that no future writes are allowed to invalidate + // prior reads performed to this point on the RHS range. + priorReadSum := cArgs.EvalCtx.GetCurrentReadSummary() + // For now, forward this summary to the freeze time. This may appear to + // undermine the benefit of the read summary, but it doesn't entirely. Until + // we ship higher-resolution read summaries, the read summary doesn't + // provide much value in avoiding transaction retries, but it is necessary + // for correctness if the RHS has served reads at future times above the + // freeze time. + // + // We can remove this in the future when we increase the resolution of read + // summaries and have a per-range closed timestamp system that is easier to + // think about. + priorReadSum.Merge(rspb.FromTimestamp(reply.FreezeStart.ToTimestamp())) + reply.ReadSummary = &priorReadSum return result.Result{ Local: result.LocalResult{FreezeStart: reply.FreezeStart.ToTimestamp()}, diff --git a/pkg/kv/kvserver/batcheval/declare.go b/pkg/kv/kvserver/batcheval/declare.go index fdf46ac19430..75db92949645 100644 --- a/pkg/kv/kvserver/batcheval/declare.go +++ b/pkg/kv/kvserver/batcheval/declare.go @@ -94,7 +94,7 @@ func declareAllKeys(latchSpans *spanset.SpanSet) { // extend beyond the Range, but this is ok for the purpose of // acquiring latches. latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.LocalPrefix, EndKey: keys.LocalMax}) - latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.LocalMax, EndKey: keys.MaxKey}) + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey}) } // CommandArgs contains all the arguments to a command. diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index 7851f091b101..747e805b885d 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -68,7 +69,6 @@ type EvalContext interface { GetTerm(uint64) (uint64, error) GetLeaseAppliedIndex() uint64 GetTracker() closedts.TrackerI - FrozenClosedTimestamp(ctx context.Context) hlc.Timestamp Desc() *roachpb.RangeDescriptor ContainsKey(key roachpb.Key) bool @@ -97,6 +97,14 @@ type EvalContext interface { GetLastReplicaGCTimestamp(context.Context) (hlc.Timestamp, error) GetLease() (roachpb.Lease, roachpb.Lease) GetRangeInfo(context.Context) roachpb.RangeInfo + GetFrozenClosedTimestamp() hlc.Timestamp + + // GetCurrentReadSummary returns a new ReadSummary reflecting all reads + // served by the range to this point. The method requires a write latch + // across all keys in the range (see declareAllKeys), because it will only + // return a meaningful summary if the caller has serialized with all other + // requests on the range. + GetCurrentReadSummary() rspb.ReadSummary GetExternalStorage(ctx context.Context, dest roachpb.ExternalStorage) (cloud.ExternalStorage, error) GetExternalStorageFromURI(ctx context.Context, uri string, user security.SQLUsername) (cloud.ExternalStorage, @@ -106,17 +114,18 @@ type EvalContext interface { // MockEvalCtx is a dummy implementation of EvalContext for testing purposes. // For technical reasons, the interface is implemented by a wrapper .EvalContext(). type MockEvalCtx struct { - ClusterSettings *cluster.Settings - Desc *roachpb.RangeDescriptor - StoreID roachpb.StoreID - Clock *hlc.Clock - Stats enginepb.MVCCStats - QPS float64 - AbortSpan *abortspan.AbortSpan - GCThreshold hlc.Timestamp - Term, FirstIndex uint64 - CanCreateTxn func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) - Lease roachpb.Lease + ClusterSettings *cluster.Settings + Desc *roachpb.RangeDescriptor + StoreID roachpb.StoreID + Clock *hlc.Clock + Stats enginepb.MVCCStats + QPS float64 + AbortSpan *abortspan.AbortSpan + GCThreshold hlc.Timestamp + Term, FirstIndex uint64 + CanCreateTxn func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) + Lease roachpb.Lease + CurrentReadSummary rspb.ReadSummary } // EvalContext returns the MockEvalCtx as an EvalContext. It will reflect future @@ -185,7 +194,7 @@ func (m *mockEvalCtxImpl) GetLeaseAppliedIndex() uint64 { func (m *mockEvalCtxImpl) GetTracker() closedts.TrackerI { panic("unimplemented") } -func (m *mockEvalCtxImpl) FrozenClosedTimestamp(ctx context.Context) hlc.Timestamp { +func (m *mockEvalCtxImpl) GetFrozenClosedTimestamp() hlc.Timestamp { panic("unimplemented") } func (m *mockEvalCtxImpl) Desc() *roachpb.RangeDescriptor { @@ -217,13 +226,14 @@ func (m *mockEvalCtxImpl) GetLease() (roachpb.Lease, roachpb.Lease) { func (m *mockEvalCtxImpl) GetRangeInfo(ctx context.Context) roachpb.RangeInfo { return roachpb.RangeInfo{Desc: *m.Desc(), Lease: m.Lease} } - +func (m *mockEvalCtxImpl) GetCurrentReadSummary() rspb.ReadSummary { + return m.CurrentReadSummary +} func (m *mockEvalCtxImpl) GetExternalStorage( ctx context.Context, dest roachpb.ExternalStorage, ) (cloud.ExternalStorage, error) { panic("unimplemented") } - func (m *mockEvalCtxImpl) GetExternalStorageFromURI( ctx context.Context, uri string, user security.SQLUsername, ) (cloud.ExternalStorage, error) { diff --git a/pkg/kv/kvserver/batcheval/result/result.go b/pkg/kv/kvserver/batcheval/result/result.go index 32271101021c..7d444371cf86 100644 --- a/pkg/kv/kvserver/batcheval/result/result.go +++ b/pkg/kv/kvserver/batcheval/result/result.go @@ -294,6 +294,13 @@ func (p *Result) MergeAndDestroy(q Result) error { } q.Replicated.PrevLeaseProposal = nil + if p.Replicated.PriorReadSummary == nil { + p.Replicated.PriorReadSummary = q.Replicated.PriorReadSummary + } else if q.Replicated.PriorReadSummary != nil { + return errors.AssertionFailedf("conflicting prior read summary") + } + q.Replicated.PriorReadSummary = nil + if p.Local.EncounteredIntents == nil { p.Local.EncounteredIntents = q.Local.EncounteredIntents } else { diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index f4aba4633955..fb5e07729b4e 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" @@ -31,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -364,9 +366,84 @@ func TestCannotTransferLeaseToVoterOutgoing(t *testing.T) { } -// Test the error returned by attempts to create a txn record after a lease -// transfer. -func TestTimestampCacheErrorAfterLeaseTransfer(t *testing.T) { +// TestStoreLeaseTransferTimestampCacheRead verifies that the timestamp cache on +// the new leaseholder is properly updated after a lease transfer to prevent new +// writes from invalidating previously served reads. +func TestStoreLeaseTransferTimestampCacheRead(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunTrueAndFalse(t, "future-read", func(t *testing.T, futureRead bool) { + manualClock := hlc.NewHybridManualClock() + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + key := []byte("a") + rangeDesc, err := tc.LookupRange(key) + require.NoError(t, err) + + // Transfer the lease to Servers[0] so we start in a known state. Otherwise, + // there might be already a lease owned by a random node. + require.NoError(t, tc.TransferRangeLease(rangeDesc, tc.Target(0))) + + // Pause the cluster's clock. This ensures that if we perform a read at + // a future timestamp, the read time remains in the future, regardless + // of the passage of real time. + manualClock.Pause() + + // Write a key. + _, pErr := kv.SendWrapped(ctx, tc.Servers[0].DistSender(), incrementArgs(key, 1)) + require.Nil(t, pErr) + + // Determine when to read. + readTS := tc.Servers[0].Clock().Now() + if futureRead { + readTS = readTS.Add(500*time.Millisecond.Nanoseconds(), 0).WithSynthetic(true) + } + + // Read the key at readTS. + // NB: don't use SendWrapped because we want access to br.Timestamp. + var ba roachpb.BatchRequest + ba.Timestamp = readTS + ba.Add(getArgs(key)) + br, pErr := tc.Servers[0].DistSender().Send(ctx, ba) + require.Nil(t, pErr) + require.Equal(t, readTS, br.Timestamp) + v, err := br.Responses[0].GetGet().Value.GetInt() + require.NoError(t, err) + require.Equal(t, int64(1), v) + + // Transfer the lease. This should carry over a summary of the old + // leaseholder's timestamp cache to prevent any writes on the new + // leaseholder from writing under the previous read. + require.NoError(t, tc.TransferRangeLease(rangeDesc, tc.Target(1))) + + // Attempt to write under the read on the new leaseholder. The batch + // should get forwarded to a timestamp after the read. + // NB: don't use SendWrapped because we want access to br.Timestamp. + ba = roachpb.BatchRequest{} + ba.Timestamp = readTS + ba.Add(incrementArgs(key, 1)) + br, pErr = tc.Servers[0].DistSender().Send(ctx, ba) + require.Nil(t, pErr) + require.NotEqual(t, readTS, br.Timestamp) + require.True(t, readTS.Less(br.Timestamp)) + require.Equal(t, readTS.Synthetic, br.Timestamp.Synthetic) + }) +} + +// TestStoreLeaseTransferTimestampCacheTxnRecord checks the error returned by +// attempts to create a txn record after a lease transfer. +func TestStoreLeaseTransferTimestampCacheTxnRecord(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index 5f4231945164..b4390c522831 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -379,44 +380,123 @@ func mergeWithData(t *testing.T, retries int64) { } // TestStoreRangeMergeTimestampCache verifies that the timestamp cache on the -// LHS is properly updated after a merge. +// LHS is properly updated after a merge. The test contains a subtest for each +// of the combinations of the following boolean options: +// +// - disjointLeaseholders: configures whether or not the leaseholder of the +// LHS range is disjoint from the leaseholder of the RHS range. If false, +// the leaseholders are collocated before the merge is initiated. +// +// - throughSnapshot: configures whether or not the leaseholder of the LHS of +// the merge hears about and applies the merge through a Raft snapshot, as +// opposed to through normal Raft log application. +// +// - futureRead: configures whether or not the reads performed on the RHS range +// before the merge is initiated are performed in the future of present +// time using synthetic timestamps. +// func TestStoreRangeMergeTimestampCache(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + skip.UnderShort(t) - testutils.RunTrueAndFalse(t, "disjoint-leaseholders", mergeCheckingTimestampCaches) + testutils.RunTrueAndFalse(t, "disjoint-leaseholders", func(t *testing.T, disjointLeaseholders bool) { + testutils.RunTrueAndFalse(t, "through-snapshot", func(t *testing.T, throughSnapshot bool) { + testutils.RunTrueAndFalse(t, "future-read", func(t *testing.T, futureRead bool) { + mergeCheckingTimestampCaches(t, disjointLeaseholders, throughSnapshot, futureRead) + }) + }) + }) } -func mergeCheckingTimestampCaches(t *testing.T, disjointLeaseholders bool) { +func mergeCheckingTimestampCaches( + t *testing.T, disjointLeaseholders, throughSnapshot, futureRead bool, +) { + // mergeCommitFilter is used to issue a sequence of operations on the LHS of + // a range merge immediately before it. + var mergeCommitFilter func() + // blockHBAndGCs is used to black hole Heartbeat and GC requests for the + // duration of the merge on the throughSnapshot path. Neither request type + // is needed and both can create issues by holding latches during the split + // leader-leaseholder state. + var blockHBAndGCs chan struct{} + var filterMu syncutil.Mutex + testingRequestFilter := func(_ context.Context, ba roachpb.BatchRequest) *roachpb.Error { + filterMu.Lock() + mergeCommitFilterCopy := mergeCommitFilter + blockHBAndGCsCopy := blockHBAndGCs + filterMu.Unlock() + for _, req := range ba.Requests { + switch v := req.GetInner().(type) { + case *roachpb.EndTxnRequest: + if v.InternalCommitTrigger.GetMergeTrigger() != nil { + if mergeCommitFilterCopy != nil { + mergeCommitFilterCopy() + } + } + case *roachpb.HeartbeatTxnRequest, *roachpb.GCRequest: + if blockHBAndGCsCopy != nil { + <-blockHBAndGCsCopy + } + } + } + return nil + } + + // snapshotFilter is used to listen for the completion of a Raft snapshot. + var snapshotFilter func(kvserver.IncomingSnapshot) + beforeSnapshotSSTIngestion := func( + inSnap kvserver.IncomingSnapshot, + snapType kvserver.SnapshotRequest_Type, + _ []string, + ) error { + filterMu.Lock() + snapshotFilterCopy := snapshotFilter + filterMu.Unlock() + if snapshotFilterCopy != nil { + snapshotFilterCopy(inSnap) + } + return nil + } + + manualClock := hlc.NewHybridManualClock() ctx := context.Background() - var tc *testcluster.TestCluster - var lhsStore, rhsStore *kvserver.Store + tc := testcluster.StartTestCluster(t, 3, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + Store: &kvserver.StoreTestingKnobs{ + TestingRequestFilter: testingRequestFilter, + BeforeSnapshotSSTIngestion: beforeSnapshotSSTIngestion, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + lhsStore := tc.GetFirstStoreFromServer(t, 0) + var rhsStore *kvserver.Store if disjointLeaseholders { - tc = testcluster.StartTestCluster(t, 2, - base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - }) - lhsStore = tc.GetFirstStoreFromServer(t, 0) rhsStore = tc.GetFirstStoreFromServer(t, 1) } else { - tc = testcluster.StartTestCluster(t, 1, - base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - }, - ) - lhsStore = tc.GetFirstStoreFromServer(t, 0) rhsStore = tc.GetFirstStoreFromServer(t, 0) } - defer tc.Stopper().Stop(context.Background()) + + // Disable closed timestamps to ensure that any writes that are bumped to + // higher timestamps are bumped by the timestamp cache, as expected. + _, err := tc.ServerConn(0).Exec(`SET CLUSTER SETTING kv.closed_timestamp.target_duration = '24h'`) + require.NoError(t, err) lhsDesc, rhsDesc, err := createSplitRanges(ctx, lhsStore) - if err != nil { - t.Fatal(err) - } + require.NoError(t, err) + + tc.AddVotersOrFatal(t, lhsDesc.StartKey.AsRawKey(), tc.Target(1), tc.Target(2)) + tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Target(1), tc.Target(2)) if disjointLeaseholders { - tc.AddVotersOrFatal(t, lhsDesc.StartKey.AsRawKey(), tc.Target(1)) - tc.AddVotersOrFatal(t, rhsDesc.StartKey.AsRawKey(), tc.Target(1)) tc.TransferRangeLeaseOrFatal(t, *rhsDesc, tc.Target(1)) testutils.SucceedsSoon(t, func() error { rhsRepl, err := rhsStore.GetReplica(rhsDesc.RangeID) @@ -439,6 +519,9 @@ func mergeCheckingTimestampCaches(t *testing.T, disjointLeaseholders bool) { } readTS := tc.Servers[0].Clock().Now() + if futureRead { + readTS = readTS.Add(500*time.Millisecond.Nanoseconds(), 0).WithSynthetic(true) + } // Simulate a read on the RHS from a node with a newer clock. var ba roachpb.BatchRequest @@ -461,7 +544,7 @@ func mergeCheckingTimestampCaches(t *testing.T, disjointLeaseholders bool) { pushee := roachpb.MakeTransaction("pushee", rhsKey, roachpb.MinUserPriority, readTS, 0) pusher := roachpb.MakeTransaction("pusher", rhsKey, roachpb.MaxUserPriority, readTS, 0) ba = roachpb.BatchRequest{} - ba.Timestamp = tc.Servers[0].Clock().Now() + ba.Timestamp = readTS.Next() ba.RangeID = rhsDesc.RangeID ba.Add(pushTxnArgs(&pusher, &pushee, roachpb.PUSH_ABORT)) if br, pErr := rhsStore.Send(ctx, ba); pErr != nil { @@ -470,10 +553,240 @@ func mergeCheckingTimestampCaches(t *testing.T, disjointLeaseholders bool) { t.Fatalf("expected aborted pushee, but got %v", txn) } - // Merge the RHS back into the LHS. - args := adminMergeArgs(lhsDesc.StartKey.AsRawKey()) - if _, pErr := kv.SendWrapped(ctx, lhsStore.TestSender(), args); pErr != nil { - t.Fatal(pErr) + // Pause the cluster's clock. This accomplishes two things: + // 1. It ensures that if we force the LHS leaseholder to learn about the + // merge through a snapshot (throughSnapshot), the merge transaction is not + // allowed to expire and be aborted due to delayed txn heartbeats. + // 2. it ensures that if we performed a read at a future timestamp, the read + // time remains in the future, regardless of the passage of real time. + manualClock.Pause() + + if !throughSnapshot { + // The easy case: merge the RHS back into the LHS normally. + args := adminMergeArgs(lhsDesc.StartKey.AsRawKey()) + _, pErr := kv.SendWrapped(ctx, lhsStore.TestSender(), args) + require.Nil(t, pErr) + } else { + // The hard case: merge the RHS back into the LHS, but make sure the LHS + // leaseholder finds out about the merge through a Raft snapshot. + // + // To do this, we partition the leaseholder from the rest of its range. + // Once partitioned, we perform another write and truncate the Raft log + // on the two connected nodes. We then complete the range merge before + // removing the partition. This ensures that that when the leaseholder + // reconnects it will require a snapshot from Raft. + // + // But there's a wrinkle here that makes things more difficult: the + // leaseholder needs to play a role in coordinating the range merge and + // the log truncation, as it is the only replica that can propose such + // changes. To accommodate this, we put the range into a split + // leader-leaseholder state and lock down all communication between the + // two _except_ for forwarded proposal from the leaseholder to the + // leader. This allows the leaseholder to make proposals, even though it + // won't be able to hear their result. Because this is such a fragile + // state, we enter it as late as possible - after the merge begins and + // only upon receiving the merge's EndTxn request. + + lhsKey := roachpb.Key("a") + var lhsStores []*kvserver.Store + var lhsRepls []*kvserver.Replica + for i := range tc.Servers { + s := tc.GetFirstStoreFromServer(t, i) + r := s.LookupReplica(roachpb.RKey(lhsKey)) + lhsStores = append(lhsStores, s) + lhsRepls = append(lhsRepls, r) + } + + // Applied to the leaseholder's raft transport during the partition. + partitionedLeaseholderFuncs := noopRaftHandlerFuncs() + partitionedLeaseholderFuncs.dropReq = func(*kvserver.RaftMessageRequest) bool { + // Ignore everything from new leader. + return true + } + + // Applied to the leader and other follower's raft transport during the + // partition. + partitionedLeaderFuncs := noopRaftHandlerFuncs() + partitionedLeaderFuncs.dropReq = func(req *kvserver.RaftMessageRequest) bool { + // Ignore everything from leaseholder, except forwarded proposals. + return req.FromReplica.StoreID == lhsStore.StoreID() && + req.Message.Type != raftpb.MsgProp + } + partitionedLeaderFuncs.dropHB = func(hb *kvserver.RaftHeartbeat) bool { + // Ignore heartbeats from leaseholder, results in campaign. + return hb.FromReplicaID == roachpb.ReplicaID(lhsRepls[0].RaftStatus().ID) + } + + // Applied to leaseholder after the partition heals. + var truncIndex uint64 + restoredLeaseholderFuncs := noopRaftHandlerFuncs() + restoredLeaseholderFuncs.dropReq = func(req *kvserver.RaftMessageRequest) bool { + // Make sure that even going forward no MsgApp for what we just + // truncated can make it through. The Raft transport is asynchronous + // so this is necessary to make the test pass reliably - otherwise + // the leaseholder may catch up without needing a snapshot, tripping + // up the test. + // + // NB: the Index on the message is the log index that _precedes_ any of the + // entries in the MsgApp, so filter where msg.Index < index, not <= index. + return req.Message.Type == raftpb.MsgApp && req.Message.Index < truncIndex + } + + // Because we enter a split leader-leaseholder state, none of the + // operations we perform on the leaseholder will return. Instead, they + // will block for the duration of the partition, even after they have + // succeeded on the majority quorum. So we launch async goroutines to + // perform the write and the log truncation and only wait for them to + // complete after the partition heals. + incChan := make(chan *roachpb.Error, 1) + truncChan := make(chan *roachpb.Error, 1) + snapChan := make(chan kvserver.IncomingSnapshot, 1) + + filterMu.Lock() + mergeCommitFilter = func() { + // Install leader-leaseholder partition. + for i, s := range lhsStores { + var funcs unreliableRaftHandlerFuncs + if i == 0 { + funcs = partitionedLeaseholderFuncs + } else { + funcs = partitionedLeaderFuncs + } + tc.Servers[i].RaftTransport().Listen(s.StoreID(), &unreliableRaftHandler{ + rangeID: lhsDesc.GetRangeID(), + RaftMessageHandler: s, + unreliableRaftHandlerFuncs: funcs, + }) + } + + // Make sure the LHS range in uniquiesced so that it elects a new + // Raft leader after the partition is established. + for _, r := range lhsRepls { + r.UnquiesceAndWakeLeader() + } + + // Issue an increment on the range. The leaseholder should evaluate + // the request and forward a proposal to the leader, but it should + // be the only replica that does not apply the proposal. + go func() { + incArgs := incrementArgs(lhsKey, 4) + _, pErr := kv.SendWrappedWith(ctx, lhsStore, roachpb.Header{RangeID: lhsDesc.RangeID}, incArgs) + incChan <- pErr + }() + // NB: the operation won't complete, so peek below Raft and wait for + // the result to apply on the majority quorum. + tc.WaitForValues(t, lhsKey, []int64{0, 4, 4}) + + // Truncate the log to eventually force a snapshot. Determining + // which log index to truncate is tricky. We need to make sure it is + // <= to the largest log index on the leaseholder or it will reject + // the request. But we also need to make sure it is <= to the + // largest log index on the leader, or it will panic. So we choose + // the minimum of these two and just pick the smallest "last index" + // in the range, which does the trick. + min := func(a, b uint64) uint64 { + if a < b { + return a + } + return b + } + minLastIndex := uint64(math.MaxUint64) + for _, r := range lhsRepls { + lastIndex, err := r.GetLastIndex() + require.NoError(t, err) + minLastIndex = min(minLastIndex, lastIndex) + } + // Truncate the log at index+1 (log entries < N are removed). + truncIndex = minLastIndex + 1 + go func() { + truncArgs := truncateLogArgs(truncIndex, lhsDesc.RangeID) + truncArgs.Key = lhsKey + _, pErr := kv.SendWrappedWith(ctx, lhsStore, roachpb.Header{RangeID: lhsDesc.RangeID}, truncArgs) + truncChan <- pErr + }() + // NB: the operation won't complete, so peek below Raft and wait for + // the result to apply on the majority quorum. + testutils.SucceedsSoon(t, func() error { + for _, r := range lhsRepls[1:] { + firstIndex, err := r.GetFirstIndex() + require.NoError(t, err) + if firstIndex < truncIndex { + return errors.Errorf("truncate not applied, %d < %d", firstIndex, truncIndex) + } + } + return nil + }) + } + + // Begin blocking txn heartbeats and GC requests. They cause issues + // because they can grab latches and then get stuck once in the split + // leader-leaseholder state. + blockHBAndGCs = make(chan struct{}) + + // Install a filter to capture the Raft snapshot. + snapshotFilter = func(inSnap kvserver.IncomingSnapshot) { + if inSnap.State.Desc.RangeID == lhsDesc.RangeID { + snapChan <- inSnap + } + } + filterMu.Unlock() + + // Merge the RHS back into the LHS. + mergeChan := make(chan *roachpb.Error, 1) + go func() { + args := adminMergeArgs(lhsDesc.StartKey.AsRawKey()) + _, pErr := kv.SendWrapped(ctx, lhsStore.TestSender(), args) + mergeChan <- pErr + }() + // NB: the operation won't complete, so peek below Raft and wait for + // the result to apply on the majority quorum. + testutils.SucceedsSoon(t, func() error { + for _, r := range lhsRepls[1:] { + desc := r.Desc() + if !desc.EndKey.Equal(rhsDesc.EndKey) { + return errors.Errorf("merge not applied") + } + } + return nil + }) + + // Remove the partition. A snapshot to the leaseholder should follow. + // This snapshot will inform the leaseholder about the range merge. + for i, s := range lhsStores { + var h kvserver.RaftMessageHandler + if i == 0 { + h = &unreliableRaftHandler{ + rangeID: lhsDesc.GetRangeID(), + RaftMessageHandler: s, + unreliableRaftHandlerFuncs: restoredLeaseholderFuncs, + } + } else { + h = s + } + tc.Servers[i].RaftTransport().Listen(s.StoreID(), h) + } + close(blockHBAndGCs) + + t.Logf("waiting for snapshot to LHS leaseholder") + inSnap := <-snapChan + inSnapDesc := inSnap.State.Desc + require.Equal(t, lhsDesc.StartKey, inSnapDesc.StartKey) + require.Equal(t, rhsDesc.EndKey, inSnapDesc.EndKey) + + // Wait for all async ops to complete. + for _, asyncRes := range []struct { + name string + ch chan *roachpb.Error + }{ + {"increment", incChan}, + {"truncate", truncChan}, + {"merge", mergeChan}, + } { + t.Logf("waiting for result of %s", asyncRes.name) + err := <-asyncRes.ch + require.NotNil(t, err, "%s should fail", asyncRes.name) + require.Regexp(t, "result is ambiguous", err, "%s's result should be ambiguous", asyncRes.name) + } } // After the merge, attempt to write under the read. The batch should get @@ -492,15 +805,17 @@ func mergeCheckingTimestampCaches(t *testing.T, disjointLeaseholders bool) { // was aborted before the merge. This should be rejected with a transaction // aborted error. The reason will depend on whether the leaseholders were // disjoint or not because disjoint leaseholders will lead to a loss of - // resolution in the timestamp cache. Either way though, the transaction - // should not be allowed to create its record. + // resolution in the timestamp cache. Similarly, the reason will depend on + // whether the LHS leaseholder learned about the merge through Raft log + // application or a Raft snapshot. Either way though, the transaction should + // not be allowed to create its record. hb, hbH := heartbeatArgs(&pushee, tc.Servers[0].Clock().Now()) ba = roachpb.BatchRequest{} ba.Header = hbH ba.RangeID = lhsDesc.RangeID ba.Add(hb) var expReason roachpb.TransactionAbortedReason - if disjointLeaseholders { + if disjointLeaseholders || throughSnapshot { expReason = roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED } else { expReason = roachpb.ABORT_REASON_ABORTED_RECORD_FOUND @@ -3312,8 +3627,6 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { if err != nil { t.Fatal(err) } - // Truncate the log at index+1 (log entries < N are removed, so this - // includes the merge). truncArgs := &roachpb.TruncateLogRequest{ RequestHeader: roachpb.RequestHeader{Key: keyA}, Index: index, diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index a3d12df4fc79..187f4680ec11 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -2030,126 +2030,127 @@ func TestLeaseTransferInSnapshotUpdatesTimestampCache(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - ctx := context.Background() - tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, - }) - defer tc.Stopper().Stop(context.Background()) - store0 := tc.GetFirstStoreFromServer(t, 0) - store2 := tc.GetFirstStoreFromServer(t, 2) - - keyA := tc.ScratchRange(t) - keyB := keyA.Next() - keyC := keyB.Next() - - // First, do a couple of writes; we'll use these to determine when - // the dust has settled. - incA := incrementArgs(keyA, 1) - if _, pErr := kv.SendWrapped(ctx, store0.TestSender(), incA); pErr != nil { - t.Fatal(pErr) - } - incC := incrementArgs(keyC, 2) - if _, pErr := kv.SendWrapped(ctx, store0.TestSender(), incC); pErr != nil { - t.Fatal(pErr) - } - - tc.AddVotersOrFatal(t, keyA, tc.Targets(1, 2)...) - tc.WaitForValues(t, keyA, []int64{1, 1, 1}) - tc.WaitForValues(t, keyC, []int64{2, 2, 2}) - - // Create a transaction that will try to write "under" a served read. - // The read will have been served by the original leaseholder (node 0) - // and the write will be attempted on the new leaseholder (node 2). - // It should not succeed because it should run into the timestamp cache. - txnOld := kv.NewTxn(ctx, store0.DB(), 0 /* gatewayNodeID */) + testutils.RunTrueAndFalse(t, "future-read", func(t *testing.T, futureRead bool) { + manualClock := hlc.NewHybridManualClock() + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manualClock.UnixNano, + }, + }, + }, + }) + defer tc.Stopper().Stop(context.Background()) + store0 := tc.GetFirstStoreFromServer(t, 0) + store2 := tc.GetFirstStoreFromServer(t, 2) + + keyA := tc.ScratchRange(t) + keyB := keyA.Next() + keyC := keyB.Next() + + // First, do a couple of writes; we'll use these to determine when + // the dust has settled. + incA := incrementArgs(keyA, 1) + if _, pErr := kv.SendWrapped(ctx, store0.TestSender(), incA); pErr != nil { + t.Fatal(pErr) + } + incC := incrementArgs(keyC, 2) + if _, pErr := kv.SendWrapped(ctx, store0.TestSender(), incC); pErr != nil { + t.Fatal(pErr) + } + + tc.AddVotersOrFatal(t, keyA, tc.Targets(1, 2)...) + tc.WaitForValues(t, keyA, []int64{1, 1, 1}) + tc.WaitForValues(t, keyC, []int64{2, 2, 2}) + + // Pause the cluster's clock. This ensures that if we perform a read at + // a future timestamp, the read time remains in the future, regardless + // of the passage of real time. + manualClock.Pause() + + // Determine when to read. + readTS := tc.Servers[0].Clock().Now() + if futureRead { + readTS = readTS.Add(500*time.Millisecond.Nanoseconds(), 0).WithSynthetic(true) + } + + // Read the key at readTS. + // NB: don't use SendWrapped because we want access to br.Timestamp. + var ba roachpb.BatchRequest + ba.Timestamp = readTS + ba.Add(getArgs(keyA)) + br, pErr := tc.Servers[0].DistSender().Send(ctx, ba) + require.Nil(t, pErr) + require.Equal(t, readTS, br.Timestamp) + v, err := br.Responses[0].GetGet().Value.GetInt() + require.NoError(t, err) + require.Equal(t, int64(1), v) - // Perform a write with txnOld so that its timestamp gets set. - if _, err := txnOld.Inc(ctx, keyB, 3); err != nil { - t.Fatal(err) - } + repl0 := store0.LookupReplica(roachpb.RKey(keyA)) - // Read keyC with txnOld, which is updated below. This prevents the - // transaction from refreshing when it hits the serializable error. - if _, err := txnOld.Get(ctx, keyC); err != nil { - t.Fatal(err) - } + // Partition node 2 from the rest of its range. Once partitioned, perform + // another write and truncate the Raft log on the two connected nodes. This + // ensures that that when node 2 comes back up it will require a snapshot + // from Raft. + funcs := noopRaftHandlerFuncs() + funcs.dropReq = func(*kvserver.RaftMessageRequest) bool { + return true + } + tc.Servers[2].RaftTransport().Listen(store2.StoreID(), &unreliableRaftHandler{ + rangeID: repl0.GetRangeID(), + RaftMessageHandler: store2, + unreliableRaftHandlerFuncs: funcs, + }) - // Ensure that the transaction sends its first hearbeat so that it creates - // its transaction record and doesn't run into trouble with the low water - // mark of the new leaseholder's timestamp cache. Amusingly, if the bug - // we're regression testing against here still existed, we would not have - // to do this. - hb, hbH := heartbeatArgs(txnOld.TestingCloneTxn(), tc.Servers[0].Clock().Now()) - if _, pErr := kv.SendWrappedWith(ctx, store0.TestSender(), hbH, hb); pErr != nil { - t.Fatal(pErr) - } + if _, pErr := kv.SendWrapped(ctx, store0.TestSender(), incC); pErr != nil { + t.Fatal(pErr) + } + tc.WaitForValues(t, keyC, []int64{4, 4, 2}) - // Another client comes along at a higher timestamp and reads. We should - // never be able to write under this time or we would be rewriting history. - if _, err := store0.DB().Get(ctx, keyA); err != nil { - t.Fatal(err) - } + // Truncate the log at index+1 (log entries < N are removed, so this + // includes the increment). This necessitates a snapshot when the + // partitioned replica rejoins the rest of the range. + index, err := repl0.GetLastIndex() + if err != nil { + t.Fatal(err) + } + truncArgs := truncateLogArgs(index+1, repl0.GetRangeID()) + truncArgs.Key = keyA + if _, err := kv.SendWrapped(ctx, store0.TestSender(), truncArgs); err != nil { + t.Fatal(err) + } - repl0 := store0.LookupReplica(roachpb.RKey(keyA)) + // Finally, transfer the lease to node 2 while it is still unavailable and + // behind. We try to avoid this case when picking new leaseholders in practice, + // but we're never 100% successful. + if err := repl0.AdminTransferLease(ctx, store2.Ident.StoreID); err != nil { + t.Fatal(err) + } - // Partition node 2 from the rest of its range. Once partitioned, perform - // another write and truncate the Raft log on the two connected nodes. This - // ensures that that when node 2 comes back up it will require a snapshot - // from Raft. - funcs := noopRaftHandlerFuncs() - funcs.dropReq = func(*kvserver.RaftMessageRequest) bool { - return true - } - tc.Servers[2].RaftTransport().Listen(store2.StoreID(), &unreliableRaftHandler{ - rangeID: repl0.GetRangeID(), - RaftMessageHandler: store2, - unreliableRaftHandlerFuncs: funcs, + // Remove the partition. A snapshot to node 2 should follow. This snapshot + // will inform node 2 that it is the new leaseholder for the range. Node 2 + // should act accordingly and update its internal state to reflect this. + tc.Servers[2].RaftTransport().Listen(store2.Ident.StoreID, store2) + tc.WaitForValues(t, keyC, []int64{4, 4, 4}) + + // Attempt to write under the read on the new leaseholder. The batch + // should get forwarded to a timestamp after the read. With the bug in + // #34025, the new leaseholder who heard about the lease transfer from a + // snapshot had an empty timestamp cache and would simply let us write + // under the previous read. + // NB: don't use SendWrapped because we want access to br.Timestamp. + ba = roachpb.BatchRequest{} + ba.Timestamp = readTS + ba.Add(incrementArgs(keyA, 1)) + br, pErr = tc.Servers[0].DistSender().Send(ctx, ba) + require.Nil(t, pErr) + require.NotEqual(t, readTS, br.Timestamp) + require.True(t, readTS.Less(br.Timestamp)) + require.Equal(t, readTS.Synthetic, br.Timestamp.Synthetic) }) - - if _, pErr := kv.SendWrapped(ctx, store0.TestSender(), incC); pErr != nil { - t.Fatal(pErr) - } - tc.WaitForValues(t, keyC, []int64{4, 4, 2}) - - // Truncate the log at index+1 (log entries < N are removed, so this - // includes the increment). This necessitates a snapshot when the - // partitioned replica rejoins the rest of the range. - index, err := repl0.GetLastIndex() - if err != nil { - t.Fatal(err) - } - truncArgs := truncateLogArgs(index+1, repl0.GetRangeID()) - truncArgs.Key = keyA - if _, err := kv.SendWrapped(ctx, store0.TestSender(), truncArgs); err != nil { - t.Fatal(err) - } - - // Finally, transfer the lease to node 2 while it is still unavailable and - // behind. We try to avoid this case when picking new leaseholders in practice, - // but we're never 100% successful. - if err := repl0.AdminTransferLease(ctx, store2.Ident.StoreID); err != nil { - t.Fatal(err) - } - - // Remove the partition. A snapshot to node 2 should follow. This snapshot - // will inform node 2 that it is the new leaseholder for the range. Node 2 - // should act accordingly and update its internal state to reflect this. - tc.Servers[2].RaftTransport().Listen(store2.Ident.StoreID, store2) - tc.WaitForValues(t, keyC, []int64{4, 4, 4}) - - // Perform a write on the new leaseholder underneath the previously served - // read. This write should hit the timestamp cache and flag the txn for a - // restart when we try to commit it below. With the bug in #34025, the new - // leaseholder who heard about the lease transfer from a snapshot had an - // empty timestamp cache and would simply let us write under the previous - // read. - if _, err := txnOld.Inc(ctx, keyA, 4); err != nil { - t.Fatal(err) - } - const exp = `TransactionRetryError: retry txn \(RETRY_SERIALIZABLE - failed preemptive refresh\)` - if err := txnOld.Commit(ctx); !testutils.IsError(err, exp) { - t.Fatalf("expected retry error, got: %v; did we write under a read?", err) - } } // TestConcurrentAdminChangeReplicasRequests ensures that when two attempts to diff --git a/pkg/kv/kvserver/kvserverpb/BUILD.bazel b/pkg/kv/kvserver/kvserverpb/BUILD.bazel index b767e4bfd4cb..3cc75f0f374e 100644 --- a/pkg/kv/kvserver/kvserverpb/BUILD.bazel +++ b/pkg/kv/kvserver/kvserverpb/BUILD.bazel @@ -30,6 +30,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/liveness/livenesspb:livenesspb_proto", + "//pkg/kv/kvserver/readsummary/rspb:rspb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/storage/enginepb:enginepb_proto", "//pkg/util/hlc:hlc_proto", @@ -46,6 +47,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/liveness/livenesspb", + "//pkg/kv/kvserver/readsummary/rspb", "//pkg/roachpb", "//pkg/storage/enginepb", "//pkg/util/hlc", diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go b/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go index 6e5a6185c5a2..4a027ae5895b 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go @@ -6,6 +6,7 @@ package kvserverpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" +import rspb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -48,7 +49,7 @@ func (m *Split) Reset() { *m = Split{} } func (m *Split) String() string { return proto.CompactTextString(m) } func (*Split) ProtoMessage() {} func (*Split) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{0} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{0} } func (m *Split) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -83,7 +84,7 @@ func (m *Merge) Reset() { *m = Merge{} } func (m *Merge) String() string { return proto.CompactTextString(m) } func (*Merge) ProtoMessage() {} func (*Merge) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{1} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{1} } func (m *Merge) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -117,7 +118,7 @@ type ChangeReplicas struct { func (m *ChangeReplicas) Reset() { *m = ChangeReplicas{} } func (*ChangeReplicas) ProtoMessage() {} func (*ChangeReplicas) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{2} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{2} } func (m *ChangeReplicas) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -169,7 +170,7 @@ func (m *ComputeChecksum) Reset() { *m = ComputeChecksum{} } func (m *ComputeChecksum) String() string { return proto.CompactTextString(m) } func (*ComputeChecksum) ProtoMessage() {} func (*ComputeChecksum) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{3} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{3} } func (m *ComputeChecksum) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -206,7 +207,7 @@ func (m *Compaction) Reset() { *m = Compaction{} } func (m *Compaction) String() string { return proto.CompactTextString(m) } func (*Compaction) ProtoMessage() {} func (*Compaction) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{4} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{4} } func (m *Compaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -243,7 +244,7 @@ func (m *SuggestedCompaction) Reset() { *m = SuggestedCompaction{} } func (m *SuggestedCompaction) String() string { return proto.CompactTextString(m) } func (*SuggestedCompaction) ProtoMessage() {} func (*SuggestedCompaction) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{5} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{5} } func (m *SuggestedCompaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -299,13 +300,40 @@ type ReplicatedEvalResult struct { // It will be used to make sure we know if a lease was extended after we sent out the request // but before we tried to apply it. PrevLeaseProposal *github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,20,opt,name=prev_lease_proposal,json=prevLeaseProposal,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"prev_lease_proposal,omitempty"` + // PriorReadSummary is a summary of the reads that have been served on the + // range prior to this proposal, which must be a lease change (request or + // transfer) if the field is set. The read summary is used to update the new + // leaseholder's timestamp cache to prevent them from serving writes that + // violate previously served reads. + // + // The summary, when available, can be used in place of bumping the new + // leaseholder's timestamp cache to the new lease's start time. It has two + // distinct advantages: + // 1. it can transfer a higher-resolution snapshot of the reads on the range + // through a lease transfer, to make the lease transfers less disruptive to + // writes because the timestamp cache won't be bumped as high. + // 2. it can transfer information about reads with synthetic timestamps, which + // are not otherwise captured by the new lease's start time. + // + // When a ReadSummary is set in a ReplicatedEvalResult, there is always also a + // write to the RangePriorReadSummaryKey in the RaftCommand.WriteBatch. The + // persisted summary may be identical to the summary in this field, but it + // does not have to be. Notably, we intended for the summary included in the + // ReplicatedEvalResult to eventually be a much higher-resolution version of + // the ReadSummmary than the version persisted. This scheme of persisting a + // compressed ReadSummary indefinitely and including a higher-resolution + // ReadSummary on the RaftCommand allows us to optimize for the common case + // where the lease transfer is applied on the new leaseholder through Raft log + // application while ensuring correctness in the case where the lease transfer + // is applied on the new leaseholder through a Raft snapshot. + PriorReadSummary *rspb.ReadSummary `protobuf:"bytes,22,opt,name=prior_read_summary,json=priorReadSummary,proto3" json:"prior_read_summary,omitempty"` } func (m *ReplicatedEvalResult) Reset() { *m = ReplicatedEvalResult{} } func (m *ReplicatedEvalResult) String() string { return proto.CompactTextString(m) } func (*ReplicatedEvalResult) ProtoMessage() {} func (*ReplicatedEvalResult) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{6} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{6} } func (m *ReplicatedEvalResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -349,7 +377,7 @@ func (m *ReplicatedEvalResult_AddSSTable) Reset() { *m = ReplicatedEvalR func (m *ReplicatedEvalResult_AddSSTable) String() string { return proto.CompactTextString(m) } func (*ReplicatedEvalResult_AddSSTable) ProtoMessage() {} func (*ReplicatedEvalResult_AddSSTable) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{6, 0} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{6, 0} } func (m *ReplicatedEvalResult_AddSSTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -386,7 +414,7 @@ func (m *WriteBatch) Reset() { *m = WriteBatch{} } func (m *WriteBatch) String() string { return proto.CompactTextString(m) } func (*WriteBatch) ProtoMessage() {} func (*WriteBatch) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{7} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{7} } func (m *WriteBatch) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -423,7 +451,7 @@ func (m *LogicalOpLog) Reset() { *m = LogicalOpLog{} } func (m *LogicalOpLog) String() string { return proto.CompactTextString(m) } func (*LogicalOpLog) ProtoMessage() {} func (*LogicalOpLog) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{8} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{8} } func (m *LogicalOpLog) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -552,7 +580,7 @@ func (m *RaftCommand) Reset() { *m = RaftCommand{} } func (m *RaftCommand) String() string { return proto.CompactTextString(m) } func (*RaftCommand) ProtoMessage() {} func (*RaftCommand) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{9} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{9} } func (m *RaftCommand) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -591,7 +619,7 @@ func (m *MaxLeaseFooter) Reset() { *m = MaxLeaseFooter{} } func (m *MaxLeaseFooter) String() string { return proto.CompactTextString(m) } func (*MaxLeaseFooter) ProtoMessage() {} func (*MaxLeaseFooter) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{10} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{10} } func (m *MaxLeaseFooter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -630,7 +658,7 @@ func (m *ClosedTimestampFooter) Reset() { *m = ClosedTimestampFooter{} } func (m *ClosedTimestampFooter) String() string { return proto.CompactTextString(m) } func (*ClosedTimestampFooter) ProtoMessage() {} func (*ClosedTimestampFooter) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_7221db15568d8c0d, []int{11} + return fileDescriptor_proposer_kv_fd16d3d3c1c124dd, []int{11} } func (m *ClosedTimestampFooter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1204,6 +1232,18 @@ func (m *ReplicatedEvalResult) MarshalTo(dAtA []byte) (int, error) { } i += n16 } + if m.PriorReadSummary != nil { + dAtA[i] = 0xb2 + i++ + dAtA[i] = 0x1 + i++ + i = encodeVarintProposerKv(dAtA, i, uint64(m.PriorReadSummary.Size())) + n17, err := m.PriorReadSummary.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n17 + } return i, nil } @@ -1314,11 +1354,11 @@ func (m *RaftCommand) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintProposerKv(dAtA, i, uint64(m.DeprecatedProposerLease.Size())) - n17, err := m.DeprecatedProposerLease.MarshalTo(dAtA[i:]) + n18, err := m.DeprecatedProposerLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n17 + i += n18 } if m.ProposerLeaseSequence != 0 { dAtA[i] = 0x30 @@ -1328,30 +1368,30 @@ func (m *RaftCommand) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintProposerKv(dAtA, i, uint64(m.ReplicatedEvalResult.Size())) - n18, err := m.ReplicatedEvalResult.MarshalTo(dAtA[i:]) + n19, err := m.ReplicatedEvalResult.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n19 if m.WriteBatch != nil { dAtA[i] = 0x72 i++ i = encodeVarintProposerKv(dAtA, i, uint64(m.WriteBatch.Size())) - n19, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n20, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n19 + i += n20 } if m.LogicalOpLog != nil { dAtA[i] = 0x7a i++ i = encodeVarintProposerKv(dAtA, i, uint64(m.LogicalOpLog.Size())) - n20, err := m.LogicalOpLog.MarshalTo(dAtA[i:]) + n21, err := m.LogicalOpLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n20 + i += n21 } if len(m.TraceData) > 0 { keysForTraceData := make([]string, 0, len(m.TraceData)) @@ -1383,11 +1423,11 @@ func (m *RaftCommand) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintProposerKv(dAtA, i, uint64(m.ClosedTimestamp.Size())) - n21, err := m.ClosedTimestamp.MarshalTo(dAtA[i:]) + n22, err := m.ClosedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n22 } return i, nil } @@ -1435,11 +1475,11 @@ func (m *ClosedTimestampFooter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintProposerKv(dAtA, i, uint64(m.ClosedTimestamp.Size())) - n22, err := m.ClosedTimestamp.MarshalTo(dAtA[i:]) + n23, err := m.ClosedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n23 return i, nil } @@ -1598,6 +1638,10 @@ func (m *ReplicatedEvalResult) Size() (n int) { l = m.ComputeChecksum.Size() n += 2 + l + sovProposerKv(uint64(l)) } + if m.PriorReadSummary != nil { + l = m.PriorReadSummary.Size() + n += 2 + l + sovProposerKv(uint64(l)) + } return n } @@ -2803,6 +2847,39 @@ func (m *ReplicatedEvalResult) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 22: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PriorReadSummary", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProposerKv + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProposerKv + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PriorReadSummary == nil { + m.PriorReadSummary = &rspb.ReadSummary{} + } + if err := m.PriorReadSummary.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProposerKv(dAtA[iNdEx:]) @@ -3709,101 +3786,104 @@ var ( ) func init() { - proto.RegisterFile("kv/kvserver/kvserverpb/proposer_kv.proto", fileDescriptor_proposer_kv_7221db15568d8c0d) + proto.RegisterFile("kv/kvserver/kvserverpb/proposer_kv.proto", fileDescriptor_proposer_kv_fd16d3d3c1c124dd) } -var fileDescriptor_proposer_kv_7221db15568d8c0d = []byte{ - // 1460 bytes of a gzipped FileDescriptorProto +var fileDescriptor_proposer_kv_fd16d3d3c1c124dd = []byte{ + // 1510 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x5f, 0x6f, 0x13, 0xc7, - 0x16, 0x8f, 0x63, 0x3b, 0x59, 0x1f, 0x27, 0xf6, 0x66, 0x08, 0xb0, 0x37, 0xf7, 0x5e, 0x3b, 0xf2, - 0xe5, 0xa2, 0xdc, 0x5b, 0xba, 0x46, 0x49, 0x2b, 0x21, 0x8a, 0x2a, 0x62, 0x07, 0x4a, 0x4c, 0x92, - 0xc2, 0x38, 0xd0, 0x8a, 0x3e, 0xac, 0xc6, 0xbb, 0xc3, 0x7a, 0xeb, 0xb5, 0x77, 0xd9, 0x19, 0x1b, - 0xf2, 0x29, 0xda, 0x4a, 0x95, 0xda, 0xa7, 0x96, 0xc7, 0x7e, 0x8d, 0xbe, 0xf1, 0xc8, 0x23, 0xea, - 0x83, 0x55, 0xc2, 0x4b, 0x3f, 0x03, 0x4f, 0xd5, 0xcc, 0xce, 0xfa, 0x0f, 0x0a, 0x8d, 0xa1, 0x6f, - 0xb3, 0x67, 0xe6, 0xfc, 0xce, 0x99, 0xf3, 0xe7, 0x77, 0x66, 0x61, 0xa3, 0x33, 0xa8, 0x76, 0x06, - 0x8c, 0x46, 0x03, 0x1a, 0x8d, 0x16, 0x61, 0xab, 0x1a, 0x46, 0x41, 0x18, 0x30, 0x1a, 0x59, 0x9d, - 0x81, 0x19, 0x46, 0x01, 0x0f, 0x50, 0xd9, 0x0e, 0xec, 0x4e, 0x14, 0x10, 0xbb, 0x6d, 0x76, 0x06, - 0x66, 0x72, 0xd4, 0x64, 0x3c, 0x88, 0x88, 0x4b, 0xc3, 0xd6, 0xda, 0x8a, 0xdc, 0x0c, 0x5b, 0x55, - 0x12, 0x7a, 0xb1, 0xce, 0x1a, 0x4a, 0x44, 0x0e, 0xe1, 0x44, 0xc9, 0xce, 0x25, 0xb2, 0x2e, 0xe5, - 0x64, 0x42, 0xfe, 0x4f, 0x85, 0x54, 0xa5, 0x3d, 0xd7, 0xeb, 0x51, 0x71, 0x60, 0x60, 0xdb, 0x6a, - 0xf3, 0x5f, 0x27, 0x6e, 0x6e, 0xa9, 0xdd, 0xca, 0x5b, 0x2e, 0xc1, 0x38, 0xe1, 0x54, 0x9d, 0x31, - 0xfa, 0xdc, 0xf3, 0xab, 0x6d, 0xdf, 0xae, 0x72, 0xaf, 0x4b, 0x19, 0x27, 0xdd, 0x50, 0xed, 0xac, - 0xba, 0x81, 0x1b, 0xc8, 0x65, 0x55, 0xac, 0x62, 0x69, 0xe5, 0x97, 0x14, 0x64, 0x9b, 0xa1, 0xef, - 0x71, 0x54, 0x87, 0x45, 0x1e, 0x79, 0xae, 0x4b, 0x23, 0x23, 0xb5, 0x9e, 0xda, 0xc8, 0x6f, 0x96, - 0xcd, 0x71, 0x28, 0xd4, 0x65, 0x4c, 0x79, 0xf4, 0x30, 0x3e, 0x56, 0xd3, 0x9e, 0x0d, 0xcb, 0x73, - 0xcf, 0x87, 0xe5, 0x14, 0x4e, 0x34, 0xd1, 0x21, 0xe4, 0xa2, 0x36, 0xb3, 0x1c, 0xea, 0x73, 0x62, - 0xcc, 0x4b, 0x98, 0xff, 0x4e, 0xc0, 0xa8, 0xeb, 0x99, 0xc9, 0xf5, 0xcc, 0xfd, 0xfb, 0xf5, 0x7a, - 0x93, 0x13, 0xce, 0x6a, 0xba, 0x00, 0x3b, 0x1e, 0x96, 0x35, 0x7c, 0xab, 0xb9, 0x23, 0xd4, 0xb1, - 0x16, 0xb5, 0x99, 0x5c, 0x5d, 0xcd, 0xfc, 0xf1, 0xb4, 0x9c, 0xaa, 0x60, 0xc8, 0xee, 0xd3, 0xc8, - 0xa5, 0xb3, 0x79, 0x2a, 0x8f, 0xbe, 0xdd, 0x53, 0x85, 0xe9, 0x40, 0xa1, 0xde, 0x26, 0x3d, 0x97, - 0x62, 0x1a, 0xfa, 0x9e, 0x4d, 0x18, 0xda, 0x7b, 0x13, 0x7c, 0xe3, 0x04, 0xf0, 0x69, 0x9d, 0xbf, - 0xb2, 0xf2, 0xe3, 0xd3, 0xf2, 0x5c, 0xe5, 0xe5, 0x3c, 0x14, 0xeb, 0x41, 0x37, 0xec, 0x73, 0x5a, - 0x6f, 0x53, 0xbb, 0xc3, 0xfa, 0x5d, 0xf4, 0x35, 0xe4, 0x6d, 0xb5, 0xb6, 0x3c, 0x47, 0xda, 0x5a, - 0xaa, 0xed, 0x0a, 0x84, 0xdf, 0x86, 0xe5, 0x2d, 0xd7, 0xe3, 0xed, 0x7e, 0xcb, 0xb4, 0x83, 0x6e, - 0x75, 0x64, 0xdd, 0x69, 0x8d, 0xd7, 0xd5, 0xb0, 0xe3, 0x56, 0x65, 0xaa, 0xfb, 0x7d, 0xcf, 0x31, - 0xef, 0xdd, 0xdb, 0xdd, 0x39, 0x1e, 0x96, 0x21, 0x41, 0xdf, 0xdd, 0xc1, 0x90, 0xa0, 0xef, 0x3a, - 0xe8, 0x3f, 0xb0, 0xcc, 0xc8, 0x80, 0x5a, 0xac, 0x47, 0x42, 0xd6, 0x0e, 0xb8, 0xcc, 0x8c, 0x86, - 0x97, 0x84, 0xb0, 0xa9, 0x64, 0x68, 0x0b, 0x32, 0xdd, 0xc0, 0xa1, 0x46, 0x7a, 0x3d, 0xb5, 0x51, - 0x38, 0x31, 0xa4, 0x09, 0xfa, 0x7e, 0xe0, 0x50, 0x2c, 0x0f, 0xa3, 0x12, 0xc4, 0x76, 0xc2, 0xc0, - 0xeb, 0x71, 0x23, 0x23, 0x61, 0x27, 0x24, 0xc8, 0x80, 0xc5, 0x01, 0x8d, 0x98, 0x17, 0xf4, 0x8c, - 0xec, 0x7a, 0x6a, 0x63, 0x19, 0x27, 0x9f, 0xe8, 0x16, 0xe4, 0x38, 0x8d, 0xba, 0x5e, 0x8f, 0x70, - 0x6a, 0x2c, 0xac, 0xa7, 0x37, 0xf2, 0x9b, 0x17, 0x4e, 0xb0, 0xa9, 0x62, 0xbc, 0x43, 0x99, 0x1d, - 0x79, 0x21, 0x0f, 0xa2, 0x5a, 0x46, 0xc4, 0x08, 0x8f, 0x95, 0x55, 0x26, 0xef, 0x03, 0x88, 0x10, - 0x13, 0x9b, 0x0b, 0xf4, 0x55, 0xc8, 0xb6, 0x8e, 0x38, 0x65, 0x32, 0xae, 0x69, 0x1c, 0x7f, 0xa0, - 0x4b, 0x80, 0x58, 0xdf, 0x75, 0x29, 0xe3, 0xd4, 0xb1, 0x08, 0xb7, 0x7a, 0xa4, 0x17, 0x30, 0x19, - 0x8c, 0x34, 0xd6, 0x47, 0x3b, 0xdb, 0xfc, 0x40, 0xc8, 0x15, 0xee, 0xf7, 0xf3, 0x70, 0xa6, 0x99, - 0x6c, 0x4d, 0x58, 0xb8, 0x0b, 0x39, 0xc6, 0x49, 0xc4, 0xad, 0x0e, 0x3d, 0x52, 0xd9, 0xfb, 0xe8, - 0xf5, 0xb0, 0x7c, 0x79, 0xa6, 0xcc, 0x25, 0xb7, 0xbb, 0x4d, 0x8f, 0xb0, 0x26, 0x61, 0x6e, 0xd3, - 0x23, 0xb4, 0x0f, 0x8b, 0xb4, 0xe7, 0x48, 0xc0, 0xf9, 0xbf, 0x01, 0xb8, 0x40, 0x7b, 0x8e, 0x80, - 0xbb, 0x07, 0x60, 0x8f, 0xfc, 0x95, 0x69, 0xcd, 0x6f, 0x7e, 0x60, 0x9e, 0x42, 0x6f, 0xe6, 0xf8, - 0x8a, 0x13, 0xf5, 0x3c, 0x01, 0xa4, 0xc2, 0xf2, 0xab, 0x06, 0xab, 0x2a, 0x37, 0x9c, 0x3a, 0x37, - 0x06, 0xc4, 0xc7, 0x94, 0xf5, 0x7d, 0x41, 0x23, 0x59, 0xc9, 0x47, 0xaa, 0xfb, 0x3f, 0x3c, 0xd5, - 0xa0, 0x42, 0x11, 0x2c, 0x40, 0x71, 0xac, 0x8b, 0xae, 0x41, 0x96, 0x09, 0xa6, 0x51, 0x5e, 0x5f, - 0x3c, 0x15, 0x44, 0xf2, 0x12, 0x8e, 0x95, 0x84, 0x76, 0x57, 0x74, 0xbf, 0xac, 0xc7, 0x59, 0xb4, - 0x25, 0x57, 0xe0, 0x58, 0x09, 0x6d, 0x80, 0xee, 0x31, 0xcb, 0xa7, 0x84, 0x51, 0x2b, 0xa2, 0x8f, - 0xfa, 0x94, 0x71, 0x63, 0x41, 0x16, 0x76, 0xc1, 0x63, 0x7b, 0x42, 0x8c, 0x63, 0x29, 0xda, 0x86, - 0xdc, 0x88, 0x64, 0x0d, 0x4d, 0xda, 0xfa, 0xf7, 0x84, 0x2d, 0xd1, 0x9e, 0x66, 0xdb, 0xb7, 0xcd, - 0xc3, 0xe4, 0xd0, 0xa8, 0x76, 0x13, 0x01, 0xba, 0x03, 0xba, 0x43, 0xc3, 0x88, 0xca, 0x28, 0x2a, - 0xda, 0x84, 0x77, 0xa0, 0x4d, 0x5c, 0x1c, 0xab, 0x4b, 0xae, 0x44, 0x5f, 0x42, 0xd1, 0x96, 0xec, - 0x64, 0x45, 0x8a, 0x9e, 0x8c, 0x25, 0x09, 0x58, 0x3d, 0x3d, 0xf5, 0x53, 0xac, 0x86, 0x0b, 0xf6, - 0x34, 0x33, 0x5e, 0x80, 0x42, 0x44, 0x1e, 0x72, 0xcb, 0x0f, 0x5c, 0xe5, 0xe9, 0xb2, 0xec, 0x9c, - 0x25, 0x21, 0xdd, 0x0b, 0xdc, 0xd8, 0xfe, 0x23, 0xc8, 0x13, 0xc7, 0xb1, 0x18, 0xe3, 0xa4, 0xe5, - 0x53, 0x63, 0x45, 0xda, 0xbe, 0x3e, 0x6b, 0x15, 0x4c, 0xd5, 0x92, 0xb9, 0xed, 0x38, 0xcd, 0xe6, - 0xa1, 0xc0, 0xa9, 0x15, 0x04, 0xbd, 0x8d, 0xbf, 0x31, 0x10, 0xc7, 0x69, 0xc6, 0x36, 0xd0, 0x4d, - 0xc8, 0xc6, 0xfe, 0x20, 0x69, 0xec, 0xff, 0x33, 0x45, 0x4e, 0x7a, 0xab, 0x12, 0x12, 0xab, 0xa3, - 0x6f, 0x52, 0x70, 0x26, 0x8c, 0xe8, 0x40, 0x25, 0x3f, 0x7e, 0x1b, 0x10, 0xdf, 0x58, 0x9d, 0x25, - 0xb5, 0xd7, 0x5f, 0x0f, 0xcb, 0xd7, 0x66, 0xa7, 0x6d, 0xa1, 0x5c, 0xf7, 0x03, 0xbb, 0x33, 0x42, - 0xc0, 0x2b, 0xc2, 0xb6, 0x2c, 0xb0, 0x3b, 0xca, 0x32, 0xfa, 0x0a, 0x74, 0x3b, 0x9e, 0x1b, 0x56, - 0x42, 0xe7, 0xc6, 0x59, 0xe9, 0xcd, 0xe5, 0x99, 0x1a, 0x79, 0x62, 0xe0, 0xe0, 0xa2, 0x3d, 0x2d, - 0x58, 0xfb, 0x0c, 0x26, 0x02, 0x8a, 0x10, 0x64, 0xc4, 0x2b, 0x25, 0xa6, 0x32, 0x2c, 0xd7, 0xa8, - 0x0c, 0x59, 0x3b, 0xb2, 0xb7, 0x36, 0x65, 0x2f, 0x2f, 0xd7, 0x72, 0xc7, 0xc3, 0x72, 0xb6, 0x8e, - 0xeb, 0x5b, 0x9b, 0x38, 0x96, 0xc7, 0x5c, 0xd0, 0xc8, 0x68, 0x29, 0x7d, 0xbe, 0x91, 0xd1, 0xb2, - 0xfa, 0x42, 0x23, 0xa3, 0x2d, 0xea, 0x5a, 0x23, 0xa3, 0xe5, 0x74, 0x68, 0x64, 0xb4, 0x82, 0x5e, - 0x6c, 0x64, 0xb4, 0xa2, 0xae, 0x37, 0x32, 0x9a, 0xae, 0xaf, 0x34, 0x32, 0xda, 0x19, 0x7d, 0xb5, - 0xb1, 0xa0, 0x7d, 0x77, 0xa0, 0xff, 0x74, 0x50, 0x59, 0x07, 0xf8, 0x22, 0xf2, 0x38, 0xad, 0x11, - 0x6e, 0xb7, 0x4f, 0x72, 0xa0, 0x72, 0x17, 0x96, 0xf6, 0x02, 0xd7, 0xb3, 0x89, 0xff, 0x79, 0xb8, - 0x17, 0xb8, 0x68, 0x1b, 0xd2, 0x41, 0x28, 0x48, 0x5d, 0x8c, 0x8b, 0xff, 0x9d, 0x96, 0xe7, 0x91, - 0xaa, 0x4a, 0xb3, 0xd0, 0xad, 0xfc, 0xb0, 0x00, 0x79, 0x4c, 0x1e, 0xf2, 0x7a, 0xd0, 0xed, 0x92, - 0x9e, 0x83, 0x2e, 0x42, 0xb1, 0x4b, 0x9e, 0xa8, 0x94, 0x7b, 0x3d, 0x87, 0x3e, 0x91, 0xb4, 0x91, - 0xc1, 0xcb, 0x5d, 0xf2, 0x44, 0x66, 0x63, 0x57, 0x08, 0xd1, 0x21, 0xfc, 0x63, 0xa2, 0x53, 0x47, - 0xef, 0x46, 0xa9, 0x27, 0x67, 0x5b, 0x7e, 0xd3, 0x38, 0x61, 0x7e, 0xc5, 0x84, 0x71, 0x7e, 0xac, - 0x7a, 0x47, 0x69, 0xca, 0x0d, 0x34, 0x80, 0xf3, 0xd3, 0x50, 0x16, 0x13, 0xe4, 0xd2, 0xb3, 0xa9, - 0xe4, 0x9c, 0x74, 0xed, 0xd3, 0xd7, 0xc3, 0xf2, 0xd5, 0x77, 0x1a, 0x01, 0x12, 0xb8, 0xa9, 0x50, - 0xf0, 0xd9, 0x70, 0xd2, 0x5e, 0x22, 0x46, 0x8f, 0xe0, 0x5c, 0x34, 0xea, 0x38, 0x8b, 0x0e, 0x88, - 0x6f, 0x45, 0xb2, 0xe7, 0x64, 0x4f, 0xe7, 0x37, 0x3f, 0x7e, 0xaf, 0x86, 0x55, 0x71, 0x5e, 0x8d, - 0x4e, 0x1a, 0x0c, 0x7b, 0x90, 0x7f, 0x2c, 0xb2, 0x6d, 0xb5, 0x44, 0xba, 0x8d, 0xc2, 0x8c, 0xf3, - 0x68, 0x5c, 0x21, 0x18, 0x1e, 0x8f, 0xab, 0xa5, 0x09, 0x05, 0x3f, 0x4e, 0xaf, 0x15, 0x84, 0x82, - 0x92, 0x8c, 0xe2, 0x8c, 0xf3, 0x66, 0xb2, 0xa0, 0xf0, 0x92, 0x3f, 0x59, 0x5e, 0x0f, 0x00, 0x78, - 0x44, 0x6c, 0x6a, 0xc9, 0x42, 0xd4, 0x65, 0x95, 0x7d, 0x72, 0x7a, 0x24, 0xc6, 0xd5, 0x64, 0x1e, - 0x0a, 0xf5, 0x1d, 0xc2, 0xc9, 0x8d, 0x1e, 0x8f, 0x8e, 0x70, 0x8e, 0x27, 0xdf, 0xe8, 0x16, 0xe8, - 0xb6, 0x1f, 0x30, 0xea, 0x58, 0xe3, 0x99, 0xb1, 0x32, 0x03, 0xb1, 0xe0, 0x62, 0xac, 0x36, 0x12, - 0xac, 0x5d, 0x83, 0xc2, 0xb4, 0x19, 0xa4, 0x43, 0x3a, 0x79, 0x85, 0xe4, 0xb0, 0x58, 0x8a, 0xf7, - 0xcf, 0x80, 0xf8, 0xfd, 0x78, 0x0a, 0xe7, 0x70, 0xfc, 0x71, 0x75, 0xfe, 0x8a, 0x68, 0xd6, 0xb4, - 0x9e, 0x19, 0xb5, 0xec, 0xbc, 0x9e, 0x8e, 0xdb, 0xf1, 0xe7, 0x83, 0xca, 0x15, 0x28, 0xec, 0xab, - 0x92, 0xbf, 0x19, 0x04, 0x9c, 0x46, 0xb3, 0xf6, 0x46, 0xc5, 0x85, 0xb3, 0xf5, 0x69, 0x27, 0x15, - 0xc0, 0xc1, 0x7b, 0x5e, 0x5a, 0x15, 0xd2, 0x9b, 0x57, 0xaf, 0x5d, 0x7a, 0xf6, 0xb2, 0x34, 0xf7, - 0xec, 0xb8, 0x94, 0x7a, 0x7e, 0x5c, 0x4a, 0xbd, 0x38, 0x2e, 0xa5, 0x7e, 0x3f, 0x2e, 0xa5, 0xbe, - 0x7d, 0x55, 0x9a, 0x7b, 0xfe, 0xaa, 0x34, 0xf7, 0xe2, 0x55, 0x69, 0xee, 0x01, 0x8c, 0xff, 0x8b, - 0x5a, 0x0b, 0xf2, 0x17, 0x67, 0xeb, 0xcf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x05, 0x7f, 0x93, 0xe3, - 0xfd, 0x0d, 0x00, 0x00, + 0x16, 0x8f, 0x63, 0x3b, 0x59, 0x8f, 0x13, 0x7b, 0x33, 0x04, 0xd8, 0x9b, 0x7b, 0xaf, 0x1d, 0xb9, + 0x14, 0xa5, 0x94, 0xae, 0x51, 0xd2, 0x4a, 0x88, 0xa2, 0x8a, 0xd8, 0x81, 0x12, 0x93, 0xa4, 0x30, + 0x0e, 0xb4, 0xa2, 0x95, 0x56, 0xe3, 0xdd, 0x61, 0xbd, 0xf5, 0x7a, 0x77, 0x99, 0x19, 0x1b, 0xf2, + 0x29, 0xda, 0x4a, 0x95, 0xda, 0xa7, 0xc2, 0x63, 0x3f, 0x0a, 0x8f, 0x3c, 0xa2, 0x3e, 0x58, 0x25, + 0xbc, 0xf4, 0x33, 0xf0, 0x54, 0xcd, 0xec, 0xac, 0xbd, 0x41, 0xa1, 0x31, 0xf4, 0x6d, 0xf6, 0xcc, + 0x39, 0xbf, 0x33, 0x73, 0xfe, 0xfc, 0xce, 0x2c, 0x58, 0xeb, 0x0d, 0xeb, 0xbd, 0x21, 0x23, 0x74, + 0x48, 0xe8, 0x78, 0x11, 0x75, 0xea, 0x11, 0x0d, 0xa3, 0x90, 0x11, 0x6a, 0xf5, 0x86, 0x66, 0x44, + 0x43, 0x1e, 0xc2, 0xaa, 0x1d, 0xda, 0x3d, 0x1a, 0x62, 0xbb, 0x6b, 0xf6, 0x86, 0x66, 0xa2, 0x6a, + 0x32, 0x1e, 0x52, 0xec, 0x92, 0xa8, 0xb3, 0xb2, 0x24, 0x37, 0xa3, 0x4e, 0x1d, 0x47, 0x5e, 0x6c, + 0xb3, 0x02, 0x13, 0x91, 0x83, 0x39, 0x56, 0xb2, 0x33, 0x89, 0xac, 0x4f, 0x38, 0x4e, 0xc9, 0xff, + 0xab, 0x90, 0xea, 0x24, 0x70, 0xbd, 0x80, 0x08, 0x85, 0xa1, 0x6d, 0xab, 0xcd, 0xff, 0x1d, 0xbb, + 0xb9, 0xa1, 0x76, 0x6b, 0x6f, 0xb9, 0x04, 0xe3, 0x98, 0x13, 0xa5, 0x73, 0x21, 0xad, 0x43, 0x09, + 0x76, 0xd8, 0xa0, 0xdf, 0xc7, 0xf4, 0xa0, 0x4e, 0x99, 0xd0, 0x8c, 0x3f, 0x94, 0xae, 0x31, 0xe0, + 0x9e, 0x5f, 0xef, 0xfa, 0x76, 0x9d, 0x7b, 0x7d, 0xc2, 0x38, 0xee, 0x47, 0x6a, 0x67, 0xd9, 0x0d, + 0xdd, 0x50, 0x2e, 0xeb, 0x62, 0x15, 0x4b, 0x6b, 0xbf, 0x67, 0x40, 0xbe, 0x1d, 0xf9, 0x1e, 0x87, + 0x4d, 0x30, 0xcf, 0xa9, 0xe7, 0xba, 0x84, 0x1a, 0x99, 0xd5, 0xcc, 0x5a, 0x71, 0xbd, 0x6a, 0x4e, + 0xc2, 0xa6, 0x2e, 0x6e, 0x4a, 0xd5, 0xfd, 0x58, 0xad, 0xa1, 0x3d, 0x1b, 0x55, 0x67, 0x9e, 0x8f, + 0xaa, 0x19, 0x94, 0x58, 0xc2, 0x7d, 0x50, 0xa0, 0x5d, 0x66, 0x39, 0xc4, 0xe7, 0xd8, 0x98, 0x95, + 0x30, 0x1f, 0xa6, 0x60, 0x54, 0x28, 0xcc, 0x24, 0x14, 0xe6, 0xee, 0xbd, 0x66, 0xb3, 0xcd, 0x31, + 0x67, 0x0d, 0x5d, 0x80, 0x1d, 0x8e, 0xaa, 0x1a, 0xba, 0xd9, 0xde, 0x12, 0xe6, 0x48, 0xa3, 0x5d, + 0x26, 0x57, 0x57, 0x72, 0x7f, 0x3d, 0xad, 0x66, 0x6a, 0x08, 0xe4, 0x77, 0x09, 0x75, 0xc9, 0x74, + 0x27, 0x95, 0xaa, 0x6f, 0x3f, 0xa9, 0xc2, 0x74, 0x40, 0xa9, 0xd9, 0xc5, 0x81, 0x4b, 0x10, 0x89, + 0x7c, 0xcf, 0xc6, 0x0c, 0xee, 0xbc, 0x09, 0xbe, 0x76, 0x0c, 0xf8, 0x51, 0x9b, 0x7f, 0xf2, 0xf2, + 0xeb, 0xd3, 0xea, 0x4c, 0xed, 0xe5, 0x2c, 0x28, 0x37, 0xc3, 0x7e, 0x34, 0xe0, 0xa4, 0xd9, 0x25, + 0x76, 0x8f, 0x0d, 0xfa, 0xf0, 0x7b, 0x50, 0xb4, 0xd5, 0xda, 0xf2, 0x1c, 0xe9, 0x6b, 0xa1, 0xb1, + 0x2d, 0x10, 0xfe, 0x18, 0x55, 0x37, 0x5c, 0x8f, 0x77, 0x07, 0x1d, 0xd3, 0x0e, 0xfb, 0xf5, 0xb1, + 0x77, 0xa7, 0x33, 0x59, 0xd7, 0xa3, 0x9e, 0x5b, 0x97, 0xa9, 0x1e, 0x0c, 0x3c, 0xc7, 0xbc, 0x7b, + 0x77, 0x7b, 0xeb, 0x70, 0x54, 0x05, 0x09, 0xfa, 0xf6, 0x16, 0x02, 0x09, 0xfa, 0xb6, 0x03, 0x3f, + 0x00, 0x8b, 0x0c, 0x0f, 0x89, 0xc5, 0x02, 0x1c, 0xb1, 0x6e, 0xc8, 0x65, 0x66, 0x34, 0xb4, 0x20, + 0x84, 0x6d, 0x25, 0x83, 0x1b, 0x20, 0xd7, 0x0f, 0x1d, 0x62, 0x64, 0x57, 0x33, 0x6b, 0xa5, 0x63, + 0x43, 0x9a, 0xa0, 0xef, 0x86, 0x0e, 0x41, 0x52, 0x19, 0x56, 0x40, 0xec, 0x27, 0x0a, 0xbd, 0x80, + 0x1b, 0x39, 0x09, 0x9b, 0x92, 0x40, 0x03, 0xcc, 0x0f, 0x09, 0x65, 0x5e, 0x18, 0x18, 0xf9, 0xd5, + 0xcc, 0xda, 0x22, 0x4a, 0x3e, 0xe1, 0x4d, 0x50, 0xe0, 0x84, 0xf6, 0xbd, 0x00, 0x73, 0x62, 0xcc, + 0xad, 0x66, 0xd7, 0x8a, 0xeb, 0xe7, 0x8e, 0xf1, 0xa9, 0x62, 0xbc, 0x45, 0x98, 0x4d, 0xbd, 0x88, + 0x87, 0xb4, 0x91, 0x13, 0x31, 0x42, 0x13, 0x63, 0x95, 0xc9, 0x7b, 0x00, 0x88, 0x10, 0x63, 0x9b, + 0x0b, 0xf4, 0x65, 0x90, 0xef, 0x1c, 0x70, 0xc2, 0x64, 0x5c, 0xb3, 0x28, 0xfe, 0x80, 0x17, 0x01, + 0x64, 0x03, 0xd7, 0x25, 0x8c, 0x13, 0xc7, 0xc2, 0xdc, 0x0a, 0x70, 0x10, 0x32, 0x19, 0x8c, 0x2c, + 0xd2, 0xc7, 0x3b, 0x9b, 0x7c, 0x4f, 0xc8, 0x15, 0xee, 0xcf, 0xb3, 0xe0, 0x54, 0x3b, 0xd9, 0x4a, + 0x79, 0xb8, 0x03, 0x0a, 0x8c, 0x63, 0xca, 0xad, 0x1e, 0x39, 0x50, 0xd9, 0xfb, 0xf4, 0xf5, 0xa8, + 0x7a, 0x69, 0xaa, 0xcc, 0x25, 0xb7, 0xbb, 0x45, 0x0e, 0x90, 0x26, 0x61, 0x6e, 0x91, 0x03, 0xb8, + 0x0b, 0xe6, 0x49, 0xe0, 0x48, 0xc0, 0xd9, 0x7f, 0x01, 0x38, 0x47, 0x02, 0x47, 0xc0, 0xdd, 0x05, + 0xc0, 0x1e, 0x9f, 0x57, 0xa6, 0xb5, 0xb8, 0xfe, 0xb1, 0x79, 0x02, 0x15, 0x9a, 0x93, 0x2b, 0xa6, + 0xea, 0x39, 0x05, 0xa4, 0xc2, 0xf2, 0xa4, 0x00, 0x96, 0x55, 0x6e, 0x38, 0x71, 0xae, 0x0f, 0xb1, + 0x8f, 0x08, 0x1b, 0xf8, 0x82, 0x46, 0xf2, 0x92, 0xbb, 0x54, 0xf7, 0x7f, 0x72, 0xa2, 0x43, 0x85, + 0x22, 0x58, 0x80, 0xa0, 0xd8, 0x16, 0x5e, 0x05, 0x79, 0x26, 0x98, 0x46, 0x9d, 0xfa, 0xfc, 0x89, + 0x20, 0x92, 0x97, 0x50, 0x6c, 0x24, 0xac, 0xfb, 0xa2, 0xfb, 0x65, 0x3d, 0x4e, 0x63, 0x2d, 0xb9, + 0x02, 0xc5, 0x46, 0x70, 0x0d, 0xe8, 0x1e, 0xb3, 0x7c, 0x82, 0x19, 0xb1, 0x28, 0x79, 0x38, 0x20, + 0x8c, 0x1b, 0x73, 0xb2, 0xb0, 0x4b, 0x1e, 0xdb, 0x11, 0x62, 0x14, 0x4b, 0xe1, 0x26, 0x28, 0x8c, + 0x49, 0xd6, 0xd0, 0xa4, 0xaf, 0xff, 0xa7, 0x7c, 0x89, 0xf6, 0x34, 0xbb, 0xbe, 0x6d, 0xee, 0x27, + 0x4a, 0xe3, 0xda, 0x4d, 0x04, 0xf0, 0x36, 0xd0, 0x1d, 0x12, 0x51, 0x22, 0xa3, 0xa8, 0x68, 0x13, + 0xbc, 0x03, 0x6d, 0xa2, 0xf2, 0xc4, 0x5c, 0x72, 0x25, 0xfc, 0x06, 0x94, 0x6d, 0xc9, 0x4e, 0x16, + 0x55, 0xf4, 0x64, 0x2c, 0x48, 0xc0, 0xfa, 0xc9, 0xa9, 0x3f, 0xc2, 0x6a, 0xa8, 0x64, 0x1f, 0x65, + 0xc6, 0x73, 0xa0, 0x44, 0xf1, 0x03, 0x6e, 0xf9, 0xa1, 0xab, 0x4e, 0xba, 0x28, 0x3b, 0x67, 0x41, + 0x48, 0x77, 0x42, 0x37, 0xf6, 0xff, 0x10, 0x14, 0xb1, 0xe3, 0x58, 0x8c, 0x71, 0xdc, 0xf1, 0x89, + 0xb1, 0x24, 0x7d, 0x5f, 0x9b, 0xb6, 0x0a, 0x8e, 0xd4, 0x92, 0xb9, 0xe9, 0x38, 0xed, 0xf6, 0xbe, + 0xc0, 0x69, 0x94, 0x04, 0xbd, 0x4d, 0xbe, 0x11, 0xc0, 0x8e, 0xd3, 0x8e, 0x7d, 0xc0, 0x1b, 0x20, + 0x1f, 0x9f, 0x07, 0x4a, 0x67, 0x17, 0xa6, 0x8a, 0x9c, 0x3c, 0xad, 0x4a, 0x48, 0x6c, 0x0e, 0x7f, + 0xc8, 0x80, 0x53, 0x11, 0x25, 0x43, 0x95, 0xfc, 0xf8, 0x1d, 0x81, 0x7d, 0x63, 0x79, 0x9a, 0xd4, + 0x5e, 0x7b, 0x3d, 0xaa, 0x5e, 0x9d, 0x9e, 0xb6, 0x85, 0x71, 0xd3, 0x0f, 0xed, 0xde, 0x18, 0x01, + 0x2d, 0x09, 0xdf, 0xb2, 0xc0, 0x6e, 0x2b, 0xcf, 0xf0, 0x5b, 0xa0, 0xdb, 0xf1, 0xdc, 0xb0, 0x12, + 0x3a, 0x37, 0x4e, 0xcb, 0xd3, 0x5c, 0x9a, 0xaa, 0x91, 0x53, 0x03, 0x07, 0x95, 0xed, 0x37, 0x26, + 0xd0, 0x77, 0x00, 0x46, 0xd4, 0x0b, 0xa9, 0x25, 0x9e, 0x14, 0x96, 0x7a, 0x46, 0x18, 0x67, 0x24, + 0xbc, 0xf9, 0x16, 0xf8, 0xd4, 0xeb, 0xc3, 0x44, 0x04, 0x3b, 0xed, 0x78, 0x8d, 0x74, 0x89, 0x94, + 0x92, 0xac, 0x7c, 0x09, 0x52, 0xe9, 0x82, 0x10, 0xe4, 0xc4, 0x7b, 0x29, 0x26, 0x4a, 0x24, 0xd7, + 0xb0, 0x0a, 0xf2, 0x36, 0xb5, 0x37, 0xd6, 0x25, 0x53, 0x2c, 0x36, 0x0a, 0x87, 0xa3, 0x6a, 0xbe, + 0x89, 0x9a, 0x1b, 0xeb, 0x28, 0x96, 0xc7, 0x4c, 0xd3, 0xca, 0x69, 0x19, 0x7d, 0xb6, 0x95, 0xd3, + 0xf2, 0xfa, 0x5c, 0x2b, 0xa7, 0xcd, 0xeb, 0x5a, 0x2b, 0xa7, 0x15, 0x74, 0xd0, 0xca, 0x69, 0x25, + 0xbd, 0xdc, 0xca, 0x69, 0x65, 0x5d, 0x6f, 0xe5, 0x34, 0x5d, 0x5f, 0x6a, 0xe5, 0xb4, 0x53, 0xfa, + 0x72, 0x6b, 0x4e, 0xfb, 0x69, 0x4f, 0xff, 0x6d, 0xaf, 0xb6, 0x0a, 0xc0, 0xd7, 0xd4, 0xe3, 0xa4, + 0x81, 0xb9, 0xdd, 0x3d, 0xee, 0x00, 0xb5, 0x3b, 0x60, 0x61, 0x27, 0x74, 0x3d, 0x1b, 0xfb, 0x5f, + 0x45, 0x3b, 0xa1, 0x0b, 0x37, 0x41, 0x36, 0x8c, 0xc4, 0xc8, 0x10, 0xc3, 0xe8, 0xa3, 0x93, 0xaa, + 0x68, 0x6c, 0xaa, 0x8a, 0x48, 0xd8, 0xd6, 0x7e, 0x99, 0x03, 0x45, 0x84, 0x1f, 0xf0, 0x66, 0xd8, + 0xef, 0xe3, 0xc0, 0x81, 0xe7, 0x41, 0xb9, 0x8f, 0x1f, 0xab, 0x82, 0xf2, 0x02, 0x87, 0x3c, 0x96, + 0xa4, 0x94, 0x43, 0x8b, 0x7d, 0xfc, 0x58, 0xe6, 0x7a, 0x5b, 0x08, 0xe1, 0x3e, 0xf8, 0x4f, 0x8a, + 0x07, 0xc6, 0x2f, 0x58, 0x69, 0x27, 0x27, 0x67, 0x71, 0xdd, 0x38, 0x66, 0x3a, 0xc6, 0x74, 0x74, + 0x76, 0x62, 0x7a, 0x5b, 0x59, 0xca, 0x0d, 0x38, 0x04, 0x67, 0x8f, 0x42, 0x59, 0x4c, 0x50, 0x57, + 0x60, 0x13, 0xc9, 0x68, 0xd9, 0xc6, 0x17, 0xaf, 0x47, 0xd5, 0x2b, 0xef, 0x34, 0x60, 0x24, 0x70, + 0x5b, 0xa1, 0xa0, 0xd3, 0x51, 0xda, 0x5f, 0x22, 0x86, 0x0f, 0xc1, 0x19, 0x3a, 0xee, 0x67, 0x8b, + 0x0c, 0xb1, 0x6f, 0x51, 0xd9, 0xd1, 0x92, 0x31, 0x8a, 0xeb, 0x9f, 0xbd, 0x17, 0x1d, 0xa8, 0x38, + 0x2f, 0xd3, 0xe3, 0xc6, 0xce, 0x0e, 0x28, 0x3e, 0x12, 0xd9, 0xb6, 0x3a, 0x22, 0xdd, 0x46, 0x69, + 0xca, 0x69, 0x37, 0xa9, 0x10, 0x04, 0x1e, 0x4d, 0xaa, 0xa5, 0x0d, 0x4a, 0x7e, 0x9c, 0x5e, 0x2b, + 0x8c, 0x04, 0xe1, 0x19, 0xe5, 0x29, 0xa7, 0x59, 0xba, 0xa0, 0xd0, 0x82, 0x9f, 0x2e, 0xaf, 0xfb, + 0x00, 0x70, 0x8a, 0x6d, 0x62, 0xc9, 0x42, 0xd4, 0x65, 0x95, 0x7d, 0x7e, 0x72, 0x24, 0x26, 0xd5, + 0x64, 0xee, 0x0b, 0xf3, 0x2d, 0xcc, 0xf1, 0xf5, 0x80, 0xd3, 0x03, 0x54, 0xe0, 0xc9, 0x37, 0xbc, + 0x09, 0x74, 0xdb, 0x0f, 0x19, 0x71, 0xac, 0xc9, 0x44, 0x5a, 0x9a, 0x82, 0xb6, 0x50, 0x39, 0x36, + 0x1b, 0x0b, 0x56, 0xae, 0x82, 0xd2, 0x51, 0x37, 0x50, 0x07, 0xd9, 0xe4, 0x8d, 0x53, 0x40, 0x62, + 0x29, 0x5e, 0x57, 0x43, 0xec, 0x0f, 0xe2, 0x19, 0x5f, 0x40, 0xf1, 0xc7, 0x95, 0xd9, 0xcb, 0xa2, + 0x59, 0xb3, 0x7a, 0x6e, 0xdc, 0xb2, 0xb3, 0x7a, 0x36, 0x6e, 0xc7, 0x27, 0x7b, 0xb5, 0xcb, 0xa0, + 0xb4, 0xab, 0x4a, 0xfe, 0x46, 0x18, 0x72, 0x42, 0xa7, 0xed, 0x8d, 0x9a, 0x0b, 0x4e, 0x37, 0x8f, + 0x1e, 0x52, 0x01, 0xec, 0xbd, 0xe7, 0xa5, 0x55, 0x21, 0xbd, 0x79, 0xf5, 0xc6, 0xc5, 0x67, 0x2f, + 0x2b, 0x33, 0xcf, 0x0e, 0x2b, 0x99, 0xe7, 0x87, 0x95, 0xcc, 0x8b, 0xc3, 0x4a, 0xe6, 0xcf, 0xc3, + 0x4a, 0xe6, 0xc7, 0x57, 0x95, 0x99, 0xe7, 0xaf, 0x2a, 0x33, 0x2f, 0x5e, 0x55, 0x66, 0xee, 0x83, + 0xc9, 0x1f, 0x5a, 0x67, 0x4e, 0xfe, 0x40, 0x6d, 0xfc, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x25, 0x40, + 0x71, 0x9a, 0x87, 0x0e, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto index f0a7e355cc90..4a6b28133b88 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto @@ -18,6 +18,7 @@ import "roachpb/metadata.proto"; import "storage/enginepb/mvcc.proto"; import "storage/enginepb/mvcc3.proto"; import "kv/kvserver/kvserverpb/state.proto"; +import "kv/kvserver/readsummary/rspb/summary.proto"; import "util/hlc/timestamp.proto"; import "gogoproto/gogo.proto"; @@ -158,6 +159,34 @@ message ReplicatedEvalResult { // but before we tried to apply it. util.hlc.Timestamp prev_lease_proposal = 20 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; + // PriorReadSummary is a summary of the reads that have been served on the + // range prior to this proposal, which must be a lease change (request or + // transfer) if the field is set. The read summary is used to update the new + // leaseholder's timestamp cache to prevent them from serving writes that + // violate previously served reads. + // + // The summary, when available, can be used in place of bumping the new + // leaseholder's timestamp cache to the new lease's start time. It has two + // distinct advantages: + // 1. it can transfer a higher-resolution snapshot of the reads on the range + // through a lease transfer, to make the lease transfers less disruptive to + // writes because the timestamp cache won't be bumped as high. + // 2. it can transfer information about reads with synthetic timestamps, which + // are not otherwise captured by the new lease's start time. + // + // When a ReadSummary is set in a ReplicatedEvalResult, there is always also a + // write to the RangePriorReadSummaryKey in the RaftCommand.WriteBatch. The + // persisted summary may be identical to the summary in this field, but it + // does not have to be. Notably, we intended for the summary included in the + // ReplicatedEvalResult to eventually be a much higher-resolution version of + // the ReadSummmary than the version persisted. This scheme of persisting a + // compressed ReadSummary indefinitely and including a higher-resolution + // ReadSummary on the RaftCommand allows us to optimize for the common case + // where the lease transfer is applied on the new leaseholder through Raft log + // application while ensuring correctness in the case where the lease transfer + // is applied on the new leaseholder through a Raft snapshot. + kv.kvserver.readsummary.ReadSummary prior_read_summary = 22; + reserved 1, 5, 7, 9, 14, 15, 16, 19, 10001 to 10013; } diff --git a/pkg/kv/kvserver/kvserverpb/state.pb.go b/pkg/kv/kvserver/kvserverpb/state.pb.go index b81a0c1f0d0d..ef7d34d1a284 100644 --- a/pkg/kv/kvserver/kvserverpb/state.pb.go +++ b/pkg/kv/kvserver/kvserverpb/state.pb.go @@ -51,9 +51,9 @@ type ReplicaState struct { // The latest range lease. // // Note that this message is both sent over the network and used to model - // replica state in memory. In memory (storage.Replica.mu.state), the lease - // is never nil (and never zero-valued), but it may be nil when sent over - // the network as part of ReplicatedEvalResult. + // replica state in memory. In memory (Replica.mu.state), the lease is never + // nil (and never zero-valued), but it may be nil when sent over the network + // as part of ReplicatedEvalResult. Lease *roachpb.Lease `protobuf:"bytes,4,opt,name=lease,proto3" json:"lease,omitempty"` // The truncation state of the Raft log. TruncatedState *roachpb.RaftTruncatedState `protobuf:"bytes,5,opt,name=truncated_state,json=truncatedState,proto3" json:"truncated_state,omitempty"` @@ -115,7 +115,7 @@ func (m *ReplicaState) Reset() { *m = ReplicaState{} } func (m *ReplicaState) String() string { return proto.CompactTextString(m) } func (*ReplicaState) ProtoMessage() {} func (*ReplicaState) Descriptor() ([]byte, []int) { - return fileDescriptor_state_884b07f3590284d4, []int{0} + return fileDescriptor_state_cf048718a8ccf178, []int{0} } func (m *ReplicaState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -187,7 +187,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_state_884b07f3590284d4, []int{1} + return fileDescriptor_state_cf048718a8ccf178, []int{1} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -223,7 +223,7 @@ func (m *RangeInfo_CTEntry) Reset() { *m = RangeInfo_CTEntry{} } func (m *RangeInfo_CTEntry) String() string { return proto.CompactTextString(m) } func (*RangeInfo_CTEntry) ProtoMessage() {} func (*RangeInfo_CTEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_state_884b07f3590284d4, []int{1, 0} + return fileDescriptor_state_cf048718a8ccf178, []int{1, 0} } func (m *RangeInfo_CTEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -259,7 +259,7 @@ func (m *LatchManagerInfo) Reset() { *m = LatchManagerInfo{} } func (m *LatchManagerInfo) String() string { return proto.CompactTextString(m) } func (*LatchManagerInfo) ProtoMessage() {} func (*LatchManagerInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_state_884b07f3590284d4, []int{2} + return fileDescriptor_state_cf048718a8ccf178, []int{2} } func (m *LatchManagerInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2002,10 +2002,10 @@ var ( ) func init() { - proto.RegisterFile("kv/kvserver/kvserverpb/state.proto", fileDescriptor_state_884b07f3590284d4) + proto.RegisterFile("kv/kvserver/kvserverpb/state.proto", fileDescriptor_state_cf048718a8ccf178) } -var fileDescriptor_state_884b07f3590284d4 = []byte{ +var fileDescriptor_state_cf048718a8ccf178 = []byte{ // 1068 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0xcf, 0x4f, 0x1b, 0x47, 0x14, 0x66, 0xe3, 0x35, 0xd8, 0x63, 0x08, 0xce, 0x14, 0xc2, 0x86, 0x04, 0x1b, 0x59, 0x6a, 0x45, diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto index 45e07446177e..ffe40837abc4 100644 --- a/pkg/kv/kvserver/kvserverpb/state.proto +++ b/pkg/kv/kvserver/kvserverpb/state.proto @@ -48,9 +48,9 @@ message ReplicaState { // The latest range lease. // // Note that this message is both sent over the network and used to model - // replica state in memory. In memory (storage.Replica.mu.state), the lease - // is never nil (and never zero-valued), but it may be nil when sent over - // the network as part of ReplicatedEvalResult. + // replica state in memory. In memory (Replica.mu.state), the lease is never + // nil (and never zero-valued), but it may be nil when sent over the network + // as part of ReplicatedEvalResult. roachpb.Lease lease = 4; // The truncation state of the Raft log. roachpb.RaftTruncatedState truncated_state = 5; diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index 1ac4ea2759ec..924d7e425dee 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -36,9 +36,9 @@ const ( // a large purgatory interval. mergeQueuePurgatoryCheckInterval = 1 * time.Minute - // The current implementation of merges requires rewriting the right-hand data - // onto the left-hand range, even when the ranges are collocated. This is - // expensive, so limit to one merge at a time. + // The current implementation of merges requires rebalancing replicas on the + // right-hand range so that they are collocated with those on the left-hand + // range. This is expensive, so limit to one merge at a time. mergeQueueConcurrency = 1 ) diff --git a/pkg/kv/kvserver/readsummary/BUILD.bazel b/pkg/kv/kvserver/readsummary/BUILD.bazel new file mode 100644 index 000000000000..1cf25b09faf6 --- /dev/null +++ b/pkg/kv/kvserver/readsummary/BUILD.bazel @@ -0,0 +1,16 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "readsummary", + srcs = ["persist.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary", + visibility = ["//visibility:public"], + deps = [ + "//pkg/keys", + "//pkg/kv/kvserver/readsummary/rspb", + "//pkg/roachpb", + "//pkg/storage", + "//pkg/storage/enginepb", + "//pkg/util/hlc", + ], +) diff --git a/pkg/kv/kvserver/readsummary/persist.go b/pkg/kv/kvserver/readsummary/persist.go new file mode 100644 index 000000000000..9c03878591ac --- /dev/null +++ b/pkg/kv/kvserver/readsummary/persist.go @@ -0,0 +1,48 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package readsummary + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" +) + +// Load loads the range's prior read summary. The function returns a nil summary +// if one does not already exist. +func Load( + ctx context.Context, reader storage.Reader, rangeID roachpb.RangeID, +) (*rspb.ReadSummary, error) { + var sum rspb.ReadSummary + key := keys.RangePriorReadSummaryKey(rangeID) + found, err := storage.MVCCGetProto(ctx, reader, key, hlc.Timestamp{}, &sum, storage.MVCCGetOptions{}) + if !found { + return nil, err + } + return &sum, err +} + +// Set persists a range's prior read summary. +func Set( + ctx context.Context, + readWriter storage.ReadWriter, + rangeID roachpb.RangeID, + ms *enginepb.MVCCStats, + sum *rspb.ReadSummary, +) error { + key := keys.RangePriorReadSummaryKey(rangeID) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil, sum) +} diff --git a/pkg/kv/kvserver/readsummary/rspb/BUILD.bazel b/pkg/kv/kvserver/readsummary/rspb/BUILD.bazel new file mode 100644 index 000000000000..acc363d5d27e --- /dev/null +++ b/pkg/kv/kvserver/readsummary/rspb/BUILD.bazel @@ -0,0 +1,38 @@ +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") + +proto_library( + name = "rspb_proto", + srcs = ["summary.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = [ + "//pkg/util/hlc:hlc_proto", + "@com_github_gogo_protobuf//gogoproto:gogo_proto", + ], +) + +go_proto_library( + name = "rspb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb", + proto = ":rspb_proto", + visibility = ["//visibility:public"], + deps = [ + "//pkg/util/hlc", + "@com_github_gogo_protobuf//gogoproto", + ], +) + +go_library( + name = "rspb", + srcs = ["summary.go"], + embed = [":rspb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb", + visibility = ["//visibility:public"], + deps = [ + "//pkg/util/hlc", + "//pkg/util/log", + ], +) diff --git a/pkg/kv/kvserver/readsummary/rspb/summary.go b/pkg/kv/kvserver/readsummary/rspb/summary.go new file mode 100644 index 000000000000..45cbd0b4cd8d --- /dev/null +++ b/pkg/kv/kvserver/readsummary/rspb/summary.go @@ -0,0 +1,64 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package rspb + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// FromTimestamp constructs a read summary from the provided timestamp, treating +// the argument as the low water mark of each segment in the summary. +func FromTimestamp(ts hlc.Timestamp) ReadSummary { + seg := Segment{LowWater: ts} + return ReadSummary{ + Local: seg, + Global: seg, + } +} + +// Clone performs a deep-copy of the receiver. +func (c ReadSummary) Clone() *ReadSummary { + // NOTE: When ReadSummary is updated to include pointers to non-contiguous + // memory, this will need to be updated. + return &c +} + +// Merge combines two read summaries, resulting in a single summary that +// reflects the combination of all reads in each original summary. The merge +// operation is commutative and idempotent. +func (c *ReadSummary) Merge(o ReadSummary) { + c.Local.merge(o.Local) + c.Global.merge(o.Global) +} + +func (c *Segment) merge(o Segment) { + c.LowWater.Forward(o.LowWater) +} + +// AssertNoRegression asserts that all reads in the parameter's summary are +// reflected in the receiver's summary with at least as high of a timestamp. +func (c *ReadSummary) AssertNoRegression(ctx context.Context, o ReadSummary) { + c.Local.assertNoRegression(ctx, o.Local, "local") + c.Global.assertNoRegression(ctx, o.Global, "global") +} + +func (c *Segment) assertNoRegression(ctx context.Context, o Segment, name string) { + if c.LowWater.Less(o.LowWater) { + log.Fatalf(ctx, "read summary regression in %s segment, was %s, now %s", + name, o.LowWater, c.LowWater) + } +} + +// Ignore unused warning. +var _ = (*ReadSummary).AssertNoRegression diff --git a/pkg/kv/kvserver/readsummary/rspb/summary.pb.go b/pkg/kv/kvserver/readsummary/rspb/summary.pb.go new file mode 100644 index 000000000000..30f72850a1fb --- /dev/null +++ b/pkg/kv/kvserver/readsummary/rspb/summary.pb.go @@ -0,0 +1,603 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: kv/kvserver/readsummary/rspb/summary.proto + +package rspb + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +// ReadSummary contains a summary of all read requests served on a range, along +// with the timestamps that those reads were served at. The structure is a +// summary in that it may not represent these reads with perfect precision. +// Instead, it is allowed to lose resolution in exchange for reduced space, as +// long as the resulting timestamp for each key does not regress. During such +// compression, the timestamp of a given key is only allowed to advance as +// precision drops. This parallels a similar ratcheting policy in the timestamp +// cache (tscache.Cache). +// +// For example, a high-resolution version of the summary may look like: +// +// # +// ^ ## ## # +// ts | # ## ## # ####### +// | # ## #### # ####### ## +// ### ## ####### ####### ## ### +// ----------- ---------------------------- +// local global +// +// While a low-resolution (compressed) version of the summary may look like: +// +// ############################ +// ^ ########### ############################ +// ts | ########### ############################ +// | ########### ############################ +// ########### ############################ +// ----------- ---------------------------- +// local global +// +type ReadSummary struct { + Local Segment `protobuf:"bytes,1,opt,name=local,proto3" json:"local"` + Global Segment `protobuf:"bytes,2,opt,name=global,proto3" json:"global"` +} + +func (m *ReadSummary) Reset() { *m = ReadSummary{} } +func (m *ReadSummary) String() string { return proto.CompactTextString(m) } +func (*ReadSummary) ProtoMessage() {} +func (*ReadSummary) Descriptor() ([]byte, []int) { + return fileDescriptor_summary_1534417f080e4598, []int{0} +} +func (m *ReadSummary) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReadSummary) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReadSummary) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReadSummary.Merge(dst, src) +} +func (m *ReadSummary) XXX_Size() int { + return m.Size() +} +func (m *ReadSummary) XXX_DiscardUnknown() { + xxx_messageInfo_ReadSummary.DiscardUnknown(m) +} + +var xxx_messageInfo_ReadSummary proto.InternalMessageInfo + +// Segment is a segement of a Range's complete ReadSummary. A ReadSummary has a +// segment for each of the Range's replicated, addressable contiguous keyspaces +// (i.e. range-local and global). +type Segment struct { + LowWater hlc.Timestamp `protobuf:"bytes,1,opt,name=low_water,json=lowWater,proto3" json:"low_water"` +} + +func (m *Segment) Reset() { *m = Segment{} } +func (m *Segment) String() string { return proto.CompactTextString(m) } +func (*Segment) ProtoMessage() {} +func (*Segment) Descriptor() ([]byte, []int) { + return fileDescriptor_summary_1534417f080e4598, []int{1} +} +func (m *Segment) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Segment) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Segment) XXX_Merge(src proto.Message) { + xxx_messageInfo_Segment.Merge(dst, src) +} +func (m *Segment) XXX_Size() int { + return m.Size() +} +func (m *Segment) XXX_DiscardUnknown() { + xxx_messageInfo_Segment.DiscardUnknown(m) +} + +var xxx_messageInfo_Segment proto.InternalMessageInfo + +func init() { + proto.RegisterType((*ReadSummary)(nil), "cockroach.kv.kvserver.readsummary.ReadSummary") + proto.RegisterType((*Segment)(nil), "cockroach.kv.kvserver.readsummary.Segment") +} +func (this *ReadSummary) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*ReadSummary) + if !ok { + that2, ok := that.(ReadSummary) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.Local.Equal(&that1.Local) { + return false + } + if !this.Global.Equal(&that1.Global) { + return false + } + return true +} +func (this *Segment) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*Segment) + if !ok { + that2, ok := that.(Segment) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.LowWater.Equal(&that1.LowWater) { + return false + } + return true +} +func (m *ReadSummary) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ReadSummary) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSummary(dAtA, i, uint64(m.Local.Size())) + n1, err := m.Local.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + dAtA[i] = 0x12 + i++ + i = encodeVarintSummary(dAtA, i, uint64(m.Global.Size())) + n2, err := m.Global.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + return i, nil +} + +func (m *Segment) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Segment) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSummary(dAtA, i, uint64(m.LowWater.Size())) + n3, err := m.LowWater.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + return i, nil +} + +func encodeVarintSummary(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *ReadSummary) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.Local.Size() + n += 1 + l + sovSummary(uint64(l)) + l = m.Global.Size() + n += 1 + l + sovSummary(uint64(l)) + return n +} + +func (m *Segment) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.LowWater.Size() + n += 1 + l + sovSummary(uint64(l)) + return n +} + +func sovSummary(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozSummary(x uint64) (n int) { + return sovSummary(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *ReadSummary) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSummary + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ReadSummary: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReadSummary: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Local", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSummary + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSummary + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Local.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Global", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSummary + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSummary + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Global.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSummary(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSummary + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Segment) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSummary + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Segment: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Segment: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LowWater", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSummary + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSummary + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.LowWater.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSummary(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSummary + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipSummary(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSummary + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSummary + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSummary + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthSummary + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSummary + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipSummary(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthSummary = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowSummary = fmt.Errorf("proto: integer overflow") +) + +func init() { + proto.RegisterFile("kv/kvserver/readsummary/rspb/summary.proto", fileDescriptor_summary_1534417f080e4598) +} + +var fileDescriptor_summary_1534417f080e4598 = []byte{ + // 282 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xd2, 0xca, 0x2e, 0xd3, 0xcf, + 0x2e, 0x2b, 0x4e, 0x2d, 0x2a, 0x4b, 0x2d, 0xd2, 0x2f, 0x4a, 0x4d, 0x4c, 0x29, 0x2e, 0xcd, 0xcd, + 0x4d, 0x2c, 0xaa, 0xd4, 0x2f, 0x2a, 0x2e, 0x48, 0xd2, 0x87, 0x72, 0xf4, 0x0a, 0x8a, 0xf2, 0x4b, + 0xf2, 0x85, 0x14, 0x93, 0xf3, 0x93, 0xb3, 0x8b, 0xf2, 0x13, 0x93, 0x33, 0xf4, 0xb2, 0xcb, 0xf4, + 0x60, 0xba, 0xf4, 0x90, 0x74, 0x49, 0x49, 0x94, 0x96, 0x64, 0xe6, 0xe8, 0x67, 0xe4, 0x24, 0xeb, + 0x97, 0x64, 0xe6, 0xa6, 0x16, 0x97, 0x24, 0xe6, 0x16, 0x40, 0x34, 0x4b, 0x89, 0xa4, 0xe7, 0xa7, + 0xe7, 0x83, 0x99, 0xfa, 0x20, 0x16, 0x44, 0x54, 0x69, 0x29, 0x23, 0x17, 0x77, 0x50, 0x6a, 0x62, + 0x4a, 0x30, 0x44, 0xbf, 0x90, 0x1b, 0x17, 0x6b, 0x4e, 0x7e, 0x72, 0x62, 0x8e, 0x04, 0xa3, 0x02, + 0xa3, 0x06, 0xb7, 0x91, 0x96, 0x1e, 0x41, 0x2b, 0xf5, 0x82, 0x53, 0xd3, 0x73, 0x53, 0xf3, 0x4a, + 0x9c, 0x58, 0x4e, 0xdc, 0x93, 0x67, 0x08, 0x82, 0x68, 0x17, 0xf2, 0xe0, 0x62, 0x4b, 0xcf, 0xc9, + 0x4f, 0x4a, 0xcc, 0x91, 0x60, 0x22, 0xd3, 0x20, 0xa8, 0x7e, 0x2b, 0x96, 0x17, 0x0b, 0xe4, 0x19, + 0x95, 0x02, 0xb9, 0xd8, 0xa1, 0xd2, 0x42, 0x0e, 0x5c, 0x9c, 0x39, 0xf9, 0xe5, 0xf1, 0xe5, 0x89, + 0x25, 0xa9, 0x45, 0x50, 0x67, 0xca, 0x22, 0x99, 0x0e, 0x0a, 0x00, 0xbd, 0x8c, 0x9c, 0x64, 0xbd, + 0x10, 0x58, 0x00, 0x40, 0x0d, 0xe4, 0xc8, 0xc9, 0x2f, 0x0f, 0x07, 0x69, 0x82, 0x18, 0xe9, 0xa4, + 0x76, 0xe2, 0xa1, 0x1c, 0xc3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0xde, 0x78, 0x24, + 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, 0xcb, 0x31, 0xdc, 0x78, + 0x2c, 0xc7, 0x10, 0xc5, 0x02, 0x8a, 0x83, 0x24, 0x36, 0x70, 0x48, 0x19, 0x03, 0x02, 0x00, 0x00, + 0xff, 0xff, 0x74, 0x7b, 0xe2, 0x8b, 0xaa, 0x01, 0x00, 0x00, +} diff --git a/pkg/kv/kvserver/readsummary/rspb/summary.proto b/pkg/kv/kvserver/readsummary/rspb/summary.proto new file mode 100644 index 000000000000..2498cf270df0 --- /dev/null +++ b/pkg/kv/kvserver/readsummary/rspb/summary.proto @@ -0,0 +1,62 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto3"; +package cockroach.kv.kvserver.readsummary; +option go_package = "rspb"; + +import "util/hlc/timestamp.proto"; +import "gogoproto/gogo.proto"; + +// ReadSummary contains a summary of all read requests served on a range, along +// with the timestamps that those reads were served at. The structure is a +// summary in that it may not represent these reads with perfect precision. +// Instead, it is allowed to lose resolution in exchange for reduced space, as +// long as the resulting timestamp for each key does not regress. During such +// compression, the timestamp of a given key is only allowed to advance as +// precision drops. This parallels a similar ratcheting policy in the timestamp +// cache (tscache.Cache). +// +// For example, a high-resolution version of the summary may look like: +// +// # +// ^ ## ## # +// ts | # ## ## # ####### +// | # ## #### # ####### ## +// ### ## ####### ####### ## ### +// ----------- ---------------------------- +// local global +// +// While a low-resolution (compressed) version of the summary may look like: +// +// ############################ +// ^ ########### ############################ +// ts | ########### ############################ +// | ########### ############################ +// ########### ############################ +// ----------- ---------------------------- +// local global +// +message ReadSummary { + option (gogoproto.equal) = true; + + Segment local = 1 [(gogoproto.nullable) = false]; + Segment global = 2 [(gogoproto.nullable) = false]; +} + +// Segment is a segement of a Range's complete ReadSummary. A ReadSummary has a +// segment for each of the Range's replicated, addressable contiguous keyspaces +// (i.e. range-local and global). +message Segment { + option (gogoproto.equal) = true; + + util.hlc.Timestamp low_water = 1 [(gogoproto.nullable) = false]; + // TODO(nvanbenschoten): add higher resolution portion. +} diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 7991047e45df..d1b483773ac3 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -293,6 +293,13 @@ type Replica struct { // right-hand range in an ongoing merge. This range will allow read-only // traffic below this timestamp, while blocking everything else, until the // merge completes. + // TODO(nvanbenschoten): get rid of this. It seemed like a good idea at + // the time (b192bba), but in retrospect, it's a premature optimization + // that prevents us from being more optimal about the read summary we + // ship to the LHS during a merge. Serving reads below the closed + // timestamp seems fine because that can't advance after the range is + // frozen, but serving reads above the closed timestamp but below the + // freeze start time is dangerous. freezeStart hlc.Timestamp // The state of the Raft state machine. state kvserverpb.ReplicaState diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 90c9d0247bf1..45fba8ea0cd3 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -256,7 +257,13 @@ func (r *Replica) handleSplitResult(ctx context.Context, split *kvserverpb.Split func (r *Replica) handleMergeResult(ctx context.Context, merge *kvserverpb.Merge) { if err := r.store.MergeRange( - ctx, r, merge.LeftDesc, merge.RightDesc, merge.FreezeStart, merge.RightClosedTimestamp, + ctx, + r, + merge.LeftDesc, + merge.RightDesc, + merge.FreezeStart, + merge.RightClosedTimestamp, + merge.RightReadSummary, ); err != nil { // Our in-memory state has diverged from the on-disk state. log.Fatalf(ctx, "failed to update store after merging range: %s", err) @@ -267,12 +274,15 @@ func (r *Replica) handleDescResult(ctx context.Context, desc *roachpb.RangeDescr r.setDescRaftMuLocked(ctx, desc) } -func (r *Replica) handleLeaseResult(ctx context.Context, lease *roachpb.Lease) { +func (r *Replica) handleLeaseResult( + ctx context.Context, lease *roachpb.Lease, priorReadSum *rspb.ReadSummary, +) { r.mu.Lock() defer r.mu.Unlock() r.leasePostApplyLocked(ctx, r.mu.state.Lease, /* prevLease */ lease, /* newLease */ + priorReadSum, assertNoLeaseJump) } diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 69130255ea85..ffd6db203916 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -1149,12 +1149,13 @@ func (sm *replicaStateMachine) handleNonTrivialReplicatedEvalResult( if rResult.State != nil { if newLease := rResult.State.Lease; newLease != nil { - sm.r.handleLeaseResult(ctx, newLease) + sm.r.handleLeaseResult(ctx, newLease, rResult.PriorReadSummary) rResult.State.Lease = nil + rResult.PriorReadSummary = nil } - if rResult.State.TruncatedState != nil { - rResult.RaftLogDelta += sm.r.handleTruncatedStateResult(ctx, rResult.State.TruncatedState) + if newTruncState := rResult.State.TruncatedState; newTruncState != nil { + rResult.RaftLogDelta += sm.r.handleTruncatedStateResult(ctx, newTruncState) rResult.State.TruncatedState = nil } diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index c80a744f599a..5de61f2a5aaa 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -748,6 +748,7 @@ func (r *Replica) AdminMerge( RightMVCCStats: rhsSnapshotRes.MVCCStats, FreezeStart: rhsSnapshotRes.FreezeStart, RightClosedTimestamp: rhsSnapshotRes.ClosedTimestamp, + RightReadSummary: rhsSnapshotRes.ReadSummary, }, }, }) diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index 3eff1ee1bf0b..5088a69a0357 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" @@ -117,8 +118,8 @@ func (rec *SpanSetReplicaEvalContext) GetTracker() closedts.TrackerI { return rec.i.GetTracker() } -// FrozenClosedTimestamp is part of the EvalContext interface. -func (rec *SpanSetReplicaEvalContext) FrozenClosedTimestamp(ctx context.Context) hlc.Timestamp { +// GetFrozenClosedTimestamp is part of the EvalContext interface. +func (rec *SpanSetReplicaEvalContext) GetFrozenClosedTimestamp() hlc.Timestamp { // To capture a closed timestamp, all keys must be latched to prevent any // concurrent writes (which could advance the closed timestamp). desc := rec.i.Desc() @@ -130,7 +131,7 @@ func (rec *SpanSetReplicaEvalContext) FrozenClosedTimestamp(ctx context.Context) Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey(), }) - return rec.i.FrozenClosedTimestamp(ctx) + return rec.i.GetFrozenClosedTimestamp() } // IsFirstRange returns true iff the replica belongs to the first range. @@ -226,6 +227,22 @@ func (rec SpanSetReplicaEvalContext) GetRangeInfo(ctx context.Context) roachpb.R return rec.i.GetRangeInfo(ctx) } +// GetCurrentReadSummary is part of the EvalContext interface. +func (rec *SpanSetReplicaEvalContext) GetCurrentReadSummary() rspb.ReadSummary { + // To capture a read summary over the range, all keys must be latched for + // writing to prevent any concurrent reads or writes. + desc := rec.i.Desc() + rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{ + Key: keys.MakeRangeKeyPrefix(desc.StartKey), + EndKey: keys.MakeRangeKeyPrefix(desc.EndKey), + }) + rec.ss.AssertAllowed(spanset.SpanReadWrite, roachpb.Span{ + Key: desc.StartKey.AsRawKey(), + EndKey: desc.EndKey.AsRawKey(), + }) + return rec.i.GetCurrentReadSummary() +} + // GetLimiters returns the per-store limiters. func (rec *SpanSetReplicaEvalContext) GetLimiters() *batcheval.Limiters { return rec.i.GetLimiters() diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index fdc55e0dad92..4bbee7dfdb28 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -158,14 +158,14 @@ func (r *Replica) maxClosedRLocked(ctx context.Context) (_ hlc.Timestamp, ok boo return maxClosed, true } -// FrozenClosedTimestamp returns the closed timestamp. Unlike +// GetFrozenClosedTimestamp returns the closed timestamp. Unlike // MaxClosedTimestamp, it only looks at the "new" closed timestamp mechanism, // ignoring the old one. It returns an empty result if the new mechanism is not // enabled yet. The new mechanism has better properties than the old one - // namely the closing of timestamps is synchronized with subsumption requests // (through latches). Callers who need that property should be prepared to get // an empty result back, meaning that the closed timestamp cannot be known. -func (r *Replica) FrozenClosedTimestamp(ctx context.Context) hlc.Timestamp { +func (r *Replica) GetFrozenClosedTimestamp() hlc.Timestamp { r.mu.RLock() defer r.mu.RUnlock() // TODO(andrei): Make sure that this synchronizes with the closed timestamps diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index c170a2c6cc17..2d3da2289028 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -327,14 +328,24 @@ const ( // found out about newLease through a snapshot; in this case the replica might // not be aware of other lease changes that happened before the snapshot was // generated. This method thus tolerates prevLease being "stale" when -// allowLeaseJump is passed. prevLease can also be the same as newLease; see below. +// allowLeaseJump is passed. prevLease can also be the same as newLease; see +// below. // // newLease represents the lease being applied. Can be the same as prevLease. // This allows leasePostApplyLocked to be called for some of its side-effects // even if the lease in question has otherwise already been applied to the // range. +// +// In addition to the leases, the method accepts a summary of the reads served +// on the range by prior leaseholders. This can be used by the new leaseholder +// to ensure that no future writes are allowed to invalidate prior reads. If a +// summary is not provided, the method pessimistically assumes that prior +// leaseholders served reads all the way up to the start of the new lease. func (r *Replica) leasePostApplyLocked( - ctx context.Context, prevLease *roachpb.Lease, newLease *roachpb.Lease, jumpOpt leaseJumpOption, + ctx context.Context, + prevLease, newLease *roachpb.Lease, + priorReadSum *rspb.ReadSummary, + jumpOpt leaseJumpOption, ) { // Note that we actually install the lease further down in this method. // Everything we do before then doesn't need to worry about requests being @@ -398,18 +409,28 @@ func (r *Replica) leasePostApplyLocked( newLease, err) } - // If this replica is a new holder of the lease, update the low water - // mark of the timestamp cache. Note that clock offset scenarios are - // handled via a stasis period inherent in the lease which is documented - // in the Lease struct. + // If this replica is a new holder of the lease, update the timestamp + // cache. Note that clock offset scenarios are handled via a stasis + // period inherent in the lease which is documented in the Lease struct. + // + // If the Raft entry included a prior read summary then we can use that + // directly to update the timestamp cache. Otherwise, we pessimistically + // assume that prior leaseholders served reads all the way up to the + // start of the new lease. // // The introduction of lease transfers implies that the previous lease - // may have been shortened and we are now applying a formally overlapping - // lease (since the old lease holder has promised not to serve any more - // requests, this is kosher). This means that we don't use the old - // lease's expiration but instead use the new lease's start to initialize - // the timestamp cache low water. - setTimestampCacheLowWaterMark(r.store.tsCache, r.descRLocked(), newLease.Start.ToTimestamp()) + // may have been shortened and we are now applying a formally + // overlapping lease (since the old lease holder has promised not to + // serve any more requests, this is kosher). This means that we don't + // use the old lease's expiration but instead use the new lease's start + // to initialize the timestamp cache low water. + var sum rspb.ReadSummary + if priorReadSum != nil { + sum = *priorReadSum + } else { + sum = rspb.FromTimestamp(newLease.Start.ToTimestamp()) + } + applyReadSummaryToTimestampCache(r.store.tsCache, r.descRLocked(), sum) // Reset the request counts used to make lease placement decisions whenever // starting a new lease. diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 85ea16017361..611aebb8c123 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftentry" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -964,6 +965,14 @@ func (r *Replica) applySnapshot( log.Fatalf(ctx, "failed to clear in-memory data of subsumed replicas while applying snapshot: %+v", err) } + // Read the prior read summary for this range, which was included in the + // snapshot. We may need to use it to bump our timestamp cache if we + // discover that we are the leaseholder as of the snapshot's log index. + prioReadSum, err := readsummary.Load(ctx, r.store.engine, r.RangeID) + if err != nil { + log.Fatalf(ctx, "failed to read prior read summary after applying snapshot: %+v", err) + } + // Atomically swap the placeholder, if any, for the replica, and update the // replica's state. Note that this is intentionally in one critical section. // to avoid exposing an inconsistent in-memory state. We did however already @@ -1011,7 +1020,16 @@ func (r *Replica) applySnapshot( // replica according to whether it holds the lease. We allow jumps in the // lease sequence because there may be multiple lease changes accounted for // in the snapshot. - r.leasePostApplyLocked(ctx, lastKnownLease, s.Lease /* newLease */, allowLeaseJump) + r.leasePostApplyLocked(ctx, lastKnownLease, s.Lease /* newLease */, prioReadSum, allowLeaseJump) + + // Similarly, if we subsumed any replicas through the snapshot (meaning that + // we missed the application of a merge) and we are the new leaseholder, we + // make sure to update the timestamp cache using the prior read summary to + // account for any reads that were served on the right-hand side range(s). + if len(subsumedRepls) > 0 && s.Lease.Replica.ReplicaID == r.mu.replicaID && prioReadSum != nil { + applyReadSummaryToTimestampCache(r.store.tsCache, r.descRLocked(), *prioReadSum) + } + // Inform the concurrency manager that this replica just applied a snapshot. r.concMgr.OnReplicaSnapshotApplied() diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 054acf694083..8a4cb17102cc 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -6238,10 +6238,12 @@ func TestRangeStatsComputation(t *testing.T) { tc.Start(t, stopper) baseStats := initialStats() - // The initial stats contain no lease, but there will be an initial - // nontrivial lease requested with the first write below. + // The initial stats contain an empty lease and no prior read summary, but + // there will be an initial nontrivial lease requested with the first write + // below. This lease acquisition will, in turn, create a prior read summary. baseStats.Add(enginepb.MVCCStats{ - SysBytes: 28, + SysBytes: 64, + SysCount: 1, }) // Our clock might not be set to zero. diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index b64123177749..cdc161f3e5df 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tscache" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -24,19 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/uuid" ) -// setTimestampCacheLowWaterMark updates the low water mark of the timestamp -// cache to the provided timestamp for all key ranges owned by the provided -// Range descriptor. This ensures that no future writes in either the local or -// global keyspace are allowed at times equal to or earlier than this timestamp, -// which could invalidate prior reads. -func setTimestampCacheLowWaterMark( - tc tscache.Cache, desc *roachpb.RangeDescriptor, ts hlc.Timestamp, -) { - for _, keyRange := range rditer.MakeReplicatedKeyRangesExceptLockTable(desc) { - tc.SetLowWater(keyRange.Start.Key, keyRange.End.Key, ts) - } -} - // addToTSCacheChecked adds the specified timestamp to the timestamp cache // covering the range of keys from start to end. Before doing so, the function // performs a few assertions to check for proper use of the timestamp cache. @@ -551,3 +538,49 @@ func transactionTombstoneMarker(key roachpb.Key, txnID uuid.UUID) roachpb.Key { func transactionPushMarker(key roachpb.Key, txnID uuid.UUID) roachpb.Key { return append(keys.TransactionKey(key, txnID), []byte("-push")...) } + +// GetCurrentReadSummary returns a new ReadSummary reflecting all reads served +// by the range to this point. +func (r *Replica) GetCurrentReadSummary() rspb.ReadSummary { + sum := collectReadSummaryFromTimestampCache(r.store.tsCache, r.Desc()) + // Forward the read summary by the range's closed timestamp, because any + // replica could have served reads below this time. + sum.Merge(rspb.FromTimestamp(r.GetFrozenClosedTimestamp())) + return sum +} + +// collectReadSummaryFromTimestampCache constucts a read summary for the range +// with the specified descriptor using the timestamp cache. +func collectReadSummaryFromTimestampCache( + tc tscache.Cache, desc *roachpb.RangeDescriptor, +) rspb.ReadSummary { + var s rspb.ReadSummary + s.Local.LowWater, _ = tc.GetMax( + keys.MakeRangeKeyPrefix(desc.StartKey), + keys.MakeRangeKeyPrefix(desc.EndKey), + ) + s.Global.LowWater, _ = tc.GetMax( + desc.StartKey.AsRawKey(), + desc.EndKey.AsRawKey(), + ) + return s +} + +// applyReadSummaryToTimestampCache updates the timestamp cache to reflect the +// reads present in the provided read summary. This ensures that no future +// writes in either the local or global keyspace are allowed to invalidate +// ("write underneath") prior reads. +func applyReadSummaryToTimestampCache( + tc tscache.Cache, desc *roachpb.RangeDescriptor, s rspb.ReadSummary, +) { + tc.SetLowWater( + keys.MakeRangeKeyPrefix(desc.StartKey), + keys.MakeRangeKeyPrefix(desc.EndKey), + s.Local.LowWater, + ) + tc.SetLowWater( + desc.StartKey.AsRawKey(), + desc.EndKey.AsRawKey(), + s.Global.LowWater, + ) +} diff --git a/pkg/kv/kvserver/store_merge.go b/pkg/kv/kvserver/store_merge.go index a8d497b71e86..22ad0ae8cef1 100644 --- a/pkg/kv/kvserver/store_merge.go +++ b/pkg/kv/kvserver/store_merge.go @@ -13,6 +13,7 @@ package kvserver import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" @@ -30,6 +31,7 @@ func (s *Store) MergeRange( newLeftDesc, rightDesc roachpb.RangeDescriptor, freezeStart hlc.ClockTimestamp, rightClosedTS hlc.Timestamp, + rightReadSum *rspb.ReadSummary, ) error { if oldLeftDesc := leftRepl.Desc(); !oldLeftDesc.EndKey.Less(newLeftDesc.EndKey) { return errors.Errorf("the new end key is not greater than the current one: %+v <= %+v", @@ -79,17 +81,26 @@ func (s *Store) MergeRange( if !rightLease.OwnedBy(s.Ident.StoreID) { // We hold the lease for the LHS, but do not hold the lease for the RHS. // That means we don't have up-to-date timestamp cache entries for the - // keyspace previously owned by the RHS. Bump the low water mark for the RHS - // keyspace to freezeStart, the time at which the RHS promised to stop - // serving traffic, as freezeStart is guaranteed to be greater than any - // entry in the RHS's timestamp cache. + // keyspace previously owned by the RHS. Update the timestamp cache for + // the RHS keyspace. If the merge trigger included a prior read summary + // then we can use that directly to update the timestamp cache. + // Otherwise, we pessimistically assume that the right-hand side served + // reads all the way up to freezeStart, the time at which the RHS + // promised to stop serving traffic. // - // Note that we need to update our clock with freezeStart to preserve the - // invariant that our clock is always greater than or equal to any - // timestamps in the timestamp cache. For a full discussion, see the comment - // on TestStoreRangeMergeTimestampCacheCausality. + // Note that we need to update our clock with freezeStart to preserve + // the invariant that our clock is always greater than or equal to any + // timestamps in the timestamp cache. For a full discussion, see the + // comment on TestStoreRangeMergeTimestampCacheCausality. s.Clock().Update(freezeStart) - setTimestampCacheLowWaterMark(s.tsCache, &rightDesc, freezeStart.ToTimestamp()) + + var sum rspb.ReadSummary + if rightReadSum != nil { + sum = *rightReadSum + } else { + sum = rspb.FromTimestamp(freezeStart.ToTimestamp()) + } + applyReadSummaryToTimestampCache(s.tsCache, &rightDesc, sum) } // When merging ranges, the closed timestamp of the RHS can regress. It's // possible that, at subsumption time, the RHS had a high closed timestamp. @@ -101,10 +112,8 @@ func (s *Store) MergeRange( // In the case when the RHS lease was not collocated with the LHS, this bump // is frequently (but not necessarily) redundant with the bumping to the // freeze time done above. - if !rightClosedTS.Synthetic { - s.Clock().Update(rightClosedTS.UnsafeToClockTimestamp()) - } - setTimestampCacheLowWaterMark(s.tsCache, &rightDesc, rightClosedTS) + sum := rspb.FromTimestamp(rightClosedTS) + applyReadSummaryToTimestampCache(s.tsCache, &rightDesc, sum) } // Update the subsuming range's descriptor. diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go index 9ec16989e7f7..26e3a3492db6 100644 --- a/pkg/kv/kvserver/store_split.go +++ b/pkg/kv/kvserver/store_split.go @@ -272,6 +272,7 @@ func prepareRightReplicaForSplit( rightRepl.leasePostApplyLocked(ctx, rightRepl.mu.state.Lease, /* prevLease */ rightRepl.mu.state.Lease, /* newLease - same as prevLease */ + nil, /* priorReadSum */ assertNoLeaseJump) // We need to explicitly wake up the Raft group on the right-hand range or diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index 4c8bc0341a3f..2ffb4484c471 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -121,6 +121,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/concurrency/lock:lock_proto", + "//pkg/kv/kvserver/readsummary/rspb:rspb_proto", "//pkg/storage/enginepb:enginepb_proto", "//pkg/util:util_proto", "//pkg/util/hlc:hlc_proto", @@ -140,6 +141,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/concurrency/lock", + "//pkg/kv/kvserver/readsummary/rspb", "//pkg/storage/enginepb", "//pkg/util", "//pkg/util/hlc", diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index d619f1df3fcb..bab2cf077073 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -7,6 +7,7 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" import lock "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" +import rspb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" import enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import tracingpb "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" @@ -74,7 +75,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -102,7 +103,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{1} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{1} } type ChecksumMode int32 @@ -149,7 +150,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{2} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -180,7 +181,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{3} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{3} } type ExternalStorageProvider int32 @@ -221,7 +222,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{4} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{4} } type MVCCFilter int32 @@ -244,7 +245,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{5} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{5} } type ResponseHeader_ResumeReason int32 @@ -270,7 +271,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{1, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -312,7 +313,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{25, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -333,7 +334,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_7ab19f7b4624e210, []int{0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -399,7 +400,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{1} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -439,7 +440,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_7ab19f7b4624e210, []int{2} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -482,7 +483,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_7ab19f7b4624e210, []int{3} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -525,7 +526,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_7ab19f7b4624e210, []int{4} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -559,7 +560,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_7ab19f7b4624e210, []int{5} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -648,7 +649,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_7ab19f7b4624e210, []int{6} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -683,7 +684,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_7ab19f7b4624e210, []int{7} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -729,7 +730,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_7ab19f7b4624e210, []int{8} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -763,7 +764,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_7ab19f7b4624e210, []int{9} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -803,7 +804,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_7ab19f7b4624e210, []int{10} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -840,7 +841,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_7ab19f7b4624e210, []int{11} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -874,7 +875,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_7ab19f7b4624e210, []int{12} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -908,7 +909,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_7ab19f7b4624e210, []int{13} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -960,7 +961,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_7ab19f7b4624e210, []int{14} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -997,7 +998,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_7ab19f7b4624e210, []int{15} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1052,7 +1053,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_7ab19f7b4624e210, []int{16} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1086,7 +1087,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_7ab19f7b4624e210, []int{17} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1128,7 +1129,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_7ab19f7b4624e210, []int{18} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1162,7 +1163,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_7ab19f7b4624e210, []int{19} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1213,7 +1214,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_7ab19f7b4624e210, []int{20} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1265,7 +1266,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_7ab19f7b4624e210, []int{21} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1316,7 +1317,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_7ab19f7b4624e210, []int{22} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1368,7 +1369,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_7ab19f7b4624e210, []int{23} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1421,7 +1422,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_7ab19f7b4624e210, []int{24} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1459,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_7ab19f7b4624e210, []int{25} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1502,7 +1503,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_7ab19f7b4624e210, []int{25, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1550,7 +1551,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_7ab19f7b4624e210, []int{26} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1586,7 +1587,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_7ab19f7b4624e210, []int{27} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1697,7 +1698,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_7ab19f7b4624e210, []int{28} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1743,7 +1744,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_7ab19f7b4624e210, []int{29} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1804,7 +1805,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_7ab19f7b4624e210, []int{30} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1839,7 +1840,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_7ab19f7b4624e210, []int{31} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1878,7 +1879,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_7ab19f7b4624e210, []int{32} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1913,7 +1914,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_7ab19f7b4624e210, []int{33} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1956,7 +1957,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_7ab19f7b4624e210, []int{34} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1991,7 +1992,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_7ab19f7b4624e210, []int{35} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2029,7 +2030,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_7ab19f7b4624e210, []int{36} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2062,7 +2063,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_7ab19f7b4624e210, []int{37} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2097,7 +2098,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_7ab19f7b4624e210, []int{38} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2155,7 +2156,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_7ab19f7b4624e210, []int{39} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2190,7 +2191,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_7ab19f7b4624e210, []int{40} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2228,7 +2229,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_7ab19f7b4624e210, []int{41} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2261,7 +2262,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_7ab19f7b4624e210, []int{42} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2303,7 +2304,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_7ab19f7b4624e210, []int{43} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2340,7 +2341,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_7ab19f7b4624e210, []int{44} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2378,7 +2379,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_7ab19f7b4624e210, []int{45} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2412,7 +2413,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_7ab19f7b4624e210, []int{45, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2446,7 +2447,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_7ab19f7b4624e210, []int{46} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2515,7 +2516,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_7ab19f7b4624e210, []int{47} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2558,7 +2559,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_7ab19f7b4624e210, []int{48} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2605,7 +2606,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_7ab19f7b4624e210, []int{49} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2641,7 +2642,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_7ab19f7b4624e210, []int{50} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2685,7 +2686,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_7ab19f7b4624e210, []int{51} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2729,7 +2730,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_7ab19f7b4624e210, []int{52} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2797,7 +2798,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_7ab19f7b4624e210, []int{53} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2833,7 +2834,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_7ab19f7b4624e210, []int{54} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2880,7 +2881,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_7ab19f7b4624e210, []int{55} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2915,7 +2916,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_7ab19f7b4624e210, []int{56} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2966,7 +2967,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_7ab19f7b4624e210, []int{57} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3001,7 +3002,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_7ab19f7b4624e210, []int{58} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3038,7 +3039,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_7ab19f7b4624e210, []int{59} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3072,7 +3073,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_7ab19f7b4624e210, []int{60} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3117,7 +3118,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_7ab19f7b4624e210, []int{61} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3151,7 +3152,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_7ab19f7b4624e210, []int{62} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3195,7 +3196,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_7ab19f7b4624e210, []int{63} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3244,7 +3245,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_7ab19f7b4624e210, []int{64} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3281,7 +3282,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_7ab19f7b4624e210, []int{65} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3318,7 +3319,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_7ab19f7b4624e210, []int{66} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3353,7 +3354,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_7ab19f7b4624e210, []int{67} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3408,7 +3409,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_7ab19f7b4624e210, []int{68} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3445,7 +3446,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_7ab19f7b4624e210, []int{69} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3485,7 +3486,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_7ab19f7b4624e210, []int{70} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3519,7 +3520,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_7ab19f7b4624e210, []int{70, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3552,7 +3553,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_7ab19f7b4624e210, []int{70, 1} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3594,7 +3595,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_7ab19f7b4624e210, []int{70, 2} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3633,7 +3634,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_7ab19f7b4624e210, []int{70, 3} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3669,7 +3670,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_7ab19f7b4624e210, []int{70, 4} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3708,7 +3709,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_7ab19f7b4624e210, []int{70, 5} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3750,7 +3751,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_7ab19f7b4624e210, []int{70, 6} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3790,7 +3791,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_7ab19f7b4624e210, []int{71} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3824,7 +3825,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_7ab19f7b4624e210, []int{72} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3860,7 +3861,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_7ab19f7b4624e210, []int{73} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3930,7 +3931,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_7ab19f7b4624e210, []int{74} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3982,7 +3983,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_7ab19f7b4624e210, []int{75} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4018,7 +4019,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_7ab19f7b4624e210, []int{76} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4058,7 +4059,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_7ab19f7b4624e210, []int{76, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4109,7 +4110,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_7ab19f7b4624e210, []int{77} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4144,7 +4145,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_7ab19f7b4624e210, []int{77, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4180,7 +4181,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_7ab19f7b4624e210, []int{77, 1} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4215,7 +4216,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_7ab19f7b4624e210, []int{78} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4253,7 +4254,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_7ab19f7b4624e210, []int{79} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4290,7 +4291,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_7ab19f7b4624e210, []int{80} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4323,7 +4324,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_7ab19f7b4624e210, []int{80, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4368,7 +4369,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_7ab19f7b4624e210, []int{81} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4406,7 +4407,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_7ab19f7b4624e210, []int{82} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4459,7 +4460,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_7ab19f7b4624e210, []int{83} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4493,7 +4494,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_7ab19f7b4624e210, []int{84} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4537,7 +4538,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_7ab19f7b4624e210, []int{85} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4571,7 +4572,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_7ab19f7b4624e210, []int{86} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4610,7 +4611,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_7ab19f7b4624e210, []int{87} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4644,7 +4645,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_7ab19f7b4624e210, []int{88} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4693,7 +4694,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_7ab19f7b4624e210, []int{89} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4734,7 +4735,8 @@ type SubsumeResponse struct { // timestamps at which any requests were serviced by the responding replica // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water - // mark for the keys previously owned by the subsumed range. + // mark for the keys previously owned by the subsumed range though this role + // is largely being... subsumed by the RightReadSummary. FreezeStart github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"freeze_start"` // closed_timestamp is the range's closed timestamp at the moment of the // subsumption. Because the SubsumeRequest synchronizes with all other @@ -4750,13 +4752,27 @@ type SubsumeResponse struct { // the two sides are collocated at merge time, we don't need to use the // read_summary and simply use this field. ClosedTimestamp hlc.Timestamp `protobuf:"bytes,6,opt,name=closed_timestamp,json=closedTimestamp,proto3" json:"closed_timestamp"` + // ReadSummary is a summary of the reads that have been performed on the range + // up to the point of the Subsume request, which serializes with past reads + // and begins blocking future reads. It is suitable for use to update the + // timestamp cache for the keys previously owned by the subsumed range. + // + // ReadSummary can be used in place of FreezeStart, when available. It has two + // distinct advantages: + // 1. it can transfer a higher-resolution snapshot of the reads on the range + // through a range merge, to make the merge less disruptive to writes on + // the post-merge range because the timestamp cache won't be bumped as + // high. + // 2. it can transfer information about reads with synthetic timestamps, which + // are not otherwise captured by the FreezeStart clock timestamp. + ReadSummary *rspb.ReadSummary `protobuf:"bytes,7,opt,name=read_summary,json=readSummary,proto3" json:"read_summary,omitempty"` } 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_7ab19f7b4624e210, []int{90} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4791,7 +4807,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_7ab19f7b4624e210, []int{91} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4832,7 +4848,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{92} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4875,7 +4891,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_7ab19f7b4624e210, []int{93} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{93} } func (m *MigrateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4909,7 +4925,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_7ab19f7b4624e210, []int{94} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{94} } func (m *MigrateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4993,7 +5009,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_7ab19f7b4624e210, []int{95} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6474,7 +6490,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_7ab19f7b4624e210, []int{96} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8030,7 +8046,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_7ab19f7b4624e210, []int{97} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8066,7 +8082,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{98} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{98} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8103,7 +8119,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{99} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{99} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8179,7 +8195,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{99, 0} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{99, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8217,7 +8233,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{100} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{100} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8255,7 +8271,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{101} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{101} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8294,7 +8310,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{102} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{102} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8335,7 +8351,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{103} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{103} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8376,7 +8392,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{104} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{104} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8413,7 +8429,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{105} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{105} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8450,7 +8466,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{106} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{106} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8494,7 +8510,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{107} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{107} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8526,7 +8542,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{108} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{108} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8567,7 +8583,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{109} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{109} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8607,7 +8623,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{110} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{110} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8643,7 +8659,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{111} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{111} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8682,7 +8698,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{112} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{112} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8722,7 +8738,7 @@ type ContentionEvent struct { func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_7ab19f7b4624e210, []int{113} + return fileDescriptor_api_22c0b4cc9c2cd251, []int{113} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -13526,6 +13542,16 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n158 + if m.ReadSummary != nil { + dAtA[i] = 0x3a + i++ + i = encodeVarintApi(dAtA, i, uint64(m.ReadSummary.Size())) + n159, err := m.ReadSummary.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n159 + } return i, nil } @@ -13547,11 +13573,11 @@ func (m *RangeStatsRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n159, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n160, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n159 + i += n160 return i, nil } @@ -13573,19 +13599,19 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n160, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n161, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n160 + i += n161 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n161, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n162, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n161 + i += n162 if m.QueriesPerSecond != 0 { dAtA[i] = 0x19 i++ @@ -13595,11 +13621,11 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeInfo.Size())) - n162, err := m.RangeInfo.MarshalTo(dAtA[i:]) + n163, err := m.RangeInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n162 + i += n163 return i, nil } @@ -13621,19 +13647,19 @@ func (m *MigrateRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n163, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n164, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n163 + i += n164 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Version.Size())) - n164, err := m.Version.MarshalTo(dAtA[i:]) + n165, err := m.Version.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n164 + i += n165 return i, nil } @@ -13655,11 +13681,11 @@ func (m *MigrateResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n165, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n166, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n165 + i += n166 return i, nil } @@ -13679,11 +13705,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn166, err := m.Value.MarshalTo(dAtA[i:]) + nn167, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn166 + i += nn167 } return i, nil } @@ -13694,11 +13720,11 @@ func (m *RequestUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n167, err := m.Get.MarshalTo(dAtA[i:]) + n168, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n167 + i += n168 } return i, nil } @@ -13708,11 +13734,11 @@ func (m *RequestUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n168, err := m.Put.MarshalTo(dAtA[i:]) + n169, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n168 + i += n169 } return i, nil } @@ -13722,11 +13748,11 @@ func (m *RequestUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n169, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n170, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n169 + i += n170 } return i, nil } @@ -13736,11 +13762,11 @@ func (m *RequestUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n170, err := m.Increment.MarshalTo(dAtA[i:]) + n171, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n170 + i += n171 } return i, nil } @@ -13750,11 +13776,11 @@ func (m *RequestUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n171, err := m.Delete.MarshalTo(dAtA[i:]) + n172, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n171 + i += n172 } return i, nil } @@ -13764,11 +13790,11 @@ func (m *RequestUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n172, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n173, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n172 + i += n173 } return i, nil } @@ -13778,11 +13804,11 @@ func (m *RequestUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n173, err := m.Scan.MarshalTo(dAtA[i:]) + n174, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n173 + i += n174 } return i, nil } @@ -13792,11 +13818,11 @@ func (m *RequestUnion_EndTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTxn.Size())) - n174, err := m.EndTxn.MarshalTo(dAtA[i:]) + n175, err := m.EndTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n174 + i += n175 } return i, nil } @@ -13806,11 +13832,11 @@ func (m *RequestUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n175, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n176, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n175 + i += n176 } return i, nil } @@ -13820,11 +13846,11 @@ func (m *RequestUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n176, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n177, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n176 + i += n177 } return i, nil } @@ -13834,11 +13860,11 @@ func (m *RequestUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n177, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n178, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n177 + i += n178 } return i, nil } @@ -13848,11 +13874,11 @@ func (m *RequestUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n178, err := m.Gc.MarshalTo(dAtA[i:]) + n179, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n178 + i += n179 } return i, nil } @@ -13862,11 +13888,11 @@ func (m *RequestUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n179, err := m.PushTxn.MarshalTo(dAtA[i:]) + n180, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n179 + i += n180 } return i, nil } @@ -13878,11 +13904,11 @@ func (m *RequestUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n180, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n181, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n180 + i += n181 } return i, nil } @@ -13894,11 +13920,11 @@ func (m *RequestUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n181, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n182, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n181 + i += n182 } return i, nil } @@ -13910,11 +13936,11 @@ func (m *RequestUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n182, err := m.Merge.MarshalTo(dAtA[i:]) + n183, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n182 + i += n183 } return i, nil } @@ -13926,11 +13952,11 @@ func (m *RequestUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n183, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n184, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n183 + i += n184 } return i, nil } @@ -13942,11 +13968,11 @@ func (m *RequestUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n184, err := m.RequestLease.MarshalTo(dAtA[i:]) + n185, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n184 + i += n185 } return i, nil } @@ -13958,11 +13984,11 @@ func (m *RequestUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n185, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n186, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n185 + i += n186 } return i, nil } @@ -13974,11 +14000,11 @@ func (m *RequestUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n186, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n187, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n186 + i += n187 } return i, nil } @@ -13990,11 +14016,11 @@ func (m *RequestUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n187, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n188, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n187 + i += n188 } return i, nil } @@ -14006,11 +14032,11 @@ func (m *RequestUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n188, err := m.InitPut.MarshalTo(dAtA[i:]) + n189, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n188 + i += n189 } return i, nil } @@ -14022,11 +14048,11 @@ func (m *RequestUnion_TransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TransferLease.Size())) - n189, err := m.TransferLease.MarshalTo(dAtA[i:]) + n190, err := m.TransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n189 + i += n190 } return i, nil } @@ -14038,11 +14064,11 @@ func (m *RequestUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n190, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n191, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n190 + i += n191 } return i, nil } @@ -14054,11 +14080,11 @@ func (m *RequestUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n191, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n192, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n191 + i += n192 } return i, nil } @@ -14070,11 +14096,11 @@ func (m *RequestUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n192, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n193, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n192 + i += n193 } return i, nil } @@ -14086,11 +14112,11 @@ func (m *RequestUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n193, err := m.Export.MarshalTo(dAtA[i:]) + n194, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n193 + i += n194 } return i, nil } @@ -14102,11 +14128,11 @@ func (m *RequestUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n194, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n195, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n194 + i += n195 } return i, nil } @@ -14118,11 +14144,11 @@ func (m *RequestUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n195, err := m.Import.MarshalTo(dAtA[i:]) + n196, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n195 + i += n196 } return i, nil } @@ -14134,11 +14160,11 @@ func (m *RequestUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n196, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n197, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n196 + i += n197 } return i, nil } @@ -14150,11 +14176,11 @@ func (m *RequestUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n197, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n198, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n197 + i += n198 } return i, nil } @@ -14166,11 +14192,11 @@ func (m *RequestUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n198, err := m.AddSstable.MarshalTo(dAtA[i:]) + n199, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n198 + i += n199 } return i, nil } @@ -14182,11 +14208,11 @@ func (m *RequestUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n199, err := m.ClearRange.MarshalTo(dAtA[i:]) + n200, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n199 + i += n200 } return i, nil } @@ -14198,11 +14224,11 @@ func (m *RequestUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n200, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n201, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n200 + i += n201 } return i, nil } @@ -14214,11 +14240,11 @@ func (m *RequestUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n201, err := m.Refresh.MarshalTo(dAtA[i:]) + n202, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n201 + i += n202 } return i, nil } @@ -14230,11 +14256,11 @@ func (m *RequestUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n202, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n203, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n202 + i += n203 } return i, nil } @@ -14246,11 +14272,11 @@ func (m *RequestUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n203, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n204, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n203 + i += n204 } return i, nil } @@ -14262,11 +14288,11 @@ func (m *RequestUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n204, err := m.Subsume.MarshalTo(dAtA[i:]) + n205, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n204 + i += n205 } return i, nil } @@ -14278,11 +14304,11 @@ func (m *RequestUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n205, err := m.RangeStats.MarshalTo(dAtA[i:]) + n206, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n205 + i += n206 } return i, nil } @@ -14294,11 +14320,11 @@ func (m *RequestUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n206, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n207, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n206 + i += n207 } return i, nil } @@ -14310,11 +14336,11 @@ func (m *RequestUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n207, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n208, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n207 + i += n208 } return i, nil } @@ -14326,11 +14352,11 @@ func (m *RequestUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size())) - n208, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) + n209, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n208 + i += n209 } return i, nil } @@ -14342,11 +14368,11 @@ func (m *RequestUnion_RevertRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.RevertRange.Size())) - n209, err := m.RevertRange.MarshalTo(dAtA[i:]) + n210, err := m.RevertRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n209 + i += n210 } return i, nil } @@ -14358,11 +14384,11 @@ func (m *RequestUnion_AdminVerifyProtectedTimestamp) MarshalTo(dAtA []byte) (int dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminVerifyProtectedTimestamp.Size())) - n210, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) + n211, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n210 + i += n211 } return i, nil } @@ -14374,11 +14400,11 @@ func (m *RequestUnion_Migrate) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.Migrate.Size())) - n211, err := m.Migrate.MarshalTo(dAtA[i:]) + n212, err := m.Migrate.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n211 + i += n212 } return i, nil } @@ -14398,11 +14424,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn212, err := m.Value.MarshalTo(dAtA[i:]) + nn213, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn212 + i += nn213 } return i, nil } @@ -14413,11 +14439,11 @@ func (m *ResponseUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n213, err := m.Get.MarshalTo(dAtA[i:]) + n214, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n213 + i += n214 } return i, nil } @@ -14427,11 +14453,11 @@ func (m *ResponseUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n214, err := m.Put.MarshalTo(dAtA[i:]) + n215, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n214 + i += n215 } return i, nil } @@ -14441,11 +14467,11 @@ func (m *ResponseUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n215, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n216, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n215 + i += n216 } return i, nil } @@ -14455,11 +14481,11 @@ func (m *ResponseUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n216, err := m.Increment.MarshalTo(dAtA[i:]) + n217, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n216 + i += n217 } return i, nil } @@ -14469,11 +14495,11 @@ func (m *ResponseUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n217, err := m.Delete.MarshalTo(dAtA[i:]) + n218, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n217 + i += n218 } return i, nil } @@ -14483,11 +14509,11 @@ func (m *ResponseUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n218, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n219, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n218 + i += n219 } return i, nil } @@ -14497,11 +14523,11 @@ func (m *ResponseUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n219, err := m.Scan.MarshalTo(dAtA[i:]) + n220, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n219 + i += n220 } return i, nil } @@ -14511,11 +14537,11 @@ func (m *ResponseUnion_EndTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTxn.Size())) - n220, err := m.EndTxn.MarshalTo(dAtA[i:]) + n221, err := m.EndTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n220 + i += n221 } return i, nil } @@ -14525,11 +14551,11 @@ func (m *ResponseUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n221, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n222, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n221 + i += n222 } return i, nil } @@ -14539,11 +14565,11 @@ func (m *ResponseUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n222, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n223, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n222 + i += n223 } return i, nil } @@ -14553,11 +14579,11 @@ func (m *ResponseUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n223, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n224, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n223 + i += n224 } return i, nil } @@ -14567,11 +14593,11 @@ func (m *ResponseUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n224, err := m.Gc.MarshalTo(dAtA[i:]) + n225, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n224 + i += n225 } return i, nil } @@ -14581,11 +14607,11 @@ func (m *ResponseUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n225, err := m.PushTxn.MarshalTo(dAtA[i:]) + n226, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n225 + i += n226 } return i, nil } @@ -14597,11 +14623,11 @@ func (m *ResponseUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n226, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n227, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n226 + i += n227 } return i, nil } @@ -14613,11 +14639,11 @@ func (m *ResponseUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n227, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n228, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n227 + i += n228 } return i, nil } @@ -14629,11 +14655,11 @@ func (m *ResponseUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n228, err := m.Merge.MarshalTo(dAtA[i:]) + n229, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n228 + i += n229 } return i, nil } @@ -14645,11 +14671,11 @@ func (m *ResponseUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n229, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n230, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n229 + i += n230 } return i, nil } @@ -14661,11 +14687,11 @@ func (m *ResponseUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n230, err := m.RequestLease.MarshalTo(dAtA[i:]) + n231, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n230 + i += n231 } return i, nil } @@ -14677,11 +14703,11 @@ func (m *ResponseUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n231, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n232, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n231 + i += n232 } return i, nil } @@ -14693,11 +14719,11 @@ func (m *ResponseUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n232, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n233, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n232 + i += n233 } return i, nil } @@ -14709,11 +14735,11 @@ func (m *ResponseUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n233, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n234, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n233 + i += n234 } return i, nil } @@ -14725,11 +14751,11 @@ func (m *ResponseUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n234, err := m.InitPut.MarshalTo(dAtA[i:]) + n235, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n234 + i += n235 } return i, nil } @@ -14741,11 +14767,11 @@ func (m *ResponseUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n235, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n236, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n235 + i += n236 } return i, nil } @@ -14757,11 +14783,11 @@ func (m *ResponseUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n236, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n237, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n236 + i += n237 } return i, nil } @@ -14773,11 +14799,11 @@ func (m *ResponseUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n237, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n238, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n237 + i += n238 } return i, nil } @@ -14789,11 +14815,11 @@ func (m *ResponseUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n238, err := m.Export.MarshalTo(dAtA[i:]) + n239, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n238 + i += n239 } return i, nil } @@ -14805,11 +14831,11 @@ func (m *ResponseUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n239, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n240, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n239 + i += n240 } return i, nil } @@ -14821,11 +14847,11 @@ func (m *ResponseUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n240, err := m.Import.MarshalTo(dAtA[i:]) + n241, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n240 + i += n241 } return i, nil } @@ -14837,11 +14863,11 @@ func (m *ResponseUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n241, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n242, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n241 + i += n242 } return i, nil } @@ -14853,11 +14879,11 @@ func (m *ResponseUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n242, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n243, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n242 + i += n243 } return i, nil } @@ -14869,11 +14895,11 @@ func (m *ResponseUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n243, err := m.AddSstable.MarshalTo(dAtA[i:]) + n244, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n243 + i += n244 } return i, nil } @@ -14885,11 +14911,11 @@ func (m *ResponseUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n244, err := m.ClearRange.MarshalTo(dAtA[i:]) + n245, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n244 + i += n245 } return i, nil } @@ -14901,11 +14927,11 @@ func (m *ResponseUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n245, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n246, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n245 + i += n246 } return i, nil } @@ -14917,11 +14943,11 @@ func (m *ResponseUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n246, err := m.Refresh.MarshalTo(dAtA[i:]) + n247, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n246 + i += n247 } return i, nil } @@ -14933,11 +14959,11 @@ func (m *ResponseUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n247, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n248, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n247 + i += n248 } return i, nil } @@ -14949,11 +14975,11 @@ func (m *ResponseUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n248, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n249, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n248 + i += n249 } return i, nil } @@ -14965,11 +14991,11 @@ func (m *ResponseUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n249, err := m.Subsume.MarshalTo(dAtA[i:]) + n250, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n249 + i += n250 } return i, nil } @@ -14981,11 +15007,11 @@ func (m *ResponseUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n250, err := m.RangeStats.MarshalTo(dAtA[i:]) + n251, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n250 + i += n251 } return i, nil } @@ -14997,11 +15023,11 @@ func (m *ResponseUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n251, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n252, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n251 + i += n252 } return i, nil } @@ -15013,11 +15039,11 @@ func (m *ResponseUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n252, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n253, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n252 + i += n253 } return i, nil } @@ -15029,11 +15055,11 @@ func (m *ResponseUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size())) - n253, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) + n254, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n253 + i += n254 } return i, nil } @@ -15045,11 +15071,11 @@ func (m *ResponseUnion_RevertRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.RevertRange.Size())) - n254, err := m.RevertRange.MarshalTo(dAtA[i:]) + n255, err := m.RevertRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n254 + i += n255 } return i, nil } @@ -15061,11 +15087,11 @@ func (m *ResponseUnion_AdminVerifyProtectedTimestamp) MarshalTo(dAtA []byte) (in dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminVerifyProtectedTimestamp.Size())) - n255, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) + n256, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n255 + i += n256 } return i, nil } @@ -15077,11 +15103,11 @@ func (m *ResponseUnion_Migrate) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.Migrate.Size())) - n256, err := m.Migrate.MarshalTo(dAtA[i:]) + n257, err := m.Migrate.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n256 + i += n257 } return i, nil } @@ -15103,19 +15129,19 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n257, err := m.Timestamp.MarshalTo(dAtA[i:]) + n258, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n257 + i += n258 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Replica.Size())) - n258, err := m.Replica.MarshalTo(dAtA[i:]) + n259, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n258 + i += n259 if m.RangeID != 0 { dAtA[i] = 0x18 i++ @@ -15131,11 +15157,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n259, err := m.Txn.MarshalTo(dAtA[i:]) + n260, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n259 + i += n260 } if m.ReadConsistency != 0 { dAtA[i] = 0x30 @@ -15194,11 +15220,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClientRangeInfo.Size())) - n260, err := m.ClientRangeInfo.MarshalTo(dAtA[i:]) + n261, err := m.ClientRangeInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n260 + i += n261 if m.WaitPolicy != 0 { dAtA[i] = 0x90 i++ @@ -15227,11 +15253,11 @@ func (m *BatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n261, err := m.Header.MarshalTo(dAtA[i:]) + n262, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n261 + i += n262 if len(m.Requests) > 0 { for _, msg := range m.Requests { dAtA[i] = 0x12 @@ -15265,11 +15291,11 @@ func (m *BatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BatchResponse_Header.Size())) - n262, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) + n263, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n262 + i += n263 if len(m.Responses) > 0 { for _, msg := range m.Responses { dAtA[i] = 0x12 @@ -15304,38 +15330,38 @@ func (m *BatchResponse_Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n263, err := m.Error.MarshalTo(dAtA[i:]) + n264, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n263 + i += n264 } dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n264, err := m.Timestamp.MarshalTo(dAtA[i:]) + n265, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n264 + i += n265 if m.Txn != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n265, err := m.Txn.MarshalTo(dAtA[i:]) + n266, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n265 + i += n266 } dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n266, err := m.Now.MarshalTo(dAtA[i:]) + n267, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n266 + i += n267 if len(m.CollectedSpans) > 0 { for _, msg := range m.CollectedSpans { dAtA[i] = 0x32 @@ -15450,11 +15476,11 @@ func (m *RangeLookupResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n267, err := m.Error.MarshalTo(dAtA[i:]) + n268, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n267 + i += n268 } return i, nil } @@ -15477,19 +15503,19 @@ func (m *RangeFeedRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n268, err := m.Header.MarshalTo(dAtA[i:]) + n269, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n268 + i += n269 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n269, err := m.Span.MarshalTo(dAtA[i:]) + n270, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n269 + i += n270 if m.WithDiff { dAtA[i] = 0x18 i++ @@ -15527,19 +15553,19 @@ func (m *RangeFeedValue) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n270, err := m.Value.MarshalTo(dAtA[i:]) + n271, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n270 + i += n271 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevValue.Size())) - n271, err := m.PrevValue.MarshalTo(dAtA[i:]) + n272, err := m.PrevValue.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n271 + i += n272 return i, nil } @@ -15561,19 +15587,19 @@ func (m *RangeFeedCheckpoint) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n272, err := m.Span.MarshalTo(dAtA[i:]) + n273, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n272 + i += n273 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolvedTS.Size())) - n273, err := m.ResolvedTS.MarshalTo(dAtA[i:]) + n274, err := m.ResolvedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n273 + i += n274 return i, nil } @@ -15595,11 +15621,11 @@ func (m *RangeFeedError) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n274, err := m.Error.MarshalTo(dAtA[i:]) + n275, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n274 + i += n275 return i, nil } @@ -15622,31 +15648,31 @@ func (m *RangeFeedEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Val.Size())) - n275, err := m.Val.MarshalTo(dAtA[i:]) + n276, err := m.Val.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n275 + i += n276 } if m.Checkpoint != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Checkpoint.Size())) - n276, err := m.Checkpoint.MarshalTo(dAtA[i:]) + n277, err := m.Checkpoint.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n276 + i += n277 } if m.Error != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n277, err := m.Error.MarshalTo(dAtA[i:]) + n278, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n277 + i += n278 } return i, nil } @@ -15749,11 +15775,11 @@ func (m *GossipSubscriptionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Content.Size())) - n278, err := m.Content.MarshalTo(dAtA[i:]) + n279, err := m.Content.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n278 + i += n279 if len(m.PatternMatched) > 0 { dAtA[i] = 0x1a i++ @@ -15764,11 +15790,11 @@ func (m *GossipSubscriptionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n279, err := m.Error.MarshalTo(dAtA[i:]) + n280, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n279 + i += n280 } return i, nil } @@ -15792,11 +15818,11 @@ func (m *JoinNodeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BinaryVersion.Size())) - n280, err := m.BinaryVersion.MarshalTo(dAtA[i:]) + n281, err := m.BinaryVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n280 + i += n281 } return i, nil } @@ -15836,11 +15862,11 @@ func (m *JoinNodeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.ActiveVersion.Size())) - n281, err := m.ActiveVersion.MarshalTo(dAtA[i:]) + n282, err := m.ActiveVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n281 + i += n282 } return i, nil } @@ -15869,19 +15895,19 @@ func (m *ContentionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.TxnMeta.Size())) - n282, err := m.TxnMeta.MarshalTo(dAtA[i:]) + n283, err := m.TxnMeta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n282 + i += n283 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration))) - n283, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:]) + n284, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:]) if err != nil { return 0, err } - i += n283 + i += n284 return i, nil } @@ -17753,6 +17779,10 @@ func (m *SubsumeResponse) Size() (n int) { n += 1 + l + sovApi(uint64(l)) l = m.ClosedTimestamp.Size() n += 1 + l + sovApi(uint64(l)) + if m.ReadSummary != nil { + l = m.ReadSummary.Size() + n += 1 + l + sovApi(uint64(l)) + } return n } @@ -33024,6 +33054,39 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReadSummary", 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 + } + if m.ReadSummary == nil { + m.ReadSummary = &rspb.ReadSummary{} + } + if err := m.ReadSummary.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -38976,521 +39039,524 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_7ab19f7b4624e210) } - -var fileDescriptor_api_7ab19f7b4624e210 = []byte{ - // 8196 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x59, - 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0x22, 0x4b, 0x57, 0xfd, 0xa3, 0xe6, 0xcc, 0xb4, 0xba, - 0xab, 0xff, 0xdb, 0x33, 0xd2, 0x74, 0xf7, 0x4e, 0x66, 0x3c, 0x3d, 0x9e, 0xb5, 0x44, 0xb1, 0x9b, - 0x94, 0x5a, 0x6a, 0x75, 0x91, 0xea, 0xc6, 0x8c, 0xd7, 0xa9, 0x2d, 0x55, 0x5d, 0x51, 0xb5, 0x22, - 0xab, 0xd8, 0x55, 0x45, 0xfd, 0x0c, 0x10, 0x20, 0x8e, 0x0d, 0x67, 0x9f, 0x8c, 0x7d, 0x30, 0x90, - 0x35, 0x1c, 0xc4, 0xeb, 0x6c, 0x10, 0x3f, 0x04, 0x48, 0x02, 0x24, 0xc8, 0x1f, 0x92, 0xf5, 0x8b, - 0x81, 0x2c, 0x02, 0x27, 0x5e, 0x3f, 0xc5, 0x08, 0x10, 0xc5, 0xd6, 0xe6, 0x25, 0x70, 0x10, 0x04, - 0x41, 0x00, 0x03, 0xf3, 0x10, 0x04, 0xf7, 0xa7, 0xfe, 0xc8, 0xe2, 0x8f, 0x7a, 0x6a, 0x93, 0x01, - 0xfc, 0x22, 0xb1, 0xce, 0xbd, 0xe7, 0xd4, 0xbd, 0xe7, 0xde, 0x7b, 0xee, 0xf9, 0x6e, 0x9d, 0x7b, - 0x2f, 0xcc, 0xd9, 0x96, 0xaa, 0xed, 0x77, 0x76, 0x97, 0xd5, 0x8e, 0xb1, 0xd4, 0xb1, 0x2d, 0xd7, - 0x42, 0x73, 0x9a, 0xa5, 0x1d, 0x50, 0xf2, 0x12, 0x4f, 0x2c, 0xdd, 0x3f, 0x38, 0x5c, 0x3e, 0x38, - 0x74, 0xb0, 0x7d, 0x88, 0xed, 0x65, 0xcd, 0x32, 0xb5, 0xae, 0x6d, 0x63, 0x53, 0x3b, 0x59, 0x6e, - 0x59, 0xda, 0x01, 0xfd, 0x63, 0x98, 0x4d, 0xc6, 0x5e, 0x42, 0x9e, 0x44, 0x5d, 0x75, 0x55, 0x4e, - 0xbb, 0xe0, 0xd1, 0xb0, 0x6d, 0x5b, 0xb6, 0xc3, 0xa9, 0x97, 0x3c, 0x6a, 0x1b, 0xbb, 0x6a, 0x28, - 0xf7, 0x5b, 0x8e, 0x6b, 0xd9, 0x6a, 0x13, 0x2f, 0x63, 0xb3, 0x69, 0x98, 0x98, 0x64, 0x38, 0xd4, - 0x34, 0x9e, 0xf8, 0x76, 0x6c, 0xe2, 0x23, 0x9e, 0xba, 0xd0, 0x75, 0x8d, 0xd6, 0xf2, 0x7e, 0x4b, - 0x5b, 0x76, 0x8d, 0x36, 0x76, 0x5c, 0xb5, 0xdd, 0xe1, 0x29, 0xf7, 0x69, 0x8a, 0x6b, 0xab, 0x9a, - 0x61, 0x36, 0xbd, 0xff, 0x9d, 0xdd, 0x65, 0x1b, 0x6b, 0x96, 0xad, 0x63, 0x5d, 0x71, 0x3a, 0xaa, - 0xe9, 0x15, 0xb7, 0x69, 0x35, 0x2d, 0xfa, 0x73, 0x99, 0xfc, 0xe2, 0xd4, 0xab, 0x4d, 0xcb, 0x6a, - 0xb6, 0xf0, 0x32, 0x7d, 0xda, 0xed, 0xee, 0x2d, 0xeb, 0x5d, 0x5b, 0x75, 0x0d, 0x8b, 0x73, 0x49, - 0xff, 0x4c, 0x80, 0x59, 0x19, 0xbf, 0xee, 0x62, 0xc7, 0xad, 0x62, 0x55, 0xc7, 0x36, 0xba, 0x02, - 0xe9, 0x03, 0x7c, 0xb2, 0x90, 0xbe, 0x26, 0xdc, 0x9d, 0x59, 0x9d, 0xfe, 0xf2, 0x74, 0x31, 0xbd, - 0x81, 0x4f, 0x64, 0x42, 0x43, 0xd7, 0x60, 0x1a, 0x9b, 0xba, 0x42, 0x92, 0x33, 0xd1, 0xe4, 0x29, - 0x6c, 0xea, 0x1b, 0xf8, 0x04, 0x7d, 0x0b, 0xb2, 0x0e, 0x91, 0x66, 0x6a, 0x78, 0x61, 0xf2, 0x9a, - 0x70, 0x77, 0x72, 0xf5, 0x17, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0x69, 0xb8, 0xfb, 0xdd, 0xdd, 0x25, - 0xcd, 0x6a, 0x2f, 0xfb, 0xed, 0xa4, 0xef, 0x06, 0xbf, 0x97, 0x3b, 0x07, 0xcd, 0xe5, 0x5e, 0x1d, - 0x2d, 0x35, 0x8e, 0xcd, 0x3a, 0x7e, 0x2d, 0xfb, 0x12, 0xd7, 0x33, 0x59, 0x41, 0x4c, 0xad, 0x67, - 0xb2, 0x29, 0x31, 0x2d, 0xfd, 0x51, 0x0a, 0x0a, 0x32, 0x76, 0x3a, 0x96, 0xe9, 0x60, 0x5e, 0xf2, - 0xf7, 0x21, 0xed, 0x1e, 0x9b, 0xb4, 0xe4, 0xf9, 0x87, 0x57, 0x97, 0xfa, 0x7a, 0xc4, 0x52, 0xc3, - 0x56, 0x4d, 0x47, 0xd5, 0x48, 0xf5, 0x65, 0x92, 0x15, 0x7d, 0x04, 0x79, 0x1b, 0x3b, 0xdd, 0x36, - 0xa6, 0x8a, 0xa4, 0x95, 0xca, 0x3f, 0xbc, 0x1c, 0xc3, 0x59, 0xef, 0xa8, 0xa6, 0x0c, 0x2c, 0x2f, - 0xf9, 0x8d, 0xae, 0x40, 0xd6, 0xec, 0xb6, 0x89, 0x2a, 0x1c, 0x5a, 0xd1, 0xb4, 0x3c, 0x6d, 0x76, - 0xdb, 0x1b, 0xf8, 0xc4, 0x41, 0x75, 0x98, 0xe5, 0x42, 0x6d, 0xac, 0x3a, 0x96, 0xb9, 0x30, 0x7d, - 0x4d, 0xb8, 0x5b, 0x78, 0xb8, 0x14, 0x23, 0x36, 0x5a, 0x01, 0xf2, 0xd8, 0x6d, 0x63, 0x99, 0x72, - 0xc9, 0x33, 0x76, 0xe8, 0x09, 0xbd, 0x05, 0x39, 0xf2, 0xbe, 0xdd, 0x13, 0x17, 0x3b, 0x0b, 0x59, - 0xfa, 0x42, 0x52, 0x80, 0x55, 0xf2, 0x2c, 0x7d, 0x0a, 0x33, 0x61, 0x56, 0x84, 0xa0, 0x20, 0x57, - 0xea, 0x3b, 0x9b, 0x15, 0x65, 0x67, 0x6b, 0x63, 0xeb, 0xf9, 0xab, 0x2d, 0x71, 0x02, 0x5d, 0x00, - 0x91, 0xd3, 0x36, 0x2a, 0x9f, 0x29, 0xcf, 0x6a, 0x9b, 0xb5, 0x86, 0x28, 0x94, 0x32, 0xdf, 0xfd, - 0xe1, 0xd5, 0x89, 0xf5, 0x4c, 0x76, 0x4a, 0x9c, 0x96, 0x7e, 0x28, 0x00, 0x3c, 0xc5, 0x2e, 0xef, - 0x0d, 0x68, 0x15, 0xa6, 0xf6, 0x69, 0xb1, 0x16, 0x04, 0xaa, 0x96, 0x6b, 0xb1, 0xe5, 0x0f, 0xf5, - 0x9c, 0xd5, 0xec, 0x8f, 0x4f, 0x17, 0x27, 0x7e, 0x72, 0xba, 0x28, 0xc8, 0x9c, 0x13, 0xbd, 0x80, - 0xfc, 0x01, 0x3e, 0x51, 0xf8, 0x58, 0x5b, 0x48, 0x51, 0x45, 0xbc, 0x1f, 0x12, 0x74, 0x70, 0xb8, - 0xe4, 0x0d, 0xd1, 0xa5, 0xd0, 0x10, 0x5d, 0x22, 0x1c, 0x4b, 0x75, 0xd7, 0xc6, 0x66, 0xd3, 0xdd, - 0x97, 0xe1, 0x00, 0x9f, 0x3c, 0x63, 0x32, 0xa4, 0xdf, 0x17, 0x20, 0x4f, 0x4b, 0xc9, 0x34, 0x87, - 0xca, 0x3d, 0xc5, 0xbc, 0x3e, 0x52, 0xcd, 0x31, 0xe5, 0x5c, 0x82, 0xc9, 0x43, 0xb5, 0xd5, 0xc5, - 0xb4, 0x84, 0xf9, 0x87, 0x0b, 0x31, 0x32, 0x5e, 0x92, 0x74, 0x99, 0x65, 0x43, 0x8f, 0x61, 0xc6, - 0x30, 0x5d, 0x6c, 0xba, 0x0a, 0x63, 0x4b, 0x8f, 0x60, 0xcb, 0xb3, 0xdc, 0xf4, 0x41, 0xfa, 0xa7, - 0x02, 0xc0, 0x76, 0x37, 0x51, 0x3d, 0x7f, 0x63, 0xcc, 0xf2, 0xaf, 0x66, 0x08, 0xab, 0x57, 0x8b, - 0x4b, 0x30, 0x65, 0x98, 0x2d, 0xc3, 0x64, 0xe5, 0xcf, 0xca, 0xfc, 0x09, 0x5d, 0x80, 0xc9, 0xdd, - 0x96, 0x61, 0xea, 0x74, 0x3c, 0x64, 0x65, 0xf6, 0x20, 0xc9, 0x90, 0xa7, 0xa5, 0x4e, 0x50, 0xef, - 0xd2, 0x69, 0x0a, 0x2e, 0x96, 0x2d, 0x53, 0x37, 0xc8, 0x90, 0x54, 0x5b, 0x5f, 0x0b, 0xad, 0xac, - 0xc3, 0x05, 0x1d, 0x77, 0x6c, 0xac, 0xa9, 0x2e, 0xd6, 0x15, 0x7c, 0xdc, 0x19, 0xb3, 0x8d, 0x51, - 0xc0, 0x55, 0x39, 0xee, 0x50, 0x5a, 0xbc, 0x26, 0xd1, 0x37, 0xe0, 0xb2, 0xda, 0x6a, 0x59, 0x47, - 0x8a, 0xb1, 0xa7, 0xe8, 0x16, 0x76, 0x14, 0xd3, 0x72, 0x15, 0x7c, 0x6c, 0x38, 0x2e, 0x35, 0x25, - 0x59, 0x79, 0x9e, 0x26, 0xd7, 0xf6, 0xd6, 0x2c, 0xec, 0x6c, 0x59, 0x6e, 0x85, 0x24, 0x11, 0x0b, - 0x40, 0x0a, 0xc3, 0x2c, 0xc0, 0x14, 0x31, 0xbf, 0x72, 0x16, 0x1f, 0x77, 0xa8, 0x05, 0x08, 0x35, - 0xe5, 0x74, 0xb8, 0x29, 0xa5, 0x5f, 0x86, 0x4b, 0xbd, 0xfa, 0x4d, 0xb2, 0xfd, 0xfe, 0x50, 0x80, - 0x42, 0xcd, 0x34, 0xdc, 0xaf, 0x45, 0xc3, 0xf9, 0xca, 0x4e, 0x87, 0x95, 0x7d, 0x1f, 0xc4, 0x3d, - 0xd5, 0x68, 0x3d, 0x37, 0x1b, 0x56, 0x7b, 0xd7, 0x71, 0x2d, 0x13, 0x3b, 0xbc, 0x35, 0xfa, 0xe8, - 0xd2, 0x4b, 0x28, 0xfa, 0xb5, 0x49, 0x52, 0x4d, 0x2e, 0x88, 0x35, 0x53, 0xb3, 0x71, 0x1b, 0x9b, - 0x89, 0xea, 0xe9, 0x6d, 0xc8, 0x19, 0x9e, 0x5c, 0xaa, 0xab, 0xb4, 0x1c, 0x10, 0xa4, 0x2e, 0xcc, - 0x85, 0xde, 0x9a, 0xa4, 0xb9, 0x24, 0x93, 0x11, 0x3e, 0x52, 0x82, 0x36, 0x22, 0x93, 0x11, 0x3e, - 0x62, 0xe6, 0xad, 0x0e, 0xb3, 0x6b, 0xb8, 0x85, 0x5d, 0x9c, 0x60, 0x4d, 0xa5, 0x1d, 0x28, 0x78, - 0x42, 0x93, 0x6c, 0x98, 0xbf, 0x25, 0x00, 0xe2, 0x72, 0x55, 0xb3, 0x99, 0x64, 0x89, 0xd1, 0x22, - 0x71, 0x2d, 0xdc, 0xae, 0x6d, 0x32, 0x1f, 0x81, 0xf5, 0x49, 0x60, 0x24, 0xea, 0x26, 0x04, 0x43, - 0x36, 0x13, 0x1e, 0xb2, 0xdc, 0xbd, 0x39, 0x82, 0xf9, 0x48, 0xc1, 0x92, 0x6d, 0xbe, 0x0c, 0x2d, - 0x53, 0xea, 0x5a, 0x3a, 0xec, 0xc3, 0x51, 0xa2, 0xf4, 0x7d, 0x01, 0xe6, 0xca, 0x2d, 0xac, 0xda, - 0x89, 0x6b, 0xe4, 0x9b, 0x90, 0xd5, 0xb1, 0xaa, 0xd3, 0x2a, 0xb3, 0x81, 0xfd, 0x4e, 0x48, 0x0a, - 0xf1, 0x74, 0x97, 0xf6, 0x5b, 0xda, 0x52, 0xc3, 0xf3, 0x81, 0xf9, 0xe8, 0xf6, 0x99, 0xa4, 0xcf, - 0x00, 0x85, 0x4b, 0x96, 0x64, 0x47, 0xf8, 0xdf, 0x02, 0x20, 0x19, 0x1f, 0x62, 0xdb, 0x4d, 0xbc, - 0xda, 0x6b, 0x90, 0x77, 0x55, 0xbb, 0x89, 0x5d, 0x85, 0x78, 0xf7, 0xe7, 0xa9, 0x39, 0x30, 0x3e, - 0x42, 0x46, 0x0d, 0xb8, 0x83, 0x4d, 0x75, 0xb7, 0x85, 0xa9, 0x14, 0x65, 0xd7, 0xea, 0x9a, 0xba, - 0x62, 0xb8, 0xd8, 0x56, 0x5d, 0xcb, 0x56, 0xac, 0x8e, 0x6b, 0xb4, 0x8d, 0x2f, 0xa8, 0x63, 0xcf, - 0xbb, 0xda, 0x0d, 0x96, 0x9d, 0x30, 0xaf, 0x92, 0xcc, 0x35, 0x9e, 0xf7, 0x79, 0x28, 0xab, 0xf4, - 0x39, 0xcc, 0x47, 0x6a, 0x9d, 0xa4, 0x4a, 0xff, 0xa7, 0x00, 0xf9, 0xba, 0xa6, 0x9a, 0x49, 0xea, - 0xf2, 0x53, 0xc8, 0x3b, 0x9a, 0x6a, 0x2a, 0x7b, 0x96, 0xdd, 0x56, 0x5d, 0x3a, 0x70, 0x0a, 0x11, - 0x5d, 0xfa, 0xfe, 0xba, 0xa6, 0x9a, 0x4f, 0x68, 0x26, 0x19, 0x1c, 0xff, 0x77, 0xaf, 0x3f, 0x3a, - 0xf9, 0xd5, 0xfd, 0x51, 0x36, 0x5c, 0xd7, 0x33, 0xd9, 0xb4, 0x98, 0x91, 0xfe, 0x42, 0x80, 0x19, - 0x56, 0xe5, 0x24, 0x87, 0xeb, 0x07, 0x90, 0xb1, 0xad, 0x23, 0x36, 0x5c, 0xf3, 0x0f, 0xdf, 0x8a, - 0x11, 0xb1, 0x81, 0x4f, 0xc2, 0xf3, 0x21, 0xcd, 0x8e, 0x56, 0x81, 0x7b, 0x9d, 0x0a, 0xe5, 0x4e, - 0x8f, 0xcb, 0x0d, 0x8c, 0x4b, 0x26, 0x32, 0xee, 0x40, 0x71, 0x57, 0x75, 0xb5, 0x7d, 0xc5, 0xe6, - 0x85, 0x24, 0x73, 0x67, 0xfa, 0xee, 0x8c, 0x5c, 0xa0, 0x64, 0xaf, 0xe8, 0x0e, 0xa9, 0x39, 0x1b, - 0x3f, 0x0e, 0xfe, 0x4b, 0xd6, 0xe6, 0xff, 0x47, 0xe0, 0x63, 0xc8, 0xab, 0xf9, 0x5f, 0xb6, 0xa6, - 0xff, 0xad, 0x14, 0x5c, 0x2e, 0xef, 0x63, 0xed, 0xa0, 0x6c, 0x99, 0x8e, 0xe1, 0xb8, 0x44, 0x77, - 0x49, 0xb6, 0xff, 0x5b, 0x90, 0x3b, 0x32, 0xdc, 0x7d, 0x45, 0x37, 0xf6, 0xf6, 0xa8, 0xf5, 0xcc, - 0xca, 0x59, 0x42, 0x58, 0x33, 0xf6, 0xf6, 0xd0, 0x23, 0xc8, 0xb4, 0x2d, 0x9d, 0x39, 0xe7, 0x85, - 0x87, 0x8b, 0x31, 0xe2, 0x69, 0xd1, 0x9c, 0x6e, 0x7b, 0xd3, 0xd2, 0xb1, 0x4c, 0x33, 0xa3, 0xab, - 0x00, 0x1a, 0xa1, 0x76, 0x2c, 0xc3, 0x74, 0xf9, 0xec, 0x1b, 0xa2, 0xa0, 0x2a, 0xe4, 0x5c, 0x6c, - 0xb7, 0x0d, 0x53, 0x75, 0xf1, 0xc2, 0x24, 0x55, 0xde, 0xcd, 0xd8, 0x82, 0x77, 0x5a, 0x86, 0xa6, - 0xae, 0x61, 0x47, 0xb3, 0x8d, 0x8e, 0x6b, 0xd9, 0x5c, 0x8b, 0x01, 0xb3, 0xf4, 0x1b, 0x19, 0x58, - 0xe8, 0xd7, 0x4d, 0x92, 0x3d, 0x64, 0x1b, 0xa6, 0x6c, 0xec, 0x74, 0x5b, 0x2e, 0xef, 0x23, 0x0f, - 0x07, 0xa9, 0x20, 0xa6, 0x04, 0x74, 0xbd, 0xa1, 0xe5, 0xf2, 0x62, 0x73, 0x39, 0xa5, 0x7f, 0x2d, - 0xc0, 0x14, 0x4b, 0x40, 0x0f, 0x20, 0x6b, 0x93, 0x89, 0x41, 0x31, 0x74, 0x5a, 0xc6, 0xf4, 0xea, - 0xa5, 0xb3, 0xd3, 0xc5, 0x69, 0x3a, 0x59, 0xd4, 0xd6, 0xbe, 0x0c, 0x7e, 0xca, 0xd3, 0x34, 0x5f, - 0x4d, 0x27, 0xad, 0xe5, 0xb8, 0xaa, 0xed, 0xd2, 0x45, 0xa2, 0x14, 0x43, 0x29, 0x94, 0xb0, 0x81, - 0x4f, 0xd0, 0x3a, 0x4c, 0x39, 0xae, 0xea, 0x76, 0x1d, 0xde, 0x5e, 0xe7, 0x2a, 0x6c, 0x9d, 0x72, - 0xca, 0x5c, 0x02, 0x71, 0x9f, 0x74, 0xec, 0xaa, 0x46, 0x8b, 0x36, 0x60, 0x4e, 0xe6, 0x4f, 0xd2, - 0x6f, 0x0b, 0x30, 0xc5, 0xb2, 0xa2, 0xcb, 0x30, 0x2f, 0xaf, 0x6c, 0x3d, 0xad, 0x28, 0xb5, 0xad, - 0xb5, 0x4a, 0xa3, 0x22, 0x6f, 0xd6, 0xb6, 0x56, 0x1a, 0x15, 0x71, 0x02, 0x5d, 0x02, 0xe4, 0x25, - 0x94, 0x9f, 0x6f, 0xd5, 0x6b, 0xf5, 0x46, 0x65, 0xab, 0x21, 0x0a, 0x74, 0x8d, 0x84, 0xd2, 0x43, - 0xd4, 0x14, 0xba, 0x09, 0xd7, 0x7a, 0xa9, 0x4a, 0xbd, 0xb1, 0xd2, 0xa8, 0x2b, 0x95, 0x7a, 0xa3, - 0xb6, 0xb9, 0xd2, 0xa8, 0xac, 0x89, 0xe9, 0x21, 0xb9, 0xc8, 0x4b, 0x64, 0xb9, 0x52, 0x6e, 0x88, - 0x19, 0xc9, 0x85, 0x8b, 0x32, 0xd6, 0xac, 0x76, 0xa7, 0xeb, 0x62, 0x52, 0x4a, 0x27, 0xc9, 0x91, - 0x72, 0x19, 0xa6, 0x75, 0xfb, 0x44, 0xb1, 0xbb, 0x26, 0x1f, 0x27, 0x53, 0xba, 0x7d, 0x22, 0x77, - 0x4d, 0xe9, 0x1f, 0x0b, 0x70, 0xa9, 0xf7, 0xb5, 0x49, 0x76, 0xc2, 0x17, 0x90, 0x57, 0x75, 0x1d, - 0xeb, 0x8a, 0x8e, 0x5b, 0xae, 0xca, 0x5d, 0x9c, 0xfb, 0x21, 0x49, 0x7c, 0x69, 0x6f, 0xc9, 0x5f, - 0xda, 0xdb, 0x7c, 0x59, 0x2e, 0xd3, 0x82, 0xac, 0x11, 0x0e, 0xcf, 0xfc, 0x50, 0x21, 0x94, 0x22, - 0xfd, 0xf7, 0x0c, 0xcc, 0x56, 0x4c, 0xbd, 0x71, 0x9c, 0xe8, 0x5c, 0x72, 0x09, 0xa6, 0x34, 0xab, - 0xdd, 0x36, 0x5c, 0x4f, 0x41, 0xec, 0x09, 0xfd, 0x7c, 0xc8, 0x35, 0x4d, 0x8f, 0xe1, 0xa0, 0x05, - 0x4e, 0x29, 0xfa, 0x36, 0x5c, 0x26, 0x56, 0xd3, 0x36, 0xd5, 0x96, 0xc2, 0xa4, 0x29, 0xae, 0x6d, - 0x34, 0x9b, 0xd8, 0xe6, 0xcb, 0x89, 0x77, 0x63, 0xca, 0x59, 0xe3, 0x1c, 0x65, 0xca, 0xd0, 0x60, - 0xf9, 0xe5, 0x8b, 0x46, 0x1c, 0x19, 0x7d, 0x02, 0x40, 0xa6, 0x22, 0xba, 0x44, 0xe9, 0x70, 0x7b, - 0x34, 0x68, 0x8d, 0xd2, 0x33, 0x41, 0x84, 0x81, 0x3c, 0x3b, 0x68, 0x99, 0xe0, 0x90, 0xd7, 0x5d, - 0xc3, 0xc6, 0xca, 0x83, 0x8e, 0x46, 0x17, 0x0e, 0xb2, 0xab, 0x85, 0xb3, 0xd3, 0x45, 0x90, 0x19, - 0xf9, 0xc1, 0x76, 0x99, 0xe0, 0x12, 0xf6, 0xbb, 0xa3, 0xa1, 0x57, 0x70, 0x2f, 0xb4, 0xfe, 0x41, - 0x66, 0x5e, 0x5e, 0x2d, 0xd5, 0x55, 0xf6, 0x8d, 0xe6, 0x3e, 0xb6, 0x15, 0x7f, 0x99, 0x9a, 0xae, - 0x44, 0x66, 0xe5, 0x9b, 0x01, 0x43, 0x59, 0x35, 0x59, 0xe9, 0x57, 0xdc, 0x2a, 0xcd, 0xec, 0xeb, - 0x8c, 0x28, 0xbf, 0x63, 0x19, 0x8e, 0x65, 0x2e, 0xe4, 0x98, 0xf2, 0xd9, 0x13, 0xba, 0x07, 0xa2, - 0x7b, 0x6c, 0x2a, 0xfb, 0x58, 0xb5, 0xdd, 0x5d, 0xac, 0xba, 0x64, 0x96, 0x06, 0x9a, 0xa3, 0xe8, - 0x1e, 0x9b, 0xd5, 0x10, 0x19, 0xbd, 0x00, 0xd1, 0x30, 0x95, 0xbd, 0x96, 0xd1, 0xdc, 0x77, 0x95, - 0x23, 0xdb, 0x70, 0xb1, 0xb3, 0x30, 0x47, 0x15, 0x12, 0xd7, 0x6f, 0xeb, 0x7c, 0xdd, 0x58, 0x7f, - 0x45, 0x72, 0x72, 0xd5, 0x14, 0x0c, 0xf3, 0x09, 0xe5, 0xa7, 0x44, 0x67, 0x3d, 0x93, 0x9d, 0x16, - 0xb3, 0xd2, 0x7f, 0x16, 0xa0, 0xe0, 0x75, 0xb7, 0x24, 0x47, 0xc6, 0x5d, 0x10, 0x2d, 0x13, 0x2b, - 0x9d, 0x7d, 0xd5, 0xc1, 0x5c, 0x8f, 0x7c, 0xc2, 0x29, 0x58, 0x26, 0xde, 0x26, 0x64, 0xa6, 0x2e, - 0xb4, 0x0d, 0x73, 0x8e, 0xab, 0x36, 0x0d, 0xb3, 0x19, 0x52, 0xef, 0xe4, 0xf8, 0x60, 0x41, 0xe4, - 0xdc, 0x3e, 0x3d, 0xe2, 0xa5, 0xfc, 0xb1, 0x00, 0x73, 0x2b, 0x7a, 0xdb, 0x30, 0xeb, 0x9d, 0x96, - 0x91, 0xe8, 0x1a, 0xc4, 0x4d, 0xc8, 0x39, 0x44, 0x66, 0x60, 0xf0, 0x03, 0x44, 0x99, 0xa5, 0x29, - 0xc4, 0xf2, 0x3f, 0x83, 0x22, 0x3e, 0xee, 0x18, 0xec, 0xd3, 0x03, 0x03, 0x42, 0x99, 0xf1, 0xeb, - 0x56, 0x08, 0x78, 0x49, 0x12, 0xaf, 0xd3, 0x67, 0x80, 0xc2, 0x55, 0x4a, 0x12, 0xbb, 0x7c, 0x06, - 0xf3, 0x54, 0xf4, 0x8e, 0xe9, 0x24, 0xac, 0x2f, 0xe9, 0x97, 0xe0, 0x42, 0x54, 0x74, 0x92, 0xe5, - 0x7e, 0xc5, 0x5b, 0x79, 0x13, 0xdb, 0x89, 0x82, 0x58, 0x5f, 0xd7, 0x5c, 0x70, 0x92, 0x65, 0xfe, - 0x35, 0x01, 0xae, 0x50, 0xd9, 0xf4, 0xeb, 0xcc, 0x1e, 0xb6, 0x9f, 0x61, 0xd5, 0x49, 0x14, 0x81, - 0xdf, 0x80, 0x29, 0x86, 0xa4, 0x69, 0xff, 0x9c, 0x5c, 0xcd, 0x13, 0xcf, 0xa5, 0xee, 0x5a, 0x36, - 0xf1, 0x5c, 0x78, 0x92, 0xa4, 0x42, 0x29, 0xae, 0x14, 0x49, 0xd6, 0xf4, 0xef, 0x08, 0x30, 0xc7, - 0x9d, 0x46, 0xd2, 0x95, 0xcb, 0xfb, 0xc4, 0x67, 0x42, 0x15, 0xc8, 0x6b, 0xf4, 0x97, 0xe2, 0x9e, - 0x74, 0x30, 0x95, 0x5f, 0x18, 0xe6, 0x6f, 0x32, 0xb6, 0xc6, 0x49, 0x07, 0x13, 0xa7, 0xd5, 0xfb, - 0x4d, 0x14, 0x15, 0xaa, 0xe4, 0x50, 0x8f, 0x95, 0x8e, 0x23, 0x9a, 0xd7, 0x73, 0xfd, 0xb8, 0x0e, - 0xfe, 0x49, 0x9a, 0x2b, 0x81, 0xbd, 0x83, 0x67, 0x4f, 0xd4, 0x47, 0xf9, 0x1c, 0x2e, 0x85, 0x67, - 0x97, 0x50, 0xc5, 0x53, 0xe7, 0xa8, 0x78, 0x68, 0x85, 0x3e, 0xa0, 0xa2, 0xcf, 0x20, 0xb4, 0x06, - 0xaf, 0xb0, 0x3a, 0x79, 0xe8, 0xe7, 0x3c, 0xea, 0x98, 0x0b, 0xa4, 0x30, 0xba, 0x83, 0xca, 0x90, - 0xc5, 0xc7, 0x1d, 0x45, 0xc7, 0x8e, 0xc6, 0x0d, 0x97, 0x14, 0x27, 0x90, 0x14, 0xa5, 0x0f, 0x0f, - 0x4c, 0xe3, 0xe3, 0x0e, 0x21, 0xa2, 0x1d, 0x32, 0x7b, 0x79, 0xae, 0x02, 0x2d, 0xb6, 0x33, 0x1a, - 0x5e, 0x04, 0x3d, 0x85, 0x8b, 0x2b, 0xfa, 0x5e, 0x02, 0x13, 0x21, 0xfd, 0x40, 0x80, 0xb7, 0x62, - 0x5b, 0x2d, 0xc9, 0x89, 0xec, 0x13, 0xc8, 0xd0, 0xca, 0xa7, 0xce, 0x59, 0x79, 0xca, 0x25, 0x7d, - 0x37, 0xc5, 0xc7, 0xb8, 0x8c, 0x5b, 0x16, 0x51, 0x6c, 0xe2, 0xab, 0x6c, 0xcf, 0x61, 0xf6, 0xd0, - 0x72, 0x89, 0x6f, 0xc2, 0x9b, 0x3d, 0x75, 0xee, 0x66, 0x9f, 0xa1, 0x02, 0xbc, 0x16, 0x7f, 0x09, - 0x73, 0xa6, 0x65, 0x2a, 0x51, 0xa1, 0xe7, 0xef, 0x4b, 0x45, 0xd3, 0x32, 0x5f, 0x86, 0xe4, 0xfa, - 0x76, 0xa6, 0x47, 0x13, 0x49, 0xda, 0x99, 0xef, 0x09, 0x30, 0xef, 0xbb, 0x4d, 0x09, 0x7b, 0xd0, - 0x1f, 0x40, 0xda, 0xb4, 0x8e, 0xce, 0xb3, 0x8a, 0x49, 0xf2, 0x93, 0x59, 0x2f, 0x5a, 0xa2, 0x24, - 0xeb, 0xfb, 0x6f, 0x52, 0x90, 0x7b, 0x5a, 0x4e, 0xb2, 0x96, 0x9f, 0xf0, 0x15, 0x72, 0xd6, 0xde, - 0x71, 0xbd, 0xdd, 0x7f, 0xdf, 0xd2, 0xd3, 0xf2, 0x06, 0x3e, 0xf1, 0x7a, 0x3b, 0xe1, 0x42, 0x2b, - 0x90, 0x73, 0xf7, 0x6d, 0xec, 0xec, 0x5b, 0x2d, 0xfd, 0x3c, 0x6e, 0x4e, 0xc0, 0x55, 0xc2, 0x30, - 0x49, 0xe5, 0x7a, 0xd1, 0x18, 0x42, 0x4c, 0x34, 0x06, 0x79, 0x8d, 0xef, 0x29, 0xa6, 0xce, 0xf3, - 0x9a, 0x90, 0x8b, 0x38, 0x29, 0x4e, 0x49, 0x2f, 0x00, 0x48, 0x75, 0x92, 0x6c, 0x92, 0x5f, 0x4f, - 0x43, 0x61, 0xbb, 0xeb, 0xec, 0x27, 0xdc, 0xfb, 0xca, 0x00, 0x9d, 0xae, 0x43, 0x21, 0xc8, 0xb1, - 0xc9, 0xeb, 0x3c, 0x22, 0xd0, 0xc3, 0xab, 0x34, 0xe3, 0x6b, 0x1c, 0x9b, 0xa8, 0xca, 0x85, 0x60, - 0x25, 0x88, 0x16, 0xb9, 0x31, 0x0c, 0xac, 0x36, 0x8e, 0xcd, 0x4d, 0xec, 0xa3, 0x54, 0x26, 0x09, - 0x13, 0x49, 0x9f, 0xc0, 0x34, 0x79, 0x50, 0x5c, 0xeb, 0x3c, 0xcd, 0x3c, 0x45, 0x78, 0x1a, 0x16, - 0x7a, 0x0c, 0x39, 0xc6, 0x4d, 0x66, 0xbf, 0x29, 0x3a, 0xfb, 0xc5, 0xd5, 0x85, 0xab, 0x91, 0xce, - 0x7b, 0x59, 0xca, 0x4a, 0xe6, 0xba, 0x0b, 0x30, 0xb9, 0x67, 0xd9, 0x9a, 0xf7, 0xbd, 0x97, 0x3d, - 0xb0, 0xf6, 0x5c, 0xcf, 0x64, 0xb3, 0x62, 0x6e, 0x3d, 0x93, 0xcd, 0x89, 0x20, 0xfd, 0xb6, 0x00, - 0x45, 0xbf, 0x21, 0x92, 0x9c, 0x10, 0xca, 0x11, 0x2d, 0x9e, 0xbf, 0x29, 0x88, 0x02, 0xa5, 0x7f, - 0x4b, 0x3d, 0x22, 0xcd, 0x3a, 0xa4, 0x2d, 0x93, 0x64, 0x4f, 0x79, 0xcc, 0x62, 0x81, 0x52, 0xe7, - 0x6d, 0x5d, 0x1a, 0x16, 0xf4, 0x00, 0x2e, 0x18, 0x6d, 0x62, 0xcf, 0x0d, 0xb7, 0x75, 0xc2, 0x61, - 0x9b, 0x8b, 0xbd, 0x0f, 0xcb, 0xf3, 0x41, 0x5a, 0xd9, 0x4b, 0x92, 0xfe, 0x1e, 0x5d, 0x00, 0x0f, - 0x6a, 0x92, 0xa4, 0xaa, 0x6b, 0x30, 0x6b, 0x33, 0xd1, 0xc4, 0xad, 0x39, 0xa7, 0xb6, 0x67, 0x7c, - 0x56, 0xa2, 0xf0, 0xdf, 0x4d, 0x41, 0xf1, 0x45, 0x17, 0xdb, 0x27, 0x5f, 0x27, 0x75, 0xdf, 0x86, - 0xe2, 0x91, 0x6a, 0xb8, 0xca, 0x9e, 0x65, 0x2b, 0xdd, 0x8e, 0xae, 0xba, 0x5e, 0x40, 0xca, 0x2c, - 0x21, 0x3f, 0xb1, 0xec, 0x1d, 0x4a, 0x44, 0x18, 0xd0, 0x81, 0x69, 0x1d, 0x99, 0x0a, 0x21, 0x53, - 0xa0, 0x7c, 0x6c, 0xf2, 0x55, 0xe9, 0xd5, 0x0f, 0xff, 0xd3, 0xe9, 0xe2, 0xa3, 0xb1, 0xc2, 0xcc, - 0x68, 0x48, 0x5d, 0xb7, 0x6b, 0xe8, 0x4b, 0x3b, 0x3b, 0xb5, 0x35, 0x59, 0xa4, 0x22, 0x5f, 0x31, - 0x89, 0x8d, 0x63, 0xd3, 0x91, 0xfe, 0x7e, 0x0a, 0xc4, 0x40, 0x47, 0x49, 0x36, 0x64, 0x05, 0xf2, - 0xaf, 0xbb, 0xd8, 0x36, 0xde, 0xa0, 0x19, 0x81, 0x33, 0x12, 0xb3, 0xf3, 0x39, 0xcc, 0x44, 0x34, - 0x90, 0xfe, 0x6a, 0x1a, 0xc8, 0x1f, 0x05, 0x95, 0x47, 0xf7, 0x61, 0xce, 0x3d, 0x36, 0x15, 0x16, - 0x60, 0xc8, 0x82, 0x52, 0xbc, 0x78, 0x89, 0xa2, 0x4b, 0xf4, 0x41, 0xe8, 0x34, 0x20, 0xc5, 0x91, - 0x7e, 0x5f, 0x00, 0x44, 0x15, 0x55, 0x63, 0x9f, 0x0d, 0xbe, 0x2e, 0xfd, 0xe9, 0x2e, 0x88, 0x34, - 0x64, 0x53, 0x31, 0xf6, 0x94, 0xb6, 0xe1, 0x38, 0x86, 0xd9, 0xe4, 0x1d, 0xaa, 0x40, 0xe9, 0xb5, - 0xbd, 0x4d, 0x46, 0x95, 0xfe, 0x1a, 0xcc, 0x47, 0x2a, 0x90, 0x64, 0x63, 0x5f, 0x87, 0x99, 0x3d, - 0xf6, 0x95, 0x96, 0x0a, 0xe7, 0x2b, 0x8e, 0x79, 0x4a, 0x63, 0xef, 0x93, 0xfe, 0x3c, 0x05, 0x17, - 0x64, 0xec, 0x58, 0xad, 0x43, 0x9c, 0xbc, 0x0a, 0xab, 0xc0, 0x3f, 0xe7, 0x28, 0x6f, 0xa4, 0xc9, - 0x1c, 0x63, 0x66, 0xd3, 0x5c, 0x74, 0xd9, 0xfe, 0xe6, 0xf0, 0x1e, 0xdb, 0xbf, 0x50, 0xcf, 0x97, - 0xfd, 0x32, 0x91, 0x65, 0x3f, 0x0b, 0x8a, 0x46, 0xd3, 0xb4, 0x88, 0x4d, 0x73, 0xf0, 0x6b, 0xb3, - 0xdb, 0xf6, 0xc0, 0xd0, 0xd2, 0xb0, 0x42, 0xd6, 0x18, 0x4b, 0x1d, 0xbf, 0xde, 0xea, 0xb6, 0xa9, - 0xef, 0xbc, 0x7a, 0x89, 0x94, 0xf7, 0xec, 0x74, 0xb1, 0x10, 0x49, 0x73, 0xe4, 0x82, 0xe1, 0x3f, - 0x13, 0xe9, 0xd2, 0xb7, 0xe0, 0x62, 0x8f, 0xb2, 0x93, 0xf4, 0x78, 0xfe, 0x55, 0x1a, 0xae, 0x44, - 0xc5, 0x27, 0x0d, 0x71, 0xbe, 0xee, 0x0d, 0x5a, 0x85, 0xd9, 0xb6, 0x61, 0xbe, 0xd9, 0xea, 0xe5, - 0x4c, 0xdb, 0x30, 0x83, 0x95, 0xe2, 0x98, 0xae, 0x31, 0xf5, 0x33, 0xed, 0x1a, 0x2a, 0x94, 0xe2, - 0xda, 0x2e, 0xc9, 0xfe, 0xf1, 0x5d, 0x01, 0x66, 0x92, 0x5e, 0x96, 0x7b, 0xb3, 0x40, 0x39, 0xa9, - 0x01, 0xb3, 0x3f, 0x83, 0x75, 0xbc, 0xdf, 0x15, 0x00, 0x35, 0xec, 0xae, 0x49, 0x40, 0xed, 0x33, - 0xab, 0x99, 0x64, 0x35, 0x2f, 0xc0, 0xa4, 0x61, 0xea, 0xf8, 0x98, 0x56, 0x33, 0x23, 0xb3, 0x87, - 0xc8, 0xd7, 0xc9, 0xf4, 0x58, 0x5f, 0x27, 0xa5, 0xcf, 0x61, 0x3e, 0x52, 0xc4, 0x24, 0xeb, 0xff, - 0xdf, 0x52, 0x30, 0xcf, 0x2b, 0x92, 0xf8, 0x0a, 0xe6, 0x37, 0x60, 0xb2, 0x45, 0x64, 0x0e, 0x69, - 0x67, 0xfa, 0x4e, 0xaf, 0x9d, 0x69, 0x66, 0xf4, 0x0b, 0x00, 0x1d, 0x1b, 0x1f, 0x2a, 0x8c, 0x35, - 0x3d, 0x16, 0x6b, 0x8e, 0x70, 0x50, 0x02, 0xfa, 0xbe, 0x00, 0x45, 0x32, 0xa0, 0x3b, 0xb6, 0xd5, - 0xb1, 0x1c, 0xe2, 0xb3, 0x38, 0xe3, 0xc1, 0x9c, 0x17, 0x67, 0xa7, 0x8b, 0xb3, 0x9b, 0x86, 0xb9, - 0xcd, 0x19, 0x1b, 0xf5, 0xb1, 0xf7, 0x00, 0x78, 0x3b, 0x21, 0x96, 0xca, 0x2d, 0x4b, 0x3b, 0x08, - 0xbe, 0xb7, 0x11, 0xcb, 0xe2, 0x8b, 0x73, 0xa4, 0x3f, 0x12, 0xe0, 0xc2, 0xcf, 0x6c, 0xb9, 0xf8, - 0xff, 0x87, 0xb2, 0xa5, 0x97, 0x20, 0xd2, 0x1f, 0x35, 0x73, 0xcf, 0x4a, 0x72, 0xe1, 0xfe, 0x37, - 0x04, 0x98, 0x0b, 0x09, 0x4e, 0xd2, 0xc1, 0x79, 0x23, 0x3d, 0x49, 0xbf, 0x44, 0x5c, 0x9e, 0xf0, - 0x28, 0x49, 0x72, 0x0c, 0xfe, 0xf3, 0x14, 0x5c, 0x2a, 0xb3, 0xcf, 0xdc, 0x5e, 0xdc, 0x47, 0x92, - 0x3d, 0x63, 0x01, 0xa6, 0x0f, 0xb1, 0xed, 0x18, 0x16, 0x9b, 0x7e, 0x67, 0x65, 0xef, 0x11, 0x95, - 0x20, 0xeb, 0x98, 0x6a, 0xc7, 0xd9, 0xb7, 0xbc, 0xef, 0x7b, 0xfe, 0xb3, 0x1f, 0xa3, 0x32, 0xf9, - 0xe6, 0x31, 0x2a, 0x53, 0xc3, 0x63, 0x54, 0xa6, 0xbf, 0x42, 0x8c, 0x0a, 0xff, 0x98, 0xf6, 0xef, - 0x04, 0xb8, 0xdc, 0xa7, 0xb9, 0x24, 0x7b, 0xcb, 0x77, 0x20, 0xaf, 0x71, 0xc1, 0xc4, 0x60, 0xb3, - 0x2f, 0x85, 0x35, 0x92, 0xed, 0x0d, 0x71, 0xcb, 0xd9, 0xe9, 0x22, 0x78, 0x45, 0xad, 0xad, 0x71, - 0xe5, 0x90, 0xdf, 0xba, 0xf4, 0xab, 0xb3, 0x50, 0xac, 0x1c, 0xb3, 0x55, 0xf2, 0x3a, 0x73, 0x13, - 0xd0, 0x13, 0xc8, 0x76, 0x6c, 0xeb, 0xd0, 0xf0, 0xaa, 0x51, 0x88, 0x04, 0x28, 0x78, 0xd5, 0xe8, - 0xe1, 0xda, 0xe6, 0x1c, 0xb2, 0xcf, 0x8b, 0x1a, 0x90, 0x7b, 0x66, 0x69, 0x6a, 0xeb, 0x89, 0xd1, - 0xf2, 0x7a, 0xfe, 0xfb, 0xa3, 0x05, 0x2d, 0xf9, 0x3c, 0xdb, 0xaa, 0xbb, 0xef, 0x35, 0x82, 0x4f, - 0x44, 0x35, 0xc8, 0x56, 0x5d, 0xb7, 0x43, 0x12, 0xb9, 0xed, 0xb8, 0x33, 0x86, 0x50, 0xc2, 0xe2, - 0x45, 0xc9, 0x7a, 0xec, 0xa8, 0x01, 0x73, 0x4f, 0xe9, 0x9e, 0xaf, 0x72, 0xcb, 0xea, 0xea, 0x65, - 0xcb, 0xdc, 0x33, 0x9a, 0xdc, 0x6e, 0xdf, 0x1e, 0x43, 0xe6, 0xd3, 0x72, 0x5d, 0xee, 0x17, 0x80, - 0x56, 0x20, 0x5b, 0x7f, 0xc4, 0x85, 0x31, 0xbf, 0xee, 0xd6, 0x18, 0xc2, 0xea, 0x8f, 0x64, 0x9f, - 0x0d, 0xad, 0x43, 0x7e, 0xe5, 0x8b, 0xae, 0x8d, 0xb9, 0x94, 0xa9, 0x81, 0xd1, 0x11, 0xbd, 0x52, - 0x28, 0x97, 0x1c, 0x66, 0x46, 0x75, 0x28, 0xbc, 0xb2, 0xec, 0x83, 0x96, 0xa5, 0x7a, 0x35, 0x9c, - 0xa6, 0xe2, 0x7e, 0x6e, 0x0c, 0x71, 0x1e, 0xa3, 0xdc, 0x23, 0x02, 0x7d, 0x0b, 0x8a, 0xa4, 0x31, - 0x1a, 0xea, 0x6e, 0xcb, 0x2b, 0x64, 0x96, 0x4a, 0x7d, 0x77, 0x0c, 0xa9, 0x3e, 0xa7, 0xb7, 0xf0, - 0xdf, 0x23, 0xaa, 0x24, 0xc3, 0x6c, 0xa4, 0x13, 0x20, 0x04, 0x99, 0x0e, 0x69, 0x6f, 0x81, 0xc6, - 0x2f, 0xd1, 0xdf, 0xe8, 0x3d, 0x98, 0x36, 0x2d, 0x1d, 0x7b, 0x23, 0x64, 0x76, 0xf5, 0xc2, 0xd9, - 0xe9, 0xe2, 0xd4, 0x96, 0xa5, 0x33, 0x8f, 0x86, 0xff, 0x92, 0xa7, 0x48, 0xa6, 0x9a, 0x5e, 0xba, - 0x06, 0x19, 0xd2, 0xee, 0xc4, 0x30, 0xed, 0xaa, 0x0e, 0xde, 0xb1, 0x0d, 0x2e, 0xcd, 0x7b, 0x2c, - 0xfd, 0xa3, 0x14, 0xa4, 0xea, 0x8f, 0x88, 0xcf, 0xbe, 0xdb, 0xd5, 0x0e, 0xb0, 0xcb, 0xd3, 0xf9, - 0x13, 0xf5, 0xe5, 0x6d, 0xbc, 0x67, 0x30, 0xd7, 0x2a, 0x27, 0xf3, 0x27, 0xf4, 0x0e, 0x80, 0xaa, - 0x69, 0xd8, 0x71, 0x14, 0x6f, 0x2f, 0x60, 0x4e, 0xce, 0x31, 0xca, 0x06, 0x3e, 0x21, 0x6c, 0x0e, - 0xd6, 0x6c, 0xec, 0x7a, 0xc1, 0x57, 0xec, 0x89, 0xb0, 0xb9, 0xb8, 0xdd, 0x51, 0x5c, 0xeb, 0x00, - 0x9b, 0xb4, 0x9f, 0xe4, 0x88, 0xa9, 0x69, 0x77, 0x1a, 0x84, 0x40, 0xac, 0x24, 0x36, 0xf5, 0xc0, - 0xa4, 0xe5, 0x64, 0xff, 0x99, 0x88, 0xb4, 0x71, 0xd3, 0xe0, 0xdb, 0xe5, 0x72, 0x32, 0x7f, 0x22, - 0x5a, 0x52, 0xbb, 0xee, 0x3e, 0x6d, 0x89, 0x9c, 0x4c, 0x7f, 0xa3, 0xdb, 0x50, 0x64, 0xf1, 0x9a, - 0x0a, 0x36, 0x35, 0x85, 0x1a, 0xd7, 0x1c, 0x4d, 0x9e, 0x65, 0xe4, 0x8a, 0xa9, 0x11, 0x53, 0x8a, - 0x1e, 0x01, 0x27, 0x28, 0x07, 0x6d, 0x87, 0xe8, 0x14, 0x48, 0xae, 0xd5, 0xe2, 0xd9, 0xe9, 0x62, - 0xbe, 0x4e, 0x13, 0x36, 0x36, 0xeb, 0xb5, 0x35, 0x39, 0xcf, 0x72, 0x6d, 0xb4, 0x9d, 0x9a, 0x5e, - 0xfa, 0x4d, 0x01, 0xd2, 0x4f, 0xcb, 0xf5, 0x73, 0xab, 0xcc, 0x2b, 0x68, 0x3a, 0x54, 0xd0, 0x3b, - 0x50, 0xdc, 0x35, 0x5a, 0x2d, 0xc3, 0x6c, 0x12, 0x2f, 0xea, 0x3b, 0x58, 0xf3, 0x14, 0x56, 0xe0, - 0xe4, 0x6d, 0x46, 0x45, 0xd7, 0x20, 0xaf, 0xd9, 0x58, 0xc7, 0xa6, 0x6b, 0xa8, 0x2d, 0x87, 0x6b, - 0x2e, 0x4c, 0x2a, 0xfd, 0x8a, 0x00, 0x93, 0x74, 0x04, 0xa0, 0xb7, 0x21, 0xa7, 0x59, 0xa6, 0xab, - 0x1a, 0x26, 0x37, 0x65, 0x39, 0x39, 0x20, 0x0c, 0x2c, 0xde, 0x75, 0x98, 0x51, 0x35, 0xcd, 0xea, - 0x9a, 0xae, 0x62, 0xaa, 0x6d, 0xcc, 0x8b, 0x99, 0xe7, 0xb4, 0x2d, 0xb5, 0x8d, 0xd1, 0x22, 0x78, - 0x8f, 0xfe, 0x16, 0xcf, 0x9c, 0x0c, 0x9c, 0xb4, 0x81, 0x4f, 0x4a, 0x7f, 0x20, 0x40, 0xd6, 0x1b, - 0x33, 0xa4, 0x18, 0x4d, 0x6c, 0xb2, 0xa0, 0x72, 0xaf, 0x18, 0x3e, 0xa1, 0x77, 0xaa, 0xcc, 0x05, - 0x53, 0xe5, 0x05, 0x98, 0x74, 0xc9, 0xb0, 0xe0, 0x25, 0x60, 0x0f, 0x74, 0x3d, 0xbb, 0xa5, 0x36, - 0xd9, 0x72, 0x5e, 0x4e, 0x66, 0x0f, 0xa4, 0x32, 0x3c, 0xec, 0x97, 0x69, 0x84, 0x3f, 0x91, 0x92, - 0xb2, 0xe0, 0xd4, 0x5d, 0xdc, 0x34, 0x4c, 0xda, 0x97, 0xd2, 0x32, 0x50, 0xd2, 0x2a, 0xa1, 0xa0, - 0xb7, 0x20, 0xc7, 0x32, 0x60, 0x53, 0xa7, 0x1d, 0x2a, 0x2d, 0x67, 0x29, 0xa1, 0x62, 0xea, 0x25, - 0x0c, 0x39, 0x7f, 0x70, 0x92, 0x66, 0xeb, 0x3a, 0xbe, 0x22, 0xe9, 0x6f, 0xf4, 0x3e, 0x5c, 0x78, - 0xdd, 0x55, 0x5b, 0xc6, 0x1e, 0x5d, 0xa9, 0xa3, 0x51, 0xf7, 0x54, 0x67, 0xac, 0x26, 0xc8, 0x4f, - 0xa3, 0x12, 0xa8, 0xea, 0xbc, 0xb1, 0x9c, 0x0e, 0xc6, 0xb2, 0xf4, 0x7b, 0x02, 0xcc, 0xb1, 0xc8, - 0x23, 0x16, 0x30, 0x9b, 0x9c, 0x1f, 0xf2, 0x31, 0xe4, 0x74, 0xd5, 0x55, 0xd9, 0xa6, 0xd5, 0xd4, - 0xd0, 0x4d, 0xab, 0xfe, 0x26, 0x0a, 0xd5, 0x55, 0xe9, 0xc6, 0x55, 0x04, 0x19, 0xf2, 0x9b, 0xed, - 0xef, 0x95, 0xe9, 0x6f, 0xe9, 0x33, 0x40, 0xe1, 0x82, 0x26, 0xe9, 0x91, 0xdd, 0x83, 0x8b, 0x44, - 0xd7, 0x15, 0x53, 0xb3, 0x4f, 0x3a, 0xae, 0x61, 0x99, 0xcf, 0xe9, 0x5f, 0x07, 0x89, 0xa1, 0x0f, - 0x5b, 0xf4, 0x7b, 0x96, 0xf4, 0xa3, 0x29, 0x98, 0xad, 0x1c, 0x77, 0x2c, 0x3b, 0xd1, 0x65, 0xb0, - 0x55, 0x98, 0xe6, 0x2b, 0x05, 0x43, 0xbe, 0x5d, 0xf7, 0x18, 0x73, 0xef, 0xc3, 0x3d, 0x67, 0x44, - 0xab, 0x00, 0x2c, 0xa8, 0x95, 0x06, 0x2e, 0xa5, 0xcf, 0xf1, 0xa9, 0x8d, 0xb2, 0xd1, 0x0d, 0x1c, - 0x5b, 0x90, 0x6f, 0x1f, 0x6a, 0x9a, 0xb2, 0x67, 0xb4, 0x5c, 0x1e, 0x1b, 0x18, 0x1f, 0xc6, 0xbe, - 0xf9, 0xb2, 0x5c, 0x7e, 0x42, 0x33, 0xb1, 0x30, 0xbd, 0xe0, 0x59, 0x06, 0x22, 0x81, 0xfd, 0x46, - 0xef, 0x02, 0xdf, 0x4c, 0xa4, 0x38, 0xde, 0xbe, 0xc1, 0xd5, 0xd9, 0xb3, 0xd3, 0xc5, 0x9c, 0x4c, - 0xa9, 0xf5, 0x7a, 0x43, 0xce, 0xb1, 0x0c, 0x75, 0xc7, 0x45, 0x37, 0x60, 0xd6, 0x6a, 0x1b, 0xae, - 0xe2, 0x39, 0x49, 0xdc, 0xa3, 0x9c, 0x21, 0x44, 0xcf, 0x89, 0x3a, 0xcf, 0x1e, 0x93, 0xe9, 0xb1, - 0xf7, 0x98, 0xa0, 0xbf, 0x29, 0xc0, 0x25, 0xae, 0x48, 0x65, 0x97, 0xc6, 0xe1, 0xab, 0x2d, 0xc3, - 0x3d, 0x51, 0x0e, 0x0e, 0x17, 0xb2, 0xd4, 0x6f, 0xfd, 0xf9, 0xd8, 0x06, 0x09, 0xf5, 0x83, 0x25, - 0xaf, 0x59, 0x4e, 0x9e, 0x71, 0xe6, 0x8d, 0xc3, 0x8a, 0xe9, 0xda, 0x27, 0xab, 0x97, 0xcf, 0x4e, - 0x17, 0xe7, 0xfb, 0x53, 0x5f, 0xca, 0xf3, 0x4e, 0x3f, 0x0b, 0xaa, 0x02, 0x60, 0xbf, 0x1f, 0xd2, - 0x19, 0x23, 0xde, 0xff, 0x88, 0xed, 0xb0, 0x72, 0x88, 0x17, 0xdd, 0x05, 0x91, 0xef, 0xe9, 0xd9, - 0x33, 0x5a, 0x58, 0x71, 0x8c, 0x2f, 0x30, 0x9d, 0x5b, 0xd2, 0x72, 0x81, 0xd1, 0x89, 0x88, 0xba, - 0xf1, 0x05, 0x2e, 0x7d, 0x07, 0x16, 0x06, 0x95, 0x3e, 0x3c, 0x04, 0x72, 0xec, 0x93, 0xee, 0x47, - 0xd1, 0xf5, 0x9c, 0x31, 0xba, 0x2a, 0x5f, 0xd3, 0xf9, 0x38, 0xf5, 0x91, 0x20, 0xfd, 0x83, 0x14, - 0xcc, 0xae, 0x76, 0x5b, 0x07, 0xcf, 0x3b, 0xf5, 0x6e, 0xbb, 0xad, 0xda, 0x27, 0xc4, 0x0c, 0x32, - 0x43, 0x41, 0x0a, 0x28, 0x30, 0x33, 0x48, 0x2d, 0x81, 0xf1, 0x05, 0x26, 0x93, 0x53, 0x28, 0x5c, - 0x86, 0xef, 0x33, 0xa0, 0x75, 0x08, 0xc8, 0x74, 0x2b, 0xc0, 0x47, 0xb0, 0x10, 0xca, 0x48, 0x17, - 0x5f, 0x14, 0x6c, 0xba, 0xb6, 0x81, 0xd9, 0x02, 0x62, 0x5a, 0x0e, 0xc5, 0xf4, 0xd4, 0x48, 0x72, - 0x85, 0xa5, 0xa2, 0x06, 0xcc, 0x90, 0x8c, 0x27, 0x0a, 0x9d, 0x42, 0xbc, 0x05, 0xde, 0x07, 0x31, - 0xd5, 0x8a, 0x94, 0x7b, 0x89, 0xea, 0xa7, 0x4c, 0x79, 0xe8, 0x4f, 0x39, 0x8f, 0x03, 0x4a, 0xe9, - 0x53, 0x10, 0x7b, 0x33, 0x84, 0x75, 0x99, 0x61, 0xba, 0xbc, 0x10, 0xd6, 0x65, 0x3a, 0xa4, 0xa7, - 0xf5, 0x4c, 0x36, 0x23, 0x4e, 0x4a, 0x7f, 0x96, 0x86, 0x82, 0xd7, 0xcd, 0x92, 0x04, 0x3a, 0xab, - 0x30, 0x49, 0x3a, 0x85, 0x17, 0x81, 0x72, 0x7b, 0x48, 0xef, 0xe6, 0x91, 0xed, 0xa4, 0xb3, 0x78, - 0x20, 0x99, 0xb2, 0x26, 0x61, 0x70, 0x4a, 0xbf, 0x92, 0x82, 0x0c, 0xc5, 0x16, 0x0f, 0x20, 0x43, - 0x27, 0x0a, 0x61, 0x9c, 0x89, 0x82, 0x66, 0xf5, 0xa7, 0xb3, 0x54, 0xc8, 0x35, 0x25, 0x3e, 0xdf, - 0xbe, 0xfa, 0xc1, 0x83, 0x87, 0xd4, 0xd8, 0xcc, 0xc8, 0xfc, 0x09, 0xad, 0xd2, 0xd0, 0x28, 0xcb, - 0x76, 0xb1, 0xce, 0x7d, 0xfa, 0x6b, 0xa3, 0xda, 0xd7, 0x9b, 0x94, 0x3c, 0x3e, 0x74, 0x05, 0xd2, - 0xc4, 0x8a, 0x4d, 0xb3, 0x28, 0x87, 0xb3, 0xd3, 0xc5, 0x34, 0xb1, 0x5f, 0x84, 0x86, 0x96, 0x21, - 0x1f, 0x35, 0x19, 0xc4, 0x83, 0xa3, 0x86, 0x31, 0x34, 0xdc, 0xa1, 0xe5, 0x0f, 0x2d, 0x86, 0x67, - 0x79, 0x1b, 0xff, 0x8f, 0x0c, 0xcc, 0xd6, 0xda, 0x49, 0x4f, 0x29, 0x2b, 0xd1, 0x16, 0x8e, 0x03, - 0x42, 0x91, 0x97, 0xc6, 0x34, 0x70, 0x64, 0x06, 0x4f, 0x9f, 0x6f, 0x06, 0xaf, 0x11, 0x4f, 0x99, - 0x1f, 0x3c, 0x91, 0x1e, 0x80, 0x79, 0xa2, 0xef, 0xa7, 0x7e, 0x8a, 0x4c, 0x78, 0x82, 0xbd, 0x1e, - 0x34, 0x52, 0xe5, 0x53, 0xea, 0x90, 0xb3, 0x5e, 0x36, 0x35, 0x7e, 0x2f, 0x9b, 0xc6, 0xa6, 0x4e, - 0x27, 0xb5, 0xa8, 0x45, 0x9d, 0x7e, 0x73, 0x8b, 0x5a, 0x72, 0x79, 0x67, 0xfd, 0x18, 0xd2, 0xba, - 0xe1, 0x35, 0xce, 0xf8, 0x53, 0x35, 0x61, 0x1a, 0xd1, 0x6b, 0x33, 0xe1, 0x5e, 0xcb, 0x7a, 0x49, - 0xa9, 0x06, 0x10, 0xe8, 0x06, 0x5d, 0x83, 0x29, 0xab, 0xa5, 0x7b, 0x9b, 0x5d, 0x66, 0x57, 0x73, - 0x67, 0xa7, 0x8b, 0x93, 0xcf, 0x5b, 0x7a, 0x6d, 0x4d, 0x9e, 0xb4, 0x5a, 0x7a, 0x4d, 0xa7, 0xa7, - 0x7e, 0xe0, 0x23, 0xc5, 0x8f, 0x84, 0x9b, 0x91, 0xa7, 0x4d, 0x7c, 0xb4, 0x86, 0x1d, 0x8d, 0x77, - 0xb8, 0xdf, 0x11, 0xa0, 0xe0, 0xe9, 0x3e, 0x59, 0xa3, 0x92, 0x35, 0xda, 0x7c, 0x90, 0xa5, 0xcf, - 0x37, 0xc8, 0x3c, 0x3e, 0xbe, 0xb1, 0xf8, 0xd7, 0x04, 0x1e, 0xdb, 0x5c, 0xd7, 0x54, 0x97, 0x38, - 0x15, 0x09, 0x0e, 0x8c, 0x7b, 0x20, 0xda, 0xaa, 0xa9, 0x5b, 0x6d, 0xe3, 0x0b, 0xcc, 0x16, 0x42, - 0x1d, 0xfe, 0xd9, 0xb3, 0xe8, 0xd3, 0xe9, 0xaa, 0x9f, 0x23, 0xfd, 0x41, 0x8a, 0xc7, 0x41, 0xfb, - 0xc5, 0x48, 0x52, 0x5d, 0xdf, 0x86, 0xb9, 0xf0, 0xd4, 0xc6, 0x42, 0x2d, 0xd9, 0x68, 0x7d, 0x2f, - 0x46, 0x5e, 0x5c, 0x41, 0x58, 0x3c, 0xa3, 0x17, 0x5c, 0x1f, 0x9a, 0x11, 0xa9, 0x30, 0x54, 0x86, - 0x3c, 0xff, 0xf8, 0x60, 0xee, 0x59, 0x5e, 0xa0, 0xd8, 0xdb, 0x83, 0xc2, 0x22, 0x6b, 0xe6, 0x9e, - 0xe5, 0x7d, 0xc8, 0xb7, 0x3d, 0x82, 0x53, 0xfa, 0x45, 0x98, 0xa4, 0xc9, 0x6f, 0x60, 0xa2, 0x79, - 0x6b, 0xfe, 0x69, 0x0a, 0x6e, 0xd2, 0xd2, 0xbf, 0xc4, 0xb6, 0xb1, 0x77, 0xb2, 0x6d, 0x5b, 0x2e, - 0xd6, 0x5c, 0xac, 0x07, 0x2b, 0xe7, 0x89, 0xda, 0xbd, 0x5c, 0xc7, 0x7b, 0xc1, 0xb9, 0x02, 0xce, - 0x7c, 0x2e, 0xb4, 0x01, 0x45, 0x1e, 0x5a, 0xa0, 0xb6, 0x8c, 0x43, 0xac, 0xa8, 0xee, 0x79, 0x66, - 0xb7, 0x59, 0xc6, 0xbb, 0x42, 0x58, 0x57, 0x5c, 0xa4, 0x43, 0x8e, 0x0b, 0x33, 0x74, 0x7e, 0x20, - 0xd1, 0xd3, 0xaf, 0xb6, 0xa0, 0x98, 0x65, 0xf1, 0x0d, 0xb5, 0x35, 0x39, 0xcb, 0x24, 0xd7, 0x74, - 0xe9, 0x3f, 0x0a, 0x70, 0x6b, 0x84, 0x8a, 0x93, 0xec, 0xba, 0x25, 0xc8, 0x1e, 0x92, 0x17, 0x19, - 0x5c, 0xc7, 0x59, 0xd9, 0x7f, 0x46, 0x9b, 0x30, 0xbb, 0xa7, 0x1a, 0xad, 0xa0, 0x4b, 0x0f, 0x8e, - 0x4f, 0x8c, 0x8f, 0xc6, 0x9d, 0x61, 0xec, 0xac, 0x0f, 0x4b, 0xbf, 0x95, 0x82, 0xb9, 0x15, 0x5d, - 0xaf, 0xd7, 0xb9, 0x0d, 0x4c, 0xae, 0xa7, 0x78, 0x20, 0x33, 0x15, 0x80, 0x4c, 0xf4, 0x1e, 0x20, - 0xdd, 0x70, 0xd8, 0xc1, 0x27, 0xce, 0xbe, 0xaa, 0x5b, 0x47, 0x41, 0x5c, 0xc6, 0x9c, 0x97, 0x52, - 0xf7, 0x12, 0x50, 0x1d, 0x28, 0xda, 0x51, 0x1c, 0x57, 0xf5, 0xbf, 0x3b, 0xdd, 0x1a, 0x6b, 0x4b, - 0x19, 0x83, 0x41, 0xfe, 0xa3, 0x9c, 0x23, 0x72, 0xe8, 0x4f, 0xe2, 0xb7, 0x1b, 0xa4, 0xea, 0xae, - 0xa2, 0x3a, 0xde, 0xfe, 0x21, 0x76, 0xe4, 0x4a, 0x81, 0xd1, 0x57, 0x1c, 0xb6, 0x2d, 0x88, 0x6d, - 0x78, 0x08, 0x54, 0x93, 0x24, 0x24, 0xfe, 0xbb, 0x02, 0x14, 0x64, 0xbc, 0x67, 0x63, 0x27, 0xd1, - 0x45, 0x81, 0x27, 0x30, 0x63, 0x33, 0xa9, 0xca, 0x9e, 0x6d, 0xb5, 0xcf, 0x33, 0xae, 0xf2, 0x9c, - 0xf1, 0x89, 0x6d, 0xb5, 0xb9, 0x61, 0x79, 0x09, 0x45, 0xbf, 0x8c, 0x49, 0x56, 0xfe, 0xf7, 0xe8, - 0x76, 0x69, 0x26, 0x38, 0xe9, 0x00, 0x89, 0x64, 0x35, 0x40, 0x3f, 0x54, 0x85, 0x0b, 0x9a, 0xa4, - 0x1a, 0xfe, 0xab, 0x00, 0x85, 0x7a, 0x77, 0x97, 0x9d, 0xd9, 0x95, 0x9c, 0x06, 0x2a, 0x90, 0x6b, - 0xe1, 0x3d, 0x57, 0x79, 0xa3, 0x50, 0xfd, 0x2c, 0x61, 0xa5, 0x1b, 0x15, 0x9e, 0x02, 0xd8, 0x74, - 0x8b, 0x1d, 0x95, 0x93, 0x3e, 0xa7, 0x9c, 0x1c, 0xe5, 0x25, 0x64, 0xe9, 0x47, 0x69, 0x28, 0xfa, - 0xd5, 0x4c, 0xd2, 0x4a, 0xbe, 0x8a, 0x58, 0x87, 0xf4, 0x79, 0xac, 0xc3, 0x1c, 0x8f, 0x09, 0x89, - 0xb7, 0x10, 0x4b, 0x30, 0x4f, 0x1d, 0x17, 0x45, 0xed, 0x74, 0x5a, 0x86, 0x07, 0x77, 0xa9, 0xfd, - 0xc9, 0xc8, 0x73, 0x34, 0x69, 0x85, 0xa5, 0x50, 0xa0, 0x8b, 0x7e, 0x5d, 0x80, 0x99, 0x3d, 0x1b, - 0xe3, 0x2f, 0xb0, 0x42, 0xa1, 0xd7, 0x78, 0x41, 0x2f, 0x6b, 0xa4, 0x0c, 0x5f, 0xf9, 0xa3, 0x78, - 0x9e, 0xbd, 0xb8, 0x4e, 0xde, 0x8b, 0xb6, 0x40, 0xd4, 0x5a, 0xec, 0x33, 0xbd, 0x1f, 0x80, 0x73, - 0x0e, 0x97, 0xbe, 0xc8, 0x98, 0x7b, 0x76, 0x0f, 0x4a, 0xaf, 0x60, 0x8e, 0x36, 0x72, 0xd2, 0x7b, - 0x95, 0xa5, 0x1f, 0xa6, 0x00, 0x85, 0x25, 0xff, 0xec, 0x3a, 0x47, 0x2a, 0xb9, 0xce, 0xf1, 0x2e, - 0x20, 0x16, 0x86, 0xe9, 0x28, 0x1d, 0x6c, 0x2b, 0x0e, 0xd6, 0x2c, 0x7e, 0xdc, 0x94, 0x20, 0x8b, - 0x3c, 0x65, 0x1b, 0xdb, 0x75, 0x4a, 0x47, 0x2b, 0x00, 0x81, 0x8b, 0xc8, 0x67, 0xb0, 0x71, 0x3c, - 0xc4, 0x9c, 0xef, 0x21, 0x4a, 0xdf, 0x13, 0xa0, 0xb0, 0x69, 0x34, 0x6d, 0x35, 0xd1, 0xd3, 0x94, - 0xd0, 0xc7, 0xd1, 0xc5, 0xf9, 0xfc, 0xc3, 0x52, 0x5c, 0xe0, 0x10, 0xcb, 0xe1, 0x41, 0x3e, 0xce, - 0x40, 0x26, 0x06, 0xbf, 0x44, 0x49, 0x5a, 0xc4, 0x7f, 0x5f, 0x82, 0x19, 0x5e, 0xee, 0x1d, 0xd3, - 0xb0, 0x4c, 0xf4, 0x00, 0xd2, 0x4d, 0xfe, 0xf1, 0x25, 0x1f, 0xbb, 0x50, 0x1a, 0x9c, 0x55, 0x58, - 0x9d, 0x90, 0x49, 0x5e, 0xc2, 0xd2, 0xe9, 0xba, 0x31, 0xae, 0x69, 0x10, 0x4b, 0x1f, 0x66, 0xe9, - 0x74, 0x5d, 0x54, 0x87, 0xa2, 0x16, 0x1c, 0x90, 0xa6, 0x10, 0xf6, 0xf4, 0x40, 0x18, 0x1b, 0x7b, - 0x54, 0x5d, 0x75, 0x42, 0x2e, 0x68, 0x91, 0x04, 0x54, 0x0e, 0x9f, 0xcb, 0x95, 0xe9, 0x0b, 0xd4, - 0x0b, 0x76, 0x81, 0x47, 0xcf, 0x04, 0xab, 0x4e, 0x84, 0x8e, 0xef, 0x42, 0x1f, 0xc3, 0x94, 0x4e, - 0x4f, 0x80, 0xe2, 0x26, 0x25, 0xae, 0xa1, 0x23, 0x07, 0x6d, 0x55, 0x27, 0x64, 0xce, 0x81, 0xd6, - 0x61, 0x86, 0xfd, 0x62, 0x8e, 0x22, 0x37, 0x04, 0xb7, 0x06, 0x4b, 0x08, 0x4d, 0xc5, 0xd5, 0x09, - 0x39, 0xaf, 0x07, 0x54, 0xf4, 0x0d, 0xc8, 0x38, 0x9a, 0xea, 0xa1, 0xfb, 0xab, 0x03, 0x0e, 0x5e, - 0x09, 0x98, 0x69, 0x6e, 0xf4, 0x98, 0x1d, 0x15, 0xea, 0x1e, 0x7b, 0x0b, 0xad, 0x71, 0xc5, 0x8f, - 0x6c, 0xf0, 0x27, 0xc5, 0xc7, 0x94, 0x80, 0x9e, 0x42, 0x5e, 0x25, 0x1e, 0xb7, 0x42, 0xb7, 0xc8, - 0xd2, 0x95, 0xd5, 0xf8, 0xf0, 0x86, 0xbe, 0x2d, 0xcd, 0x55, 0x7a, 0x8a, 0x80, 0x47, 0x0c, 0x04, - 0xb5, 0xb1, 0xdd, 0xc4, 0x0b, 0xf9, 0xe1, 0x82, 0xc2, 0xe1, 0x79, 0xbe, 0x20, 0x4a, 0x24, 0x9e, - 0xb7, 0xbf, 0x3f, 0x9d, 0x56, 0x6a, 0x66, 0xe0, 0x07, 0xf5, 0x98, 0x9d, 0x57, 0xd5, 0x09, 0x79, - 0x66, 0x3f, 0x44, 0x46, 0x4b, 0x90, 0x6a, 0x6a, 0x0b, 0xb3, 0x03, 0x4d, 0x82, 0xbf, 0xbb, 0xa8, - 0x3a, 0x21, 0xa7, 0x9a, 0x1a, 0xfa, 0x14, 0xb2, 0x6c, 0xab, 0xc8, 0xb1, 0xb9, 0x50, 0x18, 0x38, - 0xc6, 0xa2, 0x1b, 0x6e, 0xaa, 0x13, 0x32, 0xdd, 0x9d, 0x42, 0xde, 0xb7, 0x0d, 0x05, 0x9b, 0xc5, - 0x37, 0x7a, 0xd1, 0xc8, 0xe2, 0xc0, 0x20, 0x83, 0xb8, 0x80, 0xe4, 0x2a, 0xc5, 0x5e, 0x21, 0x3a, - 0xfa, 0x36, 0x5c, 0x88, 0x4a, 0xe4, 0x3d, 0x6d, 0x6e, 0xe0, 0x07, 0xf3, 0x81, 0xc1, 0xb1, 0xd5, - 0x09, 0x19, 0xd9, 0x7d, 0x89, 0xe8, 0x43, 0x98, 0x64, 0xad, 0x86, 0xa8, 0xc8, 0xb8, 0xb8, 0x99, - 0x9e, 0x06, 0x63, 0xf9, 0x49, 0xe7, 0x77, 0x79, 0x90, 0x9f, 0xd2, 0xb2, 0x9a, 0x0b, 0xf3, 0x03, - 0x3b, 0x7f, 0x7f, 0xb8, 0x22, 0xe9, 0xfc, 0x6e, 0x40, 0x25, 0xed, 0x6e, 0xb3, 0x14, 0x1e, 0xd8, - 0x75, 0x61, 0x60, 0xbb, 0xc7, 0xc4, 0xfe, 0x55, 0xe9, 0xf6, 0x8b, 0x80, 0x4c, 0x8a, 0x66, 0xb3, - 0xb3, 0x82, 0x14, 0x3a, 0xa6, 0x2e, 0x0e, 0x2c, 0x5a, 0xff, 0x61, 0x4a, 0x55, 0xea, 0xa5, 0xfa, - 0x54, 0xf4, 0x12, 0x44, 0x7e, 0xa2, 0x47, 0xf0, 0x55, 0xe7, 0x12, 0x95, 0x77, 0x2f, 0xd6, 0x74, - 0xc5, 0x45, 0x45, 0x55, 0xc9, 0xc4, 0x1f, 0x4d, 0x41, 0x9f, 0xc1, 0x1c, 0x95, 0xa7, 0x68, 0xc1, - 0x21, 0x2c, 0x0b, 0x0b, 0x7d, 0x47, 0x7a, 0x0c, 0x3e, 0xaf, 0xc5, 0x93, 0x2c, 0x6a, 0x3d, 0x49, - 0xa4, 0x1b, 0x1b, 0xa6, 0xe1, 0x52, 0x2b, 0x5b, 0x1a, 0xd8, 0x8d, 0xa3, 0x07, 0x4a, 0x92, 0x6e, - 0x6c, 0x30, 0x0a, 0xe9, 0xc6, 0x2e, 0x8f, 0xfa, 0xe3, 0xcd, 0xf1, 0xf6, 0xc0, 0x6e, 0x1c, 0x17, - 0x1e, 0x48, 0xba, 0xb1, 0x1b, 0xa6, 0x93, 0x6e, 0xcc, 0x0c, 0x44, 0x8f, 0xdc, 0x77, 0x06, 0x76, - 0xe3, 0x81, 0x5b, 0xd5, 0x49, 0x37, 0x56, 0xfb, 0x12, 0xd1, 0x1a, 0x00, 0x73, 0x28, 0xa9, 0x17, - 0x70, 0x75, 0xe0, 0x64, 0xd0, 0x1b, 0xfd, 0x47, 0x26, 0x83, 0x96, 0x47, 0x23, 0x86, 0x8c, 0xc2, - 0x55, 0x85, 0x7e, 0xc4, 0x5e, 0x58, 0x1c, 0x68, 0xc8, 0xfa, 0x3e, 0x38, 0x13, 0x43, 0x76, 0xe4, - 0x13, 0xc9, 0xac, 0xc2, 0x56, 0xdd, 0x17, 0xae, 0x0d, 0x36, 0xcb, 0xe1, 0x8f, 0x6f, 0xd4, 0x2c, - 0x53, 0x02, 0x5a, 0x81, 0x1c, 0x71, 0x72, 0x4e, 0xa8, 0x19, 0xba, 0x3e, 0x10, 0x15, 0xf4, 0xec, - 0x2f, 0xaa, 0x4e, 0xc8, 0xd9, 0xd7, 0x9c, 0x44, 0x5e, 0xcf, 0xd6, 0x23, 0x17, 0xa4, 0x81, 0xaf, - 0x8f, 0xac, 0x5d, 0x93, 0xd7, 0x33, 0x0e, 0xa4, 0xc1, 0x45, 0xd6, 0x56, 0x7c, 0xd7, 0xb8, 0xcd, - 0x37, 0x3a, 0x2f, 0xdc, 0xa0, 0xa2, 0x06, 0x2e, 0xec, 0xc5, 0x6e, 0x66, 0xaf, 0x4e, 0xc8, 0xf3, - 0x6a, 0x7f, 0x2a, 0x19, 0xf0, 0x7c, 0xea, 0x61, 0xcb, 0x81, 0x0b, 0x37, 0x07, 0x0e, 0xf8, 0x98, - 0x55, 0x54, 0x32, 0xe0, 0xd5, 0x10, 0x99, 0x4d, 0x40, 0xba, 0xe2, 0x38, 0x2c, 0xe4, 0xe1, 0xd6, - 0x90, 0x09, 0xa8, 0x67, 0x1d, 0x86, 0x4d, 0x40, 0x7a, 0x9d, 0x71, 0x12, 0x41, 0x5a, 0x0b, 0xab, - 0x36, 0x37, 0xb3, 0xb7, 0x07, 0x0a, 0xea, 0x3b, 0xbb, 0x91, 0x08, 0xd2, 0x7c, 0x22, 0x71, 0x78, - 0x6c, 0xef, 0x28, 0x20, 0xee, 0x21, 0xdf, 0x19, 0xe8, 0xf0, 0xc4, 0x9e, 0x55, 0x44, 0x1c, 0x1e, - 0x3b, 0x92, 0x80, 0x7e, 0x01, 0xa6, 0x39, 0x80, 0x5e, 0xb8, 0x3b, 0xc4, 0x05, 0x0c, 0xaf, 0x79, - 0x90, 0x71, 0xcd, 0x79, 0x98, 0x95, 0x65, 0xc0, 0x9d, 0x55, 0xef, 0xde, 0x10, 0x2b, 0xdb, 0xb7, - 0x76, 0xc0, 0xac, 0x6c, 0x40, 0x26, 0x56, 0x96, 0xf5, 0x53, 0x3e, 0xd7, 0xdd, 0x1f, 0x68, 0x65, - 0xfb, 0x77, 0x2f, 0x11, 0x2b, 0xfb, 0x3a, 0xa0, 0x92, 0x9a, 0x39, 0x0c, 0xc0, 0x2e, 0xfc, 0xdc, - 0xc0, 0x9a, 0x45, 0x91, 0x3c, 0xa9, 0x19, 0xe7, 0x21, 0xcd, 0xc6, 0x30, 0x00, 0xd3, 0xf4, 0xbb, - 0x83, 0x8f, 0x66, 0xe8, 0x45, 0x59, 0x55, 0x6f, 0xa9, 0x98, 0x69, 0xd8, 0x37, 0x54, 0x36, 0xdf, - 0x37, 0xce, 0x35, 0xf5, 0xde, 0x70, 0x43, 0x15, 0xb7, 0xdf, 0xde, 0x37, 0x54, 0x91, 0x44, 0x5a, - 0x54, 0xb6, 0x55, 0x90, 0x8e, 0xef, 0xa5, 0x21, 0xa7, 0x48, 0xf4, 0x6c, 0xd8, 0xa4, 0x45, 0xf5, - 0x89, 0xc1, 0x10, 0xea, 0xb2, 0x23, 0x4e, 0x16, 0x96, 0x87, 0x0f, 0xa1, 0xe8, 0x21, 0x2b, 0xfe, - 0x10, 0xe2, 0x64, 0x7f, 0xce, 0xf4, 0x3c, 0x8c, 0xf7, 0x87, 0xcf, 0x99, 0xbd, 0xae, 0x05, 0x9b, - 0x33, 0xb9, 0x4f, 0xf1, 0x37, 0x04, 0xb8, 0xc6, 0xca, 0x46, 0xd7, 0x54, 0x4f, 0x14, 0x7f, 0x65, - 0x3a, 0x84, 0x9a, 0x1f, 0xd0, 0x17, 0x7c, 0x38, 0xa8, 0xb8, 0x23, 0x56, 0xda, 0xab, 0x13, 0xf2, - 0x3b, 0xea, 0xb0, 0x7c, 0xa4, 0x4b, 0xb5, 0x19, 0x82, 0x5a, 0x78, 0x38, 0xb0, 0x4b, 0x45, 0x51, - 0x1f, 0xe9, 0x52, 0x9c, 0x67, 0x75, 0x9a, 0x7f, 0xd7, 0xf6, 0xb7, 0xf9, 0x16, 0x45, 0x71, 0x3d, - 0x93, 0xbd, 0x2c, 0x2e, 0xac, 0x67, 0xb2, 0x57, 0xc4, 0xd2, 0x7a, 0x26, 0xfb, 0x96, 0xf8, 0xb6, - 0xf4, 0x0f, 0x4b, 0x30, 0xeb, 0x81, 0x2e, 0x06, 0xa8, 0x1e, 0x86, 0x01, 0xd5, 0xd5, 0x41, 0x80, - 0x8a, 0xc3, 0x34, 0x8e, 0xa8, 0x1e, 0x86, 0x11, 0xd5, 0xd5, 0x41, 0x88, 0x2a, 0xe0, 0x21, 0x90, - 0xaa, 0x31, 0x08, 0x52, 0xdd, 0x1b, 0x03, 0x52, 0xf9, 0xa2, 0x7a, 0x31, 0xd5, 0x5a, 0x3f, 0xa6, - 0xba, 0x39, 0x1c, 0x53, 0xf9, 0xa2, 0x42, 0xa0, 0xea, 0x71, 0x0f, 0xa8, 0xba, 0x3e, 0x04, 0x54, - 0xf9, 0xfc, 0x1e, 0xaa, 0xda, 0x88, 0x45, 0x55, 0xb7, 0x47, 0xa1, 0x2a, 0x5f, 0x4e, 0x04, 0x56, - 0x7d, 0x10, 0x81, 0x55, 0x8b, 0x03, 0x61, 0x95, 0xcf, 0xcd, 0x70, 0xd5, 0x27, 0xbd, 0xb8, 0xea, - 0xfa, 0x10, 0x5c, 0x15, 0xd4, 0x80, 0x03, 0xab, 0x6a, 0x1c, 0xb0, 0xba, 0x35, 0x02, 0x58, 0xf9, - 0x52, 0xc2, 0xc8, 0xaa, 0x1a, 0x87, 0xac, 0x6e, 0x8d, 0x40, 0x56, 0x3d, 0x92, 0x18, 0xb4, 0xda, - 0x8a, 0x87, 0x56, 0x77, 0x46, 0x42, 0x2b, 0x5f, 0x5a, 0x14, 0x5b, 0x2d, 0x87, 0xb0, 0xd5, 0x3b, - 0x03, 0xb0, 0x95, 0xcf, 0x4a, 0xc0, 0xd5, 0x37, 0xfb, 0xc0, 0x95, 0x34, 0x0c, 0x5c, 0xf9, 0xbc, - 0x3e, 0xba, 0x7a, 0x31, 0x00, 0x5d, 0xdd, 0x1d, 0x8d, 0xae, 0x7c, 0x61, 0x3d, 0xf0, 0x4a, 0x1d, - 0x0a, 0xaf, 0xde, 0x1b, 0x13, 0x5e, 0xf9, 0xd2, 0xe3, 0xf0, 0xd5, 0x47, 0x51, 0x7c, 0x75, 0x6d, - 0x30, 0xbe, 0xf2, 0xc5, 0x70, 0x80, 0xb5, 0x11, 0x0b, 0xb0, 0x6e, 0x8f, 0x02, 0x58, 0xc1, 0x38, - 0x08, 0x23, 0xac, 0xad, 0x78, 0x84, 0x75, 0x67, 0x24, 0xc2, 0x0a, 0x9a, 0x3f, 0x02, 0xb1, 0x36, - 0x62, 0x21, 0xd6, 0xed, 0x51, 0x10, 0x2b, 0x28, 0x5c, 0x18, 0x63, 0xbd, 0x1a, 0x88, 0xb1, 0xee, - 0x8f, 0x83, 0xb1, 0x7c, 0xa1, 0x7d, 0x20, 0xeb, 0xf3, 0xc1, 0x20, 0xeb, 0xe7, 0xce, 0x71, 0x28, - 0x66, 0x2c, 0xca, 0xfa, 0x66, 0x1f, 0xca, 0x92, 0x86, 0xa1, 0xac, 0xa0, 0x3f, 0x7b, 0x30, 0x4b, - 0x1d, 0x0a, 0x8a, 0xde, 0x1b, 0x13, 0x14, 0x05, 0x9d, 0x2f, 0x06, 0x15, 0x55, 0x62, 0x50, 0xd1, - 0xcd, 0xe1, 0xa8, 0x28, 0x30, 0xe7, 0x01, 0x2c, 0xaa, 0xc6, 0xc1, 0xa2, 0x5b, 0x23, 0x60, 0x51, - 0x60, 0x85, 0x42, 0xb8, 0xe8, 0x71, 0x0f, 0x2e, 0xba, 0x3e, 0x32, 0x6c, 0x2b, 0x04, 0x8c, 0x56, - 0xfb, 0x81, 0xd1, 0x8d, 0xa1, 0xc0, 0xc8, 0x97, 0x10, 0x20, 0xa3, 0xc7, 0x3d, 0xc8, 0xe8, 0xfa, - 0x10, 0x64, 0x14, 0x14, 0x80, 0x43, 0x23, 0x7d, 0x38, 0x34, 0x5a, 0x1a, 0x17, 0x1a, 0xf9, 0x82, - 0x63, 0xb1, 0xd1, 0x56, 0x3c, 0x36, 0xba, 0x33, 0x66, 0x44, 0x45, 0x1f, 0x38, 0xaa, 0xc6, 0x81, - 0xa3, 0x5b, 0x23, 0xc0, 0x51, 0x78, 0x0e, 0xf1, 0xd1, 0x51, 0x35, 0x0e, 0x1d, 0xdd, 0x1a, 0x81, - 0x8e, 0x02, 0x49, 0x21, 0x78, 0xd4, 0x18, 0x04, 0x8f, 0xee, 0x8d, 0x01, 0x8f, 0x02, 0xe7, 0xa5, - 0x07, 0x1f, 0x7d, 0xda, 0x8b, 0x8f, 0xa4, 0x61, 0xf8, 0x28, 0x18, 0x91, 0x1e, 0x40, 0xda, 0x8a, - 0x07, 0x48, 0x77, 0x46, 0x02, 0xa4, 0xb0, 0x91, 0x0c, 0x21, 0xa4, 0x8d, 0x58, 0x84, 0x74, 0x7b, - 0x14, 0x42, 0x0a, 0x8c, 0x64, 0x18, 0x22, 0x7d, 0xda, 0x0b, 0x91, 0xa4, 0x61, 0x10, 0x29, 0xa8, - 0x9c, 0x87, 0x91, 0xaa, 0x71, 0x18, 0xe9, 0xd6, 0x08, 0x8c, 0x14, 0x34, 0x5e, 0x08, 0x24, 0xa9, - 0x43, 0x41, 0xd2, 0x7b, 0x63, 0x82, 0xa4, 0x1e, 0xc3, 0x15, 0x45, 0x49, 0xd5, 0x38, 0x94, 0x74, - 0x6b, 0x04, 0x4a, 0x0a, 0x15, 0x36, 0x80, 0x49, 0x5b, 0xf1, 0x30, 0xe9, 0xce, 0x48, 0x98, 0xd4, - 0x33, 0x9a, 0x3c, 0x9c, 0xb4, 0x11, 0x8b, 0x93, 0x6e, 0x8f, 0xc2, 0x49, 0x3d, 0x13, 0x1f, 0x77, - 0x0e, 0x7e, 0x75, 0x7c, 0xa0, 0xf4, 0xd1, 0xf9, 0x81, 0x92, 0xff, 0xce, 0x11, 0x48, 0xe9, 0xd3, - 0x5e, 0xa4, 0x24, 0x0d, 0x43, 0x4a, 0x41, 0xcf, 0x3a, 0x1f, 0x54, 0x5a, 0xcf, 0x64, 0xdf, 0x16, - 0xdf, 0x91, 0xfe, 0x7c, 0x0a, 0xa6, 0xaa, 0x7e, 0xa0, 0x53, 0x50, 0x4b, 0xe1, 0x4d, 0x4e, 0xd6, - 0x42, 0x6b, 0x64, 0xc4, 0x53, 0xbb, 0x39, 0xfa, 0x3c, 0xc6, 0xfe, 0x13, 0x03, 0x39, 0xeb, 0x1b, - 0x6c, 0x71, 0x47, 0x1f, 0xc0, 0x6c, 0xd7, 0xc1, 0xb6, 0xd2, 0xb1, 0x0d, 0xcb, 0x36, 0x5c, 0xb6, - 0x8d, 0x47, 0x58, 0x15, 0xbf, 0x3c, 0x5d, 0x9c, 0xd9, 0x71, 0xb0, 0xbd, 0xcd, 0xe9, 0xf2, 0x4c, - 0x37, 0xf4, 0xe4, 0xdd, 0x9d, 0x36, 0x39, 0xfe, 0xdd, 0x69, 0x2f, 0x40, 0xb4, 0xb1, 0xaa, 0x47, - 0x3c, 0x18, 0x76, 0x8a, 0x55, 0x7c, 0x9f, 0xa3, 0xdb, 0xec, 0xbc, 0x9c, 0xf4, 0x34, 0xab, 0xa2, - 0x1d, 0x25, 0xa2, 0x07, 0x70, 0xb1, 0xad, 0x1e, 0xd3, 0xa0, 0x58, 0xc5, 0x73, 0x0a, 0x69, 0xa0, - 0x2b, 0xbb, 0xf0, 0x0c, 0xb5, 0xd5, 0x63, 0x7a, 0x11, 0x1b, 0x4b, 0xa2, 0xb7, 0xa8, 0xdc, 0x82, - 0x82, 0x6e, 0x38, 0xae, 0x61, 0x6a, 0x2e, 0x3f, 0x20, 0x99, 0x1d, 0x2e, 0x3c, 0xeb, 0x51, 0xd9, - 0x29, 0xc8, 0x65, 0x28, 0x36, 0x55, 0x17, 0x1f, 0xa9, 0x27, 0x8a, 0xb7, 0xbf, 0x2e, 0x4f, 0xcf, - 0x02, 0x7d, 0xeb, 0xec, 0x74, 0x71, 0xf6, 0x29, 0x4b, 0xea, 0xdb, 0x66, 0x37, 0xdb, 0x0c, 0x25, - 0xe8, 0xe8, 0x0e, 0x14, 0x55, 0xe7, 0xc4, 0xd4, 0x68, 0x95, 0xb1, 0xe9, 0x74, 0x1d, 0x0a, 0x33, - 0xb2, 0x72, 0x81, 0x92, 0xcb, 0x1e, 0x15, 0x5d, 0x87, 0x19, 0xbe, 0x3d, 0x80, 0xdd, 0xd6, 0x54, - 0xa4, 0xc5, 0xe7, 0xd7, 0x80, 0xb0, 0x0b, 0x9b, 0x1e, 0x43, 0x89, 0x5f, 0x6a, 0x70, 0xa4, 0xda, - 0xba, 0x42, 0x35, 0x19, 0xf4, 0x39, 0x91, 0x8a, 0xbd, 0xcc, 0x2e, 0x31, 0x20, 0x19, 0x88, 0xfa, - 0x82, 0x71, 0xd1, 0x80, 0x39, 0xad, 0x65, 0xf8, 0xa8, 0x80, 0x39, 0x51, 0x73, 0x03, 0x47, 0x48, - 0x99, 0xe6, 0xed, 0xfd, 0xcc, 0x5c, 0xd4, 0xa2, 0x64, 0x54, 0x07, 0x7a, 0x12, 0x90, 0xd2, 0xb1, - 0x5a, 0x86, 0x76, 0x42, 0x61, 0x41, 0xf4, 0x88, 0xf6, 0xa1, 0x17, 0x25, 0xbc, 0x52, 0x0d, 0x77, - 0x9b, 0x72, 0xca, 0x70, 0xe4, 0xff, 0x66, 0xc7, 0x2b, 0xaf, 0x67, 0xb2, 0x20, 0xe6, 0xd7, 0x33, - 0xd9, 0x19, 0x71, 0x76, 0x3d, 0x93, 0x2d, 0x88, 0x45, 0xe9, 0x37, 0x05, 0x98, 0x89, 0xec, 0x8b, - 0x7a, 0xdc, 0xf3, 0x11, 0xf9, 0x4a, 0x3c, 0xfe, 0x1b, 0x14, 0x99, 0x98, 0xe5, 0xfd, 0xc5, 0x0b, - 0xf3, 0x5c, 0x1c, 0x8c, 0x1f, 0xe8, 0x6a, 0x88, 0x17, 0xf3, 0xe2, 0xb1, 0x7d, 0x9c, 0xf9, 0xfe, - 0x0f, 0x16, 0x27, 0xa4, 0xbf, 0xc8, 0xc0, 0x6c, 0x74, 0x17, 0x54, 0xad, 0xa7, 0x5c, 0x71, 0xf6, - 0x39, 0xc2, 0xb1, 0x34, 0xe4, 0x26, 0x98, 0x5c, 0x70, 0x99, 0x02, 0x2b, 0xe6, 0xb5, 0x21, 0x9f, - 0xca, 0xc3, 0xe5, 0x0c, 0x18, 0x4b, 0xff, 0x21, 0xed, 0xdb, 0xa9, 0x25, 0x98, 0xa4, 0x07, 0x1a, - 0xf1, 0xa2, 0x2d, 0xf4, 0xb6, 0x15, 0xf1, 0x59, 0x49, 0xba, 0xcc, 0xb2, 0x11, 0xbb, 0xd6, 0x78, - 0xa3, 0x13, 0x03, 0x83, 0xae, 0x77, 0xfe, 0x3b, 0x16, 0xbb, 0xec, 0xc4, 0xc8, 0xff, 0x87, 0x71, - 0x31, 0xe4, 0x7d, 0xe8, 0x97, 0xa1, 0xa8, 0x59, 0xad, 0x16, 0x9b, 0xb3, 0x98, 0x65, 0xe8, 0x3f, - 0x43, 0x86, 0x16, 0x81, 0x5f, 0xab, 0xb9, 0xe4, 0x5f, 0xaf, 0xb9, 0x24, 0xf3, 0xeb, 0x35, 0x43, - 0x41, 0xb7, 0x05, 0x5f, 0x98, 0x67, 0x50, 0x22, 0xf1, 0xbf, 0xd3, 0x6f, 0x12, 0xff, 0xcb, 0x62, - 0xc6, 0x79, 0xcf, 0xfb, 0x63, 0x81, 0x07, 0xc4, 0x3c, 0xb3, 0xac, 0x83, 0xae, 0x1f, 0xb7, 0x5b, - 0x0a, 0x9f, 0xff, 0x98, 0xfd, 0xf2, 0x74, 0x31, 0x23, 0xfb, 0x07, 0x40, 0xc6, 0x59, 0xe0, 0xd4, - 0x57, 0xb3, 0xc0, 0xd7, 0x61, 0xa6, 0x63, 0xe3, 0x3d, 0xec, 0x6a, 0xfb, 0x8a, 0xd9, 0x6d, 0xf3, - 0xed, 0x3d, 0x79, 0x8f, 0xb6, 0xd5, 0x6d, 0xa3, 0x7b, 0x20, 0xfa, 0x59, 0x38, 0x56, 0xf6, 0x0e, - 0x08, 0xf3, 0xe8, 0x1c, 0x59, 0x4b, 0xff, 0x4b, 0x80, 0xf9, 0x48, 0x9d, 0xf8, 0x98, 0x5a, 0x87, - 0xbc, 0xee, 0xcf, 0x79, 0xce, 0x82, 0x70, 0xce, 0x28, 0xd6, 0x30, 0x33, 0x52, 0xe0, 0x92, 0xf7, - 0x5a, 0x7a, 0x01, 0x41, 0x20, 0x36, 0x75, 0x4e, 0xb1, 0x17, 0x03, 0x39, 0x6b, 0xa1, 0x17, 0xf8, - 0x83, 0x2c, 0x3d, 0xd6, 0x20, 0x93, 0x7e, 0x47, 0x00, 0x91, 0xbe, 0xe0, 0x09, 0xc6, 0x7a, 0x22, - 0xd6, 0xcd, 0x8b, 0x0e, 0x4f, 0x8d, 0xbf, 0x81, 0x27, 0x72, 0x69, 0x4a, 0x3a, 0x7a, 0x69, 0x8a, - 0xf4, 0x03, 0x01, 0x0a, 0x7e, 0x09, 0xd9, 0x45, 0x85, 0x43, 0x8e, 0x19, 0x7d, 0xb3, 0xcb, 0xf8, - 0xbc, 0xe3, 0x50, 0xc6, 0xba, 0x3b, 0x31, 0x7c, 0x1c, 0x0a, 0xbb, 0x44, 0xee, 0x6f, 0x7b, 0x3d, - 0x87, 0x14, 0xb1, 0x1c, 0x1c, 0x7b, 0xf1, 0x06, 0x7b, 0x99, 0x64, 0x7a, 0xc7, 0xab, 0xd5, 0x3a, - 0x64, 0x27, 0xd8, 0x8c, 0x65, 0xf6, 0x10, 0x0f, 0x03, 0x03, 0xbe, 0xaa, 0xa6, 0x37, 0xea, 0xf4, - 0xf6, 0x57, 0xf6, 0xdb, 0x91, 0x9e, 0x84, 0x14, 0x48, 0x1b, 0x9f, 0x68, 0x69, 0x2c, 0x53, 0xec, - 0x69, 0x89, 0xf5, 0x95, 0x3f, 0x0c, 0xb7, 0x44, 0xe5, 0x90, 0xa0, 0xa9, 0x47, 0x90, 0x3e, 0x54, - 0x5b, 0xc3, 0x22, 0xa9, 0x22, 0x2d, 0x27, 0x93, 0xdc, 0xe8, 0x49, 0xe4, 0xb4, 0x90, 0xd4, 0x60, - 0xcf, 0xbf, 0x5f, 0xa5, 0x91, 0x53, 0x45, 0x3e, 0x8c, 0xf6, 0xf5, 0xa1, 0xaf, 0x0f, 0x77, 0xfa, - 0x8f, 0x33, 0x3f, 0xfe, 0xc1, 0xa2, 0x20, 0x7d, 0x02, 0x48, 0xc6, 0x0e, 0x76, 0x5f, 0x74, 0x2d, - 0x3b, 0x38, 0x79, 0xe5, 0x76, 0xcf, 0x2d, 0x32, 0x93, 0xab, 0xf9, 0xb3, 0xb8, 0xc3, 0x99, 0x2e, - 0xc2, 0x7c, 0x84, 0x9b, 0x19, 0x0b, 0xe9, 0x43, 0xb8, 0xf2, 0xd4, 0x72, 0x1c, 0xa3, 0x43, 0x20, - 0x24, 0x1d, 0x95, 0x64, 0x6a, 0xf1, 0xcd, 0x63, 0xb6, 0x43, 0x17, 0x13, 0x4c, 0x66, 0x46, 0x72, - 0xb2, 0xff, 0x2c, 0xfd, 0x48, 0x80, 0xcb, 0xfd, 0x9c, 0x4c, 0xcb, 0x71, 0x5b, 0x2f, 0xa7, 0x35, - 0x2b, 0x38, 0xa9, 0x6f, 0x74, 0x6f, 0xf5, 0xb2, 0x13, 0xb7, 0x90, 0xbf, 0x53, 0x69, 0xab, 0xd4, - 0x7c, 0xf0, 0x6d, 0xe0, 0x05, 0x4e, 0xde, 0x64, 0xd4, 0xc0, 0x92, 0x64, 0xc6, 0xb3, 0x24, 0x0d, - 0x28, 0xae, 0x5b, 0x86, 0x49, 0xbc, 0x4f, 0xaf, 0xbe, 0x2b, 0x50, 0xd8, 0x35, 0x4c, 0xd5, 0x3e, - 0x51, 0xbc, 0x00, 0x3e, 0x61, 0x54, 0x00, 0x9f, 0x3c, 0xcb, 0x38, 0xf8, 0xa3, 0xf4, 0x13, 0x01, - 0xc4, 0x40, 0x2c, 0xb7, 0xc8, 0xef, 0x02, 0x68, 0xad, 0xae, 0xe3, 0x62, 0xdb, 0x6b, 0xa5, 0x19, - 0x16, 0x46, 0x5f, 0x66, 0xd4, 0xda, 0x9a, 0x9c, 0xe3, 0x19, 0x6a, 0x3a, 0xba, 0x11, 0x3d, 0xa5, - 0x62, 0x72, 0x15, 0xce, 0xfa, 0xce, 0xa6, 0x20, 0xcd, 0xee, 0xb8, 0x96, 0xed, 0x63, 0x17, 0xde, - 0xec, 0xde, 0xb9, 0xfb, 0x74, 0x73, 0x35, 0xc9, 0xb7, 0x02, 0x05, 0xe2, 0x2e, 0x1c, 0x62, 0xbf, - 0x4a, 0x99, 0xd1, 0x55, 0x62, 0x1c, 0x5e, 0x95, 0xfe, 0xa5, 0x00, 0xc5, 0x32, 0x6b, 0x0d, 0xbf, - 0x85, 0x87, 0x58, 0xb4, 0x35, 0xc8, 0xba, 0xc7, 0xa6, 0xd2, 0xc6, 0xfe, 0x5d, 0x35, 0xe7, 0x38, - 0x46, 0x6f, 0xda, 0x65, 0x8f, 0xf4, 0x3a, 0x43, 0x7e, 0x97, 0x36, 0x1f, 0x2e, 0x57, 0x96, 0xd8, - 0x65, 0xdb, 0x4b, 0xde, 0x65, 0xdb, 0x4b, 0x6b, 0x3c, 0x03, 0x33, 0xea, 0xdf, 0xff, 0x2f, 0x8b, - 0x82, 0xec, 0x33, 0xb1, 0x79, 0xff, 0x7e, 0x9d, 0xf4, 0xfa, 0xbe, 0x99, 0x19, 0x15, 0x00, 0x42, - 0x97, 0x10, 0xf1, 0xeb, 0x9b, 0x57, 0xd6, 0x94, 0x9d, 0xad, 0xf2, 0xf3, 0xcd, 0xcd, 0x5a, 0xa3, - 0x51, 0x59, 0x13, 0x05, 0x24, 0xc2, 0x4c, 0xe4, 0x0a, 0xa3, 0x14, 0xbb, 0xd0, 0xf9, 0xfe, 0x5f, - 0x01, 0x08, 0x6e, 0x43, 0x23, 0xb2, 0x36, 0x2a, 0x9f, 0x29, 0x2f, 0x57, 0x9e, 0xed, 0x54, 0xea, - 0xe2, 0x04, 0x42, 0x50, 0x58, 0x5d, 0x69, 0x94, 0xab, 0x8a, 0x5c, 0xa9, 0x6f, 0x3f, 0xdf, 0xaa, - 0x57, 0xbc, 0x8b, 0xa0, 0xef, 0xaf, 0xc1, 0x4c, 0xf8, 0x2c, 0x22, 0x34, 0x0f, 0xc5, 0x72, 0xb5, - 0x52, 0xde, 0x50, 0x5e, 0xd6, 0x56, 0x94, 0x17, 0x3b, 0x95, 0x9d, 0x8a, 0x38, 0x41, 0x8b, 0x46, - 0x89, 0x4f, 0x76, 0x9e, 0x3d, 0x13, 0x05, 0x54, 0x84, 0x3c, 0x7b, 0xa6, 0xd7, 0x1d, 0x89, 0xa9, - 0xfb, 0x9b, 0x90, 0x0f, 0x1d, 0x5a, 0x4c, 0x5e, 0xb7, 0xbd, 0x53, 0xaf, 0x2a, 0x8d, 0xda, 0x66, - 0xa5, 0xde, 0x58, 0xd9, 0xdc, 0x66, 0x32, 0x28, 0x6d, 0x65, 0xf5, 0xb9, 0xdc, 0x10, 0x05, 0xff, - 0xb9, 0xf1, 0x7c, 0xa7, 0x5c, 0xf5, 0xaa, 0x21, 0x65, 0xb2, 0x69, 0x31, 0x7d, 0xff, 0xaf, 0x0b, - 0x70, 0x79, 0xc0, 0xb9, 0x3c, 0x28, 0x0f, 0xd3, 0x3b, 0x26, 0x3d, 0xc1, 0x55, 0x9c, 0x40, 0xb3, - 0xa1, 0xa3, 0x79, 0x44, 0x01, 0x65, 0xd9, 0xe1, 0x28, 0x62, 0x0a, 0x4d, 0x41, 0xaa, 0xfe, 0x48, - 0x4c, 0x93, 0x92, 0x86, 0x4e, 0xb6, 0x11, 0x33, 0x28, 0xc7, 0xcf, 0xd4, 0x10, 0x27, 0xd1, 0x4c, - 0x70, 0xb4, 0x85, 0x38, 0x45, 0x44, 0xf9, 0x47, 0x44, 0x88, 0xd3, 0xf7, 0xaf, 0x43, 0x68, 0x1b, - 0x3e, 0x02, 0x98, 0x7a, 0xa6, 0xba, 0xd8, 0x71, 0xc5, 0x09, 0x34, 0x0d, 0xe9, 0x95, 0x56, 0x4b, - 0x14, 0x1e, 0xfe, 0x8b, 0x0c, 0x64, 0xbd, 0x6b, 0x7d, 0xd0, 0x33, 0x98, 0x64, 0xcb, 0xca, 0x8b, - 0x83, 0xd1, 0x02, 0x1d, 0xd0, 0xa5, 0x6b, 0xa3, 0xe0, 0x84, 0x34, 0x81, 0xfe, 0x2a, 0xe4, 0x43, - 0x5e, 0x14, 0x1a, 0xb8, 0x34, 0x16, 0xf1, 0x1c, 0x4b, 0xb7, 0x47, 0x65, 0xf3, 0xe5, 0xbf, 0x82, - 0x9c, 0x6f, 0xd5, 0xd1, 0x8d, 0x61, 0x36, 0xdf, 0x93, 0x3d, 0x7c, 0x62, 0x20, 0xe3, 0x4f, 0x9a, - 0x78, 0x5f, 0x40, 0x36, 0xa0, 0x7e, 0x03, 0x8c, 0xe2, 0x82, 0x15, 0x06, 0x5a, 0xf8, 0xd2, 0xfd, - 0xb1, 0x72, 0x07, 0xef, 0x24, 0xca, 0x0a, 0x66, 0x91, 0x78, 0x65, 0xf5, 0xcd, 0x51, 0xf1, 0xca, - 0x8a, 0x99, 0x8c, 0x26, 0xd0, 0x0b, 0xc8, 0x10, 0xeb, 0x89, 0xe2, 0xfc, 0xca, 0x1e, 0x6b, 0x5d, - 0xba, 0x31, 0x34, 0x8f, 0x27, 0x72, 0xf5, 0xde, 0x8f, 0xff, 0xec, 0xea, 0xc4, 0x8f, 0xcf, 0xae, - 0x0a, 0x3f, 0x39, 0xbb, 0x2a, 0xfc, 0xc9, 0xd9, 0x55, 0xe1, 0x4f, 0xcf, 0xae, 0x0a, 0xdf, 0xfb, - 0xe9, 0xd5, 0x89, 0x9f, 0xfc, 0xf4, 0xea, 0xc4, 0x9f, 0xfc, 0xf4, 0xea, 0xc4, 0xe7, 0xd3, 0x9c, - 0x7b, 0x77, 0x8a, 0x9a, 0x96, 0x47, 0xff, 0x37, 0x00, 0x00, 0xff, 0xff, 0xd5, 0xdb, 0x79, 0x80, - 0x02, 0x81, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_22c0b4cc9c2cd251) } + +var fileDescriptor_api_22c0b4cc9c2cd251 = []byte{ + // 8245 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0xbd, 0x5d, 0x6c, 0x23, 0x59, + 0x76, 0x1f, 0xae, 0x22, 0x29, 0x89, 0x3c, 0x94, 0xc8, 0xd2, 0x55, 0x7f, 0xa8, 0x35, 0x33, 0xad, + 0xee, 0xea, 0xef, 0xde, 0x19, 0x6a, 0xba, 0x7b, 0xe7, 0x3f, 0xe3, 0xe9, 0xf1, 0xac, 0x25, 0x8a, + 0xdd, 0xa4, 0xd4, 0x52, 0xab, 0x8b, 0x54, 0x37, 0x66, 0xbc, 0xfe, 0xd7, 0x96, 0xaa, 0xae, 0xa8, + 0x5a, 0x91, 0x55, 0xec, 0xaa, 0xa2, 0x3e, 0x06, 0x08, 0x10, 0xc7, 0x86, 0xb3, 0x4f, 0xc6, 0x06, + 0x30, 0x90, 0x35, 0x1c, 0xc4, 0xeb, 0x6c, 0x10, 0x3f, 0x04, 0x48, 0x02, 0x24, 0xc8, 0x17, 0x12, + 0xfb, 0xc5, 0x40, 0x16, 0x81, 0x13, 0xaf, 0x9f, 0x62, 0x04, 0x88, 0x62, 0x6b, 0xf3, 0x12, 0x38, + 0x08, 0x82, 0x20, 0x80, 0x81, 0x79, 0x08, 0x82, 0xfb, 0x51, 0x5f, 0x64, 0xf1, 0x43, 0x3d, 0xb5, + 0xc9, 0x00, 0x7e, 0x91, 0x58, 0xa7, 0xee, 0x39, 0x75, 0xef, 0xb9, 0xe7, 0x9e, 0x7b, 0x7e, 0xb7, + 0xce, 0xbd, 0x05, 0x73, 0xb6, 0xa5, 0x6a, 0xfb, 0x9d, 0xdd, 0x65, 0xb5, 0x63, 0x94, 0x3a, 0xb6, + 0xe5, 0x5a, 0x68, 0x4e, 0xb3, 0xb4, 0x03, 0x4a, 0x2e, 0xf1, 0x9b, 0x8b, 0xf7, 0x0f, 0x0e, 0x97, + 0x0f, 0x0e, 0x1d, 0x6c, 0x1f, 0x62, 0x7b, 0x59, 0xb3, 0x4c, 0xad, 0x6b, 0xdb, 0xd8, 0xd4, 0x4e, + 0x96, 0x5b, 0x96, 0x76, 0x40, 0xff, 0x18, 0x66, 0x93, 0xb1, 0x47, 0xcb, 0xda, 0x58, 0xd5, 0x9d, + 0x6e, 0xbb, 0xad, 0xda, 0x27, 0xcb, 0xb6, 0xd3, 0xd9, 0x5d, 0xe6, 0x17, 0xbc, 0x2c, 0xf2, 0x9e, + 0xae, 0xab, 0xae, 0xca, 0x69, 0x17, 0x3c, 0x1a, 0xb6, 0x6d, 0xcb, 0x76, 0x38, 0xf5, 0x92, 0x47, + 0x6d, 0x63, 0x57, 0x0d, 0x95, 0x7e, 0xcb, 0x71, 0x2d, 0x5b, 0x6d, 0xe2, 0x65, 0x6c, 0x36, 0x0d, + 0x13, 0x93, 0x02, 0x87, 0x9a, 0xc6, 0x6f, 0xbe, 0x1d, 0x7b, 0xf3, 0x11, 0xbf, 0xbb, 0xd0, 0x75, + 0x8d, 0xd6, 0xf2, 0x7e, 0x4b, 0x5b, 0x76, 0x8d, 0x36, 0x76, 0x5c, 0xb5, 0xdd, 0xf1, 0x9a, 0x40, + 0xef, 0xb8, 0xb6, 0xaa, 0x19, 0x66, 0xd3, 0xfb, 0xdf, 0xd9, 0x5d, 0xb6, 0xb1, 0x66, 0xd9, 0x3a, + 0xd6, 0x15, 0xa7, 0xa3, 0x9a, 0x5e, 0x75, 0x9b, 0x56, 0xd3, 0xa2, 0x3f, 0x97, 0xc9, 0x2f, 0x4e, + 0xbd, 0xda, 0xb4, 0xac, 0x66, 0x0b, 0x2f, 0xd3, 0xab, 0xdd, 0xee, 0xde, 0xb2, 0xde, 0xb5, 0x55, + 0xd7, 0xb0, 0x38, 0x97, 0xf4, 0x4f, 0x05, 0x98, 0x95, 0xf1, 0xeb, 0x2e, 0x76, 0xdc, 0x2a, 0x56, + 0x75, 0x6c, 0xa3, 0x2b, 0x90, 0x3e, 0xc0, 0x27, 0x0b, 0xe9, 0x6b, 0xc2, 0xdd, 0x99, 0xd5, 0xe9, + 0x2f, 0x4f, 0x97, 0xd2, 0x1b, 0xf8, 0x44, 0x26, 0x34, 0x74, 0x0d, 0xa6, 0xb1, 0xa9, 0x2b, 0xe4, + 0x76, 0x26, 0x7a, 0x7b, 0x0a, 0x9b, 0xfa, 0x06, 0x3e, 0x41, 0xdf, 0x86, 0xac, 0x43, 0xa4, 0x99, + 0x1a, 0x5e, 0x98, 0xbc, 0x26, 0xdc, 0x9d, 0x5c, 0xfd, 0x85, 0x2f, 0x4f, 0x97, 0x3e, 0x69, 0x1a, + 0xee, 0x7e, 0x77, 0xb7, 0xa4, 0x59, 0xed, 0x65, 0xbf, 0x4f, 0xf5, 0xdd, 0xe0, 0xf7, 0x72, 0xe7, + 0xa0, 0xb9, 0xdc, 0xab, 0xa3, 0x52, 0xe3, 0xd8, 0xac, 0xe3, 0xd7, 0xb2, 0x2f, 0x71, 0x3d, 0x93, + 0x15, 0xc4, 0xd4, 0x7a, 0x26, 0x9b, 0x12, 0xd3, 0xd2, 0x1f, 0xa5, 0xa0, 0x20, 0x63, 0xa7, 0x63, + 0x99, 0x0e, 0xe6, 0x35, 0x7f, 0x1f, 0xd2, 0xee, 0xb1, 0x49, 0x6b, 0x9e, 0x7f, 0x78, 0xb5, 0xd4, + 0x67, 0x3d, 0xa5, 0x86, 0xad, 0x9a, 0x8e, 0xaa, 0x91, 0xe6, 0xcb, 0xa4, 0x28, 0xfa, 0x08, 0xf2, + 0x36, 0x76, 0xba, 0x6d, 0x4c, 0x15, 0x49, 0x1b, 0x95, 0x7f, 0x78, 0x39, 0x86, 0xb3, 0xde, 0x51, + 0x4d, 0x19, 0x58, 0x59, 0xf2, 0x1b, 0x5d, 0x81, 0xac, 0xd9, 0x6d, 0x13, 0x55, 0x38, 0xb4, 0xa1, + 0x69, 0x79, 0xda, 0xec, 0xb6, 0x37, 0xf0, 0x89, 0x83, 0xea, 0x30, 0xcb, 0x85, 0xda, 0x58, 0x75, + 0x2c, 0x73, 0x61, 0xfa, 0x9a, 0x70, 0xb7, 0xf0, 0xb0, 0x14, 0x23, 0x36, 0xda, 0x00, 0x72, 0xd9, + 0x6d, 0x63, 0x99, 0x72, 0xc9, 0x33, 0x76, 0xe8, 0x0a, 0xbd, 0x05, 0x39, 0xf2, 0xbc, 0xdd, 0x13, + 0x17, 0x3b, 0x0b, 0x59, 0xfa, 0x40, 0x52, 0x81, 0x55, 0x72, 0x2d, 0x7d, 0x0a, 0x33, 0x61, 0x56, + 0x84, 0xa0, 0x20, 0x57, 0xea, 0x3b, 0x9b, 0x15, 0x65, 0x67, 0x6b, 0x63, 0xeb, 0xf9, 0xab, 0x2d, + 0x71, 0x02, 0x5d, 0x00, 0x91, 0xd3, 0x36, 0x2a, 0x9f, 0x29, 0xcf, 0x6a, 0x9b, 0xb5, 0x86, 0x28, + 0x2c, 0x66, 0xbe, 0xf7, 0xa3, 0xab, 0x13, 0xeb, 0x99, 0xec, 0x94, 0x38, 0x2d, 0xfd, 0x48, 0x00, + 0x78, 0x8a, 0x5d, 0x6e, 0x0d, 0x68, 0x15, 0xa6, 0xf6, 0x69, 0xb5, 0x16, 0x04, 0xaa, 0x96, 0x6b, + 0xb1, 0xf5, 0x0f, 0x59, 0xce, 0x6a, 0xf6, 0xc7, 0xa7, 0x4b, 0x13, 0x3f, 0x39, 0x5d, 0x12, 0x64, + 0xce, 0x89, 0x5e, 0x40, 0xfe, 0x00, 0x9f, 0x28, 0x7c, 0x5c, 0x2e, 0xa4, 0xa8, 0x22, 0xde, 0x0f, + 0x09, 0x3a, 0x38, 0x2c, 0x79, 0x43, 0xb4, 0x14, 0x1a, 0xce, 0x25, 0xc2, 0x51, 0xaa, 0xbb, 0x36, + 0x36, 0x9b, 0xee, 0xbe, 0x0c, 0x07, 0xf8, 0xe4, 0x19, 0x93, 0x21, 0xfd, 0xbe, 0x00, 0x79, 0x5a, + 0x4b, 0xa6, 0x39, 0x54, 0xee, 0xa9, 0xe6, 0xf5, 0x91, 0x6a, 0x8e, 0xa9, 0x67, 0x09, 0x26, 0x0f, + 0xd5, 0x56, 0x17, 0xd3, 0x1a, 0xe6, 0x1f, 0x2e, 0xc4, 0xc8, 0x78, 0x49, 0xee, 0xcb, 0xac, 0x18, + 0x7a, 0x0c, 0x33, 0x86, 0xe9, 0x62, 0xd3, 0x55, 0x18, 0x5b, 0x7a, 0x04, 0x5b, 0x9e, 0x95, 0xa6, + 0x17, 0xd2, 0x3f, 0x11, 0x00, 0xb6, 0xbb, 0x89, 0xea, 0xf9, 0x9b, 0x63, 0xd6, 0x7f, 0x35, 0x43, + 0x58, 0xbd, 0x56, 0x5c, 0x82, 0x29, 0xc3, 0x6c, 0x19, 0x26, 0xab, 0x7f, 0x56, 0xe6, 0x57, 0xe8, + 0x02, 0x4c, 0xee, 0xb6, 0x0c, 0x53, 0xa7, 0xe3, 0x21, 0x2b, 0xb3, 0x0b, 0x49, 0x86, 0x3c, 0xad, + 0x75, 0x82, 0x7a, 0x97, 0x4e, 0x53, 0x70, 0xb1, 0x6c, 0x99, 0xba, 0x41, 0x86, 0xa4, 0xda, 0xfa, + 0x5a, 0x68, 0x65, 0x1d, 0x2e, 0xe8, 0xb8, 0x63, 0x63, 0x4d, 0x75, 0xb1, 0xae, 0xe0, 0xe3, 0xce, + 0x98, 0x7d, 0x8c, 0x02, 0xae, 0xca, 0x71, 0x87, 0xd2, 0xe2, 0x35, 0x89, 0xbe, 0x09, 0x97, 0xd5, + 0x56, 0xcb, 0x3a, 0x52, 0x8c, 0x3d, 0x45, 0xb7, 0xb0, 0xa3, 0x98, 0x96, 0xab, 0xe0, 0x63, 0xc3, + 0x71, 0xa9, 0x2b, 0xc9, 0xca, 0xf3, 0xf4, 0x76, 0x6d, 0x6f, 0xcd, 0xc2, 0xce, 0x96, 0xe5, 0x56, + 0xc8, 0x2d, 0xe2, 0x01, 0x48, 0x65, 0x98, 0x07, 0x98, 0x22, 0xee, 0x57, 0xce, 0xe2, 0xe3, 0x0e, + 0xf5, 0x00, 0xa1, 0xae, 0x9c, 0x0e, 0x77, 0xa5, 0xf4, 0x4b, 0x70, 0xa9, 0x57, 0xbf, 0x49, 0xf6, + 0xdf, 0x1f, 0x0a, 0x50, 0xa8, 0x99, 0x86, 0xfb, 0xb5, 0xe8, 0x38, 0x5f, 0xd9, 0xe9, 0xb0, 0xb2, + 0xef, 0x83, 0xb8, 0xa7, 0x1a, 0xad, 0xe7, 0x66, 0xc3, 0x6a, 0xef, 0x3a, 0xae, 0x65, 0x62, 0x87, + 0xf7, 0x46, 0x1f, 0x5d, 0x7a, 0x09, 0x45, 0xbf, 0x35, 0x49, 0xaa, 0xc9, 0x05, 0xb1, 0x66, 0x6a, + 0x36, 0x6e, 0x63, 0x33, 0x51, 0x3d, 0xbd, 0x0d, 0x39, 0xc3, 0x93, 0x4b, 0x75, 0x95, 0x96, 0x03, + 0x82, 0xd4, 0x85, 0xb9, 0xd0, 0x53, 0x93, 0x74, 0x97, 0x64, 0x32, 0xc2, 0x47, 0x4a, 0xd0, 0x47, + 0x64, 0x32, 0xc2, 0x47, 0xcc, 0xbd, 0xd5, 0x61, 0x76, 0x0d, 0xb7, 0xb0, 0x8b, 0x13, 0x6c, 0xa9, + 0xb4, 0x03, 0x05, 0x4f, 0x68, 0x92, 0x1d, 0xf3, 0x37, 0x05, 0x40, 0x5c, 0xae, 0x6a, 0x36, 0x93, + 0xac, 0x31, 0x5a, 0x22, 0xa1, 0x85, 0xdb, 0xb5, 0x4d, 0x16, 0x23, 0x30, 0x9b, 0x04, 0x46, 0xa2, + 0x61, 0x42, 0x30, 0x64, 0x33, 0xe1, 0x21, 0xcb, 0xc3, 0x9b, 0x23, 0x98, 0x8f, 0x54, 0x2c, 0xd9, + 0xee, 0xcb, 0xd0, 0x3a, 0xa5, 0xae, 0xa5, 0xc3, 0x31, 0x1c, 0x25, 0x4a, 0x3f, 0x10, 0x60, 0xae, + 0xdc, 0xc2, 0xaa, 0x9d, 0xb8, 0x46, 0xbe, 0x05, 0x59, 0x1d, 0xab, 0x3a, 0x6d, 0x32, 0x1b, 0xd8, + 0xef, 0x84, 0xa4, 0x90, 0x48, 0xb7, 0xb4, 0xdf, 0xd2, 0x4a, 0x0d, 0x2f, 0x06, 0xe6, 0xa3, 0xdb, + 0x67, 0x92, 0x3e, 0x03, 0x14, 0xae, 0x59, 0x92, 0x86, 0xf0, 0xbf, 0x04, 0x40, 0x32, 0x3e, 0xc4, + 0xb6, 0x9b, 0x78, 0xb3, 0xd7, 0x20, 0xef, 0xaa, 0x76, 0x13, 0xbb, 0x0a, 0x89, 0xee, 0xcf, 0xd3, + 0x72, 0x60, 0x7c, 0x84, 0x8c, 0x1a, 0x70, 0x07, 0x9b, 0xea, 0x6e, 0x0b, 0x53, 0x29, 0xca, 0xae, + 0xd5, 0x35, 0x75, 0xc5, 0x70, 0xb1, 0xad, 0xba, 0x96, 0xad, 0x58, 0x1d, 0xd7, 0x68, 0x1b, 0x5f, + 0xd0, 0xc0, 0x9e, 0x9b, 0xda, 0x0d, 0x56, 0x9c, 0x30, 0xaf, 0x92, 0xc2, 0x35, 0x5e, 0xf6, 0x79, + 0xa8, 0xa8, 0xf4, 0x39, 0xcc, 0x47, 0x5a, 0x9d, 0xa4, 0x4a, 0xff, 0x87, 0x00, 0xf9, 0xba, 0xa6, + 0x9a, 0x49, 0xea, 0xf2, 0x53, 0xc8, 0x3b, 0x9a, 0x6a, 0x2a, 0x7b, 0x96, 0xdd, 0x56, 0x5d, 0x3a, + 0x70, 0x0a, 0x11, 0x5d, 0xfa, 0xf1, 0xba, 0xa6, 0x9a, 0x4f, 0x68, 0x21, 0x19, 0x1c, 0xff, 0x77, + 0x6f, 0x3c, 0x3a, 0xf9, 0xd5, 0xe3, 0x51, 0x36, 0x5c, 0xd7, 0x33, 0xd9, 0xb4, 0x98, 0x91, 0xfe, + 0x42, 0x80, 0x19, 0xd6, 0xe4, 0x24, 0x87, 0xeb, 0x07, 0x90, 0xb1, 0xad, 0x23, 0x36, 0x5c, 0xf3, + 0x0f, 0xdf, 0x8a, 0x11, 0xb1, 0x81, 0x4f, 0xc2, 0xf3, 0x21, 0x2d, 0x8e, 0x56, 0x81, 0x47, 0x9d, + 0x0a, 0xe5, 0x4e, 0x8f, 0xcb, 0x0d, 0x8c, 0x4b, 0x26, 0x32, 0xee, 0x40, 0x71, 0x57, 0x75, 0xb5, + 0x7d, 0xc5, 0xe6, 0x95, 0x24, 0x73, 0x67, 0xfa, 0xee, 0x8c, 0x5c, 0xa0, 0x64, 0xaf, 0xea, 0x0e, + 0x69, 0x39, 0x1b, 0x3f, 0x0e, 0xfe, 0x4b, 0xd6, 0xe7, 0xff, 0x5b, 0xe0, 0x63, 0xc8, 0x6b, 0xf9, + 0x5f, 0xb6, 0xae, 0xff, 0xcd, 0x14, 0x5c, 0x2e, 0xef, 0x63, 0xed, 0xa0, 0x6c, 0x99, 0x8e, 0xe1, + 0xb8, 0x44, 0x77, 0x49, 0xf6, 0xff, 0x5b, 0x90, 0x3b, 0x32, 0xdc, 0x7d, 0x45, 0x37, 0xf6, 0xf6, + 0xa8, 0xf7, 0xcc, 0xca, 0x59, 0x42, 0x58, 0x33, 0xf6, 0xf6, 0xd0, 0x23, 0xc8, 0xb4, 0x2d, 0x9d, + 0x05, 0xe7, 0x85, 0x87, 0x4b, 0x31, 0xe2, 0x69, 0xd5, 0x9c, 0x6e, 0x7b, 0xd3, 0xd2, 0xb1, 0x4c, + 0x0b, 0xa3, 0xab, 0x00, 0x1a, 0xa1, 0x76, 0x2c, 0xc3, 0x74, 0xf9, 0xec, 0x1b, 0xa2, 0xa0, 0x2a, + 0xe4, 0x5c, 0x6c, 0xb7, 0x0d, 0x53, 0x75, 0xf1, 0xc2, 0x24, 0x55, 0xde, 0xcd, 0xd8, 0x8a, 0x77, + 0x5a, 0x86, 0xa6, 0xae, 0x61, 0x47, 0xb3, 0x8d, 0x8e, 0x6b, 0xd9, 0x5c, 0x8b, 0x01, 0xb3, 0xf4, + 0xeb, 0x19, 0x58, 0xe8, 0xd7, 0x4d, 0x92, 0x16, 0xb2, 0x0d, 0x53, 0x36, 0x76, 0xba, 0x2d, 0x97, + 0xdb, 0xc8, 0xc3, 0x41, 0x2a, 0x88, 0xa9, 0x01, 0x5d, 0x6f, 0x68, 0xb9, 0xbc, 0xda, 0x5c, 0xce, + 0xe2, 0xbf, 0x12, 0x60, 0x8a, 0xdd, 0x40, 0x0f, 0x20, 0x6b, 0x93, 0x89, 0x41, 0x31, 0x74, 0x5a, + 0xc7, 0xf4, 0xea, 0xa5, 0xb3, 0xd3, 0xa5, 0x69, 0x3a, 0x59, 0xd4, 0xd6, 0xbe, 0x0c, 0x7e, 0xca, + 0xd3, 0xb4, 0x5c, 0x4d, 0x27, 0xbd, 0xe5, 0xb8, 0xaa, 0xed, 0xd2, 0x45, 0xa2, 0x14, 0x43, 0x29, + 0x94, 0xb0, 0x81, 0x4f, 0xd0, 0x3a, 0x4c, 0x39, 0xae, 0xea, 0x76, 0x1d, 0xde, 0x5f, 0xe7, 0xaa, + 0x6c, 0x9d, 0x72, 0xca, 0x5c, 0x02, 0x09, 0x9f, 0x74, 0xec, 0xaa, 0x46, 0x8b, 0x76, 0x60, 0x4e, + 0xe6, 0x57, 0xd2, 0x6f, 0x09, 0x30, 0xc5, 0x8a, 0xa2, 0xcb, 0x30, 0x2f, 0xaf, 0x6c, 0x3d, 0xad, + 0x28, 0xb5, 0xad, 0xb5, 0x4a, 0xa3, 0x22, 0x6f, 0xd6, 0xb6, 0x56, 0x1a, 0x15, 0x71, 0x02, 0x5d, + 0x02, 0xe4, 0xdd, 0x28, 0x3f, 0xdf, 0xaa, 0xd7, 0xea, 0x8d, 0xca, 0x56, 0x43, 0x14, 0xe8, 0x1a, + 0x09, 0xa5, 0x87, 0xa8, 0x29, 0x74, 0x13, 0xae, 0xf5, 0x52, 0x95, 0x7a, 0x63, 0xa5, 0x51, 0x57, + 0x2a, 0xf5, 0x46, 0x6d, 0x73, 0xa5, 0x51, 0x59, 0x13, 0xd3, 0x43, 0x4a, 0x91, 0x87, 0xc8, 0x72, + 0xa5, 0xdc, 0x10, 0x33, 0x92, 0x0b, 0x17, 0x65, 0xac, 0x59, 0xed, 0x4e, 0xd7, 0xc5, 0xa4, 0x96, + 0x4e, 0x92, 0x23, 0xe5, 0x32, 0x4c, 0xeb, 0xf6, 0x89, 0x62, 0x77, 0x4d, 0x3e, 0x4e, 0xa6, 0x74, + 0xfb, 0x44, 0xee, 0x9a, 0xd2, 0x3f, 0x12, 0xe0, 0x52, 0xef, 0x63, 0x93, 0x34, 0xc2, 0x17, 0x90, + 0x57, 0x75, 0x1d, 0xeb, 0x8a, 0x8e, 0x5b, 0xae, 0xca, 0x43, 0x9c, 0xfb, 0x21, 0x49, 0x7c, 0x69, + 0xaf, 0xe4, 0x2f, 0xed, 0x6d, 0xbe, 0x2c, 0x97, 0x69, 0x45, 0xd6, 0x08, 0x87, 0xe7, 0x7e, 0xa8, + 0x10, 0x4a, 0x91, 0xfe, 0x5b, 0x06, 0x66, 0x2b, 0xa6, 0xde, 0x38, 0x4e, 0x74, 0x2e, 0xb9, 0x04, + 0x53, 0x9a, 0xd5, 0x6e, 0x1b, 0xae, 0xa7, 0x20, 0x76, 0x85, 0x7e, 0x2e, 0x14, 0x9a, 0xa6, 0xc7, + 0x08, 0xd0, 0x82, 0xa0, 0x14, 0x7d, 0x07, 0x2e, 0x13, 0xaf, 0x69, 0x9b, 0x6a, 0x4b, 0x61, 0xd2, + 0x14, 0xd7, 0x36, 0x9a, 0x4d, 0x6c, 0xf3, 0xe5, 0xc4, 0xbb, 0x31, 0xf5, 0xac, 0x71, 0x8e, 0x32, + 0x65, 0x68, 0xb0, 0xf2, 0xf2, 0x45, 0x23, 0x8e, 0x8c, 0x3e, 0x01, 0x20, 0x53, 0x11, 0x5d, 0xa2, + 0x74, 0xb8, 0x3f, 0x1a, 0xb4, 0x46, 0xe9, 0xb9, 0x20, 0xc2, 0x40, 0xae, 0x1d, 0xb4, 0x4c, 0x70, + 0xc8, 0xeb, 0xae, 0x61, 0x63, 0xe5, 0x41, 0x47, 0xa3, 0x0b, 0x07, 0xd9, 0xd5, 0xc2, 0xd9, 0xe9, + 0x12, 0xc8, 0x8c, 0xfc, 0x60, 0xbb, 0x4c, 0x70, 0x09, 0xfb, 0xdd, 0xd1, 0xd0, 0x2b, 0xb8, 0x17, + 0x5a, 0xff, 0x20, 0x33, 0x2f, 0x6f, 0x96, 0xea, 0x2a, 0xfb, 0x46, 0x73, 0x1f, 0xdb, 0x8a, 0xbf, + 0x4c, 0x4d, 0x57, 0x22, 0xb3, 0xf2, 0xcd, 0x80, 0xa1, 0xac, 0x9a, 0xac, 0xf6, 0x2b, 0x6e, 0x95, + 0x16, 0xf6, 0x75, 0x46, 0x94, 0xdf, 0xb1, 0x0c, 0xc7, 0x32, 0x17, 0x72, 0x4c, 0xf9, 0xec, 0x0a, + 0xdd, 0x03, 0xd1, 0x3d, 0x36, 0x95, 0x7d, 0xac, 0xda, 0xee, 0x2e, 0x56, 0x5d, 0x32, 0x4b, 0x03, + 0x2d, 0x51, 0x74, 0x8f, 0xcd, 0x6a, 0x88, 0x8c, 0x5e, 0x80, 0x68, 0x98, 0xca, 0x5e, 0xcb, 0x68, + 0xee, 0xbb, 0xca, 0x91, 0x6d, 0xb8, 0xd8, 0x59, 0x98, 0xa3, 0x0a, 0x89, 0xb3, 0xdb, 0x3a, 0x5f, + 0x37, 0xd6, 0x5f, 0x91, 0x92, 0x5c, 0x35, 0x05, 0xc3, 0x7c, 0x42, 0xf9, 0x29, 0xd1, 0x59, 0xcf, + 0x64, 0xa7, 0xc5, 0xac, 0xf4, 0x9f, 0x04, 0x28, 0x78, 0xe6, 0x96, 0xe4, 0xc8, 0xb8, 0x0b, 0xa2, + 0x65, 0x62, 0xa5, 0xb3, 0xaf, 0x3a, 0x98, 0xeb, 0x91, 0x4f, 0x38, 0x05, 0xcb, 0xc4, 0xdb, 0x84, + 0xcc, 0xd4, 0x85, 0xb6, 0x61, 0xce, 0x71, 0xd5, 0xa6, 0x61, 0x36, 0x43, 0xea, 0x9d, 0x1c, 0x1f, + 0x2c, 0x88, 0x9c, 0xdb, 0xa7, 0x47, 0xa2, 0x94, 0x3f, 0x16, 0x60, 0x6e, 0x45, 0x6f, 0x1b, 0x66, + 0xbd, 0xd3, 0x32, 0x12, 0x5d, 0x83, 0xb8, 0x09, 0x39, 0x87, 0xc8, 0x0c, 0x1c, 0x7e, 0x80, 0x28, + 0xb3, 0xf4, 0x0e, 0xf1, 0xfc, 0xcf, 0xa0, 0x88, 0x8f, 0x3b, 0x06, 0x7b, 0xf5, 0xc0, 0x80, 0x50, + 0x66, 0xfc, 0xb6, 0x15, 0x02, 0x5e, 0x72, 0x8b, 0xb7, 0xe9, 0x33, 0x40, 0xe1, 0x26, 0x25, 0x89, + 0x5d, 0x3e, 0x83, 0x79, 0x2a, 0x7a, 0xc7, 0x74, 0x12, 0xd6, 0x97, 0xf4, 0x8b, 0x70, 0x21, 0x2a, + 0x3a, 0xc9, 0x7a, 0xbf, 0xe2, 0xbd, 0xbc, 0x89, 0xed, 0x44, 0x41, 0xac, 0xaf, 0x6b, 0x2e, 0x38, + 0xc9, 0x3a, 0xff, 0xaa, 0x00, 0x57, 0xa8, 0x6c, 0xfa, 0x76, 0x66, 0x0f, 0xdb, 0xcf, 0xb0, 0xea, + 0x24, 0x8a, 0xc0, 0x6f, 0xc0, 0x14, 0x43, 0xd2, 0xd4, 0x3e, 0x27, 0x57, 0xf3, 0x24, 0x72, 0xa9, + 0xbb, 0x96, 0x4d, 0x22, 0x17, 0x7e, 0x4b, 0x52, 0x61, 0x31, 0xae, 0x16, 0x49, 0xb6, 0xf4, 0x6f, + 0x0b, 0x30, 0xc7, 0x83, 0x46, 0x62, 0xca, 0xe5, 0x7d, 0x12, 0x33, 0xa1, 0x0a, 0xe4, 0x35, 0xfa, + 0x4b, 0x71, 0x4f, 0x3a, 0x98, 0xca, 0x2f, 0x0c, 0x8b, 0x37, 0x19, 0x5b, 0xe3, 0xa4, 0x83, 0x49, + 0xd0, 0xea, 0xfd, 0x26, 0x8a, 0x0a, 0x35, 0x72, 0x68, 0xc4, 0x4a, 0xc7, 0x11, 0x2d, 0xeb, 0x85, + 0x7e, 0x5c, 0x07, 0xff, 0x38, 0xcd, 0x95, 0xc0, 0x9e, 0xc1, 0x8b, 0x27, 0x1a, 0xa3, 0x7c, 0x0e, + 0x97, 0xc2, 0xb3, 0x4b, 0xa8, 0xe1, 0xa9, 0x73, 0x34, 0x3c, 0xb4, 0x42, 0x1f, 0x50, 0xd1, 0x67, + 0x10, 0x5a, 0x83, 0x57, 0x58, 0x9b, 0x3c, 0xf4, 0x73, 0x1e, 0x75, 0xcc, 0x05, 0x52, 0x18, 0xdd, + 0x41, 0x65, 0xc8, 0xe2, 0xe3, 0x8e, 0xa2, 0x63, 0x47, 0xe3, 0x8e, 0x4b, 0x8a, 0x13, 0x48, 0xaa, + 0xd2, 0x87, 0x07, 0xa6, 0xf1, 0x71, 0x87, 0x10, 0xd1, 0x0e, 0x99, 0xbd, 0xbc, 0x50, 0x81, 0x56, + 0xdb, 0x19, 0x0d, 0x2f, 0x02, 0x4b, 0xe1, 0xe2, 0x8a, 0x7e, 0x94, 0xc0, 0x44, 0x48, 0x3f, 0x14, + 0xe0, 0xad, 0xd8, 0x5e, 0x4b, 0x72, 0x22, 0xfb, 0x04, 0x32, 0xb4, 0xf1, 0xa9, 0x73, 0x36, 0x9e, + 0x72, 0x49, 0xdf, 0x4b, 0xf1, 0x31, 0x2e, 0xe3, 0x96, 0x45, 0x14, 0x9b, 0xf8, 0x2a, 0xdb, 0x73, + 0x98, 0x3d, 0xb4, 0x5c, 0x12, 0x9b, 0xf0, 0x6e, 0x4f, 0x9d, 0xbb, 0xdb, 0x67, 0xa8, 0x00, 0xaf, + 0xc7, 0x5f, 0xc2, 0x9c, 0x69, 0x99, 0x4a, 0x54, 0xe8, 0xf9, 0x6d, 0xa9, 0x68, 0x5a, 0xe6, 0xcb, + 0x90, 0x5c, 0xdf, 0xcf, 0xf4, 0x68, 0x22, 0x49, 0x3f, 0xf3, 0x7d, 0x01, 0xe6, 0xfd, 0xb0, 0x29, + 0xe1, 0x08, 0xfa, 0x03, 0x48, 0x9b, 0xd6, 0xd1, 0x79, 0x56, 0x31, 0x49, 0x79, 0x32, 0xeb, 0x45, + 0x6b, 0x94, 0x64, 0x7b, 0xff, 0x75, 0x0a, 0x72, 0x4f, 0xcb, 0x49, 0xb6, 0xf2, 0x13, 0xbe, 0x42, + 0xce, 0xfa, 0x3b, 0xce, 0xda, 0xfd, 0xe7, 0x95, 0x9e, 0x96, 0x37, 0xf0, 0x89, 0x67, 0xed, 0x84, + 0x0b, 0xad, 0x40, 0xce, 0xdd, 0xb7, 0xb1, 0xb3, 0x6f, 0xb5, 0xf4, 0xf3, 0x84, 0x39, 0x01, 0xd7, + 0x22, 0x86, 0x49, 0x2a, 0xd7, 0xcb, 0xc6, 0x10, 0x62, 0xb2, 0x31, 0xc8, 0x63, 0xfc, 0x48, 0x31, + 0x75, 0x9e, 0xc7, 0x84, 0x42, 0xc4, 0x49, 0x71, 0x4a, 0x7a, 0x01, 0x40, 0x9a, 0x93, 0x64, 0x97, + 0xfc, 0x5a, 0x1a, 0x0a, 0xdb, 0x5d, 0x67, 0x3f, 0x61, 0xeb, 0x2b, 0x03, 0x74, 0xba, 0x0e, 0x85, + 0x20, 0xc7, 0x26, 0x6f, 0xf3, 0x88, 0x44, 0x0f, 0xaf, 0xd1, 0x8c, 0xaf, 0x71, 0x6c, 0xa2, 0x2a, + 0x17, 0x82, 0x95, 0x20, 0x5b, 0xe4, 0xc6, 0x30, 0xb0, 0xda, 0x38, 0x36, 0x37, 0xb1, 0x8f, 0x52, + 0x99, 0x24, 0x4c, 0x24, 0x7d, 0x02, 0xd3, 0xe4, 0x42, 0x71, 0xad, 0xf3, 0x74, 0xf3, 0x14, 0xe1, + 0x69, 0x58, 0xe8, 0x31, 0xe4, 0x18, 0x37, 0x99, 0xfd, 0xa6, 0xe8, 0xec, 0x17, 0xd7, 0x16, 0xae, + 0x46, 0x3a, 0xef, 0x65, 0x29, 0x2b, 0x99, 0xeb, 0x2e, 0xc0, 0xe4, 0x9e, 0x65, 0x6b, 0xde, 0xfb, + 0x5e, 0x76, 0xc1, 0xfa, 0x73, 0x3d, 0x93, 0xcd, 0x8a, 0xb9, 0xf5, 0x4c, 0x36, 0x27, 0x82, 0xf4, + 0x5b, 0x02, 0x14, 0xfd, 0x8e, 0x48, 0x72, 0x42, 0x28, 0x47, 0xb4, 0x78, 0xfe, 0xae, 0x20, 0x0a, + 0x94, 0xfe, 0x0d, 0x8d, 0x88, 0x34, 0xeb, 0x90, 0xf6, 0x4c, 0x92, 0x96, 0xf2, 0x98, 0xe5, 0x02, + 0xa5, 0xce, 0xdb, 0xbb, 0x34, 0x2d, 0xe8, 0x01, 0x5c, 0x30, 0xda, 0xc4, 0x9f, 0x1b, 0x6e, 0xeb, + 0x84, 0xc3, 0x36, 0x17, 0x7b, 0x2f, 0x96, 0xe7, 0x83, 0x7b, 0x65, 0xef, 0x96, 0xf4, 0x77, 0xe9, + 0x02, 0x78, 0xd0, 0x92, 0x24, 0x55, 0x5d, 0x83, 0x59, 0x9b, 0x89, 0x26, 0x61, 0xcd, 0x39, 0xb5, + 0x3d, 0xe3, 0xb3, 0x12, 0x85, 0xff, 0x4e, 0x0a, 0x8a, 0x2f, 0xba, 0xd8, 0x3e, 0xf9, 0x3a, 0xa9, + 0xfb, 0x36, 0x14, 0x8f, 0x54, 0xc3, 0x55, 0xf6, 0x2c, 0x5b, 0xe9, 0x76, 0x74, 0xd5, 0xf5, 0x12, + 0x52, 0x66, 0x09, 0xf9, 0x89, 0x65, 0xef, 0x50, 0x22, 0xc2, 0x80, 0x0e, 0x4c, 0xeb, 0xc8, 0x54, + 0x08, 0x99, 0x02, 0xe5, 0x63, 0x93, 0xaf, 0x4a, 0xaf, 0x7e, 0xf8, 0x1f, 0x4f, 0x97, 0x1e, 0x8d, + 0x95, 0x66, 0x46, 0x53, 0xea, 0xba, 0x5d, 0x43, 0x2f, 0xed, 0xec, 0xd4, 0xd6, 0x64, 0x91, 0x8a, + 0x7c, 0xc5, 0x24, 0x36, 0x8e, 0x4d, 0x47, 0xfa, 0x7b, 0x29, 0x10, 0x03, 0x1d, 0x25, 0xd9, 0x91, + 0x15, 0xc8, 0xbf, 0xee, 0x62, 0xdb, 0x78, 0x83, 0x6e, 0x04, 0xce, 0x48, 0xdc, 0xce, 0xe7, 0x30, + 0x13, 0xd1, 0x40, 0xfa, 0xab, 0x69, 0x20, 0x7f, 0x14, 0x34, 0x1e, 0xdd, 0x87, 0x39, 0xf7, 0xd8, + 0x54, 0x58, 0x82, 0x21, 0x4b, 0x4a, 0xf1, 0xf2, 0x25, 0x8a, 0x2e, 0xd1, 0x07, 0xa1, 0xd3, 0x84, + 0x14, 0x47, 0xfa, 0x7d, 0x01, 0x10, 0x55, 0x54, 0x8d, 0xbd, 0x36, 0xf8, 0xba, 0xd8, 0xd3, 0x5d, + 0x10, 0x69, 0xca, 0xa6, 0x62, 0xec, 0x29, 0x6d, 0xc3, 0x71, 0x0c, 0xb3, 0xc9, 0x0d, 0xaa, 0x40, + 0xe9, 0xb5, 0xbd, 0x4d, 0x46, 0x95, 0xfe, 0x0a, 0xcc, 0x47, 0x1a, 0x90, 0x64, 0x67, 0x5f, 0x87, + 0x99, 0x3d, 0xf6, 0x96, 0x96, 0x0a, 0xe7, 0x2b, 0x8e, 0x79, 0x4a, 0x63, 0xcf, 0x93, 0xfe, 0x3c, + 0x05, 0x17, 0x64, 0xec, 0x58, 0xad, 0x43, 0x9c, 0xbc, 0x0a, 0xab, 0xc0, 0x5f, 0xe7, 0x28, 0x6f, + 0xa4, 0xc9, 0x1c, 0x63, 0x66, 0xd3, 0x5c, 0x74, 0xd9, 0xfe, 0xe6, 0x70, 0x8b, 0xed, 0x5f, 0xa8, + 0xe7, 0xcb, 0x7e, 0x99, 0xc8, 0xb2, 0x9f, 0x05, 0x45, 0xa3, 0x69, 0x5a, 0xc4, 0xa7, 0x39, 0xf8, + 0xb5, 0xd9, 0x6d, 0x7b, 0x60, 0xa8, 0x34, 0xac, 0x92, 0x35, 0xc6, 0x52, 0xc7, 0xaf, 0xb7, 0xba, + 0x6d, 0x1a, 0x3b, 0xaf, 0x5e, 0x22, 0xf5, 0x3d, 0x3b, 0x5d, 0x2a, 0x44, 0xee, 0x39, 0x72, 0xc1, + 0xf0, 0xaf, 0x89, 0x74, 0xe9, 0xdb, 0x70, 0xb1, 0x47, 0xd9, 0x49, 0x46, 0x3c, 0xff, 0x32, 0x0d, + 0x57, 0xa2, 0xe2, 0x93, 0x86, 0x38, 0x5f, 0xf7, 0x0e, 0xad, 0xc2, 0x6c, 0xdb, 0x30, 0xdf, 0x6c, + 0xf5, 0x72, 0xa6, 0x6d, 0x98, 0xc1, 0x4a, 0x71, 0x8c, 0x69, 0x4c, 0xfd, 0x4c, 0x4d, 0x43, 0x85, + 0xc5, 0xb8, 0xbe, 0x4b, 0xd2, 0x3e, 0xbe, 0x27, 0xc0, 0x4c, 0xd2, 0xcb, 0x72, 0x6f, 0x96, 0x28, + 0x27, 0x35, 0x60, 0xf6, 0x67, 0xb0, 0x8e, 0xf7, 0x3b, 0x02, 0xa0, 0x86, 0xdd, 0x35, 0x09, 0xa8, + 0x7d, 0x66, 0x35, 0x93, 0x6c, 0xe6, 0x05, 0x98, 0x34, 0x4c, 0x1d, 0x1f, 0xd3, 0x66, 0x66, 0x64, + 0x76, 0x11, 0x79, 0x3b, 0x99, 0x1e, 0xeb, 0xed, 0xa4, 0xf4, 0x39, 0xcc, 0x47, 0xaa, 0x98, 0x64, + 0xfb, 0xff, 0x6b, 0x0a, 0xe6, 0x79, 0x43, 0x12, 0x5f, 0xc1, 0xfc, 0x26, 0x4c, 0xb6, 0x88, 0xcc, + 0x21, 0xfd, 0x4c, 0x9f, 0xe9, 0xf5, 0x33, 0x2d, 0x8c, 0x7e, 0x1e, 0xa0, 0x63, 0xe3, 0x43, 0x85, + 0xb1, 0xa6, 0xc7, 0x62, 0xcd, 0x11, 0x0e, 0x4a, 0x40, 0x3f, 0x10, 0xa0, 0x48, 0x06, 0x74, 0xc7, + 0xb6, 0x3a, 0x96, 0x43, 0x62, 0x16, 0x67, 0x3c, 0x98, 0xf3, 0xe2, 0xec, 0x74, 0x69, 0x76, 0xd3, + 0x30, 0xb7, 0x39, 0x63, 0xa3, 0x3e, 0xf6, 0x1e, 0x00, 0x6f, 0x27, 0x44, 0xa9, 0xdc, 0xb2, 0xb4, + 0x83, 0xe0, 0x7d, 0x1b, 0xf1, 0x2c, 0xbe, 0x38, 0x47, 0xfa, 0x23, 0x01, 0x2e, 0xfc, 0xcc, 0x96, + 0x8b, 0xff, 0x5f, 0x28, 0x5b, 0x7a, 0x09, 0x22, 0xfd, 0x51, 0x33, 0xf7, 0xac, 0x24, 0x17, 0xee, + 0x7f, 0x5d, 0x80, 0xb9, 0x90, 0xe0, 0x24, 0x03, 0x9c, 0x37, 0xd2, 0x93, 0xf4, 0x8b, 0x24, 0xe4, + 0x09, 0x8f, 0x92, 0x24, 0xc7, 0xe0, 0x3f, 0x4b, 0xc1, 0xa5, 0x32, 0x7b, 0xcd, 0xed, 0xe5, 0x7d, + 0x24, 0x69, 0x19, 0x0b, 0x30, 0x7d, 0x88, 0x6d, 0xc7, 0xb0, 0xd8, 0xf4, 0x3b, 0x2b, 0x7b, 0x97, + 0x68, 0x11, 0xb2, 0x8e, 0xa9, 0x76, 0x9c, 0x7d, 0xcb, 0x7b, 0xbf, 0xe7, 0x5f, 0xfb, 0x39, 0x2a, + 0x93, 0x6f, 0x9e, 0xa3, 0x32, 0x35, 0x3c, 0x47, 0x65, 0xfa, 0x2b, 0xe4, 0xa8, 0xf0, 0x97, 0x69, + 0xff, 0x56, 0x80, 0xcb, 0x7d, 0x9a, 0x4b, 0xd2, 0x5a, 0xbe, 0x0b, 0x79, 0x8d, 0x0b, 0x26, 0x0e, + 0x9b, 0xbd, 0x29, 0xac, 0x91, 0x62, 0x6f, 0x88, 0x5b, 0xce, 0x4e, 0x97, 0xc0, 0xab, 0x6a, 0x6d, + 0x8d, 0x2b, 0x87, 0xfc, 0xd6, 0xa5, 0x5f, 0x99, 0x85, 0x62, 0xe5, 0x98, 0xad, 0x92, 0xd7, 0x59, + 0x98, 0x80, 0x9e, 0x40, 0xb6, 0x63, 0x5b, 0x87, 0x86, 0xd7, 0x8c, 0x42, 0x24, 0x41, 0xc1, 0x6b, + 0x46, 0x0f, 0xd7, 0x36, 0xe7, 0x90, 0x7d, 0x5e, 0xd4, 0x80, 0xdc, 0x33, 0x4b, 0x53, 0x5b, 0x4f, + 0x8c, 0x96, 0x67, 0xf9, 0xef, 0x8f, 0x16, 0x54, 0xf2, 0x79, 0xb6, 0x55, 0x77, 0xdf, 0xeb, 0x04, + 0x9f, 0x88, 0x6a, 0x90, 0xad, 0xba, 0x6e, 0x87, 0xdc, 0xe4, 0xbe, 0xe3, 0xce, 0x18, 0x42, 0x09, + 0x8b, 0x97, 0x25, 0xeb, 0xb1, 0xa3, 0x06, 0xcc, 0x3d, 0xa5, 0x7b, 0xbe, 0xca, 0x2d, 0xab, 0xab, + 0x97, 0x2d, 0x73, 0xcf, 0x68, 0x72, 0xbf, 0x7d, 0x7b, 0x0c, 0x99, 0x4f, 0xcb, 0x75, 0xb9, 0x5f, + 0x00, 0x5a, 0x81, 0x6c, 0xfd, 0x11, 0x17, 0xc6, 0xe2, 0xba, 0x5b, 0x63, 0x08, 0xab, 0x3f, 0x92, + 0x7d, 0x36, 0xb4, 0x0e, 0xf9, 0x95, 0x2f, 0xba, 0x36, 0xe6, 0x52, 0xa6, 0x06, 0x66, 0x47, 0xf4, + 0x4a, 0xa1, 0x5c, 0x72, 0x98, 0x19, 0xd5, 0xa1, 0xf0, 0xca, 0xb2, 0x0f, 0x5a, 0x96, 0xea, 0xb5, + 0x70, 0x9a, 0x8a, 0xfb, 0xc6, 0x18, 0xe2, 0x3c, 0x46, 0xb9, 0x47, 0x04, 0xfa, 0x36, 0x14, 0x49, + 0x67, 0x34, 0xd4, 0xdd, 0x96, 0x57, 0xc9, 0x2c, 0x95, 0xfa, 0xee, 0x18, 0x52, 0x7d, 0x4e, 0x6f, + 0xe1, 0xbf, 0x47, 0xd4, 0xa2, 0x0c, 0xb3, 0x11, 0x23, 0x40, 0x08, 0x32, 0x1d, 0xd2, 0xdf, 0x02, + 0xcd, 0x5f, 0xa2, 0xbf, 0xd1, 0x7b, 0x30, 0x6d, 0x5a, 0x3a, 0xf6, 0x46, 0xc8, 0xec, 0xea, 0x85, + 0xb3, 0xd3, 0xa5, 0xa9, 0x2d, 0x4b, 0x67, 0x11, 0x0d, 0xff, 0x25, 0x4f, 0x91, 0x42, 0x35, 0x7d, + 0xf1, 0x1a, 0x64, 0x48, 0xbf, 0x13, 0xc7, 0xb4, 0xab, 0x3a, 0x78, 0xc7, 0x36, 0xb8, 0x34, 0xef, + 0x72, 0xf1, 0x1f, 0xa6, 0x20, 0x55, 0x7f, 0x44, 0x62, 0xf6, 0xdd, 0xae, 0x76, 0x80, 0x5d, 0x7e, + 0x9f, 0x5f, 0xd1, 0x58, 0xde, 0xc6, 0x7b, 0x06, 0x0b, 0xad, 0x72, 0x32, 0xbf, 0x42, 0xef, 0x00, + 0xa8, 0x9a, 0x86, 0x1d, 0x47, 0xf1, 0xf6, 0x02, 0xe6, 0xe4, 0x1c, 0xa3, 0x6c, 0xe0, 0x13, 0xc2, + 0xe6, 0x60, 0xcd, 0xc6, 0xae, 0x97, 0x7c, 0xc5, 0xae, 0x08, 0x9b, 0x8b, 0xdb, 0x1d, 0xc5, 0xb5, + 0x0e, 0xb0, 0x49, 0xed, 0x24, 0x47, 0x5c, 0x4d, 0xbb, 0xd3, 0x20, 0x04, 0xe2, 0x25, 0xb1, 0xa9, + 0x07, 0x2e, 0x2d, 0x27, 0xfb, 0xd7, 0x44, 0xa4, 0x8d, 0x9b, 0x06, 0xdf, 0x2e, 0x97, 0x93, 0xf9, + 0x15, 0xd1, 0x92, 0xda, 0x75, 0xf7, 0x69, 0x4f, 0xe4, 0x64, 0xfa, 0x1b, 0xdd, 0x86, 0x22, 0xcb, + 0xd7, 0x54, 0xb0, 0xa9, 0x29, 0xd4, 0xb9, 0xe6, 0xe8, 0xed, 0x59, 0x46, 0xae, 0x98, 0x1a, 0x71, + 0xa5, 0xe8, 0x11, 0x70, 0x82, 0x72, 0xd0, 0x76, 0x88, 0x4e, 0x81, 0x94, 0x5a, 0x2d, 0x9e, 0x9d, + 0x2e, 0xe5, 0xeb, 0xf4, 0xc6, 0xc6, 0x66, 0xbd, 0xb6, 0x26, 0xe7, 0x59, 0xa9, 0x8d, 0xb6, 0x53, + 0xd3, 0x17, 0x7f, 0x43, 0x80, 0xf4, 0xd3, 0x72, 0xfd, 0xdc, 0x2a, 0xf3, 0x2a, 0x9a, 0x0e, 0x55, + 0xf4, 0x0e, 0x14, 0x77, 0x8d, 0x56, 0xcb, 0x30, 0x9b, 0x24, 0x8a, 0xfa, 0x2e, 0xd6, 0x3c, 0x85, + 0x15, 0x38, 0x79, 0x9b, 0x51, 0xd1, 0x35, 0xc8, 0x6b, 0x36, 0xd6, 0xb1, 0xe9, 0x1a, 0x6a, 0xcb, + 0xe1, 0x9a, 0x0b, 0x93, 0x16, 0x7f, 0x59, 0x80, 0x49, 0x3a, 0x02, 0xd0, 0xdb, 0x90, 0xd3, 0x2c, + 0xd3, 0x55, 0x0d, 0x93, 0xbb, 0xb2, 0x9c, 0x1c, 0x10, 0x06, 0x56, 0xef, 0x3a, 0xcc, 0xa8, 0x9a, + 0x66, 0x75, 0x4d, 0x57, 0x31, 0xd5, 0x36, 0xe6, 0xd5, 0xcc, 0x73, 0xda, 0x96, 0xda, 0xc6, 0x68, + 0x09, 0xbc, 0x4b, 0x7f, 0x8b, 0x67, 0x4e, 0x06, 0x4e, 0xda, 0xc0, 0x27, 0x8b, 0x7f, 0x20, 0x40, + 0xd6, 0x1b, 0x33, 0xa4, 0x1a, 0x4d, 0x6c, 0xb2, 0xa4, 0x72, 0xaf, 0x1a, 0x3e, 0xa1, 0x77, 0xaa, + 0xcc, 0x05, 0x53, 0xe5, 0x05, 0x98, 0x74, 0xc9, 0xb0, 0xe0, 0x35, 0x60, 0x17, 0x74, 0x3d, 0xbb, + 0xa5, 0x36, 0xd9, 0x72, 0x5e, 0x4e, 0x66, 0x17, 0xa4, 0x31, 0x3c, 0xed, 0x97, 0x69, 0x84, 0x5f, + 0x91, 0x9a, 0xb2, 0xe4, 0xd4, 0x5d, 0xdc, 0x34, 0x4c, 0x6a, 0x4b, 0x69, 0x19, 0x28, 0x69, 0x95, + 0x50, 0xd0, 0x5b, 0x90, 0x63, 0x05, 0xb0, 0xa9, 0x53, 0x83, 0x4a, 0xcb, 0x59, 0x4a, 0xa8, 0x98, + 0xfa, 0x22, 0x86, 0x9c, 0x3f, 0x38, 0x49, 0xb7, 0x75, 0x1d, 0x5f, 0x91, 0xf4, 0x37, 0x7a, 0x1f, + 0x2e, 0xbc, 0xee, 0xaa, 0x2d, 0x63, 0x8f, 0xae, 0xd4, 0xd1, 0xac, 0x7b, 0xaa, 0x33, 0xd6, 0x12, + 0xe4, 0xdf, 0xa3, 0x12, 0xa8, 0xea, 0xbc, 0xb1, 0x9c, 0x0e, 0xc6, 0xb2, 0xf4, 0xbb, 0x02, 0xcc, + 0xb1, 0xcc, 0x23, 0x96, 0x30, 0x9b, 0x5c, 0x1c, 0xf2, 0x31, 0xe4, 0x74, 0xd5, 0x55, 0xd9, 0xa6, + 0xd5, 0xd4, 0xd0, 0x4d, 0xab, 0xfe, 0x26, 0x0a, 0xd5, 0x55, 0xe9, 0xc6, 0x55, 0x04, 0x19, 0xf2, + 0x9b, 0xed, 0xef, 0x95, 0xe9, 0x6f, 0xe9, 0x33, 0x40, 0xe1, 0x8a, 0x26, 0x19, 0x91, 0xdd, 0x83, + 0x8b, 0x44, 0xd7, 0x15, 0x53, 0xb3, 0x4f, 0x3a, 0xae, 0x61, 0x99, 0xcf, 0xe9, 0x5f, 0x07, 0x89, + 0xa1, 0x17, 0x5b, 0xf4, 0x7d, 0x96, 0xf4, 0x7b, 0x53, 0x30, 0x5b, 0x39, 0xee, 0x58, 0x76, 0xa2, + 0xcb, 0x60, 0xab, 0x30, 0xcd, 0x57, 0x0a, 0x86, 0xbc, 0xbb, 0xee, 0x71, 0xe6, 0xde, 0x8b, 0x7b, + 0xce, 0x88, 0x56, 0x01, 0x58, 0x52, 0x2b, 0x4d, 0x5c, 0x4a, 0x9f, 0xe3, 0x55, 0x1b, 0x65, 0xa3, + 0x1b, 0x38, 0xb6, 0x20, 0xdf, 0x3e, 0xd4, 0x34, 0x65, 0xcf, 0x68, 0xb9, 0x3c, 0x37, 0x30, 0x3e, + 0x8d, 0x7d, 0xf3, 0x65, 0xb9, 0xfc, 0x84, 0x16, 0x62, 0x69, 0x7a, 0xc1, 0xb5, 0x0c, 0x44, 0x02, + 0xfb, 0x8d, 0xde, 0x05, 0xbe, 0x99, 0x48, 0x71, 0xbc, 0x7d, 0x83, 0xab, 0xb3, 0x67, 0xa7, 0x4b, + 0x39, 0x99, 0x52, 0xeb, 0xf5, 0x86, 0x9c, 0x63, 0x05, 0xea, 0x8e, 0x8b, 0x6e, 0xc0, 0xac, 0xd5, + 0x36, 0x5c, 0xc5, 0x0b, 0x92, 0x78, 0x44, 0x39, 0x43, 0x88, 0x5e, 0x10, 0x75, 0x9e, 0x3d, 0x26, + 0xd3, 0x63, 0xef, 0x31, 0x41, 0x7f, 0x5d, 0x80, 0x4b, 0x5c, 0x91, 0xca, 0x2e, 0xcd, 0xc3, 0x57, + 0x5b, 0x86, 0x7b, 0xa2, 0x1c, 0x1c, 0x2e, 0x64, 0x69, 0xdc, 0xfa, 0x73, 0xb1, 0x1d, 0x12, 0xb2, + 0x83, 0x92, 0xd7, 0x2d, 0x27, 0xcf, 0x38, 0xf3, 0xc6, 0x61, 0xc5, 0x74, 0xed, 0x93, 0xd5, 0xcb, + 0x67, 0xa7, 0x4b, 0xf3, 0xfd, 0x77, 0x5f, 0xca, 0xf3, 0x4e, 0x3f, 0x0b, 0xaa, 0x02, 0x60, 0xdf, + 0x0e, 0xe9, 0x8c, 0x11, 0x1f, 0x7f, 0xc4, 0x1a, 0xac, 0x1c, 0xe2, 0x45, 0x77, 0x41, 0xe4, 0x7b, + 0x7a, 0xf6, 0x8c, 0x16, 0x56, 0x1c, 0xe3, 0x0b, 0x4c, 0xe7, 0x96, 0xb4, 0x5c, 0x60, 0x74, 0x22, + 0xa2, 0x6e, 0x7c, 0x81, 0x17, 0xbf, 0x0b, 0x0b, 0x83, 0x6a, 0x1f, 0x1e, 0x02, 0x39, 0xf6, 0x4a, + 0xf7, 0xa3, 0xe8, 0x7a, 0xce, 0x18, 0xa6, 0xca, 0xd7, 0x74, 0x3e, 0x4e, 0x7d, 0x24, 0x48, 0x7f, + 0x3f, 0x05, 0xb3, 0xab, 0xdd, 0xd6, 0xc1, 0xf3, 0x4e, 0x9d, 0x1d, 0x6e, 0x40, 0xdc, 0x20, 0x73, + 0x14, 0xa4, 0x82, 0x02, 0x73, 0x83, 0xd4, 0x13, 0x18, 0x5f, 0x60, 0x32, 0x39, 0x85, 0xd2, 0x65, + 0xf8, 0x3e, 0x03, 0xda, 0x86, 0x80, 0x4c, 0xb7, 0x02, 0x7c, 0x04, 0x0b, 0xa1, 0x82, 0x74, 0xf1, + 0x45, 0xc1, 0xa6, 0x6b, 0x1b, 0x98, 0x2d, 0x20, 0xa6, 0xe5, 0x50, 0x4e, 0x4f, 0x8d, 0xdc, 0xae, + 0xb0, 0xbb, 0xa8, 0x01, 0x33, 0xa4, 0xe0, 0x89, 0x42, 0xa7, 0x10, 0x6f, 0x81, 0xf7, 0x41, 0x4c, + 0xb3, 0x22, 0xf5, 0x2e, 0x51, 0xfd, 0x94, 0x29, 0x0f, 0xfd, 0x29, 0xe7, 0x71, 0x40, 0x59, 0xfc, + 0x14, 0xc4, 0xde, 0x02, 0x61, 0x5d, 0x66, 0x98, 0x2e, 0x2f, 0x84, 0x75, 0x99, 0x0e, 0xe9, 0x69, + 0x3d, 0x93, 0xcd, 0x88, 0x93, 0xd2, 0x9f, 0xa5, 0xa1, 0xe0, 0x99, 0x59, 0x92, 0x40, 0x67, 0x15, + 0x26, 0x89, 0x51, 0x78, 0x19, 0x28, 0xb7, 0x87, 0x58, 0x37, 0xcf, 0x6c, 0x27, 0xc6, 0xe2, 0x81, + 0x64, 0xca, 0x9a, 0x84, 0xc3, 0x59, 0xfc, 0xe5, 0x14, 0x64, 0x28, 0xb6, 0x78, 0x00, 0x19, 0x3a, + 0x51, 0x08, 0xe3, 0x4c, 0x14, 0xb4, 0xa8, 0x3f, 0x9d, 0xa5, 0x42, 0xa1, 0x29, 0x89, 0xf9, 0xf6, + 0xd5, 0x0f, 0x1e, 0x3c, 0xa4, 0xce, 0x66, 0x46, 0xe6, 0x57, 0x68, 0x95, 0xa6, 0x46, 0x59, 0xb6, + 0x8b, 0x75, 0x1e, 0xd3, 0x5f, 0x1b, 0xd5, 0xbf, 0xde, 0xa4, 0xe4, 0xf1, 0xa1, 0x2b, 0x90, 0x26, + 0x5e, 0x6c, 0x9a, 0x65, 0x39, 0x9c, 0x9d, 0x2e, 0xa5, 0x89, 0xff, 0x22, 0x34, 0xb4, 0x0c, 0xf9, + 0xa8, 0xcb, 0x20, 0x11, 0x1c, 0x75, 0x8c, 0xa1, 0xe1, 0x0e, 0x2d, 0x7f, 0x68, 0x31, 0x3c, 0xcb, + 0xfb, 0xf8, 0xbf, 0x67, 0x60, 0xb6, 0xd6, 0x4e, 0x7a, 0x4a, 0x59, 0x89, 0xf6, 0x70, 0x1c, 0x10, + 0x8a, 0x3c, 0x34, 0xa6, 0x83, 0x23, 0x33, 0x78, 0xfa, 0x7c, 0x33, 0x78, 0x8d, 0x44, 0xca, 0xfc, + 0xe0, 0x89, 0xf4, 0x00, 0xcc, 0x13, 0x7d, 0x3e, 0x8d, 0x53, 0x64, 0xc2, 0x13, 0xec, 0xf5, 0xa0, + 0x99, 0x2a, 0x9f, 0xd2, 0x80, 0x9c, 0x59, 0xd9, 0xd4, 0xf8, 0x56, 0x36, 0x8d, 0x4d, 0x9d, 0x4e, + 0x6a, 0x51, 0x8f, 0x3a, 0xfd, 0xe6, 0x1e, 0x75, 0xd1, 0xe5, 0xc6, 0xfa, 0x31, 0xa4, 0x75, 0xc3, + 0xeb, 0x9c, 0xf1, 0xa7, 0x6a, 0xc2, 0x34, 0xc2, 0x6a, 0x33, 0x61, 0xab, 0x65, 0x56, 0xb2, 0x58, + 0x03, 0x08, 0x74, 0x83, 0xae, 0xc1, 0x94, 0xd5, 0xd2, 0xbd, 0xcd, 0x2e, 0xb3, 0xab, 0xb9, 0xb3, + 0xd3, 0xa5, 0xc9, 0xe7, 0x2d, 0xbd, 0xb6, 0x26, 0x4f, 0x5a, 0x2d, 0xbd, 0xa6, 0xd3, 0x53, 0x3f, + 0xf0, 0x91, 0xe2, 0x67, 0xc2, 0xcd, 0xc8, 0xd3, 0x26, 0x3e, 0x5a, 0xc3, 0x8e, 0xc6, 0x0d, 0xee, + 0xb7, 0x05, 0x28, 0x78, 0xba, 0x4f, 0xd6, 0xa9, 0x64, 0x8d, 0x36, 0x1f, 0x64, 0xe9, 0xf3, 0x0d, + 0x32, 0x8f, 0x8f, 0x6f, 0x2c, 0xfe, 0x55, 0x81, 0xe7, 0x36, 0xd7, 0x35, 0xd5, 0x25, 0x41, 0x45, + 0x82, 0x03, 0xe3, 0x1e, 0x88, 0xb6, 0x6a, 0xea, 0x56, 0xdb, 0xf8, 0x02, 0xb3, 0x85, 0x50, 0x87, + 0xbf, 0xf6, 0x2c, 0xfa, 0x74, 0xba, 0xea, 0xe7, 0x48, 0x7f, 0x90, 0xe2, 0x79, 0xd0, 0x7e, 0x35, + 0x92, 0x54, 0xd7, 0x77, 0x60, 0x2e, 0x3c, 0xb5, 0xb1, 0x54, 0x4b, 0x36, 0x5a, 0xdf, 0x8b, 0x91, + 0x17, 0x57, 0x11, 0x96, 0xcf, 0xe8, 0x25, 0xd7, 0x87, 0x66, 0x44, 0x2a, 0x0c, 0x95, 0x21, 0xcf, + 0x5f, 0x3e, 0x98, 0x7b, 0x96, 0x97, 0x28, 0xf6, 0xf6, 0xa0, 0xb4, 0xc8, 0x9a, 0xb9, 0x67, 0x79, + 0x2f, 0xf2, 0x6d, 0x8f, 0xe0, 0x2c, 0xfe, 0x02, 0x4c, 0xd2, 0xdb, 0x6f, 0xe0, 0xa2, 0x79, 0x6f, + 0xfe, 0x69, 0x0a, 0x6e, 0xd2, 0xda, 0xbf, 0xc4, 0xb6, 0xb1, 0x77, 0xb2, 0x6d, 0x5b, 0x2e, 0xd6, + 0x5c, 0xac, 0x07, 0x2b, 0xe7, 0x89, 0xfa, 0xbd, 0x5c, 0xc7, 0x7b, 0xc0, 0xb9, 0x12, 0xce, 0x7c, + 0x2e, 0xb4, 0x01, 0x45, 0x9e, 0x5a, 0xa0, 0xb6, 0x8c, 0x43, 0xac, 0xa8, 0xee, 0x79, 0x66, 0xb7, + 0x59, 0xc6, 0xbb, 0x42, 0x58, 0x57, 0x5c, 0xa4, 0x43, 0x8e, 0x0b, 0x33, 0x74, 0x7e, 0x20, 0xd1, + 0xd3, 0xaf, 0xb6, 0xa0, 0x98, 0x65, 0xf9, 0x0d, 0xb5, 0x35, 0x39, 0xcb, 0x24, 0xd7, 0x74, 0xe9, + 0x3f, 0x08, 0x70, 0x6b, 0x84, 0x8a, 0x93, 0x34, 0xdd, 0x45, 0xc8, 0x1e, 0x92, 0x07, 0x19, 0x5c, + 0xc7, 0x59, 0xd9, 0xbf, 0x46, 0x9b, 0x30, 0xbb, 0xa7, 0x1a, 0xad, 0xc0, 0xa4, 0x07, 0xe7, 0x27, + 0xc6, 0x67, 0xe3, 0xce, 0x30, 0x76, 0x66, 0xc3, 0xd2, 0x6f, 0xa6, 0x60, 0x6e, 0x45, 0xd7, 0xeb, + 0x75, 0xee, 0x03, 0x93, 0xb3, 0x14, 0x0f, 0x64, 0xa6, 0x02, 0x90, 0x89, 0xde, 0x03, 0xa4, 0x1b, + 0x0e, 0x3b, 0xf8, 0xc4, 0xd9, 0x57, 0x75, 0xeb, 0x28, 0xc8, 0xcb, 0x98, 0xf3, 0xee, 0xd4, 0xbd, + 0x1b, 0xa8, 0x0e, 0x14, 0xed, 0x28, 0x8e, 0xab, 0xfa, 0xef, 0x9d, 0x6e, 0x8d, 0xb5, 0xa5, 0x8c, + 0xc1, 0x20, 0xff, 0x52, 0xce, 0x11, 0x39, 0xf4, 0x27, 0x89, 0xdb, 0x0d, 0xd2, 0x74, 0x57, 0x51, + 0x1d, 0x6f, 0xff, 0x10, 0x3b, 0x72, 0xa5, 0xc0, 0xe8, 0x2b, 0x0e, 0xdb, 0x16, 0xc4, 0x36, 0x3c, + 0x04, 0xaa, 0x49, 0x12, 0x12, 0xff, 0x1d, 0x01, 0x0a, 0x32, 0xde, 0xb3, 0xb1, 0x93, 0xe8, 0xa2, + 0xc0, 0x13, 0x98, 0xb1, 0x99, 0x54, 0x65, 0xcf, 0xb6, 0xda, 0xe7, 0x19, 0x57, 0x79, 0xce, 0xf8, + 0xc4, 0xb6, 0xda, 0xdc, 0xb1, 0xbc, 0x84, 0xa2, 0x5f, 0xc7, 0x24, 0x1b, 0xff, 0xbb, 0x74, 0xbb, + 0x34, 0x13, 0x9c, 0x74, 0x82, 0x44, 0xb2, 0x1a, 0xa0, 0x2f, 0xaa, 0xc2, 0x15, 0x4d, 0x52, 0x0d, + 0xff, 0x45, 0x80, 0x42, 0xbd, 0xbb, 0xcb, 0xce, 0xec, 0x4a, 0x4e, 0x03, 0x15, 0xc8, 0xb5, 0xf0, + 0x9e, 0xab, 0xbc, 0x51, 0xaa, 0x7e, 0x96, 0xb0, 0xd2, 0x8d, 0x0a, 0x4f, 0x01, 0x6c, 0xba, 0xc5, + 0x8e, 0xca, 0x49, 0x9f, 0x53, 0x4e, 0x8e, 0xf2, 0x12, 0xb2, 0xf4, 0x37, 0x32, 0x50, 0xf4, 0x9b, + 0x99, 0xa4, 0x97, 0x7c, 0x15, 0xf1, 0x0e, 0xe9, 0xf3, 0x78, 0x87, 0x39, 0x9e, 0x13, 0x12, 0xef, + 0x21, 0x4a, 0x30, 0x4f, 0x03, 0x17, 0x45, 0xed, 0x74, 0x5a, 0x86, 0x07, 0x77, 0xa9, 0xff, 0xc9, + 0xc8, 0x73, 0xf4, 0xd6, 0x0a, 0xbb, 0x43, 0x81, 0x2e, 0xfa, 0x35, 0x01, 0x66, 0xf6, 0x6c, 0x8c, + 0xbf, 0xc0, 0x0a, 0x85, 0x5e, 0xe3, 0x25, 0xbd, 0xac, 0x91, 0x3a, 0x7c, 0xe5, 0x97, 0xe2, 0x79, + 0xf6, 0xe0, 0x3a, 0x79, 0x2e, 0xda, 0x02, 0x51, 0x6b, 0xb1, 0xd7, 0xf4, 0x7e, 0x02, 0xce, 0x39, + 0x42, 0xfa, 0x22, 0x63, 0x0e, 0x72, 0x70, 0x5e, 0x90, 0xc1, 0xa4, 0xea, 0x0a, 0x3f, 0x27, 0x91, + 0x07, 0xf7, 0xa5, 0x01, 0xe7, 0x26, 0x84, 0x8e, 0x57, 0x2c, 0xc9, 0x58, 0xd5, 0x79, 0x0c, 0x4a, + 0xc6, 0x95, 0x7f, 0xc1, 0xc7, 0xd5, 0x2b, 0x98, 0xa3, 0x76, 0x93, 0xf4, 0xf6, 0x67, 0xe9, 0x47, + 0x29, 0x40, 0x61, 0xc9, 0x3f, 0x3b, 0x7b, 0x4b, 0x25, 0x67, 0x6f, 0xef, 0x02, 0x62, 0x99, 0x9d, + 0x8e, 0xd2, 0xc1, 0xb6, 0xe2, 0x60, 0xcd, 0xe2, 0x27, 0x58, 0x09, 0xb2, 0xc8, 0xef, 0x6c, 0x63, + 0xbb, 0x4e, 0xe9, 0x68, 0x05, 0x20, 0x88, 0x3a, 0xf9, 0xa4, 0x38, 0x4e, 0xd0, 0x99, 0xf3, 0x83, + 0x4e, 0xe9, 0xfb, 0x02, 0x14, 0x36, 0x8d, 0xa6, 0xad, 0x26, 0x7a, 0x40, 0x13, 0xfa, 0x38, 0xba, + 0xde, 0x9f, 0x7f, 0xb8, 0x18, 0x97, 0x8b, 0xc4, 0x4a, 0x78, 0x28, 0x92, 0x33, 0x90, 0xb9, 0xc6, + 0xaf, 0x51, 0x92, 0x4e, 0xf6, 0xdf, 0x2d, 0xc2, 0x0c, 0xaf, 0xf7, 0x8e, 0x69, 0x58, 0x26, 0x7a, + 0x00, 0xe9, 0x26, 0x7f, 0x9f, 0x93, 0x8f, 0x5d, 0x7b, 0x0d, 0x8e, 0x3f, 0xac, 0x4e, 0xc8, 0xa4, + 0x2c, 0x61, 0xe9, 0x74, 0xdd, 0x98, 0x68, 0x37, 0x48, 0xcf, 0x0f, 0xb3, 0x74, 0xba, 0x2e, 0xaa, + 0x43, 0x51, 0x0b, 0xce, 0x5c, 0x53, 0x08, 0x7b, 0x7a, 0x20, 0x32, 0x8e, 0x3d, 0xfd, 0xae, 0x3a, + 0x21, 0x17, 0xb4, 0xc8, 0x0d, 0x54, 0x0e, 0x1f, 0xf5, 0x95, 0xe9, 0xcb, 0xfd, 0x0b, 0x36, 0x96, + 0x47, 0x8f, 0x19, 0xab, 0x4e, 0x84, 0x4e, 0x04, 0x43, 0x1f, 0xc3, 0x94, 0x4e, 0x0f, 0x95, 0xe2, + 0x5e, 0x2a, 0xae, 0xa3, 0x23, 0x67, 0x77, 0x55, 0x27, 0x64, 0xce, 0x81, 0xd6, 0x61, 0x86, 0xfd, + 0x62, 0xb1, 0x27, 0xf7, 0x2d, 0xb7, 0x06, 0x4b, 0x08, 0xcd, 0xee, 0xd5, 0x09, 0x39, 0xaf, 0x07, + 0x54, 0xf4, 0x4d, 0xc8, 0x38, 0x9a, 0xea, 0x2d, 0x18, 0x5c, 0x1d, 0x70, 0x96, 0x4b, 0xc0, 0x4c, + 0x4b, 0xa3, 0xc7, 0xec, 0xf4, 0x51, 0xf7, 0xd8, 0x5b, 0xbb, 0x8d, 0xab, 0x7e, 0xe4, 0xcc, 0x00, + 0x52, 0x7d, 0x4c, 0x09, 0xe8, 0x29, 0xe4, 0x55, 0x12, 0xc4, 0x2b, 0x74, 0xd7, 0x2d, 0x5d, 0xac, + 0x8d, 0xcf, 0x98, 0xe8, 0xdb, 0x25, 0x5d, 0xa5, 0x07, 0x13, 0x78, 0xc4, 0x40, 0x50, 0x1b, 0xdb, + 0x4d, 0xbc, 0x90, 0x1f, 0x2e, 0x28, 0x9c, 0xf1, 0xe7, 0x0b, 0xa2, 0x44, 0x12, 0xcc, 0xfb, 0x5b, + 0xde, 0x69, 0xa3, 0x66, 0x06, 0xbe, 0xa3, 0x8f, 0xd9, 0xcc, 0x55, 0x9d, 0x90, 0x67, 0xf6, 0x43, + 0x64, 0x54, 0x82, 0x54, 0x53, 0x5b, 0x98, 0x1d, 0xe8, 0x12, 0xfc, 0x0d, 0x4b, 0xd5, 0x09, 0x39, + 0xd5, 0xd4, 0xd0, 0xa7, 0x90, 0x65, 0xbb, 0x4f, 0x8e, 0xcd, 0x85, 0xc2, 0xc0, 0x31, 0x16, 0xdd, + 0xc3, 0x53, 0x9d, 0x90, 0xe9, 0x86, 0x17, 0xf2, 0xbc, 0x6d, 0x28, 0xd8, 0x2c, 0x65, 0xd2, 0x4b, + 0x70, 0x16, 0x07, 0xe6, 0x2d, 0xc4, 0xe5, 0x38, 0x57, 0x29, 0x9c, 0x0b, 0xd1, 0xd1, 0x77, 0xe0, + 0x42, 0x54, 0x22, 0xb7, 0xb4, 0xb9, 0x81, 0xef, 0xe0, 0x07, 0xe6, 0xdb, 0x56, 0x27, 0x64, 0x64, + 0xf7, 0xdd, 0x44, 0x1f, 0xc2, 0x24, 0xeb, 0x35, 0x44, 0x45, 0xc6, 0xa5, 0xe2, 0xf4, 0x74, 0x18, + 0x2b, 0x4f, 0x8c, 0xdf, 0xe5, 0x79, 0x83, 0x4a, 0xcb, 0x6a, 0x2e, 0xcc, 0x0f, 0x34, 0xfe, 0xfe, + 0x0c, 0x48, 0x62, 0xfc, 0x6e, 0x40, 0x25, 0xfd, 0x6e, 0xb3, 0x3b, 0x3c, 0x57, 0xec, 0xc2, 0xc0, + 0x7e, 0x8f, 0x49, 0x27, 0xac, 0xd2, 0x1d, 0x1d, 0x01, 0x99, 0x54, 0xcd, 0x66, 0xc7, 0x0f, 0x29, + 0x74, 0x4c, 0x5d, 0x1c, 0x58, 0xb5, 0xfe, 0xf3, 0x99, 0xaa, 0x34, 0xf0, 0xf5, 0xa9, 0xe8, 0x25, + 0x88, 0xfc, 0x90, 0x90, 0xe0, 0x45, 0xd1, 0x25, 0x2a, 0xef, 0x5e, 0xac, 0xeb, 0x8a, 0x4b, 0xb4, + 0xaa, 0x92, 0x58, 0x22, 0x7a, 0x07, 0x7d, 0x06, 0x73, 0x54, 0x9e, 0xa2, 0x05, 0xe7, 0xba, 0x2c, + 0x2c, 0xf4, 0x9d, 0x12, 0x32, 0xf8, 0x08, 0x18, 0x4f, 0xb2, 0xa8, 0xf5, 0xdc, 0x22, 0x66, 0x6c, + 0x98, 0x86, 0x4b, 0xbd, 0xec, 0xe2, 0x40, 0x33, 0x8e, 0x9e, 0x51, 0x49, 0xcc, 0xd8, 0x60, 0x14, + 0x62, 0xc6, 0x2e, 0x4f, 0x24, 0xe4, 0xdd, 0xf1, 0xf6, 0x40, 0x33, 0x8e, 0xcb, 0x38, 0x24, 0x66, + 0xec, 0x86, 0xe9, 0xc4, 0x8c, 0x99, 0x83, 0xe8, 0x91, 0xfb, 0xce, 0x40, 0x33, 0x1e, 0xb8, 0xfb, + 0x9d, 0x98, 0xb1, 0xda, 0x77, 0x13, 0xad, 0x01, 0xb0, 0x18, 0x95, 0x46, 0x01, 0x57, 0x07, 0x4e, + 0x06, 0xbd, 0x09, 0x85, 0x64, 0x32, 0x68, 0x79, 0x34, 0xe2, 0xc8, 0x28, 0x02, 0x56, 0xe8, 0x7b, + 0xf1, 0x85, 0xa5, 0x81, 0x8e, 0xac, 0xef, 0x1d, 0x36, 0x71, 0x64, 0x47, 0x3e, 0x91, 0xcc, 0x2a, + 0x6c, 0x21, 0x7f, 0xe1, 0xda, 0x60, 0xb7, 0x1c, 0x7e, 0x9f, 0x47, 0xdd, 0x32, 0x25, 0xa0, 0x15, + 0xc8, 0x91, 0x20, 0xe7, 0x84, 0xba, 0xa1, 0xeb, 0x03, 0x81, 0x46, 0xcf, 0x96, 0xa5, 0xea, 0x84, + 0x9c, 0x7d, 0xcd, 0x49, 0xe4, 0xf1, 0x6c, 0x89, 0x73, 0x41, 0x1a, 0xf8, 0xf8, 0xc8, 0x72, 0x38, + 0x79, 0x3c, 0xe3, 0x40, 0x1a, 0x5c, 0x64, 0x7d, 0xc5, 0x37, 0xa2, 0xdb, 0x7c, 0xef, 0xf4, 0xc2, + 0x0d, 0x2a, 0x6a, 0xe0, 0x5a, 0x61, 0xec, 0xfe, 0xf8, 0xea, 0x84, 0x3c, 0xaf, 0xf6, 0xdf, 0x25, + 0x03, 0x9e, 0x4f, 0x3d, 0x6c, 0x85, 0x71, 0xe1, 0xe6, 0xc0, 0x01, 0x1f, 0xb3, 0x30, 0x4b, 0x06, + 0xbc, 0x1a, 0x22, 0xb3, 0x09, 0x48, 0x57, 0x1c, 0x87, 0x65, 0x51, 0xdc, 0x1a, 0x32, 0x01, 0xf5, + 0x2c, 0xed, 0xb0, 0x09, 0x48, 0xaf, 0x33, 0x4e, 0x22, 0x48, 0x6b, 0x61, 0xd5, 0xe6, 0x6e, 0xf6, + 0xf6, 0x40, 0x41, 0x7d, 0xc7, 0x41, 0x12, 0x41, 0x9a, 0x4f, 0x24, 0x01, 0x8f, 0xed, 0x9d, 0x2e, + 0xc4, 0x23, 0xe4, 0x3b, 0x03, 0x03, 0x9e, 0xd8, 0xe3, 0x8f, 0x48, 0xc0, 0x63, 0x47, 0x6e, 0xa0, + 0x9f, 0x87, 0x69, 0x8e, 0xc9, 0x17, 0xee, 0x0e, 0x09, 0x01, 0xc3, 0xcb, 0x28, 0x64, 0x5c, 0x73, + 0x1e, 0xe6, 0x65, 0xd9, 0x5a, 0x00, 0x6b, 0xde, 0xbd, 0x21, 0x5e, 0xb6, 0x6f, 0x39, 0x82, 0x79, + 0xd9, 0x80, 0x4c, 0xbc, 0x2c, 0xb3, 0x53, 0x3e, 0xd7, 0xdd, 0x1f, 0xe8, 0x65, 0xfb, 0x37, 0x44, + 0x11, 0x2f, 0xfb, 0x3a, 0xa0, 0x92, 0x96, 0x39, 0x0c, 0x13, 0x2f, 0x7c, 0x63, 0x60, 0xcb, 0xa2, + 0x8b, 0x03, 0xa4, 0x65, 0x9c, 0x87, 0x74, 0x1b, 0xc3, 0x00, 0x4c, 0xd3, 0xef, 0x0e, 0x3e, 0xed, + 0xa1, 0x17, 0x65, 0x55, 0xbd, 0xd5, 0x67, 0xa6, 0x61, 0xdf, 0x51, 0xd9, 0x7c, 0x2b, 0x3a, 0xd7, + 0xd4, 0x7b, 0xc3, 0x1d, 0x55, 0xdc, 0x16, 0x7e, 0xdf, 0x51, 0x45, 0x6e, 0xd2, 0xaa, 0xb2, 0xdd, + 0x87, 0x74, 0x7c, 0x97, 0x86, 0x1c, 0x4c, 0xd1, 0xb3, 0x07, 0x94, 0x56, 0xd5, 0x27, 0x06, 0x43, + 0xa8, 0xcb, 0x4e, 0x4d, 0x59, 0x58, 0x1e, 0x3e, 0x84, 0xa2, 0xe7, 0xb6, 0xf8, 0x43, 0x88, 0x93, + 0xfd, 0x39, 0xd3, 0x8b, 0x30, 0xde, 0x1f, 0x3e, 0x67, 0xf6, 0x86, 0x16, 0x6c, 0xce, 0xe4, 0x31, + 0xc5, 0x5f, 0x13, 0xe0, 0x1a, 0xab, 0x1b, 0x5d, 0xa6, 0x3d, 0x51, 0xfc, 0xc5, 0xee, 0x10, 0x10, + 0x7f, 0x40, 0x1f, 0xf0, 0xe1, 0xa0, 0xea, 0x8e, 0x58, 0xbc, 0xaf, 0x4e, 0xc8, 0xef, 0xa8, 0xc3, + 0xca, 0x11, 0x93, 0x6a, 0x33, 0x04, 0xb5, 0xf0, 0x70, 0xa0, 0x49, 0x45, 0x51, 0x1f, 0x31, 0x29, + 0xce, 0xb3, 0x3a, 0xcd, 0x5f, 0x95, 0xfb, 0x3b, 0x87, 0x8b, 0xa2, 0xb8, 0x9e, 0xc9, 0x5e, 0x16, + 0x17, 0xd6, 0x33, 0xd9, 0x2b, 0xe2, 0xe2, 0x7a, 0x26, 0xfb, 0x96, 0xf8, 0xb6, 0xf4, 0x0f, 0x16, + 0x61, 0xd6, 0x03, 0x5d, 0x0c, 0x50, 0x3d, 0x0c, 0x03, 0xaa, 0xab, 0x83, 0x00, 0x15, 0x87, 0x69, + 0x1c, 0x51, 0x3d, 0x0c, 0x23, 0xaa, 0xab, 0x83, 0x10, 0x55, 0xc0, 0x43, 0x20, 0x55, 0x63, 0x10, + 0xa4, 0xba, 0x37, 0x06, 0xa4, 0xf2, 0x45, 0xf5, 0x62, 0xaa, 0xb5, 0x7e, 0x4c, 0x75, 0x73, 0x38, + 0xa6, 0xf2, 0x45, 0x85, 0x40, 0xd5, 0xe3, 0x1e, 0x50, 0x75, 0x7d, 0x08, 0xa8, 0xf2, 0xf9, 0x3d, + 0x54, 0xb5, 0x11, 0x8b, 0xaa, 0x6e, 0x8f, 0x42, 0x55, 0xbe, 0x9c, 0x08, 0xac, 0xfa, 0x20, 0x02, + 0xab, 0x96, 0x06, 0xc2, 0x2a, 0x9f, 0x9b, 0xe1, 0xaa, 0x4f, 0x7a, 0x71, 0xd5, 0xf5, 0x21, 0xb8, + 0x2a, 0x68, 0x01, 0x07, 0x56, 0xd5, 0x38, 0x60, 0x75, 0x6b, 0x04, 0xb0, 0xf2, 0xa5, 0x84, 0x91, + 0x55, 0x35, 0x0e, 0x59, 0xdd, 0x1a, 0x81, 0xac, 0x7a, 0x24, 0x31, 0x68, 0xb5, 0x15, 0x0f, 0xad, + 0xee, 0x8c, 0x84, 0x56, 0xbe, 0xb4, 0x28, 0xb6, 0x5a, 0x0e, 0x61, 0xab, 0x77, 0x06, 0x60, 0x2b, + 0x9f, 0x95, 0x80, 0xab, 0x6f, 0xf5, 0x81, 0x2b, 0x69, 0x18, 0xb8, 0xf2, 0x79, 0x7d, 0x74, 0xf5, + 0x62, 0x00, 0xba, 0xba, 0x3b, 0x1a, 0x5d, 0xf9, 0xc2, 0x7a, 0xe0, 0x95, 0x3a, 0x14, 0x5e, 0xbd, + 0x37, 0x26, 0xbc, 0xf2, 0xa5, 0xc7, 0xe1, 0xab, 0x8f, 0xa2, 0xf8, 0xea, 0xda, 0x60, 0x7c, 0xe5, + 0x8b, 0xe1, 0x00, 0x6b, 0x23, 0x16, 0x60, 0xdd, 0x1e, 0x05, 0xb0, 0x82, 0x71, 0x10, 0x46, 0x58, + 0x5b, 0xf1, 0x08, 0xeb, 0xce, 0x48, 0x84, 0x15, 0x74, 0x7f, 0x04, 0x62, 0x6d, 0xc4, 0x42, 0xac, + 0xdb, 0xa3, 0x20, 0x56, 0x50, 0xb9, 0x30, 0xc6, 0x7a, 0x35, 0x10, 0x63, 0xdd, 0x1f, 0x07, 0x63, + 0xf9, 0x42, 0xfb, 0x40, 0xd6, 0xe7, 0x83, 0x41, 0xd6, 0x37, 0xce, 0x71, 0xce, 0x66, 0x2c, 0xca, + 0xfa, 0x56, 0x1f, 0xca, 0x92, 0x86, 0xa1, 0xac, 0xc0, 0x9e, 0x3d, 0x98, 0xa5, 0x0e, 0x05, 0x45, + 0xef, 0x8d, 0x09, 0x8a, 0x02, 0xe3, 0x8b, 0x41, 0x45, 0x95, 0x18, 0x54, 0x74, 0x73, 0x38, 0x2a, + 0x0a, 0xdc, 0x79, 0x00, 0x8b, 0xaa, 0x71, 0xb0, 0xe8, 0xd6, 0x08, 0x58, 0x14, 0x78, 0xa1, 0x10, + 0x2e, 0x7a, 0xdc, 0x83, 0x8b, 0xae, 0x8f, 0xcc, 0x04, 0x0b, 0x01, 0xa3, 0xd5, 0x7e, 0x60, 0x74, + 0x63, 0x28, 0x30, 0xf2, 0x25, 0x04, 0xc8, 0xe8, 0x71, 0x0f, 0x32, 0xba, 0x3e, 0x04, 0x19, 0x05, + 0x15, 0xe0, 0xd0, 0x48, 0x1f, 0x0e, 0x8d, 0x4a, 0xe3, 0x42, 0x23, 0x5f, 0x70, 0x2c, 0x36, 0xda, + 0x8a, 0xc7, 0x46, 0x77, 0xc6, 0x4c, 0xd2, 0xe8, 0x03, 0x47, 0xd5, 0x38, 0x70, 0x74, 0x6b, 0x04, + 0x38, 0x0a, 0xcf, 0x21, 0x3e, 0x3a, 0xaa, 0xc6, 0xa1, 0xa3, 0x5b, 0x23, 0xd0, 0x51, 0x20, 0x29, + 0x04, 0x8f, 0x1a, 0x83, 0xe0, 0xd1, 0xbd, 0x31, 0xe0, 0x51, 0x10, 0xbc, 0xf4, 0xe0, 0xa3, 0x4f, + 0x7b, 0xf1, 0x91, 0x34, 0x0c, 0x1f, 0x05, 0x23, 0xd2, 0x03, 0x48, 0x5b, 0xf1, 0x00, 0xe9, 0xce, + 0x48, 0x80, 0x14, 0x76, 0x92, 0x21, 0x84, 0xb4, 0x11, 0x8b, 0x90, 0x6e, 0x8f, 0x42, 0x48, 0x81, + 0x93, 0x0c, 0x43, 0xa4, 0x4f, 0x7b, 0x21, 0x92, 0x34, 0x0c, 0x22, 0x05, 0x8d, 0xf3, 0x30, 0x52, + 0x35, 0x0e, 0x23, 0xdd, 0x1a, 0x81, 0x91, 0x82, 0xce, 0x0b, 0x81, 0x24, 0x75, 0x28, 0x48, 0x7a, + 0x6f, 0x4c, 0x90, 0xd4, 0xe3, 0xb8, 0xa2, 0x28, 0xa9, 0x1a, 0x87, 0x92, 0x6e, 0x8d, 0x40, 0x49, + 0xa1, 0xca, 0x06, 0x30, 0x69, 0x2b, 0x1e, 0x26, 0xdd, 0x19, 0x09, 0x93, 0x7a, 0x46, 0x93, 0x87, + 0x93, 0x36, 0x62, 0x71, 0xd2, 0xed, 0x51, 0x38, 0xa9, 0x67, 0xe2, 0xe3, 0xc1, 0xc1, 0xaf, 0x8c, + 0x0f, 0x94, 0x3e, 0x3a, 0x3f, 0x50, 0xf2, 0x9f, 0x39, 0x02, 0x29, 0x7d, 0xda, 0x8b, 0x94, 0xa4, + 0x61, 0x48, 0x29, 0xb0, 0xac, 0xf3, 0x41, 0xa5, 0xf5, 0x4c, 0xf6, 0x6d, 0xf1, 0x1d, 0xe9, 0xcf, + 0xa7, 0x60, 0xaa, 0xea, 0xe7, 0x4e, 0x05, 0xad, 0x14, 0xde, 0xe4, 0xb0, 0x2e, 0xb4, 0x46, 0x46, + 0x3c, 0xf5, 0x9b, 0xa3, 0x8f, 0x78, 0xec, 0x3f, 0x84, 0x90, 0xb3, 0xbe, 0xc1, 0xae, 0x79, 0xf4, + 0x01, 0xcc, 0x76, 0x1d, 0x6c, 0x2b, 0x1d, 0xdb, 0xb0, 0x6c, 0xc3, 0x65, 0x3b, 0x83, 0x84, 0x55, + 0xf1, 0xcb, 0xd3, 0xa5, 0x99, 0x1d, 0x07, 0xdb, 0xdb, 0x9c, 0x2e, 0xcf, 0x74, 0x43, 0x57, 0xde, + 0xe7, 0xd8, 0x26, 0xc7, 0xff, 0x1c, 0xdb, 0x0b, 0x10, 0xe9, 0x3b, 0xe7, 0x70, 0x04, 0xc3, 0x0e, + 0xc6, 0x8a, 0xb7, 0x39, 0xba, 0x73, 0xcf, 0x2b, 0x49, 0x0f, 0xc8, 0x2a, 0xda, 0x51, 0x22, 0x7a, + 0x00, 0x17, 0xdb, 0xea, 0x31, 0xcd, 0xb3, 0x55, 0xbc, 0xa0, 0x90, 0xe6, 0xce, 0xb2, 0x6f, 0xa8, + 0xa1, 0xb6, 0x7a, 0x4c, 0xbf, 0xed, 0xc6, 0x6e, 0xd1, 0x0f, 0xb3, 0xdc, 0x82, 0x82, 0x6e, 0x38, + 0xae, 0x61, 0x6a, 0x2e, 0x3f, 0x73, 0x99, 0x9d, 0x57, 0x3c, 0xeb, 0x51, 0xd9, 0xc1, 0xca, 0x65, + 0x28, 0x36, 0x55, 0x17, 0x1f, 0xa9, 0x27, 0x8a, 0xb7, 0x65, 0x2f, 0x4f, 0x8f, 0x17, 0x7d, 0xeb, + 0xec, 0x74, 0x69, 0xf6, 0x29, 0xbb, 0xd5, 0xb7, 0x73, 0x6f, 0xb6, 0x19, 0xba, 0xa1, 0xa3, 0x3b, + 0x50, 0x54, 0x9d, 0x13, 0x53, 0xa3, 0x4d, 0xc6, 0xa6, 0xd3, 0x75, 0x28, 0xcc, 0xc8, 0xca, 0x05, + 0x4a, 0x2e, 0x7b, 0x54, 0x74, 0x1d, 0x66, 0xf8, 0x8e, 0x03, 0xf6, 0x01, 0xa8, 0x22, 0xad, 0x3e, + 0xff, 0xb2, 0x08, 0xfb, 0x06, 0xd4, 0x63, 0x58, 0xe4, 0xdf, 0x49, 0x38, 0x52, 0x6d, 0x5d, 0xa1, + 0x9a, 0x0c, 0x6c, 0x4e, 0xa4, 0x62, 0x2f, 0xb3, 0xef, 0x22, 0x90, 0x02, 0x44, 0x7d, 0xc1, 0xb8, + 0x68, 0xc0, 0x9c, 0xd6, 0x32, 0x7c, 0x54, 0xc0, 0x82, 0xa8, 0xb9, 0x81, 0x23, 0xa4, 0x4c, 0xcb, + 0xf6, 0xbe, 0x66, 0x2e, 0x6a, 0x51, 0x32, 0xaa, 0x03, 0x3d, 0x5c, 0x48, 0xe9, 0x58, 0x2d, 0x43, + 0x3b, 0xa1, 0xb0, 0x20, 0x7a, 0xea, 0xfb, 0xd0, 0x6f, 0x2f, 0xbc, 0x52, 0x0d, 0x77, 0x9b, 0x72, + 0xca, 0x70, 0xe4, 0xff, 0x66, 0x27, 0x36, 0xaf, 0x67, 0xb2, 0x20, 0xe6, 0xd7, 0x33, 0xd9, 0x19, + 0x71, 0x76, 0x3d, 0x93, 0x2d, 0x88, 0x45, 0xe9, 0x37, 0x04, 0x98, 0x89, 0x6c, 0xb5, 0x7a, 0xdc, + 0xf3, 0x12, 0xf9, 0x4a, 0x3c, 0xfe, 0x1b, 0x94, 0xec, 0x98, 0xe5, 0xf6, 0xe2, 0x65, 0x8e, 0x2e, + 0x0d, 0xc6, 0x0f, 0x74, 0x35, 0xc4, 0x4b, 0xa3, 0xf1, 0xd8, 0x3e, 0xce, 0xfc, 0xe0, 0x87, 0x4b, + 0x13, 0xd2, 0x5f, 0x64, 0x60, 0x36, 0xba, 0xb1, 0xaa, 0xd6, 0x53, 0xaf, 0x38, 0xff, 0x1c, 0xe1, + 0x28, 0x0d, 0xf9, 0xb8, 0x4c, 0x2e, 0xf8, 0x3e, 0x03, 0xab, 0xe6, 0xb5, 0x21, 0xaf, 0xca, 0xc3, + 0xf5, 0x0c, 0x18, 0x17, 0xff, 0x7d, 0xda, 0xf7, 0x53, 0x25, 0x98, 0xa4, 0x67, 0x24, 0xf1, 0xaa, + 0x2d, 0xf4, 0xf6, 0x15, 0x89, 0x59, 0xc9, 0x7d, 0x99, 0x15, 0x23, 0x7e, 0xad, 0xf1, 0x46, 0x87, + 0x10, 0x06, 0xa6, 0x77, 0xfe, 0xcf, 0x36, 0x76, 0xd9, 0x21, 0x94, 0xff, 0x17, 0x53, 0x6d, 0xc8, + 0xf3, 0xd0, 0x2f, 0x41, 0x51, 0xb3, 0x5a, 0x2d, 0x36, 0x67, 0x31, 0xcf, 0xd0, 0x7f, 0x2c, 0x0d, + 0xad, 0x02, 0xff, 0x52, 0x67, 0xc9, 0xff, 0x62, 0x67, 0x49, 0xe6, 0x5f, 0xec, 0x0c, 0xe5, 0xf1, + 0x16, 0x7c, 0x61, 0x9e, 0x43, 0x89, 0xa4, 0x14, 0x4f, 0xbf, 0x49, 0x4a, 0x31, 0x4b, 0x43, 0xe7, + 0x96, 0xf7, 0xc7, 0x02, 0x4f, 0x88, 0x79, 0x66, 0x59, 0x07, 0x5d, 0x3f, 0x15, 0x78, 0x31, 0x7c, + 0xa4, 0x64, 0xf6, 0xcb, 0xd3, 0xa5, 0x8c, 0xec, 0x9f, 0x29, 0x19, 0xe7, 0x81, 0x53, 0x5f, 0xcd, + 0x03, 0x5f, 0x87, 0x99, 0x8e, 0x8d, 0xf7, 0xb0, 0xab, 0xed, 0x2b, 0x66, 0xb7, 0xcd, 0x77, 0x0c, + 0xe5, 0x3d, 0xda, 0x56, 0xb7, 0x8d, 0xee, 0x81, 0xe8, 0x17, 0xe1, 0x58, 0xd9, 0x3b, 0x73, 0xcc, + 0xa3, 0x73, 0x64, 0x2d, 0xfd, 0x4f, 0x01, 0xe6, 0x23, 0x6d, 0xe2, 0x63, 0x6a, 0x1d, 0xf2, 0xba, + 0x3f, 0xe7, 0x39, 0x0b, 0xc2, 0x39, 0x13, 0x63, 0xc3, 0xcc, 0x48, 0x81, 0x4b, 0xde, 0x63, 0xe9, + 0x37, 0x0d, 0x02, 0xb1, 0xa9, 0x73, 0x8a, 0xbd, 0x18, 0xc8, 0x59, 0x0b, 0x3d, 0xc0, 0x1f, 0x64, + 0xe9, 0xb1, 0x06, 0x99, 0xf4, 0xdb, 0x02, 0x88, 0xf4, 0x01, 0x4f, 0x30, 0xd6, 0x13, 0xf1, 0x6e, + 0x5e, 0xc2, 0x79, 0x6a, 0xfc, 0x3d, 0x41, 0x91, 0xef, 0xb0, 0xa4, 0xa3, 0xdf, 0x61, 0x91, 0x7e, + 0x28, 0x40, 0xc1, 0xaf, 0x21, 0xfb, 0xf6, 0xe1, 0x90, 0x93, 0x4b, 0xdf, 0xec, 0xfb, 0x7e, 0xde, + 0x09, 0x2b, 0x63, 0x7d, 0x8e, 0x31, 0x7c, 0xc2, 0x0a, 0xfb, 0x2e, 0xdd, 0xdf, 0xf2, 0x2c, 0x87, + 0x54, 0xb1, 0x1c, 0x9c, 0xa4, 0xf1, 0x06, 0xdb, 0xa3, 0x64, 0xfa, 0xd9, 0x58, 0xab, 0x75, 0xc8, + 0x0e, 0xc5, 0x19, 0xcb, 0xed, 0x21, 0x9e, 0x06, 0x06, 0x7c, 0x55, 0x4d, 0x6f, 0xd4, 0xe9, 0x07, + 0x65, 0xd9, 0x6f, 0x47, 0x7a, 0x12, 0x52, 0x20, 0xed, 0x7c, 0xa2, 0xa5, 0xb1, 0x5c, 0xb1, 0xa7, + 0x25, 0x66, 0x2b, 0x7f, 0x18, 0xee, 0x89, 0xca, 0x21, 0x41, 0x53, 0x8f, 0x20, 0x7d, 0xa8, 0xb6, + 0x86, 0x65, 0x52, 0x45, 0x7a, 0x4e, 0x26, 0xa5, 0xd1, 0x93, 0xc8, 0x01, 0x24, 0xa9, 0xc1, 0x91, + 0x7f, 0xbf, 0x4a, 0x23, 0x07, 0x95, 0x7c, 0x18, 0xb5, 0xf5, 0xa1, 0x8f, 0x0f, 0x1b, 0xfd, 0xc7, + 0x99, 0x1f, 0xff, 0x70, 0x49, 0x90, 0x3e, 0x01, 0x24, 0x63, 0x07, 0xbb, 0x2f, 0xba, 0x96, 0x1d, + 0x1c, 0xe6, 0x72, 0xbb, 0xe7, 0xc3, 0x34, 0x93, 0xab, 0xf9, 0xb3, 0xb8, 0xf3, 0x9e, 0x2e, 0xc2, + 0x7c, 0x84, 0x9b, 0x39, 0x0b, 0xe9, 0x43, 0xb8, 0xf2, 0xd4, 0x72, 0x1c, 0xa3, 0x43, 0x20, 0x24, + 0x1d, 0x95, 0x64, 0x6a, 0xf1, 0xdd, 0x63, 0xb6, 0x43, 0x17, 0x13, 0x4c, 0xe6, 0x46, 0x72, 0xb2, + 0x7f, 0x2d, 0xfd, 0x9e, 0x00, 0x97, 0xfb, 0x39, 0x99, 0x96, 0xe3, 0x76, 0x73, 0x4e, 0x6b, 0x56, + 0x70, 0xf8, 0xdf, 0x68, 0x6b, 0xf5, 0x8a, 0x93, 0xb0, 0x90, 0x3f, 0x53, 0x69, 0xab, 0xd4, 0x7d, + 0xf0, 0x9d, 0xe5, 0x05, 0x4e, 0xde, 0x64, 0xd4, 0xc0, 0x93, 0x64, 0xc6, 0xf3, 0x24, 0x0d, 0x28, + 0xae, 0x5b, 0x86, 0x49, 0xa2, 0x4f, 0xaf, 0xbd, 0x2b, 0x50, 0xd8, 0x35, 0x4c, 0xd5, 0x3e, 0x51, + 0xbc, 0x04, 0x3e, 0x61, 0x54, 0x02, 0x9f, 0x3c, 0xcb, 0x38, 0xf8, 0xa5, 0xf4, 0x13, 0x01, 0xc4, + 0x40, 0x2c, 0xf7, 0xc8, 0xef, 0x02, 0x68, 0xad, 0xae, 0xe3, 0x62, 0xdb, 0xeb, 0xa5, 0x19, 0x96, + 0x99, 0x5f, 0x66, 0xd4, 0xda, 0x9a, 0x9c, 0xe3, 0x05, 0x6a, 0x3a, 0xba, 0x11, 0x3d, 0xf8, 0x62, + 0x72, 0x15, 0xce, 0xfa, 0x8e, 0xbb, 0x20, 0xdd, 0xee, 0xb8, 0x96, 0xed, 0x63, 0x17, 0xde, 0xed, + 0xde, 0x51, 0xfe, 0x74, 0xbf, 0x36, 0x29, 0xb7, 0x02, 0x05, 0x12, 0x2e, 0x1c, 0x62, 0xbf, 0x49, + 0x99, 0xd1, 0x4d, 0x62, 0x1c, 0x5e, 0x93, 0xfe, 0x85, 0x00, 0xc5, 0x32, 0xeb, 0x0d, 0xbf, 0x87, + 0x87, 0x78, 0xb4, 0x35, 0xc8, 0xba, 0xc7, 0xa6, 0xd2, 0xc6, 0xfe, 0xe7, 0x6f, 0xce, 0x71, 0x32, + 0xdf, 0xb4, 0xcb, 0x2e, 0xe9, 0x17, 0x12, 0xf9, 0xe7, 0xb9, 0xf9, 0x70, 0xb9, 0x52, 0x62, 0xdf, + 0xef, 0x2e, 0x79, 0xdf, 0xef, 0x2e, 0xad, 0xf1, 0x02, 0xcc, 0xa9, 0xff, 0xe0, 0x3f, 0x2f, 0x09, + 0xb2, 0xcf, 0xc4, 0xe6, 0xfd, 0xfb, 0x75, 0x62, 0xf5, 0x7d, 0x33, 0x33, 0x2a, 0x00, 0x84, 0xbe, + 0x6b, 0xc4, 0xbf, 0x08, 0xbd, 0xb2, 0xa6, 0xec, 0x6c, 0x95, 0x9f, 0x6f, 0x6e, 0xd6, 0x1a, 0x8d, + 0xca, 0x9a, 0x28, 0x20, 0x11, 0x66, 0x22, 0x5f, 0x45, 0x4a, 0xb1, 0x6f, 0x44, 0xdf, 0xff, 0xff, + 0x00, 0x82, 0x0f, 0xac, 0x11, 0x59, 0x1b, 0x95, 0xcf, 0x94, 0x97, 0x2b, 0xcf, 0x76, 0x2a, 0x75, + 0x71, 0x02, 0x21, 0x28, 0xac, 0xae, 0x34, 0xca, 0x55, 0x45, 0xae, 0xd4, 0xb7, 0x9f, 0x6f, 0xd5, + 0x2b, 0xde, 0xb7, 0xa5, 0xef, 0xaf, 0xc1, 0x4c, 0xf8, 0x78, 0x23, 0x34, 0x0f, 0xc5, 0x72, 0xb5, + 0x52, 0xde, 0x50, 0x5e, 0xd6, 0x56, 0x94, 0x17, 0x3b, 0x95, 0x9d, 0x8a, 0x38, 0x41, 0xab, 0x46, + 0x89, 0x4f, 0x76, 0x9e, 0x3d, 0x13, 0x05, 0x54, 0x84, 0x3c, 0xbb, 0xa6, 0x5f, 0x50, 0x12, 0x53, + 0xf7, 0x37, 0x21, 0x1f, 0x3a, 0x07, 0x99, 0x3c, 0x6e, 0x7b, 0xa7, 0x5e, 0x55, 0x1a, 0xb5, 0xcd, + 0x4a, 0xbd, 0xb1, 0xb2, 0xb9, 0xcd, 0x64, 0x50, 0xda, 0xca, 0xea, 0x73, 0xb9, 0x21, 0x0a, 0xfe, + 0x75, 0xe3, 0xf9, 0x4e, 0xb9, 0xea, 0x35, 0x43, 0xca, 0x64, 0xd3, 0x62, 0xfa, 0xfe, 0x5f, 0x15, + 0xe0, 0xf2, 0x80, 0xa3, 0x7e, 0x50, 0x1e, 0xa6, 0x77, 0x4c, 0x7a, 0x28, 0xac, 0x38, 0x81, 0x66, + 0x43, 0xa7, 0xfd, 0x88, 0x02, 0xca, 0xb2, 0xf3, 0x56, 0xc4, 0x14, 0x9a, 0x82, 0x54, 0xfd, 0x91, + 0x98, 0x26, 0x35, 0x0d, 0x1d, 0x96, 0x23, 0x66, 0x50, 0x8e, 0x1f, 0xd3, 0x21, 0x4e, 0xa2, 0x99, + 0xe0, 0xb4, 0x0c, 0x71, 0x8a, 0x88, 0xf2, 0x4f, 0x9d, 0x10, 0xa7, 0xef, 0x5f, 0x87, 0xd0, 0xce, + 0x7e, 0x04, 0x30, 0xf5, 0x4c, 0x75, 0xb1, 0xe3, 0x8a, 0x13, 0x68, 0x1a, 0xd2, 0x2b, 0xad, 0x96, + 0x28, 0x3c, 0xfc, 0xe7, 0x19, 0xc8, 0x7a, 0x5f, 0x0a, 0x42, 0xcf, 0x60, 0x92, 0x2d, 0x2b, 0x2f, + 0x0d, 0x46, 0x0b, 0x74, 0x40, 0x2f, 0x5e, 0x1b, 0x05, 0x27, 0xa4, 0x09, 0xf4, 0xff, 0x43, 0x3e, + 0x14, 0x45, 0xa1, 0x81, 0x4b, 0x63, 0x91, 0xc8, 0x71, 0xf1, 0xf6, 0xa8, 0x62, 0xbe, 0xfc, 0x57, + 0x90, 0xf3, 0xbd, 0x3a, 0xba, 0x31, 0xcc, 0xe7, 0x7b, 0xb2, 0x87, 0x4f, 0x0c, 0x64, 0xfc, 0x49, + 0x13, 0xef, 0x0b, 0xc8, 0x06, 0xd4, 0xef, 0x80, 0x51, 0x5c, 0xb2, 0xc2, 0x40, 0x0f, 0xbf, 0x78, + 0x7f, 0xac, 0xd2, 0xc1, 0x33, 0x89, 0xb2, 0x82, 0x59, 0x24, 0x5e, 0x59, 0x7d, 0x73, 0x54, 0xbc, + 0xb2, 0x62, 0x26, 0xa3, 0x09, 0xf4, 0x02, 0x32, 0xc4, 0x7b, 0xa2, 0xb8, 0xb8, 0xb2, 0xc7, 0x5b, + 0x2f, 0xde, 0x18, 0x5a, 0xc6, 0x13, 0xb9, 0x7a, 0xef, 0xc7, 0x7f, 0x76, 0x75, 0xe2, 0xc7, 0x67, + 0x57, 0x85, 0x9f, 0x9c, 0x5d, 0x15, 0xfe, 0xe4, 0xec, 0xaa, 0xf0, 0xa7, 0x67, 0x57, 0x85, 0xef, + 0xff, 0xf4, 0xea, 0xc4, 0x4f, 0x7e, 0x7a, 0x75, 0xe2, 0x4f, 0x7e, 0x7a, 0x75, 0xe2, 0xf3, 0x69, + 0xce, 0xbd, 0x3b, 0x45, 0x5d, 0xcb, 0xa3, 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0x1d, 0x1b, 0xee, + 0xd1, 0x81, 0x81, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 3a9f4dfd4177..bc3ec522eed3 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -13,6 +13,7 @@ package cockroach.roachpb; option go_package = "roachpb"; import "kv/kvserver/concurrency/lock/locking.proto"; +import "kv/kvserver/readsummary/rspb/summary.proto"; import "roachpb/data.proto"; import "roachpb/errors.proto"; import "roachpb/metadata.proto"; @@ -1662,7 +1663,8 @@ message SubsumeResponse { // timestamps at which any requests were serviced by the responding replica // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water - // mark for the keys previously owned by the subsumed range. + // mark for the keys previously owned by the subsumed range though this role + // is largely being... subsumed by the RightReadSummary. util.hlc.Timestamp freeze_start = 5 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; @@ -1680,6 +1682,21 @@ message SubsumeResponse { // the two sides are collocated at merge time, we don't need to use the // read_summary and simply use this field. util.hlc.Timestamp closed_timestamp = 6 [(gogoproto.nullable) = false]; + + // ReadSummary is a summary of the reads that have been performed on the range + // up to the point of the Subsume request, which serializes with past reads + // and begins blocking future reads. It is suitable for use to update the + // timestamp cache for the keys previously owned by the subsumed range. + // + // ReadSummary can be used in place of FreezeStart, when available. It has two + // distinct advantages: + // 1. it can transfer a higher-resolution snapshot of the reads on the range + // through a range merge, to make the merge less disruptive to writes on + // the post-merge range because the timestamp cache won't be bumped as + // high. + // 2. it can transfer information about reads with synthetic timestamps, which + // are not otherwise captured by the FreezeStart clock timestamp. + kv.kvserver.readsummary.ReadSummary read_summary = 7; } // RangeStatsRequest is the argument to the RangeStats() method. It requests the diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 0e0fb4a47579..2309cba13ff6 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -7,6 +7,7 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" import lock "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" +import rspb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb" import enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -98,7 +99,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{0} + return fileDescriptor_data_73b4def784a0a137, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -128,7 +129,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{1} + return fileDescriptor_data_73b4def784a0a137, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -180,7 +181,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{2} + return fileDescriptor_data_73b4def784a0a137, []int{2} } // RangeClosedTimestampPolicy represents the policy used by the leaseholder of a @@ -216,7 +217,7 @@ func (x RangeClosedTimestampPolicy) String() string { return proto.EnumName(RangeClosedTimestampPolicy_name, int32(x)) } func (RangeClosedTimestampPolicy) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{3} + return fileDescriptor_data_73b4def784a0a137, []int{3} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -233,7 +234,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{0} + return fileDescriptor_data_73b4def784a0a137, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -285,7 +286,7 @@ func (m *Value) Reset() { *m = Value{} } func (m *Value) String() string { return proto.CompactTextString(m) } func (*Value) ProtoMessage() {} func (*Value) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{1} + return fileDescriptor_data_73b4def784a0a137, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -321,7 +322,7 @@ func (m *KeyValue) Reset() { *m = KeyValue{} } func (m *KeyValue) String() string { return proto.CompactTextString(m) } func (*KeyValue) ProtoMessage() {} func (*KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{2} + return fileDescriptor_data_73b4def784a0a137, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -359,7 +360,7 @@ func (m *StoreIdent) Reset() { *m = StoreIdent{} } func (m *StoreIdent) String() string { return proto.CompactTextString(m) } func (*StoreIdent) ProtoMessage() {} func (*StoreIdent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{3} + return fileDescriptor_data_73b4def784a0a137, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -399,7 +400,7 @@ func (m *SplitTrigger) Reset() { *m = SplitTrigger{} } func (m *SplitTrigger) String() string { return proto.CompactTextString(m) } func (*SplitTrigger) ProtoMessage() {} func (*SplitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{4} + return fileDescriptor_data_73b4def784a0a137, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -439,7 +440,8 @@ type MergeTrigger struct { // timestamps at which any requests were serviced by the right-hand side range // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water - // mark for the keys previously owned by the subsumed range. + // mark for the keys previously owned by the subsumed range, though this role + // is largely being... subsumed by the RightReadSummary. FreezeStart github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"freeze_start"` // right_closed_timestamp is the closed timestamp of the RHS at the moment of // the subsumption. Because the SubsumeRequest synchronizes with all other @@ -455,13 +457,41 @@ type MergeTrigger struct { // the two sides are collocated at merge time, we don't need to use the // read_summary and simply use this field. RightClosedTimestamp hlc.Timestamp `protobuf:"bytes,6,opt,name=right_closed_timestamp,json=rightClosedTimestamp,proto3" json:"right_closed_timestamp"` + // RightReadSummary is a summary of the reads that have been performed on the + // right-hand side up to the point of the Subsume request, which serializes + // with past reads and begins blocking future reads. It is suitable for use to + // update the timestamp cache for the keys previously owned by the subsumed + // range. + // + // RightReadSummary can be used in place of FreezeStart, when available. It + // has two distinct advantages: + // 1. it can transfer a higher-resolution snapshot of the reads on the range + // through a range merge, to make the merge less disruptive to writes on + // the post-merge range because the timestamp cache won't be bumped as + // high. + // 2. it can transfer information about reads with synthetic timestamps, which + // are not otherwise captured by the FreezeStart clock timestamp. + // + // When a RightReadSummary is set in ReplicatedEvalResult.Merge.Trigger, there + // is always also a write to the RangePriorReadSummaryKey in the corresponding + // RaftCommand.WriteBatch. The persisted summary may be identical to the + // summary in this field, but it does not have to be. Notably, we intended for + // the summary included in the ReplicatedEvalResult.Merge.Trigger to + // eventually be a much higher-resolution version of the ReadSummmary than the + // version persisted. This scheme of persisting a compressed ReadSummary + // indefinitely and including a higher-resolution ReadSummary on the + // RaftCommand allows us to optimize for the common case where the merge is + // applied on the LHS's leaseholder through Raft log application while + // ensuring correctness in the case where the merge is applied on the LHS's + // leaseholder through a Raft snapshot. + RightReadSummary *rspb.ReadSummary `protobuf:"bytes,7,opt,name=right_read_summary,json=rightReadSummary,proto3" json:"right_read_summary,omitempty"` } func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } func (*MergeTrigger) ProtoMessage() {} func (*MergeTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{5} + return fileDescriptor_data_73b4def784a0a137, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -531,7 +561,7 @@ type ChangeReplicasTrigger struct { func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{6} + return fileDescriptor_data_73b4def784a0a137, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -573,7 +603,7 @@ func (m *ModifiedSpanTrigger) Reset() { *m = ModifiedSpanTrigger{} } func (m *ModifiedSpanTrigger) String() string { return proto.CompactTextString(m) } func (*ModifiedSpanTrigger) ProtoMessage() {} func (*ModifiedSpanTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{7} + return fileDescriptor_data_73b4def784a0a137, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -619,7 +649,7 @@ func (m *StickyBitTrigger) Reset() { *m = StickyBitTrigger{} } func (m *StickyBitTrigger) String() string { return proto.CompactTextString(m) } func (*StickyBitTrigger) ProtoMessage() {} func (*StickyBitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{8} + return fileDescriptor_data_73b4def784a0a137, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -658,7 +688,7 @@ func (m *InternalCommitTrigger) Reset() { *m = InternalCommitTrigger{} } func (m *InternalCommitTrigger) String() string { return proto.CompactTextString(m) } func (*InternalCommitTrigger) ProtoMessage() {} func (*InternalCommitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{9} + return fileDescriptor_data_73b4def784a0a137, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -727,7 +757,7 @@ func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } func (*ObservedTimestamp) ProtoMessage() {} func (*ObservedTimestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{10} + return fileDescriptor_data_73b4def784a0a137, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -902,7 +932,7 @@ type Transaction struct { func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{11} + return fileDescriptor_data_73b4def784a0a137, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -953,7 +983,7 @@ func (m *TransactionRecord) Reset() { *m = TransactionRecord{} } func (m *TransactionRecord) String() string { return proto.CompactTextString(m) } func (*TransactionRecord) ProtoMessage() {} func (*TransactionRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{12} + return fileDescriptor_data_73b4def784a0a137, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -993,7 +1023,7 @@ func (m *Intent) Reset() { *m = Intent{} } func (m *Intent) String() string { return proto.CompactTextString(m) } func (*Intent) ProtoMessage() {} func (*Intent) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{13} + return fileDescriptor_data_73b4def784a0a137, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1030,7 +1060,7 @@ func (m *Intent_SingleKeySpan) Reset() { *m = Intent_SingleKeySpan{} } func (m *Intent_SingleKeySpan) String() string { return proto.CompactTextString(m) } func (*Intent_SingleKeySpan) ProtoMessage() {} func (*Intent_SingleKeySpan) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{13, 0} + return fileDescriptor_data_73b4def784a0a137, []int{13, 0} } func (m *Intent_SingleKeySpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1067,7 +1097,7 @@ func (m *LockAcquisition) Reset() { *m = LockAcquisition{} } func (m *LockAcquisition) String() string { return proto.CompactTextString(m) } func (*LockAcquisition) ProtoMessage() {} func (*LockAcquisition) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{14} + return fileDescriptor_data_73b4def784a0a137, []int{14} } func (m *LockAcquisition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1108,7 +1138,7 @@ func (m *LockUpdate) Reset() { *m = LockUpdate{} } func (m *LockUpdate) String() string { return proto.CompactTextString(m) } func (*LockUpdate) ProtoMessage() {} func (*LockUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{15} + return fileDescriptor_data_73b4def784a0a137, []int{15} } func (m *LockUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1145,7 +1175,7 @@ func (m *SequencedWrite) Reset() { *m = SequencedWrite{} } func (m *SequencedWrite) String() string { return proto.CompactTextString(m) } func (*SequencedWrite) ProtoMessage() {} func (*SequencedWrite) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{16} + return fileDescriptor_data_73b4def784a0a137, []int{16} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1210,7 +1240,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{17} + return fileDescriptor_data_73b4def784a0a137, []int{17} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1255,7 +1285,7 @@ func (m *AbortSpanEntry) Reset() { *m = AbortSpanEntry{} } func (m *AbortSpanEntry) String() string { return proto.CompactTextString(m) } func (*AbortSpanEntry) ProtoMessage() {} func (*AbortSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{18} + return fileDescriptor_data_73b4def784a0a137, []int{18} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1312,7 +1342,7 @@ func (m *LeafTxnInputState) Reset() { *m = LeafTxnInputState{} } func (m *LeafTxnInputState) String() string { return proto.CompactTextString(m) } func (*LeafTxnInputState) ProtoMessage() {} func (*LeafTxnInputState) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{19} + return fileDescriptor_data_73b4def784a0a137, []int{19} } func (m *LeafTxnInputState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1365,7 +1395,7 @@ func (m *LeafTxnFinalState) Reset() { *m = LeafTxnFinalState{} } func (m *LeafTxnFinalState) String() string { return proto.CompactTextString(m) } func (*LeafTxnFinalState) ProtoMessage() {} func (*LeafTxnFinalState) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{20} + return fileDescriptor_data_73b4def784a0a137, []int{20} } func (m *LeafTxnFinalState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1404,7 +1434,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_data_91736d498bf42716, []int{21} + return fileDescriptor_data_73b4def784a0a137, []int{21} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1441,7 +1471,7 @@ type RangeInfo struct { func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_91736d498bf42716, []int{22} + return fileDescriptor_data_73b4def784a0a137, []int{22} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1557,6 +1587,9 @@ func (this *MergeTrigger) Equal(that interface{}) bool { if !this.RightClosedTimestamp.Equal(&that1.RightClosedTimestamp) { return false } + if !this.RightReadSummary.Equal(that1.RightReadSummary) { + return false + } return true } func (this *AbortSpanEntry) Equal(that interface{}) bool { @@ -1808,6 +1841,16 @@ func (m *MergeTrigger) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n10 + if m.RightReadSummary != nil { + dAtA[i] = 0x3a + i++ + i = encodeVarintData(dAtA, i, uint64(m.RightReadSummary.Size())) + n11, err := m.RightReadSummary.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n11 + } return i, nil } @@ -1834,11 +1877,11 @@ func (m *ChangeReplicasTrigger) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.DeprecatedReplica.Size())) - n11, err := m.DeprecatedReplica.MarshalTo(dAtA[i:]) + n12, err := m.DeprecatedReplica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n12 if len(m.DeprecatedUpdatedReplicas) > 0 { for _, msg := range m.DeprecatedUpdatedReplicas { dAtA[i] = 0x1a @@ -1860,11 +1903,11 @@ func (m *ChangeReplicasTrigger) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintData(dAtA, i, uint64(m.Desc.Size())) - n12, err := m.Desc.MarshalTo(dAtA[i:]) + n13, err := m.Desc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n13 } if len(m.InternalAddedReplicas) > 0 { for _, msg := range m.InternalAddedReplicas { @@ -1922,11 +1965,11 @@ func (m *ModifiedSpanTrigger) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.NodeLivenessSpan.Size())) - n13, err := m.NodeLivenessSpan.MarshalTo(dAtA[i:]) + n14, err := m.NodeLivenessSpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n14 } return i, nil } @@ -1949,11 +1992,11 @@ func (m *StickyBitTrigger) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.StickyBit.Size())) - n14, err := m.StickyBit.MarshalTo(dAtA[i:]) + n15, err := m.StickyBit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n15 return i, nil } @@ -1976,51 +2019,51 @@ func (m *InternalCommitTrigger) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.SplitTrigger.Size())) - n15, err := m.SplitTrigger.MarshalTo(dAtA[i:]) + n16, err := m.SplitTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n15 + i += n16 } if m.MergeTrigger != nil { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.MergeTrigger.Size())) - n16, err := m.MergeTrigger.MarshalTo(dAtA[i:]) + n17, err := m.MergeTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n16 + i += n17 } if m.ChangeReplicasTrigger != nil { dAtA[i] = 0x1a i++ i = encodeVarintData(dAtA, i, uint64(m.ChangeReplicasTrigger.Size())) - n17, err := m.ChangeReplicasTrigger.MarshalTo(dAtA[i:]) + n18, err := m.ChangeReplicasTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n17 + i += n18 } if m.ModifiedSpanTrigger != nil { dAtA[i] = 0x22 i++ i = encodeVarintData(dAtA, i, uint64(m.ModifiedSpanTrigger.Size())) - n18, err := m.ModifiedSpanTrigger.MarshalTo(dAtA[i:]) + n19, err := m.ModifiedSpanTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n19 } if m.StickyBitTrigger != nil { dAtA[i] = 0x2a i++ i = encodeVarintData(dAtA, i, uint64(m.StickyBitTrigger.Size())) - n19, err := m.StickyBitTrigger.MarshalTo(dAtA[i:]) + n20, err := m.StickyBitTrigger.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n19 + i += n20 } return i, nil } @@ -2048,11 +2091,11 @@ func (m *ObservedTimestamp) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Timestamp.Size())) - n20, err := m.Timestamp.MarshalTo(dAtA[i:]) + n21, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n20 + i += n21 return i, nil } @@ -2074,11 +2117,11 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.TxnMeta.Size())) - n21, err := m.TxnMeta.MarshalTo(dAtA[i:]) + n22, err := m.TxnMeta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n22 if len(m.Name) > 0 { dAtA[i] = 0x12 i++ @@ -2093,19 +2136,19 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintData(dAtA, i, uint64(m.LastHeartbeat.Size())) - n22, err := m.LastHeartbeat.MarshalTo(dAtA[i:]) + n23, err := m.LastHeartbeat.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n23 dAtA[i] = 0x3a i++ i = encodeVarintData(dAtA, i, uint64(m.GlobalUncertaintyLimit.Size())) - n23, err := m.GlobalUncertaintyLimit.MarshalTo(dAtA[i:]) + n24, err := m.GlobalUncertaintyLimit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n24 if len(m.ObservedTimestamps) > 0 { for _, msg := range m.ObservedTimestamps { dAtA[i] = 0x42 @@ -2143,11 +2186,11 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintData(dAtA, i, uint64(m.ReadTimestamp.Size())) - n24, err := m.ReadTimestamp.MarshalTo(dAtA[i:]) + n25, err := m.ReadTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n25 if m.CommitTimestampFixed { dAtA[i] = 0x80 i++ @@ -2209,11 +2252,11 @@ func (m *TransactionRecord) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.TxnMeta.Size())) - n25, err := m.TxnMeta.MarshalTo(dAtA[i:]) + n26, err := m.TxnMeta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n26 if m.Status != 0 { dAtA[i] = 0x20 i++ @@ -2222,11 +2265,11 @@ func (m *TransactionRecord) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintData(dAtA, i, uint64(m.LastHeartbeat.Size())) - n26, err := m.LastHeartbeat.MarshalTo(dAtA[i:]) + n27, err := m.LastHeartbeat.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n27 if len(m.LockSpans) > 0 { for _, msg := range m.LockSpans { dAtA[i] = 0x5a @@ -2288,19 +2331,19 @@ func (m *Intent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Intent_SingleKeySpan.Size())) - n27, err := m.Intent_SingleKeySpan.MarshalTo(dAtA[i:]) + n28, err := m.Intent_SingleKeySpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n28 dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Txn.Size())) - n28, err := m.Txn.MarshalTo(dAtA[i:]) + n29, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n29 return i, nil } @@ -2346,19 +2389,19 @@ func (m *LockAcquisition) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Span.Size())) - n29, err := m.Span.MarshalTo(dAtA[i:]) + n30, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n30 dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Txn.Size())) - n30, err := m.Txn.MarshalTo(dAtA[i:]) + n31, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n30 + i += n31 if m.Durability != 0 { dAtA[i] = 0x18 i++ @@ -2385,19 +2428,19 @@ func (m *LockUpdate) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Span.Size())) - n31, err := m.Span.MarshalTo(dAtA[i:]) + n32, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n32 dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Txn.Size())) - n32, err := m.Txn.MarshalTo(dAtA[i:]) + n33, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n33 if m.Status != 0 { dAtA[i] = 0x18 i++ @@ -2465,48 +2508,48 @@ func (m *Lease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Start.Size())) - n33, err := m.Start.MarshalTo(dAtA[i:]) + n34, err := m.Start.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n34 if m.Expiration != nil { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Expiration.Size())) - n34, err := m.Expiration.MarshalTo(dAtA[i:]) + n35, err := m.Expiration.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n35 } dAtA[i] = 0x1a i++ i = encodeVarintData(dAtA, i, uint64(m.Replica.Size())) - n35, err := m.Replica.MarshalTo(dAtA[i:]) + n36, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n35 + i += n36 if m.DeprecatedStartStasis != nil { dAtA[i] = 0x22 i++ i = encodeVarintData(dAtA, i, uint64(m.DeprecatedStartStasis.Size())) - n36, err := m.DeprecatedStartStasis.MarshalTo(dAtA[i:]) + n37, err := m.DeprecatedStartStasis.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n37 } if m.ProposedTS != nil { dAtA[i] = 0x2a i++ i = encodeVarintData(dAtA, i, uint64(m.ProposedTS.Size())) - n37, err := m.ProposedTS.MarshalTo(dAtA[i:]) + n38, err := m.ProposedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n37 + i += n38 } if m.Epoch != 0 { dAtA[i] = 0x30 @@ -2545,11 +2588,11 @@ func (m *AbortSpanEntry) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Timestamp.Size())) - n38, err := m.Timestamp.MarshalTo(dAtA[i:]) + n39, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n38 + i += n39 if m.Priority != 0 { dAtA[i] = 0x18 i++ @@ -2576,11 +2619,11 @@ func (m *LeafTxnInputState) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Txn.Size())) - n39, err := m.Txn.MarshalTo(dAtA[i:]) + n40, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n40 if m.RefreshInvalid { dAtA[i] = 0x38 i++ @@ -2639,11 +2682,11 @@ func (m *LeafTxnFinalState) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Txn.Size())) - n40, err := m.Txn.MarshalTo(dAtA[i:]) + n41, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n41 if m.DeprecatedCommandCount != 0 { dAtA[i] = 0x18 i++ @@ -2725,19 +2768,19 @@ func (m *RangeInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintData(dAtA, i, uint64(m.Desc.Size())) - n41, err := m.Desc.MarshalTo(dAtA[i:]) + n42, err := m.Desc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n42 dAtA[i] = 0x12 i++ i = encodeVarintData(dAtA, i, uint64(m.Lease.Size())) - n42, err := m.Lease.MarshalTo(dAtA[i:]) + n43, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n43 if m.ClosedTimestampPolicy != 0 { dAtA[i] = 0x18 i++ @@ -3099,6 +3142,10 @@ func (m *MergeTrigger) Size() (n int) { n += 1 + l + sovData(uint64(l)) l = m.RightClosedTimestamp.Size() n += 1 + l + sovData(uint64(l)) + if m.RightReadSummary != nil { + l = m.RightReadSummary.Size() + n += 1 + l + sovData(uint64(l)) + } return n } @@ -4265,6 +4312,39 @@ func (m *MergeTrigger) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RightReadSummary", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthData + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RightReadSummary == nil { + m.RightReadSummary = &rspb.ReadSummary{} + } + if err := m.RightReadSummary.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipData(dAtA[iNdEx:]) @@ -7249,171 +7329,174 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_91736d498bf42716) } - -var fileDescriptor_data_91736d498bf42716 = []byte{ - // 2606 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0x4f, 0x70, 0x23, 0x47, - 0xd5, 0xf7, 0x58, 0x23, 0x69, 0xf4, 0x2c, 0xc9, 0xe3, 0x5e, 0xdb, 0xab, 0x75, 0xbe, 0xcf, 0xda, - 0x4f, 0xf9, 0x20, 0xcb, 0x16, 0x91, 0x0b, 0x27, 0xa4, 0x52, 0xcb, 0x52, 0x85, 0xfe, 0xad, 0x23, - 0xad, 0x2c, 0x6d, 0x46, 0xe3, 0x0d, 0x4e, 0xa0, 0x86, 0xd1, 0x4c, 0x5b, 0x1e, 0x3c, 0x9a, 0xd1, - 0xce, 0x8c, 0x1c, 0x8b, 0x1b, 0x17, 0x2a, 0x95, 0x03, 0x70, 0xe4, 0x98, 0x2a, 0x6e, 0x70, 0x80, - 0x23, 0xc5, 0x81, 0x73, 0x0e, 0x1c, 0x72, 0x4b, 0x80, 0x2a, 0x15, 0x38, 0x97, 0x1c, 0xb8, 0x53, - 0xe5, 0x82, 0x2a, 0xaa, 0xbb, 0xe7, 0x9f, 0xbd, 0xb2, 0x91, 0xb3, 0x21, 0xa4, 0xb8, 0xd8, 0x33, - 0xaf, 0xdf, 0xfb, 0xbd, 0xee, 0xd7, 0xef, 0xdf, 0x3c, 0x01, 0x72, 0x6c, 0x55, 0x3b, 0x1c, 0xf5, - 0xb7, 0x74, 0xd5, 0x53, 0xcb, 0x23, 0xc7, 0xf6, 0x6c, 0xb4, 0xa2, 0xd9, 0xda, 0x11, 0xa5, 0x97, - 0xfd, 0xd5, 0x8d, 0xbb, 0x47, 0xc7, 0x5b, 0x47, 0xc7, 0x2e, 0x76, 0x8e, 0xb1, 0xb3, 0xa5, 0xd9, - 0x96, 0x36, 0x76, 0x1c, 0x6c, 0x69, 0x93, 0x2d, 0xd3, 0xd6, 0x8e, 0xe8, 0x1f, 0xc3, 0x1a, 0x30, - 0xf1, 0x8d, 0xf5, 0x00, 0x72, 0x88, 0x3d, 0x35, 0x82, 0xdd, 0x78, 0xce, 0xf5, 0x6c, 0x47, 0x1d, - 0xe0, 0x2d, 0x6c, 0x0d, 0x0c, 0x0b, 0x13, 0x86, 0x63, 0x4d, 0xf3, 0x17, 0xff, 0x67, 0xe6, 0xe2, - 0x4b, 0xfe, 0x6a, 0x61, 0xec, 0x19, 0xe6, 0xd6, 0xa1, 0xa9, 0x6d, 0x79, 0xc6, 0x10, 0xbb, 0x9e, - 0x3a, 0x1c, 0xf9, 0x2b, 0xab, 0x03, 0x7b, 0x60, 0xd3, 0xc7, 0x2d, 0xf2, 0xc4, 0xa8, 0xa5, 0x7d, - 0xe0, 0x7b, 0x23, 0xd5, 0x42, 0xb7, 0x20, 0x71, 0x84, 0x27, 0x85, 0xc4, 0x6d, 0xee, 0x4e, 0xb6, - 0x9a, 0x3e, 0x9b, 0x16, 0x13, 0x0f, 0xf1, 0x44, 0x22, 0x34, 0x74, 0x1b, 0xd2, 0xd8, 0xd2, 0x15, - 0xb2, 0xcc, 0x9f, 0x5f, 0x4e, 0x61, 0x4b, 0x7f, 0x88, 0x27, 0xf7, 0x84, 0x9f, 0xbd, 0x57, 0x5c, - 0xf8, 0xcd, 0x7b, 0x45, 0xae, 0xc5, 0x0b, 0x9c, 0xb8, 0xd8, 0xe2, 0x85, 0x45, 0x31, 0x51, 0x1a, - 0x40, 0xf2, 0xb1, 0x6a, 0x8e, 0x31, 0x7a, 0x0e, 0x32, 0x8e, 0xfa, 0xb6, 0xd2, 0x9f, 0x78, 0xd8, - 0x2d, 0x70, 0x04, 0x42, 0x12, 0x1c, 0xf5, 0xed, 0x2a, 0x79, 0x47, 0x15, 0xc8, 0x84, 0x3b, 0x2d, - 0x2c, 0xde, 0xe6, 0xee, 0x2c, 0x6d, 0xff, 0x6f, 0x39, 0x32, 0x2b, 0x39, 0x4e, 0xf9, 0xd0, 0xd4, - 0xca, 0x72, 0xc0, 0x54, 0xe5, 0xdf, 0x9f, 0x16, 0x17, 0xa4, 0x48, 0xaa, 0xf4, 0x16, 0x08, 0x0f, - 0xf1, 0x84, 0xe9, 0xf2, 0xcf, 0xc1, 0xcd, 0x38, 0xc7, 0xcb, 0x90, 0x3c, 0x26, 0x3c, 0xbe, 0x96, - 0x42, 0xf9, 0xa9, 0xcb, 0x2b, 0x53, 0x0c, 0x5f, 0x01, 0x63, 0x2e, 0x7d, 0xc8, 0x01, 0xf4, 0x3c, - 0xdb, 0xc1, 0x4d, 0x1d, 0x5b, 0x1e, 0x1a, 0x00, 0x68, 0xe6, 0xd8, 0xf5, 0xb0, 0xa3, 0x18, 0xba, - 0xaf, 0xe6, 0x35, 0xc2, 0xff, 0xc7, 0x69, 0xf1, 0xa5, 0x81, 0xe1, 0x1d, 0x8e, 0xfb, 0x65, 0xcd, - 0x1e, 0x6e, 0x85, 0xd8, 0x7a, 0x3f, 0x7a, 0xde, 0x1a, 0x1d, 0x0d, 0xb6, 0xe8, 0x05, 0x8d, 0xc7, - 0x86, 0x5e, 0xde, 0xdb, 0x6b, 0xd6, 0x4f, 0xa7, 0xc5, 0x4c, 0x8d, 0x01, 0x36, 0xeb, 0x52, 0xc6, - 0xc7, 0x6e, 0xea, 0xe8, 0x45, 0x48, 0x5b, 0xb6, 0x8e, 0x89, 0x16, 0xb2, 0xdf, 0x64, 0x75, 0xf5, - 0x74, 0x5a, 0x4c, 0x75, 0x6c, 0x1d, 0x37, 0xeb, 0x67, 0xe1, 0x93, 0x94, 0x22, 0x4c, 0x4d, 0x1d, - 0x7d, 0x0d, 0x04, 0xe2, 0x17, 0x94, 0x3f, 0x41, 0xf9, 0xd7, 0x4f, 0xa7, 0xc5, 0x34, 0xdb, 0x39, - 0x11, 0x08, 0x1e, 0xa5, 0xb4, 0xcb, 0x4e, 0x53, 0xfa, 0x05, 0x07, 0xd9, 0xde, 0xc8, 0x34, 0x3c, - 0xd9, 0x31, 0x06, 0x03, 0xec, 0xa0, 0x06, 0x64, 0x4c, 0x7c, 0xe0, 0x29, 0x3a, 0x76, 0x35, 0x7a, - 0xb4, 0xa5, 0xed, 0xd2, 0x0c, 0x23, 0x49, 0xaa, 0x35, 0xc0, 0x75, 0xec, 0x6a, 0x8e, 0x31, 0xf2, - 0x6c, 0xc7, 0x37, 0x97, 0x40, 0x44, 0x09, 0x15, 0xed, 0x00, 0x38, 0xc6, 0xe0, 0xd0, 0xc7, 0x59, - 0xbc, 0x26, 0x4e, 0x86, 0xca, 0x12, 0xf2, 0x3d, 0xfe, 0x13, 0xe6, 0x52, 0x09, 0x91, 0x2f, 0xfd, - 0x2d, 0x01, 0xd9, 0x5d, 0xec, 0x0c, 0xf0, 0x17, 0x74, 0xb3, 0x68, 0x00, 0x22, 0x03, 0x22, 0xd1, - 0xa8, 0xb8, 0x9e, 0xea, 0xb9, 0x34, 0x5c, 0x96, 0xb6, 0xbf, 0x14, 0x83, 0xf3, 0x63, 0xb7, 0x1c, - 0xc4, 0x6e, 0x79, 0xf7, 0x71, 0xad, 0xd6, 0x23, 0xcc, 0xd5, 0x75, 0x82, 0x78, 0x3a, 0x2d, 0xe6, - 0x25, 0x02, 0x13, 0xd2, 0xa5, 0x3c, 0x85, 0xdd, 0x3d, 0xd6, 0x34, 0xfa, 0x8e, 0x7e, 0xc4, 0x41, - 0xf6, 0xc0, 0xc1, 0xf8, 0x07, 0x98, 0x68, 0x71, 0xbc, 0x42, 0x72, 0x9e, 0xa0, 0xa9, 0x13, 0xf4, - 0xb3, 0x69, 0xf1, 0xfe, 0xfc, 0x3e, 0x4a, 0x00, 0x6a, 0x24, 0x61, 0x85, 0x28, 0xd2, 0x12, 0x53, - 0xdc, 0x23, 0x7a, 0xd1, 0x3e, 0xac, 0xb3, 0x13, 0x6b, 0xa6, 0xed, 0x62, 0x5d, 0x89, 0xc2, 0x38, - 0x35, 0x7f, 0x18, 0xaf, 0x52, 0x88, 0x1a, 0x45, 0x08, 0xd7, 0xce, 0xdd, 0xfc, 0x2f, 0x93, 0xb0, - 0x56, 0x3b, 0x24, 0xe6, 0x97, 0xf0, 0xc8, 0x34, 0x34, 0xd5, 0x0d, 0x5c, 0xe0, 0x4d, 0x58, 0xd7, - 0xf1, 0xc8, 0xc1, 0x9a, 0xea, 0x61, 0x5d, 0xd1, 0x28, 0x8f, 0xe2, 0x4d, 0x46, 0x98, 0xfa, 0x43, - 0x7e, 0xfb, 0xff, 0x67, 0xdd, 0x23, 0xc3, 0x60, 0x80, 0xf2, 0x64, 0x84, 0xa5, 0xd5, 0x08, 0x23, - 0xa2, 0xa2, 0x7d, 0x40, 0x31, 0x6c, 0x87, 0x49, 0xf9, 0xfe, 0x71, 0x05, 0xee, 0x53, 0x1e, 0xb2, - 0x12, 0xa1, 0xf8, 0x2c, 0xe8, 0xfb, 0xf0, 0x5c, 0x0c, 0x7a, 0x3c, 0xd2, 0xe3, 0x2a, 0xdc, 0x42, - 0xe2, 0x76, 0xe2, 0x9a, 0x3a, 0x6e, 0x45, 0x70, 0x7b, 0x0c, 0x2d, 0xb0, 0x14, 0xc2, 0xb0, 0x11, - 0xd3, 0x65, 0xe1, 0x13, 0x2f, 0x50, 0x44, 0x12, 0x05, 0x4f, 0x13, 0xc5, 0x9d, 0xd3, 0x69, 0xf1, - 0x66, 0x3d, 0xe4, 0xea, 0xe0, 0x13, 0xcf, 0x97, 0xa7, 0x89, 0x23, 0x13, 0xbe, 0x48, 0x37, 0xf5, - 0x99, 0x5c, 0x3a, 0x7a, 0x05, 0x78, 0x1a, 0x3f, 0xc9, 0x79, 0xe3, 0x47, 0xa2, 0xfc, 0xa8, 0x0f, - 0x37, 0x0d, 0xcb, 0xc3, 0x8e, 0xa5, 0x9a, 0x8a, 0xaa, 0xeb, 0x71, 0x33, 0xa4, 0xae, 0x6d, 0x86, - 0xb5, 0x00, 0xaa, 0x42, 0x90, 0x42, 0x13, 0x1c, 0xc0, 0xad, 0x50, 0x87, 0x83, 0x87, 0xf6, 0x71, - 0x5c, 0x4b, 0xfa, 0xda, 0x5a, 0xc2, 0x0d, 0x4b, 0x0c, 0x2b, 0xd0, 0x73, 0x8f, 0x27, 0x45, 0xb0, - 0xf4, 0x2e, 0x07, 0x37, 0x76, 0x6d, 0xdd, 0x38, 0x30, 0xb0, 0x4e, 0x0a, 0x6b, 0xe0, 0xab, 0x5f, - 0x05, 0xe4, 0x4e, 0x5c, 0x0f, 0x0f, 0x15, 0xcd, 0xb6, 0x0e, 0x8c, 0x81, 0xe2, 0x8e, 0x54, 0x8b, - 0xfa, 0xa9, 0x20, 0x89, 0x6c, 0xa5, 0x46, 0x17, 0x68, 0x35, 0x6e, 0x00, 0xa2, 0xc9, 0xdf, 0x34, - 0x8e, 0xb1, 0x85, 0x5d, 0x97, 0x71, 0x33, 0xef, 0xbb, 0x39, 0x63, 0xb3, 0x44, 0x48, 0x12, 0x89, - 0x48, 0xdb, 0x97, 0x20, 0x94, 0xd2, 0x63, 0x10, 0x7b, 0x9e, 0xa1, 0x1d, 0x4d, 0xaa, 0x51, 0x92, - 0xaf, 0x02, 0xb8, 0x94, 0xa6, 0xf4, 0x0d, 0xcf, 0x4f, 0x9c, 0xf3, 0x15, 0x5c, 0x37, 0x80, 0x2a, - 0xfd, 0x36, 0x01, 0x6b, 0x4d, 0xdf, 0x0c, 0x35, 0x7b, 0x38, 0x8c, 0xd0, 0xeb, 0x90, 0x73, 0x49, - 0x49, 0x51, 0x3c, 0x46, 0xf0, 0x15, 0x14, 0x67, 0xee, 0x39, 0x2a, 0x3d, 0x52, 0xd6, 0x8d, 0x17, - 0xa2, 0x3a, 0xe4, 0x86, 0x24, 0xd7, 0x87, 0x28, 0x8b, 0x97, 0xa2, 0xc4, 0x6b, 0x82, 0x94, 0x1d, - 0xc6, 0x2b, 0xc4, 0xf7, 0xe0, 0xa6, 0x9f, 0x13, 0x82, 0xeb, 0x0e, 0xf1, 0x12, 0x14, 0xef, 0xce, - 0x0c, 0xbc, 0x99, 0x99, 0x46, 0x5a, 0xd3, 0x2e, 0x49, 0x40, 0x6b, 0x43, 0xff, 0xae, 0xe9, 0x0d, - 0x85, 0xf8, 0x2c, 0xf1, 0x7f, 0x79, 0xd6, 0x7e, 0x9f, 0xf6, 0x0d, 0xe9, 0xc6, 0x70, 0x86, 0xc3, - 0xbc, 0x0e, 0x28, 0xba, 0xa7, 0x10, 0x98, 0x05, 0xd8, 0xf3, 0xb3, 0xcc, 0x79, 0xe1, 0xa2, 0x25, - 0xd1, 0xbd, 0x40, 0xb9, 0xc7, 0xbf, 0xf3, 0x5e, 0x91, 0x2b, 0xfd, 0x9e, 0x83, 0x95, 0x6e, 0x9f, - 0x36, 0xa8, 0x51, 0xc6, 0x8d, 0xb7, 0x1b, 0xdc, 0x1c, 0xed, 0xc6, 0x0f, 0xb9, 0x6b, 0xb7, 0x6d, - 0x9f, 0x4d, 0x05, 0x8a, 0xb4, 0xde, 0xe3, 0x49, 0xc7, 0x59, 0x3a, 0x4b, 0xc1, 0x92, 0xec, 0xa8, - 0x96, 0xab, 0x6a, 0x9e, 0x61, 0x5b, 0xa8, 0x02, 0x3c, 0xe9, 0xa6, 0x7d, 0xc7, 0x7b, 0xfe, 0xaa, - 0xda, 0x2b, 0x9f, 0x58, 0xbb, 0xd8, 0x53, 0xab, 0x02, 0xd9, 0xd9, 0x07, 0xd3, 0x22, 0x27, 0x51, - 0x51, 0x84, 0x80, 0xb7, 0xd4, 0x21, 0xeb, 0x13, 0x33, 0x12, 0x7d, 0x46, 0xf7, 0x21, 0x45, 0x6a, - 0xfa, 0x98, 0x15, 0xf5, 0xd9, 0xb5, 0x25, 0xb6, 0x8d, 0x1e, 0xe5, 0x95, 0x7c, 0x19, 0xd4, 0x82, - 0xbc, 0xa9, 0xba, 0x9e, 0x72, 0x88, 0x55, 0xc7, 0xeb, 0x63, 0x75, 0xce, 0xa2, 0xcd, 0x02, 0x2f, - 0x47, 0x44, 0x5f, 0x0b, 0x24, 0xd1, 0x77, 0xa1, 0x30, 0x30, 0xed, 0xbe, 0x6a, 0x2a, 0x63, 0x4b, - 0xc3, 0x8e, 0xa7, 0x1a, 0x96, 0x37, 0x51, 0x4c, 0x63, 0x68, 0x78, 0x85, 0xf4, 0xfc, 0xa8, 0xeb, - 0x0c, 0x64, 0x2f, 0xc2, 0x68, 0x13, 0x08, 0xf4, 0x16, 0xdc, 0xb0, 0x7d, 0xef, 0x88, 0x2a, 0xba, - 0x5b, 0x10, 0x2e, 0x4d, 0x94, 0x4f, 0xf9, 0x92, 0xaf, 0x00, 0xd9, 0x17, 0x17, 0x5c, 0x74, 0x1f, - 0x80, 0x5c, 0x27, 0x0d, 0x16, 0xb7, 0xb0, 0x44, 0x31, 0x2f, 0xcb, 0x67, 0x41, 0xda, 0x21, 0x02, - 0xe4, 0xdd, 0x45, 0x25, 0xc8, 0xbd, 0xed, 0x18, 0x1e, 0x56, 0x3c, 0xdb, 0x56, 0x6c, 0x53, 0x2f, - 0x64, 0x69, 0xfa, 0x5c, 0xa2, 0x44, 0xd9, 0xb6, 0xbb, 0xa6, 0x4e, 0x2c, 0xed, 0x60, 0x35, 0xde, - 0x8c, 0x2c, 0x5f, 0xc3, 0xd2, 0x44, 0x34, 0x8a, 0x89, 0x97, 0x61, 0x5d, 0xa3, 0xd9, 0x2d, 0x42, - 0x53, 0x0e, 0x8c, 0x13, 0xac, 0x17, 0x44, 0xaa, 0x78, 0x95, 0xad, 0x86, 0x02, 0x0f, 0xc8, 0x1a, - 0x7a, 0x1d, 0x44, 0xc3, 0x52, 0x0e, 0x4c, 0xda, 0x1a, 0xd1, 0xad, 0xb9, 0x85, 0x15, 0x7a, 0xd2, - 0xff, 0x9b, 0x75, 0x52, 0xfc, 0x64, 0x8c, 0x2d, 0x0d, 0xeb, 0x6f, 0x10, 0x4e, 0x7f, 0x1f, 0x79, - 0xc3, 0x7a, 0x40, 0xe5, 0x29, 0xd1, 0x45, 0x36, 0x2c, 0x1b, 0x03, 0xcb, 0x76, 0x48, 0x9a, 0xc1, - 0x4f, 0xac, 0xf1, 0xd0, 0x2d, 0x20, 0x8a, 0x58, 0xbe, 0xca, 0xbd, 0x9b, 0x4c, 0xa4, 0x87, 0x9f, - 0x74, 0xc6, 0x43, 0x5a, 0x7f, 0xa3, 0x1e, 0xf3, 0xdc, 0x9a, 0x2b, 0xe5, 0x8d, 0xf0, 0x9d, 0xa0, - 0x9f, 0xfb, 0xa0, 0x4b, 0x88, 0x7c, 0x8b, 0x17, 0x52, 0x62, 0xba, 0xc5, 0x0b, 0x19, 0x11, 0x5a, - 0xbc, 0x90, 0x13, 0xf3, 0x2d, 0x5e, 0xc8, 0x8b, 0xcb, 0xa5, 0x5f, 0xf3, 0xb0, 0x12, 0xf3, 0x7a, - 0x09, 0x6b, 0xb6, 0xa3, 0x7f, 0x16, 0x21, 0xf8, 0xc5, 0x09, 0xb7, 0x67, 0x73, 0xd9, 0xff, 0x06, - 0x67, 0xe0, 0x7d, 0x47, 0x58, 0x14, 0x13, 0x17, 0xdc, 0x21, 0x2d, 0x0a, 0x2d, 0x5e, 0x10, 0xc4, - 0x4c, 0xe8, 0x1a, 0x20, 0x2e, 0xb5, 0x78, 0x21, 0x2b, 0xe6, 0xe2, 0x6e, 0xd2, 0xe2, 0x85, 0x65, - 0x51, 0x6c, 0xf1, 0x82, 0x28, 0xae, 0x94, 0xa6, 0x1c, 0xa4, 0x48, 0xef, 0x60, 0x91, 0x0f, 0x88, - 0x65, 0xd7, 0xb0, 0x06, 0x26, 0x56, 0x8e, 0xf0, 0x24, 0x6a, 0x88, 0x96, 0xb6, 0x5f, 0x98, 0x61, - 0x1b, 0x26, 0x53, 0xee, 0x51, 0x81, 0x87, 0x78, 0x42, 0xed, 0x1d, 0xb9, 0x4d, 0xce, 0x8d, 0x2f, - 0xa0, 0x6f, 0x40, 0xc2, 0x3b, 0x09, 0x3a, 0xa6, 0xb9, 0x3c, 0x90, 0x19, 0x9b, 0x48, 0x6d, 0xdc, - 0x87, 0xdc, 0x39, 0x35, 0x57, 0x0c, 0x47, 0xe2, 0x03, 0x8f, 0x16, 0x2f, 0xf0, 0x62, 0xb2, 0xf4, - 0x07, 0x0e, 0x96, 0xdb, 0xb6, 0x76, 0x54, 0xd1, 0x9e, 0x8c, 0x0d, 0xd7, 0xa0, 0x45, 0xe9, 0xeb, - 0xc0, 0xc7, 0x8e, 0x77, 0xa9, 0xfb, 0xc4, 0xa2, 0xc0, 0x7d, 0xd6, 0x53, 0x20, 0x09, 0x40, 0x1f, - 0x3b, 0x6a, 0xdf, 0x30, 0x0d, 0x8f, 0xed, 0x3d, 0xbf, 0xbd, 0x1d, 0xc3, 0x38, 0x3a, 0x2e, 0x07, - 0x73, 0xaa, 0x72, 0x6c, 0x4e, 0x55, 0x26, 0xce, 0x5b, 0xae, 0x87, 0x92, 0x52, 0x0c, 0xa5, 0xf4, - 0xab, 0x45, 0x00, 0x72, 0x36, 0xf6, 0x99, 0xf1, 0x1f, 0x39, 0x56, 0x94, 0x19, 0x12, 0x9f, 0x22, - 0x33, 0xcc, 0x08, 0x1e, 0xfe, 0xdf, 0x19, 0x3c, 0xa5, 0x9f, 0x70, 0x90, 0x3f, 0x1f, 0xd6, 0x57, - 0x8d, 0xa8, 0xbe, 0x03, 0x82, 0xeb, 0x33, 0xfb, 0x53, 0x9f, 0x6f, 0xcd, 0xdd, 0x31, 0x5d, 0x1c, - 0x0b, 0x12, 0xe3, 0xf5, 0xf0, 0x13, 0x29, 0x44, 0xf4, 0x1b, 0xa6, 0xdf, 0xf1, 0x90, 0x6c, 0x63, - 0xd5, 0xc5, 0x68, 0x02, 0x49, 0x36, 0x41, 0xe0, 0x3e, 0xbf, 0xfe, 0x8d, 0x69, 0x44, 0xdf, 0x04, - 0xc0, 0x27, 0x23, 0xc3, 0x51, 0xc9, 0x1d, 0xcd, 0xd5, 0x3f, 0x4a, 0x31, 0x01, 0x54, 0x87, 0x74, - 0xf0, 0x49, 0x9e, 0xb8, 0xf6, 0x27, 0x79, 0x20, 0x8a, 0xf6, 0x20, 0xf6, 0x41, 0xcb, 0x86, 0x29, - 0xe4, 0xaf, 0x6b, 0x04, 0x93, 0x9b, 0x7f, 0xb1, 0xa3, 0xb5, 0x48, 0x9a, 0x4e, 0x44, 0x7a, 0x54, - 0x16, 0xfd, 0x98, 0x83, 0xa5, 0x91, 0x63, 0x8f, 0xd8, 0x50, 0xc4, 0x9d, 0xaf, 0xf6, 0x74, 0x4e, - 0xa7, 0x45, 0x78, 0xe4, 0x4b, 0xc9, 0xbd, 0x67, 0xb6, 0x33, 0x04, 0x3b, 0x90, 0x5d, 0xb4, 0x0a, - 0x49, 0x3c, 0xb2, 0xb5, 0x43, 0x3a, 0x97, 0x49, 0x48, 0xec, 0x05, 0xbd, 0x18, 0xf3, 0x35, 0xd2, - 0x37, 0x26, 0xaa, 0x2b, 0x67, 0xd3, 0x62, 0x8e, 0xba, 0x46, 0xe0, 0xb1, 0x31, 0xe7, 0x09, 0x5b, - 0x82, 0xd2, 0x9f, 0x38, 0xc8, 0x57, 0xfa, 0xb6, 0xe3, 0x91, 0xe0, 0x6e, 0x58, 0x9e, 0x33, 0xb9, - 0xca, 0xa5, 0x9f, 0x7d, 0xbe, 0x8b, 0x54, 0x10, 0x46, 0x8e, 0x61, 0x3b, 0x41, 0x1e, 0x4b, 0x56, - 0x1b, 0x67, 0xd3, 0x62, 0xe5, 0x53, 0x47, 0xc5, 0x23, 0x1f, 0x4c, 0x0a, 0x61, 0xef, 0x09, 0xe4, - 0x64, 0x9f, 0x90, 0xd3, 0xfd, 0x63, 0x11, 0x56, 0xda, 0x58, 0x3d, 0x90, 0x4f, 0xac, 0xa6, 0x35, - 0x1a, 0x93, 0x4b, 0xf5, 0x30, 0x7a, 0x85, 0xa5, 0x2c, 0x16, 0x28, 0x9b, 0x57, 0xa7, 0x9c, 0x78, - 0xb6, 0x7a, 0x01, 0x96, 0x1d, 0x7c, 0xe0, 0x60, 0xf7, 0x50, 0x31, 0xac, 0x63, 0xd5, 0x34, 0x74, - 0x6a, 0x6b, 0x41, 0xca, 0xfb, 0xe4, 0x26, 0xa3, 0xce, 0x6c, 0x14, 0x84, 0x67, 0x6b, 0x14, 0xb6, - 0x61, 0xcd, 0xf5, 0xf0, 0x68, 0x64, 0x58, 0x03, 0x65, 0x48, 0xbe, 0xed, 0xb0, 0xa5, 0xf6, 0x4d, - 0xac, 0x17, 0x32, 0x74, 0x07, 0x37, 0x82, 0xc5, 0x5d, 0x5b, 0xc7, 0x0d, 0xb6, 0x84, 0xfa, 0x90, - 0xa5, 0xed, 0xb3, 0x8b, 0x9f, 0x28, 0xd6, 0x78, 0x58, 0x80, 0xcf, 0x28, 0x09, 0x01, 0x41, 0x65, - 0x39, 0xf2, 0x42, 0x27, 0xc1, 0x8b, 0xc9, 0x16, 0x2f, 0x24, 0xc5, 0x14, 0xeb, 0x2a, 0x4a, 0xef, - 0x46, 0xf6, 0x7f, 0x60, 0x58, 0xaa, 0xf9, 0x6c, 0xf6, 0x7f, 0x15, 0x0a, 0xf1, 0x11, 0xa1, 0x3d, - 0x1c, 0xaa, 0x16, 0xf9, 0x3f, 0xb6, 0x3c, 0xe6, 0x4a, 0x52, 0x6c, 0x84, 0x58, 0x63, 0xcb, 0x35, - 0xb2, 0x8a, 0xaa, 0x90, 0x0b, 0x6e, 0x8e, 0xb5, 0x7e, 0xfc, 0x3c, 0xad, 0x5f, 0xd6, 0x97, 0x61, - 0xdd, 0xdf, 0xbc, 0xb7, 0x1f, 0x9a, 0x24, 0x34, 0x03, 0x6b, 0xab, 0x4a, 0x7f, 0xe7, 0x60, 0xb9, - 0x66, 0x1a, 0xd8, 0xf2, 0x68, 0xd5, 0x69, 0x5a, 0x07, 0x36, 0x7a, 0x0d, 0xd6, 0xf4, 0x30, 0xa5, - 0x29, 0x03, 0x6c, 0x61, 0x3f, 0x8b, 0x72, 0x34, 0x88, 0x6f, 0x9c, 0x4d, 0x8b, 0xcb, 0x94, 0x7b, - 0x27, 0x5c, 0x92, 0x56, 0x23, 0x89, 0x88, 0x8a, 0x5e, 0x85, 0xbc, 0x49, 0xa2, 0x5d, 0x39, 0x57, - 0x73, 0x66, 0xe6, 0x81, 0x9c, 0x19, 0x7f, 0x45, 0x18, 0x6e, 0x5e, 0x1c, 0xfa, 0x2a, 0x23, 0xdb, - 0x34, 0xb4, 0xa0, 0xd1, 0x78, 0xf1, 0xb2, 0x11, 0xe0, 0x85, 0x49, 0xef, 0x23, 0x2a, 0x24, 0xad, - 0x69, 0xb3, 0xc8, 0xa5, 0xbf, 0x72, 0x90, 0x89, 0x0e, 0x7e, 0xdf, 0x1f, 0x32, 0x5e, 0x77, 0xd8, - 0xcf, 0x46, 0x8d, 0x2f, 0x43, 0x92, 0x9e, 0xe1, 0x8a, 0x5f, 0x7f, 0xe8, 0x91, 0x83, 0x5f, 0x7f, - 0x28, 0xf3, 0xe7, 0x74, 0x50, 0x36, 0x3b, 0xbc, 0xfb, 0x21, 0x07, 0x19, 0xfa, 0x0b, 0x14, 0x9d, - 0x40, 0x2f, 0x41, 0x7a, 0xaf, 0xf3, 0xb0, 0xd3, 0x7d, 0xa3, 0x23, 0x2e, 0xa0, 0x34, 0x24, 0x9a, - 0x1d, 0x59, 0xe4, 0x50, 0x06, 0x92, 0x0f, 0xda, 0xdd, 0x8a, 0x2c, 0x2e, 0x92, 0xc7, 0xea, 0xbe, - 0xdc, 0xe8, 0x89, 0x09, 0x74, 0x03, 0x96, 0xeb, 0x8d, 0x76, 0x73, 0xb7, 0x29, 0x37, 0xea, 0x0a, - 0x23, 0x0a, 0x48, 0x00, 0x5e, 0x6e, 0xee, 0x36, 0x44, 0x9e, 0x40, 0xd5, 0x1b, 0xb5, 0xe6, 0x6e, - 0xa5, 0x2d, 0x26, 0xd1, 0x1a, 0xac, 0x44, 0xbc, 0x01, 0x39, 0x83, 0xb2, 0x20, 0xd4, 0xf7, 0xa4, - 0x8a, 0xdc, 0xec, 0x76, 0xc4, 0x14, 0x02, 0x48, 0x11, 0x59, 0xf9, 0x4d, 0x31, 0x4b, 0x74, 0xef, - 0x34, 0xba, 0x62, 0x8e, 0x2a, 0xec, 0x7e, 0x7b, 0xbb, 0x2e, 0xe6, 0xc9, 0xa3, 0xbc, 0xf7, 0xa8, - 0xdd, 0x10, 0x81, 0x08, 0x56, 0x9b, 0x72, 0x45, 0x92, 0x2a, 0xfb, 0xe2, 0x12, 0xca, 0x03, 0x10, - 0xc1, 0x5e, 0x43, 0x6a, 0x36, 0x7a, 0xa2, 0x5e, 0x22, 0x1f, 0x09, 0xe9, 0xbb, 0x1a, 0xac, 0x3c, - 0x35, 0x78, 0x47, 0x39, 0xc8, 0x54, 0xea, 0x75, 0xe5, 0x71, 0x57, 0x6e, 0x48, 0xe2, 0x02, 0x12, - 0x21, 0x2b, 0x35, 0x76, 0xbb, 0x8f, 0x1b, 0x3e, 0x85, 0x43, 0x2b, 0x90, 0x23, 0x0c, 0x9d, 0x6e, - 0xc7, 0x27, 0x2d, 0xa2, 0x55, 0x10, 0x7d, 0xa6, 0x88, 0x9a, 0xd8, 0xe0, 0xdf, 0xf9, 0xf9, 0xe6, - 0xc2, 0xdd, 0xee, 0xb9, 0x6f, 0x51, 0xd6, 0xf8, 0x91, 0xa3, 0x3f, 0x6a, 0x74, 0xea, 0xcd, 0xce, - 0x8e, 0xb8, 0x40, 0x5e, 0x7a, 0x72, 0x65, 0x87, 0xbc, 0x24, 0x88, 0xfa, 0x5a, 0x77, 0x77, 0xb7, - 0x29, 0xcb, 0x8d, 0xba, 0xc8, 0x91, 0xb5, 0x4a, 0xb5, 0x2b, 0x91, 0x97, 0x45, 0x1f, 0x70, 0x1f, - 0x36, 0x2e, 0xbf, 0x4a, 0xb4, 0x01, 0xeb, 0xed, 0xca, 0x8e, 0x52, 0xdd, 0x57, 0x6a, 0xed, 0xbd, - 0x9e, 0xdc, 0x90, 0x94, 0x5e, 0x43, 0x96, 0x99, 0xa2, 0x5b, 0xb0, 0xd6, 0x6e, 0x54, 0xea, 0xca, - 0x83, 0xae, 0xa4, 0xec, 0xb4, 0xbb, 0xd5, 0x4a, 0x5b, 0x91, 0x1a, 0x95, 0x7a, 0x4f, 0xe4, 0x18, - 0x74, 0xf5, 0x2b, 0xef, 0xff, 0x65, 0x73, 0xe1, 0xfd, 0xd3, 0x4d, 0xee, 0x83, 0xd3, 0x4d, 0xee, - 0xa3, 0xd3, 0x4d, 0xee, 0xcf, 0xa7, 0x9b, 0xdc, 0x4f, 0x3f, 0xde, 0x5c, 0xf8, 0xe0, 0xe3, 0xcd, - 0x85, 0x8f, 0x3e, 0xde, 0x5c, 0x78, 0x33, 0xed, 0x7b, 0x53, 0x3f, 0x45, 0x7f, 0xa8, 0x7d, 0xe9, - 0x9f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x08, 0xe1, 0x66, 0xfa, 0x80, 0x1e, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_73b4def784a0a137) } + +var fileDescriptor_data_73b4def784a0a137 = []byte{ + // 2654 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0xcb, 0x73, 0x23, 0x57, + 0xd5, 0x77, 0x5b, 0x2d, 0xa9, 0x75, 0x2c, 0xc9, 0xed, 0x3b, 0xb6, 0x47, 0xe3, 0x7c, 0x9f, 0x35, + 0x28, 0x40, 0x86, 0x29, 0x22, 0x15, 0x4e, 0x48, 0xa5, 0x86, 0xa1, 0x0a, 0xbd, 0xc6, 0x91, 0x46, + 0x96, 0x26, 0x2d, 0x79, 0x82, 0x93, 0x50, 0x4d, 0xab, 0xfb, 0x5a, 0x6e, 0xdc, 0xea, 0xd6, 0x74, + 0xb7, 0x1c, 0x8b, 0x1d, 0x1b, 0x2a, 0x95, 0x05, 0xb0, 0x64, 0x99, 0x2a, 0x76, 0xb0, 0x80, 0x25, + 0xc5, 0x82, 0x75, 0x16, 0x2c, 0xb2, 0xa1, 0x12, 0xa0, 0x4a, 0x05, 0xce, 0x26, 0x0b, 0xfe, 0x02, + 0x17, 0x54, 0x51, 0xf7, 0xde, 0x7e, 0xd9, 0x23, 0x1b, 0x39, 0x1e, 0x42, 0x8a, 0x8d, 0xdd, 0x7d, + 0x1e, 0xbf, 0x73, 0x1f, 0xe7, 0xd5, 0x47, 0x80, 0x6c, 0x4b, 0x51, 0x0f, 0x46, 0xfd, 0x92, 0xa6, + 0xb8, 0x4a, 0x71, 0x64, 0x5b, 0xae, 0x85, 0x56, 0x54, 0x4b, 0x3d, 0xa4, 0xf4, 0xa2, 0xc7, 0xdd, + 0xb8, 0x7b, 0x78, 0x54, 0x3a, 0x3c, 0x72, 0xb0, 0x7d, 0x84, 0xed, 0x92, 0x6a, 0x99, 0xea, 0xd8, + 0xb6, 0xb1, 0xa9, 0x4e, 0x4a, 0x86, 0xa5, 0x1e, 0xd2, 0x3f, 0xba, 0x39, 0x60, 0xea, 0x67, 0x65, + 0x6d, 0xac, 0x68, 0xce, 0x78, 0x38, 0x54, 0xec, 0x49, 0xc9, 0x76, 0x46, 0xfd, 0x92, 0xf7, 0xe2, + 0xc9, 0xae, 0xfb, 0xe6, 0x87, 0xd8, 0x55, 0xc2, 0x25, 0x6c, 0x3c, 0xe7, 0xb8, 0x96, 0xad, 0x0c, + 0x70, 0x09, 0x9b, 0x03, 0xdd, 0xc4, 0x44, 0xe0, 0x48, 0x55, 0x3d, 0xe6, 0xff, 0xcd, 0x64, 0xbe, + 0xe4, 0x71, 0x73, 0x63, 0x57, 0x37, 0x4a, 0x07, 0x86, 0x5a, 0x72, 0xf5, 0x21, 0x76, 0x5c, 0x65, + 0x38, 0xf2, 0x38, 0xab, 0x03, 0x6b, 0x60, 0xd1, 0xc7, 0x12, 0x79, 0x62, 0xd4, 0xc2, 0x1e, 0xf0, + 0xdd, 0x91, 0x62, 0xa2, 0x5b, 0x10, 0x3b, 0xc4, 0x93, 0x5c, 0xec, 0x36, 0x77, 0x27, 0x5d, 0x49, + 0x9e, 0x4e, 0xf3, 0xb1, 0x87, 0x78, 0x22, 0x11, 0x1a, 0xba, 0x0d, 0x49, 0x6c, 0x6a, 0x32, 0x61, + 0xf3, 0x67, 0xd9, 0x09, 0x6c, 0x6a, 0x0f, 0xf1, 0xe4, 0x9e, 0xf0, 0xf3, 0xf7, 0xf3, 0x0b, 0xbf, + 0x7d, 0x3f, 0xcf, 0x35, 0x79, 0x81, 0x13, 0x17, 0x9b, 0xbc, 0xb0, 0x28, 0xc6, 0x0a, 0x03, 0x88, + 0x3f, 0x56, 0x8c, 0x31, 0x46, 0xcf, 0x41, 0xca, 0x56, 0xde, 0x91, 0xfb, 0x13, 0x17, 0x3b, 0x39, + 0x8e, 0x40, 0x48, 0x82, 0xad, 0xbc, 0x53, 0x21, 0xef, 0xa8, 0x0c, 0xa9, 0x60, 0xa5, 0xb9, 0xc5, + 0xdb, 0xdc, 0x9d, 0xa5, 0xad, 0xff, 0x2f, 0x86, 0x57, 0x40, 0xb6, 0x53, 0x3c, 0x30, 0xd4, 0x62, + 0xcf, 0x17, 0xaa, 0xf0, 0x1f, 0x4c, 0xf3, 0x0b, 0x52, 0xa8, 0x55, 0x78, 0x0b, 0x84, 0x87, 0x78, + 0xc2, 0x6c, 0x79, 0xfb, 0xe0, 0x66, 0xec, 0xe3, 0x65, 0x88, 0x1f, 0x11, 0x19, 0xcf, 0x4a, 0xae, + 0xf8, 0xd4, 0x45, 0x17, 0x29, 0x86, 0x67, 0x80, 0x09, 0x17, 0x3e, 0xe2, 0x00, 0xba, 0xae, 0x65, + 0xe3, 0x86, 0x86, 0x4d, 0x17, 0x0d, 0x00, 0x54, 0x63, 0xec, 0xb8, 0xd8, 0x96, 0x75, 0xcd, 0x33, + 0xf3, 0x1a, 0x91, 0xff, 0xf3, 0x34, 0xff, 0xd2, 0x40, 0x77, 0x0f, 0xc6, 0xfd, 0xa2, 0x6a, 0x0d, + 0x4b, 0x01, 0xb6, 0xd6, 0x0f, 0x9f, 0x4b, 0xa3, 0xc3, 0x41, 0x89, 0x5e, 0xd0, 0x78, 0xac, 0x6b, + 0xc5, 0xdd, 0xdd, 0x46, 0xed, 0x64, 0x9a, 0x4f, 0x55, 0x19, 0x60, 0xa3, 0x26, 0xa5, 0x3c, 0xec, + 0x86, 0x86, 0x5e, 0x84, 0xa4, 0x69, 0x69, 0x98, 0x58, 0x21, 0xeb, 0x8d, 0x57, 0x56, 0x4f, 0xa6, + 0xf9, 0x44, 0xdb, 0xd2, 0x70, 0xa3, 0x76, 0x1a, 0x3c, 0x49, 0x09, 0x22, 0xd4, 0xd0, 0xd0, 0x37, + 0x40, 0x20, 0x7e, 0x41, 0xe5, 0x63, 0x54, 0x7e, 0xfd, 0x64, 0x9a, 0x4f, 0xb2, 0x95, 0x13, 0x05, + 0xff, 0x51, 0x4a, 0x3a, 0x6c, 0x37, 0x85, 0x5f, 0x72, 0x90, 0xee, 0x8e, 0x0c, 0xdd, 0xed, 0xd9, + 0xfa, 0x60, 0x80, 0x6d, 0x54, 0x87, 0x94, 0x81, 0xf7, 0x5d, 0x59, 0xc3, 0x8e, 0x4a, 0xb7, 0xb6, + 0xb4, 0x55, 0x98, 0x71, 0x48, 0x92, 0x62, 0x0e, 0x70, 0x0d, 0x3b, 0xaa, 0xad, 0x8f, 0x5c, 0xcb, + 0xf6, 0x8e, 0x4b, 0x20, 0xaa, 0x84, 0x8a, 0xb6, 0x01, 0x6c, 0x7d, 0x70, 0xe0, 0xe1, 0x2c, 0x5e, + 0x11, 0x27, 0x45, 0x75, 0x09, 0xf9, 0x1e, 0xff, 0x29, 0x73, 0xa9, 0x98, 0xc8, 0x17, 0xfe, 0xc8, + 0x43, 0x7a, 0x07, 0xdb, 0x03, 0xfc, 0x05, 0x5d, 0x2c, 0x1a, 0x80, 0xc8, 0x80, 0x48, 0x34, 0xca, + 0x8e, 0xab, 0xb8, 0x0e, 0x0d, 0x97, 0xa5, 0xad, 0xaf, 0x44, 0xe0, 0xbc, 0xd8, 0x2d, 0xfa, 0xb1, + 0x5b, 0xdc, 0x79, 0x5c, 0xad, 0x76, 0x89, 0x70, 0x65, 0x9d, 0x20, 0x9e, 0x4c, 0xf3, 0x59, 0x89, + 0xc0, 0x04, 0x74, 0x29, 0x4b, 0x61, 0x77, 0x8e, 0x54, 0x95, 0xbe, 0xa3, 0x1f, 0x73, 0x90, 0xde, + 0xb7, 0x31, 0xfe, 0x21, 0x26, 0x56, 0x6c, 0x37, 0x17, 0x9f, 0x27, 0x68, 0x6a, 0x04, 0xfd, 0x74, + 0x9a, 0xbf, 0x3f, 0xbf, 0x8f, 0x12, 0x80, 0x2a, 0x49, 0x6e, 0x01, 0x8a, 0xb4, 0xc4, 0x0c, 0x77, + 0x89, 0x5d, 0xb4, 0x07, 0xeb, 0x6c, 0xc7, 0xaa, 0x61, 0x39, 0x58, 0x93, 0xc3, 0x30, 0x4e, 0xcc, + 0x1f, 0xc6, 0xab, 0x14, 0xa2, 0x4a, 0x11, 0x02, 0x1e, 0x7a, 0x1b, 0x10, 0x83, 0x26, 0x09, 0x54, + 0xf6, 0x92, 0x66, 0x2e, 0x49, 0x61, 0x8b, 0x11, 0xd8, 0xc3, 0xa3, 0xa2, 0x9f, 0x6b, 0x8b, 0x91, + 0x5c, 0x5b, 0x94, 0xb0, 0xa2, 0x75, 0xd9, 0xb3, 0xc4, 0xae, 0x25, 0x42, 0x39, 0xe3, 0x57, 0xbf, + 0x8a, 0xc3, 0x5a, 0xf5, 0x80, 0x5c, 0xae, 0x84, 0x47, 0x86, 0xae, 0x2a, 0x8e, 0xef, 0x60, 0x6f, + 0xc2, 0xba, 0x86, 0x47, 0x36, 0x56, 0x15, 0x17, 0x6b, 0xb2, 0x4a, 0x65, 0x64, 0x77, 0x32, 0xc2, + 0xd4, 0xdb, 0xb2, 0x5b, 0x5f, 0x9e, 0xe5, 0x25, 0x0c, 0x83, 0x01, 0xf6, 0x26, 0x23, 0x2c, 0xad, + 0x86, 0x18, 0x21, 0x15, 0xed, 0x01, 0x8a, 0x60, 0xdb, 0x4c, 0xcb, 0xf3, 0xbe, 0x4b, 0x70, 0x9f, + 0xf2, 0xbf, 0x95, 0x10, 0xc5, 0x13, 0x41, 0x3f, 0x80, 0xe7, 0x22, 0xd0, 0xe3, 0x91, 0x16, 0x35, + 0xe1, 0xe4, 0x62, 0xb7, 0x63, 0x57, 0xb4, 0x71, 0x2b, 0x84, 0xdb, 0x65, 0x68, 0xfe, 0x49, 0x21, + 0x0c, 0x1b, 0x11, 0x5b, 0x26, 0x3e, 0x76, 0x7d, 0x43, 0x24, 0x0d, 0xf1, 0x34, 0x0d, 0xdd, 0x39, + 0x99, 0xe6, 0x6f, 0xd6, 0x02, 0xa9, 0x36, 0x3e, 0x76, 0x3d, 0x7d, 0x9a, 0x96, 0x52, 0xc1, 0x8b, + 0x74, 0x53, 0x9b, 0x29, 0xa5, 0xa1, 0x57, 0x80, 0xa7, 0xd1, 0x19, 0x9f, 0x37, 0x3a, 0x25, 0x2a, + 0x8f, 0xfa, 0x70, 0x53, 0x37, 0x5d, 0x6c, 0x9b, 0x8a, 0x21, 0x2b, 0x9a, 0x16, 0x3d, 0x86, 0xc4, + 0x95, 0x8f, 0x61, 0xcd, 0x87, 0x2a, 0x13, 0xa4, 0xe0, 0x08, 0xf6, 0xe1, 0x56, 0x60, 0xc3, 0xc6, + 0x43, 0xeb, 0x28, 0x6a, 0x25, 0x79, 0x65, 0x2b, 0xc1, 0x82, 0x25, 0x86, 0xe5, 0xdb, 0xb9, 0xc7, + 0x93, 0x12, 0x5b, 0x78, 0x8f, 0x83, 0x1b, 0x3b, 0x96, 0xa6, 0xef, 0xeb, 0x58, 0x23, 0x65, 0xdb, + 0xf7, 0xd5, 0xaf, 0x03, 0x72, 0x26, 0x8e, 0x8b, 0x87, 0xb2, 0x6a, 0x99, 0xfb, 0xfa, 0x40, 0x76, + 0x46, 0x8a, 0x49, 0xfd, 0x54, 0x90, 0x44, 0xc6, 0xa9, 0x52, 0x06, 0xad, 0xf5, 0x75, 0x40, 0xb4, + 0xb4, 0x18, 0xfa, 0x11, 0x36, 0xb1, 0xe3, 0x30, 0x69, 0xe6, 0x7d, 0x37, 0x67, 0x2c, 0x96, 0x28, + 0x49, 0x22, 0x51, 0x69, 0x79, 0x1a, 0x84, 0x52, 0x78, 0x0c, 0x62, 0xd7, 0xd5, 0xd5, 0xc3, 0x49, + 0x25, 0x2c, 0x21, 0x15, 0x00, 0x87, 0xd2, 0xe4, 0xbe, 0xee, 0x7a, 0x69, 0x79, 0xbe, 0x72, 0xee, + 0xf8, 0x50, 0x85, 0xdf, 0xc5, 0x60, 0xad, 0xe1, 0x1d, 0x43, 0xd5, 0x1a, 0x0e, 0x43, 0xf4, 0x1a, + 0x64, 0x1c, 0x52, 0xb0, 0x64, 0x97, 0x11, 0x3c, 0x03, 0xf9, 0x99, 0x6b, 0x0e, 0x0b, 0x9b, 0x94, + 0x76, 0xa2, 0x65, 0xae, 0x06, 0x99, 0x21, 0xa9, 0x24, 0x01, 0xca, 0xe2, 0x85, 0x28, 0xd1, 0x8a, + 0x23, 0xa5, 0x87, 0xd1, 0xfa, 0xf3, 0x7d, 0xb8, 0xe9, 0xe5, 0x04, 0xff, 0xba, 0x03, 0xbc, 0x18, + 0xc5, 0xbb, 0x33, 0x03, 0x6f, 0x66, 0xa6, 0x91, 0xd6, 0xd4, 0x0b, 0x12, 0xd0, 0xda, 0xd0, 0xbb, + 0x6b, 0x7a, 0x43, 0x01, 0x3e, 0x2b, 0x2b, 0x5f, 0x9d, 0xb5, 0xde, 0xa7, 0x7d, 0x43, 0xba, 0x31, + 0x9c, 0xe1, 0x30, 0xaf, 0x03, 0x0a, 0xef, 0x29, 0x00, 0x66, 0x01, 0xf6, 0xfc, 0xac, 0xe3, 0x3c, + 0x77, 0xd1, 0x92, 0xe8, 0x9c, 0xa3, 0xdc, 0xe3, 0xdf, 0x7d, 0x3f, 0xcf, 0x15, 0xfe, 0xc0, 0xc1, + 0x4a, 0xa7, 0x4f, 0x53, 0x72, 0x24, 0x9f, 0x47, 0x9a, 0x19, 0x6e, 0x8e, 0x66, 0xe6, 0x47, 0xdc, + 0x95, 0x9b, 0xc2, 0x67, 0x53, 0xdf, 0x42, 0xab, 0xf7, 0x78, 0xd2, 0xcf, 0x16, 0x4e, 0x13, 0xb0, + 0xd4, 0xb3, 0x15, 0xd3, 0x51, 0x54, 0x57, 0xb7, 0x4c, 0x54, 0x06, 0x9e, 0xf4, 0xea, 0x9e, 0xe3, + 0x3d, 0x7f, 0x59, 0x65, 0xef, 0x1d, 0x9b, 0x3b, 0xd8, 0x55, 0x2a, 0x02, 0x59, 0xd9, 0x87, 0xd3, + 0x3c, 0x27, 0x51, 0x55, 0x84, 0x80, 0x37, 0x95, 0x21, 0xeb, 0x42, 0x53, 0x12, 0x7d, 0x46, 0xf7, + 0x21, 0x41, 0x3a, 0x86, 0x31, 0x6b, 0x19, 0x66, 0xd7, 0x96, 0xc8, 0x32, 0xba, 0x54, 0x56, 0xf2, + 0x74, 0x50, 0x13, 0xb2, 0x86, 0xe2, 0xb8, 0xf2, 0x01, 0x56, 0x6c, 0xb7, 0x8f, 0x95, 0x39, 0x5b, + 0x02, 0x16, 0x78, 0x19, 0xa2, 0xfa, 0x9a, 0xaf, 0x89, 0xbe, 0x07, 0xb9, 0x81, 0x61, 0xf5, 0x15, + 0x43, 0x1e, 0x9b, 0x2a, 0xb6, 0x5d, 0x45, 0x37, 0xdd, 0x89, 0x6c, 0xe8, 0x43, 0xdd, 0xf5, 0xea, + 0xef, 0x5c, 0xa8, 0xeb, 0x0c, 0x64, 0x37, 0xc4, 0x68, 0x11, 0x08, 0xf4, 0x16, 0xdc, 0xb0, 0x3c, + 0xef, 0x08, 0xfb, 0x05, 0x27, 0x27, 0x5c, 0x98, 0x28, 0x9f, 0xf2, 0x25, 0xcf, 0x00, 0xb2, 0xce, + 0x33, 0x1c, 0x74, 0x1f, 0x80, 0x5c, 0x27, 0x0d, 0x16, 0x27, 0xb7, 0x44, 0x31, 0x2f, 0xca, 0x67, + 0x7e, 0xda, 0x21, 0x0a, 0xe4, 0xdd, 0x41, 0x05, 0xc8, 0xbc, 0x63, 0xeb, 0x2e, 0x96, 0x5d, 0xcb, + 0x92, 0x2d, 0x43, 0xcb, 0xa5, 0x69, 0xfa, 0x5c, 0xa2, 0xc4, 0x9e, 0x65, 0x75, 0x0c, 0x8d, 0x9c, + 0x34, 0xed, 0x48, 0x42, 0xe7, 0x5c, 0xbe, 0xc2, 0x49, 0x13, 0xd5, 0x30, 0x26, 0x5e, 0x86, 0x75, + 0x95, 0x66, 0xb7, 0x10, 0x4d, 0xde, 0xd7, 0x8f, 0xb1, 0x96, 0x13, 0xa9, 0xe1, 0x55, 0xc6, 0x0d, + 0x14, 0x1e, 0x10, 0x1e, 0x7a, 0x1d, 0x44, 0xdd, 0x94, 0xf7, 0x0d, 0xda, 0x1d, 0xd1, 0xa5, 0x39, + 0xb9, 0x15, 0xba, 0xd3, 0x2f, 0xcd, 0xda, 0x29, 0x7e, 0x32, 0xc6, 0xa6, 0x8a, 0xb5, 0x37, 0x88, + 0xa4, 0xb7, 0x8e, 0xac, 0x6e, 0x3e, 0xa0, 0xfa, 0x94, 0xe8, 0x20, 0x0b, 0x96, 0xf5, 0x81, 0x69, + 0xd9, 0x24, 0xcd, 0xe0, 0x27, 0xe6, 0x78, 0xe8, 0xe4, 0x10, 0x45, 0x2c, 0x5e, 0xe6, 0xde, 0x0d, + 0xa6, 0xd2, 0xc5, 0x4f, 0xda, 0xe3, 0x21, 0xad, 0xbf, 0x61, 0x07, 0x7b, 0x86, 0xe7, 0x48, 0x59, + 0x3d, 0x78, 0x27, 0xe8, 0x67, 0x3e, 0x17, 0x63, 0x22, 0xdf, 0xe4, 0x85, 0x84, 0x98, 0x6c, 0xf2, + 0x42, 0x4a, 0x84, 0x26, 0x2f, 0x64, 0xc4, 0x6c, 0x93, 0x17, 0xb2, 0xe2, 0x72, 0xe1, 0x37, 0x3c, + 0xac, 0x44, 0xbc, 0x5e, 0xc2, 0xaa, 0x65, 0x6b, 0xcf, 0x22, 0x04, 0xbf, 0x38, 0xe1, 0x76, 0x3d, + 0x97, 0xfd, 0x5f, 0x70, 0x06, 0xde, 0x73, 0x84, 0x45, 0x31, 0x76, 0xce, 0x1d, 0x92, 0xa2, 0xd0, + 0xe4, 0x05, 0x41, 0x4c, 0x05, 0xae, 0x01, 0xe2, 0x52, 0x93, 0x17, 0xd2, 0x62, 0x26, 0xea, 0x26, + 0x4d, 0x5e, 0x58, 0x16, 0xc5, 0x26, 0x2f, 0x88, 0xe2, 0x4a, 0x61, 0xca, 0x41, 0x82, 0xf4, 0x0e, + 0x26, 0xf9, 0x3c, 0x59, 0x76, 0x74, 0x73, 0x60, 0x60, 0xf9, 0x10, 0x4f, 0xc2, 0x86, 0x68, 0x69, + 0xeb, 0x85, 0x19, 0x67, 0xc3, 0x74, 0x8a, 0x5d, 0xaa, 0xf0, 0x10, 0x4f, 0xe8, 0x79, 0x87, 0x6e, + 0x93, 0x71, 0xa2, 0x0c, 0xf4, 0x2d, 0x88, 0xb9, 0xc7, 0x7e, 0xc7, 0x34, 0x97, 0x07, 0xb2, 0xc3, + 0x26, 0x5a, 0x1b, 0xf7, 0x21, 0x73, 0xc6, 0xcc, 0x25, 0xa3, 0x97, 0xe8, 0x38, 0xa5, 0xc9, 0x0b, + 0xbc, 0x18, 0x2f, 0xfc, 0x89, 0x83, 0xe5, 0x96, 0xa5, 0x1e, 0x96, 0xd5, 0x27, 0x63, 0xdd, 0xd1, + 0x69, 0x51, 0xfa, 0x26, 0xf0, 0x91, 0xed, 0x5d, 0xe8, 0x3e, 0x91, 0x28, 0x70, 0xae, 0xbb, 0x0b, + 0x24, 0x01, 0x68, 0x63, 0x5b, 0xe9, 0xeb, 0x86, 0xee, 0xb2, 0xb5, 0x67, 0xb7, 0xb6, 0x2e, 0xf8, + 0x32, 0x8b, 0x4c, 0xcc, 0x8a, 0xc4, 0x79, 0x8b, 0xb5, 0x40, 0x53, 0x8a, 0xa0, 0x14, 0x7e, 0xbd, + 0x08, 0x40, 0xf6, 0xc6, 0x3e, 0x33, 0xfe, 0x2b, 0xdb, 0x0a, 0x33, 0x43, 0xec, 0x33, 0x64, 0x86, + 0x19, 0xc1, 0xc3, 0xff, 0x27, 0x83, 0xa7, 0xf0, 0x53, 0x0e, 0xb2, 0x67, 0xc3, 0xfa, 0xb2, 0x01, + 0xd8, 0xdb, 0x20, 0x38, 0x9e, 0xb0, 0x37, 0x53, 0xfa, 0xce, 0xdc, 0x1d, 0xd3, 0xf9, 0xa1, 0x23, + 0x39, 0xbc, 0x2e, 0x7e, 0x22, 0x05, 0x88, 0x5e, 0xc3, 0xf4, 0x7b, 0x1e, 0xe2, 0x2d, 0xac, 0x38, + 0x18, 0x4d, 0x20, 0xce, 0xe6, 0x13, 0xdc, 0xe7, 0xd7, 0xbf, 0x31, 0x8b, 0xe8, 0xdb, 0x00, 0xf8, + 0x78, 0xa4, 0xdb, 0x0a, 0xb9, 0xa3, 0xb9, 0xfa, 0x47, 0x29, 0xa2, 0x80, 0x6a, 0x90, 0xf4, 0x3f, + 0xc9, 0x63, 0x57, 0xfe, 0x24, 0xf7, 0x55, 0xd1, 0x2e, 0x44, 0x3e, 0x68, 0xd9, 0xa8, 0x86, 0xfc, + 0x75, 0x74, 0x7f, 0x2e, 0xf4, 0x6f, 0x56, 0xb4, 0x16, 0x6a, 0xd3, 0x79, 0x4b, 0x97, 0xea, 0xa2, + 0x9f, 0x70, 0xb0, 0x34, 0xb2, 0xad, 0x11, 0x1b, 0xb9, 0x38, 0xf3, 0xd5, 0x9e, 0xf6, 0xc9, 0x34, + 0x0f, 0x8f, 0x3c, 0xad, 0x5e, 0xf7, 0xda, 0xe7, 0x0c, 0xfe, 0x0a, 0x7a, 0x0e, 0x5a, 0x85, 0x38, + 0x1e, 0x59, 0xea, 0x01, 0x9d, 0xfa, 0xc4, 0x24, 0xf6, 0x82, 0x5e, 0x8c, 0xf8, 0x1a, 0xe9, 0x1b, + 0x63, 0x95, 0x95, 0xd3, 0x69, 0x3e, 0x43, 0x5d, 0xc3, 0xf7, 0xd8, 0x88, 0xf3, 0x04, 0x2d, 0x41, + 0xe1, 0x2f, 0x1c, 0x64, 0xcb, 0x7d, 0xcb, 0x76, 0x49, 0x70, 0xd7, 0x4d, 0xd7, 0x9e, 0x5c, 0xe6, + 0xd2, 0xd7, 0x9f, 0x1e, 0x23, 0x05, 0x84, 0x91, 0xad, 0x5b, 0xb6, 0x9f, 0xc7, 0xe2, 0x95, 0xfa, + 0xe9, 0x34, 0x5f, 0xfe, 0xcc, 0x51, 0xf1, 0xc8, 0x03, 0x93, 0x02, 0xd8, 0x7b, 0x02, 0xd9, 0xd9, + 0xa7, 0x64, 0x77, 0xff, 0x5c, 0x84, 0x95, 0x16, 0x56, 0xf6, 0x7b, 0xc7, 0x66, 0xc3, 0x1c, 0x8d, + 0xc9, 0xa5, 0xba, 0x18, 0xbd, 0xc2, 0x52, 0x16, 0x0b, 0x94, 0xcd, 0xcb, 0x53, 0x4e, 0x34, 0x5b, + 0xbd, 0x00, 0xcb, 0x36, 0xde, 0xb7, 0xb1, 0x73, 0x20, 0xeb, 0xe6, 0x91, 0x62, 0xe8, 0x1a, 0x3d, + 0x6b, 0x41, 0xca, 0x7a, 0xe4, 0x06, 0xa3, 0xce, 0x6c, 0x14, 0x84, 0xeb, 0x35, 0x0a, 0x5b, 0xb0, + 0xe6, 0xb8, 0x78, 0x34, 0xd2, 0xcd, 0x81, 0x3c, 0x24, 0xdf, 0x76, 0xd8, 0x54, 0xfa, 0x06, 0xd6, + 0x72, 0x29, 0xba, 0x82, 0x1b, 0x3e, 0x73, 0xc7, 0xd2, 0x70, 0x9d, 0xb1, 0x50, 0x1f, 0xd2, 0x6c, + 0xa0, 0x87, 0x9f, 0xc8, 0xe6, 0x78, 0x98, 0x83, 0x67, 0x94, 0x84, 0x80, 0xa0, 0xb2, 0x1c, 0x79, + 0xae, 0x93, 0xe0, 0xc5, 0x78, 0x93, 0x17, 0xe2, 0x62, 0x82, 0x75, 0x15, 0x85, 0xf7, 0xc2, 0xf3, + 0x7f, 0xa0, 0x9b, 0x8a, 0x71, 0xbd, 0xf3, 0x7f, 0x15, 0x72, 0xd1, 0x11, 0xa1, 0x35, 0x1c, 0x2a, + 0x26, 0xf9, 0x3f, 0x36, 0x5d, 0xe6, 0x4a, 0x52, 0x64, 0x84, 0x58, 0x65, 0xec, 0x2a, 0xe1, 0xa2, + 0x0a, 0x64, 0xfc, 0x9b, 0x63, 0xad, 0x1f, 0x3f, 0x4f, 0xeb, 0x97, 0xf6, 0x74, 0x58, 0xf7, 0x37, + 0xef, 0xed, 0x07, 0x47, 0x12, 0x1c, 0x03, 0x6b, 0xab, 0x0a, 0xff, 0xe0, 0x60, 0xb9, 0x6a, 0xe8, + 0xd8, 0x74, 0x69, 0xd5, 0x69, 0x98, 0xfb, 0x16, 0x7a, 0x0d, 0xd6, 0xb4, 0x20, 0xa5, 0xc9, 0x03, + 0x6c, 0x62, 0x2f, 0x8b, 0x72, 0x34, 0x88, 0x6f, 0x9c, 0x4e, 0xf3, 0xcb, 0x54, 0x7a, 0x3b, 0x60, + 0x49, 0xab, 0xa1, 0x46, 0x48, 0x45, 0xaf, 0x42, 0xd6, 0x20, 0xd1, 0x2e, 0x9f, 0xa9, 0x39, 0x33, + 0xf3, 0x40, 0xc6, 0x88, 0xbe, 0x22, 0x0c, 0x37, 0xcf, 0x8f, 0x94, 0xe5, 0x91, 0x65, 0xe8, 0xaa, + 0xdf, 0x68, 0xbc, 0x78, 0xd1, 0x08, 0xf0, 0xdc, 0x1c, 0xf9, 0x11, 0x55, 0x92, 0xd6, 0xd4, 0x59, + 0xe4, 0xc2, 0xdf, 0x39, 0x48, 0x85, 0x1b, 0xbf, 0xef, 0x0d, 0x19, 0xaf, 0xfa, 0x53, 0x02, 0x1b, + 0x35, 0xbe, 0x0c, 0x71, 0xba, 0x87, 0x4b, 0x7e, 0x5b, 0xa2, 0x5b, 0xf6, 0x7f, 0x5b, 0xa2, 0xc2, + 0x9f, 0xd3, 0x46, 0xd9, 0xec, 0xf0, 0xee, 0x47, 0x1c, 0xa4, 0xe8, 0xef, 0x5b, 0x74, 0x02, 0xbd, + 0x04, 0xc9, 0xdd, 0xf6, 0xc3, 0x76, 0xe7, 0x8d, 0xb6, 0xb8, 0x80, 0x92, 0x10, 0x6b, 0xb4, 0x7b, + 0x22, 0x87, 0x52, 0x10, 0x7f, 0xd0, 0xea, 0x94, 0x7b, 0xe2, 0x22, 0x79, 0xac, 0xec, 0xf5, 0xea, + 0x5d, 0x31, 0x86, 0x6e, 0xc0, 0x72, 0xad, 0xde, 0x6a, 0xec, 0x34, 0x7a, 0xf5, 0x9a, 0xcc, 0x88, + 0x02, 0x12, 0x80, 0xef, 0x35, 0x76, 0xea, 0x22, 0x4f, 0xa0, 0x6a, 0xf5, 0x6a, 0x63, 0xa7, 0xdc, + 0x12, 0xe3, 0x68, 0x0d, 0x56, 0x42, 0x59, 0x9f, 0x9c, 0x42, 0x69, 0x10, 0x6a, 0xbb, 0x52, 0xb9, + 0xd7, 0xe8, 0xb4, 0xc5, 0x04, 0x02, 0x48, 0x10, 0xdd, 0xde, 0x9b, 0x62, 0x9a, 0xd8, 0xde, 0xae, + 0x77, 0xc4, 0x0c, 0x35, 0xd8, 0xf9, 0xee, 0x56, 0x4d, 0xcc, 0x92, 0xc7, 0xde, 0xee, 0xa3, 0x56, + 0x5d, 0x04, 0xa2, 0x58, 0x69, 0xf4, 0xca, 0x92, 0x54, 0xde, 0x13, 0x97, 0x50, 0x16, 0x80, 0x28, + 0x76, 0xeb, 0x52, 0xa3, 0xde, 0x15, 0xb5, 0x02, 0xf9, 0x48, 0x48, 0xde, 0x55, 0x61, 0xe5, 0xa9, + 0xc1, 0x3b, 0xca, 0x40, 0xaa, 0x5c, 0xab, 0xc9, 0x8f, 0x3b, 0xbd, 0xba, 0x24, 0x2e, 0x20, 0x11, + 0xd2, 0x52, 0x7d, 0xa7, 0xf3, 0xb8, 0xee, 0x51, 0x38, 0xb4, 0x02, 0x19, 0x22, 0xd0, 0xee, 0xb4, + 0x3d, 0xd2, 0x22, 0x5a, 0x05, 0xd1, 0x13, 0x0a, 0xa9, 0xb1, 0x0d, 0xfe, 0xdd, 0x5f, 0x6c, 0x2e, + 0xdc, 0xed, 0x9c, 0xf9, 0x16, 0x65, 0x8d, 0x1f, 0xd9, 0xfa, 0xa3, 0x7a, 0xbb, 0xd6, 0x68, 0x6f, + 0x8b, 0x0b, 0xe4, 0xa5, 0xdb, 0x2b, 0x6f, 0x93, 0x97, 0x18, 0x31, 0x5f, 0xed, 0xec, 0xec, 0x34, + 0x7a, 0xbd, 0x7a, 0x4d, 0xe4, 0x08, 0xaf, 0x5c, 0xe9, 0x48, 0xe4, 0x65, 0xd1, 0x03, 0xdc, 0x83, + 0x8d, 0x8b, 0xaf, 0x12, 0x6d, 0xc0, 0x7a, 0xab, 0xbc, 0x2d, 0x57, 0xf6, 0xe4, 0x6a, 0x6b, 0xb7, + 0xdb, 0xab, 0x4b, 0x72, 0xb7, 0xde, 0xeb, 0x31, 0x43, 0xb7, 0x60, 0xad, 0x55, 0x2f, 0xd7, 0xe4, + 0x07, 0x1d, 0x49, 0xde, 0x6e, 0x75, 0x2a, 0xe5, 0x96, 0x2c, 0xd5, 0xcb, 0xb5, 0xae, 0xc8, 0x31, + 0xe8, 0xca, 0xd7, 0x3e, 0xf8, 0xdb, 0xe6, 0xc2, 0x07, 0x27, 0x9b, 0xdc, 0x87, 0x27, 0x9b, 0xdc, + 0xc7, 0x27, 0x9b, 0xdc, 0x5f, 0x4f, 0x36, 0xb9, 0x9f, 0x7d, 0xb2, 0xb9, 0xf0, 0xe1, 0x27, 0x9b, + 0x0b, 0x1f, 0x7f, 0xb2, 0xb9, 0xf0, 0x66, 0xd2, 0xf3, 0xa6, 0x7e, 0x82, 0xfe, 0x0c, 0xfc, 0xd2, + 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xd5, 0x23, 0x4d, 0x2f, 0x0a, 0x1f, 0x00, 0x00, } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index d0789f67e8ff..e55eb7379a6a 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -13,6 +13,7 @@ package cockroach.roachpb; option go_package = "roachpb"; import "kv/kvserver/concurrency/lock/locking.proto"; +import "kv/kvserver/readsummary/rspb/summary.proto"; import "roachpb/metadata.proto"; import "storage/enginepb/mvcc.proto"; import "storage/enginepb/mvcc3.proto"; @@ -147,7 +148,8 @@ message MergeTrigger { // timestamps at which any requests were serviced by the right-hand side range // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water - // mark for the keys previously owned by the subsumed range. + // mark for the keys previously owned by the subsumed range, though this role + // is largely being... subsumed by the RightReadSummary. util.hlc.Timestamp freeze_start = 5 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; @@ -165,6 +167,35 @@ message MergeTrigger { // the two sides are collocated at merge time, we don't need to use the // read_summary and simply use this field. util.hlc.Timestamp right_closed_timestamp = 6 [(gogoproto.nullable) = false]; + + // RightReadSummary is a summary of the reads that have been performed on the + // right-hand side up to the point of the Subsume request, which serializes + // with past reads and begins blocking future reads. It is suitable for use to + // update the timestamp cache for the keys previously owned by the subsumed + // range. + // + // RightReadSummary can be used in place of FreezeStart, when available. It + // has two distinct advantages: + // 1. it can transfer a higher-resolution snapshot of the reads on the range + // through a range merge, to make the merge less disruptive to writes on + // the post-merge range because the timestamp cache won't be bumped as + // high. + // 2. it can transfer information about reads with synthetic timestamps, which + // are not otherwise captured by the FreezeStart clock timestamp. + // + // When a RightReadSummary is set in ReplicatedEvalResult.Merge.Trigger, there + // is always also a write to the RangePriorReadSummaryKey in the corresponding + // RaftCommand.WriteBatch. The persisted summary may be identical to the + // summary in this field, but it does not have to be. Notably, we intended for + // the summary included in the ReplicatedEvalResult.Merge.Trigger to + // eventually be a much higher-resolution version of the ReadSummmary than the + // version persisted. This scheme of persisting a compressed ReadSummary + // indefinitely and including a higher-resolution ReadSummary on the + // RaftCommand allows us to optimize for the common case where the merge is + // applied on the LHS's leaseholder through Raft log application while + // ensuring correctness in the case where the merge is applied on the LHS's + // leaseholder through a Raft snapshot. + kv.kvserver.readsummary.ReadSummary right_read_summary = 7; } // ReplicaChangeType is a parameter of ChangeReplicasTrigger.