diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index 237467a83db6..d5fec0a95082 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -100,6 +100,6 @@
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-24 | set the active cluster version in the format '.' |
+version | version | 20.2-26 | set the active cluster version in the format '.' |
diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go
index 726a337c6487..4ee53eac1ba2 100644
--- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go
+++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go
@@ -53,6 +53,8 @@ var followerReadMultiple = settings.RegisterFloatSetting(
// the offset from now to request a follower read. The same value less the clock
// uncertainty, then is used to determine at the kv layer if a query can use a
// follower read.
+// TODO(nvanbenschoten): don't route to followers if closedts.TargetDuration is
+// 0, as this disables closed timestamps.
func getFollowerReadDuration(st *cluster.Settings) time.Duration {
targetMultiple := followerReadMultiple.Get(&st.SV)
targetDuration := closedts.TargetDuration.Get(&st.SV)
diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go
index 4be5ed066e8a..49de2be05d5d 100644
--- a/pkg/clusterversion/cockroach_versions.go
+++ b/pkg/clusterversion/cockroach_versions.go
@@ -235,6 +235,10 @@ const (
// using the replicated legacy TruncatedState. It's also used in asserting
// that no replicated truncated state representation is found.
PostTruncatedAndRangeAppliedStateMigration
+ // PriorReadSummaries introduces support for the use of read summary objects
+ // ship information about reads on a range through lease changes and range
+ // merges.
+ PriorReadSummaries
// Step (1): Add new versions here.
)
@@ -389,6 +393,10 @@ var versionsSingleton = keyedVersions([]keyedVersion{
Key: PostTruncatedAndRangeAppliedStateMigration,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 24},
},
+ {
+ Key: PriorReadSummaries,
+ Version: roachpb.Version{Major: 20, Minor: 2, Internal: 26},
+ },
// Step (2): Add new versions here.
})
diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go
index 4c84f52af97f..e7c9e55459ae 100644
--- a/pkg/clusterversion/key_string.go
+++ b/pkg/clusterversion/key_string.go
@@ -40,11 +40,12 @@ func _() {
_ = x[LongRunningMigrations-29]
_ = x[TruncatedAndRangeAppliedStateMigration-30]
_ = x[PostTruncatedAndRangeAppliedStateMigration-31]
+ _ = x[PriorReadSummaries-32]
}
-const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigration"
+const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesNodeMembershipStatusMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsreplacedTruncatedAndRangeAppliedStateMigrationreplacedPostTruncatedAndRangeAppliedStateMigrationNewSchemaChangerLongRunningMigrationsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigrationPriorReadSummaries"
-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}
+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, 727}
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 8fc8d3b175f4..45f233f0dd91 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.
@@ -951,6 +957,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 402d71680912..59a1cefe68c6 100644
--- a/pkg/kv/kvserver/BUILD.bazel
+++ b/pkg/kv/kvserver/BUILD.bazel
@@ -132,6 +132,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..083ff4d35b2b 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,6 +108,8 @@ 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",
diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go
index c999672787ce..17cde401bb08 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,12 @@ func declareKeysEndTxn(
Key: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID),
EndKey: keys.MakeRangeIDReplicatedPrefix(mt.RightDesc.RangeID).PrefixEnd(),
})
+ // Merges the prior read summary from the RHS over to 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 +1110,30 @@ 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.
+ 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..7ace374b57ea 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,15 @@ func evalNewLease(
}
pd.Replicated.PrevLeaseProposal = prevLease.ProposedTS
+ // If we're setting a new prior read summary, store it to disk & in-memory.
+ 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 fcfa4c18b384..47340053f08b 100644
--- a/pkg/kv/kvserver/batcheval/cmd_lease_test.go
+++ b/pkg/kv/kvserver/batcheval/cmd_lease_test.go
@@ -16,6 +16,8 @@ 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/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
@@ -171,53 +173,78 @@ 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.NewDefaultInMem()
- 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.NewDefaultInMem()
+ 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{
+ 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 cc7b44015ff1..6fe48dce5b7f 100644
--- a/pkg/kv/kvserver/batcheval/cmd_subsume.go
+++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go
@@ -151,6 +151,8 @@ func Subsume(
reply.MVCCStats = cArgs.EvalCtx.GetMVCCStats()
reply.LeaseAppliedIndex = lai
reply.FreezeStart = cArgs.EvalCtx.Clock().NowAsClockTimestamp()
+ sum := cArgs.EvalCtx.GetCurrentReadSummary()
+ reply.ReadSummary = &sum
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 fa59268280a6..74f820b64435 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"
@@ -97,6 +98,13 @@ type EvalContext interface {
GetLease() (roachpb.Lease, roachpb.Lease)
GetRangeInfo(context.Context) roachpb.RangeInfo
+ // 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,
error)
@@ -105,17 +113,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
@@ -213,13 +222,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 40fa64c6df9c..de961970bd77 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 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 f2bfb7231e7d..0d94f2514a9f 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 collacted 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 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,234 @@ 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 allowed 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,
+ })
+ }
+
+ // 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 +799,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
@@ -3325,8 +3634,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 b53ea45ba111..259d33f67a43 100644
--- a/pkg/kv/kvserver/client_replica_test.go
+++ b/pkg/kv/kvserver/client_replica_test.go
@@ -2029,126 +2029,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 8b43941018d9..b221b1646c63 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_0c8837b323bf7b92, []int{0}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []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_0c8837b323bf7b92, []int{1}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []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_0c8837b323bf7b92, []int{2}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []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_0c8837b323bf7b92, []int{3}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []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_0c8837b323bf7b92, []int{4}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []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_0c8837b323bf7b92, []int{5}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []int{5}
}
func (m *SuggestedCompaction) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -299,13 +300,28 @@ 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.
+ 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_0c8837b323bf7b92, []int{6}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []int{6}
}
func (m *ReplicatedEvalResult) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -349,7 +365,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_0c8837b323bf7b92, []int{6, 0}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []int{6, 0}
}
func (m *ReplicatedEvalResult_AddSSTable) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -386,7 +402,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_0c8837b323bf7b92, []int{7}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []int{7}
}
func (m *WriteBatch) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -423,7 +439,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_0c8837b323bf7b92, []int{8}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []int{8}
}
func (m *LogicalOpLog) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -529,7 +545,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_0c8837b323bf7b92, []int{9}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []int{9}
}
func (m *RaftCommand) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -568,7 +584,7 @@ func (m *RaftCommandFooter) Reset() { *m = RaftCommandFooter{} }
func (m *RaftCommandFooter) String() string { return proto.CompactTextString(m) }
func (*RaftCommandFooter) ProtoMessage() {}
func (*RaftCommandFooter) Descriptor() ([]byte, []int) {
- return fileDescriptor_proposer_kv_0c8837b323bf7b92, []int{10}
+ return fileDescriptor_proposer_kv_004803af804b8ced, []int{10}
}
func (m *RaftCommandFooter) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1141,6 +1157,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
}
@@ -1251,11 +1279,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
@@ -1265,30 +1293,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))
@@ -1495,6 +1523,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
}
@@ -2685,6 +2717,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:])
@@ -3478,98 +3543,102 @@ var (
)
func init() {
- proto.RegisterFile("kv/kvserver/kvserverpb/proposer_kv.proto", fileDescriptor_proposer_kv_0c8837b323bf7b92)
-}
-
-var fileDescriptor_proposer_kv_0c8837b323bf7b92 = []byte{
- // 1424 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4f, 0x6f, 0xdb, 0x46,
- 0x16, 0xb7, 0x2c, 0xc9, 0xa6, 0x9e, 0x6c, 0x89, 0x9e, 0x38, 0x09, 0xd7, 0xbb, 0x2b, 0x19, 0xda,
- 0x6c, 0xe0, 0xdd, 0xcd, 0x52, 0x81, 0xbd, 0x0b, 0x14, 0x49, 0x50, 0xc4, 0x92, 0x93, 0xc6, 0x8a,
- 0xed, 0x26, 0x23, 0x27, 0x2d, 0xd2, 0x03, 0x31, 0x22, 0x27, 0x14, 0x2b, 0x8a, 0x64, 0x86, 0x23,
- 0x25, 0xfe, 0x14, 0x6d, 0x81, 0x1e, 0x7a, 0x6a, 0x73, 0xec, 0xd7, 0xe8, 0x2d, 0x97, 0x02, 0x39,
- 0x06, 0x3d, 0x08, 0x8d, 0x73, 0xe9, 0x67, 0xc8, 0xa9, 0x98, 0xe1, 0x50, 0x92, 0x0b, 0xa7, 0x56,
- 0xda, 0xdb, 0xf0, 0xcd, 0xbc, 0xdf, 0x7b, 0xf3, 0xfe, 0xfc, 0xde, 0x10, 0x36, 0x7a, 0xc3, 0x7a,
- 0x6f, 0x18, 0x53, 0x36, 0xa4, 0x6c, 0xbc, 0x88, 0x3a, 0xf5, 0x88, 0x85, 0x51, 0x18, 0x53, 0x66,
- 0xf5, 0x86, 0x66, 0xc4, 0x42, 0x1e, 0xa2, 0xaa, 0x1d, 0xda, 0x3d, 0x16, 0x12, 0xbb, 0x6b, 0xf6,
- 0x86, 0x66, 0x7a, 0xd4, 0x8c, 0x79, 0xc8, 0x88, 0x4b, 0xa3, 0xce, 0xda, 0x8a, 0xdc, 0x8c, 0x3a,
- 0x75, 0x12, 0x79, 0x89, 0xce, 0x1a, 0x4a, 0x45, 0x0e, 0xe1, 0x44, 0xc9, 0x2e, 0xa4, 0xb2, 0x3e,
- 0xe5, 0x64, 0x4a, 0xfe, 0x57, 0x85, 0x54, 0xa7, 0x81, 0xeb, 0x05, 0x54, 0x1c, 0x18, 0xda, 0xb6,
- 0xda, 0xfc, 0xdb, 0xa9, 0x9b, 0x5b, 0x6a, 0xb7, 0xf6, 0x8e, 0x4b, 0xc4, 0x9c, 0x70, 0xaa, 0xce,
- 0x18, 0x03, 0xee, 0xf9, 0xf5, 0xae, 0x6f, 0xd7, 0xb9, 0xd7, 0xa7, 0x31, 0x27, 0xfd, 0x48, 0xed,
- 0xac, 0xba, 0xa1, 0x1b, 0xca, 0x65, 0x5d, 0xac, 0x12, 0x69, 0xed, 0xfb, 0x0c, 0xe4, 0xdb, 0x91,
- 0xef, 0x71, 0xd4, 0x84, 0x45, 0xce, 0x3c, 0xd7, 0xa5, 0xcc, 0xc8, 0xac, 0x67, 0x36, 0x8a, 0x9b,
- 0x55, 0x73, 0x12, 0x0a, 0x75, 0x19, 0x53, 0x1e, 0x3d, 0x4c, 0x8e, 0x35, 0xb4, 0x17, 0xa3, 0xea,
- 0xdc, 0xcb, 0x51, 0x35, 0x83, 0x53, 0x4d, 0x74, 0x08, 0x05, 0xd6, 0x8d, 0x2d, 0x87, 0xfa, 0x9c,
- 0x18, 0xf3, 0x12, 0xe6, 0x9f, 0x53, 0x30, 0xea, 0x7a, 0x66, 0x7a, 0x3d, 0x73, 0xff, 0x61, 0xb3,
- 0xd9, 0xe6, 0x84, 0xc7, 0x0d, 0x5d, 0x80, 0x1d, 0x8f, 0xaa, 0x1a, 0xbe, 0xd3, 0xde, 0x11, 0xea,
- 0x58, 0x63, 0xdd, 0x58, 0xae, 0xae, 0xe5, 0x7e, 0x79, 0x5e, 0xcd, 0xd4, 0x30, 0xe4, 0xf7, 0x29,
- 0x73, 0xe9, 0x6c, 0x9e, 0xca, 0xa3, 0xef, 0xf6, 0x54, 0x61, 0x3a, 0x50, 0x6a, 0x76, 0x49, 0xe0,
- 0x52, 0x4c, 0x23, 0xdf, 0xb3, 0x49, 0x8c, 0xf6, 0x7e, 0x0b, 0xbe, 0x71, 0x0a, 0xf8, 0x49, 0x9d,
- 0xdf, 0xb3, 0xf2, 0xcd, 0xf3, 0xea, 0x5c, 0xed, 0xf5, 0x3c, 0x94, 0x9b, 0x61, 0x3f, 0x1a, 0x70,
- 0xda, 0xec, 0x52, 0xbb, 0x17, 0x0f, 0xfa, 0xe8, 0x73, 0x28, 0xda, 0x6a, 0x6d, 0x79, 0x8e, 0xb4,
- 0xb5, 0xd4, 0xd8, 0x15, 0x08, 0x3f, 0x8d, 0xaa, 0x5b, 0xae, 0xc7, 0xbb, 0x83, 0x8e, 0x69, 0x87,
- 0xfd, 0xfa, 0xd8, 0xba, 0xd3, 0x99, 0xac, 0xeb, 0x51, 0xcf, 0xad, 0xcb, 0x54, 0x0f, 0x06, 0x9e,
- 0x63, 0x3e, 0x78, 0xb0, 0xbb, 0x73, 0x3c, 0xaa, 0x42, 0x8a, 0xbe, 0xbb, 0x83, 0x21, 0x45, 0xdf,
- 0x75, 0xd0, 0x3f, 0x60, 0x39, 0x26, 0x43, 0x6a, 0xc5, 0x01, 0x89, 0xe2, 0x6e, 0xc8, 0x65, 0x66,
- 0x34, 0xbc, 0x24, 0x84, 0x6d, 0x25, 0x43, 0x5b, 0x90, 0xeb, 0x87, 0x0e, 0x35, 0xb2, 0xeb, 0x99,
- 0x8d, 0xd2, 0xa9, 0x21, 0x4d, 0xd1, 0xf7, 0x43, 0x87, 0x62, 0x79, 0x18, 0x55, 0x20, 0xb1, 0x13,
- 0x85, 0x5e, 0xc0, 0x8d, 0x9c, 0x84, 0x9d, 0x92, 0x20, 0x03, 0x16, 0x87, 0x94, 0xc5, 0x5e, 0x18,
- 0x18, 0xf9, 0xf5, 0xcc, 0xc6, 0x32, 0x4e, 0x3f, 0xd1, 0x1d, 0x28, 0x70, 0xca, 0xfa, 0x5e, 0x40,
- 0x38, 0x35, 0x16, 0xd6, 0xb3, 0x1b, 0xc5, 0xcd, 0x4b, 0xa7, 0xd8, 0x54, 0x31, 0xde, 0xa1, 0xb1,
- 0xcd, 0xbc, 0x88, 0x87, 0xac, 0x91, 0x13, 0x31, 0xc2, 0x13, 0x65, 0x95, 0xc9, 0x87, 0x00, 0x22,
- 0xc4, 0xc4, 0xe6, 0x02, 0x7d, 0x15, 0xf2, 0x9d, 0x23, 0x4e, 0x63, 0x19, 0xd7, 0x2c, 0x4e, 0x3e,
- 0xd0, 0x15, 0x40, 0xf1, 0xc0, 0x75, 0x69, 0xcc, 0xa9, 0x63, 0x11, 0x6e, 0x05, 0x24, 0x08, 0x63,
- 0x19, 0x8c, 0x2c, 0xd6, 0xc7, 0x3b, 0xdb, 0xfc, 0x40, 0xc8, 0x15, 0xee, 0xd7, 0xf3, 0x70, 0xae,
- 0x9d, 0x6e, 0x4d, 0x59, 0xb8, 0x0f, 0x85, 0x98, 0x13, 0xc6, 0xad, 0x1e, 0x3d, 0x52, 0xd9, 0xfb,
- 0xdf, 0xdb, 0x51, 0xf5, 0xea, 0x4c, 0x99, 0x4b, 0x6f, 0x77, 0x97, 0x1e, 0x61, 0x4d, 0xc2, 0xdc,
- 0xa5, 0x47, 0x68, 0x1f, 0x16, 0x69, 0xe0, 0x48, 0xc0, 0xf9, 0x3f, 0x01, 0xb8, 0x40, 0x03, 0x47,
- 0xc0, 0x3d, 0x00, 0xb0, 0xc7, 0xfe, 0xca, 0xb4, 0x16, 0x37, 0xff, 0x63, 0x9e, 0x41, 0x6f, 0xe6,
- 0xe4, 0x8a, 0x53, 0xf5, 0x3c, 0x05, 0xa4, 0xc2, 0xf2, 0x83, 0x06, 0xab, 0x2a, 0x37, 0x9c, 0x3a,
- 0xb7, 0x86, 0xc4, 0xc7, 0x34, 0x1e, 0xf8, 0x82, 0x46, 0xf2, 0x92, 0x8f, 0x54, 0xf7, 0xff, 0xf7,
- 0x4c, 0x83, 0x0a, 0x45, 0xb0, 0x00, 0xc5, 0x89, 0x2e, 0xba, 0x01, 0xf9, 0x58, 0x30, 0x8d, 0xf2,
- 0xfa, 0xf2, 0x99, 0x20, 0x92, 0x97, 0x70, 0xa2, 0x24, 0xb4, 0xfb, 0xa2, 0xfb, 0x65, 0x3d, 0xce,
- 0xa2, 0x2d, 0xb9, 0x02, 0x27, 0x4a, 0x68, 0x03, 0x74, 0x2f, 0xb6, 0x7c, 0x4a, 0x62, 0x6a, 0x31,
- 0xfa, 0x64, 0x40, 0x63, 0x6e, 0x2c, 0xc8, 0xc2, 0x2e, 0x79, 0xf1, 0x9e, 0x10, 0xe3, 0x44, 0x8a,
- 0xb6, 0xa1, 0x30, 0x26, 0x59, 0x43, 0x93, 0xb6, 0xfe, 0x3e, 0x65, 0x4b, 0xb4, 0xa7, 0xd9, 0xf5,
- 0x6d, 0xf3, 0x30, 0x3d, 0x34, 0xae, 0xdd, 0x54, 0x80, 0xee, 0x81, 0xee, 0xd0, 0x88, 0x51, 0x19,
- 0x45, 0x45, 0x9b, 0xf0, 0x1e, 0xb4, 0x89, 0xcb, 0x13, 0x75, 0xc9, 0x95, 0xe8, 0x53, 0x28, 0xdb,
- 0x92, 0x9d, 0x2c, 0xa6, 0xe8, 0xc9, 0x58, 0x92, 0x80, 0xf5, 0xb3, 0x53, 0x7f, 0x82, 0xd5, 0x70,
- 0xc9, 0x3e, 0xc9, 0x8c, 0x97, 0xa0, 0xc4, 0xc8, 0x63, 0x6e, 0xf9, 0xa1, 0xab, 0x3c, 0x5d, 0x96,
- 0x9d, 0xb3, 0x24, 0xa4, 0x7b, 0xa1, 0x9b, 0xd8, 0x7f, 0x02, 0x45, 0xe2, 0x38, 0x56, 0x1c, 0x73,
- 0xd2, 0xf1, 0xa9, 0xb1, 0x22, 0x6d, 0xdf, 0x9c, 0xb5, 0x0a, 0x4e, 0xd4, 0x92, 0xb9, 0xed, 0x38,
- 0xed, 0xf6, 0xa1, 0xc0, 0x69, 0x94, 0x04, 0xbd, 0x4d, 0xbe, 0x31, 0x10, 0xc7, 0x69, 0x27, 0x36,
- 0xd0, 0x6d, 0xc8, 0x27, 0xfe, 0x20, 0x69, 0xec, 0xdf, 0x33, 0x45, 0x4e, 0x7a, 0xab, 0x12, 0x92,
- 0xa8, 0xa3, 0x2f, 0x32, 0x70, 0x2e, 0x62, 0x74, 0xa8, 0x92, 0x9f, 0xbc, 0x0d, 0x88, 0x6f, 0xac,
- 0xce, 0x92, 0xda, 0x9b, 0x6f, 0x47, 0xd5, 0x1b, 0xb3, 0xd3, 0xb6, 0x50, 0x6e, 0xfa, 0xa1, 0xdd,
- 0x1b, 0x23, 0xe0, 0x15, 0x61, 0x5b, 0x16, 0xd8, 0x3d, 0x65, 0x19, 0x7d, 0x06, 0xba, 0x9d, 0xcc,
- 0x0d, 0x2b, 0xa5, 0x73, 0xe3, 0xbc, 0xf4, 0xe6, 0xea, 0x4c, 0x8d, 0x3c, 0x35, 0x70, 0x70, 0xd9,
- 0x3e, 0x29, 0x58, 0xfb, 0x08, 0xa6, 0x02, 0x8a, 0x10, 0xe4, 0xc4, 0x2b, 0x25, 0xa1, 0x32, 0x2c,
- 0xd7, 0xa8, 0x0a, 0x79, 0x9b, 0xd9, 0x5b, 0x9b, 0xb2, 0x97, 0x97, 0x1b, 0x85, 0xe3, 0x51, 0x35,
- 0xdf, 0xc4, 0xcd, 0xad, 0x4d, 0x9c, 0xc8, 0x13, 0x2e, 0x68, 0xe5, 0xb4, 0x8c, 0x3e, 0xdf, 0xca,
- 0x69, 0x79, 0x7d, 0xa1, 0x95, 0xd3, 0x16, 0x75, 0xad, 0x95, 0xd3, 0x0a, 0x3a, 0xb4, 0x72, 0x5a,
- 0x49, 0x2f, 0xb7, 0x72, 0x5a, 0x59, 0xd7, 0x5b, 0x39, 0x4d, 0xd7, 0x57, 0x5a, 0x39, 0xed, 0x9c,
- 0xbe, 0xda, 0x5a, 0xd0, 0xbe, 0x3a, 0xd0, 0xbf, 0x3d, 0xa8, 0xad, 0x03, 0x7c, 0xc2, 0x3c, 0x4e,
- 0x1b, 0x84, 0xdb, 0xdd, 0xd3, 0x1c, 0xa8, 0xdd, 0x87, 0xa5, 0xbd, 0xd0, 0xf5, 0x6c, 0xe2, 0x7f,
- 0x1c, 0xed, 0x85, 0x2e, 0xda, 0x86, 0x6c, 0x18, 0x09, 0x52, 0x17, 0xe3, 0xe2, 0x5f, 0x67, 0xe5,
- 0x79, 0xac, 0xaa, 0xd2, 0x2c, 0x74, 0x6b, 0x3f, 0xe6, 0xa1, 0x88, 0xc9, 0x63, 0xde, 0x0c, 0xfb,
- 0x7d, 0x12, 0x38, 0xe8, 0x32, 0x94, 0xfb, 0xe4, 0x99, 0x4a, 0xb9, 0x17, 0x38, 0xf4, 0x99, 0xa4,
- 0x8d, 0x1c, 0x5e, 0xee, 0x93, 0x67, 0x32, 0x1b, 0xbb, 0x42, 0x88, 0x0e, 0xe1, 0x2f, 0x53, 0x9d,
- 0x3a, 0x7e, 0x37, 0x4a, 0x3d, 0x39, 0xdb, 0x8a, 0x9b, 0xc6, 0x29, 0xf3, 0x2b, 0x21, 0x8c, 0x8b,
- 0x13, 0xd5, 0x7b, 0x4a, 0x53, 0x6e, 0xa0, 0x21, 0x5c, 0x3c, 0x09, 0x65, 0xc5, 0x82, 0x5c, 0x02,
- 0x9b, 0x4a, 0xce, 0xc9, 0x36, 0x3e, 0x7c, 0x3b, 0xaa, 0x5e, 0x7b, 0xaf, 0x11, 0x20, 0x81, 0xdb,
- 0x0a, 0x05, 0x9f, 0x8f, 0xa6, 0xed, 0xa5, 0x62, 0xf4, 0x04, 0x2e, 0xb0, 0x71, 0xc7, 0x59, 0x74,
- 0x48, 0x7c, 0x8b, 0xc9, 0x9e, 0x93, 0x3d, 0x5d, 0xdc, 0xfc, 0xff, 0x1f, 0x6a, 0x58, 0x15, 0xe7,
- 0x55, 0x76, 0xda, 0x60, 0xd8, 0x83, 0xe2, 0x53, 0x91, 0x6d, 0xab, 0x23, 0xd2, 0x6d, 0x94, 0x66,
- 0x9c, 0x47, 0x93, 0x0a, 0xc1, 0xf0, 0x74, 0x52, 0x2d, 0x6d, 0x28, 0xf9, 0x49, 0x7a, 0xad, 0x30,
- 0x12, 0x94, 0x64, 0x94, 0x67, 0x9c, 0x37, 0xd3, 0x05, 0x85, 0x97, 0xfc, 0xe9, 0xf2, 0x7a, 0x04,
- 0xc0, 0x19, 0xb1, 0xa9, 0x25, 0x0b, 0x51, 0x97, 0x55, 0x76, 0xfd, 0xec, 0x48, 0x4c, 0xaa, 0xc9,
- 0x3c, 0x14, 0xea, 0x3b, 0x84, 0x93, 0x5b, 0x01, 0x67, 0x47, 0xb8, 0xc0, 0xd3, 0xef, 0xb5, 0x1b,
- 0x50, 0x3a, 0xb9, 0x89, 0x74, 0xc8, 0xa6, 0x6f, 0x87, 0x02, 0x16, 0x4b, 0xf1, 0x6a, 0x19, 0x12,
- 0x7f, 0x90, 0xcc, 0xce, 0x02, 0x4e, 0x3e, 0xae, 0xcd, 0x7f, 0x20, 0x5a, 0x2c, 0xab, 0xe7, 0xc6,
- 0x8d, 0x36, 0xaf, 0x67, 0x93, 0x26, 0xfa, 0xee, 0xa0, 0x76, 0x1d, 0x56, 0xa6, 0x1c, 0xb8, 0x1d,
- 0x86, 0x9c, 0xb2, 0x59, 0x8b, 0xba, 0x71, 0xe5, 0xc5, 0xeb, 0xca, 0xdc, 0x8b, 0xe3, 0x4a, 0xe6,
- 0xe5, 0x71, 0x25, 0xf3, 0xea, 0xb8, 0x92, 0xf9, 0xf9, 0xb8, 0x92, 0xf9, 0xf2, 0x4d, 0x65, 0xee,
- 0xe5, 0x9b, 0xca, 0xdc, 0xab, 0x37, 0x95, 0xb9, 0x47, 0x30, 0xf9, 0xcf, 0xe8, 0x2c, 0xc8, 0x5f,
- 0x86, 0xad, 0x5f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xf1, 0xe4, 0x48, 0xb2, 0x4d, 0x0d, 0x00, 0x00,
+ proto.RegisterFile("kv/kvserver/kvserverpb/proposer_kv.proto", fileDescriptor_proposer_kv_004803af804b8ced)
+}
+
+var fileDescriptor_proposer_kv_004803af804b8ced = []byte{
+ // 1473 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0xcf, 0x6f, 0x13, 0xc7,
+ 0x17, 0x8f, 0x63, 0x3b, 0x59, 0x8f, 0x13, 0x7b, 0x33, 0x04, 0xd8, 0x6f, 0xbe, 0xad, 0x1d, 0xb9,
+ 0x14, 0xa5, 0x94, 0xae, 0x51, 0xd2, 0x4a, 0x15, 0xa0, 0x8a, 0xd8, 0x81, 0x12, 0x93, 0xa4, 0x30,
+ 0x0e, 0xb4, 0xa2, 0x95, 0x56, 0xe3, 0xdd, 0x61, 0xbd, 0xf5, 0x7a, 0x77, 0x99, 0x19, 0x1b, 0xf2,
+ 0x57, 0xb4, 0x95, 0x7a, 0xe8, 0xa9, 0x70, 0xec, 0x9f, 0xc2, 0xa5, 0x12, 0x47, 0xd4, 0x83, 0x55,
+ 0xc2, 0xa5, 0x7f, 0x03, 0xa7, 0x6a, 0x66, 0x67, 0xed, 0x0d, 0x0a, 0x8d, 0x69, 0x6f, 0x33, 0x6f,
+ 0xde, 0xfb, 0xbc, 0xd9, 0xf7, 0xe3, 0xf3, 0x66, 0xc1, 0x5a, 0x6f, 0x58, 0xef, 0x0d, 0x19, 0xa1,
+ 0x43, 0x42, 0xc7, 0x8b, 0xa8, 0x53, 0x8f, 0x68, 0x18, 0x85, 0x8c, 0x50, 0xab, 0x37, 0x34, 0x23,
+ 0x1a, 0xf2, 0x10, 0x56, 0xed, 0xd0, 0xee, 0xd1, 0x10, 0xdb, 0x5d, 0xb3, 0x37, 0x34, 0x13, 0x55,
+ 0x93, 0xf1, 0x90, 0x62, 0x97, 0x44, 0x9d, 0x95, 0x25, 0x79, 0x18, 0x75, 0xea, 0x38, 0xf2, 0x62,
+ 0x9b, 0x15, 0x98, 0x88, 0x1c, 0xcc, 0xb1, 0x92, 0x9d, 0x49, 0x64, 0x7d, 0xc2, 0x71, 0x4a, 0xfe,
+ 0x7f, 0x85, 0x54, 0x27, 0x81, 0xeb, 0x05, 0x44, 0x28, 0x0c, 0x6d, 0x5b, 0x1d, 0xbe, 0x77, 0xec,
+ 0xe1, 0x86, 0x3a, 0xad, 0xbd, 0xe5, 0x23, 0x18, 0xc7, 0x9c, 0x28, 0x9d, 0x0b, 0x69, 0x1d, 0x4a,
+ 0xb0, 0xc3, 0x06, 0xfd, 0x3e, 0xa6, 0x07, 0x75, 0xca, 0x84, 0x66, 0xbc, 0x51, 0xba, 0xc6, 0x80,
+ 0x7b, 0x7e, 0xbd, 0xeb, 0xdb, 0x75, 0xee, 0xf5, 0x09, 0xe3, 0xb8, 0x1f, 0xa9, 0x93, 0x65, 0x37,
+ 0x74, 0x43, 0xb9, 0xac, 0x8b, 0x55, 0x2c, 0xad, 0xfd, 0x96, 0x01, 0xf9, 0x76, 0xe4, 0x7b, 0x1c,
+ 0x36, 0xc1, 0x3c, 0xa7, 0x9e, 0xeb, 0x12, 0x6a, 0x64, 0x56, 0x33, 0x6b, 0xc5, 0xf5, 0xaa, 0x39,
+ 0x09, 0x9b, 0xfa, 0x70, 0x53, 0xaa, 0xee, 0xc7, 0x6a, 0x0d, 0xed, 0xd9, 0xa8, 0x3a, 0xf3, 0x7c,
+ 0x54, 0xcd, 0xa0, 0xc4, 0x12, 0xee, 0x83, 0x02, 0xed, 0x32, 0xcb, 0x21, 0x3e, 0xc7, 0xc6, 0xac,
+ 0x84, 0xf9, 0x30, 0x05, 0xa3, 0x42, 0x61, 0x26, 0xa1, 0x30, 0x77, 0xef, 0x35, 0x9b, 0x6d, 0x8e,
+ 0x39, 0x6b, 0xe8, 0x02, 0xec, 0x70, 0x54, 0xd5, 0xd0, 0xcd, 0xf6, 0x96, 0x30, 0x47, 0x1a, 0xed,
+ 0x32, 0xb9, 0xba, 0x9c, 0xfb, 0xeb, 0x69, 0x35, 0x53, 0x43, 0x20, 0xbf, 0x4b, 0xa8, 0x4b, 0xa6,
+ 0xbb, 0xa9, 0x54, 0x7d, 0xfb, 0x4d, 0x15, 0xa6, 0x03, 0x4a, 0xcd, 0x2e, 0x0e, 0x5c, 0x82, 0x48,
+ 0xe4, 0x7b, 0x36, 0x66, 0x70, 0xe7, 0x4d, 0xf0, 0xb5, 0x63, 0xc0, 0x8f, 0xda, 0xfc, 0x93, 0x97,
+ 0x5f, 0x9e, 0x56, 0x67, 0x6a, 0x2f, 0x67, 0x41, 0xb9, 0x19, 0xf6, 0xa3, 0x01, 0x27, 0xcd, 0x2e,
+ 0xb1, 0x7b, 0x6c, 0xd0, 0x87, 0xdf, 0x83, 0xa2, 0xad, 0xd6, 0x96, 0xe7, 0x48, 0x5f, 0x0b, 0x8d,
+ 0x6d, 0x81, 0xf0, 0xc7, 0xa8, 0xba, 0xe1, 0x7a, 0xbc, 0x3b, 0xe8, 0x98, 0x76, 0xd8, 0xaf, 0x8f,
+ 0xbd, 0x3b, 0x9d, 0xc9, 0xba, 0x1e, 0xf5, 0xdc, 0xba, 0x4c, 0xf5, 0x60, 0xe0, 0x39, 0xe6, 0xdd,
+ 0xbb, 0xdb, 0x5b, 0x87, 0xa3, 0x2a, 0x48, 0xd0, 0xb7, 0xb7, 0x10, 0x48, 0xd0, 0xb7, 0x1d, 0xf8,
+ 0x01, 0x58, 0x64, 0x78, 0x48, 0x2c, 0x16, 0xe0, 0x88, 0x75, 0x43, 0x2e, 0x33, 0xa3, 0xa1, 0x05,
+ 0x21, 0x6c, 0x2b, 0x19, 0xdc, 0x00, 0xb9, 0x7e, 0xe8, 0x10, 0x23, 0xbb, 0x9a, 0x59, 0x2b, 0x1d,
+ 0x1b, 0xd2, 0x04, 0x7d, 0x37, 0x74, 0x08, 0x92, 0xca, 0xb0, 0x02, 0x62, 0x3f, 0x51, 0xe8, 0x05,
+ 0xdc, 0xc8, 0x49, 0xd8, 0x94, 0x04, 0x1a, 0x60, 0x7e, 0x48, 0x28, 0xf3, 0xc2, 0xc0, 0xc8, 0xaf,
+ 0x66, 0xd6, 0x16, 0x51, 0xb2, 0x85, 0x37, 0x41, 0x81, 0x13, 0xda, 0xf7, 0x02, 0xcc, 0x89, 0x31,
+ 0xb7, 0x9a, 0x5d, 0x2b, 0xae, 0x9f, 0x3b, 0xc6, 0xa7, 0x8a, 0xf1, 0x16, 0x61, 0x36, 0xf5, 0x22,
+ 0x1e, 0xd2, 0x46, 0x4e, 0xc4, 0x08, 0x4d, 0x8c, 0x55, 0x26, 0xef, 0x01, 0x20, 0x42, 0x8c, 0x6d,
+ 0x2e, 0xd0, 0x97, 0x41, 0xbe, 0x73, 0xc0, 0x09, 0x93, 0x71, 0xcd, 0xa2, 0x78, 0x03, 0x2f, 0x02,
+ 0xc8, 0x06, 0xae, 0x4b, 0x18, 0x27, 0x8e, 0x85, 0xb9, 0x15, 0xe0, 0x20, 0x64, 0x32, 0x18, 0x59,
+ 0xa4, 0x8f, 0x4f, 0x36, 0xf9, 0x9e, 0x90, 0x2b, 0xdc, 0x9f, 0x67, 0xc1, 0xa9, 0x76, 0x72, 0x94,
+ 0xf2, 0x70, 0x07, 0x14, 0x18, 0xc7, 0x94, 0x5b, 0x3d, 0x72, 0xa0, 0xb2, 0xf7, 0xe9, 0xeb, 0x51,
+ 0xf5, 0xd2, 0x54, 0x99, 0x4b, 0xbe, 0xee, 0x16, 0x39, 0x40, 0x9a, 0x84, 0xb9, 0x45, 0x0e, 0xe0,
+ 0x2e, 0x98, 0x27, 0x81, 0x23, 0x01, 0x67, 0xff, 0x03, 0xe0, 0x1c, 0x09, 0x1c, 0x01, 0x77, 0x17,
+ 0x00, 0x7b, 0x7c, 0x5f, 0x99, 0xd6, 0xe2, 0xfa, 0xc7, 0xe6, 0x09, 0x54, 0x68, 0x4e, 0x3e, 0x31,
+ 0x55, 0xcf, 0x29, 0x20, 0x15, 0x96, 0x27, 0x05, 0xb0, 0xac, 0x72, 0xc3, 0x89, 0x73, 0x7d, 0x88,
+ 0x7d, 0x44, 0xd8, 0xc0, 0x17, 0x34, 0x92, 0x97, 0xdc, 0xa5, 0xba, 0xff, 0x93, 0x13, 0x1d, 0x2a,
+ 0x14, 0xc1, 0x02, 0x04, 0xc5, 0xb6, 0xf0, 0x2a, 0xc8, 0x33, 0xc1, 0x34, 0xea, 0xd6, 0xe7, 0x4f,
+ 0x04, 0x91, 0xbc, 0x84, 0x62, 0x23, 0x61, 0xdd, 0x17, 0xdd, 0x2f, 0xeb, 0x71, 0x1a, 0x6b, 0xc9,
+ 0x15, 0x28, 0x36, 0x82, 0x6b, 0x40, 0xf7, 0x98, 0xe5, 0x13, 0xcc, 0x88, 0x45, 0xc9, 0xc3, 0x01,
+ 0x61, 0xdc, 0x98, 0x93, 0x85, 0x5d, 0xf2, 0xd8, 0x8e, 0x10, 0xa3, 0x58, 0x0a, 0x37, 0x41, 0x61,
+ 0x4c, 0xb2, 0x86, 0x26, 0x7d, 0xbd, 0x9f, 0xf2, 0x25, 0xda, 0xd3, 0xec, 0xfa, 0xb6, 0xb9, 0x9f,
+ 0x28, 0x8d, 0x6b, 0x37, 0x11, 0xc0, 0xdb, 0x40, 0x77, 0x48, 0x44, 0x89, 0x8c, 0xa2, 0xa2, 0x4d,
+ 0xf0, 0x0e, 0xb4, 0x89, 0xca, 0x13, 0x73, 0xc9, 0x95, 0xf0, 0x1b, 0x50, 0xb6, 0x25, 0x3b, 0x59,
+ 0x54, 0xd1, 0x93, 0xb1, 0x20, 0x01, 0xeb, 0x27, 0xa7, 0xfe, 0x08, 0xab, 0xa1, 0x92, 0x7d, 0x94,
+ 0x19, 0xcf, 0x81, 0x12, 0xc5, 0x0f, 0xb8, 0xe5, 0x87, 0xae, 0xba, 0xe9, 0xa2, 0xec, 0x9c, 0x05,
+ 0x21, 0xdd, 0x09, 0xdd, 0xd8, 0xff, 0x43, 0x50, 0xc4, 0x8e, 0x63, 0x31, 0xc6, 0x71, 0xc7, 0x27,
+ 0xc6, 0x92, 0xf4, 0x7d, 0x6d, 0xda, 0x2a, 0x38, 0x52, 0x4b, 0xe6, 0xa6, 0xe3, 0xb4, 0xdb, 0xfb,
+ 0x02, 0xa7, 0x51, 0x12, 0xf4, 0x36, 0xd9, 0x23, 0x80, 0x1d, 0xa7, 0x1d, 0xfb, 0x80, 0x37, 0x40,
+ 0x3e, 0xbe, 0x0f, 0x94, 0xce, 0x2e, 0x4c, 0x15, 0x39, 0x79, 0x5b, 0x95, 0x90, 0xd8, 0x1c, 0xfe,
+ 0x90, 0x01, 0xa7, 0x22, 0x4a, 0x86, 0x2a, 0xf9, 0xf1, 0x3b, 0x02, 0xfb, 0xc6, 0xf2, 0x34, 0xa9,
+ 0xbd, 0xf6, 0x7a, 0x54, 0xbd, 0x3a, 0x3d, 0x6d, 0x0b, 0xe3, 0xa6, 0x1f, 0xda, 0xbd, 0x31, 0x02,
+ 0x5a, 0x12, 0xbe, 0x65, 0x81, 0xdd, 0x56, 0x9e, 0xe1, 0xb7, 0x40, 0xb7, 0xe3, 0xb9, 0x61, 0x25,
+ 0x74, 0x6e, 0x9c, 0x96, 0xb7, 0xb9, 0x34, 0x55, 0x23, 0xa7, 0x06, 0x0e, 0x2a, 0xdb, 0x6f, 0x4c,
+ 0xa0, 0xef, 0x00, 0x8c, 0xa8, 0x17, 0x52, 0x4b, 0x3c, 0x29, 0x2c, 0xf5, 0x8c, 0x30, 0xce, 0x48,
+ 0x78, 0xf3, 0x2d, 0xf0, 0xa9, 0xd7, 0x87, 0x89, 0x08, 0x76, 0xda, 0xf1, 0x1a, 0xe9, 0x12, 0x29,
+ 0x25, 0x59, 0xf9, 0x12, 0xa4, 0xd2, 0x05, 0x21, 0xc8, 0x89, 0xf7, 0x52, 0x4c, 0x94, 0x48, 0xae,
+ 0x61, 0x15, 0xe4, 0x6d, 0x6a, 0x6f, 0xac, 0x4b, 0xa6, 0x58, 0x6c, 0x14, 0x0e, 0x47, 0xd5, 0x7c,
+ 0x13, 0x35, 0x37, 0xd6, 0x51, 0x2c, 0x8f, 0x99, 0xa6, 0x95, 0xd3, 0x32, 0xfa, 0x6c, 0x2b, 0xa7,
+ 0xe5, 0xf5, 0xb9, 0x56, 0x4e, 0x9b, 0xd7, 0xb5, 0x56, 0x4e, 0x2b, 0xe8, 0xa0, 0x95, 0xd3, 0x4a,
+ 0x7a, 0xb9, 0x95, 0xd3, 0xca, 0xba, 0xde, 0xca, 0x69, 0xba, 0xbe, 0xd4, 0xca, 0x69, 0xa7, 0xf4,
+ 0xe5, 0xd6, 0x9c, 0xf6, 0xd3, 0x9e, 0xfe, 0xeb, 0x5e, 0x6d, 0x15, 0x80, 0xaf, 0xa9, 0xc7, 0x49,
+ 0x03, 0x73, 0xbb, 0x7b, 0xdc, 0x05, 0x6a, 0x77, 0xc0, 0xc2, 0x4e, 0xe8, 0x7a, 0x36, 0xf6, 0xbf,
+ 0x8a, 0x76, 0x42, 0x17, 0x6e, 0x82, 0x6c, 0x18, 0x89, 0x91, 0x21, 0x86, 0xd1, 0x47, 0x27, 0x55,
+ 0xd1, 0xd8, 0x54, 0x15, 0x91, 0xb0, 0xad, 0xfd, 0x9e, 0x07, 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, 0x5f, 0x8a,
+ 0x07, 0xc6, 0x2f, 0x58, 0x69, 0x27, 0x27, 0x67, 0x71, 0xdd, 0x38, 0x66, 0x3a, 0xc6, 0x74, 0x74,
+ 0x76, 0x62, 0x7a, 0x5b, 0x59, 0xca, 0x03, 0x38, 0x04, 0x67, 0x8f, 0x42, 0x59, 0x4c, 0x50, 0x57,
+ 0x60, 0x13, 0xc9, 0x68, 0xd9, 0xc6, 0x17, 0xaf, 0x47, 0xd5, 0xcb, 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, 0xfd, 0x2b, 0x3a, 0x50, 0x71,
+ 0x5e, 0xa6, 0xc7, 0x8d, 0x9d, 0x1d, 0x50, 0x7c, 0x24, 0xb2, 0x6d, 0x75, 0x44, 0xba, 0x8d, 0xd2,
+ 0x94, 0xd3, 0x6e, 0x52, 0x21, 0x08, 0x3c, 0x9a, 0x54, 0x4b, 0x1b, 0x94, 0xfc, 0x38, 0xbd, 0x56,
+ 0x18, 0x09, 0xc2, 0x33, 0xca, 0x53, 0x4e, 0xb3, 0x74, 0x41, 0xa1, 0x05, 0x3f, 0x5d, 0x5e, 0xf7,
+ 0x01, 0xe0, 0x14, 0xdb, 0xc4, 0x92, 0x85, 0xa8, 0xcb, 0x2a, 0xbb, 0x72, 0x72, 0x24, 0x26, 0xd5,
+ 0x64, 0xee, 0x0b, 0xf3, 0x2d, 0xcc, 0xf1, 0xf5, 0x80, 0xd3, 0x03, 0x54, 0xe0, 0xc9, 0x7e, 0xe5,
+ 0x2a, 0x28, 0x1d, 0x3d, 0x84, 0x3a, 0xc8, 0x26, 0x2f, 0x93, 0x02, 0x12, 0x4b, 0xf1, 0x26, 0x1a,
+ 0x62, 0x7f, 0x10, 0x4f, 0xe6, 0x02, 0x8a, 0x37, 0x97, 0x67, 0x3f, 0x17, 0x2d, 0x96, 0xd5, 0x73,
+ 0xe3, 0x46, 0x9b, 0xd5, 0xb3, 0x71, 0x13, 0x3d, 0xd9, 0xab, 0x5d, 0x01, 0x4b, 0xa9, 0x0b, 0xdc,
+ 0x08, 0x43, 0x4e, 0xe8, 0xb4, 0x45, 0xdd, 0xb8, 0xf8, 0xec, 0x65, 0x65, 0xe6, 0xd9, 0x61, 0x25,
+ 0xf3, 0xfc, 0xb0, 0x92, 0x79, 0x71, 0x58, 0xc9, 0xfc, 0x79, 0x58, 0xc9, 0xfc, 0xf8, 0xaa, 0x32,
+ 0xf3, 0xfc, 0x55, 0x65, 0xe6, 0xc5, 0xab, 0xca, 0xcc, 0x7d, 0x30, 0xf9, 0xe3, 0xe9, 0xcc, 0xc9,
+ 0x1f, 0x92, 0x8d, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0x54, 0x87, 0x06, 0x00, 0xd7, 0x0d, 0x00,
+ 0x00,
}
diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto
index 3267b2a7b4a8..63cc4df854da 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,22 @@ 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.
+ 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 d33ed0c2ccae..f2a2518f00a3 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"`
@@ -108,7 +108,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_acd314f0f91777f8, []int{0}
+ return fileDescriptor_state_a50383b8609eb8ee, []int{0}
}
func (m *ReplicaState) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -180,7 +180,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_acd314f0f91777f8, []int{1}
+ return fileDescriptor_state_a50383b8609eb8ee, []int{1}
}
func (m *RangeInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -216,7 +216,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_acd314f0f91777f8, []int{1, 0}
+ return fileDescriptor_state_a50383b8609eb8ee, []int{1, 0}
}
func (m *RangeInfo_CTEntry) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -252,7 +252,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_acd314f0f91777f8, []int{2}
+ return fileDescriptor_state_a50383b8609eb8ee, []int{2}
}
func (m *LatchManagerInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1952,10 +1952,10 @@ var (
)
func init() {
- proto.RegisterFile("kv/kvserver/kvserverpb/state.proto", fileDescriptor_state_acd314f0f91777f8)
+ proto.RegisterFile("kv/kvserver/kvserverpb/state.proto", fileDescriptor_state_a50383b8609eb8ee)
}
-var fileDescriptor_state_acd314f0f91777f8 = []byte{
+var fileDescriptor_state_a50383b8609eb8ee = []byte{
// 1061 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xcd, 0x6e, 0x1b, 0x37,
0x10, 0xf6, 0x5a, 0x2b, 0x5b, 0xa2, 0xec, 0x58, 0x61, 0x9d, 0x78, 0xe3, 0xc4, 0x92, 0x21, 0xa0,
diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto
index f8288300aeac..0bd2924a3d86 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/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..0d011082a83a
--- /dev/null
+++ b/pkg/kv/kvserver/readsummary/persist.go
@@ -0,0 +1,47 @@
+// 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.
+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..c9f4c8398956
--- /dev/null
+++ b/pkg/kv/kvserver/readsummary/rspb/summary.go
@@ -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.
+
+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 {
+ 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..1a990d3c7556
--- /dev/null
+++ b/pkg/kv/kvserver/readsummary/rspb/summary.pb.go
@@ -0,0 +1,602 @@
+// 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. However, the
+// timestamp of a key is allowed to advance as precision drops. This parallels a
+// similar ratchetting 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_6c83dfcb0ce22264, []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_6c83dfcb0ce22264, []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_6c83dfcb0ce22264)
+}
+
+var fileDescriptor_summary_6c83dfcb0ce22264 = []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..8aba8ef86f4d
--- /dev/null
+++ b/pkg/kv/kvserver/readsummary/rspb/summary.proto
@@ -0,0 +1,61 @@
+// 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. However, the
+// timestamp of a key is allowed to advance as precision drops. This parallels a
+// similar ratchetting 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 fcb1f7b447ca..9bd1e8c03a7c 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 34faae32038b..5f8f287a596f 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"
@@ -246,7 +247,7 @@ 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,
+ ctx, r, merge.LeftDesc, merge.RightDesc, merge.FreezeStart, 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)
@@ -257,10 +258,12 @@ 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, *lease, false /* permitJump */)
+ r.leasePostApplyLocked(ctx, *lease, priorReadSum, false /* permitJump */)
}
func (r *Replica) handleTruncatedStateResult(
diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go
index 0eca465b0394..d0de171934d6 100644
--- a/pkg/kv/kvserver/replica_application_state_machine.go
+++ b/pkg/kv/kvserver/replica_application_state_machine.go
@@ -1112,12 +1112,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 b3e2870fd6f0..66a2a1e788c5 100644
--- a/pkg/kv/kvserver/replica_command.go
+++ b/pkg/kv/kvserver/replica_command.go
@@ -738,10 +738,11 @@ func (r *Replica) AdminMerge(
Commit: true,
InternalCommitTrigger: &roachpb.InternalCommitTrigger{
MergeTrigger: &roachpb.MergeTrigger{
- LeftDesc: updatedLeftDesc,
- RightDesc: rightDesc,
- RightMVCCStats: rhsSnapshotRes.MVCCStats,
- FreezeStart: rhsSnapshotRes.FreezeStart,
+ LeftDesc: updatedLeftDesc,
+ RightDesc: rightDesc,
+ RightMVCCStats: rhsSnapshotRes.MVCCStats,
+ FreezeStart: rhsSnapshotRes.FreezeStart,
+ RightReadSummary: rhsSnapshotRes.ReadSummary,
},
},
})
diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go
index 0e8d519a6893..d258e228eeca 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"
@@ -210,6 +211,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_proposal.go b/pkg/kv/kvserver/replica_proposal.go
index d4d11e5b7dff..1cd2662b702f 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"
@@ -297,10 +298,16 @@ A file preventing this node from restarting was placed at:
// called repeatedly for the same lease safely. However, the method will panic
// if passed a lease with a lower sequence number than the current lease. By
// default, the method will also panic if passed a lease that indicates a
-// forward sequence number jump (i.e. a skipped lease). This behavior can
-// be disabled by passing permitJump as true.
+// forward sequence number jump (i.e. a skipped lease). This behavior can be
+// disabled by passing permitJump as true.
+//
+// In addition to a lease, 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, newLease roachpb.Lease, permitJump bool,
+ ctx context.Context, newLease roachpb.Lease, priorReadSum *rspb.ReadSummary, permitJump bool,
) {
// Pull out the last lease known to this Replica. It's possible that this is
// not actually the last lease in the Range's lease sequence because the
@@ -368,18 +375,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 e5701be49dc0..f1e20ea55c91 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"
@@ -963,6 +964,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
@@ -984,7 +993,15 @@ 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, *s.Lease, true /* permitJump */)
+ r.leasePostApplyLocked(ctx, *s.Lease, prioReadSum, true /* permitJump */)
+
+ // 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 d3c202d1aa3e..509b7132f0d8 100644
--- a/pkg/kv/kvserver/replica_test.go
+++ b/pkg/kv/kvserver/replica_test.go
@@ -6227,10 +6227,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 56b2dcdc876d..708ccd8e3969 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,55 @@ 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())
+ if maxClosed, ok := r.maxClosed(context.Background()); ok {
+ // Forward the read summary by the range's closed timestamp, because any
+ // replica could have served reads below this time.
+ // TODO DURING REVIEW: Like in #59854, this is likely not correct for
+ // the old closed timestamp system, because we can't tell what timestamp
+ // will be closed next under the current lease. This will be easier to
+ // think through with the new closed timestamp system.
+ sum.Merge(rspb.FromTimestamp(maxClosed))
+ }
+ 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 0b8c9c03f24c..4ea6189196a5 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"
@@ -29,6 +30,7 @@ func (s *Store) MergeRange(
leftRepl *Replica,
newLeftDesc, rightDesc roachpb.RangeDescriptor,
freezeStart hlc.ClockTimestamp,
+ 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",
@@ -77,17 +79,26 @@ func (s *Store) MergeRange(
if leftLease.OwnedBy(s.Ident.StoreID) && !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)
}
// Update the subsuming range's descriptor.
diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go
index c4702d445a66..9fbcfd51749a 100644
--- a/pkg/kv/kvserver/store_split.go
+++ b/pkg/kv/kvserver/store_split.go
@@ -274,7 +274,7 @@ func prepareRightReplicaForSplit(
// the txnWaitQueue.
rightRepl.mu.Lock()
defer rightRepl.mu.Unlock()
- rightRepl.leasePostApplyLocked(ctx, rightLease, false /* permitJump */)
+ rightRepl.leasePostApplyLocked(ctx, rightLease, nil /* priorReadSum */, false /* permitJump */)
return rightRepl
}
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 a4174b5977f8..586ef9d2fd4d 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_f2b2489e6aa3dd67, []int{0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{1}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{2}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{3}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{4}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{5}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{1, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{25, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{1}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{2}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{3}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{4}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{5}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{6}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{7}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{8}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{9}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{10}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{11}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{12}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{13}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{14}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{15}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{16}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{17}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{18}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{19}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{20}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{21}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{22}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{23}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{24}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{25}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{25, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{26}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{27}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{28}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{29}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{30}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{31}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{32}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{33}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{34}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{35}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{36}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{37}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{38}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{39}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{40}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{41}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{42}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{43}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{44}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{45}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{45, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{46}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{47}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{48}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{49}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{50}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{51}
+ return fileDescriptor_api_70ddf268aebf8b6a, []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_f2b2489e6aa3dd67, []int{52}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{52}
}
func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2789,7 +2790,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_f2b2489e6aa3dd67, []int{53}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{53}
}
func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2825,7 +2826,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_f2b2489e6aa3dd67, []int{54}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{54}
}
func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2872,7 +2873,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_f2b2489e6aa3dd67, []int{55}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{55}
}
func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2907,7 +2908,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_f2b2489e6aa3dd67, []int{56}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{56}
}
func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2958,7 +2959,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_f2b2489e6aa3dd67, []int{57}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{57}
}
func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2993,7 +2994,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_f2b2489e6aa3dd67, []int{58}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{58}
}
func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3030,7 +3031,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_f2b2489e6aa3dd67, []int{59}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{59}
}
func (m *MergeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3064,7 +3065,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_f2b2489e6aa3dd67, []int{60}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{60}
}
func (m *MergeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3109,7 +3110,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_f2b2489e6aa3dd67, []int{61}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{61}
}
func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3143,7 +3144,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_f2b2489e6aa3dd67, []int{62}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{62}
}
func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3187,7 +3188,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_f2b2489e6aa3dd67, []int{63}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{63}
}
func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3236,7 +3237,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_f2b2489e6aa3dd67, []int{64}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{64}
}
func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3273,7 +3274,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_f2b2489e6aa3dd67, []int{65}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{65}
}
func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3310,7 +3311,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_f2b2489e6aa3dd67, []int{66}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{66}
}
func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3345,7 +3346,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_f2b2489e6aa3dd67, []int{67}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{67}
}
func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3400,7 +3401,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_f2b2489e6aa3dd67, []int{68}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{68}
}
func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3437,7 +3438,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_f2b2489e6aa3dd67, []int{69}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{69}
}
func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3477,7 +3478,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_f2b2489e6aa3dd67, []int{70}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{70}
}
func (m *ExternalStorage) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3511,7 +3512,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_f2b2489e6aa3dd67, []int{70, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{70, 0}
}
func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3544,7 +3545,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_f2b2489e6aa3dd67, []int{70, 1}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{70, 1}
}
func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3586,7 +3587,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_f2b2489e6aa3dd67, []int{70, 2}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{70, 2}
}
func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3625,7 +3626,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_f2b2489e6aa3dd67, []int{70, 3}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{70, 3}
}
func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3661,7 +3662,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_f2b2489e6aa3dd67, []int{70, 4}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{70, 4}
}
func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3700,7 +3701,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_f2b2489e6aa3dd67, []int{70, 5}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{70, 5}
}
func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3742,7 +3743,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_f2b2489e6aa3dd67, []int{70, 6}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{70, 6}
}
func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3782,7 +3783,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_f2b2489e6aa3dd67, []int{71}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{71}
}
func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3816,7 +3817,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_f2b2489e6aa3dd67, []int{72}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{72}
}
func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3852,7 +3853,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_f2b2489e6aa3dd67, []int{73}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{73}
}
func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3922,7 +3923,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_f2b2489e6aa3dd67, []int{74}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{74}
}
func (m *ExportRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3974,7 +3975,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_f2b2489e6aa3dd67, []int{75}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{75}
}
func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4010,7 +4011,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_f2b2489e6aa3dd67, []int{76}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{76}
}
func (m *ExportResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4050,7 +4051,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_f2b2489e6aa3dd67, []int{76, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{76, 0}
}
func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4101,7 +4102,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_f2b2489e6aa3dd67, []int{77}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{77}
}
func (m *ImportRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4136,7 +4137,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_f2b2489e6aa3dd67, []int{77, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{77, 0}
}
func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4172,7 +4173,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_f2b2489e6aa3dd67, []int{77, 1}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{77, 1}
}
func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4207,7 +4208,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_f2b2489e6aa3dd67, []int{78}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{78}
}
func (m *ImportResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4245,7 +4246,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_f2b2489e6aa3dd67, []int{79}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{79}
}
func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4282,7 +4283,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_f2b2489e6aa3dd67, []int{80}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{80}
}
func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4315,7 +4316,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_f2b2489e6aa3dd67, []int{80, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{80, 0}
}
func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4360,7 +4361,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_f2b2489e6aa3dd67, []int{81}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{81}
}
func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4398,7 +4399,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_f2b2489e6aa3dd67, []int{82}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{82}
}
func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4451,7 +4452,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_f2b2489e6aa3dd67, []int{83}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{83}
}
func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4485,7 +4486,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_f2b2489e6aa3dd67, []int{84}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{84}
}
func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4529,7 +4530,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_f2b2489e6aa3dd67, []int{85}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{85}
}
func (m *RefreshRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4563,7 +4564,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_f2b2489e6aa3dd67, []int{86}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{86}
}
func (m *RefreshResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4602,7 +4603,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_f2b2489e6aa3dd67, []int{87}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{87}
}
func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4636,7 +4637,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_f2b2489e6aa3dd67, []int{88}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{88}
}
func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4685,7 +4686,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_f2b2489e6aa3dd67, []int{89}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{89}
}
func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4728,13 +4729,27 @@ type SubsumeResponse struct {
// being subsumed). It is suitable for use as the timestamp cache's low water
// mark for the keys previously owned by the subsumed range.
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"`
+ // 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,6,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_f2b2489e6aa3dd67, []int{90}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{90}
}
func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4769,7 +4784,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_f2b2489e6aa3dd67, []int{91}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{91}
}
func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4810,7 +4825,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_f2b2489e6aa3dd67, []int{92}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{92}
}
func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4853,7 +4868,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_f2b2489e6aa3dd67, []int{93}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{93}
}
func (m *MigrateRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4887,7 +4902,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_f2b2489e6aa3dd67, []int{94}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{94}
}
func (m *MigrateResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4971,7 +4986,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_f2b2489e6aa3dd67, []int{95}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{95}
}
func (m *RequestUnion) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -6452,7 +6467,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_f2b2489e6aa3dd67, []int{96}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{96}
}
func (m *ResponseUnion) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8008,7 +8023,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_f2b2489e6aa3dd67, []int{97}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{97}
}
func (m *Header) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8044,7 +8059,7 @@ type BatchRequest struct {
func (m *BatchRequest) Reset() { *m = BatchRequest{} }
func (*BatchRequest) ProtoMessage() {}
func (*BatchRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_f2b2489e6aa3dd67, []int{98}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{98}
}
func (m *BatchRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8081,7 +8096,7 @@ type BatchResponse struct {
func (m *BatchResponse) Reset() { *m = BatchResponse{} }
func (*BatchResponse) ProtoMessage() {}
func (*BatchResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_f2b2489e6aa3dd67, []int{99}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{99}
}
func (m *BatchResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8157,7 +8172,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_f2b2489e6aa3dd67, []int{99, 0}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{99, 0}
}
func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8195,7 +8210,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_f2b2489e6aa3dd67, []int{100}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{100}
}
func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8233,7 +8248,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_f2b2489e6aa3dd67, []int{101}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{101}
}
func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8272,7 +8287,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_f2b2489e6aa3dd67, []int{102}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{102}
}
func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8313,7 +8328,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_f2b2489e6aa3dd67, []int{103}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{103}
}
func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8354,7 +8369,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_f2b2489e6aa3dd67, []int{104}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{104}
}
func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8391,7 +8406,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_f2b2489e6aa3dd67, []int{105}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{105}
}
func (m *RangeFeedError) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8428,7 +8443,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_f2b2489e6aa3dd67, []int{106}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{106}
}
func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8472,7 +8487,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_f2b2489e6aa3dd67, []int{107}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{107}
}
func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8504,7 +8519,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_f2b2489e6aa3dd67, []int{108}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{108}
}
func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8545,7 +8560,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_f2b2489e6aa3dd67, []int{109}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{109}
}
func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8585,7 +8600,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_f2b2489e6aa3dd67, []int{110}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{110}
}
func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8621,7 +8636,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_f2b2489e6aa3dd67, []int{111}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{111}
}
func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8660,7 +8675,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_f2b2489e6aa3dd67, []int{112}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{112}
}
func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8700,7 +8715,7 @@ type ContentionEvent struct {
func (m *ContentionEvent) Reset() { *m = ContentionEvent{} }
func (*ContentionEvent) ProtoMessage() {}
func (*ContentionEvent) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_f2b2489e6aa3dd67, []int{113}
+ return fileDescriptor_api_70ddf268aebf8b6a, []int{113}
}
func (m *ContentionEvent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -13496,6 +13511,16 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) {
return 0, err
}
i += n157
+ if m.ReadSummary != nil {
+ dAtA[i] = 0x32
+ i++
+ i = encodeVarintApi(dAtA, i, uint64(m.ReadSummary.Size()))
+ n158, err := m.ReadSummary.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += n158
+ }
return i, nil
}
@@ -13517,11 +13542,11 @@ func (m *RangeStatsRequest) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size()))
- n158, err := m.RequestHeader.MarshalTo(dAtA[i:])
+ n159, err := m.RequestHeader.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n158
+ i += n159
return i, nil
}
@@ -13543,19 +13568,19 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size()))
- n159, err := m.ResponseHeader.MarshalTo(dAtA[i:])
+ n160, err := m.ResponseHeader.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n159
+ i += n160
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size()))
- n160, err := m.MVCCStats.MarshalTo(dAtA[i:])
+ n161, err := m.MVCCStats.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n160
+ i += n161
if m.QueriesPerSecond != 0 {
dAtA[i] = 0x19
i++
@@ -13565,11 +13590,11 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x22
i++
i = encodeVarintApi(dAtA, i, uint64(m.RangeInfo.Size()))
- n161, err := m.RangeInfo.MarshalTo(dAtA[i:])
+ n162, err := m.RangeInfo.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n161
+ i += n162
return i, nil
}
@@ -13591,19 +13616,19 @@ func (m *MigrateRequest) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size()))
- n162, err := m.RequestHeader.MarshalTo(dAtA[i:])
+ n163, err := m.RequestHeader.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n162
+ i += n163
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Version.Size()))
- n163, err := m.Version.MarshalTo(dAtA[i:])
+ n164, err := m.Version.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n163
+ i += n164
return i, nil
}
@@ -13625,11 +13650,11 @@ func (m *MigrateResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size()))
- n164, err := m.ResponseHeader.MarshalTo(dAtA[i:])
+ n165, err := m.ResponseHeader.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n164
+ i += n165
return i, nil
}
@@ -13649,11 +13674,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) {
var l int
_ = l
if m.Value != nil {
- nn165, err := m.Value.MarshalTo(dAtA[i:])
+ nn166, err := m.Value.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += nn165
+ i += nn166
}
return i, nil
}
@@ -13664,11 +13689,11 @@ func (m *RequestUnion_Get) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Get.Size()))
- n166, err := m.Get.MarshalTo(dAtA[i:])
+ n167, err := m.Get.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n166
+ i += n167
}
return i, nil
}
@@ -13678,11 +13703,11 @@ func (m *RequestUnion_Put) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Put.Size()))
- n167, err := m.Put.MarshalTo(dAtA[i:])
+ n168, err := m.Put.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n167
+ i += n168
}
return i, nil
}
@@ -13692,11 +13717,11 @@ func (m *RequestUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1a
i++
i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size()))
- n168, err := m.ConditionalPut.MarshalTo(dAtA[i:])
+ n169, err := m.ConditionalPut.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n168
+ i += n169
}
return i, nil
}
@@ -13706,11 +13731,11 @@ func (m *RequestUnion_Increment) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x22
i++
i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size()))
- n169, err := m.Increment.MarshalTo(dAtA[i:])
+ n170, err := m.Increment.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n169
+ i += n170
}
return i, nil
}
@@ -13720,11 +13745,11 @@ func (m *RequestUnion_Delete) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size()))
- n170, err := m.Delete.MarshalTo(dAtA[i:])
+ n171, err := m.Delete.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n170
+ i += n171
}
return i, nil
}
@@ -13734,11 +13759,11 @@ func (m *RequestUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x32
i++
i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size()))
- n171, err := m.DeleteRange.MarshalTo(dAtA[i:])
+ n172, err := m.DeleteRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n171
+ i += n172
}
return i, nil
}
@@ -13748,11 +13773,11 @@ func (m *RequestUnion_Scan) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x3a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size()))
- n172, err := m.Scan.MarshalTo(dAtA[i:])
+ n173, err := m.Scan.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n172
+ i += n173
}
return i, nil
}
@@ -13762,11 +13787,11 @@ func (m *RequestUnion_EndTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x4a
i++
i = encodeVarintApi(dAtA, i, uint64(m.EndTxn.Size()))
- n173, err := m.EndTxn.MarshalTo(dAtA[i:])
+ n174, err := m.EndTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n173
+ i += n174
}
return i, nil
}
@@ -13776,11 +13801,11 @@ func (m *RequestUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x52
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size()))
- n174, err := m.AdminSplit.MarshalTo(dAtA[i:])
+ n175, err := m.AdminSplit.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n174
+ i += n175
}
return i, nil
}
@@ -13790,11 +13815,11 @@ func (m *RequestUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x5a
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size()))
- n175, err := m.AdminMerge.MarshalTo(dAtA[i:])
+ n176, err := m.AdminMerge.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n175
+ i += n176
}
return i, nil
}
@@ -13804,11 +13829,11 @@ func (m *RequestUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x62
i++
i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size()))
- n176, err := m.HeartbeatTxn.MarshalTo(dAtA[i:])
+ n177, err := m.HeartbeatTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n176
+ i += n177
}
return i, nil
}
@@ -13818,11 +13843,11 @@ func (m *RequestUnion_Gc) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x6a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size()))
- n177, err := m.Gc.MarshalTo(dAtA[i:])
+ n178, err := m.Gc.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n177
+ i += n178
}
return i, nil
}
@@ -13832,11 +13857,11 @@ func (m *RequestUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x72
i++
i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size()))
- n178, err := m.PushTxn.MarshalTo(dAtA[i:])
+ n179, err := m.PushTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n178
+ i += n179
}
return i, nil
}
@@ -13848,11 +13873,11 @@ func (m *RequestUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size()))
- n179, err := m.ResolveIntent.MarshalTo(dAtA[i:])
+ n180, err := m.ResolveIntent.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n179
+ i += n180
}
return i, nil
}
@@ -13864,11 +13889,11 @@ func (m *RequestUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size()))
- n180, err := m.ResolveIntentRange.MarshalTo(dAtA[i:])
+ n181, err := m.ResolveIntentRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n180
+ i += n181
}
return i, nil
}
@@ -13880,11 +13905,11 @@ func (m *RequestUnion_Merge) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size()))
- n181, err := m.Merge.MarshalTo(dAtA[i:])
+ n182, err := m.Merge.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n181
+ i += n182
}
return i, nil
}
@@ -13896,11 +13921,11 @@ func (m *RequestUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size()))
- n182, err := m.TruncateLog.MarshalTo(dAtA[i:])
+ n183, err := m.TruncateLog.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n182
+ i += n183
}
return i, nil
}
@@ -13912,11 +13937,11 @@ func (m *RequestUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size()))
- n183, err := m.RequestLease.MarshalTo(dAtA[i:])
+ n184, err := m.RequestLease.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n183
+ i += n184
}
return i, nil
}
@@ -13928,11 +13953,11 @@ func (m *RequestUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size()))
- n184, err := m.ReverseScan.MarshalTo(dAtA[i:])
+ n185, err := m.ReverseScan.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n184
+ i += n185
}
return i, nil
}
@@ -13944,11 +13969,11 @@ func (m *RequestUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size()))
- n185, err := m.ComputeChecksum.MarshalTo(dAtA[i:])
+ n186, err := m.ComputeChecksum.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n185
+ i += n186
}
return i, nil
}
@@ -13960,11 +13985,11 @@ func (m *RequestUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size()))
- n186, err := m.CheckConsistency.MarshalTo(dAtA[i:])
+ n187, err := m.CheckConsistency.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n186
+ i += n187
}
return i, nil
}
@@ -13976,11 +14001,11 @@ func (m *RequestUnion_InitPut) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size()))
- n187, err := m.InitPut.MarshalTo(dAtA[i:])
+ n188, err := m.InitPut.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n187
+ i += n188
}
return i, nil
}
@@ -13992,11 +14017,11 @@ func (m *RequestUnion_TransferLease) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.TransferLease.Size()))
- n188, err := m.TransferLease.MarshalTo(dAtA[i:])
+ n189, err := m.TransferLease.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n188
+ i += n189
}
return i, nil
}
@@ -14008,11 +14033,11 @@ func (m *RequestUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size()))
- n189, err := m.AdminTransferLease.MarshalTo(dAtA[i:])
+ n190, err := m.AdminTransferLease.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n189
+ i += n190
}
return i, nil
}
@@ -14024,11 +14049,11 @@ func (m *RequestUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size()))
- n190, err := m.LeaseInfo.MarshalTo(dAtA[i:])
+ n191, err := m.LeaseInfo.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n190
+ i += n191
}
return i, nil
}
@@ -14040,11 +14065,11 @@ func (m *RequestUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size()))
- n191, err := m.WriteBatch.MarshalTo(dAtA[i:])
+ n192, err := m.WriteBatch.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n191
+ i += n192
}
return i, nil
}
@@ -14056,11 +14081,11 @@ func (m *RequestUnion_Export) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.Export.Size()))
- n192, err := m.Export.MarshalTo(dAtA[i:])
+ n193, err := m.Export.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n192
+ i += n193
}
return i, nil
}
@@ -14072,11 +14097,11 @@ func (m *RequestUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size()))
- n193, err := m.QueryTxn.MarshalTo(dAtA[i:])
+ n194, err := m.QueryTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n193
+ i += n194
}
return i, nil
}
@@ -14088,11 +14113,11 @@ func (m *RequestUnion_Import) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.Import.Size()))
- n194, err := m.Import.MarshalTo(dAtA[i:])
+ n195, err := m.Import.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n194
+ i += n195
}
return i, nil
}
@@ -14104,11 +14129,11 @@ func (m *RequestUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size()))
- n195, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:])
+ n196, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n195
+ i += n196
}
return i, nil
}
@@ -14120,11 +14145,11 @@ func (m *RequestUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size()))
- n196, err := m.AdminScatter.MarshalTo(dAtA[i:])
+ n197, err := m.AdminScatter.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n196
+ i += n197
}
return i, nil
}
@@ -14136,11 +14161,11 @@ func (m *RequestUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size()))
- n197, err := m.AddSstable.MarshalTo(dAtA[i:])
+ n198, err := m.AddSstable.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n197
+ i += n198
}
return i, nil
}
@@ -14152,11 +14177,11 @@ func (m *RequestUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size()))
- n198, err := m.ClearRange.MarshalTo(dAtA[i:])
+ n199, err := m.ClearRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n198
+ i += n199
}
return i, nil
}
@@ -14168,11 +14193,11 @@ func (m *RequestUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size()))
- n199, err := m.RecomputeStats.MarshalTo(dAtA[i:])
+ n200, err := m.RecomputeStats.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n199
+ i += n200
}
return i, nil
}
@@ -14184,11 +14209,11 @@ func (m *RequestUnion_Refresh) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size()))
- n200, err := m.Refresh.MarshalTo(dAtA[i:])
+ n201, err := m.Refresh.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n200
+ i += n201
}
return i, nil
}
@@ -14200,11 +14225,11 @@ func (m *RequestUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size()))
- n201, err := m.RefreshRange.MarshalTo(dAtA[i:])
+ n202, err := m.RefreshRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n201
+ i += n202
}
return i, nil
}
@@ -14216,11 +14241,11 @@ func (m *RequestUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size()))
- n202, err := m.QueryIntent.MarshalTo(dAtA[i:])
+ n203, err := m.QueryIntent.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n202
+ i += n203
}
return i, nil
}
@@ -14232,11 +14257,11 @@ func (m *RequestUnion_Subsume) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size()))
- n203, err := m.Subsume.MarshalTo(dAtA[i:])
+ n204, err := m.Subsume.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n203
+ i += n204
}
return i, nil
}
@@ -14248,11 +14273,11 @@ func (m *RequestUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size()))
- n204, err := m.RangeStats.MarshalTo(dAtA[i:])
+ n205, err := m.RangeStats.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n204
+ i += n205
}
return i, nil
}
@@ -14264,11 +14289,11 @@ func (m *RequestUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size()))
- n205, err := m.AdminRelocateRange.MarshalTo(dAtA[i:])
+ n206, err := m.AdminRelocateRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n205
+ i += n206
}
return i, nil
}
@@ -14280,11 +14305,11 @@ func (m *RequestUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size()))
- n206, err := m.RecoverTxn.MarshalTo(dAtA[i:])
+ n207, err := m.RecoverTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n206
+ i += n207
}
return i, nil
}
@@ -14296,11 +14321,11 @@ func (m *RequestUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size()))
- n207, err := m.AdminUnsplit.MarshalTo(dAtA[i:])
+ n208, err := m.AdminUnsplit.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n207
+ i += n208
}
return i, nil
}
@@ -14312,11 +14337,11 @@ func (m *RequestUnion_RevertRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x3
i++
i = encodeVarintApi(dAtA, i, uint64(m.RevertRange.Size()))
- n208, err := m.RevertRange.MarshalTo(dAtA[i:])
+ n209, err := m.RevertRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n208
+ i += n209
}
return i, nil
}
@@ -14328,11 +14353,11 @@ func (m *RequestUnion_AdminVerifyProtectedTimestamp) MarshalTo(dAtA []byte) (int
dAtA[i] = 0x3
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminVerifyProtectedTimestamp.Size()))
- n209, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:])
+ n210, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n209
+ i += n210
}
return i, nil
}
@@ -14344,11 +14369,11 @@ func (m *RequestUnion_Migrate) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x3
i++
i = encodeVarintApi(dAtA, i, uint64(m.Migrate.Size()))
- n210, err := m.Migrate.MarshalTo(dAtA[i:])
+ n211, err := m.Migrate.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n210
+ i += n211
}
return i, nil
}
@@ -14368,11 +14393,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) {
var l int
_ = l
if m.Value != nil {
- nn211, err := m.Value.MarshalTo(dAtA[i:])
+ nn212, err := m.Value.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += nn211
+ i += nn212
}
return i, nil
}
@@ -14383,11 +14408,11 @@ func (m *ResponseUnion_Get) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Get.Size()))
- n212, err := m.Get.MarshalTo(dAtA[i:])
+ n213, err := m.Get.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n212
+ i += n213
}
return i, nil
}
@@ -14397,11 +14422,11 @@ func (m *ResponseUnion_Put) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Put.Size()))
- n213, err := m.Put.MarshalTo(dAtA[i:])
+ n214, err := m.Put.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n213
+ i += n214
}
return i, nil
}
@@ -14411,11 +14436,11 @@ func (m *ResponseUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1a
i++
i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size()))
- n214, err := m.ConditionalPut.MarshalTo(dAtA[i:])
+ n215, err := m.ConditionalPut.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n214
+ i += n215
}
return i, nil
}
@@ -14425,11 +14450,11 @@ func (m *ResponseUnion_Increment) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x22
i++
i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size()))
- n215, err := m.Increment.MarshalTo(dAtA[i:])
+ n216, err := m.Increment.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n215
+ i += n216
}
return i, nil
}
@@ -14439,11 +14464,11 @@ func (m *ResponseUnion_Delete) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size()))
- n216, err := m.Delete.MarshalTo(dAtA[i:])
+ n217, err := m.Delete.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n216
+ i += n217
}
return i, nil
}
@@ -14453,11 +14478,11 @@ func (m *ResponseUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x32
i++
i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size()))
- n217, err := m.DeleteRange.MarshalTo(dAtA[i:])
+ n218, err := m.DeleteRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n217
+ i += n218
}
return i, nil
}
@@ -14467,11 +14492,11 @@ func (m *ResponseUnion_Scan) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x3a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size()))
- n218, err := m.Scan.MarshalTo(dAtA[i:])
+ n219, err := m.Scan.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n218
+ i += n219
}
return i, nil
}
@@ -14481,11 +14506,11 @@ func (m *ResponseUnion_EndTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x4a
i++
i = encodeVarintApi(dAtA, i, uint64(m.EndTxn.Size()))
- n219, err := m.EndTxn.MarshalTo(dAtA[i:])
+ n220, err := m.EndTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n219
+ i += n220
}
return i, nil
}
@@ -14495,11 +14520,11 @@ func (m *ResponseUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x52
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size()))
- n220, err := m.AdminSplit.MarshalTo(dAtA[i:])
+ n221, err := m.AdminSplit.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n220
+ i += n221
}
return i, nil
}
@@ -14509,11 +14534,11 @@ func (m *ResponseUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x5a
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size()))
- n221, err := m.AdminMerge.MarshalTo(dAtA[i:])
+ n222, err := m.AdminMerge.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n221
+ i += n222
}
return i, nil
}
@@ -14523,11 +14548,11 @@ func (m *ResponseUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x62
i++
i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size()))
- n222, err := m.HeartbeatTxn.MarshalTo(dAtA[i:])
+ n223, err := m.HeartbeatTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n222
+ i += n223
}
return i, nil
}
@@ -14537,11 +14562,11 @@ func (m *ResponseUnion_Gc) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x6a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size()))
- n223, err := m.Gc.MarshalTo(dAtA[i:])
+ n224, err := m.Gc.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n223
+ i += n224
}
return i, nil
}
@@ -14551,11 +14576,11 @@ func (m *ResponseUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x72
i++
i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size()))
- n224, err := m.PushTxn.MarshalTo(dAtA[i:])
+ n225, err := m.PushTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n224
+ i += n225
}
return i, nil
}
@@ -14567,11 +14592,11 @@ func (m *ResponseUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size()))
- n225, err := m.ResolveIntent.MarshalTo(dAtA[i:])
+ n226, err := m.ResolveIntent.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n225
+ i += n226
}
return i, nil
}
@@ -14583,11 +14608,11 @@ func (m *ResponseUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size()))
- n226, err := m.ResolveIntentRange.MarshalTo(dAtA[i:])
+ n227, err := m.ResolveIntentRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n226
+ i += n227
}
return i, nil
}
@@ -14599,11 +14624,11 @@ func (m *ResponseUnion_Merge) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size()))
- n227, err := m.Merge.MarshalTo(dAtA[i:])
+ n228, err := m.Merge.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n227
+ i += n228
}
return i, nil
}
@@ -14615,11 +14640,11 @@ func (m *ResponseUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size()))
- n228, err := m.TruncateLog.MarshalTo(dAtA[i:])
+ n229, err := m.TruncateLog.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n228
+ i += n229
}
return i, nil
}
@@ -14631,11 +14656,11 @@ func (m *ResponseUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size()))
- n229, err := m.RequestLease.MarshalTo(dAtA[i:])
+ n230, err := m.RequestLease.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n229
+ i += n230
}
return i, nil
}
@@ -14647,11 +14672,11 @@ func (m *ResponseUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size()))
- n230, err := m.ReverseScan.MarshalTo(dAtA[i:])
+ n231, err := m.ReverseScan.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n230
+ i += n231
}
return i, nil
}
@@ -14663,11 +14688,11 @@ func (m *ResponseUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size()))
- n231, err := m.ComputeChecksum.MarshalTo(dAtA[i:])
+ n232, err := m.ComputeChecksum.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n231
+ i += n232
}
return i, nil
}
@@ -14679,11 +14704,11 @@ func (m *ResponseUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size()))
- n232, err := m.CheckConsistency.MarshalTo(dAtA[i:])
+ n233, err := m.CheckConsistency.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n232
+ i += n233
}
return i, nil
}
@@ -14695,11 +14720,11 @@ func (m *ResponseUnion_InitPut) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size()))
- n233, err := m.InitPut.MarshalTo(dAtA[i:])
+ n234, err := m.InitPut.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n233
+ i += n234
}
return i, nil
}
@@ -14711,11 +14736,11 @@ func (m *ResponseUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size()))
- n234, err := m.AdminTransferLease.MarshalTo(dAtA[i:])
+ n235, err := m.AdminTransferLease.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n234
+ i += n235
}
return i, nil
}
@@ -14727,11 +14752,11 @@ func (m *ResponseUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size()))
- n235, err := m.LeaseInfo.MarshalTo(dAtA[i:])
+ n236, err := m.LeaseInfo.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n235
+ i += n236
}
return i, nil
}
@@ -14743,11 +14768,11 @@ func (m *ResponseUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size()))
- n236, err := m.WriteBatch.MarshalTo(dAtA[i:])
+ n237, err := m.WriteBatch.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n236
+ i += n237
}
return i, nil
}
@@ -14759,11 +14784,11 @@ func (m *ResponseUnion_Export) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.Export.Size()))
- n237, err := m.Export.MarshalTo(dAtA[i:])
+ n238, err := m.Export.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n237
+ i += n238
}
return i, nil
}
@@ -14775,11 +14800,11 @@ func (m *ResponseUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size()))
- n238, err := m.QueryTxn.MarshalTo(dAtA[i:])
+ n239, err := m.QueryTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n238
+ i += n239
}
return i, nil
}
@@ -14791,11 +14816,11 @@ func (m *ResponseUnion_Import) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.Import.Size()))
- n239, err := m.Import.MarshalTo(dAtA[i:])
+ n240, err := m.Import.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n239
+ i += n240
}
return i, nil
}
@@ -14807,11 +14832,11 @@ func (m *ResponseUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error)
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size()))
- n240, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:])
+ n241, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n240
+ i += n241
}
return i, nil
}
@@ -14823,11 +14848,11 @@ func (m *ResponseUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size()))
- n241, err := m.AdminScatter.MarshalTo(dAtA[i:])
+ n242, err := m.AdminScatter.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n241
+ i += n242
}
return i, nil
}
@@ -14839,11 +14864,11 @@ func (m *ResponseUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size()))
- n242, err := m.AddSstable.MarshalTo(dAtA[i:])
+ n243, err := m.AddSstable.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n242
+ i += n243
}
return i, nil
}
@@ -14855,11 +14880,11 @@ func (m *ResponseUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size()))
- n243, err := m.ClearRange.MarshalTo(dAtA[i:])
+ n244, err := m.ClearRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n243
+ i += n244
}
return i, nil
}
@@ -14871,11 +14896,11 @@ func (m *ResponseUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size()))
- n244, err := m.RecomputeStats.MarshalTo(dAtA[i:])
+ n245, err := m.RecomputeStats.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n244
+ i += n245
}
return i, nil
}
@@ -14887,11 +14912,11 @@ func (m *ResponseUnion_Refresh) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size()))
- n245, err := m.Refresh.MarshalTo(dAtA[i:])
+ n246, err := m.Refresh.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n245
+ i += n246
}
return i, nil
}
@@ -14903,11 +14928,11 @@ func (m *ResponseUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size()))
- n246, err := m.RefreshRange.MarshalTo(dAtA[i:])
+ n247, err := m.RefreshRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n246
+ i += n247
}
return i, nil
}
@@ -14919,11 +14944,11 @@ func (m *ResponseUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size()))
- n247, err := m.QueryIntent.MarshalTo(dAtA[i:])
+ n248, err := m.QueryIntent.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n247
+ i += n248
}
return i, nil
}
@@ -14935,11 +14960,11 @@ func (m *ResponseUnion_Subsume) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size()))
- n248, err := m.Subsume.MarshalTo(dAtA[i:])
+ n249, err := m.Subsume.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n248
+ i += n249
}
return i, nil
}
@@ -14951,11 +14976,11 @@ func (m *ResponseUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size()))
- n249, err := m.RangeStats.MarshalTo(dAtA[i:])
+ n250, err := m.RangeStats.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n249
+ i += n250
}
return i, nil
}
@@ -14967,11 +14992,11 @@ func (m *ResponseUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size()))
- n250, err := m.AdminRelocateRange.MarshalTo(dAtA[i:])
+ n251, err := m.AdminRelocateRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n250
+ i += n251
}
return i, nil
}
@@ -14983,11 +15008,11 @@ func (m *ResponseUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size()))
- n251, err := m.RecoverTxn.MarshalTo(dAtA[i:])
+ n252, err := m.RecoverTxn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n251
+ i += n252
}
return i, nil
}
@@ -14999,11 +15024,11 @@ func (m *ResponseUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size()))
- n252, err := m.AdminUnsplit.MarshalTo(dAtA[i:])
+ n253, err := m.AdminUnsplit.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n252
+ i += n253
}
return i, nil
}
@@ -15015,11 +15040,11 @@ func (m *ResponseUnion_RevertRange) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x3
i++
i = encodeVarintApi(dAtA, i, uint64(m.RevertRange.Size()))
- n253, err := m.RevertRange.MarshalTo(dAtA[i:])
+ n254, err := m.RevertRange.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n253
+ i += n254
}
return i, nil
}
@@ -15031,11 +15056,11 @@ func (m *ResponseUnion_AdminVerifyProtectedTimestamp) MarshalTo(dAtA []byte) (in
dAtA[i] = 0x3
i++
i = encodeVarintApi(dAtA, i, uint64(m.AdminVerifyProtectedTimestamp.Size()))
- n254, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:])
+ n255, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n254
+ i += n255
}
return i, nil
}
@@ -15047,11 +15072,11 @@ func (m *ResponseUnion_Migrate) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x3
i++
i = encodeVarintApi(dAtA, i, uint64(m.Migrate.Size()))
- n255, err := m.Migrate.MarshalTo(dAtA[i:])
+ n256, err := m.Migrate.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n255
+ i += n256
}
return i, nil
}
@@ -15073,19 +15098,19 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size()))
- n256, err := m.Timestamp.MarshalTo(dAtA[i:])
+ n257, err := m.Timestamp.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n256
+ i += n257
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Replica.Size()))
- n257, err := m.Replica.MarshalTo(dAtA[i:])
+ n258, err := m.Replica.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n257
+ i += n258
if m.RangeID != 0 {
dAtA[i] = 0x18
i++
@@ -15101,11 +15126,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size()))
- n258, err := m.Txn.MarshalTo(dAtA[i:])
+ n259, err := m.Txn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n258
+ i += n259
}
if m.ReadConsistency != 0 {
dAtA[i] = 0x30
@@ -15164,11 +15189,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1
i++
i = encodeVarintApi(dAtA, i, uint64(m.ClientRangeInfo.Size()))
- n259, err := m.ClientRangeInfo.MarshalTo(dAtA[i:])
+ n260, err := m.ClientRangeInfo.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n259
+ i += n260
if m.WaitPolicy != 0 {
dAtA[i] = 0x90
i++
@@ -15197,11 +15222,11 @@ func (m *BatchRequest) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Header.Size()))
- n260, err := m.Header.MarshalTo(dAtA[i:])
+ n261, err := m.Header.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n260
+ i += n261
if len(m.Requests) > 0 {
for _, msg := range m.Requests {
dAtA[i] = 0x12
@@ -15235,11 +15260,11 @@ func (m *BatchResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.BatchResponse_Header.Size()))
- n261, err := m.BatchResponse_Header.MarshalTo(dAtA[i:])
+ n262, err := m.BatchResponse_Header.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n261
+ i += n262
if len(m.Responses) > 0 {
for _, msg := range m.Responses {
dAtA[i] = 0x12
@@ -15274,38 +15299,38 @@ func (m *BatchResponse_Header) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Error.Size()))
- n262, err := m.Error.MarshalTo(dAtA[i:])
+ n263, err := m.Error.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n262
+ i += n263
}
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size()))
- n263, err := m.Timestamp.MarshalTo(dAtA[i:])
+ n264, err := m.Timestamp.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n263
+ i += n264
if m.Txn != nil {
dAtA[i] = 0x1a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size()))
- n264, err := m.Txn.MarshalTo(dAtA[i:])
+ n265, err := m.Txn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n264
+ i += n265
}
dAtA[i] = 0x2a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Now.Size()))
- n265, err := m.Now.MarshalTo(dAtA[i:])
+ n266, err := m.Now.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n265
+ i += n266
if len(m.CollectedSpans) > 0 {
for _, msg := range m.CollectedSpans {
dAtA[i] = 0x32
@@ -15420,11 +15445,11 @@ func (m *RangeLookupResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x1a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Error.Size()))
- n266, err := m.Error.MarshalTo(dAtA[i:])
+ n267, err := m.Error.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n266
+ i += n267
}
return i, nil
}
@@ -15447,19 +15472,19 @@ func (m *RangeFeedRequest) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Header.Size()))
- n267, err := m.Header.MarshalTo(dAtA[i:])
+ n268, err := m.Header.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n267
+ i += n268
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Span.Size()))
- n268, err := m.Span.MarshalTo(dAtA[i:])
+ n269, err := m.Span.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n268
+ i += n269
if m.WithDiff {
dAtA[i] = 0x18
i++
@@ -15497,19 +15522,19 @@ func (m *RangeFeedValue) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Value.Size()))
- n269, err := m.Value.MarshalTo(dAtA[i:])
+ n270, err := m.Value.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n269
+ i += n270
dAtA[i] = 0x1a
i++
i = encodeVarintApi(dAtA, i, uint64(m.PrevValue.Size()))
- n270, err := m.PrevValue.MarshalTo(dAtA[i:])
+ n271, err := m.PrevValue.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n270
+ i += n271
return i, nil
}
@@ -15531,19 +15556,19 @@ func (m *RangeFeedCheckpoint) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Span.Size()))
- n271, err := m.Span.MarshalTo(dAtA[i:])
+ n272, err := m.Span.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n271
+ i += n272
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.ResolvedTS.Size()))
- n272, err := m.ResolvedTS.MarshalTo(dAtA[i:])
+ n273, err := m.ResolvedTS.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n272
+ i += n273
return i, nil
}
@@ -15565,11 +15590,11 @@ func (m *RangeFeedError) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Error.Size()))
- n273, err := m.Error.MarshalTo(dAtA[i:])
+ n274, err := m.Error.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n273
+ i += n274
return i, nil
}
@@ -15592,31 +15617,31 @@ func (m *RangeFeedEvent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.Val.Size()))
- n274, err := m.Val.MarshalTo(dAtA[i:])
+ n275, err := m.Val.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n274
+ i += n275
}
if m.Checkpoint != nil {
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Checkpoint.Size()))
- n275, err := m.Checkpoint.MarshalTo(dAtA[i:])
+ n276, err := m.Checkpoint.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n275
+ i += n276
}
if m.Error != nil {
dAtA[i] = 0x1a
i++
i = encodeVarintApi(dAtA, i, uint64(m.Error.Size()))
- n276, err := m.Error.MarshalTo(dAtA[i:])
+ n277, err := m.Error.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n276
+ i += n277
}
return i, nil
}
@@ -15719,11 +15744,11 @@ func (m *GossipSubscriptionEvent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.Content.Size()))
- n277, err := m.Content.MarshalTo(dAtA[i:])
+ n278, err := m.Content.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n277
+ i += n278
if len(m.PatternMatched) > 0 {
dAtA[i] = 0x1a
i++
@@ -15734,11 +15759,11 @@ func (m *GossipSubscriptionEvent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x22
i++
i = encodeVarintApi(dAtA, i, uint64(m.Error.Size()))
- n278, err := m.Error.MarshalTo(dAtA[i:])
+ n279, err := m.Error.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n278
+ i += n279
}
return i, nil
}
@@ -15762,11 +15787,11 @@ func (m *JoinNodeRequest) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintApi(dAtA, i, uint64(m.BinaryVersion.Size()))
- n279, err := m.BinaryVersion.MarshalTo(dAtA[i:])
+ n280, err := m.BinaryVersion.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n279
+ i += n280
}
return i, nil
}
@@ -15806,11 +15831,11 @@ func (m *JoinNodeResponse) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x22
i++
i = encodeVarintApi(dAtA, i, uint64(m.ActiveVersion.Size()))
- n280, err := m.ActiveVersion.MarshalTo(dAtA[i:])
+ n281, err := m.ActiveVersion.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n280
+ i += n281
}
return i, nil
}
@@ -15839,19 +15864,19 @@ func (m *ContentionEvent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintApi(dAtA, i, uint64(m.TxnMeta.Size()))
- n281, err := m.TxnMeta.MarshalTo(dAtA[i:])
+ n282, err := m.TxnMeta.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n281
+ i += n282
dAtA[i] = 0x1a
i++
i = encodeVarintApi(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration)))
- n282, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:])
+ n283, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:])
if err != nil {
return 0, err
}
- i += n282
+ i += n283
return i, nil
}
@@ -17721,6 +17746,10 @@ func (m *SubsumeResponse) Size() (n int) {
}
l = m.FreezeStart.Size()
n += 1 + l + sovApi(uint64(l))
+ if m.ReadSummary != nil {
+ l = m.ReadSummary.Size()
+ n += 1 + l + sovApi(uint64(l))
+ }
return n
}
@@ -32962,6 +32991,39 @@ func (m *SubsumeResponse) Unmarshal(dAtA []byte) error {
return err
}
iNdEx = postIndex
+ case 6:
+ 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:])
@@ -38914,520 +38976,523 @@ var (
ErrIntOverflowApi = fmt.Errorf("proto: integer overflow")
)
-func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_f2b2489e6aa3dd67) }
+func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_70ddf268aebf8b6a) }
-var fileDescriptor_api_f2b2489e6aa3dd67 = []byte{
- // 8182 bytes of a gzipped FileDescriptorProto
+var fileDescriptor_api_70ddf268aebf8b6a = []byte{
+ // 8228 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, 0xa4, 0x48, 0xea, 0xaa, 0x3f, 0xd8, 0x9c, 0x99, 0x56,
- 0x77, 0xf5, 0x77, 0xef, 0x8c, 0x34, 0xdd, 0xbd, 0xf3, 0x9f, 0xf1, 0xf4, 0x78, 0xd6, 0x12, 0xc5,
+ 0x76, 0x1f, 0xae, 0x22, 0x29, 0x89, 0x3c, 0xa4, 0x48, 0xea, 0xaa, 0x3f, 0xd8, 0x9a, 0x99, 0x56,
+ 0x77, 0xf5, 0x77, 0xef, 0x0c, 0x35, 0xdd, 0xbd, 0xf3, 0x9f, 0xf1, 0xf4, 0x78, 0xd6, 0x12, 0xc5,
0x6e, 0x52, 0x6a, 0xa9, 0xd5, 0x45, 0xaa, 0x1b, 0x33, 0x5e, 0xff, 0x6b, 0x4b, 0x55, 0x57, 0x54,
0xad, 0xc8, 0x2a, 0x76, 0x55, 0x51, 0x1f, 0x03, 0x04, 0x88, 0x63, 0xc3, 0xd9, 0x27, 0x63, 0x1f,
0x0c, 0x64, 0x0d, 0x07, 0xf1, 0x3a, 0x1b, 0xc4, 0x0f, 0x01, 0x92, 0x00, 0x09, 0xf2, 0x85, 0xc4,
- 0x7e, 0x31, 0x90, 0x45, 0xe0, 0xc4, 0xeb, 0xa7, 0x18, 0x01, 0xa2, 0xd8, 0xda, 0xbc, 0x04, 0x0e,
- 0x8c, 0x20, 0x08, 0x60, 0x60, 0x1e, 0x82, 0xe0, 0x7e, 0xd4, 0x17, 0x59, 0xfc, 0x50, 0x4f, 0x6d,
- 0x32, 0x80, 0x5f, 0x24, 0xd6, 0xb9, 0xf7, 0x9c, 0xba, 0xf7, 0xdc, 0x7b, 0xcf, 0x3d, 0xbf, 0x5b,
- 0xe7, 0xde, 0x0b, 0xf3, 0x96, 0xa9, 0xa8, 0xfb, 0xdd, 0xdd, 0x65, 0xa5, 0xab, 0x2f, 0x75, 0x2d,
- 0xd3, 0x31, 0xd1, 0xbc, 0x6a, 0xaa, 0x07, 0x94, 0xbc, 0xc4, 0x13, 0xcb, 0xf7, 0x0f, 0x0e, 0x97,
- 0x0f, 0x0e, 0x6d, 0x6c, 0x1d, 0x62, 0x6b, 0x59, 0x35, 0x0d, 0xb5, 0x67, 0x59, 0xd8, 0x50, 0x4f,
- 0x96, 0xdb, 0xa6, 0x7a, 0x40, 0xff, 0xe8, 0x46, 0x8b, 0xb1, 0x97, 0x91, 0x2b, 0x51, 0x53, 0x1c,
- 0x85, 0xd3, 0x2e, 0xb8, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x25, 0x97, 0xda, 0xc1, 0x8e,
- 0x12, 0xc8, 0xfd, 0x96, 0xed, 0x98, 0x96, 0xd2, 0xc2, 0xcb, 0xd8, 0x68, 0xe9, 0x06, 0x26, 0x19,
- 0x0e, 0x55, 0x95, 0x27, 0xbe, 0x1d, 0x99, 0xf8, 0x88, 0xa7, 0x96, 0x7a, 0x8e, 0xde, 0x5e, 0xde,
- 0x6f, 0xab, 0xcb, 0x8e, 0xde, 0xc1, 0xb6, 0xa3, 0x74, 0xba, 0x3c, 0xe5, 0x3e, 0x4d, 0x71, 0x2c,
- 0x45, 0xd5, 0x8d, 0x96, 0xfb, 0xbf, 0xbb, 0xbb, 0x6c, 0x61, 0xd5, 0xb4, 0x34, 0xac, 0xc9, 0x76,
- 0x57, 0x31, 0xdc, 0xe2, 0xb6, 0xcc, 0x96, 0x49, 0x7f, 0x2e, 0x93, 0x5f, 0x9c, 0x7a, 0xb5, 0x65,
- 0x9a, 0xad, 0x36, 0x5e, 0xa6, 0x4f, 0xbb, 0xbd, 0xbd, 0x65, 0xad, 0x67, 0x29, 0x8e, 0x6e, 0x72,
- 0x2e, 0xf1, 0x9f, 0x09, 0x30, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac, 0x68, 0xd8, 0x42,
- 0x57, 0x20, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0x6b, 0xc2, 0xdd, 0xdc, 0xea, 0xec, 0x97, 0xa7, 0x8b,
- 0xc9, 0x0d, 0x7c, 0x22, 0x11, 0x1a, 0xba, 0x06, 0xb3, 0xd8, 0xd0, 0x64, 0x92, 0x9c, 0x0a, 0x27,
- 0xcf, 0x60, 0x43, 0xdb, 0xc0, 0x27, 0xe8, 0xdb, 0x90, 0xb6, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xe9,
- 0x6b, 0xc2, 0xdd, 0xe9, 0xd5, 0x5f, 0xf8, 0xf2, 0x74, 0xf1, 0x93, 0x96, 0xee, 0xec, 0xf7, 0x76,
- 0x97, 0x54, 0xb3, 0xb3, 0xec, 0xb5, 0x93, 0xb6, 0xeb, 0xff, 0x5e, 0xee, 0x1e, 0xb4, 0x96, 0xfb,
- 0x75, 0xb4, 0xd4, 0x3c, 0x36, 0x1a, 0xf8, 0xb5, 0xe4, 0x49, 0x5c, 0x4f, 0xa5, 0x85, 0x62, 0x62,
- 0x3d, 0x95, 0x4e, 0x14, 0x93, 0xe2, 0x1f, 0x25, 0x20, 0x2f, 0x61, 0xbb, 0x6b, 0x1a, 0x36, 0xe6,
- 0x25, 0x7f, 0x1f, 0x92, 0xce, 0xb1, 0x41, 0x4b, 0x9e, 0x7d, 0x78, 0x75, 0x69, 0xa0, 0x47, 0x2c,
- 0x35, 0x2d, 0xc5, 0xb0, 0x15, 0x95, 0x54, 0x5f, 0x22, 0x59, 0xd1, 0x47, 0x90, 0xb5, 0xb0, 0xdd,
- 0xeb, 0x60, 0xaa, 0x48, 0x5a, 0xa9, 0xec, 0xc3, 0xcb, 0x11, 0x9c, 0x8d, 0xae, 0x62, 0x48, 0xc0,
- 0xf2, 0x92, 0xdf, 0xe8, 0x0a, 0xa4, 0x8d, 0x5e, 0x87, 0xa8, 0xc2, 0xa6, 0x15, 0x4d, 0x4a, 0xb3,
- 0x46, 0xaf, 0xb3, 0x81, 0x4f, 0x6c, 0xd4, 0x80, 0x39, 0x2e, 0xd4, 0xc2, 0x8a, 0x6d, 0x1a, 0xa5,
- 0xd9, 0x6b, 0xc2, 0xdd, 0xfc, 0xc3, 0xa5, 0x08, 0xb1, 0xe1, 0x0a, 0x90, 0xc7, 0x5e, 0x07, 0x4b,
- 0x94, 0x4b, 0xca, 0x59, 0x81, 0x27, 0xf4, 0x16, 0x64, 0xc8, 0xfb, 0x76, 0x4f, 0x1c, 0x6c, 0x97,
- 0xd2, 0xf4, 0x85, 0xa4, 0x00, 0xab, 0xe4, 0x59, 0xfc, 0x14, 0x72, 0x41, 0x56, 0x84, 0x20, 0x2f,
- 0x55, 0x1b, 0x3b, 0x9b, 0x55, 0x79, 0x67, 0x6b, 0x63, 0xeb, 0xf9, 0xab, 0xad, 0xe2, 0x14, 0xba,
- 0x00, 0x45, 0x4e, 0xdb, 0xa8, 0x7e, 0x26, 0x3f, 0xab, 0x6f, 0xd6, 0x9b, 0x45, 0xa1, 0x9c, 0xfa,
- 0xde, 0x8f, 0xae, 0x4e, 0xad, 0xa7, 0xd2, 0x33, 0xc5, 0x59, 0xf1, 0x47, 0x02, 0xc0, 0x53, 0xec,
- 0xf0, 0xde, 0x80, 0x56, 0x61, 0x66, 0x9f, 0x16, 0xab, 0x24, 0x50, 0xb5, 0x5c, 0x8b, 0x2c, 0x7f,
- 0xa0, 0xe7, 0xac, 0xa6, 0x7f, 0x7c, 0xba, 0x38, 0xf5, 0x93, 0xd3, 0x45, 0x41, 0xe2, 0x9c, 0xe8,
- 0x05, 0x64, 0x0f, 0xf0, 0x89, 0xcc, 0xc7, 0x5a, 0x29, 0x41, 0x15, 0xf1, 0x7e, 0x40, 0xd0, 0xc1,
- 0xe1, 0x92, 0x3b, 0x44, 0x97, 0x02, 0x43, 0x74, 0x89, 0x70, 0x2c, 0x35, 0x1c, 0x0b, 0x1b, 0x2d,
- 0x67, 0x5f, 0x82, 0x03, 0x7c, 0xf2, 0x8c, 0xc9, 0x10, 0x7f, 0x5f, 0x80, 0x2c, 0x2d, 0x25, 0xd3,
- 0x1c, 0xaa, 0xf4, 0x15, 0xf3, 0xfa, 0x58, 0x35, 0x47, 0x94, 0x73, 0x09, 0xa6, 0x0f, 0x95, 0x76,
- 0x0f, 0xd3, 0x12, 0x66, 0x1f, 0x96, 0x22, 0x64, 0xbc, 0x24, 0xe9, 0x12, 0xcb, 0x86, 0x1e, 0x43,
- 0x4e, 0x37, 0x1c, 0x6c, 0x38, 0x32, 0x63, 0x4b, 0x8e, 0x61, 0xcb, 0xb2, 0xdc, 0xf4, 0x41, 0xfc,
- 0xa7, 0x02, 0xc0, 0x76, 0x2f, 0x56, 0x3d, 0x7f, 0x73, 0xc2, 0xf2, 0xaf, 0xa6, 0x08, 0xab, 0x5b,
- 0x8b, 0x4b, 0x30, 0xa3, 0x1b, 0x6d, 0xdd, 0x60, 0xe5, 0x4f, 0x4b, 0xfc, 0x09, 0x5d, 0x80, 0xe9,
- 0xdd, 0xb6, 0x6e, 0x68, 0x74, 0x3c, 0xa4, 0x25, 0xf6, 0x20, 0x4a, 0x90, 0xa5, 0xa5, 0x8e, 0x51,
- 0xef, 0xe2, 0x69, 0x02, 0x2e, 0x56, 0x4c, 0x43, 0xd3, 0xc9, 0x90, 0x54, 0xda, 0x5f, 0x0b, 0xad,
- 0xac, 0xc3, 0x05, 0x0d, 0x77, 0x2d, 0xac, 0x2a, 0x0e, 0xd6, 0x64, 0x7c, 0xdc, 0x9d, 0xb0, 0x8d,
- 0x91, 0xcf, 0x55, 0x3d, 0xee, 0x52, 0x5a, 0xb4, 0x26, 0xd1, 0x37, 0xe1, 0xb2, 0xd2, 0x6e, 0x9b,
- 0x47, 0xb2, 0xbe, 0x27, 0x6b, 0x26, 0xb6, 0x65, 0xc3, 0x74, 0x64, 0x7c, 0xac, 0xdb, 0x0e, 0x35,
- 0x25, 0x69, 0x69, 0x81, 0x26, 0xd7, 0xf7, 0xd6, 0x4c, 0x6c, 0x6f, 0x99, 0x4e, 0x95, 0x24, 0x11,
- 0x0b, 0x40, 0x0a, 0xc3, 0x2c, 0xc0, 0x0c, 0x31, 0xbf, 0x52, 0x1a, 0x1f, 0x77, 0xa9, 0x05, 0x08,
- 0x34, 0xe5, 0x6c, 0xb0, 0x29, 0xc5, 0x5f, 0x82, 0x4b, 0xfd, 0xfa, 0x8d, 0xb3, 0xfd, 0xfe, 0x50,
- 0x80, 0x7c, 0xdd, 0xd0, 0x9d, 0xaf, 0x45, 0xc3, 0x79, 0xca, 0x4e, 0x06, 0x95, 0x7d, 0x1f, 0x8a,
- 0x7b, 0x8a, 0xde, 0x7e, 0x6e, 0x34, 0xcd, 0xce, 0xae, 0xed, 0x98, 0x06, 0xb6, 0x79, 0x6b, 0x0c,
- 0xd0, 0xc5, 0x97, 0x50, 0xf0, 0x6a, 0x13, 0xa7, 0x9a, 0x1c, 0x28, 0xd6, 0x0d, 0xd5, 0xc2, 0x1d,
- 0x6c, 0xc4, 0xaa, 0xa7, 0xb7, 0x21, 0xa3, 0xbb, 0x72, 0xa9, 0xae, 0x92, 0x92, 0x4f, 0x10, 0x7b,
- 0x30, 0x1f, 0x78, 0x6b, 0x9c, 0xe6, 0x92, 0x4c, 0x46, 0xf8, 0x48, 0xf6, 0xdb, 0x88, 0x4c, 0x46,
- 0xf8, 0x88, 0x99, 0xb7, 0x06, 0xcc, 0xad, 0xe1, 0x36, 0x76, 0x70, 0x8c, 0x35, 0x15, 0x77, 0x20,
- 0xef, 0x0a, 0x8d, 0xb3, 0x61, 0xfe, 0x96, 0x00, 0x88, 0xcb, 0x55, 0x8c, 0x56, 0x9c, 0x25, 0x46,
- 0x8b, 0xc4, 0xb5, 0x70, 0x7a, 0x96, 0xc1, 0x7c, 0x04, 0xd6, 0x27, 0x81, 0x91, 0xa8, 0x9b, 0xe0,
- 0x0f, 0xd9, 0x54, 0x70, 0xc8, 0x72, 0xf7, 0xe6, 0x08, 0x16, 0x42, 0x05, 0x8b, 0xb7, 0xf9, 0x52,
- 0xb4, 0x4c, 0x89, 0x6b, 0xc9, 0xa0, 0x0f, 0x47, 0x89, 0xe2, 0x0f, 0x04, 0x98, 0xaf, 0xb4, 0xb1,
- 0x62, 0xc5, 0xae, 0x91, 0x6f, 0x41, 0x5a, 0xc3, 0x8a, 0x46, 0xab, 0xcc, 0x06, 0xf6, 0x3b, 0x01,
- 0x29, 0xc4, 0xd3, 0x5d, 0xda, 0x6f, 0xab, 0x4b, 0x4d, 0xd7, 0x07, 0xe6, 0xa3, 0xdb, 0x63, 0x12,
- 0x3f, 0x03, 0x14, 0x2c, 0x59, 0x9c, 0x1d, 0xe1, 0x7f, 0x09, 0x80, 0x24, 0x7c, 0x88, 0x2d, 0x27,
- 0xf6, 0x6a, 0xaf, 0x41, 0xd6, 0x51, 0xac, 0x16, 0x76, 0x64, 0xe2, 0xdd, 0x9f, 0xa7, 0xe6, 0xc0,
- 0xf8, 0x08, 0x19, 0x35, 0xe1, 0x0e, 0x36, 0x94, 0xdd, 0x36, 0xa6, 0x52, 0xe4, 0x5d, 0xb3, 0x67,
- 0x68, 0xb2, 0xee, 0x60, 0x4b, 0x71, 0x4c, 0x4b, 0x36, 0xbb, 0x8e, 0xde, 0xd1, 0xbf, 0xa0, 0x8e,
- 0x3d, 0xef, 0x6a, 0x37, 0x58, 0x76, 0xc2, 0xbc, 0x4a, 0x32, 0xd7, 0x79, 0xde, 0xe7, 0x81, 0xac,
- 0xe2, 0xe7, 0xb0, 0x10, 0xaa, 0x75, 0x9c, 0x2a, 0xfd, 0x1f, 0x02, 0x64, 0x1b, 0xaa, 0x62, 0xc4,
- 0xa9, 0xcb, 0x4f, 0x21, 0x6b, 0xab, 0x8a, 0x21, 0xef, 0x99, 0x56, 0x47, 0x71, 0xe8, 0xc0, 0xc9,
- 0x87, 0x74, 0xe9, 0xf9, 0xeb, 0xaa, 0x62, 0x3c, 0xa1, 0x99, 0x24, 0xb0, 0xbd, 0xdf, 0xfd, 0xfe,
- 0xe8, 0xf4, 0x57, 0xf7, 0x47, 0xd9, 0x70, 0x5d, 0x4f, 0xa5, 0x93, 0xc5, 0x94, 0xf8, 0x97, 0x02,
- 0xe4, 0x58, 0x95, 0xe3, 0x1c, 0xae, 0x1f, 0x40, 0xca, 0x32, 0x8f, 0xd8, 0x70, 0xcd, 0x3e, 0x7c,
- 0x2b, 0x42, 0xc4, 0x06, 0x3e, 0x09, 0xce, 0x87, 0x34, 0x3b, 0x5a, 0x05, 0xee, 0x75, 0xca, 0x94,
- 0x3b, 0x39, 0x29, 0x37, 0x30, 0x2e, 0x89, 0xc8, 0xb8, 0x03, 0x85, 0x5d, 0xc5, 0x51, 0xf7, 0x65,
- 0x8b, 0x17, 0x92, 0xcc, 0x9d, 0xc9, 0xbb, 0x39, 0x29, 0x4f, 0xc9, 0x6e, 0xd1, 0x6d, 0x52, 0x73,
- 0x36, 0x7e, 0x6c, 0xfc, 0x57, 0xac, 0xcd, 0xff, 0xb7, 0xc0, 0xc7, 0x90, 0x5b, 0xf3, 0xbf, 0x6a,
- 0x4d, 0xff, 0x9b, 0x09, 0xb8, 0x5c, 0xd9, 0xc7, 0xea, 0x41, 0xc5, 0x34, 0x6c, 0xdd, 0x76, 0x88,
- 0xee, 0xe2, 0x6c, 0xff, 0xb7, 0x20, 0x73, 0xa4, 0x3b, 0xfb, 0xb2, 0xa6, 0xef, 0xed, 0x51, 0xeb,
- 0x99, 0x96, 0xd2, 0x84, 0xb0, 0xa6, 0xef, 0xed, 0xa1, 0x47, 0x90, 0xea, 0x98, 0x1a, 0x73, 0xce,
- 0xf3, 0x0f, 0x17, 0x23, 0xc4, 0xd3, 0xa2, 0xd9, 0xbd, 0xce, 0xa6, 0xa9, 0x61, 0x89, 0x66, 0x46,
- 0x57, 0x01, 0x54, 0x42, 0xed, 0x9a, 0xba, 0xe1, 0xf0, 0xd9, 0x37, 0x40, 0x41, 0x35, 0xc8, 0x38,
- 0xd8, 0xea, 0xe8, 0x86, 0xe2, 0xe0, 0xd2, 0x34, 0x55, 0xde, 0xcd, 0xc8, 0x82, 0x77, 0xdb, 0xba,
- 0xaa, 0xac, 0x61, 0x5b, 0xb5, 0xf4, 0xae, 0x63, 0x5a, 0x5c, 0x8b, 0x3e, 0xb3, 0xf8, 0xeb, 0x29,
- 0x28, 0x0d, 0xea, 0x26, 0xce, 0x1e, 0xb2, 0x0d, 0x33, 0x16, 0xb6, 0x7b, 0x6d, 0x87, 0xf7, 0x91,
- 0x87, 0xc3, 0x54, 0x10, 0x51, 0x02, 0xba, 0xde, 0xd0, 0x76, 0x78, 0xb1, 0xb9, 0x9c, 0xf2, 0xbf,
- 0x16, 0x60, 0x86, 0x25, 0xa0, 0x07, 0x90, 0xb6, 0xc8, 0xc4, 0x20, 0xeb, 0x1a, 0x2d, 0x63, 0x72,
- 0xf5, 0xd2, 0xd9, 0xe9, 0xe2, 0x2c, 0x9d, 0x2c, 0xea, 0x6b, 0x5f, 0xfa, 0x3f, 0xa5, 0x59, 0x9a,
- 0xaf, 0xae, 0x91, 0xd6, 0xb2, 0x1d, 0xc5, 0x72, 0xe8, 0x22, 0x51, 0x82, 0xa1, 0x14, 0x4a, 0xd8,
- 0xc0, 0x27, 0x68, 0x1d, 0x66, 0x6c, 0x47, 0x71, 0x7a, 0x36, 0x6f, 0xaf, 0x73, 0x15, 0xb6, 0x41,
- 0x39, 0x25, 0x2e, 0x81, 0xb8, 0x4f, 0x1a, 0x76, 0x14, 0xbd, 0x4d, 0x1b, 0x30, 0x23, 0xf1, 0x27,
- 0xf1, 0xb7, 0x04, 0x98, 0x61, 0x59, 0xd1, 0x65, 0x58, 0x90, 0x56, 0xb6, 0x9e, 0x56, 0xe5, 0xfa,
- 0xd6, 0x5a, 0xb5, 0x59, 0x95, 0x36, 0xeb, 0x5b, 0x2b, 0xcd, 0x6a, 0x71, 0x0a, 0x5d, 0x02, 0xe4,
- 0x26, 0x54, 0x9e, 0x6f, 0x35, 0xea, 0x8d, 0x66, 0x75, 0xab, 0x59, 0x14, 0xe8, 0x1a, 0x09, 0xa5,
- 0x07, 0xa8, 0x09, 0x74, 0x13, 0xae, 0xf5, 0x53, 0xe5, 0x46, 0x73, 0xa5, 0xd9, 0x90, 0xab, 0x8d,
- 0x66, 0x7d, 0x73, 0xa5, 0x59, 0x5d, 0x2b, 0x26, 0x47, 0xe4, 0x22, 0x2f, 0x91, 0xa4, 0x6a, 0xa5,
- 0x59, 0x4c, 0x89, 0x0e, 0x5c, 0x94, 0xb0, 0x6a, 0x76, 0xba, 0x3d, 0x07, 0x93, 0x52, 0xda, 0x71,
- 0x8e, 0x94, 0xcb, 0x30, 0xab, 0x59, 0x27, 0xb2, 0xd5, 0x33, 0xf8, 0x38, 0x99, 0xd1, 0xac, 0x13,
- 0xa9, 0x67, 0x88, 0xff, 0x58, 0x80, 0x4b, 0xfd, 0xaf, 0x8d, 0xb3, 0x13, 0xbe, 0x80, 0xac, 0xa2,
- 0x69, 0x58, 0x93, 0x35, 0xdc, 0x76, 0x14, 0xee, 0xe2, 0xdc, 0x0f, 0x48, 0xe2, 0x4b, 0x7b, 0x4b,
- 0xde, 0xd2, 0xde, 0xe6, 0xcb, 0x4a, 0x85, 0x16, 0x64, 0x8d, 0x70, 0xb8, 0xe6, 0x87, 0x0a, 0xa1,
- 0x14, 0xf1, 0xbf, 0xa7, 0x60, 0xae, 0x6a, 0x68, 0xcd, 0xe3, 0x58, 0xe7, 0x92, 0x4b, 0x30, 0xa3,
- 0x9a, 0x9d, 0x8e, 0xee, 0xb8, 0x0a, 0x62, 0x4f, 0xe8, 0xe7, 0x02, 0xae, 0x69, 0x72, 0x02, 0x07,
- 0xcd, 0x77, 0x4a, 0xd1, 0x77, 0xe0, 0x32, 0xb1, 0x9a, 0x96, 0xa1, 0xb4, 0x65, 0x26, 0x4d, 0x76,
- 0x2c, 0xbd, 0xd5, 0xc2, 0x16, 0x5f, 0x4e, 0xbc, 0x1b, 0x51, 0xce, 0x3a, 0xe7, 0xa8, 0x50, 0x86,
- 0x26, 0xcb, 0x2f, 0x5d, 0xd4, 0xa3, 0xc8, 0xe8, 0x13, 0x00, 0x32, 0x15, 0xd1, 0x25, 0x4a, 0x9b,
- 0xdb, 0xa3, 0x61, 0x6b, 0x94, 0xae, 0x09, 0x22, 0x0c, 0xe4, 0xd9, 0x46, 0xcb, 0x04, 0x87, 0xbc,
- 0xee, 0xe9, 0x16, 0x96, 0x1f, 0x74, 0x55, 0xba, 0x70, 0x90, 0x5e, 0xcd, 0x9f, 0x9d, 0x2e, 0x82,
- 0xc4, 0xc8, 0x0f, 0xb6, 0x2b, 0x04, 0x97, 0xb0, 0xdf, 0x5d, 0x15, 0xbd, 0x82, 0x7b, 0x81, 0xf5,
- 0x0f, 0x32, 0xf3, 0xf2, 0x6a, 0x29, 0x8e, 0xbc, 0xaf, 0xb7, 0xf6, 0xb1, 0x25, 0x7b, 0xcb, 0xd4,
- 0x74, 0x25, 0x32, 0x2d, 0xdd, 0xf4, 0x19, 0x2a, 0x8a, 0xc1, 0x4a, 0xbf, 0xe2, 0xd4, 0x68, 0x66,
- 0x4f, 0x67, 0x44, 0xf9, 0x5d, 0x53, 0xb7, 0x4d, 0xa3, 0x94, 0x61, 0xca, 0x67, 0x4f, 0xe8, 0x1e,
- 0x14, 0x9d, 0x63, 0x43, 0xde, 0xc7, 0x8a, 0xe5, 0xec, 0x62, 0xc5, 0x21, 0xb3, 0x34, 0xd0, 0x1c,
- 0x05, 0xe7, 0xd8, 0xa8, 0x05, 0xc8, 0xe8, 0x05, 0x14, 0x75, 0x43, 0xde, 0x6b, 0xeb, 0xad, 0x7d,
- 0x47, 0x3e, 0xb2, 0x74, 0x07, 0xdb, 0xa5, 0x79, 0xaa, 0x90, 0xa8, 0x7e, 0xdb, 0xe0, 0xeb, 0xc6,
- 0xda, 0x2b, 0x92, 0x93, 0xab, 0x26, 0xaf, 0x1b, 0x4f, 0x28, 0x3f, 0x25, 0xda, 0xeb, 0xa9, 0xf4,
- 0x6c, 0x31, 0x2d, 0xfe, 0x67, 0x01, 0xf2, 0x6e, 0x77, 0x8b, 0x73, 0x64, 0xdc, 0x85, 0xa2, 0x69,
- 0x60, 0xb9, 0xbb, 0xaf, 0xd8, 0x98, 0xeb, 0x91, 0x4f, 0x38, 0x79, 0xd3, 0xc0, 0xdb, 0x84, 0xcc,
- 0xd4, 0x85, 0xb6, 0x61, 0xde, 0x76, 0x94, 0x96, 0x6e, 0xb4, 0x02, 0xea, 0x9d, 0x9e, 0x1c, 0x2c,
- 0x14, 0x39, 0xb7, 0x47, 0x0f, 0x79, 0x29, 0x7f, 0x2c, 0xc0, 0xfc, 0x8a, 0xd6, 0xd1, 0x8d, 0x46,
- 0xb7, 0xad, 0xc7, 0xba, 0x06, 0x71, 0x13, 0x32, 0x36, 0x91, 0xe9, 0x1b, 0x7c, 0x1f, 0x51, 0xa6,
- 0x69, 0x0a, 0xb1, 0xfc, 0xcf, 0xa0, 0x80, 0x8f, 0xbb, 0x3a, 0xfb, 0xf4, 0xc0, 0x80, 0x50, 0x6a,
- 0xf2, 0xba, 0xe5, 0x7d, 0x5e, 0x92, 0xc4, 0xeb, 0xf4, 0x19, 0xa0, 0x60, 0x95, 0xe2, 0xc4, 0x2e,
- 0x9f, 0xc1, 0x02, 0x15, 0xbd, 0x63, 0xd8, 0x31, 0xeb, 0x4b, 0xfc, 0x45, 0xb8, 0x10, 0x16, 0x1d,
- 0x67, 0xb9, 0x5f, 0xf1, 0x56, 0xde, 0xc4, 0x56, 0xac, 0x20, 0xd6, 0xd3, 0x35, 0x17, 0x1c, 0x67,
- 0x99, 0x7f, 0x55, 0x80, 0x2b, 0x54, 0x36, 0xfd, 0x3a, 0xb3, 0x87, 0xad, 0x67, 0x58, 0xb1, 0x63,
- 0x45, 0xe0, 0x37, 0x60, 0x86, 0x21, 0x69, 0xda, 0x3f, 0xa7, 0x57, 0xb3, 0xc4, 0x73, 0x69, 0x38,
- 0xa6, 0x45, 0x3c, 0x17, 0x9e, 0x24, 0x2a, 0x50, 0x8e, 0x2a, 0x45, 0x9c, 0x35, 0xfd, 0x3b, 0x02,
- 0xcc, 0x73, 0xa7, 0x91, 0x74, 0xe5, 0xca, 0x3e, 0xf1, 0x99, 0x50, 0x15, 0xb2, 0x2a, 0xfd, 0x25,
- 0x3b, 0x27, 0x5d, 0x4c, 0xe5, 0xe7, 0x47, 0xf9, 0x9b, 0x8c, 0xad, 0x79, 0xd2, 0xc5, 0xc4, 0x69,
- 0x75, 0x7f, 0x13, 0x45, 0x05, 0x2a, 0x39, 0xd2, 0x63, 0xa5, 0xe3, 0x88, 0xe6, 0x75, 0x5d, 0x3f,
- 0xae, 0x83, 0x7f, 0x92, 0xe4, 0x4a, 0x60, 0xef, 0xe0, 0xd9, 0x63, 0xf5, 0x51, 0x3e, 0x87, 0x4b,
- 0xc1, 0xd9, 0x25, 0x50, 0xf1, 0xc4, 0x39, 0x2a, 0x1e, 0x58, 0xa1, 0xf7, 0xa9, 0xe8, 0x33, 0x08,
- 0xac, 0xc1, 0xcb, 0xac, 0x4e, 0x2e, 0xfa, 0x39, 0x8f, 0x3a, 0xe6, 0x7d, 0x29, 0x8c, 0x6e, 0xa3,
- 0x0a, 0xa4, 0xf1, 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x46, 0x09, 0x24, 0x45, 0x19,
- 0xc0, 0x03, 0xb3, 0xf8, 0xb8, 0x4b, 0x88, 0x68, 0x87, 0xcc, 0x5e, 0xae, 0xab, 0x40, 0x8b, 0x6d,
- 0x8f, 0x87, 0x17, 0x7e, 0x4f, 0xe1, 0xe2, 0x0a, 0x9e, 0x97, 0xc0, 0x44, 0x88, 0x3f, 0x14, 0xe0,
- 0xad, 0xc8, 0x56, 0x8b, 0x73, 0x22, 0xfb, 0x04, 0x52, 0xb4, 0xf2, 0x89, 0x73, 0x56, 0x9e, 0x72,
- 0x89, 0xdf, 0x4b, 0xf0, 0x31, 0x2e, 0xe1, 0xb6, 0x49, 0x14, 0x1b, 0xfb, 0x2a, 0xdb, 0x73, 0x98,
- 0x3b, 0x34, 0x1d, 0xe2, 0x9b, 0xf0, 0x66, 0x4f, 0x9c, 0xbb, 0xd9, 0x73, 0x54, 0x80, 0xdb, 0xe2,
- 0x2f, 0x61, 0xde, 0x30, 0x0d, 0x39, 0x2c, 0xf4, 0xfc, 0x7d, 0xa9, 0x60, 0x98, 0xc6, 0xcb, 0x80,
- 0x5c, 0xcf, 0xce, 0xf4, 0x69, 0x22, 0x4e, 0x3b, 0xf3, 0x7d, 0x01, 0x16, 0x3c, 0xb7, 0x29, 0x66,
- 0x0f, 0xfa, 0x03, 0x48, 0x1a, 0xe6, 0xd1, 0x79, 0x56, 0x31, 0x49, 0x7e, 0x32, 0xeb, 0x85, 0x4b,
- 0x14, 0x67, 0x7d, 0xff, 0x4d, 0x02, 0x32, 0x4f, 0x2b, 0x71, 0xd6, 0xf2, 0x13, 0xbe, 0x42, 0xce,
- 0xda, 0x3b, 0xaa, 0xb7, 0x7b, 0xef, 0x5b, 0x7a, 0x5a, 0xd9, 0xc0, 0x27, 0x6e, 0x6f, 0x27, 0x5c,
- 0x68, 0x05, 0x32, 0xce, 0xbe, 0x85, 0xed, 0x7d, 0xb3, 0xad, 0x9d, 0xc7, 0xcd, 0xf1, 0xb9, 0xca,
- 0x18, 0xa6, 0xa9, 0x5c, 0x37, 0x1a, 0x43, 0x88, 0x88, 0xc6, 0x20, 0xaf, 0xf1, 0x3c, 0xc5, 0xc4,
- 0x79, 0x5e, 0x13, 0x70, 0x11, 0xa7, 0x8b, 0x33, 0xe2, 0x0b, 0x00, 0x52, 0x9d, 0x38, 0x9b, 0xe4,
- 0xd7, 0x92, 0x90, 0xdf, 0xee, 0xd9, 0xfb, 0x31, 0xf7, 0xbe, 0x0a, 0x40, 0xb7, 0x67, 0x53, 0x08,
- 0x72, 0x6c, 0xf0, 0x3a, 0x8f, 0x09, 0xf4, 0x70, 0x2b, 0xcd, 0xf8, 0x9a, 0xc7, 0x06, 0xaa, 0x71,
- 0x21, 0x58, 0xf6, 0xa3, 0x45, 0x6e, 0x8c, 0x02, 0xab, 0xcd, 0x63, 0x63, 0x13, 0x7b, 0x28, 0x95,
- 0x49, 0xc2, 0x44, 0xd2, 0x27, 0x30, 0x4b, 0x1e, 0x64, 0xc7, 0x3c, 0x4f, 0x33, 0xcf, 0x10, 0x9e,
- 0xa6, 0x89, 0x1e, 0x43, 0x86, 0x71, 0x93, 0xd9, 0x6f, 0x86, 0xce, 0x7e, 0x51, 0x75, 0xe1, 0x6a,
- 0xa4, 0xf3, 0x5e, 0x9a, 0xb2, 0x92, 0xb9, 0xee, 0x02, 0x4c, 0xef, 0x99, 0x96, 0xea, 0x7e, 0xef,
- 0x65, 0x0f, 0xac, 0x3d, 0xd7, 0x53, 0xe9, 0x74, 0x31, 0xb3, 0x9e, 0x4a, 0x67, 0x8a, 0x20, 0xfe,
- 0x96, 0x00, 0x05, 0xaf, 0x21, 0xe2, 0x9c, 0x10, 0x2a, 0x21, 0x2d, 0x9e, 0xbf, 0x29, 0x88, 0x02,
- 0xc5, 0x7f, 0x4b, 0x3d, 0x22, 0xd5, 0x3c, 0xa4, 0x2d, 0x13, 0x67, 0x4f, 0x79, 0xcc, 0x62, 0x81,
- 0x12, 0xe7, 0x6d, 0x5d, 0x1a, 0x16, 0xf4, 0x00, 0x2e, 0xe8, 0x1d, 0x62, 0xcf, 0x75, 0xa7, 0x7d,
- 0xc2, 0x61, 0x9b, 0x83, 0xdd, 0x0f, 0xcb, 0x0b, 0x7e, 0x5a, 0xc5, 0x4d, 0x12, 0xff, 0x1e, 0x5d,
- 0x00, 0xf7, 0x6b, 0x12, 0xa7, 0xaa, 0xeb, 0x30, 0x67, 0x31, 0xd1, 0xc4, 0xad, 0x39, 0xa7, 0xb6,
- 0x73, 0x1e, 0x2b, 0x51, 0xf8, 0xef, 0x24, 0xa0, 0xf0, 0xa2, 0x87, 0xad, 0x93, 0xaf, 0x93, 0xba,
- 0x6f, 0x43, 0xe1, 0x48, 0xd1, 0x1d, 0x79, 0xcf, 0xb4, 0xe4, 0x5e, 0x57, 0x53, 0x1c, 0x37, 0x20,
- 0x65, 0x8e, 0x90, 0x9f, 0x98, 0xd6, 0x0e, 0x25, 0x22, 0x0c, 0xe8, 0xc0, 0x30, 0x8f, 0x0c, 0x99,
- 0x90, 0x29, 0x50, 0x3e, 0x36, 0xf8, 0xaa, 0xf4, 0xea, 0x87, 0xff, 0xe9, 0x74, 0xf1, 0xd1, 0x44,
- 0x61, 0x66, 0x34, 0xa4, 0xae, 0xd7, 0xd3, 0xb5, 0xa5, 0x9d, 0x9d, 0xfa, 0x9a, 0x54, 0xa4, 0x22,
- 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x7e, 0x02, 0x8a, 0xbe, 0x8e, 0xe2, 0x6c, 0xc8,
- 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x33, 0x02, 0x67, 0x24, 0x66, 0xe7, 0x73, 0xc8,
- 0x85, 0x34, 0x90, 0xfc, 0x6a, 0x1a, 0xc8, 0x1e, 0xf9, 0x95, 0x47, 0xf7, 0x61, 0xde, 0x39, 0x36,
- 0x64, 0x16, 0x60, 0xc8, 0x82, 0x52, 0xdc, 0x78, 0x89, 0x82, 0x43, 0xf4, 0x41, 0xe8, 0x34, 0x20,
- 0xc5, 0x16, 0x7f, 0x5f, 0x00, 0x44, 0x15, 0x55, 0x67, 0x9f, 0x0d, 0xbe, 0x2e, 0xfd, 0xe9, 0x2e,
- 0x14, 0x69, 0xc8, 0xa6, 0xac, 0xef, 0xc9, 0x1d, 0xdd, 0xb6, 0x75, 0xa3, 0xc5, 0x3b, 0x54, 0x9e,
- 0xd2, 0xeb, 0x7b, 0x9b, 0x8c, 0x2a, 0xfe, 0x35, 0x58, 0x08, 0x55, 0x20, 0xce, 0xc6, 0xbe, 0x0e,
- 0xb9, 0x3d, 0xf6, 0x95, 0x96, 0x0a, 0xe7, 0x2b, 0x8e, 0x59, 0x4a, 0x63, 0xef, 0x13, 0xff, 0x3c,
- 0x01, 0x17, 0x24, 0x6c, 0x9b, 0xed, 0x43, 0x1c, 0xbf, 0x0a, 0x6b, 0xc0, 0x3f, 0xe7, 0xc8, 0x6f,
- 0xa4, 0xc9, 0x0c, 0x63, 0x66, 0xd3, 0x5c, 0x78, 0xd9, 0xfe, 0xe6, 0xe8, 0x1e, 0x3b, 0xb8, 0x50,
- 0xcf, 0x97, 0xfd, 0x52, 0xa1, 0x65, 0x3f, 0x13, 0x0a, 0x7a, 0xcb, 0x30, 0x89, 0x4d, 0xb3, 0xf1,
- 0x6b, 0xa3, 0xd7, 0x71, 0xc1, 0xd0, 0xd2, 0xa8, 0x42, 0xd6, 0x19, 0x4b, 0x03, 0xbf, 0xde, 0xea,
- 0x75, 0xa8, 0xef, 0xbc, 0x7a, 0x89, 0x94, 0xf7, 0xec, 0x74, 0x31, 0x1f, 0x4a, 0xb3, 0xa5, 0xbc,
- 0xee, 0x3d, 0x13, 0xe9, 0xe2, 0xb7, 0xe1, 0x62, 0x9f, 0xb2, 0xe3, 0xf4, 0x78, 0xfe, 0x55, 0x12,
- 0xae, 0x84, 0xc5, 0xc7, 0x0d, 0x71, 0xbe, 0xee, 0x0d, 0x5a, 0x83, 0xb9, 0x8e, 0x6e, 0xbc, 0xd9,
- 0xea, 0x65, 0xae, 0xa3, 0x1b, 0xfe, 0x4a, 0x71, 0x44, 0xd7, 0x98, 0xf9, 0x99, 0x76, 0x0d, 0x05,
- 0xca, 0x51, 0x6d, 0x17, 0x67, 0xff, 0xf8, 0x9e, 0x00, 0xb9, 0xb8, 0x97, 0xe5, 0xde, 0x2c, 0x50,
- 0x4e, 0x6c, 0xc2, 0xdc, 0xcf, 0x60, 0x1d, 0xef, 0x77, 0x04, 0x40, 0x4d, 0xab, 0x67, 0x10, 0x50,
- 0xfb, 0xcc, 0x6c, 0xc5, 0x59, 0xcd, 0x0b, 0x30, 0xad, 0x1b, 0x1a, 0x3e, 0xa6, 0xd5, 0x4c, 0x49,
- 0xec, 0x21, 0xf4, 0x75, 0x32, 0x39, 0xd1, 0xd7, 0x49, 0xf1, 0x73, 0x58, 0x08, 0x15, 0x31, 0xce,
- 0xfa, 0xff, 0xb7, 0x04, 0x2c, 0xf0, 0x8a, 0xc4, 0xbe, 0x82, 0xf9, 0x4d, 0x98, 0x6e, 0x13, 0x99,
- 0x23, 0xda, 0x99, 0xbe, 0xd3, 0x6d, 0x67, 0x9a, 0x19, 0xfd, 0x3c, 0x40, 0xd7, 0xc2, 0x87, 0x32,
- 0x63, 0x4d, 0x4e, 0xc4, 0x9a, 0x21, 0x1c, 0x94, 0x80, 0x7e, 0x20, 0x40, 0x81, 0x0c, 0xe8, 0xae,
- 0x65, 0x76, 0x4d, 0x9b, 0xf8, 0x2c, 0xf6, 0x64, 0x30, 0xe7, 0xc5, 0xd9, 0xe9, 0xe2, 0xdc, 0xa6,
- 0x6e, 0x6c, 0x73, 0xc6, 0x66, 0x63, 0xe2, 0x3d, 0x00, 0xee, 0x4e, 0x88, 0xa5, 0x4a, 0xdb, 0x54,
- 0x0f, 0xfc, 0xef, 0x6d, 0xc4, 0xb2, 0x78, 0xe2, 0x6c, 0xf1, 0x8f, 0x04, 0xb8, 0xf0, 0x33, 0x5b,
+ 0x46, 0x00, 0x03, 0x59, 0x04, 0x4e, 0xbc, 0x7e, 0x8a, 0x11, 0x20, 0x8a, 0xad, 0xcd, 0x4b, 0xe0,
+ 0x20, 0x08, 0x82, 0x00, 0x06, 0xe6, 0x21, 0x08, 0xee, 0x47, 0x7d, 0x91, 0xc5, 0x0f, 0xf5, 0xd4,
+ 0x26, 0x03, 0xf8, 0x45, 0x62, 0x9d, 0xba, 0xe7, 0xd4, 0xbd, 0xe7, 0x9e, 0x7b, 0xee, 0xf9, 0xdd,
+ 0x3a, 0xf7, 0x16, 0xcc, 0x5b, 0xa6, 0xa2, 0xee, 0x77, 0x77, 0x97, 0x95, 0xae, 0x5e, 0xee, 0x5a,
+ 0xa6, 0x63, 0xa2, 0x79, 0xd5, 0x54, 0x0f, 0x28, 0xb9, 0xcc, 0x6f, 0x2e, 0xde, 0x3f, 0x38, 0x5c,
+ 0x3e, 0x38, 0xb4, 0xb1, 0x75, 0x88, 0xad, 0x65, 0xd5, 0x34, 0xd4, 0x9e, 0x65, 0x61, 0x43, 0x3d,
+ 0x59, 0x6e, 0x9b, 0xea, 0x01, 0xfd, 0xa3, 0x1b, 0x2d, 0xc6, 0x1e, 0x2e, 0x6b, 0x61, 0x45, 0xb3,
+ 0x7b, 0x9d, 0x8e, 0x62, 0x9d, 0x2c, 0x5b, 0x76, 0x77, 0x77, 0x99, 0x5f, 0xf0, 0xb2, 0xc8, 0x7d,
+ 0xba, 0xa6, 0x38, 0x0a, 0xa7, 0x5d, 0x70, 0x69, 0xd8, 0xb2, 0x4c, 0xcb, 0xe6, 0xd4, 0x4b, 0x2e,
+ 0xb5, 0x83, 0x1d, 0x25, 0x50, 0xfa, 0x2d, 0xdb, 0x31, 0x2d, 0xa5, 0x85, 0x97, 0xb1, 0xd1, 0xd2,
+ 0x0d, 0x4c, 0x0a, 0x1c, 0xaa, 0x2a, 0xbf, 0xf9, 0x76, 0xe4, 0xcd, 0x47, 0xfc, 0x6e, 0xa9, 0xe7,
+ 0xe8, 0xed, 0xe5, 0xfd, 0xb6, 0xba, 0xec, 0xe8, 0x1d, 0x6c, 0x3b, 0x4a, 0xa7, 0xeb, 0x36, 0x81,
+ 0xde, 0x71, 0x2c, 0x45, 0xd5, 0x8d, 0x96, 0xfb, 0xbf, 0xbb, 0xbb, 0x6c, 0x61, 0xd5, 0xb4, 0x34,
+ 0xac, 0xc9, 0x76, 0x57, 0x31, 0xdc, 0xea, 0xb6, 0xcc, 0x96, 0x49, 0x7f, 0x2e, 0x93, 0x5f, 0x9c,
+ 0x7a, 0xb5, 0x65, 0x9a, 0xad, 0x36, 0x5e, 0xa6, 0x57, 0xbb, 0xbd, 0xbd, 0x65, 0xad, 0x67, 0x29,
+ 0x8e, 0x6e, 0x72, 0x2e, 0xf1, 0x9f, 0x08, 0x30, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac,
+ 0x68, 0xd8, 0x42, 0x57, 0x20, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0x6b, 0xc2, 0xdd, 0xdc, 0xea, 0xec,
+ 0x97, 0xa7, 0x4b, 0xc9, 0x0d, 0x7c, 0x22, 0x11, 0x1a, 0xba, 0x06, 0xb3, 0xd8, 0xd0, 0x64, 0x72,
+ 0x3b, 0x15, 0xbe, 0x3d, 0x83, 0x0d, 0x6d, 0x03, 0x9f, 0xa0, 0x6f, 0x43, 0xda, 0x26, 0xd2, 0x0c,
+ 0x15, 0x97, 0xa6, 0xaf, 0x09, 0x77, 0xa7, 0x57, 0x7f, 0xe1, 0xcb, 0xd3, 0xa5, 0x4f, 0x5a, 0xba,
+ 0xb3, 0xdf, 0xdb, 0x2d, 0xab, 0x66, 0x67, 0xd9, 0xeb, 0x53, 0x6d, 0xd7, 0xff, 0xbd, 0xdc, 0x3d,
+ 0x68, 0x2d, 0xf7, 0xeb, 0xa8, 0xdc, 0x3c, 0x36, 0x1a, 0xf8, 0xb5, 0xe4, 0x49, 0x5c, 0x4f, 0xa5,
+ 0x85, 0x62, 0x62, 0x3d, 0x95, 0x4e, 0x14, 0x93, 0xe2, 0x1f, 0x25, 0x20, 0x2f, 0x61, 0xbb, 0x6b,
+ 0x1a, 0x36, 0xe6, 0x35, 0x7f, 0x1f, 0x92, 0xce, 0xb1, 0x41, 0x6b, 0x9e, 0x7d, 0x78, 0xb5, 0x3c,
+ 0x60, 0x3d, 0xe5, 0xa6, 0xa5, 0x18, 0xb6, 0xa2, 0x92, 0xe6, 0x4b, 0xa4, 0x28, 0xfa, 0x08, 0xb2,
+ 0x16, 0xb6, 0x7b, 0x1d, 0x4c, 0x15, 0x49, 0x1b, 0x95, 0x7d, 0x78, 0x39, 0x82, 0xb3, 0xd1, 0x55,
+ 0x0c, 0x09, 0x58, 0x59, 0xf2, 0x1b, 0x5d, 0x81, 0xb4, 0xd1, 0xeb, 0x10, 0x55, 0xd8, 0xb4, 0xa1,
+ 0x49, 0x69, 0xd6, 0xe8, 0x75, 0x36, 0xf0, 0x89, 0x8d, 0x1a, 0x30, 0xc7, 0x85, 0x5a, 0x58, 0xb1,
+ 0x4d, 0xa3, 0x34, 0x7b, 0x4d, 0xb8, 0x9b, 0x7f, 0x58, 0x8e, 0x10, 0x1b, 0x6e, 0x00, 0xb9, 0xec,
+ 0x75, 0xb0, 0x44, 0xb9, 0xa4, 0x9c, 0x15, 0xb8, 0x42, 0x6f, 0x41, 0x86, 0x3c, 0x6f, 0xf7, 0xc4,
+ 0xc1, 0x76, 0x29, 0x4d, 0x1f, 0x48, 0x2a, 0xb0, 0x4a, 0xae, 0xc5, 0x4f, 0x21, 0x17, 0x64, 0x45,
+ 0x08, 0xf2, 0x52, 0xb5, 0xb1, 0xb3, 0x59, 0x95, 0x77, 0xb6, 0x36, 0xb6, 0x9e, 0xbf, 0xda, 0x2a,
+ 0x4e, 0xa1, 0x0b, 0x50, 0xe4, 0xb4, 0x8d, 0xea, 0x67, 0xf2, 0xb3, 0xfa, 0x66, 0xbd, 0x59, 0x14,
+ 0x16, 0x53, 0xdf, 0xfb, 0xd1, 0xd5, 0xa9, 0xf5, 0x54, 0x7a, 0xa6, 0x38, 0x2b, 0xfe, 0x48, 0x00,
+ 0x78, 0x8a, 0x1d, 0x6e, 0x0d, 0x68, 0x15, 0x66, 0xf6, 0x69, 0xb5, 0x4a, 0x02, 0x55, 0xcb, 0xb5,
+ 0xc8, 0xfa, 0x07, 0x2c, 0x67, 0x35, 0xfd, 0xe3, 0xd3, 0xa5, 0xa9, 0x9f, 0x9c, 0x2e, 0x09, 0x12,
+ 0xe7, 0x44, 0x2f, 0x20, 0x7b, 0x80, 0x4f, 0x64, 0x3e, 0x2e, 0x4b, 0x09, 0xaa, 0x88, 0xf7, 0x03,
+ 0x82, 0x0e, 0x0e, 0xcb, 0xee, 0x10, 0x2d, 0x07, 0x86, 0x73, 0x99, 0x70, 0x94, 0x1b, 0x8e, 0x85,
+ 0x8d, 0x96, 0xb3, 0x2f, 0xc1, 0x01, 0x3e, 0x79, 0xc6, 0x64, 0x88, 0xbf, 0x2f, 0x40, 0x96, 0xd6,
+ 0x92, 0x69, 0x0e, 0x55, 0xfa, 0xaa, 0x79, 0x7d, 0xac, 0x9a, 0x23, 0xea, 0x59, 0x86, 0xe9, 0x43,
+ 0xa5, 0xdd, 0xc3, 0xb4, 0x86, 0xd9, 0x87, 0xa5, 0x08, 0x19, 0x2f, 0xc9, 0x7d, 0x89, 0x15, 0x43,
+ 0x8f, 0x21, 0xa7, 0x1b, 0x0e, 0x36, 0x1c, 0x99, 0xb1, 0x25, 0xc7, 0xb0, 0x65, 0x59, 0x69, 0x7a,
+ 0x21, 0xfe, 0x63, 0x01, 0x60, 0xbb, 0x17, 0xab, 0x9e, 0xbf, 0x39, 0x61, 0xfd, 0x57, 0x53, 0x84,
+ 0xd5, 0x6d, 0xc5, 0x25, 0x98, 0xd1, 0x8d, 0xb6, 0x6e, 0xb0, 0xfa, 0xa7, 0x25, 0x7e, 0x85, 0x2e,
+ 0xc0, 0xf4, 0x6e, 0x5b, 0x37, 0x34, 0x3a, 0x1e, 0xd2, 0x12, 0xbb, 0x10, 0x25, 0xc8, 0xd2, 0x5a,
+ 0xc7, 0xa8, 0x77, 0xf1, 0x34, 0x01, 0x17, 0x2b, 0xa6, 0xa1, 0xe9, 0x64, 0x48, 0x2a, 0xed, 0xaf,
+ 0x85, 0x56, 0xd6, 0xe1, 0x82, 0x86, 0xbb, 0x16, 0x56, 0x15, 0x07, 0x6b, 0x32, 0x3e, 0xee, 0x4e,
+ 0xd8, 0xc7, 0xc8, 0xe7, 0xaa, 0x1e, 0x77, 0x29, 0x2d, 0x5a, 0x93, 0xe8, 0x9b, 0x70, 0x59, 0x69,
+ 0xb7, 0xcd, 0x23, 0x59, 0xdf, 0x93, 0x35, 0x13, 0xdb, 0xb2, 0x61, 0x3a, 0x32, 0x3e, 0xd6, 0x6d,
+ 0x87, 0xba, 0x92, 0xb4, 0xb4, 0x40, 0x6f, 0xd7, 0xf7, 0xd6, 0x4c, 0x6c, 0x6f, 0x99, 0x4e, 0x95,
+ 0xdc, 0x22, 0x1e, 0x80, 0x54, 0x86, 0x79, 0x80, 0x19, 0xe2, 0x7e, 0xa5, 0x34, 0x3e, 0xee, 0x52,
+ 0x0f, 0x10, 0xe8, 0xca, 0xd9, 0x60, 0x57, 0x8a, 0xbf, 0x04, 0x97, 0xfa, 0xf5, 0x1b, 0x67, 0xff,
+ 0xfd, 0xa1, 0x00, 0xf9, 0xba, 0xa1, 0x3b, 0x5f, 0x8b, 0x8e, 0xf3, 0x94, 0x9d, 0x0c, 0x2a, 0xfb,
+ 0x3e, 0x14, 0xf7, 0x14, 0xbd, 0xfd, 0xdc, 0x68, 0x9a, 0x9d, 0x5d, 0xdb, 0x31, 0x0d, 0x6c, 0xf3,
+ 0xde, 0x18, 0xa0, 0x8b, 0x2f, 0xa1, 0xe0, 0xb5, 0x26, 0x4e, 0x35, 0x39, 0x50, 0xac, 0x1b, 0xaa,
+ 0x85, 0x3b, 0xd8, 0x88, 0x55, 0x4f, 0x6f, 0x43, 0x46, 0x77, 0xe5, 0x52, 0x5d, 0x25, 0x25, 0x9f,
+ 0x20, 0xf6, 0x60, 0x3e, 0xf0, 0xd4, 0x38, 0xdd, 0x25, 0x99, 0x8c, 0xf0, 0x91, 0xec, 0xf7, 0x11,
+ 0x99, 0x8c, 0xf0, 0x11, 0x73, 0x6f, 0x0d, 0x98, 0x5b, 0xc3, 0x6d, 0xec, 0xe0, 0x18, 0x5b, 0x2a,
+ 0xee, 0x40, 0xde, 0x15, 0x1a, 0x67, 0xc7, 0xfc, 0x0d, 0x01, 0x10, 0x97, 0xab, 0x18, 0xad, 0x38,
+ 0x6b, 0x8c, 0x96, 0x48, 0x68, 0xe1, 0xf4, 0x2c, 0x83, 0xc5, 0x08, 0xcc, 0x26, 0x81, 0x91, 0x68,
+ 0x98, 0xe0, 0x0f, 0xd9, 0x54, 0x70, 0xc8, 0xf2, 0xf0, 0xe6, 0x08, 0x16, 0x42, 0x15, 0x8b, 0xb7,
+ 0xfb, 0x52, 0xb4, 0x4e, 0x89, 0x6b, 0xc9, 0x60, 0x0c, 0x47, 0x89, 0xe2, 0x0f, 0x04, 0x98, 0xaf,
+ 0xb4, 0xb1, 0x62, 0xc5, 0xae, 0x91, 0x6f, 0x41, 0x5a, 0xc3, 0x8a, 0x46, 0x9b, 0xcc, 0x06, 0xf6,
+ 0x3b, 0x01, 0x29, 0x24, 0xd2, 0x2d, 0xef, 0xb7, 0xd5, 0x72, 0xd3, 0x8d, 0x81, 0xf9, 0xe8, 0xf6,
+ 0x98, 0xc4, 0xcf, 0x00, 0x05, 0x6b, 0x16, 0xa7, 0x21, 0xfc, 0x2f, 0x01, 0x90, 0x84, 0x0f, 0xb1,
+ 0xe5, 0xc4, 0xde, 0xec, 0x35, 0xc8, 0x3a, 0x8a, 0xd5, 0xc2, 0x8e, 0x4c, 0xa2, 0xfb, 0xf3, 0xb4,
+ 0x1c, 0x18, 0x1f, 0x21, 0xa3, 0x26, 0xdc, 0xc1, 0x86, 0xb2, 0xdb, 0xc6, 0x54, 0x8a, 0xbc, 0x6b,
+ 0xf6, 0x0c, 0x4d, 0xd6, 0x1d, 0x6c, 0x29, 0x8e, 0x69, 0xc9, 0x66, 0xd7, 0xd1, 0x3b, 0xfa, 0x17,
+ 0x34, 0xb0, 0xe7, 0xa6, 0x76, 0x83, 0x15, 0x27, 0xcc, 0xab, 0xa4, 0x70, 0x9d, 0x97, 0x7d, 0x1e,
+ 0x28, 0x2a, 0x7e, 0x0e, 0x0b, 0xa1, 0x56, 0xc7, 0xa9, 0xd2, 0xff, 0x21, 0x40, 0xb6, 0xa1, 0x2a,
+ 0x46, 0x9c, 0xba, 0xfc, 0x14, 0xb2, 0xb6, 0xaa, 0x18, 0xf2, 0x9e, 0x69, 0x75, 0x14, 0x87, 0x0e,
+ 0x9c, 0x7c, 0x48, 0x97, 0x5e, 0xbc, 0xae, 0x2a, 0xc6, 0x13, 0x5a, 0x48, 0x02, 0xdb, 0xfb, 0xdd,
+ 0x1f, 0x8f, 0x4e, 0x7f, 0xf5, 0x78, 0x94, 0x0d, 0xd7, 0xf5, 0x54, 0x3a, 0x59, 0x4c, 0x89, 0x7f,
+ 0x21, 0x40, 0x8e, 0x35, 0x39, 0xce, 0xe1, 0xfa, 0x01, 0xa4, 0x2c, 0xf3, 0x88, 0x0d, 0xd7, 0xec,
+ 0xc3, 0xb7, 0x22, 0x44, 0x6c, 0xe0, 0x93, 0xe0, 0x7c, 0x48, 0x8b, 0xa3, 0x55, 0xe0, 0x51, 0xa7,
+ 0x4c, 0xb9, 0x93, 0x93, 0x72, 0x03, 0xe3, 0x92, 0x88, 0x8c, 0x3b, 0x50, 0xd8, 0x55, 0x1c, 0x75,
+ 0x5f, 0xb6, 0x78, 0x25, 0xc9, 0xdc, 0x99, 0xbc, 0x9b, 0x93, 0xf2, 0x94, 0xec, 0x56, 0xdd, 0x26,
+ 0x2d, 0x67, 0xe3, 0xc7, 0xc6, 0x7f, 0xc9, 0xfa, 0xfc, 0x7f, 0x0b, 0x7c, 0x0c, 0xb9, 0x2d, 0xff,
+ 0xcb, 0xd6, 0xf5, 0xbf, 0x99, 0x80, 0xcb, 0x95, 0x7d, 0xac, 0x1e, 0x54, 0x4c, 0xc3, 0xd6, 0x6d,
+ 0x87, 0xe8, 0x2e, 0xce, 0xfe, 0x7f, 0x0b, 0x32, 0x47, 0xba, 0xb3, 0x2f, 0x6b, 0xfa, 0xde, 0x1e,
+ 0xf5, 0x9e, 0x69, 0x29, 0x4d, 0x08, 0x6b, 0xfa, 0xde, 0x1e, 0x7a, 0x04, 0xa9, 0x8e, 0xa9, 0xb1,
+ 0xe0, 0x3c, 0xff, 0x70, 0x29, 0x42, 0x3c, 0xad, 0x9a, 0xdd, 0xeb, 0x6c, 0x9a, 0x1a, 0x96, 0x68,
+ 0x61, 0x74, 0x15, 0x40, 0x25, 0xd4, 0xae, 0xa9, 0x1b, 0x0e, 0x9f, 0x7d, 0x03, 0x14, 0x54, 0x83,
+ 0x8c, 0x83, 0xad, 0x8e, 0x6e, 0x28, 0x0e, 0x2e, 0x4d, 0x53, 0xe5, 0xdd, 0x8c, 0xac, 0x78, 0xb7,
+ 0xad, 0xab, 0xca, 0x1a, 0xb6, 0x55, 0x4b, 0xef, 0x3a, 0xa6, 0xc5, 0xb5, 0xe8, 0x33, 0x8b, 0xbf,
+ 0x9e, 0x82, 0xd2, 0xa0, 0x6e, 0xe2, 0xb4, 0x90, 0x6d, 0x98, 0xb1, 0xb0, 0xdd, 0x6b, 0x3b, 0xdc,
+ 0x46, 0x1e, 0x0e, 0x53, 0x41, 0x44, 0x0d, 0xe8, 0x7a, 0x43, 0xdb, 0xe1, 0xd5, 0xe6, 0x72, 0x16,
+ 0xff, 0xa5, 0x00, 0x33, 0xec, 0x06, 0x7a, 0x00, 0x69, 0x8b, 0x4c, 0x0c, 0xb2, 0xae, 0xd1, 0x3a,
+ 0x26, 0x57, 0x2f, 0x9d, 0x9d, 0x2e, 0xcd, 0xd2, 0xc9, 0xa2, 0xbe, 0xf6, 0xa5, 0xff, 0x53, 0x9a,
+ 0xa5, 0xe5, 0xea, 0x1a, 0xe9, 0x2d, 0xdb, 0x51, 0x2c, 0x87, 0x2e, 0x12, 0x25, 0x18, 0x4a, 0xa1,
+ 0x84, 0x0d, 0x7c, 0x82, 0xd6, 0x61, 0xc6, 0x76, 0x14, 0xa7, 0x67, 0xf3, 0xfe, 0x3a, 0x57, 0x65,
+ 0x1b, 0x94, 0x53, 0xe2, 0x12, 0x48, 0xf8, 0xa4, 0x61, 0x47, 0xd1, 0xdb, 0xb4, 0x03, 0x33, 0x12,
+ 0xbf, 0x12, 0x7f, 0x4b, 0x80, 0x19, 0x56, 0x14, 0x5d, 0x86, 0x05, 0x69, 0x65, 0xeb, 0x69, 0x55,
+ 0xae, 0x6f, 0xad, 0x55, 0x9b, 0x55, 0x69, 0xb3, 0xbe, 0xb5, 0xd2, 0xac, 0x16, 0xa7, 0xd0, 0x25,
+ 0x40, 0xee, 0x8d, 0xca, 0xf3, 0xad, 0x46, 0xbd, 0xd1, 0xac, 0x6e, 0x35, 0x8b, 0x02, 0x5d, 0x23,
+ 0xa1, 0xf4, 0x00, 0x35, 0x81, 0x6e, 0xc2, 0xb5, 0x7e, 0xaa, 0xdc, 0x68, 0xae, 0x34, 0x1b, 0x72,
+ 0xb5, 0xd1, 0xac, 0x6f, 0xae, 0x34, 0xab, 0x6b, 0xc5, 0xe4, 0x88, 0x52, 0xe4, 0x21, 0x92, 0x54,
+ 0xad, 0x34, 0x8b, 0x29, 0xd1, 0x81, 0x8b, 0x12, 0x56, 0xcd, 0x4e, 0xb7, 0xe7, 0x60, 0x52, 0x4b,
+ 0x3b, 0xce, 0x91, 0x72, 0x19, 0x66, 0x35, 0xeb, 0x44, 0xb6, 0x7a, 0x06, 0x1f, 0x27, 0x33, 0x9a,
+ 0x75, 0x22, 0xf5, 0x0c, 0xf1, 0x1f, 0x0a, 0x70, 0xa9, 0xff, 0xb1, 0x71, 0x1a, 0xe1, 0x0b, 0xc8,
+ 0x2a, 0x9a, 0x86, 0x35, 0x59, 0xc3, 0x6d, 0x47, 0xe1, 0x21, 0xce, 0xfd, 0x80, 0x24, 0xbe, 0xb4,
+ 0x57, 0xf6, 0x96, 0xf6, 0x36, 0x5f, 0x56, 0x2a, 0xb4, 0x22, 0x6b, 0x84, 0xc3, 0x75, 0x3f, 0x54,
+ 0x08, 0xa5, 0x88, 0xff, 0x2d, 0x05, 0x73, 0x55, 0x43, 0x6b, 0x1e, 0xc7, 0x3a, 0x97, 0x5c, 0x82,
+ 0x19, 0xd5, 0xec, 0x74, 0x74, 0xc7, 0x55, 0x10, 0xbb, 0x42, 0x3f, 0x17, 0x08, 0x4d, 0x93, 0x13,
+ 0x04, 0x68, 0x7e, 0x50, 0x8a, 0xbe, 0x03, 0x97, 0x89, 0xd7, 0xb4, 0x0c, 0xa5, 0x2d, 0x33, 0x69,
+ 0xb2, 0x63, 0xe9, 0xad, 0x16, 0xb6, 0xf8, 0x72, 0xe2, 0xdd, 0x88, 0x7a, 0xd6, 0x39, 0x47, 0x85,
+ 0x32, 0x34, 0x59, 0x79, 0xe9, 0xa2, 0x1e, 0x45, 0x46, 0x9f, 0x00, 0x90, 0xa9, 0x88, 0x2e, 0x51,
+ 0xda, 0xdc, 0x1f, 0x0d, 0x5b, 0xa3, 0x74, 0x5d, 0x10, 0x61, 0x20, 0xd7, 0x36, 0x5a, 0x26, 0x38,
+ 0xe4, 0x75, 0x4f, 0xb7, 0xb0, 0xfc, 0xa0, 0xab, 0xd2, 0x85, 0x83, 0xf4, 0x6a, 0xfe, 0xec, 0x74,
+ 0x09, 0x24, 0x46, 0x7e, 0xb0, 0x5d, 0x21, 0xb8, 0x84, 0xfd, 0xee, 0xaa, 0xe8, 0x15, 0xdc, 0x0b,
+ 0xac, 0x7f, 0x90, 0x99, 0x97, 0x37, 0x4b, 0x71, 0xe4, 0x7d, 0xbd, 0xb5, 0x8f, 0x2d, 0xd9, 0x5b,
+ 0xa6, 0xa6, 0x2b, 0x91, 0x69, 0xe9, 0xa6, 0xcf, 0x50, 0x51, 0x0c, 0x56, 0xfb, 0x15, 0xa7, 0x46,
+ 0x0b, 0x7b, 0x3a, 0x23, 0xca, 0xef, 0x9a, 0xba, 0x6d, 0x1a, 0xa5, 0x0c, 0x53, 0x3e, 0xbb, 0x42,
+ 0xf7, 0xa0, 0xe8, 0x1c, 0x1b, 0xf2, 0x3e, 0x56, 0x2c, 0x67, 0x17, 0x2b, 0x0e, 0x99, 0xa5, 0x81,
+ 0x96, 0x28, 0x38, 0xc7, 0x46, 0x2d, 0x40, 0x46, 0x2f, 0xa0, 0xa8, 0x1b, 0xf2, 0x5e, 0x5b, 0x6f,
+ 0xed, 0x3b, 0xf2, 0x91, 0xa5, 0x3b, 0xd8, 0x2e, 0xcd, 0x53, 0x85, 0x44, 0xd9, 0x6d, 0x83, 0xaf,
+ 0x1b, 0x6b, 0xaf, 0x48, 0x49, 0xae, 0x9a, 0xbc, 0x6e, 0x3c, 0xa1, 0xfc, 0x94, 0x68, 0xaf, 0xa7,
+ 0xd2, 0xb3, 0xc5, 0xb4, 0xf8, 0x9f, 0x04, 0xc8, 0xbb, 0xe6, 0x16, 0xe7, 0xc8, 0xb8, 0x0b, 0x45,
+ 0xd3, 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x9f, 0x70, 0xf2, 0xa6, 0x81, 0xb7, 0x09,
+ 0x99, 0xa9, 0x0b, 0x6d, 0xc3, 0xbc, 0xed, 0x28, 0x2d, 0xdd, 0x68, 0x05, 0xd4, 0x3b, 0x3d, 0x39,
+ 0x58, 0x28, 0x72, 0x6e, 0x8f, 0x1e, 0x8a, 0x52, 0xfe, 0x58, 0x80, 0xf9, 0x15, 0xad, 0xa3, 0x1b,
+ 0x8d, 0x6e, 0x5b, 0x8f, 0x75, 0x0d, 0xe2, 0x26, 0x64, 0x6c, 0x22, 0xd3, 0x77, 0xf8, 0x3e, 0xa2,
+ 0x4c, 0xd3, 0x3b, 0xc4, 0xf3, 0x3f, 0x83, 0x02, 0x3e, 0xee, 0xea, 0xec, 0xd5, 0x03, 0x03, 0x42,
+ 0xa9, 0xc9, 0xdb, 0x96, 0xf7, 0x79, 0xc9, 0x2d, 0xde, 0xa6, 0xcf, 0x00, 0x05, 0x9b, 0x14, 0x27,
+ 0x76, 0xf9, 0x0c, 0x16, 0xa8, 0xe8, 0x1d, 0xc3, 0x8e, 0x59, 0x5f, 0xe2, 0x2f, 0xc2, 0x85, 0xb0,
+ 0xe8, 0x38, 0xeb, 0xfd, 0x8a, 0xf7, 0xf2, 0x26, 0xb6, 0x62, 0x05, 0xb1, 0x9e, 0xae, 0xb9, 0xe0,
+ 0x38, 0xeb, 0xfc, 0xab, 0x02, 0x5c, 0xa1, 0xb2, 0xe9, 0xdb, 0x99, 0x3d, 0x6c, 0x3d, 0xc3, 0x8a,
+ 0x1d, 0x2b, 0x02, 0xbf, 0x01, 0x33, 0x0c, 0x49, 0x53, 0xfb, 0x9c, 0x5e, 0xcd, 0x92, 0xc8, 0xa5,
+ 0xe1, 0x98, 0x16, 0x89, 0x5c, 0xf8, 0x2d, 0x51, 0x81, 0xc5, 0xa8, 0x5a, 0xc4, 0xd9, 0xd2, 0xbf,
+ 0x25, 0xc0, 0x3c, 0x0f, 0x1a, 0x89, 0x29, 0x57, 0xf6, 0x49, 0xcc, 0x84, 0xaa, 0x90, 0x55, 0xe9,
+ 0x2f, 0xd9, 0x39, 0xe9, 0x62, 0x2a, 0x3f, 0x3f, 0x2a, 0xde, 0x64, 0x6c, 0xcd, 0x93, 0x2e, 0x26,
+ 0x41, 0xab, 0xfb, 0x9b, 0x28, 0x2a, 0xd0, 0xc8, 0x91, 0x11, 0x2b, 0x1d, 0x47, 0xb4, 0xac, 0x1b,
+ 0xfa, 0x71, 0x1d, 0xfc, 0xa3, 0x24, 0x57, 0x02, 0x7b, 0x06, 0x2f, 0x1e, 0x6b, 0x8c, 0xf2, 0x39,
+ 0x5c, 0x0a, 0xce, 0x2e, 0x81, 0x86, 0x27, 0xce, 0xd1, 0xf0, 0xc0, 0x0a, 0xbd, 0x4f, 0x45, 0x9f,
+ 0x41, 0x60, 0x0d, 0x5e, 0x66, 0x6d, 0x72, 0xd1, 0xcf, 0x79, 0xd4, 0x31, 0xef, 0x4b, 0x61, 0x74,
+ 0x1b, 0x55, 0x20, 0x8d, 0x8f, 0xbb, 0xb2, 0x86, 0x6d, 0x95, 0x3b, 0x2e, 0x31, 0x4a, 0x20, 0xa9,
+ 0xca, 0x00, 0x1e, 0x98, 0xc5, 0xc7, 0x5d, 0x42, 0x44, 0x3b, 0x64, 0xf6, 0x72, 0x43, 0x05, 0x5a,
+ 0x6d, 0x7b, 0x3c, 0xbc, 0xf0, 0x2d, 0x85, 0x8b, 0x2b, 0x78, 0x51, 0x02, 0x13, 0x21, 0xfe, 0x50,
+ 0x80, 0xb7, 0x22, 0x7b, 0x2d, 0xce, 0x89, 0xec, 0x13, 0x48, 0xd1, 0xc6, 0x27, 0xce, 0xd9, 0x78,
+ 0xca, 0x25, 0x7e, 0x2f, 0xc1, 0xc7, 0xb8, 0x84, 0xdb, 0x26, 0x51, 0x6c, 0xec, 0xab, 0x6c, 0xcf,
+ 0x61, 0xee, 0xd0, 0x74, 0x48, 0x6c, 0xc2, 0xbb, 0x3d, 0x71, 0xee, 0x6e, 0xcf, 0x51, 0x01, 0x6e,
+ 0x8f, 0xbf, 0x84, 0x79, 0xc3, 0x34, 0xe4, 0xb0, 0xd0, 0xf3, 0xdb, 0x52, 0xc1, 0x30, 0x8d, 0x97,
+ 0x01, 0xb9, 0x9e, 0x9f, 0xe9, 0xd3, 0x44, 0x9c, 0x7e, 0xe6, 0xfb, 0x02, 0x2c, 0x78, 0x61, 0x53,
+ 0xcc, 0x11, 0xf4, 0x07, 0x90, 0x34, 0xcc, 0xa3, 0xf3, 0xac, 0x62, 0x92, 0xf2, 0x64, 0xd6, 0x0b,
+ 0xd7, 0x28, 0xce, 0xf6, 0xfe, 0xab, 0x04, 0x64, 0x9e, 0x56, 0xe2, 0x6c, 0xe5, 0x27, 0x7c, 0x85,
+ 0x9c, 0xf5, 0x77, 0x94, 0xb5, 0x7b, 0xcf, 0x2b, 0x3f, 0xad, 0x6c, 0xe0, 0x13, 0xd7, 0xda, 0x09,
+ 0x17, 0x5a, 0x81, 0x8c, 0xb3, 0x6f, 0x61, 0x7b, 0xdf, 0x6c, 0x6b, 0xe7, 0x09, 0x73, 0x7c, 0xae,
+ 0x45, 0x0c, 0xd3, 0x54, 0xae, 0x9b, 0x8d, 0x21, 0x44, 0x64, 0x63, 0x90, 0xc7, 0x78, 0x91, 0x62,
+ 0xe2, 0x3c, 0x8f, 0x09, 0x84, 0x88, 0xd3, 0xc5, 0x19, 0xf1, 0x05, 0x00, 0x69, 0x4e, 0x9c, 0x5d,
+ 0xf2, 0x6b, 0x49, 0xc8, 0x6f, 0xf7, 0xec, 0xfd, 0x98, 0xad, 0xaf, 0x02, 0xd0, 0xed, 0xd9, 0x14,
+ 0x82, 0x1c, 0x1b, 0xbc, 0xcd, 0x63, 0x12, 0x3d, 0xdc, 0x46, 0x33, 0xbe, 0xe6, 0xb1, 0x81, 0x6a,
+ 0x5c, 0x08, 0x96, 0xfd, 0x6c, 0x91, 0x1b, 0xa3, 0xc0, 0x6a, 0xf3, 0xd8, 0xd8, 0xc4, 0x1e, 0x4a,
+ 0x65, 0x92, 0x30, 0x91, 0xf4, 0x09, 0xcc, 0x92, 0x0b, 0xd9, 0x31, 0xcf, 0xd3, 0xcd, 0x33, 0x84,
+ 0xa7, 0x69, 0xa2, 0xc7, 0x90, 0x61, 0xdc, 0x64, 0xf6, 0x9b, 0xa1, 0xb3, 0x5f, 0x54, 0x5b, 0xb8,
+ 0x1a, 0xe9, 0xbc, 0x97, 0xa6, 0xac, 0x64, 0xae, 0xbb, 0x00, 0xd3, 0x7b, 0xa6, 0xa5, 0xba, 0xef,
+ 0x7b, 0xd9, 0x05, 0xeb, 0xcf, 0xf5, 0x54, 0x3a, 0x5d, 0xcc, 0xac, 0xa7, 0xd2, 0x99, 0x22, 0x88,
+ 0xbf, 0x25, 0x40, 0xc1, 0xeb, 0x88, 0x38, 0x27, 0x84, 0x4a, 0x48, 0x8b, 0xe7, 0xef, 0x0a, 0xa2,
+ 0x40, 0xf1, 0xdf, 0xd0, 0x88, 0x48, 0x35, 0x0f, 0x69, 0xcf, 0xc4, 0x69, 0x29, 0x8f, 0x59, 0x2e,
+ 0x50, 0xe2, 0xbc, 0xbd, 0x4b, 0xd3, 0x82, 0x1e, 0xc0, 0x05, 0xbd, 0x43, 0xfc, 0xb9, 0xee, 0xb4,
+ 0x4f, 0x38, 0x6c, 0x73, 0xb0, 0xfb, 0x62, 0x79, 0xc1, 0xbf, 0x57, 0x71, 0x6f, 0x89, 0x7f, 0x87,
+ 0x2e, 0x80, 0xfb, 0x2d, 0x89, 0x53, 0xd5, 0x75, 0x98, 0xb3, 0x98, 0x68, 0x12, 0xd6, 0x9c, 0x53,
+ 0xdb, 0x39, 0x8f, 0x95, 0x28, 0xfc, 0x77, 0x12, 0x50, 0x78, 0xd1, 0xc3, 0xd6, 0xc9, 0xd7, 0x49,
+ 0xdd, 0xb7, 0xa1, 0x70, 0xa4, 0xe8, 0x8e, 0xbc, 0x67, 0x5a, 0x72, 0xaf, 0xab, 0x29, 0x8e, 0x9b,
+ 0x90, 0x32, 0x47, 0xc8, 0x4f, 0x4c, 0x6b, 0x87, 0x12, 0x11, 0x06, 0x74, 0x60, 0x98, 0x47, 0x86,
+ 0x4c, 0xc8, 0x14, 0x28, 0x1f, 0x1b, 0x7c, 0x55, 0x7a, 0xf5, 0xc3, 0xff, 0x78, 0xba, 0xf4, 0x68,
+ 0xa2, 0x34, 0x33, 0x9a, 0x52, 0xd7, 0xeb, 0xe9, 0x5a, 0x79, 0x67, 0xa7, 0xbe, 0x26, 0x15, 0xa9,
+ 0xc8, 0x57, 0x4c, 0x62, 0xf3, 0xd8, 0xb0, 0xc5, 0xbf, 0x9b, 0x80, 0xa2, 0xaf, 0xa3, 0x38, 0x3b,
+ 0xb2, 0x0a, 0xd9, 0xd7, 0x3d, 0x6c, 0xe9, 0x6f, 0xd0, 0x8d, 0xc0, 0x19, 0x89, 0xdb, 0xf9, 0x1c,
+ 0x72, 0x21, 0x0d, 0x24, 0xbf, 0x9a, 0x06, 0xb2, 0x47, 0x7e, 0xe3, 0xd1, 0x7d, 0x98, 0x77, 0x8e,
+ 0x0d, 0x99, 0x25, 0x18, 0xb2, 0xa4, 0x14, 0x37, 0x5f, 0xa2, 0xe0, 0x10, 0x7d, 0x10, 0x3a, 0x4d,
+ 0x48, 0xb1, 0xc5, 0xdf, 0x17, 0x00, 0x51, 0x45, 0xd5, 0xd9, 0x6b, 0x83, 0xaf, 0x8b, 0x3d, 0xdd,
+ 0x85, 0x22, 0x4d, 0xd9, 0x94, 0xf5, 0x3d, 0xb9, 0xa3, 0xdb, 0xb6, 0x6e, 0xb4, 0xb8, 0x41, 0xe5,
+ 0x29, 0xbd, 0xbe, 0xb7, 0xc9, 0xa8, 0xe2, 0x5f, 0x81, 0x85, 0x50, 0x03, 0xe2, 0xec, 0xec, 0xeb,
+ 0x90, 0xdb, 0x63, 0x6f, 0x69, 0xa9, 0x70, 0xbe, 0xe2, 0x98, 0xa5, 0x34, 0xf6, 0x3c, 0xf1, 0xcf,
+ 0x13, 0x70, 0x41, 0xc2, 0xb6, 0xd9, 0x3e, 0xc4, 0xf1, 0xab, 0xb0, 0x06, 0xfc, 0x75, 0x8e, 0xfc,
+ 0x46, 0x9a, 0xcc, 0x30, 0x66, 0x36, 0xcd, 0x85, 0x97, 0xed, 0x6f, 0x8e, 0xb6, 0xd8, 0xc1, 0x85,
+ 0x7a, 0xbe, 0xec, 0x97, 0x0a, 0x2d, 0xfb, 0x99, 0x50, 0xd0, 0x5b, 0x86, 0x49, 0x7c, 0x9a, 0x8d,
+ 0x5f, 0x1b, 0xbd, 0x8e, 0x0b, 0x86, 0xca, 0xa3, 0x2a, 0x59, 0x67, 0x2c, 0x0d, 0xfc, 0x7a, 0xab,
+ 0xd7, 0xa1, 0xb1, 0xf3, 0xea, 0x25, 0x52, 0xdf, 0xb3, 0xd3, 0xa5, 0x7c, 0xe8, 0x9e, 0x2d, 0xe5,
+ 0x75, 0xef, 0x9a, 0x48, 0x17, 0xbf, 0x0d, 0x17, 0xfb, 0x94, 0x1d, 0x67, 0xc4, 0xf3, 0x2f, 0x92,
+ 0x70, 0x25, 0x2c, 0x3e, 0x6e, 0x88, 0xf3, 0x75, 0xef, 0xd0, 0x1a, 0xcc, 0x75, 0x74, 0xe3, 0xcd,
+ 0x56, 0x2f, 0x73, 0x1d, 0xdd, 0xf0, 0x57, 0x8a, 0x23, 0x4c, 0x63, 0xe6, 0x67, 0x6a, 0x1a, 0x0a,
+ 0x2c, 0x46, 0xf5, 0x5d, 0x9c, 0xf6, 0xf1, 0x3d, 0x01, 0x72, 0x71, 0x2f, 0xcb, 0xbd, 0x59, 0xa2,
+ 0x9c, 0xd8, 0x84, 0xb9, 0x9f, 0xc1, 0x3a, 0xde, 0xef, 0x08, 0x80, 0x9a, 0x56, 0xcf, 0x20, 0xa0,
+ 0xf6, 0x99, 0xd9, 0x8a, 0xb3, 0x99, 0x17, 0x60, 0x5a, 0x37, 0x34, 0x7c, 0x4c, 0x9b, 0x99, 0x92,
+ 0xd8, 0x45, 0xe8, 0xed, 0x64, 0x72, 0xa2, 0xb7, 0x93, 0xe2, 0xe7, 0xb0, 0x10, 0xaa, 0x62, 0x9c,
+ 0xed, 0xff, 0xaf, 0x09, 0x58, 0xe0, 0x0d, 0x89, 0x7d, 0x05, 0xf3, 0x9b, 0x30, 0xdd, 0x26, 0x32,
+ 0x47, 0xf4, 0x33, 0x7d, 0xa6, 0xdb, 0xcf, 0xb4, 0x30, 0xfa, 0x79, 0x80, 0xae, 0x85, 0x0f, 0x65,
+ 0xc6, 0x9a, 0x9c, 0x88, 0x35, 0x43, 0x38, 0x28, 0x01, 0xfd, 0x40, 0x80, 0x02, 0x19, 0xd0, 0x5d,
+ 0xcb, 0xec, 0x9a, 0x36, 0x89, 0x59, 0xec, 0xc9, 0x60, 0xce, 0x8b, 0xb3, 0xd3, 0xa5, 0xb9, 0x4d,
+ 0xdd, 0xd8, 0xe6, 0x8c, 0xcd, 0xc6, 0xc4, 0x7b, 0x00, 0xdc, 0x9d, 0x10, 0xe5, 0x4a, 0xdb, 0x54,
+ 0x0f, 0xfc, 0xf7, 0x6d, 0xc4, 0xb3, 0x78, 0xe2, 0x6c, 0xf1, 0x8f, 0x04, 0xb8, 0xf0, 0x33, 0x5b,
0x2e, 0xfe, 0x7f, 0xa1, 0x6c, 0xf1, 0x25, 0x14, 0xe9, 0x8f, 0xba, 0xb1, 0x67, 0xc6, 0xb9, 0x70,
- 0xff, 0xeb, 0x02, 0xcc, 0x07, 0x04, 0xc7, 0xe9, 0xe0, 0xbc, 0x91, 0x9e, 0xc4, 0x5f, 0x24, 0x2e,
- 0x4f, 0x70, 0x94, 0xc4, 0x39, 0x06, 0xff, 0x79, 0x02, 0x2e, 0x55, 0xd8, 0x67, 0x6e, 0x37, 0xee,
- 0x23, 0xce, 0x9e, 0x51, 0x82, 0xd9, 0x43, 0x6c, 0xd9, 0xba, 0xc9, 0xa6, 0xdf, 0x39, 0xc9, 0x7d,
- 0x44, 0x65, 0x48, 0xdb, 0x86, 0xd2, 0xb5, 0xf7, 0x4d, 0xf7, 0xfb, 0x9e, 0xf7, 0xec, 0xc5, 0xa8,
- 0x4c, 0xbf, 0x79, 0x8c, 0xca, 0xcc, 0xe8, 0x18, 0x95, 0xd9, 0xaf, 0x10, 0xa3, 0xc2, 0x3f, 0xa6,
- 0xfd, 0x3b, 0x01, 0x2e, 0x0f, 0x68, 0x2e, 0xce, 0xde, 0xf2, 0x5d, 0xc8, 0xaa, 0x5c, 0x30, 0x31,
- 0xd8, 0xec, 0x4b, 0x61, 0x9d, 0x64, 0x7b, 0x43, 0xdc, 0x72, 0x76, 0xba, 0x08, 0x6e, 0x51, 0xeb,
- 0x6b, 0x5c, 0x39, 0xe4, 0xb7, 0x26, 0xfe, 0xca, 0x1c, 0x14, 0xaa, 0xc7, 0x6c, 0x95, 0xbc, 0xc1,
- 0xdc, 0x04, 0xf4, 0x04, 0xd2, 0x5d, 0xcb, 0x3c, 0xd4, 0xdd, 0x6a, 0xe4, 0x43, 0x01, 0x0a, 0x6e,
- 0x35, 0xfa, 0xb8, 0xb6, 0x39, 0x87, 0xe4, 0xf1, 0xa2, 0x26, 0x64, 0x9e, 0x99, 0xaa, 0xd2, 0x7e,
- 0xa2, 0xb7, 0xdd, 0x9e, 0xff, 0xfe, 0x78, 0x41, 0x4b, 0x1e, 0xcf, 0xb6, 0xe2, 0xec, 0xbb, 0x8d,
- 0xe0, 0x11, 0x51, 0x1d, 0xd2, 0x35, 0xc7, 0xe9, 0x92, 0x44, 0x6e, 0x3b, 0xee, 0x4c, 0x20, 0x94,
- 0xb0, 0xb8, 0x51, 0xb2, 0x2e, 0x3b, 0x6a, 0xc2, 0xfc, 0x53, 0xba, 0xe7, 0xab, 0xd2, 0x36, 0x7b,
- 0x5a, 0xc5, 0x34, 0xf6, 0xf4, 0x16, 0xb7, 0xdb, 0xb7, 0x27, 0x90, 0xf9, 0xb4, 0xd2, 0x90, 0x06,
- 0x05, 0xa0, 0x15, 0x48, 0x37, 0x1e, 0x71, 0x61, 0xcc, 0xaf, 0xbb, 0x35, 0x81, 0xb0, 0xc6, 0x23,
- 0xc9, 0x63, 0x43, 0xeb, 0x90, 0x5d, 0xf9, 0xa2, 0x67, 0x61, 0x2e, 0x65, 0x66, 0x68, 0x74, 0x44,
- 0xbf, 0x14, 0xca, 0x25, 0x05, 0x99, 0x51, 0x03, 0xf2, 0xaf, 0x4c, 0xeb, 0xa0, 0x6d, 0x2a, 0x6e,
- 0x0d, 0x67, 0xa9, 0xb8, 0x6f, 0x4c, 0x20, 0xce, 0x65, 0x94, 0xfa, 0x44, 0xa0, 0x6f, 0x43, 0x81,
- 0x34, 0x46, 0x53, 0xd9, 0x6d, 0xbb, 0x85, 0x4c, 0x53, 0xa9, 0xef, 0x4e, 0x20, 0xd5, 0xe3, 0x74,
- 0x17, 0xfe, 0xfb, 0x44, 0x95, 0x25, 0x98, 0x0b, 0x75, 0x02, 0x84, 0x20, 0xd5, 0x25, 0xed, 0x2d,
- 0xd0, 0xf8, 0x25, 0xfa, 0x1b, 0xbd, 0x07, 0xb3, 0x86, 0xa9, 0x61, 0x77, 0x84, 0xcc, 0xad, 0x5e,
- 0x38, 0x3b, 0x5d, 0x9c, 0xd9, 0x32, 0x35, 0xe6, 0xd1, 0xf0, 0x5f, 0xd2, 0x0c, 0xc9, 0x54, 0xd7,
- 0xca, 0xd7, 0x20, 0x45, 0xda, 0x9d, 0x18, 0xa6, 0x5d, 0xc5, 0xc6, 0x3b, 0x96, 0xce, 0xa5, 0xb9,
- 0x8f, 0xe5, 0x7f, 0x94, 0x80, 0x44, 0xe3, 0x11, 0xf1, 0xd9, 0x77, 0x7b, 0xea, 0x01, 0x76, 0x78,
- 0x3a, 0x7f, 0xa2, 0xbe, 0xbc, 0x85, 0xf7, 0x74, 0xe6, 0x5a, 0x65, 0x24, 0xfe, 0x84, 0xde, 0x01,
- 0x50, 0x54, 0x15, 0xdb, 0xb6, 0xec, 0xee, 0x05, 0xcc, 0x48, 0x19, 0x46, 0xd9, 0xc0, 0x27, 0x84,
- 0xcd, 0xc6, 0xaa, 0x85, 0x1d, 0x37, 0xf8, 0x8a, 0x3d, 0x11, 0x36, 0x07, 0x77, 0xba, 0xb2, 0x63,
- 0x1e, 0x60, 0x83, 0xf6, 0x93, 0x0c, 0x31, 0x35, 0x9d, 0x6e, 0x93, 0x10, 0x88, 0x95, 0xc4, 0x86,
- 0xe6, 0x9b, 0xb4, 0x8c, 0xe4, 0x3d, 0x13, 0x91, 0x16, 0x6e, 0xe9, 0x7c, 0xbb, 0x5c, 0x46, 0xe2,
- 0x4f, 0x44, 0x4b, 0x4a, 0xcf, 0xd9, 0xa7, 0x2d, 0x91, 0x91, 0xe8, 0x6f, 0x74, 0x1b, 0x0a, 0x2c,
- 0x5e, 0x53, 0xc6, 0x86, 0x2a, 0x53, 0xe3, 0x9a, 0xa1, 0xc9, 0x73, 0x8c, 0x5c, 0x35, 0x54, 0x62,
- 0x4a, 0xd1, 0x23, 0xe0, 0x04, 0xf9, 0xa0, 0x63, 0x13, 0x9d, 0x02, 0xc9, 0xb5, 0x5a, 0x38, 0x3b,
- 0x5d, 0xcc, 0x36, 0x68, 0xc2, 0xc6, 0x66, 0xa3, 0xbe, 0x26, 0x65, 0x59, 0xae, 0x8d, 0x8e, 0x5d,
- 0xd7, 0xca, 0xbf, 0x21, 0x40, 0xf2, 0x69, 0xa5, 0x71, 0x6e, 0x95, 0xb9, 0x05, 0x4d, 0x06, 0x0a,
- 0x7a, 0x07, 0x0a, 0xbb, 0x7a, 0xbb, 0xad, 0x1b, 0x2d, 0xe2, 0x45, 0x7d, 0x17, 0xab, 0xae, 0xc2,
- 0xf2, 0x9c, 0xbc, 0xcd, 0xa8, 0xe8, 0x1a, 0x64, 0x55, 0x0b, 0x6b, 0xd8, 0x70, 0x74, 0xa5, 0x6d,
- 0x73, 0xcd, 0x05, 0x49, 0xe5, 0x5f, 0x16, 0x60, 0x9a, 0x8e, 0x00, 0xf4, 0x36, 0x64, 0x54, 0xd3,
- 0x70, 0x14, 0xdd, 0xe0, 0xa6, 0x2c, 0x23, 0xf9, 0x84, 0xa1, 0xc5, 0xbb, 0x0e, 0x39, 0x45, 0x55,
- 0xcd, 0x9e, 0xe1, 0xc8, 0x86, 0xd2, 0xc1, 0xbc, 0x98, 0x59, 0x4e, 0xdb, 0x52, 0x3a, 0x18, 0x2d,
- 0x82, 0xfb, 0xe8, 0x6d, 0xf1, 0xcc, 0x48, 0xc0, 0x49, 0x1b, 0xf8, 0xa4, 0xfc, 0x07, 0x02, 0xa4,
- 0xdd, 0x31, 0x43, 0x8a, 0xd1, 0xc2, 0x06, 0x0b, 0x2a, 0x77, 0x8b, 0xe1, 0x11, 0xfa, 0xa7, 0xca,
- 0x8c, 0x3f, 0x55, 0x5e, 0x80, 0x69, 0x87, 0x0c, 0x0b, 0x5e, 0x02, 0xf6, 0x40, 0xd7, 0xb3, 0xdb,
- 0x4a, 0x8b, 0x2d, 0xe7, 0x65, 0x24, 0xf6, 0x40, 0x2a, 0xc3, 0xc3, 0x7e, 0x99, 0x46, 0xf8, 0x13,
- 0x29, 0x29, 0x0b, 0x4e, 0xdd, 0xc5, 0x2d, 0xdd, 0xa0, 0x7d, 0x29, 0x29, 0x01, 0x25, 0xad, 0x12,
- 0x0a, 0x7a, 0x0b, 0x32, 0x2c, 0x03, 0x36, 0x34, 0xda, 0xa1, 0x92, 0x52, 0x9a, 0x12, 0xaa, 0x86,
- 0x56, 0xc6, 0x90, 0xf1, 0x06, 0x27, 0x69, 0xb6, 0x9e, 0xed, 0x29, 0x92, 0xfe, 0x46, 0xef, 0xc3,
- 0x85, 0xd7, 0x3d, 0xa5, 0xad, 0xef, 0xd1, 0x95, 0x3a, 0x1a, 0x75, 0x4f, 0x75, 0xc6, 0x6a, 0x82,
- 0xbc, 0x34, 0x2a, 0x81, 0xaa, 0xce, 0x1d, 0xcb, 0x49, 0x7f, 0x2c, 0x8b, 0xbf, 0x2b, 0xc0, 0x3c,
- 0x8b, 0x3c, 0x62, 0x01, 0xb3, 0xf1, 0xf9, 0x21, 0x1f, 0x43, 0x46, 0x53, 0x1c, 0x85, 0x6d, 0x5a,
- 0x4d, 0x8c, 0xdc, 0xb4, 0xea, 0x6d, 0xa2, 0x50, 0x1c, 0x85, 0x6e, 0x5c, 0x45, 0x90, 0x22, 0xbf,
- 0xd9, 0xfe, 0x5e, 0x89, 0xfe, 0x16, 0x3f, 0x03, 0x14, 0x2c, 0x68, 0x9c, 0x1e, 0xd9, 0x3d, 0xb8,
- 0x48, 0x74, 0x5d, 0x35, 0x54, 0xeb, 0xa4, 0xeb, 0xe8, 0xa6, 0xf1, 0x9c, 0xfe, 0xb5, 0x51, 0x31,
- 0xf0, 0x61, 0x8b, 0x7e, 0xcf, 0x12, 0x7f, 0x6f, 0x06, 0xe6, 0xaa, 0xc7, 0x5d, 0xd3, 0x8a, 0x75,
- 0x19, 0x6c, 0x15, 0x66, 0xf9, 0x4a, 0xc1, 0x88, 0x6f, 0xd7, 0x7d, 0xc6, 0xdc, 0xfd, 0x70, 0xcf,
- 0x19, 0xd1, 0x2a, 0x00, 0x0b, 0x6a, 0xa5, 0x81, 0x4b, 0xc9, 0x73, 0x7c, 0x6a, 0xa3, 0x6c, 0x74,
- 0x03, 0xc7, 0x16, 0x64, 0x3b, 0x87, 0xaa, 0x2a, 0xef, 0xe9, 0x6d, 0x87, 0xc7, 0x06, 0x46, 0x87,
- 0xb1, 0x6f, 0xbe, 0xac, 0x54, 0x9e, 0xd0, 0x4c, 0x2c, 0x4c, 0xcf, 0x7f, 0x96, 0x80, 0x48, 0x60,
- 0xbf, 0xd1, 0xbb, 0xc0, 0x37, 0x13, 0xc9, 0xb6, 0xbb, 0x6f, 0x70, 0x75, 0xee, 0xec, 0x74, 0x31,
- 0x23, 0x51, 0x6a, 0xa3, 0xd1, 0x94, 0x32, 0x2c, 0x43, 0xc3, 0x76, 0xd0, 0x0d, 0x98, 0x33, 0x3b,
- 0xba, 0x23, 0xbb, 0x4e, 0x12, 0xf7, 0x28, 0x73, 0x84, 0xe8, 0x3a, 0x51, 0xe7, 0xd9, 0x63, 0x32,
- 0x3b, 0xf1, 0x1e, 0x13, 0xf4, 0x37, 0x05, 0xb8, 0xc4, 0x15, 0x29, 0xef, 0xd2, 0x38, 0x7c, 0xa5,
- 0xad, 0x3b, 0x27, 0xf2, 0xc1, 0x61, 0x29, 0x4d, 0xfd, 0xd6, 0x9f, 0x8b, 0x6c, 0x90, 0x40, 0x3f,
- 0x58, 0x72, 0x9b, 0xe5, 0xe4, 0x19, 0x67, 0xde, 0x38, 0xac, 0x1a, 0x8e, 0x75, 0xb2, 0x7a, 0xf9,
- 0xec, 0x74, 0x71, 0x61, 0x30, 0xf5, 0xa5, 0xb4, 0x60, 0x0f, 0xb2, 0xa0, 0x1a, 0x00, 0xf6, 0xfa,
- 0x21, 0x9d, 0x31, 0xa2, 0xfd, 0x8f, 0xc8, 0x0e, 0x2b, 0x05, 0x78, 0xd1, 0x5d, 0x28, 0xf2, 0x3d,
- 0x3d, 0x7b, 0x7a, 0x1b, 0xcb, 0xb6, 0xfe, 0x05, 0xa6, 0x73, 0x4b, 0x52, 0xca, 0x33, 0x3a, 0x11,
- 0xd1, 0xd0, 0xbf, 0xc0, 0xe5, 0xef, 0x42, 0x69, 0x58, 0xe9, 0x83, 0x43, 0x20, 0xc3, 0x3e, 0xe9,
- 0x7e, 0x14, 0x5e, 0xcf, 0x99, 0xa0, 0xab, 0xf2, 0x35, 0x9d, 0x8f, 0x13, 0x1f, 0x09, 0xe2, 0x3f,
- 0x48, 0xc0, 0xdc, 0x6a, 0xaf, 0x7d, 0xf0, 0xbc, 0xdb, 0xe8, 0x75, 0x3a, 0x8a, 0x75, 0x42, 0xcc,
- 0x20, 0x33, 0x14, 0xa4, 0x80, 0x02, 0x33, 0x83, 0xd4, 0x12, 0xe8, 0x5f, 0x60, 0x32, 0x39, 0x05,
- 0xc2, 0x65, 0xf8, 0x3e, 0x03, 0x5a, 0x07, 0x9f, 0x4c, 0xb7, 0x02, 0x7c, 0x04, 0xa5, 0x40, 0x46,
- 0xba, 0xf8, 0x22, 0x63, 0xc3, 0xb1, 0x74, 0xcc, 0x16, 0x10, 0x93, 0x52, 0x20, 0xa6, 0xa7, 0x4e,
- 0x92, 0xab, 0x2c, 0x15, 0x35, 0x21, 0x47, 0x32, 0x9e, 0xc8, 0x74, 0x0a, 0x71, 0x17, 0x78, 0x1f,
- 0x44, 0x54, 0x2b, 0x54, 0xee, 0x25, 0xaa, 0x9f, 0x0a, 0xe5, 0xa1, 0x3f, 0xa5, 0x2c, 0xf6, 0x29,
- 0xe5, 0x4f, 0xa1, 0xd8, 0x9f, 0x21, 0xa8, 0xcb, 0x14, 0xd3, 0xe5, 0x85, 0xa0, 0x2e, 0x93, 0x01,
- 0x3d, 0xad, 0xa7, 0xd2, 0xa9, 0xe2, 0xb4, 0xf8, 0x67, 0x49, 0xc8, 0xbb, 0xdd, 0x2c, 0x4e, 0xa0,
- 0xb3, 0x0a, 0xd3, 0xa4, 0x53, 0xb8, 0x11, 0x28, 0xb7, 0x47, 0xf4, 0x6e, 0x1e, 0xd9, 0x4e, 0x3a,
- 0x8b, 0x0b, 0x92, 0x29, 0x6b, 0x1c, 0x06, 0xa7, 0xfc, 0xcb, 0x09, 0x48, 0x51, 0x6c, 0xf1, 0x00,
- 0x52, 0x74, 0xa2, 0x10, 0x26, 0x99, 0x28, 0x68, 0x56, 0x6f, 0x3a, 0x4b, 0x04, 0x5c, 0x53, 0xe2,
- 0xf3, 0xed, 0x2b, 0x1f, 0x3c, 0x78, 0x48, 0x8d, 0x4d, 0x4e, 0xe2, 0x4f, 0x68, 0x95, 0x86, 0x46,
- 0x99, 0x96, 0x83, 0x35, 0xee, 0xd3, 0x5f, 0x1b, 0xd7, 0xbe, 0xee, 0xa4, 0xe4, 0xf2, 0xa1, 0x2b,
- 0x90, 0x24, 0x56, 0x6c, 0x96, 0x45, 0x39, 0x9c, 0x9d, 0x2e, 0x26, 0x89, 0xfd, 0x22, 0x34, 0xb4,
- 0x0c, 0xd9, 0xb0, 0xc9, 0x20, 0x1e, 0x1c, 0x35, 0x8c, 0x81, 0xe1, 0x0e, 0x6d, 0x6f, 0x68, 0x31,
- 0x3c, 0xcb, 0xdb, 0xf8, 0x2f, 0x52, 0x30, 0x57, 0xef, 0xc4, 0x3d, 0xa5, 0xac, 0x84, 0x5b, 0x38,
- 0x0a, 0x08, 0x85, 0x5e, 0x1a, 0xd1, 0xc0, 0xa1, 0x19, 0x3c, 0x79, 0xbe, 0x19, 0xbc, 0x4e, 0x3c,
- 0x65, 0x7e, 0xf0, 0x44, 0x72, 0x08, 0xe6, 0x09, 0xbf, 0x9f, 0xfa, 0x29, 0x12, 0xe1, 0xf1, 0xf7,
- 0x7a, 0xd0, 0x48, 0x95, 0x4f, 0xa9, 0x43, 0xce, 0x7a, 0xd9, 0xcc, 0xe4, 0xbd, 0x6c, 0x16, 0x1b,
- 0x1a, 0x9d, 0xd4, 0xc2, 0x16, 0x75, 0xf6, 0xcd, 0x2d, 0x6a, 0xd9, 0xe1, 0x9d, 0xf5, 0x63, 0x48,
- 0x6a, 0xba, 0xdb, 0x38, 0x93, 0x4f, 0xd5, 0x84, 0x69, 0x4c, 0xaf, 0x4d, 0x05, 0x7b, 0x2d, 0xeb,
- 0x25, 0xe5, 0x3a, 0x80, 0xaf, 0x1b, 0x74, 0x0d, 0x66, 0xcc, 0xb6, 0xe6, 0x6e, 0x76, 0x99, 0x5b,
- 0xcd, 0x9c, 0x9d, 0x2e, 0x4e, 0x3f, 0x6f, 0x6b, 0xf5, 0x35, 0x69, 0xda, 0x6c, 0x6b, 0x75, 0x8d,
- 0x9e, 0xfa, 0x81, 0x8f, 0x64, 0x2f, 0x12, 0x2e, 0x27, 0xcd, 0x1a, 0xf8, 0x68, 0x0d, 0xdb, 0x2a,
- 0xef, 0x70, 0xbf, 0x2d, 0x40, 0xde, 0xd5, 0x7d, 0xbc, 0x46, 0x25, 0xad, 0x77, 0xf8, 0x20, 0x4b,
- 0x9e, 0x6f, 0x90, 0xb9, 0x7c, 0x7c, 0x63, 0xf1, 0xaf, 0x0a, 0x3c, 0xb6, 0xb9, 0xa1, 0x2a, 0x0e,
- 0x71, 0x2a, 0x62, 0x1c, 0x18, 0xf7, 0xa0, 0x68, 0x29, 0x86, 0x66, 0x76, 0xf4, 0x2f, 0x30, 0x5b,
- 0x08, 0xb5, 0xf9, 0x67, 0xcf, 0x82, 0x47, 0xa7, 0xab, 0x7e, 0xb6, 0xf8, 0x07, 0x09, 0x1e, 0x07,
- 0xed, 0x15, 0x23, 0x4e, 0x75, 0x7d, 0x07, 0xe6, 0x83, 0x53, 0x1b, 0x0b, 0xb5, 0x64, 0xa3, 0xf5,
- 0xbd, 0x08, 0x79, 0x51, 0x05, 0x61, 0xf1, 0x8c, 0x6e, 0x70, 0x7d, 0x60, 0x46, 0xa4, 0xc2, 0x50,
- 0x05, 0xb2, 0xfc, 0xe3, 0x83, 0xb1, 0x67, 0xba, 0x81, 0x62, 0x6f, 0x0f, 0x0b, 0x8b, 0xac, 0x1b,
- 0x7b, 0xa6, 0xfb, 0x21, 0xdf, 0x72, 0x09, 0x76, 0xf9, 0x17, 0x60, 0x9a, 0x26, 0xbf, 0x81, 0x89,
- 0xe6, 0xad, 0xf9, 0xa7, 0x09, 0xb8, 0x49, 0x4b, 0xff, 0x12, 0x5b, 0xfa, 0xde, 0xc9, 0xb6, 0x65,
- 0x3a, 0x58, 0x75, 0xb0, 0xe6, 0xaf, 0x9c, 0xc7, 0x6a, 0xf7, 0x32, 0x5d, 0xf7, 0x05, 0xe7, 0x0a,
- 0x38, 0xf3, 0xb8, 0xd0, 0x06, 0x14, 0x78, 0x68, 0x81, 0xd2, 0xd6, 0x0f, 0xb1, 0xac, 0x38, 0xe7,
- 0x99, 0xdd, 0xe6, 0x18, 0xef, 0x0a, 0x61, 0x5d, 0x71, 0x90, 0x06, 0x19, 0x2e, 0x4c, 0xd7, 0xf8,
- 0x81, 0x44, 0x4f, 0xbf, 0xda, 0x82, 0x62, 0x9a, 0xc5, 0x37, 0xd4, 0xd7, 0xa4, 0x34, 0x93, 0x5c,
- 0xd7, 0xc4, 0xff, 0x28, 0xc0, 0xad, 0x31, 0x2a, 0x8e, 0xb3, 0xeb, 0x96, 0x21, 0x7d, 0x48, 0x5e,
- 0xa4, 0x73, 0x1d, 0xa7, 0x25, 0xef, 0x19, 0x6d, 0xc2, 0xdc, 0x9e, 0xa2, 0xb7, 0xfd, 0x2e, 0x3d,
- 0x3c, 0x3e, 0x31, 0x3a, 0x1a, 0x37, 0xc7, 0xd8, 0x59, 0x1f, 0x16, 0x7f, 0x33, 0x01, 0xf3, 0x2b,
- 0x9a, 0xd6, 0x68, 0x70, 0x1b, 0x18, 0x5f, 0x4f, 0x71, 0x41, 0x66, 0xc2, 0x07, 0x99, 0xe8, 0x3d,
- 0x40, 0x9a, 0x6e, 0xb3, 0x83, 0x4f, 0xec, 0x7d, 0x45, 0x33, 0x8f, 0xfc, 0xb8, 0x8c, 0x79, 0x37,
- 0xa5, 0xe1, 0x26, 0xa0, 0x06, 0x50, 0xb4, 0x23, 0xdb, 0x8e, 0xe2, 0x7d, 0x77, 0xba, 0x35, 0xd1,
- 0x96, 0x32, 0x06, 0x83, 0xbc, 0x47, 0x29, 0x43, 0xe4, 0xd0, 0x9f, 0xc4, 0x6f, 0xd7, 0x49, 0xd5,
- 0x1d, 0x59, 0xb1, 0xdd, 0xfd, 0x43, 0xec, 0xc8, 0x95, 0x3c, 0xa3, 0xaf, 0xd8, 0x6c, 0x5b, 0x10,
- 0xdb, 0xf0, 0xe0, 0xab, 0x26, 0x4e, 0x48, 0xfc, 0x77, 0x05, 0xc8, 0x4b, 0x78, 0xcf, 0xc2, 0x76,
- 0xac, 0x8b, 0x02, 0x4f, 0x20, 0x67, 0x31, 0xa9, 0xf2, 0x9e, 0x65, 0x76, 0xce, 0x33, 0xae, 0xb2,
- 0x9c, 0xf1, 0x89, 0x65, 0x76, 0xb8, 0x61, 0x79, 0x09, 0x05, 0xaf, 0x8c, 0x71, 0x56, 0xfe, 0x77,
- 0xe9, 0x76, 0x69, 0x26, 0x38, 0xee, 0x00, 0x89, 0x78, 0x35, 0x40, 0x3f, 0x54, 0x05, 0x0b, 0x1a,
- 0xa7, 0x1a, 0xfe, 0xab, 0x00, 0xf9, 0x46, 0x6f, 0x97, 0x9d, 0xd9, 0x15, 0x9f, 0x06, 0xaa, 0x90,
- 0x69, 0xe3, 0x3d, 0x47, 0x7e, 0xa3, 0x50, 0xfd, 0x34, 0x61, 0xa5, 0x1b, 0x15, 0x9e, 0x02, 0x58,
- 0x74, 0x8b, 0x1d, 0x95, 0x93, 0x3c, 0xa7, 0x9c, 0x0c, 0xe5, 0x25, 0x64, 0xf1, 0x2f, 0x12, 0x50,
- 0xf0, 0xaa, 0x19, 0xa7, 0x95, 0x7c, 0x15, 0xb2, 0x0e, 0xc9, 0xf3, 0x58, 0x87, 0x79, 0x1e, 0x13,
- 0x12, 0x6d, 0x21, 0x96, 0x60, 0x81, 0x3a, 0x2e, 0xb2, 0xd2, 0xed, 0xb6, 0x75, 0x17, 0xee, 0x52,
- 0xfb, 0x93, 0x92, 0xe6, 0x69, 0xd2, 0x0a, 0x4b, 0xa1, 0x40, 0x17, 0xfd, 0x9a, 0x00, 0xb9, 0x3d,
- 0x0b, 0xe3, 0x2f, 0xb0, 0x4c, 0xa1, 0xd7, 0x64, 0x41, 0x2f, 0x6b, 0xa4, 0x0c, 0x5f, 0xf9, 0xa3,
- 0x78, 0x96, 0xbd, 0xb8, 0x41, 0xde, 0xcb, 0x3b, 0xed, 0x2b, 0x98, 0xa7, 0x8d, 0x12, 0xf7, 0xde,
- 0x62, 0xf1, 0x47, 0x09, 0x40, 0x41, 0xc9, 0x3f, 0xbb, 0xc6, 0x4c, 0xc4, 0xd7, 0x98, 0xef, 0x02,
- 0x62, 0x61, 0x93, 0xb6, 0xdc, 0xc5, 0x96, 0x6c, 0x63, 0xd5, 0xe4, 0xc7, 0x43, 0x09, 0x52, 0x91,
- 0xa7, 0x6c, 0x63, 0xab, 0x41, 0xe9, 0x68, 0x05, 0xc0, 0x77, 0xe9, 0xf8, 0x8c, 0x33, 0x89, 0x47,
- 0x97, 0xf1, 0x3c, 0x3a, 0xf1, 0xfb, 0x02, 0xe4, 0x37, 0xf5, 0x96, 0xa5, 0xc4, 0x7a, 0xfa, 0x11,
- 0xfa, 0x38, 0xbc, 0x98, 0x9e, 0x7d, 0x58, 0x8e, 0x0a, 0xf4, 0x61, 0x39, 0x5c, 0x88, 0xc6, 0x19,
- 0x88, 0x21, 0xf7, 0x4a, 0x14, 0xa7, 0x05, 0xfb, 0xf7, 0x65, 0xc8, 0xf1, 0x72, 0xef, 0x18, 0xba,
- 0x69, 0xa0, 0x07, 0x90, 0x6c, 0xf1, 0x8f, 0x25, 0xd9, 0xc8, 0x85, 0x4d, 0xff, 0x6c, 0xc1, 0xda,
- 0x94, 0x44, 0xf2, 0x12, 0x96, 0x6e, 0xcf, 0x89, 0x70, 0x25, 0xfd, 0xd8, 0xf7, 0x20, 0x4b, 0xb7,
- 0xe7, 0xa0, 0x06, 0x14, 0x54, 0xff, 0x40, 0x33, 0x99, 0xb0, 0x27, 0x87, 0xc2, 0xce, 0xc8, 0xa3,
- 0xe5, 0x6a, 0x53, 0x52, 0x5e, 0x0d, 0x25, 0xa0, 0x4a, 0xf0, 0x1c, 0xad, 0xd4, 0x40, 0x60, 0x9d,
- 0xbf, 0x6b, 0x3b, 0x7c, 0x86, 0x57, 0x6d, 0x2a, 0x70, 0xdc, 0x16, 0xfa, 0x18, 0x66, 0x34, 0x7a,
- 0x62, 0x13, 0x37, 0x01, 0x51, 0x0d, 0x1d, 0x3a, 0x18, 0xab, 0x36, 0x25, 0x71, 0x0e, 0xb4, 0x0e,
- 0x39, 0xf6, 0x8b, 0x39, 0x76, 0x1c, 0x8b, 0xdf, 0x1a, 0x2e, 0x21, 0x30, 0x75, 0xd6, 0xa6, 0xa4,
- 0xac, 0xe6, 0x53, 0xd1, 0x37, 0x21, 0x65, 0xab, 0x8a, 0x8b, 0xc6, 0xaf, 0x0e, 0x39, 0x28, 0xc5,
- 0x67, 0xa6, 0xb9, 0xd1, 0x63, 0x76, 0xb4, 0xa7, 0x73, 0xec, 0x2e, 0x8c, 0x46, 0x15, 0x3f, 0xb4,
- 0x21, 0x9f, 0x14, 0x1f, 0x53, 0x02, 0x7a, 0x0a, 0x59, 0x85, 0x78, 0xc8, 0x32, 0xdd, 0xd2, 0x4a,
- 0x57, 0x42, 0xa3, 0xc3, 0x11, 0x06, 0xb6, 0x20, 0xd7, 0xe8, 0xae, 0x7f, 0x97, 0xe8, 0x0b, 0xea,
- 0x60, 0xab, 0x85, 0x4b, 0xd9, 0xd1, 0x82, 0x82, 0xe1, 0x74, 0x9e, 0x20, 0x4a, 0x24, 0x9e, 0xb2,
- 0xb7, 0x9f, 0x9c, 0x56, 0x2a, 0x37, 0xf4, 0x03, 0x78, 0xc4, 0x4e, 0xa9, 0xda, 0x94, 0x94, 0xdb,
- 0x0f, 0x90, 0xd1, 0x12, 0x24, 0x5a, 0x6a, 0x69, 0x6e, 0xa8, 0x49, 0xf0, 0x76, 0x03, 0xd5, 0xa6,
- 0xa4, 0x44, 0x4b, 0x45, 0x9f, 0x42, 0x9a, 0x6d, 0xed, 0x38, 0x36, 0x4a, 0xf9, 0xa1, 0x63, 0x2c,
- 0xbc, 0x41, 0xa6, 0x36, 0x25, 0xd1, 0xdd, 0x24, 0xe4, 0x7d, 0xdb, 0x90, 0xb7, 0x58, 0x3c, 0xa2,
- 0x1b, 0x3d, 0x5c, 0x1c, 0x1a, 0x14, 0x10, 0x15, 0x40, 0x5c, 0xa3, 0x58, 0x29, 0x40, 0x47, 0xdf,
- 0x81, 0x0b, 0x61, 0x89, 0xbc, 0xa7, 0xcd, 0x0f, 0xfd, 0xc0, 0x3d, 0x34, 0x98, 0xb5, 0x36, 0x25,
- 0x21, 0x6b, 0x20, 0x11, 0x7d, 0x08, 0xd3, 0xac, 0xd5, 0x10, 0x15, 0x19, 0x15, 0xe7, 0xd2, 0xd7,
- 0x60, 0x2c, 0x3f, 0xe9, 0xfc, 0x0e, 0x0f, 0xca, 0x93, 0xdb, 0x66, 0xab, 0xb4, 0x30, 0xb4, 0xf3,
- 0x0f, 0x86, 0x17, 0x92, 0xce, 0xef, 0xf8, 0x54, 0xd2, 0xee, 0x16, 0x4b, 0xe1, 0x81, 0x58, 0x17,
- 0x86, 0xb6, 0x7b, 0x44, 0xac, 0x5e, 0x8d, 0x6e, 0x97, 0xf0, 0xc9, 0xa4, 0x68, 0x16, 0x3b, 0xdb,
- 0x47, 0xa6, 0x63, 0xea, 0xe2, 0xd0, 0xa2, 0x0d, 0x1e, 0x7e, 0x54, 0xa3, 0x5e, 0xa5, 0x47, 0x45,
- 0x2f, 0xa1, 0xc8, 0x4f, 0xe0, 0xf0, 0xbf, 0xc2, 0x5c, 0xa2, 0xf2, 0xee, 0x45, 0x9a, 0xae, 0xa8,
- 0x28, 0xa6, 0xda, 0x94, 0x54, 0x50, 0xc3, 0x29, 0xe8, 0x33, 0x98, 0xa7, 0xf2, 0x64, 0xd5, 0x3f,
- 0x34, 0xa5, 0x54, 0x1a, 0x38, 0x82, 0x63, 0xf8, 0xf9, 0x2a, 0xae, 0xe4, 0xa2, 0xda, 0x97, 0x44,
- 0xba, 0xb1, 0x6e, 0xe8, 0x0e, 0xb5, 0xb2, 0xe5, 0xa1, 0xdd, 0x38, 0x7c, 0x00, 0x24, 0xe9, 0xc6,
- 0x3a, 0xa3, 0x90, 0x6e, 0xec, 0xf0, 0x28, 0x3d, 0xde, 0x1c, 0x6f, 0x0f, 0xed, 0xc6, 0x51, 0xe1,
- 0x7c, 0xa4, 0x1b, 0x3b, 0x41, 0x3a, 0xe9, 0xc6, 0xcc, 0x40, 0xf4, 0xc9, 0x7d, 0x67, 0x68, 0x37,
- 0x1e, 0xba, 0xb5, 0x9c, 0x74, 0x63, 0x65, 0x20, 0x11, 0xad, 0x01, 0x30, 0x07, 0x90, 0x7a, 0x01,
- 0x57, 0x87, 0x4e, 0x06, 0xfd, 0xd1, 0x7a, 0x64, 0x32, 0x68, 0xbb, 0x34, 0x62, 0xc8, 0x28, 0xbc,
- 0x94, 0xe9, 0x47, 0xe7, 0xd2, 0xe2, 0x50, 0x43, 0x36, 0xf0, 0x81, 0x98, 0x18, 0xb2, 0x23, 0x8f,
- 0x48, 0x66, 0x15, 0xb6, 0x4a, 0x5e, 0xba, 0x36, 0xdc, 0x2c, 0x07, 0x3f, 0x96, 0x51, 0xb3, 0x4c,
- 0x09, 0x68, 0x05, 0x32, 0xc4, 0xc9, 0x39, 0xa1, 0x66, 0xe8, 0xfa, 0x50, 0x2f, 0xbe, 0x6f, 0x3f,
- 0x50, 0x6d, 0x4a, 0x4a, 0xbf, 0xe6, 0x24, 0xf2, 0x7a, 0xb6, 0x7e, 0x58, 0x12, 0x87, 0xbe, 0x3e,
- 0xb4, 0xd6, 0x4c, 0x5e, 0xcf, 0x38, 0x90, 0x0a, 0x17, 0x59, 0x5b, 0xf1, 0x5d, 0xde, 0x16, 0xdf,
- 0x98, 0x5c, 0xba, 0x41, 0x45, 0x0d, 0x5d, 0x88, 0x8b, 0xdc, 0x7c, 0x5e, 0x9b, 0x92, 0x16, 0x94,
- 0xc1, 0x54, 0x32, 0xe0, 0xf9, 0xd4, 0xc3, 0x96, 0xef, 0x4a, 0x37, 0x87, 0x0e, 0xf8, 0x88, 0x55,
- 0x4f, 0x32, 0xe0, 0x95, 0x00, 0x99, 0x4d, 0x40, 0x9a, 0x6c, 0xdb, 0x2c, 0x44, 0xe1, 0xd6, 0x88,
- 0x09, 0xa8, 0x6f, 0xdd, 0x84, 0x4d, 0x40, 0x5a, 0x83, 0x71, 0x12, 0x41, 0x6a, 0x1b, 0x2b, 0x16,
- 0x37, 0xb3, 0xb7, 0x87, 0x0a, 0x1a, 0x38, 0x6b, 0x91, 0x08, 0x52, 0x3d, 0x22, 0x71, 0x78, 0x2c,
- 0xf7, 0xe8, 0x1e, 0xee, 0x21, 0xdf, 0x19, 0xea, 0xf0, 0x44, 0x9e, 0x2d, 0x44, 0x1c, 0x1e, 0x2b,
- 0x94, 0x80, 0x7e, 0x1e, 0x66, 0x39, 0xe0, 0x2d, 0xdd, 0x1d, 0xe1, 0x02, 0x06, 0xd7, 0x28, 0xc8,
- 0xb8, 0xe6, 0x3c, 0xcc, 0xca, 0x32, 0xa0, 0xcd, 0xaa, 0x77, 0x6f, 0x84, 0x95, 0x1d, 0xc0, 0xfa,
- 0xcc, 0xca, 0xfa, 0x64, 0x62, 0x65, 0x59, 0x3f, 0xe5, 0x73, 0xdd, 0xfd, 0xa1, 0x56, 0x76, 0x70,
- 0xb7, 0x11, 0xb1, 0xb2, 0xaf, 0x7d, 0x2a, 0xa9, 0x99, 0xcd, 0x00, 0x67, 0xe9, 0x1b, 0x43, 0x6b,
- 0x16, 0x46, 0xde, 0xa4, 0x66, 0x9c, 0x87, 0x34, 0x1b, 0xc3, 0x00, 0x4c, 0xd3, 0xef, 0x0e, 0x3f,
- 0x4a, 0xa1, 0x1f, 0x65, 0xd5, 0xdc, 0xa5, 0x5d, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0x7c, 0x9f, 0x37,
- 0xd7, 0xd4, 0x7b, 0xa3, 0x0d, 0x55, 0xd4, 0xfe, 0x78, 0xcf, 0x50, 0x85, 0x12, 0x69, 0x51, 0xd9,
- 0xd6, 0x3e, 0x3a, 0xbe, 0x97, 0x46, 0x9c, 0xfa, 0xd0, 0xb7, 0xc1, 0x92, 0x16, 0xd5, 0x23, 0xfa,
- 0x43, 0xa8, 0xc7, 0x8e, 0x24, 0x29, 0x2d, 0x8f, 0x1e, 0x42, 0xe1, 0x43, 0x51, 0xbc, 0x21, 0xc4,
- 0xc9, 0xde, 0x9c, 0xe9, 0x7a, 0x18, 0xef, 0x8f, 0x9e, 0x33, 0xfb, 0x5d, 0x0b, 0x36, 0x67, 0x72,
- 0x9f, 0xe2, 0x6f, 0x08, 0x70, 0x8d, 0x95, 0x8d, 0xae, 0x81, 0x9e, 0xc8, 0xde, 0x4a, 0x72, 0x60,
- 0x9b, 0xc9, 0x03, 0xfa, 0x82, 0x0f, 0x87, 0x15, 0x77, 0xcc, 0xca, 0x78, 0x6d, 0x4a, 0x7a, 0x47,
- 0x19, 0x95, 0x8f, 0x74, 0xa9, 0x0e, 0x43, 0x50, 0xa5, 0x87, 0x43, 0xbb, 0x54, 0x18, 0xf5, 0x91,
- 0x2e, 0xc5, 0x79, 0x56, 0x67, 0xf9, 0x77, 0x68, 0x6f, 0x5b, 0x6e, 0xa1, 0x58, 0x5c, 0x4f, 0xa5,
- 0x2f, 0x17, 0x4b, 0xeb, 0xa9, 0xf4, 0x95, 0x62, 0x79, 0x3d, 0x95, 0x7e, 0xab, 0xf8, 0xb6, 0xf8,
- 0x0f, 0xcb, 0x30, 0xe7, 0x82, 0x2e, 0x06, 0xa8, 0x1e, 0x06, 0x01, 0xd5, 0xd5, 0x61, 0x80, 0x8a,
- 0xc3, 0x34, 0x8e, 0xa8, 0x1e, 0x06, 0x11, 0xd5, 0xd5, 0x61, 0x88, 0xca, 0xe7, 0x21, 0x90, 0xaa,
- 0x39, 0x0c, 0x52, 0xdd, 0x9b, 0x00, 0x52, 0x79, 0xa2, 0xfa, 0x31, 0xd5, 0xda, 0x20, 0xa6, 0xba,
- 0x39, 0x1a, 0x53, 0x79, 0xa2, 0x02, 0xa0, 0xea, 0x71, 0x1f, 0xa8, 0xba, 0x3e, 0x02, 0x54, 0x79,
- 0xfc, 0x2e, 0xaa, 0xda, 0x88, 0x44, 0x55, 0xb7, 0xc7, 0xa1, 0x2a, 0x4f, 0x4e, 0x08, 0x56, 0x7d,
- 0x10, 0x82, 0x55, 0x8b, 0x43, 0x61, 0x95, 0xc7, 0xcd, 0x70, 0xd5, 0x27, 0xfd, 0xb8, 0xea, 0xfa,
- 0x08, 0x5c, 0xe5, 0xd7, 0x80, 0x03, 0xab, 0x5a, 0x14, 0xb0, 0xba, 0x35, 0x06, 0x58, 0x79, 0x52,
- 0x82, 0xc8, 0xaa, 0x16, 0x85, 0xac, 0x6e, 0x8d, 0x41, 0x56, 0x7d, 0x92, 0x18, 0xb4, 0xda, 0x8a,
- 0x86, 0x56, 0x77, 0xc6, 0x42, 0x2b, 0x4f, 0x5a, 0x18, 0x5b, 0x2d, 0x07, 0xb0, 0xd5, 0x3b, 0x43,
- 0xb0, 0x95, 0xc7, 0x4a, 0xc0, 0xd5, 0xb7, 0x06, 0xc0, 0x95, 0x38, 0x0a, 0x5c, 0x79, 0xbc, 0x1e,
- 0xba, 0x7a, 0x31, 0x04, 0x5d, 0xdd, 0x1d, 0x8f, 0xae, 0x3c, 0x61, 0x7d, 0xf0, 0x4a, 0x19, 0x09,
- 0xaf, 0xde, 0x9b, 0x10, 0x5e, 0x79, 0xd2, 0xa3, 0xf0, 0xd5, 0x47, 0x61, 0x7c, 0x75, 0x6d, 0x38,
- 0xbe, 0xf2, 0xc4, 0x70, 0x80, 0xb5, 0x11, 0x09, 0xb0, 0x6e, 0x8f, 0x03, 0x58, 0xfe, 0x38, 0x08,
- 0x22, 0xac, 0xad, 0x68, 0x84, 0x75, 0x67, 0x2c, 0xc2, 0xf2, 0x9b, 0x3f, 0x04, 0xb1, 0x36, 0x22,
- 0x21, 0xd6, 0xed, 0x71, 0x10, 0xcb, 0x2f, 0x5c, 0x10, 0x63, 0xbd, 0x1a, 0x8a, 0xb1, 0xee, 0x4f,
- 0x82, 0xb1, 0x3c, 0xa1, 0x03, 0x20, 0xeb, 0xf3, 0xe1, 0x20, 0xeb, 0x1b, 0xe7, 0x38, 0xc4, 0x32,
- 0x12, 0x65, 0x7d, 0x6b, 0x00, 0x65, 0x89, 0xa3, 0x50, 0x96, 0xdf, 0x9f, 0x5d, 0x98, 0xa5, 0x8c,
- 0x04, 0x45, 0xef, 0x4d, 0x08, 0x8a, 0xfc, 0xce, 0x17, 0x81, 0x8a, 0xaa, 0x11, 0xa8, 0xe8, 0xe6,
- 0x68, 0x54, 0xe4, 0x9b, 0x73, 0x1f, 0x16, 0xd5, 0xa2, 0x60, 0xd1, 0xad, 0x31, 0xb0, 0xc8, 0xb7,
- 0x42, 0x01, 0x5c, 0xf4, 0xb8, 0x0f, 0x17, 0x5d, 0x1f, 0x1b, 0x66, 0x15, 0x00, 0x46, 0xab, 0x83,
- 0xc0, 0xe8, 0xc6, 0x48, 0x60, 0xe4, 0x49, 0xf0, 0x91, 0xd1, 0xe3, 0x3e, 0x64, 0x74, 0x7d, 0x04,
- 0x32, 0xf2, 0x0b, 0xc0, 0xa1, 0x91, 0x36, 0x1a, 0x1a, 0x2d, 0x4d, 0x0a, 0x8d, 0x3c, 0xc1, 0x91,
- 0xd8, 0x68, 0x2b, 0x1a, 0x1b, 0xdd, 0x99, 0x30, 0x02, 0x62, 0x00, 0x1c, 0xd5, 0xa2, 0xc0, 0xd1,
- 0xad, 0x31, 0xe0, 0x28, 0x38, 0x87, 0x78, 0xe8, 0xa8, 0x16, 0x85, 0x8e, 0x6e, 0x8d, 0x41, 0x47,
- 0xbe, 0xa4, 0x00, 0x3c, 0x6a, 0x0e, 0x83, 0x47, 0xf7, 0x26, 0x80, 0x47, 0xbe, 0xf3, 0xd2, 0x87,
- 0x8f, 0x3e, 0xed, 0xc7, 0x47, 0xe2, 0x28, 0x7c, 0xe4, 0x8f, 0x48, 0x17, 0x20, 0x6d, 0x45, 0x03,
- 0xa4, 0x3b, 0x63, 0x01, 0x52, 0xd0, 0x48, 0x06, 0x10, 0xd2, 0x46, 0x24, 0x42, 0xba, 0x3d, 0x0e,
- 0x21, 0xf9, 0x46, 0x32, 0x08, 0x91, 0x3e, 0xed, 0x87, 0x48, 0xe2, 0x28, 0x88, 0xe4, 0x57, 0xce,
- 0xc5, 0x48, 0xb5, 0x28, 0x8c, 0x74, 0x6b, 0x0c, 0x46, 0xf2, 0x1b, 0x2f, 0x00, 0x92, 0x94, 0x91,
- 0x20, 0xe9, 0xbd, 0x09, 0x41, 0x52, 0x9f, 0xe1, 0x0a, 0xa3, 0xa4, 0x5a, 0x14, 0x4a, 0xba, 0x35,
- 0x06, 0x25, 0x05, 0x0a, 0xeb, 0xc3, 0xa4, 0xad, 0x68, 0x98, 0x74, 0x67, 0x2c, 0x4c, 0xea, 0x1b,
- 0x4d, 0x2e, 0x4e, 0xda, 0x88, 0xc4, 0x49, 0xb7, 0xc7, 0xe1, 0xa4, 0xbe, 0x89, 0x8f, 0x3b, 0x07,
- 0xbf, 0x32, 0x39, 0x50, 0xfa, 0xe8, 0xfc, 0x40, 0xc9, 0x7b, 0xe7, 0x18, 0xa4, 0xf4, 0x69, 0x3f,
- 0x52, 0x12, 0x47, 0x21, 0x25, 0xbf, 0x67, 0x9d, 0x0f, 0x2a, 0xad, 0xa7, 0xd2, 0x6f, 0x17, 0xdf,
- 0x11, 0xff, 0x7c, 0x06, 0x66, 0x6a, 0x5e, 0x60, 0x92, 0x5f, 0x4b, 0xe1, 0x4d, 0x4e, 0xc2, 0x42,
- 0x6b, 0x64, 0xc4, 0x53, 0xbb, 0x39, 0xfe, 0xfc, 0xc4, 0xc1, 0x13, 0xfe, 0x38, 0xeb, 0x1b, 0x6c,
- 0x49, 0x47, 0x1f, 0xc0, 0x5c, 0xcf, 0xc6, 0x96, 0xdc, 0xb5, 0x74, 0xd3, 0xd2, 0x1d, 0xb6, 0xed,
- 0x46, 0x58, 0x2d, 0x7e, 0x79, 0xba, 0x98, 0xdb, 0xb1, 0xb1, 0xb5, 0xcd, 0xe9, 0x52, 0xae, 0x17,
- 0x78, 0x72, 0xef, 0x3a, 0x9b, 0x9e, 0xfc, 0xae, 0xb3, 0x17, 0x50, 0xb4, 0xb0, 0xa2, 0x85, 0x3c,
- 0x18, 0x76, 0xea, 0x54, 0x74, 0x9f, 0xa3, 0xdb, 0xe2, 0xdc, 0x9c, 0xf4, 0xf4, 0xa9, 0x82, 0x15,
- 0x26, 0xa2, 0x07, 0x70, 0xb1, 0xa3, 0x1c, 0xd3, 0x20, 0x56, 0xd9, 0x75, 0x0a, 0x69, 0x60, 0x2a,
- 0xbb, 0xa0, 0x0c, 0x75, 0x94, 0x63, 0x7a, 0x71, 0x1a, 0x4b, 0xa2, 0xb7, 0x9e, 0xdc, 0x82, 0xbc,
- 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0x3f, 0xd0, 0x98, 0x1d, 0x06, 0x3c, 0xe7, 0x52, 0xd9, 0xa9,
- 0xc5, 0x15, 0x28, 0xb4, 0x14, 0x07, 0x1f, 0x29, 0x27, 0xb2, 0xbb, 0x1f, 0x2e, 0x4b, 0xcf, 0xee,
- 0x7c, 0xeb, 0xec, 0x74, 0x71, 0xee, 0x29, 0x4b, 0x1a, 0xd8, 0x16, 0x37, 0xd7, 0x0a, 0x24, 0x68,
- 0xe8, 0x0e, 0x14, 0x14, 0xfb, 0xc4, 0x50, 0x69, 0x95, 0xb1, 0x61, 0xf7, 0x6c, 0x0a, 0x33, 0xd2,
- 0x52, 0x9e, 0x92, 0x2b, 0x2e, 0x15, 0x5d, 0x87, 0x1c, 0x0f, 0xe7, 0x67, 0xb7, 0x2b, 0x15, 0x68,
- 0xf1, 0xf9, 0xb5, 0x1d, 0xec, 0x82, 0xa5, 0xc7, 0x50, 0xe6, 0x97, 0x10, 0x1c, 0x29, 0x96, 0x26,
- 0x53, 0x4d, 0xfa, 0x7d, 0xae, 0x48, 0xc5, 0x5e, 0x66, 0x97, 0x0e, 0x90, 0x0c, 0x44, 0x7d, 0xfe,
- 0xb8, 0x68, 0xc2, 0xbc, 0xda, 0xd6, 0x3d, 0x54, 0xc0, 0x9c, 0xa8, 0xf9, 0xa1, 0x23, 0xa4, 0x42,
- 0xf3, 0xf6, 0x7f, 0x66, 0x2e, 0xa8, 0x61, 0x32, 0x6a, 0x00, 0x3d, 0xb9, 0x47, 0xee, 0x9a, 0x6d,
- 0x5d, 0x3d, 0xa1, 0xb0, 0x20, 0x7c, 0xa4, 0xfa, 0xc8, 0x8b, 0x0d, 0x5e, 0x29, 0xba, 0xb3, 0x4d,
- 0x39, 0x25, 0x38, 0xf2, 0x7e, 0xb3, 0xe3, 0x90, 0xd7, 0x53, 0x69, 0x28, 0x66, 0xd7, 0x53, 0xe9,
- 0x5c, 0x71, 0x6e, 0x3d, 0x95, 0xce, 0x17, 0x0b, 0xe2, 0x6f, 0x08, 0x90, 0x0b, 0xed, 0x63, 0x7a,
- 0xdc, 0xf7, 0x11, 0xf9, 0x4a, 0x34, 0xfe, 0x1b, 0x16, 0x49, 0x98, 0xe6, 0xfd, 0xc5, 0x0d, 0xcb,
- 0x5c, 0x1c, 0x8e, 0x1f, 0xe8, 0x6a, 0x88, 0x1b, 0xa3, 0xe2, 0xb2, 0x7d, 0x9c, 0xfa, 0xc1, 0x0f,
- 0x17, 0xa7, 0xc4, 0xbf, 0x4c, 0xc1, 0x5c, 0x78, 0xd7, 0x52, 0xbd, 0xaf, 0x5c, 0x51, 0xf6, 0x39,
- 0xc4, 0xb1, 0x34, 0xe2, 0xe6, 0x96, 0x8c, 0x7f, 0xf9, 0x01, 0x2b, 0xe6, 0xb5, 0x11, 0x9f, 0xca,
- 0x83, 0xe5, 0xf4, 0x19, 0xcb, 0xff, 0x21, 0xe9, 0xd9, 0xa9, 0x25, 0x98, 0xa6, 0x07, 0x10, 0xf1,
- 0xa2, 0x95, 0xfa, 0xdb, 0x8a, 0xf8, 0xac, 0x24, 0x5d, 0x62, 0xd9, 0x88, 0x5d, 0x6b, 0xbe, 0xd1,
- 0x09, 0x7f, 0x7e, 0xd7, 0x3b, 0xff, 0x9d, 0x88, 0x3d, 0x76, 0xc2, 0xe3, 0xff, 0xc5, 0x38, 0x16,
- 0xf2, 0x3e, 0xf4, 0x4b, 0x50, 0x50, 0xcd, 0x76, 0x9b, 0xcd, 0x59, 0xcc, 0x32, 0x0c, 0x9e, 0xf9,
- 0x42, 0x8b, 0xc0, 0xaf, 0xc1, 0x5c, 0xf2, 0xae, 0xc3, 0x5c, 0x92, 0xf8, 0x75, 0x98, 0x81, 0x20,
- 0xd9, 0xbc, 0x27, 0xcc, 0x35, 0x28, 0xa1, 0x78, 0xdd, 0xd9, 0x37, 0x89, 0xd7, 0x65, 0x31, 0xde,
- 0xbc, 0xe7, 0xfd, 0xb1, 0xc0, 0x03, 0x62, 0x9e, 0x99, 0xe6, 0x41, 0xcf, 0x8b, 0xb3, 0x2d, 0x07,
- 0xcf, 0x6b, 0x4c, 0x7f, 0x79, 0xba, 0x98, 0x92, 0xbc, 0x03, 0x1b, 0xa3, 0x2c, 0x70, 0xe2, 0xab,
- 0x59, 0xe0, 0xeb, 0x90, 0xeb, 0x5a, 0x78, 0x0f, 0x3b, 0xea, 0xbe, 0x6c, 0xf4, 0x3a, 0x7c, 0x3b,
- 0x4e, 0xd6, 0xa5, 0x6d, 0xf5, 0x3a, 0xe8, 0x1e, 0x14, 0xbd, 0x2c, 0x1c, 0x2b, 0xbb, 0x07, 0x7a,
- 0xb9, 0x74, 0x8e, 0xac, 0xc5, 0xff, 0x29, 0xc0, 0x42, 0xa8, 0x4e, 0x7c, 0x4c, 0xad, 0x43, 0x56,
- 0xf3, 0xe6, 0x3c, 0xbb, 0x24, 0x9c, 0x33, 0xea, 0x34, 0xc8, 0x8c, 0x64, 0xb8, 0xe4, 0xbe, 0x96,
- 0x5e, 0x18, 0xe0, 0x8b, 0x4d, 0x9c, 0x53, 0xec, 0x45, 0x5f, 0xce, 0x5a, 0xe0, 0x05, 0xde, 0x20,
- 0x4b, 0x4e, 0x34, 0xc8, 0xc4, 0xdf, 0x16, 0xa0, 0x48, 0x5f, 0xf0, 0x04, 0x63, 0x2d, 0x16, 0xeb,
- 0xe6, 0x46, 0x73, 0x27, 0x26, 0xdf, 0x70, 0x13, 0xba, 0xe4, 0x24, 0x19, 0xbe, 0xe4, 0x44, 0xfc,
- 0xa1, 0x00, 0x79, 0xaf, 0x84, 0xec, 0x62, 0xc1, 0x11, 0xc7, 0x82, 0xbe, 0xd9, 0xe5, 0x79, 0xee,
- 0xf1, 0x25, 0x13, 0xdd, 0x75, 0x18, 0x3c, 0xbe, 0x84, 0x5d, 0xfa, 0xf6, 0xb7, 0xdd, 0x9e, 0x43,
- 0x8a, 0x58, 0xf1, 0x8f, 0xa9, 0x78, 0x83, 0xbd, 0x47, 0x12, 0xbd, 0x93, 0xd5, 0x6c, 0x1f, 0xb2,
- 0x13, 0x67, 0x26, 0x32, 0x7b, 0x88, 0x87, 0x81, 0x01, 0x5f, 0x55, 0xd3, 0x9a, 0x0d, 0x7a, 0x5b,
- 0x2b, 0xfb, 0x6d, 0x8b, 0x4f, 0x02, 0x0a, 0xa4, 0x8d, 0x4f, 0xb4, 0x34, 0x91, 0x29, 0x76, 0xb5,
- 0xc4, 0xfa, 0xca, 0x1f, 0x06, 0x5b, 0xa2, 0x7a, 0x48, 0xd0, 0xd4, 0x23, 0x48, 0x1e, 0x2a, 0xed,
- 0x51, 0x91, 0x54, 0xa1, 0x96, 0x93, 0x48, 0x6e, 0xf4, 0x24, 0x74, 0xba, 0x47, 0x62, 0xb8, 0xe7,
- 0x3f, 0xa8, 0xd2, 0xd0, 0x29, 0x20, 0x1f, 0x86, 0xfb, 0xfa, 0xc8, 0xd7, 0x07, 0x3b, 0xfd, 0xc7,
- 0xa9, 0x1f, 0xff, 0x70, 0x51, 0x10, 0x3f, 0x01, 0x24, 0x61, 0x1b, 0x3b, 0x2f, 0x7a, 0xa6, 0xe5,
- 0x9f, 0x94, 0x72, 0xbb, 0xef, 0xd6, 0x97, 0xe9, 0xd5, 0xec, 0x59, 0xd4, 0x61, 0x4a, 0x17, 0x61,
- 0x21, 0xc4, 0xcd, 0x8c, 0x85, 0xf8, 0x21, 0x5c, 0x79, 0x6a, 0xda, 0xb6, 0xde, 0x25, 0x10, 0x92,
- 0x8e, 0x4a, 0x32, 0xb5, 0x78, 0xe6, 0x31, 0xdd, 0xa5, 0x8b, 0x09, 0x06, 0x33, 0x23, 0x19, 0xc9,
- 0x7b, 0x16, 0x7f, 0x4f, 0x80, 0xcb, 0x83, 0x9c, 0x4c, 0xcb, 0x51, 0x5b, 0x25, 0x67, 0x55, 0xd3,
- 0x3f, 0x59, 0x6f, 0x7c, 0x6f, 0x75, 0xb3, 0x13, 0xb7, 0x90, 0xbf, 0x53, 0xee, 0x28, 0xd4, 0x7c,
- 0xf0, 0x6d, 0xdb, 0x79, 0x4e, 0xde, 0x64, 0x54, 0xdf, 0x92, 0xa4, 0x26, 0xb3, 0x24, 0x4d, 0x28,
- 0xac, 0x9b, 0xba, 0x41, 0xbc, 0x4f, 0xb7, 0xbe, 0x2b, 0x90, 0xdf, 0xd5, 0x0d, 0xc5, 0x3a, 0x91,
- 0xdd, 0x00, 0x3e, 0x61, 0x5c, 0x00, 0x9f, 0x34, 0xc7, 0x38, 0xf8, 0xa3, 0xf8, 0x13, 0x01, 0x8a,
- 0xbe, 0x58, 0x6e, 0x91, 0xdf, 0x05, 0x50, 0xdb, 0x3d, 0xdb, 0xc1, 0x96, 0xdb, 0x4a, 0x39, 0x16,
- 0xf6, 0x5e, 0x61, 0xd4, 0xfa, 0x9a, 0x94, 0xe1, 0x19, 0xea, 0x1a, 0xba, 0x11, 0x3e, 0x55, 0x62,
- 0x7a, 0x15, 0xce, 0x06, 0xce, 0x92, 0x20, 0xcd, 0x6e, 0x3b, 0xa6, 0xe5, 0x61, 0x17, 0xde, 0xec,
- 0xee, 0x39, 0xf9, 0x74, 0x33, 0x34, 0xc9, 0xb7, 0x02, 0x79, 0xe2, 0x2e, 0x1c, 0x62, 0xaf, 0x4a,
- 0xa9, 0xf1, 0x55, 0x62, 0x1c, 0x6e, 0x95, 0xfe, 0xa5, 0x00, 0x85, 0x0a, 0x6b, 0x0d, 0xaf, 0x85,
- 0x47, 0x58, 0xb4, 0x35, 0x48, 0x3b, 0xc7, 0x86, 0xdc, 0xc1, 0xde, 0xdd, 0x32, 0xe7, 0x38, 0xf6,
- 0x6e, 0xd6, 0x61, 0x8f, 0xf4, 0xfa, 0x41, 0x7e, 0xf7, 0x35, 0x1f, 0x2e, 0x57, 0x96, 0xd8, 0xe5,
- 0xd8, 0x4b, 0xee, 0xe5, 0xd8, 0x4b, 0x6b, 0x3c, 0x03, 0x33, 0xea, 0x3f, 0xf8, 0x2f, 0x8b, 0x82,
- 0xe4, 0x31, 0xb1, 0x79, 0xff, 0x7e, 0x83, 0xf4, 0xfa, 0x81, 0x99, 0x19, 0xe5, 0x01, 0x02, 0x97,
- 0x06, 0xf1, 0xeb, 0x96, 0x57, 0xd6, 0xe4, 0x9d, 0xad, 0xca, 0xf3, 0xcd, 0xcd, 0x7a, 0xb3, 0x59,
- 0x5d, 0x2b, 0x0a, 0xa8, 0x08, 0xb9, 0xd0, 0x95, 0x43, 0x09, 0x76, 0x01, 0xf3, 0xfd, 0xff, 0x0f,
- 0xc0, 0xbf, 0xbd, 0x8c, 0xc8, 0xda, 0xa8, 0x7e, 0x26, 0xbf, 0x5c, 0x79, 0xb6, 0x53, 0x6d, 0x14,
- 0xa7, 0x10, 0x82, 0xfc, 0xea, 0x4a, 0xb3, 0x52, 0x93, 0xa5, 0x6a, 0x63, 0xfb, 0xf9, 0x56, 0xa3,
- 0xea, 0x5e, 0xdc, 0x7c, 0x7f, 0x0d, 0x72, 0xc1, 0xb3, 0x83, 0xd0, 0x02, 0x14, 0x2a, 0xb5, 0x6a,
- 0x65, 0x43, 0x7e, 0x59, 0x5f, 0x91, 0x5f, 0xec, 0x54, 0x77, 0xaa, 0xc5, 0x29, 0x5a, 0x34, 0x4a,
- 0x7c, 0xb2, 0xf3, 0xec, 0x59, 0x51, 0x40, 0x05, 0xc8, 0xb2, 0x67, 0x7a, 0x3d, 0x51, 0x31, 0x71,
- 0x7f, 0x13, 0xb2, 0x81, 0x43, 0x86, 0xc9, 0xeb, 0xb6, 0x77, 0x1a, 0x35, 0xb9, 0x59, 0xdf, 0xac,
- 0x36, 0x9a, 0x2b, 0x9b, 0xdb, 0x4c, 0x06, 0xa5, 0xad, 0xac, 0x3e, 0x97, 0x9a, 0x45, 0xc1, 0x7b,
- 0x6e, 0x3e, 0xdf, 0xa9, 0xd4, 0xdc, 0x6a, 0x88, 0xa9, 0x74, 0xb2, 0x98, 0xbc, 0xff, 0xd7, 0x05,
- 0xb8, 0x3c, 0xe4, 0x1c, 0x1d, 0x94, 0x85, 0xd9, 0x1d, 0x83, 0x9e, 0xb8, 0x5a, 0x9c, 0x42, 0x73,
- 0x81, 0xa3, 0x74, 0x8a, 0x02, 0x4a, 0xb3, 0xc3, 0x4c, 0x8a, 0x09, 0x34, 0x03, 0x89, 0xc6, 0xa3,
- 0x62, 0x92, 0x94, 0x34, 0x70, 0x12, 0x4d, 0x31, 0x85, 0x32, 0xfc, 0x0c, 0x8c, 0xe2, 0x34, 0xca,
- 0xf9, 0x47, 0x51, 0x14, 0x67, 0x88, 0x28, 0xef, 0x48, 0x87, 0xe2, 0xec, 0xfd, 0xeb, 0x10, 0xd8,
- 0x36, 0x8f, 0x00, 0x66, 0x9e, 0x29, 0x0e, 0xb6, 0x9d, 0xe2, 0x14, 0x9a, 0x85, 0xe4, 0x4a, 0xbb,
- 0x5d, 0x14, 0x1e, 0xfe, 0x8b, 0x14, 0xa4, 0xdd, 0x6b, 0x78, 0xd0, 0x33, 0x98, 0x66, 0xcb, 0xca,
- 0x8b, 0xc3, 0xd1, 0x02, 0x1d, 0xd0, 0xe5, 0x6b, 0xe3, 0xe0, 0x84, 0x38, 0x85, 0xfe, 0x7f, 0xc8,
- 0x06, 0xbc, 0x28, 0x34, 0x74, 0x69, 0x2c, 0xe4, 0x39, 0x96, 0x6f, 0x8f, 0xcb, 0xe6, 0xc9, 0x7f,
- 0x05, 0x19, 0xcf, 0xaa, 0xa3, 0x1b, 0xa3, 0x6c, 0xbe, 0x2b, 0x7b, 0xf4, 0xc4, 0x40, 0xc6, 0x9f,
- 0x38, 0xf5, 0xbe, 0x80, 0x2c, 0x40, 0x83, 0x06, 0x18, 0x45, 0x05, 0x2b, 0x0c, 0xb5, 0xf0, 0xe5,
- 0xfb, 0x13, 0xe5, 0xf6, 0xdf, 0x49, 0x94, 0xe5, 0xcf, 0x22, 0xd1, 0xca, 0x1a, 0x98, 0xa3, 0xa2,
- 0x95, 0x15, 0x31, 0x19, 0x4d, 0xa1, 0x17, 0x90, 0x22, 0xd6, 0x13, 0x45, 0xf9, 0x95, 0x7d, 0xd6,
- 0xba, 0x7c, 0x63, 0x64, 0x1e, 0x57, 0xe4, 0xea, 0xbd, 0x1f, 0xff, 0xd9, 0xd5, 0xa9, 0x1f, 0x9f,
- 0x5d, 0x15, 0x7e, 0x72, 0x76, 0x55, 0xf8, 0x93, 0xb3, 0xab, 0xc2, 0x9f, 0x9e, 0x5d, 0x15, 0xbe,
- 0xff, 0xd3, 0xab, 0x53, 0x3f, 0xf9, 0xe9, 0xd5, 0xa9, 0x3f, 0xf9, 0xe9, 0xd5, 0xa9, 0xcf, 0x67,
- 0x39, 0xf7, 0xee, 0x0c, 0x35, 0x2d, 0x8f, 0xfe, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x7c, 0xb1,
- 0x3d, 0x40, 0xb2, 0x80, 0x00, 0x00,
+ 0xff, 0xeb, 0x02, 0xcc, 0x07, 0x04, 0xc7, 0x19, 0xe0, 0xbc, 0x91, 0x9e, 0xc4, 0x5f, 0x24, 0x21,
+ 0x4f, 0x70, 0x94, 0xc4, 0x39, 0x06, 0xff, 0x69, 0x02, 0x2e, 0x55, 0xd8, 0x6b, 0x6e, 0x37, 0xef,
+ 0x23, 0x4e, 0xcb, 0x28, 0xc1, 0xec, 0x21, 0xb6, 0x6c, 0xdd, 0x64, 0xd3, 0xef, 0x9c, 0xe4, 0x5e,
+ 0xa2, 0x45, 0x48, 0xdb, 0x86, 0xd2, 0xb5, 0xf7, 0x4d, 0xf7, 0xfd, 0x9e, 0x77, 0xed, 0xe5, 0xa8,
+ 0x4c, 0xbf, 0x79, 0x8e, 0xca, 0xcc, 0xe8, 0x1c, 0x95, 0xd9, 0xaf, 0x90, 0xa3, 0xc2, 0x5f, 0xa6,
+ 0xfd, 0x5b, 0x01, 0x2e, 0x0f, 0x68, 0x2e, 0x4e, 0x6b, 0xf9, 0x2e, 0x64, 0x55, 0x2e, 0x98, 0x38,
+ 0x6c, 0xf6, 0xa6, 0xb0, 0x4e, 0x8a, 0xbd, 0x21, 0x6e, 0x39, 0x3b, 0x5d, 0x02, 0xb7, 0xaa, 0xf5,
+ 0x35, 0xae, 0x1c, 0xf2, 0x5b, 0x13, 0x7f, 0x65, 0x0e, 0x0a, 0xd5, 0x63, 0xb6, 0x4a, 0xde, 0x60,
+ 0x61, 0x02, 0x7a, 0x02, 0xe9, 0xae, 0x65, 0x1e, 0xea, 0x6e, 0x33, 0xf2, 0xa1, 0x04, 0x05, 0xb7,
+ 0x19, 0x7d, 0x5c, 0xdb, 0x9c, 0x43, 0xf2, 0x78, 0x51, 0x13, 0x32, 0xcf, 0x4c, 0x55, 0x69, 0x3f,
+ 0xd1, 0xdb, 0xae, 0xe5, 0xbf, 0x3f, 0x5e, 0x50, 0xd9, 0xe3, 0xd9, 0x56, 0x9c, 0x7d, 0xb7, 0x13,
+ 0x3c, 0x22, 0xaa, 0x43, 0xba, 0xe6, 0x38, 0x5d, 0x72, 0x93, 0xfb, 0x8e, 0x3b, 0x13, 0x08, 0x25,
+ 0x2c, 0x6e, 0x96, 0xac, 0xcb, 0x8e, 0x9a, 0x30, 0xff, 0x94, 0xee, 0xf9, 0xaa, 0xb4, 0xcd, 0x9e,
+ 0x56, 0x31, 0x8d, 0x3d, 0xbd, 0xc5, 0xfd, 0xf6, 0xed, 0x09, 0x64, 0x3e, 0xad, 0x34, 0xa4, 0x41,
+ 0x01, 0x68, 0x05, 0xd2, 0x8d, 0x47, 0x5c, 0x18, 0x8b, 0xeb, 0x6e, 0x4d, 0x20, 0xac, 0xf1, 0x48,
+ 0xf2, 0xd8, 0xd0, 0x3a, 0x64, 0x57, 0xbe, 0xe8, 0x59, 0x98, 0x4b, 0x99, 0x19, 0x9a, 0x1d, 0xd1,
+ 0x2f, 0x85, 0x72, 0x49, 0x41, 0x66, 0xd4, 0x80, 0xfc, 0x2b, 0xd3, 0x3a, 0x68, 0x9b, 0x8a, 0xdb,
+ 0xc2, 0x59, 0x2a, 0xee, 0x1b, 0x13, 0x88, 0x73, 0x19, 0xa5, 0x3e, 0x11, 0xe8, 0xdb, 0x50, 0x20,
+ 0x9d, 0xd1, 0x54, 0x76, 0xdb, 0x6e, 0x25, 0xd3, 0x54, 0xea, 0xbb, 0x13, 0x48, 0xf5, 0x38, 0xdd,
+ 0x85, 0xff, 0x3e, 0x51, 0x8b, 0x12, 0xcc, 0x85, 0x8c, 0x00, 0x21, 0x48, 0x75, 0x49, 0x7f, 0x0b,
+ 0x34, 0x7f, 0x89, 0xfe, 0x46, 0xef, 0xc1, 0xac, 0x61, 0x6a, 0xd8, 0x1d, 0x21, 0x73, 0xab, 0x17,
+ 0xce, 0x4e, 0x97, 0x66, 0xb6, 0x4c, 0x8d, 0x45, 0x34, 0xfc, 0x97, 0x34, 0x43, 0x0a, 0xd5, 0xb5,
+ 0xc5, 0x6b, 0x90, 0x22, 0xfd, 0x4e, 0x1c, 0xd3, 0xae, 0x62, 0xe3, 0x1d, 0x4b, 0xe7, 0xd2, 0xdc,
+ 0xcb, 0xc5, 0x7f, 0x90, 0x80, 0x44, 0xe3, 0x11, 0x89, 0xd9, 0x77, 0x7b, 0xea, 0x01, 0x76, 0xf8,
+ 0x7d, 0x7e, 0x45, 0x63, 0x79, 0x0b, 0xef, 0xe9, 0x2c, 0xb4, 0xca, 0x48, 0xfc, 0x0a, 0xbd, 0x03,
+ 0xa0, 0xa8, 0x2a, 0xb6, 0x6d, 0xd9, 0xdd, 0x0b, 0x98, 0x91, 0x32, 0x8c, 0xb2, 0x81, 0x4f, 0x08,
+ 0x9b, 0x8d, 0x55, 0x0b, 0x3b, 0x6e, 0xf2, 0x15, 0xbb, 0x22, 0x6c, 0x0e, 0xee, 0x74, 0x65, 0xc7,
+ 0x3c, 0xc0, 0x06, 0xb5, 0x93, 0x0c, 0x71, 0x35, 0x9d, 0x6e, 0x93, 0x10, 0x88, 0x97, 0xc4, 0x86,
+ 0xe6, 0xbb, 0xb4, 0x8c, 0xe4, 0x5d, 0x13, 0x91, 0x16, 0x6e, 0xe9, 0x7c, 0xbb, 0x5c, 0x46, 0xe2,
+ 0x57, 0x44, 0x4b, 0x4a, 0xcf, 0xd9, 0xa7, 0x3d, 0x91, 0x91, 0xe8, 0x6f, 0x74, 0x1b, 0x0a, 0x2c,
+ 0x5f, 0x53, 0xc6, 0x86, 0x2a, 0x53, 0xe7, 0x9a, 0xa1, 0xb7, 0xe7, 0x18, 0xb9, 0x6a, 0xa8, 0xc4,
+ 0x95, 0xa2, 0x47, 0xc0, 0x09, 0xf2, 0x41, 0xc7, 0x26, 0x3a, 0x05, 0x52, 0x6a, 0xb5, 0x70, 0x76,
+ 0xba, 0x94, 0x6d, 0xd0, 0x1b, 0x1b, 0x9b, 0x8d, 0xfa, 0x9a, 0x94, 0x65, 0xa5, 0x36, 0x3a, 0x76,
+ 0x5d, 0x5b, 0xfc, 0x0d, 0x01, 0x92, 0x4f, 0x2b, 0x8d, 0x73, 0xab, 0xcc, 0xad, 0x68, 0x32, 0x50,
+ 0xd1, 0x3b, 0x50, 0xd8, 0xd5, 0xdb, 0x6d, 0xdd, 0x68, 0x91, 0x28, 0xea, 0xbb, 0x58, 0x75, 0x15,
+ 0x96, 0xe7, 0xe4, 0x6d, 0x46, 0x45, 0xd7, 0x20, 0xab, 0x5a, 0x58, 0xc3, 0x86, 0xa3, 0x2b, 0x6d,
+ 0x9b, 0x6b, 0x2e, 0x48, 0x5a, 0xfc, 0x65, 0x01, 0xa6, 0xe9, 0x08, 0x40, 0x6f, 0x43, 0x46, 0x35,
+ 0x0d, 0x47, 0xd1, 0x0d, 0xee, 0xca, 0x32, 0x92, 0x4f, 0x18, 0x5a, 0xbd, 0xeb, 0x90, 0x53, 0x54,
+ 0xd5, 0xec, 0x19, 0x8e, 0x6c, 0x28, 0x1d, 0xcc, 0xab, 0x99, 0xe5, 0xb4, 0x2d, 0xa5, 0x83, 0xd1,
+ 0x12, 0xb8, 0x97, 0xde, 0x16, 0xcf, 0x8c, 0x04, 0x9c, 0xb4, 0x81, 0x4f, 0x16, 0xff, 0x40, 0x80,
+ 0xb4, 0x3b, 0x66, 0x48, 0x35, 0x5a, 0xd8, 0x60, 0x49, 0xe5, 0x6e, 0x35, 0x3c, 0x42, 0xff, 0x54,
+ 0x99, 0xf1, 0xa7, 0xca, 0x0b, 0x30, 0xed, 0x90, 0x61, 0xc1, 0x6b, 0xc0, 0x2e, 0xe8, 0x7a, 0x76,
+ 0x5b, 0x69, 0xb1, 0xe5, 0xbc, 0x8c, 0xc4, 0x2e, 0x48, 0x63, 0x78, 0xda, 0x2f, 0xd3, 0x08, 0xbf,
+ 0x22, 0x35, 0x65, 0xc9, 0xa9, 0xbb, 0xb8, 0xa5, 0x1b, 0xd4, 0x96, 0x92, 0x12, 0x50, 0xd2, 0x2a,
+ 0xa1, 0xa0, 0xb7, 0x20, 0xc3, 0x0a, 0x60, 0x43, 0xa3, 0x06, 0x95, 0x94, 0xd2, 0x94, 0x50, 0x35,
+ 0xb4, 0x45, 0x0c, 0x19, 0x6f, 0x70, 0x92, 0x6e, 0xeb, 0xd9, 0x9e, 0x22, 0xe9, 0x6f, 0xf4, 0x3e,
+ 0x5c, 0x78, 0xdd, 0x53, 0xda, 0xfa, 0x1e, 0x5d, 0xa9, 0xa3, 0x59, 0xf7, 0x54, 0x67, 0xac, 0x25,
+ 0xc8, 0xbb, 0x47, 0x25, 0x50, 0xd5, 0xb9, 0x63, 0x39, 0xe9, 0x8f, 0x65, 0xf1, 0x77, 0x05, 0x98,
+ 0x67, 0x99, 0x47, 0x2c, 0x61, 0x36, 0xbe, 0x38, 0xe4, 0x63, 0xc8, 0x68, 0x8a, 0xa3, 0xb0, 0x4d,
+ 0xab, 0x89, 0x91, 0x9b, 0x56, 0xbd, 0x4d, 0x14, 0x8a, 0xa3, 0xd0, 0x8d, 0xab, 0x08, 0x52, 0xe4,
+ 0x37, 0xdb, 0xdf, 0x2b, 0xd1, 0xdf, 0xe2, 0x67, 0x80, 0x82, 0x15, 0x8d, 0x33, 0x22, 0xbb, 0x07,
+ 0x17, 0x89, 0xae, 0xab, 0x86, 0x6a, 0x9d, 0x74, 0x1d, 0xdd, 0x34, 0x9e, 0xd3, 0xbf, 0x36, 0x2a,
+ 0x06, 0x5e, 0x6c, 0xd1, 0xf7, 0x59, 0xe2, 0xef, 0xcd, 0xc0, 0x5c, 0xf5, 0xb8, 0x6b, 0x5a, 0xb1,
+ 0x2e, 0x83, 0xad, 0xc2, 0x2c, 0x5f, 0x29, 0x18, 0xf1, 0xee, 0xba, 0xcf, 0x99, 0xbb, 0x2f, 0xee,
+ 0x39, 0x23, 0x5a, 0x05, 0x60, 0x49, 0xad, 0x34, 0x71, 0x29, 0x79, 0x8e, 0x57, 0x6d, 0x94, 0x8d,
+ 0x6e, 0xe0, 0xd8, 0x82, 0x6c, 0xe7, 0x50, 0x55, 0xe5, 0x3d, 0xbd, 0xed, 0xf0, 0xdc, 0xc0, 0xe8,
+ 0x34, 0xf6, 0xcd, 0x97, 0x95, 0xca, 0x13, 0x5a, 0x88, 0xa5, 0xe9, 0xf9, 0xd7, 0x12, 0x10, 0x09,
+ 0xec, 0x37, 0x7a, 0x17, 0xf8, 0x66, 0x22, 0xd9, 0x76, 0xf7, 0x0d, 0xae, 0xce, 0x9d, 0x9d, 0x2e,
+ 0x65, 0x24, 0x4a, 0x6d, 0x34, 0x9a, 0x52, 0x86, 0x15, 0x68, 0xd8, 0x0e, 0xba, 0x01, 0x73, 0x66,
+ 0x47, 0x77, 0x64, 0x37, 0x48, 0xe2, 0x11, 0x65, 0x8e, 0x10, 0xdd, 0x20, 0xea, 0x3c, 0x7b, 0x4c,
+ 0x66, 0x27, 0xde, 0x63, 0x82, 0xfe, 0xba, 0x00, 0x97, 0xb8, 0x22, 0xe5, 0x5d, 0x9a, 0x87, 0xaf,
+ 0xb4, 0x75, 0xe7, 0x44, 0x3e, 0x38, 0x2c, 0xa5, 0x69, 0xdc, 0xfa, 0x73, 0x91, 0x1d, 0x12, 0xb0,
+ 0x83, 0xb2, 0xdb, 0x2d, 0x27, 0xcf, 0x38, 0xf3, 0xc6, 0x61, 0xd5, 0x70, 0xac, 0x93, 0xd5, 0xcb,
+ 0x67, 0xa7, 0x4b, 0x0b, 0x83, 0x77, 0x5f, 0x4a, 0x0b, 0xf6, 0x20, 0x0b, 0xaa, 0x01, 0x60, 0xcf,
+ 0x0e, 0xe9, 0x8c, 0x11, 0x1d, 0x7f, 0x44, 0x1a, 0xac, 0x14, 0xe0, 0x45, 0x77, 0xa1, 0xc8, 0xf7,
+ 0xf4, 0xec, 0xe9, 0x6d, 0x2c, 0xdb, 0xfa, 0x17, 0x98, 0xce, 0x2d, 0x49, 0x29, 0xcf, 0xe8, 0x44,
+ 0x44, 0x43, 0xff, 0x02, 0x2f, 0x7e, 0x17, 0x4a, 0xc3, 0x6a, 0x1f, 0x1c, 0x02, 0x19, 0xf6, 0x4a,
+ 0xf7, 0xa3, 0xf0, 0x7a, 0xce, 0x04, 0xa6, 0xca, 0xd7, 0x74, 0x3e, 0x4e, 0x7c, 0x24, 0x88, 0x7f,
+ 0x2f, 0x01, 0x73, 0xab, 0xbd, 0xf6, 0xc1, 0xf3, 0x6e, 0x83, 0x1d, 0x6e, 0x40, 0xdc, 0x20, 0x73,
+ 0x14, 0xa4, 0x82, 0x02, 0x73, 0x83, 0xd4, 0x13, 0xe8, 0x5f, 0x60, 0x32, 0x39, 0x05, 0xd2, 0x65,
+ 0xf8, 0x3e, 0x03, 0xda, 0x06, 0x9f, 0x4c, 0xb7, 0x02, 0x7c, 0x04, 0xa5, 0x40, 0x41, 0xba, 0xf8,
+ 0x22, 0x63, 0xc3, 0xb1, 0x74, 0xcc, 0x16, 0x10, 0x93, 0x52, 0x20, 0xa7, 0xa7, 0x4e, 0x6e, 0x57,
+ 0xd9, 0x5d, 0xd4, 0x84, 0x1c, 0x29, 0x78, 0x22, 0xd3, 0x29, 0xc4, 0x5d, 0xe0, 0x7d, 0x10, 0xd1,
+ 0xac, 0x50, 0xbd, 0xcb, 0x54, 0x3f, 0x15, 0xca, 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0x16, 0x3f,
+ 0x85, 0x62, 0x7f, 0x81, 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0x17, 0x82, 0xba, 0x4c, 0x06, 0xf4, 0xb4,
+ 0x9e, 0x4a, 0xa7, 0x8a, 0xd3, 0xe2, 0x9f, 0x25, 0x21, 0xef, 0x9a, 0x59, 0x9c, 0x40, 0x67, 0x15,
+ 0xa6, 0x89, 0x51, 0xb8, 0x19, 0x28, 0xb7, 0x47, 0x58, 0x37, 0xcf, 0x6c, 0x27, 0xc6, 0xe2, 0x82,
+ 0x64, 0xca, 0x1a, 0x87, 0xc3, 0x59, 0xfc, 0xe5, 0x04, 0xa4, 0x28, 0xb6, 0x78, 0x00, 0x29, 0x3a,
+ 0x51, 0x08, 0x93, 0x4c, 0x14, 0xb4, 0xa8, 0x37, 0x9d, 0x25, 0x02, 0xa1, 0x29, 0x89, 0xf9, 0xf6,
+ 0x95, 0x0f, 0x1e, 0x3c, 0xa4, 0xce, 0x26, 0x27, 0xf1, 0x2b, 0xb4, 0x4a, 0x53, 0xa3, 0x4c, 0xcb,
+ 0xc1, 0x1a, 0x8f, 0xe9, 0xaf, 0x8d, 0xeb, 0x5f, 0x77, 0x52, 0x72, 0xf9, 0xd0, 0x15, 0x48, 0x12,
+ 0x2f, 0x36, 0xcb, 0xb2, 0x1c, 0xce, 0x4e, 0x97, 0x92, 0xc4, 0x7f, 0x11, 0x1a, 0x5a, 0x86, 0x6c,
+ 0xd8, 0x65, 0x90, 0x08, 0x8e, 0x3a, 0xc6, 0xc0, 0x70, 0x87, 0xb6, 0x37, 0xb4, 0x18, 0x9e, 0xe5,
+ 0x7d, 0xfc, 0xdf, 0x53, 0x30, 0x57, 0xef, 0xc4, 0x3d, 0xa5, 0xac, 0x84, 0x7b, 0x38, 0x0a, 0x08,
+ 0x85, 0x1e, 0x1a, 0xd1, 0xc1, 0xa1, 0x19, 0x3c, 0x79, 0xbe, 0x19, 0xbc, 0x4e, 0x22, 0x65, 0x7e,
+ 0xf0, 0x44, 0x72, 0x08, 0xe6, 0x09, 0x3f, 0x9f, 0xc6, 0x29, 0x12, 0xe1, 0xf1, 0xf7, 0x7a, 0xd0,
+ 0x4c, 0x95, 0x4f, 0x69, 0x40, 0xce, 0xac, 0x6c, 0x66, 0x72, 0x2b, 0x9b, 0xc5, 0x86, 0x46, 0x27,
+ 0xb5, 0xb0, 0x47, 0x9d, 0x7d, 0x73, 0x8f, 0xba, 0xe8, 0x70, 0x63, 0xfd, 0x18, 0x92, 0x9a, 0xee,
+ 0x76, 0xce, 0xe4, 0x53, 0x35, 0x61, 0x1a, 0x63, 0xb5, 0xa9, 0xa0, 0xd5, 0x32, 0x2b, 0x59, 0xac,
+ 0x03, 0xf8, 0xba, 0x41, 0xd7, 0x60, 0xc6, 0x6c, 0x6b, 0xee, 0x66, 0x97, 0xb9, 0xd5, 0xcc, 0xd9,
+ 0xe9, 0xd2, 0xf4, 0xf3, 0xb6, 0x56, 0x5f, 0x93, 0xa6, 0xcd, 0xb6, 0x56, 0xd7, 0xe8, 0xa9, 0x1f,
+ 0xf8, 0x48, 0xf6, 0x32, 0xe1, 0x72, 0xd2, 0xac, 0x81, 0x8f, 0xd6, 0xb0, 0xad, 0x72, 0x83, 0xfb,
+ 0x6d, 0x01, 0xf2, 0xae, 0xee, 0xe3, 0x75, 0x2a, 0x69, 0xbd, 0xc3, 0x07, 0x59, 0xf2, 0x7c, 0x83,
+ 0xcc, 0xe5, 0xe3, 0x1b, 0x8b, 0x7f, 0x55, 0xe0, 0xb9, 0xcd, 0x0d, 0x55, 0x71, 0x48, 0x50, 0x11,
+ 0xe3, 0xc0, 0xb8, 0x07, 0x45, 0x4b, 0x31, 0x34, 0xb3, 0xa3, 0x7f, 0x81, 0xd9, 0x42, 0xa8, 0xcd,
+ 0x5f, 0x7b, 0x16, 0x3c, 0x3a, 0x5d, 0xf5, 0xb3, 0xc5, 0x3f, 0x48, 0xf0, 0x3c, 0x68, 0xaf, 0x1a,
+ 0x71, 0xaa, 0xeb, 0x3b, 0x30, 0x1f, 0x9c, 0xda, 0x58, 0xaa, 0x25, 0x1b, 0xad, 0xef, 0x45, 0xc8,
+ 0x8b, 0xaa, 0x08, 0xcb, 0x67, 0x74, 0x93, 0xeb, 0x03, 0x33, 0x22, 0x15, 0x86, 0x2a, 0x90, 0xe5,
+ 0x2f, 0x1f, 0x8c, 0x3d, 0xd3, 0x4d, 0x14, 0x7b, 0x7b, 0x58, 0x5a, 0x64, 0xdd, 0xd8, 0x33, 0xdd,
+ 0x17, 0xf9, 0x96, 0x4b, 0xb0, 0x17, 0x7f, 0x01, 0xa6, 0xe9, 0xed, 0x37, 0x70, 0xd1, 0xbc, 0x37,
+ 0xff, 0x34, 0x01, 0x37, 0x69, 0xed, 0x5f, 0x62, 0x4b, 0xdf, 0x3b, 0xd9, 0xb6, 0x4c, 0x07, 0xab,
+ 0x0e, 0xd6, 0xfc, 0x95, 0xf3, 0x58, 0xfd, 0x5e, 0xa6, 0xeb, 0x3e, 0xe0, 0x5c, 0x09, 0x67, 0x1e,
+ 0x17, 0xda, 0x80, 0x02, 0x4f, 0x2d, 0x50, 0xda, 0xfa, 0x21, 0x96, 0x15, 0xe7, 0x3c, 0xb3, 0xdb,
+ 0x1c, 0xe3, 0x5d, 0x21, 0xac, 0x2b, 0x0e, 0xd2, 0x20, 0xc3, 0x85, 0xe9, 0x1a, 0x3f, 0x90, 0xe8,
+ 0xe9, 0x57, 0x5b, 0x50, 0x4c, 0xb3, 0xfc, 0x86, 0xfa, 0x9a, 0x94, 0x66, 0x92, 0xeb, 0x9a, 0xf8,
+ 0x1f, 0x04, 0xb8, 0x35, 0x46, 0xc5, 0x71, 0x9a, 0xee, 0x22, 0xa4, 0x0f, 0xc9, 0x83, 0x74, 0xae,
+ 0xe3, 0xb4, 0xe4, 0x5d, 0xa3, 0x4d, 0x98, 0xdb, 0x53, 0xf4, 0xb6, 0x6f, 0xd2, 0xc3, 0xf3, 0x13,
+ 0xa3, 0xb3, 0x71, 0x73, 0x8c, 0x9d, 0xd9, 0xb0, 0xf8, 0x9b, 0x09, 0x98, 0x5f, 0xd1, 0xb4, 0x46,
+ 0x83, 0xfb, 0xc0, 0xf8, 0x2c, 0xc5, 0x05, 0x99, 0x09, 0x1f, 0x64, 0xa2, 0xf7, 0x00, 0x69, 0xba,
+ 0xcd, 0x0e, 0x3e, 0xb1, 0xf7, 0x15, 0xcd, 0x3c, 0xf2, 0xf3, 0x32, 0xe6, 0xdd, 0x3b, 0x0d, 0xf7,
+ 0x06, 0x6a, 0x00, 0x45, 0x3b, 0xb2, 0xed, 0x28, 0xde, 0x7b, 0xa7, 0x5b, 0x13, 0x6d, 0x29, 0x63,
+ 0x30, 0xc8, 0xbb, 0x94, 0x32, 0x44, 0x0e, 0xfd, 0x49, 0xe2, 0x76, 0x9d, 0x34, 0xdd, 0x91, 0x15,
+ 0xdb, 0xdd, 0x3f, 0xc4, 0x8e, 0x5c, 0xc9, 0x33, 0xfa, 0x8a, 0xcd, 0xb6, 0x05, 0xb1, 0x0d, 0x0f,
+ 0xbe, 0x6a, 0xe2, 0x84, 0xc4, 0x7f, 0x5b, 0x80, 0xbc, 0x84, 0xf7, 0x2c, 0x6c, 0xc7, 0xba, 0x28,
+ 0xf0, 0x04, 0x72, 0x16, 0x93, 0x2a, 0xef, 0x59, 0x66, 0xe7, 0x3c, 0xe3, 0x2a, 0xcb, 0x19, 0x9f,
+ 0x58, 0x66, 0x87, 0x3b, 0x96, 0x97, 0x50, 0xf0, 0xea, 0x18, 0x67, 0xe3, 0x7f, 0x97, 0x6e, 0x97,
+ 0x66, 0x82, 0xe3, 0x4e, 0x90, 0x88, 0x57, 0x03, 0xf4, 0x45, 0x55, 0xb0, 0xa2, 0x71, 0xaa, 0xe1,
+ 0xbf, 0x08, 0x90, 0x6f, 0xf4, 0x76, 0xd9, 0x99, 0x5d, 0xf1, 0x69, 0xa0, 0x0a, 0x99, 0x36, 0xde,
+ 0x73, 0xe4, 0x37, 0x4a, 0xd5, 0x4f, 0x13, 0x56, 0xba, 0x51, 0xe1, 0x29, 0x80, 0x45, 0xb7, 0xd8,
+ 0x51, 0x39, 0xc9, 0x73, 0xca, 0xc9, 0x50, 0x5e, 0x42, 0x16, 0xff, 0x75, 0x12, 0x0a, 0x5e, 0x33,
+ 0xe3, 0xf4, 0x92, 0xaf, 0x42, 0xde, 0x21, 0x79, 0x1e, 0xef, 0x30, 0xcf, 0x73, 0x42, 0xa2, 0x3d,
+ 0x44, 0x19, 0x16, 0x68, 0xe0, 0x22, 0x2b, 0xdd, 0x6e, 0x5b, 0x77, 0xe1, 0x2e, 0xf5, 0x3f, 0x29,
+ 0x69, 0x9e, 0xde, 0x5a, 0x61, 0x77, 0x28, 0xd0, 0x45, 0xbf, 0x26, 0x40, 0x6e, 0xcf, 0xc2, 0xf8,
+ 0x0b, 0x2c, 0x53, 0xe8, 0x35, 0x59, 0xd2, 0xcb, 0x1a, 0xa9, 0xc3, 0x57, 0x7e, 0x29, 0x9e, 0x65,
+ 0x0f, 0x6e, 0x90, 0xe7, 0xa2, 0x17, 0xc4, 0xf8, 0x15, 0x4d, 0xe6, 0xe7, 0x1a, 0xf2, 0x70, 0xbe,
+ 0x3c, 0xe4, 0x9c, 0x83, 0xc0, 0x71, 0x88, 0x65, 0x09, 0x2b, 0x1a, 0x8f, 0x19, 0xc9, 0x38, 0xf0,
+ 0x2e, 0xf8, 0x38, 0x78, 0x05, 0xf3, 0xb4, 0x9f, 0xe3, 0xde, 0xae, 0x2c, 0xfe, 0x28, 0x01, 0x28,
+ 0x28, 0xf9, 0x67, 0x67, 0x1f, 0x89, 0xf8, 0xec, 0xe3, 0x5d, 0x40, 0x2c, 0x13, 0xd3, 0x96, 0xbb,
+ 0xd8, 0x92, 0x6d, 0xac, 0x9a, 0xfc, 0xc4, 0x29, 0x41, 0x2a, 0xf2, 0x3b, 0xdb, 0xd8, 0x6a, 0x50,
+ 0x3a, 0x5a, 0x01, 0xf0, 0xa3, 0x44, 0x3e, 0x89, 0x4d, 0x12, 0x24, 0x66, 0xbc, 0x20, 0x51, 0xfc,
+ 0xbe, 0x00, 0xf9, 0x4d, 0xbd, 0x65, 0x29, 0xb1, 0x1e, 0xa8, 0x84, 0x3e, 0x0e, 0xaf, 0xcf, 0x67,
+ 0x1f, 0x2e, 0x46, 0xe5, 0x0e, 0xb1, 0x12, 0x2e, 0xea, 0xe3, 0x0c, 0x64, 0x6e, 0xf0, 0x6a, 0x14,
+ 0xa7, 0x53, 0xfc, 0x77, 0x8b, 0x90, 0xe3, 0xf5, 0xde, 0x31, 0x74, 0xd3, 0x40, 0x0f, 0x20, 0xd9,
+ 0xe2, 0xef, 0x5f, 0xb2, 0x91, 0x6b, 0xa5, 0xfe, 0x71, 0x85, 0xb5, 0x29, 0x89, 0x94, 0x25, 0x2c,
+ 0xdd, 0x9e, 0x13, 0x11, 0x9d, 0xfa, 0xe9, 0xf4, 0x41, 0x96, 0x6e, 0xcf, 0x41, 0x0d, 0x28, 0xa8,
+ 0xfe, 0x19, 0x69, 0x32, 0x61, 0x4f, 0x0e, 0x45, 0xb2, 0x91, 0xa7, 0xd5, 0xd5, 0xa6, 0xa4, 0xbc,
+ 0x1a, 0xba, 0x81, 0x2a, 0xc1, 0xa3, 0xb9, 0x52, 0x03, 0xb9, 0x7a, 0xfe, 0x46, 0xf0, 0xf0, 0xb1,
+ 0x60, 0xb5, 0xa9, 0xc0, 0x09, 0x5e, 0xe8, 0x63, 0x98, 0xd1, 0xe8, 0x21, 0x50, 0xdc, 0xab, 0x44,
+ 0x75, 0x74, 0xe8, 0xac, 0xad, 0xda, 0x94, 0xc4, 0x39, 0xd0, 0x3a, 0xe4, 0xd8, 0x2f, 0x16, 0x2b,
+ 0x72, 0x7f, 0x70, 0x6b, 0xb8, 0x84, 0xc0, 0x6c, 0x5c, 0x9b, 0x92, 0xb2, 0x9a, 0x4f, 0x45, 0xdf,
+ 0x84, 0x94, 0xad, 0x2a, 0x2e, 0xc0, 0xbf, 0x3a, 0xe4, 0xec, 0x15, 0x9f, 0x99, 0x96, 0x46, 0x8f,
+ 0xd9, 0x69, 0xa1, 0xce, 0xb1, 0xbb, 0xd6, 0x1a, 0x55, 0xfd, 0xd0, 0x1e, 0x7f, 0x52, 0x7d, 0x4c,
+ 0x09, 0xe8, 0x29, 0x64, 0x15, 0x12, 0x74, 0xcb, 0x74, 0x97, 0x2c, 0x5d, 0x5c, 0x8d, 0xce, 0x70,
+ 0x18, 0xd8, 0xd5, 0x5c, 0xa3, 0x07, 0x09, 0xb8, 0x44, 0x5f, 0x50, 0x07, 0x5b, 0x2d, 0x5c, 0xca,
+ 0x8e, 0x16, 0x14, 0xcc, 0xd0, 0xf3, 0x04, 0x51, 0x22, 0x09, 0xbe, 0xbd, 0x2d, 0xea, 0xb4, 0x51,
+ 0xb9, 0xa1, 0xef, 0xd4, 0x23, 0x36, 0x5f, 0xd5, 0xa6, 0xa4, 0xdc, 0x7e, 0x80, 0x8c, 0xca, 0x90,
+ 0x68, 0xa9, 0xa5, 0xb9, 0xa1, 0x2e, 0xc1, 0xdb, 0x60, 0x54, 0x9b, 0x92, 0x12, 0x2d, 0x15, 0x7d,
+ 0x0a, 0x69, 0xb6, 0x5b, 0xe4, 0xd8, 0x28, 0xe5, 0x87, 0x8e, 0xb1, 0xf0, 0x9e, 0x9b, 0xda, 0x94,
+ 0x44, 0x37, 0xa8, 0x90, 0xe7, 0x6d, 0x43, 0xde, 0x62, 0x29, 0x8e, 0x6e, 0x42, 0x72, 0x71, 0x68,
+ 0x9e, 0x41, 0x54, 0x4e, 0x72, 0x8d, 0xc2, 0xaf, 0x00, 0x1d, 0x7d, 0x07, 0x2e, 0x84, 0x25, 0x72,
+ 0x4b, 0x9b, 0x1f, 0xfa, 0xce, 0x7c, 0x68, 0x7e, 0x6c, 0x6d, 0x4a, 0x42, 0xd6, 0xc0, 0x4d, 0xf4,
+ 0x21, 0x4c, 0xb3, 0x5e, 0x43, 0x54, 0x64, 0x54, 0xea, 0x4c, 0x5f, 0x87, 0xb1, 0xf2, 0xc4, 0xf8,
+ 0x1d, 0x9e, 0xe7, 0x27, 0xb7, 0xcd, 0x56, 0x69, 0x61, 0xa8, 0xf1, 0x0f, 0x66, 0x2c, 0x12, 0xe3,
+ 0x77, 0x7c, 0x2a, 0xe9, 0x77, 0x8b, 0xdd, 0xe1, 0xb9, 0x5d, 0x17, 0x86, 0xf6, 0x7b, 0x44, 0xfa,
+ 0x5f, 0x8d, 0xee, 0xc0, 0xf0, 0xc9, 0xa4, 0x6a, 0x16, 0x3b, 0x2e, 0x48, 0xa6, 0x63, 0xea, 0xe2,
+ 0xd0, 0xaa, 0x0d, 0x9e, 0xa7, 0x54, 0xa3, 0x81, 0xaa, 0x47, 0x45, 0x2f, 0xa1, 0xc8, 0x0f, 0xf5,
+ 0xf0, 0x5f, 0xec, 0x5c, 0xa2, 0xf2, 0xee, 0x45, 0xba, 0xae, 0xa8, 0xc4, 0xa8, 0xda, 0x94, 0x54,
+ 0x50, 0xc3, 0x77, 0xd0, 0x67, 0x30, 0x4f, 0xe5, 0xc9, 0xaa, 0x7f, 0x0e, 0x4b, 0xa9, 0x34, 0x70,
+ 0xaa, 0xc7, 0xf0, 0x23, 0x5b, 0x5c, 0xc9, 0x45, 0xb5, 0xef, 0x16, 0x31, 0x63, 0xdd, 0xd0, 0x1d,
+ 0xea, 0x65, 0x17, 0x87, 0x9a, 0x71, 0xf8, 0x4c, 0x49, 0x62, 0xc6, 0x3a, 0xa3, 0x10, 0x33, 0x76,
+ 0x78, 0xe2, 0x1f, 0xef, 0x8e, 0xb7, 0x87, 0x9a, 0x71, 0x54, 0x86, 0x20, 0x31, 0x63, 0x27, 0x48,
+ 0x27, 0x66, 0xcc, 0x1c, 0x44, 0x9f, 0xdc, 0x77, 0x86, 0x9a, 0xf1, 0xd0, 0xdd, 0xea, 0xc4, 0x8c,
+ 0x95, 0x81, 0x9b, 0x68, 0x0d, 0x80, 0xc5, 0x94, 0x34, 0x0a, 0xb8, 0x3a, 0x74, 0x32, 0xe8, 0x4f,
+ 0x00, 0x24, 0x93, 0x41, 0xdb, 0xa5, 0x11, 0x47, 0x46, 0x11, 0xab, 0x4c, 0xdf, 0x63, 0x97, 0x96,
+ 0x86, 0x3a, 0xb2, 0x81, 0x77, 0xce, 0xc4, 0x91, 0x1d, 0x79, 0x44, 0x32, 0xab, 0xb0, 0x85, 0xf7,
+ 0xd2, 0xb5, 0xe1, 0x6e, 0x39, 0xf8, 0xfe, 0x8d, 0xba, 0x65, 0x4a, 0x40, 0x2b, 0x90, 0x21, 0x41,
+ 0xce, 0x09, 0x75, 0x43, 0xd7, 0x87, 0x02, 0x83, 0xbe, 0x2d, 0x46, 0xb5, 0x29, 0x29, 0xfd, 0x9a,
+ 0x93, 0xc8, 0xe3, 0xd9, 0x92, 0x64, 0x49, 0x1c, 0xfa, 0xf8, 0xd0, 0xf2, 0x35, 0x79, 0x3c, 0xe3,
+ 0x40, 0x2a, 0x5c, 0x64, 0x7d, 0xc5, 0x37, 0x8e, 0x5b, 0x7c, 0xaf, 0x73, 0xe9, 0x06, 0x15, 0x35,
+ 0x74, 0x6d, 0x2f, 0x72, 0x3f, 0x7b, 0x6d, 0x4a, 0x5a, 0x50, 0x06, 0xef, 0x92, 0x01, 0xcf, 0xa7,
+ 0x1e, 0xb6, 0x22, 0x58, 0xba, 0x39, 0x74, 0xc0, 0x47, 0x2c, 0xa4, 0x92, 0x01, 0xaf, 0x04, 0xc8,
+ 0x6c, 0x02, 0xd2, 0x64, 0xdb, 0x66, 0x59, 0x0f, 0xb7, 0x46, 0x4c, 0x40, 0x7d, 0x4b, 0x31, 0x6c,
+ 0x02, 0xd2, 0x1a, 0x8c, 0x93, 0x08, 0x52, 0xdb, 0x58, 0xb1, 0xb8, 0x9b, 0xbd, 0x3d, 0x54, 0xd0,
+ 0xc0, 0xf1, 0x8d, 0x44, 0x90, 0xea, 0x11, 0x49, 0xc0, 0x63, 0xb9, 0xa7, 0x01, 0xf1, 0x08, 0xf9,
+ 0xce, 0xd0, 0x80, 0x27, 0xf2, 0xb8, 0x22, 0x12, 0xf0, 0x58, 0xa1, 0x1b, 0xe8, 0xe7, 0x61, 0x96,
+ 0x63, 0xe8, 0xd2, 0xdd, 0x11, 0x21, 0x60, 0x70, 0xd9, 0x83, 0x8c, 0x6b, 0xce, 0xc3, 0xbc, 0x2c,
+ 0xc3, 0xee, 0xac, 0x79, 0xf7, 0x46, 0x78, 0xd9, 0x81, 0xe5, 0x03, 0xe6, 0x65, 0x7d, 0x32, 0xf1,
+ 0xb2, 0xcc, 0x4e, 0xf9, 0x5c, 0x77, 0x7f, 0xa8, 0x97, 0x1d, 0xdc, 0xc0, 0x44, 0xbc, 0xec, 0x6b,
+ 0x9f, 0x4a, 0x5a, 0x66, 0x33, 0x0c, 0x5b, 0xfa, 0xc6, 0xd0, 0x96, 0x85, 0xc1, 0x3c, 0x69, 0x19,
+ 0xe7, 0x21, 0xdd, 0xc6, 0x30, 0x00, 0xd3, 0xf4, 0xbb, 0xc3, 0x4f, 0x67, 0xe8, 0x47, 0x59, 0x35,
+ 0x77, 0xb5, 0x98, 0x69, 0xd8, 0x73, 0x54, 0x16, 0xdf, 0x3a, 0xce, 0x35, 0xf5, 0xde, 0x68, 0x47,
+ 0x15, 0xb5, 0xe5, 0xde, 0x73, 0x54, 0xa1, 0x9b, 0xb4, 0xaa, 0x6c, 0xb7, 0x20, 0x1d, 0xdf, 0xe5,
+ 0x11, 0x07, 0x49, 0xf4, 0xed, 0xd9, 0xa4, 0x55, 0xf5, 0x88, 0xfe, 0x10, 0xea, 0xb1, 0x53, 0x4e,
+ 0x4a, 0xcb, 0xa3, 0x87, 0x50, 0xf8, 0x9c, 0x15, 0x6f, 0x08, 0x71, 0xb2, 0x37, 0x67, 0xba, 0x11,
+ 0xc6, 0xfb, 0xa3, 0xe7, 0xcc, 0xfe, 0xd0, 0x82, 0xcd, 0x99, 0x3c, 0xa6, 0xf8, 0x6b, 0x02, 0x5c,
+ 0x63, 0x75, 0xa3, 0xcb, 0xaa, 0x27, 0xb2, 0xb7, 0x38, 0x1d, 0xd8, 0xb9, 0xf2, 0x80, 0x3e, 0xe0,
+ 0xc3, 0x61, 0xd5, 0x1d, 0xb3, 0xd8, 0x5e, 0x9b, 0x92, 0xde, 0x51, 0x46, 0x95, 0x23, 0x26, 0xd5,
+ 0x61, 0x08, 0xaa, 0xf4, 0x70, 0xa8, 0x49, 0x85, 0x51, 0x1f, 0x31, 0x29, 0xce, 0xb3, 0x3a, 0xcb,
+ 0x5f, 0x6d, 0x7b, 0x3b, 0x7d, 0x0b, 0xc5, 0xe2, 0x7a, 0x2a, 0x7d, 0xb9, 0x58, 0x5a, 0x4f, 0xa5,
+ 0xaf, 0x14, 0x17, 0xd7, 0x53, 0xe9, 0xb7, 0x8a, 0x6f, 0x8b, 0x7f, 0x7f, 0x11, 0xe6, 0x5c, 0xd0,
+ 0xc5, 0x00, 0xd5, 0xc3, 0x20, 0xa0, 0xba, 0x3a, 0x0c, 0x50, 0x71, 0x98, 0xc6, 0x11, 0xd5, 0xc3,
+ 0x20, 0xa2, 0xba, 0x3a, 0x0c, 0x51, 0xf9, 0x3c, 0x04, 0x52, 0x35, 0x87, 0x41, 0xaa, 0x7b, 0x13,
+ 0x40, 0x2a, 0x4f, 0x54, 0x3f, 0xa6, 0x5a, 0x1b, 0xc4, 0x54, 0x37, 0x47, 0x63, 0x2a, 0x4f, 0x54,
+ 0x00, 0x54, 0x3d, 0xee, 0x03, 0x55, 0xd7, 0x47, 0x80, 0x2a, 0x8f, 0xdf, 0x45, 0x55, 0x1b, 0x91,
+ 0xa8, 0xea, 0xf6, 0x38, 0x54, 0xe5, 0xc9, 0x09, 0xc1, 0xaa, 0x0f, 0x42, 0xb0, 0x6a, 0x69, 0x28,
+ 0xac, 0xf2, 0xb8, 0x19, 0xae, 0xfa, 0xa4, 0x1f, 0x57, 0x5d, 0x1f, 0x81, 0xab, 0xfc, 0x16, 0x70,
+ 0x60, 0x55, 0x8b, 0x02, 0x56, 0xb7, 0xc6, 0x00, 0x2b, 0x4f, 0x4a, 0x10, 0x59, 0xd5, 0xa2, 0x90,
+ 0xd5, 0xad, 0x31, 0xc8, 0xaa, 0x4f, 0x12, 0x83, 0x56, 0x5b, 0xd1, 0xd0, 0xea, 0xce, 0x58, 0x68,
+ 0xe5, 0x49, 0x0b, 0x63, 0xab, 0xe5, 0x00, 0xb6, 0x7a, 0x67, 0x08, 0xb6, 0xf2, 0x58, 0x09, 0xb8,
+ 0xfa, 0xd6, 0x00, 0xb8, 0x12, 0x47, 0x81, 0x2b, 0x8f, 0xd7, 0x43, 0x57, 0x2f, 0x86, 0xa0, 0xab,
+ 0xbb, 0xe3, 0xd1, 0x95, 0x27, 0xac, 0x0f, 0x5e, 0x29, 0x23, 0xe1, 0xd5, 0x7b, 0x13, 0xc2, 0x2b,
+ 0x4f, 0x7a, 0x14, 0xbe, 0xfa, 0x28, 0x8c, 0xaf, 0xae, 0x0d, 0xc7, 0x57, 0x9e, 0x18, 0x0e, 0xb0,
+ 0x36, 0x22, 0x01, 0xd6, 0xed, 0x71, 0x00, 0xcb, 0x1f, 0x07, 0x41, 0x84, 0xb5, 0x15, 0x8d, 0xb0,
+ 0xee, 0x8c, 0x45, 0x58, 0x7e, 0xf7, 0x87, 0x20, 0xd6, 0x46, 0x24, 0xc4, 0xba, 0x3d, 0x0e, 0x62,
+ 0xf9, 0x95, 0x0b, 0x62, 0xac, 0x57, 0x43, 0x31, 0xd6, 0xfd, 0x49, 0x30, 0x96, 0x27, 0x74, 0x00,
+ 0x64, 0x7d, 0x3e, 0x1c, 0x64, 0x7d, 0xe3, 0x1c, 0xe7, 0x62, 0x46, 0xa2, 0xac, 0x6f, 0x0d, 0xa0,
+ 0x2c, 0x71, 0x14, 0xca, 0xf2, 0xed, 0xd9, 0x85, 0x59, 0xca, 0x48, 0x50, 0xf4, 0xde, 0x84, 0xa0,
+ 0xc8, 0x37, 0xbe, 0x08, 0x54, 0x54, 0x8d, 0x40, 0x45, 0x37, 0x47, 0xa3, 0x22, 0xdf, 0x9d, 0xfb,
+ 0xb0, 0xa8, 0x16, 0x05, 0x8b, 0x6e, 0x8d, 0x81, 0x45, 0xbe, 0x17, 0x0a, 0xe0, 0xa2, 0xc7, 0x7d,
+ 0xb8, 0xe8, 0xfa, 0xd8, 0xcc, 0xad, 0x00, 0x30, 0x5a, 0x1d, 0x04, 0x46, 0x37, 0x46, 0x02, 0x23,
+ 0x4f, 0x82, 0x8f, 0x8c, 0x1e, 0xf7, 0x21, 0xa3, 0xeb, 0x23, 0x90, 0x91, 0x5f, 0x01, 0x0e, 0x8d,
+ 0xb4, 0xd1, 0xd0, 0xa8, 0x3c, 0x29, 0x34, 0xf2, 0x04, 0x47, 0x62, 0xa3, 0xad, 0x68, 0x6c, 0x74,
+ 0x67, 0xc2, 0xa4, 0x8a, 0x01, 0x70, 0x54, 0x8b, 0x02, 0x47, 0xb7, 0xc6, 0x80, 0xa3, 0xe0, 0x1c,
+ 0xe2, 0xa1, 0xa3, 0x5a, 0x14, 0x3a, 0xba, 0x35, 0x06, 0x1d, 0xf9, 0x92, 0x02, 0xf0, 0xa8, 0x39,
+ 0x0c, 0x1e, 0xdd, 0x9b, 0x00, 0x1e, 0xf9, 0xc1, 0x4b, 0x1f, 0x3e, 0xfa, 0xb4, 0x1f, 0x1f, 0x89,
+ 0xa3, 0xf0, 0x91, 0x3f, 0x22, 0x5d, 0x80, 0xb4, 0x15, 0x0d, 0x90, 0xee, 0x8c, 0x05, 0x48, 0x41,
+ 0x27, 0x19, 0x40, 0x48, 0x1b, 0x91, 0x08, 0xe9, 0xf6, 0x38, 0x84, 0xe4, 0x3b, 0xc9, 0x20, 0x44,
+ 0xfa, 0xb4, 0x1f, 0x22, 0x89, 0xa3, 0x20, 0x92, 0xdf, 0x38, 0x17, 0x23, 0xd5, 0xa2, 0x30, 0xd2,
+ 0xad, 0x31, 0x18, 0xc9, 0xef, 0xbc, 0x00, 0x48, 0x52, 0x46, 0x82, 0xa4, 0xf7, 0x26, 0x04, 0x49,
+ 0x7d, 0x8e, 0x2b, 0x8c, 0x92, 0x6a, 0x51, 0x28, 0xe9, 0xd6, 0x18, 0x94, 0x14, 0xa8, 0xac, 0x0f,
+ 0x93, 0xb6, 0xa2, 0x61, 0xd2, 0x9d, 0xb1, 0x30, 0xa9, 0x6f, 0x34, 0xb9, 0x38, 0x69, 0x23, 0x12,
+ 0x27, 0xdd, 0x1e, 0x87, 0x93, 0xfa, 0x26, 0x3e, 0x1e, 0x1c, 0xfc, 0xca, 0xe4, 0x40, 0xe9, 0xa3,
+ 0xf3, 0x03, 0x25, 0xef, 0x99, 0x63, 0x90, 0xd2, 0xa7, 0xfd, 0x48, 0x49, 0x1c, 0x85, 0x94, 0x7c,
+ 0xcb, 0x3a, 0x1f, 0x54, 0x5a, 0x4f, 0xa5, 0xdf, 0x2e, 0xbe, 0x23, 0xfe, 0xf9, 0x0c, 0xcc, 0xd4,
+ 0xbc, 0x5c, 0x27, 0xbf, 0x95, 0xc2, 0x9b, 0x1c, 0xae, 0x85, 0xd6, 0xc8, 0x88, 0xa7, 0x7e, 0x73,
+ 0xfc, 0x91, 0x8c, 0x83, 0x87, 0x06, 0x72, 0xd6, 0x37, 0xd8, 0xe5, 0x8e, 0x3e, 0x80, 0xb9, 0x9e,
+ 0x8d, 0x2d, 0xb9, 0x6b, 0xe9, 0xa6, 0xa5, 0x3b, 0x6c, 0x27, 0x8f, 0xb0, 0x5a, 0xfc, 0xf2, 0x74,
+ 0x29, 0xb7, 0x63, 0x63, 0x6b, 0x9b, 0xd3, 0xa5, 0x5c, 0x2f, 0x70, 0xe5, 0x7e, 0x3e, 0x6d, 0x7a,
+ 0xf2, 0xcf, 0xa7, 0xbd, 0x80, 0x22, 0x7d, 0xe7, 0x1c, 0x8c, 0x60, 0xd8, 0x41, 0x56, 0xd1, 0x36,
+ 0x47, 0x77, 0xda, 0xb9, 0x25, 0xe9, 0x81, 0x56, 0x05, 0x2b, 0x4c, 0x44, 0x0f, 0xe0, 0x62, 0x47,
+ 0x39, 0xa6, 0x79, 0xb1, 0xb2, 0x1b, 0x14, 0xd2, 0x5c, 0x57, 0xf6, 0xcd, 0x33, 0xd4, 0x51, 0x8e,
+ 0xe9, 0xb7, 0xd8, 0xd8, 0x2d, 0xfa, 0x21, 0x95, 0x5b, 0x90, 0xd7, 0x74, 0xdb, 0xd1, 0x0d, 0xd5,
+ 0xe1, 0x67, 0x24, 0xb3, 0xf3, 0x85, 0xe7, 0x5c, 0x2a, 0x3b, 0x08, 0xb9, 0x02, 0x85, 0x96, 0xe2,
+ 0xe0, 0x23, 0xe5, 0x44, 0x76, 0xb7, 0xd8, 0x65, 0xe9, 0x71, 0xa0, 0x6f, 0x9d, 0x9d, 0x2e, 0xcd,
+ 0x3d, 0x65, 0xb7, 0x06, 0x76, 0xda, 0xcd, 0xb5, 0x02, 0x37, 0x34, 0x74, 0x07, 0x0a, 0x8a, 0x7d,
+ 0x62, 0xa8, 0xb4, 0xc9, 0xd8, 0xb0, 0x7b, 0x36, 0x85, 0x19, 0x69, 0x29, 0x4f, 0xc9, 0x15, 0x97,
+ 0x8a, 0xae, 0x43, 0x8e, 0xef, 0x10, 0x60, 0x1f, 0x6c, 0x2a, 0xd0, 0xea, 0xf3, 0x2f, 0x81, 0xb0,
+ 0x6f, 0x36, 0x3d, 0x86, 0x45, 0xfe, 0x5d, 0x83, 0x23, 0xc5, 0xd2, 0x64, 0xaa, 0x49, 0xdf, 0xe6,
+ 0x8a, 0x54, 0xec, 0x65, 0xf6, 0x1d, 0x03, 0x52, 0x80, 0xa8, 0xcf, 0x1f, 0x17, 0x4d, 0x98, 0x57,
+ 0xdb, 0xba, 0x87, 0x0a, 0x58, 0x10, 0x35, 0x3f, 0x74, 0x84, 0x54, 0x68, 0xd9, 0xfe, 0xd7, 0xcc,
+ 0x05, 0x35, 0x4c, 0x46, 0x0d, 0xa0, 0x87, 0x01, 0xc9, 0x5d, 0xb3, 0xad, 0xab, 0x27, 0x14, 0x16,
+ 0x84, 0x4f, 0x69, 0x1f, 0xf9, 0xad, 0x84, 0x57, 0x8a, 0xee, 0x6c, 0x53, 0x4e, 0x09, 0x8e, 0xbc,
+ 0xdf, 0xec, 0x84, 0xe5, 0xf5, 0x54, 0x1a, 0x8a, 0xd9, 0xf5, 0x54, 0x3a, 0x57, 0x9c, 0x5b, 0x4f,
+ 0xa5, 0xf3, 0xc5, 0x82, 0xf8, 0x1b, 0x02, 0xe4, 0x42, 0x5b, 0xa3, 0x1e, 0xf7, 0xbd, 0x44, 0xbe,
+ 0x12, 0x8d, 0xff, 0x86, 0x25, 0x27, 0xa6, 0xb9, 0xbd, 0xb8, 0x99, 0x9e, 0x4b, 0xc3, 0xf1, 0x03,
+ 0x5d, 0x0d, 0x71, 0xd3, 0x5e, 0x5c, 0xb6, 0x8f, 0x53, 0x3f, 0xf8, 0xe1, 0xd2, 0x94, 0xf8, 0x17,
+ 0x29, 0x98, 0x0b, 0x6f, 0x84, 0xaa, 0xf7, 0xd5, 0x2b, 0xca, 0x3f, 0x87, 0x38, 0xca, 0x23, 0x3e,
+ 0x06, 0x93, 0xf1, 0xbf, 0xa7, 0xc0, 0xaa, 0x79, 0x6d, 0xc4, 0xab, 0xf2, 0x60, 0x3d, 0x7d, 0xc6,
+ 0xc5, 0x7f, 0x9f, 0xf4, 0xfc, 0x54, 0x19, 0xa6, 0xe9, 0x99, 0x46, 0xbc, 0x6a, 0xa5, 0xfe, 0xbe,
+ 0x22, 0x31, 0x2b, 0xb9, 0x2f, 0xb1, 0x62, 0xc4, 0xaf, 0x35, 0xdf, 0xe8, 0xd0, 0x40, 0xdf, 0xf4,
+ 0xce, 0xff, 0x99, 0xc5, 0x1e, 0x3b, 0x34, 0xf2, 0xff, 0x62, 0x6a, 0x0c, 0x79, 0x1e, 0xfa, 0x25,
+ 0x28, 0xa8, 0x66, 0xbb, 0xcd, 0xe6, 0x2c, 0xe6, 0x19, 0x06, 0x8f, 0x91, 0xa1, 0x55, 0xe0, 0x5f,
+ 0xd6, 0x2c, 0x7b, 0x5f, 0xd8, 0x2c, 0x4b, 0xfc, 0x0b, 0x9b, 0x81, 0xbc, 0xdb, 0xbc, 0x27, 0xcc,
+ 0x75, 0x28, 0xa1, 0x14, 0xe0, 0xd9, 0x37, 0x49, 0x01, 0x66, 0x69, 0xe3, 0xdc, 0xf2, 0xfe, 0x58,
+ 0xe0, 0x09, 0x31, 0xcf, 0x4c, 0xf3, 0xa0, 0xe7, 0xa5, 0xee, 0x2e, 0x06, 0x8f, 0x80, 0x4c, 0x7f,
+ 0x79, 0xba, 0x94, 0x92, 0xbc, 0x33, 0x20, 0xa3, 0x3c, 0x70, 0xe2, 0xab, 0x79, 0xe0, 0xeb, 0x90,
+ 0xeb, 0x5a, 0x78, 0x0f, 0x3b, 0xea, 0xbe, 0x6c, 0xf4, 0x3a, 0x7c, 0x87, 0x4f, 0xd6, 0xa5, 0x6d,
+ 0xf5, 0x3a, 0xe8, 0x1e, 0x14, 0xbd, 0x22, 0x1c, 0x2b, 0xbb, 0x67, 0x84, 0xb9, 0x74, 0x8e, 0xac,
+ 0xc5, 0xff, 0x29, 0xc0, 0x42, 0xa8, 0x4d, 0x7c, 0x4c, 0xad, 0x43, 0x56, 0xf3, 0xe6, 0x3c, 0xbb,
+ 0x24, 0x9c, 0x33, 0x91, 0x35, 0xc8, 0x8c, 0x64, 0xb8, 0xe4, 0x3e, 0x96, 0x7e, 0x83, 0xc0, 0x17,
+ 0x9b, 0x38, 0xa7, 0xd8, 0x8b, 0xbe, 0x9c, 0xb5, 0xc0, 0x03, 0xbc, 0x41, 0x96, 0x9c, 0x68, 0x90,
+ 0x89, 0xbf, 0x2d, 0x40, 0x91, 0x3e, 0xe0, 0x09, 0xc6, 0x5a, 0x2c, 0xde, 0xcd, 0x4d, 0x10, 0x4f,
+ 0x4c, 0xbe, 0x87, 0x27, 0xf4, 0xdd, 0x94, 0x64, 0xf8, 0xbb, 0x29, 0xe2, 0x0f, 0x05, 0xc8, 0x7b,
+ 0x35, 0x64, 0xdf, 0x2a, 0x1c, 0x71, 0xd2, 0xe8, 0x9b, 0x7d, 0x8f, 0xcf, 0x3d, 0x11, 0x65, 0xa2,
+ 0xcf, 0x27, 0x06, 0x4f, 0x44, 0x61, 0xdf, 0x91, 0xfb, 0x9b, 0xae, 0xe5, 0x90, 0x2a, 0x56, 0xfc,
+ 0x93, 0x2f, 0xde, 0x60, 0x3b, 0x93, 0x44, 0x3f, 0xf3, 0x6a, 0xb6, 0x0f, 0xd9, 0x21, 0x36, 0x13,
+ 0xb9, 0x3d, 0xc4, 0xd3, 0xc0, 0x80, 0xaf, 0xaa, 0x69, 0xcd, 0x06, 0xfd, 0x00, 0x2c, 0xfb, 0x6d,
+ 0x8b, 0x4f, 0x02, 0x0a, 0xa4, 0x9d, 0x4f, 0xb4, 0x34, 0x91, 0x2b, 0x76, 0xb5, 0xc4, 0x6c, 0xe5,
+ 0x0f, 0x83, 0x3d, 0x51, 0x3d, 0x24, 0x68, 0xea, 0x11, 0x24, 0x0f, 0x95, 0xf6, 0xa8, 0x4c, 0xaa,
+ 0x50, 0xcf, 0x49, 0xa4, 0x34, 0x7a, 0x12, 0x3a, 0x30, 0x24, 0x31, 0x3c, 0xf2, 0x1f, 0x54, 0x69,
+ 0xe8, 0x60, 0x91, 0x0f, 0xc3, 0xb6, 0x3e, 0xf2, 0xf1, 0x41, 0xa3, 0xff, 0x38, 0xf5, 0xe3, 0x1f,
+ 0x2e, 0x09, 0xe2, 0x27, 0x80, 0x24, 0x6c, 0x63, 0xe7, 0x45, 0xcf, 0xb4, 0xfc, 0xc3, 0x57, 0x6e,
+ 0xf7, 0x7d, 0x48, 0x66, 0x7a, 0x35, 0x7b, 0x16, 0x75, 0x3e, 0xd3, 0x45, 0x58, 0x08, 0x71, 0x33,
+ 0x67, 0x21, 0x7e, 0x08, 0x57, 0x9e, 0x9a, 0xb6, 0xad, 0x77, 0x09, 0x84, 0xa4, 0xa3, 0x92, 0x4c,
+ 0x2d, 0x9e, 0x7b, 0x4c, 0x77, 0xe9, 0x62, 0x82, 0xc1, 0xdc, 0x48, 0x46, 0xf2, 0xae, 0xc5, 0xdf,
+ 0x13, 0xe0, 0xf2, 0x20, 0x27, 0xd3, 0x72, 0xd4, 0xee, 0xcb, 0x59, 0xd5, 0xf4, 0x0f, 0xeb, 0x1b,
+ 0x6f, 0xad, 0x6e, 0x71, 0x12, 0x16, 0xf2, 0x67, 0xca, 0x1d, 0x85, 0xba, 0x0f, 0xbe, 0x13, 0x3c,
+ 0xcf, 0xc9, 0x9b, 0x8c, 0xea, 0x7b, 0x92, 0xd4, 0x64, 0x9e, 0xa4, 0x09, 0x85, 0x75, 0x53, 0x37,
+ 0x48, 0xf4, 0xe9, 0xb6, 0x77, 0x05, 0xf2, 0xbb, 0xba, 0xa1, 0x58, 0x27, 0xb2, 0x9b, 0xc0, 0x27,
+ 0x8c, 0x4b, 0xe0, 0x93, 0xe6, 0x18, 0x07, 0xbf, 0x14, 0x7f, 0x22, 0x40, 0xd1, 0x17, 0xcb, 0x3d,
+ 0xf2, 0xbb, 0x00, 0x6a, 0xbb, 0x67, 0x3b, 0xd8, 0x72, 0x7b, 0x29, 0xc7, 0x32, 0xe9, 0x2b, 0x8c,
+ 0x5a, 0x5f, 0x93, 0x32, 0xbc, 0x40, 0x5d, 0x43, 0x37, 0xc2, 0x07, 0x55, 0x4c, 0xaf, 0xc2, 0xd9,
+ 0xc0, 0xf1, 0x14, 0xa4, 0xdb, 0x6d, 0xc7, 0xb4, 0x3c, 0xec, 0xc2, 0xbb, 0xdd, 0x3d, 0x7a, 0x9f,
+ 0xee, 0xaf, 0x26, 0xe5, 0x56, 0x20, 0x4f, 0xc2, 0x85, 0x43, 0xec, 0x35, 0x29, 0x35, 0xbe, 0x49,
+ 0x8c, 0xc3, 0x6d, 0xd2, 0x3f, 0x17, 0xa0, 0x50, 0x61, 0xbd, 0xe1, 0xf5, 0xf0, 0x08, 0x8f, 0xb6,
+ 0x06, 0x69, 0xe7, 0xd8, 0x90, 0x3b, 0xd8, 0xfb, 0x5c, 0xcd, 0x39, 0x4e, 0xd2, 0x9b, 0x75, 0xd8,
+ 0x25, 0xfd, 0xa2, 0x21, 0xff, 0x9c, 0x36, 0x1f, 0x2e, 0x57, 0xca, 0xec, 0x7b, 0xdb, 0x65, 0xf7,
+ 0x7b, 0xdb, 0xe5, 0x35, 0x5e, 0x80, 0x39, 0xf5, 0x1f, 0xfc, 0xe7, 0x25, 0x41, 0xf2, 0x98, 0xd8,
+ 0xbc, 0x7f, 0xbf, 0x41, 0xac, 0x7e, 0x60, 0x66, 0x46, 0x79, 0x80, 0xc0, 0x77, 0x88, 0xf8, 0x17,
+ 0x9c, 0x57, 0xd6, 0xe4, 0x9d, 0xad, 0xca, 0xf3, 0xcd, 0xcd, 0x7a, 0xb3, 0x59, 0x5d, 0x2b, 0x0a,
+ 0xa8, 0x08, 0xb9, 0xd0, 0x57, 0x8c, 0x12, 0xec, 0x9b, 0xce, 0xf7, 0xff, 0x3f, 0x00, 0xff, 0x83,
+ 0x68, 0x44, 0xd6, 0x46, 0xf5, 0x33, 0xf9, 0xe5, 0xca, 0xb3, 0x9d, 0x6a, 0xa3, 0x38, 0x85, 0x10,
+ 0xe4, 0x57, 0x57, 0x9a, 0x95, 0x9a, 0x2c, 0x55, 0x1b, 0xdb, 0xcf, 0xb7, 0x1a, 0x55, 0xf7, 0x5b,
+ 0xd0, 0xf7, 0xd7, 0x20, 0x17, 0x3c, 0x8e, 0x08, 0x2d, 0x40, 0xa1, 0x52, 0xab, 0x56, 0x36, 0xe4,
+ 0x97, 0xf5, 0x15, 0xf9, 0xc5, 0x4e, 0x75, 0xa7, 0x5a, 0x9c, 0xa2, 0x55, 0xa3, 0xc4, 0x27, 0x3b,
+ 0xcf, 0x9e, 0x15, 0x05, 0x54, 0x80, 0x2c, 0xbb, 0xa6, 0x5f, 0x3c, 0x2a, 0x26, 0xee, 0x6f, 0x42,
+ 0x36, 0x70, 0x6e, 0x31, 0x79, 0xdc, 0xf6, 0x4e, 0xa3, 0x26, 0x37, 0xeb, 0x9b, 0xd5, 0x46, 0x73,
+ 0x65, 0x73, 0x9b, 0xc9, 0xa0, 0xb4, 0x95, 0xd5, 0xe7, 0x52, 0xb3, 0x28, 0x78, 0xd7, 0xcd, 0xe7,
+ 0x3b, 0x95, 0x9a, 0xdb, 0x0c, 0x31, 0x95, 0x4e, 0x16, 0x93, 0xf7, 0xff, 0xaa, 0x00, 0x97, 0x87,
+ 0x1c, 0xcd, 0x83, 0xb2, 0x30, 0xbb, 0x63, 0xd0, 0x43, 0x5c, 0x8b, 0x53, 0x68, 0x2e, 0x70, 0x3a,
+ 0x4f, 0x51, 0x40, 0x69, 0x76, 0x3e, 0x4a, 0x31, 0x81, 0x66, 0x20, 0xd1, 0x78, 0x54, 0x4c, 0x92,
+ 0x9a, 0x06, 0x0e, 0xb7, 0x29, 0xa6, 0x50, 0x86, 0x1f, 0xab, 0x51, 0x9c, 0x46, 0x39, 0xff, 0x74,
+ 0x8b, 0xe2, 0x0c, 0x11, 0xe5, 0x9d, 0x12, 0x51, 0x9c, 0xbd, 0x7f, 0x1d, 0x02, 0x3b, 0xf1, 0x11,
+ 0xc0, 0xcc, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, 0x42, 0xb3, 0x90, 0x5c, 0x69, 0xb7, 0x8b, 0xc2,
+ 0xc3, 0x7f, 0x96, 0x82, 0xb4, 0xfb, 0x65, 0x1f, 0xf4, 0x0c, 0xa6, 0xd9, 0xb2, 0xf2, 0xd2, 0x70,
+ 0xb4, 0x40, 0x07, 0xf4, 0xe2, 0xb5, 0x71, 0x70, 0x42, 0x9c, 0x42, 0xff, 0x3f, 0x64, 0x03, 0x51,
+ 0x14, 0x1a, 0xba, 0x34, 0x16, 0x8a, 0x1c, 0x17, 0x6f, 0x8f, 0x2b, 0xe6, 0xc9, 0x7f, 0x05, 0x19,
+ 0xcf, 0xab, 0xa3, 0x1b, 0xa3, 0x7c, 0xbe, 0x2b, 0x7b, 0xf4, 0xc4, 0x40, 0xc6, 0x9f, 0x38, 0xf5,
+ 0xbe, 0x80, 0x2c, 0x40, 0x83, 0x0e, 0x18, 0x45, 0x25, 0x2b, 0x0c, 0xf5, 0xf0, 0x8b, 0xf7, 0x27,
+ 0x2a, 0xed, 0x3f, 0x93, 0x28, 0xcb, 0x9f, 0x45, 0xa2, 0x95, 0x35, 0x30, 0x47, 0x45, 0x2b, 0x2b,
+ 0x62, 0x32, 0x9a, 0x42, 0x2f, 0x20, 0x45, 0xbc, 0x27, 0x8a, 0x8a, 0x2b, 0xfb, 0xbc, 0xf5, 0xe2,
+ 0x8d, 0x91, 0x65, 0x5c, 0x91, 0xab, 0xf7, 0x7e, 0xfc, 0x67, 0x57, 0xa7, 0x7e, 0x7c, 0x76, 0x55,
+ 0xf8, 0xc9, 0xd9, 0x55, 0xe1, 0x4f, 0xce, 0xae, 0x0a, 0x7f, 0x7a, 0x76, 0x55, 0xf8, 0xfe, 0x4f,
+ 0xaf, 0x4e, 0xfd, 0xe4, 0xa7, 0x57, 0xa7, 0xfe, 0xe4, 0xa7, 0x57, 0xa7, 0x3e, 0x9f, 0xe5, 0xdc,
+ 0xbb, 0x33, 0xd4, 0xb5, 0x3c, 0xfa, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x17, 0x23, 0xa7, 0xec,
+ 0x31, 0x81, 0x00, 0x00,
}
diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto
index 3f6805808049..e9d12d976cfa 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";
@@ -1657,6 +1658,21 @@ message SubsumeResponse {
// mark for the keys previously owned by the subsumed range.
util.hlc.Timestamp freeze_start = 5 [(gogoproto.nullable) = false,
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"];
+
+ // 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 = 6;
}
// 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 c2ac7657473e..43a356b1c48c 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_f7c076f1b3c5df9f, []int{0}
+ return fileDescriptor_data_4eed16a3a3304f60, []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_f7c076f1b3c5df9f, []int{1}
+ return fileDescriptor_data_4eed16a3a3304f60, []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_f7c076f1b3c5df9f, []int{2}
+ return fileDescriptor_data_4eed16a3a3304f60, []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_f7c076f1b3c5df9f, []int{3}
+ return fileDescriptor_data_4eed16a3a3304f60, []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_f7c076f1b3c5df9f, []int{0}
+ return fileDescriptor_data_4eed16a3a3304f60, []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_f7c076f1b3c5df9f, []int{1}
+ return fileDescriptor_data_4eed16a3a3304f60, []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_f7c076f1b3c5df9f, []int{2}
+ return fileDescriptor_data_4eed16a3a3304f60, []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_f7c076f1b3c5df9f, []int{3}
+ return fileDescriptor_data_4eed16a3a3304f60, []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_f7c076f1b3c5df9f, []int{4}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{4}
}
func (m *SplitTrigger) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -441,13 +442,28 @@ type MergeTrigger struct {
// being subsumed). It is suitable for use as the timestamp cache's low water
// mark for the keys previously owned by the subsumed range.
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"`
+ // 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.
+ RightReadSummary *rspb.ReadSummary `protobuf:"bytes,6,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_f7c076f1b3c5df9f, []int{5}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{5}
}
func (m *MergeTrigger) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -517,7 +533,7 @@ type ChangeReplicasTrigger struct {
func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} }
func (*ChangeReplicasTrigger) ProtoMessage() {}
func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) {
- return fileDescriptor_data_f7c076f1b3c5df9f, []int{6}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{6}
}
func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -559,7 +575,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_f7c076f1b3c5df9f, []int{7}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{7}
}
func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -605,7 +621,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_f7c076f1b3c5df9f, []int{8}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{8}
}
func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -644,7 +660,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_f7c076f1b3c5df9f, []int{9}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{9}
}
func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -713,7 +729,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_f7c076f1b3c5df9f, []int{10}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{10}
}
func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -888,7 +904,7 @@ type Transaction struct {
func (m *Transaction) Reset() { *m = Transaction{} }
func (*Transaction) ProtoMessage() {}
func (*Transaction) Descriptor() ([]byte, []int) {
- return fileDescriptor_data_f7c076f1b3c5df9f, []int{11}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{11}
}
func (m *Transaction) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -939,7 +955,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_f7c076f1b3c5df9f, []int{12}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{12}
}
func (m *TransactionRecord) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -979,7 +995,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_f7c076f1b3c5df9f, []int{13}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{13}
}
func (m *Intent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1016,7 +1032,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_f7c076f1b3c5df9f, []int{13, 0}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{13, 0}
}
func (m *Intent_SingleKeySpan) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1053,7 +1069,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_f7c076f1b3c5df9f, []int{14}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{14}
}
func (m *LockAcquisition) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1094,7 +1110,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_f7c076f1b3c5df9f, []int{15}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{15}
}
func (m *LockUpdate) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1131,7 +1147,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_f7c076f1b3c5df9f, []int{16}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{16}
}
func (m *SequencedWrite) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1196,7 +1212,7 @@ type Lease struct {
func (m *Lease) Reset() { *m = Lease{} }
func (*Lease) ProtoMessage() {}
func (*Lease) Descriptor() ([]byte, []int) {
- return fileDescriptor_data_f7c076f1b3c5df9f, []int{17}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{17}
}
func (m *Lease) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1241,7 +1257,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_f7c076f1b3c5df9f, []int{18}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{18}
}
func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1298,7 +1314,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_f7c076f1b3c5df9f, []int{19}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{19}
}
func (m *LeafTxnInputState) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1351,7 +1367,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_f7c076f1b3c5df9f, []int{20}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{20}
}
func (m *LeafTxnFinalState) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1390,7 +1406,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_f7c076f1b3c5df9f, []int{21}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{21}
}
func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1427,7 +1443,7 @@ type RangeInfo struct {
func (m *RangeInfo) Reset() { *m = RangeInfo{} }
func (*RangeInfo) ProtoMessage() {}
func (*RangeInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_data_f7c076f1b3c5df9f, []int{22}
+ return fileDescriptor_data_4eed16a3a3304f60, []int{22}
}
func (m *RangeInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1540,6 +1556,9 @@ func (this *MergeTrigger) Equal(that interface{}) bool {
if !this.FreezeStart.Equal(&that1.FreezeStart) {
return false
}
+ if !this.RightReadSummary.Equal(that1.RightReadSummary) {
+ return false
+ }
return true
}
func (this *AbortSpanEntry) Equal(that interface{}) bool {
@@ -1783,6 +1802,16 @@ func (m *MergeTrigger) MarshalTo(dAtA []byte) (int, error) {
return 0, err
}
i += n9
+ if m.RightReadSummary != nil {
+ dAtA[i] = 0x32
+ i++
+ i = encodeVarintData(dAtA, i, uint64(m.RightReadSummary.Size()))
+ n10, err := m.RightReadSummary.MarshalTo(dAtA[i:])
+ if err != nil {
+ return 0, err
+ }
+ i += n10
+ }
return i, nil
}
@@ -1809,11 +1838,11 @@ func (m *ChangeReplicasTrigger) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.DeprecatedReplica.Size()))
- n10, err := m.DeprecatedReplica.MarshalTo(dAtA[i:])
+ n11, err := m.DeprecatedReplica.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n10
+ i += n11
if len(m.DeprecatedUpdatedReplicas) > 0 {
for _, msg := range m.DeprecatedUpdatedReplicas {
dAtA[i] = 0x1a
@@ -1835,11 +1864,11 @@ func (m *ChangeReplicasTrigger) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2a
i++
i = encodeVarintData(dAtA, i, uint64(m.Desc.Size()))
- n11, err := m.Desc.MarshalTo(dAtA[i:])
+ n12, err := m.Desc.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n11
+ i += n12
}
if len(m.InternalAddedReplicas) > 0 {
for _, msg := range m.InternalAddedReplicas {
@@ -1897,11 +1926,11 @@ func (m *ModifiedSpanTrigger) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.NodeLivenessSpan.Size()))
- n12, err := m.NodeLivenessSpan.MarshalTo(dAtA[i:])
+ n13, err := m.NodeLivenessSpan.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n12
+ i += n13
}
return i, nil
}
@@ -1924,11 +1953,11 @@ func (m *StickyBitTrigger) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.StickyBit.Size()))
- n13, err := m.StickyBit.MarshalTo(dAtA[i:])
+ n14, err := m.StickyBit.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n13
+ i += n14
return i, nil
}
@@ -1951,51 +1980,51 @@ func (m *InternalCommitTrigger) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.SplitTrigger.Size()))
- n14, err := m.SplitTrigger.MarshalTo(dAtA[i:])
+ n15, err := m.SplitTrigger.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n14
+ i += n15
}
if m.MergeTrigger != nil {
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.MergeTrigger.Size()))
- n15, err := m.MergeTrigger.MarshalTo(dAtA[i:])
+ n16, err := m.MergeTrigger.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n15
+ i += n16
}
if m.ChangeReplicasTrigger != nil {
dAtA[i] = 0x1a
i++
i = encodeVarintData(dAtA, i, uint64(m.ChangeReplicasTrigger.Size()))
- n16, err := m.ChangeReplicasTrigger.MarshalTo(dAtA[i:])
+ n17, err := m.ChangeReplicasTrigger.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n16
+ i += n17
}
if m.ModifiedSpanTrigger != nil {
dAtA[i] = 0x22
i++
i = encodeVarintData(dAtA, i, uint64(m.ModifiedSpanTrigger.Size()))
- n17, err := m.ModifiedSpanTrigger.MarshalTo(dAtA[i:])
+ n18, err := m.ModifiedSpanTrigger.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n17
+ i += n18
}
if m.StickyBitTrigger != nil {
dAtA[i] = 0x2a
i++
i = encodeVarintData(dAtA, i, uint64(m.StickyBitTrigger.Size()))
- n18, err := m.StickyBitTrigger.MarshalTo(dAtA[i:])
+ n19, err := m.StickyBitTrigger.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n18
+ i += n19
}
return i, nil
}
@@ -2023,11 +2052,11 @@ func (m *ObservedTimestamp) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.Timestamp.Size()))
- n19, err := m.Timestamp.MarshalTo(dAtA[i:])
+ n20, err := m.Timestamp.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n19
+ i += n20
return i, nil
}
@@ -2049,11 +2078,11 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.TxnMeta.Size()))
- n20, err := m.TxnMeta.MarshalTo(dAtA[i:])
+ n21, err := m.TxnMeta.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n20
+ i += n21
if len(m.Name) > 0 {
dAtA[i] = 0x12
i++
@@ -2068,19 +2097,19 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2a
i++
i = encodeVarintData(dAtA, i, uint64(m.LastHeartbeat.Size()))
- n21, err := m.LastHeartbeat.MarshalTo(dAtA[i:])
+ n22, err := m.LastHeartbeat.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n21
+ i += n22
dAtA[i] = 0x3a
i++
i = encodeVarintData(dAtA, i, uint64(m.GlobalUncertaintyLimit.Size()))
- n22, err := m.GlobalUncertaintyLimit.MarshalTo(dAtA[i:])
+ n23, err := m.GlobalUncertaintyLimit.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n22
+ i += n23
if len(m.ObservedTimestamps) > 0 {
for _, msg := range m.ObservedTimestamps {
dAtA[i] = 0x42
@@ -2118,11 +2147,11 @@ func (m *Transaction) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x7a
i++
i = encodeVarintData(dAtA, i, uint64(m.ReadTimestamp.Size()))
- n23, err := m.ReadTimestamp.MarshalTo(dAtA[i:])
+ n24, err := m.ReadTimestamp.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n23
+ i += n24
if m.CommitTimestampFixed {
dAtA[i] = 0x80
i++
@@ -2184,11 +2213,11 @@ func (m *TransactionRecord) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.TxnMeta.Size()))
- n24, err := m.TxnMeta.MarshalTo(dAtA[i:])
+ n25, err := m.TxnMeta.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n24
+ i += n25
if m.Status != 0 {
dAtA[i] = 0x20
i++
@@ -2197,11 +2226,11 @@ func (m *TransactionRecord) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x2a
i++
i = encodeVarintData(dAtA, i, uint64(m.LastHeartbeat.Size()))
- n25, err := m.LastHeartbeat.MarshalTo(dAtA[i:])
+ n26, err := m.LastHeartbeat.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n25
+ i += n26
if len(m.LockSpans) > 0 {
for _, msg := range m.LockSpans {
dAtA[i] = 0x5a
@@ -2263,19 +2292,19 @@ func (m *Intent) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.Intent_SingleKeySpan.Size()))
- n26, err := m.Intent_SingleKeySpan.MarshalTo(dAtA[i:])
+ n27, err := m.Intent_SingleKeySpan.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n26
+ i += n27
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.Txn.Size()))
- n27, err := m.Txn.MarshalTo(dAtA[i:])
+ n28, err := m.Txn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n27
+ i += n28
return i, nil
}
@@ -2321,19 +2350,19 @@ func (m *LockAcquisition) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.Span.Size()))
- n28, err := m.Span.MarshalTo(dAtA[i:])
+ n29, err := m.Span.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n28
+ i += n29
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.Txn.Size()))
- n29, err := m.Txn.MarshalTo(dAtA[i:])
+ n30, err := m.Txn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n29
+ i += n30
if m.Durability != 0 {
dAtA[i] = 0x18
i++
@@ -2360,19 +2389,19 @@ func (m *LockUpdate) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.Span.Size()))
- n30, err := m.Span.MarshalTo(dAtA[i:])
+ n31, err := m.Span.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n30
+ i += n31
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.Txn.Size()))
- n31, err := m.Txn.MarshalTo(dAtA[i:])
+ n32, err := m.Txn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n31
+ i += n32
if m.Status != 0 {
dAtA[i] = 0x18
i++
@@ -2440,48 +2469,48 @@ func (m *Lease) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.Start.Size()))
- n32, err := m.Start.MarshalTo(dAtA[i:])
+ n33, err := m.Start.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n32
+ i += n33
if m.Expiration != nil {
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.Expiration.Size()))
- n33, err := m.Expiration.MarshalTo(dAtA[i:])
+ n34, err := m.Expiration.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n33
+ i += n34
}
dAtA[i] = 0x1a
i++
i = encodeVarintData(dAtA, i, uint64(m.Replica.Size()))
- n34, err := m.Replica.MarshalTo(dAtA[i:])
+ n35, err := m.Replica.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n34
+ i += n35
if m.DeprecatedStartStasis != nil {
dAtA[i] = 0x22
i++
i = encodeVarintData(dAtA, i, uint64(m.DeprecatedStartStasis.Size()))
- n35, err := m.DeprecatedStartStasis.MarshalTo(dAtA[i:])
+ n36, err := m.DeprecatedStartStasis.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n35
+ i += n36
}
if m.ProposedTS != nil {
dAtA[i] = 0x2a
i++
i = encodeVarintData(dAtA, i, uint64(m.ProposedTS.Size()))
- n36, err := m.ProposedTS.MarshalTo(dAtA[i:])
+ n37, err := m.ProposedTS.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n36
+ i += n37
}
if m.Epoch != 0 {
dAtA[i] = 0x30
@@ -2520,11 +2549,11 @@ func (m *AbortSpanEntry) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.Timestamp.Size()))
- n37, err := m.Timestamp.MarshalTo(dAtA[i:])
+ n38, err := m.Timestamp.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n37
+ i += n38
if m.Priority != 0 {
dAtA[i] = 0x18
i++
@@ -2551,11 +2580,11 @@ func (m *LeafTxnInputState) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.Txn.Size()))
- n38, err := m.Txn.MarshalTo(dAtA[i:])
+ n39, err := m.Txn.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n38
+ i += n39
if m.RefreshInvalid {
dAtA[i] = 0x38
i++
@@ -2614,11 +2643,11 @@ func (m *LeafTxnFinalState) 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.DeprecatedCommandCount != 0 {
dAtA[i] = 0x18
i++
@@ -2700,19 +2729,19 @@ func (m *RangeInfo) MarshalTo(dAtA []byte) (int, error) {
dAtA[i] = 0xa
i++
i = encodeVarintData(dAtA, i, uint64(m.Desc.Size()))
- n40, err := m.Desc.MarshalTo(dAtA[i:])
+ n41, err := m.Desc.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n40
+ i += n41
dAtA[i] = 0x12
i++
i = encodeVarintData(dAtA, i, uint64(m.Lease.Size()))
- n41, err := m.Lease.MarshalTo(dAtA[i:])
+ n42, err := m.Lease.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
- i += n41
+ i += n42
if m.ClosedTimestampPolicy != 0 {
dAtA[i] = 0x18
i++
@@ -3072,6 +3101,10 @@ func (m *MergeTrigger) Size() (n int) {
n += 1 + l + sovData(uint64(l))
l = m.FreezeStart.Size()
n += 1 + l + sovData(uint64(l))
+ if m.RightReadSummary != nil {
+ l = m.RightReadSummary.Size()
+ n += 1 + l + sovData(uint64(l))
+ }
return n
}
@@ -4208,6 +4241,39 @@ func (m *MergeTrigger) Unmarshal(dAtA []byte) error {
return err
}
iNdEx = postIndex
+ case 6:
+ 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:])
@@ -7192,170 +7258,173 @@ var (
ErrIntOverflowData = fmt.Errorf("proto: integer overflow")
)
-func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_f7c076f1b3c5df9f) }
+func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_4eed16a3a3304f60) }
-var fileDescriptor_data_f7c076f1b3c5df9f = []byte{
- // 2591 bytes of a gzipped FileDescriptorProto
+var fileDescriptor_data_4eed16a3a3304f60 = []byte{
+ // 2638 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0xcd, 0x73, 0x23, 0x47,
- 0x15, 0xf7, 0x58, 0x23, 0x69, 0xf4, 0x2c, 0xc9, 0xe3, 0x5e, 0xdb, 0xab, 0x75, 0xc0, 0x5a, 0x14,
- 0x20, 0xcb, 0x16, 0x91, 0x0b, 0x27, 0xa4, 0x52, 0xcb, 0x52, 0x85, 0xbe, 0xd6, 0x91, 0x56, 0x96,
- 0x36, 0xa3, 0xf1, 0x06, 0x27, 0x50, 0xc3, 0x68, 0xa6, 0x2d, 0x0f, 0x1e, 0xcd, 0x68, 0x67, 0x46,
- 0x8e, 0xc5, 0x8d, 0x0b, 0x95, 0xca, 0x01, 0x38, 0x72, 0xe0, 0x90, 0x2a, 0x6e, 0x70, 0x80, 0x23,
- 0xc5, 0x81, 0x73, 0x0e, 0x1c, 0x72, 0x4b, 0x80, 0x2a, 0x15, 0x38, 0x97, 0x1c, 0xf8, 0x0b, 0x5c,
- 0x50, 0x45, 0x75, 0xf7, 0x7c, 0x79, 0x57, 0x36, 0x72, 0x76, 0x09, 0x29, 0x2e, 0xf6, 0xcc, 0xfb,
- 0xf8, 0xbd, 0xee, 0xd7, 0xef, 0xbd, 0x7e, 0xf3, 0x04, 0xc8, 0xb1, 0x55, 0xed, 0x70, 0xd4, 0xdf,
- 0xd2, 0x55, 0x4f, 0x2d, 0x8f, 0x1c, 0xdb, 0xb3, 0xd1, 0x8a, 0x66, 0x6b, 0x47, 0x94, 0x5e, 0xf6,
- 0xb9, 0x1b, 0xb7, 0x8f, 0x8e, 0xb7, 0x8e, 0x8e, 0x5d, 0xec, 0x1c, 0x63, 0x67, 0x4b, 0xb3, 0x2d,
- 0x6d, 0xec, 0x38, 0xd8, 0xd2, 0x26, 0x5b, 0xa6, 0xad, 0x1d, 0xd1, 0x3f, 0x86, 0x35, 0x60, 0xea,
- 0x1b, 0xeb, 0x01, 0xe4, 0x10, 0x7b, 0x6a, 0x04, 0xbb, 0xf1, 0x9c, 0xeb, 0xd9, 0x8e, 0x3a, 0xc0,
- 0x5b, 0xd8, 0x1a, 0x18, 0x16, 0x26, 0x02, 0xc7, 0x9a, 0xe6, 0x33, 0xbf, 0x30, 0x93, 0xf9, 0x92,
- 0xcf, 0x2d, 0x8c, 0x3d, 0xc3, 0xdc, 0x3a, 0x34, 0xb5, 0x2d, 0xcf, 0x18, 0x62, 0xd7, 0x53, 0x87,
- 0x23, 0x9f, 0xb3, 0x3a, 0xb0, 0x07, 0x36, 0x7d, 0xdc, 0x22, 0x4f, 0x8c, 0x5a, 0xda, 0x07, 0xbe,
- 0x37, 0x52, 0x2d, 0x74, 0x03, 0x12, 0x47, 0x78, 0x52, 0x48, 0xdc, 0xe4, 0x6e, 0x65, 0xab, 0xe9,
- 0xb3, 0x69, 0x31, 0x71, 0x1f, 0x4f, 0x24, 0x42, 0x43, 0x37, 0x21, 0x8d, 0x2d, 0x5d, 0x21, 0x6c,
- 0xfe, 0x3c, 0x3b, 0x85, 0x2d, 0xfd, 0x3e, 0x9e, 0xdc, 0x11, 0x7e, 0xf1, 0x5e, 0x71, 0xe1, 0xf7,
- 0xef, 0x15, 0xb9, 0x16, 0x2f, 0x70, 0xe2, 0x62, 0x8b, 0x17, 0x16, 0xc5, 0x44, 0x69, 0x00, 0xc9,
- 0x87, 0xaa, 0x39, 0xc6, 0xe8, 0x39, 0xc8, 0x38, 0xea, 0xdb, 0x4a, 0x7f, 0xe2, 0x61, 0xb7, 0xc0,
- 0x11, 0x08, 0x49, 0x70, 0xd4, 0xb7, 0xab, 0xe4, 0x1d, 0x55, 0x20, 0x13, 0xae, 0xb4, 0xb0, 0x78,
- 0x93, 0xbb, 0xb5, 0xb4, 0xfd, 0xc5, 0x72, 0xe4, 0x56, 0xb2, 0x9d, 0xf2, 0xa1, 0xa9, 0x95, 0xe5,
- 0x40, 0xa8, 0xca, 0xbf, 0x3f, 0x2d, 0x2e, 0x48, 0x91, 0x56, 0xe9, 0x2d, 0x10, 0xee, 0xe3, 0x09,
- 0xb3, 0xe5, 0xef, 0x83, 0x9b, 0xb1, 0x8f, 0x97, 0x21, 0x79, 0x4c, 0x64, 0x7c, 0x2b, 0x85, 0xf2,
- 0x13, 0x87, 0x57, 0xa6, 0x18, 0xbe, 0x01, 0x26, 0x5c, 0xfa, 0x90, 0x03, 0xe8, 0x79, 0xb6, 0x83,
- 0x9b, 0x3a, 0xb6, 0x3c, 0x34, 0x00, 0xd0, 0xcc, 0xb1, 0xeb, 0x61, 0x47, 0x31, 0x74, 0xdf, 0xcc,
- 0x6b, 0x44, 0xfe, 0x2f, 0xd3, 0xe2, 0x4b, 0x03, 0xc3, 0x3b, 0x1c, 0xf7, 0xcb, 0x9a, 0x3d, 0xdc,
- 0x0a, 0xb1, 0xf5, 0x7e, 0xf4, 0xbc, 0x35, 0x3a, 0x1a, 0x6c, 0xd1, 0x03, 0x1a, 0x8f, 0x0d, 0xbd,
- 0xbc, 0xb7, 0xd7, 0xac, 0x9f, 0x4e, 0x8b, 0x99, 0x1a, 0x03, 0x6c, 0xd6, 0xa5, 0x8c, 0x8f, 0xdd,
- 0xd4, 0xd1, 0x8b, 0x90, 0xb6, 0x6c, 0x1d, 0x13, 0x2b, 0x64, 0xbd, 0xc9, 0xea, 0xea, 0xe9, 0xb4,
- 0x98, 0xea, 0xd8, 0x3a, 0x6e, 0xd6, 0xcf, 0xc2, 0x27, 0x29, 0x45, 0x84, 0x9a, 0x3a, 0xfa, 0x06,
- 0x08, 0x24, 0x2e, 0xa8, 0x7c, 0x82, 0xca, 0xaf, 0x9f, 0x4e, 0x8b, 0x69, 0xb6, 0x72, 0xa2, 0x10,
- 0x3c, 0x4a, 0x69, 0x97, 0xed, 0xa6, 0xf4, 0x6b, 0x0e, 0xb2, 0xbd, 0x91, 0x69, 0x78, 0xb2, 0x63,
- 0x0c, 0x06, 0xd8, 0x41, 0x0d, 0xc8, 0x98, 0xf8, 0xc0, 0x53, 0x74, 0xec, 0x6a, 0x74, 0x6b, 0x4b,
- 0xdb, 0xa5, 0x19, 0x4e, 0x92, 0x54, 0x6b, 0x80, 0xeb, 0xd8, 0xd5, 0x1c, 0x63, 0xe4, 0xd9, 0x8e,
- 0xef, 0x2e, 0x81, 0xa8, 0x12, 0x2a, 0xda, 0x01, 0x70, 0x8c, 0xc1, 0xa1, 0x8f, 0xb3, 0x78, 0x45,
- 0x9c, 0x0c, 0xd5, 0x25, 0xe4, 0x3b, 0xfc, 0x27, 0x2c, 0xa4, 0x12, 0x22, 0x5f, 0xfa, 0x65, 0x02,
- 0xb2, 0xbb, 0xd8, 0x19, 0xe0, 0xcf, 0xe9, 0x62, 0xd1, 0x00, 0x44, 0x06, 0x44, 0xb2, 0x51, 0x71,
- 0x3d, 0xd5, 0x73, 0x69, 0xba, 0x2c, 0x6d, 0x7f, 0x25, 0x06, 0xe7, 0xe7, 0x6e, 0x39, 0xc8, 0xdd,
- 0xf2, 0xee, 0xc3, 0x5a, 0xad, 0x47, 0x84, 0xab, 0xeb, 0x04, 0xf1, 0x74, 0x5a, 0xcc, 0x4b, 0x04,
- 0x26, 0xa4, 0x4b, 0x79, 0x0a, 0xbb, 0x7b, 0xac, 0x69, 0xf4, 0x1d, 0xfd, 0x84, 0x83, 0xec, 0x81,
- 0x83, 0xf1, 0x8f, 0x30, 0xb1, 0xe2, 0x78, 0x85, 0xe4, 0x3c, 0x49, 0x53, 0x27, 0xe8, 0x67, 0xd3,
- 0xe2, 0xdd, 0xf9, 0x63, 0x94, 0x00, 0xd4, 0x48, 0xc1, 0x0a, 0x51, 0xa4, 0x25, 0x66, 0xb8, 0x47,
- 0xec, 0x9e, 0x3b, 0x9e, 0xdf, 0x24, 0x61, 0xad, 0x76, 0x48, 0x7c, 0x24, 0xe1, 0x91, 0x69, 0x68,
- 0xaa, 0x1b, 0x9c, 0xd3, 0x9b, 0xb0, 0xae, 0xe3, 0x91, 0x83, 0x35, 0xd5, 0xc3, 0xba, 0xa2, 0x51,
- 0x19, 0xc5, 0x9b, 0x8c, 0x30, 0x3d, 0xb4, 0xfc, 0xf6, 0x97, 0x67, 0x39, 0x9b, 0x61, 0x30, 0x40,
- 0x79, 0x32, 0xc2, 0xd2, 0x6a, 0x84, 0x11, 0x51, 0xd1, 0x3e, 0xa0, 0x18, 0xb6, 0xc3, 0xb4, 0xfc,
- 0x43, 0xbc, 0x04, 0xf7, 0x89, 0x63, 0x5c, 0x89, 0x50, 0x7c, 0x11, 0xf4, 0x43, 0x78, 0x2e, 0x06,
- 0x3d, 0x1e, 0xe9, 0x71, 0x13, 0x6e, 0x21, 0x71, 0x33, 0x71, 0x45, 0x1b, 0x37, 0x22, 0xb8, 0x3d,
- 0x86, 0x16, 0x78, 0x0a, 0x61, 0xd8, 0x88, 0xd9, 0xb2, 0xf0, 0x89, 0x17, 0x18, 0x22, 0xd9, 0xcc,
- 0xd3, 0x6c, 0xbe, 0x75, 0x3a, 0x2d, 0x5e, 0xaf, 0x87, 0x52, 0x1d, 0x7c, 0xe2, 0xf9, 0xfa, 0x34,
- 0xbb, 0x33, 0xe1, 0x8b, 0x74, 0x5d, 0x9f, 0x29, 0xa5, 0xa3, 0x57, 0x80, 0xa7, 0x41, 0x9e, 0x9c,
- 0x37, 0xc8, 0x25, 0x2a, 0x8f, 0xfa, 0x70, 0xdd, 0xb0, 0x3c, 0xec, 0x58, 0xaa, 0xa9, 0xa8, 0xba,
- 0x1e, 0x77, 0x43, 0xea, 0xca, 0x6e, 0x58, 0x0b, 0xa0, 0x2a, 0x04, 0x29, 0x74, 0xc1, 0x01, 0xdc,
- 0x08, 0x6d, 0x38, 0x78, 0x68, 0x1f, 0xc7, 0xad, 0xa4, 0xaf, 0x6c, 0x25, 0x5c, 0xb0, 0xc4, 0xb0,
- 0x02, 0x3b, 0x77, 0x78, 0x72, 0x53, 0x95, 0xde, 0xe5, 0xe0, 0xda, 0xae, 0xad, 0x1b, 0x07, 0x06,
- 0xd6, 0xc9, 0xed, 0x17, 0xc4, 0xea, 0xd7, 0x01, 0xb9, 0x13, 0xd7, 0xc3, 0x43, 0x45, 0xb3, 0xad,
- 0x03, 0x63, 0xa0, 0xb8, 0x23, 0xd5, 0xa2, 0x71, 0x2a, 0x48, 0x22, 0xe3, 0xd4, 0x28, 0x83, 0x5e,
- 0x99, 0x0d, 0x40, 0xb4, 0x42, 0x9b, 0xc6, 0x31, 0xb6, 0xb0, 0xeb, 0x32, 0x69, 0x16, 0x7d, 0xd7,
- 0x67, 0x2c, 0x96, 0x28, 0x49, 0x22, 0x51, 0x69, 0xfb, 0x1a, 0x84, 0x52, 0x7a, 0x08, 0x62, 0xcf,
- 0x33, 0xb4, 0xa3, 0x49, 0x35, 0xaa, 0xc4, 0x55, 0x00, 0x97, 0xd2, 0x94, 0xbe, 0xe1, 0xf9, 0xd5,
- 0x6d, 0xbe, 0x5b, 0xd1, 0x0d, 0xa0, 0x4a, 0x7f, 0x48, 0xc0, 0x5a, 0xd3, 0x77, 0x43, 0xcd, 0x1e,
- 0x0e, 0x23, 0xf4, 0x3a, 0xe4, 0x5c, 0x52, 0xf7, 0x15, 0x8f, 0x11, 0x7c, 0x03, 0xc5, 0x99, 0x6b,
- 0x8e, 0xee, 0x07, 0x29, 0xeb, 0xc6, 0x6f, 0x8b, 0x3a, 0xe4, 0x86, 0xa4, 0x20, 0x87, 0x28, 0x8b,
- 0x17, 0xa2, 0xc4, 0x0b, 0xb7, 0x94, 0x1d, 0xc6, 0xcb, 0xf8, 0x0f, 0xe0, 0xba, 0x5f, 0x13, 0x82,
- 0xe3, 0x0e, 0xf1, 0x12, 0x14, 0xef, 0xd6, 0x0c, 0xbc, 0x99, 0x95, 0x46, 0x5a, 0xd3, 0x2e, 0x28,
- 0x40, 0x6b, 0x43, 0xff, 0xac, 0xe9, 0x09, 0x85, 0xf8, 0xac, 0x3a, 0x7f, 0x75, 0xd6, 0x7a, 0x9f,
- 0x8c, 0x0d, 0xe9, 0xda, 0x70, 0x46, 0xc0, 0xbc, 0x0e, 0x28, 0x3a, 0xa7, 0x10, 0x98, 0x25, 0xd8,
- 0xf3, 0xb3, 0xdc, 0xf9, 0xd8, 0x41, 0x4b, 0xa2, 0xfb, 0x18, 0xe5, 0x0e, 0xff, 0xce, 0x7b, 0x45,
- 0xae, 0xf4, 0x27, 0x0e, 0x56, 0xba, 0x7d, 0xda, 0x45, 0xea, 0xe1, 0x19, 0xc7, 0x7b, 0x02, 0x6e,
- 0x8e, 0x9e, 0xe0, 0xc7, 0xdc, 0x95, 0x7b, 0xab, 0x67, 0x73, 0x4d, 0x44, 0x56, 0xef, 0xf0, 0xa4,
- 0x2d, 0x2c, 0x9d, 0xa5, 0x60, 0x49, 0x76, 0x54, 0xcb, 0x55, 0x35, 0xcf, 0xb0, 0x2d, 0x54, 0x01,
- 0x9e, 0xb4, 0xbc, 0x7e, 0xe0, 0x3d, 0x7f, 0xd9, 0x05, 0x29, 0x9f, 0x58, 0xbb, 0xd8, 0x53, 0xab,
- 0x02, 0x59, 0xd9, 0x07, 0xd3, 0x22, 0x27, 0x51, 0x55, 0x84, 0x80, 0xb7, 0xd4, 0x21, 0x6b, 0xe6,
- 0x32, 0x12, 0x7d, 0x46, 0x77, 0x21, 0x45, 0x2e, 0xde, 0x31, 0xbb, 0x79, 0x67, 0xdf, 0x2d, 0xb1,
- 0x65, 0xf4, 0xa8, 0xac, 0xe4, 0xeb, 0xa0, 0x16, 0xe4, 0x4d, 0xd5, 0xf5, 0x94, 0x43, 0xac, 0x3a,
- 0x5e, 0x1f, 0xab, 0x73, 0xde, 0xac, 0x2c, 0xf1, 0x72, 0x44, 0xf5, 0xb5, 0x40, 0x13, 0x7d, 0x1f,
- 0x0a, 0x03, 0xd3, 0xee, 0xab, 0xa6, 0x32, 0xb6, 0x34, 0xec, 0x78, 0xaa, 0x61, 0x79, 0x13, 0xc5,
- 0x34, 0x86, 0x86, 0x57, 0x48, 0xcf, 0x8f, 0xba, 0xce, 0x40, 0xf6, 0x22, 0x8c, 0x36, 0x81, 0x40,
- 0x6f, 0xc1, 0x35, 0xdb, 0x8f, 0x0e, 0x25, 0xf4, 0xb5, 0x5b, 0x10, 0x2e, 0x2c, 0x94, 0x4f, 0xc4,
- 0x92, 0x6f, 0x00, 0xd9, 0x8f, 0x33, 0x5c, 0x74, 0x17, 0x80, 0x1c, 0x27, 0x4d, 0x16, 0xb7, 0xb0,
- 0x44, 0x31, 0x2f, 0xaa, 0x67, 0x41, 0xd9, 0x21, 0x0a, 0xe4, 0xdd, 0x45, 0x25, 0xc8, 0xbd, 0xed,
- 0x18, 0x1e, 0x56, 0x3c, 0xdb, 0x56, 0x6c, 0x53, 0x2f, 0x64, 0x69, 0xf9, 0x5c, 0xa2, 0x44, 0xd9,
- 0xb6, 0xbb, 0xa6, 0x4e, 0x3c, 0xed, 0x60, 0x35, 0xb6, 0xf4, 0xc2, 0xf2, 0x15, 0x3c, 0x4d, 0x54,
- 0xa3, 0x9c, 0x78, 0x19, 0xd6, 0x35, 0x5a, 0xdd, 0x22, 0x34, 0xe5, 0xc0, 0x38, 0xc1, 0x7a, 0x41,
- 0xa4, 0x86, 0x57, 0x19, 0x37, 0x54, 0xb8, 0x47, 0x78, 0xe8, 0x75, 0x10, 0x0d, 0x4b, 0x39, 0x30,
- 0x69, 0xc7, 0x46, 0x97, 0xe6, 0x16, 0x56, 0xe8, 0x4e, 0xbf, 0x34, 0x6b, 0xa7, 0xf8, 0xd1, 0x18,
- 0x5b, 0x1a, 0xd6, 0xdf, 0x20, 0x92, 0xfe, 0x3a, 0xf2, 0x86, 0x75, 0x8f, 0xea, 0x53, 0xa2, 0x8b,
- 0x6c, 0x58, 0x36, 0x06, 0x96, 0xed, 0x90, 0x32, 0x83, 0x1f, 0x59, 0xe3, 0xa1, 0x5b, 0x40, 0x14,
- 0xb1, 0x7c, 0x59, 0x78, 0x37, 0x99, 0x4a, 0x0f, 0x3f, 0xea, 0x8c, 0x87, 0xf4, 0xfe, 0x8d, 0x1a,
- 0xc1, 0x73, 0x3c, 0x57, 0xca, 0x1b, 0xe1, 0x3b, 0x41, 0x3f, 0xf7, 0xd5, 0x95, 0x10, 0xf9, 0x16,
- 0x2f, 0xa4, 0xc4, 0x74, 0x8b, 0x17, 0x32, 0x22, 0xb4, 0x78, 0x21, 0x27, 0xe6, 0x5b, 0xbc, 0x90,
- 0x17, 0x97, 0x4b, 0xbf, 0xe3, 0x61, 0x25, 0x16, 0xf5, 0x12, 0xd6, 0x6c, 0x47, 0x7f, 0x16, 0x29,
- 0xf8, 0xf9, 0x49, 0xb7, 0xa7, 0x0b, 0xd9, 0xff, 0x87, 0x60, 0xe0, 0xfd, 0x40, 0x58, 0x14, 0x13,
- 0x8f, 0x85, 0x43, 0x5a, 0x14, 0x5a, 0xbc, 0x20, 0x88, 0x99, 0x30, 0x34, 0x40, 0x5c, 0x6a, 0xf1,
- 0x42, 0x56, 0xcc, 0xc5, 0xc3, 0xa4, 0xc5, 0x0b, 0xcb, 0xa2, 0xd8, 0xe2, 0x05, 0x51, 0x5c, 0x29,
- 0x4d, 0x39, 0x48, 0x91, 0xde, 0xc1, 0xf2, 0xd0, 0x3e, 0x2c, 0xbb, 0x86, 0x35, 0x30, 0xb1, 0x72,
- 0x84, 0x27, 0x51, 0x43, 0xb4, 0xb4, 0xfd, 0xc2, 0x0c, 0xdf, 0x30, 0x9d, 0x72, 0x8f, 0x2a, 0xdc,
- 0xc7, 0x13, 0xea, 0xef, 0x28, 0x6c, 0x72, 0x6e, 0x9c, 0x81, 0xbe, 0x05, 0x09, 0xef, 0x24, 0xe8,
- 0x98, 0xe6, 0x8a, 0x40, 0xe6, 0x6c, 0xa2, 0xb5, 0x71, 0x17, 0x72, 0xe7, 0xcc, 0x5c, 0x32, 0xc1,
- 0x88, 0x4f, 0x25, 0x5a, 0xbc, 0xc0, 0x8b, 0xc9, 0xd2, 0x9f, 0x39, 0x58, 0x6e, 0xdb, 0xda, 0x51,
- 0x45, 0x7b, 0x34, 0x36, 0x5c, 0x83, 0x5e, 0x4a, 0xdf, 0x04, 0x3e, 0xb6, 0xbd, 0x0b, 0xc3, 0x27,
- 0x96, 0x05, 0xee, 0xd3, 0xee, 0x02, 0x49, 0x00, 0xfa, 0xd8, 0x51, 0xfb, 0x86, 0x69, 0x78, 0x6c,
- 0xed, 0xf9, 0xed, 0xed, 0x18, 0xc6, 0xd1, 0x71, 0x39, 0x18, 0x26, 0x95, 0x63, 0xc3, 0xa4, 0x32,
- 0x09, 0xde, 0x72, 0x3d, 0xd4, 0x94, 0x62, 0x28, 0xa5, 0xdf, 0x2e, 0x02, 0x90, 0xbd, 0xb1, 0xcf,
- 0x8c, 0xff, 0xc9, 0xb6, 0xa2, 0xca, 0x90, 0xf8, 0x14, 0x95, 0x61, 0x46, 0xf2, 0xf0, 0xff, 0xcd,
- 0xe4, 0x29, 0xfd, 0x8c, 0x83, 0xfc, 0xf9, 0xb4, 0xbe, 0x6c, 0x8e, 0xf4, 0x3d, 0x10, 0x5c, 0x5f,
- 0xd8, 0x1f, 0xcd, 0x7c, 0x67, 0xee, 0x8e, 0xe9, 0xf1, 0xd9, 0x1d, 0x71, 0x5e, 0x0f, 0x3f, 0x92,
- 0x42, 0x44, 0xbf, 0x61, 0xfa, 0x23, 0x0f, 0xc9, 0x36, 0x56, 0x5d, 0x8c, 0x26, 0x90, 0x64, 0x9f,
- 0xf9, 0xdc, 0x67, 0xd7, 0xbf, 0x31, 0x8b, 0xe8, 0xdb, 0x00, 0xf8, 0x64, 0x64, 0x38, 0x2a, 0x39,
- 0xa3, 0xb9, 0xfa, 0x47, 0x29, 0xa6, 0x80, 0xea, 0x90, 0x0e, 0x3e, 0xc9, 0x13, 0x57, 0xfe, 0x24,
- 0x0f, 0x54, 0xd1, 0x1e, 0xc4, 0x3e, 0x68, 0xd9, 0xc4, 0x83, 0xfc, 0x75, 0x8d, 0x60, 0xbc, 0xf2,
- 0x1f, 0x56, 0xb4, 0x16, 0x69, 0xd3, 0xb1, 0x45, 0x8f, 0xea, 0xa2, 0x9f, 0x72, 0xb0, 0x34, 0x72,
- 0xec, 0x91, 0xed, 0x92, 0x16, 0xca, 0x9d, 0xef, 0xee, 0xe9, 0x9c, 0x4e, 0x8b, 0xf0, 0xc0, 0xd7,
- 0x92, 0x7b, 0x4f, 0xed, 0x67, 0x08, 0x56, 0x20, 0xbb, 0x68, 0x15, 0x92, 0x78, 0x64, 0x6b, 0x87,
- 0x85, 0xd4, 0x4d, 0xee, 0x56, 0x42, 0x62, 0x2f, 0xe8, 0xc5, 0x58, 0xac, 0x91, 0xbe, 0x31, 0x51,
- 0x5d, 0x39, 0x9b, 0x16, 0x73, 0x34, 0x34, 0x82, 0x88, 0x8d, 0x05, 0x4f, 0xd8, 0x12, 0x94, 0xfe,
- 0xca, 0x41, 0xbe, 0xd2, 0xb7, 0x1d, 0x8f, 0x24, 0x77, 0xc3, 0xf2, 0x9c, 0xc9, 0x65, 0x21, 0xfd,
- 0xf4, 0x43, 0x58, 0xa4, 0x82, 0x30, 0x72, 0x0c, 0xdb, 0x09, 0xea, 0x58, 0xb2, 0xda, 0x38, 0x9b,
- 0x16, 0x2b, 0x9f, 0x3a, 0x2b, 0x1e, 0xf8, 0x60, 0x52, 0x08, 0x7b, 0x47, 0x20, 0x3b, 0xfb, 0x84,
- 0xec, 0xee, 0x5f, 0x8b, 0xb0, 0xd2, 0xc6, 0xea, 0x81, 0x7c, 0x62, 0x35, 0xad, 0xd1, 0x98, 0x1c,
- 0xaa, 0x87, 0xd1, 0x2b, 0xac, 0x64, 0xb1, 0x44, 0xd9, 0xbc, 0xbc, 0xe4, 0xc4, 0xab, 0xd5, 0x0b,
- 0xb0, 0xec, 0xe0, 0x03, 0x07, 0xbb, 0x87, 0x8a, 0x61, 0x1d, 0xab, 0xa6, 0xa1, 0x53, 0x5f, 0x0b,
- 0x52, 0xde, 0x27, 0x37, 0x19, 0x75, 0x66, 0xa3, 0x20, 0x3c, 0x5d, 0xa3, 0xb0, 0x0d, 0x6b, 0xae,
- 0x87, 0x47, 0x23, 0xc3, 0x1a, 0x28, 0x43, 0xf2, 0x6d, 0x87, 0x2d, 0xb5, 0x6f, 0x62, 0xbd, 0x90,
- 0xa1, 0x2b, 0xb8, 0x16, 0x30, 0x77, 0x6d, 0x1d, 0x37, 0x18, 0x0b, 0xf5, 0x21, 0x4b, 0xdb, 0x67,
- 0x17, 0x3f, 0x52, 0xac, 0xf1, 0xb0, 0x00, 0xcf, 0xa8, 0x08, 0x01, 0x41, 0x65, 0x35, 0xf2, 0xb1,
- 0x4e, 0x82, 0x17, 0x93, 0x2d, 0x5e, 0x48, 0x8a, 0x29, 0xd6, 0x55, 0x94, 0xde, 0x8d, 0xfc, 0x7f,
- 0xcf, 0xb0, 0x54, 0xf3, 0xe9, 0xfc, 0xff, 0x2a, 0x14, 0xe2, 0x23, 0x42, 0x7b, 0x38, 0x54, 0x2d,
- 0xf2, 0x7f, 0x6c, 0x79, 0x2c, 0x94, 0xa4, 0xd8, 0x08, 0xb1, 0xc6, 0xd8, 0x35, 0xc2, 0x45, 0x55,
- 0xc8, 0x05, 0x27, 0xc7, 0x5a, 0x3f, 0x7e, 0x9e, 0xd6, 0x2f, 0xeb, 0xeb, 0xb0, 0xee, 0x6f, 0xde,
- 0xd3, 0x0f, 0x5d, 0x12, 0xba, 0x81, 0xb5, 0x55, 0xa5, 0x7f, 0x72, 0xb0, 0x5c, 0x33, 0x0d, 0x6c,
- 0x79, 0xf4, 0xd6, 0x69, 0x5a, 0x07, 0x36, 0x7a, 0x0d, 0xd6, 0xf4, 0xb0, 0xa4, 0x29, 0x03, 0x6c,
- 0x61, 0xbf, 0x8a, 0x72, 0x34, 0x89, 0xaf, 0x9d, 0x4d, 0x8b, 0xcb, 0x54, 0x7a, 0x27, 0x64, 0x49,
- 0xab, 0x91, 0x46, 0x44, 0x45, 0xaf, 0x42, 0xde, 0x24, 0xd9, 0xae, 0x9c, 0xbb, 0x73, 0x66, 0xd6,
- 0x81, 0x9c, 0x19, 0x7f, 0x45, 0x18, 0xae, 0x6b, 0x26, 0xab, 0x6f, 0xe1, 0x97, 0xd1, 0xc8, 0x36,
- 0x0d, 0x2d, 0x68, 0x34, 0x5e, 0xbc, 0x68, 0x04, 0x58, 0xa3, 0x6a, 0x61, 0xb2, 0x3f, 0xa0, 0x4a,
- 0xd2, 0x9a, 0x36, 0x8b, 0x5c, 0xfa, 0x07, 0x07, 0x99, 0x68, 0xe3, 0x77, 0xfd, 0x21, 0xe3, 0x55,
- 0x27, 0xf2, 0x6c, 0xd4, 0xf8, 0x32, 0x24, 0xe9, 0x1e, 0x2e, 0xf9, 0x89, 0x86, 0x6e, 0x39, 0xf8,
- 0x89, 0x86, 0x0a, 0x7f, 0x46, 0x1b, 0x65, 0xb3, 0xc3, 0xdb, 0x1f, 0x72, 0x90, 0xa1, 0x3f, 0x13,
- 0xd1, 0x09, 0xf4, 0x12, 0xa4, 0xf7, 0x3a, 0xf7, 0x3b, 0xdd, 0x37, 0x3a, 0xe2, 0x02, 0x4a, 0x43,
- 0xa2, 0xd9, 0x91, 0x45, 0x0e, 0x65, 0x20, 0x79, 0xaf, 0xdd, 0xad, 0xc8, 0xe2, 0x22, 0x79, 0xac,
- 0xee, 0xcb, 0x8d, 0x9e, 0x98, 0x40, 0xd7, 0x60, 0xb9, 0xde, 0x68, 0x37, 0x77, 0x9b, 0x72, 0xa3,
- 0xae, 0x30, 0xa2, 0x80, 0x04, 0xe0, 0xe5, 0xe6, 0x6e, 0x43, 0xe4, 0x09, 0x54, 0xbd, 0x51, 0x6b,
- 0xee, 0x56, 0xda, 0x62, 0x12, 0xad, 0xc1, 0x4a, 0x24, 0x1b, 0x90, 0x33, 0x28, 0x0b, 0x42, 0x7d,
- 0x4f, 0xaa, 0xc8, 0xcd, 0x6e, 0x47, 0x4c, 0x21, 0x80, 0x14, 0xd1, 0x95, 0xdf, 0x14, 0xb3, 0xc4,
- 0xf6, 0x4e, 0xa3, 0x2b, 0xe6, 0xa8, 0xc1, 0xee, 0x77, 0xb7, 0xeb, 0x62, 0x9e, 0x3c, 0xca, 0x7b,
- 0x0f, 0xda, 0x0d, 0x11, 0x88, 0x62, 0xb5, 0x29, 0x57, 0x24, 0xa9, 0xb2, 0x2f, 0x2e, 0xa1, 0x3c,
- 0x00, 0x51, 0xec, 0x35, 0xa4, 0x66, 0xa3, 0x27, 0xea, 0x25, 0xf2, 0x91, 0x90, 0xbe, 0xad, 0xc1,
- 0xca, 0x13, 0x83, 0x77, 0x94, 0x83, 0x4c, 0xa5, 0x5e, 0x57, 0x1e, 0x76, 0xe5, 0x86, 0x24, 0x2e,
- 0x20, 0x11, 0xb2, 0x52, 0x63, 0xb7, 0xfb, 0xb0, 0xe1, 0x53, 0x38, 0xb4, 0x02, 0x39, 0x22, 0xd0,
- 0xe9, 0x76, 0x7c, 0xd2, 0x22, 0x5a, 0x05, 0xd1, 0x17, 0x8a, 0xa8, 0x89, 0x0d, 0xfe, 0x9d, 0x5f,
- 0x6d, 0x2e, 0xdc, 0xee, 0x9e, 0xfb, 0x16, 0x65, 0x8d, 0x1f, 0xd9, 0xfa, 0x83, 0x46, 0xa7, 0xde,
- 0xec, 0xec, 0x88, 0x0b, 0xe4, 0xa5, 0x27, 0x57, 0x76, 0xc8, 0x4b, 0x82, 0x98, 0xaf, 0x75, 0x77,
- 0x77, 0x9b, 0xb2, 0xdc, 0xa8, 0x8b, 0x1c, 0xe1, 0x55, 0xaa, 0x5d, 0x89, 0xbc, 0x2c, 0xfa, 0x80,
- 0xfb, 0xb0, 0x71, 0xf1, 0x51, 0xa2, 0x0d, 0x58, 0x6f, 0x57, 0x76, 0x94, 0xea, 0xbe, 0x52, 0x6b,
- 0xef, 0xf5, 0xe4, 0x86, 0xa4, 0xf4, 0x1a, 0xb2, 0xcc, 0x0c, 0xdd, 0x80, 0xb5, 0x76, 0xa3, 0x52,
- 0x57, 0xee, 0x75, 0x25, 0x65, 0xa7, 0xdd, 0xad, 0x56, 0xda, 0x8a, 0xd4, 0xa8, 0xd4, 0x7b, 0x22,
- 0xc7, 0xa0, 0xab, 0x5f, 0x7b, 0xff, 0xef, 0x9b, 0x0b, 0xef, 0x9f, 0x6e, 0x72, 0x1f, 0x9c, 0x6e,
- 0x72, 0x1f, 0x9d, 0x6e, 0x72, 0x7f, 0x3b, 0xdd, 0xe4, 0x7e, 0xfe, 0xf1, 0xe6, 0xc2, 0x07, 0x1f,
- 0x6f, 0x2e, 0x7c, 0xf4, 0xf1, 0xe6, 0xc2, 0x9b, 0x69, 0x3f, 0x9a, 0xfa, 0x29, 0xfa, 0x6b, 0xea,
- 0x4b, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x77, 0x72, 0x88, 0x5c, 0x25, 0x1e, 0x00, 0x00,
+ 0x15, 0xf7, 0x58, 0x23, 0x6b, 0xf4, 0x2c, 0xc9, 0xe3, 0x5e, 0xdb, 0xab, 0x75, 0xc0, 0x5a, 0x14,
+ 0x20, 0xcb, 0x16, 0x91, 0x0a, 0x27, 0xa4, 0x52, 0xcb, 0x52, 0x85, 0xbe, 0xd6, 0x91, 0x56, 0x96,
+ 0x36, 0xa3, 0xf1, 0x06, 0x27, 0xa1, 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, 0xf8, 0x0b, 0x5c, 0x50,
+ 0x45, 0x75, 0xf7, 0x7c, 0x79, 0x57, 0x36, 0x72, 0x76, 0x09, 0x29, 0x2e, 0xf6, 0xf4, 0xeb, 0xf7,
+ 0x7e, 0xaf, 0xfb, 0xf5, 0x7b, 0xaf, 0x5f, 0x3f, 0x01, 0x72, 0x6c, 0x55, 0x3b, 0x1c, 0xf5, 0xcb,
+ 0xba, 0xea, 0xa9, 0xa5, 0x91, 0x63, 0x7b, 0x36, 0x5a, 0xd5, 0x6c, 0xed, 0x88, 0xd2, 0x4b, 0xfe,
+ 0xec, 0xe6, 0xed, 0xa3, 0xe3, 0xf2, 0xd1, 0xb1, 0x8b, 0x9d, 0x63, 0xec, 0x94, 0x35, 0xdb, 0xd2,
+ 0xc6, 0x8e, 0x83, 0x2d, 0x6d, 0x52, 0x36, 0x6d, 0xed, 0x88, 0xfe, 0x31, 0xac, 0x01, 0x13, 0x3f,
+ 0xcf, 0xeb, 0x60, 0x55, 0x77, 0xc7, 0xc3, 0xa1, 0xea, 0x4c, 0xca, 0x8e, 0x3b, 0xea, 0x97, 0xfd,
+ 0x81, 0xcf, 0xbb, 0x11, 0xa8, 0x1f, 0x62, 0x4f, 0x8d, 0x96, 0xb0, 0xf9, 0x9c, 0xeb, 0xd9, 0x8e,
+ 0x3a, 0xc0, 0x65, 0x6c, 0x0d, 0x0c, 0x0b, 0x13, 0x86, 0x63, 0x4d, 0xf3, 0x27, 0xbf, 0x34, 0x73,
+ 0xf2, 0x25, 0x7f, 0x36, 0x3f, 0xf6, 0x0c, 0xb3, 0x7c, 0x68, 0x6a, 0x65, 0xcf, 0x18, 0x62, 0xd7,
+ 0x53, 0x87, 0x23, 0x7f, 0x66, 0x6d, 0x60, 0x0f, 0x6c, 0xfa, 0x59, 0x26, 0x5f, 0x8c, 0x5a, 0xdc,
+ 0x07, 0xbe, 0x37, 0x52, 0x2d, 0x74, 0x03, 0x12, 0x47, 0x78, 0x92, 0x4f, 0xdc, 0xe4, 0x6e, 0x65,
+ 0xaa, 0xa9, 0xb3, 0x69, 0x21, 0x71, 0x1f, 0x4f, 0x24, 0x42, 0x43, 0x37, 0x21, 0x85, 0x2d, 0x5d,
+ 0x21, 0xd3, 0xfc, 0xf9, 0xe9, 0x25, 0x6c, 0xe9, 0xf7, 0xf1, 0xe4, 0x8e, 0xf0, 0xcb, 0xf7, 0x0b,
+ 0x0b, 0xbf, 0x7f, 0xbf, 0xc0, 0xb5, 0x78, 0x81, 0x13, 0x17, 0x5b, 0xbc, 0xb0, 0x28, 0x26, 0x8a,
+ 0x03, 0x48, 0x3e, 0x54, 0xcd, 0x31, 0x46, 0xcf, 0x41, 0xda, 0x51, 0xdf, 0x51, 0xfa, 0x13, 0x0f,
+ 0xbb, 0x79, 0x8e, 0x40, 0x48, 0x82, 0xa3, 0xbe, 0x53, 0x25, 0x63, 0x54, 0x81, 0x74, 0xb8, 0xd2,
+ 0xfc, 0xe2, 0x4d, 0xee, 0xd6, 0xf2, 0xf6, 0x97, 0x4b, 0xd1, 0x11, 0x90, 0xed, 0x94, 0x0e, 0x4d,
+ 0xad, 0x24, 0x07, 0x4c, 0x55, 0xfe, 0x83, 0x69, 0x61, 0x41, 0x8a, 0xa4, 0x8a, 0x6f, 0x81, 0x70,
+ 0x1f, 0x4f, 0x98, 0x2e, 0x7f, 0x1f, 0xdc, 0x8c, 0x7d, 0xbc, 0x0c, 0xc9, 0x63, 0xc2, 0xe3, 0x6b,
+ 0xc9, 0x97, 0x9e, 0x38, 0xe8, 0x12, 0xc5, 0xf0, 0x15, 0x30, 0xe6, 0xe2, 0x47, 0x1c, 0x40, 0xcf,
+ 0xb3, 0x1d, 0xdc, 0xd4, 0xb1, 0xe5, 0xa1, 0x01, 0x80, 0x66, 0x8e, 0x5d, 0x0f, 0x3b, 0x8a, 0xa1,
+ 0xfb, 0x6a, 0x5e, 0x23, 0xfc, 0x7f, 0x99, 0x16, 0x5e, 0x1a, 0x18, 0xde, 0xe1, 0xb8, 0x5f, 0xd2,
+ 0xec, 0x61, 0x39, 0xc4, 0xd6, 0xfb, 0xd1, 0x77, 0x79, 0x74, 0x34, 0x28, 0xd3, 0x03, 0x1a, 0x8f,
+ 0x0d, 0xbd, 0xb4, 0xb7, 0xd7, 0xac, 0x9f, 0x4e, 0x0b, 0xe9, 0x1a, 0x03, 0x6c, 0xd6, 0xa5, 0xb4,
+ 0x8f, 0xdd, 0xd4, 0xd1, 0x8b, 0x90, 0xb2, 0x6c, 0x1d, 0x13, 0x2d, 0x64, 0xbd, 0xc9, 0xea, 0xda,
+ 0xe9, 0xb4, 0xb0, 0xd4, 0xb1, 0x75, 0xdc, 0xac, 0x9f, 0x85, 0x5f, 0xd2, 0x12, 0x61, 0x6a, 0xea,
+ 0xe8, 0x5b, 0x20, 0x10, 0xbf, 0xa0, 0xfc, 0x09, 0xca, 0xbf, 0x71, 0x3a, 0x2d, 0xa4, 0xd8, 0xca,
+ 0x89, 0x40, 0xf0, 0x29, 0xa5, 0x5c, 0xb6, 0x9b, 0xe2, 0xaf, 0x39, 0xc8, 0xf4, 0x46, 0xa6, 0xe1,
+ 0xc9, 0x8e, 0x31, 0x18, 0x60, 0x07, 0x35, 0x20, 0x6d, 0xe2, 0x03, 0x4f, 0xd1, 0xb1, 0xab, 0xd1,
+ 0xad, 0x2d, 0x6f, 0x17, 0x67, 0x18, 0x49, 0x52, 0xad, 0x01, 0xae, 0x63, 0x57, 0x73, 0x8c, 0x91,
+ 0x67, 0x3b, 0xbe, 0xb9, 0x04, 0x22, 0x4a, 0xa8, 0x68, 0x07, 0xc0, 0x31, 0x06, 0x87, 0x3e, 0xce,
+ 0xe2, 0x15, 0x71, 0xd2, 0x54, 0x96, 0x90, 0xef, 0xf0, 0x9f, 0x32, 0x97, 0x4a, 0x88, 0x7c, 0xf1,
+ 0x2c, 0x01, 0x99, 0x5d, 0xec, 0x0c, 0xf0, 0x17, 0x74, 0xb1, 0x68, 0x00, 0x22, 0x03, 0x22, 0xd1,
+ 0xa8, 0xb8, 0x9e, 0xea, 0xb9, 0x34, 0x5c, 0x96, 0xb7, 0xbf, 0x16, 0x83, 0xf3, 0x63, 0xb7, 0x14,
+ 0xc4, 0x6e, 0x69, 0xf7, 0x61, 0xad, 0xd6, 0x23, 0xcc, 0xd5, 0x0d, 0x82, 0x78, 0x3a, 0x2d, 0xe4,
+ 0x24, 0x02, 0x13, 0xd2, 0xa5, 0x1c, 0x85, 0xdd, 0x3d, 0xd6, 0x34, 0x3a, 0x46, 0x3f, 0xe5, 0x20,
+ 0x73, 0xe0, 0x60, 0xfc, 0x63, 0x4c, 0xb4, 0x38, 0x5e, 0x3e, 0x39, 0x4f, 0xd0, 0xd4, 0x09, 0xfa,
+ 0xd9, 0xb4, 0x70, 0x77, 0x7e, 0x1f, 0x25, 0x00, 0x35, 0x92, 0xdc, 0x42, 0x14, 0x69, 0x99, 0x29,
+ 0xee, 0x11, 0xbd, 0xe8, 0x6d, 0x40, 0x6c, 0xc7, 0x24, 0xcb, 0x29, 0x7e, 0x66, 0xcb, 0x2f, 0xd1,
+ 0xd5, 0x94, 0x62, 0xab, 0x39, 0x3a, 0x2e, 0x05, 0x09, 0xb1, 0x14, 0x4b, 0x88, 0x25, 0x09, 0xab,
+ 0x7a, 0x8f, 0x7d, 0x4b, 0xcc, 0x76, 0x31, 0xca, 0xb9, 0xc3, 0xff, 0x4d, 0x12, 0xd6, 0x6b, 0x87,
+ 0xe4, 0x04, 0x24, 0x3c, 0x32, 0x0d, 0x4d, 0x75, 0x03, 0x2f, 0x78, 0x13, 0x36, 0x74, 0x3c, 0x72,
+ 0xb0, 0xa6, 0x7a, 0x58, 0x57, 0x34, 0xca, 0xa3, 0x78, 0x93, 0x11, 0xa6, 0x2e, 0x91, 0xdb, 0xfe,
+ 0xea, 0xac, 0xa3, 0x64, 0x18, 0x0c, 0x50, 0x9e, 0x8c, 0xb0, 0xb4, 0x16, 0x61, 0x44, 0x54, 0xb4,
+ 0x0f, 0x28, 0x86, 0xed, 0x30, 0x29, 0xdf, 0x45, 0x2e, 0xc1, 0x7d, 0xc2, 0x49, 0x56, 0x23, 0x14,
+ 0x9f, 0x05, 0xfd, 0x08, 0x9e, 0x8b, 0x41, 0x8f, 0x47, 0x7a, 0x5c, 0x85, 0x9b, 0x4f, 0xdc, 0x4c,
+ 0x5c, 0x51, 0xc7, 0x8d, 0x08, 0x6e, 0x8f, 0xa1, 0x05, 0x96, 0x42, 0x18, 0x36, 0x63, 0xba, 0x2c,
+ 0x7c, 0xe2, 0x05, 0x8a, 0x48, 0xae, 0xe0, 0x69, 0xae, 0xb8, 0x75, 0x3a, 0x2d, 0x5c, 0xaf, 0x87,
+ 0x5c, 0x1d, 0x7c, 0xe2, 0xf9, 0xf2, 0x34, 0x77, 0xa4, 0xc3, 0x81, 0x74, 0x5d, 0x9f, 0xc9, 0xa5,
+ 0xa3, 0x57, 0x80, 0xa7, 0x21, 0x94, 0x9c, 0x37, 0x84, 0x24, 0xca, 0x8f, 0xfa, 0x70, 0xdd, 0xb0,
+ 0x3c, 0xec, 0x58, 0xaa, 0xa9, 0xa8, 0xba, 0x1e, 0x37, 0xc3, 0xd2, 0x95, 0xcd, 0xb0, 0x1e, 0x40,
+ 0x55, 0x08, 0x52, 0x68, 0x82, 0x03, 0xb8, 0x11, 0xea, 0x70, 0xf0, 0xd0, 0x3e, 0x8e, 0x6b, 0x49,
+ 0x5d, 0x59, 0x4b, 0xb8, 0x60, 0x89, 0x61, 0x05, 0x7a, 0xee, 0xf0, 0xe4, 0x1e, 0x2c, 0xbe, 0xc7,
+ 0xc1, 0xb5, 0x5d, 0x5b, 0x37, 0x0e, 0x0c, 0xac, 0x93, 0xbb, 0x35, 0xf0, 0xd5, 0x6f, 0x02, 0x72,
+ 0x27, 0xae, 0x87, 0x87, 0x8a, 0x66, 0x5b, 0x07, 0xc6, 0x40, 0x71, 0x47, 0xaa, 0x45, 0xfd, 0x54,
+ 0x90, 0x44, 0x36, 0x53, 0xa3, 0x13, 0xf4, 0x42, 0x6e, 0x00, 0xa2, 0xf9, 0xdf, 0x34, 0x8e, 0xb1,
+ 0x85, 0x5d, 0x97, 0x71, 0x33, 0xef, 0xbb, 0x3e, 0x63, 0xb1, 0x44, 0x48, 0x12, 0x89, 0x48, 0xdb,
+ 0x97, 0x20, 0x94, 0xe2, 0x43, 0x10, 0x7b, 0x9e, 0xa1, 0x1d, 0x4d, 0xaa, 0x51, 0x9e, 0xaf, 0x02,
+ 0xb8, 0x94, 0xa6, 0xf4, 0x0d, 0xcf, 0xcf, 0x9d, 0xf3, 0xdd, 0xb9, 0x6e, 0x00, 0x55, 0xfc, 0x43,
+ 0x02, 0xd6, 0x9b, 0xbe, 0x19, 0x6a, 0xf6, 0x70, 0x18, 0xa1, 0xd7, 0x21, 0xeb, 0x92, 0x5b, 0x45,
+ 0xf1, 0x18, 0xc1, 0x57, 0x50, 0x98, 0xb9, 0xe6, 0xe8, 0xf6, 0x91, 0x32, 0x6e, 0xfc, 0x2e, 0xaa,
+ 0x43, 0x76, 0x48, 0xd2, 0x7d, 0x88, 0xb2, 0x78, 0x21, 0x4a, 0xfc, 0x5a, 0x90, 0x32, 0xc3, 0xf8,
+ 0x25, 0xf1, 0x43, 0xb8, 0xee, 0xe7, 0x84, 0xe0, 0xb8, 0x43, 0xbc, 0x04, 0xc5, 0xbb, 0x35, 0x03,
+ 0x6f, 0x66, 0xa6, 0x91, 0xd6, 0xb5, 0x0b, 0x12, 0xd0, 0xfa, 0xd0, 0x3f, 0x6b, 0x7a, 0x42, 0x21,
+ 0x3e, 0xcb, 0xfd, 0x5f, 0x9f, 0xb5, 0xde, 0x27, 0x7d, 0x43, 0xba, 0x36, 0x9c, 0xe1, 0x30, 0xaf,
+ 0x03, 0x8a, 0xce, 0x29, 0x04, 0x66, 0x01, 0xf6, 0xfc, 0x2c, 0x73, 0x3e, 0x76, 0xd0, 0x92, 0xe8,
+ 0x3e, 0x46, 0xb9, 0xc3, 0xbf, 0xfb, 0x7e, 0x81, 0x2b, 0xfe, 0x89, 0x83, 0xd5, 0x6e, 0x9f, 0xa6,
+ 0x64, 0x3d, 0x3c, 0xe3, 0x78, 0xc5, 0xc1, 0xcd, 0x51, 0x71, 0xfc, 0x84, 0xbb, 0x72, 0xe5, 0xf6,
+ 0x6c, 0x2e, 0xa1, 0x48, 0xeb, 0x1d, 0x9e, 0x14, 0x9d, 0xc5, 0xb3, 0x25, 0x58, 0x96, 0x1d, 0xd5,
+ 0x72, 0x55, 0xcd, 0x33, 0x6c, 0x0b, 0x55, 0x80, 0x27, 0x05, 0xb5, 0xef, 0x78, 0xcf, 0x5f, 0x76,
+ 0xfd, 0xca, 0x27, 0xd6, 0x2e, 0xf6, 0xd4, 0xaa, 0x40, 0x56, 0xf6, 0xe1, 0xb4, 0xc0, 0x49, 0x54,
+ 0x14, 0x21, 0xe0, 0x2d, 0x75, 0xc8, 0x4a, 0xc5, 0xb4, 0x44, 0xbf, 0xd1, 0x5d, 0x58, 0x22, 0xd7,
+ 0xfa, 0x98, 0xdd, 0xeb, 0xb3, 0xef, 0x96, 0xd8, 0x32, 0x7a, 0x94, 0x57, 0xf2, 0x65, 0x50, 0x0b,
+ 0x72, 0xa6, 0xea, 0x7a, 0xca, 0x21, 0x56, 0x1d, 0xaf, 0x8f, 0xd5, 0x39, 0xef, 0x6d, 0x16, 0x78,
+ 0x59, 0x22, 0xfa, 0x5a, 0x20, 0x89, 0x7e, 0x00, 0xf9, 0x81, 0x69, 0xf7, 0x55, 0x53, 0x19, 0x5b,
+ 0x1a, 0x76, 0x3c, 0xd5, 0xb0, 0xbc, 0x89, 0x62, 0x1a, 0x43, 0xc3, 0xcb, 0xa7, 0xe6, 0x47, 0xdd,
+ 0x60, 0x20, 0x7b, 0x11, 0x46, 0x9b, 0x40, 0xa0, 0xb7, 0xe0, 0x9a, 0xed, 0x7b, 0x87, 0x12, 0xda,
+ 0xda, 0xcd, 0x0b, 0x17, 0x26, 0xca, 0x27, 0x7c, 0xc9, 0x57, 0x80, 0xec, 0xc7, 0x27, 0x5c, 0x74,
+ 0x17, 0x80, 0x1c, 0x27, 0x0d, 0x16, 0x37, 0xbf, 0x4c, 0x31, 0x2f, 0xca, 0x67, 0x41, 0xda, 0x21,
+ 0x02, 0x64, 0xec, 0xa2, 0x22, 0x64, 0xdf, 0x71, 0x0c, 0x0f, 0x2b, 0x9e, 0x6d, 0x2b, 0xb6, 0xa9,
+ 0xe7, 0x33, 0x34, 0x7d, 0x2e, 0x53, 0xa2, 0x6c, 0xdb, 0x5d, 0x53, 0x27, 0x96, 0xa6, 0x15, 0x49,
+ 0xe4, 0x9c, 0x2b, 0x57, 0xb0, 0x34, 0x11, 0x8d, 0x62, 0xe2, 0x65, 0xd8, 0xd0, 0x68, 0x76, 0x8b,
+ 0xd0, 0x94, 0x03, 0xe3, 0x04, 0xeb, 0x79, 0x91, 0x2a, 0x5e, 0x63, 0xb3, 0xa1, 0xc0, 0x3d, 0x32,
+ 0x87, 0x5e, 0x07, 0xd1, 0xb0, 0x94, 0x03, 0x93, 0x56, 0x47, 0x74, 0x69, 0x6e, 0x7e, 0x95, 0xee,
+ 0xf4, 0x2b, 0xb3, 0x76, 0x8a, 0x1f, 0x8d, 0xb1, 0xa5, 0x61, 0xfd, 0x0d, 0xc2, 0xe9, 0xaf, 0x23,
+ 0x67, 0x58, 0xf7, 0xa8, 0x3c, 0x25, 0xba, 0xc8, 0x86, 0x15, 0x63, 0x60, 0xd9, 0x0e, 0x49, 0x33,
+ 0xf8, 0x91, 0x35, 0x1e, 0xba, 0x79, 0x44, 0x11, 0x4b, 0x97, 0xb9, 0x77, 0x93, 0x89, 0xf4, 0xf0,
+ 0xa3, 0xce, 0x78, 0x48, 0xef, 0xdf, 0xa8, 0xcc, 0x3c, 0x37, 0xe7, 0x4a, 0x39, 0x23, 0x1c, 0x13,
+ 0xf4, 0x73, 0x6f, 0xba, 0x84, 0xc8, 0xb7, 0x78, 0x61, 0x49, 0x4c, 0xb5, 0x78, 0x21, 0x2d, 0x42,
+ 0x8b, 0x17, 0xb2, 0x62, 0xae, 0xc5, 0x0b, 0x39, 0x71, 0xa5, 0xf8, 0x3b, 0x1e, 0x56, 0x63, 0x5e,
+ 0x2f, 0x61, 0xcd, 0x76, 0xf4, 0x67, 0x11, 0x82, 0x5f, 0x9c, 0x70, 0x7b, 0x3a, 0x97, 0xfd, 0x7f,
+ 0x70, 0x06, 0xde, 0x77, 0x84, 0x45, 0x31, 0xf1, 0x98, 0x3b, 0xa4, 0x44, 0xa1, 0xc5, 0x0b, 0x82,
+ 0x98, 0x0e, 0x5d, 0x03, 0xc4, 0xe5, 0x16, 0x2f, 0x64, 0xc4, 0x6c, 0xdc, 0x4d, 0x5a, 0xbc, 0xb0,
+ 0x22, 0x8a, 0x2d, 0x5e, 0x10, 0xc5, 0xd5, 0xe2, 0x94, 0x83, 0x25, 0x52, 0x3b, 0x58, 0x1e, 0xda,
+ 0x87, 0x15, 0xd7, 0xb0, 0x06, 0x26, 0x56, 0x8e, 0xf0, 0x24, 0x2a, 0x88, 0x96, 0xb7, 0x5f, 0x98,
+ 0x61, 0x1b, 0x26, 0x53, 0xea, 0x51, 0x81, 0xfb, 0x78, 0x42, 0xed, 0x1d, 0xb9, 0x4d, 0xd6, 0x8d,
+ 0x4f, 0xa0, 0xef, 0x40, 0xc2, 0x3b, 0x09, 0x2a, 0xa6, 0xb9, 0x3c, 0x90, 0x19, 0x9b, 0x48, 0x6d,
+ 0xde, 0x85, 0xec, 0x39, 0x35, 0x97, 0xf4, 0x47, 0xe2, 0x3d, 0x8f, 0x16, 0x2f, 0xf0, 0x62, 0xb2,
+ 0xf8, 0x67, 0x0e, 0x56, 0xda, 0xb6, 0x76, 0x54, 0xd1, 0x1e, 0x8d, 0x0d, 0xd7, 0xa0, 0x97, 0xd2,
+ 0xb7, 0x81, 0x8f, 0x6d, 0xef, 0x42, 0xf7, 0x89, 0x45, 0x81, 0xfb, 0xb4, 0xbb, 0x40, 0x12, 0x80,
+ 0x3e, 0x76, 0xd4, 0xbe, 0x61, 0x1a, 0x1e, 0x5b, 0x7b, 0x6e, 0x7b, 0xfb, 0x82, 0x97, 0x59, 0xac,
+ 0xad, 0x55, 0x22, 0xce, 0x5b, 0xaa, 0x87, 0x92, 0x52, 0x0c, 0xa5, 0xf8, 0xdb, 0x45, 0x00, 0xb2,
+ 0x37, 0xf6, 0xcc, 0xf8, 0x9f, 0x6c, 0x2b, 0xca, 0x0c, 0x89, 0xcf, 0x90, 0x19, 0x66, 0x04, 0x0f,
+ 0xff, 0xdf, 0x0c, 0x9e, 0xe2, 0xcf, 0x39, 0xc8, 0x9d, 0x0f, 0xeb, 0xcb, 0xba, 0x54, 0x6f, 0x83,
+ 0xe0, 0xfa, 0xcc, 0x7e, 0xe3, 0xe7, 0x7b, 0x73, 0x57, 0x4c, 0x8f, 0x77, 0x06, 0x89, 0xf1, 0x7a,
+ 0xf8, 0x91, 0x14, 0x22, 0xfa, 0x05, 0xd3, 0x1f, 0x79, 0x48, 0xb6, 0xb1, 0xea, 0x62, 0x34, 0x81,
+ 0x24, 0x6b, 0x22, 0x70, 0x9f, 0x5f, 0xfd, 0xc6, 0x34, 0xa2, 0xef, 0x02, 0xe0, 0x93, 0x91, 0xe1,
+ 0xa8, 0xe4, 0x8c, 0xe6, 0xaa, 0x1f, 0xa5, 0x98, 0x00, 0xaa, 0x43, 0x2a, 0x78, 0x92, 0x27, 0xae,
+ 0xfc, 0x24, 0x0f, 0x44, 0xd1, 0x1e, 0xc4, 0x1e, 0xb4, 0xac, 0x9f, 0x42, 0xfe, 0xba, 0x46, 0xd0,
+ 0xbc, 0xf9, 0x0f, 0x2b, 0x5a, 0x8f, 0xa4, 0x69, 0x53, 0xa4, 0x47, 0x65, 0xd1, 0xcf, 0x38, 0x58,
+ 0x1e, 0x39, 0xf6, 0xc8, 0x76, 0x49, 0x09, 0xe5, 0xce, 0x77, 0xf7, 0x74, 0x4e, 0xa7, 0x05, 0x78,
+ 0xe0, 0x4b, 0xc9, 0xbd, 0xa7, 0xb6, 0x33, 0x04, 0x2b, 0x90, 0x5d, 0xb4, 0x06, 0x49, 0x3c, 0xb2,
+ 0xb5, 0x43, 0xda, 0x9e, 0x49, 0x48, 0x6c, 0x80, 0x5e, 0x8c, 0xf9, 0x1a, 0xa9, 0x1b, 0x13, 0xd5,
+ 0xd5, 0xb3, 0x69, 0x21, 0x4b, 0x5d, 0x23, 0xf0, 0xd8, 0x98, 0xf3, 0x84, 0x25, 0x41, 0xf1, 0xaf,
+ 0x1c, 0xe4, 0x2a, 0x7d, 0xdb, 0xf1, 0x48, 0x70, 0x37, 0x2c, 0xcf, 0x99, 0x5c, 0xe6, 0xd2, 0x4f,
+ 0xdf, 0xe2, 0x45, 0x2a, 0x08, 0x23, 0xc7, 0xb0, 0x9d, 0x20, 0x8f, 0x25, 0xab, 0x8d, 0xb3, 0x69,
+ 0xa1, 0xf2, 0x99, 0xa3, 0xe2, 0x81, 0x0f, 0x26, 0x85, 0xb0, 0x77, 0x04, 0xb2, 0xb3, 0x4f, 0xc9,
+ 0xee, 0xfe, 0xb5, 0x08, 0xab, 0x6d, 0xac, 0x1e, 0xc8, 0x27, 0x56, 0xd3, 0x1a, 0x8d, 0xc9, 0xa1,
+ 0x7a, 0x18, 0xbd, 0xc2, 0x52, 0x16, 0x0b, 0x94, 0xad, 0xcb, 0x53, 0x4e, 0x3c, 0x5b, 0xbd, 0x00,
+ 0x2b, 0x0e, 0x3e, 0x70, 0xb0, 0x7b, 0xa8, 0x18, 0xd6, 0xb1, 0x6a, 0x1a, 0x3a, 0xb5, 0xb5, 0x20,
+ 0xe5, 0x7c, 0x72, 0x93, 0x51, 0x67, 0x16, 0x0a, 0xc2, 0xd3, 0x15, 0x0a, 0xdb, 0xb0, 0xee, 0x7a,
+ 0x78, 0x34, 0x32, 0xac, 0x81, 0x32, 0x24, 0x6f, 0x3b, 0x6c, 0xa9, 0x7d, 0x13, 0xeb, 0xf9, 0x34,
+ 0x5d, 0xc1, 0xb5, 0x60, 0x72, 0xd7, 0xd6, 0x71, 0x83, 0x4d, 0xa1, 0x3e, 0x64, 0x58, 0x43, 0x0f,
+ 0x3f, 0x52, 0xac, 0xf1, 0x30, 0x0f, 0xcf, 0x28, 0x09, 0x01, 0x41, 0x65, 0x39, 0xf2, 0xb1, 0x4a,
+ 0x82, 0x17, 0x93, 0x2d, 0x5e, 0x48, 0x8a, 0x4b, 0xac, 0xaa, 0x28, 0xbe, 0x17, 0xd9, 0xff, 0x9e,
+ 0x61, 0xa9, 0xe6, 0xd3, 0xd9, 0xff, 0x55, 0xc8, 0xc7, 0x5b, 0x84, 0xf6, 0x70, 0xa8, 0x5a, 0xe4,
+ 0xff, 0xd8, 0xf2, 0x98, 0x2b, 0x49, 0xb1, 0x16, 0x62, 0x8d, 0x4d, 0xd7, 0xc8, 0x2c, 0xaa, 0x42,
+ 0x36, 0x38, 0x39, 0x56, 0xfa, 0xf1, 0xf3, 0x94, 0x7e, 0x19, 0x5f, 0x86, 0x55, 0x7f, 0xf3, 0x9e,
+ 0x7e, 0x68, 0x92, 0xd0, 0x0c, 0xac, 0xac, 0x2a, 0xfe, 0x93, 0x83, 0x95, 0x9a, 0x69, 0x60, 0xcb,
+ 0xa3, 0xb7, 0x4e, 0xd3, 0x3a, 0xb0, 0xd1, 0x6b, 0xb0, 0xae, 0x87, 0x29, 0x4d, 0x19, 0x60, 0x0b,
+ 0xfb, 0x59, 0x94, 0xa3, 0x41, 0x7c, 0xed, 0x6c, 0x5a, 0x58, 0xa1, 0xdc, 0x3b, 0xe1, 0x94, 0xb4,
+ 0x16, 0x49, 0x44, 0x54, 0xf4, 0x2a, 0xe4, 0x4c, 0x12, 0xed, 0xca, 0xb9, 0x3b, 0x67, 0x66, 0x1e,
+ 0xc8, 0x9a, 0xf1, 0x21, 0xc2, 0x70, 0x5d, 0x33, 0x59, 0x7e, 0x0b, 0x5f, 0x46, 0x23, 0xdb, 0x34,
+ 0xb4, 0xa0, 0xd0, 0x78, 0xf1, 0xa2, 0x16, 0x60, 0x8d, 0x8a, 0x85, 0xc1, 0xfe, 0x80, 0x0a, 0x49,
+ 0xeb, 0xda, 0x2c, 0x72, 0xf1, 0x1f, 0x1c, 0xa4, 0xa3, 0x8d, 0xdf, 0xf5, 0x9b, 0x8c, 0x57, 0xed,
+ 0xf7, 0xb3, 0x56, 0xe3, 0xcb, 0x90, 0xa4, 0x7b, 0xb8, 0xe4, 0x07, 0x20, 0xba, 0xe5, 0xe0, 0x07,
+ 0x20, 0xca, 0xfc, 0x39, 0x6d, 0x94, 0xf5, 0x0e, 0x6f, 0x7f, 0xc4, 0x41, 0x9a, 0xfe, 0x08, 0x45,
+ 0x3b, 0xd0, 0xcb, 0x90, 0xda, 0xeb, 0xdc, 0xef, 0x74, 0xdf, 0xe8, 0x88, 0x0b, 0x28, 0x05, 0x89,
+ 0x66, 0x47, 0x16, 0x39, 0x94, 0x86, 0xe4, 0xbd, 0x76, 0xb7, 0x22, 0x8b, 0x8b, 0xe4, 0xb3, 0xba,
+ 0x2f, 0x37, 0x7a, 0x62, 0x02, 0x5d, 0x83, 0x95, 0x7a, 0xa3, 0xdd, 0xdc, 0x6d, 0xca, 0x8d, 0xba,
+ 0xc2, 0x88, 0x02, 0x12, 0x80, 0x97, 0x9b, 0xbb, 0x0d, 0x91, 0x27, 0x50, 0xf5, 0x46, 0xad, 0xb9,
+ 0x5b, 0x69, 0x8b, 0x49, 0xb4, 0x0e, 0xab, 0x11, 0x6f, 0x40, 0x4e, 0xa3, 0x0c, 0x08, 0xf5, 0x3d,
+ 0xa9, 0x22, 0x37, 0xbb, 0x1d, 0x71, 0x09, 0x01, 0x2c, 0x11, 0x59, 0xf9, 0x4d, 0x31, 0x43, 0x74,
+ 0xef, 0x34, 0xba, 0x62, 0x96, 0x2a, 0xec, 0x7e, 0x7f, 0xbb, 0x2e, 0xe6, 0xc8, 0xa7, 0xbc, 0xf7,
+ 0xa0, 0xdd, 0x10, 0x81, 0x08, 0x56, 0x9b, 0x72, 0x45, 0x92, 0x2a, 0xfb, 0xe2, 0x32, 0xca, 0x01,
+ 0x10, 0xc1, 0x5e, 0x43, 0x6a, 0x36, 0x7a, 0xa2, 0x5e, 0x24, 0x8f, 0x84, 0xd4, 0x6d, 0x0d, 0x56,
+ 0x9f, 0x68, 0xbc, 0xa3, 0x2c, 0xa4, 0x2b, 0xf5, 0xba, 0xf2, 0xb0, 0x2b, 0x37, 0x24, 0x71, 0x01,
+ 0x89, 0x90, 0x91, 0x1a, 0xbb, 0xdd, 0x87, 0x0d, 0x9f, 0xc2, 0xa1, 0x55, 0xc8, 0x12, 0x86, 0x4e,
+ 0xb7, 0xe3, 0x93, 0x16, 0xd1, 0x1a, 0x88, 0x3e, 0x53, 0x44, 0x4d, 0x6c, 0xf2, 0xef, 0xfe, 0x6a,
+ 0x6b, 0xe1, 0x76, 0xf7, 0xdc, 0x5b, 0x94, 0x15, 0x7e, 0x64, 0xeb, 0x0f, 0x1a, 0x9d, 0x7a, 0xb3,
+ 0xb3, 0x23, 0x2e, 0x90, 0x41, 0x4f, 0xae, 0xec, 0x90, 0x41, 0x82, 0xa8, 0xaf, 0x75, 0x77, 0x77,
+ 0x9b, 0xb2, 0xdc, 0xa8, 0x8b, 0x1c, 0x99, 0xab, 0x54, 0xbb, 0x12, 0x19, 0x2c, 0xfa, 0x80, 0xfb,
+ 0xb0, 0x79, 0xf1, 0x51, 0xa2, 0x4d, 0xd8, 0x68, 0x57, 0x76, 0x94, 0xea, 0xbe, 0x52, 0x6b, 0xef,
+ 0xf5, 0xe4, 0x86, 0xa4, 0xf4, 0x1a, 0xb2, 0xcc, 0x14, 0xdd, 0x80, 0xf5, 0x76, 0xa3, 0x52, 0x57,
+ 0xee, 0x75, 0x25, 0x65, 0xa7, 0xdd, 0xad, 0x56, 0xda, 0x8a, 0xd4, 0xa8, 0xd4, 0x7b, 0x22, 0xc7,
+ 0xa0, 0xab, 0xdf, 0xf8, 0xe0, 0xef, 0x5b, 0x0b, 0x1f, 0x9c, 0x6e, 0x71, 0x1f, 0x9e, 0x6e, 0x71,
+ 0x1f, 0x9f, 0x6e, 0x71, 0x7f, 0x3b, 0xdd, 0xe2, 0x7e, 0xf1, 0xc9, 0xd6, 0xc2, 0x87, 0x9f, 0x6c,
+ 0x2d, 0x7c, 0xfc, 0xc9, 0xd6, 0xc2, 0x9b, 0x29, 0xdf, 0x9b, 0xfa, 0x4b, 0xf4, 0xb7, 0xda, 0x97,
+ 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff, 0xc9, 0x93, 0xc5, 0xc6, 0xaf, 0x1e, 0x00, 0x00,
}
diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto
index 6e730fcabc73..c24332ae58f1 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";
@@ -150,6 +151,22 @@ message MergeTrigger {
// mark for the keys previously owned by the subsumed range.
util.hlc.Timestamp freeze_start = 5 [(gogoproto.nullable) = false,
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"];
+
+ // 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.
+ kv.kvserver.readsummary.ReadSummary right_read_summary = 6;
}
// ReplicaChangeType is a parameter of ChangeReplicasTrigger.