From 4fb25e865ec161cb8f8cc3f95b651520ddcc2e6f Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Sun, 12 May 2019 17:05:41 -0400 Subject: [PATCH 1/2] roachprod: create geo-distributed clusters with extra nodes placed at the end Prior to this change, roachprod would create geo-distributed clusters by placing nodes in AZs in contiguous chunks. If the number of total nodes was not evenly divisible by the number of regions, the first regions would be allocated one additional node. This allocation pattern is rarely desirable. A user will commonly allocate a single extra node as a load generator and would generally like that load node to be the final node and for that final node to be the extra node. This changes the allocation where the extra nodes are placed in the same regions as before but are given node indices at the end rather than with the other nodes in their region. After this change a cluster created with `roachprod create $CLUSTER -n 7 --geo` will look like: ``` ajwerner-test-roachprod-gce: [gce] 12h47m58s remaining ajwerner-test-roachprod-gce-0001 ajwerner-test-roachprod-gce-0001.us-east1-b.cockroach-ephemeral 10.142.0.70 34.74.58.108 ajwerner-test-roachprod-gce-0002 ajwerner-test-roachprod-gce-0002.us-east1-b.cockroach-ephemeral 10.142.0.5 35.237.74.155 ajwerner-test-roachprod-gce-0003 ajwerner-test-roachprod-gce-0003.us-west1-b.cockroach-ephemeral 10.138.0.99 35.199.159.104 ajwerner-test-roachprod-gce-0004 ajwerner-test-roachprod-gce-0004.us-west1-b.cockroach-ephemeral 10.138.0.100 35.197.94.83 ajwerner-test-roachprod-gce-0005 ajwerner-test-roachprod-gce-0005.europe-west2-b.cockroach-ephemeral 10.154.15.237 35.230.143.190 ajwerner-test-roachprod-gce-0006 ajwerner-test-roachprod-gce-0006.europe-west2-b.cockroach-ephemeral 10.154.15.236 35.234.156.121 ajwerner-test-roachprod-gce-0007 ajwerner-test-roachprod-gce-0007.us-east1-b.cockroach-ephemeral 10.142.0.33 35.185.62.76 ``` Instead of the previous: ``` ajwerner-test-old: [gce] 12h19m21s remaining ajwerner-test-old-0001 ajwerner-test-old-0001.us-east1-b.cockroach-ephemeral 10.142.0.139 34.74.150.216 ajwerner-test-old-0002 ajwerner-test-old-0002.us-east1-b.cockroach-ephemeral 10.142.0.140 34.73.154.246 ajwerner-test-old-0003 ajwerner-test-old-0003.us-east1-b.cockroach-ephemeral 10.142.0.141 35.243.176.131 ajwerner-test-old-0004 ajwerner-test-old-0004.us-west1-b.cockroach-ephemeral 10.138.0.71 34.83.16.1 ajwerner-test-old-0005 ajwerner-test-old-0005.us-west1-b.cockroach-ephemeral 10.138.0.60 34.83.78.172 ajwerner-test-old-0006 ajwerner-test-old-0006.europe-west2-b.cockroach-ephemeral 10.154.15.200 35.234.148.191 ajwerner-test-old-0007 ajwerner-test-old-0007.europe-west2-b.cockroach-ephemeral 10.154.15.199 35.242.179.144 ``` Release note: None --- pkg/cmd/roachprod/vm/aws/aws.go | 44 +++++-------- pkg/cmd/roachprod/vm/gce/gcloud.go | 31 +++------ pkg/cmd/roachprod/vm/vm.go | 23 +++++++ pkg/cmd/roachprod/vm/vm_test.go | 40 +++++++++++ pkg/cmd/roachtest/cluster.go | 53 +++++++++++++++ pkg/cmd/roachtest/cluster_test.go | 73 +++++++++++++++++++++ pkg/cmd/roachtest/indexes.go | 16 +---- pkg/cmd/roachtest/interleavedpartitioned.go | 19 +++--- pkg/cmd/roachtest/ledger.go | 16 +---- pkg/cmd/roachtest/tpcc.go | 31 +-------- 10 files changed, 234 insertions(+), 112 deletions(-) create mode 100644 pkg/cmd/roachprod/vm/vm_test.go diff --git a/pkg/cmd/roachprod/vm/aws/aws.go b/pkg/cmd/roachprod/vm/aws/aws.go index f890af5b4ede..e0e8936231ff 100644 --- a/pkg/cmd/roachprod/vm/aws/aws.go +++ b/pkg/cmd/roachprod/vm/aws/aws.go @@ -19,7 +19,6 @@ import ( "encoding/json" "fmt" "log" - "math" "math/rand" "os" "os/exec" @@ -228,38 +227,29 @@ func (p *Provider) Create(names []string, opts vm.CreateOpts) error { regions = []string{regions[0]} } - nodeCount := len(names) - - var g errgroup.Group - // We're looping over regions to create all of the nodes in one region - // in the same iteration so they're contiguous. - node := 0 - const rateLimit = 2 // per second - limiter := rate.NewLimiter(rateLimit, 2 /* buckets */) + // Choose a random availability zone for each region. + zones := make([]string, len(regions)) for i, region := range regions { - zones, err := p.regionZones(region, p.opts.CreateZones) + regionZones, err := p.regionZones(region, p.opts.CreateZones) if err != nil { return err } - nodesPerRegion := int(math.Ceil(float64(nodeCount-node) / float64(len(regions)-i))) - // We're choosing a random availability zone now which will be consistent - // per region. - availabilityZone := rand.Int31n(int32(len(zones))) - for j := 0; j < nodesPerRegion; j++ { - if node >= nodeCount { - break - } - capName := names[node] - placement := zones[availabilityZone] - res := limiter.Reserve() - g.Go(func() error { - time.Sleep(res.Delay()) - return p.runInstance(capName, placement, opts) - }) - node++ - } + zones[i] = regionZones[rand.Int31n(int32(len(regionZones)))] } + nodeZones := vm.ZonePlacement(len(zones), len(names)) + var g errgroup.Group + const rateLimit = 2 // per second + limiter := rate.NewLimiter(rateLimit, 2 /* buckets */) + for i := range names { + capName := names[i] + placement := zones[nodeZones[i]] + res := limiter.Reserve() + g.Go(func() error { + time.Sleep(res.Delay()) + return p.runInstance(capName, placement, opts) + }) + } return g.Wait() } diff --git a/pkg/cmd/roachprod/vm/gce/gcloud.go b/pkg/cmd/roachprod/vm/gce/gcloud.go index 1b54189679d8..f08c15f0ea7c 100644 --- a/pkg/cmd/roachprod/vm/gce/gcloud.go +++ b/pkg/cmd/roachprod/vm/gce/gcloud.go @@ -20,7 +20,6 @@ import ( "context" "encoding/json" "fmt" - "math" "os" "os/exec" "regexp" @@ -332,13 +331,6 @@ func (p *Provider) Create(names []string, opts vm.CreateOpts) error { p.opts.Zones = []string{p.opts.Zones[0]} } - totalNodes := float64(len(names)) - totalZones := float64(len(p.opts.Zones)) - nodesPerZone := int(math.Ceil(totalNodes / totalZones)) - - ct := int(0) - i := 0 - // Fixed args. args := []string{ "compute", "instances", "create", @@ -384,20 +376,15 @@ func (p *Provider) Create(names []string, opts vm.CreateOpts) error { var g errgroup.Group - // This is calculating the number of machines to allocate per zone by taking the ceiling of the the total number - // of machines left divided by the number of zones left. If the the number of machines isn't - // divisible by the number of zones, then the extra machines will be allocated one per zone until there are - // no more extra machines left. - for i < len(names) { - argsWithZone := append(args[:len(args):len(args)], "--zone", p.opts.Zones[ct]) - ct++ - argsWithZone = append(argsWithZone, names[i:i+nodesPerZone]...) - i += nodesPerZone - - totalNodes -= float64(nodesPerZone) - totalZones-- - nodesPerZone = int(math.Ceil(totalNodes / totalZones)) - + nodeZones := vm.ZonePlacement(len(p.opts.Zones), len(names)) + zoneHostNames := make([][]string, len(p.opts.Zones)) + for i, name := range names { + zone := nodeZones[i] + zoneHostNames[zone] = append(zoneHostNames[zone], name) + } + for i, zoneHosts := range zoneHostNames { + argsWithZone := append(args[:len(args):len(args)], "--zone", p.opts.Zones[i]) + argsWithZone = append(argsWithZone, zoneHosts...) g.Go(func() error { cmd := exec.Command("gcloud", argsWithZone...) diff --git a/pkg/cmd/roachprod/vm/vm.go b/pkg/cmd/roachprod/vm/vm.go index 3a1ff7219697..7be52ffb6f61 100644 --- a/pkg/cmd/roachprod/vm/vm.go +++ b/pkg/cmd/roachprod/vm/vm.go @@ -289,3 +289,26 @@ func ProvidersSequential(named []string, action func(Provider) error) error { } return nil } + +// ZonePlacement allocates zones to numNodes in an equally sized groups in the +// same order as zones. If numNodes is not divisible by len(zones) the remainder +// is allocated in a round-robin fashion and placed at the end of the returned +// slice. The returned slice has a length of numNodes where each value is in +// [0, numZones). +// +// For example: +// +// ZonePlacement(3, 8) = []int{0, 0, 1, 1, 2, 2, 0, 1} +// +func ZonePlacement(numZones, numNodes int) (nodeZones []int) { + numPerZone := numNodes / numZones + extraStartIndex := numPerZone * numZones + nodeZones = make([]int, numNodes) + for i := 0; i < numNodes; i++ { + nodeZones[i] = i / numPerZone + if i >= extraStartIndex { + nodeZones[i] = i % numZones + } + } + return nodeZones +} diff --git a/pkg/cmd/roachprod/vm/vm_test.go b/pkg/cmd/roachprod/vm/vm_test.go new file mode 100644 index 000000000000..e296a1cca5ec --- /dev/null +++ b/pkg/cmd/roachprod/vm/vm_test.go @@ -0,0 +1,40 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. See the AUTHORS file +// for names of contributors. + +package vm + +import ( + "strconv" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestZonePlacement(t *testing.T) { + for i, c := range []struct { + numZones, numNodes int + expected []int + }{ + {1, 1, []int{0}}, + {1, 2, []int{0, 0}}, + {2, 4, []int{0, 0, 1, 1}}, + {2, 5, []int{0, 0, 1, 1, 0}}, + {3, 11, []int{0, 0, 0, 1, 1, 1, 2, 2, 2, 0, 1}}, + } { + t.Run(strconv.Itoa(i), func(t *testing.T) { + assert.EqualValues(t, c.expected, ZonePlacement(c.numZones, c.numNodes)) + }) + } +} diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 69519d7c54f9..45d2c588b361 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -1988,3 +1988,56 @@ func waitForFullReplication(t *test, db *gosql.DB) { } } } + +type loadGroup struct { + roachNodes nodeListOption + loadNodes nodeListOption +} + +type loadGroupList []loadGroup + +func (lg loadGroupList) roachNodes() nodeListOption { + var roachNodes nodeListOption + for _, g := range lg { + roachNodes = roachNodes.merge(g.roachNodes) + } + return roachNodes +} + +func (lg loadGroupList) loadNodes() nodeListOption { + var loadNodes nodeListOption + for _, g := range lg { + loadNodes = loadNodes.merge(g.loadNodes) + } + return loadNodes +} + +// makeLoadGroups create a loadGroupList that has an equal number of cockroach +// nodes per zone. It assumes that numLoadNodes <= numZones and that numZones is +// divisible by numLoadNodes. +func makeLoadGroups(c *cluster, numZones, numRoachNodes, numLoadNodes int) loadGroupList { + if numLoadNodes > numZones { + panic("cannot have more than one load node per zone") + } else if numZones%numLoadNodes != 0 { + panic("numZones must be divisible by numLoadNodes") + } + // roachprod allocates nodes over regions in a round-robin fashion. + // If the number of nodes is not divisible by the number of regions, the + // extra nodes are allocated in a round-robin fashion over the regions at + // the end of cluster. + loadNodesAtTheEnd := numLoadNodes%numZones != 0 + loadGroups := make(loadGroupList, numLoadNodes) + roachNodesPerGroup := numRoachNodes / numLoadNodes + for i := range loadGroups { + if loadNodesAtTheEnd { + first := i*roachNodesPerGroup + 1 + loadGroups[i].roachNodes = c.Range(first, first+roachNodesPerGroup-1) + loadGroups[i].loadNodes = c.Node(numRoachNodes + i + 1) + } else { + first := i*(roachNodesPerGroup+1) + 1 + loadGroups[i].roachNodes = c.Range(first, first+roachNodesPerGroup-1) + loadGroups[i].loadNodes = c.Node((i + 1) * (roachNodesPerGroup + 1)) + } + } + return loadGroups +} diff --git a/pkg/cmd/roachtest/cluster_test.go b/pkg/cmd/roachtest/cluster_test.go index 1a7e81efe589..e166bc594ec5 100644 --- a/pkg/cmd/roachtest/cluster_test.go +++ b/pkg/cmd/roachtest/cluster_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/pkg/errors" + "github.com/stretchr/testify/require" ) func TestClusterNodes(t *testing.T) { @@ -253,3 +254,75 @@ func TestClusterMachineType(t *testing.T) { }) } } + +func TestLoadGroups(t *testing.T) { + cfg := &loggerConfig{stdout: os.Stdout, stderr: os.Stderr} + logger, err := cfg.newLogger("" /* path */) + if err != nil { + t.Fatal(err) + } + for _, tc := range []struct { + numZones, numRoachNodes, numLoadNodes int + loadGroups loadGroupList + }{ + { + 3, 9, 3, + loadGroupList{ + { + nodeListOption{1, 2, 3}, + nodeListOption{4}, + }, + { + nodeListOption{5, 6, 7}, + nodeListOption{8}, + }, + { + nodeListOption{9, 10, 11}, + nodeListOption{12}, + }, + }, + }, + { + 3, 9, 1, + loadGroupList{ + { + nodeListOption{1, 2, 3, 4, 5, 6, 7, 8, 9}, + nodeListOption{10}, + }, + }, + }, + { + 4, 8, 2, + loadGroupList{ + { + nodeListOption{1, 2, 3, 4}, + nodeListOption{9}, + }, + { + nodeListOption{5, 6, 7, 8}, + nodeListOption{10}, + }, + }, + }, + } { + t.Run(fmt.Sprintf("%d/%d/%d", tc.numZones, tc.numRoachNodes, tc.numLoadNodes), + func(t *testing.T) { + c := &cluster{t: testWrapper{t}, l: logger, nodes: tc.numRoachNodes + tc.numLoadNodes} + lg := makeLoadGroups(c, tc.numZones, tc.numRoachNodes, tc.numLoadNodes) + require.EqualValues(t, lg, tc.loadGroups) + }) + } + t.Run("panics with too many load nodes", func(t *testing.T) { + require.Panics(t, func() { + + numZones, numRoachNodes, numLoadNodes := 2, 4, 3 + makeLoadGroups(nil, numZones, numRoachNodes, numLoadNodes) + }, "Failed to panic when number of load nodes exceeded number of zones") + }) + t.Run("panics with unequal zones per load node", func(t *testing.T) { + require.Panics(t, func() { + numZones, numRoachNodes, numLoadNodes := 4, 4, 3 + makeLoadGroups(nil, numZones, numRoachNodes, numLoadNodes) + }, "Failed to panic when number of zones is not divisible by number of load nodes") + }) +} diff --git a/pkg/cmd/roachtest/indexes.go b/pkg/cmd/roachtest/indexes.go index 2cb58e2911bc..7734801a2adf 100644 --- a/pkg/cmd/roachtest/indexes.go +++ b/pkg/cmd/roachtest/indexes.go @@ -31,19 +31,9 @@ func registerNIndexes(r *registry, secondaryIndexes int) { Cluster: makeClusterSpec(nodes+1, cpu(16), geo(), zones(geoZonesStr)), Run: func(ctx context.Context, t *test, c *cluster) { firstAZ := geoZones[0] - lastNodeInFirstAZ := nodes / 3 - var roachNodes, gatewayNodes, loadNode nodeListOption - for i := 0; i < c.nodes; i++ { - n := c.Node(i + 1) - if i == lastNodeInFirstAZ { - loadNode = n - } else { - roachNodes = roachNodes.merge(n) - if i < lastNodeInFirstAZ { - gatewayNodes = gatewayNodes.merge(n) - } - } - } + roachNodes := c.Range(1, nodes) + gatewayNodes := c.Range(1, nodes/3) + loadNode := c.Node(nodes + 1) c.Put(ctx, cockroach, "./cockroach", roachNodes) c.Put(ctx, workload, "./workload", loadNode) diff --git a/pkg/cmd/roachtest/interleavedpartitioned.go b/pkg/cmd/roachtest/interleavedpartitioned.go index 7bd717f1d394..cde3025c6546 100644 --- a/pkg/cmd/roachtest/interleavedpartitioned.go +++ b/pkg/cmd/roachtest/interleavedpartitioned.go @@ -39,15 +39,16 @@ func registerInterleaved(r *registry) { c *cluster, config config, ) { - cockroachWest := c.Range(1, 3) - workloadWest := c.Node(4) - cockroachEast := c.Range(5, 7) - workloadEast := c.Node(8) - cockroachCentral := c.Range(9, 11) - workloadCentral := c.Node(12) - - cockroachNodes := cockroachWest.merge(cockroachEast.merge(cockroachCentral)) - workloadNodes := workloadWest.merge(workloadEast.merge(workloadCentral)) + numZones, numRoachNodes, numLoadNodes := 3, 9, 3 + loadGroups := makeLoadGroups(c, numZones, numRoachNodes, numLoadNodes) + cockroachWest := loadGroups[0].roachNodes + workloadWest := loadGroups[0].loadNodes + cockroachEast := loadGroups[1].roachNodes + workloadEast := loadGroups[1].loadNodes + cockroachCentral := loadGroups[2].roachNodes + workloadCentral := loadGroups[2].loadNodes + cockroachNodes := loadGroups.roachNodes() + workloadNodes := loadGroups.loadNodes() c.l.Printf("cockroach nodes: %s", cockroachNodes.String()[1:]) c.l.Printf("workload nodes: %s", workloadNodes.String()[1:]) diff --git a/pkg/cmd/roachtest/ledger.go b/pkg/cmd/roachtest/ledger.go index 450eca752aca..54459f1deab7 100644 --- a/pkg/cmd/roachtest/ledger.go +++ b/pkg/cmd/roachtest/ledger.go @@ -27,19 +27,9 @@ func registerLedger(r *registry) { Name: fmt.Sprintf("ledger/nodes=%d/multi-az", nodes), Cluster: makeClusterSpec(nodes+1, cpu(16), geo(), zones(azs)), Run: func(ctx context.Context, t *test, c *cluster) { - lastNodeInFirstAZ := nodes / 3 - var roachNodes, gatewayNodes, loadNode nodeListOption - for i := 0; i < c.nodes; i++ { - n := c.Node(i + 1) - if i == lastNodeInFirstAZ { - loadNode = n - } else { - roachNodes = roachNodes.merge(n) - if i < lastNodeInFirstAZ { - gatewayNodes = gatewayNodes.merge(n) - } - } - } + roachNodes := c.Range(1, nodes) + gatewayNodes := c.Range(1, nodes/3) + loadNode := c.Node(nodes + 1) c.Put(ctx, cockroach, "./cockroach", roachNodes) c.Put(ctx, workload, "./workload", loadNode) diff --git a/pkg/cmd/roachtest/tpcc.go b/pkg/cmd/roachtest/tpcc.go index 706aae29452a..773da99a2f8a 100644 --- a/pkg/cmd/roachtest/tpcc.go +++ b/pkg/cmd/roachtest/tpcc.go @@ -665,36 +665,11 @@ func loadTPCCBench( func runTPCCBench(ctx context.Context, t *test, c *cluster, b tpccBenchSpec) { // Determine the nodes in each load group. A load group consists of a set of // Cockroach nodes and a single load generator. - numRoachNodes := b.Nodes numLoadGroups := b.LoadConfig.numLoadNodes(b.Distribution) numZones := len(b.Distribution.zones()) - roachNodesPerGroup := numRoachNodes / numLoadGroups - nodesPerGroup := roachNodesPerGroup + 1 // nodesPerGroup * numLoadGroups == c.nodes - zonesPerGroup := numZones / numLoadGroups - // Roachprod round-robins the allocation of nodes across zones. When running - // a single load group across zones, the first zone will always have one more - // node than the others. This extra node is the load node. - loadNodePerGroup := roachNodesPerGroup / zonesPerGroup - loadGroups := make([]struct{ roachNodes, loadNodes nodeListOption }, numLoadGroups) - for i := range loadGroups { - for j := 0; j < nodesPerGroup; j++ { - n := c.Node(i*nodesPerGroup + j + 1) - if j == loadNodePerGroup { - loadGroups[i].loadNodes = loadGroups[i].loadNodes.merge(n) - } else { - loadGroups[i].roachNodes = loadGroups[i].roachNodes.merge(n) - } - } - } - - // Aggregate nodes across load groups. - var roachNodes nodeListOption - var loadNodes nodeListOption - for _, g := range loadGroups { - roachNodes = roachNodes.merge(g.roachNodes) - loadNodes = loadNodes.merge(g.loadNodes) - } - + loadGroups := makeLoadGroups(c, numZones, b.Nodes, numLoadGroups) + roachNodes := loadGroups.roachNodes() + loadNodes := loadGroups.loadNodes() c.Put(ctx, cockroach, "./cockroach", roachNodes) c.Put(ctx, workload, "./workload", loadNodes) c.Start(ctx, t, append(b.startOpts(), roachNodes)...) From d68329f68fdd29d2d6d3023b4c5c0112cf0d5051 Mon Sep 17 00:00:00 2001 From: Jeffrey Xiao Date: Mon, 20 May 2019 14:27:05 -0400 Subject: [PATCH 2/2] sql: add ALTER TABLE/INDEX .. UNSPLIT AT .. Now that manual splits add a sticky bit to the range descriptor, and the merge queue respects this sticky bit, we can expose functionality to manually unset this sticky bit. If the key to unsplit is not the start of a range, then the unsplit command will throw an error. If the range was manually split (I.E. the sticky bit is set), then the sticky bit will be unset. Otherwise, this command is a no-op. Syntactically, the unsplit command is identical to the split command. Release note (sql change): Add ALTER TABLE/INDEX .. UNSPLIT AT .. --- docs/generated/sql/bnf/stmt_block.bnf | 9 + pkg/internal/client/batch.go | 15 + pkg/internal/client/db.go | 13 + pkg/kv/split_test.go | 25 +- pkg/roachpb/api.go | 10 + pkg/roachpb/api.pb.go | 2419 ++++++++++------- pkg/roachpb/api.proto | 20 + pkg/roachpb/batch_generated.go | 335 +-- pkg/roachpb/method.go | 2 + pkg/roachpb/method_string.go | 65 +- pkg/sql/expand_plan.go | 6 + .../logictest/testdata/logic_test/alter_table | 9 + .../testdata/logic_test/crdb_internal | 13 + pkg/sql/opt_filters.go | 5 + pkg/sql/opt_limits.go | 3 + pkg/sql/opt_needed.go | 3 + pkg/sql/parser/parse_test.go | 11 + pkg/sql/parser/sql.y | 25 +- pkg/sql/plan.go | 5 + pkg/sql/plan_columns.go | 2 + pkg/sql/plan_physical_props.go | 1 + pkg/sql/sem/tree/split.go | 21 + pkg/sql/sem/tree/stmt.go | 9 +- pkg/sql/unsplit.go | 145 + pkg/sql/unsplit_test.go | 153 ++ pkg/sql/walk.go | 4 + pkg/storage/client_merge_test.go | 32 +- pkg/storage/replica.go | 5 + pkg/storage/replica_command.go | 50 + 29 files changed, 2226 insertions(+), 1189 deletions(-) create mode 100644 pkg/sql/unsplit.go create mode 100644 pkg/sql/unsplit_test.go diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 6089d786f654..4920c24c1cc4 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -790,6 +790,7 @@ unreserved_keyword ::= | 'UNCOMMITTED' | 'UNKNOWN' | 'UNLOGGED' + | 'UNSPLIT' | 'UPDATE' | 'UPSERT' | 'UUID' @@ -908,6 +909,7 @@ opt_abort_mod ::= alter_table_stmt ::= alter_onetable_stmt | alter_split_stmt + | alter_unsplit_stmt | alter_scatter_stmt | alter_zone_table_stmt | alter_rename_table_stmt @@ -915,6 +917,7 @@ alter_table_stmt ::= alter_index_stmt ::= alter_oneindex_stmt | alter_split_index_stmt + | alter_unsplit_index_stmt | alter_scatter_index_stmt | alter_rename_index_stmt | alter_zone_index_stmt @@ -1223,6 +1226,9 @@ alter_onetable_stmt ::= alter_split_stmt ::= 'ALTER' 'TABLE' table_name 'SPLIT' 'AT' select_stmt +alter_unsplit_stmt ::= + 'ALTER' 'TABLE' table_name 'UNSPLIT' 'AT' select_stmt + alter_scatter_stmt ::= 'ALTER' 'TABLE' table_name 'SCATTER' | 'ALTER' 'TABLE' table_name 'SCATTER' 'FROM' '(' expr_list ')' 'TO' '(' expr_list ')' @@ -1242,6 +1248,9 @@ alter_oneindex_stmt ::= alter_split_index_stmt ::= 'ALTER' 'INDEX' table_index_name 'SPLIT' 'AT' select_stmt +alter_unsplit_index_stmt ::= + 'ALTER' 'INDEX' table_index_name 'UNSPLIT' 'AT' select_stmt + alter_scatter_index_stmt ::= 'ALTER' 'INDEX' table_index_name 'SCATTER' | 'ALTER' 'INDEX' table_index_name 'SCATTER' 'FROM' '(' expr_list ')' 'TO' '(' expr_list ')' diff --git a/pkg/internal/client/batch.go b/pkg/internal/client/batch.go index 4e3fb1fe08a7..d4d242b1501d 100644 --- a/pkg/internal/client/batch.go +++ b/pkg/internal/client/batch.go @@ -248,6 +248,7 @@ func (b *Batch) fillResults(ctx context.Context) { case *roachpb.EndTransactionRequest: case *roachpb.AdminMergeRequest: case *roachpb.AdminSplitRequest: + case *roachpb.AdminUnsplitRequest: case *roachpb.AdminTransferLeaseRequest: case *roachpb.AdminChangeReplicasRequest: case *roachpb.AdminRelocateRangeRequest: @@ -609,6 +610,20 @@ func (b *Batch) adminSplit(spanKeyIn, splitKeyIn interface{}, manual bool) { b.initResult(1, 0, notRaw, nil) } +func (b *Batch) adminUnsplit(splitKeyIn interface{}) { + splitKey, err := marshalKey(splitKeyIn) + if err != nil { + b.initResult(0, 0, notRaw, err) + } + req := &roachpb.AdminUnsplitRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: splitKey, + }, + } + b.appendReqs(req) + b.initResult(1, 0, notRaw, nil) +} + // adminTransferLease is only exported on DB. It is here for symmetry with the // other operations. func (b *Batch) adminTransferLease(key interface{}, target roachpb.StoreID) { diff --git a/pkg/internal/client/db.go b/pkg/internal/client/db.go index 020086c7be7c..e71867437572 100644 --- a/pkg/internal/client/db.go +++ b/pkg/internal/client/db.go @@ -488,6 +488,19 @@ func (db *DB) AdminSplit(ctx context.Context, spanKey, splitKey interface{}, man return getOneErr(db.Run(ctx, b), b) } +// AdminUnsplit removes the sticky bit of the range specified by splitKey. +// +// splitKey is the start key of the range whose sticky bit should be removed. +// +// If splitKey is not the start key of a range, then this method will throw an +// error. If the range specified by splitKey does not have a sticky bit set, +// then this method will not throw an error and is a no-op. +func (db *DB) AdminUnsplit(ctx context.Context, splitKey interface{}) error { + b := &Batch{} + b.adminUnsplit(splitKey) + return getOneErr(db.Run(ctx, b), b) +} + // AdminTransferLease transfers the lease for the range containing key to the // specified target. The target replica for the lease transfer must be one of // the existing replicas of the range. diff --git a/pkg/kv/split_test.go b/pkg/kv/split_test.go index fb42a9bdc864..2ec998ba9023 100644 --- a/pkg/kv/split_test.go +++ b/pkg/kv/split_test.go @@ -34,7 +34,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/gogo/protobuf/proto" "github.com/pkg/errors" @@ -294,30 +293,8 @@ func TestRangeSplitsStickyBit(t *testing.T) { t.Fatal("Sticky bit not set after splitting") } - // TODO(jeffreyxiao): Use same mechanism as ALTER TABLE ... UNSPLIT AT - // does when it is added. // Removing sticky bit. - desc.StickyBit = nil - if err := s.DB.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { - b := txn.NewBatch() - marshalledDesc, err := protoutil.Marshal(&desc) - if err != nil { - return err - } - b.Put(descKey, marshalledDesc) - b.Put(keys.RangeMetaKey(desc.EndKey).AsRawKey(), marshalledDesc) - // End the transaction manually in order to provide a sticky bit trigger. - // Note that this hack will be removed. See above TODO. - b.AddRawRequest(&roachpb.EndTransactionRequest{ - Commit: true, - InternalCommitTrigger: &roachpb.InternalCommitTrigger{ - StickyBitTrigger: &roachpb.StickyBitTrigger{ - StickyBit: nil, - }, - }, - }) - return txn.Run(ctx, b) - }); err != nil { + if err := s.DB.AdminUnsplit(ctx, splitKey.AsRawKey()); err != nil { t.Fatal(err) } diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 73272e1de85c..7b9eb40df7e4 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -492,6 +492,9 @@ func (*EndTransactionRequest) Method() Method { return EndTransaction } // Method implements the Request interface. func (*AdminSplitRequest) Method() Method { return AdminSplit } +// Method implements the Request interface. +func (*AdminUnsplitRequest) Method() Method { return AdminUnsplit } + // Method implements the Request interface. func (*AdminMergeRequest) Method() Method { return AdminMerge } @@ -660,6 +663,12 @@ func (asr *AdminSplitRequest) ShallowCopy() Request { return &shallowCopy } +// ShallowCopy implements the Request interface. +func (aur *AdminUnsplitRequest) ShallowCopy() Request { + shallowCopy := *aur + return &shallowCopy +} + // ShallowCopy implements the Request interface. func (amr *AdminMergeRequest) ShallowCopy() Request { shallowCopy := *amr @@ -1028,6 +1037,7 @@ func (*BeginTransactionRequest) flags() int { return isWrite | isTxn } // have Txn.WriteTooOld=true and must retry on EndTransaction. func (*EndTransactionRequest) flags() int { return isWrite | isTxn | isAlone | updatesWriteTSCache } func (*AdminSplitRequest) flags() int { return isAdmin | isAlone } +func (*AdminUnsplitRequest) flags() int { return isAdmin | isAlone } func (*AdminMergeRequest) flags() int { return isAdmin | isAlone } func (*AdminTransferLeaseRequest) flags() int { return isAdmin | isAlone } func (*AdminChangeReplicasRequest) flags() int { return isAdmin | isAlone } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index e21851f59993..048ce754539d 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -70,7 +70,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{0} + return fileDescriptor_api_c55003ffcd524477, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -98,7 +98,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{1} + return fileDescriptor_api_c55003ffcd524477, []int{1} } type ChecksumMode int32 @@ -145,7 +145,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{2} + return fileDescriptor_api_c55003ffcd524477, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -176,7 +176,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{3} + return fileDescriptor_api_c55003ffcd524477, []int{3} } type ExportStorageProvider int32 @@ -214,7 +214,7 @@ func (x ExportStorageProvider) String() string { return proto.EnumName(ExportStorageProvider_name, int32(x)) } func (ExportStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{4} + return fileDescriptor_api_c55003ffcd524477, []int{4} } type MVCCFilter int32 @@ -237,7 +237,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{5} + return fileDescriptor_api_c55003ffcd524477, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +269,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{2, 0} + return fileDescriptor_api_c55003ffcd524477, []int{2, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +311,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{25, 0} + return fileDescriptor_api_c55003ffcd524477, []int{25, 0} } // RangeInfo describes a range which executed a request. It contains @@ -327,7 +327,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_api_75ce0d0ce4ff4c16, []int{0} + return fileDescriptor_api_c55003ffcd524477, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -372,7 +372,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_75ce0d0ce4ff4c16, []int{1} + return fileDescriptor_api_c55003ffcd524477, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -432,7 +432,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_75ce0d0ce4ff4c16, []int{2} + return fileDescriptor_api_c55003ffcd524477, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -468,7 +468,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_75ce0d0ce4ff4c16, []int{3} + return fileDescriptor_api_c55003ffcd524477, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -508,7 +508,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_75ce0d0ce4ff4c16, []int{4} + return fileDescriptor_api_c55003ffcd524477, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -553,7 +553,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_75ce0d0ce4ff4c16, []int{5} + return fileDescriptor_api_c55003ffcd524477, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -589,7 +589,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_75ce0d0ce4ff4c16, []int{6} + return fileDescriptor_api_c55003ffcd524477, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -645,7 +645,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_75ce0d0ce4ff4c16, []int{7} + return fileDescriptor_api_c55003ffcd524477, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -682,7 +682,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_75ce0d0ce4ff4c16, []int{8} + return fileDescriptor_api_c55003ffcd524477, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -730,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_75ce0d0ce4ff4c16, []int{9} + return fileDescriptor_api_c55003ffcd524477, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -766,7 +766,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_75ce0d0ce4ff4c16, []int{10} + return fileDescriptor_api_c55003ffcd524477, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -808,7 +808,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_75ce0d0ce4ff4c16, []int{11} + return fileDescriptor_api_c55003ffcd524477, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -847,7 +847,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_75ce0d0ce4ff4c16, []int{12} + return fileDescriptor_api_c55003ffcd524477, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -883,7 +883,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_75ce0d0ce4ff4c16, []int{13} + return fileDescriptor_api_c55003ffcd524477, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -919,7 +919,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_75ce0d0ce4ff4c16, []int{14} + return fileDescriptor_api_c55003ffcd524477, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -970,7 +970,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_75ce0d0ce4ff4c16, []int{15} + return fileDescriptor_api_c55003ffcd524477, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1009,7 +1009,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_75ce0d0ce4ff4c16, []int{16} + return fileDescriptor_api_c55003ffcd524477, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1058,7 +1058,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_75ce0d0ce4ff4c16, []int{17} + return fileDescriptor_api_c55003ffcd524477, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1094,7 +1094,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_75ce0d0ce4ff4c16, []int{18} + return fileDescriptor_api_c55003ffcd524477, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1165,7 +1165,7 @@ func (m *ScanOptions) Reset() { *m = ScanOptions{} } func (m *ScanOptions) String() string { return proto.CompactTextString(m) } func (*ScanOptions) ProtoMessage() {} func (*ScanOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{19} + return fileDescriptor_api_c55003ffcd524477, []int{19} } func (m *ScanOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1207,7 +1207,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{20} + return fileDescriptor_api_c55003ffcd524477, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1256,7 +1256,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_75ce0d0ce4ff4c16, []int{21} + return fileDescriptor_api_c55003ffcd524477, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1298,7 +1298,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_75ce0d0ce4ff4c16, []int{22} + return fileDescriptor_api_c55003ffcd524477, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1347,7 +1347,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_75ce0d0ce4ff4c16, []int{23} + return fileDescriptor_api_c55003ffcd524477, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1395,7 +1395,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_75ce0d0ce4ff4c16, []int{24} + return fileDescriptor_api_c55003ffcd524477, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1434,7 +1434,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_75ce0d0ce4ff4c16, []int{25} + return fileDescriptor_api_c55003ffcd524477, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1480,7 +1480,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_75ce0d0ce4ff4c16, []int{25, 0} + return fileDescriptor_api_c55003ffcd524477, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1530,7 +1530,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_75ce0d0ce4ff4c16, []int{26} + return fileDescriptor_api_c55003ffcd524477, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1568,7 +1568,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_75ce0d0ce4ff4c16, []int{27} + return fileDescriptor_api_c55003ffcd524477, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1604,7 +1604,7 @@ func (m *BeginTransactionRequest) Reset() { *m = BeginTransactionRequest func (m *BeginTransactionRequest) String() string { return proto.CompactTextString(m) } func (*BeginTransactionRequest) ProtoMessage() {} func (*BeginTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{28} + return fileDescriptor_api_c55003ffcd524477, []int{28} } func (m *BeginTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1640,7 +1640,7 @@ func (m *BeginTransactionResponse) Reset() { *m = BeginTransactionRespon func (m *BeginTransactionResponse) String() string { return proto.CompactTextString(m) } func (*BeginTransactionResponse) ProtoMessage() {} func (*BeginTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{29} + return fileDescriptor_api_c55003ffcd524477, []int{29} } func (m *BeginTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1730,7 +1730,7 @@ func (m *EndTransactionRequest) Reset() { *m = EndTransactionRequest{} } func (m *EndTransactionRequest) String() string { return proto.CompactTextString(m) } func (*EndTransactionRequest) ProtoMessage() {} func (*EndTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{30} + return fileDescriptor_api_c55003ffcd524477, []int{30} } func (m *EndTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1779,7 +1779,7 @@ func (m *EndTransactionResponse) Reset() { *m = EndTransactionResponse{} func (m *EndTransactionResponse) String() string { return proto.CompactTextString(m) } func (*EndTransactionResponse) ProtoMessage() {} func (*EndTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{31} + return fileDescriptor_api_c55003ffcd524477, []int{31} } func (m *EndTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1842,7 +1842,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_75ce0d0ce4ff4c16, []int{32} + return fileDescriptor_api_c55003ffcd524477, []int{32} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1879,7 +1879,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_75ce0d0ce4ff4c16, []int{33} + return fileDescriptor_api_c55003ffcd524477, []int{33} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1904,6 +1904,84 @@ func (m *AdminSplitResponse) XXX_DiscardUnknown() { var xxx_messageInfo_AdminSplitResponse proto.InternalMessageInfo +// An AdminUnsplitRequest is the argument to the AdminUnsplit() +// method. The sticky bit of the existing range whose starting key is +// header.key is removed. +// +// Ranges that do not have the sticky bit set are eligible for +// automatic merging. +type AdminUnsplitRequest struct { + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_c55003ffcd524477, []int{34} +} +func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminUnsplitRequest) 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 *AdminUnsplitRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminUnsplitRequest.Merge(dst, src) +} +func (m *AdminUnsplitRequest) XXX_Size() int { + return m.Size() +} +func (m *AdminUnsplitRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AdminUnsplitRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_AdminUnsplitRequest proto.InternalMessageInfo + +// An AdminUnsplitResponse is the return value from the +// AdminUnsplit() method. +type AdminUnsplitResponse struct { + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_c55003ffcd524477, []int{35} +} +func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminUnsplitResponse) 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 *AdminUnsplitResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminUnsplitResponse.Merge(dst, src) +} +func (m *AdminUnsplitResponse) XXX_Size() int { + return m.Size() +} +func (m *AdminUnsplitResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AdminUnsplitResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_AdminUnsplitResponse proto.InternalMessageInfo + // An AdminMergeRequest is the argument to the AdminMerge() method. A // merge is performed by calling AdminMerge on the left-hand range of // two consecutive ranges (i.e. the range which contains keys which @@ -1923,7 +2001,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_75ce0d0ce4ff4c16, []int{34} + return fileDescriptor_api_c55003ffcd524477, []int{36} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1960,7 +2038,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_75ce0d0ce4ff4c16, []int{35} + return fileDescriptor_api_c55003ffcd524477, []int{37} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2000,7 +2078,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_75ce0d0ce4ff4c16, []int{36} + return fileDescriptor_api_c55003ffcd524477, []int{38} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2035,7 +2113,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_75ce0d0ce4ff4c16, []int{37} + return fileDescriptor_api_c55003ffcd524477, []int{39} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2089,7 +2167,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_75ce0d0ce4ff4c16, []int{38} + return fileDescriptor_api_c55003ffcd524477, []int{40} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2128,7 +2206,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_75ce0d0ce4ff4c16, []int{39} + return fileDescriptor_api_c55003ffcd524477, []int{41} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2167,7 +2245,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_75ce0d0ce4ff4c16, []int{40} + return fileDescriptor_api_c55003ffcd524477, []int{42} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2202,7 +2280,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_75ce0d0ce4ff4c16, []int{41} + return fileDescriptor_api_c55003ffcd524477, []int{43} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2243,7 +2321,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_75ce0d0ce4ff4c16, []int{42} + return fileDescriptor_api_c55003ffcd524477, []int{44} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2282,7 +2360,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_75ce0d0ce4ff4c16, []int{43} + return fileDescriptor_api_c55003ffcd524477, []int{45} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2327,7 +2405,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_75ce0d0ce4ff4c16, []int{44} + return fileDescriptor_api_c55003ffcd524477, []int{46} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2363,7 +2441,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_75ce0d0ce4ff4c16, []int{44, 0} + return fileDescriptor_api_c55003ffcd524477, []int{46, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2399,7 +2477,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_75ce0d0ce4ff4c16, []int{45} + return fileDescriptor_api_c55003ffcd524477, []int{47} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2483,7 +2561,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_75ce0d0ce4ff4c16, []int{46} + return fileDescriptor_api_c55003ffcd524477, []int{48} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2528,7 +2606,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_75ce0d0ce4ff4c16, []int{47} + return fileDescriptor_api_c55003ffcd524477, []int{49} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2577,7 +2655,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_75ce0d0ce4ff4c16, []int{48} + return fileDescriptor_api_c55003ffcd524477, []int{50} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2615,7 +2693,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_75ce0d0ce4ff4c16, []int{49} + return fileDescriptor_api_c55003ffcd524477, []int{51} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2661,7 +2739,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_75ce0d0ce4ff4c16, []int{50} + return fileDescriptor_api_c55003ffcd524477, []int{52} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2702,7 +2780,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_75ce0d0ce4ff4c16, []int{51} + return fileDescriptor_api_c55003ffcd524477, []int{53} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2764,7 +2842,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_75ce0d0ce4ff4c16, []int{52} + return fileDescriptor_api_c55003ffcd524477, []int{54} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2802,7 +2880,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_75ce0d0ce4ff4c16, []int{53} + return fileDescriptor_api_c55003ffcd524477, []int{55} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2848,7 +2926,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_75ce0d0ce4ff4c16, []int{54} + return fileDescriptor_api_c55003ffcd524477, []int{56} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2885,7 +2963,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_75ce0d0ce4ff4c16, []int{55} + return fileDescriptor_api_c55003ffcd524477, []int{57} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2935,7 +3013,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_75ce0d0ce4ff4c16, []int{56} + return fileDescriptor_api_c55003ffcd524477, []int{58} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2972,7 +3050,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_75ce0d0ce4ff4c16, []int{57} + return fileDescriptor_api_c55003ffcd524477, []int{59} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3011,7 +3089,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_75ce0d0ce4ff4c16, []int{58} + return fileDescriptor_api_c55003ffcd524477, []int{60} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3047,7 +3125,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_75ce0d0ce4ff4c16, []int{59} + return fileDescriptor_api_c55003ffcd524477, []int{61} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3094,7 +3172,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_75ce0d0ce4ff4c16, []int{60} + return fileDescriptor_api_c55003ffcd524477, []int{62} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3130,7 +3208,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_75ce0d0ce4ff4c16, []int{61} + return fileDescriptor_api_c55003ffcd524477, []int{63} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3176,7 +3254,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_75ce0d0ce4ff4c16, []int{62} + return fileDescriptor_api_c55003ffcd524477, []int{64} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3227,7 +3305,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_75ce0d0ce4ff4c16, []int{63} + return fileDescriptor_api_c55003ffcd524477, []int{65} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3266,7 +3344,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_75ce0d0ce4ff4c16, []int{64} + return fileDescriptor_api_c55003ffcd524477, []int{66} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3305,7 +3383,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_75ce0d0ce4ff4c16, []int{65} + return fileDescriptor_api_c55003ffcd524477, []int{67} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3342,7 +3420,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_75ce0d0ce4ff4c16, []int{66} + return fileDescriptor_api_c55003ffcd524477, []int{68} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3394,7 +3472,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_75ce0d0ce4ff4c16, []int{67} + return fileDescriptor_api_c55003ffcd524477, []int{69} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3433,7 +3511,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_75ce0d0ce4ff4c16, []int{68} + return fileDescriptor_api_c55003ffcd524477, []int{70} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3474,7 +3552,7 @@ func (m *ExportStorage) Reset() { *m = ExportStorage{} } func (m *ExportStorage) String() string { return proto.CompactTextString(m) } func (*ExportStorage) ProtoMessage() {} func (*ExportStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{69} + return fileDescriptor_api_c55003ffcd524477, []int{71} } func (m *ExportStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3510,7 +3588,7 @@ func (m *ExportStorage_LocalFilePath) Reset() { *m = ExportStorage_Local func (m *ExportStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExportStorage_LocalFilePath) ProtoMessage() {} func (*ExportStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{69, 0} + return fileDescriptor_api_c55003ffcd524477, []int{71, 0} } func (m *ExportStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3545,7 +3623,7 @@ func (m *ExportStorage_Http) Reset() { *m = ExportStorage_Http{} } func (m *ExportStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Http) ProtoMessage() {} func (*ExportStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{69, 1} + return fileDescriptor_api_c55003ffcd524477, []int{71, 1} } func (m *ExportStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3586,7 +3664,7 @@ func (m *ExportStorage_S3) Reset() { *m = ExportStorage_S3{} } func (m *ExportStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExportStorage_S3) ProtoMessage() {} func (*ExportStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{69, 2} + return fileDescriptor_api_c55003ffcd524477, []int{71, 2} } func (m *ExportStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3627,7 +3705,7 @@ func (m *ExportStorage_GCS) Reset() { *m = ExportStorage_GCS{} } func (m *ExportStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExportStorage_GCS) ProtoMessage() {} func (*ExportStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{69, 3} + return fileDescriptor_api_c55003ffcd524477, []int{71, 3} } func (m *ExportStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3665,7 +3743,7 @@ func (m *ExportStorage_Azure) Reset() { *m = ExportStorage_Azure{} } func (m *ExportStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Azure) ProtoMessage() {} func (*ExportStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{69, 4} + return fileDescriptor_api_c55003ffcd524477, []int{71, 4} } func (m *ExportStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3706,7 +3784,7 @@ func (m *ExportStorage_Workload) Reset() { *m = ExportStorage_Workload{} func (m *ExportStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Workload) ProtoMessage() {} func (*ExportStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{69, 5} + return fileDescriptor_api_c55003ffcd524477, []int{71, 5} } func (m *ExportStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3748,7 +3826,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_75ce0d0ce4ff4c16, []int{70} + return fileDescriptor_api_c55003ffcd524477, []int{72} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3784,7 +3862,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_75ce0d0ce4ff4c16, []int{71} + return fileDescriptor_api_c55003ffcd524477, []int{73} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3844,7 +3922,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_75ce0d0ce4ff4c16, []int{72} + return fileDescriptor_api_c55003ffcd524477, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3882,7 +3960,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{73} + return fileDescriptor_api_c55003ffcd524477, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3920,7 +3998,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_75ce0d0ce4ff4c16, []int{74} + return fileDescriptor_api_c55003ffcd524477, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3961,7 +4039,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_75ce0d0ce4ff4c16, []int{74, 0} + return fileDescriptor_api_c55003ffcd524477, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4013,7 +4091,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_75ce0d0ce4ff4c16, []int{75} + return fileDescriptor_api_c55003ffcd524477, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4050,7 +4128,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_75ce0d0ce4ff4c16, []int{75, 0} + return fileDescriptor_api_c55003ffcd524477, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4088,7 +4166,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{75, 1} + return fileDescriptor_api_c55003ffcd524477, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4125,7 +4203,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_75ce0d0ce4ff4c16, []int{76} + return fileDescriptor_api_c55003ffcd524477, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4165,7 +4243,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_75ce0d0ce4ff4c16, []int{77} + return fileDescriptor_api_c55003ffcd524477, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4202,7 +4280,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_75ce0d0ce4ff4c16, []int{78} + return fileDescriptor_api_c55003ffcd524477, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4237,7 +4315,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_75ce0d0ce4ff4c16, []int{78, 0} + return fileDescriptor_api_c55003ffcd524477, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4275,7 +4353,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_75ce0d0ce4ff4c16, []int{79} + return fileDescriptor_api_c55003ffcd524477, []int{81} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4311,7 +4389,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_75ce0d0ce4ff4c16, []int{80} + return fileDescriptor_api_c55003ffcd524477, []int{82} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4357,7 +4435,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_75ce0d0ce4ff4c16, []int{81} + return fileDescriptor_api_c55003ffcd524477, []int{83} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4393,7 +4471,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_75ce0d0ce4ff4c16, []int{82} + return fileDescriptor_api_c55003ffcd524477, []int{84} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4435,7 +4513,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_75ce0d0ce4ff4c16, []int{83} + return fileDescriptor_api_c55003ffcd524477, []int{85} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4471,7 +4549,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_75ce0d0ce4ff4c16, []int{84} + return fileDescriptor_api_c55003ffcd524477, []int{86} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4525,7 +4603,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_75ce0d0ce4ff4c16, []int{85} + return fileDescriptor_api_c55003ffcd524477, []int{87} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4572,7 +4650,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{86} + return fileDescriptor_api_c55003ffcd524477, []int{88} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4609,7 +4687,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_75ce0d0ce4ff4c16, []int{87} + return fileDescriptor_api_c55003ffcd524477, []int{89} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4650,7 +4728,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_75ce0d0ce4ff4c16, []int{88} + return fileDescriptor_api_c55003ffcd524477, []int{90} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4693,6 +4771,7 @@ type RequestUnion struct { // *RequestUnion_BeginTransaction // *RequestUnion_EndTransaction // *RequestUnion_AdminSplit + // *RequestUnion_AdminUnsplit // *RequestUnion_AdminMerge // *RequestUnion_AdminTransferLease // *RequestUnion_AdminChangeReplicas @@ -4733,7 +4812,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_75ce0d0ce4ff4c16, []int{89} + return fileDescriptor_api_c55003ffcd524477, []int{91} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4797,6 +4876,9 @@ type RequestUnion_EndTransaction struct { type RequestUnion_AdminSplit struct { AdminSplit *AdminSplitRequest `protobuf:"bytes,10,opt,name=admin_split,json=adminSplit,proto3,oneof"` } +type RequestUnion_AdminUnsplit struct { + AdminUnsplit *AdminUnsplitRequest `protobuf:"bytes,47,opt,name=admin_unsplit,json=adminUnsplit,proto3,oneof"` +} type RequestUnion_AdminMerge struct { AdminMerge *AdminMergeRequest `protobuf:"bytes,11,opt,name=admin_merge,json=adminMerge,proto3,oneof"` } @@ -4902,6 +4984,7 @@ func (*RequestUnion_Scan) isRequestUnion_Value() {} func (*RequestUnion_BeginTransaction) isRequestUnion_Value() {} func (*RequestUnion_EndTransaction) isRequestUnion_Value() {} func (*RequestUnion_AdminSplit) isRequestUnion_Value() {} +func (*RequestUnion_AdminUnsplit) isRequestUnion_Value() {} func (*RequestUnion_AdminMerge) isRequestUnion_Value() {} func (*RequestUnion_AdminTransferLease) isRequestUnion_Value() {} func (*RequestUnion_AdminChangeReplicas) isRequestUnion_Value() {} @@ -5018,6 +5101,13 @@ func (m *RequestUnion) GetAdminSplit() *AdminSplitRequest { return nil } +func (m *RequestUnion) GetAdminUnsplit() *AdminUnsplitRequest { + if x, ok := m.GetValue().(*RequestUnion_AdminUnsplit); ok { + return x.AdminUnsplit + } + return nil +} + func (m *RequestUnion) GetAdminMerge() *AdminMergeRequest { if x, ok := m.GetValue().(*RequestUnion_AdminMerge); ok { return x.AdminMerge @@ -5249,6 +5339,7 @@ func (*RequestUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*RequestUnion_BeginTransaction)(nil), (*RequestUnion_EndTransaction)(nil), (*RequestUnion_AdminSplit)(nil), + (*RequestUnion_AdminUnsplit)(nil), (*RequestUnion_AdminMerge)(nil), (*RequestUnion_AdminTransferLease)(nil), (*RequestUnion_AdminChangeReplicas)(nil), @@ -5342,6 +5433,11 @@ func _RequestUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.AdminSplit); err != nil { return err } + case *RequestUnion_AdminUnsplit: + _ = b.EncodeVarint(47<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.AdminUnsplit); err != nil { + return err + } case *RequestUnion_AdminMerge: _ = b.EncodeVarint(11<<3 | proto.WireBytes) if err := b.EncodeMessage(x.AdminMerge); err != nil { @@ -5595,6 +5691,14 @@ func _RequestUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.B err := b.DecodeMessage(msg) m.Value = &RequestUnion_AdminSplit{msg} return true, err + case 47: // value.admin_unsplit + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(AdminUnsplitRequest) + err := b.DecodeMessage(msg) + m.Value = &RequestUnion_AdminUnsplit{msg} + return true, err case 11: // value.admin_merge if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType @@ -5907,6 +6011,11 @@ func _RequestUnion_OneofSizer(msg proto.Message) (n int) { n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *RequestUnion_AdminUnsplit: + s := proto.Size(x.AdminUnsplit) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case *RequestUnion_AdminMerge: s := proto.Size(x.AdminMerge) n += 1 // tag and wire @@ -6084,6 +6193,7 @@ type ResponseUnion struct { // *ResponseUnion_BeginTransaction // *ResponseUnion_EndTransaction // *ResponseUnion_AdminSplit + // *ResponseUnion_AdminUnsplit // *ResponseUnion_AdminMerge // *ResponseUnion_AdminTransferLease // *ResponseUnion_AdminChangeReplicas @@ -6123,7 +6233,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_75ce0d0ce4ff4c16, []int{90} + return fileDescriptor_api_c55003ffcd524477, []int{92} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6187,6 +6297,9 @@ type ResponseUnion_EndTransaction struct { type ResponseUnion_AdminSplit struct { AdminSplit *AdminSplitResponse `protobuf:"bytes,10,opt,name=admin_split,json=adminSplit,proto3,oneof"` } +type ResponseUnion_AdminUnsplit struct { + AdminUnsplit *AdminUnsplitResponse `protobuf:"bytes,47,opt,name=admin_unsplit,json=adminUnsplit,proto3,oneof"` +} type ResponseUnion_AdminMerge struct { AdminMerge *AdminMergeResponse `protobuf:"bytes,11,opt,name=admin_merge,json=adminMerge,proto3,oneof"` } @@ -6289,6 +6402,7 @@ func (*ResponseUnion_Scan) isResponseUnion_Value() {} func (*ResponseUnion_BeginTransaction) isResponseUnion_Value() {} func (*ResponseUnion_EndTransaction) isResponseUnion_Value() {} func (*ResponseUnion_AdminSplit) isResponseUnion_Value() {} +func (*ResponseUnion_AdminUnsplit) isResponseUnion_Value() {} func (*ResponseUnion_AdminMerge) isResponseUnion_Value() {} func (*ResponseUnion_AdminTransferLease) isResponseUnion_Value() {} func (*ResponseUnion_AdminChangeReplicas) isResponseUnion_Value() {} @@ -6404,6 +6518,13 @@ func (m *ResponseUnion) GetAdminSplit() *AdminSplitResponse { return nil } +func (m *ResponseUnion) GetAdminUnsplit() *AdminUnsplitResponse { + if x, ok := m.GetValue().(*ResponseUnion_AdminUnsplit); ok { + return x.AdminUnsplit + } + return nil +} + func (m *ResponseUnion) GetAdminMerge() *AdminMergeResponse { if x, ok := m.GetValue().(*ResponseUnion_AdminMerge); ok { return x.AdminMerge @@ -6628,6 +6749,7 @@ func (*ResponseUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*ResponseUnion_BeginTransaction)(nil), (*ResponseUnion_EndTransaction)(nil), (*ResponseUnion_AdminSplit)(nil), + (*ResponseUnion_AdminUnsplit)(nil), (*ResponseUnion_AdminMerge)(nil), (*ResponseUnion_AdminTransferLease)(nil), (*ResponseUnion_AdminChangeReplicas)(nil), @@ -6720,6 +6842,11 @@ func _ResponseUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.AdminSplit); err != nil { return err } + case *ResponseUnion_AdminUnsplit: + _ = b.EncodeVarint(47<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.AdminUnsplit); err != nil { + return err + } case *ResponseUnion_AdminMerge: _ = b.EncodeVarint(11<<3 | proto.WireBytes) if err := b.EncodeMessage(x.AdminMerge); err != nil { @@ -6968,6 +7095,14 @@ func _ResponseUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto. err := b.DecodeMessage(msg) m.Value = &ResponseUnion_AdminSplit{msg} return true, err + case 47: // value.admin_unsplit + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(AdminUnsplitResponse) + err := b.DecodeMessage(msg) + m.Value = &ResponseUnion_AdminUnsplit{msg} + return true, err case 11: // value.admin_merge if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType @@ -7272,6 +7407,11 @@ func _ResponseUnion_OneofSizer(msg proto.Message) (n int) { n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *ResponseUnion_AdminUnsplit: + s := proto.Size(x.AdminUnsplit) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case *ResponseUnion_AdminMerge: s := proto.Size(x.AdminMerge) n += 1 // tag and wire @@ -7502,7 +7642,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_75ce0d0ce4ff4c16, []int{91} + return fileDescriptor_api_c55003ffcd524477, []int{93} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7540,7 +7680,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{92} + return fileDescriptor_api_c55003ffcd524477, []int{94} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7579,7 +7719,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_75ce0d0ce4ff4c16, []int{93} + return fileDescriptor_api_c55003ffcd524477, []int{95} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7640,7 +7780,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_75ce0d0ce4ff4c16, []int{93, 0} + return fileDescriptor_api_c55003ffcd524477, []int{95, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7678,7 +7818,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_75ce0d0ce4ff4c16, []int{94} + return fileDescriptor_api_c55003ffcd524477, []int{96} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7716,7 +7856,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_75ce0d0ce4ff4c16, []int{95} + return fileDescriptor_api_c55003ffcd524477, []int{97} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7759,7 +7899,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_75ce0d0ce4ff4c16, []int{96} + return fileDescriptor_api_c55003ffcd524477, []int{98} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7798,7 +7938,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_75ce0d0ce4ff4c16, []int{97} + return fileDescriptor_api_c55003ffcd524477, []int{99} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7837,7 +7977,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_75ce0d0ce4ff4c16, []int{98} + return fileDescriptor_api_c55003ffcd524477, []int{100} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7898,6 +8038,8 @@ func init() { proto.RegisterType((*EndTransactionResponse)(nil), "cockroach.roachpb.EndTransactionResponse") proto.RegisterType((*AdminSplitRequest)(nil), "cockroach.roachpb.AdminSplitRequest") proto.RegisterType((*AdminSplitResponse)(nil), "cockroach.roachpb.AdminSplitResponse") + proto.RegisterType((*AdminUnsplitRequest)(nil), "cockroach.roachpb.AdminUnsplitRequest") + proto.RegisterType((*AdminUnsplitResponse)(nil), "cockroach.roachpb.AdminUnsplitResponse") proto.RegisterType((*AdminMergeRequest)(nil), "cockroach.roachpb.AdminMergeRequest") proto.RegisterType((*AdminMergeResponse)(nil), "cockroach.roachpb.AdminMergeResponse") proto.RegisterType((*AdminTransferLeaseRequest)(nil), "cockroach.roachpb.AdminTransferLeaseRequest") @@ -8498,6 +8640,30 @@ func (this *AdminSplitRequest) Equal(that interface{}) bool { } return true } +func (this *AdminUnsplitRequest) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*AdminUnsplitRequest) + if !ok { + that2, ok := that.(AdminUnsplitRequest) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if !this.RequestHeader.Equal(&that1.RequestHeader) { + return false + } + return true +} func (this *AdminMergeRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -11208,7 +11374,7 @@ func (m *AdminSplitResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *AdminMergeRequest) Marshal() (dAtA []byte, err error) { +func (m *AdminUnsplitRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -11218,7 +11384,7 @@ func (m *AdminMergeRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *AdminMergeRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *AdminUnsplitRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -11234,7 +11400,7 @@ func (m *AdminMergeRequest) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *AdminMergeResponse) Marshal() (dAtA []byte, err error) { +func (m *AdminUnsplitResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -11244,7 +11410,7 @@ func (m *AdminMergeResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *AdminMergeResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *AdminUnsplitResponse) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -11260,7 +11426,7 @@ func (m *AdminMergeResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *AdminTransferLeaseRequest) Marshal() (dAtA []byte, err error) { +func (m *AdminMergeRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -11270,7 +11436,7 @@ func (m *AdminTransferLeaseRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *AdminTransferLeaseRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *AdminMergeRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -11283,6 +11449,58 @@ func (m *AdminTransferLeaseRequest) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n47 + return i, nil +} + +func (m *AdminMergeResponse) 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 *AdminMergeResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) + n48, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n48 + return i, nil +} + +func (m *AdminTransferLeaseRequest) 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 *AdminTransferLeaseRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) + n49, err := m.RequestHeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n49 if m.Target != 0 { dAtA[i] = 0x10 i++ @@ -11309,11 +11527,11 @@ func (m *AdminTransferLeaseResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n48, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n50, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n50 return i, nil } @@ -11335,11 +11553,11 @@ func (m *AdminChangeReplicasRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n49, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n51, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n51 if m.ChangeType != 0 { dAtA[i] = 0x10 i++ @@ -11361,11 +11579,11 @@ func (m *AdminChangeReplicasRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.ExpDesc.Size())) - n50, err := m.ExpDesc.MarshalTo(dAtA[i:]) + n52, err := m.ExpDesc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n52 } return i, nil } @@ -11388,20 +11606,20 @@ func (m *AdminChangeReplicasResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n51, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n53, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n53 if m.Desc != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Desc.Size())) - n52, err := m.Desc.MarshalTo(dAtA[i:]) + n54, err := m.Desc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n54 } return i, nil } @@ -11424,11 +11642,11 @@ func (m *AdminRelocateRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n53, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n55, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n55 if len(m.Targets) > 0 { for _, msg := range m.Targets { dAtA[i] = 0x12 @@ -11462,11 +11680,11 @@ func (m *AdminRelocateRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n54, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n56, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n56 return i, nil } @@ -11488,19 +11706,19 @@ func (m *HeartbeatTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n55, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n57, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n57 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n56, err := m.Now.MarshalTo(dAtA[i:]) + n58, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n58 return i, nil } @@ -11522,11 +11740,11 @@ func (m *HeartbeatTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n57, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n59, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n59 return i, nil } @@ -11548,11 +11766,11 @@ func (m *GCRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n58, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n60, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n60 if len(m.Keys) > 0 { for _, msg := range m.Keys { dAtA[i] = 0x1a @@ -11568,19 +11786,19 @@ func (m *GCRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Threshold.Size())) - n59, err := m.Threshold.MarshalTo(dAtA[i:]) + n61, err := m.Threshold.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n59 + i += n61 dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.TxnSpanGCThreshold.Size())) - n60, err := m.TxnSpanGCThreshold.MarshalTo(dAtA[i:]) + n62, err := m.TxnSpanGCThreshold.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n62 return i, nil } @@ -11608,11 +11826,11 @@ func (m *GCRequest_GCKey) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n61, err := m.Timestamp.MarshalTo(dAtA[i:]) + n63, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n63 return i, nil } @@ -11634,11 +11852,11 @@ func (m *GCResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n62, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n64, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n64 return i, nil } @@ -11660,43 +11878,43 @@ func (m *PushTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n63, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n65, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n63 + i += n65 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.PusherTxn.Size())) - n64, err := m.PusherTxn.MarshalTo(dAtA[i:]) + n66, err := m.PusherTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n64 + i += n66 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PusheeTxn.Size())) - n65, err := m.PusheeTxn.MarshalTo(dAtA[i:]) + n67, err := m.PusheeTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n65 + i += n67 dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTo.Size())) - n66, err := m.PushTo.MarshalTo(dAtA[i:]) + n68, err := m.PushTo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n66 + i += n68 dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.DeprecatedNow.Size())) - n67, err := m.DeprecatedNow.MarshalTo(dAtA[i:]) + n69, err := m.DeprecatedNow.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n67 + i += n69 if m.PushType != 0 { dAtA[i] = 0x30 i++ @@ -11743,19 +11961,19 @@ func (m *PushTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n68, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n70, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n68 + i += n70 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.PusheeTxn.Size())) - n69, err := m.PusheeTxn.MarshalTo(dAtA[i:]) + n71, err := m.PusheeTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n69 + i += n71 return i, nil } @@ -11777,19 +11995,19 @@ func (m *RecoverTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n70, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n72, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n70 + i += n72 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n71, err := m.Txn.MarshalTo(dAtA[i:]) + n73, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n71 + i += n73 if m.ImplicitlyCommitted { dAtA[i] = 0x18 i++ @@ -11821,19 +12039,19 @@ func (m *RecoverTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n72, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n74, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n72 + i += n74 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoveredTxn.Size())) - n73, err := m.RecoveredTxn.MarshalTo(dAtA[i:]) + n75, err := m.RecoveredTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n73 + i += n75 return i, nil } @@ -11855,19 +12073,19 @@ func (m *QueryTxnRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n74, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n76, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n74 + i += n76 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n75, err := m.Txn.MarshalTo(dAtA[i:]) + n77, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n75 + i += n77 if m.WaitForUpdate { dAtA[i] = 0x18 i++ @@ -11911,19 +12129,19 @@ func (m *QueryTxnResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n76, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n78, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n76 + i += n78 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueriedTxn.Size())) - n77, err := m.QueriedTxn.MarshalTo(dAtA[i:]) + n79, err := m.QueriedTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n77 + i += n79 if len(m.WaitingTxns) > 0 { for _, msg := range m.WaitingTxns { dAtA[i] = 0x1a @@ -11957,19 +12175,19 @@ func (m *QueryIntentRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n78, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n80, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n78 + i += n80 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n79, err := m.Txn.MarshalTo(dAtA[i:]) + n81, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n79 + i += n81 if m.ErrorIfMissing { dAtA[i] = 0x18 i++ @@ -12001,11 +12219,11 @@ func (m *QueryIntentResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n80, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n82, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n80 + i += n82 if m.FoundIntent { dAtA[i] = 0x10 i++ @@ -12037,19 +12255,19 @@ func (m *ResolveIntentRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n81, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n83, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n81 + i += n83 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.IntentTxn.Size())) - n82, err := m.IntentTxn.MarshalTo(dAtA[i:]) + n84, err := m.IntentTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n82 + i += n84 if m.Status != 0 { dAtA[i] = 0x18 i++ @@ -12086,11 +12304,11 @@ func (m *ResolveIntentResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n83, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n85, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n83 + i += n85 return i, nil } @@ -12112,19 +12330,19 @@ func (m *ResolveIntentRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n84, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n86, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n84 + i += n86 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.IntentTxn.Size())) - n85, err := m.IntentTxn.MarshalTo(dAtA[i:]) + n87, err := m.IntentTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n85 + i += n87 if m.Status != 0 { dAtA[i] = 0x18 i++ @@ -12143,11 +12361,11 @@ func (m *ResolveIntentRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.MinTimestamp.Size())) - n86, err := m.MinTimestamp.MarshalTo(dAtA[i:]) + n88, err := m.MinTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n86 + i += n88 return i, nil } @@ -12169,11 +12387,11 @@ func (m *ResolveIntentRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n87, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n89, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n87 + i += n89 return i, nil } @@ -12195,19 +12413,19 @@ func (m *MergeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n88, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n90, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n88 + i += n90 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n89, err := m.Value.MarshalTo(dAtA[i:]) + n91, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n89 + i += n91 return i, nil } @@ -12229,11 +12447,11 @@ func (m *MergeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n90, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n92, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n90 + i += n92 return i, nil } @@ -12255,11 +12473,11 @@ func (m *TruncateLogRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n91, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n93, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n91 + i += n93 if m.Index != 0 { dAtA[i] = 0x10 i++ @@ -12291,11 +12509,11 @@ func (m *TruncateLogResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n92, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n94, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n92 + i += n94 return i, nil } @@ -12317,36 +12535,36 @@ func (m *RequestLeaseRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n93, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n95, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n93 + i += n95 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n94, err := m.Lease.MarshalTo(dAtA[i:]) + n96, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n94 + i += n96 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevLease.Size())) - n95, err := m.PrevLease.MarshalTo(dAtA[i:]) + n97, err := m.PrevLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n95 + i += n97 if m.MinProposedTS != nil { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.MinProposedTS.Size())) - n96, err := m.MinProposedTS.MarshalTo(dAtA[i:]) + n98, err := m.MinProposedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n96 + i += n98 } return i, nil } @@ -12369,27 +12587,27 @@ func (m *TransferLeaseRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n97, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n99, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n97 + i += n99 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n98, err := m.Lease.MarshalTo(dAtA[i:]) + n100, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n98 + i += n100 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevLease.Size())) - n99, err := m.PrevLease.MarshalTo(dAtA[i:]) + n101, err := m.PrevLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n99 + i += n101 return i, nil } @@ -12411,11 +12629,11 @@ func (m *LeaseInfoRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n100, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n102, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n100 + i += n102 return i, nil } @@ -12437,19 +12655,19 @@ func (m *LeaseInfoResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n101, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n103, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n101 + i += n103 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Lease.Size())) - n102, err := m.Lease.MarshalTo(dAtA[i:]) + n104, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n102 + i += n104 return i, nil } @@ -12471,11 +12689,11 @@ func (m *RequestLeaseResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n103, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n105, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n103 + i += n105 return i, nil } @@ -12497,11 +12715,11 @@ func (m *ComputeChecksumRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n104, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n106, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n104 + i += n106 if m.Version != 0 { dAtA[i] = 0x10 i++ @@ -12553,19 +12771,19 @@ func (m *ComputeChecksumResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n105, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n107, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n105 + i += n107 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ChecksumID.Size())) - n106, err := m.ChecksumID.MarshalTo(dAtA[i:]) + n108, err := m.ChecksumID.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n106 + i += n108 return i, nil } @@ -12592,58 +12810,58 @@ func (m *ExportStorage) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.LocalFile.Size())) - n107, err := m.LocalFile.MarshalTo(dAtA[i:]) + n109, err := m.LocalFile.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n107 + i += n109 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.HttpPath.Size())) - n108, err := m.HttpPath.MarshalTo(dAtA[i:]) + n110, err := m.HttpPath.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n108 + i += n110 if m.GoogleCloudConfig != nil { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.GoogleCloudConfig.Size())) - n109, err := m.GoogleCloudConfig.MarshalTo(dAtA[i:]) + n111, err := m.GoogleCloudConfig.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n109 + i += n111 } if m.S3Config != nil { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.S3Config.Size())) - n110, err := m.S3Config.MarshalTo(dAtA[i:]) + n112, err := m.S3Config.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n110 + i += n112 } if m.AzureConfig != nil { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.AzureConfig.Size())) - n111, err := m.AzureConfig.MarshalTo(dAtA[i:]) + n113, err := m.AzureConfig.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n111 + i += n113 } if m.WorkloadConfig != nil { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.WorkloadConfig.Size())) - n112, err := m.WorkloadConfig.MarshalTo(dAtA[i:]) + n114, err := m.WorkloadConfig.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n112 + i += n114 } return i, nil } @@ -12936,19 +13154,19 @@ func (m *WriteBatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n113, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n115, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n113 + i += n115 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.DataSpan.Size())) - n114, err := m.DataSpan.MarshalTo(dAtA[i:]) + n116, err := m.DataSpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n114 + i += n116 if len(m.Data) > 0 { dAtA[i] = 0x1a i++ @@ -12976,11 +13194,11 @@ func (m *WriteBatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n115, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n117, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n115 + i += n117 return i, nil } @@ -13002,27 +13220,27 @@ func (m *ExportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n116, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n118, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n116 + i += n118 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Storage.Size())) - n117, err := m.Storage.MarshalTo(dAtA[i:]) + n119, err := m.Storage.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n117 + i += n119 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.StartTime.Size())) - n118, err := m.StartTime.MarshalTo(dAtA[i:]) + n120, err := m.StartTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n118 + i += n120 if m.MVCCFilter != 0 { dAtA[i] = 0x20 i++ @@ -13117,11 +13335,11 @@ func (m *ExportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n119, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n121, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n119 + i += n121 if len(m.Files) > 0 { for _, msg := range m.Files { dAtA[i] = 0x12 @@ -13137,11 +13355,11 @@ func (m *ExportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.StartTime.Size())) - n120, err := m.StartTime.MarshalTo(dAtA[i:]) + n122, err := m.StartTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n120 + i += n122 return i, nil } @@ -13163,11 +13381,11 @@ func (m *ExportResponse_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n121, err := m.Span.MarshalTo(dAtA[i:]) + n123, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n121 + i += n123 if len(m.Path) > 0 { dAtA[i] = 0x12 i++ @@ -13183,11 +13401,11 @@ func (m *ExportResponse_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.Exported.Size())) - n122, err := m.Exported.MarshalTo(dAtA[i:]) + n124, err := m.Exported.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n122 + i += n124 if len(m.SST) > 0 { dAtA[i] = 0x3a i++ @@ -13215,11 +13433,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n123, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n125, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n123 + i += n125 if len(m.Files) > 0 { for _, msg := range m.Files { dAtA[i] = 0x12 @@ -13235,11 +13453,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.DataSpan.Size())) - n124, err := m.DataSpan.MarshalTo(dAtA[i:]) + n126, err := m.DataSpan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n124 + i += n126 if len(m.Rekeys) > 0 { for _, msg := range m.Rekeys { dAtA[i] = 0x2a @@ -13255,11 +13473,11 @@ func (m *ImportRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTime.Size())) - n125, err := m.EndTime.MarshalTo(dAtA[i:]) + n127, err := m.EndTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n125 + i += n127 return i, nil } @@ -13281,11 +13499,11 @@ func (m *ImportRequest_File) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Dir.Size())) - n126, err := m.Dir.MarshalTo(dAtA[i:]) + n128, err := m.Dir.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n126 + i += n128 if len(m.Path) > 0 { dAtA[i] = 0x12 i++ @@ -13348,19 +13566,19 @@ func (m *ImportResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n127, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n129, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n127 + i += n129 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Imported.Size())) - n128, err := m.Imported.MarshalTo(dAtA[i:]) + n130, err := m.Imported.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n128 + i += n130 return i, nil } @@ -13382,11 +13600,11 @@ func (m *AdminScatterRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n129, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n131, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n129 + i += n131 if m.RandomizeLeases { dAtA[i] = 0x10 i++ @@ -13418,11 +13636,11 @@ func (m *AdminScatterResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n130, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n132, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n130 + i += n132 if len(m.Ranges) > 0 { for _, msg := range m.Ranges { dAtA[i] = 0x12 @@ -13456,11 +13674,11 @@ func (m *AdminScatterResponse_Range) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n131, err := m.Span.MarshalTo(dAtA[i:]) + n133, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n131 + i += n133 return i, nil } @@ -13482,11 +13700,11 @@ func (m *AddSSTableRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n132, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n134, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n132 + i += n134 if len(m.Data) > 0 { dAtA[i] = 0x12 i++ @@ -13514,11 +13732,11 @@ func (m *AddSSTableResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n133, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n135, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n133 + i += n135 return i, nil } @@ -13540,11 +13758,11 @@ func (m *RefreshRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n134, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n136, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n134 + i += n136 if m.Write { dAtA[i] = 0x10 i++ @@ -13576,11 +13794,11 @@ func (m *RefreshResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n135, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n137, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n135 + i += n137 return i, nil } @@ -13602,11 +13820,11 @@ func (m *RefreshRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n136, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n138, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n136 + i += n138 if m.Write { dAtA[i] = 0x10 i++ @@ -13638,11 +13856,11 @@ func (m *RefreshRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n137, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n139, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n137 + i += n139 return i, nil } @@ -13664,28 +13882,28 @@ func (m *SubsumeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n138, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n140, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n138 + i += n140 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeftDesc.Size())) - n139, err := m.LeftDesc.MarshalTo(dAtA[i:]) + n141, err := m.LeftDesc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n139 + i += n141 if m.RightDesc != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.RightDesc.Size())) - n140, err := m.RightDesc.MarshalTo(dAtA[i:]) + n142, err := m.RightDesc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n140 + i += n142 } return i, nil } @@ -13708,19 +13926,19 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n141, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n143, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n141 + i += n143 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n142, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n144, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n142 + i += n144 if m.LeaseAppliedIndex != 0 { dAtA[i] = 0x20 i++ @@ -13729,11 +13947,11 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.FreezeStart.Size())) - n143, err := m.FreezeStart.MarshalTo(dAtA[i:]) + n145, err := m.FreezeStart.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n143 + i += n145 return i, nil } @@ -13755,11 +13973,11 @@ func (m *RangeStatsRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n144, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n146, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n144 + i += n146 return i, nil } @@ -13781,19 +13999,19 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n145, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n147, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n145 + i += n147 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n146, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n148, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n146 + i += n148 if m.QueriesPerSecond != 0 { dAtA[i] = 0x19 i++ @@ -13819,11 +14037,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn147, err := m.Value.MarshalTo(dAtA[i:]) + nn149, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn147 + i += nn149 } return i, nil } @@ -13834,11 +14052,11 @@ func (m *RequestUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n148, err := m.Get.MarshalTo(dAtA[i:]) + n150, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n148 + i += n150 } return i, nil } @@ -13848,11 +14066,11 @@ func (m *RequestUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n149, err := m.Put.MarshalTo(dAtA[i:]) + n151, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n149 + i += n151 } return i, nil } @@ -13862,11 +14080,11 @@ func (m *RequestUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n150, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n152, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n150 + i += n152 } return i, nil } @@ -13876,11 +14094,11 @@ func (m *RequestUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n151, err := m.Increment.MarshalTo(dAtA[i:]) + n153, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n151 + i += n153 } return i, nil } @@ -13890,11 +14108,11 @@ func (m *RequestUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n152, err := m.Delete.MarshalTo(dAtA[i:]) + n154, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n152 + i += n154 } return i, nil } @@ -13904,11 +14122,11 @@ func (m *RequestUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n153, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n155, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n153 + i += n155 } return i, nil } @@ -13918,11 +14136,11 @@ func (m *RequestUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n154, err := m.Scan.MarshalTo(dAtA[i:]) + n156, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n154 + i += n156 } return i, nil } @@ -13932,11 +14150,11 @@ func (m *RequestUnion_BeginTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.BeginTransaction.Size())) - n155, err := m.BeginTransaction.MarshalTo(dAtA[i:]) + n157, err := m.BeginTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n155 + i += n157 } return i, nil } @@ -13946,11 +14164,11 @@ func (m *RequestUnion_EndTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTransaction.Size())) - n156, err := m.EndTransaction.MarshalTo(dAtA[i:]) + n158, err := m.EndTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n156 + i += n158 } return i, nil } @@ -13960,11 +14178,11 @@ func (m *RequestUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n157, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n159, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n157 + i += n159 } return i, nil } @@ -13974,11 +14192,11 @@ func (m *RequestUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n158, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n160, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n158 + i += n160 } return i, nil } @@ -13988,11 +14206,11 @@ func (m *RequestUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n159, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n161, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n159 + i += n161 } return i, nil } @@ -14002,11 +14220,11 @@ func (m *RequestUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n160, err := m.Gc.MarshalTo(dAtA[i:]) + n162, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n160 + i += n162 } return i, nil } @@ -14016,11 +14234,11 @@ func (m *RequestUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n161, err := m.PushTxn.MarshalTo(dAtA[i:]) + n163, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n161 + i += n163 } return i, nil } @@ -14032,11 +14250,11 @@ func (m *RequestUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n162, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n164, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n162 + i += n164 } return i, nil } @@ -14048,11 +14266,11 @@ func (m *RequestUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n163, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n165, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n163 + i += n165 } return i, nil } @@ -14064,11 +14282,11 @@ func (m *RequestUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n164, err := m.Merge.MarshalTo(dAtA[i:]) + n166, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n164 + i += n166 } return i, nil } @@ -14080,11 +14298,11 @@ func (m *RequestUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n165, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n167, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n165 + i += n167 } return i, nil } @@ -14096,11 +14314,11 @@ func (m *RequestUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n166, err := m.RequestLease.MarshalTo(dAtA[i:]) + n168, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n166 + i += n168 } return i, nil } @@ -14112,11 +14330,11 @@ func (m *RequestUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n167, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n169, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n167 + i += n169 } return i, nil } @@ -14128,11 +14346,11 @@ func (m *RequestUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n168, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n170, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n168 + i += n170 } return i, nil } @@ -14144,11 +14362,11 @@ func (m *RequestUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n169, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n171, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n169 + i += n171 } return i, nil } @@ -14160,11 +14378,11 @@ func (m *RequestUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n170, err := m.InitPut.MarshalTo(dAtA[i:]) + n172, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n170 + i += n172 } return i, nil } @@ -14176,11 +14394,11 @@ func (m *RequestUnion_TransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TransferLease.Size())) - n171, err := m.TransferLease.MarshalTo(dAtA[i:]) + n173, err := m.TransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n171 + i += n173 } return i, nil } @@ -14192,11 +14410,11 @@ func (m *RequestUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n172, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n174, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n172 + i += n174 } return i, nil } @@ -14208,11 +14426,11 @@ func (m *RequestUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n173, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n175, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n173 + i += n175 } return i, nil } @@ -14224,11 +14442,11 @@ func (m *RequestUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n174, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n176, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n174 + i += n176 } return i, nil } @@ -14240,11 +14458,11 @@ func (m *RequestUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n175, err := m.Export.MarshalTo(dAtA[i:]) + n177, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n175 + i += n177 } return i, nil } @@ -14256,11 +14474,11 @@ func (m *RequestUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n176, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n178, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n176 + i += n178 } return i, nil } @@ -14272,11 +14490,11 @@ func (m *RequestUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n177, err := m.Import.MarshalTo(dAtA[i:]) + n179, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n177 + i += n179 } return i, nil } @@ -14288,11 +14506,11 @@ func (m *RequestUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n178, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n180, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n178 + i += n180 } return i, nil } @@ -14304,11 +14522,11 @@ func (m *RequestUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n179, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n181, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n179 + i += n181 } return i, nil } @@ -14320,11 +14538,11 @@ func (m *RequestUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n180, err := m.AddSstable.MarshalTo(dAtA[i:]) + n182, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n180 + i += n182 } return i, nil } @@ -14336,11 +14554,11 @@ func (m *RequestUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n181, err := m.ClearRange.MarshalTo(dAtA[i:]) + n183, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n181 + i += n183 } return i, nil } @@ -14352,11 +14570,11 @@ func (m *RequestUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n182, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n184, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n182 + i += n184 } return i, nil } @@ -14368,11 +14586,11 @@ func (m *RequestUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n183, err := m.Refresh.MarshalTo(dAtA[i:]) + n185, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n183 + i += n185 } return i, nil } @@ -14384,11 +14602,11 @@ func (m *RequestUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n184, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n186, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n184 + i += n186 } return i, nil } @@ -14400,11 +14618,11 @@ func (m *RequestUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n185, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n187, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n185 + i += n187 } return i, nil } @@ -14416,11 +14634,11 @@ func (m *RequestUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n186, err := m.Subsume.MarshalTo(dAtA[i:]) + n188, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n186 + i += n188 } return i, nil } @@ -14432,11 +14650,11 @@ func (m *RequestUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n187, err := m.RangeStats.MarshalTo(dAtA[i:]) + n189, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n187 + i += n189 } return i, nil } @@ -14448,11 +14666,11 @@ func (m *RequestUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n188, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n190, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n188 + i += n190 } return i, nil } @@ -14464,11 +14682,27 @@ func (m *RequestUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n189, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n191, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n189 + i += n191 + } + return i, nil +} +func (m *RequestUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.AdminUnsplit != nil { + dAtA[i] = 0xfa + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size())) + n192, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n192 } return i, nil } @@ -14488,11 +14722,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn190, err := m.Value.MarshalTo(dAtA[i:]) + nn193, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn190 + i += nn193 } return i, nil } @@ -14503,11 +14737,11 @@ func (m *ResponseUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n191, err := m.Get.MarshalTo(dAtA[i:]) + n194, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n191 + i += n194 } return i, nil } @@ -14517,11 +14751,11 @@ func (m *ResponseUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n192, err := m.Put.MarshalTo(dAtA[i:]) + n195, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n192 + i += n195 } return i, nil } @@ -14531,11 +14765,11 @@ func (m *ResponseUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n193, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n196, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n193 + i += n196 } return i, nil } @@ -14545,11 +14779,11 @@ func (m *ResponseUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n194, err := m.Increment.MarshalTo(dAtA[i:]) + n197, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n194 + i += n197 } return i, nil } @@ -14559,11 +14793,11 @@ func (m *ResponseUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n195, err := m.Delete.MarshalTo(dAtA[i:]) + n198, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n195 + i += n198 } return i, nil } @@ -14573,11 +14807,11 @@ func (m *ResponseUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n196, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n199, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n196 + i += n199 } return i, nil } @@ -14587,11 +14821,11 @@ func (m *ResponseUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n197, err := m.Scan.MarshalTo(dAtA[i:]) + n200, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n197 + i += n200 } return i, nil } @@ -14601,11 +14835,11 @@ func (m *ResponseUnion_BeginTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.BeginTransaction.Size())) - n198, err := m.BeginTransaction.MarshalTo(dAtA[i:]) + n201, err := m.BeginTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n198 + i += n201 } return i, nil } @@ -14615,11 +14849,11 @@ func (m *ResponseUnion_EndTransaction) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTransaction.Size())) - n199, err := m.EndTransaction.MarshalTo(dAtA[i:]) + n202, err := m.EndTransaction.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n199 + i += n202 } return i, nil } @@ -14629,11 +14863,11 @@ func (m *ResponseUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n200, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n203, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n200 + i += n203 } return i, nil } @@ -14643,11 +14877,11 @@ func (m *ResponseUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n201, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n204, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n201 + i += n204 } return i, nil } @@ -14657,11 +14891,11 @@ func (m *ResponseUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n202, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n205, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n202 + i += n205 } return i, nil } @@ -14671,11 +14905,11 @@ func (m *ResponseUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n203, err := m.Gc.MarshalTo(dAtA[i:]) + n206, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n203 + i += n206 } return i, nil } @@ -14685,11 +14919,11 @@ func (m *ResponseUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n204, err := m.PushTxn.MarshalTo(dAtA[i:]) + n207, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n204 + i += n207 } return i, nil } @@ -14701,11 +14935,11 @@ func (m *ResponseUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n205, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n208, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n205 + i += n208 } return i, nil } @@ -14717,11 +14951,11 @@ func (m *ResponseUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n206, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n209, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n206 + i += n209 } return i, nil } @@ -14733,11 +14967,11 @@ func (m *ResponseUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n207, err := m.Merge.MarshalTo(dAtA[i:]) + n210, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n207 + i += n210 } return i, nil } @@ -14749,11 +14983,11 @@ func (m *ResponseUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n208, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n211, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n208 + i += n211 } return i, nil } @@ -14765,11 +14999,11 @@ func (m *ResponseUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n209, err := m.RequestLease.MarshalTo(dAtA[i:]) + n212, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n209 + i += n212 } return i, nil } @@ -14781,11 +15015,11 @@ func (m *ResponseUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n210, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n213, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n210 + i += n213 } return i, nil } @@ -14797,11 +15031,11 @@ func (m *ResponseUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n211, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n214, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n211 + i += n214 } return i, nil } @@ -14813,11 +15047,11 @@ func (m *ResponseUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n212, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n215, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n212 + i += n215 } return i, nil } @@ -14829,11 +15063,11 @@ func (m *ResponseUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n213, err := m.InitPut.MarshalTo(dAtA[i:]) + n216, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n213 + i += n216 } return i, nil } @@ -14845,11 +15079,11 @@ func (m *ResponseUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n214, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n217, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n214 + i += n217 } return i, nil } @@ -14861,11 +15095,11 @@ func (m *ResponseUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n215, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n218, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n215 + i += n218 } return i, nil } @@ -14877,11 +15111,11 @@ func (m *ResponseUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n216, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n219, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n216 + i += n219 } return i, nil } @@ -14893,11 +15127,11 @@ func (m *ResponseUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n217, err := m.Export.MarshalTo(dAtA[i:]) + n220, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n217 + i += n220 } return i, nil } @@ -14909,11 +15143,11 @@ func (m *ResponseUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n218, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n221, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n218 + i += n221 } return i, nil } @@ -14925,11 +15159,11 @@ func (m *ResponseUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n219, err := m.Import.MarshalTo(dAtA[i:]) + n222, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n219 + i += n222 } return i, nil } @@ -14941,11 +15175,11 @@ func (m *ResponseUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n220, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n223, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n220 + i += n223 } return i, nil } @@ -14957,11 +15191,11 @@ func (m *ResponseUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n221, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n224, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n221 + i += n224 } return i, nil } @@ -14973,11 +15207,11 @@ func (m *ResponseUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n222, err := m.AddSstable.MarshalTo(dAtA[i:]) + n225, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n222 + i += n225 } return i, nil } @@ -14989,11 +15223,11 @@ func (m *ResponseUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n223, err := m.ClearRange.MarshalTo(dAtA[i:]) + n226, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n223 + i += n226 } return i, nil } @@ -15005,11 +15239,11 @@ func (m *ResponseUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n224, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n227, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n224 + i += n227 } return i, nil } @@ -15021,11 +15255,11 @@ func (m *ResponseUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n225, err := m.Refresh.MarshalTo(dAtA[i:]) + n228, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n225 + i += n228 } return i, nil } @@ -15037,11 +15271,11 @@ func (m *ResponseUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n226, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n229, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n226 + i += n229 } return i, nil } @@ -15053,11 +15287,11 @@ func (m *ResponseUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n227, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n230, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n227 + i += n230 } return i, nil } @@ -15069,11 +15303,11 @@ func (m *ResponseUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n228, err := m.Subsume.MarshalTo(dAtA[i:]) + n231, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n228 + i += n231 } return i, nil } @@ -15085,11 +15319,11 @@ func (m *ResponseUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n229, err := m.RangeStats.MarshalTo(dAtA[i:]) + n232, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n229 + i += n232 } return i, nil } @@ -15101,11 +15335,11 @@ func (m *ResponseUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n230, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n233, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n230 + i += n233 } return i, nil } @@ -15117,11 +15351,27 @@ func (m *ResponseUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n231, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n234, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n231 + i += n234 + } + return i, nil +} +func (m *ResponseUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.AdminUnsplit != nil { + dAtA[i] = 0xfa + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size())) + n235, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n235 } return i, nil } @@ -15143,19 +15393,19 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n232, err := m.Timestamp.MarshalTo(dAtA[i:]) + n236, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n232 + i += n236 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Replica.Size())) - n233, err := m.Replica.MarshalTo(dAtA[i:]) + n237, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n233 + i += n237 if m.RangeID != 0 { dAtA[i] = 0x18 i++ @@ -15171,11 +15421,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n234, err := m.Txn.MarshalTo(dAtA[i:]) + n238, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n234 + i += n238 } if m.ReadConsistency != 0 { dAtA[i] = 0x30 @@ -15216,11 +15466,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.ScanOptions.Size())) - n235, err := m.ScanOptions.MarshalTo(dAtA[i:]) + n239, err := m.ScanOptions.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n235 + i += n239 } if m.AsyncConsensus { dAtA[i] = 0x68 @@ -15253,11 +15503,11 @@ func (m *BatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n236, err := m.Header.MarshalTo(dAtA[i:]) + n240, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n236 + i += n240 if len(m.Requests) > 0 { for _, msg := range m.Requests { dAtA[i] = 0x12 @@ -15291,11 +15541,11 @@ func (m *BatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BatchResponse_Header.Size())) - n237, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) + n241, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n237 + i += n241 if len(m.Responses) > 0 { for _, msg := range m.Responses { dAtA[i] = 0x12 @@ -15330,38 +15580,38 @@ func (m *BatchResponse_Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n238, err := m.Error.MarshalTo(dAtA[i:]) + n242, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n238 + i += n242 } dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n239, err := m.Timestamp.MarshalTo(dAtA[i:]) + n243, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n239 + i += n243 if m.Txn != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n240, err := m.Txn.MarshalTo(dAtA[i:]) + n244, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n240 + i += n244 } dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n241, err := m.Now.MarshalTo(dAtA[i:]) + n245, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n241 + i += n245 if len(m.CollectedSpans) > 0 { for _, msg := range m.CollectedSpans { dAtA[i] = 0x32 @@ -15395,19 +15645,19 @@ func (m *RangeFeedRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n242, err := m.Header.MarshalTo(dAtA[i:]) + n246, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n242 + i += n246 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n243, err := m.Span.MarshalTo(dAtA[i:]) + n247, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n243 + i += n247 return i, nil } @@ -15435,11 +15685,11 @@ func (m *RangeFeedValue) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n244, err := m.Value.MarshalTo(dAtA[i:]) + n248, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n244 + i += n248 return i, nil } @@ -15461,19 +15711,19 @@ func (m *RangeFeedCheckpoint) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n245, err := m.Span.MarshalTo(dAtA[i:]) + n249, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n245 + i += n249 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolvedTS.Size())) - n246, err := m.ResolvedTS.MarshalTo(dAtA[i:]) + n250, err := m.ResolvedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n246 + i += n250 return i, nil } @@ -15495,11 +15745,11 @@ func (m *RangeFeedError) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n247, err := m.Error.MarshalTo(dAtA[i:]) + n251, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n247 + i += n251 return i, nil } @@ -15522,31 +15772,31 @@ func (m *RangeFeedEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Val.Size())) - n248, err := m.Val.MarshalTo(dAtA[i:]) + n252, err := m.Val.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n248 + i += n252 } if m.Checkpoint != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Checkpoint.Size())) - n249, err := m.Checkpoint.MarshalTo(dAtA[i:]) + n253, err := m.Checkpoint.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n249 + i += n253 } if m.Error != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n250, err := m.Error.MarshalTo(dAtA[i:]) + n254, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n250 + i += n254 } return i, nil } @@ -16150,6 +16400,28 @@ func (m *AdminSplitResponse) Size() (n int) { return n } +func (m *AdminUnsplitRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.RequestHeader.Size() + n += 1 + l + sovApi(uint64(l)) + return n +} + +func (m *AdminUnsplitResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.ResponseHeader.Size() + n += 1 + l + sovApi(uint64(l)) + return n +} + func (m *AdminMergeRequest) Size() (n int) { if m == nil { return 0 @@ -17775,6 +18047,18 @@ func (m *RequestUnion_RecoverTxn) Size() (n int) { } return n } +func (m *RequestUnion_AdminUnsplit) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AdminUnsplit != nil { + l = m.AdminUnsplit.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *ResponseUnion) Size() (n int) { if m == nil { return 0 @@ -18279,6 +18563,18 @@ func (m *ResponseUnion_RecoverTxn) Size() (n int) { } return n } +func (m *ResponseUnion_AdminUnsplit) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AdminUnsplit != nil { + l = m.AdminUnsplit.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *Header) Size() (n int) { if m == nil { return 0 @@ -22645,6 +22941,166 @@ func (m *AdminSplitResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *AdminUnsplitRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AdminUnsplitRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AdminUnsplitRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", 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 err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AdminUnsplitResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AdminUnsplitResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AdminUnsplitResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResponseHeader", 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 err := m.ResponseHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *AdminMergeRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -32336,6 +32792,38 @@ func (m *RequestUnion) Unmarshal(dAtA []byte) error { } m.Value = &RequestUnion_RecoverTxn{v} iNdEx = postIndex + case 47: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AdminUnsplit", 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 + } + v := &AdminUnsplitRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &RequestUnion_AdminUnsplit{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -33698,6 +34186,38 @@ func (m *ResponseUnion) Unmarshal(dAtA []byte) error { } m.Value = &ResponseUnion_RecoverTxn{v} iNdEx = postIndex + case 47: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AdminUnsplit", 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 + } + v := &AdminUnsplitResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ResponseUnion_AdminUnsplit{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -35136,425 +35656,428 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_75ce0d0ce4ff4c16) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_c55003ffcd524477) } -var fileDescriptor_api_75ce0d0ce4ff4c16 = []byte{ - // 6666 bytes of a gzipped FileDescriptorProto +var fileDescriptor_api_c55003ffcd524477 = []byte{ + // 6714 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5b, 0x6c, 0x23, 0xc9, 0x75, 0xb6, 0x9a, 0xa4, 0x28, 0xf2, 0xf0, 0xa2, 0x56, 0x69, 0x2e, 0x1c, 0xed, 0xee, 0x48, 0xc3, 0xb9, 0xee, 0xec, 0xae, 0xc6, 0xa3, 0xf1, 0xda, 0xfe, 0x77, 0xed, 0xb5, 0x45, 0x8a, 0x33, 0xe4, 0x68, 0x74, 0xd9, 0x26, 0x35, 0xeb, 0x5d, 0xff, 0xfb, 0xb7, 0x5b, 0xdd, 0x25, 0xaa, 0x3d, 0x64, - 0x37, 0xa7, 0xbb, 0x39, 0x92, 0x06, 0xf8, 0x11, 0x20, 0x2f, 0x0e, 0x0c, 0xc3, 0xc8, 0x43, 0x10, - 0x04, 0x76, 0x82, 0x18, 0xb0, 0x81, 0x00, 0x09, 0x62, 0xc4, 0x81, 0x81, 0x20, 0x41, 0x02, 0x3f, - 0x24, 0x0f, 0x0b, 0xc3, 0x0f, 0x46, 0x10, 0x07, 0x46, 0x02, 0x08, 0x89, 0x92, 0xc0, 0x46, 0x5e, - 0xf3, 0x10, 0x60, 0x1f, 0x92, 0xa0, 0x2e, 0x7d, 0x21, 0xd9, 0xbc, 0x48, 0xee, 0x45, 0x36, 0xc8, - 0x93, 0x58, 0xa7, 0xea, 0x9c, 0xae, 0xcb, 0xa9, 0x53, 0xe7, 0xab, 0x3a, 0x55, 0x82, 0x39, 0xcb, - 0x54, 0xd4, 0xfd, 0xce, 0xee, 0x1d, 0xa5, 0xa3, 0x2f, 0x77, 0x2c, 0xd3, 0x31, 0xd1, 0x9c, 0x6a, - 0xaa, 0x4f, 0x28, 0x79, 0x99, 0x67, 0x2e, 0x20, 0xb7, 0x94, 0xa6, 0x38, 0x0a, 0x2b, 0xb6, 0x70, - 0xce, 0xa5, 0x61, 0xcb, 0x32, 0x2d, 0x9b, 0x53, 0x2f, 0xb8, 0xd4, 0x36, 0x76, 0x94, 0x40, 0xe9, - 0xa2, 0xed, 0x98, 0x96, 0xd2, 0xc4, 0x77, 0xb0, 0xd1, 0xd4, 0x0d, 0xf7, 0x0f, 0x29, 0xf7, 0x4c, - 0x55, 0x79, 0x99, 0xab, 0xa3, 0xca, 0xdc, 0xe3, 0x85, 0x0a, 0x5d, 0x47, 0x6f, 0xdd, 0xd9, 0x6f, - 0xa9, 0x77, 0x1c, 0xbd, 0x8d, 0x6d, 0x47, 0x69, 0x77, 0x78, 0xce, 0x12, 0xcd, 0x71, 0x2c, 0x45, - 0xd5, 0x8d, 0xe6, 0x1d, 0x0b, 0xab, 0xa6, 0xa5, 0x61, 0x4d, 0xb6, 0x3b, 0x8a, 0xe1, 0x56, 0xb9, - 0x69, 0x36, 0x4d, 0xfa, 0xf3, 0x0e, 0xf9, 0xc5, 0xa8, 0xc5, 0x5f, 0x81, 0xb4, 0xa4, 0x18, 0x4d, - 0x5c, 0x33, 0xf6, 0x4c, 0xf4, 0x59, 0x48, 0x68, 0xd8, 0x56, 0x0b, 0xc2, 0x92, 0x70, 0x2b, 0xb3, - 0x52, 0x5c, 0x1e, 0xe8, 0x8b, 0x65, 0x5a, 0x76, 0x0d, 0xdb, 0xaa, 0xa5, 0x77, 0x1c, 0xd3, 0x2a, - 0x25, 0x3e, 0x38, 0x5e, 0x9c, 0x92, 0x28, 0x17, 0xfa, 0x24, 0x4c, 0xb7, 0xb0, 0x62, 0xe3, 0x42, - 0x8c, 0xb2, 0x17, 0x42, 0xd8, 0x1f, 0x91, 0x7c, 0xce, 0xc4, 0x0a, 0x17, 0xff, 0x52, 0x80, 0x9c, - 0x84, 0x9f, 0x76, 0xb1, 0xed, 0x54, 0xb1, 0xa2, 0x61, 0x0b, 0x5d, 0x82, 0xf8, 0x13, 0x7c, 0x54, - 0x88, 0x2f, 0x09, 0xb7, 0xb2, 0xa5, 0x99, 0x0f, 0x8f, 0x17, 0xe3, 0xeb, 0xf8, 0x48, 0x22, 0x34, - 0xb4, 0x04, 0x33, 0xd8, 0xd0, 0x64, 0x92, 0x9d, 0xe8, 0xcd, 0x4e, 0x62, 0x43, 0x5b, 0xc7, 0x47, - 0x48, 0x85, 0x94, 0x4d, 0xa4, 0x19, 0x2a, 0x2e, 0x4c, 0x2f, 0x09, 0xb7, 0xa6, 0x4b, 0x0f, 0x3e, - 0x3c, 0x5e, 0x2c, 0x37, 0x75, 0x67, 0xbf, 0xbb, 0xbb, 0xac, 0x9a, 0xed, 0x3b, 0x5e, 0xad, 0xb4, - 0x5d, 0xff, 0xf7, 0x9d, 0xce, 0x93, 0xe6, 0x9d, 0x21, 0x23, 0xb0, 0xdc, 0x38, 0x34, 0xea, 0xf8, - 0xa9, 0xe4, 0x09, 0x7e, 0x23, 0xf1, 0x8b, 0x6f, 0x2f, 0x0a, 0x0f, 0x13, 0x29, 0x41, 0x8c, 0x3d, - 0x4c, 0xa4, 0x62, 0x62, 0xbc, 0xf8, 0xf5, 0x38, 0xe4, 0x25, 0x6c, 0x77, 0x4c, 0xc3, 0xc6, 0xbc, - 0x19, 0x9f, 0x80, 0xb8, 0x73, 0x68, 0xd0, 0x66, 0x64, 0x56, 0x2e, 0x87, 0x74, 0x46, 0xc3, 0x52, - 0x0c, 0x5b, 0x51, 0x1d, 0xdd, 0x34, 0x24, 0x52, 0x14, 0x7d, 0x06, 0x32, 0x16, 0xb6, 0xbb, 0x6d, - 0x4c, 0x87, 0x8d, 0xb6, 0x30, 0xb3, 0x72, 0x31, 0x84, 0xb3, 0xde, 0x51, 0x0c, 0x09, 0x58, 0x59, - 0xf2, 0x1b, 0x5d, 0x82, 0x94, 0xd1, 0x6d, 0x93, 0x7e, 0xb1, 0x69, 0xab, 0xe3, 0xd2, 0x8c, 0xd1, - 0x6d, 0xaf, 0xe3, 0x23, 0x1b, 0x95, 0x21, 0x63, 0x91, 0x41, 0x93, 0x75, 0x63, 0xcf, 0xb4, 0x0b, - 0xc9, 0xa5, 0xf8, 0xad, 0xcc, 0xca, 0x8b, 0xc3, 0x86, 0x96, 0xa8, 0x01, 0x1f, 0x1f, 0xb0, 0x5c, - 0x82, 0x8d, 0xea, 0x90, 0xe3, 0x35, 0xb3, 0xb0, 0x62, 0x9b, 0x46, 0x61, 0x66, 0x49, 0xb8, 0x95, - 0x5f, 0x59, 0x0e, 0x13, 0xd3, 0xd3, 0x0b, 0x24, 0xd9, 0x6d, 0x63, 0x89, 0x72, 0x49, 0x59, 0x2b, - 0x90, 0x2a, 0xbe, 0x0b, 0xd9, 0x60, 0x2e, 0x42, 0x90, 0x97, 0x2a, 0xf5, 0x9d, 0x8d, 0x8a, 0xbc, - 0xb3, 0xb9, 0xbe, 0xb9, 0xf5, 0xce, 0xa6, 0x38, 0x85, 0xce, 0x81, 0xc8, 0x69, 0xeb, 0x95, 0x77, - 0xe5, 0x47, 0xb5, 0x8d, 0x5a, 0x43, 0x14, 0xd0, 0x25, 0x38, 0xcf, 0xa9, 0xd2, 0xea, 0xe6, 0x83, - 0x8a, 0x5c, 0xda, 0xda, 0xd9, 0x5c, 0x5b, 0x95, 0xde, 0x15, 0x63, 0x0b, 0x89, 0x5f, 0xfb, 0xce, - 0xe5, 0xa9, 0xe2, 0x63, 0x80, 0x07, 0xd8, 0xe1, 0x6a, 0x85, 0x4a, 0x90, 0xdc, 0xa7, 0xb5, 0xe1, - 0x8a, 0xbd, 0x14, 0x5a, 0xed, 0x80, 0x0a, 0x96, 0x52, 0xa4, 0x07, 0x7e, 0x72, 0xbc, 0x28, 0x48, - 0x9c, 0x93, 0x0d, 0x79, 0xf1, 0x87, 0x02, 0x64, 0xa8, 0x60, 0xd6, 0x46, 0x54, 0xee, 0x93, 0x7c, - 0x65, 0x6c, 0x87, 0x0c, 0x8a, 0x46, 0xcb, 0x30, 0xfd, 0x4c, 0x69, 0x75, 0x47, 0xcd, 0x9b, 0xc7, - 0x24, 0x5f, 0x62, 0xc5, 0xd0, 0x9b, 0x90, 0xd5, 0x0d, 0x07, 0x1b, 0x8e, 0xcc, 0xd8, 0xe2, 0x63, - 0xd8, 0x32, 0xac, 0x34, 0x4d, 0x14, 0xff, 0x4c, 0x00, 0xd8, 0xee, 0x46, 0xd9, 0x35, 0x64, 0xde, - 0x4f, 0x54, 0x7f, 0x77, 0xde, 0xb3, 0x56, 0x5c, 0x80, 0xa4, 0x6e, 0xb4, 0x74, 0x83, 0xd5, 0x3f, - 0x25, 0xf1, 0x14, 0x3a, 0x07, 0xd3, 0xbb, 0x2d, 0xdd, 0xd0, 0xa8, 0xfa, 0xa7, 0x24, 0x96, 0xe0, - 0xdd, 0x2f, 0x41, 0x86, 0xd6, 0x3d, 0xc2, 0xde, 0x2f, 0x7e, 0x33, 0x06, 0xe7, 0xcb, 0xa6, 0xa1, - 0xe9, 0x64, 0x1e, 0x2a, 0xad, 0x8f, 0x45, 0xdf, 0xbc, 0x0e, 0x69, 0x7c, 0xd8, 0x99, 0x70, 0x78, - 0x53, 0xf8, 0xb0, 0x43, 0x7f, 0x85, 0x77, 0x1d, 0xfa, 0x24, 0x5c, 0x54, 0x5a, 0x2d, 0xf3, 0x40, - 0xd6, 0xf7, 0x64, 0xcd, 0xc4, 0xb6, 0x6c, 0x98, 0x8e, 0x8c, 0x0f, 0x75, 0xdb, 0xa1, 0xa6, 0x22, - 0x25, 0xcd, 0xd3, 0xec, 0xda, 0xde, 0x9a, 0x89, 0xed, 0x4d, 0xd3, 0xa9, 0x90, 0x2c, 0xde, 0xe1, - 0xef, 0xc3, 0x85, 0xfe, 0xbe, 0x89, 0xb2, 0xef, 0xff, 0x5a, 0x80, 0x7c, 0xcd, 0xd0, 0x9d, 0x8f, - 0x45, 0xa7, 0x7b, 0xbd, 0x17, 0x0f, 0xf6, 0xde, 0x6d, 0x10, 0xf7, 0x14, 0xbd, 0xb5, 0x65, 0x34, - 0xcc, 0xf6, 0xae, 0xed, 0x98, 0x06, 0xb6, 0x79, 0xf7, 0x0e, 0xd0, 0x79, 0x9f, 0x3d, 0x86, 0x59, - 0xaf, 0x4d, 0x51, 0x76, 0xd6, 0x73, 0x10, 0x6b, 0x86, 0x6a, 0xe1, 0x36, 0x36, 0x22, 0xed, 0xad, - 0x17, 0x21, 0xad, 0xbb, 0x72, 0x69, 0x8f, 0xc5, 0x25, 0x9f, 0xc0, 0xdb, 0xd4, 0x85, 0xb9, 0xc0, - 0xb7, 0xa3, 0x34, 0x7e, 0x2f, 0x40, 0xda, 0xc0, 0x07, 0xb2, 0x3f, 0x5e, 0x71, 0x29, 0x65, 0xe0, - 0x03, 0x66, 0xac, 0xde, 0x85, 0xdc, 0x1a, 0x6e, 0x61, 0x07, 0x47, 0x6f, 0xc9, 0x77, 0x20, 0xef, - 0x8a, 0x8e, 0x72, 0x90, 0x7e, 0x47, 0x00, 0xc4, 0xe5, 0x92, 0xd5, 0x33, 0xca, 0x71, 0x5a, 0x24, - 0xde, 0x81, 0xd3, 0xb5, 0x0c, 0xb6, 0xcc, 0x33, 0x2d, 0x05, 0x46, 0xa2, 0x2b, 0xbd, 0x6f, 0x51, - 0x13, 0x41, 0x8b, 0xea, 0x79, 0x2b, 0xc4, 0x4f, 0x39, 0x80, 0xf9, 0x9e, 0xea, 0x45, 0x3b, 0x94, - 0x09, 0x5a, 0xb3, 0xd8, 0x52, 0x3c, 0xe8, 0x99, 0x51, 0x62, 0xf1, 0x7d, 0x98, 0x2b, 0xb7, 0xb0, - 0x62, 0x45, 0xdd, 0x2d, 0x7c, 0x38, 0xdf, 0x05, 0x14, 0x14, 0x1f, 0xe5, 0x90, 0xea, 0x90, 0xa9, - 0xab, 0x8a, 0xb1, 0xd5, 0x21, 0x46, 0xd0, 0x46, 0xf7, 0xe0, 0x82, 0xed, 0x98, 0x1d, 0x59, 0x71, - 0x64, 0xe6, 0x57, 0xed, 0x9a, 0x5d, 0x43, 0x53, 0xac, 0x23, 0xfa, 0x8d, 0x94, 0x34, 0x4f, 0x72, - 0x57, 0x1d, 0x5a, 0x91, 0x12, 0xcf, 0x22, 0x63, 0xd7, 0xd6, 0x0d, 0x99, 0xb8, 0x3f, 0x2d, 0xc7, - 0xe6, 0x7a, 0x0e, 0x6d, 0xdd, 0x90, 0x18, 0x85, 0xb7, 0xe2, 0x3b, 0x02, 0xfb, 0x56, 0x94, 0x6a, - 0xf3, 0x16, 0x64, 0x6c, 0x55, 0x31, 0xe4, 0x3d, 0xd3, 0x6a, 0x2b, 0x0e, 0x55, 0x8d, 0xfc, 0xca, - 0x4b, 0x61, 0x4e, 0xa5, 0xaa, 0x18, 0xf7, 0x69, 0x21, 0x09, 0x6c, 0xef, 0x77, 0x50, 0x7b, 0x1e, - 0x26, 0x52, 0x71, 0x31, 0x51, 0xfc, 0x77, 0x01, 0xb2, 0xac, 0x96, 0x51, 0x6a, 0xcf, 0xeb, 0x90, - 0xb0, 0xcc, 0x03, 0xa6, 0x3d, 0x99, 0x95, 0x17, 0x42, 0x44, 0xac, 0xe3, 0xa3, 0xa0, 0xd9, 0xa6, - 0xc5, 0x51, 0x09, 0xb8, 0x7b, 0x23, 0x53, 0xee, 0xf8, 0xa4, 0xdc, 0xc0, 0xb8, 0x24, 0x22, 0xe3, - 0x26, 0xcc, 0xee, 0x2a, 0x8e, 0xba, 0x4f, 0xc6, 0x87, 0x56, 0x92, 0x98, 0xf8, 0xf8, 0xad, 0xac, - 0x94, 0xa7, 0x64, 0xb7, 0xea, 0x76, 0xf1, 0xf7, 0x05, 0x40, 0x12, 0x7e, 0x86, 0x2d, 0x1b, 0x7f, - 0xfc, 0x87, 0xe9, 0x3f, 0x04, 0x98, 0xef, 0xa9, 0xec, 0xff, 0xb6, 0xd1, 0xfa, 0xa9, 0x00, 0x17, - 0xcb, 0xfb, 0x58, 0x7d, 0x52, 0x36, 0x0d, 0x5b, 0xb7, 0x1d, 0x6c, 0xa8, 0x47, 0x51, 0x0e, 0xd9, - 0x0b, 0x90, 0x3e, 0xd0, 0x9d, 0x7d, 0x59, 0xd3, 0xf7, 0xf6, 0xe8, 0x94, 0x4e, 0x49, 0x29, 0x42, - 0x58, 0xd3, 0xf7, 0xf6, 0xd0, 0x3d, 0x48, 0xb4, 0x4d, 0x8d, 0x79, 0x6f, 0xf9, 0x95, 0xc5, 0x10, - 0xf1, 0xb4, 0x6a, 0x76, 0xb7, 0xbd, 0x61, 0x6a, 0x58, 0xa2, 0x85, 0xd1, 0x65, 0x00, 0x95, 0x50, - 0x3b, 0xa6, 0x6e, 0x38, 0xdc, 0x8a, 0x07, 0x28, 0xdc, 0x4a, 0x7c, 0x23, 0x01, 0x85, 0xc1, 0x76, - 0x45, 0x39, 0xba, 0xdb, 0x90, 0x64, 0xa6, 0x8a, 0x8f, 0xef, 0xca, 0xb0, 0xea, 0x87, 0xd4, 0x60, - 0x99, 0x99, 0x34, 0x3e, 0x70, 0x5c, 0xce, 0xc2, 0x9f, 0x0b, 0x90, 0x64, 0x19, 0xe8, 0x2e, 0xa4, - 0x38, 0x20, 0xd5, 0x68, 0x1d, 0xe3, 0xa5, 0x0b, 0x27, 0xc7, 0x8b, 0x33, 0x0c, 0x7e, 0xae, 0x7d, - 0xe8, 0xff, 0x94, 0x66, 0x18, 0x02, 0xd5, 0x48, 0x4f, 0xdb, 0x8e, 0x62, 0x39, 0x14, 0xf8, 0x93, - 0x9e, 0xce, 0x4a, 0x29, 0x4a, 0x20, 0x88, 0xff, 0x21, 0x24, 0x6d, 0x47, 0x71, 0xba, 0x36, 0xef, - 0xeb, 0x53, 0x55, 0xb6, 0x4e, 0x39, 0x25, 0x2e, 0x81, 0x2c, 0xa1, 0x1a, 0x76, 0x14, 0xbd, 0x45, - 0x3b, 0x3f, 0x2d, 0xf1, 0x54, 0xf1, 0x5b, 0x02, 0x24, 0x59, 0x51, 0x74, 0x11, 0xe6, 0x19, 0xe8, - 0xac, 0x6d, 0xae, 0x55, 0x1a, 0x15, 0x69, 0xa3, 0xb6, 0xb9, 0xda, 0xa8, 0x88, 0x53, 0xe8, 0x02, - 0x20, 0x37, 0xa3, 0xbc, 0xb5, 0x59, 0xaf, 0xd5, 0x1b, 0x95, 0x4d, 0x02, 0x56, 0x09, 0x84, 0xa5, - 0xf4, 0x00, 0x35, 0x86, 0xae, 0xc1, 0x52, 0x3f, 0x55, 0xae, 0x37, 0x56, 0x1b, 0x75, 0xb9, 0x52, - 0x6f, 0xd4, 0x36, 0x56, 0x1b, 0x95, 0x35, 0x31, 0x3e, 0xa2, 0x14, 0xf9, 0x88, 0x24, 0x55, 0xca, - 0x0d, 0x31, 0x51, 0x7c, 0x0e, 0xe7, 0x25, 0xac, 0x9a, 0xed, 0x4e, 0xd7, 0xc1, 0xa4, 0x96, 0x76, - 0x94, 0x5a, 0x7e, 0x11, 0x66, 0x34, 0xeb, 0x48, 0xb6, 0xba, 0x06, 0xd7, 0xf1, 0xa4, 0x66, 0x1d, - 0x49, 0x5d, 0x83, 0x2b, 0xe3, 0x9f, 0x08, 0x70, 0xa1, 0xff, 0xe3, 0x51, 0xaa, 0xe2, 0x17, 0x21, - 0xa3, 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0xb7, 0x1c, 0x85, 0x7b, 0xf4, 0x77, 0x03, 0x92, 0xf8, 0xa6, - 0xcd, 0x32, 0xdb, 0xad, 0x59, 0xf6, 0x36, 0x6d, 0x36, 0x1e, 0x97, 0xcb, 0xb4, 0x3e, 0x6b, 0x84, - 0xd1, 0xb5, 0x23, 0x54, 0x16, 0xa5, 0x14, 0x55, 0xb8, 0x58, 0xc2, 0x4d, 0xdd, 0x08, 0x6e, 0xc3, - 0x44, 0xee, 0x97, 0xc8, 0x50, 0x18, 0xfc, 0x48, 0x94, 0xde, 0xc9, 0x77, 0x13, 0x70, 0xbe, 0x62, - 0x68, 0x1f, 0x4d, 0x23, 0xc8, 0x7c, 0x50, 0xcd, 0x76, 0x5b, 0x77, 0xdc, 0xb1, 0x67, 0x29, 0xf4, - 0x7f, 0x20, 0xa5, 0x61, 0x45, 0xf3, 0xe0, 0x7b, 0xa6, 0x67, 0xa9, 0xea, 0x3a, 0x7a, 0x6b, 0x79, - 0xbf, 0xa5, 0x2e, 0x37, 0xdc, 0x4d, 0x4a, 0xc9, 0x2b, 0x8e, 0xbe, 0x0c, 0x17, 0x89, 0x31, 0xb7, - 0x0c, 0xa5, 0x25, 0x33, 0x69, 0xb2, 0x63, 0xe9, 0xcd, 0x26, 0xb6, 0xf8, 0x86, 0xd7, 0xad, 0x90, - 0x7a, 0xd6, 0x38, 0x47, 0x99, 0x32, 0x34, 0x58, 0x79, 0xe9, 0xbc, 0x1e, 0x46, 0x46, 0x5f, 0xf0, - 0xf6, 0x47, 0xec, 0x8e, 0x62, 0xd8, 0x85, 0x69, 0x6a, 0xc3, 0x86, 0xed, 0xa3, 0x71, 0xcd, 0xe0, - 0xeb, 0x12, 0xa1, 0xd8, 0xe8, 0x0e, 0x71, 0xb5, 0x9f, 0x76, 0x75, 0x0b, 0xcb, 0x77, 0x3b, 0x6a, - 0x21, 0x49, 0xda, 0x5e, 0xca, 0x9f, 0x1c, 0x2f, 0x82, 0xc4, 0xc8, 0x77, 0xb7, 0xcb, 0xc4, 0xf5, - 0x66, 0xbf, 0x3b, 0x2a, 0xba, 0x05, 0xa2, 0x61, 0xca, 0x16, 0xde, 0xb3, 0xb0, 0xbd, 0xcf, 0x3f, - 0x9b, 0xa2, 0x3d, 0x96, 0x37, 0x4c, 0x89, 0x91, 0x99, 0xe8, 0x0b, 0x90, 0xec, 0x98, 0xba, 0x6d, - 0x1a, 0x85, 0x34, 0xeb, 0x51, 0x96, 0x42, 0x6f, 0x83, 0xa8, 0x1b, 0xf2, 0x5e, 0x4b, 0x6f, 0xee, - 0x3b, 0xf2, 0x81, 0xa5, 0x3b, 0xd8, 0x2e, 0xcc, 0xd1, 0x8a, 0x87, 0xa9, 0x45, 0x9d, 0xef, 0x44, - 0x6a, 0xef, 0x90, 0x92, 0xbc, 0x09, 0x79, 0xdd, 0xb8, 0x4f, 0xf9, 0x29, 0xd1, 0xf6, 0x5c, 0x82, - 0x19, 0x31, 0x55, 0xfc, 0x17, 0x01, 0x2e, 0xf4, 0xab, 0x49, 0x94, 0xd3, 0xf4, 0x16, 0x88, 0xa6, - 0x81, 0xe5, 0xce, 0xbe, 0x62, 0x63, 0x3e, 0xac, 0x7c, 0xfd, 0xca, 0x9b, 0x06, 0xde, 0x26, 0x64, - 0x36, 0x48, 0x68, 0x1b, 0xe6, 0x6c, 0x47, 0x69, 0xea, 0x46, 0x53, 0xf6, 0xf6, 0xb0, 0xe9, 0x46, - 0xc4, 0x38, 0x1d, 0xe2, 0xad, 0x14, 0x39, 0xb7, 0x47, 0xef, 0x71, 0x7a, 0xbe, 0x25, 0xc0, 0xdc, - 0xaa, 0xd6, 0xd6, 0x8d, 0x7a, 0xa7, 0xa5, 0x47, 0x0a, 0x93, 0xaf, 0x41, 0xda, 0x26, 0x32, 0xfd, - 0x35, 0xc8, 0x87, 0x38, 0x29, 0x9a, 0x43, 0x16, 0xa3, 0x0b, 0x90, 0x6c, 0x2b, 0x46, 0x57, 0x69, - 0xb9, 0xbb, 0x5a, 0x2c, 0xe5, 0xa3, 0x94, 0x60, 0xe5, 0xa2, 0xb4, 0x03, 0xef, 0xf3, 0x76, 0x6f, - 0x60, 0xeb, 0x23, 0xc2, 0x57, 0x41, 0xf1, 0x51, 0xd6, 0xfc, 0x6b, 0x02, 0x5c, 0xa2, 0xb2, 0xa9, - 0x72, 0xee, 0x61, 0x8b, 0x1e, 0x12, 0x44, 0x39, 0x74, 0x57, 0x21, 0xe9, 0x28, 0x56, 0x13, 0x33, - 0x2b, 0x36, 0x5d, 0xca, 0x10, 0x27, 0xa3, 0xee, 0x98, 0x16, 0x71, 0x32, 0x78, 0x16, 0x6f, 0xa7, - 0x02, 0x0b, 0x61, 0x75, 0x89, 0xb2, 0xbd, 0x7f, 0x1c, 0xe3, 0xdf, 0x28, 0xef, 0x33, 0xb0, 0xda, - 0x69, 0xe9, 0xaa, 0x12, 0xe9, 0x9a, 0x5d, 0x81, 0x8c, 0x4a, 0x85, 0xcb, 0xce, 0x51, 0x87, 0x6d, - 0xab, 0xe4, 0x57, 0xae, 0x85, 0x0a, 0xa2, 0x1f, 0x67, 0x35, 0x69, 0x1c, 0x75, 0x30, 0x71, 0x37, - 0xdd, 0xdf, 0x68, 0x0d, 0x66, 0x58, 0xe7, 0xb8, 0x9e, 0xfa, 0x08, 0x11, 0xc4, 0xa4, 0x34, 0x68, - 0x61, 0x3e, 0x4f, 0x5d, 0x56, 0xf4, 0x39, 0x48, 0xe1, 0xc3, 0x8e, 0x4c, 0x0f, 0x96, 0x12, 0x93, - 0x1e, 0x2c, 0x49, 0x33, 0xf8, 0xb0, 0x43, 0x92, 0x7c, 0x5c, 0xbe, 0x29, 0xc0, 0x0b, 0xa1, 0x9d, - 0x16, 0xa5, 0x11, 0xfb, 0x14, 0x3f, 0xfe, 0x8a, 0x4d, 0x5c, 0x4b, 0x5a, 0xbe, 0xf8, 0x07, 0xae, - 0x06, 0x4b, 0xb8, 0x65, 0xaa, 0xca, 0x47, 0xb0, 0xf7, 0x13, 0x18, 0x89, 0xd8, 0x99, 0x47, 0xa2, - 0x4f, 0xc5, 0xfb, 0x2a, 0x1b, 0xa5, 0x8a, 0xff, 0xa6, 0x00, 0xf3, 0x55, 0xac, 0x58, 0xce, 0x2e, - 0x56, 0x9c, 0xc6, 0x61, 0xa4, 0x2e, 0xc9, 0xeb, 0x10, 0x37, 0xcc, 0x03, 0x3e, 0x46, 0x13, 0xad, - 0x18, 0xa4, 0x3c, 0x6f, 0xfb, 0x97, 0xe0, 0x5c, 0x6f, 0xbd, 0xa2, 0x6c, 0xf5, 0xf7, 0xe3, 0x90, - 0x7e, 0x50, 0x8e, 0xb2, 0xad, 0x9f, 0xe5, 0x3b, 0x6a, 0x6c, 0xf6, 0x85, 0x29, 0xa4, 0xf7, 0xbd, - 0xe5, 0x07, 0xe5, 0x75, 0x7c, 0xe4, 0x82, 0x6d, 0xc2, 0x85, 0x56, 0x21, 0xed, 0xec, 0x13, 0xcf, - 0xc3, 0x6c, 0x69, 0x7c, 0xe6, 0x4d, 0xd4, 0x5f, 0x3e, 0x17, 0x6a, 0xc1, 0x79, 0xe7, 0xd0, 0xa0, - 0x0e, 0x8d, 0xdc, 0x54, 0x65, 0x5f, 0xdc, 0x44, 0x0b, 0xf6, 0x02, 0x11, 0x77, 0x72, 0xbc, 0x88, - 0x1a, 0x87, 0x06, 0xf1, 0x7e, 0x1e, 0x94, 0x1b, 0xae, 0x00, 0x09, 0x39, 0x9c, 0xa6, 0x7a, 0xb4, - 0x85, 0x27, 0x30, 0x4d, 0x5b, 0xe1, 0x9e, 0x00, 0x0b, 0x21, 0x27, 0xc0, 0xa4, 0x51, 0x9e, 0xdb, - 0x10, 0x3b, 0x4d, 0xa3, 0x5c, 0x02, 0x53, 0x05, 0xae, 0x10, 0x6f, 0x03, 0x90, 0x2e, 0x8c, 0x52, - 0x0d, 0xfe, 0x33, 0x0e, 0xf9, 0xed, 0xae, 0xbd, 0x1f, 0xb1, 0xde, 0x97, 0x01, 0x3a, 0x5d, 0x7b, - 0x1f, 0x5b, 0xb2, 0x73, 0x68, 0xf0, 0x96, 0x8f, 0x39, 0x55, 0x76, 0x9b, 0xce, 0xf8, 0x1a, 0x87, - 0x06, 0xda, 0xe2, 0x42, 0xb0, 0xec, 0x1f, 0x4d, 0xdf, 0x9e, 0x00, 0x4c, 0x35, 0x0e, 0x8d, 0x0d, - 0xec, 0xa1, 0x28, 0x26, 0x10, 0x13, 0x81, 0x9f, 0x85, 0x19, 0x92, 0x90, 0x1d, 0xf3, 0x34, 0x1a, - 0x96, 0x24, 0x3c, 0x0d, 0x13, 0x3d, 0x84, 0xbc, 0x86, 0x3b, 0x16, 0x26, 0x66, 0x48, 0x93, 0xc9, - 0xb4, 0x3e, 0x85, 0x23, 0x98, 0xf3, 0x59, 0x37, 0xcd, 0x03, 0xf4, 0x26, 0xa4, 0x59, 0x4d, 0xc8, - 0x8a, 0x97, 0xa4, 0x2b, 0x5e, 0x58, 0xf7, 0xf0, 0x91, 0xa1, 0x6b, 0x5d, 0x8a, 0x56, 0x83, 0xac, - 0x74, 0xe7, 0x60, 0x7a, 0xcf, 0xb4, 0x54, 0x4c, 0xcf, 0xb5, 0x53, 0x12, 0x4b, 0xa0, 0xdb, 0x30, - 0xa7, 0x1b, 0x6a, 0xab, 0x6b, 0xeb, 0xcf, 0xb0, 0xec, 0x36, 0x93, 0xb9, 0xed, 0xb3, 0x5e, 0x06, - 0x15, 0x68, 0x7a, 0xce, 0x76, 0x4a, 0x4c, 0x13, 0x27, 0x74, 0xd6, 0xd3, 0x80, 0x28, 0x17, 0xa8, - 0x72, 0xcf, 0xf0, 0x9d, 0x5e, 0x07, 0xc8, 0x90, 0x15, 0xff, 0x46, 0x80, 0x39, 0x82, 0xd8, 0x9f, - 0x51, 0x95, 0x88, 0x52, 0x45, 0x4b, 0x2c, 0xe2, 0x21, 0x76, 0x46, 0xb5, 0xa2, 0x31, 0x10, 0x77, - 0xe1, 0x9c, 0xde, 0x26, 0xeb, 0x98, 0xee, 0xb4, 0x8e, 0x38, 0x92, 0x70, 0xb0, 0x7b, 0x28, 0x37, - 0xef, 0xe7, 0x95, 0xdd, 0x2c, 0x3e, 0x93, 0xbf, 0x4b, 0xf7, 0x66, 0xfd, 0x66, 0x45, 0xd9, 0xef, - 0x35, 0xc8, 0x59, 0x4c, 0x34, 0xd6, 0x4e, 0xdd, 0xf5, 0x59, 0x8f, 0x95, 0xf4, 0xfe, 0xf7, 0x62, - 0x30, 0xfb, 0x76, 0x17, 0x5b, 0x47, 0x1f, 0xc3, 0xbe, 0xbf, 0x01, 0xb3, 0x07, 0x8a, 0xee, 0xc8, - 0x7b, 0xa6, 0x25, 0x77, 0x3b, 0x9a, 0xe2, 0xb8, 0x67, 0xf3, 0x39, 0x42, 0xbe, 0x6f, 0x5a, 0x3b, - 0x94, 0x88, 0x30, 0xa0, 0x27, 0x86, 0x79, 0x60, 0xc8, 0x84, 0x4c, 0x81, 0xdc, 0xa1, 0xc1, 0x37, - 0x61, 0x4b, 0x9f, 0xfe, 0xbb, 0xe3, 0xc5, 0x7b, 0x13, 0x45, 0xdb, 0xd0, 0x80, 0xa5, 0x6e, 0x57, - 0xd7, 0x96, 0x77, 0x76, 0x6a, 0x6b, 0x92, 0x48, 0x45, 0xbe, 0xc3, 0x24, 0x36, 0x0e, 0x0d, 0xd7, - 0x5d, 0xf9, 0x50, 0x00, 0xd1, 0xef, 0xb0, 0x28, 0x47, 0xb5, 0x02, 0x99, 0xa7, 0x5d, 0x6c, 0xe9, - 0x67, 0x18, 0x53, 0xe0, 0x8c, 0xc4, 0x04, 0xbe, 0x07, 0xd9, 0x9e, 0x7e, 0x88, 0xff, 0x72, 0xfd, - 0x90, 0x39, 0xf0, 0xbb, 0xa0, 0xf8, 0x63, 0x01, 0x10, 0x6d, 0x7c, 0x8d, 0xed, 0x7f, 0x7f, 0xcc, - 0x14, 0xe6, 0x16, 0x88, 0x34, 0xfe, 0x4d, 0xd6, 0xf7, 0xe4, 0xb6, 0x6e, 0xdb, 0xba, 0xd1, 0xe4, - 0x1a, 0x93, 0xa7, 0xf4, 0xda, 0xde, 0x06, 0xa3, 0xf2, 0xb1, 0xfc, 0xff, 0x30, 0xdf, 0xd3, 0x9a, - 0x28, 0x47, 0xf3, 0x0a, 0x64, 0xf7, 0xcc, 0xae, 0xa1, 0xc9, 0x6c, 0x23, 0x87, 0x6f, 0x58, 0x65, - 0x28, 0x8d, 0x7d, 0xaf, 0xf8, 0xd5, 0x18, 0x9c, 0x93, 0xb0, 0x6d, 0xb6, 0x9e, 0xe1, 0xe8, 0xfb, - 0x73, 0x0b, 0xf8, 0x21, 0x85, 0xfc, 0xcb, 0x74, 0x6b, 0x9a, 0xc9, 0x60, 0x4b, 0x6b, 0xef, 0xbe, - 0xf6, 0xb5, 0xd1, 0x9a, 0x39, 0xb8, 0x93, 0xcd, 0xf7, 0x99, 0x12, 0xc1, 0x7d, 0x26, 0x3e, 0x10, - 0xff, 0x17, 0xce, 0xf7, 0x75, 0x44, 0x94, 0x1e, 0xd0, 0xdf, 0xc6, 0xe0, 0x52, 0xaf, 0xf8, 0xa8, - 0xf1, 0xd0, 0xff, 0x8c, 0xce, 0x46, 0x55, 0xc8, 0xb5, 0x75, 0xe3, 0x6c, 0xfb, 0x5c, 0xd9, 0xb6, - 0x6e, 0x34, 0x7a, 0x7d, 0x56, 0x02, 0xdd, 0xc2, 0xfa, 0x35, 0xca, 0xb1, 0xfb, 0x86, 0x00, 0xd9, - 0xa8, 0xf7, 0x90, 0xce, 0x16, 0x90, 0xc3, 0xdb, 0xdc, 0x80, 0xdc, 0x47, 0xb0, 0xe9, 0xf4, 0x7b, - 0x02, 0xa0, 0x86, 0xd5, 0x35, 0x88, 0xfb, 0xf8, 0xc8, 0x6c, 0x46, 0xd9, 0xd8, 0x73, 0x30, 0xad, - 0x1b, 0x1a, 0x3e, 0xa4, 0x8d, 0x4d, 0x48, 0x2c, 0xd1, 0x73, 0xea, 0x15, 0x9f, 0xe8, 0xd4, 0x8b, - 0xb7, 0xff, 0x3d, 0x98, 0xef, 0xa9, 0x68, 0x94, 0xbd, 0xf0, 0xbd, 0x18, 0xcc, 0xf3, 0xe6, 0x44, - 0xbe, 0xe9, 0x76, 0xa6, 0x68, 0x60, 0xf4, 0x39, 0x80, 0x8e, 0x85, 0x9f, 0xc9, 0x8c, 0x35, 0x3e, - 0x11, 0x6b, 0x9a, 0x70, 0x50, 0x02, 0xfa, 0x22, 0xcc, 0x92, 0x09, 0xd7, 0xb1, 0xcc, 0x8e, 0x69, - 0x93, 0x75, 0xdd, 0x9e, 0x0c, 0x96, 0xcc, 0x9d, 0x1c, 0x2f, 0xe6, 0x36, 0x74, 0x63, 0x9b, 0x33, - 0x36, 0xea, 0x12, 0x99, 0xb9, 0x5e, 0xd2, 0x75, 0x46, 0x7e, 0x2a, 0xc0, 0xb9, 0x8f, 0x6c, 0x9b, - 0xf2, 0xbf, 0xa3, 0xc7, 0xbc, 0xf5, 0x40, 0xa4, 0xc9, 0x9a, 0xb1, 0x67, 0x46, 0xbf, 0x79, 0xfc, - 0x0d, 0x01, 0xe6, 0x02, 0xe2, 0xa3, 0x5c, 0xf5, 0xcf, 0x16, 0x73, 0xfe, 0x25, 0xe2, 0x07, 0x04, - 0xd5, 0x3e, 0xca, 0x49, 0xf5, 0x6f, 0x02, 0x5c, 0x28, 0xb3, 0xf3, 0x50, 0xf7, 0x88, 0x3f, 0x4a, - 0x2d, 0x29, 0xc0, 0xcc, 0x33, 0x6c, 0xd9, 0xba, 0xc9, 0xd6, 0xbd, 0x9c, 0xe4, 0x26, 0xd1, 0x02, - 0xa4, 0x6c, 0x43, 0xe9, 0xd8, 0xfb, 0xa6, 0x7b, 0xf6, 0xe2, 0xa5, 0xbd, 0x70, 0x84, 0xe9, 0xb3, - 0x87, 0x23, 0x24, 0xc3, 0xc3, 0x11, 0xf8, 0xc1, 0xcb, 0x8f, 0x04, 0xb8, 0x38, 0xd0, 0xea, 0x28, - 0x47, 0xfa, 0x2b, 0x90, 0x51, 0xb9, 0x60, 0x62, 0x43, 0xd9, 0x09, 0x4c, 0x8d, 0x14, 0x3b, 0xa3, - 0xa7, 0x7d, 0x72, 0xbc, 0x08, 0x6e, 0x55, 0x6b, 0x6b, 0xbc, 0x61, 0xe4, 0xb7, 0x56, 0xfc, 0x39, - 0x40, 0xae, 0x72, 0xd8, 0x31, 0x2d, 0xa7, 0xce, 0x1c, 0x09, 0xb4, 0x06, 0xa9, 0x8e, 0x65, 0x3e, - 0xd3, 0xdd, 0x46, 0xe4, 0x43, 0x8f, 0x29, 0x7b, 0x78, 0xb6, 0x79, 0x79, 0xc9, 0xe3, 0x44, 0x12, - 0xa4, 0x1f, 0x99, 0xaa, 0xd2, 0xba, 0xaf, 0xb7, 0x5c, 0x8d, 0x5d, 0x1e, 0x27, 0x66, 0xd9, 0xe3, - 0xd8, 0x56, 0x9c, 0x7d, 0x77, 0x02, 0x7b, 0x44, 0xf4, 0x00, 0x52, 0x55, 0xc7, 0xe9, 0x90, 0x4c, - 0x3e, 0xfb, 0xaf, 0x8f, 0x15, 0x49, 0x18, 0xb8, 0x24, 0x8f, 0x19, 0x49, 0x30, 0xf7, 0xc0, 0x34, - 0x9b, 0x2d, 0x5c, 0x6e, 0x99, 0x5d, 0xad, 0x6c, 0x1a, 0x7b, 0x7a, 0x93, 0x5b, 0xcf, 0x6b, 0x63, - 0x25, 0x3e, 0x28, 0xd7, 0xa5, 0x41, 0x76, 0xf4, 0x79, 0x48, 0xd5, 0xef, 0x71, 0x51, 0xcc, 0xf7, - 0xb9, 0x3a, 0x56, 0x54, 0xfd, 0x9e, 0xe4, 0x31, 0xa1, 0x2a, 0x64, 0x56, 0x9f, 0x77, 0x2d, 0xcc, - 0x65, 0x24, 0xa9, 0x8c, 0x1b, 0x63, 0x65, 0x50, 0x1e, 0x29, 0xc8, 0x8a, 0xde, 0x86, 0xfc, 0x3b, - 0xa6, 0xf5, 0xa4, 0x65, 0x2a, 0x6e, 0xdb, 0x66, 0xa8, 0xb0, 0x97, 0xc7, 0x0a, 0x73, 0xd9, 0xa4, - 0x3e, 0x01, 0x0b, 0x5f, 0x84, 0x5c, 0xcf, 0xe0, 0x20, 0x04, 0x89, 0x0e, 0x19, 0x07, 0x81, 0x06, - 0x8f, 0xd0, 0xdf, 0xe8, 0x35, 0x98, 0x31, 0x4c, 0x0d, 0xbb, 0x3a, 0x9b, 0x2b, 0x9d, 0x3b, 0x39, - 0x5e, 0x4c, 0x6e, 0x9a, 0x1a, 0x5b, 0xf6, 0xf9, 0x2f, 0x29, 0x49, 0x0a, 0xb9, 0x8b, 0xfe, 0xc2, - 0x0d, 0x48, 0x90, 0x71, 0x21, 0x93, 0x7d, 0x57, 0xb1, 0xf1, 0x8e, 0xa5, 0x73, 0x99, 0x6e, 0x92, - 0x97, 0xfb, 0x2b, 0x01, 0x62, 0xf5, 0x7b, 0xc4, 0xff, 0xdc, 0xed, 0xaa, 0x4f, 0xb0, 0xc3, 0x4b, - 0xf1, 0x14, 0xf5, 0x4b, 0x2d, 0xbc, 0xa7, 0x33, 0x5f, 0x24, 0x2d, 0xf1, 0x14, 0x7a, 0x09, 0x40, - 0x51, 0x55, 0x6c, 0xdb, 0xb2, 0x7b, 0xd1, 0x26, 0x2d, 0xa5, 0x19, 0x85, 0x1f, 0x62, 0xda, 0x58, - 0xb5, 0xb0, 0xe3, 0x46, 0xc1, 0xb0, 0x14, 0x61, 0x73, 0x70, 0xbb, 0x23, 0x3b, 0xe6, 0x13, 0x6c, - 0xd0, 0xf1, 0x4c, 0x4b, 0x69, 0x42, 0x69, 0x10, 0x02, 0xb1, 0x3f, 0xd8, 0xd0, 0x7c, 0x63, 0x91, - 0x96, 0xbc, 0x34, 0x11, 0x69, 0xe1, 0xa6, 0xce, 0x6f, 0x8e, 0xa4, 0x25, 0x9e, 0xe2, 0xcd, 0xf8, - 0xa6, 0x00, 0xf1, 0x07, 0xe5, 0xfa, 0xa9, 0xdb, 0x81, 0x20, 0xa1, 0x74, 0xb9, 0xde, 0xa7, 0x25, - 0xfa, 0x9b, 0x86, 0x87, 0xe9, 0xad, 0x16, 0x81, 0xe3, 0x1d, 0xcb, 0xfc, 0x0a, 0x56, 0xdd, 0x56, - 0xe4, 0x39, 0x79, 0x9b, 0x51, 0xd1, 0x12, 0x64, 0x54, 0x0b, 0x6b, 0xd8, 0x70, 0x74, 0xa5, 0x65, - 0xf3, 0xe6, 0x04, 0x49, 0xbc, 0x72, 0x5f, 0x15, 0x60, 0x9a, 0x2a, 0x12, 0x7a, 0x11, 0xd2, 0xaa, - 0x69, 0x38, 0x8a, 0x6e, 0x70, 0x2b, 0x90, 0x96, 0x7c, 0xc2, 0xd0, 0x4a, 0x5e, 0x81, 0xac, 0xa2, - 0xaa, 0x66, 0xd7, 0x70, 0x64, 0x43, 0x69, 0x63, 0x5e, 0xd9, 0x0c, 0xa7, 0x6d, 0x2a, 0x6d, 0x8c, - 0x16, 0xc1, 0x4d, 0x7a, 0x57, 0x9b, 0xd2, 0x12, 0x70, 0xd2, 0x3a, 0x3e, 0xe2, 0x35, 0xf9, 0x91, - 0x00, 0x29, 0x57, 0x05, 0x49, 0x65, 0x9a, 0xd8, 0xc0, 0x96, 0xe2, 0x98, 0x5e, 0x65, 0x3c, 0x42, - 0xff, 0x2a, 0x91, 0xf6, 0x57, 0x89, 0x73, 0x30, 0xed, 0x28, 0xbb, 0x2d, 0xb7, 0x1e, 0x2c, 0x41, - 0x37, 0x40, 0x5b, 0x4a, 0x93, 0xed, 0xe2, 0xa4, 0x25, 0x96, 0x20, 0x4d, 0xe2, 0xf1, 0x88, 0xac, - 0x77, 0x78, 0x8a, 0xd4, 0x97, 0x85, 0xe0, 0xed, 0xe2, 0xa6, 0x6e, 0xd0, 0xc1, 0x8e, 0x4b, 0x40, - 0x49, 0x34, 0xda, 0x05, 0xbd, 0x00, 0x69, 0x56, 0x00, 0x1b, 0x1a, 0x1d, 0xf1, 0xb8, 0x94, 0xa2, - 0x84, 0x8a, 0x7b, 0x97, 0x83, 0xbb, 0x06, 0x7f, 0x28, 0xc0, 0x1c, 0x8b, 0x61, 0x60, 0xb1, 0x7b, - 0xd1, 0xad, 0x93, 0x6f, 0x40, 0x5a, 0x53, 0x1c, 0x85, 0x5d, 0xa5, 0x8a, 0x8d, 0xbc, 0x4a, 0xe5, - 0x9a, 0x42, 0x52, 0x9e, 0x5e, 0xa7, 0x42, 0x90, 0x20, 0xbf, 0xd9, 0x15, 0x34, 0x89, 0xfe, 0xf6, - 0xcf, 0xc1, 0x83, 0xd5, 0x8d, 0xd2, 0x6f, 0xf8, 0x59, 0xdc, 0x5d, 0x74, 0xa2, 0xec, 0x86, 0x2f, - 0xc0, 0x0c, 0x07, 0xc3, 0xbc, 0x13, 0x96, 0xc6, 0xd9, 0x3b, 0xf7, 0xd4, 0x90, 0xb3, 0xa1, 0x12, - 0x00, 0x0b, 0xbe, 0x23, 0x30, 0x76, 0xa2, 0x68, 0x1f, 0x77, 0x91, 0xa2, 0x6c, 0x84, 0x8a, 0x36, - 0x21, 0xd3, 0x7e, 0xa6, 0xaa, 0xf2, 0x9e, 0xde, 0x72, 0x78, 0xa0, 0x4f, 0x78, 0x74, 0xeb, 0xc6, - 0xe3, 0x72, 0xf9, 0x3e, 0x2d, 0xc4, 0xe2, 0x6d, 0xfc, 0xb4, 0x04, 0x44, 0x02, 0xfb, 0x8d, 0x5e, - 0x05, 0x1e, 0xf8, 0x2e, 0xdb, 0xee, 0x35, 0x96, 0x52, 0xee, 0xe4, 0x78, 0x31, 0x2d, 0x51, 0x6a, - 0xbd, 0xde, 0x90, 0xd2, 0xac, 0x40, 0xdd, 0x76, 0xd0, 0x55, 0xc8, 0x99, 0x6d, 0xdd, 0x91, 0xdd, - 0x15, 0x9e, 0xbb, 0x32, 0x59, 0x42, 0x74, 0x3d, 0x00, 0xd4, 0x80, 0x9b, 0xd8, 0x20, 0x73, 0x81, - 0xb6, 0x93, 0x45, 0x75, 0xcb, 0xba, 0xc3, 0xe6, 0x93, 0x6c, 0x76, 0x1c, 0xbd, 0xad, 0x3f, 0xa7, - 0x87, 0xab, 0xfc, 0x90, 0xe0, 0x2a, 0x2b, 0x4e, 0xda, 0x47, 0xe3, 0xbc, 0x6b, 0xbc, 0xec, 0x56, - 0xa0, 0x28, 0xd7, 0x9a, 0xaf, 0x0b, 0x90, 0x2b, 0x75, 0x5b, 0x4f, 0xb6, 0x3a, 0xf5, 0x6e, 0xbb, - 0xad, 0x58, 0x47, 0x64, 0x82, 0x30, 0xed, 0xd4, 0x9f, 0x63, 0x16, 0x05, 0xc9, 0xd5, 0x4f, 0x7f, - 0x8e, 0x89, 0xfa, 0xf1, 0xe0, 0x5a, 0x42, 0x67, 0x91, 0xb3, 0x57, 0x21, 0x47, 0x81, 0xa4, 0x8c, - 0x0d, 0xc7, 0xd2, 0x31, 0xdb, 0xa7, 0x88, 0x4b, 0x59, 0x4a, 0xac, 0x30, 0x1a, 0xba, 0x0e, 0x79, - 0xfb, 0xc8, 0x76, 0x70, 0x5b, 0x66, 0x17, 0x40, 0x19, 0xfa, 0x89, 0x4b, 0x39, 0x46, 0x95, 0x18, - 0xb1, 0xf8, 0x47, 0x71, 0xc8, 0xbb, 0x9a, 0x16, 0xa5, 0x8b, 0x56, 0x82, 0xe9, 0x3d, 0xbd, 0x85, - 0xdd, 0x33, 0xea, 0xe1, 0xcb, 0xb4, 0x17, 0x7e, 0x49, 0x56, 0x4e, 0xd7, 0x35, 0xa7, 0xac, 0x51, - 0x68, 0xdb, 0xc2, 0x4f, 0x04, 0x48, 0x50, 0xdf, 0xe8, 0x2e, 0x24, 0xe8, 0xf4, 0x17, 0x26, 0x99, - 0xfe, 0xb4, 0xa8, 0xb7, 0x84, 0xc7, 0x02, 0x4b, 0x38, 0x59, 0x0f, 0xf7, 0x95, 0xd7, 0xef, 0xae, - 0x50, 0x4d, 0xcb, 0x4a, 0x3c, 0x85, 0x4a, 0x34, 0xb2, 0xc1, 0xb4, 0x1c, 0xac, 0x71, 0xcf, 0x24, - 0x6c, 0x72, 0xf5, 0x0c, 0xbc, 0x6b, 0x6a, 0x5c, 0x3e, 0x74, 0x09, 0xe2, 0x44, 0x85, 0x67, 0xd8, - 0x51, 0xe7, 0xc9, 0xf1, 0x62, 0x9c, 0x28, 0x2f, 0xa1, 0x31, 0xc7, 0xfa, 0x61, 0x22, 0x95, 0x10, - 0xa7, 0x8b, 0x3f, 0x48, 0x40, 0xae, 0xd6, 0x8e, 0xda, 0x38, 0xac, 0xf6, 0x0e, 0x58, 0x98, 0xe3, - 0xd8, 0xf3, 0xd1, 0x90, 0xf1, 0xea, 0x31, 0xb3, 0xf1, 0xd3, 0x99, 0xd9, 0x1a, 0x71, 0x0a, 0xf8, - 0x9d, 0x55, 0xf2, 0xfd, 0x57, 0xc6, 0x7e, 0xbf, 0x41, 0x66, 0x9c, 0x44, 0x78, 0xfc, 0xf8, 0x62, - 0x7a, 0xd6, 0xfd, 0x16, 0xf5, 0x3d, 0x98, 0xd2, 0x24, 0x27, 0x57, 0x9a, 0x19, 0x6c, 0x68, 0x54, - 0x65, 0x0e, 0xb9, 0xc6, 0x7c, 0x06, 0xe2, 0x9a, 0x3e, 0xaa, 0x4b, 0xc3, 0x4c, 0x25, 0x61, 0x19, - 0xa3, 0x38, 0x89, 0xa0, 0xe2, 0x04, 0x81, 0xd3, 0xc2, 0x16, 0x80, 0xdf, 0x2a, 0xb4, 0x04, 0x49, - 0xb3, 0xa5, 0xb9, 0xa1, 0xd1, 0xb9, 0x52, 0xfa, 0xe4, 0x78, 0x71, 0x7a, 0xab, 0xa5, 0xd5, 0xd6, - 0xa4, 0x69, 0xb3, 0xa5, 0xd5, 0x34, 0x7a, 0xd5, 0x17, 0x1f, 0xc8, 0x5e, 0xa0, 0x4a, 0x56, 0x9a, - 0x31, 0xf0, 0x81, 0x1f, 0x2a, 0xe3, 0x09, 0x27, 0x6a, 0xf3, 0xbb, 0x02, 0xe4, 0xdd, 0x1e, 0x8c, - 0x76, 0xa6, 0xa7, 0xf4, 0x36, 0xd7, 0xfc, 0xf8, 0xe9, 0x34, 0xdf, 0xe5, 0xe3, 0x17, 0x92, 0xbe, - 0x26, 0xc0, 0x3c, 0x8b, 0x89, 0x53, 0x15, 0x87, 0x98, 0xf9, 0x08, 0xd5, 0xfb, 0x65, 0x10, 0x2d, - 0xc5, 0xd0, 0xcc, 0xb6, 0xfe, 0x1c, 0xb3, 0xfd, 0x11, 0x9b, 0x1f, 0x0b, 0xcc, 0x7a, 0x74, 0xba, - 0x01, 0xe0, 0x6e, 0xef, 0xfc, 0xab, 0x00, 0xe7, 0x7a, 0x2b, 0x13, 0x65, 0xa7, 0xad, 0x43, 0x92, - 0x6e, 0xed, 0xb9, 0xd3, 0xed, 0xb5, 0x10, 0x21, 0x61, 0x5f, 0x67, 0x51, 0x47, 0x9e, 0xc2, 0x53, - 0x11, 0x0b, 0x5f, 0x80, 0x69, 0x4a, 0x3e, 0x83, 0x8d, 0xe3, 0x3d, 0xff, 0x14, 0xe6, 0x56, 0x35, - 0xad, 0x5e, 0xe7, 0xda, 0x17, 0x5d, 0xb7, 0xbb, 0xde, 0x53, 0x2c, 0xcc, 0x7b, 0x0a, 0x7e, 0x32, - 0x4a, 0xef, 0xa9, 0x03, 0x79, 0x1e, 0x67, 0x1b, 0xf1, 0x5e, 0x2e, 0x8d, 0xc5, 0xe5, 0x6a, 0xc3, - 0x12, 0xfe, 0x3d, 0x4f, 0xef, 0x8b, 0x51, 0xb6, 0xa4, 0x0b, 0xf3, 0xae, 0xdc, 0xa8, 0x8f, 0x4d, - 0x46, 0x35, 0x87, 0xee, 0x89, 0x05, 0x3f, 0x1b, 0x65, 0x9b, 0xfe, 0x59, 0x80, 0x7c, 0xbd, 0xbb, - 0xcb, 0x2e, 0xfb, 0x47, 0x19, 0xe7, 0x98, 0x6e, 0xe1, 0x3d, 0x47, 0x3e, 0x5d, 0xd4, 0x9e, 0x6b, - 0x89, 0x08, 0x2b, 0xa1, 0xa2, 0x55, 0x00, 0x8b, 0xc6, 0x5d, 0x53, 0x39, 0xf1, 0x89, 0xa3, 0xff, - 0xd2, 0x94, 0x2b, 0x10, 0xa5, 0xf8, 0x83, 0x18, 0xcc, 0x7a, 0xcd, 0x8c, 0xd2, 0x74, 0xfc, 0x3f, - 0xa0, 0xde, 0xaf, 0x6c, 0x3b, 0x8a, 0x63, 0xf3, 0x1a, 0xbe, 0x7a, 0x9a, 0x4b, 0x10, 0xa5, 0x39, - 0x1e, 0x8b, 0x95, 0xf6, 0x48, 0x52, 0x9a, 0x88, 0xa4, 0x3f, 0xd1, 0x32, 0xcc, 0x53, 0xfb, 0x28, - 0x2b, 0x9d, 0x4e, 0x4b, 0xc7, 0x9a, 0xcc, 0x4e, 0x30, 0x12, 0xf4, 0x04, 0x63, 0x8e, 0x66, 0xad, - 0xb2, 0x9c, 0x1a, 0x3d, 0xcd, 0xb8, 0x0f, 0xd9, 0x3d, 0x0b, 0xe3, 0xe7, 0x58, 0xa6, 0x5e, 0xd7, - 0x69, 0xce, 0xb5, 0x32, 0x8c, 0xb1, 0x4e, 0xf8, 0xb8, 0x25, 0x7a, 0x1f, 0xe6, 0x68, 0xd7, 0x46, - 0x7d, 0x77, 0x85, 0x8f, 0xca, 0xcf, 0x05, 0x40, 0x41, 0xf9, 0x1f, 0xdd, 0xc0, 0xc4, 0x22, 0x1f, - 0x98, 0x57, 0x01, 0xb1, 0x50, 0x03, 0x5b, 0xee, 0x60, 0x4b, 0xb6, 0xb1, 0x6a, 0xf2, 0x1b, 0xea, - 0x82, 0x24, 0xf2, 0x9c, 0x6d, 0x6c, 0xd5, 0x29, 0xbd, 0xf8, 0xf7, 0x05, 0xc8, 0xf2, 0x3e, 0xd9, - 0x31, 0x08, 0xd8, 0xbf, 0x0b, 0xf1, 0x26, 0xdf, 0x4d, 0xc9, 0x84, 0xe2, 0x2d, 0xff, 0x95, 0x8c, - 0xea, 0x94, 0x44, 0xca, 0x12, 0x96, 0x4e, 0xd7, 0x09, 0x09, 0xad, 0xf3, 0x23, 0xa8, 0x82, 0x2c, - 0x9d, 0xae, 0x83, 0xea, 0x30, 0xab, 0xfa, 0xaf, 0x04, 0xc8, 0x84, 0x3d, 0x3e, 0xf4, 0x2a, 0x47, - 0xe8, 0x5b, 0x0b, 0xd5, 0x29, 0x29, 0xaf, 0xf6, 0x64, 0xa0, 0x72, 0xf0, 0x5a, 0x7a, 0x62, 0xe8, - 0xde, 0x61, 0xff, 0x95, 0xf8, 0xea, 0x54, 0xe0, 0xf6, 0x3a, 0x7a, 0x03, 0x92, 0x1a, 0xbd, 0xee, - 0xcc, 0x35, 0x34, 0x4c, 0x89, 0x7a, 0x6e, 0x98, 0x57, 0xa7, 0x24, 0xce, 0x81, 0x1e, 0x42, 0x96, - 0xfd, 0x62, 0xf7, 0x7c, 0xb9, 0x5b, 0x79, 0x7d, 0xb8, 0x84, 0x80, 0xb5, 0xae, 0x4e, 0x49, 0x19, - 0xcd, 0xa7, 0xa2, 0x4f, 0x42, 0xc2, 0x56, 0x15, 0x83, 0x6f, 0x39, 0x5e, 0x1e, 0x72, 0xad, 0xd3, - 0x67, 0xa6, 0xa5, 0xd1, 0xbb, 0x30, 0x47, 0x37, 0x58, 0x64, 0xc7, 0x3f, 0x79, 0xa6, 0xd7, 0x4a, - 0x7a, 0x0f, 0xbb, 0x3d, 0x77, 0x2b, 0xfc, 0x36, 0x53, 0x75, 0x4a, 0x12, 0x77, 0xfb, 0xb2, 0xc8, - 0x90, 0x51, 0x7f, 0x39, 0x20, 0x38, 0x3d, 0x74, 0xc8, 0x42, 0xef, 0x17, 0x91, 0x21, 0xc3, 0x3d, - 0x19, 0xe8, 0x01, 0x64, 0x14, 0xe2, 0xbf, 0xc8, 0xf4, 0x3a, 0x44, 0x01, 0x86, 0xee, 0x1d, 0x0f, - 0xdc, 0xd0, 0xa8, 0xd2, 0xab, 0x59, 0x2e, 0xd1, 0x17, 0xd4, 0xc6, 0x56, 0x13, 0x17, 0x32, 0xa3, - 0x05, 0x05, 0x8f, 0xab, 0x3d, 0x41, 0x94, 0x88, 0x36, 0x20, 0xb7, 0xef, 0xc6, 0xfb, 0xd2, 0x50, - 0x81, 0xec, 0xd0, 0x0d, 0xe4, 0x90, 0x78, 0xe5, 0xea, 0x94, 0x94, 0xdd, 0x0f, 0x90, 0xd1, 0x32, - 0xc4, 0x9a, 0x6a, 0x21, 0x47, 0x65, 0xbc, 0x38, 0x2a, 0x1a, 0xb7, 0x3a, 0x25, 0xc5, 0x9a, 0x2a, - 0x41, 0x25, 0x2c, 0x6c, 0xf0, 0xd0, 0x28, 0xe4, 0x87, 0x1a, 0x99, 0xde, 0x60, 0xd1, 0xea, 0x94, - 0x44, 0x43, 0x2a, 0xc9, 0xf7, 0xb6, 0x21, 0x6f, 0xb1, 0xf3, 0x7e, 0x37, 0xaa, 0x45, 0xa4, 0x52, - 0x6e, 0x86, 0x9b, 0xaa, 0x81, 0xc0, 0x96, 0xea, 0x94, 0x94, 0xb3, 0x82, 0x74, 0xf4, 0x65, 0x38, - 0xd7, 0x2b, 0x91, 0x2b, 0xf7, 0xdc, 0x80, 0xe5, 0x0a, 0x97, 0xdb, 0xab, 0xe3, 0xc8, 0x1a, 0xc8, - 0x44, 0x9f, 0x86, 0x69, 0x36, 0x6a, 0x88, 0x8a, 0x0c, 0x3b, 0x6a, 0xea, 0x1b, 0x30, 0x56, 0x9e, - 0xcc, 0x37, 0x87, 0x1f, 0x74, 0xcb, 0x2d, 0xb3, 0x59, 0x98, 0x1f, 0x3a, 0xdf, 0x06, 0x0f, 0xee, - 0xc9, 0x7c, 0x73, 0x7c, 0x2a, 0x19, 0x77, 0x8b, 0xe5, 0xf0, 0x73, 0xd1, 0x73, 0x43, 0xc7, 0x3d, - 0xe4, 0xfc, 0xbb, 0x4a, 0x83, 0xf6, 0x7c, 0x32, 0xa9, 0x9a, 0xc5, 0x6e, 0x52, 0xcb, 0x74, 0x1a, - 0x9f, 0x1f, 0x5a, 0xb5, 0xc1, 0xdb, 0xe1, 0xa4, 0x6a, 0x96, 0x4f, 0x45, 0x8f, 0x41, 0xe4, 0xb7, - 0x25, 0xfd, 0xfd, 0xa8, 0x0b, 0x43, 0x4f, 0x22, 0xc2, 0x0f, 0x12, 0xab, 0x53, 0xd2, 0xac, 0xda, - 0x9b, 0x43, 0x8c, 0x05, 0x95, 0x27, 0xab, 0xfe, 0x35, 0xd7, 0x42, 0x61, 0xa8, 0xb1, 0x18, 0x72, - 0x31, 0x9a, 0x18, 0x0b, 0xb5, 0x2f, 0x8b, 0xa8, 0xb1, 0x6e, 0xe8, 0x0e, 0x35, 0xec, 0x0b, 0x43, - 0xd5, 0xb8, 0xf7, 0x21, 0x17, 0xa2, 0xc6, 0x3a, 0xa3, 0x10, 0x35, 0x76, 0xf8, 0xa1, 0x39, 0x1f, - 0x8e, 0x17, 0x87, 0xaa, 0x71, 0xd8, 0xe9, 0x3a, 0x51, 0x63, 0x27, 0x48, 0x27, 0x6a, 0xcc, 0x0c, - 0x44, 0x9f, 0xdc, 0x97, 0x86, 0xaa, 0xf1, 0xd0, 0x1b, 0x46, 0x44, 0x8d, 0x95, 0x81, 0x4c, 0xb4, - 0x06, 0xc0, 0x3c, 0x22, 0xdd, 0xd8, 0x33, 0x0b, 0x97, 0x87, 0xae, 0x3f, 0xfd, 0xc7, 0xe6, 0x64, - 0xfd, 0x69, 0xb9, 0x34, 0x62, 0xc8, 0xa8, 0x8f, 0x2d, 0xd3, 0xcd, 0xef, 0xc2, 0xe2, 0x50, 0x43, - 0x36, 0xb0, 0x07, 0x4e, 0x0c, 0xd9, 0x81, 0x47, 0x24, 0x0b, 0x19, 0xdb, 0x32, 0x2a, 0x2c, 0x8d, - 0xd9, 0x9a, 0x08, 0x2c, 0x64, 0x8c, 0x03, 0xad, 0x42, 0x9a, 0x78, 0x0a, 0x47, 0xd4, 0x0c, 0x5d, - 0x19, 0xea, 0xdd, 0xf6, 0x45, 0xa5, 0x56, 0xa7, 0xa4, 0xd4, 0x53, 0x4e, 0x22, 0x9f, 0x67, 0xb8, - 0xbd, 0x50, 0x1c, 0xfa, 0xf9, 0x9e, 0x9d, 0x1a, 0xf2, 0x79, 0xc6, 0x81, 0x54, 0x38, 0xcf, 0xc6, - 0x8a, 0x5f, 0x49, 0xb2, 0xf8, 0xdd, 0x9d, 0xc2, 0x55, 0x2a, 0x6a, 0x28, 0x0a, 0x0e, 0xbd, 0x1e, - 0x55, 0x9d, 0x92, 0xe6, 0x95, 0xc1, 0x5c, 0x32, 0xe1, 0xf9, 0xd2, 0xc3, 0xb0, 0x73, 0xe1, 0xda, - 0xd0, 0x09, 0x1f, 0xb2, 0xdb, 0x40, 0x26, 0xbc, 0x12, 0x20, 0xb3, 0x05, 0x48, 0x93, 0x6d, 0x9b, - 0x1d, 0x95, 0x5c, 0x1f, 0xb1, 0x00, 0xf5, 0x21, 0x68, 0xb6, 0x00, 0x69, 0x75, 0xc6, 0x49, 0x04, - 0xa9, 0x2d, 0xac, 0x58, 0xdc, 0xcc, 0xde, 0x18, 0x2a, 0x68, 0xe0, 0x71, 0x14, 0x22, 0x48, 0xf5, - 0x88, 0x64, 0xc1, 0xb6, 0xdc, 0x6b, 0xd6, 0xdc, 0xdb, 0xbc, 0x39, 0x74, 0xc1, 0x0e, 0xbd, 0x0d, - 0x4e, 0x16, 0x6c, 0xab, 0x27, 0x03, 0x7d, 0x0e, 0x66, 0xf8, 0x9d, 0xd5, 0xc2, 0xad, 0x11, 0x3e, - 0x70, 0x10, 0x56, 0x93, 0x79, 0xcd, 0x79, 0x98, 0x95, 0x65, 0x57, 0x5e, 0x59, 0xf3, 0x5e, 0x1e, - 0x61, 0x65, 0x07, 0x10, 0x2d, 0xb3, 0xb2, 0x3e, 0x99, 0x58, 0x59, 0xa6, 0xa7, 0x7c, 0xad, 0xbb, - 0x3d, 0xd4, 0xca, 0x0e, 0x86, 0xc4, 0x12, 0x2b, 0xfb, 0xd4, 0xa7, 0x92, 0x96, 0xd9, 0x0c, 0x88, - 0x15, 0x5e, 0x19, 0xda, 0xb2, 0x5e, 0x44, 0x4a, 0x5a, 0xc6, 0x79, 0xc8, 0xb0, 0xb1, 0x68, 0x2d, - 0xd6, 0xd3, 0xaf, 0x0e, 0x1d, 0xb6, 0x01, 0xdc, 0x52, 0x75, 0x1f, 0xce, 0x63, 0x3d, 0xec, 0x19, - 0x2a, 0x8b, 0xdf, 0xb6, 0xe2, 0x3d, 0xf5, 0xda, 0x68, 0x43, 0x15, 0x76, 0x91, 0xcc, 0x33, 0x54, - 0x3d, 0x99, 0xb4, 0xaa, 0x2c, 0xc0, 0x9c, 0xce, 0xef, 0xe5, 0xe1, 0x55, 0xed, 0x8f, 0xf9, 0xa7, - 0x55, 0xf5, 0x88, 0xa5, 0x19, 0x1e, 0xa4, 0xf7, 0x30, 0x91, 0x9a, 0x15, 0xc5, 0x87, 0x89, 0xd4, - 0x45, 0xb1, 0xf0, 0x30, 0x91, 0xba, 0x24, 0x2e, 0x3c, 0x4c, 0xa4, 0x5e, 0x10, 0x5f, 0x2c, 0xfe, - 0x69, 0x01, 0x72, 0x2e, 0x20, 0x62, 0xf0, 0x62, 0x25, 0x08, 0x2f, 0x2e, 0x0f, 0x83, 0x17, 0x1c, - 0x42, 0x71, 0x7c, 0xb1, 0x12, 0xc4, 0x17, 0x97, 0x87, 0xe1, 0x0b, 0x9f, 0x87, 0x00, 0x8c, 0xc6, - 0x30, 0x80, 0xf1, 0xf2, 0x04, 0x00, 0xc3, 0x13, 0xd5, 0x8f, 0x30, 0xd6, 0x06, 0x11, 0xc6, 0xb5, - 0xd1, 0x08, 0xc3, 0x13, 0x15, 0x80, 0x18, 0x6f, 0xf6, 0x41, 0x8c, 0x2b, 0x23, 0x20, 0x86, 0xc7, - 0xef, 0x62, 0x8c, 0xf5, 0x50, 0x8c, 0x71, 0x63, 0x1c, 0xc6, 0xf0, 0xe4, 0xf4, 0x80, 0x8c, 0xd7, - 0x7b, 0x40, 0xc6, 0xe2, 0x50, 0x90, 0xe1, 0x71, 0x33, 0x94, 0xf1, 0xde, 0x70, 0x94, 0xf1, 0xca, - 0x44, 0x28, 0xc3, 0x93, 0x37, 0x08, 0x33, 0x1a, 0xc3, 0x60, 0xc6, 0xcb, 0x13, 0xc0, 0x0c, 0x7f, - 0xe0, 0xfa, 0x70, 0x46, 0x35, 0x0c, 0x67, 0x5c, 0x1f, 0x83, 0x33, 0x3c, 0x69, 0x41, 0xa0, 0x51, - 0x0d, 0x03, 0x1a, 0xd7, 0xc7, 0x00, 0x8d, 0x3e, 0x49, 0x0c, 0x69, 0x6c, 0x86, 0x23, 0x8d, 0x9b, - 0x63, 0x91, 0x86, 0x27, 0xad, 0x17, 0x6a, 0xdc, 0x09, 0x40, 0x8d, 0x97, 0x86, 0x40, 0x0d, 0x8f, - 0x95, 0x60, 0x8d, 0xcf, 0x0f, 0x60, 0x8d, 0xe2, 0x28, 0xac, 0xe1, 0xf1, 0x7a, 0x60, 0xe3, 0xed, - 0x21, 0x60, 0xe3, 0xd6, 0x78, 0xb0, 0xe1, 0x09, 0xeb, 0x43, 0x1b, 0xca, 0x48, 0xb4, 0xf1, 0xda, - 0x84, 0x68, 0xc3, 0x93, 0x1e, 0x06, 0x37, 0x3e, 0xd3, 0x0b, 0x37, 0x96, 0x86, 0xc3, 0x0d, 0x4f, - 0x0c, 0xc7, 0x1b, 0xeb, 0xa1, 0x78, 0xe3, 0xc6, 0x38, 0xbc, 0xe1, 0xcf, 0xbd, 0x20, 0xe0, 0xd8, - 0x0c, 0x07, 0x1c, 0x37, 0xc7, 0x02, 0x0e, 0x7f, 0xf8, 0x7b, 0x10, 0xc7, 0x7a, 0x28, 0xe2, 0xb8, - 0x31, 0x0e, 0x71, 0xf8, 0x95, 0x0b, 0x42, 0x8e, 0x77, 0x86, 0x42, 0x8e, 0xdb, 0x93, 0x40, 0x0e, - 0x4f, 0xe8, 0x00, 0xe6, 0x78, 0x6f, 0x38, 0xe6, 0x78, 0xe5, 0x14, 0xaf, 0xf0, 0x84, 0x82, 0x8e, - 0xcf, 0x0f, 0x80, 0x8e, 0xe2, 0x28, 0xd0, 0xe1, 0xeb, 0xb3, 0x8b, 0x3a, 0x94, 0x91, 0x18, 0xe1, - 0xb5, 0x09, 0x31, 0x82, 0xaf, 0x7c, 0x21, 0x20, 0xa1, 0x12, 0x02, 0x12, 0xae, 0x8d, 0x06, 0x09, - 0xfe, 0x12, 0xe2, 0xa3, 0x84, 0x6a, 0x18, 0x4a, 0xb8, 0x3e, 0x06, 0x25, 0xf8, 0x56, 0x28, 0x00, - 0x13, 0xde, 0xec, 0x83, 0x09, 0x57, 0xc6, 0x1e, 0xc1, 0x07, 0x70, 0x42, 0x69, 0x10, 0x27, 0x5c, - 0x1d, 0x89, 0x13, 0x3c, 0x09, 0x3e, 0x50, 0x78, 0xb3, 0x0f, 0x28, 0x5c, 0x19, 0x01, 0x14, 0xfc, - 0x0a, 0x70, 0xa4, 0xa0, 0x8d, 0x46, 0x0a, 0xcb, 0x93, 0x22, 0x05, 0x4f, 0x70, 0x28, 0x54, 0xd8, - 0x0c, 0x87, 0x0a, 0x37, 0x27, 0x3c, 0x8d, 0x1b, 0xc0, 0x0a, 0xd5, 0x30, 0xac, 0x70, 0x7d, 0x0c, - 0x56, 0x08, 0xae, 0x21, 0x1e, 0x58, 0xa8, 0x86, 0x81, 0x85, 0xeb, 0x63, 0xc0, 0x82, 0x2f, 0x29, - 0x80, 0x16, 0x1a, 0xc3, 0xd0, 0xc2, 0xcb, 0x13, 0xa0, 0x05, 0x7f, 0xdd, 0xed, 0x83, 0x0b, 0x6f, - 0xf5, 0xc3, 0x85, 0xe2, 0x28, 0xb8, 0xe0, 0xcf, 0x48, 0x17, 0x2f, 0x6c, 0x86, 0xe3, 0x85, 0x9b, - 0x63, 0xf1, 0x42, 0xd0, 0x48, 0x06, 0x00, 0xc3, 0x7a, 0x28, 0x60, 0xb8, 0x31, 0x0e, 0x30, 0xf8, - 0x46, 0x32, 0x88, 0x18, 0xde, 0xea, 0x47, 0x0c, 0xc5, 0x51, 0x88, 0xc1, 0x6f, 0x9c, 0x0b, 0x19, - 0xaa, 0x61, 0x90, 0xe1, 0xfa, 0x18, 0xc8, 0xe0, 0x0f, 0x5e, 0x00, 0x33, 0x28, 0x23, 0x31, 0xc3, - 0x6b, 0x13, 0x62, 0x86, 0x3e, 0xc3, 0xd5, 0x0b, 0x1a, 0xaa, 0x61, 0xa0, 0xe1, 0xfa, 0x18, 0xd0, - 0x10, 0xa8, 0xec, 0xe4, 0xa8, 0xe1, 0x61, 0x22, 0xf5, 0xa2, 0xf8, 0x52, 0xf1, 0x2f, 0xa6, 0x21, - 0x59, 0x75, 0x83, 0x4e, 0x02, 0x37, 0xf8, 0x85, 0xb3, 0xdc, 0xe0, 0x47, 0x6b, 0x44, 0x11, 0xe9, - 0x74, 0xe6, 0x38, 0x62, 0xc4, 0x73, 0x18, 0x03, 0xe7, 0x7e, 0x2e, 0xeb, 0x19, 0xae, 0xe4, 0xa0, - 0xd7, 0x21, 0xd7, 0xb5, 0xb1, 0x25, 0x77, 0x2c, 0xdd, 0xb4, 0x74, 0x87, 0x85, 0x6a, 0x0a, 0x25, - 0xf1, 0xc3, 0xe3, 0xc5, 0xec, 0x8e, 0x8d, 0xad, 0x6d, 0x4e, 0x97, 0xb2, 0xdd, 0x40, 0xca, 0x7d, - 0x0a, 0x7e, 0x7a, 0xf2, 0xa7, 0xe0, 0xdf, 0x06, 0xd1, 0xc2, 0x8a, 0xd6, 0xb3, 0xb0, 0xb2, 0x4b, - 0xed, 0xe1, 0x3e, 0x00, 0x8d, 0x49, 0x76, 0x4b, 0xd2, 0xcb, 0xed, 0xb3, 0x56, 0x2f, 0x11, 0xdd, - 0x85, 0xf3, 0x6d, 0xe5, 0x90, 0xbd, 0xe5, 0xe0, 0xfa, 0x2a, 0x34, 0xf8, 0x26, 0x45, 0x63, 0xc4, - 0x50, 0x5b, 0x39, 0xa4, 0xef, 0xca, 0xb3, 0x2c, 0xfa, 0xa2, 0xec, 0x75, 0xc8, 0x6b, 0xba, 0xed, - 0xe8, 0x86, 0xea, 0xbe, 0xa5, 0xc5, 0x6e, 0xbf, 0xe7, 0x5c, 0x2a, 0x7b, 0xd3, 0xea, 0x36, 0xcc, - 0xf1, 0x68, 0x3c, 0xff, 0xa5, 0x79, 0xea, 0x95, 0xa7, 0x48, 0x2d, 0x48, 0x86, 0xff, 0x2f, 0x06, - 0xca, 0x30, 0xdb, 0x54, 0x1c, 0x7c, 0xa0, 0x1c, 0xc9, 0x6e, 0x58, 0x74, 0x86, 0x3e, 0xca, 0xf3, - 0xc2, 0xc9, 0xf1, 0x62, 0xee, 0x01, 0xcb, 0x1a, 0x88, 0x8e, 0xce, 0x35, 0x03, 0x19, 0x1a, 0x5a, - 0x85, 0x2c, 0x7d, 0x2c, 0xd3, 0x64, 0x6f, 0xb2, 0x72, 0x5f, 0x7b, 0xd8, 0xb1, 0x0a, 0x7f, 0xb9, - 0x55, 0xa2, 0x0f, 0x6c, 0xba, 0xcf, 0xb8, 0xde, 0x84, 0x59, 0xc5, 0x3e, 0x32, 0x54, 0xda, 0xc3, - 0xd8, 0xb0, 0xbb, 0x36, 0x75, 0xb6, 0x53, 0x52, 0x9e, 0x92, 0xcb, 0x2e, 0x95, 0xbf, 0x9f, 0xf5, - 0x1b, 0x02, 0x64, 0x7b, 0x42, 0x54, 0xdf, 0xec, 0x3b, 0x3d, 0xbc, 0x14, 0xee, 0xe8, 0x0f, 0x8b, - 0xbb, 0x4a, 0xf1, 0x11, 0x70, 0x63, 0x41, 0x16, 0x87, 0x3b, 0x8a, 0x14, 0x6a, 0xbb, 0x67, 0xd6, - 0x2e, 0xdb, 0x1b, 0x89, 0xdf, 0xfa, 0xf6, 0xe2, 0x54, 0xf1, 0x17, 0x71, 0xc8, 0xf5, 0x86, 0xa2, - 0xd6, 0xfa, 0xea, 0x15, 0x66, 0x5c, 0x7b, 0x38, 0x96, 0x47, 0x3c, 0x3a, 0x93, 0xf6, 0x9f, 0xd8, - 0x64, 0xd5, 0x5c, 0x1a, 0x71, 0x46, 0x1a, 0xac, 0xa7, 0xcf, 0xb8, 0xf0, 0xc3, 0x98, 0x37, 0xf3, - 0x97, 0x61, 0x9a, 0x5e, 0x0b, 0xe6, 0x55, 0x0b, 0xbb, 0xac, 0x53, 0x21, 0xf9, 0x12, 0x2b, 0x46, - 0x2c, 0x45, 0xe3, 0x4c, 0x6f, 0x7d, 0x78, 0x84, 0x33, 0xfc, 0x13, 0x06, 0xfe, 0xbe, 0xcc, 0xf4, - 0xe9, 0xde, 0x97, 0x61, 0x67, 0xa0, 0xad, 0x16, 0x56, 0x1d, 0xfe, 0x5f, 0x37, 0xdc, 0x7f, 0xb5, - 0x70, 0xad, 0x5f, 0x04, 0xff, 0x1f, 0x1d, 0xcb, 0x12, 0xff, 0x1f, 0x1d, 0x81, 0xf0, 0x9c, 0xbc, - 0x27, 0x82, 0x4e, 0x2c, 0x16, 0xc4, 0xc5, 0x87, 0xfa, 0x57, 0x05, 0x10, 0xe9, 0x34, 0xba, 0x8f, - 0xb1, 0x16, 0x89, 0x16, 0xba, 0x91, 0x43, 0xb1, 0x89, 0x23, 0x87, 0x8a, 0x0a, 0xe4, 0xbd, 0x3a, - 0xb0, 0x37, 0xe7, 0x47, 0x3c, 0xd5, 0x72, 0xa6, 0x5b, 0x9f, 0xc5, 0xdf, 0x16, 0x60, 0xde, 0xfb, - 0x46, 0xd9, 0xbb, 0x6c, 0x74, 0x96, 0x58, 0x4e, 0x89, 0xfe, 0x3f, 0x0d, 0x82, 0x18, 0xe9, 0x4d, - 0xc0, 0x89, 0x34, 0x08, 0xf1, 0xe3, 0x78, 0xe0, 0x48, 0x54, 0x6b, 0xd4, 0xe9, 0x7f, 0xda, 0x60, - 0xbf, 0xed, 0xe2, 0xfd, 0x40, 0x0f, 0x50, 0x65, 0x25, 0xcd, 0x9c, 0x48, 0xab, 0xdd, 0x66, 0xd2, - 0xc2, 0xc5, 0x1f, 0x0b, 0x41, 0x41, 0xcf, 0x88, 0x07, 0x72, 0x0f, 0xe2, 0xcf, 0x94, 0xd6, 0xa8, - 0x68, 0x84, 0x9e, 0xae, 0x97, 0x48, 0x69, 0x74, 0xbf, 0xe7, 0x8e, 0x56, 0x6c, 0x38, 0x4c, 0x1c, - 0xec, 0xd2, 0xe0, 0x5d, 0x2e, 0xf4, 0x69, 0xb7, 0x15, 0xf1, 0xf1, 0x9f, 0x0f, 0x4e, 0xd2, 0x37, - 0x12, 0x1f, 0x7c, 0x7b, 0x51, 0xb8, 0x5d, 0x87, 0xf9, 0x90, 0x45, 0x08, 0xe5, 0x01, 0x02, 0xaf, - 0x9e, 0xf2, 0x7f, 0xe7, 0xb1, 0xba, 0x26, 0xef, 0x6c, 0x96, 0xb7, 0x36, 0x36, 0x6a, 0x8d, 0x46, - 0x65, 0x4d, 0x14, 0x90, 0x08, 0xd9, 0x9e, 0x37, 0x53, 0xf9, 0x7f, 0xf1, 0xb8, 0xfd, 0x29, 0x00, - 0xff, 0xb5, 0x63, 0x22, 0x6b, 0xbd, 0xf2, 0xae, 0xfc, 0x78, 0xf5, 0xd1, 0x4e, 0xa5, 0x2e, 0x4e, - 0x21, 0x04, 0xf9, 0xd2, 0x6a, 0xa3, 0x5c, 0x95, 0xa5, 0x4a, 0x7d, 0x7b, 0x6b, 0xb3, 0x5e, 0x11, - 0x05, 0xce, 0xb7, 0x06, 0xd9, 0xe0, 0x6d, 0x36, 0x34, 0x0f, 0xb3, 0xe5, 0x6a, 0xa5, 0xbc, 0x2e, - 0x3f, 0xae, 0xad, 0xca, 0x6f, 0xef, 0x54, 0x76, 0x2a, 0xe2, 0x14, 0xad, 0x1a, 0x25, 0xde, 0xdf, - 0x79, 0xf4, 0x48, 0x14, 0xd0, 0x2c, 0x64, 0x58, 0x9a, 0xbe, 0xaf, 0x2a, 0xc6, 0x6e, 0x6f, 0x40, - 0x26, 0xf0, 0x58, 0x0c, 0xf9, 0xdc, 0xf6, 0x4e, 0xbd, 0x2a, 0x37, 0x6a, 0x1b, 0x95, 0x7a, 0x63, - 0x75, 0x63, 0x9b, 0xc9, 0xa0, 0xb4, 0xd5, 0xd2, 0x96, 0xd4, 0x10, 0x05, 0x2f, 0xdd, 0xd8, 0xda, - 0x29, 0x57, 0xbd, 0x7f, 0x46, 0x92, 0x48, 0xc5, 0xc5, 0xf8, 0x6d, 0x13, 0xce, 0x87, 0x5e, 0x0f, - 0x43, 0x19, 0x98, 0xd9, 0x31, 0xe8, 0xe3, 0x17, 0xe2, 0x14, 0xca, 0x05, 0x6e, 0x88, 0x89, 0x02, - 0x4a, 0xb1, 0x7b, 0x40, 0x62, 0x0c, 0x25, 0x21, 0x56, 0xbf, 0x27, 0xc6, 0x49, 0x35, 0x03, 0xd7, - 0xac, 0xc4, 0x04, 0x4a, 0xf3, 0xdb, 0x29, 0xe2, 0x34, 0xca, 0xfa, 0xd7, 0x43, 0xc4, 0xe4, 0xed, - 0x2b, 0x10, 0x88, 0x9e, 0x47, 0x00, 0xc9, 0x47, 0x8a, 0x83, 0x6d, 0x47, 0x9c, 0x42, 0x33, 0x10, - 0x5f, 0x6d, 0xb5, 0x44, 0x61, 0xe5, 0xfb, 0x02, 0xa4, 0xdc, 0xa7, 0x35, 0xd1, 0x23, 0x98, 0x66, - 0x20, 0x72, 0x71, 0xf8, 0x92, 0x41, 0xad, 0xce, 0xc2, 0xd2, 0xb8, 0x35, 0xa5, 0x38, 0x85, 0xde, - 0xe1, 0xff, 0x57, 0x88, 0xe8, 0x0b, 0xba, 0x3a, 0x4a, 0x9b, 0x5c, 0xa9, 0xa3, 0x55, 0x8e, 0xcc, - 0x90, 0xe2, 0xd4, 0x27, 0x84, 0xd2, 0x95, 0x0f, 0xfe, 0xf1, 0xf2, 0xd4, 0x07, 0x27, 0x97, 0x85, - 0x9f, 0x9c, 0x5c, 0x16, 0x7e, 0x76, 0x72, 0x59, 0xf8, 0x87, 0x93, 0xcb, 0xc2, 0xaf, 0xff, 0xd3, - 0xe5, 0xa9, 0xf7, 0x66, 0x38, 0xd7, 0x6e, 0x92, 0xfe, 0x6b, 0xa3, 0x7b, 0xff, 0x15, 0x00, 0x00, - 0xff, 0xff, 0x61, 0x40, 0x93, 0xf6, 0xdf, 0x69, 0x00, 0x00, + 0x37, 0xa7, 0xbb, 0x39, 0x92, 0x06, 0xf8, 0x11, 0x20, 0x2f, 0x4e, 0x0c, 0xc3, 0xc8, 0x43, 0x10, + 0x04, 0x76, 0x82, 0x18, 0xb0, 0x81, 0x00, 0x0e, 0x62, 0xc4, 0x81, 0x81, 0x00, 0x41, 0x02, 0x3f, + 0x24, 0x0f, 0x0b, 0xc3, 0x0f, 0x46, 0x10, 0x07, 0x46, 0x1e, 0x84, 0x44, 0x49, 0x60, 0x23, 0xaf, + 0x79, 0x08, 0xb0, 0x0f, 0x49, 0x50, 0x97, 0xbe, 0x90, 0x6c, 0x5e, 0x24, 0xf7, 0x22, 0x1b, 0xe4, + 0x49, 0xac, 0x53, 0x75, 0x4e, 0xd7, 0xe5, 0xd4, 0xa9, 0xf3, 0x55, 0x9d, 0x2a, 0xc1, 0x9c, 0x65, + 0x2a, 0xea, 0x7e, 0x67, 0xf7, 0x8e, 0xd2, 0xd1, 0x97, 0x3b, 0x96, 0xe9, 0x98, 0x68, 0x4e, 0x35, + 0xd5, 0x27, 0x94, 0xbc, 0xcc, 0x33, 0x17, 0x90, 0x5b, 0x4a, 0x53, 0x1c, 0x85, 0x15, 0x5b, 0x38, + 0xe7, 0xd2, 0xb0, 0x65, 0x99, 0x96, 0xcd, 0xa9, 0x17, 0x5c, 0x6a, 0x1b, 0x3b, 0x4a, 0xa0, 0x74, + 0xd1, 0x76, 0x4c, 0x4b, 0x69, 0xe2, 0x3b, 0xd8, 0x68, 0xea, 0x86, 0xfb, 0x87, 0x94, 0x7b, 0xa6, + 0xaa, 0xbc, 0xcc, 0xd5, 0x51, 0x65, 0xee, 0xf1, 0x42, 0x85, 0xae, 0xa3, 0xb7, 0xee, 0xec, 0xb7, + 0xd4, 0x3b, 0x8e, 0xde, 0xc6, 0xb6, 0xa3, 0xb4, 0x3b, 0x3c, 0x67, 0x89, 0xe6, 0x38, 0x96, 0xa2, + 0xea, 0x46, 0xf3, 0x8e, 0x85, 0x55, 0xd3, 0xd2, 0xb0, 0x26, 0xdb, 0x1d, 0xc5, 0x70, 0xab, 0xdc, + 0x34, 0x9b, 0x26, 0xfd, 0x79, 0x87, 0xfc, 0x62, 0xd4, 0xe2, 0xaf, 0x41, 0x5a, 0x52, 0x8c, 0x26, + 0xae, 0x19, 0x7b, 0x26, 0xfa, 0x2c, 0x24, 0x34, 0x6c, 0xab, 0x05, 0x61, 0x49, 0xb8, 0x95, 0x59, + 0x29, 0x2e, 0x0f, 0xf4, 0xc5, 0x32, 0x2d, 0xbb, 0x86, 0x6d, 0xd5, 0xd2, 0x3b, 0x8e, 0x69, 0x95, + 0x12, 0x1f, 0x1c, 0x2f, 0x4e, 0x49, 0x94, 0x0b, 0x7d, 0x12, 0xa6, 0x5b, 0x58, 0xb1, 0x71, 0x21, + 0x46, 0xd9, 0x0b, 0x21, 0xec, 0x8f, 0x48, 0x3e, 0x67, 0x62, 0x85, 0x8b, 0x7f, 0x25, 0x40, 0x4e, + 0xc2, 0x4f, 0xbb, 0xd8, 0x76, 0xaa, 0x58, 0xd1, 0xb0, 0x85, 0x2e, 0x41, 0xfc, 0x09, 0x3e, 0x2a, + 0xc4, 0x97, 0x84, 0x5b, 0xd9, 0xd2, 0xcc, 0x87, 0xc7, 0x8b, 0xf1, 0x75, 0x7c, 0x24, 0x11, 0x1a, + 0x5a, 0x82, 0x19, 0x6c, 0x68, 0x32, 0xc9, 0x4e, 0xf4, 0x66, 0x27, 0xb1, 0xa1, 0xad, 0xe3, 0x23, + 0xa4, 0x42, 0xca, 0x26, 0xd2, 0x0c, 0x15, 0x17, 0xa6, 0x97, 0x84, 0x5b, 0xd3, 0xa5, 0x07, 0x1f, + 0x1e, 0x2f, 0x96, 0x9b, 0xba, 0xb3, 0xdf, 0xdd, 0x5d, 0x56, 0xcd, 0xf6, 0x1d, 0xaf, 0x56, 0xda, + 0xae, 0xff, 0xfb, 0x4e, 0xe7, 0x49, 0xf3, 0xce, 0x90, 0x11, 0x58, 0x6e, 0x1c, 0x1a, 0x75, 0xfc, + 0x54, 0xf2, 0x04, 0xbf, 0x91, 0xf8, 0xe5, 0xb7, 0x17, 0x85, 0x87, 0x89, 0x94, 0x20, 0xc6, 0x1e, + 0x26, 0x52, 0x31, 0x31, 0x5e, 0xfc, 0x7a, 0x1c, 0xf2, 0x12, 0xb6, 0x3b, 0xa6, 0x61, 0x63, 0xde, + 0x8c, 0x4f, 0x40, 0xdc, 0x39, 0x34, 0x68, 0x33, 0x32, 0x2b, 0x97, 0x43, 0x3a, 0xa3, 0x61, 0x29, + 0x86, 0xad, 0xa8, 0x8e, 0x6e, 0x1a, 0x12, 0x29, 0x8a, 0x3e, 0x03, 0x19, 0x0b, 0xdb, 0xdd, 0x36, + 0xa6, 0xc3, 0x46, 0x5b, 0x98, 0x59, 0xb9, 0x18, 0xc2, 0x59, 0xef, 0x28, 0x86, 0x04, 0xac, 0x2c, + 0xf9, 0x8d, 0x2e, 0x41, 0xca, 0xe8, 0xb6, 0x49, 0xbf, 0xd8, 0xb4, 0xd5, 0x71, 0x69, 0xc6, 0xe8, + 0xb6, 0xd7, 0xf1, 0x91, 0x8d, 0xca, 0x90, 0xb1, 0xc8, 0xa0, 0xc9, 0xba, 0xb1, 0x67, 0xda, 0x85, + 0xe4, 0x52, 0xfc, 0x56, 0x66, 0xe5, 0xc5, 0x61, 0x43, 0x4b, 0xd4, 0x80, 0x8f, 0x0f, 0x58, 0x2e, + 0xc1, 0x46, 0x75, 0xc8, 0xf1, 0x9a, 0x59, 0x58, 0xb1, 0x4d, 0xa3, 0x30, 0xb3, 0x24, 0xdc, 0xca, + 0xaf, 0x2c, 0x87, 0x89, 0xe9, 0xe9, 0x05, 0x92, 0xec, 0xb6, 0xb1, 0x44, 0xb9, 0xa4, 0xac, 0x15, + 0x48, 0x15, 0xdf, 0x85, 0x6c, 0x30, 0x17, 0x21, 0xc8, 0x4b, 0x95, 0xfa, 0xce, 0x46, 0x45, 0xde, + 0xd9, 0x5c, 0xdf, 0xdc, 0x7a, 0x67, 0x53, 0x9c, 0x42, 0xe7, 0x40, 0xe4, 0xb4, 0xf5, 0xca, 0xbb, + 0xf2, 0xa3, 0xda, 0x46, 0xad, 0x21, 0x0a, 0xe8, 0x12, 0x9c, 0xe7, 0x54, 0x69, 0x75, 0xf3, 0x41, + 0x45, 0x2e, 0x6d, 0xed, 0x6c, 0xae, 0xad, 0x4a, 0xef, 0x8a, 0xb1, 0x85, 0xc4, 0x6f, 0x7c, 0xe7, + 0xf2, 0x54, 0xf1, 0x31, 0xc0, 0x03, 0xec, 0x70, 0xb5, 0x42, 0x25, 0x48, 0xee, 0xd3, 0xda, 0x70, + 0xc5, 0x5e, 0x0a, 0xad, 0x76, 0x40, 0x05, 0x4b, 0x29, 0xd2, 0x03, 0x3f, 0x3d, 0x5e, 0x14, 0x24, + 0xce, 0xc9, 0x86, 0xbc, 0xf8, 0x23, 0x01, 0x32, 0x54, 0x30, 0x6b, 0x23, 0x2a, 0xf7, 0x49, 0xbe, + 0x32, 0xb6, 0x43, 0x06, 0x45, 0xa3, 0x65, 0x98, 0x7e, 0xa6, 0xb4, 0xba, 0xa3, 0xe6, 0xcd, 0x63, + 0x92, 0x2f, 0xb1, 0x62, 0xe8, 0x4d, 0xc8, 0xea, 0x86, 0x83, 0x0d, 0x47, 0x66, 0x6c, 0xf1, 0x31, + 0x6c, 0x19, 0x56, 0x9a, 0x26, 0x8a, 0x7f, 0x2e, 0x00, 0x6c, 0x77, 0xa3, 0xec, 0x1a, 0x32, 0xef, + 0x27, 0xaa, 0xbf, 0x3b, 0xef, 0x59, 0x2b, 0x2e, 0x40, 0x52, 0x37, 0x5a, 0xba, 0xc1, 0xea, 0x9f, + 0x92, 0x78, 0x0a, 0x9d, 0x83, 0xe9, 0xdd, 0x96, 0x6e, 0x68, 0x54, 0xfd, 0x53, 0x12, 0x4b, 0xf0, + 0xee, 0x97, 0x20, 0x43, 0xeb, 0x1e, 0x61, 0xef, 0x17, 0xbf, 0x19, 0x83, 0xf3, 0x65, 0xd3, 0xd0, + 0x74, 0x32, 0x0f, 0x95, 0xd6, 0xc7, 0xa2, 0x6f, 0x5e, 0x87, 0x34, 0x3e, 0xec, 0x4c, 0x38, 0xbc, + 0x29, 0x7c, 0xd8, 0xa1, 0xbf, 0xc2, 0xbb, 0x0e, 0x7d, 0x12, 0x2e, 0x2a, 0xad, 0x96, 0x79, 0x20, + 0xeb, 0x7b, 0xb2, 0x66, 0x62, 0x5b, 0x36, 0x4c, 0x47, 0xc6, 0x87, 0xba, 0xed, 0x50, 0x53, 0x91, + 0x92, 0xe6, 0x69, 0x76, 0x6d, 0x6f, 0xcd, 0xc4, 0xf6, 0xa6, 0xe9, 0x54, 0x48, 0x16, 0xef, 0xf0, + 0xf7, 0xe1, 0x42, 0x7f, 0xdf, 0x44, 0xd9, 0xf7, 0x7f, 0x23, 0x40, 0xbe, 0x66, 0xe8, 0xce, 0xc7, + 0xa2, 0xd3, 0xbd, 0xde, 0x8b, 0x07, 0x7b, 0xef, 0x36, 0x88, 0x7b, 0x8a, 0xde, 0xda, 0x32, 0x1a, + 0x66, 0x7b, 0xd7, 0x76, 0x4c, 0x03, 0xdb, 0xbc, 0x7b, 0x07, 0xe8, 0xbc, 0xcf, 0x1e, 0xc3, 0xac, + 0xd7, 0xa6, 0x28, 0x3b, 0xeb, 0x39, 0x88, 0x35, 0x43, 0xb5, 0x70, 0x1b, 0x1b, 0x91, 0xf6, 0xd6, + 0x8b, 0x90, 0xd6, 0x5d, 0xb9, 0xb4, 0xc7, 0xe2, 0x92, 0x4f, 0xe0, 0x6d, 0xea, 0xc2, 0x5c, 0xe0, + 0xdb, 0x51, 0x1a, 0xbf, 0x17, 0x20, 0x6d, 0xe0, 0x03, 0xd9, 0x1f, 0xaf, 0xb8, 0x94, 0x32, 0xf0, + 0x01, 0x33, 0x56, 0xef, 0x42, 0x6e, 0x0d, 0xb7, 0xb0, 0x83, 0xa3, 0xb7, 0xe4, 0x3b, 0x90, 0x77, + 0x45, 0x47, 0x39, 0x48, 0xbf, 0x2f, 0x00, 0xe2, 0x72, 0xc9, 0xea, 0x19, 0xe5, 0x38, 0x2d, 0x12, + 0xef, 0xc0, 0xe9, 0x5a, 0x06, 0x5b, 0xe6, 0x99, 0x96, 0x02, 0x23, 0xd1, 0x95, 0xde, 0xb7, 0xa8, + 0x89, 0xa0, 0x45, 0xf5, 0xbc, 0x15, 0xe2, 0xa7, 0x1c, 0xc0, 0x7c, 0x4f, 0xf5, 0xa2, 0x1d, 0xca, + 0x04, 0xad, 0x59, 0x6c, 0x29, 0x1e, 0xf4, 0xcc, 0x28, 0xb1, 0xf8, 0x3e, 0xcc, 0x95, 0x5b, 0x58, + 0xb1, 0xa2, 0xee, 0x16, 0x3e, 0x9c, 0xef, 0x02, 0x0a, 0x8a, 0x8f, 0x72, 0x48, 0x75, 0xc8, 0xd4, + 0x55, 0xc5, 0xd8, 0xea, 0x10, 0x23, 0x68, 0xa3, 0x7b, 0x70, 0xc1, 0x76, 0xcc, 0x8e, 0xac, 0x38, + 0x32, 0xf3, 0xab, 0x76, 0xcd, 0xae, 0xa1, 0x29, 0xd6, 0x11, 0xfd, 0x46, 0x4a, 0x9a, 0x27, 0xb9, + 0xab, 0x0e, 0xad, 0x48, 0x89, 0x67, 0x91, 0xb1, 0x6b, 0xeb, 0x86, 0x4c, 0xdc, 0x9f, 0x96, 0x63, + 0x73, 0x3d, 0x87, 0xb6, 0x6e, 0x48, 0x8c, 0xc2, 0x5b, 0xf1, 0x1d, 0x81, 0x7d, 0x2b, 0x4a, 0xb5, + 0x79, 0x0b, 0x32, 0xb6, 0xaa, 0x18, 0xf2, 0x9e, 0x69, 0xb5, 0x15, 0x87, 0xaa, 0x46, 0x7e, 0xe5, + 0xa5, 0x30, 0xa7, 0x52, 0x55, 0x8c, 0xfb, 0xb4, 0x90, 0x04, 0xb6, 0xf7, 0x3b, 0xa8, 0x3d, 0x0f, + 0x13, 0xa9, 0xb8, 0x98, 0x28, 0xfe, 0xbb, 0x00, 0x59, 0x56, 0xcb, 0x28, 0xb5, 0xe7, 0x75, 0x48, + 0x58, 0xe6, 0x01, 0xd3, 0x9e, 0xcc, 0xca, 0x0b, 0x21, 0x22, 0xd6, 0xf1, 0x51, 0xd0, 0x6c, 0xd3, + 0xe2, 0xa8, 0x04, 0xdc, 0xbd, 0x91, 0x29, 0x77, 0x7c, 0x52, 0x6e, 0x60, 0x5c, 0x12, 0x91, 0x71, + 0x13, 0x66, 0x77, 0x15, 0x47, 0xdd, 0x27, 0xe3, 0x43, 0x2b, 0x49, 0x4c, 0x7c, 0xfc, 0x56, 0x56, + 0xca, 0x53, 0xb2, 0x5b, 0x75, 0xbb, 0xf8, 0x3d, 0x01, 0x90, 0x84, 0x9f, 0x61, 0xcb, 0xc6, 0x1f, + 0xff, 0x61, 0xfa, 0x0f, 0x01, 0xe6, 0x7b, 0x2a, 0xfb, 0xbf, 0x6d, 0xb4, 0x7e, 0x26, 0xc0, 0xc5, + 0xf2, 0x3e, 0x56, 0x9f, 0x94, 0x4d, 0xc3, 0xd6, 0x6d, 0x07, 0x1b, 0xea, 0x51, 0x94, 0x43, 0xf6, + 0x02, 0xa4, 0x0f, 0x74, 0x67, 0x5f, 0xd6, 0xf4, 0xbd, 0x3d, 0x3a, 0xa5, 0x53, 0x52, 0x8a, 0x10, + 0xd6, 0xf4, 0xbd, 0x3d, 0x74, 0x0f, 0x12, 0x6d, 0x53, 0x63, 0xde, 0x5b, 0x7e, 0x65, 0x31, 0x44, + 0x3c, 0xad, 0x9a, 0xdd, 0x6d, 0x6f, 0x98, 0x1a, 0x96, 0x68, 0x61, 0x74, 0x19, 0x40, 0x25, 0xd4, + 0x8e, 0xa9, 0x1b, 0x0e, 0xb7, 0xe2, 0x01, 0x0a, 0xb7, 0x12, 0xdf, 0x48, 0x40, 0x61, 0xb0, 0x5d, + 0x51, 0x8e, 0xee, 0x36, 0x24, 0x99, 0xa9, 0xe2, 0xe3, 0xbb, 0x32, 0xac, 0xfa, 0x21, 0x35, 0x58, + 0x66, 0x26, 0x8d, 0x0f, 0x1c, 0x97, 0xb3, 0xf0, 0x17, 0x02, 0x24, 0x59, 0x06, 0xba, 0x0b, 0x29, + 0x0e, 0x48, 0x35, 0x5a, 0xc7, 0x78, 0xe9, 0xc2, 0xc9, 0xf1, 0xe2, 0x0c, 0x83, 0x9f, 0x6b, 0x1f, + 0xfa, 0x3f, 0xa5, 0x19, 0x86, 0x40, 0x35, 0xd2, 0xd3, 0xb6, 0xa3, 0x58, 0x0e, 0x05, 0xfe, 0xa4, + 0xa7, 0xb3, 0x52, 0x8a, 0x12, 0x08, 0xe2, 0x7f, 0x08, 0x49, 0xdb, 0x51, 0x9c, 0xae, 0xcd, 0xfb, + 0xfa, 0x54, 0x95, 0xad, 0x53, 0x4e, 0x89, 0x4b, 0x20, 0x4b, 0xa8, 0x86, 0x1d, 0x45, 0x6f, 0xd1, + 0xce, 0x4f, 0x4b, 0x3c, 0x55, 0xfc, 0x96, 0x00, 0x49, 0x56, 0x14, 0x5d, 0x84, 0x79, 0x06, 0x3a, + 0x6b, 0x9b, 0x6b, 0x95, 0x46, 0x45, 0xda, 0xa8, 0x6d, 0xae, 0x36, 0x2a, 0xe2, 0x14, 0xba, 0x00, + 0xc8, 0xcd, 0x28, 0x6f, 0x6d, 0xd6, 0x6b, 0xf5, 0x46, 0x65, 0x93, 0x80, 0x55, 0x02, 0x61, 0x29, + 0x3d, 0x40, 0x8d, 0xa1, 0x6b, 0xb0, 0xd4, 0x4f, 0x95, 0xeb, 0x8d, 0xd5, 0x46, 0x5d, 0xae, 0xd4, + 0x1b, 0xb5, 0x8d, 0xd5, 0x46, 0x65, 0x4d, 0x8c, 0x8f, 0x28, 0x45, 0x3e, 0x22, 0x49, 0x95, 0x72, + 0x43, 0x4c, 0x14, 0x9f, 0xc3, 0x79, 0x09, 0xab, 0x66, 0xbb, 0xd3, 0x75, 0x30, 0xa9, 0xa5, 0x1d, + 0xa5, 0x96, 0x5f, 0x84, 0x19, 0xcd, 0x3a, 0x92, 0xad, 0xae, 0xc1, 0x75, 0x3c, 0xa9, 0x59, 0x47, + 0x52, 0xd7, 0xe0, 0xca, 0xf8, 0x67, 0x02, 0x5c, 0xe8, 0xff, 0x78, 0x94, 0xaa, 0xf8, 0x45, 0xc8, + 0x28, 0x9a, 0x86, 0x35, 0x59, 0xc3, 0x2d, 0x47, 0xe1, 0x1e, 0xfd, 0xdd, 0x80, 0x24, 0xbe, 0x69, + 0xb3, 0xcc, 0x76, 0x6b, 0x96, 0xbd, 0x4d, 0x9b, 0x8d, 0xc7, 0xe5, 0x32, 0xad, 0xcf, 0x1a, 0x61, + 0x74, 0xed, 0x08, 0x95, 0x45, 0x29, 0x45, 0x15, 0x2e, 0x96, 0x70, 0x53, 0x37, 0x82, 0xdb, 0x30, + 0x91, 0xfb, 0x25, 0x32, 0x14, 0x06, 0x3f, 0x12, 0xa5, 0x77, 0xf2, 0xdd, 0x04, 0x9c, 0xaf, 0x18, + 0xda, 0x47, 0xd3, 0x08, 0x32, 0x1f, 0x54, 0xb3, 0xdd, 0xd6, 0x1d, 0x77, 0xec, 0x59, 0x0a, 0xfd, + 0x1f, 0x48, 0x69, 0x58, 0xd1, 0x3c, 0xf8, 0x9e, 0xe9, 0x59, 0xaa, 0xba, 0x8e, 0xde, 0x5a, 0xde, + 0x6f, 0xa9, 0xcb, 0x0d, 0x77, 0x93, 0x52, 0xf2, 0x8a, 0xa3, 0x2f, 0xc3, 0x45, 0x62, 0xcc, 0x2d, + 0x43, 0x69, 0xc9, 0x4c, 0x9a, 0xec, 0x58, 0x7a, 0xb3, 0x89, 0x2d, 0xbe, 0xe1, 0x75, 0x2b, 0xa4, + 0x9e, 0x35, 0xce, 0x51, 0xa6, 0x0c, 0x0d, 0x56, 0x5e, 0x3a, 0xaf, 0x87, 0x91, 0xd1, 0x17, 0xbc, + 0xfd, 0x11, 0xbb, 0xa3, 0x18, 0x76, 0x61, 0x9a, 0xda, 0xb0, 0x61, 0xfb, 0x68, 0x5c, 0x33, 0xf8, + 0xba, 0x44, 0x28, 0x36, 0xba, 0x43, 0x5c, 0xed, 0xa7, 0x5d, 0xdd, 0xc2, 0xf2, 0xdd, 0x8e, 0x5a, + 0x48, 0x92, 0xb6, 0x97, 0xf2, 0x27, 0xc7, 0x8b, 0x20, 0x31, 0xf2, 0xdd, 0xed, 0x32, 0x71, 0xbd, + 0xd9, 0xef, 0x8e, 0x8a, 0x6e, 0x81, 0x68, 0x98, 0xb2, 0x85, 0xf7, 0x2c, 0x6c, 0xef, 0xf3, 0xcf, + 0xa6, 0x68, 0x8f, 0xe5, 0x0d, 0x53, 0x62, 0x64, 0x26, 0xfa, 0x02, 0x24, 0x3b, 0xa6, 0x6e, 0x9b, + 0x46, 0x21, 0xcd, 0x7a, 0x94, 0xa5, 0xd0, 0xdb, 0x20, 0xea, 0x86, 0xbc, 0xd7, 0xd2, 0x9b, 0xfb, + 0x8e, 0x7c, 0x60, 0xe9, 0x0e, 0xb6, 0x0b, 0x73, 0xb4, 0xe2, 0x61, 0x6a, 0x51, 0xe7, 0x3b, 0x91, + 0xda, 0x3b, 0xa4, 0x24, 0x6f, 0x42, 0x5e, 0x37, 0xee, 0x53, 0x7e, 0x4a, 0xb4, 0x3d, 0x97, 0x60, + 0x46, 0x4c, 0x15, 0xff, 0x45, 0x80, 0x0b, 0xfd, 0x6a, 0x12, 0xe5, 0x34, 0xbd, 0x05, 0xa2, 0x69, + 0x60, 0xb9, 0xb3, 0xaf, 0xd8, 0x98, 0x0f, 0x2b, 0x5f, 0xbf, 0xf2, 0xa6, 0x81, 0xb7, 0x09, 0x99, + 0x0d, 0x12, 0xda, 0x86, 0x39, 0xdb, 0x51, 0x9a, 0xba, 0xd1, 0x94, 0xbd, 0x3d, 0x6c, 0xba, 0x11, + 0x31, 0x4e, 0x87, 0x78, 0x2b, 0x45, 0xce, 0xed, 0xd1, 0x7b, 0x9c, 0x9e, 0x6f, 0x09, 0x30, 0xb7, + 0xaa, 0xb5, 0x75, 0xa3, 0xde, 0x69, 0xe9, 0x91, 0xc2, 0xe4, 0x6b, 0x90, 0xb6, 0x89, 0x4c, 0x7f, + 0x0d, 0xf2, 0x21, 0x4e, 0x8a, 0xe6, 0x90, 0xc5, 0xe8, 0x02, 0x24, 0xdb, 0x8a, 0xd1, 0x55, 0x5a, + 0xee, 0xae, 0x16, 0x4b, 0xf9, 0x28, 0x25, 0x58, 0xb9, 0x28, 0xed, 0x80, 0x0c, 0xf3, 0x54, 0xf4, + 0x8e, 0x61, 0x47, 0xdc, 0x72, 0x5e, 0xf7, 0x2f, 0xc1, 0xb9, 0xde, 0x0f, 0x44, 0x59, 0xfb, 0xf7, + 0xf9, 0xa8, 0x6d, 0x60, 0xeb, 0x23, 0x42, 0x87, 0x41, 0xf1, 0x51, 0xd6, 0xfc, 0x6b, 0x02, 0x5c, + 0xa2, 0xb2, 0xe9, 0xd4, 0xda, 0xc3, 0x16, 0x3d, 0xe2, 0x88, 0x52, 0xf1, 0xae, 0x42, 0xd2, 0x51, + 0xac, 0x26, 0x66, 0x36, 0x78, 0xba, 0x94, 0x21, 0x2e, 0x52, 0xdd, 0x31, 0x2d, 0xe2, 0x22, 0xf1, + 0x2c, 0xde, 0x4e, 0x05, 0x16, 0xc2, 0xea, 0x12, 0x65, 0x7b, 0xff, 0x34, 0xc6, 0xbf, 0x51, 0xde, + 0x67, 0x50, 0xbb, 0xd3, 0xd2, 0x55, 0x25, 0x52, 0x8f, 0xa3, 0x02, 0x19, 0x95, 0x0a, 0x97, 0x9d, + 0xa3, 0x0e, 0xdb, 0x14, 0xca, 0xaf, 0x5c, 0x0b, 0x15, 0x44, 0x3f, 0xce, 0x6a, 0xd2, 0x38, 0xea, + 0x60, 0xe2, 0x2c, 0xbb, 0xbf, 0xd1, 0x1a, 0xcc, 0xb0, 0xce, 0x71, 0x71, 0xc6, 0x08, 0x11, 0xc4, + 0x20, 0x36, 0x68, 0x61, 0x6e, 0x65, 0x5c, 0x56, 0xf4, 0x39, 0x48, 0xe1, 0xc3, 0x8e, 0x4c, 0x8f, + 0xc5, 0x12, 0x93, 0x1e, 0x8b, 0x49, 0x33, 0xf8, 0xb0, 0x43, 0x92, 0x7c, 0x5c, 0xbe, 0x29, 0xc0, + 0x0b, 0xa1, 0x9d, 0x16, 0xa5, 0x09, 0xfe, 0x14, 0x3f, 0xbc, 0x8b, 0x4d, 0x5c, 0x4b, 0x5a, 0xbe, + 0xf8, 0x47, 0xae, 0x06, 0x4b, 0xb8, 0x65, 0xaa, 0xca, 0x47, 0xb0, 0x73, 0x15, 0x18, 0x89, 0xd8, + 0x99, 0x47, 0xa2, 0x4f, 0xc5, 0xfb, 0x2a, 0x1b, 0xa5, 0x8a, 0xff, 0x8e, 0x00, 0xf3, 0x55, 0xac, + 0x58, 0xce, 0x2e, 0x56, 0x9c, 0xc6, 0x61, 0xa4, 0x0e, 0xd5, 0xeb, 0x10, 0x37, 0xcc, 0x03, 0x3e, + 0x46, 0x13, 0xad, 0x77, 0xa4, 0xbc, 0x6f, 0x82, 0x7b, 0xeb, 0x15, 0x65, 0xab, 0x7f, 0x10, 0x87, + 0xf4, 0x83, 0x72, 0x94, 0x6d, 0xfd, 0x2c, 0xdf, 0x0f, 0x64, 0xb3, 0x2f, 0x4c, 0x21, 0xbd, 0xef, + 0x2d, 0x3f, 0x28, 0xaf, 0xe3, 0x23, 0x77, 0xab, 0x80, 0x70, 0xa1, 0x55, 0x48, 0x3b, 0xfb, 0xc4, + 0x6f, 0x32, 0x5b, 0x1a, 0x9f, 0x79, 0x13, 0xf5, 0x97, 0xcf, 0x85, 0x5a, 0x70, 0xde, 0x39, 0x34, + 0xa8, 0x3b, 0x26, 0x37, 0x55, 0xd9, 0x17, 0x37, 0x91, 0xbb, 0xb1, 0x40, 0xc4, 0x9d, 0x1c, 0x2f, + 0xa2, 0xc6, 0xa1, 0x41, 0x7c, 0xb7, 0x07, 0xe5, 0x86, 0x2b, 0x40, 0x42, 0x0e, 0xa7, 0xa9, 0x1e, + 0x6d, 0xe1, 0x09, 0x4c, 0xd3, 0x56, 0xb8, 0xe7, 0xd7, 0x42, 0xc8, 0xf9, 0x35, 0x69, 0x94, 0xe7, + 0xf4, 0xc4, 0x4e, 0xd3, 0x28, 0x97, 0xc0, 0x54, 0x81, 0x2b, 0xc4, 0xdb, 0x00, 0xa4, 0x0b, 0xa3, + 0x54, 0x83, 0xff, 0x8c, 0x43, 0x7e, 0xbb, 0x6b, 0xef, 0x47, 0xac, 0xf7, 0x65, 0x80, 0x4e, 0xd7, + 0xde, 0xc7, 0x96, 0xec, 0x1c, 0x1a, 0xbc, 0xe5, 0x63, 0xce, 0xc4, 0xdd, 0xa6, 0x33, 0xbe, 0xc6, + 0xa1, 0x81, 0xb6, 0xb8, 0x10, 0x2c, 0xfb, 0x07, 0xeb, 0xb7, 0x27, 0x80, 0x82, 0x8d, 0x43, 0x63, + 0x03, 0x7b, 0x18, 0x90, 0x09, 0xc4, 0x44, 0xe0, 0x67, 0x61, 0x86, 0x24, 0x64, 0xc7, 0x3c, 0x8d, + 0x86, 0x25, 0x09, 0x4f, 0xc3, 0x44, 0x0f, 0x21, 0xaf, 0xe1, 0x8e, 0x85, 0x89, 0x19, 0xd2, 0x64, + 0x32, 0xad, 0x4f, 0xe1, 0xc6, 0xe6, 0x7c, 0xd6, 0x4d, 0xf3, 0x00, 0xbd, 0x09, 0x69, 0x56, 0x13, + 0xb2, 0xe2, 0x25, 0xe9, 0x8a, 0x17, 0xd6, 0x3d, 0x7c, 0x64, 0xe8, 0x5a, 0x97, 0xa2, 0xd5, 0x20, + 0x2b, 0xdd, 0x39, 0x98, 0xde, 0x33, 0x2d, 0x15, 0xd3, 0x53, 0xf9, 0x94, 0xc4, 0x12, 0xe8, 0x36, + 0xcc, 0xe9, 0x86, 0xda, 0xea, 0xda, 0xfa, 0x33, 0x2c, 0xbb, 0xcd, 0x64, 0xa0, 0x63, 0xd6, 0xcb, + 0xa0, 0x02, 0x4d, 0x0f, 0x2a, 0xa4, 0xc4, 0x34, 0x71, 0xa1, 0x67, 0x3d, 0x0d, 0x88, 0x72, 0x81, + 0x2a, 0xf7, 0x0c, 0xdf, 0xe9, 0x75, 0x80, 0x0c, 0x59, 0xf1, 0x6f, 0x05, 0x98, 0x93, 0xb0, 0x6a, + 0x3e, 0xa3, 0x2a, 0x11, 0xa5, 0x8a, 0x96, 0x58, 0xbc, 0x46, 0xec, 0x8c, 0x6a, 0x45, 0x23, 0x38, + 0xee, 0xc2, 0x39, 0xbd, 0x4d, 0xd6, 0x31, 0xdd, 0x69, 0x1d, 0x71, 0x1c, 0xe4, 0x60, 0xf7, 0x48, + 0x71, 0xde, 0xcf, 0x2b, 0xbb, 0x59, 0x7c, 0x26, 0x7f, 0x97, 0xee, 0x2c, 0xfb, 0xcd, 0x8a, 0xb2, + 0xdf, 0x6b, 0x90, 0xb3, 0x98, 0x68, 0xac, 0x9d, 0xba, 0xeb, 0xb3, 0x1e, 0x2b, 0xe9, 0xfd, 0xef, + 0xc7, 0x60, 0xf6, 0xed, 0x2e, 0xb6, 0x8e, 0x3e, 0x86, 0x7d, 0x7f, 0x03, 0x66, 0x0f, 0x14, 0xdd, + 0x91, 0xf7, 0x4c, 0x4b, 0xee, 0x76, 0x34, 0xc5, 0x71, 0x23, 0x0b, 0x72, 0x84, 0x7c, 0xdf, 0xb4, + 0x76, 0x28, 0x11, 0x61, 0x40, 0x4f, 0x0c, 0xf3, 0xc0, 0x90, 0x09, 0x99, 0xc2, 0xd0, 0x43, 0x83, + 0x6f, 0x21, 0x97, 0x3e, 0xfd, 0xf7, 0xc7, 0x8b, 0xf7, 0x26, 0x8a, 0x15, 0xa2, 0xe1, 0x56, 0xdd, + 0xae, 0xae, 0x2d, 0xef, 0xec, 0xd4, 0xd6, 0x24, 0x91, 0x8a, 0x7c, 0x87, 0x49, 0x6c, 0x1c, 0x1a, + 0xae, 0xbb, 0xf2, 0xa1, 0x00, 0xa2, 0xdf, 0x61, 0x51, 0x8e, 0x6a, 0x05, 0x32, 0x4f, 0xbb, 0xd8, + 0xd2, 0xcf, 0x30, 0xa6, 0xc0, 0x19, 0x89, 0x09, 0x7c, 0x0f, 0xb2, 0x3d, 0xfd, 0x10, 0xff, 0xd5, + 0xfa, 0x21, 0x73, 0xe0, 0x77, 0x41, 0xf1, 0x27, 0x02, 0x20, 0xda, 0xf8, 0x1a, 0xdb, 0xbd, 0xff, + 0x98, 0x29, 0xcc, 0x2d, 0x10, 0x69, 0xf4, 0x9e, 0xac, 0xef, 0xc9, 0x6d, 0xdd, 0xb6, 0x75, 0xa3, + 0xc9, 0x35, 0x26, 0x4f, 0xe9, 0xb5, 0xbd, 0x0d, 0x46, 0xe5, 0x63, 0xf9, 0xff, 0x61, 0xbe, 0xa7, + 0x35, 0x51, 0x8e, 0xe6, 0x15, 0xc8, 0xee, 0x99, 0x5d, 0x43, 0x93, 0xd9, 0x36, 0x14, 0xdf, 0x6e, + 0xcb, 0x50, 0x1a, 0xfb, 0x5e, 0xf1, 0xab, 0x31, 0x38, 0x27, 0x61, 0xdb, 0x6c, 0x3d, 0xc3, 0xd1, + 0xf7, 0xe7, 0x16, 0xf0, 0x23, 0x16, 0xf9, 0x57, 0xe9, 0xd6, 0x34, 0x93, 0xc1, 0x96, 0xd6, 0xde, + 0x5d, 0xf9, 0x6b, 0xa3, 0x35, 0x73, 0x70, 0x1f, 0x9e, 0xef, 0x92, 0x25, 0x82, 0xbb, 0x64, 0x7c, + 0x20, 0xfe, 0x2f, 0x9c, 0xef, 0xeb, 0x88, 0x28, 0x3d, 0xa0, 0xbf, 0x8b, 0xc1, 0xa5, 0x5e, 0xf1, + 0x51, 0xe3, 0xa1, 0xff, 0x19, 0x9d, 0x8d, 0xaa, 0x90, 0x6b, 0xeb, 0xc6, 0xd9, 0x76, 0xe9, 0xb2, + 0x6d, 0xdd, 0x68, 0xf4, 0xfa, 0xac, 0x04, 0xba, 0x85, 0xf5, 0x6b, 0x94, 0x63, 0xf7, 0x0d, 0x01, + 0xb2, 0x51, 0xef, 0x21, 0x9d, 0x2d, 0x9c, 0x88, 0xb7, 0xb9, 0x01, 0xb9, 0x8f, 0x60, 0xd3, 0xe9, + 0x0f, 0x05, 0x40, 0x0d, 0xab, 0x6b, 0x10, 0xf7, 0xf1, 0x91, 0xd9, 0x8c, 0xb2, 0xb1, 0xe7, 0x60, + 0x5a, 0x37, 0x34, 0x7c, 0x48, 0x1b, 0x9b, 0x90, 0x58, 0xa2, 0xe7, 0xcc, 0x2e, 0x3e, 0xd1, 0x99, + 0x1d, 0x6f, 0xff, 0x7b, 0x30, 0xdf, 0x53, 0xd1, 0x28, 0x7b, 0xe1, 0xfb, 0x31, 0x98, 0xe7, 0xcd, + 0x89, 0x7c, 0xd3, 0xed, 0x4c, 0xb1, 0xcc, 0xe8, 0x73, 0x00, 0x1d, 0x0b, 0x3f, 0x93, 0x19, 0x6b, + 0x7c, 0x22, 0xd6, 0x34, 0xe1, 0xa0, 0x04, 0xf4, 0x45, 0x98, 0x25, 0x13, 0xae, 0x63, 0x99, 0x1d, + 0xd3, 0x26, 0xeb, 0xba, 0x3d, 0x19, 0x2c, 0x99, 0x3b, 0x39, 0x5e, 0xcc, 0x6d, 0xe8, 0xc6, 0x36, + 0x67, 0x6c, 0xd4, 0x25, 0x32, 0x73, 0xbd, 0xa4, 0xeb, 0x8c, 0xfc, 0x4c, 0x80, 0x73, 0x1f, 0xd9, + 0x36, 0xe5, 0x7f, 0x47, 0x8f, 0x79, 0xeb, 0x81, 0x48, 0x93, 0x35, 0x63, 0xcf, 0x8c, 0x7e, 0xf3, + 0xf8, 0x1b, 0x02, 0xcc, 0x05, 0xc4, 0x47, 0xb9, 0xea, 0x9f, 0x2d, 0x62, 0xfe, 0x4b, 0xc4, 0x0f, + 0x08, 0xaa, 0x7d, 0x94, 0x93, 0xea, 0xdf, 0x04, 0xb8, 0x50, 0x66, 0xa7, 0xb9, 0x6e, 0x80, 0x42, + 0x94, 0x5a, 0x52, 0x80, 0x99, 0x67, 0xd8, 0xb2, 0x75, 0x93, 0xad, 0x7b, 0x39, 0xc9, 0x4d, 0xa2, + 0x05, 0x48, 0xd9, 0x86, 0xd2, 0xb1, 0xf7, 0x4d, 0xf7, 0xe4, 0xc8, 0x4b, 0x7b, 0xc1, 0x14, 0xd3, + 0x67, 0x0f, 0xa6, 0x48, 0x86, 0x07, 0x53, 0xf0, 0x63, 0xa3, 0x1f, 0x0b, 0x70, 0x71, 0xa0, 0xd5, + 0x51, 0x8e, 0xf4, 0x57, 0x20, 0xa3, 0x72, 0xc1, 0xc4, 0x86, 0xb2, 0xf3, 0xa3, 0x1a, 0x29, 0x76, + 0x46, 0x4f, 0xfb, 0xe4, 0x78, 0x11, 0xdc, 0xaa, 0xd6, 0xd6, 0x78, 0xc3, 0xc8, 0x6f, 0xad, 0xf8, + 0x0b, 0x80, 0x5c, 0xe5, 0xb0, 0x63, 0x5a, 0x4e, 0x9d, 0x39, 0x12, 0x68, 0x0d, 0x52, 0x1d, 0xcb, + 0x7c, 0xa6, 0xbb, 0x8d, 0xc8, 0x87, 0x1e, 0xb2, 0xf6, 0xf0, 0x6c, 0xf3, 0xf2, 0x92, 0xc7, 0x89, + 0x24, 0x48, 0x3f, 0x32, 0x55, 0xa5, 0x75, 0x5f, 0x6f, 0xb9, 0x1a, 0xbb, 0x3c, 0x4e, 0xcc, 0xb2, + 0xc7, 0xb1, 0xad, 0x38, 0xfb, 0xee, 0x04, 0xf6, 0x88, 0xe8, 0x01, 0xa4, 0xaa, 0x8e, 0xd3, 0x21, + 0x99, 0x7c, 0xf6, 0x5f, 0x1f, 0x2b, 0x92, 0x30, 0x70, 0x49, 0x1e, 0x33, 0x92, 0x60, 0xee, 0x81, + 0x69, 0x36, 0x5b, 0xb8, 0xdc, 0x32, 0xbb, 0x5a, 0xd9, 0x34, 0xf6, 0xf4, 0x26, 0xb7, 0x9e, 0xd7, + 0xc6, 0x4a, 0x7c, 0x50, 0xae, 0x4b, 0x83, 0xec, 0xe8, 0xf3, 0x90, 0xaa, 0xdf, 0xe3, 0xa2, 0x98, + 0xef, 0x73, 0x75, 0xac, 0xa8, 0xfa, 0x3d, 0xc9, 0x63, 0x42, 0x55, 0xc8, 0xac, 0x3e, 0xef, 0x5a, + 0x98, 0xcb, 0x48, 0x52, 0x19, 0x37, 0xc6, 0xca, 0xa0, 0x3c, 0x52, 0x90, 0x15, 0xbd, 0x0d, 0xf9, + 0x77, 0x4c, 0xeb, 0x49, 0xcb, 0x54, 0xdc, 0xb6, 0xcd, 0x50, 0x61, 0x2f, 0x8f, 0x15, 0xe6, 0xb2, + 0x49, 0x7d, 0x02, 0x16, 0xbe, 0x08, 0xb9, 0x9e, 0xc1, 0x41, 0x08, 0x12, 0x1d, 0x32, 0x0e, 0x02, + 0x0d, 0x7d, 0xa1, 0xbf, 0xd1, 0x6b, 0x30, 0x63, 0x98, 0x1a, 0x76, 0x75, 0x36, 0x57, 0x3a, 0x77, + 0x72, 0xbc, 0x98, 0xdc, 0x34, 0x35, 0xb6, 0xec, 0xf3, 0x5f, 0x52, 0x92, 0x14, 0x72, 0x17, 0xfd, + 0x85, 0x1b, 0x90, 0x20, 0xe3, 0x42, 0x26, 0xfb, 0xae, 0x62, 0xe3, 0x1d, 0x4b, 0xe7, 0x32, 0xdd, + 0x24, 0x2f, 0xf7, 0xd7, 0x02, 0xc4, 0xea, 0xf7, 0x88, 0xff, 0xb9, 0xdb, 0x55, 0x9f, 0x60, 0x87, + 0x97, 0xe2, 0x29, 0xea, 0x97, 0x5a, 0x78, 0x4f, 0x67, 0xbe, 0x48, 0x5a, 0xe2, 0x29, 0xf4, 0x12, + 0x80, 0xa2, 0xaa, 0xd8, 0xb6, 0x65, 0xf7, 0x9a, 0x50, 0x5a, 0x4a, 0x33, 0x0a, 0x3f, 0x82, 0xb5, + 0xb1, 0x6a, 0x61, 0xc7, 0x8d, 0xe1, 0x61, 0x29, 0xc2, 0xe6, 0xe0, 0x76, 0x47, 0x76, 0xcc, 0x27, + 0xd8, 0xa0, 0xe3, 0x99, 0x96, 0xd2, 0x84, 0xd2, 0x20, 0x04, 0x62, 0x7f, 0xb0, 0xa1, 0xf9, 0xc6, + 0x22, 0x2d, 0x79, 0x69, 0x22, 0xd2, 0xc2, 0x4d, 0x9d, 0xdf, 0x7b, 0x49, 0x4b, 0x3c, 0xc5, 0x9b, + 0xf1, 0x4d, 0x01, 0xe2, 0x0f, 0xca, 0xf5, 0x53, 0xb7, 0x03, 0x41, 0x42, 0xe9, 0x72, 0xbd, 0x4f, + 0x4b, 0xf4, 0x37, 0x0d, 0x6e, 0xd3, 0x5b, 0x2d, 0x02, 0xc7, 0x3b, 0x96, 0xf9, 0x15, 0xac, 0xba, + 0xad, 0xc8, 0x73, 0xf2, 0x36, 0xa3, 0xa2, 0x25, 0xc8, 0xa8, 0x16, 0xd6, 0xb0, 0xe1, 0xe8, 0x4a, + 0xcb, 0xe6, 0xcd, 0x09, 0x92, 0x78, 0xe5, 0xbe, 0x2a, 0xc0, 0x34, 0x55, 0x24, 0xf4, 0x22, 0xa4, + 0x55, 0xd3, 0x70, 0x14, 0xdd, 0xe0, 0x56, 0x20, 0x2d, 0xf9, 0x84, 0xa1, 0x95, 0xbc, 0x02, 0x59, + 0x45, 0x55, 0xcd, 0xae, 0xe1, 0xc8, 0x86, 0xd2, 0xc6, 0xbc, 0xb2, 0x19, 0x4e, 0xdb, 0x54, 0xda, + 0x18, 0x2d, 0x82, 0x9b, 0xf4, 0x2e, 0x66, 0xa5, 0x25, 0xe0, 0xa4, 0x75, 0x7c, 0xc4, 0x6b, 0xf2, + 0x63, 0x01, 0x52, 0xae, 0x0a, 0x92, 0xca, 0x34, 0xb1, 0x81, 0x2d, 0xc5, 0x31, 0xbd, 0xca, 0x78, + 0x84, 0xfe, 0x55, 0x22, 0xed, 0xaf, 0x12, 0xe7, 0x60, 0xda, 0x51, 0x76, 0x5b, 0x6e, 0x3d, 0x58, + 0x82, 0x6e, 0x80, 0xb6, 0x94, 0x26, 0xdb, 0xc5, 0x49, 0x4b, 0x2c, 0x41, 0x9a, 0xc4, 0xa3, 0x29, + 0x59, 0xef, 0xf0, 0x14, 0xa9, 0x2f, 0x0b, 0x20, 0xdc, 0xc5, 0x4d, 0xdd, 0xa0, 0x83, 0x1d, 0x97, + 0x80, 0x92, 0x68, 0xac, 0x0e, 0x7a, 0x01, 0xd2, 0xac, 0x00, 0x36, 0x34, 0x3a, 0xe2, 0x71, 0x29, + 0x45, 0x09, 0x15, 0xf7, 0x26, 0x0a, 0x77, 0x0d, 0xfe, 0x58, 0x80, 0x39, 0x16, 0x81, 0xc1, 0x22, + 0x0f, 0xa3, 0x5b, 0x27, 0xdf, 0x80, 0xb4, 0xa6, 0x38, 0x0a, 0xbb, 0x08, 0x16, 0x1b, 0x79, 0x11, + 0xcc, 0x35, 0x85, 0xa4, 0x3c, 0xbd, 0x0c, 0x86, 0x20, 0x41, 0x7e, 0xb3, 0x0b, 0x74, 0x12, 0xfd, + 0xed, 0x9f, 0x83, 0x07, 0xab, 0x1b, 0xa5, 0xdf, 0xf0, 0xf3, 0xb8, 0xbb, 0xe8, 0x44, 0xd9, 0x0d, + 0x5f, 0x80, 0x19, 0x0e, 0x86, 0x79, 0x27, 0x2c, 0x8d, 0xb3, 0x77, 0xee, 0xa9, 0x21, 0x67, 0x43, + 0x25, 0x00, 0x16, 0x3a, 0x48, 0x60, 0xec, 0x44, 0xb1, 0x4a, 0xee, 0x22, 0x45, 0xd9, 0x08, 0x15, + 0x6d, 0x42, 0xa6, 0xfd, 0x4c, 0x55, 0xe5, 0x3d, 0xbd, 0xe5, 0xf0, 0x30, 0xa5, 0xf0, 0xd8, 0xdc, + 0x8d, 0xc7, 0xe5, 0xf2, 0x7d, 0x5a, 0x88, 0x45, 0x0b, 0xf9, 0x69, 0x09, 0x88, 0x04, 0xf6, 0x1b, + 0xbd, 0x0a, 0x3c, 0x6c, 0x5f, 0xb6, 0xdd, 0x4b, 0x38, 0xa5, 0xdc, 0xc9, 0xf1, 0x62, 0x5a, 0xa2, + 0xd4, 0x7a, 0xbd, 0x21, 0xa5, 0x59, 0x81, 0xba, 0xed, 0xa0, 0xab, 0x90, 0x33, 0xdb, 0xba, 0x23, + 0xbb, 0x2b, 0x3c, 0x77, 0x65, 0xb2, 0x84, 0xe8, 0x7a, 0x00, 0xa8, 0x01, 0x37, 0xb1, 0x41, 0xe6, + 0x02, 0x6d, 0x27, 0x8b, 0x49, 0x97, 0x75, 0x87, 0xcd, 0x27, 0xd9, 0xec, 0x38, 0x7a, 0x5b, 0x7f, + 0x4e, 0x0f, 0x57, 0xf9, 0x21, 0xc1, 0x55, 0x56, 0x9c, 0xb4, 0x8f, 0x46, 0xa9, 0xd7, 0x78, 0xd9, + 0xad, 0x40, 0x51, 0xae, 0x35, 0x5f, 0x17, 0x20, 0x57, 0xea, 0xb6, 0x9e, 0x6c, 0x75, 0xea, 0xdd, + 0x76, 0x5b, 0xb1, 0x8e, 0xc8, 0x04, 0x61, 0xda, 0xa9, 0x3f, 0xc7, 0x2c, 0x86, 0x93, 0xab, 0x9f, + 0xfe, 0x1c, 0x13, 0xf5, 0xe3, 0xa1, 0xc1, 0x84, 0xce, 0xe2, 0x7e, 0xaf, 0x42, 0x8e, 0x02, 0x49, + 0x19, 0x1b, 0x8e, 0xa5, 0x63, 0xb6, 0x4f, 0x11, 0x97, 0xb2, 0x94, 0x58, 0x61, 0x34, 0x74, 0x1d, + 0xf2, 0xf6, 0x91, 0xed, 0xe0, 0xb6, 0xcc, 0xae, 0xaf, 0x32, 0xf4, 0x13, 0x97, 0x72, 0x8c, 0x2a, + 0x31, 0x62, 0xf1, 0x4f, 0xe2, 0x90, 0x77, 0x35, 0x2d, 0x4a, 0x17, 0xad, 0x04, 0xd3, 0x7b, 0x7a, + 0x0b, 0xbb, 0x67, 0xd4, 0xc3, 0x97, 0x69, 0x2f, 0x78, 0x94, 0xac, 0x9c, 0xae, 0x6b, 0x4e, 0x59, + 0xa3, 0xd0, 0xb6, 0x85, 0x9f, 0x0a, 0x90, 0xa0, 0xbe, 0xd1, 0x5d, 0x48, 0xd0, 0xe9, 0x2f, 0x4c, + 0x32, 0xfd, 0x69, 0x51, 0x6f, 0x09, 0x8f, 0x05, 0x96, 0x70, 0xb2, 0x1e, 0xee, 0x2b, 0xaf, 0xdf, + 0x5d, 0xa1, 0x9a, 0x96, 0x95, 0x78, 0x0a, 0x95, 0x68, 0x64, 0x83, 0x69, 0x39, 0x58, 0xe3, 0x9e, + 0x49, 0xd8, 0xe4, 0xea, 0x19, 0x78, 0xd7, 0xd4, 0xb8, 0x7c, 0xe8, 0x12, 0xc4, 0x89, 0x0a, 0xcf, + 0xb0, 0xa3, 0xce, 0x93, 0xe3, 0xc5, 0x38, 0x51, 0x5e, 0x42, 0x63, 0x8e, 0xf5, 0xc3, 0x44, 0x2a, + 0x21, 0x4e, 0x17, 0x7f, 0x98, 0x80, 0x5c, 0xad, 0x1d, 0xb5, 0x71, 0x58, 0xed, 0x1d, 0xb0, 0x30, + 0xc7, 0xb1, 0xe7, 0xa3, 0x21, 0xe3, 0xd5, 0x63, 0x66, 0xe3, 0xa7, 0x33, 0xb3, 0x35, 0xe2, 0x14, + 0xf0, 0x1b, 0xb7, 0xe4, 0xfb, 0xaf, 0x8c, 0xfd, 0x7e, 0x83, 0xcc, 0x38, 0x89, 0xf0, 0xf8, 0xd1, + 0xd1, 0xf4, 0xac, 0xfb, 0x2d, 0xea, 0x7b, 0x30, 0xa5, 0x49, 0x4e, 0xae, 0x34, 0x33, 0xd8, 0xd0, + 0xa8, 0xca, 0x1c, 0x72, 0x8d, 0xf9, 0x0c, 0xc4, 0x35, 0x7d, 0x54, 0x97, 0x86, 0x99, 0x4a, 0xc2, + 0x32, 0x46, 0x71, 0x12, 0x41, 0xc5, 0x09, 0x02, 0xa7, 0x85, 0x2d, 0x00, 0xbf, 0x55, 0x68, 0x09, + 0x92, 0x66, 0x4b, 0x73, 0x03, 0xbb, 0x73, 0xa5, 0xf4, 0xc9, 0xf1, 0xe2, 0xf4, 0x56, 0x4b, 0xab, + 0xad, 0x49, 0xd3, 0x66, 0x4b, 0xab, 0x69, 0xf4, 0xa2, 0x32, 0x3e, 0x90, 0xbd, 0x40, 0x95, 0xac, + 0x34, 0x63, 0xe0, 0x03, 0x3f, 0x54, 0xc6, 0x13, 0x4e, 0xd4, 0xe6, 0x0f, 0x04, 0xc8, 0xbb, 0x3d, + 0x18, 0xed, 0x4c, 0x4f, 0xe9, 0x6d, 0xae, 0xf9, 0xf1, 0xd3, 0x69, 0xbe, 0xcb, 0xc7, 0xaf, 0x53, + 0x7d, 0x4d, 0xe0, 0x61, 0x77, 0x75, 0x55, 0x71, 0x88, 0x99, 0x8f, 0x50, 0xbd, 0x5f, 0x06, 0xd1, + 0x52, 0x0c, 0xcd, 0x6c, 0xeb, 0xcf, 0x31, 0xdb, 0x1f, 0xb1, 0xf9, 0xb1, 0xc0, 0xac, 0x47, 0xa7, + 0x1b, 0x00, 0xee, 0xf6, 0xce, 0xbf, 0x0a, 0x3c, 0x44, 0xcf, 0xab, 0x4c, 0x94, 0x9d, 0xb6, 0x0e, + 0x49, 0xba, 0xb5, 0xe7, 0x4e, 0xb7, 0xd7, 0x42, 0x84, 0x84, 0x7d, 0x9d, 0x45, 0x1d, 0x79, 0x0a, + 0x4f, 0x45, 0x2c, 0x7c, 0x01, 0xa6, 0x29, 0xf9, 0x0c, 0x36, 0x8e, 0xf7, 0xfc, 0x53, 0x98, 0x5b, + 0xd5, 0xb4, 0x7a, 0x9d, 0x6b, 0x5f, 0x74, 0xdd, 0xee, 0x7a, 0x4f, 0xb1, 0x30, 0xef, 0x29, 0xf8, + 0xc9, 0x28, 0xbd, 0xa7, 0x0e, 0xe4, 0x79, 0x94, 0x70, 0xc4, 0x7b, 0xb9, 0x34, 0x92, 0x98, 0xab, + 0x0d, 0x4b, 0xf8, 0xb7, 0x54, 0xbd, 0x2f, 0x46, 0xd9, 0x92, 0x2e, 0xcc, 0xbb, 0x72, 0xa3, 0x3e, + 0x36, 0x19, 0xd5, 0x1c, 0xba, 0x27, 0x16, 0xfc, 0x6c, 0x94, 0x6d, 0xfa, 0x67, 0x01, 0xf2, 0xf5, + 0xee, 0x2e, 0x7b, 0xaa, 0x20, 0xca, 0x38, 0xc7, 0x74, 0x0b, 0xef, 0x39, 0xf2, 0xe9, 0xa2, 0xf6, + 0x5c, 0x4b, 0x44, 0x58, 0x09, 0x15, 0xad, 0x02, 0x58, 0x34, 0x6a, 0x9c, 0xca, 0x89, 0x4f, 0x1c, + 0xfd, 0x97, 0xa6, 0x5c, 0x81, 0x28, 0xc5, 0x1f, 0xc6, 0x60, 0xd6, 0x6b, 0x66, 0x94, 0xa6, 0xe3, + 0xff, 0x01, 0xf5, 0x7e, 0x65, 0xdb, 0x51, 0x1c, 0x9b, 0xd7, 0xf0, 0xd5, 0xd3, 0x5c, 0xe1, 0x28, + 0xcd, 0xf1, 0x58, 0xac, 0xb4, 0x47, 0x92, 0xd2, 0x44, 0x24, 0xfd, 0x89, 0x96, 0x61, 0x9e, 0xda, + 0x47, 0x59, 0xe9, 0x74, 0x5a, 0x3a, 0xd6, 0x64, 0x76, 0x82, 0x91, 0xa0, 0x27, 0x18, 0x73, 0x34, + 0x6b, 0x95, 0xe5, 0xd4, 0xe8, 0x69, 0xc6, 0x7d, 0xc8, 0xee, 0x59, 0x18, 0x3f, 0xc7, 0x32, 0xf5, + 0xba, 0x4e, 0x73, 0xae, 0x95, 0x61, 0x8c, 0x75, 0xc2, 0xc7, 0x2d, 0xd1, 0xfb, 0x30, 0x47, 0xbb, + 0x36, 0xea, 0x9b, 0x37, 0x7c, 0x54, 0x7e, 0x21, 0x00, 0x0a, 0xca, 0xff, 0xe8, 0x06, 0x26, 0x16, + 0xf9, 0xc0, 0xbc, 0x0a, 0x88, 0x85, 0x1a, 0xd8, 0x72, 0x07, 0x5b, 0xb2, 0x8d, 0x55, 0x93, 0xdf, + 0xaf, 0x17, 0x24, 0x91, 0xe7, 0x6c, 0x63, 0xab, 0x4e, 0xe9, 0xc5, 0xef, 0x5d, 0x82, 0x2c, 0xef, + 0x93, 0x1d, 0x83, 0x80, 0xfd, 0xbb, 0x10, 0x6f, 0xf2, 0xdd, 0x94, 0x4c, 0x28, 0xde, 0xf2, 0xdf, + 0xf8, 0xa8, 0x4e, 0x49, 0xa4, 0x2c, 0x61, 0xe9, 0x74, 0x9d, 0x90, 0xd0, 0x3a, 0x3f, 0x82, 0x2a, + 0xc8, 0xd2, 0xe9, 0x3a, 0xa8, 0x0e, 0xb3, 0xaa, 0xff, 0xc6, 0x81, 0x4c, 0xd8, 0xe3, 0x43, 0x2f, + 0xa2, 0x84, 0xbe, 0x14, 0x51, 0x9d, 0x92, 0xf2, 0x6a, 0x4f, 0x06, 0x2a, 0x07, 0x2f, 0xd5, 0x27, + 0x86, 0xee, 0x1d, 0xf6, 0x5f, 0xe8, 0xaf, 0x4e, 0x05, 0xee, 0xde, 0xa3, 0x37, 0x20, 0xa9, 0xd1, + 0xcb, 0xda, 0x5c, 0x43, 0xc3, 0x94, 0xa8, 0xe7, 0x7e, 0x7c, 0x75, 0x4a, 0xe2, 0x1c, 0xe8, 0x21, + 0x64, 0xd9, 0x2f, 0x76, 0x4b, 0x99, 0xbb, 0x95, 0xd7, 0x87, 0x4b, 0x08, 0x58, 0xeb, 0xea, 0x94, + 0x94, 0xd1, 0x7c, 0x2a, 0xfa, 0x24, 0x24, 0x6c, 0x55, 0x31, 0xf8, 0x96, 0xe3, 0xe5, 0x21, 0x97, + 0x52, 0x7d, 0x66, 0x5a, 0x1a, 0xbd, 0x0b, 0x73, 0x74, 0x83, 0x45, 0x76, 0xfc, 0x93, 0x67, 0x7a, + 0x29, 0xa6, 0xf7, 0xb0, 0xdb, 0x73, 0xb7, 0xc2, 0xef, 0x62, 0x55, 0xa7, 0x24, 0x71, 0xb7, 0x2f, + 0x8b, 0x0c, 0x19, 0xf5, 0x97, 0x03, 0x82, 0xd3, 0x43, 0x87, 0x2c, 0xf4, 0x76, 0x14, 0x19, 0x32, + 0xdc, 0x93, 0x81, 0x1e, 0x40, 0x46, 0x21, 0xfe, 0x8b, 0x4c, 0xef, 0x37, 0x14, 0x60, 0xe8, 0xde, + 0xf1, 0xc0, 0xfd, 0x92, 0x2a, 0xbd, 0x58, 0xe6, 0x12, 0x7d, 0x41, 0x6d, 0x6c, 0x35, 0x71, 0x21, + 0x33, 0x5a, 0x50, 0xf0, 0xb8, 0xda, 0x13, 0x44, 0x89, 0x68, 0x03, 0x72, 0xfb, 0x6e, 0xbc, 0x2f, + 0x0d, 0x15, 0xc8, 0x0e, 0xdd, 0x40, 0x0e, 0x89, 0x57, 0xae, 0x4e, 0x49, 0xd9, 0xfd, 0x00, 0x19, + 0x2d, 0x43, 0xac, 0xa9, 0x16, 0x72, 0x54, 0xc6, 0x8b, 0xa3, 0xa2, 0x71, 0xab, 0x53, 0x52, 0xac, + 0xa9, 0x12, 0x54, 0xc2, 0xc2, 0x06, 0x0f, 0x8d, 0x42, 0x7e, 0xa8, 0x91, 0xe9, 0x0d, 0x16, 0xad, + 0x4e, 0x49, 0x34, 0xa4, 0x92, 0x7c, 0x6f, 0x1b, 0xf2, 0x16, 0x3b, 0xef, 0x77, 0xa3, 0x5a, 0x44, + 0x2a, 0xe5, 0x66, 0xb8, 0xa9, 0x1a, 0x08, 0x6c, 0xa9, 0x4e, 0x49, 0x39, 0x2b, 0x48, 0x47, 0x5f, + 0x86, 0x73, 0xbd, 0x12, 0xb9, 0x72, 0xcf, 0x0d, 0x58, 0xae, 0x70, 0xb9, 0xbd, 0x3a, 0x8e, 0xac, + 0x81, 0x4c, 0xf4, 0x69, 0x98, 0x66, 0xa3, 0x86, 0xa8, 0xc8, 0xb0, 0xa3, 0xa6, 0xbe, 0x01, 0x63, + 0xe5, 0xc9, 0x7c, 0x73, 0xf8, 0x41, 0xb7, 0xdc, 0x32, 0x9b, 0x85, 0xf9, 0xa1, 0xf3, 0x6d, 0xf0, + 0xe0, 0x9e, 0xcc, 0x37, 0xc7, 0xa7, 0x92, 0x71, 0xb7, 0x58, 0x0e, 0x3f, 0x17, 0x3d, 0x37, 0x74, + 0xdc, 0x43, 0xce, 0xbf, 0xab, 0x34, 0x68, 0xcf, 0x27, 0x93, 0xaa, 0x59, 0xec, 0x1e, 0xb8, 0x4c, + 0xa7, 0xf1, 0xf9, 0xa1, 0x55, 0x1b, 0xbc, 0xdb, 0x4e, 0xaa, 0x66, 0xf9, 0x54, 0xf4, 0x18, 0x44, + 0x7e, 0xd7, 0xd3, 0xdf, 0x8f, 0xba, 0x30, 0xf4, 0x24, 0x22, 0xfc, 0x20, 0xb1, 0x3a, 0x25, 0xcd, + 0xaa, 0xbd, 0x39, 0xc4, 0x58, 0x50, 0x79, 0xb2, 0xea, 0x5f, 0xd2, 0x2d, 0x14, 0x86, 0x1a, 0x8b, + 0x21, 0xd7, 0xba, 0x89, 0xb1, 0x50, 0xfb, 0xb2, 0x88, 0x1a, 0xeb, 0x86, 0xee, 0x50, 0xc3, 0xbe, + 0x30, 0x54, 0x8d, 0x7b, 0x9f, 0xa1, 0x21, 0x6a, 0xac, 0x33, 0x0a, 0x51, 0x63, 0x87, 0x1f, 0x9a, + 0xf3, 0xe1, 0x78, 0x71, 0xa8, 0x1a, 0x87, 0x9d, 0xae, 0x13, 0x35, 0x76, 0x82, 0x74, 0xa2, 0xc6, + 0xcc, 0x40, 0xf4, 0xc9, 0x7d, 0x69, 0xa8, 0x1a, 0x0f, 0xbd, 0x61, 0x44, 0xd4, 0x58, 0x19, 0xc8, + 0x44, 0x6b, 0x00, 0xcc, 0x23, 0xd2, 0x8d, 0x3d, 0xb3, 0x70, 0x79, 0xe8, 0xfa, 0xd3, 0x7f, 0x6c, + 0x4e, 0xd6, 0x9f, 0x96, 0x4b, 0x23, 0x86, 0x8c, 0xfa, 0xd8, 0x32, 0xdd, 0xfc, 0x2e, 0x2c, 0x0e, + 0x35, 0x64, 0x03, 0x7b, 0xe0, 0xc4, 0x90, 0x1d, 0x78, 0x44, 0xb2, 0x90, 0xb1, 0x2d, 0xa3, 0xc2, + 0xd2, 0x98, 0xad, 0x89, 0xc0, 0x42, 0xc6, 0x38, 0xd0, 0x2a, 0xa4, 0x89, 0xa7, 0x70, 0x44, 0xcd, + 0xd0, 0x95, 0xa1, 0xde, 0x6d, 0x5f, 0x54, 0x6a, 0x75, 0x4a, 0x4a, 0x3d, 0xe5, 0x24, 0xf2, 0x79, + 0x86, 0xdb, 0x0b, 0xc5, 0xa1, 0x9f, 0xef, 0xd9, 0xa9, 0x21, 0x9f, 0x67, 0x1c, 0x48, 0x85, 0xf3, + 0x6c, 0xac, 0xf8, 0x95, 0x24, 0x8b, 0xdf, 0xdd, 0x29, 0x5c, 0xa5, 0xa2, 0x86, 0xa2, 0xe0, 0xd0, + 0xeb, 0x51, 0xd5, 0x29, 0x69, 0x5e, 0x19, 0xcc, 0x25, 0x13, 0x9e, 0x2f, 0x3d, 0x0c, 0x3b, 0x17, + 0xae, 0x0d, 0x9d, 0xf0, 0x21, 0xbb, 0x0d, 0x64, 0xc2, 0x2b, 0x01, 0x32, 0x5b, 0x80, 0x34, 0xd9, + 0xb6, 0xd9, 0x51, 0xc9, 0xf5, 0x11, 0x0b, 0x50, 0x1f, 0x82, 0x66, 0x0b, 0x90, 0x56, 0x67, 0x9c, + 0x44, 0x90, 0xda, 0xc2, 0x8a, 0xc5, 0xcd, 0xec, 0x8d, 0xa1, 0x82, 0x06, 0x9e, 0x76, 0x21, 0x82, + 0x54, 0x8f, 0x48, 0x16, 0x6c, 0xcb, 0xbd, 0x24, 0xce, 0xbd, 0xcd, 0x9b, 0x43, 0x17, 0xec, 0xd0, + 0xbb, 0xec, 0x64, 0xc1, 0xb6, 0x7a, 0x32, 0xd0, 0xe7, 0x60, 0x86, 0xdf, 0xb8, 0x2d, 0xdc, 0x1a, + 0xe1, 0x03, 0x07, 0x61, 0x35, 0x99, 0xd7, 0x9c, 0x87, 0x59, 0x59, 0x76, 0x61, 0x97, 0x35, 0xef, + 0xe5, 0x11, 0x56, 0x76, 0x00, 0xd1, 0x32, 0x2b, 0xeb, 0x93, 0x89, 0x95, 0x65, 0x7a, 0xca, 0xd7, + 0xba, 0xdb, 0x43, 0xad, 0xec, 0x60, 0x48, 0x2c, 0xb1, 0xb2, 0x4f, 0x7d, 0x2a, 0x69, 0x99, 0xcd, + 0x80, 0x58, 0xe1, 0x95, 0xa1, 0x2d, 0xeb, 0x45, 0xa4, 0xa4, 0x65, 0x9c, 0x87, 0x0c, 0x1b, 0x8b, + 0xd6, 0x62, 0x3d, 0xfd, 0xea, 0xd0, 0x61, 0x1b, 0xc0, 0x2d, 0x55, 0xf7, 0xd9, 0x3f, 0xd6, 0xc3, + 0x9e, 0xa1, 0xb2, 0xf8, 0x6d, 0x2b, 0xde, 0x53, 0xaf, 0x8d, 0x36, 0x54, 0x61, 0x17, 0xc9, 0x3c, + 0x43, 0xd5, 0x93, 0x49, 0xab, 0xca, 0x02, 0xcc, 0xe9, 0xfc, 0x5e, 0x1e, 0x5e, 0xd5, 0xfe, 0x98, + 0x7f, 0x5a, 0x55, 0x8f, 0xe8, 0x4f, 0xa1, 0x2e, 0xbb, 0x9f, 0x5a, 0xb8, 0x33, 0x7a, 0x0a, 0xf5, + 0xde, 0x93, 0xf5, 0xa6, 0x10, 0x27, 0x97, 0x66, 0x78, 0xcc, 0xdf, 0xc3, 0x44, 0x6a, 0x56, 0x14, + 0x1f, 0x26, 0x52, 0x17, 0xc5, 0xc2, 0xc3, 0x44, 0xea, 0x92, 0xb8, 0xf0, 0x30, 0x91, 0x7a, 0x41, + 0x7c, 0xb1, 0xf8, 0x9b, 0x97, 0x20, 0xe7, 0xe2, 0x2b, 0x86, 0x56, 0x56, 0x82, 0x68, 0xe5, 0xf2, + 0x30, 0xb4, 0xc2, 0x11, 0x19, 0x87, 0x2b, 0x2b, 0x41, 0xb8, 0x72, 0x79, 0x18, 0x5c, 0xf1, 0x79, + 0x08, 0x5e, 0x69, 0x0c, 0xc3, 0x2b, 0x2f, 0x4f, 0x80, 0x57, 0x3c, 0x51, 0xfd, 0x80, 0x65, 0x6d, + 0x10, 0xb0, 0x5c, 0x1b, 0x0d, 0x58, 0x3c, 0x51, 0x01, 0xc4, 0xf2, 0x66, 0x1f, 0x62, 0xb9, 0x32, + 0x02, 0xb1, 0x78, 0xfc, 0x2e, 0x64, 0x59, 0x0f, 0x85, 0x2c, 0x37, 0xc6, 0x41, 0x16, 0x4f, 0x4e, + 0x0f, 0x66, 0x79, 0xbd, 0x07, 0xb3, 0x2c, 0x0e, 0xc5, 0x2c, 0x1e, 0x37, 0x03, 0x2d, 0xef, 0x0d, + 0x07, 0x2d, 0xaf, 0x4c, 0x04, 0x5a, 0x3c, 0x79, 0x83, 0xa8, 0xa5, 0x31, 0x0c, 0xb5, 0xbc, 0x3c, + 0x01, 0x6a, 0xf1, 0x07, 0xae, 0x0f, 0xb6, 0x54, 0xc3, 0x60, 0xcb, 0xf5, 0x31, 0xb0, 0xc5, 0x93, + 0x16, 0xc4, 0x2d, 0xd5, 0x30, 0xdc, 0x72, 0x7d, 0x0c, 0x6e, 0xe9, 0x93, 0xc4, 0x80, 0xcb, 0x66, + 0x38, 0x70, 0xb9, 0x39, 0x16, 0xb8, 0x78, 0xd2, 0x7a, 0x91, 0xcb, 0x9d, 0x00, 0x72, 0x79, 0x69, + 0x08, 0x72, 0xf1, 0x58, 0x09, 0x74, 0xf9, 0xfc, 0x00, 0x74, 0x29, 0x8e, 0x82, 0x2e, 0x1e, 0xaf, + 0x87, 0x5d, 0xde, 0x1e, 0x82, 0x5d, 0x6e, 0x8d, 0xc7, 0x2e, 0x9e, 0xb0, 0x3e, 0xf0, 0xa2, 0x8c, + 0x04, 0x2f, 0xaf, 0x4d, 0x08, 0x5e, 0x3c, 0xe9, 0x61, 0xe8, 0xe5, 0x33, 0xbd, 0xe8, 0x65, 0x69, + 0x38, 0x7a, 0xf1, 0xc4, 0x70, 0xf8, 0xb2, 0x1e, 0x0a, 0x5f, 0x6e, 0x8c, 0x83, 0x2f, 0xfe, 0xdc, + 0x0b, 0xe2, 0x97, 0xcd, 0x70, 0xfc, 0x72, 0x73, 0x2c, 0x7e, 0xf1, 0x87, 0xbf, 0x07, 0xc0, 0xac, + 0x87, 0x02, 0x98, 0x1b, 0xe3, 0x00, 0x8c, 0x5f, 0xb9, 0x20, 0x82, 0x79, 0x67, 0x28, 0x82, 0xb9, + 0x3d, 0x09, 0x82, 0xf1, 0x84, 0x0e, 0x40, 0x98, 0xf7, 0x86, 0x43, 0x98, 0x57, 0x4e, 0xf1, 0x24, + 0x51, 0x28, 0x86, 0xf9, 0xfc, 0x00, 0x86, 0x29, 0x8e, 0xc2, 0x30, 0xbe, 0x3e, 0xbb, 0x20, 0x46, + 0x19, 0x09, 0x39, 0x5e, 0x9b, 0x10, 0x72, 0xf8, 0xca, 0x17, 0x82, 0x39, 0x2a, 0x21, 0x98, 0xe3, + 0xda, 0x68, 0xcc, 0xe1, 0x2f, 0x21, 0x3e, 0xe8, 0xa8, 0x86, 0x81, 0x8e, 0xeb, 0x63, 0x40, 0x87, + 0x6f, 0x85, 0x02, 0xa8, 0xe3, 0xcd, 0x3e, 0xd4, 0x71, 0x65, 0xec, 0x89, 0x7e, 0x00, 0x76, 0x94, + 0x06, 0x61, 0xc7, 0xd5, 0x91, 0xb0, 0xc3, 0x93, 0xe0, 0xe3, 0x8e, 0x37, 0xfb, 0x70, 0xc7, 0x95, + 0x11, 0xb8, 0xc3, 0xaf, 0x00, 0x07, 0x1e, 0xda, 0x68, 0xe0, 0xb1, 0x3c, 0x29, 0xf0, 0xf0, 0x04, + 0x87, 0x22, 0x8f, 0xcd, 0x70, 0xe4, 0x71, 0x73, 0xc2, 0xc3, 0xbd, 0x01, 0xe8, 0x51, 0x0d, 0x83, + 0x1e, 0xd7, 0xc7, 0x40, 0x8f, 0xe0, 0x1a, 0xe2, 0x61, 0x8f, 0x6a, 0x18, 0xf6, 0xb8, 0x3e, 0x06, + 0x7b, 0xf8, 0x92, 0x02, 0xe0, 0xa3, 0x31, 0x0c, 0x7c, 0xbc, 0x3c, 0x01, 0xf8, 0xf0, 0xd7, 0xdd, + 0x3e, 0xf4, 0xf1, 0x56, 0x3f, 0xfa, 0x28, 0x8e, 0x42, 0x1f, 0xfe, 0x8c, 0x74, 0xe1, 0xc7, 0x66, + 0x38, 0xfc, 0xb8, 0x39, 0x16, 0x7e, 0x04, 0x8d, 0x64, 0x00, 0x7f, 0xac, 0x87, 0xe2, 0x8f, 0x1b, + 0xe3, 0xf0, 0x87, 0x6f, 0x24, 0x83, 0x00, 0xe4, 0xad, 0x7e, 0x00, 0x52, 0x1c, 0x05, 0x40, 0xfc, + 0xc6, 0xb9, 0x08, 0xa4, 0x1a, 0x86, 0x40, 0xae, 0x8f, 0x41, 0x20, 0xfe, 0xe0, 0x05, 0x20, 0x88, + 0x32, 0x12, 0x82, 0xbc, 0x36, 0x21, 0x04, 0xe9, 0x33, 0x5c, 0xbd, 0x18, 0xa4, 0x1a, 0x86, 0x41, + 0xae, 0x8f, 0xc1, 0x20, 0x81, 0xca, 0xfa, 0x20, 0x64, 0x33, 0x1c, 0x84, 0xdc, 0x1c, 0x0b, 0x42, + 0xfa, 0x66, 0xd3, 0xa4, 0x28, 0xe4, 0x61, 0x22, 0xf5, 0xa2, 0xf8, 0x52, 0xf1, 0x2f, 0xa7, 0x21, + 0x59, 0x75, 0x63, 0x62, 0x02, 0x0f, 0x0c, 0x08, 0x67, 0x79, 0x60, 0x00, 0xad, 0x11, 0xc5, 0xa6, + 0xe6, 0x81, 0xe3, 0x92, 0x11, 0xaf, 0x75, 0x0c, 0x1c, 0x4b, 0xba, 0xac, 0x67, 0xb8, 0x31, 0x84, + 0x5e, 0x87, 0x5c, 0xd7, 0xc6, 0x96, 0xdc, 0xb1, 0x74, 0xd3, 0xd2, 0x1d, 0x16, 0x49, 0x2a, 0x94, + 0xc4, 0x0f, 0x8f, 0x17, 0xb3, 0x3b, 0x36, 0xb6, 0xb6, 0x39, 0x5d, 0xca, 0x76, 0x03, 0x29, 0xf7, + 0x9d, 0xfd, 0xe9, 0xc9, 0xdf, 0xd9, 0x7f, 0x1b, 0x44, 0x0b, 0x2b, 0x5a, 0xcf, 0x42, 0xcd, 0xee, + 0xdc, 0x87, 0xfb, 0x14, 0x34, 0x64, 0xda, 0x2d, 0x49, 0xef, 0xde, 0xcf, 0x5a, 0xbd, 0x44, 0x74, + 0x17, 0xce, 0xb7, 0x95, 0x43, 0xf6, 0xd4, 0x84, 0xeb, 0xfb, 0xd0, 0xd8, 0xa0, 0x14, 0x0d, 0x61, + 0x43, 0x6d, 0xe5, 0x90, 0x3e, 0xda, 0xcf, 0xb2, 0xe8, 0x73, 0xbd, 0xd7, 0x21, 0xaf, 0xe9, 0xb6, + 0xa3, 0x1b, 0xaa, 0xfb, 0x50, 0x19, 0xbb, 0x9c, 0x9f, 0x73, 0xa9, 0xec, 0xc1, 0xb0, 0xdb, 0x30, + 0xc7, 0x83, 0x05, 0xfd, 0x67, 0xfc, 0xa9, 0x97, 0x9f, 0x22, 0xb5, 0x20, 0x19, 0xfe, 0xff, 0x6f, + 0x28, 0xc3, 0x6c, 0x53, 0x71, 0xf0, 0x81, 0x72, 0x24, 0xbb, 0x51, 0xdb, 0x19, 0xfa, 0x66, 0xd0, + 0x0b, 0x27, 0xc7, 0x8b, 0xb9, 0x07, 0x2c, 0x6b, 0x20, 0x78, 0x3b, 0xd7, 0x0c, 0x64, 0x68, 0x68, + 0x15, 0xb2, 0xf4, 0x25, 0x52, 0x93, 0x3d, 0x78, 0xcb, 0x7d, 0xf7, 0x61, 0xa7, 0x3e, 0xfc, 0x59, + 0x5c, 0x89, 0xbe, 0x5e, 0xea, 0xbe, 0x91, 0x7b, 0x13, 0x66, 0x15, 0xfb, 0xc8, 0x50, 0x69, 0x0f, + 0x63, 0xc3, 0xee, 0xda, 0xd4, 0x79, 0x4f, 0x49, 0x79, 0x4a, 0x2e, 0xbb, 0x54, 0xfe, 0x38, 0xd9, + 0x6f, 0x0b, 0x90, 0xed, 0x89, 0xa0, 0x7d, 0xb3, 0xef, 0x70, 0xf3, 0x52, 0x38, 0x70, 0x18, 0x16, + 0x16, 0x96, 0xe2, 0x23, 0xe0, 0x86, 0xaa, 0x2c, 0x0e, 0x77, 0x3c, 0x29, 0x74, 0x77, 0x8f, 0xd4, + 0x5d, 0xb6, 0x37, 0x12, 0xbf, 0xfb, 0xed, 0xc5, 0xa9, 0xe2, 0x2f, 0xe3, 0x90, 0xeb, 0x8d, 0x94, + 0xad, 0xf5, 0xd5, 0x2b, 0x6c, 0x62, 0xf7, 0x70, 0x2c, 0x8f, 0x78, 0x13, 0x27, 0xed, 0xbf, 0x5f, + 0xca, 0xaa, 0xb9, 0x34, 0xe2, 0x08, 0x37, 0x58, 0x4f, 0x9f, 0x71, 0xe1, 0x47, 0x31, 0x6f, 0xe6, + 0x2f, 0xc3, 0x34, 0xbd, 0xb5, 0xcc, 0xab, 0x16, 0x76, 0x97, 0xa8, 0x42, 0xf2, 0x25, 0x56, 0x8c, + 0x58, 0x8a, 0xc6, 0x99, 0x9e, 0x22, 0xf1, 0x08, 0x67, 0xf8, 0x0f, 0x17, 0xfc, 0xf9, 0x9b, 0xe9, + 0xd3, 0x3d, 0x7f, 0xc3, 0x8e, 0x68, 0x5b, 0x2d, 0xac, 0x3a, 0xfc, 0x5f, 0x9a, 0xb8, 0xff, 0xc7, + 0xe2, 0x5a, 0xbf, 0x08, 0xfe, 0x0f, 0x50, 0x96, 0x25, 0xfe, 0x0f, 0x50, 0x02, 0xd1, 0x43, 0x79, + 0x4f, 0x04, 0x9d, 0x58, 0x2c, 0xc6, 0x8c, 0x0f, 0xf5, 0xaf, 0x0b, 0x20, 0xd2, 0x69, 0x74, 0x1f, + 0x63, 0x2d, 0x12, 0x2d, 0x74, 0x03, 0x9b, 0x62, 0x13, 0x07, 0x36, 0x15, 0x15, 0xc8, 0x7b, 0x75, + 0x60, 0x0f, 0xfa, 0x8f, 0x78, 0x49, 0xe6, 0x4c, 0x97, 0x52, 0x8b, 0xbf, 0x27, 0xc0, 0xbc, 0xf7, + 0x8d, 0xb2, 0x77, 0x17, 0xea, 0x2c, 0xa1, 0xa6, 0x12, 0xfd, 0x67, 0x25, 0x04, 0x81, 0xd2, 0x8b, + 0x8a, 0x13, 0x69, 0x10, 0xe2, 0xd1, 0x02, 0xc0, 0x91, 0xad, 0xd6, 0xa8, 0xd3, 0x7f, 0x63, 0xc2, + 0x7e, 0xdb, 0xc5, 0xfb, 0x81, 0x1e, 0xa0, 0xca, 0x4a, 0x9a, 0x39, 0x91, 0x56, 0xbb, 0xcd, 0xa4, + 0x85, 0x8b, 0x3f, 0x11, 0x82, 0x82, 0x9e, 0x11, 0x8f, 0xe6, 0x1e, 0xc4, 0x9f, 0x29, 0xad, 0x51, + 0xc1, 0x12, 0x3d, 0x5d, 0x2f, 0x91, 0xd2, 0xe8, 0x7e, 0xcf, 0x15, 0xb2, 0xd8, 0x70, 0xd8, 0x39, + 0xd8, 0xa5, 0xc1, 0xab, 0x66, 0xe8, 0xd3, 0x6e, 0x2b, 0xe2, 0xe3, 0x3f, 0x1f, 0x9c, 0xa4, 0x6f, + 0x24, 0x3e, 0xf8, 0xf6, 0xa2, 0x70, 0xbb, 0x0e, 0xf3, 0x21, 0x8b, 0x10, 0xca, 0x03, 0x04, 0x9e, + 0x94, 0xe5, 0xff, 0x2b, 0x65, 0x75, 0x4d, 0xde, 0xd9, 0x2c, 0x6f, 0x6d, 0x6c, 0xd4, 0x1a, 0x8d, + 0xca, 0x9a, 0x28, 0x20, 0x11, 0xb2, 0x3d, 0x0f, 0xd2, 0xf2, 0x7f, 0x91, 0x72, 0xfb, 0x53, 0x00, + 0xfe, 0x53, 0xd2, 0x44, 0xd6, 0x7a, 0xe5, 0x5d, 0xf9, 0xf1, 0xea, 0xa3, 0x9d, 0x4a, 0x5d, 0x9c, + 0x42, 0x08, 0xf2, 0xa5, 0xd5, 0x46, 0xb9, 0x2a, 0x4b, 0x95, 0xfa, 0xf6, 0xd6, 0x66, 0xbd, 0x22, + 0x0a, 0x9c, 0x6f, 0x0d, 0xb2, 0xc1, 0xcb, 0x76, 0x68, 0x1e, 0x66, 0xcb, 0xd5, 0x4a, 0x79, 0x5d, + 0x7e, 0x5c, 0x5b, 0x95, 0xdf, 0xde, 0xa9, 0xec, 0x54, 0xc4, 0x29, 0x5a, 0x35, 0x4a, 0xbc, 0xbf, + 0xf3, 0xe8, 0x91, 0x28, 0xa0, 0x59, 0xc8, 0xb0, 0x34, 0x7d, 0xbc, 0x56, 0x8c, 0xdd, 0xde, 0x80, + 0x4c, 0xe0, 0x2d, 0x1b, 0xf2, 0xb9, 0xed, 0x9d, 0x7a, 0x55, 0x6e, 0xd4, 0x36, 0x2a, 0xf5, 0xc6, + 0xea, 0xc6, 0x36, 0x93, 0x41, 0x69, 0xab, 0xa5, 0x2d, 0xa9, 0x21, 0x0a, 0x5e, 0xba, 0xb1, 0xb5, + 0x53, 0xae, 0x7a, 0xff, 0xe9, 0x25, 0x91, 0x8a, 0x8b, 0xf1, 0xdb, 0x26, 0x9c, 0x0f, 0xbd, 0xbd, + 0x86, 0x32, 0x30, 0xb3, 0x63, 0xd0, 0xb7, 0x39, 0xc4, 0x29, 0x94, 0x0b, 0x5c, 0x60, 0x13, 0x05, + 0x94, 0x62, 0xd7, 0x94, 0xc4, 0x18, 0x4a, 0x42, 0xac, 0x7e, 0x4f, 0x8c, 0x93, 0x6a, 0x06, 0x6e, + 0x81, 0x89, 0x09, 0x94, 0xe6, 0x97, 0x67, 0xc4, 0x69, 0x94, 0xf5, 0x6f, 0xaf, 0x88, 0xc9, 0xdb, + 0x57, 0x20, 0x10, 0xdc, 0x8f, 0x00, 0x92, 0x8f, 0x14, 0x07, 0xdb, 0x8e, 0x38, 0x85, 0x66, 0x20, + 0xbe, 0xda, 0x6a, 0x89, 0xc2, 0xca, 0x0f, 0x04, 0x48, 0xb9, 0xef, 0x96, 0xa2, 0x47, 0x30, 0xcd, + 0x40, 0xe9, 0xe2, 0xf0, 0x25, 0x83, 0x5a, 0x9d, 0x85, 0xa5, 0x71, 0x6b, 0x4a, 0x71, 0x0a, 0xbd, + 0xc3, 0xff, 0x69, 0x13, 0xd1, 0x17, 0x74, 0x75, 0x94, 0x36, 0xb9, 0x52, 0x47, 0xab, 0x1c, 0x99, + 0x21, 0xc5, 0xa9, 0x4f, 0x08, 0xa5, 0x2b, 0x1f, 0xfc, 0xe3, 0xe5, 0xa9, 0x0f, 0x4e, 0x2e, 0x0b, + 0x3f, 0x3d, 0xb9, 0x2c, 0xfc, 0xfc, 0xe4, 0xb2, 0xf0, 0x0f, 0x27, 0x97, 0x85, 0xdf, 0xfa, 0xa7, + 0xcb, 0x53, 0xef, 0xcd, 0x70, 0xae, 0xdd, 0x24, 0xfd, 0xbf, 0x51, 0xf7, 0xfe, 0x2b, 0x00, 0x00, + 0xff, 0xff, 0x1a, 0x05, 0x6e, 0x16, 0x3c, 0x6b, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index e226c02c7505..ee723cbb8f09 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -675,6 +675,24 @@ message AdminSplitResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; } +// An AdminUnsplitRequest is the argument to the AdminUnsplit() +// method. The sticky bit of the existing range whose starting key is +// header.key is removed. +// +// Ranges that do not have the sticky bit set are eligible for +// automatic merging. +message AdminUnsplitRequest { + option (gogoproto.equal) = true; + + RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; +} + +// An AdminUnsplitResponse is the return value from the +// AdminUnsplit() method. +message AdminUnsplitResponse { + ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; +} + // An AdminMergeRequest is the argument to the AdminMerge() method. A // merge is performed by calling AdminMerge on the left-hand range of // two consecutive ranges (i.e. the range which contains keys which @@ -1555,6 +1573,7 @@ message RequestUnion { BeginTransactionRequest begin_transaction = 8; EndTransactionRequest end_transaction = 9; AdminSplitRequest admin_split = 10; + AdminUnsplitRequest admin_unsplit = 47; AdminMergeRequest admin_merge = 11; AdminTransferLeaseRequest admin_transfer_lease = 29; AdminChangeReplicasRequest admin_change_replicas = 35; @@ -1605,6 +1624,7 @@ message ResponseUnion { BeginTransactionResponse begin_transaction = 8; EndTransactionResponse end_transaction = 9; AdminSplitResponse admin_split = 10; + AdminUnsplitResponse admin_unsplit = 47; AdminMergeResponse admin_merge = 11; AdminTransferLeaseResponse admin_transfer_lease = 29; AdminChangeReplicasResponse admin_change_replicas = 35; diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index 279598122484..578cb7433812 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -102,6 +102,8 @@ func (ru RequestUnion) GetInner() Request { return t.EndTransaction case *RequestUnion_AdminSplit: return t.AdminSplit + case *RequestUnion_AdminUnsplit: + return t.AdminUnsplit case *RequestUnion_AdminMerge: return t.AdminMerge case *RequestUnion_AdminTransferLease: @@ -194,6 +196,8 @@ func (ru ResponseUnion) GetInner() Response { return t.EndTransaction case *ResponseUnion_AdminSplit: return t.AdminSplit + case *ResponseUnion_AdminUnsplit: + return t.AdminUnsplit case *ResponseUnion_AdminMerge: return t.AdminMerge case *ResponseUnion_AdminTransferLease: @@ -356,6 +360,8 @@ func (ru *RequestUnion) SetInner(r Request) bool { union = &RequestUnion_EndTransaction{t} case *AdminSplitRequest: union = &RequestUnion_AdminSplit{t} + case *AdminUnsplitRequest: + union = &RequestUnion_AdminUnsplit{t} case *AdminMergeRequest: union = &RequestUnion_AdminMerge{t} case *AdminTransferLeaseRequest: @@ -451,6 +457,8 @@ func (ru *ResponseUnion) SetInner(r Response) bool { union = &ResponseUnion_EndTransaction{t} case *AdminSplitResponse: union = &ResponseUnion_AdminSplit{t} + case *AdminUnsplitResponse: + union = &ResponseUnion_AdminUnsplit{t} case *AdminMergeResponse: union = &ResponseUnion_AdminMerge{t} case *AdminTransferLeaseResponse: @@ -518,7 +526,7 @@ func (ru *ResponseUnion) SetInner(r Response) bool { return true } -type reqCounts [42]int32 +type reqCounts [43]int32 // getReqCounts returns the number of times each // request type appears in the batch. @@ -548,68 +556,70 @@ func (ba *BatchRequest) getReqCounts() reqCounts { counts[9]++ case *RequestUnion_AdminSplit: counts[10]++ - case *RequestUnion_AdminMerge: + case *RequestUnion_AdminUnsplit: counts[11]++ - case *RequestUnion_AdminTransferLease: + case *RequestUnion_AdminMerge: counts[12]++ - case *RequestUnion_AdminChangeReplicas: + case *RequestUnion_AdminTransferLease: counts[13]++ - case *RequestUnion_AdminRelocateRange: + case *RequestUnion_AdminChangeReplicas: counts[14]++ - case *RequestUnion_HeartbeatTxn: + case *RequestUnion_AdminRelocateRange: counts[15]++ - case *RequestUnion_Gc: + case *RequestUnion_HeartbeatTxn: counts[16]++ - case *RequestUnion_PushTxn: + case *RequestUnion_Gc: counts[17]++ - case *RequestUnion_RecoverTxn: + case *RequestUnion_PushTxn: counts[18]++ - case *RequestUnion_ResolveIntent: + case *RequestUnion_RecoverTxn: counts[19]++ - case *RequestUnion_ResolveIntentRange: + case *RequestUnion_ResolveIntent: counts[20]++ - case *RequestUnion_Merge: + case *RequestUnion_ResolveIntentRange: counts[21]++ - case *RequestUnion_TruncateLog: + case *RequestUnion_Merge: counts[22]++ - case *RequestUnion_RequestLease: + case *RequestUnion_TruncateLog: counts[23]++ - case *RequestUnion_ReverseScan: + case *RequestUnion_RequestLease: counts[24]++ - case *RequestUnion_ComputeChecksum: + case *RequestUnion_ReverseScan: counts[25]++ - case *RequestUnion_CheckConsistency: + case *RequestUnion_ComputeChecksum: counts[26]++ - case *RequestUnion_InitPut: + case *RequestUnion_CheckConsistency: counts[27]++ - case *RequestUnion_TransferLease: + case *RequestUnion_InitPut: counts[28]++ - case *RequestUnion_LeaseInfo: + case *RequestUnion_TransferLease: counts[29]++ - case *RequestUnion_WriteBatch: + case *RequestUnion_LeaseInfo: counts[30]++ - case *RequestUnion_Export: + case *RequestUnion_WriteBatch: counts[31]++ - case *RequestUnion_Import: + case *RequestUnion_Export: counts[32]++ - case *RequestUnion_QueryTxn: + case *RequestUnion_Import: counts[33]++ - case *RequestUnion_QueryIntent: + case *RequestUnion_QueryTxn: counts[34]++ - case *RequestUnion_AdminScatter: + case *RequestUnion_QueryIntent: counts[35]++ - case *RequestUnion_AddSstable: + case *RequestUnion_AdminScatter: counts[36]++ - case *RequestUnion_RecomputeStats: + case *RequestUnion_AddSstable: counts[37]++ - case *RequestUnion_Refresh: + case *RequestUnion_RecomputeStats: counts[38]++ - case *RequestUnion_RefreshRange: + case *RequestUnion_Refresh: counts[39]++ - case *RequestUnion_Subsume: + case *RequestUnion_RefreshRange: counts[40]++ - case *RequestUnion_RangeStats: + case *RequestUnion_Subsume: counts[41]++ + case *RequestUnion_RangeStats: + counts[42]++ default: panic(fmt.Sprintf("unsupported request: %+v", ru)) } @@ -629,6 +639,7 @@ var requestNames = []string{ "BeginTxn", "EndTxn", "AdmSplit", + "AdmUnsplit", "AdmMerge", "AdmTransferLease", "AdmChangeReplicas", @@ -739,6 +750,10 @@ type adminSplitResponseAlloc struct { union ResponseUnion_AdminSplit resp AdminSplitResponse } +type adminUnsplitResponseAlloc struct { + union ResponseUnion_AdminUnsplit + resp AdminUnsplitResponse +} type adminMergeResponseAlloc struct { union ResponseUnion_AdminMerge resp AdminMergeResponse @@ -880,37 +895,38 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { var buf8 []beginTransactionResponseAlloc var buf9 []endTransactionResponseAlloc var buf10 []adminSplitResponseAlloc - var buf11 []adminMergeResponseAlloc - var buf12 []adminTransferLeaseResponseAlloc - var buf13 []adminChangeReplicasResponseAlloc - var buf14 []adminRelocateRangeResponseAlloc - var buf15 []heartbeatTxnResponseAlloc - var buf16 []gCResponseAlloc - var buf17 []pushTxnResponseAlloc - var buf18 []recoverTxnResponseAlloc - var buf19 []resolveIntentResponseAlloc - var buf20 []resolveIntentRangeResponseAlloc - var buf21 []mergeResponseAlloc - var buf22 []truncateLogResponseAlloc - var buf23 []requestLeaseResponseAlloc - var buf24 []reverseScanResponseAlloc - var buf25 []computeChecksumResponseAlloc - var buf26 []checkConsistencyResponseAlloc - var buf27 []initPutResponseAlloc - var buf28 []requestLeaseResponseAlloc - var buf29 []leaseInfoResponseAlloc - var buf30 []writeBatchResponseAlloc - var buf31 []exportResponseAlloc - var buf32 []importResponseAlloc - var buf33 []queryTxnResponseAlloc - var buf34 []queryIntentResponseAlloc - var buf35 []adminScatterResponseAlloc - var buf36 []addSSTableResponseAlloc - var buf37 []recomputeStatsResponseAlloc - var buf38 []refreshResponseAlloc - var buf39 []refreshRangeResponseAlloc - var buf40 []subsumeResponseAlloc - var buf41 []rangeStatsResponseAlloc + var buf11 []adminUnsplitResponseAlloc + var buf12 []adminMergeResponseAlloc + var buf13 []adminTransferLeaseResponseAlloc + var buf14 []adminChangeReplicasResponseAlloc + var buf15 []adminRelocateRangeResponseAlloc + var buf16 []heartbeatTxnResponseAlloc + var buf17 []gCResponseAlloc + var buf18 []pushTxnResponseAlloc + var buf19 []recoverTxnResponseAlloc + var buf20 []resolveIntentResponseAlloc + var buf21 []resolveIntentRangeResponseAlloc + var buf22 []mergeResponseAlloc + var buf23 []truncateLogResponseAlloc + var buf24 []requestLeaseResponseAlloc + var buf25 []reverseScanResponseAlloc + var buf26 []computeChecksumResponseAlloc + var buf27 []checkConsistencyResponseAlloc + var buf28 []initPutResponseAlloc + var buf29 []requestLeaseResponseAlloc + var buf30 []leaseInfoResponseAlloc + var buf31 []writeBatchResponseAlloc + var buf32 []exportResponseAlloc + var buf33 []importResponseAlloc + var buf34 []queryTxnResponseAlloc + var buf35 []queryIntentResponseAlloc + var buf36 []adminScatterResponseAlloc + var buf37 []addSSTableResponseAlloc + var buf38 []recomputeStatsResponseAlloc + var buf39 []refreshResponseAlloc + var buf40 []refreshRangeResponseAlloc + var buf41 []subsumeResponseAlloc + var buf42 []rangeStatsResponseAlloc for i, r := range ba.Requests { switch r.GetValue().(type) { @@ -991,223 +1007,230 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { buf10[0].union.AdminSplit = &buf10[0].resp br.Responses[i].Value = &buf10[0].union buf10 = buf10[1:] - case *RequestUnion_AdminMerge: + case *RequestUnion_AdminUnsplit: if buf11 == nil { - buf11 = make([]adminMergeResponseAlloc, counts[11]) + buf11 = make([]adminUnsplitResponseAlloc, counts[11]) } - buf11[0].union.AdminMerge = &buf11[0].resp + buf11[0].union.AdminUnsplit = &buf11[0].resp br.Responses[i].Value = &buf11[0].union buf11 = buf11[1:] - case *RequestUnion_AdminTransferLease: + case *RequestUnion_AdminMerge: if buf12 == nil { - buf12 = make([]adminTransferLeaseResponseAlloc, counts[12]) + buf12 = make([]adminMergeResponseAlloc, counts[12]) } - buf12[0].union.AdminTransferLease = &buf12[0].resp + buf12[0].union.AdminMerge = &buf12[0].resp br.Responses[i].Value = &buf12[0].union buf12 = buf12[1:] - case *RequestUnion_AdminChangeReplicas: + case *RequestUnion_AdminTransferLease: if buf13 == nil { - buf13 = make([]adminChangeReplicasResponseAlloc, counts[13]) + buf13 = make([]adminTransferLeaseResponseAlloc, counts[13]) } - buf13[0].union.AdminChangeReplicas = &buf13[0].resp + buf13[0].union.AdminTransferLease = &buf13[0].resp br.Responses[i].Value = &buf13[0].union buf13 = buf13[1:] - case *RequestUnion_AdminRelocateRange: + case *RequestUnion_AdminChangeReplicas: if buf14 == nil { - buf14 = make([]adminRelocateRangeResponseAlloc, counts[14]) + buf14 = make([]adminChangeReplicasResponseAlloc, counts[14]) } - buf14[0].union.AdminRelocateRange = &buf14[0].resp + buf14[0].union.AdminChangeReplicas = &buf14[0].resp br.Responses[i].Value = &buf14[0].union buf14 = buf14[1:] - case *RequestUnion_HeartbeatTxn: + case *RequestUnion_AdminRelocateRange: if buf15 == nil { - buf15 = make([]heartbeatTxnResponseAlloc, counts[15]) + buf15 = make([]adminRelocateRangeResponseAlloc, counts[15]) } - buf15[0].union.HeartbeatTxn = &buf15[0].resp + buf15[0].union.AdminRelocateRange = &buf15[0].resp br.Responses[i].Value = &buf15[0].union buf15 = buf15[1:] - case *RequestUnion_Gc: + case *RequestUnion_HeartbeatTxn: if buf16 == nil { - buf16 = make([]gCResponseAlloc, counts[16]) + buf16 = make([]heartbeatTxnResponseAlloc, counts[16]) } - buf16[0].union.Gc = &buf16[0].resp + buf16[0].union.HeartbeatTxn = &buf16[0].resp br.Responses[i].Value = &buf16[0].union buf16 = buf16[1:] - case *RequestUnion_PushTxn: + case *RequestUnion_Gc: if buf17 == nil { - buf17 = make([]pushTxnResponseAlloc, counts[17]) + buf17 = make([]gCResponseAlloc, counts[17]) } - buf17[0].union.PushTxn = &buf17[0].resp + buf17[0].union.Gc = &buf17[0].resp br.Responses[i].Value = &buf17[0].union buf17 = buf17[1:] - case *RequestUnion_RecoverTxn: + case *RequestUnion_PushTxn: if buf18 == nil { - buf18 = make([]recoverTxnResponseAlloc, counts[18]) + buf18 = make([]pushTxnResponseAlloc, counts[18]) } - buf18[0].union.RecoverTxn = &buf18[0].resp + buf18[0].union.PushTxn = &buf18[0].resp br.Responses[i].Value = &buf18[0].union buf18 = buf18[1:] - case *RequestUnion_ResolveIntent: + case *RequestUnion_RecoverTxn: if buf19 == nil { - buf19 = make([]resolveIntentResponseAlloc, counts[19]) + buf19 = make([]recoverTxnResponseAlloc, counts[19]) } - buf19[0].union.ResolveIntent = &buf19[0].resp + buf19[0].union.RecoverTxn = &buf19[0].resp br.Responses[i].Value = &buf19[0].union buf19 = buf19[1:] - case *RequestUnion_ResolveIntentRange: + case *RequestUnion_ResolveIntent: if buf20 == nil { - buf20 = make([]resolveIntentRangeResponseAlloc, counts[20]) + buf20 = make([]resolveIntentResponseAlloc, counts[20]) } - buf20[0].union.ResolveIntentRange = &buf20[0].resp + buf20[0].union.ResolveIntent = &buf20[0].resp br.Responses[i].Value = &buf20[0].union buf20 = buf20[1:] - case *RequestUnion_Merge: + case *RequestUnion_ResolveIntentRange: if buf21 == nil { - buf21 = make([]mergeResponseAlloc, counts[21]) + buf21 = make([]resolveIntentRangeResponseAlloc, counts[21]) } - buf21[0].union.Merge = &buf21[0].resp + buf21[0].union.ResolveIntentRange = &buf21[0].resp br.Responses[i].Value = &buf21[0].union buf21 = buf21[1:] - case *RequestUnion_TruncateLog: + case *RequestUnion_Merge: if buf22 == nil { - buf22 = make([]truncateLogResponseAlloc, counts[22]) + buf22 = make([]mergeResponseAlloc, counts[22]) } - buf22[0].union.TruncateLog = &buf22[0].resp + buf22[0].union.Merge = &buf22[0].resp br.Responses[i].Value = &buf22[0].union buf22 = buf22[1:] - case *RequestUnion_RequestLease: + case *RequestUnion_TruncateLog: if buf23 == nil { - buf23 = make([]requestLeaseResponseAlloc, counts[23]) + buf23 = make([]truncateLogResponseAlloc, counts[23]) } - buf23[0].union.RequestLease = &buf23[0].resp + buf23[0].union.TruncateLog = &buf23[0].resp br.Responses[i].Value = &buf23[0].union buf23 = buf23[1:] - case *RequestUnion_ReverseScan: + case *RequestUnion_RequestLease: if buf24 == nil { - buf24 = make([]reverseScanResponseAlloc, counts[24]) + buf24 = make([]requestLeaseResponseAlloc, counts[24]) } - buf24[0].union.ReverseScan = &buf24[0].resp + buf24[0].union.RequestLease = &buf24[0].resp br.Responses[i].Value = &buf24[0].union buf24 = buf24[1:] - case *RequestUnion_ComputeChecksum: + case *RequestUnion_ReverseScan: if buf25 == nil { - buf25 = make([]computeChecksumResponseAlloc, counts[25]) + buf25 = make([]reverseScanResponseAlloc, counts[25]) } - buf25[0].union.ComputeChecksum = &buf25[0].resp + buf25[0].union.ReverseScan = &buf25[0].resp br.Responses[i].Value = &buf25[0].union buf25 = buf25[1:] - case *RequestUnion_CheckConsistency: + case *RequestUnion_ComputeChecksum: if buf26 == nil { - buf26 = make([]checkConsistencyResponseAlloc, counts[26]) + buf26 = make([]computeChecksumResponseAlloc, counts[26]) } - buf26[0].union.CheckConsistency = &buf26[0].resp + buf26[0].union.ComputeChecksum = &buf26[0].resp br.Responses[i].Value = &buf26[0].union buf26 = buf26[1:] - case *RequestUnion_InitPut: + case *RequestUnion_CheckConsistency: if buf27 == nil { - buf27 = make([]initPutResponseAlloc, counts[27]) + buf27 = make([]checkConsistencyResponseAlloc, counts[27]) } - buf27[0].union.InitPut = &buf27[0].resp + buf27[0].union.CheckConsistency = &buf27[0].resp br.Responses[i].Value = &buf27[0].union buf27 = buf27[1:] - case *RequestUnion_TransferLease: + case *RequestUnion_InitPut: if buf28 == nil { - buf28 = make([]requestLeaseResponseAlloc, counts[28]) + buf28 = make([]initPutResponseAlloc, counts[28]) } - buf28[0].union.RequestLease = &buf28[0].resp + buf28[0].union.InitPut = &buf28[0].resp br.Responses[i].Value = &buf28[0].union buf28 = buf28[1:] - case *RequestUnion_LeaseInfo: + case *RequestUnion_TransferLease: if buf29 == nil { - buf29 = make([]leaseInfoResponseAlloc, counts[29]) + buf29 = make([]requestLeaseResponseAlloc, counts[29]) } - buf29[0].union.LeaseInfo = &buf29[0].resp + buf29[0].union.RequestLease = &buf29[0].resp br.Responses[i].Value = &buf29[0].union buf29 = buf29[1:] - case *RequestUnion_WriteBatch: + case *RequestUnion_LeaseInfo: if buf30 == nil { - buf30 = make([]writeBatchResponseAlloc, counts[30]) + buf30 = make([]leaseInfoResponseAlloc, counts[30]) } - buf30[0].union.WriteBatch = &buf30[0].resp + buf30[0].union.LeaseInfo = &buf30[0].resp br.Responses[i].Value = &buf30[0].union buf30 = buf30[1:] - case *RequestUnion_Export: + case *RequestUnion_WriteBatch: if buf31 == nil { - buf31 = make([]exportResponseAlloc, counts[31]) + buf31 = make([]writeBatchResponseAlloc, counts[31]) } - buf31[0].union.Export = &buf31[0].resp + buf31[0].union.WriteBatch = &buf31[0].resp br.Responses[i].Value = &buf31[0].union buf31 = buf31[1:] - case *RequestUnion_Import: + case *RequestUnion_Export: if buf32 == nil { - buf32 = make([]importResponseAlloc, counts[32]) + buf32 = make([]exportResponseAlloc, counts[32]) } - buf32[0].union.Import = &buf32[0].resp + buf32[0].union.Export = &buf32[0].resp br.Responses[i].Value = &buf32[0].union buf32 = buf32[1:] - case *RequestUnion_QueryTxn: + case *RequestUnion_Import: if buf33 == nil { - buf33 = make([]queryTxnResponseAlloc, counts[33]) + buf33 = make([]importResponseAlloc, counts[33]) } - buf33[0].union.QueryTxn = &buf33[0].resp + buf33[0].union.Import = &buf33[0].resp br.Responses[i].Value = &buf33[0].union buf33 = buf33[1:] - case *RequestUnion_QueryIntent: + case *RequestUnion_QueryTxn: if buf34 == nil { - buf34 = make([]queryIntentResponseAlloc, counts[34]) + buf34 = make([]queryTxnResponseAlloc, counts[34]) } - buf34[0].union.QueryIntent = &buf34[0].resp + buf34[0].union.QueryTxn = &buf34[0].resp br.Responses[i].Value = &buf34[0].union buf34 = buf34[1:] - case *RequestUnion_AdminScatter: + case *RequestUnion_QueryIntent: if buf35 == nil { - buf35 = make([]adminScatterResponseAlloc, counts[35]) + buf35 = make([]queryIntentResponseAlloc, counts[35]) } - buf35[0].union.AdminScatter = &buf35[0].resp + buf35[0].union.QueryIntent = &buf35[0].resp br.Responses[i].Value = &buf35[0].union buf35 = buf35[1:] - case *RequestUnion_AddSstable: + case *RequestUnion_AdminScatter: if buf36 == nil { - buf36 = make([]addSSTableResponseAlloc, counts[36]) + buf36 = make([]adminScatterResponseAlloc, counts[36]) } - buf36[0].union.AddSstable = &buf36[0].resp + buf36[0].union.AdminScatter = &buf36[0].resp br.Responses[i].Value = &buf36[0].union buf36 = buf36[1:] - case *RequestUnion_RecomputeStats: + case *RequestUnion_AddSstable: if buf37 == nil { - buf37 = make([]recomputeStatsResponseAlloc, counts[37]) + buf37 = make([]addSSTableResponseAlloc, counts[37]) } - buf37[0].union.RecomputeStats = &buf37[0].resp + buf37[0].union.AddSstable = &buf37[0].resp br.Responses[i].Value = &buf37[0].union buf37 = buf37[1:] - case *RequestUnion_Refresh: + case *RequestUnion_RecomputeStats: if buf38 == nil { - buf38 = make([]refreshResponseAlloc, counts[38]) + buf38 = make([]recomputeStatsResponseAlloc, counts[38]) } - buf38[0].union.Refresh = &buf38[0].resp + buf38[0].union.RecomputeStats = &buf38[0].resp br.Responses[i].Value = &buf38[0].union buf38 = buf38[1:] - case *RequestUnion_RefreshRange: + case *RequestUnion_Refresh: if buf39 == nil { - buf39 = make([]refreshRangeResponseAlloc, counts[39]) + buf39 = make([]refreshResponseAlloc, counts[39]) } - buf39[0].union.RefreshRange = &buf39[0].resp + buf39[0].union.Refresh = &buf39[0].resp br.Responses[i].Value = &buf39[0].union buf39 = buf39[1:] - case *RequestUnion_Subsume: + case *RequestUnion_RefreshRange: if buf40 == nil { - buf40 = make([]subsumeResponseAlloc, counts[40]) + buf40 = make([]refreshRangeResponseAlloc, counts[40]) } - buf40[0].union.Subsume = &buf40[0].resp + buf40[0].union.RefreshRange = &buf40[0].resp br.Responses[i].Value = &buf40[0].union buf40 = buf40[1:] - case *RequestUnion_RangeStats: + case *RequestUnion_Subsume: if buf41 == nil { - buf41 = make([]rangeStatsResponseAlloc, counts[41]) + buf41 = make([]subsumeResponseAlloc, counts[41]) } - buf41[0].union.RangeStats = &buf41[0].resp + buf41[0].union.Subsume = &buf41[0].resp br.Responses[i].Value = &buf41[0].union buf41 = buf41[1:] + case *RequestUnion_RangeStats: + if buf42 == nil { + buf42 = make([]rangeStatsResponseAlloc, counts[42]) + } + buf42[0].union.RangeStats = &buf42[0].resp + br.Responses[i].Value = &buf42[0].union + buf42 = buf42[1:] default: panic(fmt.Sprintf("unsupported request: %+v", r)) } diff --git a/pkg/roachpb/method.go b/pkg/roachpb/method.go index de777f195f42..37e56b3a8e18 100644 --- a/pkg/roachpb/method.go +++ b/pkg/roachpb/method.go @@ -64,6 +64,8 @@ const ( EndTransaction // AdminSplit is called to coordinate a split of a range. AdminSplit + // AdminUnsplit is called to remove the sticky bit of a manually split range. + AdminUnsplit // AdminMerge is called to coordinate a merge of two adjacent ranges. AdminMerge // AdminTransferLease is called to initiate a range lease transfer. diff --git a/pkg/roachpb/method_string.go b/pkg/roachpb/method_string.go index 0ac4100f475e..599c822f2f4f 100644 --- a/pkg/roachpb/method_string.go +++ b/pkg/roachpb/method_string.go @@ -20,41 +20,42 @@ func _() { _ = x[BeginTransaction-9] _ = x[EndTransaction-10] _ = x[AdminSplit-11] - _ = x[AdminMerge-12] - _ = x[AdminTransferLease-13] - _ = x[AdminChangeReplicas-14] - _ = x[AdminRelocateRange-15] - _ = x[HeartbeatTxn-16] - _ = x[GC-17] - _ = x[PushTxn-18] - _ = x[RecoverTxn-19] - _ = x[QueryTxn-20] - _ = x[QueryIntent-21] - _ = x[ResolveIntent-22] - _ = x[ResolveIntentRange-23] - _ = x[Merge-24] - _ = x[TruncateLog-25] - _ = x[RequestLease-26] - _ = x[TransferLease-27] - _ = x[LeaseInfo-28] - _ = x[ComputeChecksum-29] - _ = x[CheckConsistency-30] - _ = x[InitPut-31] - _ = x[WriteBatch-32] - _ = x[Export-33] - _ = x[Import-34] - _ = x[AdminScatter-35] - _ = x[AddSSTable-36] - _ = x[RecomputeStats-37] - _ = x[Refresh-38] - _ = x[RefreshRange-39] - _ = x[Subsume-40] - _ = x[RangeStats-41] + _ = x[AdminUnsplit-12] + _ = x[AdminMerge-13] + _ = x[AdminTransferLease-14] + _ = x[AdminChangeReplicas-15] + _ = x[AdminRelocateRange-16] + _ = x[HeartbeatTxn-17] + _ = x[GC-18] + _ = x[PushTxn-19] + _ = x[RecoverTxn-20] + _ = x[QueryTxn-21] + _ = x[QueryIntent-22] + _ = x[ResolveIntent-23] + _ = x[ResolveIntentRange-24] + _ = x[Merge-25] + _ = x[TruncateLog-26] + _ = x[RequestLease-27] + _ = x[TransferLease-28] + _ = x[LeaseInfo-29] + _ = x[ComputeChecksum-30] + _ = x[CheckConsistency-31] + _ = x[InitPut-32] + _ = x[WriteBatch-33] + _ = x[Export-34] + _ = x[Import-35] + _ = x[AdminScatter-36] + _ = x[AddSSTable-37] + _ = x[RecomputeStats-38] + _ = x[Refresh-39] + _ = x[RefreshRange-40] + _ = x[Subsume-41] + _ = x[RangeStats-42] } -const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeScanReverseScanBeginTransactionEndTransactionAdminSplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportImportAdminScatterAddSSTableRecomputeStatsRefreshRefreshRangeSubsumeRangeStats" +const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeScanReverseScanBeginTransactionEndTransactionAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportImportAdminScatterAddSSTableRecomputeStatsRefreshRefreshRangeSubsumeRangeStats" -var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 60, 71, 87, 101, 111, 121, 139, 158, 176, 188, 190, 197, 207, 215, 226, 239, 257, 262, 273, 285, 298, 307, 322, 338, 345, 355, 361, 367, 379, 389, 403, 410, 422, 429, 439} +var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 60, 71, 87, 101, 111, 123, 133, 151, 170, 188, 200, 202, 209, 219, 227, 238, 251, 269, 274, 285, 297, 310, 319, 334, 350, 357, 367, 373, 379, 391, 401, 415, 422, 434, 441, 451} func (i Method) String() string { if i < 0 || i >= Method(len(_Method_index)-1) { diff --git a/pkg/sql/expand_plan.go b/pkg/sql/expand_plan.go index d2d478bd4a37..68e7cbcf1599 100644 --- a/pkg/sql/expand_plan.go +++ b/pkg/sql/expand_plan.go @@ -334,6 +334,9 @@ func doExpandPlan( case *splitNode: n.rows, err = doExpandPlan(ctx, p, noParams, n.rows) + case *unsplitNode: + n.rows, err = doExpandPlan(ctx, p, noParams, n.rows) + case *relocateNode: n.rows, err = doExpandPlan(ctx, p, noParams, n.rows) @@ -853,6 +856,9 @@ func (p *planner) simplifyOrderings(plan planNode, usefulOrdering sqlbase.Column case *splitNode: n.rows = p.simplifyOrderings(n.rows, nil) + case *unsplitNode: + n.rows = p.simplifyOrderings(n.rows, nil) + case *relocateNode: n.rows = p.simplifyOrderings(n.rows, nil) diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index f3f7003b0c14..f2b83b0f88ea 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -677,6 +677,12 @@ ALTER TABLE nonexistent SPLIT AT VALUES (42) statement error pgcode 42P01 relation "nonexistent" does not exist ALTER INDEX nonexistent@noindex SPLIT AT VALUES (42) +statement error pgcode 42P01 relation "nonexistent" does not exist +ALTER TABLE nonexistent UNSPLIT AT VALUES (42) + +statement error pgcode 42P01 relation "nonexistent" does not exist +ALTER INDEX nonexistent@noindex UNSPLIT AT VALUES (42) + user root statement ok @@ -688,6 +694,9 @@ ALTER TABLE privsview ADD d INT statement error pgcode 42809 "privsview" is not a table ALTER TABLE privsview SPLIT AT VALUES (42) +statement error pgcode 42809 "privsview" is not a table +ALTER TABLE privsview UNSPLIT AT VALUES (42) + # Verify that impure defaults are evaluated separately on each row # (#14352) statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index c5a034f2a4e7..784249ccdd5a 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -342,6 +342,19 @@ SELECT start_pretty, end_pretty FROM crdb_internal.ranges_no_leases WHERE manual start_pretty end_pretty /Table/56/1/2 /Max +statement ok +ALTER TABLE foo UNSPLIT AT VALUES(2) + +query TT colnames +SELECT start_pretty, end_pretty FROM crdb_internal.ranges WHERE manual_split_time IS NOT NULL +---- +start_pretty end_pretty + +query TT colnames +SELECT start_pretty, end_pretty FROM crdb_internal.ranges_no_leases WHERE manual_split_time IS NOT NULL +---- +start_pretty end_pretty + # Make sure that the cluster id isn't unset. query B select crdb_internal.cluster_id() != '00000000-0000-0000-0000-000000000000' FROM foo diff --git a/pkg/sql/opt_filters.go b/pkg/sql/opt_filters.go index 5dd71744a3a4..96863f661f02 100644 --- a/pkg/sql/opt_filters.go +++ b/pkg/sql/opt_filters.go @@ -334,6 +334,11 @@ func (p *planner) propagateFilters( return plan, extraFilter, err } + case *unsplitNode: + if n.rows, err = p.triggerFilterPropagation(ctx, n.rows); err != nil { + return plan, extraFilter, err + } + case *relocateNode: if n.rows, err = p.triggerFilterPropagation(ctx, n.rows); err != nil { return plan, extraFilter, err diff --git a/pkg/sql/opt_limits.go b/pkg/sql/opt_limits.go index f5349c7e5f52..734f48fcbd9b 100644 --- a/pkg/sql/opt_limits.go +++ b/pkg/sql/opt_limits.go @@ -184,6 +184,9 @@ func (p *planner) applyLimit(plan planNode, numRows int64, soft bool) { case *splitNode: p.setUnlimited(n.rows) + case *unsplitNode: + p.setUnlimited(n.rows) + case *relocateNode: p.setUnlimited(n.rows) diff --git a/pkg/sql/opt_needed.go b/pkg/sql/opt_needed.go index 3b1eabfcf79a..73b8e2b29f3e 100644 --- a/pkg/sql/opt_needed.go +++ b/pkg/sql/opt_needed.go @@ -234,6 +234,9 @@ func setNeededColumns(plan planNode, needed []bool) { case *splitNode: setNeededColumns(n.rows, allColumns(n.rows)) + case *unsplitNode: + setNeededColumns(n.rows, allColumns(n.rows)) + case *relocateNode: setNeededColumns(n.rows, allColumns(n.rows)) diff --git a/pkg/sql/parser/parse_test.go b/pkg/sql/parser/parse_test.go index 10d94408116a..4ade74df8ce0 100644 --- a/pkg/sql/parser/parse_test.go +++ b/pkg/sql/parser/parse_test.go @@ -1111,6 +1111,17 @@ func TestParse(t *testing.T) { {`ALTER INDEX "primary" SPLIT AT VALUES (2)`}, {`ALTER INDEX public.public."primary" SPLIT AT VALUES (2)`}, + {`ALTER TABLE a UNSPLIT AT VALUES (1)`}, + {`EXPLAIN ALTER TABLE a UNSPLIT AT VALUES (1)`}, + {`ALTER TABLE a UNSPLIT AT SELECT * FROM t`}, + {`ALTER TABLE d.a UNSPLIT AT VALUES ('b', 2)`}, + {`ALTER INDEX a@i UNSPLIT AT VALUES (1)`}, + {`ALTER INDEX d.a@i UNSPLIT AT VALUES (2)`}, + {`ALTER INDEX i UNSPLIT AT VALUES (1)`}, + {`ALTER INDEX d.i UNSPLIT AT VALUES (2)`}, + {`ALTER INDEX "primary" UNSPLIT AT VALUES (2)`}, + {`ALTER INDEX public.public."primary" UNSPLIT AT VALUES (2)`}, + {`ALTER TABLE a EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 1)`}, {`EXPLAIN ALTER TABLE a EXPERIMENTAL_RELOCATE TABLE b`}, {`ALTER TABLE a EXPERIMENTAL_RELOCATE SELECT * FROM t`}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 99a456852ff5..1e7e4975897e 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -555,7 +555,7 @@ func newNameFromStr(s string) *tree.Name { %token TRUNCATE TRUSTED TYPE %token TRACING -%token UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN UNLOGGED +%token UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN UNLOGGED UNSPLIT %token UPDATE UPSERT USE USER USERS USING UUID %token VALID VALIDATE VALUE VALUES VARBIT VARCHAR VARIADIC VIEW VARYING VIRTUAL @@ -602,6 +602,7 @@ func newNameFromStr(s string) *tree.Name { // ALTER TABLE %type alter_onetable_stmt %type alter_split_stmt +%type alter_unsplit_stmt %type alter_rename_table_stmt %type alter_scatter_stmt %type alter_relocate_stmt @@ -619,6 +620,7 @@ func newNameFromStr(s string) *tree.Name { %type alter_oneindex_stmt %type alter_scatter_index_stmt %type alter_split_index_stmt +%type alter_unsplit_index_stmt %type alter_rename_index_stmt %type alter_relocate_index_stmt %type alter_relocate_index_lease_stmt @@ -1107,6 +1109,7 @@ alter_ddl_stmt: // ALTER TABLE ... RENAME [COLUMN] TO // ALTER TABLE ... VALIDATE CONSTRAINT // ALTER TABLE ... SPLIT AT +// ALTER TABLE ... UNSPLIT AT // ALTER TABLE ... SCATTER [ FROM ( ) TO ( ) ] // ALTER TABLE ... INJECT STATISTICS ... (experimental) // ALTER TABLE ... PARTITION BY RANGE ( ) ( ) @@ -1133,6 +1136,7 @@ alter_table_stmt: | alter_relocate_stmt | alter_relocate_lease_stmt | alter_split_stmt +| alter_unsplit_stmt | alter_scatter_stmt | alter_zone_table_stmt | alter_rename_table_stmt @@ -1225,6 +1229,7 @@ alter_range_stmt: // Commands: // ALTER INDEX ... RENAME TO // ALTER INDEX ... SPLIT AT +// ALTER INDEX ... UNSPLIT AT // ALTER INDEX ... SCATTER [ FROM ( ) TO ( ) ] // ALTER PARTITION ... OF INDEX ... CONFIGURE ZONE // @@ -1240,6 +1245,7 @@ alter_index_stmt: | alter_relocate_index_stmt | alter_relocate_index_lease_stmt | alter_split_index_stmt +| alter_unsplit_index_stmt | alter_scatter_index_stmt | alter_rename_index_stmt | alter_zone_index_stmt @@ -1283,6 +1289,22 @@ alter_split_index_stmt: $$.val = &tree.Split{TableOrIndex: $3.tableIndexName(), Rows: $6.slct()} } +alter_unsplit_stmt: + ALTER TABLE table_name UNSPLIT AT select_stmt + { + name := $3.unresolvedObjectName().ToTableName() + $$.val = &tree.Unsplit{ + TableOrIndex: tree.TableIndexName{Table: name}, + Rows: $6.slct(), + } + } + +alter_unsplit_index_stmt: + ALTER INDEX table_index_name UNSPLIT AT select_stmt + { + $$.val = &tree.Unsplit{TableOrIndex: $3.tableIndexName(), Rows: $6.slct()} + } + relocate_kw: TESTING_RELOCATE | EXPERIMENTAL_RELOCATE @@ -9059,6 +9081,7 @@ unreserved_keyword: | UNCOMMITTED | UNKNOWN | UNLOGGED +| UNSPLIT | UPDATE | UPSERT | UUID diff --git a/pkg/sql/plan.go b/pkg/sql/plan.go index 671d6987a138..21e4659bc8f3 100644 --- a/pkg/sql/plan.go +++ b/pkg/sql/plan.go @@ -219,6 +219,7 @@ var _ planNode = &showFingerprintsNode{} var _ planNode = &showTraceNode{} var _ planNode = &sortNode{} var _ planNode = &splitNode{} +var _ planNode = &unsplitNode{} var _ planNode = &truncateNode{} var _ planNode = &unaryNode{} var _ planNode = &unionNode{} @@ -710,6 +711,8 @@ func (p *planner) newPlan( return p.ShowFingerprints(ctx, n) case *tree.Split: return p.Split(ctx, n) + case *tree.Unsplit: + return p.Unsplit(ctx, n) case *tree.Truncate: return p.Truncate(ctx, n) case *tree.UnionClause: @@ -811,6 +814,8 @@ func (p *planner) doPrepare(ctx context.Context, stmt tree.Statement) (planNode, return p.ShowZoneConfig(ctx, n) case *tree.Split: return p.Split(ctx, n) + case *tree.Unsplit: + return p.Unsplit(ctx, n) case *tree.Truncate: return p.Truncate(ctx, n) case *tree.Relocate: diff --git a/pkg/sql/plan_columns.go b/pkg/sql/plan_columns.go index 2d59d36ec358..ea2f12b7dd16 100644 --- a/pkg/sql/plan_columns.go +++ b/pkg/sql/plan_columns.go @@ -111,6 +111,8 @@ func getPlanColumns(plan planNode, mut bool) sqlbase.ResultColumns { return n.getColumns(mut, showFingerprintsColumns) case *splitNode: return n.getColumns(mut, splitNodeColumns) + case *unsplitNode: + return n.getColumns(mut, unsplitNodeColumns) case *showTraceReplicaNode: return n.getColumns(mut, sqlbase.ShowReplicaTraceColumns) case *sequenceSelectNode: diff --git a/pkg/sql/plan_physical_props.go b/pkg/sql/plan_physical_props.go index d9bfc8fd9497..64eef62d6fcb 100644 --- a/pkg/sql/plan_physical_props.go +++ b/pkg/sql/plan_physical_props.go @@ -142,6 +142,7 @@ func planPhysicalProps(plan planNode) physicalProps { case *showTraceNode: case *showTraceReplicaNode: case *splitNode: + case *unsplitNode: case *truncateNode: case *unaryNode: case *valuesNode: diff --git a/pkg/sql/sem/tree/split.go b/pkg/sql/sem/tree/split.go index d21ce931972c..886c61f02694 100644 --- a/pkg/sql/sem/tree/split.go +++ b/pkg/sql/sem/tree/split.go @@ -35,6 +35,27 @@ func (node *Split) Format(ctx *FmtCtx) { ctx.FormatNode(node.Rows) } +// Unsplit represents an `ALTER TABLE/INDEX .. UNSPLIT AT ..` statement. +type Unsplit struct { + TableOrIndex TableIndexName + // Each row contains values for the columns in the PK or index (or a prefix + // of the columns). + Rows *Select +} + +// Format implements the NodeFormatter interface. +func (node *Unsplit) Format(ctx *FmtCtx) { + ctx.WriteString("ALTER ") + if node.TableOrIndex.Index != "" { + ctx.WriteString("INDEX ") + } else { + ctx.WriteString("TABLE ") + } + ctx.FormatNode(&node.TableOrIndex) + ctx.WriteString(" UNSPLIT AT ") + ctx.FormatNode(node.Rows) +} + // Relocate represents an `ALTER TABLE/INDEX .. EXPERIMENTAL_RELOCATE ..` // statement. type Relocate struct { diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index cdb4bb2e3875..96eeca4e076a 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -109,7 +109,7 @@ func CanWriteData(stmt Statement) bool { case *CopyFrom, *Import, *Restore: return true // CockroachDB extensions. - case *Split, *Relocate, *Scatter: + case *Split, *Unsplit, *Relocate, *Scatter: return true } return false @@ -802,6 +802,12 @@ func (*Split) StatementType() StatementType { return Rows } // StatementTag returns a short string identifying the type of statement. func (*Split) StatementTag() string { return "SPLIT" } +// StatementType implements the Statement interface. +func (*Unsplit) StatementType() StatementType { return Rows } + +// StatementTag returns a short string identifying the type of statement. +func (*Unsplit) StatementTag() string { return "UNSPLIT" } + // StatementType implements the Statement interface. func (*Truncate) StatementType() StatementType { return Ack } @@ -930,6 +936,7 @@ func (n *ShowVar) String() string { return AsString(n) } func (n *ShowZoneConfig) String() string { return AsString(n) } func (n *ShowFingerprints) String() string { return AsString(n) } func (n *Split) String() string { return AsString(n) } +func (n *Unsplit) String() string { return AsString(n) } func (n *Truncate) String() string { return AsString(n) } func (n *UnionClause) String() string { return AsString(n) } func (n *Update) String() string { return AsString(n) } diff --git a/pkg/sql/unsplit.go b/pkg/sql/unsplit.go new file mode 100644 index 000000000000..44b9ca9b23aa --- /dev/null +++ b/pkg/sql/unsplit.go @@ -0,0 +1,145 @@ +// Copyright 2019 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/pkg/errors" +) + +type unsplitNode struct { + optColumnsSlot + + tableDesc *sqlbase.TableDescriptor + index *sqlbase.IndexDescriptor + rows planNode + run unsplitRun +} + +// Unsplit executes a KV unsplit. +// Privileges: INSERT on table. +func (p *planner) Unsplit(ctx context.Context, n *tree.Unsplit) (planNode, error) { + tableDesc, index, err := p.getTableAndIndex(ctx, &n.TableOrIndex, privilege.INSERT) + if err != nil { + return nil, err + } + // Calculate the desired types for the select statement. It is OK if the + // select statement returns fewer columns (the relevant prefix is used). + desiredTypes := make([]*types.T, len(index.ColumnIDs)) + for i, colID := range index.ColumnIDs { + c, err := tableDesc.FindColumnByID(colID) + if err != nil { + return nil, err + } + desiredTypes[i] = &c.Type + } + + // Create the plan for the unsplit rows source. + rows, err := p.newPlan(ctx, n.Rows, desiredTypes) + if err != nil { + return nil, err + } + + cols := planColumns(rows) + if len(cols) == 0 { + return nil, errors.Errorf("no columns in UNSPLIT AT data") + } + if len(cols) > len(index.ColumnIDs) { + return nil, errors.Errorf("too many columns in UNSPLIT AT data") + } + for i := range cols { + if !cols[i].Typ.Equivalent(desiredTypes[i]) { + return nil, errors.Errorf( + "UNSPLIT AT data column %d (%s) must be of type %s, not type %s", + i+1, index.ColumnNames[i], desiredTypes[i], cols[i].Typ, + ) + } + } + + return &unsplitNode{ + tableDesc: tableDesc.TableDesc(), + index: index, + rows: rows, + }, nil +} + +var unsplitNodeColumns = sqlbase.ResultColumns{ + { + Name: "key", + Typ: types.Bytes, + }, + { + Name: "pretty", + Typ: types.String, + }, +} + +// unsplitRun contains the run-time state of unsplitNode during local execution. +type unsplitRun struct { + lastUnsplitKey []byte +} + +func (n *unsplitNode) startExec(params runParams) error { + stickyBitEnabled := params.EvalContext().Settings.Version.IsActive(cluster.VersionStickyBit) + // TODO(jeffreyxiao): Remove this error in v20.1. + if !stickyBitEnabled { + return pgerror.Newf(pgerror.CodeObjectNotInPrerequisiteStateError, + `UNSPLIT AT requires all nodes to be upgraded to %s`, + cluster.VersionByKey(cluster.VersionCreateStats), + ) + } + return nil +} + +func (n *unsplitNode) Next(params runParams) (bool, error) { + if ok, err := n.rows.Next(params); err != nil || !ok { + return ok, err + } + + row := n.rows.Values() + rowKey, err := getRowKey(n.tableDesc, n.index, row) + if err != nil { + return false, err + } + + if err := params.extendedEvalCtx.ExecCfg.DB.AdminUnsplit(params.ctx, rowKey); err != nil { + ctx := tree.NewFmtCtx(tree.FmtSimple) + row.Format(ctx) + return false, pgerror.Wrapf(err, pgerror.CodeDataExceptionError, "could not UNSPLIT AT %s", ctx) + } + + n.run.lastUnsplitKey = rowKey + + return true, nil +} + +func (n *unsplitNode) Values() tree.Datums { + return tree.Datums{ + tree.NewDBytes(tree.DBytes(n.run.lastUnsplitKey)), + tree.NewDString(keys.PrettyPrint(nil /* valDirs */, n.run.lastUnsplitKey)), + } +} + +func (n *unsplitNode) Close(ctx context.Context) { + n.rows.Close(ctx) +} diff --git a/pkg/sql/unsplit_test.go b/pkg/sql/unsplit_test.go new file mode 100644 index 000000000000..a212fce08927 --- /dev/null +++ b/pkg/sql/unsplit_test.go @@ -0,0 +1,153 @@ +// Copyright 2019 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package sql_test + +import ( + "context" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +func TestUnsplitAt(t *testing.T) { + defer leaktest.AfterTest(t)() + + params, _ := tests.CreateTestServerParams() + s, db, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(context.TODO()) + + r := sqlutils.MakeSQLRunner(db) + + r.Exec(t, "CREATE DATABASE d") + r.Exec(t, `CREATE TABLE d.t ( + i INT, + s STRING, + PRIMARY KEY (i, s), + INDEX s_idx (s) + )`) + r.Exec(t, `CREATE TABLE d.i (k INT PRIMARY KEY)`) + + // Create initial splits + splitStmts := []string{ + "ALTER TABLE d.t SPLIT AT VALUES (2, 'b'), (3, 'c'), (4, 'd'), (5, 'd'), (6, 'e'), (7, 'f')", + "ALTER TABLE d.t SPLIT AT VALUES (10)", + "ALTER TABLE d.i SPLIT AT VALUES (1), (8)", + "ALTER INDEX d.t@s_idx SPLIT AT VALUES ('f')", + } + + for _, splitStmt := range splitStmts { + var key roachpb.Key + var pretty string + if err := db.QueryRow(splitStmt).Scan(&key, &pretty); err != nil { + t.Fatalf("unexpected error setting up test: %s", err) + } + } + + tests := []struct { + in string + error string + args []interface{} + }{ + { + in: "ALTER TABLE d.t UNSPLIT AT VALUES (2, 'b')", + }, + { + in: "ALTER TABLE d.t UNSPLIT AT VALUES (3, 'c'), (4, 'd')", + }, + { + in: "ALTER TABLE d.t UNSPLIT AT SELECT 5, 'd'", + }, + { + in: "ALTER TABLE d.t UNSPLIT AT SELECT * FROM (VALUES (6, 'e'), (7, 'f')) AS a", + }, + { + in: "ALTER TABLE d.t UNSPLIT AT VALUES (10)", + }, + { + in: "ALTER TABLE d.i UNSPLIT AT VALUES ($1)", + args: []interface{}{8}, + }, + { + in: "ALTER TABLE d.i UNSPLIT AT VALUES ((SELECT 1))", + }, + { + in: "ALTER INDEX d.t@s_idx UNSPLIT AT VALUES ('f')", + }, + { + in: "ALTER TABLE d.t UNSPLIT AT VALUES (1, 'non-existent')", + error: "could not UNSPLIT AT (1, 'non-existent')", + }, + { + in: "ALTER TABLE d.t UNSPLIT AT VALUES ('c', 3)", + error: "could not parse \"c\" as type int", + }, + { + in: "ALTER TABLE d.t UNSPLIT AT VALUES (i, s)", + error: `column "i" does not exist`, + }, + { + in: "ALTER INDEX d.t@not_present UNSPLIT AT VALUES ('g')", + error: `index "not_present" does not exist`, + }, + { + in: "ALTER TABLE d.i UNSPLIT AT VALUES (avg(1::float))", + error: "aggregate functions are not allowed in VALUES", + }, + { + in: "ALTER TABLE d.i UNSPLIT AT VALUES ($1)", + error: "no value provided for placeholder: $1", + }, + { + in: "ALTER TABLE d.i UNSPLIT AT VALUES ($1)", + args: []interface{}{"blah"}, + error: "error in argument for $1: strconv.ParseInt", + }, + { + in: "ALTER TABLE d.i UNSPLIT AT VALUES ($1::string)", + args: []interface{}{"1"}, + error: "UNSPLIT AT data column 1 (k) must be of type int, not type string", + }, + } + + for _, tt := range tests { + var key roachpb.Key + var pretty string + err := db.QueryRow(tt.in, tt.args...).Scan(&key, &pretty) + if err != nil && tt.error == "" { + t.Fatalf("%s: unexpected error: %s", tt.in, err) + } else if tt.error != "" && err == nil { + t.Fatalf("%s: expected error: %s", tt.in, tt.error) + } else if err != nil && tt.error != "" { + if !strings.Contains(err.Error(), tt.error) { + t.Fatalf("%s: unexpected error: %s", tt.in, err) + } + } else { + // Successful unsplit, verify it happened. + rng, err := s.(*server.TestServer).LookupRange(key) + if err != nil { + t.Fatal(err) + } + if rng.StickyBit != nil { + t.Fatalf("%s: expected range sticky bit to be nil, got %s", tt.in, pretty) + } + } + } +} diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 1be7964fec67..702d1688356d 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -423,6 +423,9 @@ func (v *planVisitor) visitInternal(plan planNode, name string) { case *splitNode: n.rows = v.visit(n.rows) + case *unsplitNode: + n.rows = v.visit(n.rows) + case *relocateNode: n.rows = v.visit(n.rows) @@ -791,6 +794,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&showTraceReplicaNode{}): "replica trace", reflect.TypeOf(&sortNode{}): "sort", reflect.TypeOf(&splitNode{}): "split", + reflect.TypeOf(&unsplitNode{}): "unsplit", reflect.TypeOf(&spoolNode{}): "spool", reflect.TypeOf(&truncateNode{}): "truncate", reflect.TypeOf(&unaryNode{}): "emptyrow", diff --git a/pkg/storage/client_merge_test.go b/pkg/storage/client_merge_test.go index 787f6931b02c..6586dd75d4ad 100644 --- a/pkg/storage/client_merge_test.go +++ b/pkg/storage/client_merge_test.go @@ -3431,35 +3431,13 @@ func TestMergeQueue(t *testing.T) { store.MustForceMergeScanAndProcess() verifyUnmerged(t) - // TODO(jeffreyxiao): Use same mechanism as ALTER TABLE ... UNSPLIT AT - // does when it is added. // Delete sticky bit and verify that merge occurs. - rhsDescKey := keys.RangeDescriptorKey(rhsStartKey) - var rhsDesc roachpb.RangeDescriptor - if err := store.DB().GetProto(ctx, rhsDescKey, &rhsDesc); err != nil { - t.Fatal(err) + unsplitArgs := &roachpb.AdminUnsplitRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: rhsStartKey.AsRawKey(), + }, } - rhsDesc.StickyBit = nil - if err := store.DB().Txn(ctx, func(ctx context.Context, txn *client.Txn) error { - b := txn.NewBatch() - marshalledDesc, err := protoutil.Marshal(&rhsDesc) - if err != nil { - return err - } - b.Put(rhsDescKey, marshalledDesc) - b.Put(keys.RangeMetaKey(rhsDesc.EndKey).AsRawKey(), marshalledDesc) - // End the transaction manually in order to provide a sticky bit trigger. - // Note that this hack will be removed. See above TODO. - b.AddRawRequest(&roachpb.EndTransactionRequest{ - Commit: true, - InternalCommitTrigger: &roachpb.InternalCommitTrigger{ - StickyBitTrigger: &roachpb.StickyBitTrigger{ - StickyBit: nil, - }, - }, - }) - return txn.Run(ctx, b) - }); err != nil { + if _, err := client.SendWrapped(ctx, store.DB().NonTransactionalSender(), unsplitArgs); err != nil { t.Fatal(err) } store.MustForceMergeScanAndProcess() diff --git a/pkg/storage/replica.go b/pkg/storage/replica.go index 32c1f771624e..92a24c3a2091 100644 --- a/pkg/storage/replica.go +++ b/pkg/storage/replica.go @@ -1256,6 +1256,11 @@ func (r *Replica) executeAdminBatch( reply, pErr = r.AdminSplit(ctx, *tArgs, "manual") resp = &reply + case *roachpb.AdminUnsplitRequest: + var reply roachpb.AdminUnsplitResponse + reply, pErr = r.AdminUnsplit(ctx, *tArgs, "manual") + resp = &reply + case *roachpb.AdminMergeRequest: var reply roachpb.AdminMergeResponse reply, pErr = r.AdminMerge(ctx, *tArgs, "manual") diff --git a/pkg/storage/replica_command.go b/pkg/storage/replica_command.go index f155f43ed8b6..62118d1aa2e3 100644 --- a/pkg/storage/replica_command.go +++ b/pkg/storage/replica_command.go @@ -380,6 +380,56 @@ func (r *Replica) adminSplitWithDescriptor( return reply, nil } +// AdminUnsplit removes the sticky bit of the range specified by the +// args.Key. +func (r *Replica) AdminUnsplit( + ctx context.Context, args roachpb.AdminUnsplitRequest, reason string, +) (roachpb.AdminUnsplitResponse, *roachpb.Error) { + // TODO(jeffreyxiao): Have a retry loop for ConditionalFailed errors similar + // to AdminSplit + var reply roachpb.AdminUnsplitResponse + + desc := *r.Desc() + if !bytes.Equal(desc.StartKey.AsRawKey(), args.Header().Key) { + return reply, roachpb.NewErrorf("key %s is not the start of a range", args.Header().Key) + } + + // If the range's sticky bit is not set, we treat the unsplit command + // as a no-op and return success instead of throwing an error. + if desc.StickyBit == nil { + return reply, nil + } + + if err := r.store.DB().Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + b := txn.NewBatch() + newDesc := desc + newDesc.StickyBit = nil + descKey := keys.RangeDescriptorKey(newDesc.StartKey) + + if err := updateRangeDescriptor(b, descKey, &desc, &newDesc); err != nil { + return err + } + if err := updateRangeAddressing(b, &newDesc); err != nil { + return err + } + // End the transaction manually in order to provide a sticky bit trigger. + b.AddRawRequest(&roachpb.EndTransactionRequest{ + Commit: true, + InternalCommitTrigger: &roachpb.InternalCommitTrigger{ + StickyBitTrigger: &roachpb.StickyBitTrigger{ + // Setting StickyBit to nil unsets the sticky bit. + StickyBit: nil, + }, + }, + }) + return txn.Run(ctx, b) + }); err != nil { + return reply, roachpb.NewErrorf("unsplit at key %s failed: %s", args.Header().Key, err) + } + + return reply, nil +} + // AdminMerge extends this range to subsume the range that comes next // in the key space. The merge is performed inside of a distributed // transaction which writes the left hand side range descriptor (the