From 217031a0c342a0f0adfdd7c893163bc3de9fbbd9 Mon Sep 17 00:00:00 2001 From: Shen Li Date: Mon, 16 Apr 2018 22:57:57 +0800 Subject: [PATCH] Merge master to 2.0 to release rc5 (#6288) --- Gopkg.lock | 2 +- Makefile | 18 +- README.md | 24 +- ast/ast.go | 4 +- ast/ddl.go | 18 + ast/misc.go | 1 + cmd/benchdb/main.go | 2 +- config/config.go | 8 +- config/config.toml.example | 15 +- ddl/callback_test.go | 15 +- ddl/column.go | 19 +- ddl/column_change_test.go | 7 +- ddl/column_test.go | 5 +- ddl/ddl.go | 3 +- ddl/ddl_api.go | 118 +- ddl/ddl_db_change_test.go | 199 +- ddl/ddl_db_test.go | 114 +- ddl/ddl_test.go | 1 - ddl/ddl_worker.go | 147 +- ddl/ddl_worker_test.go | 182 +- ddl/delete_range.go | 10 + ddl/fail_test.go | 2 - ddl/index.go | 3 +- ddl/index_change_test.go | 7 +- ddl/reorg_test.go | 3 - ddl/schema_test.go | 16 +- ddl/stat_test.go | 10 +- ddl/table.go | 21 + ddl/table_test.go | 4 +- ddl/util/util.go | 2 +- distsql/distsql.go | 87 +- distsql/distsql_test.go | 12 +- distsql/request_builder.go | 8 +- distsql/stream.go | 29 +- domain/domain.go | 29 +- executor/adapter.go | 8 +- executor/admin.go | 273 +- executor/admin_test.go | 184 + executor/aggregate.go | 18 +- executor/aggregate_test.go | 25 +- executor/analyze.go | 8 +- executor/builder.go | 54 +- executor/checksum.go | 4 +- executor/ddl.go | 19 +- executor/ddl_test.go | 6 +- executor/distsql.go | 56 +- executor/distsql_test.go | 8 +- executor/executor.go | 182 +- executor/executor_pkg_test.go | 19 +- executor/executor_test.go | 42 +- executor/explain.go | 4 +- executor/grant.go | 4 +- executor/grant_test.go | 14 +- executor/index_lookup_join.go | 17 +- executor/index_lookup_join_test.go | 39 + executor/join.go | 120 +- executor/join_result_generators.go | 364 +- executor/join_test.go | 37 +- executor/load_stats.go | 4 +- executor/merge_join.go | 40 +- executor/pkg_test.go | 11 +- executor/prepared.go | 12 +- executor/prepared_test.go | 4 +- executor/revoke.go | 4 +- executor/set.go | 7 +- executor/show.go | 25 +- executor/show_stats.go | 10 +- executor/show_test.go | 15 +- executor/simple.go | 4 +- executor/simple_test.go | 8 +- executor/sort.go | 14 +- executor/union_scan.go | 32 +- executor/write.go | 588 +- executor/write_test.go | 146 + expression/bench_test.go | 13 + expression/builtin.go | 16 + expression/builtin_arithmetic.go | 107 + expression/builtin_cast.go | 294 + expression/builtin_compare.go | 614 +- expression/builtin_control.go | 120 + expression/builtin_encryption.go | 60 + expression/builtin_info.go | 60 + expression/builtin_json.go | 60 + expression/builtin_like.go | 18 + expression/builtin_math.go | 276 + expression/builtin_miscellaneous.go | 114 + expression/builtin_op.go | 150 + expression/builtin_other.go | 114 + expression/builtin_string.go | 379 + expression/builtin_time.go | 661 +- expression/builtin_time_test.go | 11 +- expression/constant.go | 7 +- expression/evaluator_test.go | 2 +- expression/expr_to_pb_test.go | 2 +- expression/integration_test.go | 22 +- expression/scalar_function.go | 33 +- expression/typeinfer_test.go | 33 +- kv/txn.go | 53 +- meta/meta.go | 54 +- meta/meta_test.go | 10 + metrics/server.go | 9 + metrics/stats.go | 18 + model/ddl.go | 115 +- model/model.go | 44 + model/model_test.go | 34 + mysql/const_test.go | 4 +- parser/misc.go | 1 + parser/parser.y | 119 +- parser/parser_test.go | 11 +- plan/cache.go | 6 - plan/cbo_test.go | 41 + plan/common_plans.go | 10 + plan/gen_physical_plans.go | 32 +- plan/logical_plan_builder.go | 12 +- plan/logical_plan_test.go | 3 + plan/optimizer.go | 2 +- plan/physical_plan_builder.go | 64 +- plan/physical_plan_test.go | 6 +- plan/plan.go | 8 +- plan/planbuilder.go | 14 +- plan/predicate_push_down.go | 10 +- plan/property_cols_prune.go | 21 +- plan/stats.go | 55 +- privilege/privileges/cache.go | 2 +- server/conn.go | 6 +- server/driver.go | 2 +- server/driver_tidb.go | 4 +- server/http_handler.go | 71 + server/http_handler_test.go | 37 +- server/http_status.go | 1 + server/statistics_handler_test.go | 2 +- server/tidb_test.go | 4 +- session/bench_test.go | 2 +- session/bootstrap.go | 47 +- session/bootstrap_test.go | 22 +- session/isolation_test.go | 4 +- session/session.go | 25 +- session/session_test.go | 75 +- session/tidb.go | 2 +- session/tidb_test.go | 2 +- sessionctx/variable/session.go | 73 +- sessionctx/variable/sysvar.go | 6 + sessionctx/variable/tidb_vars.go | 100 +- sessionctx/variable/varsutil.go | 2 +- sessionctx/variable/varsutil_test.go | 28 + statistics/boostrap.go | 8 +- statistics/cmsketch.go | 30 +- statistics/ddl.go | 12 +- statistics/ddl_test.go | 20 +- statistics/dump.go | 2 +- statistics/dump_test.go | 2 +- statistics/feedback.go | 233 +- statistics/feedback_test.go | 54 +- statistics/handle.go | 21 +- statistics/handle_test.go | 89 +- statistics/histogram.go | 44 +- statistics/sample.go | 2 +- statistics/selectivity.go | 68 +- statistics/selectivity_test.go | 10 + statistics/statistics_test.go | 19 +- statistics/table.go | 70 +- statistics/update.go | 184 +- statistics/update_test.go | 83 +- store/mockstore/mocktikv/analyze.go | 11 +- store/mockstore/mocktikv/cluster.go | 8 +- store/mockstore/mocktikv/cluster_test.go | 2 +- store/mockstore/mocktikv/cop_handler_dag.go | 5 - store/mockstore/mocktikv/mock_tikv_test.go | 58 +- store/mockstore/mocktikv/mvcc.go | 415 +- store/mockstore/mocktikv/mvcc_leveldb.go | 96 +- store/mockstore/mocktikv/rpc.go | 38 +- store/store_test.go | 18 +- store/tikv/2pc_test.go | 3 +- store/tikv/client.go | 7 +- store/tikv/coprocessor.go | 176 +- store/tikv/gcworker/gc_worker.go | 2 +- store/tikv/latch/latch.go | 199 + store/tikv/latch/latch_test.go | 173 + store/tikv/rawkv.go | 79 + store/tikv/rawkv_test.go | 89 +- store/tikv/region_cache_test.go | 5 +- store/tikv/region_request_test.go | 20 +- store/tikv/sql_fail_test.go | 4 +- store/tikv/tikvrpc/tikvrpc.go | 15 + structure/list.go | 20 + structure/structure_test.go | 12 +- table/column.go | 9 +- table/column_test.go | 10 + table/tables/index.go | 5 +- table/tables/tables.go | 50 +- table/tables/tables_test.go | 6 +- tidb-server/main.go | 12 +- types/datum.go | 6 +- util/admin/admin.go | 2 +- util/prefix_helper_test.go | 4 +- util/printer/printer.go | 2 +- util/processinfo.go | 1 + util/ranger/ranger_test.go | 2 +- .../kvproto/pkg/coprocessor/coprocessor.pb.go | 130 +- .../pingcap/kvproto/pkg/eraftpb/eraftpb.pb.go | 177 +- .../pingcap/kvproto/pkg/errorpb/errorpb.pb.go | 21 +- .../pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go | 6089 ++++++++++++----- .../pingcap/kvproto/pkg/metapb/metapb.pb.go | 21 +- .../pingcap/kvproto/pkg/pdpb/pdpb.pb.go | 1838 ++++- .../pkg/raft_serverpb/raft_serverpb.pb.go | 480 +- .../pingcap/kvproto/pkg/tikvpb/tikvpb.pb.go | 252 +- 206 files changed, 14875 insertions(+), 4661 deletions(-) create mode 100644 executor/index_lookup_join_test.go create mode 100644 store/tikv/latch/latch.go create mode 100644 store/tikv/latch/latch_test.go diff --git a/Gopkg.lock b/Gopkg.lock index 87fff41338351..4215542250993 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -210,7 +210,7 @@ "pkg/raft_serverpb", "pkg/tikvpb" ] - revision = "7b013aefd7210f4f4dbfca3330b462ddce5dd496" + revision = "2cf2492c337c4d0281d8ead8686d47764f1df6c4" [[projects]] name = "github.com/pingcap/pd" diff --git a/Makefile b/Makefile index 69036e466beff..e07f84fc7f33f 100644 --- a/Makefile +++ b/Makefile @@ -71,13 +71,9 @@ parserlib: parser/parser.go parser/parser.go: parser/parser.y make parser -check: errcheck - go get github.com/golang/lint/golint +check: fmt errcheck lint vet - @echo "vet" - @ go tool vet -all -shadow $(TOPDIRS) 2>&1 | awk '{print} END{if(NR>0) {exit 1}}' - @echo "golint" - @ golint -set_exit_status $(PACKAGES) +fmt: @echo "gofmt (simplify)" @ gofmt -s -l -w $(FILES) 2>&1 | grep -v "vendor|parser/parser.go" | awk '{print} END{if(NR>0) {exit 1}}' @@ -88,8 +84,18 @@ goword: errcheck: go get github.com/kisielk/errcheck + @echo "errcheck" @ GOPATH=$(GOPATH) errcheck -blank $(PACKAGES) | grep -v "_test\.go" | awk '{print} END{if(NR>0) {exit 1}}' +lint: + go get github.com/golang/lint/golint + @echo "golint" + @ golint -set_exit_status $(PACKAGES) + +vet: + @echo "vet" + @ go tool vet -all -shadow $(TOPDIRS) 2>&1 | awk '{print} END{if(NR>0) {exit 1}}' + clean: $(GO) clean -i ./... rm -rf *.out diff --git a/README.md b/README.md index 09294b03db46c..de8050eafd16f 100644 --- a/README.md +++ b/README.md @@ -8,31 +8,31 @@ ## What is TiDB? -TiDB (The pronunciation is: /'taɪdiːbi:/ tai-D-B, etymology: titanium) is a Hybrid Transactional/Analytical Processing (HTAP) database. Inspired by the design of Google F1 and Google Spanner, TiDB features infinite horizontal scalability, strong consistency, and high availability. The goal of TiDB is to serve as a one-stop solution for online transactions and analyses. +TiDB (The pronunciation is: /'taɪdiːbi:/ tai-D-B, etymology: titanium) is an open source distributed scalable Hybrid Transactional and Analytical Processing (HTAP) database built by PingCAP. Inspired by the design of Google F1 and Google Spanner, TiDB features infinite horizontal scalability, strong consistency, and high availability. The goal of TiDB is to serve as a one-stop solution for both OLTP (Online Transactional Processing) and OLAP (Online Analytical Processing). - __Horizontal scalability__ -Grow TiDB as your business grows. You can increase the capacity for storage and computation simply by adding more machines. + TiDB provides horizontal scalability simply by adding new nodes. Never worry about infrastructure capacity ever again. -- __Compatible with MySQL protocol__ +- __MySQL compatibility__ -Use TiDB as MySQL. You can replace MySQL with TiDB to power your application without changing a single line of code in most cases. + Easily replace MySQL with TiDB to power your applications without changing a single line of code in most cases and still benefit from the MySQL ecosystem. -- __Automatic Failover and high availability__ +- __Distributed transaction__ -Your data and applications are always-on. TiDB automatically handles malfunctions and protects your applications from machine failures or even downtime of an entire data-center. + TiDB is your source of truth, guaranteeing ACID compliance, so your data is accurate and reliable anytime, anywhere. -- __Consistent distributed transactions__ +- __Cloud Native__ -Think of TiDB as a single-machine RDBMS. You can start a transaction that crosses multiple machines without worrying about consistency. TiDB makes your application code simple and robust. + TiDB is designed to work in the cloud -- public, private, or hybrid -- making deployment, provisioning, and maintenance drop-dead simple. -- __Online DDL__ +- __No more ETL__ -Evolve TiDB schemas as your requirement changes. You can add new columns and indexes without stopping or affecting the on-going operations. + ETL (Extract, Transform and Load) is no longer necessary with TiDB's hybrid OLTP/OLAP architecture, enabling you to create new values for your users, easier and faster. -- __Multiple storage engine support__ +- __High availability__ -Power TiDB with your most favorite engines. TiDB supports local storage engines such as GolevelDB, as well as [TiKV](https://github.com/pingcap/tikv), a distributed storage engine. + With TiDB, your data and applications are always on and continuously available, so your users are never disappointed. For more details, see [How we build TiDB](https://pingcap.github.io/blog/2016/10/17/how-we-build-tidb/). diff --git a/ast/ast.go b/ast/ast.go index dab6c3ac5d4fc..f61e9519b473c 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -137,8 +137,8 @@ type RecordSet interface { // Fields gets result fields. Fields() []*ResultField - // NextChunk reads records into chunk. - NextChunk(ctx context.Context, chk *chunk.Chunk) error + // Next reads records into chunk. + Next(ctx context.Context, chk *chunk.Chunk) error // NewChunk creates a new chunk with initial capacity. NewChunk() *chunk.Chunk diff --git a/ast/ddl.go b/ast/ddl.go index f0d1f9b986df5..2bae74f85d148 100644 --- a/ast/ddl.go +++ b/ast/ddl.go @@ -403,6 +403,7 @@ type CreateTableStmt struct { Cols []*ColumnDef Constraints []*Constraint Options []*TableOption + Partition *PartitionOptions } // Accept implements Node Accept interface. @@ -745,6 +746,7 @@ type AlterTableSpec struct { OldColumnName *ColumnName Position *ColumnPosition LockType LockType + Comment string } // Accept implements Node Accept interface. @@ -845,3 +847,19 @@ func (n *TruncateTableStmt) Accept(v Visitor) (Node, bool) { n.Table = node.(*TableName) return v.Leave(n) } + +// PartitionDefinition defines a single partition. +type PartitionDefinition struct { + Name string + LessThan []ExprNode + MaxValue bool + Comment string +} + +// PartitionOptions specifies the partition options. +type PartitionOptions struct { + Tp model.PartitionType + Expr ExprNode + ColumnNames []*ColumnName + Definitions []*PartitionDefinition +} diff --git a/ast/misc.go b/ast/misc.go index baf9bab2d9c7e..0cf7e4d6550b8 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -591,6 +591,7 @@ const ( AdminCancelDDLJobs AdminCheckIndex AdminRecoverIndex + AdminCleanupIndex AdminCheckIndexRange AdminShowDDLJobQueries AdminChecksumTable diff --git a/cmd/benchdb/main.go b/cmd/benchdb/main.go index 67305fdeccb51..e6a91936b9869 100644 --- a/cmd/benchdb/main.go +++ b/cmd/benchdb/main.go @@ -119,7 +119,7 @@ func (ut *benchDB) mustExec(sql string) { rs := rss[0] chk := rs.NewChunk() for { - err := rs.NextChunk(ctx, chk) + err := rs.Next(ctx, chk) if err != nil { log.Fatal(err) } diff --git a/config/config.go b/config/config.go index 5ef4d0d823ac6..20db6e9c9f359 100644 --- a/config/config.go +++ b/config/config.go @@ -144,12 +144,13 @@ type Performance struct { MaxProcs uint `toml:"max-procs" json:"max-procs"` TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"` RetryLimit uint `toml:"retry-limit" json:"retry-limit"` - JoinConcurrency uint `toml:"join-concurrency" json:"join-concurrency"` CrossJoin bool `toml:"cross-join" json:"cross-join"` StatsLease string `toml:"stats-lease" json:"stats-lease"` RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"` StmtCountLimit uint `toml:"stmt-count-limit" json:"stmt-count-limit"` FeedbackProbability float64 `toml:"feedback-probability" json:"feedback-probability"` + QueryFeedbackLimit uint `toml:"query-feedback-limit" json:"query-feedback-limit"` + PseudoEstimateRatio float64 `toml:"pseudo-estimate-ratio" json:"pseudo-estimate-ratio"` } // XProtocol is the XProtocol section of the config. @@ -226,7 +227,7 @@ var defaultConf = Config{ Path: "/tmp/tidb", RunDDL: true, SplitTable: true, - Lease: "10s", + Lease: "45s", TokenLimit: 1000, OOMAction: "log", EnableStreaming: false, @@ -250,12 +251,13 @@ var defaultConf = Config{ Performance: Performance{ TCPKeepAlive: true, RetryLimit: 10, - JoinConcurrency: 5, CrossJoin: true, StatsLease: "3s", RunAutoAnalyze: true, StmtCountLimit: 5000, FeedbackProbability: 0, + QueryFeedbackLimit: 1024, + PseudoEstimateRatio: 0.7, }, XProtocol: XProtocol{ XHost: "", diff --git a/config/config.toml.example b/config/config.toml.example index cb230446d2921..f367d158be80f 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -22,7 +22,7 @@ binlog-socket = "" run-ddl = true # Schema lease duration, very dangerous to change only if you know what you do. -lease = "10s" +lease = "45s" # When create table, split a separated region for it. It is recommended to # turn off this option if there will be a large number of tables created. @@ -31,9 +31,6 @@ split-table = true # The limit of concurrent executed sessions. token-limit = 1000 -# Enable chunk executors. -enable-chunk = true - # Only print a log when out of memory quota. # Valid options: ["log", "cancel"] oom-action = "log" @@ -127,9 +124,6 @@ tcp-keep-alive = true # The maximum number of retries when commit a transaction. retry-limit = 10 -# The number of goroutines that participate joining. -join-concurrency = 5 - # Whether support cartesian product. cross-join = true @@ -142,6 +136,13 @@ run-auto-analyze = true # Probability to use the query feedback to update stats, 0 or 1 for always false/true. feedback-probability = 0.0 +# The max number of query feedback that cache in memory. +query-feedback-limit = 1024 + +# Pseudo stats will be used if the ratio between the modify count and +# row count in statistics of a table is greater than it. +pseudo-estimate-ratio = 0.7 + [proxy-protocol] # PROXY protocol acceptable client networks. # Empty string means disable PROXY protocol, * means all networks. diff --git a/ddl/callback_test.go b/ddl/callback_test.go index 7d81f7f42334c..12fa3b1fb34d8 100644 --- a/ddl/callback_test.go +++ b/ddl/callback_test.go @@ -16,20 +16,24 @@ package ddl import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" ) type TestDDLCallback struct { *BaseCallback - onJobRunBefore func(*model.Job) - onJobUpdated func(*model.Job) - OnJobUpdatedExported func(*model.Job) - onWatched func(ctx context.Context) + onJobRunBefore func(*model.Job) + OnJobRunBeforeExported func(*model.Job) + onJobUpdated func(*model.Job) + OnJobUpdatedExported func(*model.Job) + onWatched func(ctx context.Context) } func (tc *TestDDLCallback) OnJobRunBefore(job *model.Job) { + if tc.OnJobRunBeforeExported != nil { + tc.OnJobRunBeforeExported(job) + return + } if tc.onJobRunBefore != nil { tc.onJobRunBefore(job) return @@ -61,7 +65,6 @@ func (tc *TestDDLCallback) OnWatched(ctx context.Context) { } func (s *testDDLSuite) TestCallback(c *C) { - defer testleak.AfterTest(c)() cb := &BaseCallback{} c.Assert(cb.OnChanged(nil), IsNil) cb.OnJobRunBefore(nil) diff --git a/ddl/column.go b/ddl/column.go index ab40ba07cb013..100ce7af3de7a 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -440,7 +440,7 @@ func (d *ddl) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) } // doModifyColumn updates the column information and reorders all columns. -func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, col *model.ColumnInfo, oldName *model.CIStr, pos *ast.ColumnPosition) (ver int64, _ error) { +func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr, pos *ast.ColumnPosition) (ver int64, _ error) { tblInfo, err := getTableInfo(t, job, job.SchemaID) if err != nil { return ver, errors.Trace(err) @@ -451,7 +451,18 @@ func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, col *model.ColumnInfo job.State = model.JobStateCancelled return ver, infoschema.ErrColumnNotExists.GenByArgs(oldName, tblInfo.Name) } + // If we want to rename the column name, we need to check whether it already exists. + if newCol.Name.L != oldName.L { + c := findCol(tblInfo.Columns, newCol.Name.L) + if c != nil { + job.State = model.JobStateCancelled + return ver, infoschema.ErrColumnExists.GenByArgs(newCol.Name) + } + } + // We need the latest column's offset and state. This information can be obtained from the store. + newCol.Offset = oldCol.Offset + newCol.State = oldCol.State // Calculate column's new position. oldPos, newPos := oldCol.Offset, oldCol.Offset if pos.Tp == ast.ColumnPositionAfter { @@ -477,10 +488,10 @@ func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, col *model.ColumnInfo } columnChanged := make(map[string]*model.ColumnInfo) - columnChanged[oldName.L] = col + columnChanged[oldName.L] = newCol if newPos == oldPos { - tblInfo.Columns[newPos] = col + tblInfo.Columns[newPos] = newCol } else { cols := tblInfo.Columns @@ -490,7 +501,7 @@ func (d *ddl) doModifyColumn(t *meta.Meta, job *model.Job, col *model.ColumnInfo } else { copy(cols[oldPos:], cols[oldPos+1:newPos+1]) } - cols[newPos] = col + cols[newPos] = newCol for i, col := range tblInfo.Columns { if col.Offset != i { diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index a3b3ff51db4e0..4f61022f780df 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -41,6 +41,7 @@ type testColumnChangeSuite struct { } func (s *testColumnChangeSuite) SetUpSuite(c *C) { + testleak.BeforeTest() s.store = testCreateStore(c, "test_column_change") s.dbInfo = &model.DBInfo{ Name: model.NewCIStr("test_column_change"), @@ -53,8 +54,12 @@ func (s *testColumnChangeSuite) SetUpSuite(c *C) { c.Check(err, IsNil) } +func (s *testColumnChangeSuite) TearDownSuite(c *C) { + s.store.Close() + testleak.AfterTest(c)() +} + func (s *testColumnChangeSuite) TestColumnChange(c *C) { - defer testleak.AfterTest(c)() d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) defer d.Stop() // create table t (c1 int, c2 int); diff --git a/ddl/column_test.go b/ddl/column_test.go index 2a50d24553a87..7098d49beb5ca 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -44,6 +44,7 @@ type testColumnSuite struct { } func (s *testColumnSuite) SetUpSuite(c *C) { + testleak.BeforeTest() s.store = testCreateStore(c, "test_column") s.d = testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) @@ -57,6 +58,7 @@ func (s *testColumnSuite) TearDownSuite(c *C) { err := s.store.Close() c.Assert(err, IsNil) + testleak.AfterTest(c)() } func testCreateColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, @@ -105,7 +107,6 @@ func testDropColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, } func (s *testColumnSuite) TestColumn(c *C) { - defer testleak.AfterTest(c)() tblInfo := testTableInfo(c, s.d, "t1", 3) ctx := testNewContext(s.d) @@ -736,7 +737,6 @@ func (s *testColumnSuite) testGetColumn(t table.Table, name string, isExist bool } func (s *testColumnSuite) TestAddColumn(c *C) { - defer testleak.AfterTest(c)() d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) tblInfo := testTableInfo(c, d, "t", 3) ctx := testNewContext(d) @@ -822,7 +822,6 @@ func (s *testColumnSuite) TestAddColumn(c *C) { } func (s *testColumnSuite) TestDropColumn(c *C) { - defer testleak.AfterTest(c)() d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) tblInfo := testTableInfo(c, d, "t", 4) ctx := testNewContext(d) diff --git a/ddl/ddl.go b/ddl/ddl.go index 730166f716bf6..337a3b5c59580 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -154,8 +154,7 @@ var ( type DDL interface { CreateSchema(ctx sessionctx.Context, name model.CIStr, charsetInfo *ast.CharsetOpt) error DropSchema(ctx sessionctx.Context, schema model.CIStr) error - CreateTable(ctx sessionctx.Context, ident ast.Ident, cols []*ast.ColumnDef, - constrs []*ast.Constraint, options []*ast.TableOption) error + CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error CreateTableWithLike(ctx sessionctx.Context, ident, referIdent ast.Ident) error DropTable(ctx sessionctx.Context, tableIdent ast.Ident) (err error) CreateIndex(ctx sessionctx.Context, tableIdent ast.Ident, unique bool, indexName model.CIStr, diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d707b8eb0c4bf..4455603772c02 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -255,6 +255,14 @@ func checkColumnCantHaveDefaultValue(col *table.Column, value interface{}) (err return nil } +// isExplicitTimeStamp is used to check if explicit_defaults_for_timestamp is on or off. +// Check out this link for more details. +// https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_explicit_defaults_for_timestamp +func isExplicitTimeStamp() bool { + // TODO: implement the behavior as MySQL when explicit_defaults_for_timestamp = off, then this function could return false. + return true +} + // columnDefToCol converts ColumnDef to Col and TableConstraints. func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef) (*table.Column, []*ast.Constraint, error) { var constraints = make([]*ast.Constraint, 0) @@ -264,11 +272,13 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef) ( FieldType: *colDef.Tp, }) - // Check and set TimestampFlag and OnUpdateNowFlag. - if col.Tp == mysql.TypeTimestamp { - col.Flag |= mysql.TimestampFlag - col.Flag |= mysql.OnUpdateNowFlag - col.Flag |= mysql.NotNullFlag + if !isExplicitTimeStamp() { + // Check and set TimestampFlag, OnUpdateNowFlag and NotNullFlag. + if col.Tp == mysql.TypeTimestamp { + col.Flag |= mysql.TimestampFlag + col.Flag |= mysql.OnUpdateNowFlag + col.Flag |= mysql.NotNullFlag + } } setOnUpdateNow := false @@ -740,14 +750,22 @@ func (d *ddl) CreateTableWithLike(ctx sessionctx.Context, ident, referIdent ast. return errors.Trace(err) } -func (d *ddl) CreateTable(ctx sessionctx.Context, ident ast.Ident, colDefs []*ast.ColumnDef, - constraints []*ast.Constraint, options []*ast.TableOption) (err error) { +func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err error) { + ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} + if s.ReferTable != nil { + referIdent := ast.Ident{Schema: s.ReferTable.Schema, Name: s.ReferTable.Name} + return d.CreateTableWithLike(ctx, ident, referIdent) + } + colDefs := s.Cols is := d.GetInformationSchema() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenByArgs(ident.Schema) } if is.TableExists(ident.Schema, ident.Name) { + if s.IfNotExists { + return nil + } return infoschema.ErrTableExists.GenByArgs(ident) } if err = checkTooLongTable(ident.Name); err != nil { @@ -766,7 +784,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, ident ast.Ident, colDefs []*as return errors.Trace(err) } - cols, newConstraints, err := buildColumnsAndConstraints(ctx, colDefs, constraints) + cols, newConstraints, err := buildColumnsAndConstraints(ctx, colDefs, s.Constraints) if err != nil { return errors.Trace(err) } @@ -780,6 +798,42 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, ident ast.Ident, colDefs []*as if err != nil { return errors.Trace(err) } + if s.Partition != nil { + pi := &model.PartitionInfo{ + Type: s.Partition.Tp, + Expr: s.Partition.Expr.Text(), + } + if s.Partition.Expr != nil { + buf := new(bytes.Buffer) + s.Partition.Expr.Format(buf) + pi.Expr = buf.String() + } else if s.Partition.ColumnNames != nil { + pi.Columns = make([]model.CIStr, 0, len(s.Partition.ColumnNames)) + for _, cn := range s.Partition.ColumnNames { + pi.Columns = append(pi.Columns, cn.Name) + } + } + for _, def := range s.Partition.Definitions { + // TODO: generate multiple global ID for paritions. + pid, err1 := d.genGlobalID() + if err1 != nil { + return errors.Trace(err1) + } + piDef := model.PartitionDefinition{ + Name: def.Name, + ID: pid, + Comment: def.Comment, + MaxValue: def.MaxValue, + } + for _, expr := range def.LessThan { + buf := new(bytes.Buffer) + expr.Format(buf) + piDef.LessThan = append(piDef.LessThan, buf.String()) + } + pi.Definitions = append(pi.Definitions, piDef) + } + tbInfo.Partition = pi + } job := &model.Job{ SchemaID: schema.ID, @@ -789,7 +843,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, ident ast.Ident, colDefs []*as Args: []interface{}{tbInfo}, } - handleTableOptions(options, tbInfo) + handleTableOptions(s.Options, tbInfo) err = checkCharsetAndCollation(tbInfo.Charset, tbInfo.Collate) if err != nil { return errors.Trace(err) @@ -802,6 +856,11 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, ident ast.Ident, colDefs []*as err = d.handleAutoIncID(tbInfo, schema.ID) } } + + // table exists, but if_not_exists flags is true, so we ignore this error. + if infoschema.ErrTableExists.Equal(err) && s.IfNotExists { + return nil + } err = d.callHookOnChanged(err) return errors.Trace(err) } @@ -934,6 +993,9 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A err = d.ShardRowID(ctx, ident, opt.UintValue) case ast.TableOptionAutoIncrement: err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue)) + case ast.TableOptionComment: + spec.Comment = opt.StrValue + err = d.AlterTableComment(ctx, ident, spec) } if err != nil { return errors.Trace(err) @@ -1293,6 +1355,14 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or if col == nil { return nil, infoschema.ErrColumnNotExists.GenByArgs(originalColName, ident.Name) } + newColName := specNewColumn.Name.Name + // If we want to rename the column name, we need to check whether it already exists. + if newColName.L != originalColName.L { + c := table.FindCol(t.Cols(), newColName.L) + if c != nil { + return nil, infoschema.ErrColumnExists.GenByArgs(newColName) + } + } // Constraints in the new column means adding new constraints. Errors should thrown, // which will be done by `setDefaultAndComment` later. @@ -1303,11 +1373,9 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or newCol := table.ToColumn(&model.ColumnInfo{ ID: col.ID, - Offset: col.Offset, - State: col.State, OriginDefaultValue: col.OriginDefaultValue, FieldType: *specNewColumn.Tp, - Name: specNewColumn.Name.Name, + Name: newColName, }) err = setCharsetCollationFlenDecimal(&newCol.FieldType) @@ -1448,6 +1516,32 @@ func (d *ddl) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Alt return errors.Trace(err) } +// AlterTableComment updates the table comment information. +func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { + is := d.infoHandle.Get() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenByArgs(ident.Schema) + } + + tb, err := is.TableByName(ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ident.Schema, ident.Name)) + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: tb.Meta().ID, + Type: model.ActionModifyTableComment, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{spec.Comment}, + } + + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} + // DropTable will proceed even if some table in the list does not exists. func (d *ddl) DropTable(ctx sessionctx.Context, ti ast.Ident) (err error) { is := d.GetInformationSchema() diff --git a/ddl/ddl_db_change_test.go b/ddl/ddl_db_change_test.go index 01c5d091d0419..9d749ba8ee8f8 100644 --- a/ddl/ddl_db_change_test.go +++ b/ddl/ddl_db_change_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "fmt" "strings" + "sync" "time" "github.com/juju/errors" @@ -25,6 +26,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/session" @@ -47,6 +49,7 @@ type testStateChangeSuite struct { } func (s *testStateChangeSuite) SetUpSuite(c *C) { + testleak.BeforeTest() s.lease = 200 * time.Millisecond var err error s.store, err = mockstore.NewMockTikvStore() @@ -68,6 +71,7 @@ func (s *testStateChangeSuite) TearDownSuite(c *C) { s.se.Close() s.dom.Close() s.store.Close() + testleak.AfterTest(c)() } func (s *testStateChangeSuite) TestTwoStates(c *C) { @@ -103,7 +107,6 @@ func (s *testStateChangeSuite) TestTwoStates(c *C) { } func (s *testStateChangeSuite) test(c *C, tableName, alterTableSQL string, testInfo *testExecInfo) { - defer testleak.AfterTest(c)() _, err := s.se.Execute(context.Background(), `create table t ( c1 int, c2 varchar(64), @@ -312,7 +315,6 @@ func (s *testStateChangeSuite) TestDeleteOnly(c *C) { func (s *testStateChangeSuite) runTestInSchemaState(c *C, state model.SchemaState, tableName, alterTableSQL string, sqlWithErrs []sqlWithErr) { - defer testleak.AfterTest(c)() _, err := s.se.Execute(context.Background(), `create table t ( c1 varchar(64), c2 enum('N','Y') not null default 'N', @@ -390,7 +392,6 @@ func (s *testStateChangeSuite) CheckResult(tk *testkit.TestKit, sql string, args } func (s *testStateChangeSuite) TestShowIndex(c *C) { - defer testleak.AfterTest(c)() _, err := s.se.Execute(context.Background(), `create table t(c1 int primary key, c2 int)`) c.Assert(err, IsNil) defer s.se.Execute(context.Background(), "drop table t") @@ -430,3 +431,195 @@ func (s *testStateChangeSuite) TestShowIndex(c *C) { callback = &ddl.TestDDLCallback{} d.SetHook(callback) } + +func (s *testStateChangeSuite) TestParallelAlterModifyColumn(c *C) { + _, err := s.se.Execute(context.Background(), "use test_db_state") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "create table t(a int, b int, c int)") + c.Assert(err, IsNil) + defer s.se.Execute(context.Background(), "drop table t") + + callback := &ddl.TestDDLCallback{} + times := 0 + callback.OnJobUpdatedExported = func(job *model.Job) { + if times != 0 { + return + } + var qLen int64 + var err1 error + for { + kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + qLen, err1 = m.DDLJobQueueLen() + if err1 != nil { + return err1 + } + return nil + }) + if qLen == 2 { + break + } + time.Sleep(5 * time.Millisecond) + } + times++ + } + d := s.dom.DDL() + d.SetHook(callback) + + wg := sync.WaitGroup{} + var err1 error + var err2 error + se, err := session.CreateSession(s.store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test_db_state") + c.Assert(err, IsNil) + se1, err := session.CreateSession(s.store) + c.Assert(err, IsNil) + _, err = se1.Execute(context.Background(), "use test_db_state") + c.Assert(err, IsNil) + wg.Add(2) + go func() { + defer wg.Done() + _, err1 = se.Execute(context.Background(), "ALTER TABLE t MODIFY COLUMN b int FIRST;") + }() + + go func() { + defer wg.Done() + _, err2 = se1.Execute(context.Background(), "ALTER TABLE t MODIFY COLUMN b int FIRST;") + }() + + time.Sleep(1 * time.Second) + wg.Wait() + c.Assert(err1, IsNil) + c.Assert(err2, IsNil) + + _, err = s.se.Execute(context.Background(), "select * from t") + c.Assert(err, IsNil) + + callback = &ddl.TestDDLCallback{} + d.SetHook(callback) +} + +func (s *testStateChangeSuite) testParrallelExecSQL(c *C, sql string) { + se, err := session.CreateSession(s.store) + _, err = se.Execute(context.Background(), "use test_db_state") + c.Assert(err, IsNil) + + se1, err1 := session.CreateSession(s.store) + _, err = se1.Execute(context.Background(), "use test_db_state") + c.Assert(err1, IsNil) + + var err2, err3 error + wg := sync.WaitGroup{} + + callback := &ddl.TestDDLCallback{} + once := sync.Once{} + callback.OnJobUpdatedExported = func(job *model.Job) { + // sleep a while, let other job enqueue. + once.Do(func() { + time.Sleep(time.Millisecond * 10) + }) + } + + d := s.dom.DDL() + d.SetHook(callback) + + wg.Add(2) + go func() { + defer wg.Done() + _, err2 = se.Execute(context.Background(), sql) + }() + + go func() { + defer wg.Done() + _, err3 = se1.Execute(context.Background(), sql) + }() + wg.Wait() + c.Assert(err2, IsNil) + c.Assert(err3, IsNil) + callback = &ddl.TestDDLCallback{} + d.SetHook(callback) +} + +// TestCreateTableIfNotExists parallel exec create table if not exists xxx. No error returns is expected. +func (s *testStateChangeSuite) TestCreateTableIfNotExists(c *C) { + defer s.se.Execute(context.Background(), "drop table test_not_exists") + s.testParrallelExecSQL(c, "create table if not exists test_not_exists(a int);") +} + +// TestCreateDBIfNotExists parallel exec create database if not exists xxx. No error returns is expected. +func (s *testStateChangeSuite) TestCreateDBIfNotExists(c *C) { + defer s.se.Execute(context.Background(), "drop database test_not_exists") + s.testParrallelExecSQL(c, "create database if not exists test_not_exists;") +} + +func (s *testStateChangeSuite) TestParallelChangeColumnName(c *C) { + _, err := s.se.Execute(context.Background(), "use test_db_state") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "create table t(a int, b int, c int)") + c.Assert(err, IsNil) + defer s.se.Execute(context.Background(), "drop table t") + + callback := &ddl.TestDDLCallback{} + once := sync.Once{} + callback.OnJobUpdatedExported = func(job *model.Job) { + // Make sure the both DDL statements have entered the DDL queue before running the DDL jobs. + once.Do(func() { + var qLen int64 + var err error + for { + kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + qLen, err = m.DDLJobQueueLen() + if err != nil { + return err + } + return nil + }) + if qLen == 2 { + break + } + time.Sleep(5 * time.Millisecond) + } + }) + } + d := s.dom.DDL() + d.SetHook(callback) + + // Use two sessions to run DDL statements in parallel. + wg := sync.WaitGroup{} + var err1 error + var err2 error + se, err := session.CreateSession(s.store) + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "use test_db_state") + c.Assert(err, IsNil) + se1, err := session.CreateSession(s.store) + c.Assert(err, IsNil) + _, err = se1.Execute(context.Background(), "use test_db_state") + c.Assert(err, IsNil) + wg.Add(2) + go func() { + defer wg.Done() + _, err1 = se.Execute(context.Background(), "ALTER TABLE t CHANGE a aa int;") + }() + go func() { + defer wg.Done() + _, err2 = se1.Execute(context.Background(), "ALTER TABLE t CHANGE b aa int;") + }() + + wg.Wait() + // Make sure only a DDL encounters the error of 'duplicate column name'. + var oneErr error + if (err1 != nil && err2 == nil) || (err1 == nil && err2 != nil) { + if err1 != nil { + oneErr = err1 + } else { + oneErr = err2 + } + } + c.Assert(oneErr.Error(), Equals, "[schema:1060]Duplicate column name 'aa'") + + callback = &ddl.TestDDLCallback{} + d.SetHook(callback) +} diff --git a/ddl/ddl_db_test.go b/ddl/ddl_db_test.go index 1eaddb1054e7f..b86c5654bf091 100644 --- a/ddl/ddl_db_test.go +++ b/ddl/ddl_db_test.go @@ -20,6 +20,7 @@ import ( "math/rand" "strconv" "strings" + "sync" "time" "github.com/juju/errors" @@ -51,7 +52,7 @@ import ( const ( // waitForCleanDataRound indicates how many times should we check data is cleaned or not. - waitForCleanDataRound = 60 + waitForCleanDataRound = 150 // waitForCleanDataInterval is a min duration between 2 check for data clean. waitForCleanDataInterval = time.Millisecond * 100 ) @@ -167,6 +168,9 @@ func (s *testDBSuite) TestMySQLErrorCode(c *C) { s.testErrorCode(c, sql, tmysql.ErrDupFieldName) sql = "alter table test_error_code_succ add column aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa int" s.testErrorCode(c, sql, tmysql.ErrTooLongIdent) + sql = "alter table test_comment comment 'test comment'" + s.testErrorCode(c, sql, tmysql.ErrNoSuchTable) + // drop column sql = "alter table test_error_code_succ drop c_not_exist" s.testErrorCode(c, sql, tmysql.ErrCantDropFieldOrKey) @@ -323,6 +327,7 @@ func (s *testDBSuite) TestCancelAddIndex(c *C) { } var checkErr error + var c3IdxInfo *model.IndexInfo hook := &ddl.TestDDLCallback{} first := true oldReorgWaitTimeout := ddl.ReorgWaitTimeout @@ -335,6 +340,16 @@ func (s *testDBSuite) TestCancelAddIndex(c *C) { // If the action is adding index and the state is writing reorganization, it want to test the case of cancelling the job when backfilling indexes. // When the job satisfies this case of addIndexNotFirstReorg, the worker will start to backfill indexes. if !addIndexNotFirstReorg { + // Get the index's meta. + if c3IdxInfo != nil { + return + } + t := s.testGetTable(c, "t1") + for _, index := range t.WritableIndices() { + if index.Meta().Name.L == "c3_index" { + c3IdxInfo = index.Meta() + } + } return } // The job satisfies the case of addIndexNotFirst for the first time, the worker hasn't finished a batch of backfill indexes. @@ -407,6 +422,10 @@ LOOP: c.Assert(strings.EqualFold(tidx.Meta().Name.L, "c3_index"), IsFalse) } + ctx := s.s.(sessionctx.Context) + idx := tables.NewIndex(t.Meta(), c3IdxInfo) + checkDelRangeDone(c, ctx, idx) + s.mustExec(c, "drop table t1") ddl.ReorgWaitTimeout = oldReorgWaitTimeout } @@ -461,7 +480,7 @@ func (s *testDBSuite) TestAddAnonymousIndex(c *C) { c.Assert(t.Indices()[1].Meta().Name.String(), Equals, "primary_3") } -// Issue 5134 +// TestModifyColumnAfterAddIndex Issue 5134 func (s *testDBSuite) TestModifyColumnAfterAddIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -713,6 +732,12 @@ LOOP: c.Assert(nidx, IsNil) idx := tables.NewIndex(t.Meta(), c3idx.Meta()) + checkDelRangeDone(c, ctx, idx) + s.tk.MustExec("drop table test_drop_index") +} + +func checkDelRangeDone(c *C, ctx sessionctx.Context, idx table.Index) { + startTime := time.Now() f := func() map[int64]struct{} { handles := make(map[int64]struct{}) @@ -744,9 +769,7 @@ LOOP: break } } - c.Assert(handles, HasLen, 0) - - s.tk.MustExec("drop table test_drop_index") + c.Assert(handles, HasLen, 0, Commentf("take time %v", time.Since(startTime))) } func (s *testDBSuite) TestAddIndexWithDupCols(c *C) { @@ -785,6 +808,22 @@ func (s *testDBSuite) TestIssue2293(c *C) { s.tk.MustQuery("select * from t_issue_2293").Check(testkit.Rows("1")) } +func (s *testDBSuite) TestIssue6101(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + s.tk.MustExec("use " + s.schemaName) + s.tk.MustExec("create table t1 (quantity decimal(2) unsigned);") + _, err := s.tk.Exec("insert into t1 values (500), (-500), (~0), (-1);") + terr := errors.Trace(err).(*errors.Err).Cause().(*terror.Error) + c.Assert(terr.Code(), Equals, terror.ErrCode(tmysql.ErrDataOutOfRange)) + s.tk.MustExec("drop table t1") + + s.tk.MustExec("set sql_mode=''") + s.tk.MustExec("create table t1 (quantity decimal(2) unsigned);") + s.tk.MustExec("insert into t1 values (500), (-500), (~0), (-1);") + s.tk.MustQuery("select * from t1").Check(testkit.Rows("99", "0", "99", "0")) + s.tk.MustExec("drop table t1") +} + func (s *testDBSuite) TestCreateIndexType(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -1013,7 +1052,7 @@ LOOP: c.Assert(count, Greater, int64(0)) } -// This test is for insert value with a to-be-dropped column when do drop column. +// testDropColumn2 is for inserting value with a to-be-dropped column when do drop column. // Column info from schema in build-insert-plan should be public only, // otherwise they will not be consist with Table.Col(), then the server will panic. func (s *testDBSuite) testDropColumn2(c *C) { @@ -1106,6 +1145,10 @@ func (s *testDBSuite) TestChangeColumn(c *C) { s.testErrorCode(c, sql, tmysql.ErrUnknown) sql = "alter table t3 modify en enum('a', 'z', 'b', 'c') not null default 'a'" s.testErrorCode(c, sql, tmysql.ErrUnknown) + // Rename to an existing column. + s.mustExec(c, "alter table t3 add column a bigint") + sql = "alter table t3 change aa a bigint" + s.testErrorCode(c, sql, tmysql.ErrDupFieldName) s.tk.MustExec("drop table t3") } @@ -1368,6 +1411,7 @@ func (s *testDBSuite) TestCreateTableWithLike(c *C) { func (s *testDBSuite) TestCreateTable(c *C) { s.tk.MustExec("use test") s.tk.MustExec("CREATE TABLE `t` (`a` double DEFAULT 1.0 DEFAULT now() DEFAULT 2.0 );") + s.tk.MustExec("CREATE TABLE IF NOT EXISTS `t` (`a` double DEFAULT 1.0 DEFAULT now() DEFAULT 2.0 );") ctx := s.tk.Se.(sessionctx.Context) is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -1387,6 +1431,33 @@ func (s *testDBSuite) TestCreateTable(c *C) { c.Assert(err, NotNil) } +func (s *testDBSuite) TestCreateTableWithPartition(c *C) { + s.tk.MustExec("use test") + s.tk.MustExec(`CREATE TABLE tp (a int) PARTITION BY RANGE(a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20), + PARTITION p2 VALUES LESS THAN (MAXVALUE) +);`) + ctx := s.tk.Se.(sessionctx.Context) + is := domain.GetDomain(ctx).InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) + c.Assert(err, IsNil) + c.Assert(tbl.Meta().Partition, NotNil) + part := tbl.Meta().Partition + c.Assert(part.Type, Equals, model.PartitionTypeRange) + c.Assert(part.Expr, Equals, "`a`") + for _, pdef := range part.Definitions { + c.Assert(pdef.ID, Greater, int64(0)) + } + c.Assert(part.Definitions, HasLen, 3) + c.Assert(part.Definitions[0].LessThan[0], Equals, "10") + c.Assert(part.Definitions[0].Name, Equals, "p0") + c.Assert(part.Definitions[1].LessThan[0], Equals, "20") + c.Assert(part.Definitions[1].Name, Equals, "p1") + c.Assert(part.Definitions[2].MaxValue, IsTrue) + c.Assert(part.Definitions[2].Name, Equals, "p2") +} + func (s *testDBSuite) TestTruncateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1773,3 +1844,34 @@ func (s *testDBSuite) TestCharacterSetInColumns(c *C) { s.tk.MustQuery("select count(*) from information_schema.columns where table_schema = 'varchar_test' and character_set_name != 'utf8'").Check(testkit.Rows("0")) s.tk.MustQuery("select count(*) from information_schema.columns where table_schema = 'varchar_test' and character_set_name = 'utf8'").Check(testkit.Rows("2")) } + +func (s *testDBSuite) TestAddNotNullColumnWhileInsertOnDupUpdate(c *C) { + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("use " + s.schemaName) + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec("use " + s.schemaName) + closeCh := make(chan bool) + wg := new(sync.WaitGroup) + wg.Add(1) + tk1.MustExec("create table nn (a int primary key, b int)") + tk1.MustExec("insert nn values (1, 1)") + var tk2Err error + go func() { + defer wg.Done() + for { + select { + case <-closeCh: + return + default: + } + _, tk2Err = tk2.Exec("insert nn (a, b) values (1, 1) on duplicate key update a = 1, b = b + 1") + if tk2Err != nil { + return + } + } + }() + tk1.MustExec("alter table nn add column c int not null default 0") + close(closeCh) + wg.Wait() + c.Assert(tk2Err, IsNil) +} diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 33c17e9b18645..b54e76a182dbc 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -91,7 +91,6 @@ func checkHistoryJob(c *C, job *model.Job) { } func checkHistoryJobArgs(c *C, ctx sessionctx.Context, id int64, args *historyJobArgs) { - c.Assert(ctx.NewTxn(), IsNil) t := meta.NewMeta(ctx.Txn()) historyJob, err := t.GetHistoryDDLJob(id) c.Assert(err, IsNil) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 46689a2ef573d..5f16429e2e6c5 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -52,6 +52,9 @@ func (d *ddl) onDDLWorker() { metrics.PanicCounter.WithLabelValues(metrics.LabelDDL).Inc() } }() + + // shouldCleanJobs is used to determine whether to clean up the job in adding index queue. + shouldCleanJobs := true for { select { case <-ticker.C: @@ -61,9 +64,12 @@ func (d *ddl) onDDLWorker() { return } - err := d.handleDDLJobQueue() + err := d.handleDDLJobQueue(shouldCleanJobs) if err != nil { log.Errorf("[ddl] handle ddl job err %v", errors.ErrorStack(err)) + } else if shouldCleanJobs { + log.Info("[ddl] cleaning jobs in the adding index queue finished.") + shouldCleanJobs = false } } } @@ -91,6 +97,29 @@ func (d *ddl) isOwner() bool { return isOwner } +// buildJobDependence sets the curjob's dependency-ID. +// The dependency-job's ID must less than the current job's ID, and we need the largest one in the list. +func buildJobDependence(t *meta.Meta, curJob *model.Job) error { + jobs, err := t.GetAllDDLJobs() + if err != nil { + return errors.Trace(err) + } + for _, job := range jobs { + if curJob.ID < job.ID { + continue + } + isDependent, err := curJob.IsDependentOn(job) + if err != nil { + return errors.Trace(err) + } + if isDependent { + curJob.DependencyID = job.ID + break + } + } + return nil +} + // addDDLJob gets a global job ID and puts the DDL job in the DDL queue. func (d *ddl) addDDLJob(ctx sessionctx.Context, job *model.Job) error { startTime := time.Now() @@ -147,6 +176,16 @@ func (d *ddl) updateDDLJob(t *meta.Meta, job *model.Job, meetErr bool) error { return errors.Trace(t.UpdateDDLJob(0, job, updateRawArgs)) } +func (d *ddl) deleteRange(job *model.Job) error { + var err error + if job.Version <= currentVersion { + err = d.delRangeManager.addDelRangeJob(job) + } else { + err = errInvalidJobVersion.GenByArgs(job.Version, currentVersion) + } + return errors.Trace(err) +} + // finishDDLJob deletes the finished DDL job in the ddl queue and puts it to history queue. // If the DDL job need to handle in background, it will prepare a background job. func (d *ddl) finishDDLJob(t *meta.Meta, job *model.Job) (err error) { @@ -154,16 +193,19 @@ func (d *ddl) finishDDLJob(t *meta.Meta, job *model.Job) (err error) { defer func() { metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerFinishDDLJob, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() + switch job.Type { - case model.ActionDropSchema, model.ActionDropTable, model.ActionTruncateTable, model.ActionDropIndex: - if job.Version <= currentVersion { - err = d.delRangeManager.addDelRangeJob(job) - } else { - err = errInvalidJobVersion.GenByArgs(job.Version, currentVersion) - } - if err != nil { - return errors.Trace(err) + case model.ActionAddIndex: + if job.State != model.JobStateRollbackDone { + break } + // After rolling back an AddIndex operation, we need to use delete-range to delete the half-done index data. + err = d.deleteRange(job) + case model.ActionDropSchema, model.ActionDropTable, model.ActionTruncateTable, model.ActionDropIndex: + err = d.deleteRange(job) + } + if err != nil { + return errors.Trace(err) } _, err = t.DeQueueDDLJob() @@ -191,7 +233,9 @@ func (d *ddl) getHistoryDDLJob(id int64) (*model.Job, error) { return job, errors.Trace(err) } -func (d *ddl) handleDDLJobQueue() error { +// handleDDLJobQueue handles DDL jobs in DDL Job queue. +// shouldCleanJobs is used to determine whether to clean up the job in adding index queue. +func (d *ddl) handleDDLJobQueue(shouldCleanJobs bool) error { once := true for { if d.isClosed() { @@ -207,6 +251,12 @@ func (d *ddl) handleDDLJobQueue() error { return nil } + // It's used for clean up the job in adding index queue before we support adding index queue. + // TODO: Remove this logic after we support the adding index queue. + if shouldCleanJobs { + return errors.Trace(d.cleanAddIndexQueueJobs(txn)) + } + var err error t := meta.NewMeta(txn) // We become the owner. Get the first job and run it. @@ -221,9 +271,11 @@ func (d *ddl) handleDDLJobQueue() error { return nil } - if job.IsDone() { + if job.IsDone() || job.IsRollbackDone() { binloginfo.SetDDLBinlog(d.workerVars.BinlogClient, txn, job.ID, job.Query) - job.State = model.JobStateSynced + if !job.IsRollbackDone() { + job.State = model.JobStateSynced + } err = d.finishDDLJob(t, job) return errors.Trace(err) } @@ -254,9 +306,9 @@ func (d *ddl) handleDDLJobQueue() error { d.hookMu.Unlock() // Here means the job enters another state (delete only, write only, public, etc...) or is cancelled. - // If the job is done or still running, we will wait 2 * lease time to guarantee other servers to update + // If the job is done or still running or rolling back, we will wait 2 * lease time to guarantee other servers to update // the newest schema. - if job.State == model.JobStateRunning || job.State == model.JobStateDone { + if job.IsRunning() || job.IsRollingback() || job.IsDone() || job.IsRollbackDone() { d.waitSchemaChanged(nil, waitTime, schemaVer) } if job.IsSynced() { @@ -329,6 +381,8 @@ func (d *ddl) runDDLJob(t *meta.Meta, job *model.Job) (ver int64, err error) { ver, err = d.onSetDefaultValue(t, job) case model.ActionShardRowID: ver, err = d.onShardRowID(t, job) + case model.ActionModifyTableComment: + ver, err = d.onModifyTableComment(t, job) default: // Invalid job, cancel it. job.State = model.JobStateCancelled @@ -417,7 +471,7 @@ func (d *ddl) waitSchemaChanged(ctx context.Context, waitTime time.Duration, lat // So here we get the latest schema version to make sure all servers' schema version update to the latest schema version // in a cluster, or to wait for 2 * lease time. func (d *ddl) waitSchemaSynced(job *model.Job, waitTime time.Duration) { - if !job.IsRunning() && !job.IsDone() { + if !job.IsRunning() && !job.IsRollingback() && !job.IsDone() && !job.IsRollbackDone() { return } // TODO: Make ctx exits when the d is close. @@ -464,3 +518,66 @@ func updateSchemaVersion(t *meta.Meta, job *model.Job) (int64, error) { err = t.SetSchemaDiff(diff) return schemaVersion, errors.Trace(err) } + +// cleanAddIndexQueueJobs cleans jobs in adding index queue. +// It's only done once after the worker become the owner. +// TODO: Remove this logic after we support the adding index queue. +func (d *ddl) cleanAddIndexQueueJobs(txn kv.Transaction) error { + startTime := time.Now() + m := meta.NewMeta(txn) + m.SetJobListKey(meta.AddIndexJobListKey) + for { + job, err := d.getFirstDDLJob(m) + if err != nil { + return errors.Trace(err) + } + if job == nil { + log.Infof("[ddl] cleaning jobs in the adding index queue takes time %v.", time.Since(startTime)) + return nil + } + log.Infof("[ddl] cleaning job %v in the adding index queue.", job) + + // The types of these jobs must be ActionAddIndex. + if job.SchemaState == model.StatePublic || job.SchemaState == model.StateNone { + if job.SchemaState == model.StateNone { + job.State = model.JobStateCancelled + } else { + binloginfo.SetDDLBinlog(d.workerVars.BinlogClient, txn, job.ID, job.Query) + job.State = model.JobStateSynced + } + err = d.finishDDLJob(m, job) + if err != nil { + return errors.Trace(err) + } + continue + } + + // When the job not in "none" and "public" state, we need to rollback it. + schemaID := job.SchemaID + tblInfo, err := getTableInfo(m, job, schemaID) + if err != nil { + return errors.Trace(err) + } + var indexName model.CIStr + var unique bool + err = job.DecodeArgs(&unique, &indexName) + if err != nil { + return errors.Trace(err) + } + indexInfo := findIndexByName(indexName.L, tblInfo.Indices) + _, err = d.convert2RollbackJob(m, job, tblInfo, indexInfo, nil) + if err == nil { + _, err = m.DeQueueDDLJob() + } + if err != nil { + return errors.Trace(err) + } + // Put the job to the default job list. + m.SetJobListKey(meta.DefaultJobListKey) + err = m.EnQueueDDLJob(job) + m.SetJobListKey(meta.AddIndexJobListKey) + if err != nil { + return errors.Trace(err) + } + } +} diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 7e344d39ae8b6..78b469b52980b 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -38,12 +38,16 @@ type testDDLSuite struct{} const testLease = 5 * time.Millisecond func (s *testDDLSuite) SetUpSuite(c *C) { + testleak.BeforeTest() // set ReorgWaitTimeout to small value, make test to be faster. ReorgWaitTimeout = 50 * time.Millisecond } +func (s *testDDLSuite) TearDownSuite(c *C) { + testleak.AfterTest(c)() +} + func (s *testDDLSuite) TestCheckOwner(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_owner") defer store.Close() @@ -59,7 +63,6 @@ func (s *testDDLSuite) TestCheckOwner(c *C) { // TestRunWorker tests no job is handled when the value of RunWorker is false. func (s *testDDLSuite) TestRunWorker(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_run_worker") defer store.Close() @@ -105,8 +108,124 @@ func (s *testDDLSuite) TestRunWorker(c *C) { <-exitCh } -func (s *testDDLSuite) TestSchemaError(c *C) { +func (s *testDDLSuite) TestCleanJobs(c *C) { defer testleak.AfterTest(c)() + store := testCreateStore(c, "test_clean_jobs") + defer store.Close() + d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) + + ctx := testNewContext(d) + dbInfo := testSchemaInfo(c, d, "test") + testCreateSchema(c, ctx, d, dbInfo) + tblInfo := testTableInfo(c, d, "t", 2) + testCreateTable(c, ctx, d, dbInfo, tblInfo) + + var failedJobIDs []int64 + job := &model.Job{ + SchemaID: dbInfo.ID, + TableID: tblInfo.ID, + Type: model.ActionAddIndex, + BinlogInfo: &model.HistoryInfo{}, + } + idxColNames := []*ast.IndexColName{{ + Column: &ast.ColumnName{Name: model.NewCIStr("c1")}, + Length: types.UnspecifiedLength}} + // Add some adding index jobs to AddIndexJobList. + backfillAddIndexJob := func(jobArgs []interface{}) { + kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { + var err error + t := meta.NewMeta(txn) + t.SetJobListKey(meta.AddIndexJobListKey) + job.ID, err = t.GenGlobalID() + c.Assert(err, IsNil) + failedJobIDs = append(failedJobIDs, job.ID) + job.Args = jobArgs + err = t.EnQueueDDLJob(job) + c.Assert(err, IsNil) + return nil + }) + } + + // Add a StateNone job. + indexName := model.NewCIStr("idx_none") + args := []interface{}{false, indexName, idxColNames, nil} + backfillAddIndexJob(args) + // Add a StateDeleteOnly job. + indexName = model.NewCIStr("idx_delete_only") + args = []interface{}{false, indexName, idxColNames, nil} + backfillAddIndexJob(args) + changeJobState := func() { + kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + t.SetJobListKey(meta.AddIndexJobListKey) + lastJobID := int64(len(failedJobIDs) - 1) + job, err1 := t.GetDDLJob(lastJobID) + c.Assert(err1, IsNil) + _, err1 = d.runDDLJob(t, job) + c.Assert(err1, IsNil) + _, err1 = updateSchemaVersion(t, job) + c.Assert(err1, IsNil) + err1 = t.UpdateDDLJob(lastJobID, job, true) + c.Assert(err1, IsNil) + return nil + }) + err := d.callHookOnChanged(nil) + c.Assert(err, IsNil) + } + changeJobState() + // Add a StateWriteReorganization job. + indexName = model.NewCIStr("idx_write_reorg") + args = []interface{}{false, indexName, idxColNames, nil} + backfillAddIndexJob(args) + changeJobState() // convert to delete only + changeJobState() // convert to write only + changeJobState() // convert to write reorg + + err := d.Stop() + c.Assert(err, IsNil) + // Make sure shouldCleanJobs is ture. + d = testNewDDL(context.Background(), nil, store, nil, nil, testLease) + defer d.Stop() + + // Make sure all DDL jobs are done. + for { + var isAllJobDone bool + kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + len, err := t.DDLJobQueueLen() + c.Assert(err, IsNil) + t.SetJobListKey(meta.AddIndexJobListKey) + addIndexLen, err := t.DDLJobQueueLen() + c.Assert(err, IsNil) + if len == 0 && addIndexLen == 0 { + isAllJobDone = true + } + return nil + }) + if isAllJobDone { + break + } + time.Sleep(time.Millisecond) + } + + // Check that the jobs in add index list are finished. + kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + for i, id := range failedJobIDs { + historyJob, err := t.GetHistoryDDLJob(id) + c.Assert(err, IsNil) + c.Assert(historyJob, NotNil, Commentf("job %v", historyJob)) + if i == 0 { + c.Assert(historyJob.State, Equals, model.JobStateCancelled) + } else { + c.Assert(historyJob.State, Equals, model.JobStateRollbackDone) + } + } + return nil + }) +} + +func (s *testDDLSuite) TestSchemaError(c *C) { store := testCreateStore(c, "test_schema_error") defer store.Close() @@ -118,7 +237,6 @@ func (s *testDDLSuite) TestSchemaError(c *C) { } func (s *testDDLSuite) TestTableError(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_table_error") defer store.Close() @@ -160,7 +278,6 @@ func (s *testDDLSuite) TestTableError(c *C) { } func (s *testDDLSuite) TestForeignKeyError(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_foreign_key_error") defer store.Close() @@ -179,7 +296,6 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) { } func (s *testDDLSuite) TestIndexError(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_index_error") defer store.Close() @@ -216,7 +332,6 @@ func (s *testDDLSuite) TestIndexError(c *C) { } func (s *testDDLSuite) TestColumnError(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_column_error") defer store.Close() d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) @@ -274,7 +389,7 @@ func testCheckJobCancelled(c *C, d *ddl, job *model.Job, state *model.SchemaStat t := meta.NewMeta(txn) historyJob, err := t.GetHistoryDDLJob(job.ID) c.Assert(err, IsNil) - c.Assert(historyJob.IsCancelled(), IsTrue, Commentf("histroy job %s", historyJob)) + c.Assert(historyJob.IsCancelled() || historyJob.IsRollbackDone(), IsTrue, Commentf("histroy job %s", historyJob)) if state != nil { c.Assert(historyJob.SchemaState, Equals, *state) } @@ -358,7 +473,6 @@ func buildCancelJobTests(firstID int64) []testCancelJob { } func (s *testDDLSuite) TestCancelJob(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_cancel_job") defer store.Close() d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) @@ -476,3 +590,53 @@ func (s *testDDLSuite) TestIgnorableSpec(c *C) { c.Assert(isIgnorableSpec(spec), IsTrue) } } + +func (s *testDDLSuite) TestBuildJobDependence(c *C) { + defer testleak.AfterTest(c)() + store := testCreateStore(c, "test_set_job_relation") + defer store.Close() + + job1 := &model.Job{ID: 1, TableID: 1} + job2 := &model.Job{ID: 2, TableID: 1} + job3 := &model.Job{ID: 3, TableID: 2} + job6 := &model.Job{ID: 6, TableID: 1} + job7 := &model.Job{ID: 7, TableID: 2} + kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + err := t.EnQueueDDLJob(job1) + c.Assert(err, IsNil) + err = t.EnQueueDDLJob(job2) + c.Assert(err, IsNil) + err = t.EnQueueDDLJob(job3) + c.Assert(err, IsNil) + err = t.EnQueueDDLJob(job6) + c.Assert(err, IsNil) + err = t.EnQueueDDLJob(job7) + c.Assert(err, IsNil) + return nil + }) + job4 := &model.Job{ID: 4, TableID: 1} + kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + err := buildJobDependence(t, job4) + c.Assert(err, IsNil) + c.Assert(job4.DependencyID, Equals, int64(2)) + return nil + }) + job5 := &model.Job{ID: 5, TableID: 2} + kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + err := buildJobDependence(t, job5) + c.Assert(err, IsNil) + c.Assert(job5.DependencyID, Equals, int64(3)) + return nil + }) + job8 := &model.Job{ID: 8, TableID: 3} + kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + err := buildJobDependence(t, job8) + c.Assert(err, IsNil) + c.Assert(job8.DependencyID, Equals, int64(0)) + return nil + }) +} diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 34b2ae98da71f..99da225c0b097 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -234,6 +234,16 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error startKey := tablecodec.EncodeTablePrefix(tableID) endKey := tablecodec.EncodeTablePrefix(tableID + 1) return doInsert(s, job.ID, tableID, startKey, endKey, now) + // ActionAddIndex needs do it, because it needs to be rolled back when it's canceled. + case model.ActionAddIndex: + tableID := job.TableID + var indexID int64 + if err := job.DecodeArgs(&indexID); err != nil { + return errors.Trace(err) + } + startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) + return doInsert(s, job.ID, indexID, startKey, endKey, now) case model.ActionDropIndex: tableID := job.TableID var indexName interface{} diff --git a/ddl/fail_test.go b/ddl/fail_test.go index 5ebd6a23b5ab0..eb5704d23a117 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -19,12 +19,10 @@ import ( "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" ) func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) { - defer testleak.AfterTest(c)() d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) defer d.Stop() // create table t_fail (c1 int, c2 int); diff --git a/ddl/index.go b/ddl/index.go index da0cc16392fab..7ba65e7edeb6f 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -391,10 +391,11 @@ func (d *ddl) onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { // Finish this job. if job.IsRollingback() { job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) + job.Args[0] = indexInfo.ID } else { job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + job.Args = append(job.Args, indexInfo.ID) } - job.Args = append(job.Args, indexInfo.ID) default: err = ErrInvalidTableState.Gen("invalid table state %v", tblInfo.State) } diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 43bb026667c5c..96caee931ac6a 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -35,6 +35,7 @@ type testIndexChangeSuite struct { } func (s *testIndexChangeSuite) SetUpSuite(c *C) { + testleak.BeforeTest() s.store = testCreateStore(c, "test_index_change") s.dbInfo = &model.DBInfo{ Name: model.NewCIStr("test_index_change"), @@ -47,8 +48,12 @@ func (s *testIndexChangeSuite) SetUpSuite(c *C) { c.Check(err, IsNil, Commentf("err %v", errors.ErrorStack(err))) } +func (s *testIndexChangeSuite) TearDownSuite(c *C) { + s.store.Close() + testleak.AfterTest(c)() +} + func (s *testIndexChangeSuite) TestIndexChange(c *C) { - defer testleak.AfterTest(c)() d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) defer d.Stop() // create table t (c1 int primary key, c2 int); diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index a0d8b64427f61..9b89757f9dae8 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testleak" "golang.org/x/net/context" ) @@ -34,7 +33,6 @@ func (k testCtxKeyType) String() string { const testCtxKey testCtxKeyType = 0 func (s *testDDLSuite) TestReorg(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_reorg") defer store.Close() @@ -142,7 +140,6 @@ func (s *testDDLSuite) TestReorg(c *C) { } func (s *testDDLSuite) TestReorgOwner(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_reorg_owner") defer store.Close() diff --git a/ddl/schema_test.go b/ddl/schema_test.go index d314243a52a96..cd9bef47f92bb 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -33,6 +33,14 @@ var _ = Suite(&testSchemaSuite{}) type testSchemaSuite struct{} +func (s *testSchemaSuite) SetUpSuite(c *C) { + testleak.BeforeTest() +} + +func (s *testSchemaSuite) TearDownSuite(c *C) { + testleak.AfterTest(c)() +} + func testSchemaInfo(c *C, d *ddl, name string) *model.DBInfo { var err error dbInfo := &model.DBInfo{ @@ -115,7 +123,6 @@ func testCheckSchemaState(c *C, d *ddl, dbInfo *model.DBInfo, state model.Schema } func (s *testSchemaSuite) TestSchema(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_schema") defer store.Close() d := testNewDDL(context.Background(), nil, store, nil, nil, testLease) @@ -176,7 +183,6 @@ func (s *testSchemaSuite) TestSchema(c *C) { } func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_schema_wait") defer store.Close() @@ -221,6 +227,7 @@ LOOP: case <-ticker.C: d.Stop() d.start(context.Background()) + time.Sleep(time.Millisecond * 20) case err := <-done: c.Assert(err, IsNil) break LOOP @@ -229,7 +236,6 @@ LOOP: } func (s *testSchemaSuite) TestSchemaResume(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_schema_resume") defer store.Close() @@ -239,22 +245,20 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) { testCheckOwner(c, d1, true) dbInfo := testSchemaInfo(c, d1, "test") - job := &model.Job{ SchemaID: dbInfo.ID, Type: model.ActionCreateSchema, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{dbInfo}, } - testRunInterruptedJob(c, d1, job) testCheckSchemaState(c, d1, dbInfo, model.StatePublic) + job = &model.Job{ SchemaID: dbInfo.ID, Type: model.ActionDropSchema, BinlogInfo: &model.HistoryInfo{}, } - testRunInterruptedJob(c, d1, job) testCheckSchemaState(c, d1, dbInfo, model.StateNone) } diff --git a/ddl/stat_test.go b/ddl/stat_test.go index e98d3528ac4ef..817919074d150 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -28,6 +28,14 @@ var _ = Suite(&testStatSuite{}) type testStatSuite struct { } +func (s *testStatSuite) SetUpSuite(c *C) { + testleak.BeforeTest() +} + +func (s *testStatSuite) TearDownSuite(c *C) { + testleak.AfterTest(c)() +} + func (s *testStatSuite) getDDLSchemaVer(c *C, d *ddl) int64 { m, err := d.Stats(nil) c.Assert(err, IsNil) @@ -36,7 +44,6 @@ func (s *testStatSuite) getDDLSchemaVer(c *C, d *ddl) int64 { } func (s *testStatSuite) TestStat(c *C) { - defer testleak.AfterTest(c)() store := testCreateStore(c, "test_stat") defer store.Close() @@ -78,6 +85,7 @@ LOOP: d.close() c.Assert(s.getDDLSchemaVer(c, d), GreaterEqual, ver) d.start(context.Background()) + time.Sleep(time.Millisecond * 20) case err := <-done: c.Assert(err, IsNil) // TODO: Get this information from etcd. diff --git a/ddl/table.go b/ddl/table.go index 19c5b2f65685e..7b63a971591f6 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -335,6 +335,27 @@ func (d *ddl) onRenameTable(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, nil } +func (d *ddl) onModifyTableComment(t *meta.Meta, job *model.Job) (ver int64, _ error) { + var comment string + if err := job.DecodeArgs(&comment); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + tblInfo, err := getTableInfo(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + tblInfo.Comment = comment + ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + return ver, nil +} + func checkTableNotExists(t *meta.Meta, job *model.Job, schemaID int64, tableName string) error { // Check this table's database. tables, err := t.ListTables(schemaID) diff --git a/ddl/table_test.go b/ddl/table_test.go index 633943436b132..3b3994700610f 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -186,6 +186,7 @@ func testGetTableWithError(d *ddl, schemaID, tableID int64) (table.Table, error) } func (s *testTableSuite) SetUpSuite(c *C) { + testleak.BeforeTest() s.store = testCreateStore(c, "test_table") s.d = testNewDDL(context.Background(), nil, s.store, nil, nil, testLease) @@ -197,10 +198,10 @@ func (s *testTableSuite) TearDownSuite(c *C) { testDropSchema(c, testNewContext(s.d), s.d, s.dbInfo) s.d.Stop() s.store.Close() + testleak.AfterTest(c)() } func (s *testTableSuite) TestTable(c *C) { - defer testleak.AfterTest(c)() d := s.d ctx := testNewContext(d) @@ -242,7 +243,6 @@ func (s *testTableSuite) TestTable(c *C) { } func (s *testTableSuite) TestTableResume(c *C) { - defer testleak.AfterTest(c)() d := s.d testCheckOwner(c, d, true) diff --git a/ddl/util/util.go b/ddl/util/util.go index eff6b8769431e..3761356bceb95 100644 --- a/ddl/util/util.go +++ b/ddl/util/util.go @@ -58,7 +58,7 @@ func LoadDeleteRanges(ctx sessionctx.Context, safePoint uint64) (ranges []DelRan chk := rs.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) if err != nil { return nil, errors.Trace(err) } diff --git a/distsql/distsql.go b/distsql/distsql.go index 4fca9ba2bd854..89a9de36ac4f1 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -36,18 +36,15 @@ var ( ) var ( - _ SelectResult = &selectResult{} - _ PartialResult = &partialResult{} + _ SelectResult = &selectResult{} ) // SelectResult is an iterator of coprocessor partial results. type SelectResult interface { - // Next gets the next partial result. - Next(context.Context) (PartialResult, error) // NextRaw gets the next raw result. NextRaw(context.Context) ([]byte, error) - // NextChunk reads the data into chunk. - NextChunk(context.Context, *chunk.Chunk) error + // Next reads the data into chunk. + Next(context.Context, *chunk.Chunk) error // Close closes the iterator. Close() error // Fetch fetches partial results from client. @@ -55,15 +52,6 @@ type SelectResult interface { Fetch(context.Context) } -// PartialResult is the result from a single region server. -type PartialResult interface { - // Next returns the next rowData of the sub result. - // If no more row to return, rowData would be nil. - Next(context.Context) (rowData []types.Datum, err error) - // Close closes the partial result. - Close() error -} - type selectResult struct { label string resp kv.Response @@ -121,23 +109,6 @@ func (r *selectResult) fetch(ctx context.Context) { } } -// Next returns the next row. -func (r *selectResult) Next(ctx context.Context) (PartialResult, error) { - re := <-r.results - if re.err != nil { - return nil, errors.Trace(re.err) - } - if re.result == nil { - return nil, nil - } - pr := &partialResult{} - pr.rowLen = r.rowLen - err := pr.unmarshal(re.result.GetData()) - r.feedback.Update(re.result.GetStartKey(), pr.resp.OutputCounts) - r.partialCount++ - return pr, errors.Trace(err) -} - // NextRaw returns the next raw partial result. func (r *selectResult) NextRaw(ctx context.Context) ([]byte, error) { re := <-r.results @@ -149,8 +120,8 @@ func (r *selectResult) NextRaw(ctx context.Context) ([]byte, error) { return re.result.GetData(), nil } -// NextChunk reads data to the chunk. -func (r *selectResult) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next reads data to the chunk. +func (r *selectResult) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() for chk.NumRows() < r.ctx.GetSessionVars().MaxChunkSize { if r.selectResp == nil || r.respChkIdx == len(r.selectResp.Chunks) { @@ -228,36 +199,6 @@ func (r *selectResult) Close() error { return r.resp.Close() } -type partialResult struct { - resp *tipb.SelectResponse - chunkIdx int - rowLen int -} - -func (pr *partialResult) unmarshal(resultSubset []byte) error { - pr.resp = new(tipb.SelectResponse) - err := pr.resp.Unmarshal(resultSubset) - if err != nil { - return errors.Trace(err) - } - - if pr.resp.Error != nil { - return errInvalidResp.Gen("[%d %s]", pr.resp.Error.GetCode(), pr.resp.Error.GetMsg()) - } - - return nil -} - -// Next returns the next row of the sub result. -// If no more row to return, data would be nil. -func (pr *partialResult) Next(ctx context.Context) (data []types.Datum, err error) { - nextChunk := pr.getChunk() - if nextChunk == nil { - return nil, nil - } - return readRowFromChunk(nextChunk, pr.rowLen) -} - func readRowFromChunk(chunk *tipb.Chunk, numCols int) (row []types.Datum, err error) { row = make([]types.Datum, numCols) for i := 0; i < numCols; i++ { @@ -271,24 +212,6 @@ func readRowFromChunk(chunk *tipb.Chunk, numCols int) (row []types.Datum, err er return } -func (pr *partialResult) getChunk() *tipb.Chunk { - for { - if pr.chunkIdx >= len(pr.resp.Chunks) { - return nil - } - currentChunk := &pr.resp.Chunks[pr.chunkIdx] - if len(currentChunk.RowsData) > 0 { - return currentChunk - } - pr.chunkIdx++ - } -} - -// Close closes the sub result. -func (pr *partialResult) Close() error { - return nil -} - // Select sends a DAG request, returns SelectResult. // In kvReq, KeyRanges is required, Concurrency/KeepOrder/Desc/IsolationLevel/Priority are optional. func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fieldTypes []*types.FieldType, fb *statistics.QueryFeedback) (SelectResult, error) { diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 3bdd8ed723d6a..27c1fbbb411ac 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -54,7 +54,7 @@ func (s *testSuite) TestSelectNormal(c *C) { colTypes = append(colTypes, colTypes[0]) colTypes = append(colTypes, colTypes[0]) - // Test NextChunk. + // Test Next. response, err := Select(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) c.Assert(err, IsNil) result, ok := response.(*selectResult) @@ -64,11 +64,11 @@ func (s *testSuite) TestSelectNormal(c *C) { response.Fetch(context.TODO()) - // Test NextChunk. + // Test Next. chk := chunk.NewChunk(colTypes) numAllRows := 0 for { - err = response.NextChunk(context.TODO(), chk) + err = response.Next(context.TODO(), chk) c.Assert(err, IsNil) numAllRows += chk.NumRows() if chk.NumRows() == 0 { @@ -108,7 +108,7 @@ func (s *testSuite) TestSelectStreaming(c *C) { s.sctx.GetSessionVars().EnableStreaming = true - // Test NextChunk. + // Test Next. response, err := Select(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) c.Assert(err, IsNil) result, ok := response.(*streamResult) @@ -117,11 +117,11 @@ func (s *testSuite) TestSelectStreaming(c *C) { response.Fetch(context.TODO()) - // Test NextChunk. + // Test Next. chk := chunk.NewChunk(colTypes) numAllRows := 0 for { - err = response.NextChunk(context.TODO(), chk) + err = response.Next(context.TODO(), chk) c.Assert(err, IsNil) numAllRows += chk.NumRows() if chk.NumRows() == 0 { diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 328977a98247b..83161d0f25dd7 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -125,7 +125,13 @@ func (builder *RequestBuilder) SetKeepOrder(order bool) *RequestBuilder { } func (builder *RequestBuilder) getIsolationLevel(sv *variable.SessionVars) kv.IsoLevel { - isoLevel, _ := sv.GetSystemVar(variable.TxnIsolation) + var isoLevel string + if sv.TxnIsolationLevelOneShot.State == 2 { + isoLevel = sv.TxnIsolationLevelOneShot.Value + } + if isoLevel == "" { + isoLevel, _ = sv.GetSystemVar(variable.TxnIsolation) + } if isoLevel == ast.ReadCommitted { return kv.RC } diff --git a/distsql/stream.go b/distsql/stream.go index bdbf241efb130..b91066f36e305 100644 --- a/distsql/stream.go +++ b/distsql/stream.go @@ -41,17 +41,7 @@ type streamResult struct { func (r *streamResult) Fetch(context.Context) {} -func (r *streamResult) Next(ctx context.Context) (PartialResult, error) { - var ret streamPartialResult - ret.rowLen = r.rowLen - finished, err := r.readDataFromResponse(ctx, r.resp, &ret.Chunk) - if err != nil || finished { - return nil, errors.Trace(err) - } - return &ret, nil -} - -func (r *streamResult) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (r *streamResult) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() maxChunkSize := r.ctx.GetSessionVars().MaxChunkSize for chk.NumRows() < maxChunkSize { @@ -157,20 +147,3 @@ func (r *streamResult) Close() error { metrics.DistSQLPartialCountHistogram.Observe(float64(r.partialCount)) return nil } - -// streamPartialResult implements PartialResult. -type streamPartialResult struct { - tipb.Chunk - rowLen int -} - -func (pr *streamPartialResult) Next(ctx context.Context) (data []types.Datum, err error) { - if len(pr.Chunk.RowsData) == 0 { - return nil, nil // partial result finished. - } - return readRowFromChunk(&pr.Chunk, pr.rowLen) -} - -func (pr *streamPartialResult) Close() error { - return nil -} diff --git a/domain/domain.go b/domain/domain.go index 41fd0bef3bd07..0e659ea7973e8 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -632,6 +632,10 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) defer loadHistogramTicker.Stop() gcStatsTicker := time.NewTicker(100 * lease) defer gcStatsTicker.Stop() + dumpFeedbackTicker := time.NewTicker(200 * lease) + defer dumpFeedbackTicker.Stop() + loadFeedbackTicker := time.NewTicker(5 * lease) + defer loadFeedbackTicker.Stop() statsHandle := do.StatsHandle() t := time.Now() err := statsHandle.InitStats(do.InfoSchema()) @@ -644,7 +648,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) for { select { case <-loadTicker.C: - err := statsHandle.Update(do.InfoSchema()) + err = statsHandle.Update(do.InfoSchema()) if err != nil { log.Error("[stats] update stats info fail: ", errors.ErrorStack(err)) } @@ -653,32 +657,45 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) return // This channel is sent only by ddl owner or the drop stats executor. case t := <-statsHandle.DDLEventCh(): - err := statsHandle.HandleDDLEvent(t) + err = statsHandle.HandleDDLEvent(t) if err != nil { log.Error("[stats] handle ddl event fail: ", errors.ErrorStack(err)) } case t := <-statsHandle.AnalyzeResultCh(): for i, hg := range t.Hist { - err := statistics.SaveStatsToStorage(ctx, t.TableID, t.Count, t.IsIndex, hg, t.Cms[i]) + err = statistics.SaveStatsToStorage(ctx, t.TableID, t.Count, t.IsIndex, hg, t.Cms[i]) if err != nil { log.Error("[stats] save histogram to storage fail: ", errors.ErrorStack(err)) } } case <-deltaUpdateTicker.C: - err := statsHandle.DumpStatsDeltaToKV() + err = statsHandle.DumpStatsDeltaToKV() if err != nil { log.Error("[stats] dump stats delta fail: ", errors.ErrorStack(err)) } case <-loadHistogramTicker.C: - err := statsHandle.LoadNeededHistograms() + err = statsHandle.LoadNeededHistograms() if err != nil { log.Error("[stats] load histograms fail: ", errors.ErrorStack(err)) } + case <-loadFeedbackTicker.C: + if !owner.IsOwner() { + continue + } + err = statsHandle.HandleUpdateStats(do.InfoSchema()) + if err != nil { + log.Errorf("[stats] update stats using feedback fail: ", errors.ErrorStack(err)) + } + case <-dumpFeedbackTicker.C: + err = statsHandle.DumpStatsFeedbackToKV() + if err != nil { + log.Error("[stats] dump stats feedback fail: ", errors.ErrorStack(err)) + } case <-gcStatsTicker.C: if !owner.IsOwner() { continue } - err := statsHandle.GCStats(do.InfoSchema(), do.DDL().GetLease()) + err = statsHandle.GCStats(do.InfoSchema(), do.DDL().GetLease()) if err != nil { log.Error("[stats] gc stats fail: ", errors.ErrorStack(err)) } diff --git a/executor/adapter.go b/executor/adapter.go index 8186c4db59326..63388b0660c70 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -79,13 +79,13 @@ func schema2ResultFields(schema *expression.Schema, defaultDB string) (rfs []*as return rfs } -// NextChunk use uses recordSet's executor to get next available chunk for later usage. +// Next use uses recordSet's executor to get next available chunk for later usage. // If chunk does not contain any rows, then we update last query found rows in session variable as current found rows. // The reason we need update is that chunk with 0 rows indicating we already finished current query, we need prepare for // next query. // If stmt is not nil and chunk with some rows inside, we simply update last query found rows by the number of row in chunk. -func (a *recordSet) NextChunk(ctx context.Context, chk *chunk.Chunk) error { - err := a.executor.NextChunk(ctx, chk) +func (a *recordSet) Next(ctx context.Context, chk *chunk.Chunk) error { + err := a.executor.Next(ctx, chk) if err != nil { a.lastErr = err return errors.Trace(err) @@ -262,7 +262,7 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co a.logSlowQuery(txnTS, err == nil) }() - err = e.NextChunk(ctx, e.newChunk()) + err = e.Next(ctx, e.newChunk()) if err != nil { return nil, errors.Trace(err) } diff --git a/executor/admin.go b/executor/admin.go index 9a72c0c80d54a..20483347d371c 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -38,6 +39,7 @@ import ( var ( _ Executor = &CheckIndexRangeExec{} _ Executor = &RecoverIndexExec{} + _ Executor = &CleanupIndexExec{} ) // CheckIndexRangeExec outputs the index values which has handle between begin and end. @@ -56,12 +58,12 @@ type CheckIndexRangeExec struct { cols []*model.ColumnInfo } -// NextChunk implements the Executor NextChunk interface. -func (e *CheckIndexRangeExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CheckIndexRangeExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() handleIdx := e.schema.Len() - 1 for { - err := e.result.NextChunk(ctx, e.srcChunk) + err := e.result.Next(ctx, e.srcChunk) if err != nil { return errors.Trace(err) } @@ -304,20 +306,6 @@ func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, err return totalAddedCnt, totalScanCnt, nil } -func (e *RecoverIndexExec) extractIdxVals(row chunk.Row, idxVals []types.Datum) []types.Datum { - if idxVals == nil { - idxVals = make([]types.Datum, 0, row.Len()-1) - } else { - idxVals = idxVals[:0] - } - - for i := 0; i < row.Len()-1; i++ { - colVal := row.GetDatum(i, &e.columns[i].FieldType) - idxVals = append(idxVals, *colVal.Copy()) - } - return idxVals -} - type recoverRows struct { handle int64 idxVals []types.Datum @@ -330,7 +318,7 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists result.scanRowCount = 0 for { - err := srcResult.NextChunk(ctx, e.srcChunk) + err := srcResult.Next(ctx, e.srcChunk) if err != nil { return nil, errors.Trace(err) } @@ -344,7 +332,7 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists return e.recoverRows, nil } handle := row.GetInt64(handleIdx) - idxVals := e.extractIdxVals(row, e.idxValsBufs[result.scanRowCount]) + idxVals := extractIdxVals(row, e.idxValsBufs[result.scanRowCount], e.colFieldTypes) e.idxValsBufs[result.scanRowCount] = idxVals e.recoverRows = append(e.recoverRows, recoverRows{handle: handle, idxVals: idxVals, skip: false}) result.scanRowCount++ @@ -373,7 +361,7 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) distinctFlags[i] = distinct } - values, err := txn.GetSnapshot().BatchGet(e.batchKeys) + values, err := kv.BatchGetValues(txn, e.batchKeys) if err != nil { return errors.Trace(err) } @@ -440,8 +428,8 @@ func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transa return result, nil } -// NextChunk implements the Executor NextChunk interface. -func (e *RecoverIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *RecoverIndexExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil @@ -457,3 +445,244 @@ func (e *RecoverIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) erro e.done = true return nil } + +// CleanupIndexExec represents a cleanup index executor. +// It is built from "admin cleanup index" statement, is used to delete +// dangling index data. +type CleanupIndexExec struct { + baseExecutor + + done bool + removeCnt uint64 + + index table.Index + table table.Table + + idxCols []*model.ColumnInfo + idxColFieldTypes []*types.FieldType + idxChunk *chunk.Chunk + + idxValues map[int64][]types.Datum + batchSize uint64 + batchKeys []kv.Key + idxValsBufs [][]types.Datum + lastIdxKey []byte + scanRowCnt uint64 +} + +func (e *CleanupIndexExec) getIdxColTypes() []*types.FieldType { + if e.idxColFieldTypes != nil { + return e.idxColFieldTypes + } + e.idxColFieldTypes = make([]*types.FieldType, 0, len(e.idxCols)) + for _, col := range e.idxCols { + e.idxColFieldTypes = append(e.idxColFieldTypes, &col.FieldType) + } + return e.idxColFieldTypes +} + +func (e *CleanupIndexExec) batchGetRecord(txn kv.Transaction) (map[string][]byte, error) { + for handle := range e.idxValues { + e.batchKeys = append(e.batchKeys, e.table.RecordKey(handle)) + } + values, err := kv.BatchGetValues(txn, e.batchKeys) + if err != nil { + return nil, errors.Trace(err) + } + return values, nil +} + +func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[string][]byte) error { + for _, k := range e.batchKeys { + if _, found := values[string(k)]; !found { + _, handle, err := tablecodec.DecodeRecordKey(k) + if err != nil { + return errors.Trace(err) + } + if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn, e.idxValues[handle], + handle); err != nil { + return errors.Trace(err) + } + e.removeCnt++ + if e.removeCnt%e.batchSize == 0 { + log.Infof("[cleaning up dangling index] table: %v, index: %v, count: %v.", + e.table.Meta().Name.String(), e.index.Meta().Name.String(), e.removeCnt) + } + } + } + return nil +} + +func extractIdxVals(row chunk.Row, idxVals []types.Datum, + fieldTypes []*types.FieldType) []types.Datum { + if idxVals == nil { + idxVals = make([]types.Datum, 0, row.Len()-1) + } else { + idxVals = idxVals[:0] + } + + for i := 0; i < row.Len()-1; i++ { + colVal := row.GetDatum(i, fieldTypes[i]) + idxVals = append(idxVals, *colVal.Copy()) + } + return idxVals +} + +func (e *CleanupIndexExec) fetchIndex(ctx context.Context, txn kv.Transaction) error { + result, err := e.buildIndexScan(ctx, txn) + if err != nil { + return errors.Trace(err) + } + defer terror.Call(result.Close) + + sc := e.ctx.GetSessionVars().StmtCtx + for { + err := result.Next(ctx, e.idxChunk) + if err != nil { + return errors.Trace(err) + } + if e.idxChunk.NumRows() == 0 { + return nil + } + iter := chunk.NewIterator4Chunk(e.idxChunk) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + handle := row.GetInt64(len(e.idxCols) - 1) + idxVals := extractIdxVals(row, e.idxValsBufs[e.scanRowCnt], e.idxColFieldTypes) + e.idxValsBufs[e.scanRowCnt] = idxVals + e.idxValues[handle] = idxVals + idxKey, _, err := e.index.GenIndexKey(sc, idxVals, handle, nil) + if err != nil { + return errors.Trace(err) + } + e.scanRowCnt++ + e.lastIdxKey = idxKey + if e.scanRowCnt >= e.batchSize { + return nil + } + } + } +} + +// Next implements the Executor Next interface. +func (e *CleanupIndexExec) Next(ctx context.Context, chk *chunk.Chunk) error { + chk.Reset() + if e.done { + return nil + } + for { + errInTxn := kv.RunInNewTxn(e.ctx.GetStore(), true, func(txn kv.Transaction) error { + err := e.fetchIndex(ctx, txn) + if err != nil { + return errors.Trace(err) + } + values, err := e.batchGetRecord(txn) + if err != nil { + return errors.Trace(err) + } + err = e.deleteDanglingIdx(txn, values) + if err != nil { + return errors.Trace(err) + } + return nil + }) + if errInTxn != nil { + return errors.Trace(errInTxn) + } + if e.scanRowCnt == 0 { + break + } + e.scanRowCnt = 0 + e.batchKeys = e.batchKeys[:0] + for k := range e.idxValues { + delete(e.idxValues, k) + } + } + e.done = true + chk.AppendUint64(0, e.removeCnt) + return nil +} + +func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transaction) (distsql.SelectResult, error) { + dagPB, err := e.buildIdxDAGPB(txn) + if err != nil { + return nil, errors.Trace(err) + } + sc := e.ctx.GetSessionVars().StmtCtx + var builder distsql.RequestBuilder + ranges := ranger.FullNewRange() + kvReq, err := builder.SetIndexRanges(sc, e.table.Meta().ID, e.index.Meta().ID, ranges). + SetDAGRequest(dagPB). + SetKeepOrder(true). + SetFromSessionVars(e.ctx.GetSessionVars()). + Build() + kvReq.KeyRanges[0].StartKey = kv.Key(e.lastIdxKey).PrefixNext() + kvReq.Concurrency = 1 + result, err := distsql.Select(ctx, e.ctx, kvReq, e.getIdxColTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) + if err != nil { + return nil, errors.Trace(err) + } + result.Fetch(ctx) + return result, nil +} + +// Open implements the Executor Open interface. +func (e *CleanupIndexExec) Open(ctx context.Context) error { + if err := e.baseExecutor.Open(ctx); err != nil { + return errors.Trace(err) + } + e.idxChunk = chunk.NewChunk(e.getIdxColTypes()) + e.idxValues = make(map[int64][]types.Datum, e.batchSize) + e.batchKeys = make([]kv.Key, 0, e.batchSize) + e.idxValsBufs = make([][]types.Datum, e.batchSize) + sc := e.ctx.GetSessionVars().StmtCtx + idxKey, _, err := e.index.GenIndexKey(sc, []types.Datum{{}}, math.MinInt64, nil) + if err != nil { + return errors.Trace(err) + } + e.lastIdxKey = idxKey + return nil +} + +func (e *CleanupIndexExec) buildIdxDAGPB(txn kv.Transaction) (*tipb.DAGRequest, error) { + dagReq := &tipb.DAGRequest{} + dagReq.StartTs = txn.StartTS() + dagReq.TimeZoneOffset = timeZoneOffset(e.ctx) + sc := e.ctx.GetSessionVars().StmtCtx + dagReq.Flags = statementContextToFlags(sc) + for i := range e.idxCols { + dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(i)) + } + + execPB := e.constructIndexScanPB() + dagReq.Executors = append(dagReq.Executors, execPB) + err := plan.SetPBColumnsDefaultValue(e.ctx, dagReq.Executors[0].IdxScan.Columns, e.idxCols) + if err != nil { + return nil, errors.Trace(err) + } + + limitExec := e.constructLimitPB() + dagReq.Executors = append(dagReq.Executors, limitExec) + + return dagReq, nil +} + +func (e *CleanupIndexExec) constructIndexScanPB() *tipb.Executor { + idxExec := &tipb.IndexScan{ + TableId: e.table.Meta().ID, + IndexId: e.index.Meta().ID, + Columns: plan.ColumnsToProto(e.idxCols, e.table.Meta().PKIsHandle), + } + return &tipb.Executor{Tp: tipb.ExecType_TypeIndexScan, IdxScan: idxExec} +} + +func (e *CleanupIndexExec) constructLimitPB() *tipb.Executor { + limitExec := &tipb.Limit{ + Limit: e.batchSize, + } + return &tipb.Executor{Tp: tipb.ExecType_TypeLimit, Limit: limitExec} +} + +// Close implements the Executor Close interface. +func (e *CleanupIndexExec) Close() error { + return nil +} diff --git a/executor/admin_test.go b/executor/admin_test.go index a7a799756f921..f083c1b75bf9a 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -213,6 +213,190 @@ func (s *testSuite) TestAdminRecoverIndex1(c *C) { tk.MustExec("admin check index admin_test `primary`") } +func (s *testSuite) TestAdminCleanupIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists admin_test") + tk.MustExec("create table admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), unique key(c2), key (c3))") + tk.MustExec("insert admin_test (c1, c2) values (1, 2), (3, 4), (-5, NULL)") + tk.MustExec("insert admin_test (c1, c3) values (7, 100), (9, 100), (11, NULL)") + + // pk is handle, no need to cleanup + _, err := tk.Exec("admin cleanup index admin_test `primary`") + c.Assert(err, NotNil) + r := tk.MustQuery("admin cleanup index admin_test c2") + r.Check(testkit.Rows("0")) + r = tk.MustQuery("admin cleanup index admin_test c3") + r.Check(testkit.Rows("0")) + + // Make some dangling index. + s.ctx = mock.NewContext() + s.ctx.Store = s.store + is := s.domain.InfoSchema() + dbName := model.NewCIStr("test") + tblName := model.NewCIStr("admin_test") + tbl, err := is.TableByName(dbName, tblName) + c.Assert(err, IsNil) + + tblInfo := tbl.Meta() + idxInfo2 := findIndexByName("c2", tblInfo.Indices) + indexOpr2 := tables.NewIndex(tblInfo, idxInfo2) + idxInfo3 := findIndexByName("c3", tblInfo.Indices) + indexOpr3 := tables.NewIndex(tblInfo, idxInfo3) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(1), -100) + c.Assert(err, IsNil) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(6), 100) + c.Assert(err, IsNil) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(nil), 101) + c.Assert(err, IsNil) + _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), 200) + c.Assert(err, IsNil) + _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), -200) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + _, err = tk.Exec("admin check table admin_test") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test c2") + c.Assert(err, NotNil) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("9")) + r = tk.MustQuery("admin cleanup index admin_test c2") + r.Check(testkit.Rows("3")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("6")) + tk.MustExec("admin check index admin_test c2") + + _, err = tk.Exec("admin check table admin_test") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test c3") + c.Assert(err, NotNil) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c3)") + r.Check(testkit.Rows("8")) + r = tk.MustQuery("admin cleanup index admin_test c3") + r.Check(testkit.Rows("2")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c3)") + r.Check(testkit.Rows("6")) + tk.MustExec("admin check index admin_test c3") + + tk.MustExec("admin check table admin_test") +} + +func (s *testSuite) TestAdminCleanupIndexPKNotHandle(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists admin_test") + tk.MustExec("create table admin_test (c1 int, c2 int, c3 int, primary key (c1, c2))") + tk.MustExec("insert admin_test (c1, c2) values (1, 2), (3, 4), (-5, 5)") + + r := tk.MustQuery("admin cleanup index admin_test `primary`") + r.Check(testkit.Rows("0")) + + // Make some dangling index. + s.ctx = mock.NewContext() + s.ctx.Store = s.store + is := s.domain.InfoSchema() + dbName := model.NewCIStr("test") + tblName := model.NewCIStr("admin_test") + tbl, err := is.TableByName(dbName, tblName) + c.Assert(err, IsNil) + + tblInfo := tbl.Meta() + idxInfo := findIndexByName("primary", tblInfo.Indices) + indexOpr := tables.NewIndex(tblInfo, idxInfo) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(7, 10), -100) + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(4, 6), 100) + c.Assert(err, IsNil) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(-7, 4), 101) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + _, err = tk.Exec("admin check table admin_test") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test `primary`") + c.Assert(err, NotNil) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(`primary`)") + r.Check(testkit.Rows("6")) + r = tk.MustQuery("admin cleanup index admin_test `primary`") + r.Check(testkit.Rows("3")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(`primary`)") + r.Check(testkit.Rows("3")) + tk.MustExec("admin check index admin_test `primary`") + tk.MustExec("admin check table admin_test") +} + +func (s *testSuite) TestAdminCleanupIndexMore(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists admin_test") + tk.MustExec("create table admin_test (c1 int, c2 int, unique key (c1, c2), key (c2))") + tk.MustExec("insert admin_test values (1, 2), (3, 4), (5, 6)") + + tk.MustExec("admin cleanup index admin_test c1") + tk.MustExec("admin cleanup index admin_test c2") + + // Make some dangling index. + s.ctx = mock.NewContext() + s.ctx.Store = s.store + is := s.domain.InfoSchema() + dbName := model.NewCIStr("test") + tblName := model.NewCIStr("admin_test") + tbl, err := is.TableByName(dbName, tblName) + c.Assert(err, IsNil) + + tblInfo := tbl.Meta() + idxInfo1 := findIndexByName("c1", tblInfo.Indices) + indexOpr1 := tables.NewIndex(tblInfo, idxInfo1) + idxInfo2 := findIndexByName("c2", tblInfo.Indices) + indexOpr2 := tables.NewIndex(tblInfo, idxInfo2) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + for i := 0; i < 2000; i++ { + c1 := int64(2*i + 7) + c2 := int64(2*i + 8) + _, err = indexOpr1.Create(s.ctx, txn, types.MakeDatums(c1, c2), c1) + c.Assert(err, IsNil) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(c2), c1) + c.Assert(err, IsNil) + } + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + _, err = tk.Exec("admin check table admin_test") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test c1") + c.Assert(err, NotNil) + _, err = tk.Exec("admin check index admin_test c2") + c.Assert(err, NotNil) + r := tk.MustQuery("SELECT COUNT(*) FROM admin_test") + r.Check(testkit.Rows("3")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c1)") + r.Check(testkit.Rows("2003")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("2003")) + r = tk.MustQuery("admin cleanup index admin_test c1") + r.Check(testkit.Rows("2000")) + r = tk.MustQuery("admin cleanup index admin_test c2") + r.Check(testkit.Rows("2000")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c1)") + r.Check(testkit.Rows("3")) + r = tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") + r.Check(testkit.Rows("3")) + tk.MustExec("admin check index admin_test c1") + tk.MustExec("admin check index admin_test c2") + tk.MustExec("admin check table admin_test") +} + func (s *testSuite) TestAdminCheckTable(c *C) { // test NULL value. tk := testkit.NewTestKit(c, s.store) diff --git a/executor/aggregate.go b/executor/aggregate.go index 1dd38edf2be3d..ab94b2f138d98 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -74,8 +74,8 @@ func (e *HashAggExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *HashAggExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *HashAggExec) Next(ctx context.Context, chk *chunk.Chunk) error { // In this stage we consider all data from src as a single group. if !e.executed { err := e.execute(ctx) @@ -110,11 +110,11 @@ func (e *HashAggExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { } } -// innerNextChunk fetches Chunks from src and update each aggregate function for each row in Chunk. +// execute fetches Chunks from src and update each aggregate function for each row in Chunk. func (e *HashAggExec) execute(ctx context.Context) (err error) { inputIter := chunk.NewIterator4Chunk(e.childrenResults[0]) for { - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -141,7 +141,7 @@ func (e *HashAggExec) execute(ctx context.Context) (err error) { } } -func (e *HashAggExec) getGroupKey(row types.Row) ([]byte, error) { +func (e *HashAggExec) getGroupKey(row chunk.Row) ([]byte, error) { vals := make([]types.Datum, 0, len(e.GroupByItems)) for _, item := range e.GroupByItems { v, err := item.Eval(row) @@ -217,8 +217,8 @@ func (e *StreamAggExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *StreamAggExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *StreamAggExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() for !e.executed && chk.NumRows() < e.maxChunkSize { @@ -264,7 +264,7 @@ func (e *StreamAggExec) fetchChildIfNecessary(ctx context.Context, chk *chunk.Ch return nil } - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -296,7 +296,7 @@ func (e *StreamAggExec) appendResult2Chunk(chk *chunk.Chunk) { } // meetNewGroup returns a value that represents if the new group is different from last group. -func (e *StreamAggExec) meetNewGroup(row types.Row) (bool, error) { +func (e *StreamAggExec) meetNewGroup(row chunk.Row) (bool, error) { if len(e.GroupByItems) == 0 { return false, nil } diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 01a9fe9bfeaa8..0b3453dc6a7b8 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -22,11 +22,8 @@ import ( ) func (s *testSuite) TestAggregation(c *C) { - plan.JoinConcurrency = 1 - defer func() { - plan.JoinConcurrency = 5 - }() tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_hash_join_concurrency=1") tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c int, d int)") @@ -243,7 +240,7 @@ func (s *testSuite) TestAggregation(c *C) { result = tk.MustQuery("select count(*) from information_schema.columns") // When adding new memory columns in information_schema, please update this variable. - columnCountOfAllInformationSchemaTables := "737" + columnCountOfAllInformationSchemaTables := "741" result.Check(testkit.Rows(columnCountOfAllInformationSchemaTables)) tk.MustExec("drop table if exists t1") @@ -296,6 +293,8 @@ func (s *testSuite) TestAggregation(c *C) { tk.MustExec(`insert into t values (6, '{"i": 0, "n": "n6"}')`) tk.MustExec(`insert into t values (7, '{"i": -1, "n": "n7"}')`) tk.MustQuery("select sum(tags->'$.i') from t").Check(testkit.Rows("14")) + + tk.MustExec("set @@tidb_hash_join_concurrency=5") } func (s *testSuite) TestStreamAggPushDown(c *C) { @@ -357,8 +356,8 @@ func (s *testSuite) TestGroupConcatAggr(c *C) { result = tk.MustQuery("select id, group_concat(name SEPARATOR ',') from test group by id") result.Check(testkit.Rows("1 10,20,30", "2 20", "3 200,500")) - result = tk.MustQuery("select id, group_concat(name SEPARATOR '%') from test group by id") - result.Check(testkit.Rows("1 10%20%30", "2 20", "3 200%500")) + result = tk.MustQuery(`select id, group_concat(name SEPARATOR '%') from test group by id`) + result.Check(testkit.Rows("1 10%20%30", "2 20", `3 200%500`)) result = tk.MustQuery("select id, group_concat(name SEPARATOR '') from test group by id") result.Check(testkit.Rows("1 102030", "2 20", "3 200500")) @@ -527,15 +526,13 @@ func (s *testSuite) TestAggEliminator(c *C) { func (s *testSuite) TestIssue5663(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - plan.GlobalPlanCache = kvcache.NewShardedLRUCache(2, 1) - if plan.GlobalPlanCache != nil { - plan.PlanCacheEnabled = true - defer func() { - plan.PlanCacheEnabled = false - }() - } + planCahche := tk.Se.GetSessionVars().PlanCacheEnabled + defer func() { + tk.Se.GetSessionVars().PlanCacheEnabled = planCahche + }() + tk.Se.GetSessionVars().PlanCacheEnabled = true tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1 (i int unsigned, primary key(i));") tk.MustExec("insert into t1 values (1),(2),(3);") diff --git a/executor/analyze.go b/executor/analyze.go index acfaba16b5e0a..534d9e961efd3 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" tipb "github.com/pingcap/tipb/go-tipb" @@ -52,8 +51,8 @@ const ( defaultCMSketchWidth = 2048 ) -// NextChunk implements the Executor NextChunk interface. -func (e *AnalyzeExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *AnalyzeExec) Next(ctx context.Context, chk *chunk.Chunk) error { concurrency, err := getBuildStatsConcurrency(e.ctx) if err != nil { return errors.Trace(err) @@ -176,9 +175,8 @@ type AnalyzeIndexExec struct { } func (e *AnalyzeIndexExec) open() error { - idxRange := &ranger.NewRange{LowVal: []types.Datum{types.MinNotNullDatum()}, HighVal: []types.Datum{types.MaxValueDatum()}} var builder distsql.RequestBuilder - kvReq, err := builder.SetIndexRanges(e.ctx.GetSessionVars().StmtCtx, e.tblInfo.ID, e.idxInfo.ID, []*ranger.NewRange{idxRange}). + kvReq, err := builder.SetIndexRanges(e.ctx.GetSessionVars().StmtCtx, e.tblInfo.ID, e.idxInfo.ID, ranger.FullNewRange()). SetAnalyzeRequest(e.analyzePB). SetKeepOrder(true). SetPriority(e.priority). diff --git a/executor/builder.go b/executor/builder.go index 24f2e3e327357..85fc49b163f04 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" tipb "github.com/pingcap/tipb/go-tipb" "golang.org/x/net/context" @@ -75,6 +74,8 @@ func (b *executorBuilder) build(p plan.Plan) Executor { return b.buildCheckIndex(v) case *plan.RecoverIndex: return b.buildRecoverIndex(v) + case *plan.CleanupIndex: + return b.buildCleanupIndex(v) case *plan.CheckIndexRange: return b.buildCheckIndexRange(v) case *plan.ChecksumTable: @@ -295,6 +296,54 @@ func (b *executorBuilder) buildRecoverIndex(v *plan.RecoverIndex) Executor { return e } +func buildCleanupIndexCols(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) []*model.ColumnInfo { + columns := make([]*model.ColumnInfo, 0, len(indexInfo.Columns)+1) + for _, idxCol := range indexInfo.Columns { + columns = append(columns, tblInfo.Columns[idxCol.Offset]) + } + handleColsInfo := &model.ColumnInfo{ + ID: model.ExtraHandleID, + Name: model.ExtraHandleName, + Offset: len(tblInfo.Columns), + } + handleColsInfo.FieldType = *types.NewFieldType(mysql.TypeLonglong) + columns = append(columns, handleColsInfo) + return columns +} + +func (b *executorBuilder) buildCleanupIndex(v *plan.CleanupIndex) Executor { + tblInfo := v.Table.TableInfo + t, err := b.is.TableByName(v.Table.Schema, tblInfo.Name) + if err != nil { + b.err = errors.Trace(err) + return nil + } + idxName := strings.ToLower(v.IndexName) + var index table.Index + for _, idx := range t.Indices() { + if idx.Meta().State != model.StatePublic { + continue + } + if idxName == idx.Meta().Name.L { + index = idx + break + } + } + + if index == nil { + b.err = errors.Errorf("index `%v` is not found in table `%v`.", v.IndexName, v.Table.Name.O) + return nil + } + e := &CleanupIndexExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + idxCols: buildCleanupIndexCols(tblInfo, index.Meta()), + index: index, + table: t, + batchSize: 20000, + } + return e +} + func (b *executorBuilder) buildCheckIndexRange(v *plan.CheckIndexRange) Executor { tb, err := b.is.TableByName(v.Table.Schema, v.Table.Name) if err != nil { @@ -970,10 +1019,7 @@ func (b *executorBuilder) buildApply(apply *plan.PhysicalApply) *NestedLoopApply outer: v.JoinType != plan.InnerJoin, resultGenerator: generator, outerSchema: apply.OuterSchema, - outerChunk: outerExec.newChunk(), - innerChunk: innerExec.newChunk(), } - e.innerList = chunk.NewList(innerExec.retTypes(), e.maxChunkSize) metrics.ExecutorCounter.WithLabelValues("NestedLoopApplyExec").Inc() return e } diff --git a/executor/checksum.go b/executor/checksum.go index e3b21953c6c0b..0beeaa89ef7ad 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -82,8 +82,8 @@ func (e *ChecksumTableExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ChecksumTableExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ChecksumTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil diff --git a/executor/ddl.go b/executor/ddl.go index bcf2f77a30cb4..23bf112f73c5c 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -38,8 +38,8 @@ type DDLExec struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *DDLExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err error) { +// Next implements the Executor Next interface. +func (e *DDLExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { if e.done { return nil } @@ -120,20 +120,7 @@ func (e *DDLExec) executeCreateDatabase(s *ast.CreateDatabaseStmt) error { } func (e *DDLExec) executeCreateTable(s *ast.CreateTableStmt) error { - ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} - var err error - if s.ReferTable == nil { - err = domain.GetDomain(e.ctx).DDL().CreateTable(e.ctx, ident, s.Cols, s.Constraints, s.Options) - } else { - referIdent := ast.Ident{Schema: s.ReferTable.Schema, Name: s.ReferTable.Name} - err = domain.GetDomain(e.ctx).DDL().CreateTableWithLike(e.ctx, ident, referIdent) - } - if infoschema.ErrTableExists.Equal(err) { - if s.IfNotExists { - return nil - } - return err - } + err := domain.GetDomain(e.ctx).DDL().CreateTable(e.ctx, s) return errors.Trace(err) } diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 265034802b25e..11597c2a36c8c 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -68,7 +68,7 @@ func (s *testSuite) TestCreateTable(c *C) { chk := rs.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err1 := rs.NextChunk(ctx, chk) + err1 := rs.Next(ctx, chk) c.Assert(err1, IsNil) if chk.NumRows() == 0 { break @@ -82,7 +82,7 @@ func (s *testSuite) TestCreateTable(c *C) { chk = rs.NewChunk() it = chunk.NewIterator4Chunk(chk) for { - err1 := rs.NextChunk(ctx, chk) + err1 := rs.Next(ctx, chk) c.Assert(err1, IsNil) if chk.NumRows() == 0 { break @@ -156,7 +156,7 @@ func (s *testSuite) TestAlterTableAddColumn(c *C) { r, err := tk.Exec("select c2 from alter_test") c.Assert(err, IsNil) chk := r.NewChunk() - err = r.NextChunk(context.Background(), chk) + err = r.Next(context.Background(), chk) c.Assert(err, IsNil) row := chk.GetRow(0) c.Assert(row.Len(), Equals, 1) diff --git a/executor/distsql.go b/executor/distsql.go index 3cda65c3c6e60..465e6746226a3 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -79,7 +78,7 @@ type lookupTableTask struct { // 4. task.memTracker.Consume(-task.memUsage) // // Step 1~3 are completed in "tableWorker.executeTask". - // Step 4 is completed in "IndexLookUpExecutor.NextChunk". + // Step 4 is completed in "IndexLookUpExecutor.Next". memUsage int64 memTracker *memory.Tracker } @@ -118,19 +117,6 @@ func closeAll(objs ...Closeable) error { return errors.Trace(err) } -func decodeRawValues(values []types.Datum, schema *expression.Schema, loc *time.Location) error { - var err error - for i := 0; i < schema.Len(); i++ { - if values[i].Kind() == types.KindRaw { - values[i], err = tablecodec.DecodeColumnValue(values[i].GetRaw(), schema.Columns[i].RetType, loc) - if err != nil { - return errors.Trace(err) - } - } - } - return nil -} - // timeZoneOffset returns the local time zone offset in seconds. func timeZoneOffset(ctx sessionctx.Context) int64 { loc := ctx.GetSessionVars().GetTimeZone() @@ -233,9 +219,9 @@ func (e *TableReaderExecutor) Close() error { return errors.Trace(err) } -// NextChunk fills data into the chunk passed by its caller. +// Next fills data into the chunk passed by its caller. // The task was actually done by tableReaderHandler. -func (e *TableReaderExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (e *TableReaderExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { err := e.resultHandler.nextChunk(ctx, chk) if err != nil { e.feedback.Invalidate() @@ -371,9 +357,9 @@ func (e *IndexReaderExecutor) Close() error { return errors.Trace(err) } -// NextChunk implements the Executor NextChunk interface. -func (e *IndexReaderExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { - err := e.result.NextChunk(ctx, chk) +// Next implements the Executor Next interface. +func (e *IndexReaderExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { + err := e.result.Next(ctx, chk) if err != nil { e.feedback.Invalidate() } @@ -602,8 +588,8 @@ func (e *IndexLookUpExecutor) Close() error { return nil } -// NextChunk implements Exec NextChunk interface. -func (e *IndexLookUpExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements Exec Next interface. +func (e *IndexLookUpExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() for { resultTask, err := e.getResultTask() @@ -705,7 +691,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult) (handles []int64, err error) { handles = make([]int64, 0, w.batchSize) for len(handles) < w.batchSize { - err = errors.Trace(idxResult.NextChunk(ctx, chk)) + err = errors.Trace(idxResult.Next(ctx, chk)) if err != nil { return handles, err } @@ -803,7 +789,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er task.rows = make([]chunk.Row, 0, handleCnt) for { chk := tableReader.newChunk() - err = tableReader.NextChunk(ctx, chk) + err = tableReader.Next(ctx, chk) if err != nil { log.Error(err) return errors.Trace(err) @@ -892,27 +878,9 @@ func (tr *tableResultHandler) open(optionalResult, result distsql.SelectResult) tr.optionalFinished = false } -func (tr *tableResultHandler) next(ctx context.Context) (partialResult distsql.PartialResult, err error) { - if !tr.optionalFinished { - partialResult, err = tr.optionalResult.Next(ctx) - if err != nil { - return nil, errors.Trace(err) - } - if partialResult != nil { - return partialResult, nil - } - tr.optionalFinished = true - } - partialResult, err = tr.result.Next(ctx) - if err != nil { - return nil, errors.Trace(err) - } - return partialResult, nil -} - func (tr *tableResultHandler) nextChunk(ctx context.Context, chk *chunk.Chunk) error { if !tr.optionalFinished { - err := tr.optionalResult.NextChunk(ctx, chk) + err := tr.optionalResult.Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -921,7 +889,7 @@ func (tr *tableResultHandler) nextChunk(ctx context.Context, chk *chunk.Chunk) e } tr.optionalFinished = true } - return tr.result.NextChunk(ctx, chk) + return tr.result.Next(ctx, chk) } func (tr *tableResultHandler) nextRaw(ctx context.Context) (data []byte, err error) { diff --git a/executor/distsql_test.go b/executor/distsql_test.go index e0dcd23d6ddc3..988c627741d75 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -55,7 +55,7 @@ func (s *testSuite) TestIndexDoubleReadClose(c *C) { rs, err := tk.Exec("select * from dist where c_idx between 0 and 100") c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.Background(), chk) + err = rs.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(err, IsNil) keyword := "pickAndExecTask" @@ -106,7 +106,7 @@ func (s *testSuite) TestCopClientSend(c *C) { c.Assert(err, IsNil) defer rs.Close() chk := rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).GetMyDecimal(0).String(), Equals, "499500") @@ -120,7 +120,7 @@ func (s *testSuite) TestCopClientSend(c *C) { rs, err = tk.Exec("select sum(id) from copclient") c.Assert(err, IsNil) chk = rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).GetMyDecimal(0).String(), Equals, "499500") rs.Close() @@ -129,7 +129,7 @@ func (s *testSuite) TestCopClientSend(c *C) { rs, err = tk.Exec("select * from copclient order by id") c.Assert(err, IsNil) chk = rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) rs.Close() keyword := "(*copIterator).work" diff --git a/executor/executor.go b/executor/executor.go index 59a1157790b27..882db36685ebf 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -66,42 +66,33 @@ var ( // Error instances. var ( - ErrUnknownPlan = terror.ClassExecutor.New(codeUnknownPlan, "Unknown plan") - ErrPrepareMulti = terror.ClassExecutor.New(codePrepareMulti, "Can not prepare multiple statements") - ErrPrepareDDL = terror.ClassExecutor.New(codePrepareDDL, "Can not prepare DDL statements") - ErrPasswordNoMatch = terror.ClassExecutor.New(CodePasswordNoMatch, "Can't find any matching row in the user table") - ErrResultIsEmpty = terror.ClassExecutor.New(codeResultIsEmpty, "result is empty") - ErrBuildExecutor = terror.ClassExecutor.New(codeErrBuildExec, "Failed to build executor") - ErrBatchInsertFail = terror.ClassExecutor.New(codeBatchInsertFail, "Batch insert failed, please clean the table and try again.") - ErrWrongValueCountOnRow = terror.ClassExecutor.New(codeWrongValueCountOnRow, "Column count doesn't match value count at row %d") - ErrPasswordFormat = terror.ClassExecutor.New(codePasswordFormat, "The password hash doesn't have the expected format. Check if the correct password algorithm is being used with the PASSWORD() function.") + ErrUnknownPlan = terror.ClassExecutor.New(codeUnknownPlan, "Unknown plan") + ErrPrepareMulti = terror.ClassExecutor.New(codePrepareMulti, "Can not prepare multiple statements") + ErrPrepareDDL = terror.ClassExecutor.New(codePrepareDDL, "Can not prepare DDL statements") + ErrPasswordNoMatch = terror.ClassExecutor.New(CodePasswordNoMatch, "Can't find any matching row in the user table") + ErrResultIsEmpty = terror.ClassExecutor.New(codeResultIsEmpty, "result is empty") + ErrBuildExecutor = terror.ClassExecutor.New(codeErrBuildExec, "Failed to build executor") + ErrBatchInsertFail = terror.ClassExecutor.New(codeBatchInsertFail, "Batch insert failed, please clean the table and try again.") + ErrWrongValueCountOnRow = terror.ClassExecutor.New(codeWrongValueCountOnRow, "Column count doesn't match value count at row %d") + ErrPasswordFormat = terror.ClassExecutor.New(codePasswordFormat, "The password hash doesn't have the expected format. Check if the correct password algorithm is being used with the PASSWORD() function.") + ErrCantChangeTxCharacteristics = terror.ClassExecutor.New(codeErrCantChangeTxCharacteristics, "Transaction characteristics can't be changed while a transaction is in progress") ) // Error codes. const ( - codeUnknownPlan terror.ErrCode = 1 - codePrepareMulti terror.ErrCode = 2 - codePrepareDDL terror.ErrCode = 7 - codeResultIsEmpty terror.ErrCode = 8 - codeErrBuildExec terror.ErrCode = 9 - codeBatchInsertFail terror.ErrCode = 10 - CodePasswordNoMatch terror.ErrCode = 1133 // MySQL error code - CodeCannotUser terror.ErrCode = 1396 // MySQL error code - codeWrongValueCountOnRow terror.ErrCode = 1136 // MySQL error code - codePasswordFormat terror.ErrCode = 1827 // MySQL error code + codeUnknownPlan terror.ErrCode = 1 + codePrepareMulti terror.ErrCode = 2 + codePrepareDDL terror.ErrCode = 7 + codeResultIsEmpty terror.ErrCode = 8 + codeErrBuildExec terror.ErrCode = 9 + codeBatchInsertFail terror.ErrCode = 10 + CodePasswordNoMatch terror.ErrCode = 1133 // MySQL error code + CodeCannotUser terror.ErrCode = 1396 // MySQL error code + codeWrongValueCountOnRow terror.ErrCode = 1136 // MySQL error code + codePasswordFormat terror.ErrCode = 1827 // MySQL error code + codeErrCantChangeTxCharacteristics terror.ErrCode = 1568 ) -// Row represents a result set row, it may be returned from a table, a join, or a projection. -// -// The following cases will need store the handle information: -// -// If the top plan is update or delete, then every executor will need the handle. -// If there is an union scan, then the below scan plan must store the handle. -// If there is sort need in the double read, then the table scan of the double read must store the handle. -// If there is a select for update. then we need to store the handle until the lock plan. But if there is aggregation, the handle info can be removed. -// Otherwise the executor's returned rows don't need to store the handle information. -type Row = types.DatumRow - type baseExecutor struct { ctx sessionctx.Context id string @@ -127,12 +118,6 @@ func (e *baseExecutor) Open(ctx context.Context) error { return nil } -// Next implements interface Executor. -// To be removed in near future. -func (e *baseExecutor) Next(context.Context) (Row, error) { - return nil, nil -} - // Close closes all executors and release all resources. func (e *baseExecutor) Close() error { for _, child := range e.children { @@ -163,8 +148,8 @@ func (e *baseExecutor) retTypes() []*types.FieldType { return e.retFieldTypes } -// NextChunk fills mutiple rows into a chunk. -func (e *baseExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next fills mutiple rows into a chunk. +func (e *baseExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { return nil } @@ -186,17 +171,24 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id strin return e } -// Executor executes a query. +// Executor is the physical implementation of a algebra operator. +// +// In TiDB, all algebra operators are implemented as iterators, i.e., they +// support a simple Open-Next-Close protocol. See this paper for more details: +// +// "Volcano-An Extensible and Parallel Query Evaluation System" +// +// Different from Volcano's execution model, a "Next" function call in TiDB will +// return a batch of rows, other than a single row in Volcano. +// NOTE: Executors must call "chk.Reset()" before appending their results to it. type Executor interface { - Next(context.Context) (Row, error) - Close() error Open(context.Context) error + Next(ctx context.Context, chk *chunk.Chunk) error + Close() error Schema() *expression.Schema + retTypes() []*types.FieldType newChunk() *chunk.Chunk - // NextChunk fills a chunk with multiple rows - // NOTE: chunk has to call Reset() method before any use. - NextChunk(ctx context.Context, chk *chunk.Chunk) error } // CancelDDLJobsExec represents a cancel DDL jobs executor. @@ -208,8 +200,8 @@ type CancelDDLJobsExec struct { errs []error } -// NextChunk implements the Executor NextChunk interface. -func (e *CancelDDLJobsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CancelDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= len(e.jobIDs) { return nil @@ -237,8 +229,8 @@ type ShowDDLExec struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *ShowDDLExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ShowDDLExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil @@ -296,8 +288,8 @@ func (e *ShowDDLJobQueriesExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ShowDDLJobQueriesExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ShowDDLJobQueriesExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= len(e.jobs) { return nil @@ -337,8 +329,8 @@ func (e *ShowDDLJobsExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ShowDDLJobsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ShowDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= len(e.jobs) { return nil @@ -372,8 +364,8 @@ func (e *CheckTableExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *CheckTableExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { if e.done { return nil } @@ -426,8 +418,8 @@ func (e *CheckIndexExec) Close() error { return errors.Trace(e.src.Close()) } -// NextChunk implements the Executor NextChunk interface. -func (e *CheckIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CheckIndexExec) Next(ctx context.Context, chk *chunk.Chunk) error { if e.done { return nil } @@ -439,7 +431,7 @@ func (e *CheckIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error } chk = e.src.newChunk() for { - err := e.src.NextChunk(ctx, chk) + err := e.src.Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -472,15 +464,15 @@ func (e *SelectLockExec) Open(ctx context.Context) error { txnCtx.ForUpdate = true for id := range e.Schema().TblID2Handle { // This operation is only for schema validator check. - txnCtx.UpdateDeltaForTable(id, 0, 0) + txnCtx.UpdateDeltaForTable(id, 0, 0, map[int64]int64{}) } return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *SelectLockExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *SelectLockExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -519,14 +511,14 @@ type LimitExec struct { meetFirstBatch bool } -// NextChunk implements the Executor NextChunk interface. -func (e *LimitExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *LimitExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= e.end { return nil } for !e.meetFirstBatch { - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -547,7 +539,7 @@ func (e *LimitExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { } e.cursor += batchSize } - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -596,7 +588,7 @@ func init() { } for { chk := exec.newChunk() - err = exec.NextChunk(ctx, chk) + err = exec.Next(ctx, chk) if err != nil { return rows, errors.Trace(err) } @@ -611,10 +603,11 @@ func init() { } } tableMySQLErrCodes := map[terror.ErrCode]uint16{ - CodeCannotUser: mysql.ErrCannotUser, - CodePasswordNoMatch: mysql.ErrPasswordNoMatch, - codeWrongValueCountOnRow: mysql.ErrWrongValueCountOnRow, - codePasswordFormat: mysql.ErrPasswordFormat, + CodeCannotUser: mysql.ErrCannotUser, + CodePasswordNoMatch: mysql.ErrPasswordNoMatch, + codeWrongValueCountOnRow: mysql.ErrWrongValueCountOnRow, + codePasswordFormat: mysql.ErrPasswordFormat, + codeErrCantChangeTxCharacteristics: mysql.ErrCantChangeTxCharacteristics, } terror.ErrClassToMySQLCodes[terror.ClassExecutor] = tableMySQLErrCodes } @@ -635,10 +628,10 @@ func (e *ProjectionExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ProjectionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ProjectionExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - if err := e.children[0].NextChunk(ctx, e.childrenResults[0]); err != nil { + if err := e.children[0].Next(ctx, e.childrenResults[0]); err != nil { return errors.Trace(err) } return errors.Trace(e.evaluatorSuit.Run(e.ctx, e.childrenResults[0], chk)) @@ -659,8 +652,8 @@ func (e *TableDualExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *TableDualExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *TableDualExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.numReturned >= e.numDualRows { return nil @@ -710,12 +703,12 @@ func (e *SelectionExec) Close() error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *SelectionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *SelectionExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.batched { - return errors.Trace(e.unBatchedNextChunk(ctx, chk)) + return errors.Trace(e.unBatchedNext(ctx, chk)) } for { @@ -728,7 +721,7 @@ func (e *SelectionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { } chk.AppendRow(e.inputRow) } - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -744,10 +737,10 @@ func (e *SelectionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { } } -// unBatchedNextChunk filters input rows one by one and returns once an input row is selected. +// unBatchedNext filters input rows one by one and returns once an input row is selected. // For sql with "SETVAR" in filter and "GETVAR" in projection, for example: "SELECT @a FROM t WHERE (@a := 2) > 0", // we have to set batch size to 1 to do the evaluation of filter and projection. -func (e *SelectionExec) unBatchedNextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (e *SelectionExec) unBatchedNext(ctx context.Context, chk *chunk.Chunk) error { for { for ; e.inputRow != e.inputIter.End(); e.inputRow = e.inputIter.Next() { selected, err := expression.EvalBool(e.ctx, e.filters, e.inputRow) @@ -760,7 +753,7 @@ func (e *SelectionExec) unBatchedNextChunk(ctx context.Context, chk *chunk.Chunk return nil } } - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -787,8 +780,8 @@ type TableScanExec struct { virtualTableChunkIdx int } -// NextChunk implements the Executor NextChunk interface. -func (e *TableScanExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *TableScanExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.isVirtualTable { return errors.Trace(e.nextChunk4InfoSchema(ctx, chk)) @@ -891,7 +884,7 @@ func (e *TableScanExec) seekRange(handle int64) (inRange bool) { } } -func (e *TableScanExec) getRow(handle int64) (Row, error) { +func (e *TableScanExec) getRow(handle int64) (types.DatumRow, error) { columns := make([]*table.Column, e.schema.Len()) for i, v := range e.columns { columns[i] = table.ToColumn(v) @@ -928,12 +921,12 @@ func (e *ExistsExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ExistsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ExistsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.evaluated { e.evaluated = true - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -963,14 +956,14 @@ func (e *MaxOneRowExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *MaxOneRowExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *MaxOneRowExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.evaluated { return nil } e.evaluated = true - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -1009,7 +1002,6 @@ type UnionExec struct { baseExecutor stopFetchData atomic.Value - cursor int wg sync.WaitGroup finished chan struct{} @@ -1083,7 +1075,7 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) { return case result.chk = <-e.resourcePools[childID]: } - result.err = errors.Trace(e.children[childID].NextChunk(ctx, result.chk)) + result.err = errors.Trace(e.children[childID].Next(ctx, result.chk)) if result.err == nil && result.chk.NumRows() == 0 { return } @@ -1095,8 +1087,8 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) { } } -// NextChunk implements the Executor NextChunk interface. -func (e *UnionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *UnionExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.initialized { e.initialize(ctx) diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 0df8915d0a4b9..3712b829b16c0 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -52,9 +52,9 @@ func (msm *mockSessionManager) Kill(cid uint64, query bool) { func (s *testExecSuite) TestShowProcessList(c *C) { // Compose schema. - names := []string{"Id", "User", "Host", "db", "Command", "Time", "State", "Info"} + names := []string{"Id", "User", "Host", "db", "Command", "Time", "State", "Info", "Mem"} ftypes := []byte{mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar, - mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeVarchar, mysql.TypeString} + mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeVarchar, mysql.TypeString, mysql.TypeLonglong} schema := buildSchema(names, ftypes) // Compose a mocked session manager. @@ -82,19 +82,24 @@ func (s *testExecSuite) TestShowProcessList(c *C) { } ctx := context.Background() + err := e.Open(ctx) + c.Assert(err, IsNil) + + chk := e.newChunk() + it := chunk.NewIterator4Chunk(chk) // Run test and check results. for _, p := range ps { - chk := e.newChunk() - err := e.NextChunk(context.Background(), chk) + err = e.Next(context.Background(), chk) c.Assert(err, IsNil) - it := chunk.NewIterator4Chunk(chk) for row := it.Begin(); row != it.End(); row = it.Next() { c.Assert(row.GetUint64(0), Equals, p.ID) } } - r, err := e.Next(ctx) + err = e.Next(context.Background(), chk) + c.Assert(err, IsNil) + c.Assert(chk.NumRows(), Equals, 0) + err = e.Close() c.Assert(err, IsNil) - c.Assert(r, IsNil) } func buildSchema(names []string, ftypes []byte) *expression.Schema { diff --git a/executor/executor_test.go b/executor/executor_test.go index aed0dd9dc9591..6cd648b821bf9 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -73,7 +73,7 @@ var _ = Suite(&testContextOptionSuite{}) type testSuite struct { cluster *mocktikv.Cluster - mvccStore *mocktikv.MvccStore + mvccStore mocktikv.MVCCStore store kv.Storage domain *domain.Domain *parser.Parser @@ -85,6 +85,7 @@ type testSuite struct { var mockTikv = flag.Bool("mockTikv", true, "use mock tikv store in executor test") func (s *testSuite) SetUpSuite(c *C) { + testleak.BeforeTest() s.autoIDStep = autoid.GetStep() autoid.SetStep(5000) s.Parser = parser.New() @@ -93,7 +94,7 @@ func (s *testSuite) SetUpSuite(c *C) { if useMockTikv { s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.NewMvccStore() + s.mvccStore = mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(s.cluster), mockstore.WithMVCCStore(s.mvccStore), @@ -112,10 +113,7 @@ func (s *testSuite) TearDownSuite(c *C) { s.domain.Close() s.store.Close() autoid.SetStep(s.autoIDStep) -} - -func (s *testSuite) SetUpTest(c *C) { - testleak.BeforeTest() + testleak.AfterTest(c)() } func (s *testSuite) TearDownTest(c *C) { @@ -126,7 +124,6 @@ func (s *testSuite) TearDownTest(c *C) { tableName := tb[0] tk.MustExec(fmt.Sprintf("drop table %v", tableName)) } - testleak.AfterTest(c)() } func (s *testSuite) TestAdmin(c *C) { @@ -141,7 +138,7 @@ func (s *testSuite) TestAdmin(c *C) { r, err := tk.Exec("admin cancel ddl jobs 1") c.Assert(err, IsNil, Commentf("err %v", err)) chk := r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) row := chk.GetRow(0) c.Assert(row.Len(), Equals, 2) @@ -151,7 +148,7 @@ func (s *testSuite) TestAdmin(c *C) { r, err = tk.Exec("admin show ddl") c.Assert(err, IsNil) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) row = chk.GetRow(0) c.Assert(row.Len(), Equals, 4) @@ -166,7 +163,7 @@ func (s *testSuite) TestAdmin(c *C) { // c.Assert(rowOwnerInfos[0], Equals, ownerInfos[0]) c.Assert(row.GetString(2), Equals, "") chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsTrue) err = txn.Rollback() @@ -176,7 +173,7 @@ func (s *testSuite) TestAdmin(c *C) { r, err = tk.Exec("admin show ddl jobs") c.Assert(err, IsNil) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) row = chk.GetRow(0) c.Assert(row.Len(), Equals, 2) @@ -759,7 +756,7 @@ func (s *testSuite) TestIssue2612(c *C) { rs, err := tk.Exec(`select timediff(finish_at, create_at) from t;`) c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.Background(), chk) + err = rs.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).GetDuration(0).String(), Equals, "-46:09:02") } @@ -2158,6 +2155,19 @@ func (s *testContextOptionSuite) TestAddIndexPriority(c *C) { store.Close() } +func (s *testContextOptionSuite) TestAlterTableComment(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t_1") + tk.MustExec("create table t_1 (c1 int, c2 int, c3 int default 1, index (c1)) comment = 'test table';") + tk.MustExec("alter table `t_1` comment 'this is table comment';") + result := tk.MustQuery("select table_comment from information_schema.tables where table_name = 't_1';") + result.Check(testkit.Rows("this is table comment")) + tk.MustExec("alter table `t_1` comment 'table t comment';") + result = tk.MustQuery("select table_comment from information_schema.tables where table_name = 't_1';") + result.Check(testkit.Rows("table t comment")) +} + func (s *testContextOptionSuite) TestCoprocessorPriority(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -2300,7 +2310,7 @@ func (s *testSuite) TestBit(c *C) { r, err := tk.Exec("select * from t where c1 = 2") c.Assert(err, IsNil) chk := r.NewChunk() - err = r.NextChunk(context.Background(), chk) + err = r.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(types.BinaryLiteral(chk.GetRow(0).GetBytes(0)), DeepEquals, types.NewBinaryLiteralFromUint(2, -1)) @@ -2465,7 +2475,7 @@ func (s *testSuite) TestEarlyClose(c *C) { c.Assert(err1, IsNil) rs := rss[0] chk := rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) rs.Close() } @@ -2477,7 +2487,7 @@ func (s *testSuite) TestEarlyClose(c *C) { c.Assert(err, IsNil) rs := rss[0] chk := rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, NotNil) rs.Close() } @@ -2643,7 +2653,7 @@ func (s *testSuite) TestCoprocessorStreamingFlag(c *C) { rs, err := tk.Se.Execute(ctx1, test.sql) c.Assert(err, IsNil) chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) rs[0].Close() } diff --git a/executor/explain.go b/executor/explain.go index f7af00687fe49..781f026c736af 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -33,8 +33,8 @@ func (e *ExplainExec) Close() error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ExplainExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ExplainExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= len(e.rows) { return nil diff --git a/executor/grant.go b/executor/grant.go index d9522f6566250..5d82141cc8cea 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -52,8 +52,8 @@ type GrantExec struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *GrantExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *GrantExec) Next(ctx context.Context, chk *chunk.Chunk) error { if e.done { return nil } diff --git a/executor/grant_test.go b/executor/grant_test.go index e16ec96c1179a..db198f94f454e 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -183,19 +183,19 @@ func (s *testSuite) TestIssue2456(c *C) { func (s *testSuite) TestCreateUserWhenGrant(c *C) { tk := testkit.NewTestKit(c, s.store) - tk.MustExec("DROP USER IF EXISTS 'test'@'%'") - tk.MustExec("GRANT ALL PRIVILEGES ON *.* to 'test'@'%' IDENTIFIED BY 'xxx'") + tk.MustExec(`DROP USER IF EXISTS 'test'@'%'`) + tk.MustExec(`GRANT ALL PRIVILEGES ON *.* to 'test'@'%' IDENTIFIED BY 'xxx'`) // Make sure user is created automatically when grant to a non-exists one. - tk.MustQuery("SELECT user FROM mysql.user WHERE user='test' and host='%'").Check( + tk.MustQuery(`SELECT user FROM mysql.user WHERE user='test' and host='%'`).Check( testkit.Rows("test"), ) } func (s *testSuite) TestIssue2654(c *C) { tk := testkit.NewTestKit(c, s.store) - tk.MustExec("DROP USER IF EXISTS 'test'@'%'") - tk.MustExec("CREATE USER 'test'@'%' IDENTIFIED BY 'test'") + tk.MustExec(`DROP USER IF EXISTS 'test'@'%'`) + tk.MustExec(`CREATE USER 'test'@'%' IDENTIFIED BY 'test'`) tk.MustExec("GRANT SELECT ON test.* to 'test'") - rows := tk.MustQuery("SELECT user,host FROM mysql.user WHERE user='test' and host='%'") - rows.Check(testkit.Rows("test %")) + rows := tk.MustQuery(`SELECT user,host FROM mysql.user WHERE user='test' and host='%'`) + rows.Check(testkit.Rows(`test %`)) } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 970ea14c4c3f3..270036ac91b12 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -139,7 +139,7 @@ func (e *IndexLookUpJoin) Open(ctx context.Context) error { } func (e *IndexLookUpJoin) startWorkers(ctx context.Context) { - concurrency := e.ctx.GetSessionVars().IndexLookupConcurrency + concurrency := e.ctx.GetSessionVars().IndexLookupJoinConcurrency resultCh := make(chan *lookUpJoinTask, concurrency) e.resultCh = resultCh workerCtx, cancelFunc := context.WithCancel(ctx) @@ -186,8 +186,8 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork return iw } -// NextChunk implements the Executor interface. -func (e *IndexLookUpJoin) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor interface. +func (e *IndexLookUpJoin) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() e.joinResult.Reset() for { @@ -206,9 +206,9 @@ func (e *IndexLookUpJoin) NextChunk(ctx context.Context, chk *chunk.Chunk) error outerRow := task.outerResult.GetRow(task.cursor) if e.innerIter.Len() == 0 { - err = e.resultGenerator.emitToChunk(outerRow, nil, chk) + err = e.resultGenerator.emit(outerRow, nil, chk) } else if e.innerIter.Current() != e.innerIter.End() { - err = e.resultGenerator.emitToChunk(outerRow, e.innerIter, chk) + err = e.resultGenerator.emit(outerRow, e.innerIter, chk) } if err != nil { return errors.Trace(err) @@ -328,7 +328,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { task.memTracker.Consume(task.outerResult.MemoryUsage()) for task.outerResult.NumRows() < ow.batchSize { - err := ow.executor.NextChunk(ctx, ow.executorChk) + err := ow.executor.Next(ctx, ow.executorChk) if err != nil { return task, errors.Trace(err) } @@ -392,9 +392,6 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { err := iw.handleTask(ctx, task) task.doneCh <- errors.Trace(err) - if err != nil { - return - } } } @@ -514,7 +511,7 @@ func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTa innerResult.GetMemTracker().SetLabel("inner result") innerResult.GetMemTracker().AttachTo(task.memTracker) for { - err := innerExec.NextChunk(ctx, iw.executorChk) + err := innerExec.Next(ctx, iw.executorChk) if err != nil { return errors.Trace(err) } diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go new file mode 100644 index 0000000000000..ecca46773e1d9 --- /dev/null +++ b/executor/index_lookup_join_test.go @@ -0,0 +1,39 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor_test + +import ( + "context" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/util/testkit" +) + +func (s *testSuite) TestIndexLookupJoinHang(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table idxJoinOuter (a int unsigned)") + tk.MustExec("create table idxJoinInner (a int unsigned unique)") + tk.MustExec("insert idxJoinOuter values (1), (1), (1), (1), (1)") + tk.MustExec("insert idxJoinInner values (1)") + tk.Se.GetSessionVars().IndexJoinBatchSize = 1 + tk.Se.GetSessionVars().IndexLookupJoinConcurrency = 1 + + rs, err := tk.Exec("select /*+ TIDB_INLJ(o, i)*/ * from idxJoinOuter o left join idxJoinInner i on o.a = i.a where o.a in (1, 2) and (i.a - 3) > 0") + c.Assert(err, IsNil) + chk := rs.NewChunk() + for i := 0; i < 5; i++ { + rs.Next(context.Background(), chk) + } + rs.Close() +} diff --git a/executor/join.go b/executor/join.go index 12b8932ed9e65..1047002cce571 100644 --- a/executor/join.go +++ b/executor/join.go @@ -211,7 +211,7 @@ func (e *HashJoinExec) fetchOuterChunks(ctx context.Context) { } } outerResult := outerResource.chk - err := e.outerExec.NextChunk(ctx, outerResult) + err := e.outerExec.Next(ctx, outerResult) if err != nil { e.joinResultCh <- &hashjoinWorkerResult{ err: errors.Trace(err), @@ -233,7 +233,7 @@ func (e *HashJoinExec) fetchInnerRows(ctx context.Context) (err error) { e.innerResult.GetMemTracker().SetLabel("innerResult") for { chk := e.children[e.innerIdx].newChunk() - err = e.innerExec.NextChunk(ctx, chk) + err = e.innerExec.Next(ctx, chk) if err != nil || chk.NumRows() == 0 { return errors.Trace(err) } @@ -356,7 +356,7 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R return false, joinResult } if hasNull { - err = e.resultGenerators[workerID].emitToChunk(outerRow, nil, joinResult.chk) + err = e.resultGenerators[workerID].emit(outerRow, nil, joinResult.chk) if err != nil { joinResult.err = errors.Trace(err) } @@ -365,7 +365,7 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R e.hashTableValBufs[workerID] = e.hashTable.Get(joinKey, e.hashTableValBufs[workerID][:0]) innerPtrs := e.hashTableValBufs[workerID] if len(innerPtrs) == 0 { - err = e.resultGenerators[workerID].emitToChunk(outerRow, nil, joinResult.chk) + err = e.resultGenerators[workerID].emit(outerRow, nil, joinResult.chk) if err != nil { joinResult.err = errors.Trace(err) } @@ -379,7 +379,7 @@ func (e *HashJoinExec) joinMatchedOuterRow2Chunk(workerID uint, outerRow chunk.R } iter := chunk.NewIterator4Slice(innerRows) for iter.Begin(); iter.Current() != iter.End(); { - err = e.resultGenerators[workerID].emitToChunk(outerRow, iter, joinResult.chk) + err = e.resultGenerators[workerID].emit(outerRow, iter, joinResult.chk) if err != nil { joinResult.err = errors.Trace(err) return false, joinResult @@ -419,7 +419,7 @@ func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResu } for i := range selected { if !selected[i] { // process unmatched outer rows - err = e.resultGenerators[workerID].emitToChunk(outerChk.GetRow(i), nil, joinResult.chk) + err = e.resultGenerators[workerID].emit(outerChk.GetRow(i), nil, joinResult.chk) if err != nil { joinResult.err = errors.Trace(err) return false, joinResult @@ -441,11 +441,11 @@ func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResu return true, joinResult } -// NextChunk implements the Executor NextChunk interface. +// Next implements the Executor Next interface. // hash join constructs the result following these steps: // step 1. fetch data from inner child and build a hash table; // step 2. fetch data from outer child in a background goroutine and probe the hash table in multiple join workers. -func (e *HashJoinExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err error) { +func (e *HashJoinExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { if !e.prepared { if err = e.fetchInnerRows(ctx); err != nil { return errors.Trace(err) @@ -473,13 +473,45 @@ func (e *HashJoinExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err err return nil } +// buildHashTableForList builds hash table from `list`. +// key of hash table: hash value of key columns +// value of hash table: RowPtr of the corresponded row +func (e *HashJoinExec) buildHashTableForList() error { + e.hashTable = mvmap.NewMVMap() + e.innerKeyColIdx = make([]int, len(e.innerKeys)) + for i := range e.innerKeys { + e.innerKeyColIdx[i] = e.innerKeys[i].Index + } + var ( + hasNull bool + err error + keyBuf = make([]byte, 0, 64) + valBuf = make([]byte, 8) + ) + for i := 0; i < e.innerResult.NumChunks(); i++ { + chk := e.innerResult.GetChunk(i) + for j := 0; j < chk.NumRows(); j++ { + hasNull, keyBuf, err = e.getJoinKeyFromChkRow(false, chk.GetRow(j), keyBuf) + if err != nil { + return errors.Trace(err) + } + if hasNull { + continue + } + rowPtr := chunk.RowPtr{ChkIdx: uint32(i), RowIdx: uint32(j)} + *(*chunk.RowPtr)(unsafe.Pointer(&valBuf[0])) = rowPtr + e.hashTable.Put(keyBuf, valBuf) + } + } + return nil +} + // NestedLoopApplyExec is the executor for apply. type NestedLoopApplyExec struct { baseExecutor - innerRows []Row + innerRows []chunk.Row cursor int - resultRows []Row innerExec Executor outerExec Executor innerFilter expression.CNFExprs @@ -498,28 +530,45 @@ type NestedLoopApplyExec struct { innerSelected []bool innerIter chunk.Iterator outerRow *chunk.Row + + memTracker *memory.Tracker // track memory usage. } // Close implements the Executor interface. func (e *NestedLoopApplyExec) Close() error { - e.resultRows = nil e.innerRows = nil + + e.memTracker.Detach() + e.memTracker = nil return errors.Trace(e.outerExec.Close()) } // Open implements the Executor interface. func (e *NestedLoopApplyExec) Open(ctx context.Context) error { + err := e.outerExec.Open(ctx) + if err != nil { + return errors.Trace(err) + } e.cursor = 0 - e.resultRows = e.resultRows[:0] e.innerRows = e.innerRows[:0] - return errors.Trace(e.outerExec.Open(ctx)) + e.outerChunk = e.outerExec.newChunk() + e.innerChunk = e.innerExec.newChunk() + e.innerList = chunk.NewList(e.innerExec.retTypes(), e.maxChunkSize) + + e.memTracker = memory.NewTracker(e.id, e.ctx.GetSessionVars().MemQuotaNestedLoopApply) + e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) + + e.innerList.GetMemTracker().SetLabel("innerList") + e.innerList.GetMemTracker().AttachTo(e.memTracker) + + return nil } func (e *NestedLoopApplyExec) fetchSelectedOuterRow(ctx context.Context, chk *chunk.Chunk) (*chunk.Row, error) { outerIter := chunk.NewIterator4Chunk(e.outerChunk) for { if e.outerChunkCursor >= e.outerChunk.NumRows() { - err := e.outerExec.NextChunk(ctx, e.outerChunk) + err := e.outerExec.Next(ctx, e.outerChunk) if err != nil { return nil, errors.Trace(err) } @@ -538,7 +587,7 @@ func (e *NestedLoopApplyExec) fetchSelectedOuterRow(ctx context.Context, chk *ch if selected { return &outerRow, nil } else if e.outer { - err := e.resultGenerator.emitToChunk(outerRow, nil, chk) + err := e.resultGenerator.emit(outerRow, nil, chk) if err != nil || chk.NumRows() == e.maxChunkSize { return nil, errors.Trace(err) } @@ -556,7 +605,7 @@ func (e *NestedLoopApplyExec) fetchAllInners(ctx context.Context) error { e.innerList.Reset() innerIter := chunk.NewIterator4Chunk(e.innerChunk) for { - err := e.innerExec.NextChunk(ctx, e.innerChunk) + err := e.innerExec.Next(ctx, e.innerChunk) if err != nil { return errors.Trace(err) } @@ -576,41 +625,8 @@ func (e *NestedLoopApplyExec) fetchAllInners(ctx context.Context) error { } } -// buildHashTableForList builds hash table from `list`. -// key of hash table: hash value of key columns -// value of hash table: RowPtr of the corresponded row -func (e *HashJoinExec) buildHashTableForList() error { - e.hashTable = mvmap.NewMVMap() - e.innerKeyColIdx = make([]int, len(e.innerKeys)) - for i := range e.innerKeys { - e.innerKeyColIdx[i] = e.innerKeys[i].Index - } - var ( - hasNull bool - err error - keyBuf = make([]byte, 0, 64) - valBuf = make([]byte, 8) - ) - for i := 0; i < e.innerResult.NumChunks(); i++ { - chk := e.innerResult.GetChunk(i) - for j := 0; j < chk.NumRows(); j++ { - hasNull, keyBuf, err = e.getJoinKeyFromChkRow(false, chk.GetRow(j), keyBuf) - if err != nil { - return errors.Trace(err) - } - if hasNull { - continue - } - rowPtr := chunk.RowPtr{ChkIdx: uint32(i), RowIdx: uint32(j)} - *(*chunk.RowPtr)(unsafe.Pointer(&valBuf[0])) = rowPtr - e.hashTable.Put(keyBuf, valBuf) - } - } - return nil -} - -// NextChunk implements the Executor interface. -func (e *NestedLoopApplyExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err error) { +// Next implements the Executor interface. +func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { chk.Reset() for { if e.innerIter == nil || e.innerIter.Current() == e.innerIter.End() { @@ -629,7 +645,7 @@ func (e *NestedLoopApplyExec) NextChunk(ctx context.Context, chk *chunk.Chunk) ( e.innerIter.Begin() } - err = e.resultGenerator.emitToChunk(*e.outerRow, e.innerIter, chk) + err = e.resultGenerator.emit(*e.outerRow, e.innerIter, chk) if err != nil || chk.NumRows() == e.maxChunkSize { return errors.Trace(err) } diff --git a/executor/join_result_generators.go b/executor/join_result_generators.go index 7a08c488a0d4d..62e6d2a2321fc 100644 --- a/executor/join_result_generators.go +++ b/executor/join_result_generators.go @@ -35,7 +35,7 @@ var ( // joinResultGenerator is used to generate join results according the join type, see every implementor for detailed information. type joinResultGenerator interface { // emit tries to join an outer row with a batch of inner rows. - // When len(inners) == 0, it means that the outer row can not be joined with any inner row: + // When inners == nil or inners.Len() == 0, it means that the outer row can not be joined with any inner row: // 1. SemiJoin: unmatched outer row is ignored. // 2. AntiSemiJoin: unmatched outer row is appended to the result buffer. // 3. LeftOuterSemiJoin: unmatched outer row is appended with 0 and appended to the result buffer. @@ -43,22 +43,18 @@ type joinResultGenerator interface { // 5. LeftOuterJoin: unmatched outer row is joined with a row of NULLs and appended to the result buffer. // 6. RightOuterJoin: unmatched outer row is joined with a row of NULLs and appended to the result buffer. // 7. InnerJoin: unmatched outer row is ignored. - // When len(inner) != 0 but all the joined rows are filtered, this means that the outer row is unmatched and the above action is tacked as well. - // Otherwise, the outer row is matched and some joined rows is appended to the result buffer. - emit(outer Row, inners []Row, resultBuffer []Row) ([]Row, error) - - // emitToChunk takes the same operation as emit, but the joined rows is appended to `chk` instead of a result buffer. + // When inners.Len != 0 but all the joined rows are filtered, this means that the outer row is unmatched and the above action is tacked as well. + // Otherwise, the outer row is matched and some joined rows is appended to the `chk`. // The size of `chk` is MaxChunkSize at most. - emitToChunk(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error + emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error } func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, - outerIsRight bool, defaultInner Row, filter []expression.Expression, + outerIsRight bool, defaultInner types.DatumRow, filter []expression.Expression, lhsColTypes, rhsColTypes []*types.FieldType) joinResultGenerator { base := baseJoinResultGenerator{ ctx: ctx, - filter: filter, - defaultInner: defaultInner, + conditions: filter, outerIsRight: outerIsRight, maxChunkSize: ctx.GetSessionVars().MaxChunkSize, } @@ -72,7 +68,7 @@ func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, if !outerIsRight { innerColTypes = rhsColTypes } - base.initDefaultChunkInner(innerColTypes) + base.initDefaultInner(innerColTypes, defaultInner) } switch joinType { case plan.SemiJoin: @@ -96,29 +92,19 @@ func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, // baseJoinResultGenerator is not thread-safe, // so we should build individual generator for every join goroutine. type baseJoinResultGenerator struct { - ctx sessionctx.Context - filter []expression.Expression - defaultChunkInner chunk.Row - outerIsRight bool - chk *chunk.Chunk - selected []bool - defaultInner Row - maxChunkSize int + ctx sessionctx.Context + conditions []expression.Expression + defaultInner chunk.Row + outerIsRight bool + chk *chunk.Chunk + selected []bool + maxChunkSize int } -func (outputer *baseJoinResultGenerator) initDefaultChunkInner(innerTypes []*types.FieldType) { +func (outputer *baseJoinResultGenerator) initDefaultInner(innerTypes []*types.FieldType, defaultInner types.DatumRow) { mutableRow := chunk.MutRowFromTypes(innerTypes) - mutableRow.SetDatums(outputer.defaultInner[:len(innerTypes)]...) - outputer.defaultChunkInner = mutableRow.ToRow() -} - -// makeJoinRowToBuffer concatenates "lhs" and "rhs" to "buffer" and return that buffer. -// With the help of this function, we can make all of the joined rows to a consecutive -// memory buffer and explore the best cache performance. -func (outputer *baseJoinResultGenerator) makeJoinRowToBuffer(buffer []types.Datum, lhs, rhs Row) []types.Datum { - buffer = append(buffer, lhs...) - buffer = append(buffer, rhs...) - return buffer + mutableRow.SetDatums(defaultInner[:len(innerTypes)]...) + outputer.defaultInner = mutableRow.ToRow() } func (outputer *baseJoinResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { @@ -128,39 +114,8 @@ func (outputer *baseJoinResultGenerator) makeJoinRowToChunk(chk *chunk.Chunk, lh chk.AppendPartialRow(lhs.Len(), rhs) } -// growResultBufferIfNecessary grows resultBuffer if necessary. -func (outputer *baseJoinResultGenerator) growResultBufferIfNecessary(resultBuffer []Row, numToAppend int) []Row { - length := len(resultBuffer) - if cap(resultBuffer)-length >= numToAppend { - return resultBuffer - } - newResultBuffer := make([]Row, length, length+numToAppend) - copy(newResultBuffer, resultBuffer) - return newResultBuffer -} - -// filterResult filters resultBuffer according to filter. -func (outputer *baseJoinResultGenerator) filterResult(resultBuffer []Row, originLen int) ([]Row, bool, error) { - if outputer.filter == nil { - return resultBuffer, len(resultBuffer) > originLen, nil - } - - curLen := originLen - for _, joinedRow := range resultBuffer[originLen:] { - matched, err := expression.EvalBool(outputer.ctx, outputer.filter, joinedRow) - if err != nil { - return nil, false, errors.Trace(err) - } - if matched { - resultBuffer[curLen] = joinedRow - curLen++ - } - } - return resultBuffer[:curLen], curLen > originLen, nil -} - -func (outputer *baseJoinResultGenerator) filterChunk(input, output *chunk.Chunk) (matched bool, err error) { - outputer.selected, err = expression.VectorizedFilter(outputer.ctx, outputer.filter, chunk.NewIterator4Chunk(input), outputer.selected) +func (outputer *baseJoinResultGenerator) filter(input, output *chunk.Chunk) (matched bool, err error) { + outputer.selected, err = expression.VectorizedFilter(outputer.ctx, outputer.conditions, chunk.NewIterator4Chunk(input), outputer.selected) if err != nil { return false, errors.Trace(err) } @@ -179,44 +134,12 @@ type semiJoinResultGenerator struct { } // emit implements joinResultGenerator interface. -func (outputer *semiJoinResultGenerator) emit(outer Row, inners []Row, resultBuffer []Row) ([]Row, error) { - // outer row can not be joined with any inner row. - if len(inners) == 0 { - return resultBuffer, nil - } - // outer row can be joined with an inner row. - if len(outputer.filter) == 0 { - return append(resultBuffer, outer), nil - } - - buffer := make(Row, 0, len(inners[0])+len(outer)) - for _, inner := range inners { - if outputer.outerIsRight { - buffer = outputer.makeJoinRowToBuffer(buffer[:0], inner, outer) - } else { - buffer = outputer.makeJoinRowToBuffer(buffer[:0], outer, inner) - } - - matched, err := expression.EvalBool(outputer.ctx, outputer.filter, buffer) - if err != nil { - return resultBuffer, errors.Trace(err) - } - if matched { - // outer row can be joined with an inner row. - return append(resultBuffer, outer), nil - } - } - // outer row can not be joined with any inner row. - return resultBuffer, nil -} - -// emitToChunk implements joinResultGenerator interface. -func (outputer *semiJoinResultGenerator) emitToChunk(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { +func (outputer *semiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { if inners == nil || inners.Len() == 0 { return nil } defer inners.ReachEnd() - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { chk.AppendPartialRow(0, outer) return nil } @@ -228,7 +151,7 @@ func (outputer *semiJoinResultGenerator) emitToChunk(outer chunk.Row, inners chu } else { outputer.makeJoinRowToChunk(outputer.chk, outer, inner) } - selected, err := expression.EvalBool(outputer.ctx, outputer.filter, outputer.chk.GetRow(0)) + selected, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { return errors.Trace(err) } @@ -245,45 +168,13 @@ type antiSemiJoinResultGenerator struct { } // emit implements joinResultGenerator interface. -func (outputer *antiSemiJoinResultGenerator) emit(outer Row, inners []Row, resultBuffer []Row) (_ []Row, err error) { - // outer row can not be joined with any inner row. - if len(inners) == 0 { - return append(resultBuffer, outer), nil - } - // outer row can be joined with an inner row. - if len(outputer.filter) == 0 { - return resultBuffer, nil - } - - buffer := make(Row, 0, len(outer)+len(inners[0])) - for _, inner := range inners { - if outputer.outerIsRight { - buffer = outputer.makeJoinRowToBuffer(buffer[:0], inner, outer) - } else { - buffer = outputer.makeJoinRowToBuffer(buffer[:0], outer, inner) - } - - matched, err1 := expression.EvalBool(outputer.ctx, outputer.filter, buffer) - if err1 != nil { - return nil, errors.Trace(err1) - } - if matched { - // outer row can be joined with an inner row. - return resultBuffer, nil - } - } - // outer row can not be joined with any inner row. - return append(resultBuffer, outer), nil -} - -// emitToChunk implements joinResultGenerator interface. -func (outputer *antiSemiJoinResultGenerator) emitToChunk(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { +func (outputer *antiSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { if inners == nil || inners.Len() == 0 { chk.AppendRow(outer) return nil } defer inners.ReachEnd() - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { return nil } @@ -295,7 +186,7 @@ func (outputer *antiSemiJoinResultGenerator) emitToChunk(outer chunk.Row, inners outputer.makeJoinRowToChunk(outputer.chk, outer, inner) } - matched, err := expression.EvalBool(outputer.ctx, outputer.filter, outputer.chk.GetRow(0)) + matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { return errors.Trace(err) } @@ -312,36 +203,7 @@ type leftOuterSemiJoinResultGenerator struct { } // emit implements joinResultGenerator interface. -func (outputer *leftOuterSemiJoinResultGenerator) emit(outer Row, inners []Row, resultBuffer []Row) ([]Row, error) { - // outer row can not be joined with any inner row. - if len(inners) == 0 { - return outputer.emitUnMatchedOuter(outer, resultBuffer), nil - } - buffer := make(Row, 0, len(outer)+len(inners[0])) - // outer row can be joined with an inner row. - if len(outputer.filter) == 0 { - joinedRow := outputer.makeJoinRowToBuffer(buffer[:0], outer, Row{types.NewIntDatum(1)}) - return append(resultBuffer, joinedRow), nil - } - - for _, inner := range inners { - buffer = outputer.makeJoinRowToBuffer(buffer[:0], outer, inner) - matched, err := expression.EvalBool(outputer.ctx, outputer.filter, buffer) - if err != nil { - return resultBuffer, errors.Trace(err) - } - if matched { - // outer row can be joined with an inner row. - buffer = append(buffer[:len(outer)], types.NewDatum(true)) - return append(resultBuffer, buffer), nil - } - } - // outer row can not be joined with any inner row. - return outputer.emitUnMatchedOuter(outer, resultBuffer), nil -} - -// emitToChunk implements joinResultGenerator interface. -func (outputer *leftOuterSemiJoinResultGenerator) emitToChunk(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { +func (outputer *leftOuterSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { if inners == nil || inners.Len() == 0 { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) @@ -349,7 +211,7 @@ func (outputer *leftOuterSemiJoinResultGenerator) emitToChunk(outer chunk.Row, i } defer inners.ReachEnd() - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 1) return nil @@ -358,7 +220,7 @@ func (outputer *leftOuterSemiJoinResultGenerator) emitToChunk(outer chunk.Row, i for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { outputer.chk.Reset() outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - matched, err := expression.EvalBool(outputer.ctx, outputer.filter, outputer.chk.GetRow(0)) + matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { return errors.Trace(err) } @@ -373,48 +235,12 @@ func (outputer *leftOuterSemiJoinResultGenerator) emitToChunk(outer chunk.Row, i return nil } -// emitUnMatchedOuter implements joinResultGenerator interface. -func (outputer *leftOuterSemiJoinResultGenerator) emitUnMatchedOuter(outer Row, resultBuffer []Row) []Row { - buffer := make(Row, 0, len(outer)+1) - joinedRow := outputer.makeJoinRowToBuffer(buffer, outer, Row{types.NewIntDatum(0)}) - return append(resultBuffer, joinedRow) -} - type antiLeftOuterSemiJoinResultGenerator struct { baseJoinResultGenerator } // emit implements joinResultGenerator interface. -func (outputer *antiLeftOuterSemiJoinResultGenerator) emit(outer Row, inners []Row, resultBuffer []Row) ([]Row, error) { - // outer row can not be joined with any inner row. - if len(inners) == 0 { - return outputer.emitUnMatchedOuter(outer, resultBuffer), nil - } - buffer := make(Row, 0, len(outer)+len(inners[0])) - // outer row can be joined with an inner row. - if len(outputer.filter) == 0 { - joinedRow := outputer.makeJoinRowToBuffer(buffer[:0], outer, Row{types.NewIntDatum(0)}) - return append(resultBuffer, joinedRow), nil - } - - for _, inner := range inners { - buffer = outputer.makeJoinRowToBuffer(buffer[:0], outer, inner) - matched, err := expression.EvalBool(outputer.ctx, outputer.filter, buffer) - if err != nil { - return resultBuffer, errors.Trace(err) - } - if matched { - // outer row can be joined with an inner row. - buffer = append(buffer[:len(outer)], types.NewDatum(false)) - return append(resultBuffer, buffer), nil - } - } - // outer row can not be joined with any inner row. - return outputer.emitUnMatchedOuter(outer, resultBuffer), nil -} - -// emitToChunk implements joinResultGenerator interface. -func (outputer *antiLeftOuterSemiJoinResultGenerator) emitToChunk(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { +func (outputer *antiLeftOuterSemiJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { // outer row can not be joined with any inner row. if inners == nil || inners.Len() == 0 { chk.AppendPartialRow(0, outer) @@ -424,7 +250,7 @@ func (outputer *antiLeftOuterSemiJoinResultGenerator) emitToChunk(outer chunk.Ro defer inners.ReachEnd() // outer row can be joined with an inner row. - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { chk.AppendPartialRow(0, outer) chk.AppendInt64(outer.Len(), 0) return nil @@ -433,7 +259,7 @@ func (outputer *antiLeftOuterSemiJoinResultGenerator) emitToChunk(outer chunk.Ro for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { outputer.chk.Reset() outputer.makeJoinRowToChunk(outputer.chk, outer, inner) - matched, err := expression.EvalBool(outputer.ctx, outputer.filter, outputer.chk.GetRow(0)) + matched, err := expression.EvalBool(outputer.ctx, outputer.conditions, outputer.chk.GetRow(0)) if err != nil { return errors.Trace(err) } @@ -451,54 +277,21 @@ func (outputer *antiLeftOuterSemiJoinResultGenerator) emitToChunk(outer chunk.Ro return nil } -// emitUnMatchedOuter implements joinResultGenerator interface. -func (outputer *antiLeftOuterSemiJoinResultGenerator) emitUnMatchedOuter(outer Row, resultBuffer []Row) []Row { - buffer := make(Row, 0, len(outer)+1) - joinedRow := outputer.makeJoinRowToBuffer(buffer, outer, Row{types.NewIntDatum(1)}) - return append(resultBuffer, joinedRow) -} - type leftOuterJoinResultGenerator struct { baseJoinResultGenerator } // emit implements joinResultGenerator interface. -func (outputer *leftOuterJoinResultGenerator) emit(outer Row, inners []Row, resultBuffer []Row) ([]Row, error) { - // outer row can not be joined with any inner row. - if len(inners) == 0 { - return append(resultBuffer, makeJoinRow(outer, outputer.defaultInner)), nil - } - resultBuffer = outputer.growResultBufferIfNecessary(resultBuffer, len(inners)) - originLen := len(resultBuffer) - buffer := make([]types.Datum, 0, len(inners)*(len(outer)+len(inners[0]))) - for _, inner := range inners { - buffer = outputer.makeJoinRowToBuffer(buffer[len(buffer):], outer, inner) - resultBuffer = append(resultBuffer, buffer) - } - var matched bool - var err error - resultBuffer, matched, err = outputer.filterResult(resultBuffer, originLen) - if err != nil { - return nil, errors.Trace(err) - } - if !matched { - // outer row can not be joined with any inner row. - return append(resultBuffer, makeJoinRow(outer, outputer.defaultInner)), nil - } - return resultBuffer, nil -} - -// emitToChunk implements joinResultGenerator interface. -func (outputer *leftOuterJoinResultGenerator) emitToChunk(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { +func (outputer *leftOuterJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { // outer row can not be joined with any inner row. if inners == nil || inners.Len() == 0 { chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputer.defaultChunkInner) + chk.AppendPartialRow(outer.Len(), outputer.defaultInner) return nil } outputer.chk.Reset() chkForJoin := outputer.chk - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { chkForJoin = chk } numToAppend := outputer.maxChunkSize - chk.NumRows() @@ -506,11 +299,11 @@ func (outputer *leftOuterJoinResultGenerator) emitToChunk(outer chunk.Row, inner outputer.makeJoinRowToChunk(chkForJoin, outer, inners.Current()) inners.Next() } - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { return nil } // reach here, chkForJoin is outputer.chk - matched, err := outputer.filterChunk(chkForJoin, chk) + matched, err := outputer.filter(chkForJoin, chk) if err != nil { return errors.Trace(err) } @@ -518,7 +311,7 @@ func (outputer *leftOuterJoinResultGenerator) emitToChunk(outer chunk.Row, inner if !matched { // outer row can not be joined with any inner row. chk.AppendPartialRow(0, outer) - chk.AppendPartialRow(outer.Len(), outputer.defaultChunkInner) + chk.AppendPartialRow(outer.Len(), outputer.defaultInner) } return nil } @@ -528,42 +321,16 @@ type rightOuterJoinResultGenerator struct { } // emit implements joinResultGenerator interface. -func (outputer *rightOuterJoinResultGenerator) emit(outer Row, inners []Row, resultBuffer []Row) ([]Row, error) { - // outer row can not be joined with any inner row. - if len(inners) == 0 { - return append(resultBuffer, makeJoinRow(outputer.defaultInner, outer)), nil - } - resultBuffer = outputer.growResultBufferIfNecessary(resultBuffer, len(inners)) - originLen := len(resultBuffer) - buffer := make([]types.Datum, 0, len(inners)*(len(outer)+len(inners[0]))) - for _, inner := range inners { - buffer = outputer.makeJoinRowToBuffer(buffer[len(buffer):], inner, outer) - resultBuffer = append(resultBuffer, buffer) - } - var matched bool - var err error - resultBuffer, matched, err = outputer.filterResult(resultBuffer, originLen) - if err != nil { - return nil, errors.Trace(err) - } - // outer row can not be joined with any inner row. - if !matched { - return append(resultBuffer, makeJoinRow(outputer.defaultInner, outer)), nil - } - return resultBuffer, nil -} - -// emitToChunk implements joinResultGenerator interface. -func (outputer *rightOuterJoinResultGenerator) emitToChunk(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { +func (outputer *rightOuterJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { // outer row can not be joined with any inner row. if inners == nil || inners.Len() == 0 { - chk.AppendPartialRow(0, outputer.defaultChunkInner) - chk.AppendPartialRow(outputer.defaultChunkInner.Len(), outer) + chk.AppendPartialRow(0, outputer.defaultInner) + chk.AppendPartialRow(outputer.defaultInner.Len(), outer) return nil } outputer.chk.Reset() chkForJoin := outputer.chk - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { chkForJoin = chk } numToAppend := outputer.maxChunkSize - chk.NumRows() @@ -571,19 +338,19 @@ func (outputer *rightOuterJoinResultGenerator) emitToChunk(outer chunk.Row, inne outputer.makeJoinRowToChunk(chkForJoin, inners.Current(), outer) inners.Next() } - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { return nil } // reach here, chkForJoin is outputer.chk - matched, err := outputer.filterChunk(chkForJoin, chk) + matched, err := outputer.filter(chkForJoin, chk) if err != nil { return errors.Trace(err) } chkForJoin.Reset() // outer row can not be joined with any inner row. if !matched { - chk.AppendPartialRow(0, outputer.defaultChunkInner) - chk.AppendPartialRow(outputer.defaultChunkInner.Len(), outer) + chk.AppendPartialRow(0, outputer.defaultInner) + chk.AppendPartialRow(outputer.defaultInner.Len(), outer) } return nil } @@ -593,38 +360,13 @@ type innerJoinResultGenerator struct { } // emit implements joinResultGenerator interface. -func (outputer *innerJoinResultGenerator) emit(outer Row, inners []Row, resultBuffer []Row) ([]Row, error) { - // outer row can not be joined with any inner row. - if len(inners) == 0 { - return resultBuffer, nil - } - resultBuffer = outputer.growResultBufferIfNecessary(resultBuffer, len(inners)) - originLen := len(resultBuffer) - buffer := make([]types.Datum, 0, (len(outer)+len(inners[0]))*len(inners)) - if outputer.outerIsRight { - for _, inner := range inners { - buffer = outputer.makeJoinRowToBuffer(buffer[len(buffer):], inner, outer) - resultBuffer = append(resultBuffer, buffer) - } - } else { - for _, inner := range inners { - buffer = outputer.makeJoinRowToBuffer(buffer[len(buffer):], outer, inner) - resultBuffer = append(resultBuffer, buffer) - } - } - var err error - resultBuffer, _, err = outputer.filterResult(resultBuffer, originLen) - return resultBuffer, errors.Trace(err) -} - -// emitToChunk implements joinResultGenerator interface. -func (outputer *innerJoinResultGenerator) emitToChunk(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { +func (outputer *innerJoinResultGenerator) emit(outer chunk.Row, inners chunk.Iterator, chk *chunk.Chunk) error { if inners == nil || inners.Len() == 0 { return nil } outputer.chk.Reset() chkForJoin := outputer.chk - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { chkForJoin = chk } inner, numToAppend := inners.Current(), outputer.maxChunkSize-chk.NumRows() @@ -635,11 +377,11 @@ func (outputer *innerJoinResultGenerator) emitToChunk(outer chunk.Row, inners ch outputer.makeJoinRowToChunk(chkForJoin, outer, inner) } } - if len(outputer.filter) == 0 { + if len(outputer.conditions) == 0 { return nil } // reach here, chkForJoin is outputer.chk - _, err := outputer.filterChunk(chkForJoin, chk) + _, err := outputer.filter(chkForJoin, chk) if err != nil { return errors.Trace(err) } @@ -647,11 +389,3 @@ func (outputer *innerJoinResultGenerator) emitToChunk(outer chunk.Row, inners ch return nil } - -// makeJoinRow simply creates a new row that appends row b to row a. -func makeJoinRow(a Row, b Row) Row { - ret := make([]types.Datum, 0, len(a)+len(b)) - ret = append(ret, a...) - ret = append(ret, b...) - return ret -} diff --git a/executor/join_test.go b/executor/join_test.go index 4f98122176b2c..9a2c521e20ca3 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -37,6 +37,13 @@ func (s *testSuite) TestJoinPanic(c *C) { func (s *testSuite) TestJoin(c *C) { tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("set @@tidb_index_lookup_join_concurrency = 200") + c.Assert(tk.Se.GetSessionVars().IndexLookupJoinConcurrency, Equals, 200) + + tk.MustExec("set @@tidb_index_lookup_join_concurrency = 4") + c.Assert(tk.Se.GetSessionVars().IndexLookupJoinConcurrency, Equals, 4) + tk.MustExec("set @@tidb_index_lookup_size = 2") tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -237,8 +244,7 @@ func (s *testSuite) TestJoin(c *C) { // This case is for testing: // when the main thread calls Executor.Close() while the out data fetch worker and join workers are still working, // we need to stop the goroutines as soon as possible to avoid unexpected error. - savedConcurrency := plan.JoinConcurrency - plan.JoinConcurrency = 5 + tk.MustExec("set @@tidb_hash_join_concurrency=5") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int)") for i := 0; i < 100; i++ { @@ -246,7 +252,15 @@ func (s *testSuite) TestJoin(c *C) { } result = tk.MustQuery("select /*+ TIDB_HJ(s, r) */ * from t as s join t as r on s.a = r.a limit 1;") result.Check(testkit.Rows("1 1")) - plan.JoinConcurrency = savedConcurrency + + tk.MustExec("drop table if exists user, aa, bb") + tk.MustExec("create table aa(id int)") + tk.MustExec("insert into aa values(1)") + tk.MustExec("create table bb(id int)") + tk.MustExec("insert into bb values(1)") + tk.MustExec("create table user(id int, name varchar(20))") + tk.MustExec("insert into user values(1, 'a'), (2, 'b')") + tk.MustQuery("select user.id,user.name from user left join aa on aa.id = user.id left join bb on aa.id = bb.id where bb.id < 10;").Check(testkit.Rows("1 a")) } func (s *testSuite) TestJoinCast(c *C) { @@ -487,11 +501,8 @@ func (s *testSuite) TestSubquerySameTable(c *C) { } func (s *testSuite) TestSubquery(c *C) { - plan.JoinConcurrency = 1 - defer func() { - plan.JoinConcurrency = 5 - }() tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_hash_join_concurrency=1") tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c int, d int)") @@ -652,6 +663,8 @@ func (s *testSuite) TestSubquery(c *C) { tk.MustExec("insert into t1 values(1)") tk.MustExec("insert into t2 values(1)") tk.MustQuery("select * from t1 where a in (select a from t2)").Check(testkit.Rows("1")) + + tk.MustExec("set @@tidb_hash_join_concurrency=5") } func (s *testSuite) TestInSubquery(c *C) { @@ -725,12 +738,8 @@ func (s *testSuite) TestInSubquery(c *C) { } func (s *testSuite) TestJoinLeak(c *C) { - savedConcurrency := plan.JoinConcurrency - plan.JoinConcurrency = 1 - defer func() { - plan.JoinConcurrency = savedConcurrency - }() tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_hash_join_concurrency=1") tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (d int)") @@ -742,10 +751,12 @@ func (s *testSuite) TestJoinLeak(c *C) { result, err := tk.Exec("select * from t t1 left join (select 1) t2 on 1") c.Assert(err, IsNil) chk := result.NewChunk() - err = result.NextChunk(context.Background(), chk) + err = result.Next(context.Background(), chk) c.Assert(err, IsNil) time.Sleep(100 * time.Millisecond) result.Close() + + tk.MustExec("set @@tidb_hash_join_concurrency=5") } func (s *testSuite) TestHashJoinExecEncodeDecodeRow(c *C) { diff --git a/executor/load_stats.go b/executor/load_stats.go index 6f524081ddba8..2e20a074206ed 100644 --- a/executor/load_stats.go +++ b/executor/load_stats.go @@ -50,8 +50,8 @@ func (k loadStatsVarKeyType) String() string { // LoadStatsVarKey is a variable key for load statistic. const LoadStatsVarKey loadStatsVarKeyType = 0 -// NextChunk implements the Executor NextChunk interface. -func (e *LoadStatsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *LoadStatsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if len(e.info.Path) == 0 { return errors.New("Load Stats: file path is empty") diff --git a/executor/merge_join.go b/executor/merge_join.go index 05178ee7906af..b869ce2449b1f 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/memory" "golang.org/x/net/context" ) @@ -43,6 +44,8 @@ type MergeJoinExec struct { innerRows []chunk.Row innerIter4Row chunk.Iterator + + memTracker *memory.Tracker } type mergeJoinOuterTable struct { @@ -75,6 +78,8 @@ type mergeJoinInnerTable struct { curResultInUse bool resultQueue []*chunk.Chunk resourceQueue []*chunk.Chunk + + memTracker *memory.Tracker } func (t *mergeJoinInnerTable) init(ctx context.Context, chk4Reader *chunk.Chunk) (err error) { @@ -87,6 +92,7 @@ func (t *mergeJoinInnerTable) init(ctx context.Context, chk4Reader *chunk.Chunk) t.curRow = t.curIter.End() t.curResultInUse = false t.resultQueue = append(t.resultQueue, chk4Reader) + t.memTracker.Consume(chk4Reader.MemoryUsage()) t.firstRow4Key, err = t.nextRow() t.compareFuncs = make([]chunk.CompareFunc, 0, len(t.joinKeys)) for i := range t.joinKeys { @@ -125,12 +131,15 @@ func (t *mergeJoinInnerTable) rowsWithSameKey() ([]chunk.Row, error) { func (t *mergeJoinInnerTable) nextRow() (chunk.Row, error) { if t.curRow == t.curIter.End() { t.reallocReaderResult() - err := t.reader.NextChunk(t.ctx, t.curResult) + oldMemUsage := t.curResult.MemoryUsage() + err := t.reader.Next(t.ctx, t.curResult) // error happens or no more data. if err != nil || t.curResult.NumRows() == 0 { t.curRow = t.curIter.End() return t.curRow, errors.Trace(err) } + newMemUsage := t.curResult.MemoryUsage() + t.memTracker.Consume(newMemUsage - oldMemUsage) t.curRow = t.curIter.Begin() } result := t.curRow @@ -151,7 +160,9 @@ func (t *mergeJoinInnerTable) reallocReaderResult() { // Create a new Chunk and append it to "resourceQueue" if there is no more // available chunk in "resourceQueue". if len(t.resourceQueue) == 0 { - t.resourceQueue = append(t.resourceQueue, t.reader.newChunk()) + newChunk := t.reader.newChunk() + t.memTracker.Consume(newChunk.MemoryUsage()) + t.resourceQueue = append(t.resourceQueue, newChunk) } // NOTE: "t.curResult" is always the last element of "resultQueue". @@ -165,10 +176,10 @@ func (t *mergeJoinInnerTable) reallocReaderResult() { // Close implements the Executor Close interface. func (e *MergeJoinExec) Close() error { - if err := e.baseExecutor.Close(); err != nil { - return errors.Trace(err) - } - return nil + e.memTracker.Detach() + e.memTracker = nil + + return errors.Trace(e.baseExecutor.Close()) } // Open implements the Executor Open interface. @@ -176,7 +187,14 @@ func (e *MergeJoinExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return errors.Trace(err) } + e.prepared = false + e.memTracker = memory.NewTracker(e.id, e.ctx.GetSessionVars().MemQuotaMergeJoin) + e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) + + e.innerTable.memTracker = memory.NewTracker("innerTable", -1) + e.innerTable.memTracker.AttachTo(e.memTracker) + return nil } @@ -216,8 +234,8 @@ func (e *MergeJoinExec) prepare(ctx context.Context, chk *chunk.Chunk) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *MergeJoinExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *MergeJoinExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.prepared { if err := e.prepare(ctx, chk); err != nil { @@ -256,7 +274,7 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM } if cmpResult < 0 { - err = e.resultGenerator.emitToChunk(e.outerTable.row, nil, chk) + err = e.resultGenerator.emit(e.outerTable.row, nil, chk) if err != nil { return false, errors.Trace(err) } @@ -269,7 +287,7 @@ func (e *MergeJoinExec) joinToChunk(ctx context.Context, chk *chunk.Chunk) (hasM continue } - err = e.resultGenerator.emitToChunk(e.outerTable.row, e.innerIter4Row, chk) + err = e.resultGenerator.emit(e.outerTable.row, e.innerIter4Row, chk) if err != nil { return false, errors.Trace(err) } @@ -301,7 +319,7 @@ func (e *MergeJoinExec) fetchNextInnerRows() (err error) { // may not all belong to the same join key, but are guaranteed to be sorted // according to the join key. func (e *MergeJoinExec) fetchNextOuterRows(ctx context.Context) (err error) { - err = e.outerTable.reader.NextChunk(ctx, e.outerTable.chk) + err = e.outerTable.reader.Next(ctx, e.outerTable.chk) if err != nil { return errors.Trace(err) } diff --git a/executor/pkg_test.go b/executor/pkg_test.go index ac16633c3d06f..c2e40dd3ba4ab 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -22,12 +22,11 @@ type pkgTestSuite struct { type MockExec struct { baseExecutor - fields []*ast.ResultField - Rows []Row + Rows []types.DatumRow curRowIdx int } -func (m *MockExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (m *MockExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() colTypes := m.retTypes() for ; m.curRowIdx < len(m.Rows) && chk.NumRows() < m.maxChunkSize; m.curRowIdx++ { @@ -59,7 +58,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { outerSchema := expression.NewSchema(col0) outerExec := &MockExec{ baseExecutor: newBaseExecutor(sctx, outerSchema, ""), - Rows: []Row{ + Rows: []types.DatumRow{ types.MakeDatums(1), types.MakeDatums(2), types.MakeDatums(3), @@ -70,7 +69,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { innerSchema := expression.NewSchema(col1) innerExec := &MockExec{ baseExecutor: newBaseExecutor(sctx, innerSchema, ""), - Rows: []Row{ + Rows: []types.DatumRow{ types.MakeDatums(1), types.MakeDatums(2), types.MakeDatums(3), @@ -98,7 +97,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { joinChk := join.newChunk() it := chunk.NewIterator4Chunk(joinChk) for rowIdx := 1; ; { - err := join.NextChunk(ctx, joinChk) + err := join.Next(ctx, joinChk) c.Check(err, IsNil) if joinChk.NumRows() == 0 { break diff --git a/executor/prepared.go b/executor/prepared.go index aff2d676a22da..bd84bead3fa91 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -93,8 +93,8 @@ func NewPrepareExec(ctx sessionctx.Context, is infoschema.InfoSchema, sqlTxt str } } -// NextChunk implements the Executor NextChunk interface. -func (e *PrepareExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *PrepareExec) Next(ctx context.Context, chk *chunk.Chunk) error { vars := e.ctx.GetSessionVars() if e.ID != 0 { // Must be the case when we retry a prepare. @@ -184,8 +184,8 @@ type ExecuteExec struct { plan plan.Plan } -// NextChunk implements the Executor NextChunk interface. -func (e *ExecuteExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ExecuteExec) Next(ctx context.Context, chk *chunk.Chunk) error { return nil } @@ -220,8 +220,8 @@ type DeallocateExec struct { Name string } -// NextChunk implements the Executor NextChunk interface. -func (e *DeallocateExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *DeallocateExec) Next(ctx context.Context, chk *chunk.Chunk) error { vars := e.ctx.GetSessionVars() id, ok := vars.PreparedStmtNameToID[e.Name] if !ok { diff --git a/executor/prepared_test.go b/executor/prepared_test.go index ec35ff2e6be7d..3c9f51016cc93 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -105,7 +105,7 @@ func (s *testSuite) TestPrepared(c *C) { rs, err = stmt.Exec(ctx) c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(rs.Close(), IsNil) @@ -176,7 +176,7 @@ func (s *testSuite) TestPrepared(c *C) { // Coverage. exec := &executor.ExecuteExec{} - exec.NextChunk(ctx, nil) + exec.Next(ctx, nil) exec.Close() } cfg.PreparedPlanCache.Enabled = orgEnable diff --git a/executor/revoke.go b/executor/revoke.go index 6e98bbcc50652..38abbf07a8f22 100644 --- a/executor/revoke.go +++ b/executor/revoke.go @@ -50,8 +50,8 @@ type RevokeExec struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *RevokeExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *RevokeExec) Next(ctx context.Context, chk *chunk.Chunk) error { if e.done { return nil } diff --git a/executor/set.go b/executor/set.go index ee4f9fcff943e..3575de68d365b 100644 --- a/executor/set.go +++ b/executor/set.go @@ -42,8 +42,8 @@ type SetExecutor struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *SetExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *SetExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil @@ -150,6 +150,9 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e return errors.Trace(err) } oldSnapshotTS := sessionVars.SnapshotTS + if name == variable.TxnIsolationOneShot && sessionVars.InTxn() { + return errors.Trace(ErrCantChangeTxCharacteristics) + } err = variable.SetSessionSystemVar(sessionVars, name, value) if err != nil { return errors.Trace(err) diff --git a/executor/show.go b/executor/show.go index 8ca2463ac9eec..9f9179f04af3a 100644 --- a/executor/show.go +++ b/executor/show.go @@ -61,8 +61,8 @@ type ShowExec struct { cursor int } -// NextChunk implements the Executor NextChunk interface. -func (e *ShowExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ShowExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.result == nil { e.result = e.newChunk() @@ -204,6 +204,7 @@ func (e *ShowExec) fetchShowProcessList() error { t, fmt.Sprintf("%d", pi.State), info, + pi.Mem, }) } return nil @@ -488,7 +489,7 @@ func (e *ShowExec) fetchShowCreateTable() error { switch col.DefaultValue { case nil: if !mysql.HasNotNullFlag(col.Flag) { - if mysql.HasTimestampFlag(col.Flag) { + if col.Tp == mysql.TypeTimestamp { buf.WriteString(" NULL") } buf.WriteString(" DEFAULT NULL") @@ -613,6 +614,24 @@ func (e *ShowExec) fetchShowCreateTable() error { buf.WriteString(fmt.Sprintf(" DEFAULT CHARSET=%s COLLATE=%s", charsetName, collate)) } + // add partition info here. + partitionInfo := tb.Meta().Partition + if partitionInfo != nil { + buf.WriteString(fmt.Sprintf("\nPARTITION BY %s ( %s ) (\n", partitionInfo.Type.String(), partitionInfo.Expr)) + for i, def := range partitionInfo.Definitions { + if i < len(partitionInfo.Definitions)-1 { + buf.WriteString(fmt.Sprintf(" PARTITION %s VALUES LESS THAN %s,\n", def.Name, def.LessThan[0])) + } else { + if def.MaxValue { + buf.WriteString(fmt.Sprintf(" PARTITION %s VALUES LESS THAN %s\n", def.Name, "MAXVALUE")) + } else { + buf.WriteString(fmt.Sprintf(" PARTITION %s VALUES LESS THAN %s\n", def.Name, def.LessThan[0])) + } + } + } + buf.WriteString(")") + } + if hasAutoIncID { autoIncID, err := tb.Allocator(e.ctx).NextGlobalAutoID(tb.Meta().ID) if err != nil { diff --git a/executor/show_stats.go b/executor/show_stats.go index cf4f0676e5dab..3cc82f941b5c0 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -30,7 +30,7 @@ func (e *ShowExec) fetchShowStatsMeta() error { dbs := do.InfoSchema().AllSchemas() for _, db := range dbs { for _, tbl := range db.Tables { - statsTbl := h.GetTableStats(tbl.ID) + statsTbl := h.GetTableStats(tbl) if !statsTbl.Pseudo { e.appendRow([]interface{}{ db.Name.O, @@ -51,10 +51,10 @@ func (e *ShowExec) fetchShowStatsHistogram() error { dbs := do.InfoSchema().AllSchemas() for _, db := range dbs { for _, tbl := range db.Tables { - statsTbl := h.GetTableStats(tbl.ID) + statsTbl := h.GetTableStats(tbl) if !statsTbl.Pseudo { for _, col := range statsTbl.Columns { - e.histogramToRow(db.Name.O, tbl.Name.O, col.Info.Name.O, 0, col.Histogram, col.AvgColSize()) + e.histogramToRow(db.Name.O, tbl.Name.O, col.Info.Name.O, 0, col.Histogram, col.AvgColSize(statsTbl.Count)) } for _, idx := range statsTbl.Indices { e.histogramToRow(db.Name.O, tbl.Name.O, idx.Info.Name.O, 1, idx.Histogram, 0) @@ -89,7 +89,7 @@ func (e *ShowExec) fetchShowStatsBuckets() error { dbs := do.InfoSchema().AllSchemas() for _, db := range dbs { for _, tbl := range db.Tables { - statsTbl := h.GetTableStats(tbl.ID) + statsTbl := h.GetTableStats(tbl) if !statsTbl.Pseudo { for _, col := range statsTbl.Columns { err := e.bucketsToRows(db.Name.O, tbl.Name.O, col.Info.Name.O, 0, col.Histogram) @@ -146,7 +146,7 @@ func (e *ShowExec) fetchShowStatsHealthy() { dbs := do.InfoSchema().AllSchemas() for _, db := range dbs { for _, tbl := range db.Tables { - statsTbl := h.GetTableStats(tbl.ID) + statsTbl := h.GetTableStats(tbl) if !statsTbl.Pseudo { var healthy int64 if statsTbl.ModifyCount < statsTbl.Count { diff --git a/executor/show_test.go b/executor/show_test.go index 5249cadb03e02..7bc0c14d481c7 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -64,7 +64,8 @@ func (s *testSuite) TestShow(c *C) { b double NOT NULL DEFAULT 2.0, c varchar(10) NOT NULL, d time unique, - e timestamp NULL + e timestamp NULL, + f timestamp );` tk.MustExec(testSQL) testSQL = "show create table ptest;" @@ -72,7 +73,7 @@ func (s *testSuite) TestShow(c *C) { c.Check(result.Rows(), HasLen, 1) row = result.Rows()[0] expectedRow = []interface{}{ - "ptest", "CREATE TABLE `ptest` (\n `a` int(11) NOT NULL,\n `b` double NOT NULL DEFAULT '2.0',\n `c` varchar(10) NOT NULL,\n `d` time DEFAULT NULL,\n `e` timestamp NULL DEFAULT NULL,\n PRIMARY KEY (`a`),\n UNIQUE KEY `d` (`d`)\n) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin"} + "ptest", "CREATE TABLE `ptest` (\n `a` int(11) NOT NULL,\n `b` double NOT NULL DEFAULT '2.0',\n `c` varchar(10) NOT NULL,\n `d` time DEFAULT NULL,\n `e` timestamp NULL DEFAULT NULL,\n `f` timestamp NULL DEFAULT NULL,\n PRIMARY KEY (`a`),\n UNIQUE KEY `d` (`d`)\n) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin"} for i, r := range row { c.Check(r, Equals, expectedRow[i]) } @@ -311,7 +312,17 @@ func (s *testSuite) TestShow(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", )) + // Test range partition tk.MustExec(`drop table if exists t`) + tk.MustExec(`CREATE TABLE t (a int) PARTITION BY RANGE(a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20), + PARTITION p2 VALUES LESS THAN (MAXVALUE))`) + tk.MustQuery("show create table t").Check(testutil.RowsWithSep("|", + "t CREATE TABLE `t` (\n"+ + " `a` int(11) DEFAULT NULL\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin"+"\nPARTITION BY RANGE ( `a` ) (\n PARTITION p0 VALUES LESS THAN 10,\n PARTITION p1 VALUES LESS THAN 20,\n PARTITION p2 VALUES LESS THAN MAXVALUE\n)", + )) } func (s *testSuite) TestShowVisibility(c *C) { diff --git a/executor/simple.go b/executor/simple.go index 59883b4025e45..a4d4387bccd81 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -47,8 +47,8 @@ type SimpleExec struct { is infoschema.InfoSchema } -// NextChunk implements the Executor NextChunk interface. -func (e *SimpleExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err error) { +// Next implements the Executor Next interface. +func (e *SimpleExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { if e.done { return nil } diff --git a/executor/simple_test.go b/executor/simple_test.go index c7a3697c07731..ebe61a4c5b3f7 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -263,23 +263,23 @@ func (s *testSuite) TestDropStats(c *C) { h := do.StatsHandle() h.Clear() testKit.MustExec("analyze table t") - statsTbl := h.GetTableStats(tableInfo.ID) + statsTbl := h.GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) testKit.MustExec("drop stats t") h.Update(is) - statsTbl = h.GetTableStats(tableInfo.ID) + statsTbl = h.GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsTrue) testKit.MustExec("analyze table t") - statsTbl = h.GetTableStats(tableInfo.ID) + statsTbl = h.GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) h.Lease = 1 testKit.MustExec("drop stats t") h.HandleDDLEvent(<-h.DDLEventCh()) h.Update(is) - statsTbl = h.GetTableStats(tableInfo.ID) + statsTbl = h.GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsTrue) h.Lease = 0 } diff --git a/executor/sort.go b/executor/sort.go index d93f4b024db00..e77455dd87eed 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -71,8 +71,8 @@ func (e *SortExec) Open(ctx context.Context) error { return errors.Trace(e.children[0].Open(ctx)) } -// NextChunk implements the Executor NextChunk interface. -func (e *SortExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *SortExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.fetched { err := e.fetchRowChunks(ctx) @@ -112,7 +112,7 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.rowChunks.GetMemTracker().SetLabel("rowChunks") for { chk := chunk.NewChunk(fields) - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -294,8 +294,8 @@ func (e *TopNExec) Open(ctx context.Context) error { return errors.Trace(e.SortExec.Open(ctx)) } -// NextChunk implements the Executor NextChunk interface. -func (e *TopNExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *TopNExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.fetched { e.totalLimit = int(e.limit.Offset + e.limit.Count) @@ -328,7 +328,7 @@ func (e *TopNExec) loadChunksUntilTotalLimit(ctx context.Context) error { e.rowChunks.GetMemTracker().SetLabel("rowChunks") for e.rowChunks.Len() < e.totalLimit { srcChk := e.children[0].newChunk() - err := e.children[0].NextChunk(ctx, srcChk) + err := e.children[0].Next(ctx, srcChk) if err != nil { return errors.Trace(err) } @@ -364,7 +364,7 @@ func (e *TopNExec) executeTopN(ctx context.Context) error { } childRowChk := e.children[0].newChunk() for { - err := e.children[0].NextChunk(ctx, childRowChk) + err := e.children[0].Next(ctx, childRowChk) if err != nil { return errors.Trace(err) } diff --git a/executor/union_scan.go b/executor/union_scan.go index fec8cc32da422..7980b3e8e670c 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -53,7 +53,7 @@ func (udb *DirtyDB) DeleteRow(tid int64, handle int64) { // TruncateTable truncates a table. func (udb *DirtyDB) TruncateTable(tid int64) { dt := udb.GetDirtyTable(tid) - dt.addedRows = make(map[int64]Row) + dt.addedRows = make(map[int64]types.DatumRow) dt.truncated = true } @@ -62,7 +62,7 @@ func (udb *DirtyDB) GetDirtyTable(tid int64) *DirtyTable { dt, ok := udb.tables[tid] if !ok { dt = &DirtyTable{ - addedRows: make(map[int64]Row), + addedRows: make(map[int64]types.DatumRow), deletedRows: make(map[int64]struct{}), } udb.tables[tid] = dt @@ -74,7 +74,7 @@ func (udb *DirtyDB) GetDirtyTable(tid int64) *DirtyTable { type DirtyTable struct { // addedRows ... // the key is handle. - addedRows map[int64]Row + addedRows map[int64]types.DatumRow deletedRows map[int64]struct{} truncated bool } @@ -106,15 +106,15 @@ type UnionScanExec struct { // belowHandleIndex is the handle's position of the below scan plan. belowHandleIndex int - addedRows []Row + addedRows []types.DatumRow cursor4AddRows int sortErr error - snapshotRows []Row + snapshotRows []types.DatumRow cursor4SnapshotRows int } -// NextChunk implements the Executor NextChunk interface. -func (us *UnionScanExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (us *UnionScanExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() mutableRow := chunk.MutRowFromTypes(us.retTypes()) for i, batchSize := 0, us.ctx.GetSessionVars().MaxChunkSize; i < batchSize; i++ { @@ -133,14 +133,14 @@ func (us *UnionScanExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error } // getOneRow gets one result row from dirty table or child. -func (us *UnionScanExec) getOneRow(ctx context.Context) (Row, error) { +func (us *UnionScanExec) getOneRow(ctx context.Context) (types.DatumRow, error) { for { snapshotRow, err := us.getSnapshotRow(ctx) if err != nil { return nil, errors.Trace(err) } addedRow := us.getAddedRow() - var row Row + var row types.DatumRow var isSnapshotRow bool if addedRow == nil { row = snapshotRow @@ -171,7 +171,7 @@ func (us *UnionScanExec) getOneRow(ctx context.Context) (Row, error) { } } -func (us *UnionScanExec) getSnapshotRow(ctx context.Context) (Row, error) { +func (us *UnionScanExec) getSnapshotRow(ctx context.Context) (types.DatumRow, error) { if us.dirty.truncated { return nil, nil } @@ -183,7 +183,7 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) (Row, error) { us.snapshotRows = us.snapshotRows[:0] for len(us.snapshotRows) == 0 { chk := chunk.NewChunkWithCapacity(us.retTypes(), us.maxChunkSize) - err = us.children[0].NextChunk(ctx, chk) + err = us.children[0].Next(ctx, chk) if err != nil || chk.NumRows() == 0 { return nil, errors.Trace(err) } @@ -204,8 +204,8 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) (Row, error) { return us.snapshotRows[0], nil } -func (us *UnionScanExec) getAddedRow() Row { - var addedRow Row +func (us *UnionScanExec) getAddedRow() types.DatumRow { + var addedRow types.DatumRow if us.cursor4AddRows < len(us.addedRows) { addedRow = us.addedRows[us.cursor4AddRows] } @@ -214,7 +214,7 @@ func (us *UnionScanExec) getAddedRow() Row { // shouldPickFirstRow picks the suitable row in order. // The value returned is used to determine whether to pick the first input row. -func (us *UnionScanExec) shouldPickFirstRow(a, b Row) (bool, error) { +func (us *UnionScanExec) shouldPickFirstRow(a, b types.DatumRow) (bool, error) { var isFirstRow bool addedCmpSrc, err := us.compare(a, b) if err != nil { @@ -233,7 +233,7 @@ func (us *UnionScanExec) shouldPickFirstRow(a, b Row) (bool, error) { return isFirstRow, nil } -func (us *UnionScanExec) compare(a, b Row) (int, error) { +func (us *UnionScanExec) compare(a, b types.DatumRow) (int, error) { sc := us.ctx.GetSessionVars().StmtCtx for _, colOff := range us.usedIndex { aColumn := a[colOff] @@ -260,7 +260,7 @@ func (us *UnionScanExec) compare(a, b Row) (int, error) { } func (us *UnionScanExec) buildAndSortAddedRows() error { - us.addedRows = make([]Row, 0, len(us.dirty.addedRows)) + us.addedRows = make([]types.DatumRow, 0, len(us.dirty.addedRows)) for h, data := range us.dirty.addedRows { newData := make(types.DatumRow, 0, us.schema.Len()) for _, col := range us.columns { diff --git a/executor/write.go b/executor/write.go index 1143e931b9f65..5f86688e42cdf 100644 --- a/executor/write.go +++ b/executor/write.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" log "github.com/sirupsen/logrus" @@ -46,7 +47,13 @@ var ( // Length of `oldData` and `newData` equals to length of `t.WritableCols()`. // ignoreErr indicate that update statement has the `IGNORE` modifier, in this situation, update statement will not update // the keys which cause duplicate conflicts and ignore the error. -func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, modified []bool, t table.Table, onDup, ignoreErr bool) (bool, error) { +// The return values: +// 1. changed (bool) : does the update really change the row values. e.g. update set i = 1 where i = 1; +// 2. handleChanged (bool) : is the handle changed after the update. +// 3. newHandle (int64) : if handleChanged == true, the newHandle means the new handle after update. +// 4. err (error) : error in the update. +func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, modified []bool, t table.Table, + onDup, ignoreErr bool) (bool, bool, int64, error) { var sc = ctx.GetSessionVars().StmtCtx var changed, handleChanged = false, false // onUpdateSpecified is for "UPDATE SET ts_field = old_value", the @@ -62,7 +69,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu // Cast changed fields with respective columns. v, err := table.CastValue(ctx, newData[i], col.ToInfo()) if err != nil { - return false, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } newData[i] = v } @@ -70,20 +77,20 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu // Rebase auto increment id if the field is changed. if mysql.HasAutoIncrementFlag(col.Flag) { if newData[i].IsNull() { - return false, errors.Errorf("Column '%v' cannot be null", col.Name.O) + return false, handleChanged, newHandle, errors.Errorf("Column '%v' cannot be null", col.Name.O) } val, errTI := newData[i].ToInt64(sc) if errTI != nil { - return false, errors.Trace(errTI) + return false, handleChanged, newHandle, errors.Trace(errTI) } err := t.RebaseAutoID(ctx, val, true) if err != nil { - return false, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } } cmp, err := newData[i].CompareDatum(sc, &oldData[i]) if err != nil { - return false, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } if cmp != 0 { changed = true @@ -104,7 +111,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu // Check the not-null constraints. err := table.CheckNotNull(t.Cols(), newData) if err != nil { - return false, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } if !changed { @@ -112,7 +119,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu if ctx.GetSessionVars().ClientCapability&mysql.ClientFoundRows > 0 { sc.AddAffectedRows(1) } - return false, nil + return false, handleChanged, newHandle, nil } // Fill values into on-update-now fields, only if they are really changed. @@ -120,7 +127,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu if mysql.HasOnUpdateNowFlag(col.Flag) && !modified[i] && !onUpdateSpecified[i] { v, errGT := expression.GetTimeValue(ctx, strings.ToUpper(ast.CurrentTimestamp), col.Tp, col.Decimal) if errGT != nil { - return false, errors.Trace(errGT) + return false, handleChanged, newHandle, errors.Trace(errGT) } newData[i] = v modified[i] = true @@ -132,21 +139,21 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu if ignoreErr { // if the new handle exists. `UPDATE IGNORE` will avoid removing record, and do nothing. if err = tables.CheckHandleExists(ctx, t, newHandle); err != nil { - return false, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } skipHandleCheck = true } err = t.RemoveRecord(ctx, h, oldData) if err != nil { - return false, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } - _, err = t.AddRecord(ctx, newData, skipHandleCheck) + newHandle, err = t.AddRecord(ctx, newData, skipHandleCheck) } else { // Update record to new value and update index. err = t.UpdateRecord(ctx, h, oldData, newData, modified) } if err != nil { - return false, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } tid := t.Meta().ID @@ -158,9 +165,15 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu } else { sc.AddAffectedRows(1) } - - ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1) - return true, nil + colSize := make(map[int64]int64) + for id, col := range t.Cols() { + val := int64(len(newData[id].GetBytes()) - len(oldData[id].GetBytes())) + if val != 0 { + colSize[col.ID] = val + } + } + ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1, colSize) + return true, handleChanged, newHandle, nil } // DeleteExec represents a delete executor. @@ -182,8 +195,8 @@ type DeleteExec struct { finished bool } -// NextChunk implements the Executor NextChunk interface. -func (e *DeleteExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *DeleteExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.finished { return nil @@ -208,7 +221,7 @@ type tblColPosInfo struct { // tableRowMapType is a map for unique (Table, Row) pair. key is the tableID. // the key in map[int64]Row is the joined table handle, which represent a unique reference row. // the value in map[int64]Row is the deleting row. -type tableRowMapType map[int64]map[int64]Row +type tableRowMapType map[int64]map[int64]types.DatumRow // matchingDeletingTable checks whether this column is from the table which is in the deleting list. func (e *DeleteExec) matchingDeletingTable(tableID int64, col *expression.Column) bool { @@ -224,7 +237,7 @@ func (e *DeleteExec) matchingDeletingTable(tableID int64, col *expression.Column return false } -func (e *DeleteExec) deleteOneRow(tbl table.Table, handleCol *expression.Column, row Row) error { +func (e *DeleteExec) deleteOneRow(tbl table.Table, handleCol *expression.Column, row types.DatumRow) error { end := len(row) if handleIsExtra(handleCol) { end-- @@ -259,7 +272,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error { chk := e.children[0].newChunk() iter := chunk.NewIterator4Chunk(chk) - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -314,11 +327,11 @@ func (e *DeleteExec) getColPosInfos(schema *expression.Schema) []tblColPosInfo { return colPosInfos } -func (e *DeleteExec) composeTblRowMap(tblRowMap tableRowMapType, colPosInfos []tblColPosInfo, joinedRow Row) { +func (e *DeleteExec) composeTblRowMap(tblRowMap tableRowMapType, colPosInfos []tblColPosInfo, joinedRow types.DatumRow) { // iterate all the joined tables, and got the copresonding rows in joinedRow. for _, info := range colPosInfos { if tblRowMap[info.tblID] == nil { - tblRowMap[info.tblID] = make(map[int64]Row) + tblRowMap[info.tblID] = make(map[int64]types.DatumRow) } handle := joinedRow[info.handleIndex].GetInt64() // tblRowMap[info.tblID][handle] hold the row datas binding to this table and this handle. @@ -339,7 +352,7 @@ func (e *DeleteExec) deleteMultiTablesByChunk(ctx context.Context) error { chk := e.children[0].newChunk() iter := chunk.NewIterator4Chunk(chk) - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -385,7 +398,14 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, d } ctx.StmtAddDirtyTableOP(DirtyTableDeleteRow, t.Meta().ID, h, nil) ctx.GetSessionVars().StmtCtx.AddAffectedRows(1) - ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1) + colSize := make(map[int64]int64) + for id, col := range t.Cols() { + val := -int64(len(data[id].GetBytes())) + if val != 0 { + colSize[col.ID] = val + } + } + ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, -1, 1, colSize) return nil } @@ -711,8 +731,8 @@ func (k loadDataVarKeyType) String() string { // LoadDataVarKey is a variable key for load data. const LoadDataVarKey loadDataVarKeyType = 0 -// NextChunk implements the Executor NextChunk interface. -func (e *LoadData) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *LoadData) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() // TODO: support load data without local field. if !e.IsLocal { @@ -789,74 +809,103 @@ type InsertExec struct { IgnoreErr bool finished bool + rowCount int + + // For duplicate key update + uniqueKeysInRows [][]keyWithDupError + dupKeyValues map[string][]byte + dupOldRowValues map[string][]byte +} + +func (e *InsertExec) insertOneRow(row []types.Datum) (int64, error) { + if err := e.checkBatchLimit(); err != nil { + return 0, errors.Trace(err) + } + e.ctx.Txn().SetOption(kv.PresumeKeyNotExists, nil) + h, err := e.Table.AddRecord(e.ctx, row, false) + e.ctx.Txn().DelOption(kv.PresumeKeyNotExists) + if err != nil { + return 0, errors.Trace(err) + } + if !e.ctx.GetSessionVars().ImportingData { + e.ctx.StmtAddDirtyTableOP(DirtyTableAddRow, e.Table.Meta().ID, h, row) + } + e.rowCount++ + return h, nil } -func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) (Row, error) { +func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) (types.DatumRow, error) { // If tidb_batch_insert is ON and not in a transaction, we could use BatchInsert mode. sessVars := e.ctx.GetSessionVars() defer sessVars.CleanBuffers() - batchInsert := sessVars.BatchInsert && !sessVars.InTxn() - batchSize := sessVars.DMLBatchSize - txn := e.ctx.Txn() - rowCount := 0 - writeBufs := sessVars.GetWriteStmtBufs() + e.rowCount = 0 if !sessVars.ImportingData { - writeBufs.BufStore = kv.NewBufferStore(txn, kv.TempTxnMemBufCap) - } - // If you use the IGNORE keyword, duplicate-key error that occurs while executing the INSERT statement are ignored. - // For example, without IGNORE, a row that duplicates an existing UNIQUE index or PRIMARY KEY value in - // the table causes a duplicate-key error and the statement is aborted. With IGNORE, the row is discarded and no error occurs. - // However, if the `on duplicate update` is also specified, the duplicated row will be updated. - // Using BatchGet in insert ignore to mark rows as duplicated before we add records to the table. - if e.IgnoreErr && len(e.OnDuplicate) == 0 { - var err error - rows, err = batchMarkDupRows(e.ctx, e.Table, rows) + sessVars.GetWriteStmtBufs().BufStore = kv.NewBufferStore(e.ctx.Txn(), kv.TempTxnMemBufCap) + } + + // If `ON DUPLICATE KEY UPDATE` is specified, and no `IGNORE` keyword, + // the to-be-insert rows will be check on duplicate keys and update to the new rows. + if len(e.OnDuplicate) > 0 && !e.IgnoreErr { + err := e.batchUpdateDupRows(rows, e.OnDuplicate) if err != nil { return nil, errors.Trace(err) } - } - for _, row := range rows { - // duplicate row will be marked as nil in batchMarkDupRows if - // IgnoreErr is true. For IgnoreErr is false, it is a protection. - if row == nil { - continue + } else { + if len(e.OnDuplicate) == 0 && e.IgnoreErr { + // If you use the IGNORE keyword, duplicate-key error that occurs while executing the INSERT statement are ignored. + // For example, without IGNORE, a row that duplicates an existing UNIQUE index or PRIMARY KEY value in + // the table causes a duplicate-key error and the statement is aborted. With IGNORE, the row is discarded and no error occurs. + // However, if the `on duplicate update` is also specified, the duplicated row will be updated. + // Using BatchGet in insert ignore to mark rows as duplicated before we add records to the table. + var err error + rows, err = batchMarkDupRows(e.ctx, e.Table, rows) + if err != nil { + return nil, errors.Trace(err) + } } - if batchInsert && rowCount >= batchSize { - e.ctx.StmtCommit() - if err := e.ctx.NewTxn(); err != nil { - // We should return a special error for batch insert. - return nil, ErrBatchInsertFail.Gen("BatchInsert failed with error: %v", err) + for _, row := range rows { + // duplicate row will be marked as nil in batchMarkDupRows if + // IgnoreErr is true. For IgnoreErr is false, it is a protection. + if row == nil { + continue } - txn = e.ctx.Txn() - rowCount = 0 - if !sessVars.ImportingData { - writeBufs.BufStore = kv.NewBufferStore(txn, kv.TempTxnMemBufCap) + if err := e.checkBatchLimit(); err != nil { + return nil, errors.Trace(err) } - } - if len(e.OnDuplicate) == 0 && !e.IgnoreErr { - txn.SetOption(kv.PresumeKeyNotExists, nil) - } - h, err := e.Table.AddRecord(e.ctx, row, false) - txn.DelOption(kv.PresumeKeyNotExists) - if err == nil { - if !sessVars.ImportingData { - e.ctx.StmtAddDirtyTableOP(DirtyTableAddRow, e.Table.Meta().ID, h, row) + if len(e.OnDuplicate) == 0 && !e.IgnoreErr { + e.ctx.Txn().SetOption(kv.PresumeKeyNotExists, nil) } - rowCount++ - continue - } - - if kv.ErrKeyExists.Equal(err) { - if len(e.OnDuplicate) > 0 { - if err = e.onDuplicateUpdate(row, h, e.OnDuplicate); err != nil { - return nil, errors.Trace(err) + h, err := e.Table.AddRecord(e.ctx, row, false) + e.ctx.Txn().DelOption(kv.PresumeKeyNotExists) + if err == nil { + if !sessVars.ImportingData { + e.ctx.StmtAddDirtyTableOP(DirtyTableAddRow, e.Table.Meta().ID, h, row) } - rowCount++ + e.rowCount++ continue } + if kv.ErrKeyExists.Equal(err) { + // TODO: Use batch get to speed up `insert ignore on duplicate key update`. + if len(e.OnDuplicate) > 0 && e.IgnoreErr { + data, err1 := e.Table.RowWithCols(e.ctx, h, e.Table.WritableCols()) + if err1 != nil { + return nil, errors.Trace(err1) + } + _, _, _, err = e.doDupRowUpdate(h, data, row, e.OnDuplicate) + if kv.ErrKeyExists.Equal(err) && e.IgnoreErr { + e.ctx.GetSessionVars().StmtCtx.AppendWarning(err) + continue + } + if err != nil { + return nil, errors.Trace(err) + } + e.rowCount++ + continue + } + } + return nil, errors.Trace(err) } - return nil, errors.Trace(err) } if e.lastInsertID != 0 { @@ -867,38 +916,46 @@ func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) (Row, error } type keyWithDupError struct { - key kv.Key - dupErr error + isRecordKey bool + key kv.Key + dupErr error + newRowValue []byte } -func getRecordIDs(ctx sessionctx.Context, t table.Table, rows [][]types.Datum) ([]int64, error) { - recordIDs := make([]int64, 0, len(rows)) - if t.Meta().PKIsHandle { - var handleCol *table.Column - for _, col := range t.Cols() { - if col.IsPKHandleColumn(t.Meta()) { - handleCol = col - break - } - } - for _, row := range rows { - recordIDs = append(recordIDs, row[handleCol.Offset].GetInt64()) - } - } else { - for range rows { - recordID, err := t.AllocAutoID(ctx) - if err != nil { - return nil, errors.Trace(err) - } - recordIDs = append(recordIDs, recordID) +// batchGetOldValues gets the values of storage in batch. +func batchGetOldValues(ctx sessionctx.Context, t table.Table, handles []int64) (map[string][]byte, error) { + batchKeys := make([]kv.Key, 0, len(handles)) + for _, handle := range handles { + batchKeys = append(batchKeys, t.RecordKey(handle)) + } + values, err := kv.BatchGetValues(ctx.Txn(), batchKeys) + if err != nil { + return nil, errors.Trace(err) + } + return values, nil +} + +// encodeNewRow encodes a new row to value. +func encodeNewRow(ctx sessionctx.Context, t table.Table, row []types.Datum) ([]byte, error) { + colIDs := make([]int64, 0, len(row)) + skimmedRow := make([]types.Datum, 0, len(row)) + for _, col := range t.Cols() { + if !tables.CanSkip(t.Meta(), col, row[col.Offset]) { + colIDs = append(colIDs, col.ID) + skimmedRow = append(skimmedRow, row[col.Offset]) } } - return recordIDs, nil + newRowValue, err := tablecodec.EncodeRow(ctx.GetSessionVars().StmtCtx, skimmedRow, colIDs, nil, nil) + if err != nil { + return nil, errors.Trace(err) + } + return newRowValue, nil } // getKeysNeedCheck gets keys converted from to-be-insert rows to record keys and unique index keys, // which need to be checked whether they are duplicate keys. -func getKeysNeedCheck(ctx sessionctx.Context, t table.Table, rows [][]types.Datum) ([][]keyWithDupError, error) { +func getKeysNeedCheck(ctx sessionctx.Context, t table.Table, rows [][]types.Datum) ([][]keyWithDupError, + error) { nUnique := 0 for _, v := range t.WritableIndices() { if v.Meta().Unique { @@ -907,17 +964,32 @@ func getKeysNeedCheck(ctx sessionctx.Context, t table.Table, rows [][]types.Datu } rowWithKeys := make([][]keyWithDupError, 0, len(rows)) - // get recordIDs - recordIDs, err := getRecordIDs(ctx, t, rows) - if err != nil { - return nil, errors.Trace(err) + var handleCol *table.Column + // Get handle column if PK is handle. + if t.Meta().PKIsHandle { + for _, col := range t.Cols() { + if col.IsPKHandleColumn(t.Meta()) { + handleCol = col + break + } + } } - for i, row := range rows { + for _, row := range rows { keysWithErr := make([]keyWithDupError, 0, nUnique+1) - // append record keys and errors + newRowValue, err := encodeNewRow(ctx, t, row) + if err != nil { + return nil, errors.Trace(err) + } + // Append record keys and errors. if t.Meta().PKIsHandle { - keysWithErr = append(keysWithErr, keyWithDupError{t.RecordKey(recordIDs[i]), kv.ErrKeyExists.FastGen("Duplicate entry '%d' for key 'PRIMARY'", recordIDs[i])}) + handle := row[handleCol.Offset].GetInt64() + keysWithErr = append(keysWithErr, keyWithDupError{ + true, + t.RecordKey(handle), + kv.ErrKeyExists.FastGen("Duplicate entry '%d' for key 'PRIMARY'", handle), + newRowValue, + }) } // append unique keys and errors @@ -925,51 +997,243 @@ func getKeysNeedCheck(ctx sessionctx.Context, t table.Table, rows [][]types.Datu if !v.Meta().Unique { continue } - var colVals []types.Datum - colVals, err = v.FetchValues(row, nil) - if err != nil { - return nil, errors.Trace(err) + colVals, err1 := v.FetchValues(row, nil) + if err1 != nil { + return nil, errors.Trace(err1) } - var key []byte - var distinct bool - key, distinct, err = v.GenIndexKey(ctx.GetSessionVars().StmtCtx, - colVals, recordIDs[i], nil) - if err != nil { - return nil, errors.Trace(err) + // Pass handle = 0 to GenIndexKey, + // due to we only care about distinct key. + key, distinct, err1 := v.GenIndexKey(ctx.GetSessionVars().StmtCtx, + colVals, 0, nil) + if err1 != nil { + return nil, errors.Trace(err1) } + // Skip the non-distinct keys. if !distinct { continue } - keysWithErr = append(keysWithErr, keyWithDupError{key, kv.ErrKeyExists.FastGen("Duplicate entry '%d' for key '%s'", recordIDs[i], v.Meta().Name)}) + colValStr, err1 := types.DatumsToString(colVals) + if err1 != nil { + return nil, errors.Trace(err1) + } + keysWithErr = append(keysWithErr, keyWithDupError{ + false, + key, + kv.ErrKeyExists.FastGen("Duplicate entry '%s' for key '%s'", + colValStr, v.Meta().Name), + newRowValue, + }) } rowWithKeys = append(rowWithKeys, keysWithErr) } return rowWithKeys, nil } -// batchMarkDupRows marks rows with duplicate errors as nil. -// All duplicate rows were marked and appended as duplicate warnings -// to the statement context in batch. -func batchMarkDupRows(ctx sessionctx.Context, t table.Table, rows [][]types.Datum) ([][]types.Datum, error) { - // get keys need to be checked - rowWithKeys, err := getKeysNeedCheck(ctx, t, rows) +// batchGetInsertKeys uses batch-get to fetch all key-value pairs to be checked for ignore or duplicate key update. +func batchGetInsertKeys(ctx sessionctx.Context, t table.Table, newRows [][]types.Datum) ([][]keyWithDupError, map[string][]byte, error) { + // Get keys need to be checked. + keysInRows, err := getKeysNeedCheck(ctx, t, newRows) + if err != nil { + return nil, nil, errors.Trace(err) + } - // batch get values + // Batch get values. nKeys := 0 - for _, v := range rowWithKeys { + for _, v := range keysInRows { nKeys += len(v) } batchKeys := make([]kv.Key, 0, nKeys) - for _, v := range rowWithKeys { + for _, v := range keysInRows { for _, k := range v { batchKeys = append(batchKeys, k.key) } } - values, err := ctx.Txn().GetSnapshot().BatchGet(batchKeys) + values, err := kv.BatchGetValues(ctx.Txn(), batchKeys) if err != nil { - return nil, errors.Trace(err) + return nil, nil, errors.Trace(err) + } + return keysInRows, values, nil +} + +// checkBatchLimit check the batchSize limitation. +func (e *InsertExec) checkBatchLimit() error { + sessVars := e.ctx.GetSessionVars() + batchInsert := sessVars.BatchInsert && !sessVars.InTxn() + batchSize := sessVars.DMLBatchSize + if batchInsert && e.rowCount >= batchSize { + e.ctx.StmtCommit() + if err := e.ctx.NewTxn(); err != nil { + // We should return a special error for batch insert. + return ErrBatchInsertFail.Gen("BatchInsert failed with error: %v", err) + } + e.rowCount = 0 + if !sessVars.ImportingData { + sessVars.GetWriteStmtBufs().BufStore = kv.NewBufferStore(e.ctx.Txn(), kv.TempTxnMemBufCap) + } + } + return nil +} + +// initDupOldRowValue initializes dupOldRowValues which contain the to-be-updated rows from storage. +func (e *InsertExec) initDupOldRowValue(newRows [][]types.Datum) (err error) { + e.dupOldRowValues = make(map[string][]byte, len(newRows)) + handles := make([]int64, 0, len(newRows)) + for _, keysInRow := range e.uniqueKeysInRows { + for _, k := range keysInRow { + if val, found := e.dupKeyValues[string(k.key)]; found { + var handle int64 + handle, err = decodeOldHandle(k, val) + if err != nil { + return errors.Trace(err) + } + handles = append(handles, handle) + break + } + } + } + e.dupOldRowValues, err = batchGetOldValues(e.ctx, e.Table, handles) + if err != nil { + return errors.Trace(err) + } + return nil +} + +// decodeOldHandle decode old handle by key-value pair. +// The key-value pair should only be a table record or a distinct index record. +// If the key is a record key, decode handle from the key, else decode handle from the value. +func decodeOldHandle(k keyWithDupError, value []byte) (oldHandle int64, err error) { + if k.isRecordKey { + oldHandle, err = tablecodec.DecodeRowKey(k.key) + } else { + oldHandle, err = tables.DecodeHandle(value) + } + if err != nil { + return 0, errors.Trace(err) + } + return oldHandle, nil +} + +// updateDupRow updates a duplicate row to a new row. +func (e *InsertExec) updateDupRow(keys []keyWithDupError, k keyWithDupError, val []byte, newRow []types.Datum, onDuplicate []*expression.Assignment) (err error) { + oldHandle, err := decodeOldHandle(k, val) + if err != nil { + return errors.Trace(err) + } + + // Get the table record row from storage for update. + oldValue, ok := e.dupOldRowValues[string(e.Table.RecordKey(oldHandle))] + if !ok { + return errors.NotFoundf("can not be duplicated row, due to old row not found. handle %d", oldHandle) + } + cols := e.Table.WritableCols() + oldRow, oldRowMap, err := tables.DecodeRawRowData(e.ctx, e.Table.Meta(), oldHandle, cols, oldValue) + if err != nil { + return errors.Trace(err) + } + // Fill write-only and write-reorg columns with originDefaultValue if not found in oldValue. + for _, col := range cols { + if col.State != model.StatePublic && oldRow[col.Offset].IsNull() { + _, found := oldRowMap[col.ID] + if !found { + oldRow[col.Offset], err = table.GetColOriginDefaultValue(e.ctx, col.ToInfo()) + if err != nil { + return errors.Trace(err) + } + } + } + } + + // Do update row. + updatedRow, handleChanged, newHandle, err := e.doDupRowUpdate(oldHandle, oldRow, newRow, onDuplicate) + if err != nil { + return errors.Trace(err) + } + return e.updateDupKeyValues(keys, oldHandle, newHandle, handleChanged, updatedRow) +} + +// updateDupKeyValues updates the dupKeyValues for further duplicate key check. +func (e *InsertExec) updateDupKeyValues(keys []keyWithDupError, oldHandle int64, + newHandle int64, handleChanged bool, updatedRow []types.Datum) error { + // There is only one row per update. + fillBackKeysInRows, err := getKeysNeedCheck(e.ctx, e.Table, [][]types.Datum{updatedRow}) + if err != nil { + return errors.Trace(err) + } + // Delete key-values belong to the old row. + for _, del := range keys { + delete(e.dupKeyValues, string(del.key)) + } + // Fill back new key-values of the updated row. + if handleChanged { + delete(e.dupOldRowValues, string(e.Table.RecordKey(oldHandle))) + e.fillBackKeys(fillBackKeysInRows[0], newHandle) + } else { + e.fillBackKeys(fillBackKeysInRows[0], oldHandle) } + return nil +} + +// batchUpdateDupRows updates multi-rows in batch if they are duplicate with rows in table. +func (e *InsertExec) batchUpdateDupRows(newRows [][]types.Datum, onDuplicate []*expression.Assignment) error { + var err error + e.uniqueKeysInRows, e.dupKeyValues, err = batchGetInsertKeys(e.ctx, e.Table, newRows) + if err != nil { + return errors.Trace(err) + } + + // Batch get the to-be-updated rows in storage. + err = e.initDupOldRowValue(newRows) + if err != nil { + return errors.Trace(err) + } + + for i, keysInRow := range e.uniqueKeysInRows { + for _, k := range keysInRow { + if val, found := e.dupKeyValues[string(k.key)]; found { + err := e.updateDupRow(keysInRow, k, val, newRows[i], e.OnDuplicate) + if err != nil { + return errors.Trace(err) + } + // Clean up row for latest add record operation. + newRows[i] = nil + break + } + } + // If row was checked with no duplicate keys, + // we should do insert the row, + // and key-values should be filled back to dupOldRowValues for the further row check, + // due to there may be duplicate keys inside the insert statement. + if newRows[i] != nil { + newHandle, err := e.insertOneRow(newRows[i]) + if err != nil { + return errors.Trace(err) + } + e.fillBackKeys(keysInRow, newHandle) + } + } + return nil +} +// fillBackKeys fills the updated key-value pair to the dupKeyValues for further check. +func (e *InsertExec) fillBackKeys(fillBackKeysInRow []keyWithDupError, handle int64) { + e.dupOldRowValues[string(e.Table.RecordKey(handle))] = fillBackKeysInRow[0].newRowValue + for _, insert := range fillBackKeysInRow { + if insert.isRecordKey { + e.dupKeyValues[string(e.Table.RecordKey(handle))] = insert.newRowValue + } else { + e.dupKeyValues[string(insert.key)] = tables.EncodeHandle(handle) + } + } +} + +// batchMarkDupRows marks rows with duplicate errors as nil. +// All duplicate rows were marked and appended as duplicate warnings +// to the statement context in batch. +func batchMarkDupRows(ctx sessionctx.Context, t table.Table, rows [][]types.Datum) ([][]types.Datum, error) { + rowWithKeys, values, err := batchGetInsertKeys(ctx, t, rows) + if err != nil { + return nil, errors.Trace(err) + } // append warnings and get no duplicated error rows for i, v := range rowWithKeys { for _, k := range v { @@ -989,14 +1253,13 @@ func batchMarkDupRows(ctx sessionctx.Context, t table.Table, rows [][]types.Datu } } } - // this statement was already been checked ctx.GetSessionVars().StmtCtx.BatchCheck = true return rows, nil } -// NextChunk implements Exec NextChunk interface. -func (e *InsertExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements Exec Next interface. +func (e *InsertExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.finished { return nil @@ -1268,7 +1531,7 @@ func (e *InsertValues) getRowsSelectChunk(ctx context.Context, cols []*table.Col chk := selectExec.newChunk() iter := chunk.NewIterator4Chunk(chk) - err := selectExec.NextChunk(ctx, chk) + err := selectExec.Next(ctx, chk) if err != nil { return nil, errors.Trace(err) } @@ -1468,33 +1731,32 @@ func (e *InsertValues) adjustAutoIncrementDatum(row []types.Datum, i int, c *tab return nil } -// onDuplicateUpdate updates the duplicate row. +// doDupRowUpdate updates the duplicate row. // TODO: Report rows affected and last insert id. -func (e *InsertExec) onDuplicateUpdate(row []types.Datum, h int64, cols []*expression.Assignment) error { - data, err := e.Table.RowWithCols(e.ctx, h, e.Table.WritableCols()) - if err != nil { - return errors.Trace(err) - } - - // See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values - e.ctx.GetSessionVars().CurrInsertValues = types.DatumRow(row) - - // evaluate assignment +func (e *InsertExec) doDupRowUpdate(handle int64, oldRow []types.Datum, newRow []types.Datum, + cols []*expression.Assignment) ([]types.Datum, bool, int64, error) { assignFlag := make([]bool, len(e.Table.WritableCols())) - newData := make(types.DatumRow, len(data)) - copy(newData, data) + // See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values + e.ctx.GetSessionVars().CurrInsertValues = types.DatumRow(newRow) + newData := make(types.DatumRow, len(oldRow)) + copy(newData, oldRow) for _, col := range cols { val, err1 := col.Expr.Eval(newData) if err1 != nil { - return errors.Trace(err1) + return nil, false, 0, errors.Trace(err1) } newData[col.Col.Index] = val assignFlag[col.Col.Index] = true } - if _, err = updateRecord(e.ctx, h, data, newData, assignFlag, e.Table, true, false); err != nil { - return errors.Trace(err) + _, handleChanged, newHandle, err := updateRecord(e.ctx, handle, oldRow, newData, assignFlag, e.Table, true, e.IgnoreErr) + if err != nil { + return nil, false, 0, errors.Trace(err) } - return nil + e.rowCount++ + if err := e.checkBatchLimit(); err != nil { + return nil, false, 0, errors.Trace(err) + } + return newData, handleChanged, newHandle, nil } // ReplaceExec represents a replace executor. @@ -1520,7 +1782,7 @@ func (e *ReplaceExec) Open(ctx context.Context) error { return nil } -func (e *ReplaceExec) exec(ctx context.Context, rows [][]types.Datum) (Row, error) { +func (e *ReplaceExec) exec(ctx context.Context, rows [][]types.Datum) (types.DatumRow, error) { /* * MySQL uses the following algorithm for REPLACE (and LOAD DATA ... REPLACE): * 1. Try to insert the new row into the table @@ -1580,8 +1842,8 @@ func (e *ReplaceExec) exec(ctx context.Context, rows [][]types.Datum) (Row, erro return nil, nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ReplaceExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ReplaceExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.finished { return nil @@ -1617,13 +1879,13 @@ type UpdateExec struct { updatedRowKeys map[int64]map[int64]struct{} tblID2table map[int64]table.Table - rows []Row // The rows fetched from TableExec. - newRowsData [][]types.Datum // The new values to be set. + rows []types.DatumRow // The rows fetched from TableExec. + newRowsData []types.DatumRow // The new values to be set. fetched bool cursor int } -func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema) (Row, error) { +func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema) (types.DatumRow, error) { assignFlag, err := getUpdateColumns(e.OrderedList, schema.Len()) if err != nil { return nil, errors.Trace(err) @@ -1654,7 +1916,7 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema) (Row, continue } // Update row - changed, err1 := updateRecord(e.ctx, handle, oldData, newTableData, flags, tbl, false, e.IgnoreErr) + changed, _, _, err1 := updateRecord(e.ctx, handle, oldData, newTableData, flags, tbl, false, e.IgnoreErr) if err1 == nil { if changed { e.updatedRowKeys[id][handle] = struct{}{} @@ -1670,11 +1932,11 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema) (Row, } } e.cursor++ - return Row{}, nil + return types.DatumRow{}, nil } -// NextChunk implements the Executor NextChunk interface. -func (e *UpdateExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *UpdateExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.fetched { err := e.fetchChunkRows(ctx) @@ -1714,7 +1976,7 @@ func (e *UpdateExec) fetchChunkRows(ctx context.Context) error { globalRowIdx := 0 for { chk := chunk.NewChunk(fields) - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -1750,7 +2012,7 @@ func (e *UpdateExec) handleErr(colName model.CIStr, rowIdx int, err error) error return errors.Trace(err) } -func (e *UpdateExec) composeNewRow(rowIdx int, oldRow Row) (Row, error) { +func (e *UpdateExec) composeNewRow(rowIdx int, oldRow types.DatumRow) (types.DatumRow, error) { newRowData := oldRow.Copy() for _, assign := range e.OrderedList { val, err := assign.Expr.Eval(newRowData) diff --git a/executor/write_test.go b/executor/write_test.go index 147b252fd99b4..36d7de1257904 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -409,6 +409,152 @@ func (s *testSuite) TestInsertIgnore(c *C) { c.Assert(err, IsNil) r = tk.MustQuery("SHOW WARNINGS") r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 'PRIMARY'")) + + testSQL = `drop table if exists test; +create table test (i int primary key, j int unique); +begin; +insert into test values (1,1); +insert ignore into test values (2,1); +commit;` + tk.MustExec(testSQL) + testSQL = `select * from test;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("1 1")) + + testSQL = `delete from test; +insert into test values (1, 1); +begin; +delete from test where i = 1; +insert ignore into test values (2, 1); +commit;` + tk.MustExec(testSQL) + testSQL = `select * from test;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("2 1")) + + testSQL = `delete from test; +insert into test values (1, 1); +begin; +update test set i = 2, j = 2 where i = 1; +insert ignore into test values (1, 3); +insert ignore into test values (2, 4); +commit;` + tk.MustExec(testSQL) + testSQL = `select * from test order by i;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("1 3", "2 2")) +} + +func (s *testSuite) TestInsertOnDup(c *C) { + var cfg kv.InjectionConfig + tk := testkit.NewTestKit(c, kv.NewInjectedStore(s.store, &cfg)) + tk.MustExec("use test") + testSQL := `drop table if exists t; + create table t (i int unique key);` + tk.MustExec(testSQL) + testSQL = `insert into t values (1),(2);` + tk.MustExec(testSQL) + + r := tk.MustQuery("select * from t;") + rowStr1 := fmt.Sprintf("%v", "1") + rowStr2 := fmt.Sprintf("%v", "2") + r.Check(testkit.Rows(rowStr1, rowStr2)) + + tk.MustExec("insert into t values (1), (2) on duplicate key update i = values(i)") + r = tk.MustQuery("select * from t;") + r.Check(testkit.Rows(rowStr1, rowStr2)) + + tk.MustExec("insert into t values (2), (3) on duplicate key update i = 3") + r = tk.MustQuery("select * from t;") + rowStr3 := fmt.Sprintf("%v", "3") + r.Check(testkit.Rows(rowStr1, rowStr3)) + + testSQL = `drop table if exists t; + create table t (i int primary key, j int unique key);` + tk.MustExec(testSQL) + testSQL = `insert into t values (-1, 1);` + tk.MustExec(testSQL) + + r = tk.MustQuery("select * from t;") + rowStr1 = fmt.Sprintf("%v %v", "-1", "1") + r.Check(testkit.Rows(rowStr1)) + + tk.MustExec("insert into t values (1, 1) on duplicate key update j = values(j)") + r = tk.MustQuery("select * from t;") + r.Check(testkit.Rows(rowStr1)) + + testSQL = `drop table if exists test; +create table test (i int primary key, j int unique); +begin; +insert into test values (1,1); +insert into test values (2,1) on duplicate key update i = -i, j = -j; +commit;` + tk.MustExec(testSQL) + testSQL = `select * from test;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("-1 -1")) + + testSQL = `delete from test; +insert into test values (1, 1); +begin; +delete from test where i = 1; +insert into test values (2, 1) on duplicate key update i = -i, j = -j; +commit;` + tk.MustExec(testSQL) + testSQL = `select * from test;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("2 1")) + + testSQL = `delete from test; +insert into test values (1, 1); +begin; +update test set i = 2, j = 2 where i = 1; +insert into test values (1, 3) on duplicate key update i = -i, j = -j; +insert into test values (2, 4) on duplicate key update i = -i, j = -j; +commit;` + tk.MustExec(testSQL) + testSQL = `select * from test order by i;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("-2 -2", "1 3")) + + testSQL = `delete from test; +begin; +insert into test values (1, 3), (1, 3) on duplicate key update i = values(i), j = values(j); +commit;` + tk.MustExec(testSQL) + testSQL = `select * from test order by i;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("1 3")) + + testSQL = `create table tmp (id int auto_increment, code int, primary key(id, code)); + create table m (id int primary key auto_increment, code int unique); + insert tmp (code) values (1); + insert tmp (code) values (1); + insert m (code) select code from tmp on duplicate key update code = values(code);` + tk.MustExec(testSQL) + testSQL = `select * from m;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("1 1")) +} + +func (s *testSuite) TestInsertIgnoreOnDup(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + testSQL := `drop table if exists t; + create table t (i int not null primary key, j int unique key);` + tk.MustExec(testSQL) + testSQL = `insert into t values (1, 1), (2, 2);` + tk.MustExec(testSQL) + testSQL = `insert ignore into t values(1, 1) on duplicate key update i = 2;` + tk.MustExec(testSQL) + testSQL = `select * from t;` + r := tk.MustQuery(testSQL) + r.Check(testkit.Rows("1 1", "2 2")) + testSQL = `insert ignore into t values(1, 1) on duplicate key update j = 2;` + tk.MustExec(testSQL) + testSQL = `select * from t;` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("1 1", "2 2")) } func (s *testSuite) TestReplace(c *C) { diff --git a/expression/bench_test.go b/expression/bench_test.go index be813816d8288..1704afa516d3c 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -175,3 +175,16 @@ func BenchmarkVectorizedExecute(b *testing.B) { } } } + +func BenchmarkScalarFunctionClone(b *testing.B) { + col := &Column{RetType: types.NewFieldType(mysql.TypeLonglong)} + con1 := One.Clone() + con2 := Zero.Clone() + add := NewFunctionInternal(mock.NewContext(), ast.Plus, types.NewFieldType(mysql.TypeLonglong), col, con1) + sub := NewFunctionInternal(mock.NewContext(), ast.Plus, types.NewFieldType(mysql.TypeLonglong), add, con2) + b.ResetTimer() + for i := 0; i < b.N; i++ { + sub.Clone() + } + b.ReportAllocs() +} diff --git a/expression/builtin.go b/expression/builtin.go index f8f31af39ec62..53b614844d807 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -221,6 +221,20 @@ func (b *baseBuiltinFunc) getCtx() sessionctx.Context { return b.ctx } +func (b *baseBuiltinFunc) cloneFrom(from *baseBuiltinFunc) { + b.args = make([]Expression, 0, len(b.args)) + for _, arg := range from.args { + b.args = append(b.args, arg.Clone()) + } + b.ctx = from.ctx + b.tp = from.tp + b.pbCode = from.pbCode +} + +func (b *baseBuiltinFunc) Clone() builtinFunc { + panic("you should not call this method.") +} + // builtinFunc stands for a particular function signature. type builtinFunc interface { // evalInt evaluates int result of builtinFunc by given row. @@ -249,6 +263,8 @@ type builtinFunc interface { setPbCode(tipb.ScalarFuncSig) // PbCode returns PbCode of this signature. PbCode() tipb.ScalarFuncSig + // Clone returns a copy of itself. + Clone() builtinFunc } // baseFunctionClass will be contained in every struct that implement functionClass interface. diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index b9cff5e77ebbb..95663ce29a8eb 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -180,6 +180,12 @@ type builtinArithmeticPlusIntSig struct { baseBuiltinFunc } +func (s *builtinArithmeticPlusIntSig) Clone() builtinFunc { + newSig := &builtinArithmeticPlusIntSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticPlusIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { a, isNull, err := s.args[0].EvalInt(s.ctx, row) if isNull || err != nil { @@ -226,6 +232,12 @@ type builtinArithmeticPlusDecimalSig struct { baseBuiltinFunc } +func (s *builtinArithmeticPlusDecimalSig) Clone() builtinFunc { + newSig := &builtinArithmeticPlusDecimalSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticPlusDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { a, isNull, err := s.args[0].EvalDecimal(s.ctx, row) if isNull || err != nil { @@ -247,6 +259,12 @@ type builtinArithmeticPlusRealSig struct { baseBuiltinFunc } +func (s *builtinArithmeticPlusRealSig) Clone() builtinFunc { + newSig := &builtinArithmeticPlusRealSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticPlusRealSig) evalReal(row types.Row) (float64, bool, error) { a, isNull, err := s.args[0].EvalReal(s.ctx, row) if isNull || err != nil { @@ -301,6 +319,12 @@ type builtinArithmeticMinusRealSig struct { baseBuiltinFunc } +func (s *builtinArithmeticMinusRealSig) Clone() builtinFunc { + newSig := &builtinArithmeticMinusRealSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticMinusRealSig) evalReal(row types.Row) (float64, bool, error) { a, isNull, err := s.args[0].EvalReal(s.ctx, row) if isNull || err != nil { @@ -320,6 +344,12 @@ type builtinArithmeticMinusDecimalSig struct { baseBuiltinFunc } +func (s *builtinArithmeticMinusDecimalSig) Clone() builtinFunc { + newSig := &builtinArithmeticMinusDecimalSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticMinusDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { a, isNull, err := s.args[0].EvalDecimal(s.ctx, row) if isNull || err != nil { @@ -341,6 +371,12 @@ type builtinArithmeticMinusIntSig struct { baseBuiltinFunc } +func (s *builtinArithmeticMinusIntSig) Clone() builtinFunc { + newSig := &builtinArithmeticMinusIntSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticMinusIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { a, isNull, err := s.args[0].EvalInt(s.ctx, row) if isNull || err != nil { @@ -429,10 +465,37 @@ func (c *arithmeticMultiplyFunctionClass) getFunction(ctx sessionctx.Context, ar } type builtinArithmeticMultiplyRealSig struct{ baseBuiltinFunc } + +func (s *builtinArithmeticMultiplyRealSig) Clone() builtinFunc { + newSig := &builtinArithmeticMultiplyRealSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + type builtinArithmeticMultiplyDecimalSig struct{ baseBuiltinFunc } + +func (s *builtinArithmeticMultiplyDecimalSig) Clone() builtinFunc { + newSig := &builtinArithmeticMultiplyDecimalSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + type builtinArithmeticMultiplyIntUnsignedSig struct{ baseBuiltinFunc } + +func (s *builtinArithmeticMultiplyIntUnsignedSig) Clone() builtinFunc { + newSig := &builtinArithmeticMultiplyIntUnsignedSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + type builtinArithmeticMultiplyIntSig struct{ baseBuiltinFunc } +func (s *builtinArithmeticMultiplyIntSig) Clone() builtinFunc { + newSig := &builtinArithmeticMultiplyIntSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticMultiplyRealSig) evalReal(row types.Row) (float64, bool, error) { a, isNull, err := s.args[0].EvalReal(s.ctx, row) if isNull || err != nil { @@ -524,8 +587,21 @@ func (c *arithmeticDivideFunctionClass) getFunction(ctx sessionctx.Context, args } type builtinArithmeticDivideRealSig struct{ baseBuiltinFunc } + +func (s *builtinArithmeticDivideRealSig) Clone() builtinFunc { + newSig := &builtinArithmeticDivideRealSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + type builtinArithmeticDivideDecimalSig struct{ baseBuiltinFunc } +func (s *builtinArithmeticDivideDecimalSig) Clone() builtinFunc { + newSig := &builtinArithmeticDivideDecimalSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticDivideRealSig) evalReal(row types.Row) (float64, bool, error) { a, isNull, err := s.args[0].EvalReal(s.ctx, row) if isNull || err != nil { @@ -592,8 +668,21 @@ func (c *arithmeticIntDivideFunctionClass) getFunction(ctx sessionctx.Context, a } type builtinArithmeticIntDivideIntSig struct{ baseBuiltinFunc } + +func (s *builtinArithmeticIntDivideIntSig) Clone() builtinFunc { + newSig := &builtinArithmeticIntDivideIntSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + type builtinArithmeticIntDivideDecimalSig struct{ baseBuiltinFunc } +func (s *builtinArithmeticIntDivideDecimalSig) Clone() builtinFunc { + newSig := &builtinArithmeticIntDivideDecimalSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticIntDivideIntSig) evalInt(row types.Row) (int64, bool, error) { b, isNull, err := s.args[1].EvalInt(s.ctx, row) if isNull || err != nil { @@ -722,6 +811,12 @@ type builtinArithmeticModRealSig struct { baseBuiltinFunc } +func (s *builtinArithmeticModRealSig) Clone() builtinFunc { + newSig := &builtinArithmeticModRealSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticModRealSig) evalReal(row types.Row) (float64, bool, error) { b, isNull, err := s.args[1].EvalReal(s.ctx, row) if isNull || err != nil { @@ -744,6 +839,12 @@ type builtinArithmeticModDecimalSig struct { baseBuiltinFunc } +func (s *builtinArithmeticModDecimalSig) Clone() builtinFunc { + newSig := &builtinArithmeticModDecimalSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticModDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { a, isNull, err := s.args[0].EvalDecimal(s.ctx, row) if isNull || err != nil { @@ -765,6 +866,12 @@ type builtinArithmeticModIntSig struct { baseBuiltinFunc } +func (s *builtinArithmeticModIntSig) Clone() builtinFunc { + newSig := &builtinArithmeticModIntSig{} + newSig.cloneFrom(&s.baseBuiltinFunc) + return newSig +} + func (s *builtinArithmeticModIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { b, isNull, err := s.args[1].EvalInt(s.ctx, row) if isNull || err != nil { diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index c635443f5dc96..f534619f19002 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -428,6 +428,12 @@ type builtinCastIntAsIntSig struct { baseBuiltinFunc } +func (b *builtinCastIntAsIntSig) Clone() builtinFunc { + newSig := &builtinCastIntAsIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastIntAsIntSig) evalInt(row types.Row) (res int64, isNull bool, err error) { return b.args[0].EvalInt(b.ctx, row) } @@ -436,6 +442,12 @@ type builtinCastIntAsRealSig struct { baseBuiltinFunc } +func (b *builtinCastIntAsRealSig) Clone() builtinFunc { + newSig := &builtinCastIntAsRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastIntAsRealSig) evalReal(row types.Row) (res float64, isNull bool, err error) { val, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -455,6 +467,12 @@ type builtinCastIntAsDecimalSig struct { baseBuiltinFunc } +func (b *builtinCastIntAsDecimalSig) Clone() builtinFunc { + newSig := &builtinCastIntAsDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastIntAsDecimalSig) evalDecimal(row types.Row) (res *types.MyDecimal, isNull bool, err error) { val, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -478,6 +496,12 @@ type builtinCastIntAsStringSig struct { baseBuiltinFunc } +func (b *builtinCastIntAsStringSig) Clone() builtinFunc { + newSig := &builtinCastIntAsStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastIntAsStringSig) evalString(row types.Row) (res string, isNull bool, err error) { val, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -501,6 +525,12 @@ type builtinCastIntAsTimeSig struct { baseBuiltinFunc } +func (b *builtinCastIntAsTimeSig) Clone() builtinFunc { + newSig := &builtinCastIntAsTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastIntAsTimeSig) evalTime(row types.Row) (res types.Time, isNull bool, err error) { val, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -521,6 +551,12 @@ type builtinCastIntAsDurationSig struct { baseBuiltinFunc } +func (b *builtinCastIntAsDurationSig) Clone() builtinFunc { + newSig := &builtinCastIntAsDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastIntAsDurationSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { val, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -540,6 +576,12 @@ type builtinCastIntAsJSONSig struct { baseBuiltinFunc } +func (b *builtinCastIntAsJSONSig) Clone() builtinFunc { + newSig := &builtinCastIntAsJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastIntAsJSONSig) evalJSON(row types.Row) (res json.BinaryJSON, isNull bool, err error) { val, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -559,6 +601,12 @@ type builtinCastRealAsJSONSig struct { baseBuiltinFunc } +func (b *builtinCastRealAsJSONSig) Clone() builtinFunc { + newSig := &builtinCastRealAsJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastRealAsJSONSig) evalJSON(row types.Row) (res json.BinaryJSON, isNull bool, err error) { val, isNull, err := b.args[0].EvalReal(b.ctx, row) // FIXME: `select json_type(cast(1111.11 as json))` should return `DECIMAL`, we return `DOUBLE` now. @@ -569,6 +617,12 @@ type builtinCastDecimalAsJSONSig struct { baseBuiltinFunc } +func (b *builtinCastDecimalAsJSONSig) Clone() builtinFunc { + newSig := &builtinCastDecimalAsJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDecimalAsJSONSig) evalJSON(row types.Row) (json.BinaryJSON, bool, error) { val, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if isNull || err != nil { @@ -586,6 +640,12 @@ type builtinCastStringAsJSONSig struct { baseBuiltinFunc } +func (b *builtinCastStringAsJSONSig) Clone() builtinFunc { + newSig := &builtinCastStringAsJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastStringAsJSONSig) evalJSON(row types.Row) (res json.BinaryJSON, isNull bool, err error) { val, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { @@ -603,6 +663,12 @@ type builtinCastDurationAsJSONSig struct { baseBuiltinFunc } +func (b *builtinCastDurationAsJSONSig) Clone() builtinFunc { + newSig := &builtinCastDurationAsJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDurationAsJSONSig) evalJSON(row types.Row) (res json.BinaryJSON, isNull bool, err error) { val, isNull, err := b.args[0].EvalDuration(b.ctx, row) if isNull || err != nil { @@ -616,6 +682,12 @@ type builtinCastTimeAsJSONSig struct { baseBuiltinFunc } +func (b *builtinCastTimeAsJSONSig) Clone() builtinFunc { + newSig := &builtinCastTimeAsJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastTimeAsJSONSig) evalJSON(row types.Row) (res json.BinaryJSON, isNull bool, err error) { val, isNull, err := b.args[0].EvalTime(b.ctx, row) if isNull || err != nil { @@ -631,6 +703,12 @@ type builtinCastRealAsRealSig struct { baseBuiltinFunc } +func (b *builtinCastRealAsRealSig) Clone() builtinFunc { + newSig := &builtinCastRealAsRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastRealAsRealSig) evalReal(row types.Row) (res float64, isNull bool, err error) { return b.args[0].EvalReal(b.ctx, row) } @@ -639,6 +717,12 @@ type builtinCastRealAsIntSig struct { baseBuiltinFunc } +func (b *builtinCastRealAsIntSig) Clone() builtinFunc { + newSig := &builtinCastRealAsIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastRealAsIntSig) evalInt(row types.Row) (res int64, isNull bool, err error) { val, isNull, err := b.args[0].EvalReal(b.ctx, row) if isNull || err != nil { @@ -658,6 +742,12 @@ type builtinCastRealAsDecimalSig struct { baseBuiltinFunc } +func (b *builtinCastRealAsDecimalSig) Clone() builtinFunc { + newSig := &builtinCastRealAsDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastRealAsDecimalSig) evalDecimal(row types.Row) (res *types.MyDecimal, isNull bool, err error) { val, isNull, err := b.args[0].EvalReal(b.ctx, row) if isNull || err != nil { @@ -676,6 +766,12 @@ type builtinCastRealAsStringSig struct { baseBuiltinFunc } +func (b *builtinCastRealAsStringSig) Clone() builtinFunc { + newSig := &builtinCastRealAsStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastRealAsStringSig) evalString(row types.Row) (res string, isNull bool, err error) { val, isNull, err := b.args[0].EvalReal(b.ctx, row) if isNull || err != nil { @@ -689,6 +785,12 @@ type builtinCastRealAsTimeSig struct { baseBuiltinFunc } +func (b *builtinCastRealAsTimeSig) Clone() builtinFunc { + newSig := &builtinCastRealAsTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastRealAsTimeSig) evalTime(row types.Row) (types.Time, bool, error) { val, isNull, err := b.args[0].EvalReal(b.ctx, row) if isNull || err != nil { @@ -710,6 +812,12 @@ type builtinCastRealAsDurationSig struct { baseBuiltinFunc } +func (b *builtinCastRealAsDurationSig) Clone() builtinFunc { + newSig := &builtinCastRealAsDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastRealAsDurationSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { val, isNull, err := b.args[0].EvalReal(b.ctx, row) if isNull || err != nil { @@ -723,6 +831,12 @@ type builtinCastDecimalAsDecimalSig struct { baseBuiltinFunc } +func (b *builtinCastDecimalAsDecimalSig) Clone() builtinFunc { + newSig := &builtinCastDecimalAsDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDecimalAsDecimalSig) evalDecimal(row types.Row) (res *types.MyDecimal, isNull bool, err error) { res, isNull, err = b.args[0].EvalDecimal(b.ctx, row) if isNull || err != nil { @@ -737,6 +851,12 @@ type builtinCastDecimalAsIntSig struct { baseBuiltinFunc } +func (b *builtinCastDecimalAsIntSig) Clone() builtinFunc { + newSig := &builtinCastDecimalAsIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDecimalAsIntSig) evalInt(row types.Row) (res int64, isNull bool, err error) { val, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if isNull || err != nil { @@ -770,6 +890,12 @@ type builtinCastDecimalAsStringSig struct { baseBuiltinFunc } +func (b *builtinCastDecimalAsStringSig) Clone() builtinFunc { + newSig := &builtinCastDecimalAsStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDecimalAsStringSig) evalString(row types.Row) (res string, isNull bool, err error) { val, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if isNull || err != nil { @@ -784,6 +910,12 @@ type builtinCastDecimalAsRealSig struct { baseBuiltinFunc } +func (b *builtinCastDecimalAsRealSig) Clone() builtinFunc { + newSig := &builtinCastDecimalAsRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDecimalAsRealSig) evalReal(row types.Row) (res float64, isNull bool, err error) { val, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if isNull || err != nil { @@ -797,6 +929,12 @@ type builtinCastDecimalAsTimeSig struct { baseBuiltinFunc } +func (b *builtinCastDecimalAsTimeSig) Clone() builtinFunc { + newSig := &builtinCastDecimalAsTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDecimalAsTimeSig) evalTime(row types.Row) (res types.Time, isNull bool, err error) { val, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if isNull || err != nil { @@ -818,6 +956,12 @@ type builtinCastDecimalAsDurationSig struct { baseBuiltinFunc } +func (b *builtinCastDecimalAsDurationSig) Clone() builtinFunc { + newSig := &builtinCastDecimalAsDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDecimalAsDurationSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { val, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if isNull || err != nil { @@ -838,6 +982,12 @@ type builtinCastStringAsStringSig struct { baseBuiltinFunc } +func (b *builtinCastStringAsStringSig) Clone() builtinFunc { + newSig := &builtinCastStringAsStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastStringAsStringSig) evalString(row types.Row) (res string, isNull bool, err error) { res, isNull, err = b.args[0].EvalString(b.ctx, row) if isNull || err != nil { @@ -852,6 +1002,12 @@ type builtinCastStringAsIntSig struct { baseBuiltinFunc } +func (b *builtinCastStringAsIntSig) Clone() builtinFunc { + newSig := &builtinCastStringAsIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // handleOverflow handles the overflow caused by cast string as int, // see https://dev.mysql.com/doc/refman/5.7/en/out-of-range-and-overflow.html. // When an out-of-range value is assigned to an integer column, MySQL stores the value representing the corresponding endpoint of the column data type range. If it is in select statement, it will return the @@ -916,6 +1072,12 @@ type builtinCastStringAsRealSig struct { baseBuiltinFunc } +func (b *builtinCastStringAsRealSig) Clone() builtinFunc { + newSig := &builtinCastStringAsRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastStringAsRealSig) evalReal(row types.Row) (res float64, isNull bool, err error) { if IsBinaryLiteral(b.args[0]) { return b.args[0].EvalReal(b.ctx, row) @@ -937,6 +1099,12 @@ type builtinCastStringAsDecimalSig struct { baseBuiltinFunc } +func (b *builtinCastStringAsDecimalSig) Clone() builtinFunc { + newSig := &builtinCastStringAsDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastStringAsDecimalSig) evalDecimal(row types.Row) (res *types.MyDecimal, isNull bool, err error) { if IsBinaryLiteral(b.args[0]) { return b.args[0].EvalDecimal(b.ctx, row) @@ -959,6 +1127,12 @@ type builtinCastStringAsTimeSig struct { baseBuiltinFunc } +func (b *builtinCastStringAsTimeSig) Clone() builtinFunc { + newSig := &builtinCastStringAsTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastStringAsTimeSig) evalTime(row types.Row) (res types.Time, isNull bool, err error) { val, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { @@ -980,6 +1154,12 @@ type builtinCastStringAsDurationSig struct { baseBuiltinFunc } +func (b *builtinCastStringAsDurationSig) Clone() builtinFunc { + newSig := &builtinCastStringAsDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastStringAsDurationSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { val, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { @@ -1001,6 +1181,12 @@ type builtinCastTimeAsTimeSig struct { baseBuiltinFunc } +func (b *builtinCastTimeAsTimeSig) Clone() builtinFunc { + newSig := &builtinCastTimeAsTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastTimeAsTimeSig) evalTime(row types.Row) (res types.Time, isNull bool, err error) { res, isNull, err = b.args[0].EvalTime(b.ctx, row) if isNull || err != nil { @@ -1024,6 +1210,12 @@ type builtinCastTimeAsIntSig struct { baseBuiltinFunc } +func (b *builtinCastTimeAsIntSig) Clone() builtinFunc { + newSig := &builtinCastTimeAsIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastTimeAsIntSig) evalInt(row types.Row) (res int64, isNull bool, err error) { val, isNull, err := b.args[0].EvalTime(b.ctx, row) if isNull || err != nil { @@ -1042,6 +1234,12 @@ type builtinCastTimeAsRealSig struct { baseBuiltinFunc } +func (b *builtinCastTimeAsRealSig) Clone() builtinFunc { + newSig := &builtinCastTimeAsRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastTimeAsRealSig) evalReal(row types.Row) (res float64, isNull bool, err error) { val, isNull, err := b.args[0].EvalTime(b.ctx, row) if isNull || err != nil { @@ -1055,6 +1253,12 @@ type builtinCastTimeAsDecimalSig struct { baseBuiltinFunc } +func (b *builtinCastTimeAsDecimalSig) Clone() builtinFunc { + newSig := &builtinCastTimeAsDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastTimeAsDecimalSig) evalDecimal(row types.Row) (res *types.MyDecimal, isNull bool, err error) { val, isNull, err := b.args[0].EvalTime(b.ctx, row) if isNull || err != nil { @@ -1069,6 +1273,12 @@ type builtinCastTimeAsStringSig struct { baseBuiltinFunc } +func (b *builtinCastTimeAsStringSig) Clone() builtinFunc { + newSig := &builtinCastTimeAsStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastTimeAsStringSig) evalString(row types.Row) (res string, isNull bool, err error) { val, isNull, err := b.args[0].EvalTime(b.ctx, row) if isNull || err != nil { @@ -1083,6 +1293,12 @@ type builtinCastTimeAsDurationSig struct { baseBuiltinFunc } +func (b *builtinCastTimeAsDurationSig) Clone() builtinFunc { + newSig := &builtinCastTimeAsDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastTimeAsDurationSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { val, isNull, err := b.args[0].EvalTime(b.ctx, row) if isNull || err != nil { @@ -1100,6 +1316,12 @@ type builtinCastDurationAsDurationSig struct { baseBuiltinFunc } +func (b *builtinCastDurationAsDurationSig) Clone() builtinFunc { + newSig := &builtinCastDurationAsDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDurationAsDurationSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { res, isNull, err = b.args[0].EvalDuration(b.ctx, row) if isNull || err != nil { @@ -1113,6 +1335,12 @@ type builtinCastDurationAsIntSig struct { baseBuiltinFunc } +func (b *builtinCastDurationAsIntSig) Clone() builtinFunc { + newSig := &builtinCastDurationAsIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDurationAsIntSig) evalInt(row types.Row) (res int64, isNull bool, err error) { val, isNull, err := b.args[0].EvalDuration(b.ctx, row) if isNull || err != nil { @@ -1130,6 +1358,12 @@ type builtinCastDurationAsRealSig struct { baseBuiltinFunc } +func (b *builtinCastDurationAsRealSig) Clone() builtinFunc { + newSig := &builtinCastDurationAsRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDurationAsRealSig) evalReal(row types.Row) (res float64, isNull bool, err error) { val, isNull, err := b.args[0].EvalDuration(b.ctx, row) if isNull || err != nil { @@ -1143,6 +1377,12 @@ type builtinCastDurationAsDecimalSig struct { baseBuiltinFunc } +func (b *builtinCastDurationAsDecimalSig) Clone() builtinFunc { + newSig := &builtinCastDurationAsDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDurationAsDecimalSig) evalDecimal(row types.Row) (res *types.MyDecimal, isNull bool, err error) { val, isNull, err := b.args[0].EvalDuration(b.ctx, row) if isNull || err != nil { @@ -1157,6 +1397,12 @@ type builtinCastDurationAsStringSig struct { baseBuiltinFunc } +func (b *builtinCastDurationAsStringSig) Clone() builtinFunc { + newSig := &builtinCastDurationAsStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDurationAsStringSig) evalString(row types.Row) (res string, isNull bool, err error) { val, isNull, err := b.args[0].EvalDuration(b.ctx, row) if isNull || err != nil { @@ -1171,6 +1417,12 @@ type builtinCastDurationAsTimeSig struct { baseBuiltinFunc } +func (b *builtinCastDurationAsTimeSig) Clone() builtinFunc { + newSig := &builtinCastDurationAsTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastDurationAsTimeSig) evalTime(row types.Row) (res types.Time, isNull bool, err error) { val, isNull, err := b.args[0].EvalDuration(b.ctx, row) if isNull || err != nil { @@ -1189,6 +1441,12 @@ type builtinCastJSONAsJSONSig struct { baseBuiltinFunc } +func (b *builtinCastJSONAsJSONSig) Clone() builtinFunc { + newSig := &builtinCastJSONAsJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastJSONAsJSONSig) evalJSON(row types.Row) (res json.BinaryJSON, isNull bool, err error) { return b.args[0].EvalJSON(b.ctx, row) } @@ -1197,6 +1455,12 @@ type builtinCastJSONAsIntSig struct { baseBuiltinFunc } +func (b *builtinCastJSONAsIntSig) Clone() builtinFunc { + newSig := &builtinCastJSONAsIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastJSONAsIntSig) evalInt(row types.Row) (res int64, isNull bool, err error) { val, isNull, err := b.args[0].EvalJSON(b.ctx, row) if isNull || err != nil { @@ -1211,6 +1475,12 @@ type builtinCastJSONAsRealSig struct { baseBuiltinFunc } +func (b *builtinCastJSONAsRealSig) Clone() builtinFunc { + newSig := &builtinCastJSONAsRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastJSONAsRealSig) evalReal(row types.Row) (res float64, isNull bool, err error) { val, isNull, err := b.args[0].EvalJSON(b.ctx, row) if isNull || err != nil { @@ -1225,6 +1495,12 @@ type builtinCastJSONAsDecimalSig struct { baseBuiltinFunc } +func (b *builtinCastJSONAsDecimalSig) Clone() builtinFunc { + newSig := &builtinCastJSONAsDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastJSONAsDecimalSig) evalDecimal(row types.Row) (res *types.MyDecimal, isNull bool, err error) { val, isNull, err := b.args[0].EvalJSON(b.ctx, row) if isNull || err != nil { @@ -1243,6 +1519,12 @@ type builtinCastJSONAsStringSig struct { baseBuiltinFunc } +func (b *builtinCastJSONAsStringSig) Clone() builtinFunc { + newSig := &builtinCastJSONAsStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastJSONAsStringSig) evalString(row types.Row) (res string, isNull bool, err error) { val, isNull, err := b.args[0].EvalJSON(b.ctx, row) if isNull || err != nil { @@ -1255,6 +1537,12 @@ type builtinCastJSONAsTimeSig struct { baseBuiltinFunc } +func (b *builtinCastJSONAsTimeSig) Clone() builtinFunc { + newSig := &builtinCastJSONAsTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastJSONAsTimeSig) evalTime(row types.Row) (res types.Time, isNull bool, err error) { val, isNull, err := b.args[0].EvalJSON(b.ctx, row) if isNull || err != nil { @@ -1280,6 +1568,12 @@ type builtinCastJSONAsDurationSig struct { baseBuiltinFunc } +func (b *builtinCastJSONAsDurationSig) Clone() builtinFunc { + newSig := &builtinCastJSONAsDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCastJSONAsDurationSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { val, isNull, err := b.args[0].EvalJSON(b.ctx, row) if isNull || err != nil { diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 7561b90cdfee6..72c9edd2dddb3 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -206,6 +206,12 @@ type builtinCoalesceIntSig struct { baseBuiltinFunc } +func (b *builtinCoalesceIntSig) Clone() builtinFunc { + newSig := &builtinCoalesceIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCoalesceIntSig) evalInt(row types.Row) (res int64, isNull bool, err error) { for _, a := range b.getArgs() { res, isNull, err = a.EvalInt(b.ctx, row) @@ -222,6 +228,12 @@ type builtinCoalesceRealSig struct { baseBuiltinFunc } +func (b *builtinCoalesceRealSig) Clone() builtinFunc { + newSig := &builtinCoalesceRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCoalesceRealSig) evalReal(row types.Row) (res float64, isNull bool, err error) { for _, a := range b.getArgs() { res, isNull, err = a.EvalReal(b.ctx, row) @@ -238,6 +250,12 @@ type builtinCoalesceDecimalSig struct { baseBuiltinFunc } +func (b *builtinCoalesceDecimalSig) Clone() builtinFunc { + newSig := &builtinCoalesceDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCoalesceDecimalSig) evalDecimal(row types.Row) (res *types.MyDecimal, isNull bool, err error) { for _, a := range b.getArgs() { res, isNull, err = a.EvalDecimal(b.ctx, row) @@ -254,6 +272,12 @@ type builtinCoalesceStringSig struct { baseBuiltinFunc } +func (b *builtinCoalesceStringSig) Clone() builtinFunc { + newSig := &builtinCoalesceStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCoalesceStringSig) evalString(row types.Row) (res string, isNull bool, err error) { for _, a := range b.getArgs() { res, isNull, err = a.EvalString(b.ctx, row) @@ -270,6 +294,12 @@ type builtinCoalesceTimeSig struct { baseBuiltinFunc } +func (b *builtinCoalesceTimeSig) Clone() builtinFunc { + newSig := &builtinCoalesceTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCoalesceTimeSig) evalTime(row types.Row) (res types.Time, isNull bool, err error) { for _, a := range b.getArgs() { res, isNull, err = a.EvalTime(b.ctx, row) @@ -286,6 +316,12 @@ type builtinCoalesceDurationSig struct { baseBuiltinFunc } +func (b *builtinCoalesceDurationSig) Clone() builtinFunc { + newSig := &builtinCoalesceDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCoalesceDurationSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { for _, a := range b.getArgs() { res, isNull, err = a.EvalDuration(b.ctx, row) @@ -385,6 +421,12 @@ type builtinGreatestIntSig struct { baseBuiltinFunc } +func (b *builtinGreatestIntSig) Clone() builtinFunc { + newSig := &builtinGreatestIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinGreatestIntSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_greatest func (b *builtinGreatestIntSig) evalInt(row types.Row) (max int64, isNull bool, err error) { @@ -409,6 +451,12 @@ type builtinGreatestRealSig struct { baseBuiltinFunc } +func (b *builtinGreatestRealSig) Clone() builtinFunc { + newSig := &builtinGreatestRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinGreatestRealSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_greatest func (b *builtinGreatestRealSig) evalReal(row types.Row) (max float64, isNull bool, err error) { @@ -433,6 +481,12 @@ type builtinGreatestDecimalSig struct { baseBuiltinFunc } +func (b *builtinGreatestDecimalSig) Clone() builtinFunc { + newSig := &builtinGreatestDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals a builtinGreatestDecimalSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_greatest func (b *builtinGreatestDecimalSig) evalDecimal(row types.Row) (max *types.MyDecimal, isNull bool, err error) { @@ -457,6 +511,12 @@ type builtinGreatestStringSig struct { baseBuiltinFunc } +func (b *builtinGreatestStringSig) Clone() builtinFunc { + newSig := &builtinGreatestStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinGreatestStringSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_greatest func (b *builtinGreatestStringSig) evalString(row types.Row) (max string, isNull bool, err error) { @@ -481,6 +541,12 @@ type builtinGreatestTimeSig struct { baseBuiltinFunc } +func (b *builtinGreatestTimeSig) Clone() builtinFunc { + newSig := &builtinGreatestTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinGreatestTimeSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_greatest func (b *builtinGreatestTimeSig) evalString(row types.Row) (_ string, isNull bool, err error) { @@ -549,6 +615,12 @@ type builtinLeastIntSig struct { baseBuiltinFunc } +func (b *builtinLeastIntSig) Clone() builtinFunc { + newSig := &builtinLeastIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinLeastIntSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#functionleast func (b *builtinLeastIntSig) evalInt(row types.Row) (min int64, isNull bool, err error) { @@ -573,6 +645,12 @@ type builtinLeastRealSig struct { baseBuiltinFunc } +func (b *builtinLeastRealSig) Clone() builtinFunc { + newSig := &builtinLeastRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinLeastRealSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#functionleast func (b *builtinLeastRealSig) evalReal(row types.Row) (min float64, isNull bool, err error) { @@ -597,6 +675,12 @@ type builtinLeastDecimalSig struct { baseBuiltinFunc } +func (b *builtinLeastDecimalSig) Clone() builtinFunc { + newSig := &builtinLeastDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals a builtinLeastDecimalSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#functionleast func (b *builtinLeastDecimalSig) evalDecimal(row types.Row) (min *types.MyDecimal, isNull bool, err error) { @@ -621,6 +705,12 @@ type builtinLeastStringSig struct { baseBuiltinFunc } +func (b *builtinLeastStringSig) Clone() builtinFunc { + newSig := &builtinLeastStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinLeastStringSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#functionleast func (b *builtinLeastStringSig) evalString(row types.Row) (min string, isNull bool, err error) { @@ -645,6 +735,12 @@ type builtinLeastTimeSig struct { baseBuiltinFunc } +func (b *builtinLeastTimeSig) Clone() builtinFunc { + newSig := &builtinLeastTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinLeastTimeSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#functionleast func (b *builtinLeastTimeSig) evalString(row types.Row) (res string, isNull bool, err error) { @@ -720,6 +816,12 @@ type builtinIntervalIntSig struct { baseBuiltinFunc } +func (b *builtinIntervalIntSig) Clone() builtinFunc { + newSig := &builtinIntervalIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinIntervalIntSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_interval func (b *builtinIntervalIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -774,6 +876,12 @@ type builtinIntervalRealSig struct { baseBuiltinFunc } +func (b *builtinIntervalRealSig) Clone() builtinFunc { + newSig := &builtinIntervalRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinIntervalRealSig. // See http://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_interval func (b *builtinIntervalRealSig) evalInt(row types.Row) (int64, bool, error) { @@ -1186,352 +1294,610 @@ type builtinLTIntSig struct { baseBuiltinFunc } -func (s *builtinLTIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLT(compareInt(s.ctx, s.args, row)) +func (b *builtinLTIntSig) Clone() builtinFunc { + newSig := &builtinLTIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLTIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLT(compareInt(b.ctx, b.args, row)) } type builtinLTRealSig struct { baseBuiltinFunc } -func (s *builtinLTRealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLT(compareReal(s.ctx, s.args, row)) +func (b *builtinLTRealSig) Clone() builtinFunc { + newSig := &builtinLTRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLTRealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLT(compareReal(b.ctx, b.args, row)) } type builtinLTDecimalSig struct { baseBuiltinFunc } -func (s *builtinLTDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLT(compareDecimal(s.ctx, s.args, row)) +func (b *builtinLTDecimalSig) Clone() builtinFunc { + newSig := &builtinLTDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLTDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLT(compareDecimal(b.ctx, b.args, row)) } type builtinLTStringSig struct { baseBuiltinFunc } -func (s *builtinLTStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLT(compareString(s.args, row, s.ctx)) +func (b *builtinLTStringSig) Clone() builtinFunc { + newSig := &builtinLTStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLTStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLT(compareString(b.args, row, b.ctx)) } type builtinLTDurationSig struct { baseBuiltinFunc } -func (s *builtinLTDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLT(compareDuration(s.args, row, s.ctx)) +func (b *builtinLTDurationSig) Clone() builtinFunc { + newSig := &builtinLTDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLTDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLT(compareDuration(b.args, row, b.ctx)) } type builtinLTTimeSig struct { baseBuiltinFunc } -func (s *builtinLTTimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLT(compareTime(s.ctx, s.args, row)) +func (b *builtinLTTimeSig) Clone() builtinFunc { + newSig := &builtinLTTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLTTimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLT(compareTime(b.ctx, b.args, row)) } type builtinLTJSONSig struct { baseBuiltinFunc } -func (s *builtinLTJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLT(compareJSON(s.ctx, s.args, row)) +func (b *builtinLTJSONSig) Clone() builtinFunc { + newSig := &builtinLTJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLTJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLT(compareJSON(b.ctx, b.args, row)) } type builtinLEIntSig struct { baseBuiltinFunc } -func (s *builtinLEIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLE(compareInt(s.ctx, s.args, row)) +func (b *builtinLEIntSig) Clone() builtinFunc { + newSig := &builtinLEIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLEIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLE(compareInt(b.ctx, b.args, row)) } type builtinLERealSig struct { baseBuiltinFunc } -func (s *builtinLERealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLE(compareReal(s.ctx, s.args, row)) +func (b *builtinLERealSig) Clone() builtinFunc { + newSig := &builtinLERealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLERealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLE(compareReal(b.ctx, b.args, row)) } type builtinLEDecimalSig struct { baseBuiltinFunc } -func (s *builtinLEDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLE(compareDecimal(s.ctx, s.args, row)) +func (b *builtinLEDecimalSig) Clone() builtinFunc { + newSig := &builtinLEDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLEDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLE(compareDecimal(b.ctx, b.args, row)) } type builtinLEStringSig struct { baseBuiltinFunc } -func (s *builtinLEStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLE(compareString(s.args, row, s.ctx)) +func (b *builtinLEStringSig) Clone() builtinFunc { + newSig := &builtinLEStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLEStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLE(compareString(b.args, row, b.ctx)) } type builtinLEDurationSig struct { baseBuiltinFunc } -func (s *builtinLEDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLE(compareDuration(s.args, row, s.ctx)) +func (b *builtinLEDurationSig) Clone() builtinFunc { + newSig := &builtinLEDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLEDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLE(compareDuration(b.args, row, b.ctx)) } type builtinLETimeSig struct { baseBuiltinFunc } -func (s *builtinLETimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLE(compareTime(s.ctx, s.args, row)) +func (b *builtinLETimeSig) Clone() builtinFunc { + newSig := &builtinLETimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLETimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLE(compareTime(b.ctx, b.args, row)) } type builtinLEJSONSig struct { baseBuiltinFunc } -func (s *builtinLEJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfLE(compareJSON(s.ctx, s.args, row)) +func (b *builtinLEJSONSig) Clone() builtinFunc { + newSig := &builtinLEJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinLEJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfLE(compareJSON(b.ctx, b.args, row)) } type builtinGTIntSig struct { baseBuiltinFunc } -func (s *builtinGTIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGT(compareInt(s.ctx, s.args, row)) +func (b *builtinGTIntSig) Clone() builtinFunc { + newSig := &builtinGTIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGTIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGT(compareInt(b.ctx, b.args, row)) } type builtinGTRealSig struct { baseBuiltinFunc } -func (s *builtinGTRealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGT(compareReal(s.ctx, s.args, row)) +func (b *builtinGTRealSig) Clone() builtinFunc { + newSig := &builtinGTRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGTRealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGT(compareReal(b.ctx, b.args, row)) } type builtinGTDecimalSig struct { baseBuiltinFunc } -func (s *builtinGTDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGT(compareDecimal(s.ctx, s.args, row)) +func (b *builtinGTDecimalSig) Clone() builtinFunc { + newSig := &builtinGTDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGTDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGT(compareDecimal(b.ctx, b.args, row)) } type builtinGTStringSig struct { baseBuiltinFunc } -func (s *builtinGTStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGT(compareString(s.args, row, s.ctx)) +func (b *builtinGTStringSig) Clone() builtinFunc { + newSig := &builtinGTStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGTStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGT(compareString(b.args, row, b.ctx)) } type builtinGTDurationSig struct { baseBuiltinFunc } -func (s *builtinGTDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGT(compareDuration(s.args, row, s.ctx)) +func (b *builtinGTDurationSig) Clone() builtinFunc { + newSig := &builtinGTDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGTDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGT(compareDuration(b.args, row, b.ctx)) } type builtinGTTimeSig struct { baseBuiltinFunc } -func (s *builtinGTTimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGT(compareTime(s.ctx, s.args, row)) +func (b *builtinGTTimeSig) Clone() builtinFunc { + newSig := &builtinGTTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGTTimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGT(compareTime(b.ctx, b.args, row)) } type builtinGTJSONSig struct { baseBuiltinFunc } -func (s *builtinGTJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGT(compareJSON(s.ctx, s.args, row)) +func (b *builtinGTJSONSig) Clone() builtinFunc { + newSig := &builtinGTJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGTJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGT(compareJSON(b.ctx, b.args, row)) } type builtinGEIntSig struct { baseBuiltinFunc } -func (s *builtinGEIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGE(compareInt(s.ctx, s.args, row)) +func (b *builtinGEIntSig) Clone() builtinFunc { + newSig := &builtinGEIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGEIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGE(compareInt(b.ctx, b.args, row)) } type builtinGERealSig struct { baseBuiltinFunc } -func (s *builtinGERealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGE(compareReal(s.ctx, s.args, row)) +func (b *builtinGERealSig) Clone() builtinFunc { + newSig := &builtinGERealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGERealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGE(compareReal(b.ctx, b.args, row)) } type builtinGEDecimalSig struct { baseBuiltinFunc } -func (s *builtinGEDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGE(compareDecimal(s.ctx, s.args, row)) +func (b *builtinGEDecimalSig) Clone() builtinFunc { + newSig := &builtinGEDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGEDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGE(compareDecimal(b.ctx, b.args, row)) } type builtinGEStringSig struct { baseBuiltinFunc } -func (s *builtinGEStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGE(compareString(s.args, row, s.ctx)) +func (b *builtinGEStringSig) Clone() builtinFunc { + newSig := &builtinGEStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGEStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGE(compareString(b.args, row, b.ctx)) } type builtinGEDurationSig struct { baseBuiltinFunc } -func (s *builtinGEDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGE(compareDuration(s.args, row, s.ctx)) +func (b *builtinGEDurationSig) Clone() builtinFunc { + newSig := &builtinGEDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGEDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGE(compareDuration(b.args, row, b.ctx)) } type builtinGETimeSig struct { baseBuiltinFunc } -func (s *builtinGETimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGE(compareTime(s.ctx, s.args, row)) +func (b *builtinGETimeSig) Clone() builtinFunc { + newSig := &builtinGETimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGETimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGE(compareTime(b.ctx, b.args, row)) } type builtinGEJSONSig struct { baseBuiltinFunc } -func (s *builtinGEJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfGE(compareJSON(s.ctx, s.args, row)) +func (b *builtinGEJSONSig) Clone() builtinFunc { + newSig := &builtinGEJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinGEJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfGE(compareJSON(b.ctx, b.args, row)) } type builtinEQIntSig struct { baseBuiltinFunc } -func (s *builtinEQIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfEQ(compareInt(s.ctx, s.args, row)) +func (b *builtinEQIntSig) Clone() builtinFunc { + newSig := &builtinEQIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinEQIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfEQ(compareInt(b.ctx, b.args, row)) } type builtinEQRealSig struct { baseBuiltinFunc } -func (s *builtinEQRealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfEQ(compareReal(s.ctx, s.args, row)) +func (b *builtinEQRealSig) Clone() builtinFunc { + newSig := &builtinEQRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinEQRealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfEQ(compareReal(b.ctx, b.args, row)) } type builtinEQDecimalSig struct { baseBuiltinFunc } -func (s *builtinEQDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfEQ(compareDecimal(s.ctx, s.args, row)) +func (b *builtinEQDecimalSig) Clone() builtinFunc { + newSig := &builtinEQDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinEQDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfEQ(compareDecimal(b.ctx, b.args, row)) } type builtinEQStringSig struct { baseBuiltinFunc } -func (s *builtinEQStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfEQ(compareString(s.args, row, s.ctx)) +func (b *builtinEQStringSig) Clone() builtinFunc { + newSig := &builtinEQStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinEQStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfEQ(compareString(b.args, row, b.ctx)) } type builtinEQDurationSig struct { baseBuiltinFunc } -func (s *builtinEQDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfEQ(compareDuration(s.args, row, s.ctx)) +func (b *builtinEQDurationSig) Clone() builtinFunc { + newSig := &builtinEQDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinEQDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfEQ(compareDuration(b.args, row, b.ctx)) } type builtinEQTimeSig struct { baseBuiltinFunc } -func (s *builtinEQTimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfEQ(compareTime(s.ctx, s.args, row)) +func (b *builtinEQTimeSig) Clone() builtinFunc { + newSig := &builtinEQTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinEQTimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfEQ(compareTime(b.ctx, b.args, row)) } type builtinEQJSONSig struct { baseBuiltinFunc } -func (s *builtinEQJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfEQ(compareJSON(s.ctx, s.args, row)) +func (b *builtinEQJSONSig) Clone() builtinFunc { + newSig := &builtinEQJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinEQJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfEQ(compareJSON(b.ctx, b.args, row)) } type builtinNEIntSig struct { baseBuiltinFunc } -func (s *builtinNEIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfNE(compareInt(s.ctx, s.args, row)) +func (b *builtinNEIntSig) Clone() builtinFunc { + newSig := &builtinNEIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNEIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfNE(compareInt(b.ctx, b.args, row)) } type builtinNERealSig struct { baseBuiltinFunc } -func (s *builtinNERealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfNE(compareReal(s.ctx, s.args, row)) +func (b *builtinNERealSig) Clone() builtinFunc { + newSig := &builtinNERealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNERealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfNE(compareReal(b.ctx, b.args, row)) } type builtinNEDecimalSig struct { baseBuiltinFunc } -func (s *builtinNEDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfNE(compareDecimal(s.ctx, s.args, row)) +func (b *builtinNEDecimalSig) Clone() builtinFunc { + newSig := &builtinNEDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNEDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfNE(compareDecimal(b.ctx, b.args, row)) } type builtinNEStringSig struct { baseBuiltinFunc } -func (s *builtinNEStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfNE(compareString(s.args, row, s.ctx)) +func (b *builtinNEStringSig) Clone() builtinFunc { + newSig := &builtinNEStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNEStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfNE(compareString(b.args, row, b.ctx)) } type builtinNEDurationSig struct { baseBuiltinFunc } -func (s *builtinNEDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfNE(compareDuration(s.args, row, s.ctx)) +func (b *builtinNEDurationSig) Clone() builtinFunc { + newSig := &builtinNEDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNEDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfNE(compareDuration(b.args, row, b.ctx)) } type builtinNETimeSig struct { baseBuiltinFunc } -func (s *builtinNETimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfNE(compareTime(s.ctx, s.args, row)) +func (b *builtinNETimeSig) Clone() builtinFunc { + newSig := &builtinNETimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNETimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfNE(compareTime(b.ctx, b.args, row)) } type builtinNEJSONSig struct { baseBuiltinFunc } -func (s *builtinNEJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - return resOfNE(compareJSON(s.ctx, s.args, row)) +func (b *builtinNEJSONSig) Clone() builtinFunc { + newSig := &builtinNEJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNEJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + return resOfNE(compareJSON(b.ctx, b.args, row)) } type builtinNullEQIntSig struct { baseBuiltinFunc } -func (s *builtinNullEQIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - arg0, isNull0, err := s.args[0].EvalInt(s.ctx, row) +func (b *builtinNullEQIntSig) Clone() builtinFunc { + newSig := &builtinNullEQIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNullEQIntSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { return 0, isNull0, errors.Trace(err) } - arg1, isNull1, err := s.args[1].EvalInt(s.ctx, row) + arg1, isNull1, err := b.args[1].EvalInt(b.ctx, row) if err != nil { return 0, isNull1, errors.Trace(err) } - isUnsigned0, isUnsigned1 := mysql.HasUnsignedFlag(s.args[0].GetType().Flag), mysql.HasUnsignedFlag(s.args[1].GetType().Flag) + isUnsigned0, isUnsigned1 := mysql.HasUnsignedFlag(b.args[0].GetType().Flag), mysql.HasUnsignedFlag(b.args[1].GetType().Flag) var res int64 switch { case isNull0 && isNull1: @@ -1564,12 +1930,18 @@ type builtinNullEQRealSig struct { baseBuiltinFunc } -func (s *builtinNullEQRealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - arg0, isNull0, err := s.args[0].EvalReal(s.ctx, row) +func (b *builtinNullEQRealSig) Clone() builtinFunc { + newSig := &builtinNullEQRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNullEQRealSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + arg0, isNull0, err := b.args[0].EvalReal(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } - arg1, isNull1, err := s.args[1].EvalReal(s.ctx, row) + arg1, isNull1, err := b.args[1].EvalReal(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } @@ -1589,12 +1961,18 @@ type builtinNullEQDecimalSig struct { baseBuiltinFunc } -func (s *builtinNullEQDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - arg0, isNull0, err := s.args[0].EvalDecimal(s.ctx, row) +func (b *builtinNullEQDecimalSig) Clone() builtinFunc { + newSig := &builtinNullEQDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNullEQDecimalSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + arg0, isNull0, err := b.args[0].EvalDecimal(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } - arg1, isNull1, err := s.args[1].EvalDecimal(s.ctx, row) + arg1, isNull1, err := b.args[1].EvalDecimal(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } @@ -1614,12 +1992,18 @@ type builtinNullEQStringSig struct { baseBuiltinFunc } -func (s *builtinNullEQStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - arg0, isNull0, err := s.args[0].EvalString(s.ctx, row) +func (b *builtinNullEQStringSig) Clone() builtinFunc { + newSig := &builtinNullEQStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNullEQStringSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + arg0, isNull0, err := b.args[0].EvalString(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } - arg1, isNull1, err := s.args[1].EvalString(s.ctx, row) + arg1, isNull1, err := b.args[1].EvalString(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } @@ -1639,12 +2023,18 @@ type builtinNullEQDurationSig struct { baseBuiltinFunc } -func (s *builtinNullEQDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - arg0, isNull0, err := s.args[0].EvalDuration(s.ctx, row) +func (b *builtinNullEQDurationSig) Clone() builtinFunc { + newSig := &builtinNullEQDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNullEQDurationSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + arg0, isNull0, err := b.args[0].EvalDuration(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } - arg1, isNull1, err := s.args[1].EvalDuration(s.ctx, row) + arg1, isNull1, err := b.args[1].EvalDuration(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } @@ -1664,12 +2054,18 @@ type builtinNullEQTimeSig struct { baseBuiltinFunc } -func (s *builtinNullEQTimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - arg0, isNull0, err := s.args[0].EvalTime(s.ctx, row) +func (b *builtinNullEQTimeSig) Clone() builtinFunc { + newSig := &builtinNullEQTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNullEQTimeSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + arg0, isNull0, err := b.args[0].EvalTime(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } - arg1, isNull1, err := s.args[1].EvalTime(s.ctx, row) + arg1, isNull1, err := b.args[1].EvalTime(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } @@ -1689,12 +2085,18 @@ type builtinNullEQJSONSig struct { baseBuiltinFunc } -func (s *builtinNullEQJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { - arg0, isNull0, err := s.args[0].EvalJSON(s.ctx, row) +func (b *builtinNullEQJSONSig) Clone() builtinFunc { + newSig := &builtinNullEQJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinNullEQJSONSig) evalInt(row types.Row) (val int64, isNull bool, err error) { + arg0, isNull0, err := b.args[0].EvalJSON(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } - arg1, isNull1, err := s.args[1].EvalJSON(s.ctx, row) + arg1, isNull1, err := b.args[1].EvalJSON(b.ctx, row) if err != nil { return 0, true, errors.Trace(err) } diff --git a/expression/builtin_control.go b/expression/builtin_control.go index 6b5c08c6ddd93..bc4d8dc2b3d41 100644 --- a/expression/builtin_control.go +++ b/expression/builtin_control.go @@ -212,6 +212,12 @@ type builtinCaseWhenIntSig struct { baseBuiltinFunc } +func (b *builtinCaseWhenIntSig) Clone() builtinFunc { + newSig := &builtinCaseWhenIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinCaseWhenIntSig. // See https://dev.mysql.com/doc/refman/5.7/en/case.html func (b *builtinCaseWhenIntSig) evalInt(row types.Row) (ret int64, isNull bool, err error) { @@ -242,6 +248,12 @@ type builtinCaseWhenRealSig struct { baseBuiltinFunc } +func (b *builtinCaseWhenRealSig) Clone() builtinFunc { + newSig := &builtinCaseWhenRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinCaseWhenRealSig. // See https://dev.mysql.com/doc/refman/5.7/en/case.html func (b *builtinCaseWhenRealSig) evalReal(row types.Row) (ret float64, isNull bool, err error) { @@ -272,6 +284,12 @@ type builtinCaseWhenDecimalSig struct { baseBuiltinFunc } +func (b *builtinCaseWhenDecimalSig) Clone() builtinFunc { + newSig := &builtinCaseWhenDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals a builtinCaseWhenDecimalSig. // See https://dev.mysql.com/doc/refman/5.7/en/case.html func (b *builtinCaseWhenDecimalSig) evalDecimal(row types.Row) (ret *types.MyDecimal, isNull bool, err error) { @@ -302,6 +320,12 @@ type builtinCaseWhenStringSig struct { baseBuiltinFunc } +func (b *builtinCaseWhenStringSig) Clone() builtinFunc { + newSig := &builtinCaseWhenStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinCaseWhenStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/case.html func (b *builtinCaseWhenStringSig) evalString(row types.Row) (ret string, isNull bool, err error) { @@ -332,6 +356,12 @@ type builtinCaseWhenTimeSig struct { baseBuiltinFunc } +func (b *builtinCaseWhenTimeSig) Clone() builtinFunc { + newSig := &builtinCaseWhenTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinCaseWhenTimeSig. // See https://dev.mysql.com/doc/refman/5.7/en/case.html func (b *builtinCaseWhenTimeSig) evalTime(row types.Row) (ret types.Time, isNull bool, err error) { @@ -362,6 +392,12 @@ type builtinCaseWhenDurationSig struct { baseBuiltinFunc } +func (b *builtinCaseWhenDurationSig) Clone() builtinFunc { + newSig := &builtinCaseWhenDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinCaseWhenDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/case.html func (b *builtinCaseWhenDurationSig) evalDuration(row types.Row) (ret types.Duration, isNull bool, err error) { @@ -432,6 +468,12 @@ type builtinIfIntSig struct { baseBuiltinFunc } +func (b *builtinIfIntSig) Clone() builtinFunc { + newSig := &builtinIfIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfIntSig) evalInt(row types.Row) (ret int64, isNull bool, err error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -449,6 +491,12 @@ type builtinIfRealSig struct { baseBuiltinFunc } +func (b *builtinIfRealSig) Clone() builtinFunc { + newSig := &builtinIfRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfRealSig) evalReal(row types.Row) (ret float64, isNull bool, err error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -466,6 +514,12 @@ type builtinIfDecimalSig struct { baseBuiltinFunc } +func (b *builtinIfDecimalSig) Clone() builtinFunc { + newSig := &builtinIfDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfDecimalSig) evalDecimal(row types.Row) (ret *types.MyDecimal, isNull bool, err error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -483,6 +537,12 @@ type builtinIfStringSig struct { baseBuiltinFunc } +func (b *builtinIfStringSig) Clone() builtinFunc { + newSig := &builtinIfStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfStringSig) evalString(row types.Row) (ret string, isNull bool, err error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -500,6 +560,12 @@ type builtinIfTimeSig struct { baseBuiltinFunc } +func (b *builtinIfTimeSig) Clone() builtinFunc { + newSig := &builtinIfTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfTimeSig) evalTime(row types.Row) (ret types.Time, isNull bool, err error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -517,6 +583,12 @@ type builtinIfDurationSig struct { baseBuiltinFunc } +func (b *builtinIfDurationSig) Clone() builtinFunc { + newSig := &builtinIfDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfDurationSig) evalDuration(row types.Row) (ret types.Duration, isNull bool, err error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -534,6 +606,12 @@ type builtinIfJSONSig struct { baseBuiltinFunc } +func (b *builtinIfJSONSig) Clone() builtinFunc { + newSig := &builtinIfJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfJSONSig) evalJSON(row types.Row) (ret json.BinaryJSON, isNull bool, err error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -605,6 +683,12 @@ type builtinIfNullIntSig struct { baseBuiltinFunc } +func (b *builtinIfNullIntSig) Clone() builtinFunc { + newSig := &builtinIfNullIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfNullIntSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull, err := b.args[0].EvalInt(b.ctx, row) if !isNull || err != nil { @@ -618,6 +702,12 @@ type builtinIfNullRealSig struct { baseBuiltinFunc } +func (b *builtinIfNullRealSig) Clone() builtinFunc { + newSig := &builtinIfNullRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfNullRealSig) evalReal(row types.Row) (float64, bool, error) { arg0, isNull, err := b.args[0].EvalReal(b.ctx, row) if !isNull || err != nil { @@ -631,6 +721,12 @@ type builtinIfNullDecimalSig struct { baseBuiltinFunc } +func (b *builtinIfNullDecimalSig) Clone() builtinFunc { + newSig := &builtinIfNullDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfNullDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { arg0, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if !isNull || err != nil { @@ -644,6 +740,12 @@ type builtinIfNullStringSig struct { baseBuiltinFunc } +func (b *builtinIfNullStringSig) Clone() builtinFunc { + newSig := &builtinIfNullStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfNullStringSig) evalString(row types.Row) (string, bool, error) { arg0, isNull, err := b.args[0].EvalString(b.ctx, row) if !isNull || err != nil { @@ -657,6 +759,12 @@ type builtinIfNullTimeSig struct { baseBuiltinFunc } +func (b *builtinIfNullTimeSig) Clone() builtinFunc { + newSig := &builtinIfNullTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfNullTimeSig) evalTime(row types.Row) (types.Time, bool, error) { arg0, isNull, err := b.args[0].EvalTime(b.ctx, row) if !isNull || err != nil { @@ -670,6 +778,12 @@ type builtinIfNullDurationSig struct { baseBuiltinFunc } +func (b *builtinIfNullDurationSig) Clone() builtinFunc { + newSig := &builtinIfNullDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfNullDurationSig) evalDuration(row types.Row) (types.Duration, bool, error) { arg0, isNull, err := b.args[0].EvalDuration(b.ctx, row) if !isNull || err != nil { @@ -683,6 +797,12 @@ type builtinIfNullJSONSig struct { baseBuiltinFunc } +func (b *builtinIfNullJSONSig) Clone() builtinFunc { + newSig := &builtinIfNullJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIfNullJSONSig) evalJSON(row types.Row) (json.BinaryJSON, bool, error) { arg0, isNull, err := b.args[0].EvalJSON(b.ctx, row) if !isNull { diff --git a/expression/builtin_encryption.go b/expression/builtin_encryption.go index 8b47d5ad0dae2..bfce4fbbe52a1 100644 --- a/expression/builtin_encryption.go +++ b/expression/builtin_encryption.go @@ -91,6 +91,12 @@ type builtinAesDecryptSig struct { baseBuiltinFunc } +func (b *builtinAesDecryptSig) Clone() builtinFunc { + newSig := &builtinAesDecryptSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals AES_DECRYPT(crypt_str, key_key). // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_aes-decrypt func (b *builtinAesDecryptSig) evalString(row types.Row) (string, bool, error) { @@ -133,6 +139,12 @@ type builtinAesEncryptSig struct { baseBuiltinFunc } +func (b *builtinAesEncryptSig) Clone() builtinFunc { + newSig := &builtinAesEncryptSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals AES_ENCRYPT(str, key_str). // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_aes-decrypt func (b *builtinAesEncryptSig) evalString(row types.Row) (string, bool, error) { @@ -222,6 +234,12 @@ type builtinPasswordSig struct { baseBuiltinFunc } +func (b *builtinPasswordSig) Clone() builtinFunc { + newSig := &builtinPasswordSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinPasswordSig. // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_password func (b *builtinPasswordSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -260,6 +278,12 @@ type builtinRandomBytesSig struct { baseBuiltinFunc } +func (b *builtinRandomBytesSig) Clone() builtinFunc { + newSig := &builtinRandomBytesSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals RANDOM_BYTES(len). // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_random-bytes func (b *builtinRandomBytesSig) evalString(row types.Row) (string, bool, error) { @@ -297,6 +321,12 @@ type builtinMD5Sig struct { baseBuiltinFunc } +func (b *builtinMD5Sig) Clone() builtinFunc { + newSig := &builtinMD5Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinMD5Sig. // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_md5 func (b *builtinMD5Sig) evalString(row types.Row) (string, bool, error) { @@ -327,6 +357,12 @@ type builtinSHA1Sig struct { baseBuiltinFunc } +func (b *builtinSHA1Sig) Clone() builtinFunc { + newSig := &builtinSHA1Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals SHA1(str). // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_sha1 // The value is returned as a string of 40 hexadecimal digits, or NULL if the argument was NULL. @@ -361,6 +397,12 @@ type builtinSHA2Sig struct { baseBuiltinFunc } +func (b *builtinSHA2Sig) Clone() builtinFunc { + newSig := &builtinSHA2Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // Supported hash length of SHA-2 family const ( SHA0 = 0 @@ -455,6 +497,12 @@ type builtinCompressSig struct { baseBuiltinFunc } +func (b *builtinCompressSig) Clone() builtinFunc { + newSig := &builtinCompressSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals COMPRESS(str). // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_compress func (b *builtinCompressSig) evalString(row types.Row) (string, bool, error) { @@ -511,6 +559,12 @@ type builtinUncompressSig struct { baseBuiltinFunc } +func (b *builtinUncompressSig) Clone() builtinFunc { + newSig := &builtinUncompressSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals UNCOMPRESS(compressed_string). // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_uncompress func (b *builtinUncompressSig) evalString(row types.Row) (string, bool, error) { @@ -553,6 +607,12 @@ type builtinUncompressedLengthSig struct { baseBuiltinFunc } +func (b *builtinUncompressedLengthSig) Clone() builtinFunc { + newSig := &builtinUncompressedLengthSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals UNCOMPRESSED_LENGTH(str). // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_uncompressed-length func (b *builtinUncompressedLengthSig) evalInt(row types.Row) (int64, bool, error) { diff --git a/expression/builtin_info.go b/expression/builtin_info.go index 6da8c16c59ce4..e9436c4432914 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -72,6 +72,12 @@ type builtinDatabaseSig struct { baseBuiltinFunc } +func (b *builtinDatabaseSig) Clone() builtinFunc { + newSig := &builtinDatabaseSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinDatabaseSig. // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html func (b *builtinDatabaseSig) evalString(row types.Row) (string, bool, error) { @@ -97,6 +103,12 @@ type builtinFoundRowsSig struct { baseBuiltinFunc } +func (b *builtinFoundRowsSig) Clone() builtinFunc { + newSig := &builtinFoundRowsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinFoundRowsSig. // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_found-rows // TODO: SQL_CALC_FOUND_ROWS and LIMIT not support for now, We will finish in another PR. @@ -126,6 +138,12 @@ type builtinCurrentUserSig struct { baseBuiltinFunc } +func (b *builtinCurrentUserSig) Clone() builtinFunc { + newSig := &builtinCurrentUserSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinCurrentUserSig. // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_current-user // TODO: The value of CURRENT_USER() can differ from the value of USER(). We will finish this after we support grant tables. @@ -156,6 +174,12 @@ type builtinUserSig struct { baseBuiltinFunc } +func (b *builtinUserSig) Clone() builtinFunc { + newSig := &builtinUserSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // eval evals a builtinUserSig. // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_user func (b *builtinUserSig) evalString(row types.Row) (string, bool, error) { @@ -185,6 +209,12 @@ type builtinConnectionIDSig struct { baseBuiltinFunc } +func (b *builtinConnectionIDSig) Clone() builtinFunc { + newSig := &builtinConnectionIDSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinConnectionIDSig) evalInt(_ types.Row) (int64, bool, error) { data := b.ctx.GetSessionVars() if data == nil { @@ -221,6 +251,12 @@ type builtinLastInsertIDSig struct { baseBuiltinFunc } +func (b *builtinLastInsertIDSig) Clone() builtinFunc { + newSig := &builtinLastInsertIDSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals LAST_INSERT_ID(). // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_last-insert-id. func (b *builtinLastInsertIDSig) evalInt(row types.Row) (res int64, isNull bool, err error) { @@ -232,6 +268,12 @@ type builtinLastInsertIDWithIDSig struct { baseBuiltinFunc } +func (b *builtinLastInsertIDWithIDSig) Clone() builtinFunc { + newSig := &builtinLastInsertIDWithIDSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals LAST_INSERT_ID(expr). // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_last-insert-id. func (b *builtinLastInsertIDWithIDSig) evalInt(row types.Row) (res int64, isNull bool, err error) { @@ -262,6 +304,12 @@ type builtinVersionSig struct { baseBuiltinFunc } +func (b *builtinVersionSig) Clone() builtinFunc { + newSig := &builtinVersionSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinVersionSig. // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_version func (b *builtinVersionSig) evalString(row types.Row) (string, bool, error) { @@ -286,6 +334,12 @@ type builtinTiDBVersionSig struct { baseBuiltinFunc } +func (b *builtinTiDBVersionSig) Clone() builtinFunc { + newSig := &builtinTiDBVersionSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinTiDBVersionSig. // This will show git hash and build time for tidb-server. func (b *builtinTiDBVersionSig) evalString(_ types.Row) (string, bool, error) { @@ -341,6 +395,12 @@ type builtinRowCountSig struct { baseBuiltinFunc } +func (b *builtinRowCountSig) Clone() builtinFunc { + newSig := &builtinRowCountSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals ROW_COUNT(). // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_row-count. func (b *builtinRowCountSig) evalInt(_ types.Row) (res int64, isNull bool, err error) { diff --git a/expression/builtin_json.go b/expression/builtin_json.go index 29de8747aaa2f..f5d0085725032 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -81,6 +81,12 @@ type builtinJSONTypeSig struct { baseBuiltinFunc } +func (b *builtinJSONTypeSig) Clone() builtinFunc { + newSig := &builtinJSONTypeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonTypeFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -109,6 +115,12 @@ type builtinJSONExtractSig struct { baseBuiltinFunc } +func (b *builtinJSONExtractSig) Clone() builtinFunc { + newSig := &builtinJSONExtractSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonExtractFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -157,6 +169,12 @@ type builtinJSONUnquoteSig struct { baseBuiltinFunc } +func (b *builtinJSONUnquoteSig) Clone() builtinFunc { + newSig := &builtinJSONUnquoteSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonUnquoteFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -186,6 +204,12 @@ type builtinJSONSetSig struct { baseBuiltinFunc } +func (b *builtinJSONSetSig) Clone() builtinFunc { + newSig := &builtinJSONSetSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonSetFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -220,6 +244,12 @@ type builtinJSONInsertSig struct { baseBuiltinFunc } +func (b *builtinJSONInsertSig) Clone() builtinFunc { + newSig := &builtinJSONInsertSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonInsertFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -254,6 +284,12 @@ type builtinJSONReplaceSig struct { baseBuiltinFunc } +func (b *builtinJSONReplaceSig) Clone() builtinFunc { + newSig := &builtinJSONReplaceSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonReplaceFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -288,6 +324,12 @@ type builtinJSONRemoveSig struct { baseBuiltinFunc } +func (b *builtinJSONRemoveSig) Clone() builtinFunc { + newSig := &builtinJSONRemoveSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonRemoveFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -337,6 +379,12 @@ type builtinJSONMergeSig struct { baseBuiltinFunc } +func (b *builtinJSONMergeSig) Clone() builtinFunc { + newSig := &builtinJSONMergeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonMergeFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -373,6 +421,12 @@ type builtinJSONObjectSig struct { baseBuiltinFunc } +func (b *builtinJSONObjectSig) Clone() builtinFunc { + newSig := &builtinJSONObjectSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonObjectFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) @@ -433,6 +487,12 @@ type builtinJSONArraySig struct { baseBuiltinFunc } +func (b *builtinJSONArraySig) Clone() builtinFunc { + newSig := &builtinJSONArraySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonArrayFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, errors.Trace(err) diff --git a/expression/builtin_like.go b/expression/builtin_like.go index af90f9c75e2b8..806eeeeb249cd 100644 --- a/expression/builtin_like.go +++ b/expression/builtin_like.go @@ -54,6 +54,12 @@ type builtinLikeSig struct { baseBuiltinFunc } +func (b *builtinLikeSig) Clone() builtinFunc { + newSig := &builtinLikeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinLikeSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-comparison-functions.html#operator_like // NOTE: Currently tikv's like function is case sensitive, so we keep its behavior here. @@ -101,6 +107,12 @@ type builtinRegexpBinarySig struct { baseBuiltinFunc } +func (b *builtinRegexpBinarySig) Clone() builtinFunc { + newSig := &builtinRegexpBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinRegexpBinarySig) evalInt(row types.Row) (int64, bool, error) { expr, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { @@ -124,6 +136,12 @@ type builtinRegexpSig struct { baseBuiltinFunc } +func (b *builtinRegexpSig) Clone() builtinFunc { + newSig := &builtinRegexpSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals `expr REGEXP pat`, or `expr RLIKE pat`. // See https://dev.mysql.com/doc/refman/5.7/en/regexp.html#operator_regexp func (b *builtinRegexpSig) evalInt(row types.Row) (int64, bool, error) { diff --git a/expression/builtin_math.go b/expression/builtin_math.go index 215271eaed97a..88f063a64d795 100644 --- a/expression/builtin_math.go +++ b/expression/builtin_math.go @@ -160,6 +160,12 @@ type builtinAbsRealSig struct { baseBuiltinFunc } +func (b *builtinAbsRealSig) Clone() builtinFunc { + newSig := &builtinAbsRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals ABS(value). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_abs func (b *builtinAbsRealSig) evalReal(row types.Row) (float64, bool, error) { @@ -174,6 +180,12 @@ type builtinAbsIntSig struct { baseBuiltinFunc } +func (b *builtinAbsIntSig) Clone() builtinFunc { + newSig := &builtinAbsIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals ABS(value). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_abs func (b *builtinAbsIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -194,6 +206,12 @@ type builtinAbsUIntSig struct { baseBuiltinFunc } +func (b *builtinAbsUIntSig) Clone() builtinFunc { + newSig := &builtinAbsUIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals ABS(value). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_abs func (b *builtinAbsUIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -204,6 +222,12 @@ type builtinAbsDecSig struct { baseBuiltinFunc } +func (b *builtinAbsDecSig) Clone() builtinFunc { + newSig := &builtinAbsDecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals ABS(value). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_abs func (b *builtinAbsDecSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { @@ -272,6 +296,12 @@ type builtinRoundRealSig struct { baseBuiltinFunc } +func (b *builtinRoundRealSig) Clone() builtinFunc { + newSig := &builtinRoundRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals ROUND(value). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_round func (b *builtinRoundRealSig) evalReal(row types.Row) (float64, bool, error) { @@ -286,6 +316,12 @@ type builtinRoundIntSig struct { baseBuiltinFunc } +func (b *builtinRoundIntSig) Clone() builtinFunc { + newSig := &builtinRoundIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals ROUND(value). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_round func (b *builtinRoundIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -296,6 +332,12 @@ type builtinRoundDecSig struct { baseBuiltinFunc } +func (b *builtinRoundDecSig) Clone() builtinFunc { + newSig := &builtinRoundDecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals ROUND(value). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_round func (b *builtinRoundDecSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { @@ -314,6 +356,12 @@ type builtinRoundWithFracRealSig struct { baseBuiltinFunc } +func (b *builtinRoundWithFracRealSig) Clone() builtinFunc { + newSig := &builtinRoundWithFracRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals ROUND(value, frac). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_round func (b *builtinRoundWithFracRealSig) evalReal(row types.Row) (float64, bool, error) { @@ -332,6 +380,12 @@ type builtinRoundWithFracIntSig struct { baseBuiltinFunc } +func (b *builtinRoundWithFracIntSig) Clone() builtinFunc { + newSig := &builtinRoundWithFracIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals ROUND(value, frac). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_round func (b *builtinRoundWithFracIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -350,6 +404,12 @@ type builtinRoundWithFracDecSig struct { baseBuiltinFunc } +func (b *builtinRoundWithFracDecSig) Clone() builtinFunc { + newSig := &builtinRoundWithFracDecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals ROUND(value, frac). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_round func (b *builtinRoundWithFracDecSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { @@ -411,6 +471,12 @@ type builtinCeilRealSig struct { baseBuiltinFunc } +func (b *builtinCeilRealSig) Clone() builtinFunc { + newSig := &builtinCeilRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinCeilRealSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_ceil func (b *builtinCeilRealSig) evalReal(row types.Row) (float64, bool, error) { @@ -425,6 +491,12 @@ type builtinCeilIntToIntSig struct { baseBuiltinFunc } +func (b *builtinCeilIntToIntSig) Clone() builtinFunc { + newSig := &builtinCeilIntToIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinCeilIntToIntSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_ceil func (b *builtinCeilIntToIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -435,6 +507,12 @@ type builtinCeilIntToDecSig struct { baseBuiltinFunc } +func (b *builtinCeilIntToDecSig) Clone() builtinFunc { + newSig := &builtinCeilIntToDecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDec evals a builtinCeilIntToDecSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_Ceil func (b *builtinCeilIntToDecSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { @@ -446,6 +524,12 @@ type builtinCeilDecToIntSig struct { baseBuiltinFunc } +func (b *builtinCeilDecToIntSig) Clone() builtinFunc { + newSig := &builtinCeilDecToIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinCeilDecToIntSig. // Ceil receives func (b *builtinCeilDecToIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -468,6 +552,12 @@ type builtinCeilDecToDecSig struct { baseBuiltinFunc } +func (b *builtinCeilDecToDecSig) Clone() builtinFunc { + newSig := &builtinCeilDecToDecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDec evals a builtinCeilDecToDecSig. func (b *builtinCeilDecToDecSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { val, isNull, err := b.args[0].EvalDecimal(b.ctx, row) @@ -557,6 +647,12 @@ type builtinFloorRealSig struct { baseBuiltinFunc } +func (b *builtinFloorRealSig) Clone() builtinFunc { + newSig := &builtinFloorRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinFloorRealSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_floor func (b *builtinFloorRealSig) evalReal(row types.Row) (float64, bool, error) { @@ -571,6 +667,12 @@ type builtinFloorIntToIntSig struct { baseBuiltinFunc } +func (b *builtinFloorIntToIntSig) Clone() builtinFunc { + newSig := &builtinFloorIntToIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinFloorIntToIntSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_floor func (b *builtinFloorIntToIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -581,6 +683,12 @@ type builtinFloorIntToDecSig struct { baseBuiltinFunc } +func (b *builtinFloorIntToDecSig) Clone() builtinFunc { + newSig := &builtinFloorIntToDecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDec evals a builtinFloorIntToDecSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_floor func (b *builtinFloorIntToDecSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { @@ -592,6 +700,12 @@ type builtinFloorDecToIntSig struct { baseBuiltinFunc } +func (b *builtinFloorDecToIntSig) Clone() builtinFunc { + newSig := &builtinFloorDecToIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinFloorDecToIntSig. // floor receives func (b *builtinFloorDecToIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -614,6 +728,12 @@ type builtinFloorDecToDecSig struct { baseBuiltinFunc } +func (b *builtinFloorDecToDecSig) Clone() builtinFunc { + newSig := &builtinFloorDecToDecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDec evals a builtinFloorDecToDecSig. func (b *builtinFloorDecToDecSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { val, isNull, err := b.args[0].EvalDecimal(b.ctx, row) @@ -667,6 +787,12 @@ type builtinLog1ArgSig struct { baseBuiltinFunc } +func (b *builtinLog1ArgSig) Clone() builtinFunc { + newSig := &builtinLog1ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinLog1ArgSig, corresponding to log(x). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_log func (b *builtinLog1ArgSig) evalReal(row types.Row) (float64, bool, error) { @@ -684,6 +810,12 @@ type builtinLog2ArgsSig struct { baseBuiltinFunc } +func (b *builtinLog2ArgsSig) Clone() builtinFunc { + newSig := &builtinLog2ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinLog2ArgsSig, corresponding to log(b, x). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_log func (b *builtinLog2ArgsSig) evalReal(row types.Row) (float64, bool, error) { @@ -721,6 +853,12 @@ type builtinLog2Sig struct { baseBuiltinFunc } +func (b *builtinLog2Sig) Clone() builtinFunc { + newSig := &builtinLog2Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinLog2Sig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_log2 func (b *builtinLog2Sig) evalReal(row types.Row) (float64, bool, error) { @@ -751,6 +889,12 @@ type builtinLog10Sig struct { baseBuiltinFunc } +func (b *builtinLog10Sig) Clone() builtinFunc { + newSig := &builtinLog10Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinLog10Sig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_log10 func (b *builtinLog10Sig) evalReal(row types.Row) (float64, bool, error) { @@ -792,6 +936,12 @@ type builtinRandSig struct { randGen *rand.Rand } +func (b *builtinRandSig) Clone() builtinFunc { + newSig := &builtinRandSig{randGen: b.randGen} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals RAND(). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_rand func (b *builtinRandSig) evalReal(row types.Row) (float64, bool, error) { @@ -806,6 +956,12 @@ type builtinRandWithSeedSig struct { randGen *rand.Rand } +func (b *builtinRandWithSeedSig) Clone() builtinFunc { + newSig := &builtinRandWithSeedSig{randGen: b.randGen} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals RAND(N). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_rand func (b *builtinRandWithSeedSig) evalReal(row types.Row) (float64, bool, error) { @@ -841,6 +997,12 @@ type builtinPowSig struct { baseBuiltinFunc } +func (b *builtinPowSig) Clone() builtinFunc { + newSig := &builtinPowSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals POW(x, y). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_pow func (b *builtinPowSig) evalReal(row types.Row) (float64, bool, error) { @@ -882,6 +1044,12 @@ type builtinConvSig struct { baseBuiltinFunc } +func (b *builtinConvSig) Clone() builtinFunc { + newSig := &builtinConvSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals CONV(N,from_base,to_base). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_conv. func (b *builtinConvSig) evalString(row types.Row) (res string, isNull bool, err error) { @@ -981,6 +1149,12 @@ type builtinCRC32Sig struct { baseBuiltinFunc } +func (b *builtinCRC32Sig) Clone() builtinFunc { + newSig := &builtinCRC32Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a CRC32(expr). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_crc32 func (b *builtinCRC32Sig) evalInt(row types.Row) (int64, bool, error) { @@ -1009,6 +1183,12 @@ type builtinSignSig struct { baseBuiltinFunc } +func (b *builtinSignSig) Clone() builtinFunc { + newSig := &builtinSignSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals SIGN(v). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_sign func (b *builtinSignSig) evalInt(row types.Row) (int64, bool, error) { @@ -1042,6 +1222,12 @@ type builtinSqrtSig struct { baseBuiltinFunc } +func (b *builtinSqrtSig) Clone() builtinFunc { + newSig := &builtinSqrtSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a SQRT(x). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_sqrt func (b *builtinSqrtSig) evalReal(row types.Row) (float64, bool, error) { @@ -1072,6 +1258,12 @@ type builtinAcosSig struct { baseBuiltinFunc } +func (b *builtinAcosSig) Clone() builtinFunc { + newSig := &builtinAcosSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinAcosSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_acos func (b *builtinAcosSig) evalReal(row types.Row) (float64, bool, error) { @@ -1103,6 +1295,12 @@ type builtinAsinSig struct { baseBuiltinFunc } +func (b *builtinAsinSig) Clone() builtinFunc { + newSig := &builtinAsinSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinAsinSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_asin func (b *builtinAsinSig) evalReal(row types.Row) (float64, bool, error) { @@ -1151,6 +1349,12 @@ type builtinAtan1ArgSig struct { baseBuiltinFunc } +func (b *builtinAtan1ArgSig) Clone() builtinFunc { + newSig := &builtinAtan1ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinAtan1ArgSig, corresponding to atan(x). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_atan func (b *builtinAtan1ArgSig) evalReal(row types.Row) (float64, bool, error) { @@ -1166,6 +1370,12 @@ type builtinAtan2ArgsSig struct { baseBuiltinFunc } +func (b *builtinAtan2ArgsSig) Clone() builtinFunc { + newSig := &builtinAtan2ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinAtan1ArgSig, corresponding to atan(y, x). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_atan func (b *builtinAtan2ArgsSig) evalReal(row types.Row) (float64, bool, error) { @@ -1199,6 +1409,12 @@ type builtinCosSig struct { baseBuiltinFunc } +func (b *builtinCosSig) Clone() builtinFunc { + newSig := &builtinCosSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinCosSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_cos func (b *builtinCosSig) evalReal(row types.Row) (float64, bool, error) { @@ -1226,6 +1442,12 @@ type builtinCotSig struct { baseBuiltinFunc } +func (b *builtinCotSig) Clone() builtinFunc { + newSig := &builtinCotSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinCotSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_cot func (b *builtinCotSig) evalReal(row types.Row) (float64, bool, error) { @@ -1261,6 +1483,12 @@ type builtinDegreesSig struct { baseBuiltinFunc } +func (b *builtinDegreesSig) Clone() builtinFunc { + newSig := &builtinDegreesSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinDegreesSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_degrees func (b *builtinDegreesSig) evalReal(row types.Row) (float64, bool, error) { @@ -1289,6 +1517,12 @@ type builtinExpSig struct { baseBuiltinFunc } +func (b *builtinExpSig) Clone() builtinFunc { + newSig := &builtinExpSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinExpSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_exp func (b *builtinExpSig) evalReal(row types.Row) (float64, bool, error) { @@ -1328,6 +1562,12 @@ type builtinPISig struct { baseBuiltinFunc } +func (b *builtinPISig) Clone() builtinFunc { + newSig := &builtinPISig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinPISig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_pi func (b *builtinPISig) evalReal(_ types.Row) (float64, bool, error) { @@ -1351,6 +1591,12 @@ type builtinRadiansSig struct { baseBuiltinFunc } +func (b *builtinRadiansSig) Clone() builtinFunc { + newSig := &builtinRadiansSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals RADIANS(X). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_radians func (b *builtinRadiansSig) evalReal(row types.Row) (float64, bool, error) { @@ -1378,6 +1624,12 @@ type builtinSinSig struct { baseBuiltinFunc } +func (b *builtinSinSig) Clone() builtinFunc { + newSig := &builtinSinSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalreal evals a builtinSinSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_sin func (b *builtinSinSig) evalReal(row types.Row) (float64, bool, error) { @@ -1405,6 +1657,12 @@ type builtinTanSig struct { baseBuiltinFunc } +func (b *builtinTanSig) Clone() builtinFunc { + newSig := &builtinTanSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // eval evals a builtinTanSig. // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_tan func (b *builtinTanSig) evalReal(row types.Row) (float64, bool, error) { @@ -1472,6 +1730,12 @@ type builtinTruncateDecimalSig struct { baseBuiltinFunc } +func (b *builtinTruncateDecimalSig) Clone() builtinFunc { + newSig := &builtinTruncateDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals a TRUNCATE(X,D). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_truncate func (b *builtinTruncateDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { @@ -1496,6 +1760,12 @@ type builtinTruncateRealSig struct { baseBuiltinFunc } +func (b *builtinTruncateRealSig) Clone() builtinFunc { + newSig := &builtinTruncateRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a TRUNCATE(X,D). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_truncate func (b *builtinTruncateRealSig) evalReal(row types.Row) (float64, bool, error) { @@ -1516,6 +1786,12 @@ type builtinTruncateIntSig struct { baseBuiltinFunc } +func (b *builtinTruncateIntSig) Clone() builtinFunc { + newSig := &builtinTruncateIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a TRUNCATE(X,D). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_truncate func (b *builtinTruncateIntSig) evalInt(row types.Row) (int64, bool, error) { diff --git a/expression/builtin_miscellaneous.go b/expression/builtin_miscellaneous.go index 07a7aeca794e9..a0c01ded2bd51 100644 --- a/expression/builtin_miscellaneous.go +++ b/expression/builtin_miscellaneous.go @@ -93,6 +93,12 @@ type builtinSleepSig struct { baseBuiltinFunc } +func (b *builtinSleepSig) Clone() builtinFunc { + newSig := &builtinSleepSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinSleepSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_sleep func (b *builtinSleepSig) evalInt(row types.Row) (int64, bool, error) { @@ -145,6 +151,12 @@ type builtinLockSig struct { baseBuiltinFunc } +func (b *builtinLockSig) Clone() builtinFunc { + newSig := &builtinLockSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinLockSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_get-lock // The lock function will do nothing. @@ -171,6 +183,12 @@ type builtinReleaseLockSig struct { baseBuiltinFunc } +func (b *builtinReleaseLockSig) Clone() builtinFunc { + newSig := &builtinReleaseLockSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinReleaseLockSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_release-lock // The release lock function will do nothing. @@ -220,6 +238,12 @@ type builtinDecimalAnyValueSig struct { baseBuiltinFunc } +func (b *builtinDecimalAnyValueSig) Clone() builtinFunc { + newSig := &builtinDecimalAnyValueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals a builtinDecimalAnyValueSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_any-value func (b *builtinDecimalAnyValueSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { @@ -230,6 +254,12 @@ type builtinDurationAnyValueSig struct { baseBuiltinFunc } +func (b *builtinDurationAnyValueSig) Clone() builtinFunc { + newSig := &builtinDurationAnyValueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinDurationAnyValueSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_any-value func (b *builtinDurationAnyValueSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -240,6 +270,12 @@ type builtinIntAnyValueSig struct { baseBuiltinFunc } +func (b *builtinIntAnyValueSig) Clone() builtinFunc { + newSig := &builtinIntAnyValueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinIntAnyValueSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_any-value func (b *builtinIntAnyValueSig) evalInt(row types.Row) (int64, bool, error) { @@ -250,6 +286,12 @@ type builtinJSONAnyValueSig struct { baseBuiltinFunc } +func (b *builtinJSONAnyValueSig) Clone() builtinFunc { + newSig := &builtinJSONAnyValueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalJSON evals a builtinJSONAnyValueSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_any-value func (b *builtinJSONAnyValueSig) evalJSON(row types.Row) (json.BinaryJSON, bool, error) { @@ -260,6 +302,12 @@ type builtinRealAnyValueSig struct { baseBuiltinFunc } +func (b *builtinRealAnyValueSig) Clone() builtinFunc { + newSig := &builtinRealAnyValueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinRealAnyValueSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_any-value func (b *builtinRealAnyValueSig) evalReal(row types.Row) (float64, bool, error) { @@ -270,6 +318,12 @@ type builtinStringAnyValueSig struct { baseBuiltinFunc } +func (b *builtinStringAnyValueSig) Clone() builtinFunc { + newSig := &builtinStringAnyValueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinStringAnyValueSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_any-value func (b *builtinStringAnyValueSig) evalString(row types.Row) (string, bool, error) { @@ -280,6 +334,12 @@ type builtinTimeAnyValueSig struct { baseBuiltinFunc } +func (b *builtinTimeAnyValueSig) Clone() builtinFunc { + newSig := &builtinTimeAnyValueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinTimeAnyValueSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_any-value func (b *builtinTimeAnyValueSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -313,6 +373,12 @@ type builtinInetAtonSig struct { baseBuiltinFunc } +func (b *builtinInetAtonSig) Clone() builtinFunc { + newSig := &builtinInetAtonSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinInetAtonSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_inet-aton func (b *builtinInetAtonSig) evalInt(row types.Row) (int64, bool, error) { @@ -380,6 +446,12 @@ type builtinInetNtoaSig struct { baseBuiltinFunc } +func (b *builtinInetNtoaSig) Clone() builtinFunc { + newSig := &builtinInetNtoaSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinInetNtoaSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_inet-ntoa func (b *builtinInetNtoaSig) evalString(row types.Row) (string, bool, error) { @@ -423,6 +495,12 @@ type builtinInet6AtonSig struct { baseBuiltinFunc } +func (b *builtinInet6AtonSig) Clone() builtinFunc { + newSig := &builtinInet6AtonSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinInet6AtonSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_inet6-aton func (b *builtinInet6AtonSig) evalString(row types.Row) (string, bool, error) { @@ -485,6 +563,12 @@ type builtinInet6NtoaSig struct { baseBuiltinFunc } +func (b *builtinInet6NtoaSig) Clone() builtinFunc { + newSig := &builtinInet6NtoaSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinInet6NtoaSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_inet6-ntoa func (b *builtinInet6NtoaSig) evalString(row types.Row) (string, bool, error) { @@ -530,6 +614,12 @@ type builtinIsIPv4Sig struct { baseBuiltinFunc } +func (b *builtinIsIPv4Sig) Clone() builtinFunc { + newSig := &builtinIsIPv4Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinIsIPv4Sig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_is-ipv4 func (b *builtinIsIPv4Sig) evalInt(row types.Row) (int64, bool, error) { @@ -588,6 +678,12 @@ type builtinIsIPv4CompatSig struct { baseBuiltinFunc } +func (b *builtinIsIPv4CompatSig) Clone() builtinFunc { + newSig := &builtinIsIPv4CompatSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals Is_IPv4_Compat // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_is-ipv4-compat func (b *builtinIsIPv4CompatSig) evalInt(row types.Row) (int64, bool, error) { @@ -627,6 +723,12 @@ type builtinIsIPv4MappedSig struct { baseBuiltinFunc } +func (b *builtinIsIPv4MappedSig) Clone() builtinFunc { + newSig := &builtinIsIPv4MappedSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals Is_IPv4_Mapped // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_is-ipv4-mapped func (b *builtinIsIPv4MappedSig) evalInt(row types.Row) (int64, bool, error) { @@ -666,6 +768,12 @@ type builtinIsIPv6Sig struct { baseBuiltinFunc } +func (b *builtinIsIPv6Sig) Clone() builtinFunc { + newSig := &builtinIsIPv6Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinIsIPv6Sig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_is-ipv6 func (b *builtinIsIPv6Sig) evalInt(row types.Row) (int64, bool, error) { @@ -730,6 +838,12 @@ type builtinUUIDSig struct { baseBuiltinFunc } +func (b *builtinUUIDSig) Clone() builtinFunc { + newSig := &builtinUUIDSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinUUIDSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_uuid func (b *builtinUUIDSig) evalString(_ types.Row) (d string, isNull bool, err error) { diff --git a/expression/builtin_op.go b/expression/builtin_op.go index 29f9d76fb75d4..0e162c9326eff 100644 --- a/expression/builtin_op.go +++ b/expression/builtin_op.go @@ -75,6 +75,12 @@ type builtinLogicAndSig struct { baseBuiltinFunc } +func (b *builtinLogicAndSig) Clone() builtinFunc { + newSig := &builtinLogicAndSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinLogicAndSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil || (!isNull0 && arg0 == 0) { @@ -110,6 +116,12 @@ type builtinLogicOrSig struct { baseBuiltinFunc } +func (b *builtinLogicOrSig) Clone() builtinFunc { + newSig := &builtinLogicOrSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinLogicOrSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -151,6 +163,12 @@ type builtinLogicXorSig struct { baseBuiltinFunc } +func (b *builtinLogicXorSig) Clone() builtinFunc { + newSig := &builtinLogicXorSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinLogicXorSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -186,6 +204,12 @@ type builtinBitAndSig struct { baseBuiltinFunc } +func (b *builtinBitAndSig) Clone() builtinFunc { + newSig := &builtinBitAndSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinBitAndSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -218,6 +242,12 @@ type builtinBitOrSig struct { baseBuiltinFunc } +func (b *builtinBitOrSig) Clone() builtinFunc { + newSig := &builtinBitOrSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinBitOrSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -250,6 +280,12 @@ type builtinBitXorSig struct { baseBuiltinFunc } +func (b *builtinBitXorSig) Clone() builtinFunc { + newSig := &builtinBitXorSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinBitXorSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -281,6 +317,12 @@ type builtinLeftShiftSig struct { baseBuiltinFunc } +func (b *builtinLeftShiftSig) Clone() builtinFunc { + newSig := &builtinLeftShiftSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinLeftShiftSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -312,6 +354,12 @@ type builtinRightShiftSig struct { baseBuiltinFunc } +func (b *builtinRightShiftSig) Clone() builtinFunc { + newSig := &builtinRightShiftSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinRightShiftSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -376,6 +424,12 @@ type builtinRealIsTrueSig struct { baseBuiltinFunc } +func (b *builtinRealIsTrueSig) Clone() builtinFunc { + newSig := &builtinRealIsTrueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinRealIsTrueSig) evalInt(row types.Row) (int64, bool, error) { input, isNull, err := b.args[0].EvalReal(b.ctx, row) if err != nil { @@ -391,6 +445,12 @@ type builtinDecimalIsTrueSig struct { baseBuiltinFunc } +func (b *builtinDecimalIsTrueSig) Clone() builtinFunc { + newSig := &builtinDecimalIsTrueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinDecimalIsTrueSig) evalInt(row types.Row) (int64, bool, error) { input, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if err != nil { @@ -406,6 +466,12 @@ type builtinIntIsTrueSig struct { baseBuiltinFunc } +func (b *builtinIntIsTrueSig) Clone() builtinFunc { + newSig := &builtinIntIsTrueSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIntIsTrueSig) evalInt(row types.Row) (int64, bool, error) { input, isNull, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -421,6 +487,12 @@ type builtinRealIsFalseSig struct { baseBuiltinFunc } +func (b *builtinRealIsFalseSig) Clone() builtinFunc { + newSig := &builtinRealIsFalseSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinRealIsFalseSig) evalInt(row types.Row) (int64, bool, error) { input, isNull, err := b.args[0].EvalReal(b.ctx, row) if err != nil { @@ -436,6 +508,12 @@ type builtinDecimalIsFalseSig struct { baseBuiltinFunc } +func (b *builtinDecimalIsFalseSig) Clone() builtinFunc { + newSig := &builtinDecimalIsFalseSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinDecimalIsFalseSig) evalInt(row types.Row) (int64, bool, error) { input, isNull, err := b.args[0].EvalDecimal(b.ctx, row) if err != nil { @@ -451,6 +529,12 @@ type builtinIntIsFalseSig struct { baseBuiltinFunc } +func (b *builtinIntIsFalseSig) Clone() builtinFunc { + newSig := &builtinIntIsFalseSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIntIsFalseSig) evalInt(row types.Row) (int64, bool, error) { input, isNull, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -481,6 +565,12 @@ type builtinBitNegSig struct { baseBuiltinFunc } +func (b *builtinBitNegSig) Clone() builtinFunc { + newSig := &builtinBitNegSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinBitNegSig) evalInt(row types.Row) (int64, bool, error) { arg, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -510,6 +600,12 @@ type builtinUnaryNotSig struct { baseBuiltinFunc } +func (b *builtinUnaryNotSig) Clone() builtinFunc { + newSig := &builtinUnaryNotSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinUnaryNotSig) evalInt(row types.Row) (int64, bool, error) { arg, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { @@ -614,6 +710,12 @@ type builtinUnaryMinusIntSig struct { baseBuiltinFunc } +func (b *builtinUnaryMinusIntSig) Clone() builtinFunc { + newSig := &builtinUnaryMinusIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinUnaryMinusIntSig) evalInt(row types.Row) (res int64, isNull bool, err error) { var val int64 val, isNull, err = b.args[0].EvalInt(b.ctx, row) @@ -640,6 +742,12 @@ type builtinUnaryMinusDecimalSig struct { constantArgOverflow bool } +func (b *builtinUnaryMinusDecimalSig) Clone() builtinFunc { + newSig := &builtinUnaryMinusDecimalSig{constantArgOverflow: b.constantArgOverflow} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinUnaryMinusDecimalSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { var dec *types.MyDecimal dec, isNull, err := b.args[0].EvalDecimal(b.ctx, row) @@ -656,6 +764,12 @@ type builtinUnaryMinusRealSig struct { baseBuiltinFunc } +func (b *builtinUnaryMinusRealSig) Clone() builtinFunc { + newSig := &builtinUnaryMinusRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinUnaryMinusRealSig) evalReal(row types.Row) (float64, bool, error) { val, isNull, err := b.args[0].EvalReal(b.ctx, row) return -val, isNull, errors.Trace(err) @@ -707,6 +821,12 @@ type builtinDecimalIsNullSig struct { baseBuiltinFunc } +func (b *builtinDecimalIsNullSig) Clone() builtinFunc { + newSig := &builtinDecimalIsNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func evalIsNull(isNull bool, err error) (int64, bool, error) { if err != nil { return 0, true, errors.Trace(err) @@ -726,6 +846,12 @@ type builtinDurationIsNullSig struct { baseBuiltinFunc } +func (b *builtinDurationIsNullSig) Clone() builtinFunc { + newSig := &builtinDurationIsNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinDurationIsNullSig) evalInt(row types.Row) (int64, bool, error) { _, isNull, err := b.args[0].EvalDuration(b.ctx, row) return evalIsNull(isNull, err) @@ -735,6 +861,12 @@ type builtinIntIsNullSig struct { baseBuiltinFunc } +func (b *builtinIntIsNullSig) Clone() builtinFunc { + newSig := &builtinIntIsNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinIntIsNullSig) evalInt(row types.Row) (int64, bool, error) { _, isNull, err := b.args[0].EvalInt(b.ctx, row) return evalIsNull(isNull, err) @@ -744,6 +876,12 @@ type builtinRealIsNullSig struct { baseBuiltinFunc } +func (b *builtinRealIsNullSig) Clone() builtinFunc { + newSig := &builtinRealIsNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinRealIsNullSig) evalInt(row types.Row) (int64, bool, error) { _, isNull, err := b.args[0].EvalReal(b.ctx, row) return evalIsNull(isNull, err) @@ -753,6 +891,12 @@ type builtinStringIsNullSig struct { baseBuiltinFunc } +func (b *builtinStringIsNullSig) Clone() builtinFunc { + newSig := &builtinStringIsNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinStringIsNullSig) evalInt(row types.Row) (int64, bool, error) { _, isNull, err := b.args[0].EvalString(b.ctx, row) return evalIsNull(isNull, err) @@ -762,6 +906,12 @@ type builtinTimeIsNullSig struct { baseBuiltinFunc } +func (b *builtinTimeIsNullSig) Clone() builtinFunc { + newSig := &builtinTimeIsNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinTimeIsNullSig) evalInt(row types.Row) (int64, bool, error) { _, isNull, err := b.args[0].EvalTime(b.ctx, row) return evalIsNull(isNull, err) diff --git a/expression/builtin_other.go b/expression/builtin_other.go index 8d4dfde7450d0..f86363dd7715a 100644 --- a/expression/builtin_other.go +++ b/expression/builtin_other.go @@ -106,6 +106,12 @@ type builtinInIntSig struct { baseBuiltinFunc } +func (b *builtinInIntSig) Clone() builtinFunc { + newSig := &builtinInIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinInIntSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalInt(b.ctx, row) if isNull0 || err != nil { @@ -149,6 +155,12 @@ type builtinInStringSig struct { baseBuiltinFunc } +func (b *builtinInStringSig) Clone() builtinFunc { + newSig := &builtinInStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinInStringSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalString(b.ctx, row) if isNull0 || err != nil { @@ -176,6 +188,12 @@ type builtinInRealSig struct { baseBuiltinFunc } +func (b *builtinInRealSig) Clone() builtinFunc { + newSig := &builtinInRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinInRealSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalReal(b.ctx, row) if isNull0 || err != nil { @@ -203,6 +221,12 @@ type builtinInDecimalSig struct { baseBuiltinFunc } +func (b *builtinInDecimalSig) Clone() builtinFunc { + newSig := &builtinInDecimalSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinInDecimalSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalDecimal(b.ctx, row) if isNull0 || err != nil { @@ -230,6 +254,12 @@ type builtinInTimeSig struct { baseBuiltinFunc } +func (b *builtinInTimeSig) Clone() builtinFunc { + newSig := &builtinInTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinInTimeSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalTime(b.ctx, row) if isNull0 || err != nil { @@ -257,6 +287,12 @@ type builtinInDurationSig struct { baseBuiltinFunc } +func (b *builtinInDurationSig) Clone() builtinFunc { + newSig := &builtinInDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinInDurationSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalDuration(b.ctx, row) if isNull0 || err != nil { @@ -284,6 +320,12 @@ type builtinInJSONSig struct { baseBuiltinFunc } +func (b *builtinInJSONSig) Clone() builtinFunc { + newSig := &builtinInJSONSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinInJSONSig) evalInt(row types.Row) (int64, bool, error) { arg0, isNull0, err := b.args[0].EvalJSON(b.ctx, row) if isNull0 || err != nil { @@ -328,6 +370,12 @@ type builtinRowSig struct { baseBuiltinFunc } +func (b *builtinRowSig) Clone() builtinFunc { + newSig := &builtinRowSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString rowFunc should always be flattened in expression rewrite phrase. func (b *builtinRowSig) evalString(row types.Row) (string, bool, error) { panic("builtinRowSig.evalString() should never be called.") @@ -352,6 +400,12 @@ type builtinSetVarSig struct { baseBuiltinFunc } +func (b *builtinSetVarSig) Clone() builtinFunc { + newSig := &builtinSetVarSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinSetVarSig) evalString(row types.Row) (res string, isNull bool, err error) { var varName string sessionVars := b.ctx.GetSessionVars() @@ -389,6 +443,12 @@ type builtinGetVarSig struct { baseBuiltinFunc } +func (b *builtinGetVarSig) Clone() builtinFunc { + newSig := &builtinGetVarSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinGetVarSig) evalString(row types.Row) (string, bool, error) { sessionVars := b.ctx.GetSessionVars() varName, isNull, err := b.args[0].EvalString(b.ctx, row) @@ -442,6 +502,12 @@ type builtinValuesIntSig struct { offset int } +func (b *builtinValuesIntSig) Clone() builtinFunc { + newSig := &builtinValuesIntSig{offset: b.offset} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinValuesIntSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesIntSig) evalInt(_ types.Row) (int64, bool, error) { @@ -465,6 +531,12 @@ type builtinValuesRealSig struct { offset int } +func (b *builtinValuesRealSig) Clone() builtinFunc { + newSig := &builtinValuesRealSig{offset: b.offset} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalReal evals a builtinValuesRealSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesRealSig) evalReal(_ types.Row) (float64, bool, error) { @@ -488,6 +560,12 @@ type builtinValuesDecimalSig struct { offset int } +func (b *builtinValuesDecimalSig) Clone() builtinFunc { + newSig := &builtinValuesDecimalSig{offset: b.offset} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals a builtinValuesDecimalSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesDecimalSig) evalDecimal(_ types.Row) (*types.MyDecimal, bool, error) { @@ -511,6 +589,12 @@ type builtinValuesStringSig struct { offset int } +func (b *builtinValuesStringSig) Clone() builtinFunc { + newSig := &builtinValuesStringSig{offset: b.offset} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinValuesStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesStringSig) evalString(_ types.Row) (string, bool, error) { @@ -534,6 +618,12 @@ type builtinValuesTimeSig struct { offset int } +func (b *builtinValuesTimeSig) Clone() builtinFunc { + newSig := &builtinValuesTimeSig{offset: b.offset} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinValuesTimeSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesTimeSig) evalTime(_ types.Row) (types.Time, bool, error) { @@ -557,6 +647,12 @@ type builtinValuesDurationSig struct { offset int } +func (b *builtinValuesDurationSig) Clone() builtinFunc { + newSig := &builtinValuesDurationSig{offset: b.offset} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinValuesDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesDurationSig) evalDuration(_ types.Row) (types.Duration, bool, error) { @@ -580,6 +676,12 @@ type builtinValuesJSONSig struct { offset int } +func (b *builtinValuesJSONSig) Clone() builtinFunc { + newSig := &builtinValuesJSONSig{offset: b.offset} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalJSON evals a builtinValuesJSONSig. // See https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values func (b *builtinValuesJSONSig) evalJSON(_ types.Row) (json.BinaryJSON, bool, error) { @@ -615,6 +717,12 @@ type builtinBitCountSig struct { baseBuiltinFunc } +func (b *builtinBitCountSig) Clone() builtinFunc { + newSig := &builtinBitCountSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals BIT_COUNT(N). // See https://dev.mysql.com/doc/refman/5.7/en/bit-functions.html#function_bit-count func (b *builtinBitCountSig) evalInt(row types.Row) (int64, bool, error) { @@ -654,6 +762,12 @@ type builtinGetParamStringSig struct { baseBuiltinFunc } +func (b *builtinGetParamStringSig) Clone() builtinFunc { + newSig := &builtinGetParamStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinGetParamStringSig) evalString(row types.Row) (string, bool, error) { sessionVars := b.ctx.GetSessionVars() idx, isNull, err := b.args[0].EvalInt(b.ctx, row) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index fe877bc3760d2..d94b5e6e700f4 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -190,6 +190,12 @@ type builtinLengthSig struct { baseBuiltinFunc } +func (b *builtinLengthSig) Clone() builtinFunc { + newSig := &builtinLengthSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evaluates a builtinLengthSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html func (b *builtinLengthSig) evalInt(row types.Row) (int64, bool, error) { @@ -218,6 +224,12 @@ type builtinASCIISig struct { baseBuiltinFunc } +func (b *builtinASCIISig) Clone() builtinFunc { + newSig := &builtinASCIISig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // eval evals a builtinASCIISig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_ascii func (b *builtinASCIISig) evalInt(row types.Row) (int64, bool, error) { @@ -265,6 +277,12 @@ type builtinConcatSig struct { baseBuiltinFunc } +func (b *builtinConcatSig) Clone() builtinFunc { + newSig := &builtinConcatSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_concat func (b *builtinConcatSig) evalString(row types.Row) (d string, isNull bool, err error) { var s []byte @@ -323,6 +341,12 @@ type builtinConcatWSSig struct { baseBuiltinFunc } +func (b *builtinConcatWSSig) Clone() builtinFunc { + newSig := &builtinConcatWSSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinConcatWSSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_concat-ws func (b *builtinConcatWSSig) evalString(row types.Row) (string, bool, error) { @@ -380,6 +404,12 @@ type builtinLeftBinarySig struct { baseBuiltinFunc } +func (b *builtinLeftBinarySig) Clone() builtinFunc { + newSig := &builtinLeftBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals LEFT(str,len). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_left func (b *builtinLeftBinarySig) evalString(row types.Row) (string, bool, error) { @@ -404,6 +434,12 @@ type builtinLeftSig struct { baseBuiltinFunc } +func (b *builtinLeftSig) Clone() builtinFunc { + newSig := &builtinLeftSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals LEFT(str,len). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_left func (b *builtinLeftSig) evalString(row types.Row) (string, bool, error) { @@ -448,6 +484,12 @@ type builtinRightBinarySig struct { baseBuiltinFunc } +func (b *builtinRightBinarySig) Clone() builtinFunc { + newSig := &builtinRightBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals RIGHT(str,len). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_right func (b *builtinRightBinarySig) evalString(row types.Row) (string, bool, error) { @@ -472,6 +514,12 @@ type builtinRightSig struct { baseBuiltinFunc } +func (b *builtinRightSig) Clone() builtinFunc { + newSig := &builtinRightSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals RIGHT(str,len). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_right func (b *builtinRightSig) evalString(row types.Row) (string, bool, error) { @@ -512,6 +560,12 @@ type builtinRepeatSig struct { baseBuiltinFunc } +func (b *builtinRepeatSig) Clone() builtinFunc { + newSig := &builtinRepeatSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // eval evals a builtinRepeatSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_repeat func (b *builtinRepeatSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -557,6 +611,12 @@ type builtinLowerSig struct { baseBuiltinFunc } +func (b *builtinLowerSig) Clone() builtinFunc { + newSig := &builtinLowerSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinLowerSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_lower func (b *builtinLowerSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -598,6 +658,12 @@ type builtinReverseBinarySig struct { baseBuiltinFunc } +func (b *builtinReverseBinarySig) Clone() builtinFunc { + newSig := &builtinReverseBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a REVERSE(str). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_reverse func (b *builtinReverseBinarySig) evalString(row types.Row) (string, bool, error) { @@ -613,6 +679,12 @@ type builtinReverseSig struct { baseBuiltinFunc } +func (b *builtinReverseSig) Clone() builtinFunc { + newSig := &builtinReverseSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a REVERSE(str). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_reverse func (b *builtinReverseSig) evalString(row types.Row) (string, bool, error) { @@ -642,6 +714,12 @@ type builtinSpaceSig struct { baseBuiltinFunc } +func (b *builtinSpaceSig) Clone() builtinFunc { + newSig := &builtinSpaceSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinSpaceSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_space func (b *builtinSpaceSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -680,6 +758,12 @@ type builtinUpperSig struct { baseBuiltinFunc } +func (b *builtinUpperSig) Clone() builtinFunc { + newSig := &builtinUpperSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinUpperSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_upper func (b *builtinUpperSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -714,6 +798,12 @@ type builtinStrcmpSig struct { baseBuiltinFunc } +func (b *builtinStrcmpSig) Clone() builtinFunc { + newSig := &builtinStrcmpSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinStrcmpSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-comparison-functions.html func (b *builtinStrcmpSig) evalInt(row types.Row) (int64, bool, error) { @@ -767,6 +857,12 @@ type builtinReplaceSig struct { baseBuiltinFunc } +func (b *builtinReplaceSig) Clone() builtinFunc { + newSig := &builtinReplaceSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinReplaceSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_replace func (b *builtinReplaceSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -811,6 +907,12 @@ type builtinConvertSig struct { baseBuiltinFunc } +func (b *builtinConvertSig) Clone() builtinFunc { + newSig := &builtinConvertSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals CONVERT(expr USING transcoding_name). // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert func (b *builtinConvertSig) evalString(row types.Row) (string, bool, error) { @@ -872,6 +974,12 @@ type builtinSubstringBinary2ArgsSig struct { baseBuiltinFunc } +func (b *builtinSubstringBinary2ArgsSig) Clone() builtinFunc { + newSig := &builtinSubstringBinary2ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals SUBSTR(str,pos), SUBSTR(str FROM pos), SUBSTR() is a synonym for SUBSTRING(). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_substr func (b *builtinSubstringBinary2ArgsSig) evalString(row types.Row) (string, bool, error) { @@ -899,6 +1007,12 @@ type builtinSubstring2ArgsSig struct { baseBuiltinFunc } +func (b *builtinSubstring2ArgsSig) Clone() builtinFunc { + newSig := &builtinSubstring2ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals SUBSTR(str,pos), SUBSTR(str FROM pos), SUBSTR() is a synonym for SUBSTRING(). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_substr func (b *builtinSubstring2ArgsSig) evalString(row types.Row) (string, bool, error) { @@ -927,6 +1041,12 @@ type builtinSubstringBinary3ArgsSig struct { baseBuiltinFunc } +func (b *builtinSubstringBinary3ArgsSig) Clone() builtinFunc { + newSig := &builtinSubstringBinary3ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals SUBSTR(str,pos,len), SUBSTR(str FROM pos FOR len), SUBSTR() is a synonym for SUBSTRING(). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_substr func (b *builtinSubstringBinary3ArgsSig) evalString(row types.Row) (string, bool, error) { @@ -964,6 +1084,12 @@ type builtinSubstring3ArgsSig struct { baseBuiltinFunc } +func (b *builtinSubstring3ArgsSig) Clone() builtinFunc { + newSig := &builtinSubstring3ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals SUBSTR(str,pos,len), SUBSTR(str FROM pos FOR len), SUBSTR() is a synonym for SUBSTRING(). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_substr func (b *builtinSubstring3ArgsSig) evalString(row types.Row) (string, bool, error) { @@ -1018,6 +1144,12 @@ type builtinSubstringIndexSig struct { baseBuiltinFunc } +func (b *builtinSubstringIndexSig) Clone() builtinFunc { + newSig := &builtinSubstringIndexSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinSubstringIndexSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_substring-index func (b *builtinSubstringIndexSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -1092,6 +1224,12 @@ type builtinLocateBinary2ArgsSig struct { baseBuiltinFunc } +func (b *builtinLocateBinary2ArgsSig) Clone() builtinFunc { + newSig := &builtinLocateBinary2ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals LOCATE(substr,str), case-sensitive. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_locate func (b *builtinLocateBinary2ArgsSig) evalInt(row types.Row) (int64, bool, error) { @@ -1118,6 +1256,12 @@ type builtinLocate2ArgsSig struct { baseBuiltinFunc } +func (b *builtinLocate2ArgsSig) Clone() builtinFunc { + newSig := &builtinLocate2ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals LOCATE(substr,str), non case-sensitive. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_locate func (b *builtinLocate2ArgsSig) evalInt(row types.Row) (int64, bool, error) { @@ -1144,6 +1288,12 @@ type builtinLocateBinary3ArgsSig struct { baseBuiltinFunc } +func (b *builtinLocateBinary3ArgsSig) Clone() builtinFunc { + newSig := &builtinLocateBinary3ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals LOCATE(substr,str,pos), case-sensitive. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_locate func (b *builtinLocateBinary3ArgsSig) evalInt(row types.Row) (int64, bool, error) { @@ -1179,6 +1329,12 @@ type builtinLocate3ArgsSig struct { baseBuiltinFunc } +func (b *builtinLocate3ArgsSig) Clone() builtinFunc { + newSig := &builtinLocate3ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals LOCATE(substr,str,pos), non case-sensitive. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_locate func (b *builtinLocate3ArgsSig) evalInt(row types.Row) (int64, bool, error) { @@ -1241,6 +1397,12 @@ type builtinHexStrArgSig struct { baseBuiltinFunc } +func (b *builtinHexStrArgSig) Clone() builtinFunc { + newSig := &builtinHexStrArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinHexStrArgSig, corresponding to hex(str) // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_hex func (b *builtinHexStrArgSig) evalString(row types.Row) (string, bool, error) { @@ -1255,6 +1417,12 @@ type builtinHexIntArgSig struct { baseBuiltinFunc } +func (b *builtinHexIntArgSig) Clone() builtinFunc { + newSig := &builtinHexIntArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinHexIntArgSig, corresponding to hex(N) // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_hex func (b *builtinHexIntArgSig) evalString(row types.Row) (string, bool, error) { @@ -1302,6 +1470,12 @@ type builtinUnHexSig struct { baseBuiltinFunc } +func (b *builtinUnHexSig) Clone() builtinFunc { + newSig := &builtinUnHexSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinUnHexSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_unhex func (b *builtinUnHexSig) evalString(row types.Row) (string, bool, error) { @@ -1368,6 +1542,12 @@ type builtinTrim1ArgSig struct { baseBuiltinFunc } +func (b *builtinTrim1ArgSig) Clone() builtinFunc { + newSig := &builtinTrim1ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinTrim1ArgSig, corresponding to trim(str) // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim func (b *builtinTrim1ArgSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -1382,6 +1562,12 @@ type builtinTrim2ArgsSig struct { baseBuiltinFunc } +func (b *builtinTrim2ArgsSig) Clone() builtinFunc { + newSig := &builtinTrim2ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinTrim2ArgsSig, corresponding to trim(str, remstr) // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim func (b *builtinTrim2ArgsSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -1404,6 +1590,12 @@ type builtinTrim3ArgsSig struct { baseBuiltinFunc } +func (b *builtinTrim3ArgsSig) Clone() builtinFunc { + newSig := &builtinTrim3ArgsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinTrim3ArgsSig, corresponding to trim(str, remstr, direction) // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim func (b *builtinTrim3ArgsSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -1469,6 +1661,12 @@ type builtinLTrimSig struct { baseBuiltinFunc } +func (b *builtinLTrimSig) Clone() builtinFunc { + newSig := &builtinLTrimSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinLTrimSig // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_ltrim func (b *builtinLTrimSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -1499,6 +1697,12 @@ type builtinRTrimSig struct { baseBuiltinFunc } +func (b *builtinRTrimSig) Clone() builtinFunc { + newSig := &builtinRTrimSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinRTrimSig // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_rtrim func (b *builtinRTrimSig) evalString(row types.Row) (d string, isNull bool, err error) { @@ -1570,6 +1774,12 @@ type builtinLpadBinarySig struct { baseBuiltinFunc } +func (b *builtinLpadBinarySig) Clone() builtinFunc { + newSig := &builtinLpadBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals LPAD(str,len,padstr). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_lpad func (b *builtinLpadBinarySig) evalString(row types.Row) (string, bool, error) { @@ -1606,6 +1816,12 @@ type builtinLpadSig struct { baseBuiltinFunc } +func (b *builtinLpadSig) Clone() builtinFunc { + newSig := &builtinLpadSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals LPAD(str,len,padstr). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_lpad func (b *builtinLpadSig) evalString(row types.Row) (string, bool, error) { @@ -1665,6 +1881,12 @@ type builtinRpadBinarySig struct { baseBuiltinFunc } +func (b *builtinRpadBinarySig) Clone() builtinFunc { + newSig := &builtinRpadBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals RPAD(str,len,padstr). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_rpad func (b *builtinRpadBinarySig) evalString(row types.Row) (string, bool, error) { @@ -1701,6 +1923,12 @@ type builtinRpadSig struct { baseBuiltinFunc } +func (b *builtinRpadSig) Clone() builtinFunc { + newSig := &builtinRpadSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals RPAD(str,len,padstr). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_rpad func (b *builtinRpadSig) evalString(row types.Row) (string, bool, error) { @@ -1751,6 +1979,12 @@ type builtinBitLengthSig struct { baseBuiltinFunc } +func (b *builtinBitLengthSig) Clone() builtinFunc { + newSig := &builtinBitLengthSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evaluates a builtinBitLengthSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_bit-length func (b *builtinBitLengthSig) evalInt(row types.Row) (int64, bool, error) { @@ -1787,6 +2021,12 @@ type builtinCharSig struct { baseBuiltinFunc } +func (b *builtinCharSig) Clone() builtinFunc { + newSig := &builtinCharSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCharSig) convertToBytes(ints []int64) []byte { buffer := bytes.NewBuffer([]byte{}) for i := len(ints) - 1; i >= 0; i-- { @@ -1859,6 +2099,12 @@ type builtinCharLengthSig struct { baseBuiltinFunc } +func (b *builtinCharLengthSig) Clone() builtinFunc { + newSig := &builtinCharLengthSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinCharLengthSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_char-length func (b *builtinCharLengthSig) evalInt(row types.Row) (int64, bool, error) { @@ -1887,6 +2133,12 @@ type builtinFindInSetSig struct { baseBuiltinFunc } +func (b *builtinFindInSetSig) Clone() builtinFunc { + newSig := &builtinFindInSetSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals FIND_IN_SET(str,strlist). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_find-in-set // TODO: This function can be optimized by using bit arithmetic when the first argument is @@ -1957,6 +2209,12 @@ type builtinFieldIntSig struct { baseBuiltinFunc } +func (b *builtinFieldIntSig) Clone() builtinFunc { + newSig := &builtinFieldIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals FIELD(str,str1,str2,str3,...). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_field func (b *builtinFieldIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -1980,6 +2238,12 @@ type builtinFieldRealSig struct { baseBuiltinFunc } +func (b *builtinFieldRealSig) Clone() builtinFunc { + newSig := &builtinFieldRealSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals FIELD(str,str1,str2,str3,...). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_field func (b *builtinFieldRealSig) evalInt(row types.Row) (int64, bool, error) { @@ -2003,6 +2267,12 @@ type builtinFieldStringSig struct { baseBuiltinFunc } +func (b *builtinFieldStringSig) Clone() builtinFunc { + newSig := &builtinFieldStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals FIELD(str,str1,str2,str3,...). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_field func (b *builtinFieldStringSig) evalInt(row types.Row) (int64, bool, error) { @@ -2074,6 +2344,12 @@ type builtinMakeSetSig struct { baseBuiltinFunc } +func (b *builtinMakeSetSig) Clone() builtinFunc { + newSig := &builtinMakeSetSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals MAKE_SET(bits,str1,str2,...). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_make-set func (b *builtinMakeSetSig) evalString(row types.Row) (string, bool, error) { @@ -2125,6 +2401,12 @@ type builtinOctIntSig struct { baseBuiltinFunc } +func (b *builtinOctIntSig) Clone() builtinFunc { + newSig := &builtinOctIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals OCT(N). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_oct func (b *builtinOctIntSig) evalString(row types.Row) (string, bool, error) { @@ -2140,6 +2422,12 @@ type builtinOctStringSig struct { baseBuiltinFunc } +func (b *builtinOctStringSig) Clone() builtinFunc { + newSig := &builtinOctStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // // evalString evals OCT(N). // // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_oct func (b *builtinOctStringSig) evalString(row types.Row) (string, bool, error) { @@ -2189,6 +2477,12 @@ type builtinOrdSig struct { baseBuiltinFunc } +func (b *builtinOrdSig) Clone() builtinFunc { + newSig := &builtinOrdSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinOrdSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_ord func (b *builtinOrdSig) evalInt(row types.Row) (int64, bool, error) { @@ -2234,6 +2528,12 @@ type builtinQuoteSig struct { baseBuiltinFunc } +func (b *builtinQuoteSig) Clone() builtinFunc { + newSig := &builtinQuoteSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals QUOTE(str). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_quote func (b *builtinQuoteSig) evalString(row types.Row) (string, bool, error) { @@ -2283,6 +2583,12 @@ type builtinBinSig struct { baseBuiltinFunc } +func (b *builtinBinSig) Clone() builtinFunc { + newSig := &builtinBinSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals BIN(N). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_bin func (b *builtinBinSig) evalString(row types.Row) (string, bool, error) { @@ -2324,6 +2630,12 @@ type builtinEltSig struct { baseBuiltinFunc } +func (b *builtinEltSig) Clone() builtinFunc { + newSig := &builtinEltSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinEltSig. // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_elt func (b *builtinEltSig) evalString(row types.Row) (string, bool, error) { @@ -2391,6 +2703,12 @@ type builtinExportSet3ArgSig struct { baseBuiltinFunc } +func (b *builtinExportSet3ArgSig) Clone() builtinFunc { + newSig := &builtinExportSet3ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals EXPORT_SET(bits,on,off). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_export-set func (b *builtinExportSet3ArgSig) evalString(row types.Row) (string, bool, error) { @@ -2416,6 +2734,12 @@ type builtinExportSet4ArgSig struct { baseBuiltinFunc } +func (b *builtinExportSet4ArgSig) Clone() builtinFunc { + newSig := &builtinExportSet4ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals EXPORT_SET(bits,on,off,separator). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_export-set func (b *builtinExportSet4ArgSig) evalString(row types.Row) (string, bool, error) { @@ -2446,6 +2770,12 @@ type builtinExportSet5ArgSig struct { baseBuiltinFunc } +func (b *builtinExportSet5ArgSig) Clone() builtinFunc { + newSig := &builtinExportSet5ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals EXPORT_SET(bits,on,off,separator,number_of_bits). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_export-set func (b *builtinExportSet5ArgSig) evalString(row types.Row) (string, bool, error) { @@ -2508,6 +2838,12 @@ type builtinFormatWithLocaleSig struct { baseBuiltinFunc } +func (b *builtinFormatWithLocaleSig) Clone() builtinFunc { + newSig := &builtinFormatWithLocaleSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals FORMAT(X,D,locale). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_format func (b *builtinFormatWithLocaleSig) evalString(row types.Row) (string, bool, error) { @@ -2534,6 +2870,12 @@ type builtinFormatSig struct { baseBuiltinFunc } +func (b *builtinFormatSig) Clone() builtinFunc { + newSig := &builtinFormatSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals FORMAT(X,D). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_format func (b *builtinFormatSig) evalString(row types.Row) (string, bool, error) { @@ -2570,6 +2912,12 @@ type builtinFromBase64Sig struct { baseBuiltinFunc } +func (b *builtinFromBase64Sig) Clone() builtinFunc { + newSig := &builtinFromBase64Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals FROM_BASE64(str). // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_from-base64 func (b *builtinFromBase64Sig) evalString(row types.Row) (string, bool, error) { @@ -2605,6 +2953,12 @@ type builtinToBase64Sig struct { baseBuiltinFunc } +func (b *builtinToBase64Sig) Clone() builtinFunc { + newSig := &builtinToBase64Sig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // base64NeededEncodedLength return the base64 encoded string length. func base64NeededEncodedLength(n int) int { // Returns -1 indicate the result will overflow. @@ -2688,6 +3042,12 @@ type builtinInsertBinarySig struct { baseBuiltinFunc } +func (b *builtinInsertBinarySig) Clone() builtinFunc { + newSig := &builtinInsertBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals INSERT(str,pos,len,newstr). // See https://dev.mysql.com/doc/refman/5.6/en/string-functions.html#function_insert func (b *builtinInsertBinarySig) evalString(row types.Row) (string, bool, error) { @@ -2725,6 +3085,12 @@ type builtinInsertSig struct { baseBuiltinFunc } +func (b *builtinInsertSig) Clone() builtinFunc { + newSig := &builtinInsertSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals INSERT(str,pos,len,newstr). // See https://dev.mysql.com/doc/refman/5.6/en/string-functions.html#function_insert func (b *builtinInsertSig) evalString(row types.Row) (string, bool, error) { @@ -2778,8 +3144,21 @@ func (c *instrFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi } type builtinInstrSig struct{ baseBuiltinFunc } + +func (b *builtinInstrSig) Clone() builtinFunc { + newSig := &builtinInstrSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + type builtinInstrBinarySig struct{ baseBuiltinFunc } +func (b *builtinInstrBinarySig) Clone() builtinFunc { + newSig := &builtinInstrBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals INSTR(str,substr), case insensitive // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_instr func (b *builtinInstrSig) evalInt(row types.Row) (int64, bool, error) { diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 470c2162a493f..8cdb0c5125235 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -257,6 +257,12 @@ type builtinDateSig struct { baseBuiltinFunc } +func (b *builtinDateSig) Clone() builtinFunc { + newSig := &builtinDateSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals DATE(expr). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_date func (b *builtinDateSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -309,6 +315,12 @@ type builtinDateLiteralSig struct { literal types.Time } +func (b *builtinDateLiteralSig) Clone() builtinFunc { + newSig := &builtinDateLiteralSig{literal: b.literal} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals DATE 'stringLit'. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-literals.html func (b *builtinDateLiteralSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -339,6 +351,12 @@ type builtinDateDiffSig struct { baseBuiltinFunc } +func (b *builtinDateDiffSig) Clone() builtinFunc { + newSig := &builtinDateDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinDateDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_datediff func (b *builtinDateDiffSig) evalInt(row types.Row) (int64, bool, error) { @@ -426,6 +444,12 @@ type builtinDurationDurationTimeDiffSig struct { baseBuiltinFunc } +func (b *builtinDurationDurationTimeDiffSig) Clone() builtinFunc { + newSig := &builtinDurationDurationTimeDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinDurationDurationTimeDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timediff func (b *builtinDurationDurationTimeDiffSig) evalDuration(row types.Row) (d types.Duration, isNull bool, err error) { @@ -447,6 +471,12 @@ type builtinTimeTimeTimeDiffSig struct { baseBuiltinFunc } +func (b *builtinTimeTimeTimeDiffSig) Clone() builtinFunc { + newSig := &builtinTimeTimeTimeDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinTimeTimeTimeDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timediff func (b *builtinTimeTimeTimeDiffSig) evalDuration(row types.Row) (d types.Duration, isNull bool, err error) { @@ -469,6 +499,12 @@ type builtinDurationStringTimeDiffSig struct { baseBuiltinFunc } +func (b *builtinDurationStringTimeDiffSig) Clone() builtinFunc { + newSig := &builtinDurationStringTimeDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinDurationStringTimeDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timediff func (b *builtinDurationStringTimeDiffSig) evalDuration(row types.Row) (d types.Duration, isNull bool, err error) { @@ -496,6 +532,12 @@ type builtinStringDurationTimeDiffSig struct { baseBuiltinFunc } +func (b *builtinStringDurationTimeDiffSig) Clone() builtinFunc { + newSig := &builtinStringDurationTimeDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinStringDurationTimeDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timediff func (b *builtinStringDurationTimeDiffSig) evalDuration(row types.Row) (d types.Duration, isNull bool, err error) { @@ -548,6 +590,12 @@ type builtinTimeStringTimeDiffSig struct { baseBuiltinFunc } +func (b *builtinTimeStringTimeDiffSig) Clone() builtinFunc { + newSig := &builtinTimeStringTimeDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinTimeStringTimeDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timediff func (b *builtinTimeStringTimeDiffSig) evalDuration(row types.Row) (d types.Duration, isNull bool, err error) { @@ -575,6 +623,12 @@ type builtinStringTimeTimeDiffSig struct { baseBuiltinFunc } +func (b *builtinStringTimeTimeDiffSig) Clone() builtinFunc { + newSig := &builtinStringTimeTimeDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinStringTimeTimeDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timediff func (b *builtinStringTimeTimeDiffSig) evalDuration(row types.Row) (d types.Duration, isNull bool, err error) { @@ -602,6 +656,12 @@ type builtinStringStringTimeDiffSig struct { baseBuiltinFunc } +func (b *builtinStringStringTimeDiffSig) Clone() builtinFunc { + newSig := &builtinStringStringTimeDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinStringStringTimeDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timediff func (b *builtinStringStringTimeDiffSig) evalDuration(row types.Row) (d types.Duration, isNull bool, err error) { @@ -644,6 +704,12 @@ type builtinNullTimeDiffSig struct { baseBuiltinFunc } +func (b *builtinNullTimeDiffSig) Clone() builtinFunc { + newSig := &builtinNullTimeDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinNullTimeDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timediff func (b *builtinNullTimeDiffSig) evalDuration(row types.Row) (d types.Duration, isNull bool, err error) { @@ -683,6 +749,12 @@ type builtinDateFormatSig struct { baseBuiltinFunc } +func (b *builtinDateFormatSig) Clone() builtinFunc { + newSig := &builtinDateFormatSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinDateFormatSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_date-format func (b *builtinDateFormatSig) evalString(row types.Row) (string, bool, error) { @@ -720,6 +792,12 @@ type builtinFromDaysSig struct { baseBuiltinFunc } +func (b *builtinFromDaysSig) Clone() builtinFunc { + newSig := &builtinFromDaysSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals FROM_DAYS(N). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_from-days func (b *builtinFromDaysSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -749,6 +827,12 @@ type builtinHourSig struct { baseBuiltinFunc } +func (b *builtinHourSig) Clone() builtinFunc { + newSig := &builtinHourSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals HOUR(time). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_hour func (b *builtinHourSig) evalInt(row types.Row) (int64, bool, error) { @@ -778,6 +862,12 @@ type builtinMinuteSig struct { baseBuiltinFunc } +func (b *builtinMinuteSig) Clone() builtinFunc { + newSig := &builtinMinuteSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals MINUTE(time). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_minute func (b *builtinMinuteSig) evalInt(row types.Row) (int64, bool, error) { @@ -807,6 +897,12 @@ type builtinSecondSig struct { baseBuiltinFunc } +func (b *builtinSecondSig) Clone() builtinFunc { + newSig := &builtinSecondSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals SECOND(time). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_second func (b *builtinSecondSig) evalInt(row types.Row) (int64, bool, error) { @@ -836,6 +932,12 @@ type builtinMicroSecondSig struct { baseBuiltinFunc } +func (b *builtinMicroSecondSig) Clone() builtinFunc { + newSig := &builtinMicroSecondSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals MICROSECOND(expr). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_microsecond func (b *builtinMicroSecondSig) evalInt(row types.Row) (int64, bool, error) { @@ -865,6 +967,12 @@ type builtinMonthSig struct { baseBuiltinFunc } +func (b *builtinMonthSig) Clone() builtinFunc { + newSig := &builtinMonthSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals MONTH(date). // see: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_month func (b *builtinMonthSig) evalInt(row types.Row) (int64, bool, error) { @@ -900,6 +1008,12 @@ type builtinMonthNameSig struct { baseBuiltinFunc } +func (b *builtinMonthNameSig) Clone() builtinFunc { + newSig := &builtinMonthNameSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinMonthNameSig) evalString(row types.Row) (string, bool, error) { arg, isNull, err := b.args[0].EvalTime(b.ctx, row) if isNull || err != nil { @@ -932,6 +1046,12 @@ type builtinDayNameSig struct { baseBuiltinFunc } +func (b *builtinDayNameSig) Clone() builtinFunc { + newSig := &builtinDayNameSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinDayNameSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_dayname func (b *builtinDayNameSig) evalString(row types.Row) (string, bool, error) { @@ -967,6 +1087,12 @@ type builtinDayOfMonthSig struct { baseBuiltinFunc } +func (b *builtinDayOfMonthSig) Clone() builtinFunc { + newSig := &builtinDayOfMonthSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinDayOfMonthSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_dayofmonth func (b *builtinDayOfMonthSig) evalInt(row types.Row) (int64, bool, error) { @@ -998,6 +1124,12 @@ type builtinDayOfWeekSig struct { baseBuiltinFunc } +func (b *builtinDayOfWeekSig) Clone() builtinFunc { + newSig := &builtinDayOfWeekSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinDayOfWeekSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_dayofweek func (b *builtinDayOfWeekSig) evalInt(row types.Row) (int64, bool, error) { @@ -1030,6 +1162,12 @@ type builtinDayOfYearSig struct { baseBuiltinFunc } +func (b *builtinDayOfYearSig) Clone() builtinFunc { + newSig := &builtinDayOfYearSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinDayOfYearSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_dayofyear func (b *builtinDayOfYearSig) evalInt(row types.Row) (int64, bool, error) { @@ -1075,6 +1213,12 @@ type builtinWeekWithModeSig struct { baseBuiltinFunc } +func (b *builtinWeekWithModeSig) Clone() builtinFunc { + newSig := &builtinWeekWithModeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals WEEK(date, mode). // see: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_week func (b *builtinWeekWithModeSig) evalInt(row types.Row) (int64, bool, error) { @@ -1101,6 +1245,12 @@ type builtinWeekWithoutModeSig struct { baseBuiltinFunc } +func (b *builtinWeekWithoutModeSig) Clone() builtinFunc { + newSig := &builtinWeekWithoutModeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals WEEK(date). // see: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_week func (b *builtinWeekWithoutModeSig) evalInt(row types.Row) (int64, bool, error) { @@ -1138,6 +1288,12 @@ type builtinWeekDaySig struct { baseBuiltinFunc } +func (b *builtinWeekDaySig) Clone() builtinFunc { + newSig := &builtinWeekDaySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals WEEKDAY(date). func (b *builtinWeekDaySig) evalInt(row types.Row) (int64, bool, error) { date, isNull, err := b.args[0].EvalTime(b.ctx, row) @@ -1170,6 +1326,12 @@ type builtinWeekOfYearSig struct { baseBuiltinFunc } +func (b *builtinWeekOfYearSig) Clone() builtinFunc { + newSig := &builtinWeekOfYearSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals WEEKOFYEAR(date). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_weekofyear func (b *builtinWeekOfYearSig) evalInt(row types.Row) (int64, bool, error) { @@ -1205,6 +1367,12 @@ type builtinYearSig struct { baseBuiltinFunc } +func (b *builtinYearSig) Clone() builtinFunc { + newSig := &builtinYearSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals YEAR(date). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_year func (b *builtinYearSig) evalInt(row types.Row) (int64, bool, error) { @@ -1251,6 +1419,12 @@ type builtinYearWeekWithModeSig struct { baseBuiltinFunc } +func (b *builtinYearWeekWithModeSig) Clone() builtinFunc { + newSig := &builtinYearWeekWithModeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals YEARWEEK(date,mode). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_yearweek func (b *builtinYearWeekWithModeSig) evalInt(row types.Row) (int64, bool, error) { @@ -1282,6 +1456,12 @@ type builtinYearWeekWithoutModeSig struct { baseBuiltinFunc } +func (b *builtinYearWeekWithoutModeSig) Clone() builtinFunc { + newSig := &builtinYearWeekWithoutModeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals YEARWEEK(date). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_yearweek func (b *builtinYearWeekWithoutModeSig) evalInt(row types.Row) (int64, bool, error) { @@ -1401,6 +1581,12 @@ type builtinFromUnixTime1ArgSig struct { baseBuiltinFunc } +func (b *builtinFromUnixTime1ArgSig) Clone() builtinFunc { + newSig := &builtinFromUnixTime1ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinFromUnixTime1ArgSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_from-unixtime func (b *builtinFromUnixTime1ArgSig) evalTime(row types.Row) (res types.Time, isNull bool, err error) { @@ -1411,6 +1597,12 @@ type builtinFromUnixTime2ArgSig struct { baseBuiltinFunc } +func (b *builtinFromUnixTime2ArgSig) Clone() builtinFunc { + newSig := &builtinFromUnixTime2ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinFromUnixTime2ArgSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_from-unixtime func (b *builtinFromUnixTime2ArgSig) evalString(row types.Row) (res string, isNull bool, err error) { @@ -1444,6 +1636,12 @@ type builtinGetFormatSig struct { baseBuiltinFunc } +func (b *builtinGetFormatSig) Clone() builtinFunc { + newSig := &builtinGetFormatSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinGetFormatSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_get-format func (b *builtinGetFormatSig) evalString(row types.Row) (string, bool, error) { @@ -1563,6 +1761,12 @@ type builtinStrToDateDateSig struct { baseBuiltinFunc } +func (b *builtinStrToDateDateSig) Clone() builtinFunc { + newSig := &builtinStrToDateDateSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinStrToDateDateSig) evalTime(row types.Row) (types.Time, bool, error) { date, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { @@ -1586,6 +1790,12 @@ type builtinStrToDateDatetimeSig struct { baseBuiltinFunc } +func (b *builtinStrToDateDatetimeSig) Clone() builtinFunc { + newSig := &builtinStrToDateDatetimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinStrToDateDatetimeSig) evalTime(row types.Row) (types.Time, bool, error) { date, isNull, err := b.args[0].EvalString(b.ctx, row) if isNull || err != nil { @@ -1609,6 +1819,12 @@ type builtinStrToDateDurationSig struct { baseBuiltinFunc } +func (b *builtinStrToDateDurationSig) Clone() builtinFunc { + newSig := &builtinStrToDateDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // TODO: If the NO_ZERO_DATE or NO_ZERO_IN_DATE SQL mode is enabled, zero dates or part of dates are disallowed. // In that case, STR_TO_DATE() returns NULL and generates a warning. func (b *builtinStrToDateDurationSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -1659,6 +1875,12 @@ type builtinSysDateWithFspSig struct { baseBuiltinFunc } +func (b *builtinSysDateWithFspSig) Clone() builtinFunc { + newSig := &builtinSysDateWithFspSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals SYSDATE(fsp). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_sysdate func (b *builtinSysDateWithFspSig) evalTime(row types.Row) (d types.Time, isNull bool, err error) { @@ -1680,6 +1902,12 @@ type builtinSysDateWithoutFspSig struct { baseBuiltinFunc } +func (b *builtinSysDateWithoutFspSig) Clone() builtinFunc { + newSig := &builtinSysDateWithoutFspSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals SYSDATE(). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_sysdate func (b *builtinSysDateWithoutFspSig) evalTime(row types.Row) (d types.Time, isNull bool, err error) { @@ -1710,6 +1938,12 @@ type builtinCurrentDateSig struct { baseBuiltinFunc } +func (b *builtinCurrentDateSig) Clone() builtinFunc { + newSig := &builtinCurrentDateSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals CURDATE(). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_curdate func (b *builtinCurrentDateSig) evalTime(row types.Row) (d types.Time, isNull bool, err error) { @@ -1761,6 +1995,12 @@ type builtinCurrentTime0ArgSig struct { baseBuiltinFunc } +func (b *builtinCurrentTime0ArgSig) Clone() builtinFunc { + newSig := &builtinCurrentTime0ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCurrentTime0ArgSig) evalDuration(row types.Row) (types.Duration, bool, error) { tz := b.ctx.GetSessionVars().GetTimeZone() dur := time.Now().In(tz).Format(types.TimeFormat) @@ -1775,6 +2015,12 @@ type builtinCurrentTime1ArgSig struct { baseBuiltinFunc } +func (b *builtinCurrentTime1ArgSig) Clone() builtinFunc { + newSig := &builtinCurrentTime1ArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinCurrentTime1ArgSig) evalDuration(row types.Row) (types.Duration, bool, error) { fsp, _, err := b.args[0].EvalInt(b.ctx, row) if err != nil { @@ -1806,6 +2052,12 @@ type builtinTimeSig struct { baseBuiltinFunc } +func (b *builtinTimeSig) Clone() builtinFunc { + newSig := &builtinTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinTimeSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_time. func (b *builtinTimeSig) evalDuration(row types.Row) (res types.Duration, isNull bool, err error) { @@ -1869,6 +2121,12 @@ type builtinTimeLiteralSig struct { duration types.Duration } +func (b *builtinTimeLiteralSig) Clone() builtinFunc { + newSig := &builtinTimeLiteralSig{duration: b.duration} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals TIME 'stringLit'. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-literals.html func (b *builtinTimeLiteralSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -1893,6 +2151,12 @@ type builtinUTCDateSig struct { baseBuiltinFunc } +func (b *builtinUTCDateSig) Clone() builtinFunc { + newSig := &builtinUTCDateSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals UTC_DATE, UTC_DATE(). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-date func (b *builtinUTCDateSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -1964,6 +2228,12 @@ type builtinUTCTimestampWithArgSig struct { baseBuiltinFunc } +func (b *builtinUTCTimestampWithArgSig) Clone() builtinFunc { + newSig := &builtinUTCTimestampWithArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals UTC_TIMESTAMP(fsp). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-timestamp func (b *builtinUTCTimestampWithArgSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -1987,6 +2257,12 @@ type builtinUTCTimestampWithoutArgSig struct { baseBuiltinFunc } +func (b *builtinUTCTimestampWithoutArgSig) Clone() builtinFunc { + newSig := &builtinUTCTimestampWithoutArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals UTC_TIMESTAMP(). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-timestamp func (b *builtinUTCTimestampWithoutArgSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2046,6 +2322,12 @@ type builtinNowWithArgSig struct { baseBuiltinFunc } +func (b *builtinNowWithArgSig) Clone() builtinFunc { + newSig := &builtinNowWithArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals NOW(fsp) // see: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_now func (b *builtinNowWithArgSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2071,6 +2353,12 @@ type builtinNowWithoutArgSig struct { baseBuiltinFunc } +func (b *builtinNowWithoutArgSig) Clone() builtinFunc { + newSig := &builtinNowWithoutArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals NOW() // see: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_now func (b *builtinNowWithoutArgSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2123,6 +2411,12 @@ type builtinExtractDatetimeSig struct { baseBuiltinFunc } +func (b *builtinExtractDatetimeSig) Clone() builtinFunc { + newSig := &builtinExtractDatetimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinExtractDatetimeSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_extract func (b *builtinExtractDatetimeSig) evalInt(row types.Row) (int64, bool, error) { @@ -2142,6 +2436,12 @@ type builtinExtractDurationSig struct { baseBuiltinFunc } +func (b *builtinExtractDurationSig) Clone() builtinFunc { + newSig := &builtinExtractDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinExtractDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_extract func (b *builtinExtractDurationSig) evalInt(row types.Row) (int64, bool, error) { @@ -2398,6 +2698,12 @@ type builtinAddDateStringStringSig struct { baseDateArithmitical } +func (b *builtinAddDateStringStringSig) Clone() builtinFunc { + newSig := &builtinAddDateStringStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals ADDDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate func (b *builtinAddDateStringStringSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2425,6 +2731,12 @@ type builtinAddDateStringIntSig struct { baseDateArithmitical } +func (b *builtinAddDateStringIntSig) Clone() builtinFunc { + newSig := &builtinAddDateStringIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals ADDDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate func (b *builtinAddDateStringIntSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2452,6 +2764,12 @@ type builtinAddDateStringDecimalSig struct { baseDateArithmitical } +func (b *builtinAddDateStringDecimalSig) Clone() builtinFunc { + newSig := &builtinAddDateStringDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals ADDDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate func (b *builtinAddDateStringDecimalSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2479,6 +2797,12 @@ type builtinAddDateIntStringSig struct { baseDateArithmitical } +func (b *builtinAddDateIntStringSig) Clone() builtinFunc { + newSig := &builtinAddDateIntStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals ADDDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate func (b *builtinAddDateIntStringSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2506,6 +2830,12 @@ type builtinAddDateIntIntSig struct { baseDateArithmitical } +func (b *builtinAddDateIntIntSig) Clone() builtinFunc { + newSig := &builtinAddDateIntIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals ADDDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate func (b *builtinAddDateIntIntSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2533,6 +2863,12 @@ type builtinAddDateDatetimeStringSig struct { baseDateArithmitical } +func (b *builtinAddDateDatetimeStringSig) Clone() builtinFunc { + newSig := &builtinAddDateDatetimeStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals ADDDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate func (b *builtinAddDateDatetimeStringSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2560,6 +2896,12 @@ type builtinAddDateDatetimeIntSig struct { baseDateArithmitical } +func (b *builtinAddDateDatetimeIntSig) Clone() builtinFunc { + newSig := &builtinAddDateDatetimeIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals ADDDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_adddate func (b *builtinAddDateDatetimeIntSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2650,6 +2992,12 @@ type builtinSubDateStringStringSig struct { baseDateArithmitical } +func (b *builtinSubDateStringStringSig) Clone() builtinFunc { + newSig := &builtinSubDateStringStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals SUBDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate func (b *builtinSubDateStringStringSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2677,6 +3025,12 @@ type builtinSubDateStringIntSig struct { baseDateArithmitical } +func (b *builtinSubDateStringIntSig) Clone() builtinFunc { + newSig := &builtinSubDateStringIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals SUBDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate func (b *builtinSubDateStringIntSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2704,6 +3058,12 @@ type builtinSubDateStringDecimalSig struct { baseDateArithmitical } +func (b *builtinSubDateStringDecimalSig) Clone() builtinFunc { + newSig := &builtinSubDateStringDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (b *builtinSubDateStringDecimalSig) evalTime(row types.Row) (types.Time, bool, error) { unit, isNull, err := b.args[2].EvalString(b.ctx, row) if isNull || err != nil { @@ -2729,6 +3089,12 @@ type builtinSubDateIntStringSig struct { baseDateArithmitical } +func (b *builtinSubDateIntStringSig) Clone() builtinFunc { + newSig := &builtinSubDateIntStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals SUBDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate func (b *builtinSubDateIntStringSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2756,6 +3122,12 @@ type builtinSubDateIntIntSig struct { baseDateArithmitical } +func (b *builtinSubDateIntIntSig) Clone() builtinFunc { + newSig := &builtinSubDateIntIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals SUBDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate func (b *builtinSubDateIntIntSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2783,6 +3155,12 @@ type builtinSubDateDatetimeStringSig struct { baseDateArithmitical } +func (b *builtinSubDateDatetimeStringSig) Clone() builtinFunc { + newSig := &builtinSubDateDatetimeStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals SUBDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate func (b *builtinSubDateDatetimeStringSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2810,6 +3188,12 @@ type builtinSubDateDatetimeIntSig struct { baseDateArithmitical } +func (b *builtinSubDateDatetimeIntSig) Clone() builtinFunc { + newSig := &builtinSubDateDatetimeIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals SUBDATE(date,INTERVAL expr unit). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subdate func (b *builtinSubDateDatetimeIntSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -2849,6 +3233,12 @@ type builtinTimestampDiffSig struct { baseBuiltinFunc } +func (b *builtinTimestampDiffSig) Clone() builtinFunc { + newSig := &builtinTimestampDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinTimestampDiffSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timestampdiff func (b *builtinTimestampDiffSig) evalInt(row types.Row) (int64, bool, error) { @@ -2897,6 +3287,16 @@ func (c *unixTimestampFunctionClass) getFunction(ctx sessionctx.Context, args [] if argEvaltp == types.ETString { // Treat types.ETString as unspecified decimal. retDecimal = types.UnspecifiedLength + if cnst, ok := args[0].(*Constant); ok { + tmpStr, _, err := cnst.EvalString(ctx, nil) + if err != nil { + return nil, errors.Trace(err) + } + retDecimal = 0 + if dotIdx := strings.LastIndex(tmpStr, "."); dotIdx >= 0 { + retDecimal = len(tmpStr) - dotIdx - 1 + } + } } else { retDecimal = argType.Decimal } @@ -2928,10 +3328,7 @@ func (c *unixTimestampFunctionClass) getFunction(ctx sessionctx.Context, args [] sig = &builtinUnixTimestampIntSig{bf} } else if retTp == types.ETDecimal { sig = &builtinUnixTimestampDecSig{bf} - } else { - panic("Unexpected retTp") } - return sig, nil } @@ -2957,6 +3354,12 @@ type builtinUnixTimestampCurrentSig struct { baseBuiltinFunc } +func (b *builtinUnixTimestampCurrentSig) Clone() builtinFunc { + newSig := &builtinUnixTimestampCurrentSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a UNIX_TIMESTAMP(). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_unix-timestamp func (b *builtinUnixTimestampCurrentSig) evalInt(row types.Row) (int64, bool, error) { @@ -2973,6 +3376,12 @@ type builtinUnixTimestampIntSig struct { baseBuiltinFunc } +func (b *builtinUnixTimestampIntSig) Clone() builtinFunc { + newSig := &builtinUnixTimestampIntSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a UNIX_TIMESTAMP(time). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_unix-timestamp func (b *builtinUnixTimestampIntSig) evalInt(row types.Row) (int64, bool, error) { @@ -3001,6 +3410,12 @@ type builtinUnixTimestampDecSig struct { baseBuiltinFunc } +func (b *builtinUnixTimestampDecSig) Clone() builtinFunc { + newSig := &builtinUnixTimestampDecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDecimal evals a UNIX_TIMESTAMP(time). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_unix-timestamp func (b *builtinUnixTimestampDecSig) evalDecimal(row types.Row) (*types.MyDecimal, bool, error) { @@ -3077,6 +3492,12 @@ type builtinTimestamp1ArgSig struct { isFloat bool } +func (b *builtinTimestamp1ArgSig) Clone() builtinFunc { + newSig := &builtinTimestamp1ArgSig{isFloat: b.isFloat} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinTimestamp1ArgSig. // See https://dev.mysql.com/doc/refman/5.5/en/date-and-time-functions.html#function_timestamp func (b *builtinTimestamp1ArgSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -3103,6 +3524,12 @@ type builtinTimestamp2ArgsSig struct { isFloat bool } +func (b *builtinTimestamp2ArgsSig) Clone() builtinFunc { + newSig := &builtinTimestamp2ArgsSig{isFloat: b.isFloat} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinTimestamp2ArgsSig. // See https://dev.mysql.com/doc/refman/5.5/en/date-and-time-functions.html#function_timestamp func (b *builtinTimestamp2ArgsSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -3179,6 +3606,12 @@ type builtinTimestampLiteralSig struct { tm types.Time } +func (b *builtinTimestampLiteralSig) Clone() builtinFunc { + newSig := &builtinTimestampLiteralSig{tm: b.tm} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals TIMESTAMP 'stringLit'. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-literals.html func (b *builtinTimestampLiteralSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -3382,6 +3815,12 @@ type builtinAddTimeDateTimeNullSig struct { baseBuiltinFunc } +func (b *builtinAddTimeDateTimeNullSig) Clone() builtinFunc { + newSig := &builtinAddTimeDateTimeNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinAddTimeDateTimeNullSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddTimeDateTimeNullSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -3392,6 +3831,12 @@ type builtinAddDatetimeAndDurationSig struct { baseBuiltinFunc } +func (b *builtinAddDatetimeAndDurationSig) Clone() builtinFunc { + newSig := &builtinAddDatetimeAndDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinAddDatetimeAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddDatetimeAndDurationSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -3411,6 +3856,12 @@ type builtinAddDatetimeAndStringSig struct { baseBuiltinFunc } +func (b *builtinAddDatetimeAndStringSig) Clone() builtinFunc { + newSig := &builtinAddDatetimeAndStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinAddDatetimeAndStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddDatetimeAndStringSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -3437,6 +3888,12 @@ type builtinAddTimeDurationNullSig struct { baseBuiltinFunc } +func (b *builtinAddTimeDurationNullSig) Clone() builtinFunc { + newSig := &builtinAddTimeDurationNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinAddTimeDurationNullSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddTimeDurationNullSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -3447,6 +3904,12 @@ type builtinAddDurationAndDurationSig struct { baseBuiltinFunc } +func (b *builtinAddDurationAndDurationSig) Clone() builtinFunc { + newSig := &builtinAddDurationAndDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinAddDurationAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddDurationAndDurationSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -3469,6 +3932,12 @@ type builtinAddDurationAndStringSig struct { baseBuiltinFunc } +func (b *builtinAddDurationAndStringSig) Clone() builtinFunc { + newSig := &builtinAddDurationAndStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinAddDurationAndStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddDurationAndStringSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -3498,6 +3967,12 @@ type builtinAddTimeStringNullSig struct { baseBuiltinFunc } +func (b *builtinAddTimeStringNullSig) Clone() builtinFunc { + newSig := &builtinAddTimeStringNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinAddDurationAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddTimeStringNullSig) evalString(row types.Row) (string, bool, error) { @@ -3508,6 +3983,12 @@ type builtinAddStringAndDurationSig struct { baseBuiltinFunc } +func (b *builtinAddStringAndDurationSig) Clone() builtinFunc { + newSig := &builtinAddStringAndDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinAddStringAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddStringAndDurationSig) evalString(row types.Row) (result string, isNull bool, err error) { @@ -3539,6 +4020,12 @@ type builtinAddStringAndStringSig struct { baseBuiltinFunc } +func (b *builtinAddStringAndStringSig) Clone() builtinFunc { + newSig := &builtinAddStringAndStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinAddStringAndStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddStringAndStringSig) evalString(row types.Row) (result string, isNull bool, err error) { @@ -3574,6 +4061,12 @@ type builtinAddDateAndDurationSig struct { baseBuiltinFunc } +func (b *builtinAddDateAndDurationSig) Clone() builtinFunc { + newSig := &builtinAddDateAndDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinAddDurationAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddDateAndDurationSig) evalString(row types.Row) (string, bool, error) { @@ -3593,6 +4086,12 @@ type builtinAddDateAndStringSig struct { baseBuiltinFunc } +func (b *builtinAddDateAndStringSig) Clone() builtinFunc { + newSig := &builtinAddDateAndStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinAddDateAndStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_addtime func (b *builtinAddDateAndStringSig) evalString(row types.Row) (string, bool, error) { @@ -3668,6 +4167,12 @@ type builtinConvertTzSig struct { timezoneRegex *regexp.Regexp } +func (b *builtinConvertTzSig) Clone() builtinFunc { + newSig := &builtinConvertTzSig{timezoneRegex: b.timezoneRegex} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals CONVERT_TZ(dt,from_tz,to_tz). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_convert-tz func (b *builtinConvertTzSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -3745,6 +4250,12 @@ type builtinMakeDateSig struct { baseBuiltinFunc } +func (b *builtinMakeDateSig) Clone() builtinFunc { + newSig := &builtinMakeDateSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evaluates a builtinMakeDateSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_makedate func (b *builtinMakeDateSig) evalTime(row types.Row) (d types.Time, isNull bool, err error) { @@ -3814,6 +4325,12 @@ type builtinMakeTimeSig struct { baseBuiltinFunc } +func (b *builtinMakeTimeSig) Clone() builtinFunc { + newSig := &builtinMakeTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinMakeTimeIntSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_maketime func (b *builtinMakeTimeSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -3916,6 +4433,12 @@ type builtinPeriodAddSig struct { baseBuiltinFunc } +func (b *builtinPeriodAddSig) Clone() builtinFunc { + newSig := &builtinPeriodAddSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals PERIOD_ADD(P,N). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_period-add func (b *builtinPeriodAddSig) evalInt(row types.Row) (int64, bool, error) { @@ -3955,6 +4478,12 @@ type builtinPeriodDiffSig struct { baseBuiltinFunc } +func (b *builtinPeriodDiffSig) Clone() builtinFunc { + newSig := &builtinPeriodDiffSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals PERIOD_DIFF(P1,P2). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_period-diff func (b *builtinPeriodDiffSig) evalInt(row types.Row) (int64, bool, error) { @@ -3991,6 +4520,12 @@ type builtinQuarterSig struct { baseBuiltinFunc } +func (b *builtinQuarterSig) Clone() builtinFunc { + newSig := &builtinQuarterSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals QUARTER(date). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_quarter func (b *builtinQuarterSig) evalInt(row types.Row) (int64, bool, error) { @@ -4041,6 +4576,12 @@ type builtinSecToTimeSig struct { baseBuiltinFunc } +func (b *builtinSecToTimeSig) Clone() builtinFunc { + newSig := &builtinSecToTimeSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals SEC_TO_TIME(seconds). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_sec-to-time func (b *builtinSecToTimeSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -4137,6 +4678,12 @@ type builtinSubDatetimeAndDurationSig struct { baseBuiltinFunc } +func (b *builtinSubDatetimeAndDurationSig) Clone() builtinFunc { + newSig := &builtinSubDatetimeAndDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinSubDatetimeAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubDatetimeAndDurationSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -4162,6 +4709,12 @@ type builtinSubDatetimeAndStringSig struct { baseBuiltinFunc } +func (b *builtinSubDatetimeAndStringSig) Clone() builtinFunc { + newSig := &builtinSubDatetimeAndStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinSubDatetimeAndStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubDatetimeAndStringSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -4197,6 +4750,12 @@ type builtinSubTimeDateTimeNullSig struct { baseBuiltinFunc } +func (b *builtinSubTimeDateTimeNullSig) Clone() builtinFunc { + newSig := &builtinSubTimeDateTimeNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinSubTimeDateTimeNullSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubTimeDateTimeNullSig) evalTime(row types.Row) (types.Time, bool, error) { @@ -4207,6 +4766,12 @@ type builtinSubStringAndDurationSig struct { baseBuiltinFunc } +func (b *builtinSubStringAndDurationSig) Clone() builtinFunc { + newSig := &builtinSubStringAndDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinSubStringAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubStringAndDurationSig) evalString(row types.Row) (result string, isNull bool, err error) { @@ -4238,6 +4803,12 @@ type builtinSubStringAndStringSig struct { baseBuiltinFunc } +func (b *builtinSubStringAndStringSig) Clone() builtinFunc { + newSig := &builtinSubStringAndStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinAddStringAndStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubStringAndStringSig) evalString(row types.Row) (result string, isNull bool, err error) { @@ -4273,6 +4844,12 @@ type builtinSubTimeStringNullSig struct { baseBuiltinFunc } +func (b *builtinSubTimeStringNullSig) Clone() builtinFunc { + newSig := &builtinSubTimeStringNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinSubTimeStringNullSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubTimeStringNullSig) evalString(row types.Row) (string, bool, error) { @@ -4283,6 +4860,12 @@ type builtinSubDurationAndDurationSig struct { baseBuiltinFunc } +func (b *builtinSubDurationAndDurationSig) Clone() builtinFunc { + newSig := &builtinSubDurationAndDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinAddDurationAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubDurationAndDurationSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -4305,6 +4888,12 @@ type builtinSubDurationAndStringSig struct { baseBuiltinFunc } +func (b *builtinSubDurationAndStringSig) Clone() builtinFunc { + newSig := &builtinSubDurationAndStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinAddDurationAndStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubDurationAndStringSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -4331,6 +4920,12 @@ type builtinSubTimeDurationNullSig struct { baseBuiltinFunc } +func (b *builtinSubTimeDurationNullSig) Clone() builtinFunc { + newSig := &builtinSubTimeDurationNullSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinSubTimeDurationNullSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubTimeDurationNullSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -4341,6 +4936,12 @@ type builtinSubDateAndDurationSig struct { baseBuiltinFunc } +func (b *builtinSubDateAndDurationSig) Clone() builtinFunc { + newSig := &builtinSubDateAndDurationSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinAddDateAndDurationSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubDateAndDurationSig) evalString(row types.Row) (string, bool, error) { @@ -4360,6 +4961,12 @@ type builtinSubDateAndStringSig struct { baseBuiltinFunc } +func (b *builtinSubDateAndStringSig) Clone() builtinFunc { + newSig := &builtinSubDateAndStringSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinAddDateAndStringSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_subtime func (b *builtinSubDateAndStringSig) evalString(row types.Row) (string, bool, error) { @@ -4404,6 +5011,12 @@ type builtinTimeFormatSig struct { baseBuiltinFunc } +func (b *builtinTimeFormatSig) Clone() builtinFunc { + newSig := &builtinTimeFormatSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinTimeFormatSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_time-format func (b *builtinTimeFormatSig) evalString(row types.Row) (string, bool, error) { @@ -4451,6 +5064,12 @@ type builtinTimeToSecSig struct { baseBuiltinFunc } +func (b *builtinTimeToSecSig) Clone() builtinFunc { + newSig := &builtinTimeToSecSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals TIME_TO_SEC(time). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_time-to-sec func (b *builtinTimeToSecSig) evalInt(row types.Row) (int64, bool, error) { @@ -4486,6 +5105,12 @@ type builtinTimestampAddSig struct { baseBuiltinFunc } +func (b *builtinTimestampAddSig) Clone() builtinFunc { + newSig := &builtinTimestampAddSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalString evals a builtinTimestampAddSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_timestampadd func (b *builtinTimestampAddSig) evalString(row types.Row) (string, bool, error) { @@ -4554,6 +5179,12 @@ type builtinToDaysSig struct { baseBuiltinFunc } +func (b *builtinToDaysSig) Clone() builtinFunc { + newSig := &builtinToDaysSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinToDaysSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_to-days func (b *builtinToDaysSig) evalInt(row types.Row) (int64, bool, error) { @@ -4586,6 +5217,12 @@ type builtinToSecondsSig struct { baseBuiltinFunc } +func (b *builtinToSecondsSig) Clone() builtinFunc { + newSig := &builtinToSecondsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalInt evals a builtinToSecondsSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_to-seconds func (b *builtinToSecondsSig) evalInt(row types.Row) (int64, bool, error) { @@ -4651,6 +5288,12 @@ type builtinUTCTimeWithoutArgSig struct { baseBuiltinFunc } +func (b *builtinUTCTimeWithoutArgSig) Clone() builtinFunc { + newSig := &builtinUTCTimeWithoutArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinUTCTimeWithoutArgSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-time func (b *builtinUTCTimeWithoutArgSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -4662,6 +5305,12 @@ type builtinUTCTimeWithArgSig struct { baseBuiltinFunc } +func (b *builtinUTCTimeWithArgSig) Clone() builtinFunc { + newSig := &builtinUTCTimeWithArgSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalDuration evals a builtinUTCTimeWithArgSig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-time func (b *builtinUTCTimeWithArgSig) evalDuration(row types.Row) (types.Duration, bool, error) { @@ -4697,6 +5346,12 @@ type builtinLastDaySig struct { baseBuiltinFunc } +func (b *builtinLastDaySig) Clone() builtinFunc { + newSig := &builtinLastDaySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + // evalTime evals a builtinLastDaySig. // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_last-day func (b *builtinLastDaySig) evalTime(row types.Row) (types.Time, bool, error) { diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 5e976af88f52f..3fb851a2b9c4e 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -1467,14 +1467,13 @@ func (s *testEvaluatorSuite) TestUnixTimestamp(c *C) { {2, types.NewDecimalDatum(types.NewDecFromStringForTest("151113102019.12")), types.KindMysqlDecimal, "1447410019.12"}, // YYMMDDHHMMSS {7, types.NewDecimalDatum(types.NewDecFromStringForTest("151113102019.1234567")), types.KindMysqlDecimal, "1447410019.123457"}, // YYMMDDHHMMSS {0, types.NewIntDatum(20151113102019), types.KindInt64, "1447410019"}, // YYYYMMDDHHMMSS - // TODO: for string literal inputs as below, fsp should be based on user input. - {0, types.NewStringDatum("2015-11-13 10:20:19"), types.KindMysqlDecimal, "1447410019.000000"}, - {0, types.NewStringDatum("2015-11-13 10:20:19.012"), types.KindMysqlDecimal, "1447410019.012000"}, - {0, types.NewStringDatum("1970-01-01 00:00:00"), types.KindMysqlDecimal, "0.000000"}, // Min timestamp + {0, types.NewStringDatum("2015-11-13 10:20:19"), types.KindInt64, "1447410019"}, + {0, types.NewStringDatum("2015-11-13 10:20:19.012"), types.KindMysqlDecimal, "1447410019.012"}, + {0, types.NewStringDatum("1970-01-01 00:00:00"), types.KindInt64, "0"}, // Min timestamp {0, types.NewStringDatum("2038-01-19 03:14:07.999999"), types.KindMysqlDecimal, "2147483647.999999"}, // Max timestamp - {0, types.NewStringDatum("2017-00-02"), types.KindMysqlDecimal, "0"}, // Invalid date + {0, types.NewStringDatum("2017-00-02"), types.KindInt64, "0"}, // Invalid date {0, types.NewStringDatum("1969-12-31 23:59:59.999999"), types.KindMysqlDecimal, "0"}, // Invalid timestamp - {0, types.NewStringDatum("2038-01-19 03:14:08"), types.KindMysqlDecimal, "0"}, // Invalid timestamp + {0, types.NewStringDatum("2038-01-19 03:14:08"), types.KindInt64, "0"}, // Invalid timestamp // Below tests irregular inputs. {0, types.NewIntDatum(0), types.KindInt64, "0"}, {0, types.NewIntDatum(-1), types.KindInt64, "0"}, diff --git a/expression/constant.go b/expression/constant.go index 5a2e2c04319f6..927f938ff5d4e 100644 --- a/expression/constant.go +++ b/expression/constant.go @@ -75,8 +75,11 @@ func (c *Constant) MarshalJSON() ([]byte, error) { // Clone implements Expression interface. func (c *Constant) Clone() Expression { - con := *c - return &con + if c.DeferredExpr != nil { + con := *c + return &con + } + return c } // GetType implements Expression interface. diff --git a/expression/evaluator_test.go b/expression/evaluator_test.go index c19fde11fe803..5bae92c1be1fe 100644 --- a/expression/evaluator_test.go +++ b/expression/evaluator_test.go @@ -544,7 +544,7 @@ func (s *testEvaluatorSuite) TestLike(c *C) { {"a", "a", 1}, {"a", "b", 0}, {"aA", "Aa", 0}, - {"aAb", "Aa%", 0}, + {"aAb", `Aa%`, 0}, {"aAb", "aA_", 1}, } for _, tt := range tests { diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index f72b3964a37e4..ef52f283524ab 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -328,7 +328,7 @@ func (s *testEvaluatorSuite) TestLikeFunc2Pb(c *C) { args := []Expression{ &Constant{RetType: retTp, Value: types.NewDatum("string")}, &Constant{RetType: retTp, Value: types.NewDatum("pattern")}, - &Constant{RetType: retTp, Value: types.NewDatum("%abc%")}, + &Constant{RetType: retTp, Value: types.NewDatum(`%abc%`)}, &Constant{RetType: retTp, Value: types.NewDatum("\\")}, } ctx := mock.NewContext() diff --git a/expression/integration_test.go b/expression/integration_test.go index c41e449e19859..395f61a93e29f 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -1378,11 +1378,11 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { result.Check(testkit.Rows("00:01:23 AM")) // for date_format - result = tk.MustQuery("SELECT DATE_FORMAT('2017-06-15', '%W %M %e %Y %r %y');") + result = tk.MustQuery(`SELECT DATE_FORMAT('2017-06-15', '%W %M %e %Y %r %y');`) result.Check(testkit.Rows("Thursday June 15 2017 12:00:00 AM 17")) - result = tk.MustQuery("SELECT DATE_FORMAT(151113102019.12, '%W %M %e %Y %r %y');") + result = tk.MustQuery(`SELECT DATE_FORMAT(151113102019.12, '%W %M %e %Y %r %y');`) result.Check(testkit.Rows("Friday November 13 2015 10:20:19 AM 15")) - result = tk.MustQuery("SELECT DATE_FORMAT('0000-00-00', '%W %M %e %Y %r %y');") + result = tk.MustQuery(`SELECT DATE_FORMAT('0000-00-00', '%W %M %e %Y %r %y');`) result.Check(testkit.Rows("")) // for yearweek @@ -1451,11 +1451,11 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { result = tk.MustQuery("SELECT UNIX_TIMESTAMP(20151113102019);") result.Check(testkit.Rows("1447410019")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2015-11-13 10:20:19');") - result.Check(testkit.Rows("1447410019.000000")) + result.Check(testkit.Rows("1447410019")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2015-11-13 10:20:19.012');") - result.Check(testkit.Rows("1447410019.012000")) + result.Check(testkit.Rows("1447410019.012")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 00:00:00');") - result.Check(testkit.Rows("0.000000")) + result.Check(testkit.Rows("0")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1969-12-31 23:59:59');") result.Check(testkit.Rows("0")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-13-01 00:00:00');") @@ -1470,14 +1470,16 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { result.Check(testkit.Rows("0")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP(12345);") result.Check(testkit.Rows("0")) + result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2017-01-01')") + result.Check(testkit.Rows("1483228800")) // Test different time zone. tk.MustExec("SET time_zone = '+08:00';") result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 00:00:00');") result.Check(testkit.Rows("0")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 08:00:00');") - result.Check(testkit.Rows("0.000000")) - result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2015-11-13 18:20:19.012');") - result.Check(testkit.Rows("1447410019.012000")) + result.Check(testkit.Rows("0")) + result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2015-11-13 18:20:19.012'), UNIX_TIMESTAMP('2015-11-13 18:20:19.0123');") + result.Check(testkit.Rows("1447410019.012 1447410019.0123")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2038-01-19 11:14:07.999999');") result.Check(testkit.Rows("2147483647.999999")) @@ -2150,7 +2152,7 @@ func (s *testIntegrationSuite) TestBuiltin(c *C) { {"a", "a", 1}, {"a", "b", 0}, {"aA", "Aa", 0}, - {"aA%", "aAab", 1}, + {`aA%`, "aAab", 1}, {"aA_", "Aaab", 0}, {"Aa_", "Aab", 1}, {"", "", 1}, diff --git a/expression/scalar_function.go b/expression/scalar_function.go index 40aa20c15a8c2..b3742176aae88 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -116,35 +116,12 @@ func ScalarFuncs2Exprs(funcs []*ScalarFunction) []Expression { // Clone implements Expression interface. func (sf *ScalarFunction) Clone() Expression { - newArgs := make([]Expression, 0, len(sf.GetArgs())) - for _, arg := range sf.GetArgs() { - newArgs = append(newArgs, arg.Clone()) - } - switch sf.FuncName.L { - case ast.Cast: - return BuildCastFunction(sf.GetCtx(), sf.GetArgs()[0], sf.GetType()) - case ast.Values: - var offset int - switch sf.GetType().EvalType() { - case types.ETInt: - offset = sf.Function.(*builtinValuesIntSig).offset - case types.ETReal: - offset = sf.Function.(*builtinValuesRealSig).offset - case types.ETDecimal: - offset = sf.Function.(*builtinValuesDecimalSig).offset - case types.ETString: - offset = sf.Function.(*builtinValuesStringSig).offset - case types.ETDatetime, types.ETTimestamp: - offset = sf.Function.(*builtinValuesTimeSig).offset - case types.ETDuration: - offset = sf.Function.(*builtinValuesDurationSig).offset - case types.ETJson: - offset = sf.Function.(*builtinValuesJSONSig).offset - } - return NewValuesFunc(sf.GetCtx(), offset, sf.GetType()) + return &ScalarFunction{ + FuncName: sf.FuncName, + RetType: sf.RetType, + Function: sf.Function.Clone(), + hashcode: sf.hashcode, } - newFunc := NewFunctionInternal(sf.GetCtx(), sf.FuncName.L, sf.RetType, newArgs...) - return newFunc } // GetType implements Expression interface. diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 5ea9e68832695..6725b15c05c57 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -183,7 +183,7 @@ func (s *testInferTypeSuite) createTestCase4Constants() []typeInferTestCase { func (s *testInferTypeSuite) createTestCase4Cast() []typeInferTestCase { return []typeInferTestCase{ {"CAST(c_int_d AS BINARY)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, -1, -1}, // TODO: Flen should be 11. - {"CAST(c_int_d AS BINARY(5))", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 5, -1}, + {"CAST(c_int_d AS BINARY(5))", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 5, -1}, {"CAST(c_int_d AS CHAR)", mysql.TypeVarString, charset.CharsetUTF8, 0, -1, -1}, // TODO: Flen should be 11. {"CAST(c_int_d AS CHAR(5))", mysql.TypeVarString, charset.CharsetUTF8, 0, 5, -1}, {"CAST(c_int_d AS DATE)", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, @@ -220,8 +220,8 @@ func (s *testInferTypeSuite) createTestCase4Columns() []typeInferTestCase { {"c_datetime_d ", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 19, 0}, {"c_time ", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 14, 3}, {"c_time_d ", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, - {"c_timestamp ", mysql.TypeTimestamp, charset.CharsetBin, mysql.NotNullFlag | mysql.BinaryFlag | mysql.TimestampFlag, 24, 4}, - {"c_timestamp_d", mysql.TypeTimestamp, charset.CharsetBin, mysql.NotNullFlag | mysql.BinaryFlag | mysql.TimestampFlag, 19, 0}, + {"c_timestamp ", mysql.TypeTimestamp, charset.CharsetBin, mysql.BinaryFlag, 24, 4}, + {"c_timestamp_d", mysql.TypeTimestamp, charset.CharsetBin, mysql.BinaryFlag, 19, 0}, {"c_char ", mysql.TypeString, charset.CharsetUTF8, 0, 20, 0}, // TODO: flag should be BinaryFlag {"c_bchar ", mysql.TypeString, charset.CharsetUTF8, mysql.BinaryFlag, 20, 0}, {"c_varchar ", mysql.TypeVarchar, charset.CharsetUTF8, 0, 20, 0}, // TODO: BinaryFlag, tp should be TypeVarString @@ -273,7 +273,7 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { {"trim(c_binary)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength}, {"ascii(c_char)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 3, 0}, {"ord(c_char)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, - {"c_int_d like 'abc%'", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, + {`c_int_d like 'abc%'`, mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, {"tidb_version()", mysql.TypeVarString, charset.CharsetUTF8, 0, len(printer.GetTiDBInfo()), types.UnspecifiedLength}, {"password(c_char)", mysql.TypeVarString, charset.CharsetUTF8, 0, mysql.PWDHashLen + 1, types.UnspecifiedLength}, {"elt(c_int_d, c_char, c_char, c_char)", mysql.TypeVarString, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength}, @@ -798,7 +798,7 @@ func (s *testInferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase { {"ifnull(c_int_d, c_binary)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength}, {"ifnull(c_char, c_binary)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength}, {"ifnull(null, null)", mysql.TypeNull, charset.CharsetBin, mysql.BinaryFlag, 0, types.UnspecifiedLength}, - {"ifnull(c_double_d, c_timestamp_d)", mysql.TypeVarchar, charset.CharsetUTF8, mysql.NotNullFlag, 22, types.UnspecifiedLength}, + {"ifnull(c_double_d, c_timestamp_d)", mysql.TypeVarchar, charset.CharsetUTF8, 0, 22, types.UnspecifiedLength}, {"ifnull(c_json, c_decimal)", mysql.TypeLongBlob, charset.CharsetUTF8, 0, math.MaxUint32, types.UnspecifiedLength}, {"if(c_int_d, c_decimal, c_int_d)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 14, 3}, {"if(c_int_d, c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8, mysql.BinaryFlag, 20, types.UnspecifiedLength}, @@ -1013,9 +1013,9 @@ func (s *testInferTypeSuite) createTestCase4CompareFuncs() []typeInferTestCase { {"nullif(c_float_d , 123)", mysql.TypeFloat, charset.CharsetBin, mysql.BinaryFlag, 12, types.UnspecifiedLength}, // TODO: tp should be TypeDouble {"nullif(c_double_d , 123)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, types.UnspecifiedLength}, {"nullif(c_decimal , 123)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 6, 3}, - {"nullif(c_datetime , 123)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 22, 2}, // TODO: tp should be TypeVarString, no binary flag - {"nullif(c_time_d , 123)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, // TODO: tp should be TypeVarString, no binary flag - {"nullif(c_timestamp_d, 123)", mysql.TypeTimestamp, charset.CharsetBin, mysql.BinaryFlag | mysql.TimestampFlag | mysql.NotNullFlag, 19, 0}, // TODO: tp should be TypeVarString, no binary flag, no timestamp flag, no not null flag + {"nullif(c_datetime , 123)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 22, 2}, // TODO: tp should be TypeVarString, no binary flag + {"nullif(c_time_d , 123)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, // TODO: tp should be TypeVarString, no binary flag + {"nullif(c_timestamp_d, 123)", mysql.TypeTimestamp, charset.CharsetBin, mysql.BinaryFlag, 19, 0}, // TODO: tp should be TypeVarString, no binary flag {"nullif(c_char , 123)", mysql.TypeString, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength}, {"nullif(c_varchar , 123)", mysql.TypeVarchar, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength}, // TODO: tp should be TypeVarString {"nullif(c_text_d , 123)", mysql.TypeBlob, charset.CharsetUTF8, 0, 65535, types.UnspecifiedLength}, // TODO: tp should be TypeMediumBlob @@ -1191,14 +1191,14 @@ func (s *testInferTypeSuite) createTestCase4OtherFuncs() []typeInferTestCase { func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { return []typeInferTestCase{ - {"time_format('150:02:28', '%r%r%r%r')", mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, - {"time_format(123456, '%r%r%r%r')", mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, - {"time_format('bad string', '%r%r%r%r')", mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, - {"time_format(null, '%r%r%r%r')", mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, + {`time_format('150:02:28', '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, + {`time_format(123456, '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, + {`time_format('bad string', '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, + {`time_format(null, '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, - {"date_format(null, '%r%r%r%r')", mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, - {"date_format('2017-06-15', '%r%r%r%r')", mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, - {"date_format(151113102019.12, '%r%r%r%r')", mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, + {`date_format(null, '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, + {`date_format('2017-06-15', '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, + {`date_format(151113102019.12, '%r%r%r%r')`, mysql.TypeVarString, charset.CharsetUTF8, 0, 44, types.UnspecifiedLength}, {"timestampadd(HOUR, c_int_d, c_timestamp_d)", mysql.TypeString, charset.CharsetUTF8, 0, 19, types.UnspecifiedLength}, {"timestampadd(minute, c_double_d, c_timestamp_d)", mysql.TypeString, charset.CharsetUTF8, 0, 19, types.UnspecifiedLength}, @@ -1231,6 +1231,9 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"unix_timestamp(c_blob_d )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6}, {"unix_timestamp(c_set )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6}, {"unix_timestamp(c_enum )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6}, + {"unix_timestamp(null )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, + {"unix_timestamp('12:12:12.123')", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 15, 3}, + {"unix_timestamp('12:12:12.1234')", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 16, 4}, // TODO: Add string literal tests for UNIX_TIMESTAMP. UNIX_TIMESTAMP respects the fsp in string literals. {"timestampdiff(MONTH, c_datetime, c_datetime)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, diff --git a/kv/txn.go b/kv/txn.go index 95149989bb184..5bea8f384ae94 100644 --- a/kv/txn.go +++ b/kv/txn.go @@ -38,24 +38,26 @@ func RunInNewTxn(store Storage, retryable bool, f func(txn Transaction) error) e return errors.Trace(err) } + // originalTxnTS is used to trace the original transaction when the function is retryable. if i == 0 { originalTxnTS = txn.StartTS() } err = f(txn) - if retryable && IsRetryableError(err) { - log.Warnf("[kv] Retry txn %v original txn %v err %v", txn, originalTxnTS, err) - err1 := txn.Rollback() - terror.Log(errors.Trace(err1)) - continue - } if err != nil { err1 := txn.Rollback() terror.Log(errors.Trace(err1)) + if retryable && IsRetryableError(err) { + log.Warnf("[kv] Retry txn %v original txn %v err %v", txn, originalTxnTS, err) + continue + } return errors.Trace(err) } err = txn.Commit(context.Background()) + if err == nil { + break + } if retryable && IsRetryableError(err) { log.Warnf("[kv] Retry txn %v original txn %v err %v", txn, originalTxnTS, err) err1 := txn.Rollback() @@ -63,10 +65,7 @@ func RunInNewTxn(store Storage, retryable bool, f func(txn Transaction) error) e BackOff(i) continue } - if err != nil { - return errors.Trace(err) - } - break + return errors.Trace(err) } return errors.Trace(err) } @@ -89,3 +88,37 @@ func BackOff(attempts uint) int { time.Sleep(sleep) return int(sleep) } + +// BatchGetValues gets values in batch. +// The values from buffer in transaction and the values from the storage node are merged together. +func BatchGetValues(txn Transaction, keys []Key) (map[string][]byte, error) { + if txn.IsReadOnly() { + return txn.GetSnapshot().BatchGet(keys) + } + bufferValues := make([][]byte, len(keys)) + shrinkKeys := make([]Key, 0, len(keys)) + for i, key := range keys { + val, err := txn.GetMemBuffer().Get(key) + if IsErrNotFound(err) { + shrinkKeys = append(shrinkKeys, key) + continue + } + if err != nil { + return nil, errors.Trace(err) + } + if len(val) != 0 { + bufferValues[i] = val + } + } + storageValues, err := txn.GetSnapshot().BatchGet(shrinkKeys) + if err != nil { + return nil, errors.Trace(err) + } + for i, key := range keys { + if bufferValues[i] == nil { + continue + } + storageValues[string(key)] = bufferValues[i] + } + return storageValues, nil +} diff --git a/meta/meta.go b/meta/meta.go index 286d1d1ad3fb5..0c0b4309b8e8d 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -80,8 +80,9 @@ var ( // Meta is for handling meta information in a transaction. type Meta struct { - txn *structure.TxStructure - StartTS uint64 // StartTS is the txn's start TS. + txn *structure.TxStructure + StartTS uint64 // StartTS is the txn's start TS. + jobListKey JobListKeyType } // NewMeta creates a Meta in transaction txn. @@ -89,7 +90,7 @@ func NewMeta(txn kv.Transaction) *Meta { txn.SetOption(kv.Priority, kv.PriorityHigh) txn.SetOption(kv.SyncLog, true) t := structure.NewStructure(txn, txn, mMetaPrefix) - return &Meta{txn: t, StartTS: txn.StartTS()} + return &Meta{txn: t, StartTS: txn.StartTS(), jobListKey: DefaultJobListKey} } // NewSnapshotMeta creates a Meta with snapshot. @@ -434,7 +435,6 @@ func (m *Meta) GetTable(dbID int64, tableID int64) (*model.TableInfo, error) { } // DDL job structure -// DDLOnwer: []byte // DDLJobList: list jobs // DDLJobHistory: hash // DDLJobReorg: hash @@ -444,10 +444,26 @@ func (m *Meta) GetTable(dbID int64, tableID int64) (*model.TableInfo, error) { var ( mDDLJobListKey = []byte("DDLJobList") + mDDLJobAddIdxList = []byte("DDLJobAddIdxList") mDDLJobHistoryKey = []byte("DDLJobHistory") mDDLJobReorgKey = []byte("DDLJobReorg") ) +// JobListKeyType is a key type of the DDL job queue. +type JobListKeyType []byte + +var ( + // DefaultJobListKey keeps all actions of DDL jobs. + DefaultJobListKey JobListKeyType = mDDLJobListKey + // AddIndexJobListKey only keeps the action of adding index. + AddIndexJobListKey JobListKeyType = mDDLJobAddIdxList +) + +// SetJobListKey sets the job list key. +func (m *Meta) SetJobListKey(key []byte) { + m.jobListKey = key +} + func (m *Meta) enQueueDDLJob(key []byte, job *model.Job) error { b, err := job.Encode(true) if err != nil { @@ -458,7 +474,7 @@ func (m *Meta) enQueueDDLJob(key []byte, job *model.Job) error { // EnQueueDDLJob adds a DDL job to the list. func (m *Meta) EnQueueDDLJob(job *model.Job) error { - return m.enQueueDDLJob(mDDLJobListKey, job) + return m.enQueueDDLJob(m.jobListKey, job) } func (m *Meta) deQueueDDLJob(key []byte) (*model.Job, error) { @@ -474,7 +490,7 @@ func (m *Meta) deQueueDDLJob(key []byte) (*model.Job, error) { // DeQueueDDLJob pops a DDL job from the list. func (m *Meta) DeQueueDDLJob() (*model.Job, error) { - return m.deQueueDDLJob(mDDLJobListKey) + return m.deQueueDDLJob(m.jobListKey) } func (m *Meta) getDDLJob(key []byte, index int64) (*model.Job, error) { @@ -491,7 +507,7 @@ func (m *Meta) getDDLJob(key []byte, index int64) (*model.Job, error) { // GetDDLJob returns the DDL job with index. func (m *Meta) GetDDLJob(index int64) (*model.Job, error) { startTime := time.Now() - job, err := m.getDDLJob(mDDLJobListKey, index) + job, err := m.getDDLJob(m.jobListKey, index) metrics.MetaHistogram.WithLabelValues(metrics.GetDDLJob, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) return job, errors.Trace(err) } @@ -510,14 +526,34 @@ func (m *Meta) updateDDLJob(index int64, job *model.Job, key []byte, updateRawAr // updateRawArgs is used to determine whether to update the raw args when encode the job. func (m *Meta) UpdateDDLJob(index int64, job *model.Job, updateRawArgs bool) error { startTime := time.Now() - err := m.updateDDLJob(index, job, mDDLJobListKey, updateRawArgs) + err := m.updateDDLJob(index, job, m.jobListKey, updateRawArgs) metrics.MetaHistogram.WithLabelValues(metrics.UpdateDDLJob, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) return errors.Trace(err) } // DDLJobQueueLen returns the DDL job queue length. func (m *Meta) DDLJobQueueLen() (int64, error) { - return m.txn.LLen(mDDLJobListKey) + return m.txn.LLen(m.jobListKey) +} + +// GetAllDDLJobs gets all DDL Jobs. +func (m *Meta) GetAllDDLJobs() ([]*model.Job, error) { + values, err := m.txn.LGetAll(mDDLJobListKey) + if err != nil || values == nil { + return nil, errors.Trace(err) + } + + jobs := make([]*model.Job, 0, len(values)) + for _, val := range values { + job := &model.Job{} + err = job.Decode(val) + if err != nil { + return nil, errors.Trace(err) + } + jobs = append(jobs, job) + } + + return jobs, nil } func (m *Meta) jobIDKey(id int64) []byte { diff --git a/meta/meta_test.go b/meta/meta_test.go index 23dbdfd827136..d514752dc082c 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -311,6 +311,16 @@ func (s *testSuite) TestDDL(c *C) { lastID = job.ID } + // Test GetAllDDLJobs. + err = t.EnQueueDDLJob(job) + job1 := &model.Job{ID: 2} + err = t.EnQueueDDLJob(job1) + c.Assert(err, IsNil) + jobs, err := t.GetAllDDLJobs() + c.Assert(err, IsNil) + expectJobs := []*model.Job{job, job1} + c.Assert(jobs, DeepEquals, expectJobs) + err = txn.Commit(context.Background()) c.Assert(err, IsNil) } diff --git a/metrics/server.go b/metrics/server.go index ed86109d42388..8c5f5122e56c9 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -94,6 +94,14 @@ var ( Name: "keep_alive_total", Help: "Counter of TiDB keep alive.", }) + + PlanCacheCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "plan_cache_total", + Help: "Counter of query using plan cache.", + }, []string{LblType}) ) func init() { @@ -105,6 +113,7 @@ func init() { prometheus.MustRegister(ServerEventCounter) prometheus.MustRegister(TimeJumpBackCounter) prometheus.MustRegister(KeepAliveCounter) + prometheus.MustRegister(PlanCacheCounter) } // ExecuteErrorToLabel converts an execute error to label. diff --git a/metrics/stats.go b/metrics/stats.go index 5456248410e78..0642972faf340 100644 --- a/metrics/stats.go +++ b/metrics/stats.go @@ -52,6 +52,22 @@ var ( Name: "pseudo_estimation_total", Help: "Counter of pseudo estimation caused by outdated stats.", }) + + DumpFeedbackCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "statistics", + Name: "dump_feedback_total", + Help: "Counter of dumping feedback.", + }, []string{LblType}) + + UpdateStatsCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "statistics", + Name: "update_stats_total", + Help: "Counter of updating stats using feedback.", + }, []string{LblType}) ) func init() { @@ -59,4 +75,6 @@ func init() { prometheus.MustRegister(AutoAnalyzeCounter) prometheus.MustRegister(StatsInaccuracyRate) prometheus.MustRegister(PseudoEstimation) + prometheus.MustRegister(DumpFeedbackCounter) + prometheus.MustRegister(UpdateStatsCounter) } diff --git a/model/ddl.go b/model/ddl.go index 0c4fb4a5b36ee..fe8481d75ca62 100644 --- a/model/ddl.go +++ b/model/ddl.go @@ -28,42 +28,44 @@ type ActionType byte // List DDL actions. const ( - ActionNone ActionType = 0 - ActionCreateSchema ActionType = 1 - ActionDropSchema ActionType = 2 - ActionCreateTable ActionType = 3 - ActionDropTable ActionType = 4 - ActionAddColumn ActionType = 5 - ActionDropColumn ActionType = 6 - ActionAddIndex ActionType = 7 - ActionDropIndex ActionType = 8 - ActionAddForeignKey ActionType = 9 - ActionDropForeignKey ActionType = 10 - ActionTruncateTable ActionType = 11 - ActionModifyColumn ActionType = 12 - ActionRebaseAutoID ActionType = 13 - ActionRenameTable ActionType = 14 - ActionSetDefaultValue ActionType = 15 - ActionShardRowID ActionType = 16 + ActionNone ActionType = 0 + ActionCreateSchema ActionType = 1 + ActionDropSchema ActionType = 2 + ActionCreateTable ActionType = 3 + ActionDropTable ActionType = 4 + ActionAddColumn ActionType = 5 + ActionDropColumn ActionType = 6 + ActionAddIndex ActionType = 7 + ActionDropIndex ActionType = 8 + ActionAddForeignKey ActionType = 9 + ActionDropForeignKey ActionType = 10 + ActionTruncateTable ActionType = 11 + ActionModifyColumn ActionType = 12 + ActionRebaseAutoID ActionType = 13 + ActionRenameTable ActionType = 14 + ActionSetDefaultValue ActionType = 15 + ActionShardRowID ActionType = 16 + ActionModifyTableComment ActionType = 17 ) var actionMap = map[ActionType]string{ - ActionCreateSchema: "create schema", - ActionDropSchema: "drop schema", - ActionCreateTable: "create table", - ActionDropTable: "drop table", - ActionAddColumn: "add column", - ActionDropColumn: "drop column", - ActionAddIndex: "add index", - ActionDropIndex: "drop index", - ActionAddForeignKey: "add foreign key", - ActionDropForeignKey: "drop foreign key", - ActionTruncateTable: "truncate table", - ActionModifyColumn: "modify column", - ActionRebaseAutoID: "rebase auto_increment ID", - ActionRenameTable: "rename table", - ActionSetDefaultValue: "set default value", - ActionShardRowID: "shard row ID", + ActionCreateSchema: "create schema", + ActionDropSchema: "drop schema", + ActionCreateTable: "create table", + ActionDropTable: "drop table", + ActionAddColumn: "add column", + ActionDropColumn: "drop column", + ActionAddIndex: "add index", + ActionDropIndex: "drop index", + ActionAddForeignKey: "add foreign key", + ActionDropForeignKey: "drop foreign key", + ActionTruncateTable: "truncate table", + ActionModifyColumn: "modify column", + ActionRebaseAutoID: "rebase auto_increment ID", + ActionRenameTable: "rename table", + ActionSetDefaultValue: "set default value", + ActionShardRowID: "shard row ID", + ActionModifyTableComment: "modify table comment", } // String return current ddl action in string @@ -125,6 +127,8 @@ type Job struct { // StartTS uses timestamp allocated by TSO. // Now it's the TS when we put the job to TiKV queue. StartTS uint64 `json:"start_ts"` + // DependencyID is the job's ID that the current job depends on. + DependencyID int64 `json:"dependency_id"` // Query string of the ddl job. Query string `json:"query"` BinlogInfo *HistoryInfo `json:"binlog"` @@ -211,6 +215,44 @@ func (job *Job) String() string { job.ID, job.Type, job.State, job.SchemaState, job.SchemaID, job.TableID, rowCount, len(job.Args), tsConvert2Time(job.StartTS), job.Error, job.ErrorCount, job.SnapshotVer) } +func (job *Job) hasDependentSchema(other *Job) (bool, error) { + if other.Type == ActionDropSchema || other.Type == ActionCreateSchema { + if other.SchemaID == job.SchemaID { + return true, nil + } + if job.Type == ActionRenameTable { + var oldSchemaID int64 + if err := job.DecodeArgs(&oldSchemaID); err != nil { + return false, errors.Trace(err) + } + if other.SchemaID == oldSchemaID { + return true, nil + } + } + } + return false, nil +} + +// IsDependentOn returns whether the job depends on "other". +// How to check the job depends on "other"? +// 1. The two jobs handle the same database when one of the two jobs is an ActionDropSchema or ActionCreateSchema type. +// 2. Or the two jobs handle the same table. +func (job *Job) IsDependentOn(other *Job) (bool, error) { + isDependent, err := job.hasDependentSchema(other) + if err != nil || isDependent { + return isDependent, errors.Trace(err) + } + isDependent, err = other.hasDependentSchema(job) + if err != nil || isDependent { + return isDependent, errors.Trace(err) + } + + if other.TableID == job.TableID { + return true, nil + } + return false, nil +} + // IsFinished returns whether job is finished or not. // If the job state is Done or Cancelled, it is finished. func (job *Job) IsFinished() bool { @@ -219,7 +261,12 @@ func (job *Job) IsFinished() bool { // IsCancelled returns whether the job is cancelled or not. func (job *Job) IsCancelled() bool { - return job.State == JobStateCancelled || job.State == JobStateRollbackDone + return job.State == JobStateCancelled +} + +// IsRollbackDone returns whether the job is rolled back or not. +func (job *Job) IsRollbackDone() bool { + return job.State == JobStateRollbackDone } // IsRollingback returns whether the job is rolling back or not. diff --git a/model/model.go b/model/model.go index ef23b415b1bfb..f3c6e2f6cf5d7 100644 --- a/model/model.go +++ b/model/model.go @@ -120,6 +120,8 @@ type TableInfo struct { // ShardRowIDBits specify if the implicit row ID is sharded. ShardRowIDBits uint64 + + Partition *PartitionInfo } // GetUpdateTime gets the table's updating time. @@ -221,6 +223,48 @@ func (t *TableInfo) ColumnIsInIndex(c *ColumnInfo) bool { return false } +// PartitionType is the type for PartitionInfo +type PartitionType int + +// Partition types. +const ( + PartitionTypeRange PartitionType = 1 + PartitionTypeHash PartitionType = 2 + PartitionTypeList PartitionType = 3 +) + +func (p PartitionType) String() string { + switch p { + case PartitionTypeRange: + return "RANGE" + case PartitionTypeHash: + return "HASH" + case PartitionTypeList: + return "LIST" + default: + return "" + } + +} + +// PartitionInfo provides table partition info. +type PartitionInfo struct { + Type PartitionType + Expr string + Columns []CIStr + + Definitions []PartitionDefinition +} + +// PartitionDefinition defines a single partition. +type PartitionDefinition struct { + ID int64 + Name string + LessThan []string + Comment string `json:"omit_empty"` + MaxValue bool +} + // IndexColumn provides index column info. type IndexColumn struct { Name CIStr `json:"name"` // Index name diff --git a/model/model_test.go b/model/model_test.go index 35df690616dbb..f0697ab0cdd64 100644 --- a/model/model_test.go +++ b/model/model_test.go @@ -14,6 +14,7 @@ package model import ( + "encoding/json" "fmt" "testing" "time" @@ -142,12 +143,44 @@ func (*testModelSuite) TestJobCodec(c *C) { } job := &Job{ ID: 1, + TableID: 2, + SchemaID: 1, BinlogInfo: &HistoryInfo{}, Args: []interface{}{NewCIStr("a"), A{Name: "abc"}}, } job.BinlogInfo.AddDBInfo(123, &DBInfo{ID: 1, Name: NewCIStr("test_history_db")}) job.BinlogInfo.AddTableInfo(123, &TableInfo{ID: 1, Name: NewCIStr("test_history_tbl")}) + // Test IsDependentOn. + // job: table ID is 2 + // job1: table ID is 2 + var err error + job1 := &Job{ + ID: 2, + TableID: 2, + SchemaID: 1, + Type: ActionRenameTable, + BinlogInfo: &HistoryInfo{}, + Args: []interface{}{int64(3), NewCIStr("new_table_name")}, + } + job1.RawArgs, err = json.Marshal(job1.Args) + c.Assert(err, IsNil) + isDependent, err := job.IsDependentOn(job1) + c.Assert(err, IsNil) + c.Assert(isDependent, IsTrue) + // job1: rename table, old schema ID is 3 + // job2: create schema, schema ID is 3 + job2 := &Job{ + ID: 3, + TableID: 3, + SchemaID: 3, + Type: ActionCreateSchema, + BinlogInfo: &HistoryInfo{}, + } + isDependent, err = job2.IsDependentOn(job1) + c.Assert(err, IsNil) + c.Assert(isDependent, IsTrue) + c.Assert(job.IsCancelled(), Equals, false) b, err := job.Encode(false) c.Assert(err, IsNil) @@ -195,6 +228,7 @@ func (*testModelSuite) TestJobCodec(c *C) { c.Assert(job.IsFinished(), IsTrue) c.Assert(job.IsRunning(), IsFalse) c.Assert(job.IsSynced(), IsFalse) + c.Assert(job.IsRollbackDone(), IsFalse) job.SetRowCount(3) c.Assert(job.GetRowCount(), Equals, int64(3)) } diff --git a/mysql/const_test.go b/mysql/const_test.go index 56d3117827e4c..dd2c34d2c1963 100644 --- a/mysql/const_test.go +++ b/mysql/const_test.go @@ -38,7 +38,7 @@ var _ = Suite(&testMySQLConstSuite{}) type testMySQLConstSuite struct { cluster *mocktikv.Cluster - mvccStore *mocktikv.MvccStore + mvccStore mocktikv.MVCCStore store kv.Storage *parser.Parser } @@ -52,7 +52,7 @@ func (s *testMySQLConstSuite) SetUpSuite(c *C) { if useMockTikv { s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.NewMvccStore() + s.mvccStore = mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(s.cluster), mockstore.WithMVCCStore(s.mvccStore), diff --git a/parser/misc.go b/parser/misc.go index 3e2ce62c28983..1c41bd47a566e 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -178,6 +178,7 @@ var tokenMap = map[string]int{ "CHARSET": charsetKwd, "CHECK": check, "CHECKSUM": checksum, + "CLEANUP": cleanup, "CLIENT": client, "COALESCE": coalesce, "COLLATE": collate, diff --git a/parser/parser.y b/parser/parser.y index a14b3cccd57e6..c199b2c009bcd 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -260,6 +260,7 @@ import ( cascaded "CASCADED" charsetKwd "CHARSET" checksum "CHECKSUM" + cleanup "CLEANUP" client "CLIENT" coalesce "COALESCE" collation "COLLATION" @@ -665,6 +666,7 @@ import ( PartitionDefinitionListOpt "Partition definition list option" PartitionOpt "Partition option" PartitionNumOpt "PARTITION NUM option" + PartDefCommentOpt "Partition comment" PartDefValuesOpt "VALUES {LESS THAN {(expr | value_list) | MAXVALUE} | IN {value_list}" PartDefStorageOpt "ENGINE = xxx or empty" PasswordOpt "Password option" @@ -1702,12 +1704,17 @@ CreateTableStmt: yylex.Errorf("Column Definition List can't be empty.") return 1 } + var part *ast.PartitionOptions + if $9 != nil { + part = $9.(*ast.PartitionOptions) + } $$ = &ast.CreateTableStmt{ Table: $4.(*ast.TableName), IfNotExists: $3.(bool), Cols: columnDefs, Constraints: constraints, Options: $8.([]*ast.TableOption), + Partition: part, } } | "CREATE" "TABLE" IfNotExists TableName "LIKE" TableName @@ -1724,15 +1731,41 @@ DefaultKwdOpt: | "DEFAULT" PartitionOpt: - {} + { + $$ = nil + } | "PARTITION" "BY" "KEY" '(' ColumnNameList ')' PartitionNumOpt PartitionDefinitionListOpt - {} + { + $$ = nil + } | "PARTITION" "BY" "HASH" '(' Expression ')' PartitionNumOpt PartitionDefinitionListOpt - {} + { + $$ = nil + } | "PARTITION" "BY" "RANGE" '(' Expression ')' PartitionNumOpt PartitionDefinitionListOpt - {} + { + var defs []*ast.PartitionDefinition + if $8 != nil { + defs = $8.([]*ast.PartitionDefinition) + } + $$ = &ast.PartitionOptions{ + Tp: model.PartitionTypeRange, + Expr: $5.(ast.ExprNode), + Definitions: defs, + } + } | "PARTITION" "BY" "RANGE" "COLUMNS" '(' ColumnNameList ')' PartitionNumOpt PartitionDefinitionListOpt - {} + { + var defs []*ast.PartitionDefinition + if $9 != nil { + defs = $9.([]*ast.PartitionDefinition) + } + $$ = &ast.PartitionOptions{ + Tp: model.PartitionTypeRange, + ColumnNames: $6.([]*ast.ColumnName), + Definitions: defs, + } + } PartitionNumOpt: {} @@ -1740,33 +1773,65 @@ PartitionNumOpt: {} PartitionDefinitionListOpt: - {} + { + $$ = nil + } | '(' PartitionDefinitionList ')' - {} + { + $$ = $2.([]*ast.PartitionDefinition) + } PartitionDefinitionList: PartitionDefinition - {} + { + $$ = []*ast.PartitionDefinition{$1.(*ast.PartitionDefinition)} + } | PartitionDefinitionList ',' PartitionDefinition - {} + { + $$ = append($1.([]*ast.PartitionDefinition), $3.(*ast.PartitionDefinition)) + } PartitionDefinition: "PARTITION" Identifier PartDefValuesOpt PartDefCommentOpt PartDefStorageOpt - {} + { + partDef := &ast.PartitionDefinition{ + Name: $2, + Comment: $4.(string), + } + switch $3.(type) { + case []ast.ExprNode: + partDef.LessThan = $3.([]ast.ExprNode) + case bool: + partDef.MaxValue = true + } + $$ = partDef + } PartDefCommentOpt: - {} + { + $$ = "" + } | "COMMENT" eq stringLit - {} + { + $$ = $3 + } PartDefValuesOpt: - {} + { + $$ = nil + } | "VALUES" "LESS" "THAN" "MAXVALUE" - {} + { + $$ = true + } | "VALUES" "LESS" "THAN" '(' "MAXVALUE" ')' - {} + { + $$ = true + } | "VALUES" "LESS" "THAN" '(' ExpressionList ')' - {} + { + $$ = $5 + } PartDefStorageOpt: {} @@ -2532,7 +2597,7 @@ Identifier: identifier | UnReservedKeyword | NotKeywordToken | TiDBKeyword UnReservedKeyword: - "ACTION" | "ASCII" | "AUTO_INCREMENT" | "AFTER" | "ALWAYS" | "AVG" | "BEGIN" | "BIT" | "BOOL" | "BOOLEAN" | "BTREE" | "BYTE" | "CHARSET" + "ACTION" | "ASCII" | "AUTO_INCREMENT" | "AFTER" | "ALWAYS" | "AVG" | "BEGIN" | "BIT" | "BOOL" | "BOOLEAN" | "BTREE" | "BYTE" | "CLEANUP" | "CHARSET" | "COLUMNS" | "COMMIT" | "COMPACT" | "COMPRESSED" | "CONSISTENT" | "DATA" | "DATE" %prec lowerThanStringLitToken| "DATETIME" | "DAY" | "DEALLOCATE" | "DO" | "DUPLICATE" | "DYNAMIC"| "END" | "ENGINE" | "ENGINES" | "ENUM" | "ESCAPE" | "EXECUTE" | "FIELDS" | "FIRST" | "FIXED" | "FLUSH" | "FORMAT" | "FULL" |"GLOBAL" | "HASH" | "HOUR" | "LESS" | "LOCAL" | "NAMES" | "OFFSET" | "PASSWORD" %prec lowerThanEq | "PREPARE" | "QUICK" | "REDUNDANT" @@ -3709,6 +3774,9 @@ CastType: { x := types.NewFieldType(mysql.TypeVarString) x.Flen = $2.(int) // TODO: Flen should be the flen of expression + if x.Flen != types.UnspecifiedLength { + x.Tp = mysql.TypeString + } x.Charset = charset.CharsetBin x.Collate = charset.CollationBin x.Flag |= mysql.BinaryFlag @@ -4556,7 +4624,14 @@ SetStmt: } | "SET" "TRANSACTION" TransactionChars { - $$ = &ast.SetStmt{Variables: $3.([]*ast.VariableAssignment)} + assigns := $3.([]*ast.VariableAssignment) + for i:=0; i,], keep order:false 2.00", + "TableScan_6 cop table:t, keep order:false 2.00", + "IndexLookUp_7 root index:IndexScan_5, table:TableScan_6 2.00", + )) + h := dom.StatsHandle() + h.Clear() + h.Lease = 1 + defer func() { h.Lease = 0 }() + c.Assert(h.Update(dom.InfoSchema()), IsNil) + testKit.MustQuery("explain select * from t where b = 1").Check(testkit.Rows( + "TableScan_5 Selection_6 cop table:t, range:[-inf,+inf], keep order:false 2.00", + "Selection_6 TableScan_5 cop eq(test.t.b, 1) 0.00", + "TableReader_7 root data:Selection_6 0.00", + )) + testKit.MustQuery("explain select * from t where b < 1").Check(testkit.Rows( + "TableScan_5 Selection_6 cop table:t, range:[-inf,+inf], keep order:false 2.00", + "Selection_6 TableScan_5 cop lt(test.t.b, 1) 0.00", + "TableReader_7 root data:Selection_6 0.00", + )) +} + func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { store, err := mockstore.NewMockTikvStore() if err != nil { diff --git a/plan/common_plans.go b/plan/common_plans.go index eae7fa569004c..cfefc40c68d9e 100644 --- a/plan/common_plans.go +++ b/plan/common_plans.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" @@ -64,6 +65,14 @@ type RecoverIndex struct { IndexName string } +// CleanupIndex is used to delete dangling index data. +type CleanupIndex struct { + baseSchemaProducer + + Table *ast.TableName + IndexName string +} + // CheckIndex is used for checking index data, built from the 'admin check index' statement. type CheckIndex struct { baseSchemaProducer @@ -175,6 +184,7 @@ func (e *Execute) getPhysicalPlan(ctx sessionctx.Context, is infoschema.InfoSche if prepared.UseCache { cacheKey = NewPSTMTPlanCacheKey(sessionVars, e.ExecID, prepared.SchemaVersion) if cacheValue, exists := ctx.PreparedPlanCache().Get(cacheKey); exists { + metrics.PlanCacheCounter.WithLabelValues("prepare").Inc() plan := cacheValue.(*PSTMTPlanCacheValue).Plan err := e.rebuildRange(plan) if err != nil { diff --git a/plan/gen_physical_plans.go b/plan/gen_physical_plans.go index 5befde20905ef..371f3485bce0a 100644 --- a/plan/gen_physical_plans.go +++ b/plan/gen_physical_plans.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/util/ranger" ) -func (p *LogicalUnionScan) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { us := PhysicalUnionScan{Conditions: p.conditions}.init(p.ctx, p.stats, prop) return []PhysicalPlan{us} } @@ -170,7 +170,7 @@ func (p *LogicalJoin) getHashJoin(prop *requiredProp, innerIdx int) *PhysicalHas RightConditions: p.RightConditions, OtherConditions: p.OtherConditions, JoinType: p.JoinType, - Concurrency: JoinConcurrency, + Concurrency: uint(p.ctx.GetSessionVars().HashJoinConcurrency), DefaultValues: p.DefaultValues, InnerChildIdx: innerIdx, }.init(p.ctx, p.stats.scaleByExpectCnt(prop.expectedCnt), chReqProps...) @@ -405,7 +405,7 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *requiredProp) ([]PhysicalPlan, boo // Firstly we check the hint, if hint is figured by user, we force to choose the corresponding physical plan. // If the hint is not matched, it will get other candidates. // If the hint is not figured, we will pick all candidates. -func (p *LogicalJoin) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalJoin) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { mergeJoins := p.getMergeJoin(prop) if (p.preferJoinType&preferMergeJoin) > 0 && len(mergeJoins) > 0 { return mergeJoins @@ -447,7 +447,7 @@ func (p *LogicalProjection) tryToGetChildProp(prop *requiredProp) (*requiredProp return newProp, true } -func (p *LogicalProjection) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalProjection) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { newProp, ok := p.tryToGetChildProp(prop) if !ok { return nil @@ -493,14 +493,14 @@ func (lt *LogicalTopN) getPhysLimits() []PhysicalPlan { return ret } -func (lt *LogicalTopN) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (lt *LogicalTopN) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { if prop.matchItems(lt.ByItems) { return append(lt.getPhysTopN(), lt.getPhysLimits()...) } return nil } -func (la *LogicalApply) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (la *LogicalApply) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { if !prop.allColsFromSchema(la.children[0].Schema()) { // for convenient, we don't pass through any prop return nil } @@ -516,8 +516,8 @@ func (la *LogicalApply) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan return []PhysicalPlan{apply} } -// genPhysPlansByReqProp is only for implementing interface. DataSource and Dual generate task in `convert2PhysicalPlan` directly. -func (p *baseLogicalPlan) genPhysPlansByReqProp(_ *requiredProp) []PhysicalPlan { +// exhaustPhysicalPlans is only for implementing interface. DataSource and Dual generate task in `findBestTask` directly. +func (p *baseLogicalPlan) exhaustPhysicalPlans(_ *requiredProp) []PhysicalPlan { panic("This function should not be called") } @@ -584,7 +584,7 @@ func (la *LogicalAggregation) getHashAggs(prop *requiredProp) []PhysicalPlan { return hashAggs } -func (la *LogicalAggregation) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (la *LogicalAggregation) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { aggs := make([]PhysicalPlan, 0, len(la.possibleProperties)+1) aggs = append(aggs, la.getHashAggs(prop)...) @@ -594,14 +594,14 @@ func (la *LogicalAggregation) genPhysPlansByReqProp(prop *requiredProp) []Physic return aggs } -func (p *LogicalSelection) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalSelection) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { sel := PhysicalSelection{ Conditions: p.Conditions, }.init(p.ctx, p.stats.scaleByExpectCnt(prop.expectedCnt), prop) return []PhysicalPlan{sel} } -func (p *LogicalLimit) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalLimit) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { if !prop.isEmpty() { return nil } @@ -618,14 +618,14 @@ func (p *LogicalLimit) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan return ret } -func (p *LogicalLock) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalLock) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { lock := PhysicalLock{ Lock: p.Lock, }.init(p.ctx, p.stats.scaleByExpectCnt(prop.expectedCnt), prop) return []PhysicalPlan{lock} } -func (p *LogicalUnionAll) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { // TODO: UnionAll can not pass any order, but we can change it to sort merge to keep order. if !prop.isEmpty() { return nil @@ -653,7 +653,7 @@ func (ls *LogicalSort) getNominalSort(reqProp *requiredProp) *NominalSort { return ps } -func (ls *LogicalSort) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (ls *LogicalSort) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { if prop.matchItems(ls.ByItems) { ret := make([]PhysicalPlan, 0, 2) ret = append(ret, ls.getPhysicalSort(prop)) @@ -666,7 +666,7 @@ func (ls *LogicalSort) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan return nil } -func (p *LogicalExists) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalExists) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { if !prop.isEmpty() { return nil } @@ -675,7 +675,7 @@ func (p *LogicalExists) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan return []PhysicalPlan{exists} } -func (p *LogicalMaxOneRow) genPhysPlansByReqProp(prop *requiredProp) []PhysicalPlan { +func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *requiredProp) []PhysicalPlan { if !prop.isEmpty() { return nil } diff --git a/plan/logical_plan_builder.go b/plan/logical_plan_builder.go index ffa3d6fb9bc5e..3d4be5dc6c067 100644 --- a/plan/logical_plan_builder.go +++ b/plan/logical_plan_builder.go @@ -1629,25 +1629,25 @@ var RatioOfPseudoEstimate = 0.7 // 1. tidb-server started and statistics handle has not been initialized. // 2. table row count from statistics is zero. // 3. statistics is outdated. -func (b *planBuilder) getStatsTable(tableID int64) *statistics.Table { +func (b *planBuilder) getStatsTable(tblInfo *model.TableInfo) *statistics.Table { statsHandle := domain.GetDomain(b.ctx).StatsHandle() // 1. tidb-server started and statistics handle has not been initialized. if statsHandle == nil { - return statistics.PseudoTable(tableID) + return statistics.PseudoTable(tblInfo) } - statsTbl := statsHandle.GetTableStats(tableID) + statsTbl := statsHandle.GetTableStats(tblInfo) // 2. table row count from statistics is zero. if statsTbl.Count == 0 { - return statistics.PseudoTable(tableID) + return statistics.PseudoTable(tblInfo) } // 3. statistics is outdated. if float64(statsTbl.ModifyCount)/float64(statsTbl.Count) > RatioOfPseudoEstimate { countFromStats := statsTbl.Count - statsTbl = statistics.PseudoTable(tableID) + statsTbl = statistics.PseudoTable(tblInfo) // Table row count from statistics is more meaningful than the // pseudo row count in most cases. statsTbl.Count = countFromStats @@ -1687,7 +1687,7 @@ func (b *planBuilder) buildDataSource(tn *ast.TableName) LogicalPlan { ds := DataSource{ DBName: dbName, tableInfo: tableInfo, - statisticTable: b.getStatsTable(tableInfo.ID), + statisticTable: b.getStatsTable(tableInfo), indexHints: tn.IndexHints, availableIndices: availableIdxes, Columns: make([]*model.ColumnInfo, 0, len(columns)), diff --git a/plan/logical_plan_test.go b/plan/logical_plan_test.go index def55388fef69..1c4e3691e6ab6 100644 --- a/plan/logical_plan_test.go +++ b/plan/logical_plan_test.go @@ -610,6 +610,7 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) { stmt, err := s.ParseOneStmt(ca.sql, "", "") c.Assert(err, IsNil, comment) + s.ctx.GetSessionVars().HashJoinConcurrency = 1 Preprocess(s.ctx, stmt, s.is, false) p, err := BuildLogicalPlan(s.ctx, stmt, s.is) c.Assert(err, IsNil) @@ -1361,6 +1362,7 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { ctx: mockContext(), is: s.is, } + builder.ctx.GetSessionVars().HashJoinConcurrency = 1 builder.build(stmt) c.Assert(builder.err, IsNil, comment) @@ -1570,6 +1572,7 @@ func (s *testPlanSuite) TestNameResolver(c *C) { comment := Commentf("for %s", t.sql) stmt, err := s.ParseOneStmt(t.sql, "", "") c.Assert(err, IsNil, comment) + s.ctx.GetSessionVars().HashJoinConcurrency = 1 _, err = BuildLogicalPlan(s.ctx, stmt, s.is) if t.err == "" { diff --git a/plan/optimizer.go b/plan/optimizer.go index f633391c17e49..47e0de2a08035 100644 --- a/plan/optimizer.go +++ b/plan/optimizer.go @@ -148,7 +148,7 @@ func logicalOptimize(flag uint64, logic LogicalPlan) (LogicalPlan, error) { func physicalOptimize(logic LogicalPlan) (PhysicalPlan, error) { logic.preparePossibleProperties() logic.deriveStats() - t, err := logic.convert2PhysicalPlan(&requiredProp{taskTp: rootTaskType, expectedCnt: math.MaxFloat64}) + t, err := logic.findBestTask(&requiredProp{taskTp: rootTaskType, expectedCnt: math.MaxFloat64}) if err != nil { return nil, errors.Trace(err) } diff --git a/plan/physical_plan_builder.go b/plan/physical_plan_builder.go index 842401a429c84..358825d20cfd6 100644 --- a/plan/physical_plan_builder.go +++ b/plan/physical_plan_builder.go @@ -39,9 +39,6 @@ const ( cpuFactor = 0.9 ) -// JoinConcurrency means the number of goroutines that participate in joining. -var JoinConcurrency uint = 5 - // wholeTaskTypes records all possible kinds of task that a plan can return. For Agg, TopN and Limit, we will try to get // these tasks one by one. var wholeTaskTypes = [...]taskType{copSingleReadTaskType, copDoubleReadTaskType, rootTaskType} @@ -67,7 +64,7 @@ func getPropByOrderByItems(items []*ByItems) (*requiredProp, bool) { return &requiredProp{cols: cols, desc: desc}, true } -func (p *LogicalTableDual) convert2PhysicalPlan(prop *requiredProp) (task, error) { +func (p *LogicalTableDual) findBestTask(prop *requiredProp) (task, error) { if !prop.isEmpty() { return invalidTask, nil } @@ -76,43 +73,44 @@ func (p *LogicalTableDual) convert2PhysicalPlan(prop *requiredProp) (task, error return &rootTask{p: dual}, nil } -// convert2PhysicalPlan implements LogicalPlan interface. -func (p *baseLogicalPlan) convert2PhysicalPlan(prop *requiredProp) (t task, err error) { +// findBestTask implements LogicalPlan interface. +func (p *baseLogicalPlan) findBestTask(prop *requiredProp) (bestTask task, err error) { // Look up the task with this prop in the task map. // It's used to reduce double counting. - t = p.getTask(prop) - if t != nil { - return t, nil + bestTask = p.getTask(prop) + if bestTask != nil { + return bestTask, nil } - t = invalidTask + if prop.taskTp != rootTaskType { // Currently all plan cannot totally push down. - p.storeTask(prop, t) - return t, nil + p.storeTask(prop, invalidTask) + return invalidTask, nil } - for _, pp := range p.self.genPhysPlansByReqProp(prop) { - t, err = p.getBestTask(t, pp) - if err != nil { - return nil, errors.Trace(err) + + bestTask = invalidTask + childTasks := make([]task, 0, len(p.children)) + for _, pp := range p.self.exhaustPhysicalPlans(prop) { + // find best child tasks firstly. + childTasks = childTasks[:0] + for i, child := range p.children { + childTask, err := child.findBestTask(pp.getChildReqProps(i)) + if err != nil { + return nil, errors.Trace(err) + } + childTasks = append(childTasks, childTask) } - } - p.storeTask(prop, t) - return t, nil -} -func (p *baseLogicalPlan) getBestTask(bestTask task, pp PhysicalPlan) (task, error) { - tasks := make([]task, 0, len(p.children)) - for i, child := range p.children { - childTask, err := child.convert2PhysicalPlan(pp.getChildReqProps(i)) - if err != nil { - return nil, errors.Trace(err) + // combine best child tasks with parent physical plan. + curTask := pp.attach2Task(childTasks...) + + // get the most efficient one. + if curTask.cost() < bestTask.cost() { + bestTask = curTask } - tasks = append(tasks, childTask) - } - resultTask := pp.attach2Task(tasks...) - if resultTask.cost() < bestTask.cost() { - bestTask = resultTask } + + p.storeTask(prop, bestTask) return bestTask, nil } @@ -169,9 +167,9 @@ func (ds *DataSource) tryToGetDualTask() (task, error) { return nil, nil } -// convert2PhysicalPlan implements the PhysicalPlan interface. +// findBestTask implements the PhysicalPlan interface. // It will enumerate all the available indices and choose a plan with least cost. -func (ds *DataSource) convert2PhysicalPlan(prop *requiredProp) (task, error) { +func (ds *DataSource) findBestTask(prop *requiredProp) (task, error) { // If ds is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself. // So here we do nothing. // TODO: Add a special prop to handle IndexJoin's inner plan. diff --git a/plan/physical_plan_test.go b/plan/physical_plan_test.go index 91dc562c9307d..95077298a67af 100644 --- a/plan/physical_plan_test.go +++ b/plan/physical_plan_test.go @@ -769,8 +769,8 @@ func (s *testPlanSuite) TestDAGPlanBuilderAgg(c *C) { }, // Test stream agg + index double. { - sql: "select sum(e), avg(b + c) from t where c = 1 and e = 1 group by c", - best: "IndexLookUp(Index(t.c_d_e)[[1,1]]->Sel([eq(test.t.e, 1)]), Table(t))->StreamAgg", + sql: "select sum(e), avg(b + c) from t where c = 1 and b = 1 group by c", + best: "IndexLookUp(Index(t.c_d_e)[[1,1]], Table(t)->Sel([eq(test.t.b, 1)]))->StreamAgg", }, // Test hash agg + order. { @@ -1015,7 +1015,7 @@ func (s *testPlanSuite) TestRefine(c *C) { best: "TableReader(Table(t)->Sel([like(test.t.c_str, _abc, 92)]))->Projection", }, { - sql: "select a from t where c_str like 'abc%'", + sql: `select a from t where c_str like 'abc%'`, best: "IndexReader(Index(t.c_d_e_str)[[abc,abd)])->Projection", }, { diff --git a/plan/plan.go b/plan/plan.go index 0cfd29fd45afd..5296b44c35b81 100644 --- a/plan/plan.go +++ b/plan/plan.go @@ -148,11 +148,11 @@ type LogicalPlan interface { // PruneColumns prunes the unused columns. PruneColumns([]*expression.Column) - // convert2PhysicalPlan converts the logical plan to the physical plan. It's a new interface. + // findBestTask converts the logical plan to the physical plan. It's a new interface. // It is called recursively from the parent to the children to create the result physical plan. // Some logical plans will convert the children to the physical plans in different ways, and return the one // with the lowest cost. - convert2PhysicalPlan(prop *requiredProp) (task, error) + findBestTask(prop *requiredProp) (task, error) // buildKeyInfo will collect the information of unique keys into schema. buildKeyInfo() @@ -169,8 +169,8 @@ type LogicalPlan interface { // so we can prepare possible properties for every LogicalPlan node. preparePossibleProperties() [][]*expression.Column - // genPhysPlansByReqProp generates all possible plans that can match the required property. - genPhysPlansByReqProp(*requiredProp) []PhysicalPlan + // exhaustPhysicalPlans generates all possible plans that can match the required property. + exhaustPhysicalPlans(*requiredProp) []PhysicalPlan extractCorrelatedCols() []*expression.CorrelatedColumn diff --git a/plan/planbuilder.go b/plan/planbuilder.go index a40f4f932ca42..b9340b5d14abf 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -491,6 +491,10 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) Plan { p := &RecoverIndex{Table: as.Tables[0], IndexName: as.Index} p.SetSchema(buildRecoverIndexFields()) ret = p + case ast.AdminCleanupIndex: + p := &CleanupIndex{Table: as.Tables[0], IndexName: as.Index} + p.SetSchema(buildCleanupIndexFields()) + ret = p case ast.AdminChecksumTable: p := &ChecksumTable{Tables: as.Tables} p.SetSchema(buildChecksumTableSchema()) @@ -665,6 +669,12 @@ func buildRecoverIndexFields() *expression.Schema { return schema } +func buildCleanupIndexFields() *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, 1)...) + schema.Append(buildColumn("", "REMOVED_COUNT", mysql.TypeLonglong, 4)) + return schema +} + func buildShowDDLJobsFields() *expression.Schema { schema := expression.NewSchema(make([]*expression.Column, 0, 2)...) schema.Append(buildColumn("", "JOBS", mysql.TypeVarchar, 128)) @@ -1367,9 +1377,9 @@ func buildShowSchema(s *ast.ShowStmt) (schema *expression.Schema) { mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, } case ast.ShowProcessList: - names = []string{"Id", "User", "Host", "db", "Command", "Time", "State", "Info"} + names = []string{"Id", "User", "Host", "db", "Command", "Time", "State", "Info", "Mem"} ftypes = []byte{mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar, - mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeVarchar, mysql.TypeString} + mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLong, mysql.TypeVarchar, mysql.TypeString, mysql.TypeLonglong} case ast.ShowStatsMeta: names = []string{"Db_name", "Table_name", "Update_time", "Modify_count", "Row_count"} ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeLonglong, mysql.TypeLonglong} diff --git a/plan/predicate_push_down.go b/plan/predicate_push_down.go index 5de0c8d8b63e1..5eacb5b8a9b43 100644 --- a/plan/predicate_push_down.go +++ b/plan/predicate_push_down.go @@ -133,8 +133,14 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression) (ret case InnerJoin: p.LeftConditions = nil p.RightConditions = nil - p.EqualConditions = equalCond - p.OtherConditions = otherCond + p.EqualConditions = make([]*expression.ScalarFunction, 0, len(equalCond)) + for _, cond := range equalCond { + p.EqualConditions = append(p.EqualConditions, cond.Clone().(*expression.ScalarFunction)) + } + p.OtherConditions = make([]expression.Expression, 0, len(otherCond)) + for _, cond := range otherCond { + p.OtherConditions = append(p.OtherConditions, cond.Clone()) + } leftCond = leftPushCond rightCond = rightPushCond } diff --git a/plan/property_cols_prune.go b/plan/property_cols_prune.go index 1f672840f45de..5449ed87ae3c4 100644 --- a/plan/property_cols_prune.go +++ b/plan/property_cols_prune.go @@ -17,23 +17,28 @@ import ( "github.com/pingcap/tidb/expression" ) -func (ds *DataSource) preparePossibleProperties() (result [][]*expression.Column) { +func (ds *DataSource) preparePossibleProperties() [][]*expression.Column { indices := ds.availableIndices.indices includeTS := ds.availableIndices.includeTableScan + + result := make([][]*expression.Column, 0, len(indices)) ds.relevantIndices = make([]bool, len(indices)) + if includeTS { col := ds.getPKIsHandleCol() if col != nil { result = append(result, []*expression.Column{col}) } + cols := expression.ExtractColumnsFromExpressions(make([]*expression.Column, 0, 10), ds.pushedDownConds, nil) + colsSet := make(map[string]struct{}, len(cols)) + for _, col := range cols { + colsSet[col.ColName.L] = struct{}{} + } + for i, idx := range indices { - for _, col := range cols { - if col.ColName.L == idx.Columns[0].Name.L { - ds.relevantIndices[i] = true - break - } - } + _, ok := colsSet[idx.Columns[0].Name.L] + ds.relevantIndices[i] = ok } } else { for i := range ds.relevantIndices { @@ -46,7 +51,7 @@ func (ds *DataSource) preparePossibleProperties() (result [][]*expression.Column result = append(result, cols) } } - return + return result } func (p *LogicalSelection) preparePossibleProperties() (result [][]*expression.Column) { diff --git a/plan/stats.go b/plan/stats.go index fb1b9a886f74f..b1d7ea7081264 100644 --- a/plan/stats.go +++ b/plan/stats.go @@ -75,19 +75,24 @@ func (p *LogicalTableDual) deriveStats() *statsInfo { } func (p *baseLogicalPlan) deriveStats() *statsInfo { - if len(p.children) == 0 { - profile := &statsInfo{ - count: float64(1), - cardinality: make([]float64, p.self.Schema().Len()), - } - for i := range profile.cardinality { - profile.cardinality[i] = float64(1) - } - p.stats = profile - return profile + if len(p.children) > 1 { + panic("LogicalPlans with more than one child should implement their own deriveStats().") } - p.stats = p.children[0].deriveStats() - return p.stats + + if len(p.children) == 1 { + p.stats = p.children[0].deriveStats() + return p.stats + } + + profile := &statsInfo{ + count: float64(1), + cardinality: make([]float64, p.self.Schema().Len()), + } + for i := range profile.cardinality { + profile.cardinality[i] = float64(1) + } + p.stats = profile + return profile } func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) *statsInfo { @@ -145,17 +150,11 @@ func (p *LogicalUnionAll) deriveStats() *statsInfo { func (p *LogicalLimit) deriveStats() *statsInfo { childProfile := p.children[0].deriveStats() p.stats = &statsInfo{ - count: float64(p.Count), + count: math.Min(float64(p.Count), childProfile.count), cardinality: make([]float64, len(childProfile.cardinality)), } - if p.stats.count > childProfile.count { - p.stats.count = childProfile.count - } for i := range p.stats.cardinality { - p.stats.cardinality[i] = childProfile.cardinality[i] - if p.stats.cardinality[i] > p.stats.count { - p.stats.cardinality[i] = p.stats.count - } + p.stats.cardinality[i] = math.Min(childProfile.cardinality[i], p.stats.count) } return p.stats } @@ -163,17 +162,11 @@ func (p *LogicalLimit) deriveStats() *statsInfo { func (lt *LogicalTopN) deriveStats() *statsInfo { childProfile := lt.children[0].deriveStats() lt.stats = &statsInfo{ - count: float64(lt.Count), + count: math.Min(float64(lt.Count), childProfile.count), cardinality: make([]float64, len(childProfile.cardinality)), } - if lt.stats.count > childProfile.count { - lt.stats.count = childProfile.count - } for i := range lt.stats.cardinality { - lt.stats.cardinality[i] = childProfile.cardinality[i] - if lt.stats.cardinality[i] > lt.stats.count { - lt.stats.cardinality[i] = lt.stats.count - } + lt.stats.cardinality[i] = math.Min(childProfile.cardinality[i], lt.stats.count) } return lt.stats } @@ -188,10 +181,8 @@ func getCardinality(cols []*expression.Column, schema *expression.Schema, profil } var cardinality = 1.0 for _, idx := range indices { - if cardinality < profile.cardinality[idx] { - // It is a very elementary estimation. - cardinality = profile.cardinality[idx] - } + // It is a very elementary estimation. + cardinality = math.Max(cardinality, profile.cardinality[idx]) } return cardinality } diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 4f7d067841b61..6a127362a634d 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -186,7 +186,7 @@ func (p *MySQLPrivilege) loadTable(sctx sessionctx.Context, sql string, chk := rs.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) if err != nil { return errors.Trace(err) } diff --git a/server/conn.go b/server/conn.go index 74db5aba8c5f6..df2cb26a7b25a 100644 --- a/server/conn.go +++ b/server/conn.go @@ -955,13 +955,13 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool chk := rs.NewChunk() gotColumnInfo := false for { - // Here server.tidbResultSet implements NextChunk method. - err := rs.NextChunk(ctx, chk) + // Here server.tidbResultSet implements Next method. + err := rs.Next(ctx, chk) if err != nil { return errors.Trace(err) } if !gotColumnInfo { - // We need to call NextChunk before we get columns. + // We need to call Next before we get columns. // Otherwise, we will get incorrect columns info. columns := rs.Columns() err = cc.writeColumnInfo(columns) diff --git a/server/driver.go b/server/driver.go index 42968c6c2fd23..9b2b1eaceb598 100644 --- a/server/driver.go +++ b/server/driver.go @@ -119,6 +119,6 @@ type PreparedStatement interface { type ResultSet interface { Columns() []*ColumnInfo NewChunk() *chunk.Chunk - NextChunk(context.Context, *chunk.Chunk) error + Next(context.Context, *chunk.Chunk) error Close() error } diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 85735baccddc1..016561b2b18cd 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -293,8 +293,8 @@ func (trs *tidbResultSet) NewChunk() *chunk.Chunk { return trs.recordSet.NewChunk() } -func (trs *tidbResultSet) NextChunk(ctx context.Context, chk *chunk.Chunk) error { - return trs.recordSet.NextChunk(ctx, chk) +func (trs *tidbResultSet) Next(ctx context.Context, chk *chunk.Chunk) error { + return trs.recordSet.Next(ctx, chk) } func (trs *tidbResultSet) Close() error { diff --git a/server/http_handler.go b/server/http_handler.go index 13fd4a733e444..ff0f310ae7ac8 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -66,6 +67,7 @@ const ( // For query string const qTableID = "table_id" +const qLimit = "limit" const ( headerContentType = "Content-Type" @@ -273,6 +275,31 @@ func (t *tikvHandlerTool) handleMvccGetByHex(params map[string]string) (interfac return t.getMvccByEncodedKey(encodedKey) } +func (t *tikvHandlerTool) getAllHistoryDDL() ([]*model.Job, error) { + s, err := session.CreateSession(t.store.(kv.Storage)) + if err != nil { + return nil, errors.Trace(err) + } + + if s != nil { + defer s.Close() + } + + store := domain.GetDomain(s.(sessionctx.Context)).Store() + txn, err := store.Begin() + + if err != nil { + return nil, errors.Trace(err) + } + txnMeta := meta.NewMeta(txn) + + jobs, err := txnMeta.GetAllHistoryDDLJobs() + if err != nil { + return nil, errors.Trace(err) + } + return jobs, nil +} + // settingsHandler is the handler for list tidb server settings. type settingsHandler struct { } @@ -294,6 +321,11 @@ type tableHandler struct { op string } +// ddlHistoryJobHandler is the handler for list job history. +type ddlHistoryJobHandler struct { + *tikvHandlerTool +} + // valueHandle is the handler for get value. type valueHandler struct { } @@ -576,6 +608,45 @@ func (h tableHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } +// ServeHTTP handles request of ddl jobs history. +func (h ddlHistoryJobHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + if limitID := req.FormValue(qLimit); len(limitID) > 0 { + lid, err := strconv.Atoi(limitID) + + if err != nil { + writeError(w, err) + return + } + + if lid < 1 { + writeError(w, errors.New("ddl history limit must be greater than 1")) + return + } + + jobs, err := h.getAllHistoryDDL() + if err != nil { + writeError(w, errors.New("ddl history not found")) + return + } + + jobsLen := len(jobs) + if jobsLen > lid { + start := jobsLen - lid + jobs = jobs[start:] + } + + writeData(w, jobs) + return + } + jobs, err := h.getAllHistoryDDL() + if err != nil { + writeError(w, errors.New("ddl history not found")) + return + } + writeData(w, jobs) + return +} + func (h tableHandler) handleRegionRequest(schema infoschema.InfoSchema, tbl table.Table, w http.ResponseWriter, req *http.Request) { tableID := tbl.Meta().ID // for record diff --git a/server/http_handler_test.go b/server/http_handler_test.go index ab1f5638c881e..837bee205225f 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -28,9 +28,13 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" @@ -221,7 +225,7 @@ func (ts *HTTPHandlerTestSuite) TestRegionsFromMeta(c *C) { } func (ts *HTTPHandlerTestSuite) startServer(c *C) { - mvccStore := mocktikv.NewMvccStore() + mvccStore := mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore(mockstore.WithMVCCStore(mvccStore)) c.Assert(err, IsNil) _, err = session.BootstrapSession(store) @@ -290,6 +294,8 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { ts.startServer(c) ts.prepareData(c) defer ts.stopServer(c) + + c.Skip("MVCCLevelDB doesn't implement MVCCDebugger interface.") resp, err := http.Get(fmt.Sprintf("http://127.0.0.1:10090/mvcc/key/tidb/test/1")) c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) @@ -345,6 +351,7 @@ func (ts *HTTPHandlerTestSuite) TestGetMVCCNotFound(c *C) { c.Assert(err, IsNil) c.Assert(data.Info, IsNil) + c.Skip("MVCCLevelDB doesn't implement MVCCDebugger interface.") resp, err = http.Get(fmt.Sprintf("http://127.0.0.1:10090/mvcc/txn/0")) c.Assert(err, IsNil) var p kvrpcpb.MvccGetByStartTsResponse @@ -422,6 +429,7 @@ func (ts *HTTPHandlerTestSuite) TestGetIndexMVCC(c *C) { ts.prepareData(c) defer ts.stopServer(c) + c.Skip("MVCCLevelDB doesn't implement MVCCDebugger interface.") // tests for normal index key resp, err := http.Get("http://127.0.0.1:10090/mvcc/index/tidb/test/idx1/1?a=1&b=2") c.Assert(err, IsNil) @@ -543,3 +551,30 @@ func (ts *HTTPHandlerTestSuite) TestGetSchema(c *C) { _, err = http.Get(fmt.Sprintf("http://127.0.0.1:10090/schema/tidb/abc")) c.Assert(err, IsNil) } + +func (ts *HTTPHandlerTestSuite) TestAllHistory(c *C) { + ts.startServer(c) + ts.prepareData(c) + defer ts.stopServer(c) + resp, err := http.Get(fmt.Sprintf("http://127.0.0.1:10090/ddl/history/?limit=3")) + c.Assert(err, IsNil) + resp, err = http.Get(fmt.Sprintf("http://127.0.0.1:10090/ddl/history/?limit=-1")) + c.Assert(err, IsNil) + + resp, err = http.Get(fmt.Sprintf("http://127.0.0.1:10090/ddl/history")) + c.Assert(err, IsNil) + decoder := json.NewDecoder(resp.Body) + + var jobs []*model.Job + s, _ := session.CreateSession(ts.server.newTikvHandlerTool().store.(kv.Storage)) + defer s.Close() + store := domain.GetDomain(s.(sessionctx.Context)).Store() + txn, _ := store.Begin() + txnMeta := meta.NewMeta(txn) + txnMeta.GetAllHistoryDDLJobs() + data, _ := txnMeta.GetAllHistoryDDLJobs() + err = decoder.Decode(&jobs) + + c.Assert(err, IsNil) + c.Assert(jobs, DeepEquals, data) +} diff --git a/server/http_status.go b/server/http_status.go index 3a39340864065..6facde077282d 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -54,6 +54,7 @@ func (s *Server) startHTTPServer() { router.Handle("/schema/{db}", schemaHandler{tikvHandlerTool}) router.Handle("/schema/{db}/{table}", schemaHandler{tikvHandlerTool}) router.Handle("/tables/{colID}/{colTp}/{colFlag}/{colLen}", valueHandler{}) + router.Handle("/ddl/history", ddlHistoryJobHandler{tikvHandlerTool}) if s.cfg.Store == "tikv" { // HTTP path for tikv router.Handle("/tables/{db}/{table}/regions", tableHandler{tikvHandlerTool, opTableRegions}) diff --git a/server/statistics_handler_test.go b/server/statistics_handler_test.go index 6240a0900a745..5d495128b6dc5 100644 --- a/server/statistics_handler_test.go +++ b/server/statistics_handler_test.go @@ -61,7 +61,7 @@ func (ds *testDumpStatsSuite) TestDumpStatsAPI(c *C) { } func (ds *testDumpStatsSuite) startServer(c *C) { - mvccStore := mocktikv.NewMvccStore() + mvccStore := mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore(mockstore.WithMVCCStore(mvccStore)) c.Assert(err, IsNil) diff --git a/server/tidb_test.go b/server/tidb_test.go index 3d030a8107dd6..ffdb3fdad39d0 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -400,7 +400,7 @@ func (ts *TidbTestSuite) TestCreateTableFlen(c *C) { c.Assert(err, IsNil) rs, err := qctx.Execute(ctx, "show create table t1") chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) cols := rs[0].Columns() c.Assert(err, IsNil) @@ -429,7 +429,7 @@ func (ts *TidbTestSuite) TestShowTablesFlen(c *C) { c.Assert(err, IsNil) rs, err := qctx.Execute(ctx, "show tables") chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) cols := rs[0].Columns() c.Assert(err, IsNil) diff --git a/session/bench_test.go b/session/bench_test.go index accdb4e117935..777876e570c02 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -84,7 +84,7 @@ func prepareJoinBenchData(se Session, colType string, valueFormat string, valueC func readResult(ctx context.Context, rs ast.RecordSet, count int) { chk := rs.NewChunk() for count > 0 { - err := rs.NextChunk(ctx, chk) + err := rs.Next(ctx, chk) if err != nil { log.Fatal(err) } diff --git a/session/bootstrap.go b/session/bootstrap.go index c66b8994abb44..fcd0e930d1a7a 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -71,7 +71,7 @@ const ( CreateDBPrivTable = `CREATE TABLE if not exists mysql.db ( Host CHAR(60), DB CHAR(64), - User CHAR(16), + User CHAR(32), Select_priv ENUM('N','Y') Not Null DEFAULT 'N', Insert_priv ENUM('N','Y') Not Null DEFAULT 'N', Update_priv ENUM('N','Y') Not Null DEFAULT 'N', @@ -95,8 +95,8 @@ const ( // CreateTablePrivTable is the SQL statement creates table scope privilege table in system db. CreateTablePrivTable = `CREATE TABLE if not exists mysql.tables_priv ( Host CHAR(60), - DB CHAR(64), - User CHAR(16), + DB CHAR(64), + User CHAR(32), Table_name CHAR(64), Grantor CHAR(77), Timestamp Timestamp DEFAULT CURRENT_TIMESTAMP, @@ -106,8 +106,8 @@ const ( // CreateColumnPrivTable is the SQL statement creates column scope privilege table in system db. CreateColumnPrivTable = `CREATE TABLE if not exists mysql.columns_priv( Host CHAR(60), - DB CHAR(64), - User CHAR(16), + DB CHAR(64), + User CHAR(32), Table_name CHAR(64), Column_name CHAR(64), Timestamp Timestamp DEFAULT CURRENT_TIMESTAMP, @@ -188,6 +188,15 @@ const ( UNIQUE KEY (element_id), KEY (job_id, element_id) );` + + // CreateStatsFeedbackTable stores the feedback info which is used to update stats. + CreateStatsFeedbackTable = `CREATE TABLE IF NOT EXISTS mysql.stats_feedback ( + table_id bigint(64) NOT NULL, + is_index tinyint(2) NOT NULL, + hist_id bigint(64) NOT NULL, + feedback blob NOT NULL, + index hist(table_id, is_index, hist_id) + );` ) // bootstrap initiates system DB for a store. @@ -232,6 +241,8 @@ const ( version16 = 16 version17 = 17 version18 = 18 + version19 = 19 + version20 = 20 ) func checkBootstrapped(s Session) (bool, error) { @@ -274,7 +285,7 @@ func getTiDBVar(s Session, name string) (sVal string, isNull bool, e error) { r := rs[0] defer terror.Call(r.Close) chk := r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) if err != nil || chk.NumRows() == 0 { return "", true, errors.Trace(err) } @@ -362,6 +373,14 @@ func upgrade(s Session) { upgradeToVer18(s) } + if ver < version19 { + upgradeToVer19(s) + } + + if ver < version20 { + upgradeToVer20(s) + } + updateBootstrapVer(s) _, err = s.Execute(context.Background(), "COMMIT") @@ -484,7 +503,7 @@ func upgradeToVer12(s Session) { defer terror.Call(r.Close) chk := r.NewChunk() it := chunk.NewIterator4Chunk(chk) - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) for err == nil && chk.NumRows() != 0 { for row := it.Begin(); row != it.End(); row = it.Next() { user := row.GetString(0) @@ -496,7 +515,7 @@ func upgradeToVer12(s Session) { updateSQL := fmt.Sprintf(`UPDATE mysql.user set password = "%s" where user="%s" and host="%s"`, newPass, user, host) sqls = append(sqls, updateSQL) } - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) } terror.MustNil(err) @@ -578,6 +597,16 @@ func upgradeToVer18(s Session) { doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `tot_col_size` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } +func upgradeToVer19(s Session) { + doReentrantDDL(s, "ALTER TABLE mysql.db MODIFY User CHAR(32)") + doReentrantDDL(s, "ALTER TABLE mysql.tables_priv MODIFY User CHAR(32)") + doReentrantDDL(s, "ALTER TABLE mysql.columns_priv MODIFY User CHAR(32)") +} + +func upgradeToVer20(s Session) { + doReentrantDDL(s, CreateStatsFeedbackTable) +} + // updateBootstrapVer updates bootstrap version variable in mysql.TiDB table. func updateBootstrapVer(s Session) { // Update bootstrap version. @@ -624,6 +653,8 @@ func doDDLWorks(s Session) { mustExecute(s, CreateStatsBucketsTable) // Create gc_delete_range table. mustExecute(s, CreateGCDeleteRangeTable) + // Create stats_feedback table. + mustExecute(s, CreateStatsFeedbackTable) } // doDMLWorks executes DML statements in bootstrap stage. diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index b4d96a130194f..2d6a7cbcb7f42 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -51,11 +51,11 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { c.Assert(r, NotNil) ctx := context.Background() chk := r.NewChunk() - err := r.NextChunk(ctx, chk) + err := r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) datums := ast.RowToDatums(chk.GetRow(0), r.Fields()) - match(c, datums, []byte("%"), []byte("root"), []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y") + match(c, datums, []byte(`%`), []byte("root"), []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y") c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "anyhost"}, []byte(""), []byte("")), IsTrue) mustExecSQL(c, se, "USE test;") @@ -67,7 +67,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { r = mustExecSQL(c, se, "SELECT COUNT(*) from mysql.global_variables;") c.Assert(r, NotNil) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).GetInt64(0), Equals, globalVarsCount()) @@ -88,7 +88,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { c.Assert(r, NotNil) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) datums = ast.RowToDatums(chk.GetRow(0), r.Fields()) match(c, datums, 3) @@ -148,12 +148,12 @@ func (s *testBootstrapSuite) testBootstrapWithError(c *C) { mustExecSQL(c, se, "USE mysql;") r := mustExecSQL(c, se, `select * from user;`) chk := r.NewChunk() - err := r.NextChunk(ctx, chk) + err := r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row := chk.GetRow(0) datums := ast.RowToDatums(row, r.Fields()) - match(c, datums, []byte("%"), []byte("root"), []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y") + match(c, datums, []byte(`%`), []byte("root"), []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y") mustExecSQL(c, se, "USE test;") // Check privilege tables. mustExecSQL(c, se, "SELECT * from mysql.db;") @@ -162,14 +162,14 @@ func (s *testBootstrapSuite) testBootstrapWithError(c *C) { // Check global variables. r = mustExecSQL(c, se, "SELECT COUNT(*) from mysql.global_variables;") chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) v := chk.GetRow(0) c.Assert(v.GetInt64(0), Equals, globalVarsCount()) r = mustExecSQL(c, se, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="bootstrapped";`) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row = chk.GetRow(0) @@ -192,7 +192,7 @@ func (s *testBootstrapSuite) TestUpgrade(c *C) { // bootstrap with currentBootstrapVersion r := mustExecSQL(c, se, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`) chk := r.NewChunk() - err := r.NextChunk(ctx, chk) + err := r.Next(ctx, chk) row := chk.GetRow(0) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) @@ -221,7 +221,7 @@ func (s *testBootstrapSuite) TestUpgrade(c *C) { // Make sure the version is downgraded. r = mustExecSQL(c, se1, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsTrue) @@ -236,7 +236,7 @@ func (s *testBootstrapSuite) TestUpgrade(c *C) { se2 := newSession(c, store, s.dbName) r = mustExecSQL(c, se2, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row = chk.GetRow(0) diff --git a/session/isolation_test.go b/session/isolation_test.go index 9d505d4c50990..5cfeaf172b3f6 100644 --- a/session/isolation_test.go +++ b/session/isolation_test.go @@ -28,7 +28,7 @@ var _ = Suite(&testIsolationSuite{}) type testIsolationSuite struct { cluster *mocktikv.Cluster - mvccStore *mocktikv.MvccStore + mvccStore mocktikv.MVCCStore store kv.Storage dom *domain.Domain } @@ -37,7 +37,7 @@ func (s *testIsolationSuite) SetUpSuite(c *C) { testleak.BeforeTest() s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.NewMvccStore() + s.mvccStore = mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(s.cluster), mockstore.WithMVCCStore(s.mvccStore), diff --git a/session/session.go b/session/session.go index 1e0ab0c920380..7bd920e4590c6 100644 --- a/session/session.go +++ b/session/session.go @@ -350,6 +350,17 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { metrics.StatementPerTransaction.WithLabelValues(metrics.RetLabel(err)).Observe(float64(counter)) metrics.TransactionDuration.WithLabelValues(metrics.RetLabel(err)).Observe(float64(duration)) s.cleanRetryInfo() + + if isoLevelOneShot := &s.sessionVars.TxnIsolationLevelOneShot; isoLevelOneShot.State != 0 { + switch isoLevelOneShot.State { + case 1: + isoLevelOneShot.State = 2 + case 2: + isoLevelOneShot.State = 0 + isoLevelOneShot.Value = "" + } + } + if err != nil { log.Warnf("[%d] finished txn:%v, %v", s.sessionVars.ConnectionID, s.txn, err) return errors.Trace(err) @@ -357,7 +368,7 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { mapper := s.GetSessionVars().TxnCtx.TableDeltaMap if s.statsCollector != nil && mapper != nil { for id, item := range mapper { - s.statsCollector.Update(id, item.Delta, item.Count) + s.statsCollector.Update(id, item.Delta, item.Count, &item.ColSize) } } return nil @@ -610,7 +621,7 @@ func drainRecordSet(ctx context.Context, rs ast.RecordSet) ([]types.Row, error) var rows []types.Row for { chk := rs.NewChunk() - err := rs.NextChunk(ctx, chk) + err := rs.Next(ctx, chk) if err != nil || chk.NumRows() == 0 { return rows, errors.Trace(err) } @@ -754,7 +765,7 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []ast.Rec cacheValue kvcache.Value hitCache = false connID = s.sessionVars.ConnectionID - planCacheEnabled = plan.PlanCacheEnabled // Read global configuration only once. + planCacheEnabled = s.sessionVars.PlanCacheEnabled // Its value is read from the global configuration, and it will be only updated in tests. ) if planCacheEnabled { @@ -766,6 +777,7 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []ast.Rec } if hitCache { + metrics.PlanCacheCounter.WithLabelValues("select").Inc() stmtNode := cacheValue.(*plan.SQLCacheValue).StmtNode stmt := &executor.ExecStmt{ InfoSchema: executor.GetInfoSchema(s), @@ -859,7 +871,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields // So we have to call PrepareTxnCtx here. s.PrepareTxnCtx(ctx) prepareExec := executor.NewPrepareExec(s, executor.GetInfoSchema(s), sql) - err = prepareExec.NextChunk(ctx, nil) + err = prepareExec.Next(ctx, nil) if err != nil { err = errors.Trace(err) return @@ -1196,7 +1208,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er const ( notBootstrapped = 0 - currentBootstrapVersion = 18 + currentBootstrapVersion = 20 ) func getStoreBootstrapVersion(store kv.Storage) int64 { @@ -1255,7 +1267,9 @@ const loadCommonGlobalVarsSQL = "select HIGH_PRIORITY * from mysql.global_variab variable.TiDBIndexJoinBatchSize + quoteCommaQuote + variable.TiDBIndexLookupSize + quoteCommaQuote + variable.TiDBIndexLookupConcurrency + quoteCommaQuote + + variable.TiDBIndexLookupJoinConcurrency + quoteCommaQuote + variable.TiDBIndexSerialScanConcurrency + quoteCommaQuote + + variable.TiDBHashJoinConcurrency + quoteCommaQuote + variable.TiDBDistSQLScanConcurrency + "')" // loadCommonGlobalVariablesIfNeeded loads and applies commonly used global variables for the session. @@ -1367,6 +1381,7 @@ func (s *session) ShowProcess() util.ProcessInfo { tmp := s.processInfo.Load() if tmp != nil { pi = tmp.(util.ProcessInfo) + pi.Mem = s.GetSessionVars().StmtCtx.MemTracker.BytesConsumed() } return pi } diff --git a/session/session_test.go b/session/session_test.go index 63a8170a66710..44ccf55445e23 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -52,7 +52,7 @@ var _ = Suite(&testSessionSuite{}) type testSessionSuite struct { cluster *mocktikv.Cluster - mvccStore *mocktikv.MvccStore + mvccStore mocktikv.MVCCStore store kv.Storage dom *domain.Domain } @@ -61,7 +61,7 @@ func (s *testSessionSuite) SetUpSuite(c *C) { testleak.BeforeTest() s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.NewMvccStore() + s.mvccStore = mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(s.cluster), mockstore.WithMVCCStore(s.mvccStore), @@ -111,18 +111,18 @@ func (p *mockBinlogPump) PullBinlogs(ctx context.Context, in *binlog.PullBinlogR } func (s *testSessionSuite) TestForCoverage(c *C) { - planCache := plan.PlanCacheEnabled plan.GlobalPlanCache = kvcache.NewShardedLRUCache(2, 1) - defer func() { - plan.PlanCacheEnabled = planCache - }() // Just for test coverage. tk := testkit.NewTestKitWithInit(c, s.store) + planCache := tk.Se.GetSessionVars().PlanCacheEnabled + defer func() { + tk.Se.GetSessionVars().PlanCacheEnabled = planCache + }() tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int auto_increment, v int, index (id))") tk.MustExec("insert t values ()") - plan.PlanCacheEnabled = true + tk.Se.GetSessionVars().PlanCacheEnabled = true tk.MustExec("insert t values ()") tk.MustExec("insert t values ()") @@ -564,8 +564,8 @@ func (s *testSessionSuite) TestSkipWithGrant(c *C) { c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "user_not_exist"}, []byte("yyy"), []byte("zzz")), IsFalse) privileges.SkipWithGrant = true - c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "xxx", Hostname: "%"}, []byte("yyy"), []byte("zzz")), IsTrue) - c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, []byte(""), []byte("")), IsTrue) + c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "xxx", Hostname: `%`}, []byte("yyy"), []byte("zzz")), IsTrue) + c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: `%`}, []byte(""), []byte("")), IsTrue) tk.MustExec("create table t (id int)") privileges.Enable = save1 @@ -934,7 +934,7 @@ func (s *testSessionSuite) TestResultType(c *C) { rs, err := tk.Exec(`select cast(null as char(30))`) c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.Background(), chk) + err = rs.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).IsNull(0), IsTrue) c.Assert(rs.Fields()[0].Column.FieldType.Tp, Equals, mysql.TypeVarString) @@ -1332,10 +1332,11 @@ func (s *testSessionSuite) TestIssue986(c *C) { tk.MustExec(`insert into address values ('10')`) } -func (s *testSessionSuite) TestIssue1089(c *C) { +func (s *testSessionSuite) TestCast(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustQuery("select cast(0.5 as unsigned)") tk.MustQuery("select cast(-0.5 as signed)") + tk.MustQuery("select hex(cast(0x10 as binary(2)))").Check(testkit.Rows("1000")) } func (s *testSessionSuite) TestTableInfoMeta(c *C) { @@ -1419,7 +1420,7 @@ var _ = Suite(&testSchemaSuite{}) type testSchemaSuite struct { cluster *mocktikv.Cluster - mvccStore *mocktikv.MvccStore + mvccStore mocktikv.MVCCStore store kv.Storage lease time.Duration dom *domain.Domain @@ -1439,7 +1440,7 @@ func (s *testSchemaSuite) SetUpSuite(c *C) { testleak.BeforeTest() s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.NewMvccStore() + s.mvccStore = mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(s.cluster), mockstore.WithMVCCStore(s.mvccStore), @@ -1655,7 +1656,7 @@ func (s *testSchemaSuite) TestTableReaderChunk(c *C) { var count int var numChunks int for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) numRows := chk.NumRows() if numRows == 0 { @@ -1688,7 +1689,7 @@ func (s *testSchemaSuite) TestInsertExecChunk(c *C) { var idx int for { chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) if chk.NumRows() == 0 { break @@ -1722,7 +1723,7 @@ func (s *testSchemaSuite) TestUpdateExecChunk(c *C) { var idx int for { chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) if chk.NumRows() == 0 { break @@ -1757,7 +1758,7 @@ func (s *testSchemaSuite) TestDeleteExecChunk(c *C) { c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) c.Assert(chk.NumRows(), Equals, 1) @@ -1789,7 +1790,7 @@ func (s *testSchemaSuite) TestDeleteMultiTableExecChunk(c *C) { var idx int for { chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) if chk.NumRows() == 0 { @@ -1809,7 +1810,7 @@ func (s *testSchemaSuite) TestDeleteMultiTableExecChunk(c *C) { c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) c.Assert(chk.NumRows(), Equals, 0) rs.Close() @@ -1834,7 +1835,7 @@ func (s *testSchemaSuite) TestIndexLookUpReaderChunk(c *C) { chk := rs.NewChunk() var count int for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) numRows := chk.NumRows() if numRows == 0 { @@ -1854,7 +1855,7 @@ func (s *testSchemaSuite) TestIndexLookUpReaderChunk(c *C) { chk = rs.NewChunk() count = 0 for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) numRows := chk.NumRows() if numRows == 0 { @@ -1975,3 +1976,35 @@ func (s *testSessionSuite) TestRollbackOnCompileError(c *C) { } c.Assert(recoverErr, IsTrue) } + +func (s *testSessionSuite) TestSetTransactionIsolationOneShot(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t (k int, v int)") + tk.MustExec("insert t values (1, 42)") + tk.MustExec("set transaction isolation level read committed") + + // Check isolation level is set to read committed. + ctx := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { + c.Assert(req.IsolationLevel, Equals, kv.RC) + }) + tk.Se.Execute(ctx, "select * from t where k = 1") + + // Check it just take effect for one time. + ctx = context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { + c.Assert(req.IsolationLevel, Equals, kv.SI) + }) + tk.Se.Execute(ctx, "select * from t where k = 1") + + // Can't change isolation level when it's inside a transaction. + tk.MustExec("begin") + _, err := tk.Se.Execute(ctx, "set transaction isolation level read committed") + c.Assert(err, NotNil) +} + +func (s *testSessionSuite) TestDBUserNameLength(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table if not exists t (a int)") + // Test user name length can be longer than 16. + tk.MustExec(`grant all privileges on test.* to 'abcddfjakldfjaldddds'@'%' identified by ''`) + tk.MustExec(`grant all privileges on test.t to 'abcddfjakldfjaldddds'@'%' identified by ''`) +} diff --git a/session/tidb.go b/session/tidb.go index 55eda92682c26..edfd96c2c4ec3 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -222,7 +222,7 @@ func GetRows4Test(ctx context.Context, sctx sessionctx.Context, rs ast.RecordSet chk := rs.NewChunk() iter := chunk.NewIterator4Chunk(chk) - err := rs.NextChunk(ctx, chk) + err := rs.Next(ctx, chk) if err != nil { return nil, errors.Trace(err) } diff --git a/session/tidb_test.go b/session/tidb_test.go index f5d2d344e913b..0a41ff89e6a05 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -242,7 +242,7 @@ func newSession(c *C, store kv.Storage, dbName string) Session { id := atomic.AddUint64(&testConnID, 1) se.SetConnectionID(id) c.Assert(err, IsNil) - se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, []byte("012345678901234567890")) + se.Auth(&auth.UserIdentity{Username: "root", Hostname: `%`}, nil, []byte("012345678901234567890")) mustExecSQL(c, se, "create database if not exists "+dbName) mustExecSQL(c, se, "use "+dbName) return se diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index e77c4fa4a327b..b95770dd643b8 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -101,13 +101,19 @@ type TransactionContext struct { } // UpdateDeltaForTable updates the delta info for some table. -func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, count int64) { +func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, count int64, colSize map[int64]int64) { if tc.TableDeltaMap == nil { tc.TableDeltaMap = make(map[int64]TableDelta) } item := tc.TableDeltaMap[tableID] + if item.ColSize == nil { + item.ColSize = make(map[int64]int64) + } item.Delta += delta item.Count += count + for key, val := range colSize { + item.ColSize[key] += val + } tc.TableDeltaMap[tableID] = item } @@ -162,6 +168,15 @@ type SessionVars struct { // Should be reset on transaction finished. TxnCtx *TransactionContext + // TxnIsolationLevelOneShot is used to implements "set transaction isolation level ..." + TxnIsolationLevelOneShot struct { + // state 0 means default + // state 1 means it's set in current transaction. + // state 2 means it should be used in current transaction. + State int + Value string + } + // Following variables are special for current session. Status uint16 @@ -183,6 +198,9 @@ type SessionVars struct { // PlanID is the unique id of logical and physical plan. PlanID int + // PlanCacheEnabled stores the global config "plan-cache-enabled", and it will be only updated in tests. + PlanCacheEnabled bool + // User is the user identity with which the session login. User *auth.UserIdentity @@ -253,9 +271,15 @@ type SessionVars struct { // IndexLookupConcurrency is the number of concurrent index lookup worker. IndexLookupConcurrency int + // IndexLookupJoinConcurrency is the number of concurrent index lookup join inner worker. + IndexLookupJoinConcurrency int + // DistSQLScanConcurrency is the number of concurrent dist SQL scan worker. DistSQLScanConcurrency int + // HashJoinConcurrency is the number of concurrent hash join outer worker. + HashJoinConcurrency int + // IndexSerialScanConcurrency is the number of concurrent index serial scan worker. IndexSerialScanConcurrency int @@ -283,6 +307,8 @@ type SessionVars struct { MemQuotaQuery int64 // MemQuotaHashJoin defines the memory quota for a hash join executor. MemQuotaHashJoin int64 + // MemQuotaMergeJoin defines the memory quota for a merge join executor. + MemQuotaMergeJoin int64 // MemQuotaSort defines the memory quota for a sort executor. MemQuotaSort int64 // MemQuotaTopn defines the memory quota for a top n executor. @@ -291,6 +317,10 @@ type SessionVars struct { MemQuotaIndexLookupReader int64 // MemQuotaIndexLookupJoin defines the memory quota for a index lookup join executor. MemQuotaIndexLookupJoin int64 + // MemQuotaNestedLoopApply defines the memory quota for a nested loop apply executor. + MemQuotaNestedLoopApply int64 + // OptimizerSelectivityLevel defines the level of the selectivity estimation in planner. + OptimizerSelectivityLevel int // EnableStreaming indicates whether the coprocessor request can use streaming API. // TODO: remove this after tidb-server configuration "enable-streaming' removed. @@ -318,15 +348,20 @@ func NewSessionVars() *SessionVars { IndexLookupSize: DefIndexLookupSize, IndexLookupConcurrency: DefIndexLookupConcurrency, IndexSerialScanConcurrency: DefIndexSerialScanConcurrency, + IndexLookupJoinConcurrency: DefIndexLookupJoinConcurrency, + HashJoinConcurrency: DefTiDBHashJoinConcurrency, DistSQLScanConcurrency: DefDistSQLScanConcurrency, MaxChunkSize: DefMaxChunkSize, DMLBatchSize: DefDMLBatchSize, MemQuotaQuery: DefTiDBMemQuotaQuery, MemQuotaHashJoin: DefTiDBMemQuotaHashJoin, + MemQuotaMergeJoin: DefTiDBMemQuotaMergeJoin, MemQuotaSort: DefTiDBMemQuotaSort, MemQuotaTopn: DefTiDBMemQuotaTopn, MemQuotaIndexLookupReader: DefTiDBMemQuotaIndexLookupReader, MemQuotaIndexLookupJoin: DefTiDBMemQuotaIndexLookupJoin, + MemQuotaNestedLoopApply: DefTiDBMemQuotaNestedLoopApply, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, } var enableStreaming string if config.GetGlobalConfig().EnableStreaming { @@ -334,6 +369,7 @@ func NewSessionVars() *SessionVars { } else { enableStreaming = "0" } + vars.PlanCacheEnabled = config.GetGlobalConfig().PlanCache.Enabled terror.Log(vars.SetSystemVar(TiDBEnableStreaming, enableStreaming)) return vars } @@ -442,6 +478,9 @@ func (s *SessionVars) deleteSystemVar(name string) error { // SetSystemVar sets the value of a system variable. func (s *SessionVars) SetSystemVar(name string, val string) error { switch name { + case TxnIsolationOneShot: + s.TxnIsolationLevelOneShot.State = 1 + s.TxnIsolationLevelOneShot.Value = val case TimeZone: tz, err := parseTimeZone(val) if err != nil { @@ -478,29 +517,35 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { case TiDBOptInSubqUnFolding: s.AllowInSubqueryUnFolding = TiDBOptOn(val) case TiDBIndexLookupConcurrency: - s.IndexLookupConcurrency = tidbOptPositiveInt(val, DefIndexLookupConcurrency) + s.IndexLookupConcurrency = tidbOptPositiveInt32(val, DefIndexLookupConcurrency) + case TiDBIndexLookupJoinConcurrency: + s.IndexLookupJoinConcurrency = tidbOptPositiveInt32(val, DefIndexLookupJoinConcurrency) case TiDBIndexJoinBatchSize: - s.IndexJoinBatchSize = tidbOptPositiveInt(val, DefIndexJoinBatchSize) + s.IndexJoinBatchSize = tidbOptPositiveInt32(val, DefIndexJoinBatchSize) case TiDBIndexLookupSize: - s.IndexLookupSize = tidbOptPositiveInt(val, DefIndexLookupSize) + s.IndexLookupSize = tidbOptPositiveInt32(val, DefIndexLookupSize) + case TiDBHashJoinConcurrency: + s.HashJoinConcurrency = tidbOptPositiveInt32(val, DefTiDBHashJoinConcurrency) case TiDBDistSQLScanConcurrency: - s.DistSQLScanConcurrency = tidbOptPositiveInt(val, DefDistSQLScanConcurrency) + s.DistSQLScanConcurrency = tidbOptPositiveInt32(val, DefDistSQLScanConcurrency) case TiDBIndexSerialScanConcurrency: - s.IndexSerialScanConcurrency = tidbOptPositiveInt(val, DefIndexSerialScanConcurrency) + s.IndexSerialScanConcurrency = tidbOptPositiveInt32(val, DefIndexSerialScanConcurrency) case TiDBBatchInsert: s.BatchInsert = TiDBOptOn(val) case TiDBBatchDelete: s.BatchDelete = TiDBOptOn(val) case TiDBDMLBatchSize: - s.DMLBatchSize = tidbOptPositiveInt(val, DefDMLBatchSize) + s.DMLBatchSize = tidbOptPositiveInt32(val, DefDMLBatchSize) case TiDBCurrentTS, TiDBConfig: return ErrReadOnly case TiDBMaxChunkSize: - s.MaxChunkSize = tidbOptPositiveInt(val, DefMaxChunkSize) + s.MaxChunkSize = tidbOptPositiveInt32(val, DefMaxChunkSize) case TIDBMemQuotaQuery: s.MemQuotaQuery = tidbOptInt64(val, DefTiDBMemQuotaQuery) case TIDBMemQuotaHashJoin: s.MemQuotaHashJoin = tidbOptInt64(val, DefTiDBMemQuotaHashJoin) + case TIDBMemQuotaMergeJoin: + s.MemQuotaMergeJoin = tidbOptInt64(val, DefTiDBMemQuotaMergeJoin) case TIDBMemQuotaSort: s.MemQuotaSort = tidbOptInt64(val, DefTiDBMemQuotaSort) case TIDBMemQuotaTopn: @@ -509,10 +554,14 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.MemQuotaIndexLookupReader = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupReader) case TIDBMemQuotaIndexLookupJoin: s.MemQuotaIndexLookupJoin = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupJoin) + case TIDBMemQuotaNestedLoopApply: + s.MemQuotaNestedLoopApply = tidbOptInt64(val, DefTiDBMemQuotaNestedLoopApply) case TiDBGeneralLog: - atomic.StoreUint32(&ProcessGeneralLog, uint32(tidbOptPositiveInt(val, DefTiDBGeneralLog))) + atomic.StoreUint32(&ProcessGeneralLog, uint32(tidbOptPositiveInt32(val, DefTiDBGeneralLog))) case TiDBEnableStreaming: s.EnableStreaming = TiDBOptOn(val) + case TiDBOptimizerSelectivityLevel: + s.OptimizerSelectivityLevel = tidbOptPositiveInt32(val, DefTiDBOptimizerSelectivityLevel) } s.systems[name] = val return nil @@ -526,10 +575,12 @@ const ( MaxAllowedPacket = "max_allowed_packet" TimeZone = "time_zone" TxnIsolation = "tx_isolation" + TxnIsolationOneShot = "tx_isolation_one_shot" ) // TableDelta stands for the changed count for one table. type TableDelta struct { - Delta int64 - Count int64 + Delta int64 + Count int64 + ColSize map[int64]int64 } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 087e1b23ed835..ecd2e6b9407ef 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -615,6 +615,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal | ScopeSession, TiDBIndexJoinBatchSize, strconv.Itoa(DefIndexJoinBatchSize)}, {ScopeGlobal | ScopeSession, TiDBIndexLookupSize, strconv.Itoa(DefIndexLookupSize)}, {ScopeGlobal | ScopeSession, TiDBIndexLookupConcurrency, strconv.Itoa(DefIndexLookupConcurrency)}, + {ScopeGlobal | ScopeSession, TiDBIndexLookupJoinConcurrency, strconv.Itoa(DefIndexLookupJoinConcurrency)}, {ScopeGlobal | ScopeSession, TiDBIndexSerialScanConcurrency, strconv.Itoa(DefIndexSerialScanConcurrency)}, {ScopeGlobal | ScopeSession, TiDBSkipUTF8Check, boolToIntStr(DefSkipUTF8Check)}, {ScopeSession, TiDBBatchInsert, boolToIntStr(DefBatchInsert)}, @@ -624,11 +625,16 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBMaxChunkSize, strconv.Itoa(DefMaxChunkSize)}, {ScopeSession, TIDBMemQuotaQuery, strconv.FormatInt(DefTiDBMemQuotaQuery, 10)}, {ScopeSession, TIDBMemQuotaHashJoin, strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10)}, + {ScopeSession, TIDBMemQuotaMergeJoin, strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10)}, {ScopeSession, TIDBMemQuotaSort, strconv.FormatInt(DefTiDBMemQuotaSort, 10)}, {ScopeSession, TIDBMemQuotaTopn, strconv.FormatInt(DefTiDBMemQuotaTopn, 10)}, {ScopeSession, TIDBMemQuotaIndexLookupReader, strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10)}, {ScopeSession, TIDBMemQuotaIndexLookupJoin, strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10)}, + {ScopeSession, TIDBMemQuotaNestedLoopApply, strconv.FormatInt(DefTiDBMemQuotaNestedLoopApply, 10)}, {ScopeSession, TiDBEnableStreaming, "0"}, + {ScopeSession, TxnIsolationOneShot, ""}, + {ScopeGlobal | ScopeSession, TiDBHashJoinConcurrency, strconv.Itoa(DefTiDBHashJoinConcurrency)}, + {ScopeSession, TiDBOptimizerSelectivityLevel, strconv.Itoa(DefTiDBOptimizerSelectivityLevel)}, /* The following variable is defined as session scope but is actually server scope. */ {ScopeSession, TiDBGeneralLog, strconv.Itoa(DefTiDBGeneralLog)}, {ScopeSession, TiDBConfig, ""}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 1d020782d3aa5..8cd5cf2c3d37d 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -26,10 +26,8 @@ package variable 8. Use this variable to control the behavior in code. */ -// TiDB system variable names. +// TiDB system variable names that only in session scope. const ( - /* Session only */ - // tidb_snapshot is used for reading history data, the default value is empty string. // When the value is set to a datetime string like '2017-11-11 20:20:20', the session reads history data of that time. TiDBSnapshot = "tidb_snapshot" @@ -38,7 +36,7 @@ const ( // When the value is set to true, unique index constraint is not checked. TiDBImportingData = "tidb_import_data" - // tidb_opt_agg_push_down is used to endable/disable the optimizer rule of aggregation push down. + // tidb_opt_agg_push_down is used to enable/disable the optimizer rule of aggregation push down. TiDBOptAggPushDown = "tidb_opt_agg_push_down" // tidb_opt_insubquery_unfold is used to enable/disable the optimizer rule of in subquery unfold. @@ -60,42 +58,6 @@ const ( // tidb_config is a read-only variable that shows the config of the current server. TiDBConfig = "tidb_config" - /* Session and global */ - - // tidb_distsql_scan_concurrency is used to set the concurrency of a distsql scan task. - // A distsql scan task can be a table scan or a index scan, which may be distributed to many TiKV nodes. - // Higher concurrency may reduce latency, but with the cost of higher memory usage and system performance impact. - // If the query has a LIMIT clause, high concurrency makes the system do much more work than needed. - TiDBDistSQLScanConcurrency = "tidb_distsql_scan_concurrency" - - // tidb_index_join_batch_size is used to set the batch size of a index lookup join. - // The index lookup join fetches batches of data from outer executor and constructs ranges for inner executor. - // This value controls how much of data in a batch to do the index join. - // Large value may reduce the latency but consumes more system resource. - TiDBIndexJoinBatchSize = "tidb_index_join_batch_size" - - // tidb_index_lookup_size is used for index lookup executor. - // The index lookup executor first scan a batch of handles from a index, then use those handles to lookup the table - // rows, this value controls how much of handles in a batch to do a lookup task. - // Small value sends more RPCs to TiKV, consume more system resource. - // Large value may do more work than needed if the query has a limit. - TiDBIndexLookupSize = "tidb_index_lookup_size" - - // tidb_index_lookup_concurrency is used for index lookup executor. - // A lookup task may have 'tidb_index_lookup_size' of handles at maximun, the handles may be distributed - // in many TiKV nodes, we executes multiple concurrent index lookup tasks concurrently to reduce the time - // waiting for a task to finish. - // Set this value higher may reduce the latency but consumes more system resource. - TiDBIndexLookupConcurrency = "tidb_index_lookup_concurrency" - - // tidb_index_serial_scan_concurrency is used for controlling the concurrency of index scan operation - // when we need to keep the data output order the same as the order of index data. - TiDBIndexSerialScanConcurrency = "tidb_index_serial_scan_concurrency" - - // tidb_skip_utf8_check skips the UTF8 validate process, validate UTF8 has performance cost, if we can make sure - // the input string values are valid, we can skip the check. - TiDBSkipUTF8Check = "tidb_skip_utf8_check" - // tidb_batch_insert is used to enable/disable auto-split insert data. If set this option on, insert executor will automatically // insert data into multiple batches and use a single txn for each batch. This will be helpful when inserting large data. TiDBBatchInsert = "tidb_batch_insert" @@ -116,27 +78,81 @@ const ( // The following session variables controls the memory quota during query execution. // "tidb_mem_quota_query": control the memory quota of a query. // "tidb_mem_quota_hashjoin": control the memory quota of "HashJoinExec". + // "tidb_mem_quota_mergejoin": control the memory quota of "MergeJoinExec". // "tidb_mem_quota_sort": control the memory quota of "SortExec". // "tidb_mem_quota_topn": control the memory quota of "TopNExec". // "tidb_mem_quota_indexlookupreader": control the memory quota of "IndexLookUpExecutor". // "tidb_mem_quota_indexlookupjoin": control the memory quota of "IndexLookUpJoin". + // "tidb_mem_quota_nestedloopapply": control the memory quota of "NestedLoopApplyExec". TIDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes. TIDBMemQuotaHashJoin = "tidb_mem_quota_hashjoin" // Bytes. + TIDBMemQuotaMergeJoin = "tidb_mem_quota_mergejoin" // Bytes. TIDBMemQuotaSort = "tidb_mem_quota_sort" // Bytes. TIDBMemQuotaTopn = "tidb_mem_quota_topn" // Bytes. TIDBMemQuotaIndexLookupReader = "tidb_mem_quota_indexlookupreader" // Bytes. TIDBMemQuotaIndexLookupJoin = "tidb_mem_quota_indexlookupjoin" // Bytes. + TIDBMemQuotaNestedLoopApply = "tidb_mem_quota_nestedloopapply" // Bytes. // tidb_general_log is used to log every query in the server in info level. TiDBGeneralLog = "tidb_general_log" // tidb_enable_streaming enables TiDB to use streaming API for coprocessor requests. TiDBEnableStreaming = "tidb_enable_streaming" + + // tidb_optimizer_selectivity_level is used to control the selectivity estimation level. + TiDBOptimizerSelectivityLevel = "tidb_optimizer_selectivity_level" +) + +// TiDB system variable names that both in session and global scope. +const ( + // tidb_distsql_scan_concurrency is used to set the concurrency of a distsql scan task. + // A distsql scan task can be a table scan or a index scan, which may be distributed to many TiKV nodes. + // Higher concurrency may reduce latency, but with the cost of higher memory usage and system performance impact. + // If the query has a LIMIT clause, high concurrency makes the system do much more work than needed. + TiDBDistSQLScanConcurrency = "tidb_distsql_scan_concurrency" + + // tidb_index_join_batch_size is used to set the batch size of a index lookup join. + // The index lookup join fetches batches of data from outer executor and constructs ranges for inner executor. + // This value controls how much of data in a batch to do the index join. + // Large value may reduce the latency but consumes more system resource. + TiDBIndexJoinBatchSize = "tidb_index_join_batch_size" + + // tidb_index_lookup_size is used for index lookup executor. + // The index lookup executor first scan a batch of handles from a index, then use those handles to lookup the table + // rows, this value controls how much of handles in a batch to do a lookup task. + // Small value sends more RPCs to TiKV, consume more system resource. + // Large value may do more work than needed if the query has a limit. + TiDBIndexLookupSize = "tidb_index_lookup_size" + + // tidb_index_lookup_concurrency is used for index lookup executor. + // A lookup task may have 'tidb_index_lookup_size' of handles at maximun, the handles may be distributed + // in many TiKV nodes, we executes multiple concurrent index lookup tasks concurrently to reduce the time + // waiting for a task to finish. + // Set this value higher may reduce the latency but consumes more system resource. + TiDBIndexLookupConcurrency = "tidb_index_lookup_concurrency" + + // tidb_index_lookup_join_concurrency is used for index lookup join executor. + // IndexLookUpJoin starts "tidb_index_lookup_join_concurrency" inner workers + // to fetch inner rows and join the matched (outer, inner) row pairs. + TiDBIndexLookupJoinConcurrency = "tidb_index_lookup_join_concurrency" + + // tidb_index_serial_scan_concurrency is used for controlling the concurrency of index scan operation + // when we need to keep the data output order the same as the order of index data. + TiDBIndexSerialScanConcurrency = "tidb_index_serial_scan_concurrency" + + // tidb_skip_utf8_check skips the UTF8 validate process, validate UTF8 has performance cost, if we can make sure + // the input string values are valid, we can skip the check. + TiDBSkipUTF8Check = "tidb_skip_utf8_check" + + // tidb_hash_join_concurrency is used for hash join executor. + // The hash join outer executor starts multiple concurrent join workers to probe the hash table. + TiDBHashJoinConcurrency = "tidb_hash_join_concurrency" ) // Default TiDB system variable values. const ( DefIndexLookupConcurrency = 4 + DefIndexLookupJoinConcurrency = 4 DefIndexSerialScanConcurrency = 1 DefIndexJoinBatchSize = 25000 DefIndexLookupSize = 20000 @@ -153,11 +169,15 @@ const ( DefDMLBatchSize = 20000 DefTiDBMemQuotaQuery = 32 << 30 // 32GB. DefTiDBMemQuotaHashJoin = 32 << 30 // 32GB. + DefTiDBMemQuotaMergeJoin = 32 << 30 // 32GB. DefTiDBMemQuotaSort = 32 << 30 // 32GB. DefTiDBMemQuotaTopn = 32 << 30 // 32GB. DefTiDBMemQuotaIndexLookupReader = 32 << 30 // 32GB. DefTiDBMemQuotaIndexLookupJoin = 32 << 30 // 32GB. + DefTiDBMemQuotaNestedLoopApply = 32 << 30 // 32GB. DefTiDBGeneralLog = 0 + DefTiDBHashJoinConcurrency = 5 + DefTiDBOptimizerSelectivityLevel = 0 ) // Process global variables. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 09ee2013860f5..4f617a1211b83 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -128,7 +128,7 @@ func TiDBOptOn(opt string) bool { return strings.EqualFold(opt, "ON") || opt == "1" } -func tidbOptPositiveInt(opt string, defaultVal int) int { +func tidbOptPositiveInt32(opt string, defaultVal int) int { val, err := strconv.Atoi(opt) if err != nil || val <= 0 { return defaultVal diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 74eb2d14bdbc8..253c658dbdd91 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -52,6 +52,30 @@ func (s *testVarsutilSuite) TestTiDBOptOn(c *C) { } } +func (s *testVarsutilSuite) TestNewSessionVars(c *C) { + defer testleak.AfterTest(c)() + vars := NewSessionVars() + + c.Assert(vars.BuildStatsConcurrencyVar, Equals, DefBuildStatsConcurrency) + c.Assert(vars.IndexJoinBatchSize, Equals, DefIndexJoinBatchSize) + c.Assert(vars.IndexLookupSize, Equals, DefIndexLookupSize) + c.Assert(vars.IndexLookupConcurrency, Equals, DefIndexLookupConcurrency) + c.Assert(vars.IndexSerialScanConcurrency, Equals, DefIndexSerialScanConcurrency) + c.Assert(vars.IndexLookupJoinConcurrency, Equals, DefIndexLookupJoinConcurrency) + c.Assert(vars.HashJoinConcurrency, Equals, DefTiDBHashJoinConcurrency) + c.Assert(vars.DistSQLScanConcurrency, Equals, DefDistSQLScanConcurrency) + c.Assert(vars.MaxChunkSize, Equals, DefMaxChunkSize) + c.Assert(vars.DMLBatchSize, Equals, DefDMLBatchSize) + c.Assert(vars.MemQuotaQuery, Equals, int64(DefTiDBMemQuotaQuery)) + c.Assert(vars.MemQuotaHashJoin, Equals, int64(DefTiDBMemQuotaHashJoin)) + c.Assert(vars.MemQuotaMergeJoin, Equals, int64(DefTiDBMemQuotaMergeJoin)) + c.Assert(vars.MemQuotaSort, Equals, int64(DefTiDBMemQuotaSort)) + c.Assert(vars.MemQuotaTopn, Equals, int64(DefTiDBMemQuotaTopn)) + c.Assert(vars.MemQuotaIndexLookupReader, Equals, int64(DefTiDBMemQuotaIndexLookupReader)) + c.Assert(vars.MemQuotaIndexLookupJoin, Equals, int64(DefTiDBMemQuotaIndexLookupJoin)) + c.Assert(vars.MemQuotaNestedLoopApply, Equals, int64(DefTiDBMemQuotaNestedLoopApply)) +} + func (s *testVarsutilSuite) TestVarsutil(c *C) { defer testleak.AfterTest(c)() v := NewSessionVars() @@ -176,6 +200,10 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(err, IsNil) c.Assert(val, Equals, "0") c.Assert(v.EnableStreaming, Equals, false) + + c.Assert(v.OptimizerSelectivityLevel, Equals, DefTiDBOptimizerSelectivityLevel) + SetSessionSystemVar(v, TiDBOptimizerSelectivityLevel, types.NewIntDatum(1)) + c.Assert(v.OptimizerSelectivityLevel, Equals, 1) } type mockGlobalAccessor struct { diff --git a/statistics/boostrap.go b/statistics/boostrap.go index 8df521565221e..6971b51fe798d 100644 --- a/statistics/boostrap.go +++ b/statistics/boostrap.go @@ -61,7 +61,7 @@ func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (statsCache, error) { chk := rc[0].NewChunk() iter := chunk.NewIterator4Chunk(chk) for { - err := rc[0].NextChunk(context.TODO(), chk) + err := rc[0].Next(context.TODO(), chk) if err != nil { return nil, errors.Trace(err) } @@ -111,7 +111,7 @@ func initStatsHistograms4Chunk(is infoschema.InfoSchema, tables statsCache, iter continue } hist := NewHistogram(id, ndv, nullCount, version, &colInfo.FieldType, 0, totColSize) - table.Columns[hist.ID] = &Column{Histogram: *hist, Info: colInfo} + table.Columns[hist.ID] = &Column{Histogram: *hist, Info: colInfo, Count: nullCount} } } } @@ -128,7 +128,7 @@ func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, tables statsCache chk := rc[0].NewChunk() iter := chunk.NewIterator4Chunk(chk) for { - err := rc[0].NextChunk(context.TODO(), chk) + err := rc[0].Next(context.TODO(), chk) if err != nil { return errors.Trace(err) } @@ -198,7 +198,7 @@ func (h *Handle) initStatsBuckets(tables statsCache) error { chk := rc[0].NewChunk() iter := chunk.NewIterator4Chunk(chk) for { - err := rc[0].NextChunk(context.TODO(), chk) + err := rc[0].Next(context.TODO(), chk) if err != nil { return errors.Trace(err) } diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index a7f402f3600c8..181fbf2aa23fd 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" - tipb "github.com/pingcap/tipb/go-tipb" + "github.com/pingcap/tipb/go-tipb" "github.com/spaolacci/murmur3" ) @@ -54,6 +54,18 @@ func (c *CMSketch) InsertBytes(bytes []byte) { } } +// setValue sets the count for value that hashed into (h1, h2). +func (c *CMSketch) setValue(h1, h2 uint64, count uint32) { + oriCount := c.queryHashValue(h1, h2) + c.count += uint64(count) - uint64(oriCount) + // let it overflow naturally + deltaCount := count - oriCount + for i := range c.table { + j := (h1 + h2*uint64(i)) % uint64(c.width) + c.table[i][j] = c.table[i][j] + deltaCount + } +} + func (c *CMSketch) queryValue(sc *stmtctx.StatementContext, val types.Datum) (uint32, error) { bytes, err := codec.EncodeValue(sc, nil, val) if err != nil { @@ -64,6 +76,10 @@ func (c *CMSketch) queryValue(sc *stmtctx.StatementContext, val types.Datum) (ui func (c *CMSketch) queryBytes(bytes []byte) uint32 { h1, h2 := murmur3.Sum128(bytes) + return c.queryHashValue(h1, h2) +} + +func (c *CMSketch) queryHashValue(h1, h2 uint64) uint32 { vals := make([]uint32, c.depth) min := uint32(math.MaxUint32) for i := range c.table { @@ -173,3 +189,15 @@ func (c *CMSketch) Equal(rc *CMSketch) bool { } return true } + +func (c *CMSketch) copy() *CMSketch { + if c == nil { + return nil + } + tbl := make([][]uint32, c.depth) + for i := range tbl { + tbl[i] = make([]uint32, c.width) + copy(tbl[i], c.table[i]) + } + return &CMSketch{count: c.count, width: c.width, depth: c.depth, table: tbl} +} diff --git a/statistics/ddl.go b/statistics/ddl.go index 145cb0b282906..cb5ea99becc4d 100644 --- a/statistics/ddl.go +++ b/statistics/ddl.go @@ -101,7 +101,7 @@ func (h *Handle) insertColStats2KV(tableID int64, colInfo *model.ColumnInfo) err return errors.Trace(err) } chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -118,16 +118,8 @@ func (h *Handle) insertColStats2KV(tableID int64, colInfo *model.ColumnInfo) err return errors.Trace(err) } } else { - var totColSize int64 - switch colInfo.Tp { - case mysql.TypeFloat, mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, - mysql.TypeDouble, mysql.TypeYear, mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDecimal: - totColSize = 0 - default: - totColSize = int64(len(value.GetBytes())) - } // If this stats exists, we insert histogram meta first, the distinct_count will always be one. - _, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, tot_col_size) values (%d, %d, 0, %d, 1, %d)", h.ctx.Txn().StartTS(), tableID, colInfo.ID, totColSize*count)) + _, err = exec.Execute(ctx, fmt.Sprintf("insert into mysql.stats_histograms (version, table_id, is_index, hist_id, distinct_count, tot_col_size) values (%d, %d, 0, %d, 1, %d)", h.ctx.Txn().StartTS(), tableID, colInfo.ID, int64(len(value.GetBytes()))*count)) if err != nil { return errors.Trace(err) } diff --git a/statistics/ddl_test.go b/statistics/ddl_test.go index 46ee0f6ed12a1..92317973de2a2 100644 --- a/statistics/ddl_test.go +++ b/statistics/ddl_test.go @@ -32,14 +32,14 @@ func (s *testStatsCacheSuite) TestDDLAfterLoad(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - statsTbl := do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) recordCount := 1000 for i := 0; i < recordCount; i++ { testKit.MustExec("insert into t values (?, ?)", i, i+1) } testKit.MustExec("analyze table t") - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) // add column testKit.MustExec("alter table t add column c10 int") @@ -69,7 +69,7 @@ func (s *testStatsCacheSuite) TestDDLTable(c *C) { err = h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(err, IsNil) h.Update(is) - statsTbl := h.GetTableStats(tableInfo.ID) + statsTbl := h.GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) testKit.MustExec("create table t1 (c1 int, c2 int, index idx(c1))") @@ -80,7 +80,7 @@ func (s *testStatsCacheSuite) TestDDLTable(c *C) { err = h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(err, IsNil) h.Update(is) - statsTbl = h.GetTableStats(tableInfo.ID) + statsTbl = h.GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) } @@ -104,10 +104,10 @@ func (s *testStatsCacheSuite) TestDDLHistogram(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - statsTbl := do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) sc := new(stmtctx.StatementContext) - c.Assert(statsTbl.ColumnIsInvalid(sc, tableInfo.Columns[2].ID), IsTrue) + c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].NullCount, Equals, int64(2)) c.Check(statsTbl.Columns[tableInfo.Columns[2].ID].NDV, Equals, int64(0)) testKit.MustExec("alter table t add column c3 int NOT NULL") @@ -118,7 +118,7 @@ func (s *testStatsCacheSuite) TestDDLHistogram(c *C) { tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo = tbl.Meta() - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) sc = new(stmtctx.StatementContext) count, err := statsTbl.ColumnEqualRowCount(sc, types.NewIntDatum(0), tableInfo.Columns[3].ID) @@ -136,7 +136,7 @@ func (s *testStatsCacheSuite) TestDDLHistogram(c *C) { tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo = tbl.Meta() - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) // If we don't use original default value, we will get a pseudo table. c.Assert(statsTbl.Pseudo, IsFalse) @@ -148,10 +148,10 @@ func (s *testStatsCacheSuite) TestDDLHistogram(c *C) { tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo = tbl.Meta() - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) sc = new(stmtctx.StatementContext) - c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(), Equals, 3.0) + c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(statsTbl.Count), Equals, 3.0) testKit.MustExec("create index i on t(c2, c1)") testKit.MustExec("analyze table t") diff --git a/statistics/dump.go b/statistics/dump.go index 16fa0c116c84e..19bd55ad38044 100644 --- a/statistics/dump.go +++ b/statistics/dump.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - tipb "github.com/pingcap/tipb/go-tipb" + "github.com/pingcap/tipb/go-tipb" ) // JSONTable is used for dumping statistics. diff --git a/statistics/dump_test.go b/statistics/dump_test.go index dc95eab6f671a..e7fb09c404ef8 100644 --- a/statistics/dump_test.go +++ b/statistics/dump_test.go @@ -61,6 +61,6 @@ func (s *testDumpStatsSuite) TestConversion(c *C) { c.Assert(err, IsNil) loadTbl, err := h.LoadStatsFromJSON(tableInfo.Meta(), jsonTbl) c.Assert(err, IsNil) - tbl := h.GetTableStats(tableInfo.Meta().ID) + tbl := h.GetTableStats(tableInfo.Meta()) assertTableEqual(c, loadTbl, tbl) } diff --git a/statistics/feedback.go b/statistics/feedback.go index 889e89b1f6ea0..d61999d03ce4b 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -15,6 +15,7 @@ package statistics import ( "bytes" + "encoding/gob" "math" "math/rand" "sort" @@ -30,6 +31,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/ranger" log "github.com/sirupsen/logrus" + "github.com/spaolacci/murmur3" ) // `feedback` represents the total scan count in range [lower, upper). @@ -126,17 +128,8 @@ func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { q.Invalidate() return } - length := len(counts) - // The `counts` was the output count of each push down executor. - if counts[length-1] != -1 { - metrics.DistSQLScanKeysPartialHistogram.Observe(float64(counts[0])) - q.actual += counts[0] - return - } - // The counts is the scan count of each range now. sum := int64(0) - rangeCounts := counts[:length-1] - for _, count := range rangeCounts { + for _, count := range counts { sum += count } metrics.DistSQLScanKeysPartialHistogram.Observe(float64(sum)) @@ -145,10 +138,10 @@ func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { return } - if q.hist.tp.Tp == mysql.TypeLong { - startKey = tablecodec.CutRowKeyPrefix(startKey) - } else { + if q.hist.tp.Tp == mysql.TypeBlob { startKey = tablecodec.CutIndexPrefix(startKey) + } else { + startKey = tablecodec.CutRowKeyPrefix(startKey) } // Find the range that startKey falls in. idx := sort.Search(len(q.feedback), func(i int) bool { @@ -160,13 +153,13 @@ func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { } // If the desc is true, the counts is reversed, so here we need to reverse it back. if q.desc { - for i := 0; i < len(rangeCounts)/2; i++ { - j := len(rangeCounts) - i - 1 - rangeCounts[i], rangeCounts[j] = rangeCounts[j], rangeCounts[i] + for i := 0; i < len(counts)/2; i++ { + j := len(counts) - i - 1 + counts[i], counts[j] = counts[j], counts[i] } } // Update the feedback count info. - for i, count := range rangeCounts { + for i, count := range counts { if i+idx >= len(q.feedback) { q.Invalidate() break @@ -176,7 +169,7 @@ func (q *QueryFeedback) Update(startKey kv.Key, counts []int64) { return } -// DecodeInt decodes the int value stored in the feedback, only used for test. +// DecodeInt decodes the int value stored in the feedback. func (q *QueryFeedback) DecodeInt() error { for _, fb := range q.feedback { _, v, err := codec.DecodeInt(fb.lower.GetBytes()) @@ -202,47 +195,45 @@ type BucketFeedback struct { } // buildBucketFeedback build the feedback for each bucket from the histogram feedback. -func buildBucketFeedback(h *Histogram, feedbacks []*QueryFeedback) (map[int]*BucketFeedback, int) { +func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*BucketFeedback, int) { bktID2FB := make(map[int]*BucketFeedback) total := 0 - for _, feedback := range feedbacks { - for _, ran := range feedback.feedback { - idx, _ := h.Bounds.LowerBound(0, ran.lower) - bktIdx := 0 - // The last bucket also stores the feedback that falls outside the upper bound. - if idx >= h.Bounds.NumRows()-2 { - bktIdx = h.Len() - 1 - } else { - bktIdx = idx / 2 - // Make sure that this feedback lies within the bucket. - if chunk.Compare(h.Bounds.GetRow(2*bktIdx+1), 0, ran.upper) < 0 { - continue - } - } - total++ - bkt := bktID2FB[bktIdx] - if bkt == nil { - bkt = &BucketFeedback{lower: h.GetLower(bktIdx), upper: h.GetUpper(bktIdx)} - bktID2FB[bktIdx] = bkt - } - bkt.feedback = append(bkt.feedback, ran) - // Update the bound if necessary. - res, err := bkt.lower.CompareDatum(nil, ran.lower) - if err != nil { - log.Debugf("compare datum %v with %v failed, err: %v", bkt.lower, ran.lower, errors.ErrorStack(err)) - continue - } - if res > 0 { - bkt.lower = ran.lower - } - res, err = bkt.upper.CompareDatum(nil, ran.upper) - if err != nil { - log.Debugf("compare datum %v with %v failed, err: %v", bkt.upper, ran.upper, errors.ErrorStack(err)) + for _, ran := range feedback.feedback { + idx, _ := h.Bounds.LowerBound(0, ran.lower) + bktIdx := 0 + // The last bucket also stores the feedback that falls outside the upper bound. + if idx >= h.Bounds.NumRows()-2 { + bktIdx = h.Len() - 1 + } else { + bktIdx = idx / 2 + // Make sure that this feedback lies within the bucket. + if chunk.Compare(h.Bounds.GetRow(2*bktIdx+1), 0, ran.upper) < 0 { continue } - if res < 0 { - bkt.upper = ran.upper - } + } + total++ + bkt := bktID2FB[bktIdx] + if bkt == nil { + bkt = &BucketFeedback{lower: h.GetLower(bktIdx), upper: h.GetUpper(bktIdx)} + bktID2FB[bktIdx] = bkt + } + bkt.feedback = append(bkt.feedback, ran) + // Update the bound if necessary. + res, err := bkt.lower.CompareDatum(nil, ran.lower) + if err != nil { + log.Debugf("compare datum %v with %v failed, err: %v", bkt.lower, ran.lower, errors.ErrorStack(err)) + continue + } + if res > 0 { + bkt.lower = ran.lower + } + res, err = bkt.upper.CompareDatum(nil, ran.upper) + if err != nil { + log.Debugf("compare datum %v with %v failed, err: %v", bkt.upper, ran.upper, errors.ErrorStack(err)) + continue + } + if res < 0 { + bkt.upper = ran.upper } } return bktID2FB, total @@ -438,8 +429,8 @@ func mergeBuckets(bkts []bucket, isNewBuckets []bool, totalCount float64) []buck return bkts } -func splitBuckets(h *Histogram, feedbacks []*QueryFeedback) ([]bucket, []bool, int64) { - bktID2FB, fbNum := buildBucketFeedback(h, feedbacks) +func splitBuckets(h *Histogram, feedback *QueryFeedback) ([]bucket, []bool, int64) { + bktID2FB, fbNum := buildBucketFeedback(h, feedback) counts := make([]int64, 0, h.Len()) for i := 0; i < h.Len(); i++ { bkt, ok := bktID2FB[i] @@ -478,8 +469,8 @@ func splitBuckets(h *Histogram, feedbacks []*QueryFeedback) ([]bucket, []bool, i } // UpdateHistogram updates the histogram according buckets. -func UpdateHistogram(h *Histogram, feedbacks []*QueryFeedback) *Histogram { - buckets, isNewBuckets, totalCount := splitBuckets(h, feedbacks) +func UpdateHistogram(h *Histogram, feedback *QueryFeedback) *Histogram { + buckets, isNewBuckets, totalCount := splitBuckets(h, feedback) buckets = mergeBuckets(buckets, isNewBuckets, float64(totalCount)) return buildNewHistogram(h, buckets) } @@ -493,3 +484,127 @@ func buildNewHistogram(h *Histogram, buckets []bucket) *Histogram { } return hist } + +// queryFeedback is used to serialize the QueryFeedback. +type queryFeedback struct { + IntRanges []int64 + HashValues []uint64 // HashValues is the murmur hash values for each index point. + IndexRanges [][]byte + Counts []int64 // Counts is the number of scan keys in each range. +} + +func encodePKFeedback(q *QueryFeedback) (*queryFeedback, error) { + pb := &queryFeedback{} + for _, fb := range q.feedback { + err := q.DecodeInt() + if err != nil { + return nil, errors.Trace(err) + } + // There is no need to update the point queries. + if fb.upper.GetInt64()-fb.lower.GetInt64() <= 1 { + continue + } + pb.IntRanges = append(pb.IntRanges, fb.lower.GetInt64(), fb.upper.GetInt64()) + pb.Counts = append(pb.Counts, fb.count) + } + return pb, nil +} + +func encodeIndexFeedback(q *QueryFeedback) *queryFeedback { + pb := &queryFeedback{} + var pointCounts []int64 + for _, fb := range q.feedback { + if bytes.Equal(kv.Key(fb.lower.GetBytes()).PrefixNext(), fb.upper.GetBytes()) { + h1, h2 := murmur3.Sum128(fb.lower.GetBytes()) + pb.HashValues = append(pb.HashValues, h1, h2) + pointCounts = append(pointCounts, fb.count) + } else { + pb.IndexRanges = append(pb.IndexRanges, fb.lower.GetBytes(), fb.upper.GetBytes()) + pb.Counts = append(pb.Counts, fb.count) + } + } + pb.Counts = append(pb.Counts, pointCounts...) + return pb +} + +func encodeFeedback(q *QueryFeedback) ([]byte, error) { + var pb *queryFeedback + var err error + if q.hist.tp.Tp == mysql.TypeBlob { + pb = encodeIndexFeedback(q) + } else { + pb, err = encodePKFeedback(q) + if err != nil { + return nil, errors.Trace(err) + } + } + var buf bytes.Buffer + enc := gob.NewEncoder(&buf) + err = enc.Encode(pb) + if err != nil { + return nil, errors.Trace(err) + } + return buf.Bytes(), nil +} + +func decodeFeedback(val []byte, q *QueryFeedback, c *CMSketch) error { + buf := bytes.NewBuffer(val) + dec := gob.NewDecoder(buf) + pb := &queryFeedback{} + err := dec.Decode(pb) + if err != nil { + return errors.Trace(err) + } + // decode feedback for index + if len(pb.IndexRanges) > 0 { + // decode the index range feedback + for i := 0; i < len(pb.IndexRanges); i += 2 { + lower, upper := types.NewBytesDatum(pb.IndexRanges[i]), types.NewBytesDatum(pb.IndexRanges[i+1]) + q.feedback = append(q.feedback, feedback{&lower, &upper, pb.Counts[i/2], 0}) + } + if c == nil { + return nil + } + // decode the index point feedback, just set value count in CM Sketch + start := len(pb.IndexRanges) / 2 + for i := 0; i < len(pb.HashValues); i += 2 { + c.setValue(pb.HashValues[i], pb.HashValues[i+1], uint32(pb.Counts[start+i/2])) + } + return nil + } + // decode feedback for primary key + for i := 0; i < len(pb.IntRanges); i += 2 { + lower, upper := types.NewIntDatum(pb.IntRanges[i]), types.NewIntDatum(pb.IntRanges[i+1]) + q.feedback = append(q.feedback, feedback{&lower, &upper, pb.Counts[i/2], 0}) + } + return nil +} + +// Equal tests if two query feedback equal, it is only used in test. +func (q *QueryFeedback) Equal(rq *QueryFeedback) bool { + if len(q.feedback) != len(rq.feedback) { + return false + } + for i, fb := range q.feedback { + rfb := rq.feedback[i] + if fb.count != rfb.count { + return false + } + if fb.lower.Kind() == types.KindInt64 { + if fb.lower.GetInt64() != rfb.lower.GetInt64() { + return false + } + if fb.upper.GetInt64() != rfb.upper.GetInt64() { + return false + } + } else { + if bytes.Compare(fb.lower.GetBytes(), rfb.lower.GetBytes()) != 0 { + return false + } + if bytes.Compare(fb.upper.GetBytes(), rfb.upper.GetBytes()) != 0 { + return false + } + } + } + return true +} diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go index 69eb0f60eb379..d72742727ca11 100644 --- a/statistics/feedback_test.go +++ b/statistics/feedback_test.go @@ -17,6 +17,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/codec" ) var _ = Suite(&testFeedbackSuite{}) @@ -68,8 +69,8 @@ func (s *testFeedbackSuite) TestUpdateHistogram(c *C) { originBucketCount := defaultBucketCount defaultBucketCount = 5 defer func() { defaultBucketCount = originBucketCount }() - c.Assert(UpdateHistogram(q.Hist(), []*QueryFeedback{q}).ToString(0), Equals, - "column:0 ndv:0\n"+ + c.Assert(UpdateHistogram(q.Hist(), q).ToString(0), Equals, + "column:0 ndv:0 totColSize:0\n"+ "num: 10000\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ "num: 10003\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ "num: 10021\tlower_bound: 4\tupper_bound: 20\trepeats: 0\n"+ @@ -85,9 +86,9 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { } q := NewQueryFeedback(0, genHistogram(), 0, false) q.feedback = feedbacks - buckets, isNewBuckets, totalCount := splitBuckets(q.Hist(), []*QueryFeedback{q}) + buckets, isNewBuckets, totalCount := splitBuckets(q.Hist(), q) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, - "column:0 ndv:0\n"+ + "column:0 ndv:0 totColSize:0\n"+ "num: 1\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ "num: 1\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ "num: 1\tlower_bound: 5\tupper_bound: 7\trepeats: 0\n"+ @@ -104,9 +105,9 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { } q = NewQueryFeedback(0, genHistogram(), 0, false) q.feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), []*QueryFeedback{q}) + buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), q) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, - "column:0 ndv:0\n"+ + "column:0 ndv:0 totColSize:0\n"+ "num: 100000\tlower_bound: 0\tupper_bound: 1\trepeats: 0\n"+ "num: 100000\tlower_bound: 2\tupper_bound: 3\trepeats: 0\n"+ "num: 100000\tlower_bound: 5\tupper_bound: 7\trepeats: 0\n"+ @@ -124,9 +125,9 @@ func (s *testFeedbackSuite) TestSplitBuckets(c *C) { } q = NewQueryFeedback(0, h, 0, false) q.feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), []*QueryFeedback{q}) + buckets, isNewBuckets, totalCount = splitBuckets(q.Hist(), q) c.Assert(buildNewHistogram(q.Hist(), buckets).ToString(0), Equals, - "column:0 ndv:0\n"+ + "column:0 ndv:0 totColSize:0\n"+ "num: 1000000\tlower_bound: 0\tupper_bound: 1000000\trepeats: 0") c.Assert(isNewBuckets, DeepEquals, []bool{false}) c.Assert(totalCount, Equals, int64(1000000)) @@ -147,14 +148,14 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { counts: []int64{1}, isNewBuckets: []bool{false}, bucketCount: 1, - result: "column:0 ndv:0\nnum: 1\tlower_bound: 1\tupper_bound: 2\trepeats: 0", + result: "column:0 ndv:0 totColSize:0\nnum: 1\tlower_bound: 1\tupper_bound: 2\trepeats: 0", }, { points: []int64{1, 2, 2, 3, 3, 4}, counts: []int64{100000, 1, 1}, isNewBuckets: []bool{false, false, false}, bucketCount: 2, - result: "column:0 ndv:0\n" + + result: "column:0 ndv:0 totColSize:0\n" + "num: 100000\tlower_bound: 1\tupper_bound: 2\trepeats: 0\n" + "num: 100002\tlower_bound: 2\tupper_bound: 4\trepeats: 0", }, @@ -164,7 +165,7 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { counts: []int64{1, 1, 100000, 100000}, isNewBuckets: []bool{false, false, false, false}, bucketCount: 3, - result: "column:0 ndv:0\n" + + result: "column:0 ndv:0 totColSize:0\n" + "num: 2\tlower_bound: 1\tupper_bound: 3\trepeats: 0\n" + "num: 100002\tlower_bound: 3\tupper_bound: 4\trepeats: 0\n" + "num: 200002\tlower_bound: 4\tupper_bound: 5\trepeats: 0", @@ -184,3 +185,34 @@ func (s *testFeedbackSuite) TestMergeBuckets(c *C) { c.Assert(result, Equals, t.result) } } + +func encodeInt(v int64) *types.Datum { + val := codec.EncodeInt(nil, v) + d := types.NewBytesDatum(val) + return &d +} + +func (s *testFeedbackSuite) TestFeedbackEncoding(c *C) { + hist := NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 0, 0) + q := &QueryFeedback{hist: hist} + q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(3), 1, 0}) + q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(5), 1, 0}) + val, err := encodeFeedback(q) + c.Assert(err, IsNil) + rq := &QueryFeedback{} + c.Assert(decodeFeedback(val, rq, nil), IsNil) + c.Assert(q.Equal(rq), IsTrue) + + hist.tp = types.NewFieldType(mysql.TypeBlob) + q = &QueryFeedback{hist: hist} + q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(3), 1, 0}) + q.feedback = append(q.feedback, feedback{encodeInt(0), encodeInt(1), 1, 0}) + val, err = encodeFeedback(q) + c.Assert(err, IsNil) + rq = &QueryFeedback{} + cms := NewCMSketch(4, 4) + c.Assert(decodeFeedback(val, rq, cms), IsNil) + c.Assert(cms.queryBytes(codec.EncodeInt(nil, 0)), Equals, uint32(1)) + q.feedback = q.feedback[:1] + c.Assert(q.Equal(rq), IsTrue) +} diff --git a/statistics/handle.go b/statistics/handle.go index 1a687224b14a7..fdb9ec9474604 100644 --- a/statistics/handle.go +++ b/statistics/handle.go @@ -21,6 +21,7 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/sqlexec" log "github.com/sirupsen/logrus" @@ -71,8 +72,8 @@ func (h *Handle) Clear() { h.globalMap = make(tableDeltaMap) } -// For now, we do not use the query feedback, so just set it to 1. -const maxQueryFeedBackCount = 1 +// MaxQueryFeedbackCount is the max number of feedback that cache in memory. +var MaxQueryFeedbackCount = 1 << 10 // NewHandle creates a Handle for update stats. func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { @@ -83,7 +84,7 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { listHead: &SessionStatsCollector{mapper: make(tableDeltaMap)}, globalMap: make(tableDeltaMap), Lease: lease, - feedback: make([]*QueryFeedback, 0, maxQueryFeedBackCount), + feedback: make([]*QueryFeedback, 0, MaxQueryFeedbackCount), } handle.statsCache.Store(statsCache{}) return handle @@ -145,10 +146,12 @@ func (h *Handle) Update(is infoschema.InfoSchema) error { } // GetTableStats retrieves the statistics table from cache, and the cache will be updated by a goroutine. -func (h *Handle) GetTableStats(tblID int64) *Table { - tbl, ok := h.statsCache.Load().(statsCache)[tblID] +func (h *Handle) GetTableStats(tblInfo *model.TableInfo) *Table { + tbl, ok := h.statsCache.Load().(statsCache)[tblInfo.ID] if !ok { - return PseudoTable(tblID) + tbl = PseudoTable(tblInfo) + h.UpdateTableStats([]*Table{tbl}, nil) + return tbl } return tbl } @@ -179,7 +182,11 @@ func (h *Handle) UpdateTableStats(tables []*Table, deletedIDs []int64) { func (h *Handle) LoadNeededHistograms() error { cols := histogramNeededColumns.allCols() for _, col := range cols { - tbl := h.GetTableStats(col.tableID).copy() + tbl, ok := h.statsCache.Load().(statsCache)[col.tableID] + if !ok { + continue + } + tbl = tbl.copy() c, ok := tbl.Columns[col.columnID] if !ok || c.Len() > 0 { histogramNeededColumns.delete(col) diff --git a/statistics/handle_test.go b/statistics/handle_test.go index 4dc0e387c5164..94efbf44062c6 100644 --- a/statistics/handle_test.go +++ b/statistics/handle_test.go @@ -76,28 +76,28 @@ func (s *testStatsCacheSuite) TestStatsCache(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - statsTbl := do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsTrue) testKit.MustExec("analyze table t") - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) testKit.MustExec("create index idx_t on t(c1)") is = do.InfoSchema() - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) // If index is build, but stats is not updated. statsTbl can also work. c.Assert(statsTbl.Pseudo, IsFalse) // But the added index will not work. c.Assert(statsTbl.Indices[int64(1)], IsNil) testKit.MustExec("analyze table t") - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) // If the new schema drop a column, the table stats can still work. testKit.MustExec("alter table t drop column c2") is = do.InfoSchema() do.StatsHandle().Clear() do.StatsHandle().Update(is) - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) // If the new schema add a column, the table stats can still work. @@ -106,7 +106,7 @@ func (s *testStatsCacheSuite) TestStatsCache(c *C) { do.StatsHandle().Clear() do.StatsHandle().Update(is) - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Pseudo, IsFalse) } @@ -149,11 +149,11 @@ func (s *testStatsCacheSuite) TestStatsStoreAndLoad(c *C) { tableInfo := tbl.Meta() testKit.MustExec("analyze table t") - statsTbl1 := do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl1 := do.StatsHandle().GetTableStats(tableInfo) do.StatsHandle().Clear() do.StatsHandle().Update(is) - statsTbl2 := do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl2 := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl2.Pseudo, IsFalse) c.Assert(statsTbl2.Count, Equals, int64(recordCount)) assertTableEqual(c, statsTbl1, statsTbl2) @@ -170,7 +170,7 @@ func (s *testStatsCacheSuite) TestEmptyTable(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - statsTbl := do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) sc := new(stmtctx.StatementContext) count := statsTbl.ColumnGreaterRowCount(sc, types.NewDatum(1), tableInfo.Columns[0].ID) c.Assert(count, Equals, 0.0) @@ -188,7 +188,7 @@ func (s *testStatsCacheSuite) TestColumnIDs(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - statsTbl := do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) sc := new(stmtctx.StatementContext) count := statsTbl.ColumnLessRowCount(sc, types.NewDatum(2), tableInfo.Columns[0].ID) c.Assert(count, Equals, float64(1)) @@ -201,7 +201,7 @@ func (s *testStatsCacheSuite) TestColumnIDs(c *C) { tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo = tbl.Meta() - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) // At that time, we should get c2's stats instead of c1's. count = statsTbl.ColumnLessRowCount(sc, types.NewDatum(2), tableInfo.Columns[0].ID) c.Assert(count, Equals, 0.0) @@ -219,20 +219,20 @@ func (s *testStatsCacheSuite) TestAvgColLen(c *C) { tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - statsTbl := do.StatsHandle().GetTableStats(tableInfo.ID) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(), Equals, 8.0) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) + c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count), Equals, 8.0) // The size of varchar type is LEN + BYTE, here is 1 + 7 = 8 - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(), Equals, 4.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(), Equals, 16.0) + c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count), Equals, 8.0) + c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count), Equals, 4.0) + c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count), Equals, 16.0) testKit.MustExec("insert into t values(132, '123456789112', 1232.3, '2018-03-07 19:17:29')") testKit.MustExec("analyze table t") - statsTbl = do.StatsHandle().GetTableStats(tableInfo.ID) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(), Equals, 10.5) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(), Equals, 4.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(), Equals, 16.0) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) + c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count), Equals, 8.0) + c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count), Equals, 10.5) + c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count), Equals, 4.0) + c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count), Equals, 16.0) } func (s *testStatsCacheSuite) TestVersion(c *C) { @@ -252,7 +252,7 @@ func (s *testStatsCacheSuite) TestVersion(c *C) { h.Update(is) c.Assert(h.LastVersion, Equals, uint64(2)) c.Assert(h.PrevLastVersion, Equals, uint64(0)) - statsTbl1 := h.GetTableStats(tableInfo1.ID) + statsTbl1 := h.GetTableStats(tableInfo1) c.Assert(statsTbl1.Pseudo, IsFalse) testKit.MustExec("create table t2 (c1 int, c2 int)") @@ -266,7 +266,7 @@ func (s *testStatsCacheSuite) TestVersion(c *C) { h.Update(is) c.Assert(h.LastVersion, Equals, uint64(2)) c.Assert(h.PrevLastVersion, Equals, uint64(2)) - statsTbl2 := h.GetTableStats(tableInfo2.ID) + statsTbl2 := h.GetTableStats(tableInfo2) c.Assert(statsTbl2.Pseudo, IsFalse) testKit.MustExec("insert t1 values(1,2)") @@ -275,7 +275,7 @@ func (s *testStatsCacheSuite) TestVersion(c *C) { h.Update(is) c.Assert(h.LastVersion, Equals, uint64(4)) c.Assert(h.PrevLastVersion, Equals, uint64(2)) - statsTbl1 = h.GetTableStats(tableInfo1.ID) + statsTbl1 = h.GetTableStats(tableInfo1) c.Assert(statsTbl1.Count, Equals, int64(1)) testKit.MustExec("insert t2 values(1,2)") @@ -285,7 +285,7 @@ func (s *testStatsCacheSuite) TestVersion(c *C) { h.Update(is) c.Assert(h.LastVersion, Equals, uint64(4)) c.Assert(h.PrevLastVersion, Equals, uint64(4)) - statsTbl2 = h.GetTableStats(tableInfo2.ID) + statsTbl2 = h.GetTableStats(tableInfo2) c.Assert(statsTbl2.Count, Equals, int64(1)) testKit.MustExec("insert t2 values(1,2)") @@ -295,7 +295,7 @@ func (s *testStatsCacheSuite) TestVersion(c *C) { h.Update(is) c.Assert(h.LastVersion, Equals, uint64(4)) c.Assert(h.PrevLastVersion, Equals, uint64(4)) - statsTbl2 = h.GetTableStats(tableInfo2.ID) + statsTbl2 = h.GetTableStats(tableInfo2) c.Assert(statsTbl2.Count, Equals, int64(1)) // We add an index and analyze it, but DDL doesn't load. @@ -303,13 +303,13 @@ func (s *testStatsCacheSuite) TestVersion(c *C) { testKit.MustExec("analyze table t2") // load it with old schema. h.Update(is) - statsTbl2 = h.GetTableStats(tableInfo2.ID) + statsTbl2 = h.GetTableStats(tableInfo2) c.Assert(statsTbl2.Pseudo, IsFalse) c.Assert(statsTbl2.Columns[int64(3)], IsNil) // Next time DDL updated. is = do.InfoSchema() h.Update(is) - statsTbl2 = h.GetTableStats(tableInfo2.ID) + statsTbl2 = h.GetTableStats(tableInfo2) c.Assert(statsTbl2.Pseudo, IsFalse) // We can read it without analyze again! Thanks for PrevLastVersion. c.Assert(statsTbl2.Columns[int64(3)], NotNil) @@ -319,32 +319,41 @@ func (s *testStatsCacheSuite) TestLoadHist(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec("create table t (c1 int, c2 int)") + testKit.MustExec("create table t (c1 varchar(12), c2 char(12))") do := s.do h := do.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(err, IsNil) rowCount := 10 for i := 0; i < rowCount; i++ { - testKit.MustExec("insert into t values(1,2)") + testKit.MustExec("insert into t values('a','ddd')") } testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := tbl.Meta() - oldStatsTbl := h.GetTableStats(tableInfo.ID) + oldStatsTbl := h.GetTableStats(tableInfo) for i := 0; i < rowCount; i++ { - testKit.MustExec("insert into t values(1,2)") + testKit.MustExec("insert into t values('bb','sdfga')") } h.DumpStatsDeltaToKV() h.Update(do.InfoSchema()) - newStatsTbl := h.GetTableStats(tableInfo.ID) + newStatsTbl := h.GetTableStats(tableInfo) // The stats table is updated. c.Assert(oldStatsTbl == newStatsTbl, IsFalse) - // The histograms is not updated. + // Only the TotColSize of histograms is updated. for id, hist := range oldStatsTbl.Columns { - c.Assert(hist, Equals, newStatsTbl.Columns[id]) + c.Assert(hist.TotColSize, Less, newStatsTbl.Columns[id].TotColSize) + + temp := hist.TotColSize + hist.TotColSize = newStatsTbl.Columns[id].TotColSize + c.Assert(statistics.HistogramEqual(&hist.Histogram, &newStatsTbl.Columns[id].Histogram, false), IsTrue) + hist.TotColSize = temp + + c.Assert(hist.CMSketch.Equal(newStatsTbl.Columns[id].CMSketch), IsTrue) + c.Assert(hist.Count, Equals, newStatsTbl.Columns[id].Count) + c.Assert(hist.Info, Equals, newStatsTbl.Columns[id].Info) } // Add column c3, we only update c3. testKit.MustExec("alter table t add column c3 int") @@ -355,7 +364,7 @@ func (s *testStatsCacheSuite) TestLoadHist(c *C) { c.Assert(err, IsNil) tableInfo = tbl.Meta() h.Update(is) - newStatsTbl2 := h.GetTableStats(tableInfo.ID) + newStatsTbl2 := h.GetTableStats(tableInfo) c.Assert(newStatsTbl2 == newStatsTbl, IsFalse) // The histograms is not updated. for id, hist := range newStatsTbl.Columns { @@ -381,10 +390,10 @@ func (s *testStatsCacheSuite) TestInitStats(c *C) { h.Clear() c.Assert(h.InitStats(is), IsNil) - table0 := h.GetTableStats(tbl.Meta().ID) + table0 := h.GetTableStats(tbl.Meta()) h.Clear() c.Assert(h.Update(is), IsNil) - table1 := h.GetTableStats(tbl.Meta().ID) + table1 := h.GetTableStats(tbl.Meta()) assertTableEqual(c, table0, table1) h.Lease = 0 } @@ -406,7 +415,7 @@ func (s *testStatsUpdateSuite) TestLoadStats(c *C) { tableInfo := tbl.Meta() h := do.StatsHandle() time.Sleep(1 * time.Second) - stat := h.GetTableStats(tableInfo.ID) + stat := h.GetTableStats(tableInfo) hg := stat.Columns[tableInfo.Columns[0].ID].Histogram c.Assert(hg.Len(), Greater, 0) cms := stat.Columns[tableInfo.Columns[0].ID].CMSketch @@ -422,7 +431,7 @@ func (s *testStatsUpdateSuite) TestLoadStats(c *C) { _, err = stat.ColumnEqualRowCount(testKit.Se.GetSessionVars().StmtCtx, types.NewIntDatum(1), tableInfo.Columns[2].ID) c.Assert(err, IsNil) time.Sleep(1 * time.Second) - stat = h.GetTableStats(tableInfo.ID) + stat = h.GetTableStats(tableInfo) hg = stat.Columns[tableInfo.Columns[2].ID].Histogram c.Assert(hg.Len(), Greater, 0) } diff --git a/statistics/histogram.go b/statistics/histogram.go index d2fe37bfc263b..15ebc173a9fe7 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -105,7 +105,10 @@ func (hg *Histogram) GetUpper(idx int) *types.Datum { } // AvgColSize is the average column size of the histogram. -func (c *Column) AvgColSize() float64 { +func (c *Column) AvgColSize(count int64) float64 { + if count == 0 { + return 0 + } switch c.Histogram.tp.Tp { case mysql.TypeFloat: return 4 @@ -117,10 +120,8 @@ func (c *Column) AvgColSize() float64 { case mysql.TypeNewDecimal: return types.MyDecimalStructSize default: - if c.Count == 0 { - return 0 - } - return float64(c.TotColSize) / float64(c.Count) + // Keep two decimal place. + return math.Round(float64(c.TotColSize)/float64(count)*100) / 100 } } @@ -194,8 +195,14 @@ func SaveStatsToStorage(sctx sessionctx.Context, tableID int64, count int64, isI } txn := sctx.Txn() version := txn.StartTS() - replaceSQL := fmt.Sprintf("replace into mysql.stats_meta (version, table_id, count) values (%d, %d, %d)", version, tableID, count) - _, err = exec.Execute(ctx, replaceSQL) + var sql string + // If the count is less than 0, then we do not want to update the modify count and count. + if count >= 0 { + sql = fmt.Sprintf("replace into mysql.stats_meta (version, table_id, count) values (%d, %d, %d)", version, tableID, count) + } else { + sql = fmt.Sprintf("update mysql.stats_meta set version = %d where table_id = %d", version, tableID) + } + _, err = exec.Execute(ctx, sql) if err != nil { return errors.Trace(err) } @@ -203,7 +210,7 @@ func SaveStatsToStorage(sctx sessionctx.Context, tableID int64, count int64, isI if err != nil { return errors.Trace(err) } - replaceSQL = fmt.Sprintf("replace into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size) values (%d, %d, %d, %d, %d, %d, X'%X', %d)", + replaceSQL := fmt.Sprintf("replace into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size) values (%d, %d, %d, %d, %d, %d, X'%X', %d)", tableID, isIndex, hg.ID, hg.NDV, version, hg.NullCount, data, hg.TotColSize) _, err = exec.Execute(ctx, replaceSQL) if err != nil { @@ -310,7 +317,7 @@ func (hg *Histogram) ToString(idxCols int) string { if idxCols > 0 { strs = append(strs, fmt.Sprintf("index:%d ndv:%d", hg.ID, hg.NDV)) } else { - strs = append(strs, fmt.Sprintf("column:%d ndv:%d", hg.ID, hg.NDV)) + strs = append(strs, fmt.Sprintf("column:%d ndv:%d totColSize:%d", hg.ID, hg.NDV, hg.TotColSize)) } for i := 0; i < hg.Len(); i++ { upperVal, err := ValueToString(hg.GetUpper(i), idxCols) @@ -358,6 +365,10 @@ func (hg *Histogram) greaterAndEqRowCount(value types.Datum) float64 { // lessRowCount estimates the row count where the column less than value. func (hg *Histogram) lessRowCount(value types.Datum) float64 { + // all the values is null + if hg.Bounds == nil { + return 0 + } index, match := hg.Bounds.LowerBound(0, &value) if index == hg.Bounds.NumRows() { return hg.totalRowCount() @@ -389,7 +400,7 @@ func (hg *Histogram) betweenRowCount(a, b types.Datum) float64 { lessCountB := hg.lessRowCount(b) // If lessCountA is not less than lessCountB, it may be that they fall to the same bucket and we cannot estimate // the fraction, so we use `totalCount / NDV` to estimate the row count, but the result should not greater than lessCountB. - if lessCountA >= lessCountB { + if lessCountA >= lessCountB && hg.NDV > 0 { return math.Min(lessCountB, hg.totalRowCount()/float64(hg.NDV)) } return lessCountB - lessCountA @@ -397,9 +408,9 @@ func (hg *Histogram) betweenRowCount(a, b types.Datum) float64 { func (hg *Histogram) totalRowCount() float64 { if hg.Len() == 0 { - return 0 + return float64(hg.NullCount) } - return float64(hg.Buckets[hg.Len()-1].Count) + return float64(hg.Buckets[hg.Len()-1].Count + hg.NullCount) } // mergeBuckets is used to merge every two neighbor buckets. @@ -425,7 +436,7 @@ func (hg *Histogram) mergeBuckets(bucketIdx int) { // getIncreaseFactor will return a factor of data increasing after the last analysis. func (hg *Histogram) getIncreaseFactor(totalCount int64) float64 { - columnCount := int64(hg.totalRowCount()) + hg.NullCount + columnCount := int64(hg.totalRowCount()) if columnCount == 0 { // avoid dividing by 0 return 1.0 @@ -615,10 +626,17 @@ func (c *Column) String() string { } func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum) (float64, error) { + if val.IsNull() { + return float64(c.NullCount), nil + } if c.CMSketch != nil { count, err := c.CMSketch.queryValue(sc, val) return float64(count), errors.Trace(err) } + // all the values is null + if c.Histogram.Bounds == nil { + return 0.0, nil + } return c.Histogram.equalRowCount(val), nil } diff --git a/statistics/sample.go b/statistics/sample.go index 33065fa35f986..279fb883b14f7 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -158,7 +158,7 @@ func (s SampleBuilder) CollectColumnStats() ([]*SampleCollector, *SortedBuilder, chk := s.RecordSet.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err := s.RecordSet.NextChunk(ctx, chk) + err := s.RecordSet.Next(ctx, chk) if err != nil { return nil, nil, errors.Trace(err) } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 1a6afa818d939..2feb50620ebf2 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -44,33 +44,69 @@ const ( colType ) -// checkColumnConstant receives two expressions and makes sure one of them is column and another is constant. -func checkColumnConstant(e []expression.Expression) bool { +const unknownColumnID = math.MinInt64 + +// getConstantColumnID receives two expressions and if one of them is column and another is constant, it returns the +// ID of the column. +func getConstantColumnID(e []expression.Expression) int64 { if len(e) != 2 { - return false + return unknownColumnID } - _, ok1 := e[0].(*expression.Column) + col, ok1 := e[0].(*expression.Column) _, ok2 := e[1].(*expression.Constant) if ok1 && ok2 { - return true + return col.ID } - _, ok1 = e[1].(*expression.Column) + col, ok1 = e[1].(*expression.Column) _, ok2 = e[0].(*expression.Constant) - return ok1 && ok2 + if ok1 && ok2 { + return col.ID + } + return unknownColumnID } -func pseudoSelectivity(exprs []expression.Expression) float64 { +func pseudoSelectivity(t *Table, exprs []expression.Expression) float64 { minFactor := selectionFactor + uniqueCol := make(map[string]bool) for _, expr := range exprs { - if fun, ok := expr.(*expression.ScalarFunction); ok && checkColumnConstant(fun.GetArgs()) { - switch fun.FuncName.L { - case ast.EQ, ast.NullEQ: - minFactor = math.Min(minFactor, 1.0/pseudoEqualRate) - case ast.GE, ast.GT, ast.LE, ast.LT: - minFactor = math.Min(minFactor, 1.0/pseudoLessRate) - // FIXME: To resolve the between case. + fun, ok := expr.(*expression.ScalarFunction) + if !ok { + continue + } + colID := getConstantColumnID(fun.GetArgs()) + if colID == unknownColumnID { + continue + } + switch fun.FuncName.L { + case ast.EQ, ast.NullEQ, ast.In: + col, ok := t.Columns[colID] + if ok && (mysql.HasUniKeyFlag(col.Info.Flag) || mysql.HasPriKeyFlag(col.Info.Flag)) { + uniqueCol[col.Info.Name.L] = true + } + minFactor = math.Min(minFactor, 1.0/pseudoEqualRate) + case ast.GE, ast.GT, ast.LE, ast.LT: + minFactor = math.Min(minFactor, 1.0/pseudoLessRate) + // FIXME: To resolve the between case. + } + } + if len(uniqueCol) == 0 { + return minFactor + } + // use the unique key info + for _, idx := range t.Indices { + if !idx.Info.Unique { + continue + } + unique := true + for _, col := range idx.Info.Columns { + if !uniqueCol[col.Name.L] { + unique = false + break } } + if unique { + return 1.0 / float64(t.Count) + } } return minFactor } @@ -88,7 +124,7 @@ func (t *Table) Selectivity(ctx sessionctx.Context, exprs []expression.Expressio // TODO: If len(exprs) is bigger than 63, we could use bitset structure to replace the int64. // This will simplify some code and speed up if we use this rather than a boolean slice. if t.Pseudo || len(exprs) > 63 || (len(t.Columns) == 0 && len(t.Indices) == 0) { - return pseudoSelectivity(exprs), nil + return pseudoSelectivity(t, exprs), nil } var sets []*exprSet sc := ctx.GetSessionVars().StmtCtx diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 27c9227b88ee4..d6adc000beb9c 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -198,6 +198,16 @@ func (s *testSelectivitySuite) TestSelectivity(c *C) { } } +func (s *testSelectivitySuite) TestPseudoSelectivity(c *C) { + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int, b int, unique key idx(a,b))") + testKit.MustQuery("explain select * from t where a = 1 and b = 1").Check(testkit.Rows( + "IndexScan_8 cop table:t, index:a, b, range:[1 1,1 1], keep order:false 1.00", + "IndexReader_9 root index:IndexScan_8 1.00")) +} + func BenchmarkSelectivity(b *testing.B) { c := &C{} s := &testSelectivitySuite{} diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 716331f1c41b8..98e709816a5d6 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -82,7 +82,7 @@ func (r *recordSet) getNext() []types.Datum { return row } -func (r *recordSet) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (r *recordSet) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() row := r.getNext() if row != nil { @@ -174,7 +174,7 @@ func buildPK(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSe ctx := context.Background() for { chk := records.NewChunk() - err := records.NextChunk(ctx, chk) + err := records.Next(ctx, chk) if err != nil { return 0, nil, errors.Trace(err) } @@ -200,7 +200,7 @@ func buildIndex(sctx sessionctx.Context, numBuckets, id int64, records ast.Recor chk := records.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err := records.NextChunk(ctx, chk) + err := records.Next(ctx, chk) if err != nil { return 0, nil, nil, errors.Trace(err) } @@ -418,7 +418,7 @@ func (s *testStatisticsSuite) TestPseudoTable(c *C) { FieldType: *types.NewFieldType(mysql.TypeLonglong), } ti.Columns = append(ti.Columns, colInfo) - tbl := PseudoTable(ti.ID) + tbl := PseudoTable(ti) c.Assert(tbl.Count, Greater, int64(0)) sc := new(stmtctx.StatementContext) count := tbl.ColumnLessRowCount(sc, types.NewIntDatum(100), colInfo.ID) @@ -453,7 +453,7 @@ func (s *testStatisticsSuite) TestColumnRange(c *C) { hg, err := BuildColumn(ctx, bucketCount, 2, collector, types.NewFieldType(mysql.TypeLonglong)) hg.PreCalculateScalar() c.Check(err, IsNil) - col := &Column{Histogram: *hg, CMSketch: buildCMSketch(s.rc.(*recordSet).data)} + col := &Column{Histogram: *hg, CMSketch: buildCMSketch(s.rc.(*recordSet).data), Info: &model.ColumnInfo{}} tbl := &Table{ Count: int64(col.totalRowCount()), Columns: make(map[int64]*Column), @@ -520,7 +520,7 @@ func (s *testStatisticsSuite) TestIntColumnRanges(c *C) { hg.PreCalculateScalar() c.Check(err, IsNil) c.Check(rowCount, Equals, int64(100000)) - col := &Column{Histogram: *hg} + col := &Column{Histogram: *hg, Info: &model.ColumnInfo{}} tbl := &Table{ Count: int64(col.totalRowCount()), Columns: make(map[int64]*Column), @@ -638,6 +638,13 @@ func (s *testStatisticsSuite) TestIndexRanges(c *C) { c.Assert(err, IsNil) c.Assert(int(count), Equals, 100) + tbl.Indices[0] = &Index{Info: &model.IndexInfo{Columns: []*model.IndexColumn{{Offset: 0}}, Unique: true}} + ran[0].LowVal[0] = types.NewIntDatum(1000) + ran[0].HighVal[0] = types.NewIntDatum(1000) + count, err = tbl.GetRowCountByIndexRanges(sc, 0, ran) + c.Assert(err, IsNil) + c.Assert(int(count), Equals, 1) + tbl.Indices[0] = idx ran[0].LowVal[0] = types.MinNotNullDatum() ran[0].HighVal[0] = types.MaxValueDatum() diff --git a/statistics/table.go b/statistics/table.go index 8e48d10b28ddf..9c74a0887d2ac 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -122,16 +122,32 @@ func (h *Handle) columnStatsFromStorage(row types.Row, table *Table, tableInfo * continue } isHandle := tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.Flag) - needNotLoad := col == nil || (col.Len() == 0 && col.LastUpdateVersion < histVer) - if h.Lease > 0 && !isHandle && needNotLoad && !loadAll { + // We will not load buckets if: + // 1. Lease > 0, and: + // 2. this column is not handle, and: + // 3. the column doesn't has buckets before, and: + // 4. loadAll is false. + notNeedLoad := h.Lease > 0 && + !isHandle && + (col == nil || col.Len() == 0 && col.LastUpdateVersion < histVer) && + !loadAll + if notNeedLoad { count, err := columnCountFromStorage(h.ctx, table.TableID, histID) if err != nil { return errors.Trace(err) } col = &Column{ - Histogram: Histogram{ID: histID, NDV: distinct, NullCount: nullCount, tp: &colInfo.FieldType, LastUpdateVersion: histVer, TotColSize: totColSize}, - Info: colInfo, - Count: count} + Histogram: Histogram{ + ID: histID, + NDV: distinct, + NullCount: nullCount, + tp: &colInfo.FieldType, + LastUpdateVersion: histVer, + TotColSize: totColSize, + }, + Info: colInfo, + Count: count + nullCount, + } break } if col == nil || col.LastUpdateVersion < histVer || loadAll { @@ -143,7 +159,12 @@ func (h *Handle) columnStatsFromStorage(row types.Row, table *Table, tableInfo * if err != nil { return errors.Trace(err) } - col = &Column{Histogram: *hg, Info: colInfo, CMSketch: cms, Count: int64(hg.totalRowCount())} + col = &Column{ + Histogram: *hg, + Info: colInfo, + CMSketch: cms, + Count: int64(hg.totalRowCount()), + } } break } @@ -161,7 +182,7 @@ func (h *Handle) columnStatsFromStorage(row types.Row, table *Table, tableInfo * // tableStatsFromStorage loads table stats info from storage. func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, loadAll bool) (*Table, error) { table, ok := h.statsCache.Load().(statsCache)[tableInfo.ID] - if !ok { + if !ok || table.Pseudo { table = &Table{ TableID: tableInfo.ID, Columns: make(map[int64]*Column, len(tableInfo.Columns)), @@ -252,7 +273,7 @@ func (t *Table) ColumnIsInvalid(sc *stmtctx.StatementContext, colID int64) bool sc.SetHistogramsNotLoad() histogramNeededColumns.insert(tableColumnID{tableID: t.TableID, columnID: colID}) } - return !ok || col.Len() == 0 + return !ok || col.totalRowCount() == 0 || (col.NDV > 0 && col.Len() == 0) } // ColumnGreaterRowCount estimates the row count where the column greater than value. @@ -325,7 +346,11 @@ func (t *Table) GetRowCountByColumnRanges(sc *stmtctx.StatementContext, colID in func (t *Table) GetRowCountByIndexRanges(sc *stmtctx.StatementContext, idxID int64, indexRanges []*ranger.NewRange) (float64, error) { idx := t.Indices[idxID] if t.Pseudo || idx == nil || idx.Len() == 0 { - return getPseudoRowCountByIndexRanges(sc, indexRanges, float64(t.Count)) + colsLen := -1 + if idx != nil && idx.Info.Unique { + colsLen = len(idx.Info.Columns) + } + return getPseudoRowCountByIndexRanges(sc, indexRanges, float64(t.Count), colsLen) } result, err := idx.getRowCount(sc, indexRanges) result *= idx.getIncreaseFactor(t.Count) @@ -333,18 +358,29 @@ func (t *Table) GetRowCountByIndexRanges(sc *stmtctx.StatementContext, idxID int } // PseudoTable creates a pseudo table statistics. -func PseudoTable(tableID int64) *Table { - return &Table{ - TableID: tableID, +func PseudoTable(tblInfo *model.TableInfo) *Table { + t := &Table{ + TableID: tblInfo.ID, Pseudo: true, Count: pseudoRowCount, - Columns: make(map[int64]*Column), - Indices: make(map[int64]*Index), + Columns: make(map[int64]*Column, len(tblInfo.Columns)), + Indices: make(map[int64]*Index, len(tblInfo.Indices)), + } + for _, col := range tblInfo.Columns { + if col.State == model.StatePublic { + t.Columns[col.ID] = &Column{Info: col} + } } + for _, idx := range tblInfo.Indices { + if idx.State == model.StatePublic { + t.Indices[idx.ID] = &Index{Info: idx} + } + } + return t } func getPseudoRowCountByIndexRanges(sc *stmtctx.StatementContext, indexRanges []*ranger.NewRange, - tableRowCount float64) (float64, error) { + tableRowCount float64, colsLen int) (float64, error) { if tableRowCount == 0 { return 0, nil } @@ -355,6 +391,10 @@ func getPseudoRowCountByIndexRanges(sc *stmtctx.StatementContext, indexRanges [] if err != nil { return 0, errors.Trace(err) } + if i == colsLen && !indexRange.LowExclude && !indexRange.HighExclude { + totalCount += 1.0 + continue + } if i >= len(indexRange.LowVal) { i = len(indexRange.LowVal) - 1 } diff --git a/statistics/update.go b/statistics/update.go index 86a82fe52b79e..f7d1f7d5a0a34 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/sqlexec" @@ -32,10 +33,18 @@ import ( type tableDeltaMap map[int64]variable.TableDelta -func (m tableDeltaMap) update(id int64, delta int64, count int64) { +func (m tableDeltaMap) update(id int64, delta int64, count int64, colSize *map[int64]int64) { item := m[id] item.Delta += delta item.Count += count + if item.ColSize == nil { + item.ColSize = make(map[int64]int64) + } + if colSize != nil { + for key, val := range *colSize { + item.ColSize[key] += val + } + } m[id] = item } @@ -43,7 +52,7 @@ func (h *Handle) merge(s *SessionStatsCollector) { s.Lock() defer s.Unlock() for id, item := range s.mapper { - h.globalMap.update(id, item.Delta, item.Count) + h.globalMap.update(id, item.Delta, item.Count, &item.ColSize) } h.feedback = mergeQueryFeedback(h.feedback, s.feedback) s.mapper = make(tableDeltaMap) @@ -70,15 +79,15 @@ func (s *SessionStatsCollector) Delete() { } // Update will updates the delta and count for one table id. -func (s *SessionStatsCollector) Update(id int64, delta int64, count int64) { +func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSize *map[int64]int64) { s.Lock() defer s.Unlock() - s.mapper.update(id, delta, count) + s.mapper.update(id, delta, count, colSize) } func mergeQueryFeedback(lq []*QueryFeedback, rq []*QueryFeedback) []*QueryFeedback { for _, q := range rq { - if len(lq) >= maxQueryFeedBackCount { + if len(lq) >= MaxQueryFeedbackCount { break } lq = append(lq, q) @@ -108,7 +117,7 @@ func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}) { s.Lock() defer s.Unlock() - if len(s.feedback) >= maxQueryFeedBackCount { + if len(s.feedback) >= MaxQueryFeedbackCount { return } s.feedback = append(s.feedback, q) @@ -154,19 +163,29 @@ func (h *Handle) DumpStatsDeltaToKV() error { } h.listHead.Unlock() for id, item := range h.globalMap { - updated, err := h.dumpTableStatDeltaToKV(id, item) + updated, err := h.dumpTableStatCountToKV(id, item) if err != nil { return errors.Trace(err) } + if updated { + h.globalMap.update(id, -item.Delta, -item.Count, nil) + } + if err = h.dumpTableStatColSizeToKV(id, item); err != nil { + return errors.Trace(err) + } if updated { delete(h.globalMap, id) + } else { + m := h.globalMap[id] + m.ColSize = nil + h.globalMap[id] = m } } return nil } // dumpTableStatDeltaToKV dumps a single delta with some table to KV and updates the version. -func (h *Handle) dumpTableStatDeltaToKV(id int64, delta variable.TableDelta) (bool, error) { +func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (bool, error) { if delta.Count == 0 { return true, nil } @@ -190,6 +209,153 @@ func (h *Handle) dumpTableStatDeltaToKV(id int64, delta variable.TableDelta) (bo return updated, errors.Trace(err) } +func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) error { + if len(delta.ColSize) == 0 { + return nil + } + ctx := context.TODO() + _, err := h.ctx.(sqlexec.SQLExecutor).Execute(ctx, "begin") + if err != nil { + return errors.Trace(err) + } + version := h.ctx.Txn().StartTS() + + for key, val := range delta.ColSize { + if val == 0 { + continue + } + sql := fmt.Sprintf("update mysql.stats_histograms set version = %d, tot_col_size = tot_col_size + %d where hist_id = %d and table_id = %d and is_index = 0", version, val, key, id) + _, err = h.ctx.(sqlexec.SQLExecutor).Execute(ctx, sql) + if err != nil { + return errors.Trace(err) + } + } + _, err = h.ctx.(sqlexec.SQLExecutor).Execute(ctx, "commit") + return errors.Trace(err) +} + +// DumpStatsFeedbackToKV dumps the stats feedback to KV. +func (h *Handle) DumpStatsFeedbackToKV() error { + var err error + var successCount int + for _, fb := range h.feedback { + err = h.dumpFeedbackToKV(fb) + if err != nil { + break + } + successCount++ + } + h.feedback = h.feedback[successCount:] + return errors.Trace(err) +} + +func (h *Handle) dumpFeedbackToKV(fb *QueryFeedback) error { + vals, err := encodeFeedback(fb) + if err != nil { + log.Debugf("error occurred when encoding feedback, err: ", errors.ErrorStack(err)) + return nil + } + var isIndex int64 + if fb.hist.tp.Tp == mysql.TypeBlob { + isIndex = 1 + } else { + isIndex = 0 + } + sql := fmt.Sprintf("insert into mysql.stats_feedback (table_id, hist_id, is_index, feedback) values "+ + "(%d, %d, %d, X'%X')", fb.tableID, fb.hist.ID, isIndex, vals) + _, err = h.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), sql) + if err != nil { + metrics.DumpFeedbackCounter.WithLabelValues(metrics.LblError).Inc() + } else { + metrics.DumpFeedbackCounter.WithLabelValues(metrics.LblOK).Inc() + } + return errors.Trace(err) +} + +// HandleUpdateStats update the stats using feedback. +func (h *Handle) HandleUpdateStats(is infoschema.InfoSchema) error { + sql := "select table_id, hist_id, is_index, feedback from mysql.stats_feedback order by table_id, hist_id, is_index" + rows, _, err := h.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(h.ctx, sql) + if len(rows) == 0 || err != nil { + return errors.Trace(err) + } + tableID, histID, isIndex := int64(-1), int64(-1), int64(-1) + q := &QueryFeedback{} + var ( + cms *CMSketch + hist *Histogram + ) + for _, row := range rows { + // merge into previous feedback + if row.GetInt64(0) == tableID && row.GetInt64(1) == histID && row.GetInt64(2) == isIndex { + err = decodeFeedback(row.GetBytes(3), q, cms) + if err != nil { + log.Debugf("decode feedback failed, err: %v", errors.ErrorStack(err)) + } + continue + } + // dump the stats into kv + if hist != nil { + err = h.dumpStatsUpdateToKV(tableID, int(isIndex), q, hist, cms) + if err != nil { + return errors.Trace(err) + } + } + // initialize new feedback + tableID, histID, isIndex = row.GetInt64(0), row.GetInt64(1), row.GetInt64(2) + table, ok := is.TableByID(tableID) + if !ok { + hist, cms = nil, nil + continue + } + tbl := h.GetTableStats(table.Meta()) + if isIndex == 1 { + idx, ok := tbl.Indices[histID] + if !ok { + hist, cms = nil, nil + continue + } + hist = &idx.Histogram + cms = idx.CMSketch.copy() + } else { + col, ok := tbl.Columns[histID] + if !ok { + hist, cms = nil, nil + continue + } + hist = &col.Histogram + cms = nil + } + err = decodeFeedback(row.GetBytes(3), q, cms) + if err != nil { + log.Debugf("decode feedback failed, err: %v", errors.ErrorStack(err)) + } + } + // dump the last feedback into kv + err = h.dumpStatsUpdateToKV(tableID, int(isIndex), q, hist, cms) + return errors.Trace(err) +} + +func (h *Handle) dumpStatsUpdateToKV(tableID int64, isIndex int, q *QueryFeedback, hist *Histogram, cms *CMSketch) (err error) { + defer func() { + if err != nil { + metrics.UpdateStatsCounter.WithLabelValues(metrics.LblError).Inc() + } else { + metrics.UpdateStatsCounter.WithLabelValues(metrics.LblOK).Inc() + } + }() + hist = UpdateHistogram(hist, q) + err = SaveStatsToStorage(h.ctx, tableID, -1, isIndex, hist, cms) + if err != nil { + return errors.Trace(err) + } + h.ctx.GetSessionVars().BatchDelete = true + sql := fmt.Sprintf("delete from mysql.stats_feedback where table_id = %d and hist_id = %d and is_index = %d", tableID, hist.ID, isIndex) + _, err = h.ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), sql) + q.feedback = q.feedback[:0] + return errors.Trace(err) +} + const ( // StatsOwnerKey is the stats owner path that is saved to etcd. StatsOwnerKey = "/tidb/stats/owner" @@ -228,7 +394,7 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) error { tbls := is.SchemaTables(model.NewCIStr(db)) for _, tbl := range tbls { tblInfo := tbl.Meta() - statsTbl := h.GetTableStats(tblInfo.ID) + statsTbl := h.GetTableStats(tblInfo) if statsTbl.Pseudo || statsTbl.Count == 0 { continue } diff --git a/statistics/update_test.go b/statistics/update_test.go index 38c8723569d26..d3f5b2d809f9d 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -77,13 +77,13 @@ func (s *testStatsUpdateSuite) TestSingleSessionInsert(c *C) { h.DumpStatsDeltaToKV() h.Update(is) - stats1 := h.GetTableStats(tableInfo1.ID) + stats1 := h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) tbl2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) c.Assert(err, IsNil) tableInfo2 := tbl2.Meta() - stats2 := h.GetTableStats(tableInfo2.ID) + stats2 := h.GetTableStats(tableInfo2) c.Assert(stats2.Count, Equals, int64(rowCount2)) testKit.MustExec("analyze table t1") @@ -93,7 +93,7 @@ func (s *testStatsUpdateSuite) TestSingleSessionInsert(c *C) { } h.DumpStatsDeltaToKV() h.Update(is) - stats1 = h.GetTableStats(tableInfo1.ID) + stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*2)) // Test IncreaseFactor. @@ -108,7 +108,7 @@ func (s *testStatsUpdateSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("commit") h.DumpStatsDeltaToKV() h.Update(is) - stats1 = h.GetTableStats(tableInfo1.ID) + stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*3)) testKit.MustExec("begin") @@ -124,9 +124,9 @@ func (s *testStatsUpdateSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("commit") h.DumpStatsDeltaToKV() h.Update(is) - stats1 = h.GetTableStats(tableInfo1.ID) + stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*3)) - stats2 = h.GetTableStats(tableInfo2.ID) + stats2 = h.GetTableStats(tableInfo2) c.Assert(stats2.Count, Equals, int64(rowCount2)) testKit.MustExec("begin") @@ -134,11 +134,14 @@ func (s *testStatsUpdateSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("commit") h.DumpStatsDeltaToKV() h.Update(is) - stats1 = h.GetTableStats(tableInfo1.ID) + stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(0)) rs := testKit.MustQuery("select modify_count from mysql.stats_meta") rs.Check(testkit.Rows("40", "70")) + + rs = testKit.MustQuery("select tot_col_size from mysql.stats_histograms") + rs.Check(testkit.Rows("0", "0", "10", "10")) } func (s *testStatsUpdateSuite) TestRollback(c *C) { @@ -159,7 +162,7 @@ func (s *testStatsUpdateSuite) TestRollback(c *C) { h.DumpStatsDeltaToKV() h.Update(is) - stats := h.GetTableStats(tableInfo.ID) + stats := h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(0)) c.Assert(stats.ModifyCount, Equals, int64(0)) } @@ -193,7 +196,7 @@ func (s *testStatsUpdateSuite) TestMultiSession(c *C) { h.DumpStatsDeltaToKV() h.Update(is) - stats1 := h.GetTableStats(tableInfo1.ID) + stats1 := h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) for i := 0; i < rowCount1; i++ { @@ -213,7 +216,7 @@ func (s *testStatsUpdateSuite) TestMultiSession(c *C) { h.DumpStatsDeltaToKV() h.Update(is) - stats1 = h.GetTableStats(tableInfo1.ID) + stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1*2)) // The session in testKit is already Closed, set it to nil will create a new session. testKit.Se = nil @@ -242,14 +245,14 @@ func (s *testStatsUpdateSuite) TestTxnWithFailure(c *C) { } h.DumpStatsDeltaToKV() h.Update(is) - stats1 := h.GetTableStats(tableInfo1.ID) + stats1 := h.GetTableStats(tableInfo1) // have not commit c.Assert(stats1.Count, Equals, int64(0)) testKit.MustExec("commit") h.DumpStatsDeltaToKV() h.Update(is) - stats1 = h.GetTableStats(tableInfo1.ID) + stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) _, err = testKit.Exec("insert into t1 values(0, 2)") @@ -257,13 +260,13 @@ func (s *testStatsUpdateSuite) TestTxnWithFailure(c *C) { h.DumpStatsDeltaToKV() h.Update(is) - stats1 = h.GetTableStats(tableInfo1.ID) + stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1)) testKit.MustExec("insert into t1 values(-1, 2)") h.DumpStatsDeltaToKV() h.Update(is) - stats1 = h.GetTableStats(tableInfo1.ID) + stats1 = h.GetTableStats(tableInfo1) c.Assert(stats1.Count, Equals, int64(rowCount1+1)) } @@ -271,7 +274,7 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec("create table t (a int)") + testKit.MustExec("create table t (a varchar(20))") statistics.AutoAnalyzeMinCnt = 0 defer func() { @@ -287,21 +290,26 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { h.HandleDDLEvent(<-h.DDLEventCh()) h.Update(is) - stats := h.GetTableStats(tableInfo.ID) + stats := h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(0)) - _, err = testKit.Exec("insert into t values (1)") + _, err = testKit.Exec("insert into t values ('ss')") c.Assert(err, IsNil) h.DumpStatsDeltaToKV() h.Update(is) err = h.HandleAutoAnalyze(is) c.Assert(err, IsNil) h.Update(is) - stats = h.GetTableStats(tableInfo.ID) + stats = h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(1)) c.Assert(stats.ModifyCount, Equals, int64(0)) + for _, item := range stats.Columns { + // TotColSize = 2(length of 'ss') + 1(size of len byte). + c.Assert(item.TotColSize, Equals, int64(3)) + break + } - _, err = testKit.Exec("insert into t values (1)") + _, err = testKit.Exec("insert into t values ('eee')") c.Assert(err, IsNil) h.DumpStatsDeltaToKV() h.Clear() @@ -312,10 +320,15 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { err = h.HandleAutoAnalyze(is) c.Assert(err, IsNil) h.Update(is) - stats = h.GetTableStats(tableInfo.ID) + stats = h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(2)) // Modify count is non-zero means that we do not analyze the table. c.Assert(stats.ModifyCount, Equals, int64(1)) + for _, item := range stats.Columns { + // TotColSize = 6, because the table has not been analyzed, and insert statement will add 3(length of 'eee') to TotColSize. + c.Assert(item.TotColSize, Equals, int64(6)) + break + } _, err = testKit.Exec("create index idx on t(a)") c.Assert(err, IsNil) @@ -325,13 +338,13 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { tableInfo = tbl.Meta() h.HandleAutoAnalyze(is) h.Update(is) - stats = h.GetTableStats(tableInfo.ID) + stats = h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(2)) c.Assert(stats.ModifyCount, Equals, int64(0)) hg, ok := stats.Indices[tableInfo.Indices[0].ID] c.Assert(ok, IsTrue) - c.Assert(hg.NDV, Equals, int64(1)) - c.Assert(hg.Len(), Equals, 1) + c.Assert(hg.NDV, Equals, int64(2)) + c.Assert(hg.Len(), Equals, 2) } func appendBucket(h *statistics.Histogram, l, r int64) { @@ -397,7 +410,7 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec("create table t (a int, b int, primary key(a), index idx(b))") + testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (3,4)") @@ -416,14 +429,16 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { idxCols int }{ { + // test primary key feedback sql: "select * from t where t.a <= 5", - hist: "column:1 ndv:3\n" + + hist: "column:1 ndv:3 totColSize:0\n" + "num: 1\tlower_bound: 1\tupper_bound: 1\trepeats: 1\n" + "num: 2\tlower_bound: 2\tupper_bound: 2\trepeats: 1\n" + "num: 4\tlower_bound: 3\tupper_bound: 6\trepeats: 0", idxCols: 0, }, { + // test index feedback by double read sql: "select * from t use index(idx) where t.b <= 5", hist: "index:1 ndv:2\n" + "num: 2\tlower_bound: 2\tupper_bound: 2\trepeats: 2\n" + @@ -431,6 +446,7 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { idxCols: 1, }, { + // test index feedback by single read sql: "select b from t use index(idx) where t.b <= 5", hist: "index:1 ndv:2\n" + "num: 2\tlower_bound: 2\tupper_bound: 2\trepeats: 2\n" + @@ -446,8 +462,23 @@ func (s *testStatsUpdateSuite) TestQueryFeedback(c *C) { if t.idxCols == 0 { c.Assert(feedback[0].DecodeInt(), IsNil) } - c.Assert(statistics.UpdateHistogram(feedback[0].Hist(), feedback).ToString(t.idxCols), Equals, t.hist) + c.Assert(statistics.UpdateHistogram(feedback[0].Hist(), feedback[0]).ToString(t.idxCols), Equals, t.hist) + } + + for _, t := range tests { + testKit.MustQuery(t.sql) } + c.Assert(h.DumpStatsDeltaToKV(), IsNil) + c.Assert(h.DumpStatsFeedbackToKV(), IsNil) + c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) + is := s.do.InfoSchema() + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + h.Update(s.do.InfoSchema()) + tblInfo := table.Meta() + tbl := h.GetTableStats(tblInfo) + c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, tests[0].hist) + c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, tests[1].hist) // Feedback from limit executor may not be accurate. testKit.MustQuery("select * from t where t.a <= 2 limit 1") diff --git a/store/mockstore/mocktikv/analyze.go b/store/mockstore/mocktikv/analyze.go index 7f8e0737613e2..b04647b19e997 100644 --- a/store/mockstore/mocktikv/analyze.go +++ b/store/mockstore/mocktikv/analyze.go @@ -210,16 +210,7 @@ func (e *analyzeColumnsExec) getNext(ctx context.Context) ([]types.Datum, error) return datumRow, nil } -// Next implements the ast.RecordSet Next interface. -func (e *analyzeColumnsExec) Next(ctx context.Context) (types.Row, error) { - row, err := e.getNext(ctx) - if row == nil || err != nil { - return nil, errors.Trace(err) - } - return types.DatumRow(row), nil -} - -func (e *analyzeColumnsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (e *analyzeColumnsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() row, err := e.getNext(ctx) if row == nil || err != nil { diff --git a/store/mockstore/mocktikv/cluster.go b/store/mockstore/mocktikv/cluster.go index 995592bf91b9c..e1e42108ad170 100644 --- a/store/mockstore/mocktikv/cluster.go +++ b/store/mockstore/mocktikv/cluster.go @@ -297,7 +297,7 @@ func (c *Cluster) Merge(regionID1, regionID2 uint64) { // SplitTable evenly splits the data in table into count regions. // Only works for single store. -func (c *Cluster) SplitTable(mvccStore *MvccStore, tableID int64, count int) { +func (c *Cluster) SplitTable(mvccStore MVCCStore, tableID int64, count int) { tableStart := tablecodec.GenTableRecordPrefix(tableID) tableEnd := tableStart.PrefixNext() c.splitRange(mvccStore, NewMvccKey(tableStart), NewMvccKey(tableEnd), count) @@ -305,13 +305,13 @@ func (c *Cluster) SplitTable(mvccStore *MvccStore, tableID int64, count int) { // SplitIndex evenly splits the data in index into count regions. // Only works for single store. -func (c *Cluster) SplitIndex(mvccStore *MvccStore, tableID, indexID int64, count int) { +func (c *Cluster) SplitIndex(mvccStore MVCCStore, tableID, indexID int64, count int) { indexStart := tablecodec.EncodeTableIndexPrefix(tableID, indexID) indexEnd := indexStart.PrefixNext() c.splitRange(mvccStore, NewMvccKey(indexStart), NewMvccKey(indexEnd), count) } -func (c *Cluster) splitRange(mvccStore *MvccStore, start, end MvccKey, count int) { +func (c *Cluster) splitRange(mvccStore MVCCStore, start, end MvccKey, count int) { c.Lock() defer c.Unlock() c.evacuateOldRegionRanges(start, end) @@ -320,7 +320,7 @@ func (c *Cluster) splitRange(mvccStore *MvccStore, start, end MvccKey, count int } // getPairsGroupByRegions groups the key value pairs into splitted regions. -func (c *Cluster) getEntriesGroupByRegions(mvccStore *MvccStore, start, end MvccKey, count int) [][]Pair { +func (c *Cluster) getEntriesGroupByRegions(mvccStore MVCCStore, start, end MvccKey, count int) [][]Pair { startTS := uint64(math.MaxUint64) limit := int(math.MaxInt32) pairs := mvccStore.Scan(start.Raw(), end.Raw(), limit, startTS, kvrpcpb.IsolationLevel_SI) diff --git a/store/mockstore/mocktikv/cluster_test.go b/store/mockstore/mocktikv/cluster_test.go index 18da909ffe82f..1e3dd5e558025 100644 --- a/store/mockstore/mocktikv/cluster_test.go +++ b/store/mockstore/mocktikv/cluster_test.go @@ -40,7 +40,7 @@ type testClusterSuite struct { func (s *testClusterSuite) TestClusterSplit(c *C) { cluster := mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(cluster) - mvccStore := mocktikv.NewMvccStore() + mvccStore := mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(cluster), mockstore.WithMVCCStore(mvccStore), diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index adbaecbe490ee..fc10ea2669fc5 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -534,11 +534,6 @@ func (mock *mockCopStreamClient) readBlockFromExecutor() (tipb.Chunk, bool, *cop func buildResp(chunks []tipb.Chunk, counts []int64, err error, warnings []error) *coprocessor.Response { resp := &coprocessor.Response{} - // The counts was the output count of each executor, but now it is the scan count of each range, - // so we need a flag to tell them apart. - if counts != nil { - counts = append(counts, -1) - } selResp := &tipb.SelectResponse{ Error: toPBError(err), Chunks: chunks, diff --git a/store/mockstore/mocktikv/mock_tikv_test.go b/store/mockstore/mocktikv/mock_tikv_test.go index 8f6b780b996cc..4c7585fe533ee 100644 --- a/store/mockstore/mocktikv/mock_tikv_test.go +++ b/store/mockstore/mocktikv/mock_tikv_test.go @@ -33,26 +33,16 @@ type testMockTiKVSuite struct { type testMarshal struct{} -var ( - _ = Suite(&testMvccStore{}) - _ = Suite(&testMVCCLevelDB{}) - _ = Suite(testMarshal{}) -) - -// testMvccStore is used to test MvccStore implementation. -type testMvccStore struct { - testMockTiKVSuite -} - -func (s *testMvccStore) SetUpTest(c *C) { - s.store = NewMvccStore() -} - // testMVCCLevelDB is used to test MVCCLevelDB implementation. type testMVCCLevelDB struct { testMockTiKVSuite } +var ( + _ = Suite(&testMVCCLevelDB{}) + _ = Suite(testMarshal{}) +) + func (s *testMockTiKVSuite) SetUpTest(c *C) { var err error s.store, err = NewMVCCLevelDB("") @@ -188,6 +178,11 @@ func (s *testMockTiKVSuite) mustBatchResolveLock(c *C, txnInfos map[uint64]uint6 c.Assert(s.store.BatchResolveLock(nil, nil, txnInfos), IsNil) } +func (s *testMockTiKVSuite) mustDeleteRange(c *C, startKey, endKey string) { + err := s.store.DeleteRange([]byte(startKey), []byte(endKey)) + c.Assert(err, IsNil) +} + func (s *testMockTiKVSuite) TestGet(c *C) { s.mustGetNone(c, "x", 10) s.mustPutOK(c, "x", "x", 5, 10) @@ -354,6 +349,14 @@ func (s *testMockTiKVSuite) TestScanLock(c *C) { s.mustPrewriteOK(c, putMutations("p1", "v5", "s1", "v5"), "p1", 5) s.mustPrewriteOK(c, putMutations("p2", "v10", "s2", "v10"), "p2", 10) s.mustPrewriteOK(c, putMutations("p3", "v20", "s3", "v20"), "p3", 20) + + locks, err := s.store.ScanLock([]byte("a"), []byte("r"), 12) + c.Assert(err, IsNil) + c.Assert(locks, DeepEquals, []*kvrpcpb.LockInfo{ + lock("p1", "p1", 5), + lock("p2", "p2", 10), + }) + s.mustScanLock(c, 10, []*kvrpcpb.LockInfo{ lock("p1", "p1", 5), lock("p2", "p2", 10), @@ -440,6 +443,31 @@ func (s *testMockTiKVSuite) TestRollbackAndWriteConflict(c *C) { s.mustWriteWriteConflict(c, errs, 0) } +func (s *testMockTiKVSuite) TestDeleteRange(c *C) { + for i := 1; i <= 5; i++ { + key := string(byte(i) + byte('0')) + value := "v" + key + s.mustPutOK(c, key, value, uint64(1+2*i), uint64(2+2*i)) + } + + s.mustScanOK(c, "0", 10, 20, "1", "v1", "2", "v2", "3", "v3", "4", "v4", "5", "v5") + + s.mustDeleteRange(c, "2", "4") + s.mustScanOK(c, "0", 10, 30, "1", "v1", "4", "v4", "5", "v5") + + s.mustDeleteRange(c, "5", "5") + s.mustScanOK(c, "0", 10, 40, "1", "v1", "4", "v4", "5", "v5") + + s.mustDeleteRange(c, "41", "42") + s.mustScanOK(c, "0", 10, 50, "1", "v1", "4", "v4", "5", "v5") + + s.mustDeleteRange(c, "4\x00", "5\x00") + s.mustScanOK(c, "0", 10, 60, "1", "v1", "4", "v4") + + s.mustDeleteRange(c, "0", "9") + s.mustScanOK(c, "0", 10, 70) +} + func (s *testMockTiKVSuite) mustWriteWriteConflict(c *C, errs []error, i int) { c.Assert(errs[i], NotNil) c.Assert(strings.Contains(errs[i].Error(), "write conflict"), IsTrue) diff --git a/store/mockstore/mocktikv/mvcc.go b/store/mockstore/mocktikv/mvcc.go index 8428395c59ca8..f8f8927436b28 100644 --- a/store/mockstore/mocktikv/mvcc.go +++ b/store/mockstore/mocktikv/mvcc.go @@ -18,7 +18,6 @@ import ( "encoding/binary" "io" "sort" - "sync" "github.com/google/btree" "github.com/juju/errors" @@ -28,8 +27,6 @@ import ( type mvccValueType int -const btreeDegree = 32 - const ( typePut mvccValueType = iota typeDelete @@ -364,16 +361,16 @@ func (e *mvccEntry) containsStartTS(startTS uint64) bool { func (e *mvccEntry) dumpMvccInfo() *kvrpcpb.MvccInfo { info := &kvrpcpb.MvccInfo{} if e.lock != nil { - info.Lock = &kvrpcpb.LockInfo{ - Key: e.key, - PrimaryLock: e.lock.primary, - LockVersion: e.lock.startTS, - LockTtl: e.lock.ttl, + info.Lock = &kvrpcpb.MvccLock{ + Type: e.lock.op, + StartTs: e.lock.startTS, + Primary: e.lock.primary, + ShortValue: e.lock.value, } } - info.Writes = make([]*kvrpcpb.WriteInfo, len(e.values)) - info.Values = make([]*kvrpcpb.ValueInfo, len(e.values)) + info.Writes = make([]*kvrpcpb.MvccWrite, len(e.values)) + info.Values = make([]*kvrpcpb.MvccValue, len(e.values)) for id, item := range e.values { var tp kvrpcpb.Op @@ -385,15 +382,15 @@ func (e *mvccEntry) dumpMvccInfo() *kvrpcpb.MvccInfo { case typeRollback: tp = kvrpcpb.Op_Rollback } - info.Writes[id] = &kvrpcpb.WriteInfo{ - StartTs: item.startTS, + info.Writes[id] = &kvrpcpb.MvccWrite{ Type: tp, + StartTs: item.startTS, CommitTs: item.commitTS, } - info.Values[id] = &kvrpcpb.ValueInfo{ - Value: item.value, - Ts: item.startTS, + info.Values[id] = &kvrpcpb.MvccValue{ + Value: item.value, + StartTs: item.startTS, } } return info @@ -427,6 +424,8 @@ type MVCCStore interface { ScanLock(startKey, endKey []byte, maxTS uint64) ([]*kvrpcpb.LockInfo, error) ResolveLock(startKey, endKey []byte, startTS, commitTS uint64) error BatchResolveLock(startKey, endKey []byte, txnInfos map[uint64]uint64) error + DeleteRange(startKey, endKey []byte) error + Close() error } // RawKV is a key-value storage. MVCCStore can be implemented upon it with timestamp encoded into key. @@ -435,6 +434,7 @@ type RawKV interface { RawScan(startKey, endKey []byte, limit int) []Pair RawPut(key, value []byte) RawDelete(key []byte) + RawDeleteRange(startKey, endKey []byte) } // MVCCDebugger is for debugging. @@ -443,37 +443,6 @@ type MVCCDebugger interface { MvccGetByKey(key []byte) *kvrpcpb.MvccInfo } -// MvccStore is an in-memory, multi-versioned, transaction-supported kv storage. -type MvccStore struct { - sync.RWMutex - tree *btree.BTree - rawkv *btree.BTree -} - -// NewMvccStore creates a MvccStore. -func NewMvccStore() *MvccStore { - return &MvccStore{ - tree: btree.New(btreeDegree), - rawkv: btree.New(btreeDegree), - } -} - -// Get reads a key by ts. -func (s *MvccStore) Get(key []byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel) ([]byte, error) { - s.RLock() - defer s.RUnlock() - - return s.get(NewMvccKey(key), startTS, isoLevel) -} - -func (s *MvccStore) get(key MvccKey, startTS uint64, isoLevel kvrpcpb.IsolationLevel) ([]byte, error) { - entry := s.tree.Get(newEntry(key)) - if entry == nil { - return nil, nil - } - return entry.(*mvccEntry).Get(startTS, isoLevel) -} - // Pair is a KV pair read from MvccStore or an error if any occurs. type Pair struct { Key []byte @@ -481,365 +450,11 @@ type Pair struct { Err error } -// BatchGet gets values with keys and ts. -func (s *MvccStore) BatchGet(ks [][]byte, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair { - s.RLock() - defer s.RUnlock() - - var pairs []Pair - for _, k := range ks { - val, err := s.get(NewMvccKey(k), startTS, isoLevel) - if val == nil && err == nil { - continue - } - pairs = append(pairs, Pair{ - Key: k, - Value: val, - Err: err, - }) - } - return pairs -} - func regionContains(startKey []byte, endKey []byte, key []byte) bool { return bytes.Compare(startKey, key) <= 0 && (bytes.Compare(key, endKey) < 0 || len(endKey) == 0) } -// Scan reads up to a limited number of Pairs that greater than or equal to startKey and less than endKey. -func (s *MvccStore) Scan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair { - s.RLock() - defer s.RUnlock() - - startKey = NewMvccKey(startKey) - endKey = NewMvccKey(endKey) - - var pairs []Pair - iterator := func(item btree.Item) bool { - if len(pairs) >= limit { - return false - } - k := item.(*mvccEntry).key - if !regionContains(startKey, endKey, k) { - return false - } - val, err := s.get(k, startTS, isoLevel) - if val != nil || err != nil { - pairs = append(pairs, Pair{ - Key: k.Raw(), - Value: val, - Err: err, - }) - } - return true - } - s.tree.AscendGreaterOrEqual(newEntry(startKey), iterator) - return pairs -} - -// ReverseScan reads up to a limited number of Pairs that greater than or equal to startKey and less than endKey -// in descending order. -func (s *MvccStore) ReverseScan(startKey, endKey []byte, limit int, startTS uint64, isoLevel kvrpcpb.IsolationLevel) []Pair { - s.RLock() - defer s.RUnlock() - - startKey = NewMvccKey(startKey) - endKey = NewMvccKey(endKey) - - var pairs []Pair - iterator := func(item btree.Item) bool { - if len(pairs) >= limit { - return false - } - k := item.(*mvccEntry).key - if bytes.Equal(k, endKey) { - return true - } - if bytes.Compare(k, startKey) < 0 { - return false - } - val, err := s.get(k, startTS, isoLevel) - if val != nil || err != nil { - pairs = append(pairs, Pair{ - Key: k.Raw(), - Value: val, - Err: err, - }) - } - return true - } - s.tree.DescendLessOrEqual(newEntry(endKey), iterator) - return pairs -} - -func (s *MvccStore) getOrNewEntry(key []byte) *mvccEntry { - if item := s.tree.Get(newEntry(key)); item != nil { - return item.(*mvccEntry).Clone() - } - return newEntry(key) -} - -// submit writes entries into the rbtree. -func (s *MvccStore) submit(ents ...*mvccEntry) { - for _, ent := range ents { - s.tree.ReplaceOrInsert(ent) - } -} - -// Prewrite acquires a lock on a key. (1st phase of 2PC). -func (s *MvccStore) Prewrite(mutations []*kvrpcpb.Mutation, primary []byte, startTS uint64, ttl uint64) []error { - s.Lock() - defer s.Unlock() - - errs := make([]error, 0, len(mutations)) - for _, m := range mutations { - entry := s.getOrNewEntry(NewMvccKey(m.Key)) - err := entry.Prewrite(m, startTS, primary, ttl) - s.submit(entry) - errs = append(errs, err) - } - return errs -} - -// Commit commits the lock on a key. (2nd phase of 2PC). -func (s *MvccStore) Commit(keys [][]byte, startTS, commitTS uint64) error { - s.Lock() - defer s.Unlock() - - var ents []*mvccEntry - for _, k := range keys { - entry := s.getOrNewEntry(NewMvccKey(k)) - err := entry.Commit(startTS, commitTS) - if err != nil { - return err - } - ents = append(ents, entry) - } - s.submit(ents...) - return nil -} - -// Cleanup cleanups a lock, often used when resolving a expired lock. -func (s *MvccStore) Cleanup(key []byte, startTS uint64) error { - s.Lock() - defer s.Unlock() - - entry := s.getOrNewEntry(NewMvccKey(key)) - err := entry.Rollback(startTS) - if err != nil { - return err - } - s.submit(entry) - return nil -} - -// Rollback cleanups multiple locks, often used when rolling back a conflict txn. -func (s *MvccStore) Rollback(keys [][]byte, startTS uint64) error { - s.Lock() - defer s.Unlock() - - var ents []*mvccEntry - for _, k := range keys { - entry := s.getOrNewEntry(NewMvccKey(k)) - err := entry.Rollback(startTS) - if err != nil { - return err - } - ents = append(ents, entry) - } - s.submit(ents...) - return nil -} - -// ScanLock scans all orphan locks in a Region. -func (s *MvccStore) ScanLock(startKey, endKey []byte, maxTS uint64) ([]*kvrpcpb.LockInfo, error) { - s.RLock() - defer s.RUnlock() - - var locks []*kvrpcpb.LockInfo - iterator := func(item btree.Item) bool { - ent := item.(*mvccEntry) - if !regionContains(startKey, endKey, ent.key) { - return false - } - if ent.lock != nil && ent.lock.startTS <= maxTS { - locks = append(locks, &kvrpcpb.LockInfo{ - PrimaryLock: ent.lock.primary, - LockVersion: ent.lock.startTS, - Key: ent.key.Raw(), - }) - } - return true - } - s.tree.AscendGreaterOrEqual(newEntry(startKey), iterator) - return locks, nil -} - -// ResolveLock resolves all orphan locks belong to a transaction. -func (s *MvccStore) ResolveLock(startKey, endKey []byte, startTS, commitTS uint64) error { - s.Lock() - defer s.Unlock() - - var ents []*mvccEntry - var err error - iterator := func(item btree.Item) bool { - ent := item.(*mvccEntry) - if !regionContains(startKey, endKey, ent.key) { - return false - } - if ent.lock != nil && ent.lock.startTS == startTS { - if commitTS > 0 { - err = ent.Commit(startTS, commitTS) - } else { - err = ent.Rollback(startTS) - } - if err != nil { - return false - } - ents = append(ents, ent) - } - return true - } - s.tree.AscendGreaterOrEqual(newEntry(startKey), iterator) - if err != nil { - return errors.Trace(err) - } - s.submit(ents...) - return nil -} - -// BatchResolveLock resolves all orphan locks belong to a transaction. -func (s *MvccStore) BatchResolveLock(startKey, endKey []byte, txnInfos map[uint64]uint64) error { - s.Lock() - defer s.Unlock() - - var ents []*mvccEntry - var err error - iterator := func(item btree.Item) bool { - ent := item.(*mvccEntry) - if !regionContains(startKey, endKey, ent.key) { - return false - } - if ent.lock != nil { - if commitTS, ok := txnInfos[ent.lock.startTS]; ok { - if commitTS > 0 { - err = ent.Commit(ent.lock.startTS, commitTS) - } else { - err = ent.Rollback(ent.lock.startTS) - } - if err != nil { - return false - } - ents = append(ents, ent) - } - } - return true - } - s.tree.AscendGreaterOrEqual(newEntry(startKey), iterator) - if err != nil { - return errors.Trace(err) - } - s.submit(ents...) - return nil -} - -// RawGet queries value with the key. -func (s *MvccStore) RawGet(key []byte) []byte { - s.RLock() - defer s.RUnlock() - - entry := s.rawkv.Get(newRawEntry(key)) - if entry == nil { - return nil - } - return entry.(*rawEntry).value -} - -// RawPut stores a key-value pair. -func (s *MvccStore) RawPut(key, value []byte) { - s.Lock() - defer s.Unlock() - if value == nil { - value = []byte{} - } - entry := s.rawkv.Get(newRawEntry(key)) - if entry != nil { - entry.(*rawEntry).value = value - } else { - s.rawkv.ReplaceOrInsert(&rawEntry{ - key: key, - value: value, - }) - } -} - -// RawDelete deletes a key-value pair. -func (s *MvccStore) RawDelete(key []byte) { - s.Lock() - defer s.Unlock() - s.rawkv.Delete(newRawEntry(key)) -} - -// RawScan reads up to a limited number of rawkv Pairs. -func (s *MvccStore) RawScan(startKey, endKey []byte, limit int) []Pair { - s.RLock() - defer s.RUnlock() - - var pairs []Pair - iterator := func(item btree.Item) bool { - if len(pairs) >= limit { - return false - } - k := item.(*rawEntry).key - if !regionContains(startKey, endKey, k) { - return false - } - pairs = append(pairs, Pair{ - Key: k, - Value: item.(*rawEntry).value, - }) - return true - } - s.rawkv.AscendGreaterOrEqual(newRawEntry(startKey), iterator) - return pairs -} - -// MvccGetByStartTS gets mvcc info for the primary key with startTS -func (s *MvccStore) MvccGetByStartTS(startKey, endKey []byte, starTS uint64) (*kvrpcpb.MvccInfo, []byte) { - s.RLock() - defer s.RUnlock() - - var info *kvrpcpb.MvccInfo - var key []byte - iterator := func(item btree.Item) bool { - k := item.(*mvccEntry) - if !regionContains(startKey, endKey, k.key) { - return false - } - if k.containsStartTS(starTS) { - info = k.dumpMvccInfo() - key = k.key - return false - } - return true - } - s.tree.AscendGreaterOrEqual(newEntry(startKey), iterator) - return info, key -} - -// MvccGetByKey gets mvcc info for the key -func (s *MvccStore) MvccGetByKey(key []byte) *kvrpcpb.MvccInfo { - s.RLock() - defer s.RUnlock() - - resp := s.tree.Get(newEntry(NewMvccKey(key))) - if resp == nil { - return nil - } - entry := resp.(*mvccEntry) - return entry.dumpMvccInfo() -} - // MvccKey is the encoded key type. // On TiKV, keys are encoded before they are saved into storage engine. type MvccKey []byte diff --git a/store/mockstore/mocktikv/mvcc_leveldb.go b/store/mockstore/mocktikv/mvcc_leveldb.go index fe6cea087dade..56d4f889a4759 100644 --- a/store/mockstore/mocktikv/mvcc_leveldb.go +++ b/store/mockstore/mocktikv/mvcc_leveldb.go @@ -53,10 +53,12 @@ type MVCCLevelDB struct { // ... // EOF db *leveldb.DB + // leveldb can not guarantee multiple operations to be atomic, for example, read + // then write, another write may happen during it, so this lock is necessory. mu sync.RWMutex } -var lockVer uint64 = math.MaxUint64 +const lockVer uint64 = math.MaxUint64 // ErrInvalidEncodedKey describes parsing an invalid format of EncodedKey. var ErrInvalidEncodedKey = errors.New("invalid encoded key") @@ -93,6 +95,15 @@ func mvccDecode(encodedKey []byte) ([]byte, uint64, error) { return key, ver, nil } +// MustNewMVCCStore is used for testing, use NewMVCCLevelDB instead. +func MustNewMVCCStore() MVCCStore { + mvccStore, err := NewMVCCLevelDB("") + if err != nil { + panic(err) + } + return mvccStore +} + // NewMVCCLevelDB returns a new MVCCLevelDB object. func NewMVCCLevelDB(path string) (*MVCCLevelDB, error) { var ( @@ -864,7 +875,90 @@ func (mvcc *MVCCLevelDB) BatchResolveLock(startKey, endKey []byte, txnInfos map[ return mvcc.db.Write(batch, nil) } +// DeleteRange implements the MVCCStore interface. +func (mvcc *MVCCLevelDB) DeleteRange(startKey, endKey []byte) error { + return mvcc.doRawDeleteRange(codec.EncodeBytes(nil, startKey), codec.EncodeBytes(nil, endKey)) +} + // Close calls leveldb's Close to free resources. func (mvcc *MVCCLevelDB) Close() error { return mvcc.db.Close() } + +// RawPut implements the RawKV interface. +func (mvcc *MVCCLevelDB) RawPut(key, value []byte) { + mvcc.mu.Lock() + defer mvcc.mu.Unlock() + + if value == nil { + value = []byte{} + } + terror.Log(mvcc.db.Put(key, value, nil)) +} + +// RawGet implements the RawKV interface. +func (mvcc *MVCCLevelDB) RawGet(key []byte) []byte { + mvcc.mu.Lock() + defer mvcc.mu.Unlock() + + ret, err := mvcc.db.Get(key, nil) + terror.Log(err) + return ret +} + +// RawDelete implements the RawKV interface. +func (mvcc *MVCCLevelDB) RawDelete(key []byte) { + mvcc.mu.Lock() + defer mvcc.mu.Unlock() + + terror.Log(mvcc.db.Delete(key, nil)) +} + +// RawScan implements the RawKV interface. +func (mvcc *MVCCLevelDB) RawScan(startKey, endKey []byte, limit int) []Pair { + mvcc.mu.Lock() + defer mvcc.mu.Unlock() + + iter := mvcc.db.NewIterator(&util.Range{ + Start: startKey, + }, nil) + + var pairs []Pair + for iter.Next() && len(pairs) < limit { + key := iter.Key() + value := iter.Value() + err := iter.Error() + if len(endKey) > 0 && bytes.Compare(key, endKey) >= 0 { + break + } + pairs = append(pairs, Pair{ + Key: append([]byte{}, key...), + Value: append([]byte{}, value...), + Err: err, + }) + } + return pairs +} + +// RawDeleteRange implements the RawKV interface. +func (mvcc *MVCCLevelDB) RawDeleteRange(startKey, endKey []byte) { + terror.Log(mvcc.doRawDeleteRange(startKey, endKey)) +} + +// doRawDeleteRange deletes all keys in a range and return the error if any. +func (mvcc *MVCCLevelDB) doRawDeleteRange(startKey, endKey []byte) error { + mvcc.mu.Lock() + defer mvcc.mu.Unlock() + + batch := &leveldb.Batch{} + + iter := mvcc.db.NewIterator(&util.Range{ + Start: startKey, + Limit: endKey, + }, nil) + for iter.Next() { + batch.Delete(iter.Key()) + } + + return mvcc.db.Write(batch, nil) +} diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index d192848d100ab..3a512d7a6230f 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -326,7 +326,9 @@ func (h *rpcHandler) handleKvBatchRollback(req *kvrpcpb.BatchRollbackRequest) *k } func (h *rpcHandler) handleKvScanLock(req *kvrpcpb.ScanLockRequest) *kvrpcpb.ScanLockResponse { - locks, err := h.mvccStore.ScanLock(h.startKey, h.endKey, req.GetMaxVersion()) + startKey := MvccKey(h.startKey).Raw() + endKey := MvccKey(h.endKey).Raw() + locks, err := h.mvccStore.ScanLock(startKey, endKey, req.GetMaxVersion()) if err != nil { return &kvrpcpb.ScanLockResponse{ Error: convertToKeyError(err), @@ -338,7 +340,9 @@ func (h *rpcHandler) handleKvScanLock(req *kvrpcpb.ScanLockRequest) *kvrpcpb.Sca } func (h *rpcHandler) handleKvResolveLock(req *kvrpcpb.ResolveLockRequest) *kvrpcpb.ResolveLockResponse { - err := h.mvccStore.ResolveLock(h.startKey, h.endKey, req.GetStartVersion(), req.GetCommitVersion()) + startKey := MvccKey(h.startKey).Raw() + endKey := MvccKey(h.endKey).Raw() + err := h.mvccStore.ResolveLock(startKey, endKey, req.GetStartVersion(), req.GetCommitVersion()) if err != nil { return &kvrpcpb.ResolveLockResponse{ Error: convertToKeyError(err), @@ -348,9 +352,15 @@ func (h *rpcHandler) handleKvResolveLock(req *kvrpcpb.ResolveLockRequest) *kvrpc } func (h *rpcHandler) handleKvDeleteRange(req *kvrpcpb.DeleteRangeRequest) *kvrpcpb.DeleteRangeResponse { - return &kvrpcpb.DeleteRangeResponse{ - Error: "not implemented", + if !h.checkKeyInRegion(req.StartKey) { + panic("KvDeleteRange: key not in region") } + var resp kvrpcpb.DeleteRangeResponse + err := h.mvccStore.DeleteRange(req.StartKey, req.EndKey) + if err != nil { + resp.Error = err.Error() + } + return &resp } func (h *rpcHandler) handleKvRawGet(req *kvrpcpb.RawGetRequest) *kvrpcpb.RawGetResponse { @@ -387,6 +397,17 @@ func (h *rpcHandler) handleKvRawDelete(req *kvrpcpb.RawDeleteRequest) *kvrpcpb.R return &kvrpcpb.RawDeleteResponse{} } +func (h *rpcHandler) handleKvRawDeleteRange(req *kvrpcpb.RawDeleteRangeRequest) *kvrpcpb.RawDeleteRangeResponse { + rawKV, ok := h.mvccStore.(RawKV) + if !ok { + return &kvrpcpb.RawDeleteRangeResponse{ + Error: "not implemented", + } + } + rawKV.RawDeleteRange(req.GetStartKey(), req.GetEndKey()) + return &kvrpcpb.RawDeleteRangeResponse{} +} + func (h *rpcHandler) handleKvRawScan(req *kvrpcpb.RawScanRequest) *kvrpcpb.RawScanResponse { rawKV, ok := h.mvccStore.(RawKV) if !ok { @@ -576,7 +597,7 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R resp.DeleteRange = &kvrpcpb.DeleteRangeResponse{RegionError: err} return resp, nil } - resp.DeleteRange = &kvrpcpb.DeleteRangeResponse{} + resp.DeleteRange = handler.handleKvDeleteRange(r) case tikvrpc.CmdRawGet: r := req.RawGet if err := handler.checkRequest(reqCtx, r.Size()); err != nil { @@ -598,6 +619,13 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R return resp, nil } resp.RawDelete = handler.handleKvRawDelete(r) + case tikvrpc.CmdRawDeleteRange: + r := req.RawDeleteRange + if err := handler.checkRequest(reqCtx, r.Size()); err != nil { + resp.RawDeleteRange = &kvrpcpb.RawDeleteRangeResponse{RegionError: err} + return resp, nil + } + resp.RawDeleteRange = handler.handleKvRawDeleteRange(r) case tikvrpc.CmdRawScan: r := req.RawScan if err := handler.checkRequest(reqCtx, r.Size()); err != nil { diff --git a/store/store_test.go b/store/store_test.go index 114771f9ead92..b69f92dbb7014 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -51,6 +51,7 @@ type testKVSuite struct { } func (s *testKVSuite) SetUpSuite(c *C) { + testleak.BeforeTest() store, err := mockstore.NewMockTikvStore() c.Assert(err, IsNil) s.s = store @@ -59,6 +60,7 @@ func (s *testKVSuite) SetUpSuite(c *C) { func (s *testKVSuite) TearDownSuite(c *C) { err := s.s.Close() c.Assert(err, IsNil) + testleak.AfterTest(c)() } func insertData(c *C, txn kv.Transaction) { @@ -155,7 +157,6 @@ func mustGet(c *C, txn kv.Transaction) { } func (s *testKVSuite) TestGetSet(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) @@ -176,7 +177,6 @@ func (s *testKVSuite) TestGetSet(c *C) { } func (s *testKVSuite) TestSeek(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) @@ -196,7 +196,6 @@ func (s *testKVSuite) TestSeek(c *C) { } func (s *testKVSuite) TestInc(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) @@ -231,7 +230,6 @@ func (s *testKVSuite) TestInc(c *C) { } func (s *testKVSuite) TestDelete(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) @@ -267,7 +265,6 @@ func (s *testKVSuite) TestDelete(c *C) { } func (s *testKVSuite) TestDelete2(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) val := []byte("test") @@ -299,7 +296,6 @@ func (s *testKVSuite) TestDelete2(c *C) { } func (s *testKVSuite) TestSetNil(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() defer txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -308,7 +304,6 @@ func (s *testKVSuite) TestSetNil(c *C) { } func (s *testKVSuite) TestBasicSeek(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) txn.Set([]byte("1"), []byte("1")) @@ -324,7 +319,6 @@ func (s *testKVSuite) TestBasicSeek(c *C) { } func (s *testKVSuite) TestBasicTable(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) for i := 1; i < 5; i++ { @@ -372,7 +366,6 @@ func (s *testKVSuite) TestBasicTable(c *C) { } func (s *testKVSuite) TestRollback(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) @@ -400,7 +393,6 @@ func (s *testKVSuite) TestRollback(c *C) { } func (s *testKVSuite) TestSeekMin(c *C) { - defer testleak.AfterTest(c)() rows := []struct { key string value string @@ -435,7 +427,6 @@ func (s *testKVSuite) TestSeekMin(c *C) { } func (s *testKVSuite) TestConditionIfNotExist(c *C) { - defer testleak.AfterTest(c)() var success int64 cnt := 100 b := []byte("1") @@ -470,7 +461,6 @@ func (s *testKVSuite) TestConditionIfNotExist(c *C) { } func (s *testKVSuite) TestConditionIfEqual(c *C) { - defer testleak.AfterTest(c)() var success int64 cnt := 100 b := []byte("1") @@ -510,7 +500,6 @@ func (s *testKVSuite) TestConditionIfEqual(c *C) { } func (s *testKVSuite) TestConditionUpdate(c *C) { - defer testleak.AfterTest(c)() txn, err := s.s.Begin() c.Assert(err, IsNil) txn.Delete([]byte("b")) @@ -521,7 +510,6 @@ func (s *testKVSuite) TestConditionUpdate(c *C) { func (s *testKVSuite) TestDBClose(c *C) { c.Skip("don't know why it fails.") - defer testleak.AfterTest(c)() store, err := mockstore.NewMockTikvStore() c.Assert(err, IsNil) @@ -564,7 +552,6 @@ func (s *testKVSuite) TestDBClose(c *C) { } func (s *testKVSuite) TestIsolationInc(c *C) { - defer testleak.AfterTest(c)() threadCnt := 4 ids := make(map[int64]struct{}, threadCnt*100) @@ -603,7 +590,6 @@ func (s *testKVSuite) TestIsolationInc(c *C) { } func (s *testKVSuite) TestIsolationMultiInc(c *C) { - defer testleak.AfterTest(c)() threadCnt := 4 incCnt := 100 keyCnt := 4 diff --git a/store/tikv/2pc_test.go b/store/tikv/2pc_test.go index d932b93183e76..54a738d1c2989 100644 --- a/store/tikv/2pc_test.go +++ b/store/tikv/2pc_test.go @@ -38,7 +38,8 @@ var _ = Suite(&testCommitterSuite{}) func (s *testCommitterSuite) SetUpTest(c *C) { s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithMultiRegions(s.cluster, []byte("a"), []byte("b"), []byte("c")) - mvccStore := mocktikv.NewMvccStore() + mvccStore, err := mocktikv.NewMVCCLevelDB("") + c.Assert(err, IsNil) client := mocktikv.NewRPCClient(s.cluster, mvccStore) pdCli := &codecPDClient{mocktikv.NewPDClient(s.cluster)} spkv := NewMockSafePointKV() diff --git a/store/tikv/client.go b/store/tikv/client.go index f71e8df034aed..30909d9264cbd 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -15,6 +15,7 @@ package tikv import ( + "io" "strconv" "sync" "sync/atomic" @@ -30,6 +31,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/terror" + log "github.com/sirupsen/logrus" "golang.org/x/net/context" "google.golang.org/grpc" "google.golang.org/grpc/credentials" @@ -254,7 +256,10 @@ func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R var first *coprocessor.Response first, err = copStream.Recv() if err != nil { - return nil, errors.Trace(err) + if errors.Cause(err) != io.EOF { + return nil, errors.Trace(err) + } + log.Debug("copstream returns nothing for the request.") } copStream.Response = first return resp, nil diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index e7178379a7cb8..59934a1d3a6dc 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -128,7 +128,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request) kv.Response { if !it.req.KeepOrder { it.respChan = make(chan copResponse, it.concurrency) } - it.run(ctx) + it.open(ctx) return it } @@ -382,6 +382,25 @@ type copIterator struct { wg sync.WaitGroup } +// copIteratorWorker receives tasks from copIteratorTaskSender, handles tasks and sends the copResponse to respChan. +type copIteratorWorker struct { + taskCh <-chan *copTask + wg *sync.WaitGroup + store *tikvStore + req *kv.Request + respChan chan<- copResponse + finished <-chan struct{} +} + +// copIteratorTaskSender sends tasks to taskCh then wait for the workers to exit. +type copIteratorTaskSender struct { + taskCh chan<- *copTask + wg *sync.WaitGroup + tasks []*copTask + finished <-chan struct{} + respChan chan<- copResponse +} + type copResponse struct { *coprocessor.Response startKey kv.Key @@ -390,63 +409,75 @@ type copResponse struct { const minLogCopTaskTime = 300 * time.Millisecond -// work is a worker function that get a copTask from channel, handle it and +// run is a worker function that get a copTask from channel, handle it and // send the result back. -func (it *copIterator) work(ctx context.Context, taskCh <-chan *copTask) { - span, ctx1 := opentracing.StartSpanFromContext(ctx, "copIterator.work") +func (worker *copIteratorWorker) run(ctx context.Context) { + span, ctx1 := opentracing.StartSpanFromContext(ctx, "copIteratorWorker.run") defer span.Finish() - defer it.wg.Done() - for task := range taskCh { - var ch chan copResponse - if !it.req.KeepOrder { - ch = it.respChan - } else { - ch = task.respChan + defer worker.wg.Done() + for task := range worker.taskCh { + respCh := worker.respChan + if respCh == nil { + respCh = task.respChan } bo := NewBackoffer(ctx1, copNextMaxBackoff) - it.handleTask(bo, task, ch) + worker.handleTask(bo, task, respCh) if bo.totalSleep > 0 { metrics.TiKVBackoffHistogram.Observe(float64(bo.totalSleep) / 1000) } - if it.req.KeepOrder { - close(ch) - } + close(task.respChan) select { - case <-it.finished: + case <-worker.finished: return default: } } } -func (it *copIterator) run(ctx context.Context) { +// open starts workers and sender goroutines. +func (it *copIterator) open(ctx context.Context) { taskCh := make(chan *copTask, 1) it.wg.Add(it.concurrency) // Start it.concurrency number of workers to handle cop requests. for i := 0; i < it.concurrency; i++ { + worker := &copIteratorWorker{ + taskCh: taskCh, + wg: &it.wg, + store: it.store, + req: it.req, + respChan: it.respChan, + finished: it.finished, + } copIteratorGP.Go(func() { - it.work(ctx, taskCh) + worker.run(ctx) }) } + taskSender := &copIteratorTaskSender{ + taskCh: taskCh, + wg: &it.wg, + tasks: it.tasks, + } + taskSender.respChan = it.respChan + copIteratorGP.Go(taskSender.run) +} - copIteratorGP.Go(func() { - // Send tasks to feed the worker goroutines. - for _, t := range it.tasks { - exit := it.sendToTaskCh(t, taskCh) - if exit { - break - } +func (sender *copIteratorTaskSender) run() { + // Send tasks to feed the worker goroutines. + for _, t := range sender.tasks { + exit := sender.sendToTaskCh(t) + if exit { + break } - close(taskCh) + } + close(sender.taskCh) - // Wait for worker goroutines to exit. - it.wg.Wait() - if !it.req.KeepOrder { - close(it.respChan) - } - }) + // Wait for worker goroutines to exit. + sender.wg.Wait() + if sender.respChan != nil { + close(sender.respChan) + } } func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan copResponse) (resp copResponse, ok bool, exit bool) { @@ -464,19 +495,19 @@ func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan copResp return } -func (it *copIterator) sendToTaskCh(t *copTask, taskCh chan<- *copTask) (exit bool) { +func (sender *copIteratorTaskSender) sendToTaskCh(t *copTask) (exit bool) { select { - case taskCh <- t: - case <-it.finished: + case sender.taskCh <- t: + case <-sender.finished: exit = true } return } -func (it *copIterator) sendToRespCh(resp copResponse, respCh chan<- copResponse) (exit bool) { +func (worker *copIteratorWorker) sendToRespCh(resp copResponse, respCh chan<- copResponse) (exit bool) { select { case respCh <- resp: - case <-it.finished: + case <-worker.finished: exit = true } return @@ -510,7 +541,7 @@ func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) { ) // If data order matters, response should be returned in the same order as copTask slice. // Otherwise all responses are returned from a single channel. - if !it.req.KeepOrder { + if it.respChan != nil { // Get next fetched resp from chan resp, ok, closed = it.recvFromRespCh(ctx, it.respChan) if !ok || closed { @@ -553,13 +584,13 @@ func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) { } // handleTask handles single copTask, sends the result to channel, retry automatically on error. -func (it *copIterator) handleTask(bo *Backoffer, task *copTask, ch chan copResponse) { +func (worker *copIteratorWorker) handleTask(bo *Backoffer, task *copTask, respCh chan<- copResponse) { remainTasks := []*copTask{task} for len(remainTasks) > 0 { - tasks, err := it.handleTaskOnce(bo, remainTasks[0], ch) + tasks, err := worker.handleTaskOnce(bo, remainTasks[0], respCh) if err != nil { resp := copResponse{err: errors.Trace(err)} - it.sendToRespCh(resp, ch) + worker.sendToRespCh(resp, respCh) return } if len(tasks) > 0 { @@ -572,25 +603,25 @@ func (it *copIterator) handleTask(bo *Backoffer, task *copTask, ch chan copRespo // handleTaskOnce handles single copTask, successful results are send to channel. // If error happened, returns error. If region split or meet lock, returns the remain tasks. -func (it *copIterator) handleTaskOnce(bo *Backoffer, task *copTask, ch chan copResponse) ([]*copTask, error) { +func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch chan<- copResponse) ([]*copTask, error) { // gofail: var handleTaskOnceError bool // if handleTaskOnceError { // return nil, errors.New("mock handleTaskOnce error") // } - sender := NewRegionRequestSender(it.store.regionCache, it.store.client) + sender := NewRegionRequestSender(worker.store.regionCache, worker.store.client) req := &tikvrpc.Request{ Type: task.cmdType, Cop: &coprocessor.Request{ - Tp: it.req.Tp, - Data: it.req.Data, + Tp: worker.req.Tp, + Data: worker.req.Data, Ranges: task.ranges.toPBRanges(), }, Context: kvrpcpb.Context{ - IsolationLevel: pbIsolationLevel(it.req.IsolationLevel), - Priority: kvPriorityToCommandPri(it.req.Priority), - NotFillCache: it.req.NotFillCache, + IsolationLevel: pbIsolationLevel(worker.req.IsolationLevel), + Priority: kvPriorityToCommandPri(worker.req.Priority), + NotFillCache: worker.req.NotFillCache, }, } startTime := time.Now() @@ -608,19 +639,24 @@ func (it *copIterator) handleTaskOnce(bo *Backoffer, task *copTask, ch chan copR task.storeAddr = sender.storeAddr if task.cmdType == tikvrpc.CmdCopStream { - return it.handleCopStreamResult(bo, resp.CopStream, task, ch) + return worker.handleCopStreamResult(bo, resp.CopStream, task, ch) } // Handles the response for non-streaming copTask. - return it.handleCopResponse(bo, resp.Cop, task, ch) + return worker.handleCopResponse(bo, resp.Cop, task, ch, nil) } -func (it *copIterator) handleCopStreamResult(bo *Backoffer, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan copResponse) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- copResponse) ([]*copTask, error) { defer stream.Close() - var resp, lastResp *coprocessor.Response + var resp *coprocessor.Response + var lastRange *coprocessor.KeyRange resp = stream.Response + if resp == nil { + // streaming request returns io.EOF, so the first Response is nil. + return nil, nil + } for { - remainedTasks, err := it.handleCopResponse(bo, resp, task, ch) + remainedTasks, err := worker.handleCopResponse(bo, resp, task, ch, lastRange) if err != nil || len(remainedTasks) != 0 { return remainedTasks, errors.Trace(err) } @@ -635,35 +671,29 @@ func (it *copIterator) handleCopStreamResult(bo *Backoffer, stream *tikvrpc.CopS } // No coprocessor.Response for network error, rebuild task based on the last success one. - ranges := task.ranges - if lastResp != nil { - if it.req.Desc { - ranges, _ = ranges.split(lastResp.GetRange().Start) - } else { - _, ranges = ranges.split(lastResp.GetRange().End) - } - } log.Info("stream recv timeout:", err) - return buildCopTasks(bo, it.store.regionCache, ranges, it.req.Desc, true) + return worker.buildCopTasksFromRemain(bo, lastRange, task) } - lastResp = resp + lastRange = resp.Range } } // handleCopResponse checks coprocessor Response for region split and lock, // returns more tasks when that happens, or handles the response if no error. -func (it *copIterator) handleCopResponse(bo *Backoffer, resp *coprocessor.Response, task *copTask, ch chan copResponse) ([]*copTask, error) { +// if we're handling streaming coprocessor response, lastRange is the range of last +// successful response, otherwise it's nil. +func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, resp *coprocessor.Response, task *copTask, ch chan<- copResponse, lastRange *coprocessor.KeyRange) ([]*copTask, error) { if regionErr := resp.GetRegionError(); regionErr != nil { if err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())); err != nil { return nil, errors.Trace(err) } // We may meet RegionError at the first packet, but not during visiting the stream. metrics.TiKVCoprocessorCounter.WithLabelValues("rebuild_task").Inc() - return buildCopTasks(bo, it.store.regionCache, task.ranges, it.req.Desc, it.req.Streaming) + return buildCopTasks(bo, worker.store.regionCache, task.ranges, worker.req.Desc, worker.req.Streaming) } if lockErr := resp.GetLocked(); lockErr != nil { log.Debugf("coprocessor encounters lock: %v", lockErr) - ok, err1 := it.store.lockResolver.ResolveLocks(bo, []*Lock{NewLock(lockErr)}) + ok, err1 := worker.store.lockResolver.ResolveLocks(bo, []*Lock{NewLock(lockErr)}) if err1 != nil { return nil, errors.Trace(err1) } @@ -672,7 +702,7 @@ func (it *copIterator) handleCopResponse(bo *Backoffer, resp *coprocessor.Respon return nil, errors.Trace(err) } } - return buildCopTasksFromRemain(bo, it.store.regionCache, resp, task, it.req.Desc, it.req.Streaming) + return worker.buildCopTasksFromRemain(bo, lastRange, task) } if otherErr := resp.GetOtherError(); otherErr != "" { err := errors.Errorf("other error: %s", otherErr) @@ -686,16 +716,16 @@ func (it *copIterator) handleCopResponse(bo *Backoffer, resp *coprocessor.Respon } else { startKey = task.ranges.at(0).StartKey } - it.sendToRespCh(copResponse{resp, startKey, nil}, ch) + worker.sendToRespCh(copResponse{resp, startKey, nil}, ch) return nil, nil } -func buildCopTasksFromRemain(bo *Backoffer, cache *RegionCache, resp *coprocessor.Response, task *copTask, desc bool, streaming bool) ([]*copTask, error) { +func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *Backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { remainedRanges := task.ranges - if streaming { - remainedRanges = calculateRemain(task.ranges, resp.Range, desc) + if worker.req.Streaming && lastRange != nil { + remainedRanges = worker.calculateRemain(task.ranges, lastRange, worker.req.Desc) } - return buildCopTasks(bo, cache, remainedRanges, desc, streaming) + return buildCopTasks(bo, worker.store.regionCache, remainedRanges, worker.req.Desc, worker.req.Streaming) } // calculateRemain splits the input ranges into two, and take one of them according to desc flag. @@ -705,7 +735,7 @@ func buildCopTasksFromRemain(bo *Backoffer, cache *RegionCache, resp *coprocesso // split: [s1 --> s2) // In normal scan order, all data before s1 is consumed, so the remain ranges should be [s1 --> r2) [r3 --> r4) // In reverse scan order, all data after s2 is consumed, so the remain ranges should be [r1 --> r2) [r3 --> s2) -func calculateRemain(ranges *copRanges, split *coprocessor.KeyRange, desc bool) *copRanges { +func (worker *copIteratorWorker) calculateRemain(ranges *copRanges, split *coprocessor.KeyRange, desc bool) *copRanges { if desc { left, _ := ranges.split(split.End) return left diff --git a/store/tikv/gcworker/gc_worker.go b/store/tikv/gcworker/gc_worker.go index 955225ab549b6..5aa1622c1d415 100644 --- a/store/tikv/gcworker/gc_worker.go +++ b/store/tikv/gcworker/gc_worker.go @@ -889,7 +889,7 @@ func (w *GCWorker) loadValueFromSysTable(key string, s session.Session) (string, return "", errors.Trace(err) } chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) if err != nil { return "", errors.Trace(err) } diff --git a/store/tikv/latch/latch.go b/store/tikv/latch/latch.go new file mode 100644 index 0000000000000..5bcac1817c1c3 --- /dev/null +++ b/store/tikv/latch/latch.go @@ -0,0 +1,199 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package latch + +import ( + "fmt" + "math/bits" + "sort" + "sync" + + "github.com/cznic/mathutil" + "github.com/spaolacci/murmur3" +) + +// Latch stores a key's waiting transactions information. +type Latch struct { + // Whether there is any transaction in waitingQueue except head. + hasMoreWaiting bool + // The startTS of the transaction which is the head of waiting transactions. + waitingQueueHead uint64 + maxCommitTS uint64 + sync.Mutex +} + +func (l *Latch) occupied() bool { + return l.waitingQueueHead != 0 +} + +func (l *Latch) free() { + l.waitingQueueHead = 0 +} + +// Lock is the locks' information required for a transaction. +type Lock struct { + // The slot IDs of the latches(keys) that a startTS must acquire before being able to processed. + requiredSlots []int + // The number of latches that the transaction has acquired. + acquiredCount int + // Whether current transaction is waiting + isWaiting bool + // Current transaction's startTS. + startTS uint64 +} + +// NewLock creates a new lock. +func NewLock(startTS uint64, requiredSlots []int) Lock { + return Lock{ + requiredSlots: requiredSlots, + acquiredCount: 0, + isWaiting: false, + startTS: startTS, + } +} + +// Latches which are used for concurrency control. +// Each latch is indexed by a slot's ID, hence the term latch and slot are used in interchangeable, +// but conceptually a latch is a queue, and a slot is an index to the queue +type Latches struct { + slots []Latch + // The waiting queue for each slot(slotID => slice of startTS). + waitingQueues map[int][]uint64 + sync.RWMutex +} + +// NewLatches create a Latches with fixed length, +// the size will be rounded up to the power of 2. +func NewLatches(size int) *Latches { + powerOfTwoSize := 1 << uint32(bits.Len32(uint32(size-1))) + slots := make([]Latch, powerOfTwoSize) + return &Latches{ + slots: slots, + waitingQueues: make(map[int][]uint64), + } +} + +// GenLock generates Lock for the transaction with startTS and keys. +func (latches *Latches) GenLock(startTS uint64, keys [][]byte) Lock { + slots := make([]int, 0, len(keys)) + for _, key := range keys { + slots = append(slots, latches.slotID(key)) + } + sort.Ints(slots) + if len(slots) <= 1 { + return NewLock(startTS, slots) + } + dedup := slots[:1] + for i := 1; i < len(slots); i++ { + if slots[i] != slots[i-1] { + dedup = append(dedup, slots[i]) + } + } + return NewLock(startTS, dedup) +} + +// slotID return slotID for current key. +func (latches *Latches) slotID(key []byte) int { + return int(murmur3.Sum32(key)) & (len(latches.slots) - 1) +} + +// Acquire tries to acquire the lock for a transaction. +// It returns with stale = true when the transaction is stale( +// when the lock.startTS is smaller than any key's last commitTS). +func (latches *Latches) Acquire(lock *Lock) (success, stale bool) { + for lock.acquiredCount < len(lock.requiredSlots) { + slotID := lock.requiredSlots[lock.acquiredCount] + success, stale = latches.acquireSlot(slotID, lock.startTS) + if success { + lock.acquiredCount++ + lock.isWaiting = false + continue + } + if !stale { + lock.isWaiting = true + } + return + } + return +} + +// Release releases all latches owned by the `lock` and returns the wakeup list. +// Preconditions: the caller must ensure the transaction is at the front of the latches. +func (latches *Latches) Release(lock *Lock, commitTS uint64) (wakeupList []uint64) { + releaseCount := lock.acquiredCount + if lock.isWaiting { + releaseCount++ + } + wakeupList = make([]uint64, 0, releaseCount) + for i := 0; i < releaseCount; i++ { + slotID := lock.requiredSlots[i] + + if hasNext, nextStartTS := latches.releaseSlot(slotID, lock.startTS, commitTS); hasNext { + wakeupList = append(wakeupList, nextStartTS) + } + } + return +} + +func (latches *Latches) releaseSlot(slotID int, startTS, commitTS uint64) (hasNext bool, nextStartTS uint64) { + latch := &latches.slots[slotID] + latch.Lock() + defer latch.Unlock() + if startTS != latch.waitingQueueHead { + panic(fmt.Sprintf("invalid front ts %d, latch:%#v", startTS, latch)) + } + latch.maxCommitTS = mathutil.MaxUint64(latch.maxCommitTS, commitTS) + if !latch.hasMoreWaiting { + latch.free() + return + } + latch.waitingQueueHead, latch.hasMoreWaiting = latches.popFromWaitingQueue(slotID) + return true, latch.waitingQueueHead +} + +func (latches *Latches) popFromWaitingQueue(slotID int) (front uint64, hasMoreWaiting bool) { + latches.Lock() + defer latches.Unlock() + waiting := latches.waitingQueues[slotID] + front = waiting[0] + if len(waiting) == 1 { + delete(latches.waitingQueues, slotID) + } else { + latches.waitingQueues[slotID] = waiting[1:] + hasMoreWaiting = true + } + return +} + +func (latches *Latches) acquireSlot(slotID int, startTS uint64) (success, stale bool) { + latch := &latches.slots[slotID] + latch.Lock() + defer latch.Unlock() + if stale = latch.maxCommitTS > startTS; stale { + return + } + // Empty latch + if !latch.occupied() { + latch.waitingQueueHead = startTS + } + if success = latch.waitingQueueHead == startTS; success { + return + } + // push current transaction into waitingQueue + latch.hasMoreWaiting = true + latches.Lock() + defer latches.Unlock() + latches.waitingQueues[slotID] = append(latches.waitingQueues[slotID], startTS) + return +} diff --git a/store/tikv/latch/latch_test.go b/store/tikv/latch/latch_test.go new file mode 100644 index 0000000000000..ed2066a8a04fc --- /dev/null +++ b/store/tikv/latch/latch_test.go @@ -0,0 +1,173 @@ +// Copyright 2018 PingCAP, Inc. +// +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package latch + +import ( + "sync" + "sync/atomic" + "testing" + + . "github.com/pingcap/check" +) + +func TestT(t *testing.T) { + TestingT(t) +} + +var _ = Suite(&testLatchSuite{}) + +var baseTso uint64 + +type testLatchSuite struct { + latches *Latches +} + +func (s *testLatchSuite) SetUpTest(c *C) { + s.latches = NewLatches(256) +} + +func (s *testLatchSuite) newLock(keys [][]byte) (startTS uint64, lock Lock) { + startTS = getTso() + lock = s.latches.GenLock(startTS, keys) + return +} + +func getTso() uint64 { + return atomic.AddUint64(&baseTso, uint64(1)) +} + +func (s *testLatchSuite) TestWakeUp(c *C) { + keysA := [][]byte{ + []byte("a"), []byte("b"), []byte("c"), []byte("c")} + _, lockA := s.newLock(keysA) + + keysB := [][]byte{[]byte("d"), []byte("e"), []byte("a"), []byte("c")} + startTSB, lockB := s.newLock(keysB) + + // A acquire lock success. + acquired, stale := s.latches.Acquire(&lockA) + c.Assert(stale, IsFalse) + c.Assert(acquired, IsTrue) + + // B acquire lock failed. + acquired, stale = s.latches.Acquire(&lockB) + c.Assert(stale, IsFalse) + c.Assert(acquired, IsFalse) + + // A release lock, and get wakeup list. + commitTSA := getTso() + wakeupList := s.latches.Release(&lockA, commitTSA) + c.Assert(wakeupList[0], Equals, startTSB) + + // B acquire failed since startTSB has stale for some keys. + acquired, stale = s.latches.Acquire(&lockB) + c.Assert(stale, IsTrue) + c.Assert(acquired, IsFalse) + + // B release lock since it received a stale. + wakeupList = s.latches.Release(&lockB, 0) + c.Assert(len(wakeupList), Equals, 0) + + // B restart:get a new startTS. + startTSB = getTso() + lockB = s.latches.GenLock(startTSB, keysB) + acquired, stale = s.latches.Acquire(&lockB) + c.Assert(acquired, IsTrue) + c.Assert(stale, IsFalse) +} + +type txn struct { + keys [][]byte + startTS uint64 + lock Lock +} + +func newTxn(keys [][]byte, startTS uint64, lock Lock) txn { + return txn{ + keys: keys, + startTS: startTS, + lock: lock, + } +} + +type txnScheduler struct { + txns map[uint64]*txn + latches *Latches + lock sync.Mutex + wait *sync.WaitGroup +} + +func newTxnScheduler(wait *sync.WaitGroup, latches *Latches) *txnScheduler { + return &txnScheduler{ + txns: make(map[uint64]*txn), + latches: latches, + wait: wait, + } +} + +func (store *txnScheduler) runTxn(startTS uint64) { + store.lock.Lock() + txn, ok := store.txns[startTS] + store.lock.Unlock() + if !ok { + panic(startTS) + } + acquired, stale := store.latches.Acquire(&txn.lock) + + if !stale && !acquired { + return + } + commitTs := uint64(0) + if stale { + // restart Txn + go store.newTxn(txn.keys) + } else { + // DO commit + commitTs = getTso() + store.wait.Done() + } + wakeupList := store.latches.Release(&txn.lock, commitTs) + for _, s := range wakeupList { + go store.runTxn(s) + } + store.lock.Lock() + delete(store.txns, startTS) + store.lock.Unlock() +} + +func (store *txnScheduler) newTxn(keys [][]byte) { + startTS := getTso() + lock := store.latches.GenLock(startTS, keys) + t := newTxn(keys, startTS, lock) + store.lock.Lock() + store.txns[t.startTS] = &t + store.lock.Unlock() + go store.runTxn(t.startTS) +} + +func (s *testLatchSuite) TestWithConcurrency(c *C) { + waitGroup := sync.WaitGroup{} + txns := [][][]byte{ + {[]byte("a"), []byte("a"), []byte("b"), []byte("c")}, + {[]byte("a"), []byte("d"), []byte("e"), []byte("f")}, + {[]byte("e"), []byte("f"), []byte("g"), []byte("h")}, + } + + store := newTxnScheduler(&waitGroup, s.latches) + waitGroup.Add(len(txns)) + for _, txn := range txns { + go store.newTxn(txn) + } + waitGroup.Wait() +} diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go index 92c056faa360e..dad5426d3d91c 100644 --- a/store/tikv/rawkv.go +++ b/store/tikv/rawkv.go @@ -14,6 +14,7 @@ package tikv import ( + "bytes" "time" "github.com/juju/errors" @@ -155,6 +156,39 @@ func (c *RawKVClient) Delete(key []byte) error { return nil } +// DeleteRange deletes all key-value pairs in a range from TiKV +func (c *RawKVClient) DeleteRange(startKey []byte, endKey []byte) error { + start := time.Now() + var err error + defer func() { + var label = "delete_range" + if err != nil { + label += "_error" + } + metrics.TiKVRawkvCmdHistogram.WithLabelValues(label).Observe(time.Since(start).Seconds()) + }() + + // Process each affected region respectively + for !bytes.Equal(startKey, endKey) { + var resp *tikvrpc.Response + var actualEndKey []byte + resp, actualEndKey, err = c.sendDeleteRangeReq(startKey, endKey) + if err != nil { + return errors.Trace(err) + } + cmdResp := resp.RawDeleteRange + if cmdResp == nil { + return errors.Trace(ErrBodyMissing) + } + if cmdResp.GetError() != "" { + return errors.New(cmdResp.GetError()) + } + startKey = actualEndKey + } + + return nil +} + // Scan queries continuous kv pairs, starts from startKey, up to limit pairs. // If you want to exclude the startKey, append a '\0' to the key: `Scan(append(startKey, '\0'), limit)`. func (c *RawKVClient) Scan(startKey []byte, limit int) (keys [][]byte, values [][]byte, err error) { @@ -219,3 +253,48 @@ func (c *RawKVClient) sendReq(key []byte, req *tikvrpc.Request) (*tikvrpc.Respon return resp, loc, nil } } + +// sendDeleteRangeReq sends a raw delete range request and returns the response and the actual endKey. +// If the given range spans over more than one regions, the actual endKey is the end of the first region. +// We can't use sendReq directly, because we need to know the end of the region before we send the request +// TODO: Is there any better way to avoid duplicating code with func `sendReq` ? +func (c *RawKVClient) sendDeleteRangeReq(startKey []byte, endKey []byte) (*tikvrpc.Response, []byte, error) { + bo := NewBackoffer(context.Background(), rawkvMaxBackoff) + sender := NewRegionRequestSender(c.regionCache, c.rpcClient) + for { + loc, err := c.regionCache.LocateKey(bo, startKey) + if err != nil { + return nil, nil, errors.Trace(err) + } + + actualEndKey := endKey + if len(loc.EndKey) > 0 && bytes.Compare(loc.EndKey, endKey) < 0 { + actualEndKey = loc.EndKey + } + + req := &tikvrpc.Request{ + Type: tikvrpc.CmdRawDeleteRange, + RawDeleteRange: &kvrpcpb.RawDeleteRangeRequest{ + StartKey: startKey, + EndKey: actualEndKey, + }, + } + + resp, err := sender.SendReq(bo, req, loc.Region, readTimeoutShort) + if err != nil { + return nil, nil, errors.Trace(err) + } + regionErr, err := resp.GetRegionError() + if err != nil { + return nil, nil, errors.Trace(err) + } + if regionErr != nil { + err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + return nil, nil, errors.Trace(err) + } + continue + } + return resp, actualEndKey, nil + } +} diff --git a/store/tikv/rawkv_test.go b/store/tikv/rawkv_test.go index f0ec33984e6ae..58a1113d8031c 100644 --- a/store/tikv/rawkv_test.go +++ b/store/tikv/rawkv_test.go @@ -14,6 +14,8 @@ package tikv import ( + "bytes" + . "github.com/pingcap/check" "github.com/pingcap/tidb/store/mockstore/mocktikv" "golang.org/x/net/context" @@ -32,11 +34,12 @@ func (s *testRawKVSuite) SetUpTest(c *C) { s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) pdClient := mocktikv.NewPDClient(s.cluster) + mvccStore := mocktikv.MustNewMVCCStore() s.client = &RawKVClient{ clusterID: 0, regionCache: NewRegionCache(pdClient), pdClient: pdClient, - rpcClient: mocktikv.NewRPCClient(s.cluster, mocktikv.NewMvccStore()), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore), } s.bo = NewBackoffer(context.Background(), 5000) } @@ -78,6 +81,41 @@ func (s *testRawKVSuite) mustScan(c *C, startKey string, limit int, expect ...st } } +func (s *testRawKVSuite) mustDeleteRange(c *C, startKey, endKey []byte, expected map[string]string) { + err := s.client.DeleteRange(startKey, endKey) + c.Assert(err, IsNil) + + for keyStr := range expected { + key := []byte(keyStr) + if bytes.Compare(startKey, key) <= 0 && bytes.Compare(key, endKey) < 0 { + delete(expected, keyStr) + } + } + + s.checkData(c, expected) +} + +func (s *testRawKVSuite) checkData(c *C, expected map[string]string) { + keys, values, err := s.client.Scan([]byte(""), len(expected)+1) + c.Assert(err, IsNil) + + c.Assert(len(expected), Equals, len(keys)) + for i, key := range keys { + c.Assert(expected[string(key)], Equals, string(values[i])) + } +} + +func (s *testRawKVSuite) split(c *C, regionKey, splitKey string) error { + loc, err := s.client.regionCache.LocateKey(s.bo, []byte(regionKey)) + if err != nil { + return err + } + + newRegionID, peerID := s.cluster.AllocID(), s.cluster.AllocID() + s.cluster.SplitRaw(loc.Region.id, newRegionID, []byte(splitKey), []uint64{peerID}, peerID) + return nil +} + func (s *testRawKVSuite) TestSimple(c *C) { s.mustNotExist(c, []byte("key")) s.mustPut(c, []byte("key"), []byte("value")) @@ -89,13 +127,11 @@ func (s *testRawKVSuite) TestSimple(c *C) { } func (s *testRawKVSuite) TestSplit(c *C) { - loc, err := s.client.regionCache.LocateKey(s.bo, []byte("k")) - c.Assert(err, IsNil) s.mustPut(c, []byte("k1"), []byte("v1")) s.mustPut(c, []byte("k3"), []byte("v3")) - newRegionID, peerID := s.cluster.AllocID(), s.cluster.AllocID() - s.cluster.SplitRaw(loc.Region.id, newRegionID, []byte("k2"), []uint64{peerID}, peerID) + err := s.split(c, "k", "k2") + c.Assert(err, IsNil) s.mustGet(c, []byte("k1"), []byte("v1")) s.mustGet(c, []byte("k3"), []byte("v3")) @@ -115,16 +151,41 @@ func (s *testRawKVSuite) TestScan(c *C) { s.mustScan(c, "k2", 3, "k3", "v3", "k5", "v5", "k7", "v7") } - split := func(regionKey, splitKey string) { - loc, err := s.client.regionCache.LocateKey(s.bo, []byte(regionKey)) - c.Assert(err, IsNil) - newRegionID, peerID := s.cluster.AllocID(), s.cluster.AllocID() - s.cluster.SplitRaw(loc.Region.id, newRegionID, []byte(splitKey), []uint64{peerID}, peerID) - } - check() - split("k", "k2") + + err := s.split(c, "k", "k2") + c.Assert(err, IsNil) check() - split("k2", "k5") + + err = s.split(c, "k2", "k5") + c.Assert(err, IsNil) check() } + +func (s *testRawKVSuite) TestDeleteRange(c *C) { + // Init data + testData := map[string]string{} + for _, i := range []byte("abcd") { + for j := byte('0'); j <= byte('9'); j++ { + key := []byte{i, j} + value := []byte{'v', i, j} + s.mustPut(c, key, value) + + testData[string(key)] = string(value) + } + } + + err := s.split(c, "b", "b") + c.Assert(err, IsNil) + err = s.split(c, "c", "c") + c.Assert(err, IsNil) + err = s.split(c, "d", "d") + c.Assert(err, IsNil) + + s.checkData(c, testData) + s.mustDeleteRange(c, []byte("b"), []byte("c0"), testData) + s.mustDeleteRange(c, []byte("c11"), []byte("c12"), testData) + s.mustDeleteRange(c, []byte("d0"), []byte("d0"), testData) + s.mustDeleteRange(c, []byte("c5"), []byte("d5"), testData) + s.mustDeleteRange(c, []byte("a"), []byte("z"), testData) +} diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index 5cd29cf4daf15..761c14239c74e 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -285,10 +285,13 @@ func (s *testRegionCacheSuite) TestRequestFail2(c *C) { } func (s *testRegionCacheSuite) TestUpdateStoreAddr(c *C) { + mvccStore := mocktikv.MustNewMVCCStore() + defer mvccStore.Close() + client := &RawKVClient{ clusterID: 0, regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mocktikv.NewMvccStore()), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore), } testKey := []byte("test_key") testValue := []byte("test_value") diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index fb65358ae79c3..6d302de7b46dd 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -39,7 +39,7 @@ type testRegionRequestSuite struct { cache *RegionCache bo *Backoffer regionRequestSender *RegionRequestSender - mvccStore *mocktikv.MvccStore + mvccStore mocktikv.MVCCStore } var _ = Suite(&testRegionRequestSuite{}) @@ -50,7 +50,7 @@ func (s *testRegionRequestSuite) SetUpTest(c *C) { pdCli := &codecPDClient{mocktikv.NewPDClient(s.cluster)} s.cache = NewRegionCache(pdCli) s.bo = NewBackoffer(context.Background(), 1) - s.mvccStore = mocktikv.NewMvccStore() + s.mvccStore = mocktikv.MustNewMVCCStore() client := mocktikv.NewRPCClient(s.cluster, s.mvccStore) s.regionRequestSender = NewRegionRequestSender(s.cache, client) } @@ -196,19 +196,33 @@ func (s *mockTikvGrpcServer) KvGC(context.Context, *kvrpcpb.GCRequest) (*kvrpcpb func (s *mockTikvGrpcServer) KvDeleteRange(context.Context, *kvrpcpb.DeleteRangeRequest) (*kvrpcpb.DeleteRangeResponse, error) { return nil, errors.New("unreachable") } - func (s *mockTikvGrpcServer) RawGet(context.Context, *kvrpcpb.RawGetRequest) (*kvrpcpb.RawGetResponse, error) { return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) RawBatchGet(context.Context, *kvrpcpb.RawBatchGetRequest) (*kvrpcpb.RawBatchGetResponse, error) { + return nil, errors.New("unreachable") +} func (s *mockTikvGrpcServer) RawPut(context.Context, *kvrpcpb.RawPutRequest) (*kvrpcpb.RawPutResponse, error) { return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) RawBatchPut(context.Context, *kvrpcpb.RawBatchPutRequest) (*kvrpcpb.RawBatchPutResponse, error) { + return nil, errors.New("unreachable") +} func (s *mockTikvGrpcServer) RawDelete(context.Context, *kvrpcpb.RawDeleteRequest) (*kvrpcpb.RawDeleteResponse, error) { return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) RawBatchDelete(context.Context, *kvrpcpb.RawBatchDeleteRequest) (*kvrpcpb.RawBatchDeleteResponse, error) { + return nil, errors.New("unreachable") +} func (s *mockTikvGrpcServer) RawScan(context.Context, *kvrpcpb.RawScanRequest) (*kvrpcpb.RawScanResponse, error) { return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) RawDeleteRange(context.Context, *kvrpcpb.RawDeleteRangeRequest) (*kvrpcpb.RawDeleteRangeResponse, error) { + return nil, errors.New("unreachable") +} +func (s *mockTikvGrpcServer) RawBatchScan(context.Context, *kvrpcpb.RawBatchScanRequest) (*kvrpcpb.RawBatchScanResponse, error) { + return nil, errors.New("unreachable") +} func (s *mockTikvGrpcServer) Coprocessor(context.Context, *coprocessor.Request) (*coprocessor.Response, error) { return nil, errors.New("unreachable") } diff --git a/store/tikv/sql_fail_test.go b/store/tikv/sql_fail_test.go index 643194cf2b72a..eec2437652c7f 100644 --- a/store/tikv/sql_fail_test.go +++ b/store/tikv/sql_fail_test.go @@ -73,7 +73,7 @@ func (s *testSQLSuite) TestFailBusyServerCop(c *C) { } c.Assert(err, IsNil) chk := rs[0].NewChunk() - err = rs[0].NextChunk(context.Background(), chk) + err = rs[0].Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) c.Assert(chk.GetRow(0).GetString(0), Equals, "True") @@ -109,7 +109,7 @@ func (s *testSQLSuite) TestCoprocessorStreamRecvTimeout(c *C) { chk := res[0].NewChunk() for { - err := res[0].NextChunk(ctx, chk) + err := res[0].Next(ctx, chk) c.Assert(err, IsNil) if chk.NumRows() == 0 { break diff --git a/store/tikv/tikvrpc/tikvrpc.go b/store/tikv/tikvrpc/tikvrpc.go index 943ab2a4ed62c..6862c4b0f892e 100644 --- a/store/tikv/tikvrpc/tikvrpc.go +++ b/store/tikv/tikvrpc/tikvrpc.go @@ -47,6 +47,7 @@ const ( CmdRawGet CmdType = 256 + iota CmdRawPut CmdRawDelete + CmdRawDeleteRange CmdRawScan CmdCop CmdType = 512 + iota @@ -87,6 +88,8 @@ func (t CmdType) String() string { return "RawPut" case CmdRawDelete: return "RawDelete" + case CmdRawDeleteRange: + return "RawDeleteRange" case CmdRawScan: return "RawScan" case CmdCop: @@ -121,6 +124,7 @@ type Request struct { RawGet *kvrpcpb.RawGetRequest RawPut *kvrpcpb.RawPutRequest RawDelete *kvrpcpb.RawDeleteRequest + RawDeleteRange *kvrpcpb.RawDeleteRangeRequest RawScan *kvrpcpb.RawScanRequest Cop *coprocessor.Request MvccGetByKey *kvrpcpb.MvccGetByKeyRequest @@ -145,6 +149,7 @@ type Response struct { RawGet *kvrpcpb.RawGetResponse RawPut *kvrpcpb.RawPutResponse RawDelete *kvrpcpb.RawDeleteResponse + RawDeleteRange *kvrpcpb.RawDeleteRangeResponse RawScan *kvrpcpb.RawScanResponse Cop *coprocessor.Response CopStream *CopStreamResponse @@ -199,6 +204,8 @@ func SetContext(req *Request, region *metapb.Region, peer *metapb.Peer) error { req.RawPut.Context = ctx case CmdRawDelete: req.RawDelete.Context = ctx + case CmdRawDeleteRange: + req.RawDeleteRange.Context = ctx case CmdRawScan: req.RawScan.Context = ctx case CmdCop: @@ -279,6 +286,10 @@ func GenRegionErrorResp(req *Request, e *errorpb.Error) (*Response, error) { resp.RawDelete = &kvrpcpb.RawDeleteResponse{ RegionError: e, } + case CmdRawDeleteRange: + resp.RawDeleteRange = &kvrpcpb.RawDeleteRangeResponse{ + RegionError: e, + } case CmdRawScan: resp.RawScan = &kvrpcpb.RawScanResponse{ RegionError: e, @@ -343,6 +354,8 @@ func (resp *Response) GetRegionError() (*errorpb.Error, error) { e = resp.RawPut.GetRegionError() case CmdRawDelete: e = resp.RawDelete.GetRegionError() + case CmdRawDeleteRange: + e = resp.RawDeleteRange.GetRegionError() case CmdRawScan: e = resp.RawScan.GetRegionError() case CmdCop: @@ -397,6 +410,8 @@ func CallRPC(ctx context.Context, client tikvpb.TikvClient, req *Request) (*Resp resp.RawPut, err = client.RawPut(ctx, req.RawPut) case CmdRawDelete: resp.RawDelete, err = client.RawDelete(ctx, req.RawDelete) + case CmdRawDeleteRange: + resp.RawDeleteRange, err = client.RawDeleteRange(ctx, req.RawDeleteRange) case CmdRawScan: resp.RawScan, err = client.RawScan(ctx, req.RawScan) case CmdCop: diff --git a/structure/list.go b/structure/list.go index 921ec3c179280..e46567539f107 100644 --- a/structure/list.go +++ b/structure/list.go @@ -136,6 +136,26 @@ func (t *TxStructure) LLen(key []byte) (int64, error) { return meta.RIndex - meta.LIndex, errors.Trace(err) } +// LGetAll gets all elements of this list in order from right to left. +func (t *TxStructure) LGetAll(key []byte) ([][]byte, error) { + metaKey := t.encodeListMetaKey(key) + meta, err := t.loadListMeta(metaKey) + if err != nil || meta.IsEmpty() { + return nil, errors.Trace(err) + } + + length := int(meta.RIndex - meta.LIndex) + elements := make([][]byte, 0, length) + for index := meta.RIndex - 1; index >= meta.LIndex; index-- { + e, err := t.reader.Get(t.encodeListDataKey(key, index)) + if err != nil { + return nil, errors.Trace(err) + } + elements = append(elements, e) + } + return elements, nil +} + // LIndex gets an element from a list by its index. func (t *TxStructure) LIndex(key []byte, index int64) ([]byte, error) { metaKey := t.encodeListMetaKey(key) diff --git a/structure/structure_test.go b/structure/structure_test.go index 89db3c98e96a0..f5dbfc0fc474f 100644 --- a/structure/structure_test.go +++ b/structure/structure_test.go @@ -98,11 +98,21 @@ func (s *testTxStructureSuite) TestList(c *C) { err = tx.LPush(key, []byte("3"), []byte("2"), []byte("1")) c.Assert(err, IsNil) + // Test LGetAll. + err = tx.LPush(key, []byte("11")) + c.Assert(err, IsNil) + values, err := tx.LGetAll(key) + c.Assert(err, IsNil) + c.Assert(values, DeepEquals, [][]byte{[]byte("3"), []byte("2"), []byte("1"), []byte("11")}) + value, err := tx.LPop(key) + c.Assert(err, IsNil) + c.Assert(value, DeepEquals, []byte("11")) + l, err := tx.LLen(key) c.Assert(err, IsNil) c.Assert(l, Equals, int64(3)) - value, err := tx.LIndex(key, 1) + value, err = tx.LIndex(key, 1) c.Assert(err, IsNil) c.Assert(value, DeepEquals, []byte("2")) diff --git a/table/column.go b/table/column.go index 5e5e7b92a6f71..5bc63ff27c777 100644 --- a/table/column.go +++ b/table/column.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" + "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/hack" log "github.com/sirupsen/logrus" ) @@ -370,7 +371,13 @@ func GetZeroValue(col *model.ColumnInfo) types.Datum { d.SetFloat64(0) case mysql.TypeNewDecimal: d.SetMysqlDecimal(new(types.MyDecimal)) - case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: + case mysql.TypeString: + if col.Flen > 0 && col.Charset == charset.CharsetBin { + d.SetBytes(make([]byte, col.Flen)) + } else { + d.SetString("") + } + case mysql.TypeVarString, mysql.TypeVarchar: d.SetString("") case mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: d.SetBytes([]byte{}) diff --git a/table/column_test.go b/table/column_test.go index 29e6d7d9b8706..45d2d5bdaf529 100644 --- a/table/column_test.go +++ b/table/column_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" ) @@ -193,6 +194,15 @@ func (t *testTableSuite) TestGetZeroValue(c *C) { types.NewFieldType(mysql.TypeEnum), types.NewDatum(types.Enum{}), }, + { + &types.FieldType{ + Tp: mysql.TypeString, + Flen: 2, + Charset: charset.CharsetBin, + Collate: charset.CollationBin, + }, + types.NewDatum(make([]byte, 2)), + }, } sc := new(stmtctx.StatementContext) for _, tt := range tests { diff --git a/table/tables/index.go b/table/tables/index.go index 92f38cc07f03f..98df4b3e73c7e 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -29,7 +29,8 @@ import ( "github.com/pingcap/tidb/util/codec" ) -func encodeHandle(h int64) []byte { +// EncodeHandle encodes handle in data. +func EncodeHandle(h int64) []byte { buf := &bytes.Buffer{} err := binary.Write(buf, binary.BigEndian, h) if err != nil { @@ -189,7 +190,7 @@ func (c *index) Create(ctx sessionctx.Context, rm kv.RetrieverMutator, indexedVa } if skipCheck || kv.IsErrNotFound(err) { - err = rm.Set(key, encodeHandle(h)) + err = rm.Set(key, EncodeHandle(h)) return 0, errors.Trace(err) } diff --git a/table/tables/tables.go b/table/tables/tables.go index 0a7c2e94fba71..f486977e446c8 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -74,12 +74,18 @@ func TableFromMeta(alloc autoid.Allocator, tblInfo *model.TableInfo) (table.Tabl return nil, table.ErrTableStateCantNone.Gen("table %s can't be in none state", tblInfo.Name) } - columns := make([]*table.Column, 0, len(tblInfo.Columns)) - for _, colInfo := range tblInfo.Columns { + colsLen := len(tblInfo.Columns) + columns := make([]*table.Column, 0, colsLen) + for i, colInfo := range tblInfo.Columns { if colInfo.State == model.StateNone { return nil, table.ErrColumnStateCantNone.Gen("column %s can't be in none state", colInfo.Name) } + // Print some information when the column's offset isn't equal to i. + if colInfo.Offset != i { + log.Errorf("[tables] table %#v schema is wrong, no.%d col %#v, cols len %v", tblInfo, i, tblInfo.Columns[i], colsLen) + } + col := table.ToColumn(colInfo) if col.IsGenerated() { expr, err := parseExpression(colInfo.GeneratedExprString) @@ -434,7 +440,14 @@ func (t *Table) AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleChe } } sessVars.StmtCtx.AddAffectedRows(1) - sessVars.TxnCtx.UpdateDeltaForTable(t.ID, 1, 1) + colSize := make(map[int64]int64) + for id, col := range t.Cols() { + val := int64(len(r[id].GetBytes())) + if val != 0 { + colSize[col.ID] = val + } + } + sessVars.TxnCtx.UpdateDeltaForTable(t.ID, 1, 1, colSize) return recordID, nil } @@ -506,14 +519,23 @@ func (t *Table) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Colum if err != nil { return nil, errors.Trace(err) } - // Decode raw row data. + v, _, err := DecodeRawRowData(ctx, t.Meta(), h, cols, value) + if err != nil { + return nil, errors.Trace(err) + } + return v, nil +} + +// DecodeRawRowData decodes raw row data into a datum slice and a (columnID:columnValue) map. +func DecodeRawRowData(ctx sessionctx.Context, meta *model.TableInfo, h int64, cols []*table.Column, + value []byte) ([]types.Datum, map[int64]types.Datum, error) { v := make([]types.Datum, len(cols)) colTps := make(map[int64]*types.FieldType, len(cols)) for i, col := range cols { if col == nil { continue } - if col.IsPKHandleColumn(t.meta) { + if col.IsPKHandleColumn(meta) { if mysql.HasUnsignedFlag(col.Flag) { v[i].SetUint64(uint64(h)) } else { @@ -525,14 +547,14 @@ func (t *Table) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Colum } rowMap, err := tablecodec.DecodeRow(value, colTps, ctx.GetSessionVars().GetTimeZone()) if err != nil { - return nil, errors.Trace(err) + return nil, rowMap, errors.Trace(err) } defaultVals := make([]types.Datum, len(cols)) for i, col := range cols { if col == nil { continue } - if col.IsPKHandleColumn(t.meta) { + if col.IsPKHandleColumn(meta) { continue } ri, ok := rowMap[col.ID] @@ -542,10 +564,10 @@ func (t *Table) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Colum } v[i], err = GetColDefaultValue(ctx, col, defaultVals) if err != nil { - return nil, errors.Trace(err) + return nil, rowMap, errors.Trace(err) } } - return v, nil + return v, rowMap, nil } // Row implements table.Table Row interface. @@ -837,12 +859,16 @@ func (t *Table) getMutation(ctx sessionctx.Context) *binlog.TableMutation { return ctx.StmtGetMutation(t.ID) } -// canSkip is for these cases, we can skip the columns in encoded row: +func (t *Table) canSkip(col *table.Column, value types.Datum) bool { + return CanSkip(t.Meta(), col, value) +} + +// CanSkip is for these cases, we can skip the columns in encoded row: // 1. the column is included in primary key; // 2. the column's default value is null, and the value equals to that; // 3. the column is virtual generated. -func (t *Table) canSkip(col *table.Column, value types.Datum) bool { - if col.IsPKHandleColumn(t.meta) { +func CanSkip(info *model.TableInfo, col *table.Column, value types.Datum) bool { + if col.IsPKHandleColumn(info) { return true } if col.DefaultValue == nil && value.IsNull() { diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index 4602503c9a6db..a056d7eec50ed 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -159,7 +159,7 @@ func (ts *testSuite) TestTypes(c *C) { rs, err := ts.se.Execute(ctx, "select * from test.t where c1 = 1") c.Assert(err, IsNil) chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) _, err = ts.se.Execute(ctx, "drop table test.t") @@ -172,7 +172,7 @@ func (ts *testSuite) TestTypes(c *C) { rs, err = ts.se.Execute(ctx, "select * from test.t where c1 = 1") c.Assert(err, IsNil) chk = rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row := chk.GetRow(0) @@ -187,7 +187,7 @@ func (ts *testSuite) TestTypes(c *C) { rs, err = ts.se.Execute(ctx, "select c1 + 1 from test.t where c1 = 1") c.Assert(err, IsNil) chk = rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) c.Assert(chk.GetRow(0).GetFloat64(0), DeepEquals, float64(2)) diff --git a/tidb-server/main.go b/tidb-server/main.go index 4c2da35c87eef..20a3d26bd5b29 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -93,7 +93,7 @@ var ( socket = flag.String(nmSocket, "", "The socket file to use for connection.") binlogSocket = flag.String(nmBinlogSocket, "", "socket file to write binlog") runDDL = flagBoolean(nmRunDDL, true, "run ddl worker on this tidb-server") - ddlLease = flag.String(nmDdlLease, "10s", "schema lease duration, very dangerous to change only if you know what you do") + ddlLease = flag.String(nmDdlLease, "45s", "schema lease duration, very dangerous to change only if you know what you do") tokenLimit = flag.Int(nmTokenLimit, 1000, "the limit of concurrent executed sessions") // Log @@ -365,18 +365,16 @@ func setGlobalVars() { session.SetStatsLease(statsLeaseDuration) domain.RunAutoAnalyze = cfg.Performance.RunAutoAnalyze statistics.FeedbackProbability = cfg.Performance.FeedbackProbability + statistics.MaxQueryFeedbackCount = int(cfg.Performance.QueryFeedbackLimit) + plan.RatioOfPseudoEstimate = cfg.Performance.PseudoEstimateRatio ddl.RunWorker = cfg.RunDDL ddl.EnableSplitTableRegion = cfg.SplitTable session.SetCommitRetryLimit(cfg.Performance.RetryLimit) - plan.JoinConcurrency = cfg.Performance.JoinConcurrency plan.AllowCartesianProduct = cfg.Performance.CrossJoin privileges.SkipWithGrant = cfg.Security.SkipGrantTable - plan.PlanCacheEnabled = cfg.PlanCache.Enabled - if plan.PlanCacheEnabled { - plan.PlanCacheCapacity = cfg.PlanCache.Capacity - plan.PlanCacheShards = cfg.PlanCache.Shards - plan.GlobalPlanCache = kvcache.NewShardedLRUCache(plan.PlanCacheCapacity, plan.PlanCacheShards) + if cfg.PlanCache.Enabled { + plan.GlobalPlanCache = kvcache.NewShardedLRUCache(cfg.PlanCache.Capacity, cfg.PlanCache.Shards) } plan.PreparedPlanCacheEnabled = cfg.PreparedPlanCache.Enabled diff --git a/types/datum.go b/types/datum.go index 1605a03ed3546..549c3127cd21d 100644 --- a/types/datum.go +++ b/types/datum.go @@ -112,7 +112,7 @@ func (d *Datum) Length() int { return int(d.length) } -// SetLength sets the length of the datum +// SetLength sets the length of the datum. func (d *Datum) SetLength(l int) { d.length = uint32(l) } @@ -1103,6 +1103,10 @@ func ProduceDecWithSpecifiedTp(dec *MyDecimal, tp *FieldType, sc *stmtctx.Statem // TODO: warnErr need to be ErrWarnDataOutOfRange err = sc.HandleOverflow(err, err) } + unsigned := mysql.HasUnsignedFlag(tp.Flag) + if unsigned && dec.IsNegative() { + dec = dec.FromUint(0) + } return dec, errors.Trace(err) } diff --git a/util/admin/admin.go b/util/admin/admin.go index aa9b82301efb3..dd39f551c5332 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -95,7 +95,7 @@ func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { continue } // If the state is rolling back, it means the work is cleaning the data after cancelling the job. - if job.IsCancelled() || job.IsRollingback() { + if job.IsCancelled() || job.IsRollingback() || job.IsRollbackDone() { continue } job.State = model.JobStateCancelling diff --git a/util/prefix_helper_test.go b/util/prefix_helper_test.go index 9e3e7c62c0506..9edcef49a64c1 100644 --- a/util/prefix_helper_test.go +++ b/util/prefix_helper_test.go @@ -127,7 +127,7 @@ func (s *testPrefixSuite) TestPrefix(c *C) { txn, err = s.s.Begin() c.Assert(err, IsNil) k := []byte("key100jfowi878230") - err = txn.Set(k, []byte("val32dfaskli384757^*&%^")) + err = txn.Set(k, []byte(`val32dfaskli384757^*&%^`)) c.Assert(err, IsNil) err = util.ScanMetaWithPrefix(txn, k, func(kv.Key, []byte) bool { return true @@ -138,7 +138,7 @@ func (s *testPrefixSuite) TestPrefix(c *C) { } func (s *testPrefixSuite) TestPrefixFilter(c *C) { - rowKey := []byte("test@#$%l(le[0]..prefix) 2uio") + rowKey := []byte(`test@#$%l(le[0]..prefix) 2uio`) rowKey[8] = 0x00 rowKey[9] = 0x00 f := util.RowKeyPrefixFilter(rowKey) diff --git a/util/printer/printer.go b/util/printer/printer.go index f391ce5554457..843e287591fc2 100644 --- a/util/printer/printer.go +++ b/util/printer/printer.go @@ -30,7 +30,7 @@ var ( TiDBGitBranch = "None" GoVersion = "None" // TiKVMinVersion is the minimum version of TiKV that can be compatible with the current TiDB. - TiKVMinVersion = "1.1.0-dev.2" + TiKVMinVersion = "2.0.0-rc.4.1" ) // PrintTiDBInfo prints the TiDB version information. diff --git a/util/processinfo.go b/util/processinfo.go index c719ff62b0611..2d204c8a2a5c5 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -27,6 +27,7 @@ type ProcessInfo struct { Time time.Time State uint16 Info string + Mem int64 } // SessionManager is an interface for session manage. Show processlist and diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index fc34a67fe851a..241e5699ac719 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -51,7 +51,7 @@ func (s *testRangerSuite) SetUpSuite(c *C) { func newStoreWithBootstrap(c *C) (kv.Storage, error) { cluster := mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(cluster) - mvccStore := mocktikv.NewMvccStore() + mvccStore := mocktikv.MustNewMVCCStore() store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(cluster), mockstore.WithMVCCStore(mvccStore), diff --git a/vendor/github.com/pingcap/kvproto/pkg/coprocessor/coprocessor.pb.go b/vendor/github.com/pingcap/kvproto/pkg/coprocessor/coprocessor.pb.go index 427fbe792cd08..03a1644d7f451 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/coprocessor/coprocessor.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/coprocessor/coprocessor.pb.go @@ -1,5 +1,6 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. +// Code generated by protoc-gen-gogo. // source: coprocessor.proto +// DO NOT EDIT! /* Package coprocessor is a generated protocol buffer package. @@ -110,6 +111,7 @@ type Response struct { Locked *kvrpcpb.LockInfo `protobuf:"bytes,3,opt,name=locked" json:"locked,omitempty"` OtherError string `protobuf:"bytes,4,opt,name=other_error,json=otherError,proto3" json:"other_error,omitempty"` Range *KeyRange `protobuf:"bytes,5,opt,name=range" json:"range,omitempty"` + ExecDetails *kvrpcpb.ExecDetails `protobuf:"bytes,6,opt,name=exec_details,json=execDetails" json:"exec_details,omitempty"` } func (m *Response) Reset() { *m = Response{} } @@ -145,6 +147,13 @@ func (m *Response) GetRange() *KeyRange { return nil } +func (m *Response) GetExecDetails() *kvrpcpb.ExecDetails { + if m != nil { + return m.ExecDetails + } + return nil +} + func init() { proto.RegisterType((*KeyRange)(nil), "coprocessor.KeyRange") proto.RegisterType((*Request)(nil), "coprocessor.Request") @@ -290,9 +299,37 @@ func (m *Response) MarshalTo(dAtA []byte) (int, error) { } i += n5 } + if m.ExecDetails != nil { + dAtA[i] = 0x32 + i++ + i = encodeVarintCoprocessor(dAtA, i, uint64(m.ExecDetails.Size())) + n6, err := m.ExecDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n6 + } return i, nil } +func encodeFixed64Coprocessor(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + dAtA[offset+4] = uint8(v >> 32) + dAtA[offset+5] = uint8(v >> 40) + dAtA[offset+6] = uint8(v >> 48) + dAtA[offset+7] = uint8(v >> 56) + return offset + 8 +} +func encodeFixed32Coprocessor(dAtA []byte, offset int, v uint32) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + return offset + 4 +} func encodeVarintCoprocessor(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -360,6 +397,10 @@ func (m *Response) Size() (n int) { l = m.Range.Size() n += 1 + l + sovCoprocessor(uint64(l)) } + if m.ExecDetails != nil { + l = m.ExecDetails.Size() + n += 1 + l + sovCoprocessor(uint64(l)) + } return n } @@ -839,6 +880,39 @@ func (m *Response) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCoprocessor + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthCoprocessor + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExecDetails == nil { + m.ExecDetails = &kvrpcpb.ExecDetails{} + } + if err := m.ExecDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipCoprocessor(dAtA[iNdEx:]) @@ -968,30 +1042,32 @@ var ( func init() { proto.RegisterFile("coprocessor.proto", fileDescriptorCoprocessor) } var fileDescriptorCoprocessor = []byte{ - // 394 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x51, 0xb1, 0x6e, 0xd4, 0x40, - 0x10, 0x8d, 0xed, 0xbb, 0x4b, 0x18, 0x27, 0x51, 0xb2, 0x0a, 0x92, 0x95, 0xc2, 0x77, 0xba, 0xea, - 0x08, 0x62, 0x2d, 0x4c, 0x41, 0x7f, 0x88, 0x02, 0x85, 0x6a, 0xf9, 0x00, 0x64, 0xaf, 0x17, 0x9f, - 0x65, 0xe2, 0x59, 0x76, 0x37, 0x27, 0x52, 0xd3, 0xf0, 0x09, 0x7c, 0x52, 0x4a, 0x6a, 0x8a, 0x08, - 0x1d, 0x3f, 0x82, 0x3c, 0x6b, 0x47, 0xd7, 0xa4, 0x9a, 0x37, 0xcf, 0xcf, 0xcf, 0x6f, 0x9e, 0xe1, - 0x5c, 0xa2, 0x36, 0x28, 0x95, 0xb5, 0x68, 0xb8, 0x36, 0xe8, 0x90, 0xc5, 0x7b, 0xd4, 0xe5, 0x89, - 0x32, 0x06, 0x8d, 0x2e, 0xfd, 0xb3, 0xcb, 0x93, 0x76, 0x6b, 0xb4, 0x7c, 0x5c, 0x2f, 0x6a, 0xac, - 0x91, 0x60, 0xd6, 0x23, 0xcf, 0x2e, 0x73, 0x38, 0xba, 0x56, 0x77, 0xa2, 0xe8, 0x6a, 0xc5, 0x2e, - 0x60, 0x6a, 0x5d, 0x61, 0x5c, 0x12, 0x2c, 0x82, 0xd5, 0xb1, 0xf0, 0x0b, 0x3b, 0x83, 0x48, 0x75, - 0x55, 0x12, 0x12, 0xd7, 0xc3, 0xe5, 0xcf, 0x00, 0x0e, 0x85, 0xfa, 0x76, 0xab, 0xac, 0x63, 0x57, - 0x70, 0x28, 0xb1, 0x73, 0xea, 0xbb, 0x7f, 0x2b, 0xce, 0xcf, 0xf8, 0xf8, 0xd9, 0x77, 0x9e, 0x17, - 0xa3, 0x80, 0x9d, 0x42, 0xe8, 0x34, 0x19, 0x45, 0x22, 0x74, 0x9a, 0x31, 0x98, 0x54, 0x85, 0x2b, - 0x92, 0x88, 0xac, 0x09, 0xb3, 0x57, 0x30, 0x33, 0x7d, 0x18, 0x9b, 0x4c, 0x16, 0xd1, 0x2a, 0xce, - 0x9f, 0xf3, 0xfd, 0xa3, 0xc7, 0xa8, 0x62, 0x10, 0x2d, 0x7f, 0x84, 0x70, 0x24, 0x94, 0xd5, 0xd8, - 0x59, 0xc5, 0xae, 0x07, 0x3f, 0x8a, 0xbf, 0x7e, 0x7b, 0xff, 0x30, 0x3f, 0xf8, 0xf3, 0x30, 0xcf, - 0xea, 0xc6, 0x6d, 0x6e, 0x4b, 0x2e, 0xf1, 0x26, 0xd3, 0x4d, 0x57, 0xcb, 0x42, 0x67, 0xae, 0xd1, - 0x65, 0x66, 0x37, 0x85, 0x51, 0x55, 0x79, 0xe7, 0x94, 0xe5, 0x9f, 0x08, 0xaf, 0x7b, 0x3c, 0x04, - 0x79, 0x0d, 0xc7, 0x46, 0xd5, 0x0d, 0x76, 0x9f, 0xa9, 0x55, 0x8a, 0x1d, 0xe7, 0xa7, 0x7c, 0xec, - 0xf8, 0x7d, 0x3f, 0x45, 0xec, 0x35, 0xb4, 0xb0, 0x17, 0x30, 0xfb, 0x8a, 0xb2, 0x55, 0x15, 0x5d, - 0x14, 0xe7, 0xe7, 0x8f, 0x55, 0x7c, 0x44, 0xd9, 0x7e, 0xe8, 0xbe, 0xa0, 0x18, 0x04, 0x6c, 0x0e, - 0x31, 0xba, 0x8d, 0x32, 0x83, 0xf9, 0x64, 0x11, 0xac, 0x9e, 0x09, 0x20, 0xca, 0x7b, 0xbd, 0x84, - 0x29, 0x9d, 0x98, 0x4c, 0xc9, 0xea, 0x89, 0x1a, 0xbc, 0x66, 0x7d, 0x75, 0xbf, 0x4b, 0x83, 0xdf, - 0xbb, 0x34, 0xf8, 0xbb, 0x4b, 0x83, 0x5f, 0xff, 0xd2, 0x03, 0x48, 0x24, 0xde, 0xf0, 0xe1, 0x5a, - 0xee, 0x9a, 0x76, 0xcb, 0xdb, 0x2d, 0xfd, 0xf0, 0x72, 0x46, 0xe3, 0xcd, 0xff, 0x00, 0x00, 0x00, - 0xff, 0xff, 0xa7, 0x5f, 0x92, 0x38, 0x4d, 0x02, 0x00, 0x00, + // 424 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x52, 0x4b, 0x6e, 0xdb, 0x30, + 0x14, 0x8c, 0xfc, 0x4b, 0x4a, 0x39, 0x41, 0x42, 0xb8, 0x80, 0x90, 0x85, 0x6d, 0x78, 0xe5, 0xa6, + 0x28, 0x85, 0xaa, 0x8b, 0xec, 0xdd, 0x66, 0x51, 0xa4, 0x2b, 0xf6, 0x00, 0x81, 0x44, 0xbd, 0xca, + 0x82, 0x12, 0x3d, 0x96, 0x64, 0x0c, 0xe7, 0x06, 0x3d, 0x42, 0x0f, 0xd3, 0x03, 0x64, 0xd9, 0x75, + 0x17, 0x41, 0xe1, 0x5e, 0xa4, 0xd0, 0xa3, 0xe4, 0x66, 0xd3, 0x95, 0xe6, 0x0d, 0x87, 0xc3, 0x79, + 0x03, 0xb1, 0x33, 0x85, 0xda, 0xa0, 0x02, 0x6b, 0xd1, 0x08, 0x6d, 0xd0, 0x21, 0x0f, 0x9f, 0x51, + 0xe7, 0xc7, 0x60, 0x0c, 0x1a, 0x9d, 0xf9, 0xb3, 0xf3, 0xe3, 0x6a, 0x63, 0xb4, 0xda, 0x8f, 0x93, + 0x02, 0x0b, 0x24, 0x18, 0x37, 0xc8, 0xb3, 0x8b, 0x84, 0x1d, 0x5d, 0xc3, 0x83, 0x4c, 0xeb, 0x02, + 0xf8, 0x84, 0x0d, 0xad, 0x4b, 0x8d, 0x8b, 0x82, 0x79, 0xb0, 0x1c, 0x4b, 0x3f, 0xf0, 0x53, 0xd6, + 0x87, 0x3a, 0x8f, 0x7a, 0xc4, 0x35, 0x70, 0xf1, 0x2d, 0x60, 0x87, 0x12, 0xbe, 0xde, 0x83, 0x75, + 0xfc, 0x82, 0x1d, 0x2a, 0xac, 0x1d, 0x6c, 0xfd, 0xad, 0x30, 0x39, 0x15, 0xdd, 0xb3, 0xef, 0x3d, + 0x2f, 0x3b, 0x01, 0x3f, 0x61, 0x3d, 0xa7, 0xc9, 0xa8, 0x2f, 0x7b, 0x4e, 0x73, 0xce, 0x06, 0x79, + 0xea, 0xd2, 0xa8, 0x4f, 0xd6, 0x84, 0xf9, 0x1b, 0x36, 0x32, 0x4d, 0x18, 0x1b, 0x0d, 0xe6, 0xfd, + 0x65, 0x98, 0xbc, 0x14, 0xcf, 0x97, 0xee, 0xa2, 0xca, 0x56, 0xb4, 0xf8, 0xd1, 0x63, 0x47, 0x12, + 0xac, 0xc6, 0xda, 0x02, 0xbf, 0x6e, 0xfd, 0x28, 0xfe, 0xea, 0xf2, 0xf1, 0x69, 0x76, 0xf0, 0xeb, + 0x69, 0x16, 0x17, 0xa5, 0x5b, 0xdf, 0x67, 0x42, 0xe1, 0x5d, 0xac, 0xcb, 0xba, 0x50, 0xa9, 0x8e, + 0x5d, 0xa9, 0xb3, 0xd8, 0xae, 0x53, 0x03, 0x79, 0xf6, 0xe0, 0xc0, 0x8a, 0xcf, 0x84, 0x57, 0x0d, + 0x6e, 0x83, 0xbc, 0x65, 0x63, 0x03, 0x45, 0x89, 0xf5, 0x0d, 0xb5, 0x4a, 0xb1, 0xc3, 0xe4, 0x44, + 0x74, 0x1d, 0x5f, 0x35, 0x5f, 0x19, 0x7a, 0x0d, 0x0d, 0xfc, 0x15, 0x1b, 0xdd, 0xa2, 0xaa, 0x20, + 0xa7, 0x8d, 0xc2, 0xe4, 0x6c, 0x5f, 0xc5, 0x27, 0x54, 0xd5, 0xc7, 0xfa, 0x0b, 0xca, 0x56, 0xc0, + 0x67, 0x2c, 0x44, 0xb7, 0x06, 0xd3, 0x9a, 0x0f, 0xe6, 0xc1, 0xf2, 0x85, 0x64, 0x44, 0x79, 0xaf, + 0xd7, 0x6c, 0x48, 0x2b, 0x46, 0x43, 0xb2, 0xfa, 0x4f, 0x0d, 0x5e, 0xc3, 0x2f, 0xd9, 0x18, 0xb6, + 0xa0, 0x6e, 0x72, 0x70, 0x69, 0x79, 0x6b, 0xa3, 0x11, 0xdd, 0x99, 0xec, 0x9f, 0xbf, 0xda, 0x82, + 0xfa, 0xe0, 0xcf, 0x64, 0x08, 0xff, 0x86, 0xd5, 0xc5, 0xe3, 0x6e, 0x1a, 0xfc, 0xdc, 0x4d, 0x83, + 0xdf, 0xbb, 0x69, 0xf0, 0xfd, 0xcf, 0xf4, 0x80, 0x45, 0x0a, 0xef, 0x44, 0x5b, 0x93, 0x70, 0x65, + 0xb5, 0x11, 0xd5, 0x86, 0xfe, 0x94, 0x6c, 0x44, 0x9f, 0x77, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, + 0x12, 0x5b, 0x89, 0x3d, 0x86, 0x02, 0x00, 0x00, } diff --git a/vendor/github.com/pingcap/kvproto/pkg/eraftpb/eraftpb.pb.go b/vendor/github.com/pingcap/kvproto/pkg/eraftpb/eraftpb.pb.go index a312ad02eea06..f3876fc9e1c5b 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/eraftpb/eraftpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/eraftpb/eraftpb.pb.go @@ -1,5 +1,6 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. +// Code generated by protoc-gen-gogo. // source: eraftpb.proto +// DO NOT EDIT! /* Package eraftpb is a generated protocol buffer package. @@ -811,6 +812,24 @@ func (m *ConfChange) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func encodeFixed64Eraftpb(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + dAtA[offset+4] = uint8(v >> 32) + dAtA[offset+5] = uint8(v >> 40) + dAtA[offset+6] = uint8(v >> 48) + dAtA[offset+7] = uint8(v >> 56) + return offset + 8 +} +func encodeFixed32Eraftpb(dAtA []byte, offset int, v uint32) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + return offset + 4 +} func encodeVarintEraftpb(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -1833,24 +1852,7 @@ func (m *ConfState) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: - if wireType == 0 { - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowEraftpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.Nodes = append(m.Nodes, v) - } else if wireType == 2 { + if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -1891,11 +1893,7 @@ func (m *ConfState) Unmarshal(dAtA []byte) error { } m.Nodes = append(m.Nodes, v) } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field Nodes", wireType) - } - case 2: - if wireType == 0 { + } else if wireType == 0 { var v uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -1911,8 +1909,12 @@ func (m *ConfState) Unmarshal(dAtA []byte) error { break } } - m.Learners = append(m.Learners, v) - } else if wireType == 2 { + m.Nodes = append(m.Nodes, v) + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Nodes", wireType) + } + case 2: + if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -1953,6 +1955,23 @@ func (m *ConfState) Unmarshal(dAtA []byte) error { } m.Learners = append(m.Learners, v) } + } else if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEraftpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Learners = append(m.Learners, v) } else { return fmt.Errorf("proto: wrong wireType = %d for field Learners", wireType) } @@ -2223,58 +2242,56 @@ var ( func init() { proto.RegisterFile("eraftpb.proto", fileDescriptorEraftpb) } var fileDescriptorEraftpb = []byte{ - // 834 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x55, 0xcd, 0x6e, 0xe3, 0x36, - 0x10, 0x8e, 0xe4, 0x1f, 0xc9, 0xa3, 0xc4, 0x61, 0xd8, 0x74, 0x57, 0x59, 0xa0, 0xa9, 0xe1, 0x93, - 0x11, 0xa0, 0x2e, 0x92, 0xa2, 0x40, 0x2f, 0x3d, 0x64, 0x83, 0x02, 0x59, 0x34, 0x0e, 0xb6, 0xda, - 0x6c, 0xaf, 0x06, 0x23, 0x8d, 0x65, 0x35, 0x16, 0xa9, 0x92, 0x74, 0xba, 0x79, 0x84, 0x02, 0xbd, - 0xf5, 0xd2, 0x47, 0xea, 0xb1, 0x8f, 0x50, 0xa4, 0x87, 0xbe, 0xc6, 0x82, 0x34, 0xa5, 0xc8, 0x9b, - 0x93, 0xe7, 0x1b, 0x0e, 0x39, 0xdf, 0x7c, 0xdf, 0x08, 0x86, 0x3d, 0x94, 0x6c, 0xa1, 0xab, 0xdb, - 0x69, 0x25, 0x85, 0x16, 0x34, 0x70, 0x70, 0xfc, 0xa7, 0x07, 0xbd, 0x1f, 0xb8, 0x96, 0x0f, 0xf4, - 0x14, 0x00, 0x4d, 0x30, 0xd7, 0x0f, 0x15, 0xc6, 0xde, 0xc8, 0x9b, 0x0c, 0xcf, 0xe8, 0xb4, 0xbe, - 0x66, 0x6b, 0x6e, 0x1e, 0x2a, 0x4c, 0x06, 0x58, 0x87, 0x94, 0x42, 0x57, 0xa3, 0x2c, 0x63, 0x7f, - 0xe4, 0x4d, 0xba, 0x89, 0x8d, 0xe9, 0x21, 0xf4, 0x0a, 0x9e, 0xe1, 0x87, 0xb8, 0x63, 0x93, 0x1b, - 0x60, 0x2a, 0x33, 0xa6, 0x59, 0xdc, 0x1d, 0x79, 0x93, 0xdd, 0xc4, 0xc6, 0xf4, 0x08, 0x42, 0xf5, - 0xc0, 0xd3, 0xf9, 0x4a, 0xe4, 0x71, 0x6f, 0xe4, 0x4d, 0xc2, 0x24, 0x30, 0xf8, 0x4a, 0xe4, 0x63, - 0x01, 0xe4, 0x1d, 0x67, 0x95, 0x5a, 0x0a, 0x3d, 0x43, 0xcd, 0x6c, 0xf9, 0x29, 0x40, 0x2a, 0xf8, - 0x62, 0xae, 0x34, 0xd3, 0x1b, 0x7e, 0x51, 0x8b, 0xdf, 0x85, 0xe0, 0x8b, 0x77, 0xe6, 0x24, 0x19, - 0xa4, 0x75, 0xf8, 0xc4, 0xc5, 0xff, 0x84, 0x8b, 0x65, 0xdd, 0x79, 0x62, 0x3d, 0x7e, 0x0f, 0x61, - 0xdd, 0xb0, 0xe1, 0xea, 0xb5, 0xb8, 0x7e, 0x0b, 0x61, 0xe9, 0x88, 0xd8, 0xc7, 0xa2, 0xb3, 0xa3, - 0xa6, 0xf5, 0xa7, 0x4c, 0x93, 0xa6, 0x74, 0xfc, 0xbf, 0x0f, 0xc1, 0x0c, 0x95, 0x62, 0x39, 0xd2, - 0xaf, 0x21, 0x2c, 0x55, 0xde, 0x56, 0xf7, 0xb0, 0x79, 0xc2, 0xd5, 0x58, 0x7d, 0x83, 0x52, 0xe5, - 0x56, 0xdd, 0x21, 0xf8, 0x5a, 0x38, 0xea, 0xbe, 0x16, 0x86, 0xd7, 0x42, 0x8a, 0x86, 0xb7, 0x89, - 0x9b, 0x59, 0xba, 0x2d, 0x07, 0x8e, 0x20, 0x5c, 0x89, 0x7c, 0x6e, 0xf3, 0x3d, 0x9b, 0x0f, 0x56, - 0x22, 0xbf, 0xd9, 0x32, 0xa7, 0xdf, 0x16, 0x64, 0x02, 0x81, 0xf1, 0xb4, 0x40, 0x15, 0x07, 0xa3, - 0xce, 0x24, 0x3a, 0x1b, 0x6e, 0xdb, 0x9e, 0xd4, 0xc7, 0xf4, 0x05, 0xf4, 0x53, 0x51, 0x96, 0x85, - 0x8e, 0x43, 0xfb, 0x80, 0x43, 0xf4, 0x2b, 0x08, 0x95, 0x53, 0x21, 0x1e, 0x58, 0x79, 0x0e, 0x9e, - 0xc9, 0x93, 0x34, 0x25, 0xe6, 0x19, 0x89, 0xbf, 0x60, 0xaa, 0x63, 0xb0, 0xbe, 0x3b, 0x44, 0xbf, - 0x84, 0x68, 0x13, 0xcd, 0x97, 0x05, 0xd7, 0x71, 0x64, 0x7b, 0xc0, 0x26, 0x75, 0x59, 0x70, 0x4d, - 0x63, 0x08, 0x52, 0xc1, 0x35, 0x7e, 0xd0, 0xf1, 0xae, 0x75, 0xa7, 0x86, 0xe3, 0x1f, 0x61, 0x70, - 0xc9, 0x64, 0xb6, 0xf1, 0xbd, 0x56, 0xc5, 0x6b, 0xa9, 0x42, 0xa1, 0x7b, 0x2f, 0x34, 0xd6, 0xbb, - 0x6a, 0xe2, 0xd6, 0x38, 0x9d, 0xf6, 0x38, 0xe3, 0xef, 0x61, 0x70, 0xd1, 0x5e, 0x22, 0x2e, 0x32, - 0x54, 0xb1, 0x37, 0xea, 0x18, 0xcd, 0x2c, 0xa0, 0xaf, 0x20, 0x5c, 0x21, 0x93, 0x1c, 0xa5, 0x8a, - 0x7d, 0x7b, 0xd0, 0xe0, 0xf1, 0xef, 0x1e, 0x80, 0xb9, 0x7f, 0xb1, 0x64, 0x3c, 0xb7, 0x3e, 0x16, - 0x99, 0xe3, 0xe2, 0x17, 0x19, 0xfd, 0x0e, 0xa2, 0xd4, 0x9e, 0x6c, 0x76, 0xc1, 0xb7, 0xbb, 0xf0, - 0x72, 0x6b, 0x93, 0x37, 0x37, 0xed, 0x3a, 0x40, 0xda, 0xc4, 0xf4, 0x25, 0x04, 0xa6, 0xfb, 0xbc, - 0xc8, 0x6a, 0xc2, 0x06, 0xbe, 0xc9, 0xda, 0xba, 0x74, 0xb7, 0x74, 0x39, 0x39, 0x85, 0x41, 0xf3, - 0xe9, 0xd2, 0x7d, 0x88, 0x2c, 0xb8, 0x16, 0xb2, 0x64, 0x2b, 0xb2, 0x43, 0x3f, 0x83, 0x7d, 0x9b, - 0x78, 0xea, 0x49, 0xbc, 0x93, 0x3f, 0x3a, 0x10, 0xb5, 0x16, 0x92, 0x02, 0xf4, 0x67, 0x2a, 0xbf, - 0x5c, 0x57, 0x64, 0x87, 0x46, 0x10, 0xcc, 0x54, 0xfe, 0x1a, 0x99, 0x26, 0x1e, 0x1d, 0x02, 0xcc, - 0x54, 0xfe, 0x56, 0x8a, 0x4a, 0x28, 0x24, 0x3e, 0xdd, 0x83, 0xc1, 0x4c, 0xe5, 0xe7, 0x55, 0x85, - 0x3c, 0x23, 0x1d, 0xfa, 0x39, 0x1c, 0x34, 0x30, 0x41, 0x55, 0x09, 0xae, 0x90, 0x74, 0x29, 0x85, - 0xe1, 0x4c, 0xe5, 0x09, 0xfe, 0xba, 0x46, 0xa5, 0x7f, 0x16, 0x1a, 0x49, 0x8f, 0xbe, 0x82, 0x17, - 0xdb, 0xb9, 0xa6, 0xbe, 0x6f, 0x48, 0xcf, 0x54, 0x5e, 0x6f, 0x11, 0x09, 0x28, 0x81, 0x5d, 0xc3, - 0x07, 0x99, 0xd4, 0xb7, 0x86, 0x48, 0x48, 0x63, 0x38, 0x6c, 0x67, 0x9a, 0xcb, 0x03, 0xd7, 0xec, - 0x3d, 0x97, 0xc8, 0xd2, 0x25, 0xbb, 0x5d, 0x21, 0x01, 0x7a, 0x00, 0x7b, 0xee, 0x41, 0x63, 0xf0, - 0x5a, 0x91, 0xc8, 0x95, 0x5d, 0x2c, 0x31, 0xbd, 0xfb, 0x69, 0x2d, 0xe4, 0xba, 0x24, 0xbb, 0x8e, - 0xfe, 0x8d, 0x64, 0x5c, 0x2d, 0x50, 0x5e, 0x21, 0xcb, 0x50, 0x92, 0x3d, 0x77, 0xfb, 0xa6, 0x28, - 0x51, 0xac, 0xf5, 0xb5, 0xf8, 0x8d, 0x0c, 0x1d, 0xa1, 0x04, 0x59, 0xf6, 0xc6, 0x7c, 0x4f, 0x64, - 0x9f, 0x1e, 0x02, 0x69, 0x67, 0x0c, 0x21, 0x42, 0xdc, 0x8b, 0x6e, 0xca, 0xb7, 0x12, 0xed, 0xf0, - 0x07, 0xf4, 0x0b, 0x38, 0x7a, 0x96, 0x6e, 0x46, 0xa0, 0x27, 0xe7, 0x30, 0xdc, 0x5e, 0x09, 0x63, - 0xc2, 0x79, 0x96, 0x5d, 0x8b, 0x0c, 0xc9, 0x8e, 0x31, 0x21, 0xc1, 0x52, 0xdc, 0xa3, 0xc5, 0x9e, - 0x19, 0xe5, 0x3c, 0xcb, 0xae, 0x36, 0xbb, 0x68, 0x73, 0xfe, 0xeb, 0x93, 0xbf, 0x1f, 0x8f, 0xbd, - 0x7f, 0x1e, 0x8f, 0xbd, 0x7f, 0x1f, 0x8f, 0xbd, 0xbf, 0xfe, 0x3b, 0xde, 0x81, 0x38, 0x15, 0xe5, - 0xb4, 0x2a, 0x78, 0x9e, 0xb2, 0x6a, 0xaa, 0x8b, 0xbb, 0xfb, 0xe9, 0xdd, 0xbd, 0xfd, 0x67, 0xb8, - 0xed, 0xdb, 0x9f, 0x6f, 0x3e, 0x06, 0x00, 0x00, 0xff, 0xff, 0x9b, 0x21, 0xc6, 0xf6, 0x31, 0x06, - 0x00, 0x00, + // 815 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x55, 0xcd, 0x6e, 0xe4, 0x44, + 0x10, 0x8e, 0x3d, 0x3f, 0xf6, 0x94, 0x93, 0x49, 0xa7, 0x08, 0xbb, 0xce, 0x4a, 0x84, 0xd1, 0x9c, + 0x46, 0x91, 0x58, 0x94, 0x20, 0x24, 0x2e, 0x1c, 0xb2, 0x11, 0x52, 0x56, 0x64, 0xa2, 0xc5, 0x9b, + 0xe5, 0x3a, 0xea, 0xd8, 0x35, 0x1e, 0xc3, 0xd8, 0x6d, 0xba, 0x7b, 0x96, 0x9d, 0x47, 0x40, 0xe2, + 0xc6, 0x85, 0x47, 0xe2, 0xc8, 0x23, 0xa0, 0x70, 0xe0, 0x35, 0x50, 0xb7, 0x7f, 0xe2, 0xd9, 0xdc, + 0xea, 0xab, 0xae, 0xee, 0xfa, 0xea, 0xfb, 0xca, 0x32, 0x1c, 0x90, 0xe4, 0x4b, 0x5d, 0xde, 0xbf, + 0x2c, 0xa5, 0xd0, 0x02, 0xbd, 0x1a, 0x4e, 0xff, 0x70, 0x60, 0xf0, 0x5d, 0xa1, 0xe5, 0x16, 0xcf, + 0x01, 0xc8, 0x04, 0x0b, 0xbd, 0x2d, 0x29, 0x74, 0x26, 0xce, 0x6c, 0x7c, 0x81, 0x2f, 0x9b, 0x6b, + 0xb6, 0xe6, 0x6e, 0x5b, 0x52, 0x34, 0xa2, 0x26, 0x44, 0x84, 0xbe, 0x26, 0x99, 0x87, 0xee, 0xc4, + 0x99, 0xf5, 0x23, 0x1b, 0xe3, 0x31, 0x0c, 0xb2, 0x22, 0xa1, 0x0f, 0x61, 0xcf, 0x26, 0x2b, 0x60, + 0x2a, 0x13, 0xae, 0x79, 0xd8, 0x9f, 0x38, 0xb3, 0xfd, 0xc8, 0xc6, 0x78, 0x02, 0xbe, 0xda, 0x16, + 0xf1, 0x62, 0x2d, 0xd2, 0x70, 0x30, 0x71, 0x66, 0x7e, 0xe4, 0x19, 0x7c, 0x23, 0xd2, 0xa9, 0x00, + 0xf6, 0xb6, 0xe0, 0xa5, 0x5a, 0x09, 0x3d, 0x27, 0xcd, 0x6d, 0xf9, 0x39, 0x40, 0x2c, 0x8a, 0xe5, + 0x42, 0x69, 0xae, 0x2b, 0x7e, 0x41, 0x87, 0xdf, 0x95, 0x28, 0x96, 0x6f, 0xcd, 0x49, 0x34, 0x8a, + 0x9b, 0xf0, 0x91, 0x8b, 0xfb, 0x11, 0x17, 0xcb, 0xba, 0xf7, 0xc8, 0x7a, 0xfa, 0x0e, 0xfc, 0xa6, + 0x61, 0xcb, 0xd5, 0xe9, 0x70, 0xfd, 0x1a, 0xfc, 0xbc, 0x26, 0x62, 0x1f, 0x0b, 0x2e, 0x4e, 0xda, + 0xd6, 0x1f, 0x33, 0x8d, 0xda, 0xd2, 0xe9, 0x7f, 0x2e, 0x78, 0x73, 0x52, 0x8a, 0xa7, 0x84, 0x5f, + 0x82, 0x9f, 0xab, 0xb4, 0xab, 0xee, 0x71, 0xfb, 0x44, 0x5d, 0x63, 0xf5, 0xf5, 0x72, 0x95, 0x5a, + 0x75, 0xc7, 0xe0, 0x6a, 0x51, 0x53, 0x77, 0xb5, 0x30, 0xbc, 0x96, 0x52, 0xb4, 0xbc, 0x4d, 0xdc, + 0xce, 0xd2, 0xef, 0x38, 0x70, 0x02, 0xfe, 0x5a, 0xa4, 0x0b, 0x9b, 0x1f, 0xd8, 0xbc, 0xb7, 0x16, + 0xe9, 0xdd, 0x8e, 0x39, 0xc3, 0xae, 0x20, 0x33, 0xf0, 0x8c, 0xa7, 0x19, 0xa9, 0xd0, 0x9b, 0xf4, + 0x66, 0xc1, 0xc5, 0x78, 0xd7, 0xf6, 0xa8, 0x39, 0xc6, 0x67, 0x30, 0x8c, 0x45, 0x9e, 0x67, 0x3a, + 0xf4, 0xed, 0x03, 0x35, 0xc2, 0x2f, 0xc0, 0x57, 0xb5, 0x0a, 0xe1, 0xc8, 0xca, 0x73, 0xf4, 0x44, + 0x9e, 0xa8, 0x2d, 0x31, 0xcf, 0x48, 0xfa, 0x89, 0x62, 0x1d, 0x82, 0xf5, 0xbd, 0x46, 0xf8, 0x39, + 0x04, 0x55, 0xb4, 0x58, 0x65, 0x85, 0x0e, 0x03, 0xdb, 0x03, 0xaa, 0xd4, 0x75, 0x56, 0x68, 0x0c, + 0xc1, 0x8b, 0x45, 0xa1, 0xe9, 0x83, 0x0e, 0xf7, 0xad, 0x3b, 0x0d, 0x9c, 0x7e, 0x0f, 0xa3, 0x6b, + 0x2e, 0x93, 0xca, 0xf7, 0x46, 0x15, 0xa7, 0xa3, 0x0a, 0x42, 0xff, 0xbd, 0xd0, 0xd4, 0xec, 0xaa, + 0x89, 0x3b, 0xe3, 0xf4, 0xba, 0xe3, 0x4c, 0xbf, 0x85, 0xd1, 0x55, 0x77, 0x89, 0x0a, 0x91, 0x90, + 0x0a, 0x9d, 0x49, 0xcf, 0x68, 0x66, 0x01, 0xbe, 0x00, 0x7f, 0x4d, 0x5c, 0x16, 0x24, 0x55, 0xe8, + 0xda, 0x83, 0x16, 0x4f, 0x7f, 0x73, 0x00, 0xcc, 0xfd, 0xab, 0x15, 0x2f, 0x52, 0xeb, 0x63, 0x96, + 0xd4, 0x5c, 0xdc, 0x2c, 0xc1, 0x6f, 0x20, 0x88, 0xed, 0x49, 0xb5, 0x0b, 0xae, 0xdd, 0x85, 0xe7, + 0x3b, 0x9b, 0x5c, 0xdd, 0xb4, 0xeb, 0x00, 0x71, 0x1b, 0xe3, 0x73, 0xf0, 0x4c, 0xf7, 0x45, 0x96, + 0x34, 0x84, 0x0d, 0x7c, 0x9d, 0x74, 0x75, 0xe9, 0xef, 0xe8, 0x72, 0x76, 0x0e, 0xa3, 0xf6, 0xd3, + 0xc5, 0x43, 0x08, 0x2c, 0xb8, 0x15, 0x32, 0xe7, 0x6b, 0xb6, 0x87, 0x9f, 0xc0, 0xa1, 0x4d, 0x3c, + 0xf6, 0x64, 0xce, 0xd9, 0xef, 0x3d, 0x08, 0x3a, 0x0b, 0x89, 0x00, 0xc3, 0xb9, 0x4a, 0xaf, 0x37, + 0x25, 0xdb, 0xc3, 0x00, 0xbc, 0xb9, 0x4a, 0x5f, 0x11, 0xd7, 0xcc, 0xc1, 0x31, 0xc0, 0x5c, 0xa5, + 0x6f, 0xa4, 0x28, 0x85, 0x22, 0xe6, 0xe2, 0x01, 0x8c, 0xe6, 0x2a, 0xbd, 0x2c, 0x4b, 0x2a, 0x12, + 0xd6, 0xc3, 0x4f, 0xe1, 0xa8, 0x85, 0x11, 0xa9, 0x52, 0x14, 0x8a, 0x58, 0x1f, 0x11, 0xc6, 0x73, + 0x95, 0x46, 0xf4, 0xcb, 0x86, 0x94, 0xfe, 0x51, 0x68, 0x62, 0x03, 0x7c, 0x01, 0xcf, 0x76, 0x73, + 0x6d, 0xfd, 0xd0, 0x90, 0x9e, 0xab, 0xb4, 0xd9, 0x22, 0xe6, 0x21, 0x83, 0x7d, 0xc3, 0x87, 0xb8, + 0xd4, 0xf7, 0x86, 0x88, 0x8f, 0x21, 0x1c, 0x77, 0x33, 0xed, 0xe5, 0x51, 0xdd, 0xec, 0x5d, 0x21, + 0x89, 0xc7, 0x2b, 0x7e, 0xbf, 0x26, 0x06, 0x78, 0x04, 0x07, 0xf5, 0x83, 0xc6, 0xe0, 0x8d, 0x62, + 0x41, 0x5d, 0x76, 0xb5, 0xa2, 0xf8, 0xe7, 0x1f, 0x36, 0x42, 0x6e, 0x72, 0xb6, 0x5f, 0xd3, 0xbf, + 0x93, 0xbc, 0x50, 0x4b, 0x92, 0x37, 0xc4, 0x13, 0x92, 0xec, 0xa0, 0xbe, 0x7d, 0x97, 0xe5, 0x24, + 0x36, 0xfa, 0x56, 0xfc, 0xca, 0xc6, 0x35, 0xa1, 0x88, 0x78, 0xf2, 0xda, 0x7c, 0x4f, 0xec, 0x10, + 0x8f, 0x81, 0x75, 0x33, 0x86, 0x10, 0x63, 0xf5, 0x8b, 0xf5, 0x94, 0x6f, 0x24, 0xd9, 0xe1, 0x8f, + 0xf0, 0x33, 0x38, 0x79, 0x92, 0x6e, 0x47, 0xc0, 0xb3, 0x4b, 0x18, 0xef, 0xae, 0x84, 0x31, 0xe1, + 0x32, 0x49, 0x6e, 0x45, 0x42, 0x6c, 0xcf, 0x98, 0x10, 0x51, 0x2e, 0xde, 0x93, 0xc5, 0x8e, 0x19, + 0xe5, 0x32, 0x49, 0x6e, 0xaa, 0x5d, 0xb4, 0x39, 0xf7, 0x15, 0xfb, 0xeb, 0xe1, 0xd4, 0xf9, 0xfb, + 0xe1, 0xd4, 0xf9, 0xe7, 0xe1, 0xd4, 0xf9, 0xf3, 0xdf, 0xd3, 0xbd, 0xfb, 0xa1, 0xfd, 0x0d, 0x7c, + 0xf5, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf4, 0xcf, 0x7f, 0x2b, 0x17, 0x06, 0x00, 0x00, } diff --git a/vendor/github.com/pingcap/kvproto/pkg/errorpb/errorpb.pb.go b/vendor/github.com/pingcap/kvproto/pkg/errorpb/errorpb.pb.go index de9c89fdc1d40..4e18d72fb9f74 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/errorpb/errorpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/errorpb/errorpb.pb.go @@ -1,5 +1,6 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. +// Code generated by protoc-gen-gogo. // source: errorpb.proto +// DO NOT EDIT! /* Package errorpb is a generated protocol buffer package. @@ -616,6 +617,24 @@ func (m *Error) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func encodeFixed64Errorpb(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + dAtA[offset+4] = uint8(v >> 32) + dAtA[offset+5] = uint8(v >> 40) + dAtA[offset+6] = uint8(v >> 48) + dAtA[offset+7] = uint8(v >> 56) + return offset + 8 +} +func encodeFixed32Errorpb(dAtA []byte, offset int, v uint32) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + return offset + 4 +} func encodeVarintErrorpb(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) diff --git a/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go b/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go index 2a48b739536f7..2bb6984dde512 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go @@ -45,14 +45,26 @@ RawGetResponse RawPutRequest RawPutResponse + RawBatchPutRequest + RawBatchPutResponse + RawBatchGetRequest + RawBatchGetResponse RawDeleteRequest RawDeleteResponse + RawBatchDeleteRequest + RawBatchDeleteResponse DeleteRangeRequest DeleteRangeResponse + RawDeleteRangeRequest + RawDeleteRangeResponse RawScanRequest RawScanResponse - WriteInfo - ValueInfo + KeyRange + RawBatchScanRequest + RawBatchScanResponse + MvccWrite + MvccValue + MvccLock MvccInfo MvccGetByKeyRequest MvccGetByKeyResponse @@ -1312,6 +1324,102 @@ func (m *RawPutResponse) GetError() string { return "" } +type RawBatchPutRequest struct { + Context *Context `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` + Pairs []*KvPair `protobuf:"bytes,2,rep,name=pairs" json:"pairs,omitempty"` +} + +func (m *RawBatchPutRequest) Reset() { *m = RawBatchPutRequest{} } +func (m *RawBatchPutRequest) String() string { return proto.CompactTextString(m) } +func (*RawBatchPutRequest) ProtoMessage() {} +func (*RawBatchPutRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{36} } + +func (m *RawBatchPutRequest) GetContext() *Context { + if m != nil { + return m.Context + } + return nil +} + +func (m *RawBatchPutRequest) GetPairs() []*KvPair { + if m != nil { + return m.Pairs + } + return nil +} + +type RawBatchPutResponse struct { + RegionError *errorpb.Error `protobuf:"bytes,1,opt,name=region_error,json=regionError" json:"region_error,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` +} + +func (m *RawBatchPutResponse) Reset() { *m = RawBatchPutResponse{} } +func (m *RawBatchPutResponse) String() string { return proto.CompactTextString(m) } +func (*RawBatchPutResponse) ProtoMessage() {} +func (*RawBatchPutResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{37} } + +func (m *RawBatchPutResponse) GetRegionError() *errorpb.Error { + if m != nil { + return m.RegionError + } + return nil +} + +func (m *RawBatchPutResponse) GetError() string { + if m != nil { + return m.Error + } + return "" +} + +type RawBatchGetRequest struct { + Context *Context `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` + Keys [][]byte `protobuf:"bytes,2,rep,name=keys" json:"keys,omitempty"` +} + +func (m *RawBatchGetRequest) Reset() { *m = RawBatchGetRequest{} } +func (m *RawBatchGetRequest) String() string { return proto.CompactTextString(m) } +func (*RawBatchGetRequest) ProtoMessage() {} +func (*RawBatchGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{38} } + +func (m *RawBatchGetRequest) GetContext() *Context { + if m != nil { + return m.Context + } + return nil +} + +func (m *RawBatchGetRequest) GetKeys() [][]byte { + if m != nil { + return m.Keys + } + return nil +} + +type RawBatchGetResponse struct { + RegionError *errorpb.Error `protobuf:"bytes,1,opt,name=region_error,json=regionError" json:"region_error,omitempty"` + Pairs []*KvPair `protobuf:"bytes,2,rep,name=pairs" json:"pairs,omitempty"` +} + +func (m *RawBatchGetResponse) Reset() { *m = RawBatchGetResponse{} } +func (m *RawBatchGetResponse) String() string { return proto.CompactTextString(m) } +func (*RawBatchGetResponse) ProtoMessage() {} +func (*RawBatchGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{39} } + +func (m *RawBatchGetResponse) GetRegionError() *errorpb.Error { + if m != nil { + return m.RegionError + } + return nil +} + +func (m *RawBatchGetResponse) GetPairs() []*KvPair { + if m != nil { + return m.Pairs + } + return nil +} + type RawDeleteRequest struct { Context *Context `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` @@ -1320,7 +1428,7 @@ type RawDeleteRequest struct { func (m *RawDeleteRequest) Reset() { *m = RawDeleteRequest{} } func (m *RawDeleteRequest) String() string { return proto.CompactTextString(m) } func (*RawDeleteRequest) ProtoMessage() {} -func (*RawDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{36} } +func (*RawDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{40} } func (m *RawDeleteRequest) GetContext() *Context { if m != nil { @@ -1344,7 +1452,7 @@ type RawDeleteResponse struct { func (m *RawDeleteResponse) Reset() { *m = RawDeleteResponse{} } func (m *RawDeleteResponse) String() string { return proto.CompactTextString(m) } func (*RawDeleteResponse) ProtoMessage() {} -func (*RawDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{37} } +func (*RawDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{41} } func (m *RawDeleteResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1360,6 +1468,54 @@ func (m *RawDeleteResponse) GetError() string { return "" } +type RawBatchDeleteRequest struct { + Context *Context `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` + Keys [][]byte `protobuf:"bytes,2,rep,name=keys" json:"keys,omitempty"` +} + +func (m *RawBatchDeleteRequest) Reset() { *m = RawBatchDeleteRequest{} } +func (m *RawBatchDeleteRequest) String() string { return proto.CompactTextString(m) } +func (*RawBatchDeleteRequest) ProtoMessage() {} +func (*RawBatchDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{42} } + +func (m *RawBatchDeleteRequest) GetContext() *Context { + if m != nil { + return m.Context + } + return nil +} + +func (m *RawBatchDeleteRequest) GetKeys() [][]byte { + if m != nil { + return m.Keys + } + return nil +} + +type RawBatchDeleteResponse struct { + RegionError *errorpb.Error `protobuf:"bytes,1,opt,name=region_error,json=regionError" json:"region_error,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` +} + +func (m *RawBatchDeleteResponse) Reset() { *m = RawBatchDeleteResponse{} } +func (m *RawBatchDeleteResponse) String() string { return proto.CompactTextString(m) } +func (*RawBatchDeleteResponse) ProtoMessage() {} +func (*RawBatchDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{43} } + +func (m *RawBatchDeleteResponse) GetRegionError() *errorpb.Error { + if m != nil { + return m.RegionError + } + return nil +} + +func (m *RawBatchDeleteResponse) GetError() string { + if m != nil { + return m.Error + } + return "" +} + type DeleteRangeRequest struct { Context *Context `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` StartKey []byte `protobuf:"bytes,2,opt,name=start_key,json=startKey,proto3" json:"start_key,omitempty"` @@ -1369,7 +1525,7 @@ type DeleteRangeRequest struct { func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} -func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{38} } +func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{44} } func (m *DeleteRangeRequest) GetContext() *Context { if m != nil { @@ -1400,7 +1556,7 @@ type DeleteRangeResponse struct { func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} -func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{39} } +func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{45} } func (m *DeleteRangeResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1416,16 +1572,73 @@ func (m *DeleteRangeResponse) GetError() string { return "" } +type RawDeleteRangeRequest struct { + Context *Context `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` + StartKey []byte `protobuf:"bytes,2,opt,name=start_key,json=startKey,proto3" json:"start_key,omitempty"` + EndKey []byte `protobuf:"bytes,3,opt,name=end_key,json=endKey,proto3" json:"end_key,omitempty"` +} + +func (m *RawDeleteRangeRequest) Reset() { *m = RawDeleteRangeRequest{} } +func (m *RawDeleteRangeRequest) String() string { return proto.CompactTextString(m) } +func (*RawDeleteRangeRequest) ProtoMessage() {} +func (*RawDeleteRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{46} } + +func (m *RawDeleteRangeRequest) GetContext() *Context { + if m != nil { + return m.Context + } + return nil +} + +func (m *RawDeleteRangeRequest) GetStartKey() []byte { + if m != nil { + return m.StartKey + } + return nil +} + +func (m *RawDeleteRangeRequest) GetEndKey() []byte { + if m != nil { + return m.EndKey + } + return nil +} + +type RawDeleteRangeResponse struct { + RegionError *errorpb.Error `protobuf:"bytes,1,opt,name=region_error,json=regionError" json:"region_error,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` +} + +func (m *RawDeleteRangeResponse) Reset() { *m = RawDeleteRangeResponse{} } +func (m *RawDeleteRangeResponse) String() string { return proto.CompactTextString(m) } +func (*RawDeleteRangeResponse) ProtoMessage() {} +func (*RawDeleteRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{47} } + +func (m *RawDeleteRangeResponse) GetRegionError() *errorpb.Error { + if m != nil { + return m.RegionError + } + return nil +} + +func (m *RawDeleteRangeResponse) GetError() string { + if m != nil { + return m.Error + } + return "" +} + type RawScanRequest struct { Context *Context `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` StartKey []byte `protobuf:"bytes,2,opt,name=start_key,json=startKey,proto3" json:"start_key,omitempty"` Limit uint32 `protobuf:"varint,3,opt,name=limit,proto3" json:"limit,omitempty"` + KeyOnly bool `protobuf:"varint,4,opt,name=key_only,json=keyOnly,proto3" json:"key_only,omitempty"` } func (m *RawScanRequest) Reset() { *m = RawScanRequest{} } func (m *RawScanRequest) String() string { return proto.CompactTextString(m) } func (*RawScanRequest) ProtoMessage() {} -func (*RawScanRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{40} } +func (*RawScanRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{48} } func (m *RawScanRequest) GetContext() *Context { if m != nil { @@ -1448,6 +1661,13 @@ func (m *RawScanRequest) GetLimit() uint32 { return 0 } +func (m *RawScanRequest) GetKeyOnly() bool { + if m != nil { + return m.KeyOnly + } + return false +} + type RawScanResponse struct { RegionError *errorpb.Error `protobuf:"bytes,1,opt,name=region_error,json=regionError" json:"region_error,omitempty"` Kvs []*KvPair `protobuf:"bytes,2,rep,name=kvs" json:"kvs,omitempty"` @@ -1456,7 +1676,7 @@ type RawScanResponse struct { func (m *RawScanResponse) Reset() { *m = RawScanResponse{} } func (m *RawScanResponse) String() string { return proto.CompactTextString(m) } func (*RawScanResponse) ProtoMessage() {} -func (*RawScanResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{41} } +func (*RawScanResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{49} } func (m *RawScanResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1472,96 +1692,224 @@ func (m *RawScanResponse) GetKvs() []*KvPair { return nil } -type WriteInfo struct { - StartTs uint64 `protobuf:"varint,1,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` - Type Op `protobuf:"varint,2,opt,name=type,proto3,enum=kvrpcpb.Op" json:"type,omitempty"` - CommitTs uint64 `protobuf:"varint,3,opt,name=commit_ts,json=commitTs,proto3" json:"commit_ts,omitempty"` +type KeyRange struct { + StartKey []byte `protobuf:"bytes,1,opt,name=start_key,json=startKey,proto3" json:"start_key,omitempty"` + EndKey []byte `protobuf:"bytes,2,opt,name=end_key,json=endKey,proto3" json:"end_key,omitempty"` } -func (m *WriteInfo) Reset() { *m = WriteInfo{} } -func (m *WriteInfo) String() string { return proto.CompactTextString(m) } -func (*WriteInfo) ProtoMessage() {} -func (*WriteInfo) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{42} } +func (m *KeyRange) Reset() { *m = KeyRange{} } +func (m *KeyRange) String() string { return proto.CompactTextString(m) } +func (*KeyRange) ProtoMessage() {} +func (*KeyRange) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{50} } -func (m *WriteInfo) GetStartTs() uint64 { +func (m *KeyRange) GetStartKey() []byte { if m != nil { - return m.StartTs + return m.StartKey + } + return nil +} + +func (m *KeyRange) GetEndKey() []byte { + if m != nil { + return m.EndKey + } + return nil +} + +type RawBatchScanRequest struct { + Context *Context `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` + Ranges []*KeyRange `protobuf:"bytes,2,rep,name=ranges" json:"ranges,omitempty"` + EachLimit uint32 `protobuf:"varint,3,opt,name=each_limit,json=eachLimit,proto3" json:"each_limit,omitempty"` + KeyOnly bool `protobuf:"varint,4,opt,name=key_only,json=keyOnly,proto3" json:"key_only,omitempty"` +} + +func (m *RawBatchScanRequest) Reset() { *m = RawBatchScanRequest{} } +func (m *RawBatchScanRequest) String() string { return proto.CompactTextString(m) } +func (*RawBatchScanRequest) ProtoMessage() {} +func (*RawBatchScanRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{51} } + +func (m *RawBatchScanRequest) GetContext() *Context { + if m != nil { + return m.Context + } + return nil +} + +func (m *RawBatchScanRequest) GetRanges() []*KeyRange { + if m != nil { + return m.Ranges + } + return nil +} + +func (m *RawBatchScanRequest) GetEachLimit() uint32 { + if m != nil { + return m.EachLimit } return 0 } -func (m *WriteInfo) GetType() Op { +func (m *RawBatchScanRequest) GetKeyOnly() bool { + if m != nil { + return m.KeyOnly + } + return false +} + +type RawBatchScanResponse struct { + RegionError *errorpb.Error `protobuf:"bytes,1,opt,name=region_error,json=regionError" json:"region_error,omitempty"` + Kvs []*KvPair `protobuf:"bytes,2,rep,name=kvs" json:"kvs,omitempty"` +} + +func (m *RawBatchScanResponse) Reset() { *m = RawBatchScanResponse{} } +func (m *RawBatchScanResponse) String() string { return proto.CompactTextString(m) } +func (*RawBatchScanResponse) ProtoMessage() {} +func (*RawBatchScanResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{52} } + +func (m *RawBatchScanResponse) GetRegionError() *errorpb.Error { + if m != nil { + return m.RegionError + } + return nil +} + +func (m *RawBatchScanResponse) GetKvs() []*KvPair { + if m != nil { + return m.Kvs + } + return nil +} + +type MvccWrite struct { + Type Op `protobuf:"varint,1,opt,name=type,proto3,enum=kvrpcpb.Op" json:"type,omitempty"` + StartTs uint64 `protobuf:"varint,2,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` + CommitTs uint64 `protobuf:"varint,3,opt,name=commit_ts,json=commitTs,proto3" json:"commit_ts,omitempty"` + ShortValue []byte `protobuf:"bytes,4,opt,name=short_value,json=shortValue,proto3" json:"short_value,omitempty"` +} + +func (m *MvccWrite) Reset() { *m = MvccWrite{} } +func (m *MvccWrite) String() string { return proto.CompactTextString(m) } +func (*MvccWrite) ProtoMessage() {} +func (*MvccWrite) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{53} } + +func (m *MvccWrite) GetType() Op { if m != nil { return m.Type } return Op_Put } -func (m *WriteInfo) GetCommitTs() uint64 { +func (m *MvccWrite) GetStartTs() uint64 { + if m != nil { + return m.StartTs + } + return 0 +} + +func (m *MvccWrite) GetCommitTs() uint64 { if m != nil { return m.CommitTs } return 0 } -type ValueInfo struct { - Value []byte `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` - Ts uint64 `protobuf:"varint,2,opt,name=ts,proto3" json:"ts,omitempty"` - IsShortValue bool `protobuf:"varint,3,opt,name=is_short_value,json=isShortValue,proto3" json:"is_short_value,omitempty"` +func (m *MvccWrite) GetShortValue() []byte { + if m != nil { + return m.ShortValue + } + return nil +} + +type MvccValue struct { + StartTs uint64 `protobuf:"varint,1,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` + Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` } -func (m *ValueInfo) Reset() { *m = ValueInfo{} } -func (m *ValueInfo) String() string { return proto.CompactTextString(m) } -func (*ValueInfo) ProtoMessage() {} -func (*ValueInfo) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{43} } +func (m *MvccValue) Reset() { *m = MvccValue{} } +func (m *MvccValue) String() string { return proto.CompactTextString(m) } +func (*MvccValue) ProtoMessage() {} +func (*MvccValue) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{54} } + +func (m *MvccValue) GetStartTs() uint64 { + if m != nil { + return m.StartTs + } + return 0 +} -func (m *ValueInfo) GetValue() []byte { +func (m *MvccValue) GetValue() []byte { if m != nil { return m.Value } return nil } -func (m *ValueInfo) GetTs() uint64 { +type MvccLock struct { + Type Op `protobuf:"varint,1,opt,name=type,proto3,enum=kvrpcpb.Op" json:"type,omitempty"` + StartTs uint64 `protobuf:"varint,2,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` + Primary []byte `protobuf:"bytes,3,opt,name=primary,proto3" json:"primary,omitempty"` + ShortValue []byte `protobuf:"bytes,4,opt,name=short_value,json=shortValue,proto3" json:"short_value,omitempty"` +} + +func (m *MvccLock) Reset() { *m = MvccLock{} } +func (m *MvccLock) String() string { return proto.CompactTextString(m) } +func (*MvccLock) ProtoMessage() {} +func (*MvccLock) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{55} } + +func (m *MvccLock) GetType() Op { + if m != nil { + return m.Type + } + return Op_Put +} + +func (m *MvccLock) GetStartTs() uint64 { if m != nil { - return m.Ts + return m.StartTs } return 0 } -func (m *ValueInfo) GetIsShortValue() bool { +func (m *MvccLock) GetPrimary() []byte { if m != nil { - return m.IsShortValue + return m.Primary } - return false + return nil +} + +func (m *MvccLock) GetShortValue() []byte { + if m != nil { + return m.ShortValue + } + return nil } type MvccInfo struct { - Lock *LockInfo `protobuf:"bytes,1,opt,name=lock" json:"lock,omitempty"` - Writes []*WriteInfo `protobuf:"bytes,2,rep,name=writes" json:"writes,omitempty"` - Values []*ValueInfo `protobuf:"bytes,3,rep,name=values" json:"values,omitempty"` + Lock *MvccLock `protobuf:"bytes,1,opt,name=lock" json:"lock,omitempty"` + Writes []*MvccWrite `protobuf:"bytes,2,rep,name=writes" json:"writes,omitempty"` + Values []*MvccValue `protobuf:"bytes,3,rep,name=values" json:"values,omitempty"` } func (m *MvccInfo) Reset() { *m = MvccInfo{} } func (m *MvccInfo) String() string { return proto.CompactTextString(m) } func (*MvccInfo) ProtoMessage() {} -func (*MvccInfo) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{44} } +func (*MvccInfo) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{56} } -func (m *MvccInfo) GetLock() *LockInfo { +func (m *MvccInfo) GetLock() *MvccLock { if m != nil { return m.Lock } return nil } -func (m *MvccInfo) GetWrites() []*WriteInfo { +func (m *MvccInfo) GetWrites() []*MvccWrite { if m != nil { return m.Writes } return nil } -func (m *MvccInfo) GetValues() []*ValueInfo { +func (m *MvccInfo) GetValues() []*MvccValue { if m != nil { return m.Values } @@ -1576,7 +1924,7 @@ type MvccGetByKeyRequest struct { func (m *MvccGetByKeyRequest) Reset() { *m = MvccGetByKeyRequest{} } func (m *MvccGetByKeyRequest) String() string { return proto.CompactTextString(m) } func (*MvccGetByKeyRequest) ProtoMessage() {} -func (*MvccGetByKeyRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{45} } +func (*MvccGetByKeyRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{57} } func (m *MvccGetByKeyRequest) GetContext() *Context { if m != nil { @@ -1601,7 +1949,7 @@ type MvccGetByKeyResponse struct { func (m *MvccGetByKeyResponse) Reset() { *m = MvccGetByKeyResponse{} } func (m *MvccGetByKeyResponse) String() string { return proto.CompactTextString(m) } func (*MvccGetByKeyResponse) ProtoMessage() {} -func (*MvccGetByKeyResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{46} } +func (*MvccGetByKeyResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{58} } func (m *MvccGetByKeyResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1632,7 +1980,7 @@ type MvccGetByStartTsRequest struct { func (m *MvccGetByStartTsRequest) Reset() { *m = MvccGetByStartTsRequest{} } func (m *MvccGetByStartTsRequest) String() string { return proto.CompactTextString(m) } func (*MvccGetByStartTsRequest) ProtoMessage() {} -func (*MvccGetByStartTsRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{47} } +func (*MvccGetByStartTsRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{59} } func (m *MvccGetByStartTsRequest) GetContext() *Context { if m != nil { @@ -1658,7 +2006,7 @@ type MvccGetByStartTsResponse struct { func (m *MvccGetByStartTsResponse) Reset() { *m = MvccGetByStartTsResponse{} } func (m *MvccGetByStartTsResponse) String() string { return proto.CompactTextString(m) } func (*MvccGetByStartTsResponse) ProtoMessage() {} -func (*MvccGetByStartTsResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{48} } +func (*MvccGetByStartTsResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{60} } func (m *MvccGetByStartTsResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1696,7 +2044,7 @@ type SplitRegionRequest struct { func (m *SplitRegionRequest) Reset() { *m = SplitRegionRequest{} } func (m *SplitRegionRequest) String() string { return proto.CompactTextString(m) } func (*SplitRegionRequest) ProtoMessage() {} -func (*SplitRegionRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{49} } +func (*SplitRegionRequest) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{61} } func (m *SplitRegionRequest) GetContext() *Context { if m != nil { @@ -1721,7 +2069,7 @@ type SplitRegionResponse struct { func (m *SplitRegionResponse) Reset() { *m = SplitRegionResponse{} } func (m *SplitRegionResponse) String() string { return proto.CompactTextString(m) } func (*SplitRegionResponse) ProtoMessage() {} -func (*SplitRegionResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{50} } +func (*SplitRegionResponse) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{62} } func (m *SplitRegionResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1781,14 +2129,26 @@ func init() { proto.RegisterType((*RawGetResponse)(nil), "kvrpcpb.RawGetResponse") proto.RegisterType((*RawPutRequest)(nil), "kvrpcpb.RawPutRequest") proto.RegisterType((*RawPutResponse)(nil), "kvrpcpb.RawPutResponse") + proto.RegisterType((*RawBatchPutRequest)(nil), "kvrpcpb.RawBatchPutRequest") + proto.RegisterType((*RawBatchPutResponse)(nil), "kvrpcpb.RawBatchPutResponse") + proto.RegisterType((*RawBatchGetRequest)(nil), "kvrpcpb.RawBatchGetRequest") + proto.RegisterType((*RawBatchGetResponse)(nil), "kvrpcpb.RawBatchGetResponse") proto.RegisterType((*RawDeleteRequest)(nil), "kvrpcpb.RawDeleteRequest") proto.RegisterType((*RawDeleteResponse)(nil), "kvrpcpb.RawDeleteResponse") + proto.RegisterType((*RawBatchDeleteRequest)(nil), "kvrpcpb.RawBatchDeleteRequest") + proto.RegisterType((*RawBatchDeleteResponse)(nil), "kvrpcpb.RawBatchDeleteResponse") proto.RegisterType((*DeleteRangeRequest)(nil), "kvrpcpb.DeleteRangeRequest") proto.RegisterType((*DeleteRangeResponse)(nil), "kvrpcpb.DeleteRangeResponse") + proto.RegisterType((*RawDeleteRangeRequest)(nil), "kvrpcpb.RawDeleteRangeRequest") + proto.RegisterType((*RawDeleteRangeResponse)(nil), "kvrpcpb.RawDeleteRangeResponse") proto.RegisterType((*RawScanRequest)(nil), "kvrpcpb.RawScanRequest") proto.RegisterType((*RawScanResponse)(nil), "kvrpcpb.RawScanResponse") - proto.RegisterType((*WriteInfo)(nil), "kvrpcpb.WriteInfo") - proto.RegisterType((*ValueInfo)(nil), "kvrpcpb.ValueInfo") + proto.RegisterType((*KeyRange)(nil), "kvrpcpb.KeyRange") + proto.RegisterType((*RawBatchScanRequest)(nil), "kvrpcpb.RawBatchScanRequest") + proto.RegisterType((*RawBatchScanResponse)(nil), "kvrpcpb.RawBatchScanResponse") + proto.RegisterType((*MvccWrite)(nil), "kvrpcpb.MvccWrite") + proto.RegisterType((*MvccValue)(nil), "kvrpcpb.MvccValue") + proto.RegisterType((*MvccLock)(nil), "kvrpcpb.MvccLock") proto.RegisterType((*MvccInfo)(nil), "kvrpcpb.MvccInfo") proto.RegisterType((*MvccGetByKeyRequest)(nil), "kvrpcpb.MvccGetByKeyRequest") proto.RegisterType((*MvccGetByKeyResponse)(nil), "kvrpcpb.MvccGetByKeyResponse") @@ -3302,7 +3662,7 @@ func (m *RawPutResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *RawDeleteRequest) Marshal() (dAtA []byte, err error) { +func (m *RawBatchPutRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3312,7 +3672,7 @@ func (m *RawDeleteRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RawDeleteRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *RawBatchPutRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3327,16 +3687,22 @@ func (m *RawDeleteRequest) MarshalTo(dAtA []byte) (int, error) { } i += n42 } - if len(m.Key) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Key))) - i += copy(dAtA[i:], m.Key) + if len(m.Pairs) > 0 { + for _, msg := range m.Pairs { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } } return i, nil } -func (m *RawDeleteResponse) Marshal() (dAtA []byte, err error) { +func (m *RawBatchPutResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3346,7 +3712,7 @@ func (m *RawDeleteResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RawDeleteResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *RawBatchPutResponse) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3370,7 +3736,7 @@ func (m *RawDeleteResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *DeleteRangeRequest) Marshal() (dAtA []byte, err error) { +func (m *RawBatchGetRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3380,7 +3746,7 @@ func (m *DeleteRangeRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *DeleteRangeRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *RawBatchGetRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3395,22 +3761,18 @@ func (m *DeleteRangeRequest) MarshalTo(dAtA []byte) (int, error) { } i += n44 } - if len(m.StartKey) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.StartKey))) - i += copy(dAtA[i:], m.StartKey) - } - if len(m.EndKey) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.EndKey))) - i += copy(dAtA[i:], m.EndKey) + if len(m.Keys) > 0 { + for _, b := range m.Keys { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(b))) + i += copy(dAtA[i:], b) + } } return i, nil } -func (m *DeleteRangeResponse) Marshal() (dAtA []byte, err error) { +func (m *RawBatchGetResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3420,7 +3782,7 @@ func (m *DeleteRangeResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *DeleteRangeResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *RawBatchGetResponse) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3435,16 +3797,22 @@ func (m *DeleteRangeResponse) MarshalTo(dAtA []byte) (int, error) { } i += n45 } - if len(m.Error) > 0 { - dAtA[i] = 0x12 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Error))) - i += copy(dAtA[i:], m.Error) + if len(m.Pairs) > 0 { + for _, msg := range m.Pairs { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } } return i, nil } -func (m *RawScanRequest) Marshal() (dAtA []byte, err error) { +func (m *RawDeleteRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3454,7 +3822,7 @@ func (m *RawScanRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RawScanRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *RawDeleteRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3469,21 +3837,16 @@ func (m *RawScanRequest) MarshalTo(dAtA []byte) (int, error) { } i += n46 } - if len(m.StartKey) > 0 { + if len(m.Key) > 0 { dAtA[i] = 0x12 i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.StartKey))) - i += copy(dAtA[i:], m.StartKey) - } - if m.Limit != 0 { - dAtA[i] = 0x18 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Limit)) + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Key))) + i += copy(dAtA[i:], m.Key) } return i, nil } -func (m *RawScanResponse) Marshal() (dAtA []byte, err error) { +func (m *RawDeleteResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3493,7 +3856,7 @@ func (m *RawScanResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RawScanResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *RawDeleteResponse) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3508,55 +3871,16 @@ func (m *RawScanResponse) MarshalTo(dAtA []byte) (int, error) { } i += n47 } - if len(m.Kvs) > 0 { - for _, msg := range m.Kvs { - dAtA[i] = 0x12 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } - } - return i, nil -} - -func (m *WriteInfo) 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 *WriteInfo) MarshalTo(dAtA []byte) (int, error) { - var i int - _ = i - var l int - _ = l - if m.StartTs != 0 { - dAtA[i] = 0x8 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.StartTs)) - } - if m.Type != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Type)) - } - if m.CommitTs != 0 { - dAtA[i] = 0x18 + if len(m.Error) > 0 { + dAtA[i] = 0x12 i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.CommitTs)) + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Error))) + i += copy(dAtA[i:], m.Error) } return i, nil } -func (m *ValueInfo) Marshal() (dAtA []byte, err error) { +func (m *RawBatchDeleteRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3566,36 +3890,33 @@ func (m *ValueInfo) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ValueInfo) MarshalTo(dAtA []byte) (int, error) { +func (m *RawBatchDeleteRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int _ = l - if len(m.Value) > 0 { + if m.Context != nil { dAtA[i] = 0xa i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Value))) - i += copy(dAtA[i:], m.Value) - } - if m.Ts != 0 { - dAtA[i] = 0x10 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Ts)) + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) + n48, err := m.Context.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n48 } - if m.IsShortValue { - dAtA[i] = 0x18 - i++ - if m.IsShortValue { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + if len(m.Keys) > 0 { + for _, b := range m.Keys { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(b))) + i += copy(dAtA[i:], b) } - i++ } return i, nil } -func (m *MvccInfo) Marshal() (dAtA []byte, err error) { +func (m *RawBatchDeleteResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3605,49 +3926,31 @@ func (m *MvccInfo) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *MvccInfo) MarshalTo(dAtA []byte) (int, error) { +func (m *RawBatchDeleteResponse) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int _ = l - if m.Lock != nil { + if m.RegionError != nil { dAtA[i] = 0xa i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Lock.Size())) - n48, err := m.Lock.MarshalTo(dAtA[i:]) + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) + n49, err := m.RegionError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 - } - if len(m.Writes) > 0 { - for _, msg := range m.Writes { - dAtA[i] = 0x12 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } + i += n49 } - if len(m.Values) > 0 { - for _, msg := range m.Values { - dAtA[i] = 0x1a - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) - n, err := msg.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n - } + if len(m.Error) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Error))) + i += copy(dAtA[i:], m.Error) } return i, nil } -func (m *MvccGetByKeyRequest) Marshal() (dAtA []byte, err error) { +func (m *DeleteRangeRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3657,7 +3960,7 @@ func (m *MvccGetByKeyRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *MvccGetByKeyRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *DeleteRangeRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3666,22 +3969,28 @@ func (m *MvccGetByKeyRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) - n49, err := m.Context.MarshalTo(dAtA[i:]) + n50, err := m.Context.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n50 } - if len(m.Key) > 0 { + if len(m.StartKey) > 0 { dAtA[i] = 0x12 i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Key))) - i += copy(dAtA[i:], m.Key) + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.StartKey))) + i += copy(dAtA[i:], m.StartKey) + } + if len(m.EndKey) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.EndKey))) + i += copy(dAtA[i:], m.EndKey) } return i, nil } -func (m *MvccGetByKeyResponse) Marshal() (dAtA []byte, err error) { +func (m *DeleteRangeResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3691,7 +4000,7 @@ func (m *MvccGetByKeyResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *MvccGetByKeyResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *DeleteRangeResponse) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3700,11 +4009,11 @@ func (m *MvccGetByKeyResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) - n50, err := m.RegionError.MarshalTo(dAtA[i:]) + n51, err := m.RegionError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n51 } if len(m.Error) > 0 { dAtA[i] = 0x12 @@ -3712,20 +4021,10 @@ func (m *MvccGetByKeyResponse) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Error))) i += copy(dAtA[i:], m.Error) } - if m.Info != nil { - dAtA[i] = 0x1a - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Info.Size())) - n51, err := m.Info.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n51 - } return i, nil } -func (m *MvccGetByStartTsRequest) Marshal() (dAtA []byte, err error) { +func (m *RawDeleteRangeRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3735,7 +4034,7 @@ func (m *MvccGetByStartTsRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *MvccGetByStartTsRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *RawDeleteRangeRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3750,15 +4049,22 @@ func (m *MvccGetByStartTsRequest) MarshalTo(dAtA []byte) (int, error) { } i += n52 } - if m.StartTs != 0 { - dAtA[i] = 0x10 + if len(m.StartKey) > 0 { + dAtA[i] = 0x12 i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.StartTs)) + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.StartKey))) + i += copy(dAtA[i:], m.StartKey) + } + if len(m.EndKey) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.EndKey))) + i += copy(dAtA[i:], m.EndKey) } return i, nil } -func (m *MvccGetByStartTsResponse) Marshal() (dAtA []byte, err error) { +func (m *RawDeleteRangeResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3768,7 +4074,7 @@ func (m *MvccGetByStartTsResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *MvccGetByStartTsResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *RawDeleteRangeResponse) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3789,26 +4095,10 @@ func (m *MvccGetByStartTsResponse) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Error))) i += copy(dAtA[i:], m.Error) } - if len(m.Key) > 0 { - dAtA[i] = 0x1a - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Key))) - i += copy(dAtA[i:], m.Key) - } - if m.Info != nil { - dAtA[i] = 0x22 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Info.Size())) - n54, err := m.Info.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n54 - } return i, nil } -func (m *SplitRegionRequest) Marshal() (dAtA []byte, err error) { +func (m *RawScanRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3818,7 +4108,7 @@ func (m *SplitRegionRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *SplitRegionRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *RawScanRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3827,22 +4117,37 @@ func (m *SplitRegionRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) - n55, err := m.Context.MarshalTo(dAtA[i:]) + n54, err := m.Context.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n54 } - if len(m.SplitKey) > 0 { + if len(m.StartKey) > 0 { dAtA[i] = 0x12 i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.SplitKey))) - i += copy(dAtA[i:], m.SplitKey) + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.StartKey))) + i += copy(dAtA[i:], m.StartKey) + } + if m.Limit != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Limit)) + } + if m.KeyOnly { + dAtA[i] = 0x20 + i++ + if m.KeyOnly { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ } return i, nil } -func (m *SplitRegionResponse) Marshal() (dAtA []byte, err error) { +func (m *RawScanResponse) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -3852,7 +4157,7 @@ func (m *SplitRegionResponse) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *SplitRegionResponse) MarshalTo(dAtA []byte) (int, error) { +func (m *RawScanResponse) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3861,590 +4166,732 @@ func (m *SplitRegionResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) - n56, err := m.RegionError.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err - } - i += n56 - } - if m.Left != nil { - dAtA[i] = 0x12 - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Left.Size())) - n57, err := m.Left.MarshalTo(dAtA[i:]) + n55, err := m.RegionError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n55 } - if m.Right != nil { - dAtA[i] = 0x1a - i++ - i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Right.Size())) - n58, err := m.Right.MarshalTo(dAtA[i:]) - if err != nil { - return 0, err + if len(m.Kvs) > 0 { + for _, msg := range m.Kvs { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n } - i += n58 } return i, nil } -func encodeFixed64Kvrpcpb(dAtA []byte, offset int, v uint64) int { - dAtA[offset] = uint8(v) - dAtA[offset+1] = uint8(v >> 8) - dAtA[offset+2] = uint8(v >> 16) - dAtA[offset+3] = uint8(v >> 24) - dAtA[offset+4] = uint8(v >> 32) - dAtA[offset+5] = uint8(v >> 40) - dAtA[offset+6] = uint8(v >> 48) - dAtA[offset+7] = uint8(v >> 56) - return offset + 8 -} -func encodeFixed32Kvrpcpb(dAtA []byte, offset int, v uint32) int { - dAtA[offset] = uint8(v) - dAtA[offset+1] = uint8(v >> 8) - dAtA[offset+2] = uint8(v >> 16) - dAtA[offset+3] = uint8(v >> 24) - return offset + 4 -} -func encodeVarintKvrpcpb(dAtA []byte, offset int, v uint64) int { - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ +func (m *KeyRange) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - dAtA[offset] = uint8(v) - return offset + 1 + return dAtA[:n], nil } -func (m *LockInfo) Size() (n int) { + +func (m *KeyRange) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - l = len(m.PrimaryLock) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) - } - if m.LockVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.LockVersion)) - } - l = len(m.Key) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if len(m.StartKey) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.StartKey))) + i += copy(dAtA[i:], m.StartKey) } - if m.LockTtl != 0 { - n += 1 + sovKvrpcpb(uint64(m.LockTtl)) + if len(m.EndKey) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.EndKey))) + i += copy(dAtA[i:], m.EndKey) } - return n + return i, nil } -func (m *KeyError) Size() (n int) { - var l int - _ = l - if m.Locked != nil { - l = m.Locked.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.Retryable) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.Abort) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) +func (m *RawBatchScanRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *Context) Size() (n int) { +func (m *RawBatchScanRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.RegionId != 0 { - n += 1 + sovKvrpcpb(uint64(m.RegionId)) - } - if m.RegionEpoch != nil { - l = m.RegionEpoch.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - if m.Peer != nil { - l = m.Peer.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - if m.Term != 0 { - n += 1 + sovKvrpcpb(uint64(m.Term)) - } - if m.Priority != 0 { - n += 1 + sovKvrpcpb(uint64(m.Priority)) - } - if m.IsolationLevel != 0 { - n += 1 + sovKvrpcpb(uint64(m.IsolationLevel)) + if m.Context != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) + n56, err := m.Context.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n56 } - if m.NotFillCache { - n += 2 + if len(m.Ranges) > 0 { + for _, msg := range m.Ranges { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } } - if m.SyncLog { - n += 2 + if m.EachLimit != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.EachLimit)) } - if m.HandleTime { - n += 2 + if m.KeyOnly { + dAtA[i] = 0x20 + i++ + if m.KeyOnly { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ } - if m.ScanDetail { - n += 2 + return i, nil +} + +func (m *RawBatchScanResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *HandleTime) Size() (n int) { +func (m *RawBatchScanResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.WaitMs != 0 { - n += 1 + sovKvrpcpb(uint64(m.WaitMs)) + if m.RegionError != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) + n57, err := m.RegionError.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n57 } - if m.ProcessMs != 0 { - n += 1 + sovKvrpcpb(uint64(m.ProcessMs)) + if len(m.Kvs) > 0 { + for _, msg := range m.Kvs { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } } - return n + return i, nil } -func (m *ScanInfo) Size() (n int) { - var l int - _ = l - if m.Total != 0 { - n += 1 + sovKvrpcpb(uint64(m.Total)) - } - if m.Processed != 0 { - n += 1 + sovKvrpcpb(uint64(m.Processed)) +func (m *MvccWrite) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *ScanDetail) Size() (n int) { +func (m *MvccWrite) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.Write != nil { - l = m.Write.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.Type != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Type)) } - if m.Lock != nil { - l = m.Lock.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.StartTs != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.StartTs)) } - if m.Data != nil { - l = m.Data.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.CommitTs != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.CommitTs)) } - return n + if len(m.ShortValue) > 0 { + dAtA[i] = 0x22 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.ShortValue))) + i += copy(dAtA[i:], m.ShortValue) + } + return i, nil } -func (m *ExecDetails) Size() (n int) { - var l int - _ = l - if m.HandleTime != nil { - l = m.HandleTime.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - if m.ScanDetail != nil { - l = m.ScanDetail.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) +func (m *MvccValue) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *GetRequest) Size() (n int) { +func (m *MvccValue) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.Context != nil { - l = m.Context.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.Key) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.StartTs != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.StartTs)) } - if m.Version != 0 { - n += 1 + sovKvrpcpb(uint64(m.Version)) + if len(m.Value) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Value))) + i += copy(dAtA[i:], m.Value) } - return n + return i, nil } -func (m *GetResponse) Size() (n int) { - var l int - _ = l - if m.RegionError != nil { - l = m.RegionError.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - if m.Error != nil { - l = m.Error.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.Value) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) +func (m *MvccLock) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *ScanRequest) Size() (n int) { +func (m *MvccLock) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.Context != nil { - l = m.Context.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.StartKey) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.Type != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Type)) } - if m.Limit != 0 { - n += 1 + sovKvrpcpb(uint64(m.Limit)) + if m.StartTs != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.StartTs)) } - if m.Version != 0 { - n += 1 + sovKvrpcpb(uint64(m.Version)) + if len(m.Primary) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Primary))) + i += copy(dAtA[i:], m.Primary) } - if m.KeyOnly { - n += 2 + if len(m.ShortValue) > 0 { + dAtA[i] = 0x22 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.ShortValue))) + i += copy(dAtA[i:], m.ShortValue) } - return n + return i, nil } -func (m *KvPair) Size() (n int) { - var l int - _ = l - if m.Error != nil { - l = m.Error.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.Key) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.Value) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) +func (m *MvccInfo) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *ScanResponse) Size() (n int) { +func (m *MvccInfo) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.RegionError != nil { - l = m.RegionError.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.Lock != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Lock.Size())) + n58, err := m.Lock.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n58 } - if len(m.Pairs) > 0 { - for _, e := range m.Pairs { - l = e.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if len(m.Writes) > 0 { + for _, msg := range m.Writes { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n } } - return n + if len(m.Values) > 0 { + for _, msg := range m.Values { + dAtA[i] = 0x1a + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil } -func (m *Mutation) Size() (n int) { - var l int - _ = l - if m.Op != 0 { - n += 1 + sovKvrpcpb(uint64(m.Op)) - } - l = len(m.Key) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.Value) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) +func (m *MvccGetByKeyRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *PrewriteRequest) Size() (n int) { +func (m *MvccGetByKeyRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l if m.Context != nil { - l = m.Context.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - if len(m.Mutations) > 0 { - for _, e := range m.Mutations { - l = e.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) + n59, err := m.Context.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } + i += n59 } - l = len(m.PrimaryLock) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) - } - if m.StartVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.StartVersion)) - } - if m.LockTtl != 0 { - n += 1 + sovKvrpcpb(uint64(m.LockTtl)) + if len(m.Key) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Key))) + i += copy(dAtA[i:], m.Key) } - if m.SkipConstraintCheck { - n += 2 + return i, nil +} + +func (m *MvccGetByKeyResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *PrewriteResponse) Size() (n int) { +func (m *MvccGetByKeyResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l if m.RegionError != nil { - l = m.RegionError.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) + n60, err := m.RegionError.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n60 } - if len(m.Errors) > 0 { - for _, e := range m.Errors { - l = e.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if len(m.Error) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Error))) + i += copy(dAtA[i:], m.Error) + } + if m.Info != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Info.Size())) + n61, err := m.Info.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } + i += n61 } - return n + return i, nil } -func (m *CommitRequest) Size() (n int) { - var l int - _ = l - if m.Context != nil { - l = m.Context.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) +func (m *MvccGetByStartTsRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - if m.StartVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.StartVersion)) - } - if len(m.Keys) > 0 { - for _, b := range m.Keys { - l = len(b) - n += 1 + l + sovKvrpcpb(uint64(l)) - } - } - if m.CommitVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.CommitVersion)) - } - return n + return dAtA[:n], nil } -func (m *CommitResponse) Size() (n int) { +func (m *MvccGetByStartTsRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.RegionError != nil { - l = m.RegionError.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.Context != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) + n62, err := m.Context.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n62 } - if m.Error != nil { - l = m.Error.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.StartTs != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.StartTs)) } - return n + return i, nil } -func (m *ImportRequest) Size() (n int) { - var l int - _ = l - if len(m.Mutations) > 0 { - for _, e := range m.Mutations { - l = e.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - } - if m.CommitVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.CommitVersion)) +func (m *MvccGetByStartTsResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *ImportResponse) Size() (n int) { +func (m *MvccGetByStartTsResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l if m.RegionError != nil { - l = m.RegionError.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) + n63, err := m.RegionError.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n63 } - l = len(m.Error) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if len(m.Error) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Error))) + i += copy(dAtA[i:], m.Error) } - return n + if len(m.Key) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.Key))) + i += copy(dAtA[i:], m.Key) + } + if m.Info != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Info.Size())) + n64, err := m.Info.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n64 + } + return i, nil } -func (m *BatchRollbackRequest) Size() (n int) { +func (m *SplitRegionRequest) 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 *SplitRegionRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l if m.Context != nil { - l = m.Context.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) + n65, err := m.Context.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n65 } - if m.StartVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.StartVersion)) + if len(m.SplitKey) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(len(m.SplitKey))) + i += copy(dAtA[i:], m.SplitKey) } - if len(m.Keys) > 0 { - for _, b := range m.Keys { - l = len(b) - n += 1 + l + sovKvrpcpb(uint64(l)) - } + return i, nil +} + +func (m *SplitRegionResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *BatchRollbackResponse) Size() (n int) { +func (m *SplitRegionResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l if m.RegionError != nil { - l = m.RegionError.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) + n66, err := m.RegionError.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n66 } - if m.Error != nil { - l = m.Error.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.Left != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Left.Size())) + n67, err := m.Left.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n67 } - return n + if m.Right != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Right.Size())) + n68, err := m.Right.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n68 + } + return i, nil } -func (m *CleanupRequest) Size() (n int) { +func encodeFixed64Kvrpcpb(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + dAtA[offset+4] = uint8(v >> 32) + dAtA[offset+5] = uint8(v >> 40) + dAtA[offset+6] = uint8(v >> 48) + dAtA[offset+7] = uint8(v >> 56) + return offset + 8 +} +func encodeFixed32Kvrpcpb(dAtA []byte, offset int, v uint32) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + return offset + 4 +} +func encodeVarintKvrpcpb(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *LockInfo) Size() (n int) { var l int _ = l - if m.Context != nil { - l = m.Context.Size() + l = len(m.PrimaryLock) + if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } + if m.LockVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.LockVersion)) + } l = len(m.Key) if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.StartVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.StartVersion)) + if m.LockTtl != 0 { + n += 1 + sovKvrpcpb(uint64(m.LockTtl)) } return n } -func (m *CleanupResponse) Size() (n int) { +func (m *KeyError) Size() (n int) { var l int _ = l - if m.RegionError != nil { - l = m.RegionError.Size() + if m.Locked != nil { + l = m.Locked.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Error != nil { - l = m.Error.Size() + l = len(m.Retryable) + if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.CommitVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.CommitVersion)) + l = len(m.Abort) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *BatchGetRequest) Size() (n int) { +func (m *Context) Size() (n int) { var l int _ = l - if m.Context != nil { - l = m.Context.Size() + if m.RegionId != 0 { + n += 1 + sovKvrpcpb(uint64(m.RegionId)) + } + if m.RegionEpoch != nil { + l = m.RegionEpoch.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if len(m.Keys) > 0 { - for _, b := range m.Keys { - l = len(b) - n += 1 + l + sovKvrpcpb(uint64(l)) - } + if m.Peer != nil { + l = m.Peer.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Version != 0 { - n += 1 + sovKvrpcpb(uint64(m.Version)) + if m.Term != 0 { + n += 1 + sovKvrpcpb(uint64(m.Term)) } - return n -} - -func (m *BatchGetResponse) Size() (n int) { - var l int - _ = l - if m.RegionError != nil { - l = m.RegionError.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.Priority != 0 { + n += 1 + sovKvrpcpb(uint64(m.Priority)) } - if len(m.Pairs) > 0 { - for _, e := range m.Pairs { - l = e.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } + if m.IsolationLevel != 0 { + n += 1 + sovKvrpcpb(uint64(m.IsolationLevel)) + } + if m.NotFillCache { + n += 2 + } + if m.SyncLog { + n += 2 + } + if m.HandleTime { + n += 2 + } + if m.ScanDetail { + n += 2 } return n } -func (m *ScanLockRequest) Size() (n int) { +func (m *HandleTime) Size() (n int) { var l int _ = l - if m.Context != nil { - l = m.Context.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.WaitMs != 0 { + n += 1 + sovKvrpcpb(uint64(m.WaitMs)) } - if m.MaxVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.MaxVersion)) + if m.ProcessMs != 0 { + n += 1 + sovKvrpcpb(uint64(m.ProcessMs)) } - l = len(m.StartKey) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + return n +} + +func (m *ScanInfo) Size() (n int) { + var l int + _ = l + if m.Total != 0 { + n += 1 + sovKvrpcpb(uint64(m.Total)) } - if m.Limit != 0 { - n += 1 + sovKvrpcpb(uint64(m.Limit)) + if m.Processed != 0 { + n += 1 + sovKvrpcpb(uint64(m.Processed)) } return n } -func (m *ScanLockResponse) Size() (n int) { +func (m *ScanDetail) Size() (n int) { var l int _ = l - if m.RegionError != nil { - l = m.RegionError.Size() + if m.Write != nil { + l = m.Write.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Error != nil { - l = m.Error.Size() + if m.Lock != nil { + l = m.Lock.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if len(m.Locks) > 0 { - for _, e := range m.Locks { - l = e.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } + if m.Data != nil { + l = m.Data.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *TxnInfo) Size() (n int) { +func (m *ExecDetails) Size() (n int) { var l int _ = l - if m.Txn != 0 { - n += 1 + sovKvrpcpb(uint64(m.Txn)) + if m.HandleTime != nil { + l = m.HandleTime.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Status != 0 { - n += 1 + sovKvrpcpb(uint64(m.Status)) + if m.ScanDetail != nil { + l = m.ScanDetail.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *ResolveLockRequest) Size() (n int) { +func (m *GetRequest) Size() (n int) { var l int _ = l if m.Context != nil { l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.StartVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.StartVersion)) - } - if m.CommitVersion != 0 { - n += 1 + sovKvrpcpb(uint64(m.CommitVersion)) + l = len(m.Key) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) } - if len(m.TxnInfos) > 0 { - for _, e := range m.TxnInfos { - l = e.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } + if m.Version != 0 { + n += 1 + sovKvrpcpb(uint64(m.Version)) } return n } -func (m *ResolveLockResponse) Size() (n int) { +func (m *GetResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { @@ -4455,58 +4902,77 @@ func (m *ResolveLockResponse) Size() (n int) { l = m.Error.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } return n } -func (m *GCRequest) Size() (n int) { +func (m *ScanRequest) Size() (n int) { var l int _ = l if m.Context != nil { l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.SafePoint != 0 { - n += 1 + sovKvrpcpb(uint64(m.SafePoint)) + l = len(m.StartKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Limit != 0 { + n += 1 + sovKvrpcpb(uint64(m.Limit)) + } + if m.Version != 0 { + n += 1 + sovKvrpcpb(uint64(m.Version)) + } + if m.KeyOnly { + n += 2 } return n } -func (m *GCResponse) Size() (n int) { +func (m *KvPair) Size() (n int) { var l int _ = l - if m.RegionError != nil { - l = m.RegionError.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } if m.Error != nil { l = m.Error.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - return n -} - -func (m *RawGetRequest) Size() (n int) { - var l int - _ = l - if m.Context != nil { - l = m.Context.Size() + l = len(m.Key) + if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.Key) + l = len(m.Value) if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *RawGetResponse) Size() (n int) { +func (m *ScanResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { l = m.RegionError.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.Error) + if len(m.Pairs) > 0 { + for _, e := range m.Pairs { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + return n +} + +func (m *Mutation) Size() (n int) { + var l int + _ = l + if m.Op != 0 { + n += 1 + sovKvrpcpb(uint64(m.Op)) + } + l = len(m.Key) if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } @@ -4517,85 +4983,103 @@ func (m *RawGetResponse) Size() (n int) { return n } -func (m *RawPutRequest) Size() (n int) { +func (m *PrewriteRequest) Size() (n int) { var l int _ = l if m.Context != nil { l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.Key) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if len(m.Mutations) > 0 { + for _, e := range m.Mutations { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } } - l = len(m.Value) + l = len(m.PrimaryLock) if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } + if m.StartVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartVersion)) + } + if m.LockTtl != 0 { + n += 1 + sovKvrpcpb(uint64(m.LockTtl)) + } + if m.SkipConstraintCheck { + n += 2 + } return n } -func (m *RawPutResponse) Size() (n int) { +func (m *PrewriteResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { l = m.RegionError.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.Error) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if len(m.Errors) > 0 { + for _, e := range m.Errors { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } } return n } -func (m *RawDeleteRequest) Size() (n int) { +func (m *CommitRequest) Size() (n int) { var l int _ = l if m.Context != nil { l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.Key) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.StartVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartVersion)) + } + if len(m.Keys) > 0 { + for _, b := range m.Keys { + l = len(b) + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + if m.CommitVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.CommitVersion)) } return n } -func (m *RawDeleteResponse) Size() (n int) { +func (m *CommitResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { l = m.RegionError.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.Error) - if l > 0 { + if m.Error != nil { + l = m.Error.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *DeleteRangeRequest) Size() (n int) { +func (m *ImportRequest) Size() (n int) { var l int _ = l - if m.Context != nil { - l = m.Context.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - l = len(m.StartKey) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if len(m.Mutations) > 0 { + for _, e := range m.Mutations { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } } - l = len(m.EndKey) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.CommitVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.CommitVersion)) } return n } -func (m *DeleteRangeResponse) Size() (n int) { +func (m *ImportResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { @@ -4609,85 +5093,141 @@ func (m *DeleteRangeResponse) Size() (n int) { return n } -func (m *RawScanRequest) Size() (n int) { +func (m *BatchRollbackRequest) Size() (n int) { var l int _ = l if m.Context != nil { l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.StartKey) - if l > 0 { - n += 1 + l + sovKvrpcpb(uint64(l)) + if m.StartVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartVersion)) } - if m.Limit != 0 { - n += 1 + sovKvrpcpb(uint64(m.Limit)) + if len(m.Keys) > 0 { + for _, b := range m.Keys { + l = len(b) + n += 1 + l + sovKvrpcpb(uint64(l)) + } } return n } -func (m *RawScanResponse) Size() (n int) { +func (m *BatchRollbackResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { l = m.RegionError.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if len(m.Kvs) > 0 { - for _, e := range m.Kvs { - l = e.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } + if m.Error != nil { + l = m.Error.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *WriteInfo) Size() (n int) { +func (m *CleanupRequest) Size() (n int) { var l int _ = l - if m.StartTs != 0 { - n += 1 + sovKvrpcpb(uint64(m.StartTs)) + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Type != 0 { - n += 1 + sovKvrpcpb(uint64(m.Type)) + l = len(m.Key) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.CommitTs != 0 { - n += 1 + sovKvrpcpb(uint64(m.CommitTs)) + if m.StartVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartVersion)) } return n } -func (m *ValueInfo) Size() (n int) { +func (m *CleanupResponse) Size() (n int) { var l int _ = l - l = len(m.Value) - if l > 0 { + if m.RegionError != nil { + l = m.RegionError.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Ts != 0 { - n += 1 + sovKvrpcpb(uint64(m.Ts)) + if m.Error != nil { + l = m.Error.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.IsShortValue { - n += 2 + if m.CommitVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.CommitVersion)) } return n } -func (m *MvccInfo) Size() (n int) { +func (m *BatchGetRequest) Size() (n int) { var l int _ = l - if m.Lock != nil { - l = m.Lock.Size() + if m.Context != nil { + l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if len(m.Writes) > 0 { - for _, e := range m.Writes { + if len(m.Keys) > 0 { + for _, b := range m.Keys { + l = len(b) + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + if m.Version != 0 { + n += 1 + sovKvrpcpb(uint64(m.Version)) + } + return n +} + +func (m *BatchGetResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Pairs) > 0 { + for _, e := range m.Pairs { l = e.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } } - if len(m.Values) > 0 { - for _, e := range m.Values { + return n +} + +func (m *ScanLockRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.MaxVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.MaxVersion)) + } + l = len(m.StartKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Limit != 0 { + n += 1 + sovKvrpcpb(uint64(m.Limit)) + } + return n +} + +func (m *ScanLockResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Error != nil { + l = m.Error.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Locks) > 0 { + for _, e := range m.Locks { l = e.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } @@ -4695,7 +5235,82 @@ func (m *MvccInfo) Size() (n int) { return n } -func (m *MvccGetByKeyRequest) Size() (n int) { +func (m *TxnInfo) Size() (n int) { + var l int + _ = l + if m.Txn != 0 { + n += 1 + sovKvrpcpb(uint64(m.Txn)) + } + if m.Status != 0 { + n += 1 + sovKvrpcpb(uint64(m.Status)) + } + return n +} + +func (m *ResolveLockRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.StartVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartVersion)) + } + if m.CommitVersion != 0 { + n += 1 + sovKvrpcpb(uint64(m.CommitVersion)) + } + if len(m.TxnInfos) > 0 { + for _, e := range m.TxnInfos { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + return n +} + +func (m *ResolveLockResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Error != nil { + l = m.Error.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *GCRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.SafePoint != 0 { + n += 1 + sovKvrpcpb(uint64(m.SafePoint)) + } + return n +} + +func (m *GCResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Error != nil { + l = m.Error.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *RawGetRequest) Size() (n int) { var l int _ = l if m.Context != nil { @@ -4709,7 +5324,7 @@ func (m *MvccGetByKeyRequest) Size() (n int) { return n } -func (m *MvccGetByKeyResponse) Size() (n int) { +func (m *RawGetResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { @@ -4720,27 +5335,32 @@ func (m *MvccGetByKeyResponse) Size() (n int) { if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Info != nil { - l = m.Info.Size() + l = len(m.Value) + if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *MvccGetByStartTsRequest) Size() (n int) { +func (m *RawPutRequest) Size() (n int) { var l int _ = l if m.Context != nil { l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.StartTs != 0 { - n += 1 + sovKvrpcpb(uint64(m.StartTs)) + l = len(m.Key) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *MvccGetByStartTsResponse) Size() (n int) { +func (m *RawPutResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { @@ -4751,63 +5371,1472 @@ func (m *MvccGetByStartTsResponse) Size() (n int) { if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.Key) + return n +} + +func (m *RawBatchPutRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Pairs) > 0 { + for _, e := range m.Pairs { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + return n +} + +func (m *RawBatchPutResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Error) if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Info != nil { - l = m.Info.Size() + return n +} + +func (m *RawBatchGetRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } + if len(m.Keys) > 0 { + for _, b := range m.Keys { + l = len(b) + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } return n } -func (m *SplitRegionRequest) Size() (n int) { +func (m *RawBatchGetResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Pairs) > 0 { + for _, e := range m.Pairs { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + return n +} + +func (m *RawDeleteRequest) Size() (n int) { var l int _ = l if m.Context != nil { l = m.Context.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - l = len(m.SplitKey) + l = len(m.Key) if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func (m *SplitRegionResponse) Size() (n int) { +func (m *RawDeleteResponse) Size() (n int) { var l int _ = l if m.RegionError != nil { l = m.RegionError.Size() n += 1 + l + sovKvrpcpb(uint64(l)) } - if m.Left != nil { - l = m.Left.Size() - n += 1 + l + sovKvrpcpb(uint64(l)) - } - if m.Right != nil { - l = m.Right.Size() + l = len(m.Error) + if l > 0 { n += 1 + l + sovKvrpcpb(uint64(l)) } return n } -func sovKvrpcpb(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { +func (m *RawBatchDeleteRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Keys) > 0 { + for _, b := range m.Keys { + l = len(b) + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + return n +} + +func (m *RawBatchDeleteResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Error) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *DeleteRangeRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.StartKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.EndKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *DeleteRangeResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Error) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *RawDeleteRangeRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.StartKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.EndKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *RawDeleteRangeResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Error) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *RawScanRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.StartKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Limit != 0 { + n += 1 + sovKvrpcpb(uint64(m.Limit)) + } + if m.KeyOnly { + n += 2 + } + return n +} + +func (m *RawScanResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Kvs) > 0 { + for _, e := range m.Kvs { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + return n +} + +func (m *KeyRange) Size() (n int) { + var l int + _ = l + l = len(m.StartKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.EndKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *RawBatchScanRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Ranges) > 0 { + for _, e := range m.Ranges { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + if m.EachLimit != 0 { + n += 1 + sovKvrpcpb(uint64(m.EachLimit)) + } + if m.KeyOnly { + n += 2 + } + return n +} + +func (m *RawBatchScanResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Kvs) > 0 { + for _, e := range m.Kvs { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + return n +} + +func (m *MvccWrite) Size() (n int) { + var l int + _ = l + if m.Type != 0 { + n += 1 + sovKvrpcpb(uint64(m.Type)) + } + if m.StartTs != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartTs)) + } + if m.CommitTs != 0 { + n += 1 + sovKvrpcpb(uint64(m.CommitTs)) + } + l = len(m.ShortValue) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *MvccValue) Size() (n int) { + var l int + _ = l + if m.StartTs != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartTs)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *MvccLock) Size() (n int) { + var l int + _ = l + if m.Type != 0 { + n += 1 + sovKvrpcpb(uint64(m.Type)) + } + if m.StartTs != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartTs)) + } + l = len(m.Primary) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.ShortValue) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *MvccInfo) Size() (n int) { + var l int + _ = l + if m.Lock != nil { + l = m.Lock.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if len(m.Writes) > 0 { + for _, e := range m.Writes { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + if len(m.Values) > 0 { + for _, e := range m.Values { + l = e.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + } + return n +} + +func (m *MvccGetByKeyRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Key) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *MvccGetByKeyResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Error) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Info != nil { + l = m.Info.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *MvccGetByStartTsRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.StartTs != 0 { + n += 1 + sovKvrpcpb(uint64(m.StartTs)) + } + return n +} + +func (m *MvccGetByStartTsResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Error) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.Key) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Info != nil { + l = m.Info.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *SplitRegionRequest) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + l = len(m.SplitKey) + if l > 0 { + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func (m *SplitRegionResponse) Size() (n int) { + var l int + _ = l + if m.RegionError != nil { + l = m.RegionError.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Left != nil { + l = m.Left.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + if m.Right != nil { + l = m.Right.Size() + n += 1 + l + sovKvrpcpb(uint64(l)) + } + return n +} + +func sovKvrpcpb(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { break } } - return n -} -func sozKvrpcpb(x uint64) (n int) { - return sovKvrpcpb(uint64((x << 1) ^ uint64((int64(x) >> 63)))) + return n +} +func sozKvrpcpb(x uint64) (n int) { + return sovKvrpcpb(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *LockInfo) 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 ErrIntOverflowKvrpcpb + } + 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: LockInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LockInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryLock", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrimaryLock = append(m.PrimaryLock[:0], dAtA[iNdEx:postIndex]...) + if m.PrimaryLock == nil { + m.PrimaryLock = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockVersion", wireType) + } + m.LockVersion = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockVersion |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockTtl", wireType) + } + m.LockTtl = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockTtl |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *KeyError) 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 ErrIntOverflowKvrpcpb + } + 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: KeyError: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: KeyError: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Locked", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Locked == nil { + m.Locked = &LockInfo{} + } + if err := m.Locked.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Retryable", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Retryable = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Abort", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Abort = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Context) 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 ErrIntOverflowKvrpcpb + } + 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: Context: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Context: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionId", wireType) + } + m.RegionId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RegionId |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionEpoch", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RegionEpoch == nil { + m.RegionEpoch = &metapb.RegionEpoch{} + } + if err := m.RegionEpoch.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Peer", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Peer == nil { + m.Peer = &metapb.Peer{} + } + if err := m.Peer.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Term", wireType) + } + m.Term = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Term |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Priority", wireType) + } + m.Priority = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Priority |= (CommandPri(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsolationLevel", wireType) + } + m.IsolationLevel = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.IsolationLevel |= (IsolationLevel(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NotFillCache", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.NotFillCache = bool(v != 0) + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncLog", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SyncLog = bool(v != 0) + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field HandleTime", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.HandleTime = bool(v != 0) + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ScanDetail", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.ScanDetail = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *HandleTime) 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 ErrIntOverflowKvrpcpb + } + 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: HandleTime: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HandleTime: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WaitMs", wireType) + } + m.WaitMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.WaitMs |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ProcessMs", wireType) + } + m.ProcessMs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ProcessMs |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ScanInfo) 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 ErrIntOverflowKvrpcpb + } + 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: ScanInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ScanInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Total", wireType) + } + m.Total = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Total |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Processed", wireType) + } + m.Processed = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Processed |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ScanDetail) 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 ErrIntOverflowKvrpcpb + } + 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: ScanDetail: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ScanDetail: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Write", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Write == nil { + m.Write = &ScanInfo{} + } + if err := m.Write.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Lock", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Lock == nil { + m.Lock = &ScanInfo{} + } + if err := m.Lock.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Data == nil { + m.Data = &ScanInfo{} + } + if err := m.Data.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExecDetails) 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 ErrIntOverflowKvrpcpb + } + 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: ExecDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExecDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HandleTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.HandleTime == nil { + m.HandleTime = &HandleTime{} + } + if err := m.HandleTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ScanDetail", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ScanDetail == nil { + m.ScanDetail = &ScanDetail{} + } + if err := m.ScanDetail.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil } -func (m *LockInfo) Unmarshal(dAtA []byte) error { +func (m *GetRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4830,17 +6859,17 @@ func (m *LockInfo) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: LockInfo: wiretype end group for non-group") + return fmt.Errorf("proto: GetRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: LockInfo: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PrimaryLock", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -4850,43 +6879,26 @@ func (m *LockInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.PrimaryLock = append(m.PrimaryLock[:0], dAtA[iNdEx:postIndex]...) - if m.PrimaryLock == nil { - m.PrimaryLock = []byte{} + if m.Context == nil { + m.Context = &Context{} + } + if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LockVersion", wireType) - } - m.LockVersion = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.LockVersion |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) } @@ -4917,11 +6929,11 @@ func (m *LockInfo) Unmarshal(dAtA []byte) error { m.Key = []byte{} } iNdEx = postIndex - case 4: + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LockTtl", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) } - m.LockTtl = 0 + m.Version = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -4931,7 +6943,7 @@ func (m *LockInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LockTtl |= (uint64(b) & 0x7F) << shift + m.Version |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } @@ -4957,7 +6969,7 @@ func (m *LockInfo) Unmarshal(dAtA []byte) error { } return nil } -func (m *KeyError) Unmarshal(dAtA []byte) error { +func (m *GetResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -4980,15 +6992,15 @@ func (m *KeyError) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: KeyError: wiretype end group for non-group") + return fmt.Errorf("proto: GetResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: KeyError: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Locked", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5012,18 +7024,18 @@ func (m *KeyError) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Locked == nil { - m.Locked = &LockInfo{} + if m.RegionError == nil { + m.RegionError = &errorpb.Error{} } - if err := m.Locked.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Retryable", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5033,26 +7045,30 @@ func (m *KeyError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.Retryable = string(dAtA[iNdEx:postIndex]) + if m.Error == nil { + m.Error = &KeyError{} + } + if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Abort", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) } - var stringLen uint64 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5062,20 +7078,22 @@ func (m *KeyError) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if byteLen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + byteLen if postIndex > l { return io.ErrUnexpectedEOF } - m.Abort = string(dAtA[iNdEx:postIndex]) + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} + } iNdEx = postIndex default: iNdEx = preIndex @@ -5098,7 +7116,7 @@ func (m *KeyError) Unmarshal(dAtA []byte) error { } return nil } -func (m *Context) Unmarshal(dAtA []byte) error { +func (m *ScanRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5121,17 +7139,17 @@ func (m *Context) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Context: wiretype end group for non-group") + return fmt.Errorf("proto: ScanRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Context: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ScanRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field RegionId", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) } - m.RegionId = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5141,16 +7159,30 @@ func (m *Context) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RegionId |= (uint64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Context == nil { + m.Context = &Context{} + } + if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RegionEpoch", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartKey", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5160,30 +7192,28 @@ func (m *Context) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex > l { return io.ErrUnexpectedEOF } - if m.RegionEpoch == nil { - m.RegionEpoch = &metapb.RegionEpoch{} - } - if err := m.RegionEpoch.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.StartKey = append(m.StartKey[:0], dAtA[iNdEx:postIndex]...) + if m.StartKey == nil { + m.StartKey = []byte{} } iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Peer", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) } - var msglen int + m.Limit = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5193,30 +7223,35 @@ func (m *Context) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + m.Limit |= (uint32(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Peer == nil { - m.Peer = &metapb.Peer{} + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) } - if err := m.Peer.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.Version = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Version |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } } - iNdEx = postIndex case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Term", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field KeyOnly", wireType) } - m.Term = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5226,16 +7261,67 @@ func (m *Context) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Term |= (uint64(b) & 0x7F) << shift + v |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Priority", wireType) + m.KeyOnly = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err } - m.Priority = 0 + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *KvPair) 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 ErrIntOverflowKvrpcpb + } + 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: KvPair: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: KvPair: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5245,55 +7331,30 @@ func (m *Context) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Priority |= (CommandPri(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IsolationLevel", wireType) + if msglen < 0 { + return ErrInvalidLengthKvrpcpb } - m.IsolationLevel = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.IsolationLevel |= (IsolationLevel(b) & 0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF } - case 8: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field NotFillCache", wireType) + if m.Error == nil { + m.Error = &KeyError{} } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } + if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.NotFillCache = bool(v != 0) - case 9: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SyncLog", wireType) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) } - var v int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5303,37 +7364,28 @@ func (m *Context) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - m.SyncLog = bool(v != 0) - case 10: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field HandleTime", wireType) + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF } - m.HandleTime = bool(v != 0) - case 11: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ScanDetail", wireType) + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} } - var v int + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5343,12 +7395,23 @@ func (m *Context) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - m.ScanDetail = bool(v != 0) + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -5370,7 +7433,7 @@ func (m *Context) Unmarshal(dAtA []byte) error { } return nil } -func (m *HandleTime) Unmarshal(dAtA []byte) error { +func (m *ScanResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5393,17 +7456,17 @@ func (m *HandleTime) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: HandleTime: wiretype end group for non-group") + return fmt.Errorf("proto: ScanResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: HandleTime: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ScanResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field WaitMs", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) } - m.WaitMs = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5413,16 +7476,30 @@ func (m *HandleTime) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.WaitMs |= (int64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RegionError == nil { + m.RegionError = &errorpb.Error{} + } + if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ProcessMs", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Pairs", wireType) } - m.ProcessMs = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5432,11 +7509,23 @@ func (m *HandleTime) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ProcessMs |= (int64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Pairs = append(m.Pairs, &KvPair{}) + if err := m.Pairs[len(m.Pairs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -5458,7 +7547,7 @@ func (m *HandleTime) Unmarshal(dAtA []byte) error { } return nil } -func (m *ScanInfo) Unmarshal(dAtA []byte) error { +func (m *Mutation) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5481,17 +7570,17 @@ func (m *ScanInfo) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ScanInfo: wiretype end group for non-group") + return fmt.Errorf("proto: Mutation: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ScanInfo: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Mutation: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Total", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) } - m.Total = 0 + m.Op = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5501,16 +7590,16 @@ func (m *ScanInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Total |= (int64(b) & 0x7F) << shift + m.Op |= (Op(b) & 0x7F) << shift if b < 0x80 { break } } case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Processed", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) } - m.Processed = 0 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5520,11 +7609,54 @@ func (m *ScanInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Processed |= (int64(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -5546,7 +7678,7 @@ func (m *ScanInfo) Unmarshal(dAtA []byte) error { } return nil } -func (m *ScanDetail) Unmarshal(dAtA []byte) error { +func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5569,15 +7701,48 @@ func (m *ScanDetail) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ScanDetail: wiretype end group for non-group") + return fmt.Errorf("proto: PrewriteRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ScanDetail: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrewriteRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Write", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Context == nil { + m.Context = &Context{} + } + if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Mutations", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5601,18 +7766,16 @@ func (m *ScanDetail) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Write == nil { - m.Write = &ScanInfo{} - } - if err := m.Write.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Mutations = append(m.Mutations, &Mutation{}) + if err := m.Mutations[len(m.Mutations)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Lock", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryLock", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5622,30 +7785,28 @@ func (m *ScanDetail) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex > l { return io.ErrUnexpectedEOF } - if m.Lock == nil { - m.Lock = &ScanInfo{} - } - if err := m.Lock.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.PrimaryLock = append(m.PrimaryLock[:0], dAtA[iNdEx:postIndex]...) + if m.PrimaryLock == nil { + m.PrimaryLock = []byte{} } iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartVersion", wireType) } - var msglen int + m.StartVersion = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5655,25 +7816,50 @@ func (m *ScanDetail) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + m.StartVersion |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthKvrpcpb + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LockTtl", wireType) } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF + m.LockTtl = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LockTtl |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if m.Data == nil { - m.Data = &ScanInfo{} + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SkipConstraintCheck", wireType) } - if err := m.Data.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } } - iNdEx = postIndex + m.SkipConstraintCheck = bool(v != 0) default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -5695,7 +7881,7 @@ func (m *ScanDetail) Unmarshal(dAtA []byte) error { } return nil } -func (m *ExecDetails) Unmarshal(dAtA []byte) error { +func (m *PrewriteResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5718,15 +7904,15 @@ func (m *ExecDetails) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ExecDetails: wiretype end group for non-group") + return fmt.Errorf("proto: PrewriteResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ExecDetails: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: PrewriteResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field HandleTime", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5750,16 +7936,16 @@ func (m *ExecDetails) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.HandleTime == nil { - m.HandleTime = &HandleTime{} + if m.RegionError == nil { + m.RegionError = &errorpb.Error{} } - if err := m.HandleTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ScanDetail", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Errors", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5783,10 +7969,8 @@ func (m *ExecDetails) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ScanDetail == nil { - m.ScanDetail = &ScanDetail{} - } - if err := m.ScanDetail.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Errors = append(m.Errors, &KeyError{}) + if err := m.Errors[len(m.Errors)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -5811,7 +7995,7 @@ func (m *ExecDetails) Unmarshal(dAtA []byte) error { } return nil } -func (m *GetRequest) Unmarshal(dAtA []byte) error { +func (m *CommitRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5834,10 +8018,10 @@ func (m *GetRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetRequest: wiretype end group for non-group") + return fmt.Errorf("proto: CommitRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CommitRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -5874,8 +8058,27 @@ func (m *GetRequest) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartVersion", wireType) + } + m.StartVersion = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StartVersion |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -5899,16 +8102,14 @@ func (m *GetRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) - if m.Key == nil { - m.Key = []byte{} - } + m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) + copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field CommitVersion", wireType) } - m.Version = 0 + m.CommitVersion = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -5918,7 +8119,7 @@ func (m *GetRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Version |= (uint64(b) & 0x7F) << shift + m.CommitVersion |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } @@ -5944,7 +8145,7 @@ func (m *GetRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *GetResponse) Unmarshal(dAtA []byte) error { +func (m *CommitResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5967,10 +8168,10 @@ func (m *GetResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetResponse: wiretype end group for non-group") + return fmt.Errorf("proto: CommitResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CommitResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6039,37 +8240,6 @@ func (m *GetResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + byteLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) - if m.Value == nil { - m.Value = []byte{} - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -6091,7 +8261,7 @@ func (m *GetResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *ScanRequest) Unmarshal(dAtA []byte) error { +func (m *ImportRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6114,15 +8284,15 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ScanRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ImportRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ScanRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ImportRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Mutations", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6146,18 +8316,16 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Context == nil { - m.Context = &Context{} - } - if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Mutations = append(m.Mutations, &Mutation{}) + if err := m.Mutations[len(m.Mutations)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field StartKey", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CommitVersion", wireType) } - var byteLen int + m.CommitVersion = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6167,28 +8335,66 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + m.CommitVersion |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + byteLen - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.StartKey = append(m.StartKey[:0], dAtA[iNdEx:postIndex]...) - if m.StartKey == nil { - m.StartKey = []byte{} + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ImportResponse) 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 ErrIntOverflowKvrpcpb } - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) + 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: ImportResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ImportResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) } - m.Limit = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6198,35 +8404,30 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Limit |= (uint32(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + if msglen < 0 { + return ErrInvalidLengthKvrpcpb } - m.Version = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Version |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF } - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field KeyOnly", wireType) + if m.RegionError == nil { + m.RegionError = &errorpb.Error{} } - var v int + if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + } + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6236,12 +8437,21 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + stringLen |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - m.KeyOnly = bool(v != 0) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Error = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -6263,7 +8473,7 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *KvPair) Unmarshal(dAtA []byte) error { +func (m *BatchRollbackRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6286,15 +8496,15 @@ func (m *KvPair) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: KvPair: wiretype end group for non-group") + return fmt.Errorf("proto: BatchRollbackRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: KvPair: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BatchRollbackRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6318,18 +8528,18 @@ func (m *KvPair) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Error == nil { - m.Error = &KeyError{} + if m.Context == nil { + m.Context = &Context{} } - if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartVersion", wireType) } - var byteLen int + m.StartVersion = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6339,26 +8549,14 @@ func (m *KvPair) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + m.StartVersion |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + byteLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) - if m.Key == nil { - m.Key = []byte{} - } - iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -6382,10 +8580,8 @@ func (m *KvPair) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) - if m.Value == nil { - m.Value = []byte{} - } + m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) + copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -6408,7 +8604,7 @@ func (m *KvPair) Unmarshal(dAtA []byte) error { } return nil } -func (m *ScanResponse) Unmarshal(dAtA []byte) error { +func (m *BatchRollbackResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6431,10 +8627,10 @@ func (m *ScanResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ScanResponse: wiretype end group for non-group") + return fmt.Errorf("proto: BatchRollbackResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ScanResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BatchRollbackResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6472,7 +8668,7 @@ func (m *ScanResponse) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Pairs", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6496,8 +8692,10 @@ func (m *ScanResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Pairs = append(m.Pairs, &KvPair{}) - if err := m.Pairs[len(m.Pairs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.Error == nil { + m.Error = &KeyError{} + } + if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -6522,7 +8720,7 @@ func (m *ScanResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *Mutation) Unmarshal(dAtA []byte) error { +func (m *CleanupRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6545,17 +8743,17 @@ func (m *Mutation) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Mutation: wiretype end group for non-group") + return fmt.Errorf("proto: CleanupRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Mutation: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CleanupRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) } - m.Op = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6565,11 +8763,25 @@ func (m *Mutation) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Op |= (Op(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Context == nil { + m.Context = &Context{} + } + if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) @@ -6602,10 +8814,10 @@ func (m *Mutation) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartVersion", wireType) } - var byteLen int + m.StartVersion = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6615,23 +8827,11 @@ func (m *Mutation) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + m.StartVersion |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + byteLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) - if m.Value == nil { - m.Value = []byte{} - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -6653,7 +8853,7 @@ func (m *Mutation) Unmarshal(dAtA []byte) error { } return nil } -func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { +func (m *CleanupResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6676,15 +8876,15 @@ func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrewriteRequest: wiretype end group for non-group") + return fmt.Errorf("proto: CleanupResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrewriteRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CleanupResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6708,16 +8908,16 @@ func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Context == nil { - m.Context = &Context{} + if m.RegionError == nil { + m.RegionError = &errorpb.Error{} } - if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Mutations", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6741,16 +8941,18 @@ func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Mutations = append(m.Mutations, &Mutation{}) - if err := m.Mutations[len(m.Mutations)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if m.Error == nil { + m.Error = &KeyError{} + } + if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PrimaryLock", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CommitVersion", wireType) } - var byteLen int + m.CommitVersion = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6760,28 +8962,66 @@ func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + m.CommitVersion |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + byteLen - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.PrimaryLock = append(m.PrimaryLock[:0], dAtA[iNdEx:postIndex]...) - if m.PrimaryLock == nil { - m.PrimaryLock = []byte{} + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BatchGetRequest) 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 ErrIntOverflowKvrpcpb } - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StartVersion", wireType) + 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: BatchGetRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BatchGetRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) } - m.StartVersion = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6791,16 +9031,30 @@ func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.StartVersion |= (uint64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field LockTtl", wireType) + if msglen < 0 { + return ErrInvalidLengthKvrpcpb } - m.LockTtl = 0 + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Context == nil { + m.Context = &Context{} + } + if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) + } + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6810,16 +9064,26 @@ func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.LockTtl |= (uint64(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 6: + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) + copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SkipConstraintCheck", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) } - var v int + m.Version = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -6829,12 +9093,11 @@ func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + m.Version |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - m.SkipConstraintCheck = bool(v != 0) default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -6856,7 +9119,7 @@ func (m *PrewriteRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *PrewriteResponse) Unmarshal(dAtA []byte) error { +func (m *BatchGetResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6879,10 +9142,10 @@ func (m *PrewriteResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: PrewriteResponse: wiretype end group for non-group") + return fmt.Errorf("proto: BatchGetResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: PrewriteResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: BatchGetResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6920,7 +9183,7 @@ func (m *PrewriteResponse) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Errors", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Pairs", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -6944,8 +9207,8 @@ func (m *PrewriteResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Errors = append(m.Errors, &KeyError{}) - if err := m.Errors[len(m.Errors)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Pairs = append(m.Pairs, &KvPair{}) + if err := m.Pairs[len(m.Pairs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -6970,7 +9233,7 @@ func (m *PrewriteResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *CommitRequest) Unmarshal(dAtA []byte) error { +func (m *ScanLockRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6993,10 +9256,10 @@ func (m *CommitRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CommitRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ScanLockRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CommitRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ScanLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7034,9 +9297,9 @@ func (m *CommitRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StartVersion", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MaxVersion", wireType) } - m.StartVersion = 0 + m.MaxVersion = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -7046,14 +9309,14 @@ func (m *CommitRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.StartVersion |= (uint64(b) & 0x7F) << shift + m.MaxVersion |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartKey", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -7077,14 +9340,16 @@ func (m *CommitRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) - copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) + m.StartKey = append(m.StartKey[:0], dAtA[iNdEx:postIndex]...) + if m.StartKey == nil { + m.StartKey = []byte{} + } iNdEx = postIndex case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CommitVersion", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) } - m.CommitVersion = 0 + m.Limit = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -7094,7 +9359,7 @@ func (m *CommitRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.CommitVersion |= (uint64(b) & 0x7F) << shift + m.Limit |= (uint32(b) & 0x7F) << shift if b < 0x80 { break } @@ -7120,7 +9385,7 @@ func (m *CommitRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *CommitResponse) Unmarshal(dAtA []byte) error { +func (m *ScanLockResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7143,10 +9408,10 @@ func (m *CommitResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CommitResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ScanLockResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CommitResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ScanLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7215,59 +9480,9 @@ func (m *CommitResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipKvrpcpb(dAtA[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthKvrpcpb - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *ImportRequest) 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 ErrIntOverflowKvrpcpb - } - 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: ImportRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: ImportRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Mutations", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Locks", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -7291,30 +9506,11 @@ func (m *ImportRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Mutations = append(m.Mutations, &Mutation{}) - if err := m.Mutations[len(m.Mutations)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Locks = append(m.Locks, &LockInfo{}) + if err := m.Locks[len(m.Locks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CommitVersion", wireType) - } - m.CommitVersion = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.CommitVersion |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -7336,7 +9532,7 @@ func (m *ImportRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *ImportResponse) Unmarshal(dAtA []byte) error { +func (m *TxnInfo) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7359,17 +9555,17 @@ func (m *ImportResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ImportResponse: wiretype end group for non-group") + return fmt.Errorf("proto: TxnInfo: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ImportResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TxnInfo: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Txn", wireType) } - var msglen int + m.Txn = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -7379,30 +9575,16 @@ func (m *ImportResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + m.Txn |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.RegionError == nil { - m.RegionError = &errorpb.Error{} - } - if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) } - var stringLen uint64 + m.Status = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -7412,21 +9594,11 @@ func (m *ImportResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + m.Status |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Error = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -7448,7 +9620,7 @@ func (m *ImportResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *BatchRollbackRequest) Unmarshal(dAtA []byte) error { +func (m *ResolveLockRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7471,10 +9643,10 @@ func (m *BatchRollbackRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BatchRollbackRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ResolveLockRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BatchRollbackRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ResolveLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7530,10 +9702,29 @@ func (m *BatchRollbackRequest) Unmarshal(dAtA []byte) error { } } case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CommitVersion", wireType) + } + m.CommitVersion = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CommitVersion |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TxnInfos", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -7543,20 +9734,22 @@ func (m *BatchRollbackRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) - copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) + m.TxnInfos = append(m.TxnInfos, &TxnInfo{}) + if err := m.TxnInfos[len(m.TxnInfos)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -7579,7 +9772,7 @@ func (m *BatchRollbackRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *BatchRollbackResponse) Unmarshal(dAtA []byte) error { +func (m *ResolveLockResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7602,10 +9795,10 @@ func (m *BatchRollbackResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BatchRollbackResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ResolveLockResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BatchRollbackResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ResolveLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7695,7 +9888,7 @@ func (m *BatchRollbackResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *CleanupRequest) Unmarshal(dAtA []byte) error { +func (m *GCRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7718,10 +9911,10 @@ func (m *CleanupRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CleanupRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GCRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CleanupRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GCRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7758,41 +9951,10 @@ func (m *CleanupRequest) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + byteLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) - if m.Key == nil { - m.Key = []byte{} - } - iNdEx = postIndex - case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StartVersion", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SafePoint", wireType) } - m.StartVersion = 0 + m.SafePoint = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -7802,7 +9964,7 @@ func (m *CleanupRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.StartVersion |= (uint64(b) & 0x7F) << shift + m.SafePoint |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } @@ -7828,7 +9990,7 @@ func (m *CleanupRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *CleanupResponse) Unmarshal(dAtA []byte) error { +func (m *GCResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7851,10 +10013,10 @@ func (m *CleanupResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: CleanupResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GCResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: CleanupResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GCResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7923,25 +10085,6 @@ func (m *CleanupResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CommitVersion", wireType) - } - m.CommitVersion = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.CommitVersion |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -7963,7 +10106,7 @@ func (m *CleanupResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *BatchGetRequest) Unmarshal(dAtA []byte) error { +func (m *RawGetRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7986,10 +10129,10 @@ func (m *BatchGetRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BatchGetRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RawGetRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BatchGetRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawGetRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -8027,7 +10170,7 @@ func (m *BatchGetRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -8051,28 +10194,11 @@ func (m *BatchGetRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) - copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) - } - m.Version = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Version |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -8094,7 +10220,7 @@ func (m *BatchGetRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *BatchGetResponse) Unmarshal(dAtA []byte) error { +func (m *RawGetResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8117,10 +10243,10 @@ func (m *BatchGetResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: BatchGetResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RawGetResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: BatchGetResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawGetResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -8158,9 +10284,9 @@ func (m *BatchGetResponse) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Pairs", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -8170,21 +10296,50 @@ func (m *BatchGetResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + stringLen |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex > l { return io.ErrUnexpectedEOF } - m.Pairs = append(m.Pairs, &KvPair{}) - if err := m.Pairs[len(m.Pairs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.Error = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} } iNdEx = postIndex default: @@ -8208,7 +10363,7 @@ func (m *BatchGetResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *ScanLockRequest) Unmarshal(dAtA []byte) error { +func (m *RawPutRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8231,10 +10386,10 @@ func (m *ScanLockRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ScanLockRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RawPutRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ScanLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawPutRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -8271,10 +10426,10 @@ func (m *ScanLockRequest) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxVersion", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) } - m.MaxVersion = 0 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -8284,16 +10439,109 @@ func (m *ScanLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaxVersion |= (uint64(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} + } + iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field StartKey", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RawPutResponse) 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 ErrIntOverflowKvrpcpb + } + 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: RawPutResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RawPutResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -8303,28 +10551,30 @@ func (m *ScanLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.StartKey = append(m.StartKey[:0], dAtA[iNdEx:postIndex]...) - if m.StartKey == nil { - m.StartKey = []byte{} + if m.RegionError == nil { + m.RegionError = &errorpb.Error{} + } + if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) } - m.Limit = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -8334,11 +10584,21 @@ func (m *ScanLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Limit |= (uint32(b) & 0x7F) << shift + stringLen |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Error = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -8360,7 +10620,7 @@ func (m *ScanLockRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *ScanLockResponse) Unmarshal(dAtA []byte) error { +func (m *RawBatchPutRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8383,15 +10643,15 @@ func (m *ScanLockResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ScanLockResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RawBatchPutRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ScanLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawBatchPutRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -8415,49 +10675,16 @@ func (m *ScanLockResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.RegionError == nil { - m.RegionError = &errorpb.Error{} + if m.Context == nil { + m.Context = &Context{} } - if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Error == nil { - m.Error = &KeyError{} - } - if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Locks", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Pairs", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -8481,8 +10708,8 @@ func (m *ScanLockResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Locks = append(m.Locks, &LockInfo{}) - if err := m.Locks[len(m.Locks)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Pairs = append(m.Pairs, &KvPair{}) + if err := m.Pairs[len(m.Pairs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -8507,7 +10734,7 @@ func (m *ScanLockResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *TxnInfo) Unmarshal(dAtA []byte) error { +func (m *RawBatchPutResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8530,17 +10757,17 @@ func (m *TxnInfo) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TxnInfo: wiretype end group for non-group") + return fmt.Errorf("proto: RawBatchPutResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TxnInfo: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawBatchPutResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Txn", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) } - m.Txn = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -8550,16 +10777,30 @@ func (m *TxnInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Txn |= (uint64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RegionError == nil { + m.RegionError = &errorpb.Error{} + } + if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Status", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) } - m.Status = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -8569,11 +10810,21 @@ func (m *TxnInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Status |= (uint64(b) & 0x7F) << shift + stringLen |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Error = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -8595,7 +10846,7 @@ func (m *TxnInfo) Unmarshal(dAtA []byte) error { } return nil } -func (m *ResolveLockRequest) Unmarshal(dAtA []byte) error { +func (m *RawBatchGetRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8618,10 +10869,10 @@ func (m *ResolveLockRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ResolveLockRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RawBatchGetRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ResolveLockRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawBatchGetRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -8658,48 +10909,10 @@ func (m *ResolveLockRequest) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StartVersion", wireType) - } - m.StartVersion = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.StartVersion |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field CommitVersion", wireType) - } - m.CommitVersion = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.CommitVersion |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TxnInfos", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -8709,22 +10922,20 @@ func (m *ResolveLockRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex > l { return io.ErrUnexpectedEOF } - m.TxnInfos = append(m.TxnInfos, &TxnInfo{}) - if err := m.TxnInfos[len(m.TxnInfos)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) + copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -8747,7 +10958,7 @@ func (m *ResolveLockRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *ResolveLockResponse) Unmarshal(dAtA []byte) error { +func (m *RawBatchGetResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8770,10 +10981,10 @@ func (m *ResolveLockResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ResolveLockResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RawBatchGetResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ResolveLockResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawBatchGetResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -8811,7 +11022,7 @@ func (m *ResolveLockResponse) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Pairs", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -8835,10 +11046,8 @@ func (m *ResolveLockResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Error == nil { - m.Error = &KeyError{} - } - if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.Pairs = append(m.Pairs, &KvPair{}) + if err := m.Pairs[len(m.Pairs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -8863,7 +11072,7 @@ func (m *ResolveLockResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *GCRequest) Unmarshal(dAtA []byte) error { +func (m *RawDeleteRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8886,10 +11095,10 @@ func (m *GCRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GCRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RawDeleteRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GCRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawDeleteRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -8926,10 +11135,10 @@ func (m *GCRequest) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SafePoint", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) } - m.SafePoint = 0 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -8939,11 +11148,23 @@ func (m *GCRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.SafePoint |= (uint64(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -8965,7 +11186,7 @@ func (m *GCRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *GCResponse) Unmarshal(dAtA []byte) error { +func (m *RawDeleteResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8988,10 +11209,10 @@ func (m *GCResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GCResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RawDeleteResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GCResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9031,7 +11252,7 @@ func (m *GCResponse) Unmarshal(dAtA []byte) error { if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -9041,24 +11262,20 @@ func (m *GCResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + stringLen |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Error == nil { - m.Error = &KeyError{} - } - if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + return io.ErrUnexpectedEOF } + m.Error = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -9081,7 +11298,7 @@ func (m *GCResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *RawGetRequest) Unmarshal(dAtA []byte) error { +func (m *RawBatchDeleteRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9104,10 +11321,10 @@ func (m *RawGetRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RawGetRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RawBatchDeleteRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RawGetRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawBatchDeleteRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9145,7 +11362,7 @@ func (m *RawGetRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -9169,10 +11386,8 @@ func (m *RawGetRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) - if m.Key == nil { - m.Key = []byte{} - } + m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) + copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -9195,7 +11410,7 @@ func (m *RawGetRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *RawGetResponse) Unmarshal(dAtA []byte) error { +func (m *RawBatchDeleteResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9218,10 +11433,10 @@ func (m *RawGetResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RawGetResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RawBatchDeleteResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RawGetResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawBatchDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9286,37 +11501,6 @@ func (m *RawGetResponse) Unmarshal(dAtA []byte) error { } m.Error = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowKvrpcpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLengthKvrpcpb - } - postIndex := iNdEx + byteLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) - if m.Value == nil { - m.Value = []byte{} - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -9338,7 +11522,7 @@ func (m *RawGetResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *RawPutRequest) Unmarshal(dAtA []byte) error { +func (m *DeleteRangeRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9361,10 +11545,10 @@ func (m *RawPutRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RawPutRequest: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteRangeRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RawPutRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteRangeRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9402,7 +11586,7 @@ func (m *RawPutRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartKey", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -9426,14 +11610,14 @@ func (m *RawPutRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) - if m.Key == nil { - m.Key = []byte{} + m.StartKey = append(m.StartKey[:0], dAtA[iNdEx:postIndex]...) + if m.StartKey == nil { + m.StartKey = []byte{} } iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field EndKey", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -9457,9 +11641,9 @@ func (m *RawPutRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) - if m.Value == nil { - m.Value = []byte{} + m.EndKey = append(m.EndKey[:0], dAtA[iNdEx:postIndex]...) + if m.EndKey == nil { + m.EndKey = []byte{} } iNdEx = postIndex default: @@ -9483,7 +11667,7 @@ func (m *RawPutRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *RawPutResponse) Unmarshal(dAtA []byte) error { +func (m *DeleteRangeResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9506,10 +11690,10 @@ func (m *RawPutResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RawPutResponse: wiretype end group for non-group") + return fmt.Errorf("proto: DeleteRangeResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RawPutResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DeleteRangeResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9595,7 +11779,7 @@ func (m *RawPutResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *RawDeleteRequest) Unmarshal(dAtA []byte) error { +func (m *RawDeleteRangeRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9618,10 +11802,10 @@ func (m *RawDeleteRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RawDeleteRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RawDeleteRangeRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RawDeleteRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawDeleteRangeRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9659,7 +11843,7 @@ func (m *RawDeleteRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartKey", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -9683,9 +11867,40 @@ func (m *RawDeleteRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) - if m.Key == nil { - m.Key = []byte{} + m.StartKey = append(m.StartKey[:0], dAtA[iNdEx:postIndex]...) + if m.StartKey == nil { + m.StartKey = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EndKey", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.EndKey = append(m.EndKey[:0], dAtA[iNdEx:postIndex]...) + if m.EndKey == nil { + m.EndKey = []byte{} } iNdEx = postIndex default: @@ -9709,7 +11924,7 @@ func (m *RawDeleteRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *RawDeleteResponse) Unmarshal(dAtA []byte) error { +func (m *RawDeleteRangeResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9732,10 +11947,10 @@ func (m *RawDeleteResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RawDeleteResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RawDeleteRangeResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RawDeleteResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawDeleteRangeResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9821,7 +12036,7 @@ func (m *RawDeleteResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *DeleteRangeRequest) Unmarshal(dAtA []byte) error { +func (m *RawScanRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9844,10 +12059,10 @@ func (m *DeleteRangeRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteRangeRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RawScanRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteRangeRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawScanRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9915,10 +12130,99 @@ func (m *DeleteRangeRequest) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) + } + m.Limit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Limit |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeyOnly", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.KeyOnly = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RawScanResponse) 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 ErrIntOverflowKvrpcpb + } + 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: RawScanResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RawScanResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field EndKey", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -9928,21 +12232,54 @@ func (m *DeleteRangeRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.RegionError == nil { + m.RegionError = &errorpb.Error{} + } + if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Kvs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.EndKey = append(m.EndKey[:0], dAtA[iNdEx:postIndex]...) - if m.EndKey == nil { - m.EndKey = []byte{} + m.Kvs = append(m.Kvs, &KvPair{}) + if err := m.Kvs[len(m.Kvs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex default: @@ -9966,7 +12303,7 @@ func (m *DeleteRangeRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *DeleteRangeResponse) Unmarshal(dAtA []byte) error { +func (m *KeyRange) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9989,17 +12326,17 @@ func (m *DeleteRangeResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteRangeResponse: wiretype end group for non-group") + return fmt.Errorf("proto: KeyRange: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteRangeResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: KeyRange: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RegionError", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartKey", wireType) } - var msglen int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -10009,30 +12346,28 @@ func (m *DeleteRangeResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + if byteLen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + msglen + postIndex := iNdEx + byteLen if postIndex > l { return io.ErrUnexpectedEOF } - if m.RegionError == nil { - m.RegionError = &errorpb.Error{} - } - if err := m.RegionError.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err + m.StartKey = append(m.StartKey[:0], dAtA[iNdEx:postIndex]...) + if m.StartKey == nil { + m.StartKey = []byte{} } iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Error", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field EndKey", wireType) } - var stringLen uint64 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -10042,20 +12377,22 @@ func (m *DeleteRangeResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if byteLen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + byteLen if postIndex > l { return io.ErrUnexpectedEOF } - m.Error = string(dAtA[iNdEx:postIndex]) + m.EndKey = append(m.EndKey[:0], dAtA[iNdEx:postIndex]...) + if m.EndKey == nil { + m.EndKey = []byte{} + } iNdEx = postIndex default: iNdEx = preIndex @@ -10078,7 +12415,7 @@ func (m *DeleteRangeResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *RawScanRequest) Unmarshal(dAtA []byte) error { +func (m *RawBatchScanRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10101,10 +12438,10 @@ func (m *RawScanRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RawScanRequest: wiretype end group for non-group") + return fmt.Errorf("proto: RawBatchScanRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RawScanRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawBatchScanRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -10142,9 +12479,9 @@ func (m *RawScanRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field StartKey", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Ranges", wireType) } - var byteLen int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -10154,28 +12491,28 @@ func (m *RawScanRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - byteLen |= (int(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - if byteLen < 0 { + if msglen < 0 { return ErrInvalidLengthKvrpcpb } - postIndex := iNdEx + byteLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.StartKey = append(m.StartKey[:0], dAtA[iNdEx:postIndex]...) - if m.StartKey == nil { - m.StartKey = []byte{} + m.Ranges = append(m.Ranges, &KeyRange{}) + if err := m.Ranges[len(m.Ranges)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } iNdEx = postIndex case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field EachLimit", wireType) } - m.Limit = 0 + m.EachLimit = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -10185,11 +12522,31 @@ func (m *RawScanRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Limit |= (uint32(b) & 0x7F) << shift + m.EachLimit |= (uint32(b) & 0x7F) << shift if b < 0x80 { break } } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeyOnly", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.KeyOnly = bool(v != 0) default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -10211,7 +12568,7 @@ func (m *RawScanRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *RawScanResponse) Unmarshal(dAtA []byte) error { +func (m *RawBatchScanResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10234,10 +12591,10 @@ func (m *RawScanResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RawScanResponse: wiretype end group for non-group") + return fmt.Errorf("proto: RawBatchScanResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RawScanResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RawBatchScanResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -10325,7 +12682,7 @@ func (m *RawScanResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *WriteInfo) Unmarshal(dAtA []byte) error { +func (m *MvccWrite) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10348,17 +12705,17 @@ func (m *WriteInfo) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: WriteInfo: wiretype end group for non-group") + return fmt.Errorf("proto: MvccWrite: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: WriteInfo: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: MvccWrite: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StartTs", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) } - m.StartTs = 0 + m.Type = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -10368,16 +12725,16 @@ func (m *WriteInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.StartTs |= (uint64(b) & 0x7F) << shift + m.Type |= (Op(b) & 0x7F) << shift if b < 0x80 { break } } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartTs", wireType) } - m.Type = 0 + m.StartTs = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -10387,7 +12744,7 @@ func (m *WriteInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (Op(b) & 0x7F) << shift + m.StartTs |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } @@ -10411,6 +12768,37 @@ func (m *WriteInfo) Unmarshal(dAtA []byte) error { break } } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShortValue", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ShortValue = append(m.ShortValue[:0], dAtA[iNdEx:postIndex]...) + if m.ShortValue == nil { + m.ShortValue = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -10432,7 +12820,7 @@ func (m *WriteInfo) Unmarshal(dAtA []byte) error { } return nil } -func (m *ValueInfo) Unmarshal(dAtA []byte) error { +func (m *MvccValue) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10455,13 +12843,32 @@ func (m *ValueInfo) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ValueInfo: wiretype end group for non-group") + return fmt.Errorf("proto: MvccValue: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ValueInfo: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: MvccValue: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartTs", wireType) + } + m.StartTs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StartTs |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) } @@ -10492,11 +12899,80 @@ func (m *ValueInfo) Unmarshal(dAtA []byte) error { m.Value = []byte{} } iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipKvrpcpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthKvrpcpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MvccLock) 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 ErrIntOverflowKvrpcpb + } + 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: MvccLock: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MvccLock: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= (Op(b) & 0x7F) << shift + if b < 0x80 { + break + } + } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Ts", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartTs", wireType) } - m.Ts = 0 + m.StartTs = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -10506,16 +12982,16 @@ func (m *ValueInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Ts |= (uint64(b) & 0x7F) << shift + m.StartTs |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IsShortValue", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Primary", wireType) } - var v int + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowKvrpcpb @@ -10525,12 +13001,54 @@ func (m *ValueInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Primary = append(m.Primary[:0], dAtA[iNdEx:postIndex]...) + if m.Primary == nil { + m.Primary = []byte{} + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ShortValue", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - m.IsShortValue = bool(v != 0) + if byteLen < 0 { + return ErrInvalidLengthKvrpcpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ShortValue = append(m.ShortValue[:0], dAtA[iNdEx:postIndex]...) + if m.ShortValue == nil { + m.ShortValue = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -10608,7 +13126,7 @@ func (m *MvccInfo) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Lock == nil { - m.Lock = &LockInfo{} + m.Lock = &MvccLock{} } if err := m.Lock.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -10640,7 +13158,7 @@ func (m *MvccInfo) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Writes = append(m.Writes, &WriteInfo{}) + m.Writes = append(m.Writes, &MvccWrite{}) if err := m.Writes[len(m.Writes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10671,7 +13189,7 @@ func (m *MvccInfo) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Values = append(m.Values, &ValueInfo{}) + m.Values = append(m.Values, &MvccValue{}) if err := m.Values[len(m.Values)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11605,117 +14123,126 @@ var ( func init() { proto.RegisterFile("kvrpcpb.proto", fileDescriptorKvrpcpb) } var fileDescriptorKvrpcpb = []byte{ - // 1784 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x39, 0xdd, 0x6e, 0x23, 0x49, - 0xd5, 0xd3, 0xed, 0xb6, 0xdd, 0x3e, 0x76, 0x1c, 0x6f, 0x25, 0xfb, 0x8d, 0x77, 0xe7, 0xdb, 0x99, - 0x6c, 0x31, 0xa3, 0xc9, 0x06, 0x91, 0x11, 0xd9, 0x15, 0x97, 0x08, 0x4d, 0x66, 0x98, 0xcd, 0x66, - 0xc2, 0x44, 0x95, 0x68, 0x56, 0x23, 0x01, 0x4d, 0xa7, 0x5d, 0xb1, 0x4b, 0x6e, 0x77, 0xf5, 0x74, - 0x95, 0x9d, 0xb4, 0x10, 0xe2, 0x02, 0x81, 0xc4, 0x25, 0x08, 0x09, 0x2e, 0xe0, 0x82, 0x4b, 0x9e, - 0x81, 0x17, 0xe0, 0x92, 0x47, 0x40, 0xc3, 0x8b, 0xa0, 0xaa, 0xea, 0x1f, 0x77, 0x9c, 0xc0, 0xc8, - 0xf2, 0xe6, 0xca, 0x55, 0xe7, 0x9c, 0x3a, 0xff, 0x3f, 0x5d, 0x65, 0x58, 0x1b, 0xcf, 0x92, 0x38, - 0x88, 0xcf, 0x76, 0xe3, 0x84, 0x4b, 0x8e, 0x9a, 0xd9, 0xf6, 0xe3, 0xce, 0x84, 0x4a, 0x3f, 0x07, - 0x7f, 0xbc, 0x46, 0x93, 0x84, 0x27, 0xc5, 0x76, 0x73, 0xc8, 0x87, 0x5c, 0x2f, 0x9f, 0xa8, 0x95, - 0x81, 0xe2, 0x5f, 0x80, 0xfb, 0x92, 0x07, 0xe3, 0x83, 0xe8, 0x9c, 0xa3, 0x4f, 0xa1, 0x13, 0x27, - 0x6c, 0xe2, 0x27, 0xa9, 0x17, 0xf2, 0x60, 0xdc, 0xb7, 0xb6, 0xac, 0xed, 0x0e, 0x69, 0x67, 0x30, - 0x45, 0xa6, 0x48, 0x14, 0xca, 0x9b, 0xd1, 0x44, 0x30, 0x1e, 0xf5, 0xed, 0x2d, 0x6b, 0xdb, 0x21, - 0x6d, 0x05, 0x7b, 0x6d, 0x40, 0xa8, 0x07, 0xb5, 0x31, 0x4d, 0xfb, 0x35, 0x7d, 0x58, 0x2d, 0xd1, - 0x47, 0xe0, 0xea, 0x43, 0x52, 0x86, 0x7d, 0x47, 0x1f, 0x68, 0xaa, 0xfd, 0xa9, 0x0c, 0x31, 0x03, - 0xf7, 0x90, 0xa6, 0xcf, 0x95, 0xa2, 0xe8, 0x33, 0x68, 0x28, 0x30, 0x1d, 0x68, 0xc1, 0xed, 0xbd, - 0x0f, 0x76, 0x73, 0x33, 0x73, 0x0d, 0x49, 0x46, 0x80, 0xfe, 0x1f, 0x5a, 0x09, 0x95, 0x49, 0xea, - 0x9f, 0x85, 0x54, 0xeb, 0xd0, 0x22, 0x25, 0x00, 0x6d, 0x42, 0xdd, 0x3f, 0xe3, 0x89, 0xd4, 0x3a, - 0xb4, 0x88, 0xd9, 0xe0, 0xbf, 0xd4, 0xa0, 0xb9, 0xcf, 0x23, 0x49, 0x2f, 0x25, 0xba, 0xa7, 0xce, - 0x0f, 0x19, 0x8f, 0x3c, 0x66, 0xa4, 0x39, 0xc4, 0x35, 0x80, 0x83, 0x01, 0xfa, 0x1e, 0x74, 0x32, - 0x24, 0x8d, 0x79, 0x30, 0xd2, 0xfc, 0xdb, 0x7b, 0x1b, 0xbb, 0x99, 0x73, 0x89, 0xc6, 0x3d, 0x57, - 0x28, 0xd2, 0x4e, 0xca, 0x0d, 0xda, 0x02, 0x27, 0xa6, 0x34, 0xd1, 0x52, 0xdb, 0x7b, 0x9d, 0x9c, - 0xfe, 0x98, 0xd2, 0x84, 0x68, 0x0c, 0x42, 0xe0, 0x48, 0x9a, 0x4c, 0xfa, 0x75, 0x2d, 0x51, 0xaf, - 0xd1, 0x13, 0x70, 0xe3, 0x84, 0xf1, 0x84, 0xc9, 0xb4, 0xdf, 0xd8, 0xb2, 0xb6, 0xbb, 0x7b, 0x1b, - 0x85, 0xdd, 0xfb, 0x7c, 0x32, 0xf1, 0xa3, 0xc1, 0x71, 0xc2, 0x48, 0x41, 0x84, 0x7e, 0x00, 0xeb, - 0x4c, 0xf0, 0xd0, 0x97, 0x4a, 0xc3, 0x90, 0xce, 0x68, 0xd8, 0x6f, 0xea, 0x73, 0x77, 0x8b, 0x73, - 0x07, 0x39, 0xfe, 0xa5, 0x42, 0x93, 0x2e, 0xab, 0xec, 0xd1, 0x43, 0xe8, 0x46, 0x5c, 0x7a, 0xe7, - 0x2c, 0x0c, 0xbd, 0xc0, 0x0f, 0x46, 0xb4, 0xef, 0x6e, 0x59, 0xdb, 0x2e, 0xe9, 0x44, 0x5c, 0xfe, - 0x90, 0x85, 0xe1, 0xbe, 0x82, 0xa9, 0xa8, 0x89, 0x34, 0x0a, 0xbc, 0x90, 0x0f, 0xfb, 0x2d, 0x8d, - 0x6f, 0xaa, 0xfd, 0x4b, 0x3e, 0x44, 0x0f, 0xa0, 0x3d, 0xf2, 0xa3, 0x41, 0x48, 0x3d, 0xc9, 0x26, - 0xb4, 0x0f, 0x1a, 0x0b, 0x06, 0x74, 0xca, 0x26, 0x54, 0x11, 0x88, 0xc0, 0x8f, 0xbc, 0x01, 0x95, - 0x3e, 0x0b, 0xfb, 0x6d, 0x43, 0xa0, 0x40, 0xcf, 0x34, 0xe4, 0x2b, 0xc7, 0x75, 0x7a, 0x75, 0xe5, - 0x3e, 0x7f, 0xe0, 0xbd, 0x9d, 0xf2, 0x64, 0x3a, 0xc1, 0xcf, 0x00, 0xbe, 0x2c, 0x39, 0xdc, 0x85, - 0xe6, 0x85, 0xcf, 0xa4, 0x37, 0x11, 0x3a, 0x3e, 0x35, 0xd2, 0x50, 0xdb, 0x23, 0x81, 0x3e, 0x01, - 0x88, 0x13, 0x1e, 0x50, 0x21, 0x14, 0xce, 0xd6, 0xb8, 0x56, 0x06, 0x39, 0x12, 0xf8, 0xfb, 0xe0, - 0x9e, 0x04, 0x7e, 0xa4, 0xf3, 0x79, 0x13, 0xea, 0x92, 0x4b, 0x3f, 0xcc, 0x38, 0x98, 0x8d, 0xca, - 0x9d, 0x8c, 0x9c, 0x0e, 0xae, 0x9c, 0xa7, 0x03, 0xfc, 0x2b, 0x0b, 0xe0, 0xa4, 0xd0, 0x13, 0x3d, - 0x86, 0xfa, 0x45, 0xc2, 0x24, 0x5d, 0x48, 0xc9, 0x5c, 0x08, 0x31, 0x78, 0xf4, 0x08, 0x1c, 0x5d, - 0x33, 0xf6, 0x4d, 0x74, 0x1a, 0xad, 0xc8, 0x06, 0xbe, 0xf4, 0xb3, 0x1c, 0xb9, 0x8e, 0x4c, 0xa1, - 0x71, 0x0a, 0xed, 0xe7, 0x97, 0x34, 0x30, 0x4a, 0x08, 0xf4, 0x45, 0xd5, 0xdf, 0x56, 0x96, 0x90, - 0xf9, 0xe1, 0xd2, 0x6d, 0x95, 0x20, 0x7c, 0x51, 0x0d, 0x82, 0x7d, 0xe5, 0x54, 0x69, 0xe5, 0x7c, - 0x64, 0xf0, 0x00, 0xe0, 0x05, 0x95, 0x84, 0xbe, 0x9d, 0x52, 0x21, 0xd1, 0x0e, 0x34, 0x03, 0x53, - 0x33, 0x99, 0xd4, 0xde, 0x5c, 0x72, 0x6a, 0x38, 0xc9, 0x09, 0xf2, 0xc2, 0xb7, 0xcb, 0xc2, 0xef, - 0x43, 0x33, 0x6f, 0x14, 0x35, 0x53, 0xf7, 0xd9, 0x16, 0xff, 0x12, 0xda, 0x5a, 0x8a, 0x88, 0x79, - 0x24, 0x28, 0xfa, 0x6e, 0x59, 0x72, 0xaa, 0x15, 0x64, 0xb2, 0xba, 0xbb, 0x79, 0x07, 0xd3, 0x0d, - 0xa2, 0xa8, 0x36, 0xdd, 0x2d, 0x1e, 0x43, 0xdd, 0xd0, 0x5e, 0xf5, 0x78, 0xde, 0x4f, 0x88, 0xc1, - 0xab, 0x2c, 0x98, 0xf9, 0xe1, 0x94, 0x66, 0x1d, 0xc9, 0x6c, 0xf0, 0x5f, 0x2d, 0x68, 0x2b, 0x0f, - 0x2c, 0x63, 0xe8, 0x3d, 0x68, 0x09, 0xe9, 0x27, 0xd2, 0x2b, 0xcd, 0x75, 0x35, 0xe0, 0x90, 0xa6, - 0x4a, 0x5c, 0xc8, 0x26, 0xcc, 0x34, 0x9f, 0x35, 0x62, 0x36, 0xf3, 0x9e, 0x70, 0x2a, 0x9e, 0x50, - 0x65, 0x36, 0xa6, 0xa9, 0xc7, 0xa3, 0x30, 0xd5, 0x7d, 0xc1, 0x25, 0xcd, 0x31, 0x4d, 0x5f, 0x45, - 0x61, 0x8a, 0xdf, 0x40, 0xe3, 0x70, 0x76, 0xec, 0xb3, 0x39, 0x63, 0xad, 0xff, 0x61, 0xec, 0x62, - 0x0c, 0xae, 0x37, 0x7f, 0x04, 0x1d, 0x63, 0xfd, 0xf2, 0x01, 0x78, 0x04, 0xf5, 0xd8, 0x67, 0x89, - 0xaa, 0xc1, 0xda, 0x76, 0x7b, 0x6f, 0xbd, 0xd4, 0x49, 0xeb, 0x4c, 0x0c, 0x16, 0xbf, 0x02, 0xf7, - 0x68, 0x2a, 0x75, 0xf7, 0x41, 0xf7, 0xc0, 0xe6, 0xb1, 0xe6, 0xdd, 0xdd, 0x6b, 0x17, 0xf4, 0xaf, - 0x62, 0x62, 0xf3, 0xf8, 0xbd, 0x55, 0xff, 0xb5, 0x0d, 0xeb, 0xc7, 0x09, 0xd5, 0x65, 0xb7, 0x4c, - 0xf4, 0x9e, 0x40, 0x6b, 0x92, 0x29, 0x94, 0xeb, 0x5e, 0xfa, 0x33, 0x57, 0x95, 0x94, 0x34, 0x0b, - 0x63, 0xb1, 0xb6, 0x38, 0x16, 0xbf, 0x05, 0x6b, 0x26, 0x23, 0xaa, 0x41, 0xee, 0x68, 0xe0, 0xeb, - 0x32, 0xd2, 0xc5, 0x18, 0xac, 0x57, 0xc6, 0x20, 0xda, 0x83, 0x0f, 0xc5, 0x98, 0xc5, 0x5e, 0xc0, - 0x23, 0x21, 0x13, 0x9f, 0x45, 0xd2, 0x0b, 0x46, 0x34, 0x18, 0xeb, 0x89, 0xe0, 0x92, 0x0d, 0x85, - 0xdc, 0x2f, 0x70, 0xfb, 0x0a, 0x85, 0x63, 0xe8, 0x95, 0x6e, 0x58, 0x3e, 0x8c, 0x9f, 0x41, 0x43, - 0x63, 0x17, 0x7d, 0x51, 0xe4, 0x56, 0x46, 0x80, 0xff, 0x66, 0xc1, 0x9a, 0x1a, 0x49, 0x6c, 0xa9, - 0xf6, 0xb0, 0xe0, 0x23, 0xfb, 0x1a, 0x1f, 0x21, 0x70, 0xc6, 0x34, 0x15, 0xfd, 0xda, 0x56, 0x6d, - 0xbb, 0x43, 0xf4, 0x1a, 0x3d, 0x82, 0x6e, 0xa0, 0xa5, 0x5e, 0xf1, 0xee, 0x9a, 0x81, 0x66, 0x47, - 0xbf, 0x72, 0xdc, 0x7a, 0xaf, 0x41, 0x1a, 0x67, 0x2c, 0x0a, 0xf9, 0x10, 0x87, 0xd0, 0xcd, 0x55, - 0xfd, 0xe6, 0x7b, 0x0c, 0x1e, 0xc2, 0xda, 0xc1, 0x24, 0xe6, 0x49, 0xe1, 0x98, 0x4a, 0x92, 0x59, - 0xef, 0x91, 0x64, 0x8b, 0x46, 0xda, 0xd7, 0x18, 0x89, 0xdf, 0x40, 0x37, 0x17, 0xb4, 0xbc, 0x59, - 0x9b, 0xf3, 0x66, 0xb5, 0x72, 0x1b, 0x7e, 0x0e, 0x9b, 0x4f, 0x7d, 0x19, 0x8c, 0x08, 0x0f, 0xc3, - 0x33, 0x3f, 0x18, 0xdf, 0x66, 0x8c, 0xb1, 0x80, 0x0f, 0xaf, 0x08, 0xbf, 0x85, 0xa8, 0x09, 0xe8, - 0xee, 0x87, 0xd4, 0x8f, 0xa6, 0xf1, 0x6a, 0xc6, 0xdd, 0x82, 0xf5, 0xb5, 0x45, 0xeb, 0xf1, 0x1f, - 0x2c, 0x58, 0x2f, 0xa4, 0xde, 0xc2, 0xf8, 0x5b, 0x4c, 0xac, 0xda, 0x75, 0x89, 0x35, 0x86, 0x75, - 0x1d, 0x80, 0x25, 0x67, 0x7f, 0x1e, 0x53, 0x7b, 0xae, 0x6e, 0x6f, 0x9e, 0xfe, 0x21, 0xf4, 0x4a, - 0x61, 0xdf, 0xf8, 0x04, 0xfa, 0x9d, 0x05, 0xeb, 0x6a, 0xd8, 0xa9, 0x4e, 0xbd, 0x8c, 0x6d, 0x0f, - 0xa0, 0x3d, 0xf1, 0x2f, 0xaf, 0xa4, 0x34, 0x4c, 0xfc, 0xcb, 0x3c, 0xa1, 0x2b, 0xdf, 0x03, 0xb5, - 0x9b, 0xbe, 0x07, 0x9c, 0xb9, 0xef, 0x01, 0xfc, 0x47, 0x0b, 0x7a, 0xa5, 0x4e, 0xb7, 0x90, 0x06, - 0x8f, 0xa1, 0xae, 0x86, 0x8d, 0xa9, 0xba, 0x6b, 0xef, 0x56, 0x06, 0x8f, 0x3f, 0x87, 0xe6, 0xe9, - 0xa5, 0xf9, 0x7e, 0xee, 0x41, 0x4d, 0x5e, 0x46, 0xd9, 0xfd, 0x48, 0x2d, 0xd1, 0xff, 0x41, 0x43, - 0x48, 0x5f, 0x4e, 0x45, 0xe6, 0x85, 0x6c, 0x87, 0xff, 0x6e, 0x01, 0x22, 0x54, 0xf0, 0x70, 0x46, - 0x97, 0xf5, 0xf2, 0x7b, 0xb5, 0x8e, 0xf7, 0x4b, 0x66, 0xf4, 0x1d, 0x68, 0xc9, 0xcb, 0xc8, 0x63, - 0xd1, 0x39, 0x17, 0x7d, 0x47, 0x1b, 0x5c, 0x4a, 0xce, 0xac, 0x23, 0xae, 0x34, 0x0b, 0x81, 0xdf, - 0xc2, 0x46, 0x45, 0xf9, 0x5b, 0x68, 0x3d, 0xaf, 0xa1, 0xf5, 0x62, 0x7f, 0x19, 0x37, 0x7d, 0x02, - 0x20, 0xfc, 0x73, 0xea, 0xc5, 0x9c, 0x45, 0x32, 0xf3, 0x51, 0x4b, 0x41, 0x8e, 0x15, 0x00, 0x8f, - 0x00, 0x14, 0xdf, 0x5b, 0xb0, 0xe0, 0x08, 0xd6, 0x88, 0x7f, 0xb1, 0xaa, 0xab, 0x02, 0xe6, 0xd0, - 0xcd, 0xd9, 0xad, 0x78, 0xb0, 0xdd, 0xf0, 0x19, 0x19, 0x68, 0xfd, 0x8f, 0xa7, 0x2b, 0xba, 0xea, - 0x5c, 0x2f, 0xe4, 0x8d, 0xb6, 0x4a, 0x0b, 0x59, 0xf5, 0xb8, 0x3e, 0x86, 0x1e, 0xf1, 0x2f, 0x9e, - 0xd1, 0x90, 0x2e, 0xf7, 0x19, 0xbc, 0x18, 0x82, 0x1f, 0xc3, 0x07, 0x73, 0x1c, 0x57, 0xad, 0xef, - 0x0c, 0x50, 0xc6, 0xda, 0x8f, 0x86, 0x74, 0xe5, 0xd7, 0xae, 0xbb, 0xd0, 0xa4, 0xd1, 0x60, 0xae, - 0x03, 0x37, 0x68, 0x34, 0x38, 0xa4, 0x29, 0xfe, 0x29, 0x6c, 0x54, 0xe4, 0xae, 0xda, 0x2e, 0x93, - 0xb8, 0xb7, 0x77, 0x95, 0xc4, 0x43, 0x58, 0x2f, 0x04, 0x2e, 0x6f, 0xcc, 0xa7, 0x50, 0x1b, 0xcf, - 0x6e, 0x9c, 0x9c, 0x0a, 0x87, 0x07, 0xd0, 0xfa, 0x5a, 0xdd, 0x2e, 0xf4, 0x2c, 0xf8, 0x08, 0x8c, - 0x5e, 0x9e, 0x14, 0xd9, 0x40, 0x68, 0xea, 0xfd, 0xa9, 0x40, 0x0f, 0xc0, 0x91, 0x69, 0x6c, 0xde, - 0xe1, 0xae, 0xdc, 0xeb, 0x34, 0x42, 0x19, 0x99, 0x75, 0x6d, 0x29, 0xb2, 0x86, 0xed, 0x1a, 0xc0, - 0xa9, 0xc0, 0x5f, 0x43, 0xeb, 0xb5, 0xaa, 0x95, 0xfc, 0xc5, 0xc6, 0x54, 0x91, 0x35, 0x57, 0x45, - 0xa8, 0x0b, 0xb6, 0xcc, 0x27, 0x8e, 0x2d, 0x05, 0x7a, 0x08, 0x5d, 0x26, 0x3c, 0x31, 0xe2, 0x6a, - 0x5a, 0x14, 0x45, 0xe7, 0x92, 0x0e, 0x13, 0x27, 0x0a, 0xa8, 0xf9, 0xe1, 0xdf, 0x5a, 0xe0, 0x1e, - 0xcd, 0x82, 0x40, 0x33, 0xce, 0x9f, 0x67, 0x6e, 0x7c, 0x59, 0x34, 0xcf, 0x33, 0x3b, 0xd0, 0xd0, - 0x17, 0xaa, 0xdc, 0x31, 0xa8, 0x20, 0x2c, 0x3c, 0x41, 0x32, 0x0a, 0x45, 0xab, 0x85, 0xe7, 0x23, - 0xb5, 0xa4, 0x2d, 0xec, 0x21, 0x19, 0x05, 0x3e, 0x81, 0x0d, 0xa5, 0xca, 0x0b, 0x2a, 0x9f, 0xa6, - 0x87, 0x34, 0x5d, 0x4d, 0xbd, 0xfe, 0xc6, 0x82, 0xcd, 0x2a, 0xd7, 0x55, 0x77, 0xce, 0x47, 0xe0, - 0xa8, 0x19, 0xba, 0xf0, 0x5a, 0x95, 0xbb, 0x95, 0x68, 0x34, 0xfe, 0x19, 0xdc, 0x2d, 0xf4, 0x38, - 0x31, 0x49, 0xb1, 0x8c, 0x85, 0xf3, 0x29, 0x66, 0x57, 0x52, 0x0c, 0xff, 0xd9, 0x82, 0xfe, 0xa2, - 0x88, 0x55, 0x9b, 0xbb, 0xf8, 0x72, 0x9d, 0x3b, 0xc0, 0xf9, 0xef, 0x0e, 0xf8, 0x09, 0xa0, 0x93, - 0x38, 0x54, 0x77, 0x4d, 0x25, 0x62, 0xd9, 0x3e, 0xa0, 0x38, 0x54, 0xfa, 0x80, 0x02, 0xa8, 0x16, - 0xf6, 0x7b, 0x0b, 0x36, 0x2a, 0xfc, 0x97, 0x37, 0x1c, 0x83, 0x13, 0xd2, 0x73, 0x99, 0x4d, 0xf7, - 0x6e, 0xf5, 0x4d, 0x9b, 0x68, 0x1c, 0x7a, 0x08, 0xf5, 0x84, 0x0d, 0x47, 0x32, 0x0b, 0xfb, 0x55, - 0x22, 0x83, 0xdc, 0xf9, 0x36, 0x40, 0xf9, 0x3c, 0x8d, 0x00, 0x1a, 0x3f, 0xe2, 0xc9, 0xc4, 0x0f, - 0x7b, 0x77, 0x50, 0x13, 0x6a, 0x2f, 0xf9, 0x45, 0xcf, 0x42, 0x2e, 0x38, 0x5f, 0xb2, 0xe1, 0xa8, - 0x67, 0xef, 0x6c, 0x41, 0xb7, 0xfa, 0x26, 0x8d, 0x1a, 0x60, 0x9f, 0x1c, 0xf4, 0xee, 0xa8, 0x5f, - 0xb2, 0xdf, 0xb3, 0x76, 0x76, 0xc1, 0x7e, 0x15, 0xab, 0xa3, 0xc7, 0x53, 0x69, 0x78, 0x3c, 0xa3, - 0xa1, 0xe1, 0xa1, 0x6a, 0xb4, 0x67, 0xa3, 0x0e, 0xb8, 0xf9, 0xed, 0xb0, 0x57, 0x7b, 0xba, 0xf3, - 0x8f, 0x77, 0xf7, 0xad, 0x7f, 0xbe, 0xbb, 0x6f, 0xfd, 0xeb, 0xdd, 0x7d, 0xeb, 0x4f, 0xff, 0xbe, - 0x7f, 0x07, 0xfa, 0x01, 0x9f, 0xec, 0xc6, 0x2c, 0x1a, 0x06, 0x7e, 0xbc, 0x2b, 0xd9, 0x78, 0xb6, - 0x3b, 0x9e, 0xe9, 0xff, 0x38, 0xce, 0x1a, 0xfa, 0xe7, 0xf3, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, - 0x03, 0x7e, 0xb6, 0x04, 0x37, 0x19, 0x00, 0x00, + // 1922 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0x51, 0x8f, 0x1b, 0x49, + 0x11, 0xce, 0xd8, 0x63, 0x7b, 0x5c, 0xf6, 0x7a, 0x7d, 0xbd, 0x9b, 0x8b, 0xb9, 0x70, 0xc9, 0xde, + 0x70, 0x51, 0x36, 0x8b, 0xd8, 0x88, 0xbd, 0x13, 0x4f, 0x08, 0x9d, 0xb2, 0x09, 0xb9, 0x5c, 0x76, + 0xc9, 0xaa, 0x77, 0x95, 0x53, 0x24, 0xc0, 0xf4, 0x8e, 0x7b, 0xed, 0x91, 0xc7, 0xd3, 0x93, 0x99, + 0xb6, 0x77, 0x47, 0x08, 0x21, 0x84, 0x0e, 0x09, 0xde, 0x40, 0x48, 0xf0, 0x00, 0x0f, 0xbc, 0x20, + 0xf1, 0x1b, 0xf8, 0x03, 0x3c, 0xf2, 0x13, 0x50, 0xf8, 0x23, 0xa8, 0x6b, 0x7a, 0x3c, 0x1e, 0xdb, + 0x4b, 0x56, 0x83, 0xb3, 0x3c, 0x79, 0xba, 0xaa, 0xba, 0xab, 0xbe, 0xaa, 0xea, 0xaa, 0xee, 0x36, + 0xac, 0x0d, 0x27, 0x61, 0xe0, 0x04, 0xa7, 0xbb, 0x41, 0x28, 0xa4, 0x20, 0x35, 0x3d, 0xfc, 0xa0, + 0x39, 0xe2, 0x92, 0xa5, 0xe4, 0x0f, 0xd6, 0x78, 0x18, 0x8a, 0x70, 0x3a, 0xdc, 0xec, 0x8b, 0xbe, + 0xc0, 0xcf, 0x87, 0xea, 0x2b, 0xa1, 0xda, 0x3f, 0x03, 0xeb, 0x40, 0x38, 0xc3, 0x67, 0xfe, 0x99, + 0x20, 0x1f, 0x41, 0x33, 0x08, 0xdd, 0x11, 0x0b, 0xe3, 0xae, 0x27, 0x9c, 0x61, 0xc7, 0xd8, 0x32, + 0xb6, 0x9b, 0xb4, 0xa1, 0x69, 0x4a, 0x4c, 0x89, 0x28, 0x56, 0x77, 0xc2, 0xc3, 0xc8, 0x15, 0x7e, + 0xa7, 0xb4, 0x65, 0x6c, 0x9b, 0xb4, 0xa1, 0x68, 0x2f, 0x13, 0x12, 0x69, 0x43, 0x79, 0xc8, 0xe3, + 0x4e, 0x19, 0x27, 0xab, 0x4f, 0xf2, 0x35, 0xb0, 0x70, 0x92, 0x94, 0x5e, 0xc7, 0xc4, 0x09, 0x35, + 0x35, 0x3e, 0x91, 0x9e, 0xed, 0x82, 0xf5, 0x9c, 0xc7, 0x4f, 0x94, 0xa1, 0xe4, 0x01, 0x54, 0x15, + 0x99, 0xf7, 0x50, 0x71, 0x63, 0xef, 0xbd, 0xdd, 0x14, 0x66, 0x6a, 0x21, 0xd5, 0x02, 0xe4, 0xeb, + 0x50, 0x0f, 0xb9, 0x0c, 0x63, 0x76, 0xea, 0x71, 0xb4, 0xa1, 0x4e, 0x33, 0x02, 0xd9, 0x84, 0x0a, + 0x3b, 0x15, 0xa1, 0x44, 0x1b, 0xea, 0x34, 0x19, 0xd8, 0x7f, 0x2e, 0x43, 0x6d, 0x5f, 0xf8, 0x92, + 0x5f, 0x48, 0x72, 0x5b, 0xcd, 0xef, 0xbb, 0xc2, 0xef, 0xba, 0x89, 0x36, 0x93, 0x5a, 0x09, 0xe1, + 0x59, 0x8f, 0x7c, 0x07, 0x9a, 0x9a, 0xc9, 0x03, 0xe1, 0x0c, 0x70, 0xfd, 0xc6, 0xde, 0xc6, 0xae, + 0x76, 0x2e, 0x45, 0xde, 0x13, 0xc5, 0xa2, 0x8d, 0x30, 0x1b, 0x90, 0x2d, 0x30, 0x03, 0xce, 0x43, + 0xd4, 0xda, 0xd8, 0x6b, 0xa6, 0xf2, 0x47, 0x9c, 0x87, 0x14, 0x39, 0x84, 0x80, 0x29, 0x79, 0x38, + 0xea, 0x54, 0x50, 0x23, 0x7e, 0x93, 0x87, 0x60, 0x05, 0xa1, 0x2b, 0x42, 0x57, 0xc6, 0x9d, 0xea, + 0x96, 0xb1, 0xdd, 0xda, 0xdb, 0x98, 0xe2, 0xde, 0x17, 0xa3, 0x11, 0xf3, 0x7b, 0x47, 0xa1, 0x4b, + 0xa7, 0x42, 0xe4, 0x33, 0x58, 0x77, 0x23, 0xe1, 0x31, 0xa9, 0x2c, 0xf4, 0xf8, 0x84, 0x7b, 0x9d, + 0x1a, 0xce, 0xbb, 0x35, 0x9d, 0xf7, 0x2c, 0xe5, 0x1f, 0x28, 0x36, 0x6d, 0xb9, 0xb9, 0x31, 0xf9, + 0x18, 0x5a, 0xbe, 0x90, 0xdd, 0x33, 0xd7, 0xf3, 0xba, 0x0e, 0x73, 0x06, 0xbc, 0x63, 0x6d, 0x19, + 0xdb, 0x16, 0x6d, 0xfa, 0x42, 0x7e, 0xdf, 0xf5, 0xbc, 0x7d, 0x45, 0x53, 0x51, 0x8b, 0x62, 0xdf, + 0xe9, 0x7a, 0xa2, 0xdf, 0xa9, 0x23, 0xbf, 0xa6, 0xc6, 0x07, 0xa2, 0x4f, 0xee, 0x42, 0x63, 0xc0, + 0xfc, 0x9e, 0xc7, 0xbb, 0xd2, 0x1d, 0xf1, 0x0e, 0x20, 0x17, 0x12, 0xd2, 0x89, 0x3b, 0xe2, 0x4a, + 0x20, 0x72, 0x98, 0xdf, 0xed, 0x71, 0xc9, 0x5c, 0xaf, 0xd3, 0x48, 0x04, 0x14, 0xe9, 0x31, 0x52, + 0xbe, 0x30, 0x2d, 0xb3, 0x5d, 0x51, 0xee, 0x63, 0xbd, 0xee, 0xeb, 0xb1, 0x08, 0xc7, 0x23, 0xfb, + 0x31, 0xc0, 0xe7, 0xd9, 0x0a, 0xb7, 0xa0, 0x76, 0xce, 0x5c, 0xd9, 0x1d, 0x45, 0x18, 0x9f, 0x32, + 0xad, 0xaa, 0xe1, 0x61, 0x44, 0x3e, 0x04, 0x08, 0x42, 0xe1, 0xf0, 0x28, 0x52, 0xbc, 0x12, 0xf2, + 0xea, 0x9a, 0x72, 0x18, 0xd9, 0xdf, 0x03, 0xeb, 0xd8, 0x61, 0x3e, 0xe6, 0xf3, 0x26, 0x54, 0xa4, + 0x90, 0xcc, 0xd3, 0x2b, 0x24, 0x03, 0x95, 0x3b, 0x5a, 0x9c, 0xf7, 0xe6, 0xe6, 0xf3, 0x9e, 0xfd, + 0x4b, 0x03, 0xe0, 0x78, 0x6a, 0x27, 0xb9, 0x0f, 0x95, 0xf3, 0xd0, 0x95, 0x7c, 0x21, 0x25, 0x53, + 0x25, 0x34, 0xe1, 0x93, 0x7b, 0x60, 0xe2, 0x9e, 0x29, 0x5d, 0x26, 0x87, 0x6c, 0x25, 0xd6, 0x63, + 0x92, 0xe9, 0x1c, 0x59, 0x26, 0xa6, 0xd8, 0x76, 0x0c, 0x8d, 0x27, 0x17, 0xdc, 0x49, 0x8c, 0x88, + 0xc8, 0xa7, 0x79, 0x7f, 0x1b, 0x3a, 0x21, 0xd3, 0xc9, 0x99, 0xdb, 0x72, 0x41, 0xf8, 0x34, 0x1f, + 0x84, 0xd2, 0xdc, 0xac, 0x0c, 0xe5, 0x6c, 0x64, 0xec, 0x1e, 0xc0, 0x53, 0x2e, 0x29, 0x7f, 0x3d, + 0xe6, 0x91, 0x24, 0x3b, 0x50, 0x73, 0x92, 0x3d, 0xa3, 0xb5, 0xb6, 0x67, 0x92, 0x13, 0xe9, 0x34, + 0x15, 0x48, 0x37, 0x7e, 0x29, 0xdb, 0xf8, 0x1d, 0xa8, 0xa5, 0x85, 0xa2, 0x9c, 0xec, 0x7b, 0x3d, + 0xb4, 0x7f, 0x0e, 0x0d, 0xd4, 0x12, 0x05, 0xc2, 0x8f, 0x38, 0xf9, 0x76, 0xb6, 0xe5, 0x54, 0x29, + 0xd0, 0xba, 0x5a, 0xbb, 0x69, 0x05, 0xc3, 0x02, 0x31, 0xdd, 0x6d, 0x58, 0x2d, 0xee, 0x43, 0x25, + 0x91, 0x9d, 0xf7, 0x78, 0x5a, 0x4f, 0x68, 0xc2, 0x57, 0x59, 0x30, 0x61, 0xde, 0x98, 0xeb, 0x8a, + 0x94, 0x0c, 0xec, 0xbf, 0x18, 0xd0, 0x50, 0x1e, 0x28, 0x02, 0xf4, 0x36, 0xd4, 0x23, 0xc9, 0x42, + 0xd9, 0xcd, 0xe0, 0x5a, 0x48, 0x78, 0xce, 0x63, 0xa5, 0xce, 0x73, 0x47, 0x6e, 0x52, 0x7c, 0xd6, + 0x68, 0x32, 0x98, 0xf5, 0x84, 0x99, 0xf3, 0x84, 0xda, 0x66, 0x43, 0x1e, 0x77, 0x85, 0xef, 0xc5, + 0x58, 0x17, 0x2c, 0x5a, 0x1b, 0xf2, 0xf8, 0x85, 0xef, 0xc5, 0xf6, 0x2b, 0xa8, 0x3e, 0x9f, 0x1c, + 0x31, 0x77, 0x06, 0xac, 0xf1, 0x16, 0xb0, 0x8b, 0x31, 0x58, 0x0e, 0x7f, 0x00, 0xcd, 0x04, 0x7d, + 0xf1, 0x00, 0xdc, 0x83, 0x4a, 0xc0, 0xdc, 0x50, 0xed, 0xc1, 0xf2, 0x76, 0x63, 0x6f, 0x3d, 0xb3, + 0x09, 0x6d, 0xa6, 0x09, 0xd7, 0x7e, 0x01, 0xd6, 0xe1, 0x58, 0x62, 0xf5, 0x21, 0xb7, 0xa1, 0x24, + 0x02, 0x5c, 0xbb, 0xb5, 0xd7, 0x98, 0xca, 0xbf, 0x08, 0x68, 0x49, 0x04, 0x57, 0x36, 0xfd, 0xab, + 0x12, 0xac, 0x1f, 0x85, 0x1c, 0xb7, 0x5d, 0x91, 0xe8, 0x3d, 0x84, 0xfa, 0x48, 0x1b, 0x94, 0xda, + 0x9e, 0xf9, 0x33, 0x35, 0x95, 0x66, 0x32, 0x0b, 0x6d, 0xb1, 0xbc, 0xd8, 0x16, 0xbf, 0x01, 0x6b, + 0x49, 0x46, 0xe4, 0x83, 0xdc, 0x44, 0xe2, 0xcb, 0x2c, 0xd2, 0xd3, 0x36, 0x58, 0xc9, 0xb5, 0x41, + 0xb2, 0x07, 0x37, 0xa3, 0xa1, 0x1b, 0x74, 0x1d, 0xe1, 0x47, 0x32, 0x64, 0xae, 0x2f, 0xbb, 0xce, + 0x80, 0x3b, 0x43, 0xec, 0x08, 0x16, 0xdd, 0x50, 0xcc, 0xfd, 0x29, 0x6f, 0x5f, 0xb1, 0xec, 0x00, + 0xda, 0x99, 0x1b, 0x8a, 0x87, 0xf1, 0x01, 0x54, 0x91, 0xbb, 0xe8, 0x8b, 0x69, 0x6e, 0x69, 0x01, + 0xfb, 0x6f, 0x06, 0xac, 0xa9, 0x96, 0xe4, 0x16, 0x2a, 0x0f, 0x0b, 0x3e, 0x2a, 0x2d, 0xf1, 0x11, + 0x01, 0x73, 0xc8, 0xe3, 0xa8, 0x53, 0xde, 0x2a, 0x6f, 0x37, 0x29, 0x7e, 0x93, 0x7b, 0xd0, 0x72, + 0x50, 0xeb, 0x9c, 0x77, 0xd7, 0x12, 0xaa, 0x9e, 0xfa, 0x85, 0x69, 0x55, 0xda, 0x55, 0x5a, 0x3d, + 0x75, 0x7d, 0x4f, 0xf4, 0x6d, 0x0f, 0x5a, 0xa9, 0xa9, 0xef, 0xbe, 0xc6, 0xd8, 0x7d, 0x58, 0x7b, + 0x36, 0x0a, 0x44, 0x38, 0x75, 0x4c, 0x2e, 0xc9, 0x8c, 0x2b, 0x24, 0xd9, 0x22, 0xc8, 0xd2, 0x12, + 0x90, 0xf6, 0x2b, 0x68, 0xa5, 0x8a, 0x8a, 0xc3, 0xda, 0x9c, 0x85, 0x55, 0x4f, 0x31, 0xfc, 0x14, + 0x36, 0x1f, 0x31, 0xe9, 0x0c, 0xa8, 0xf0, 0xbc, 0x53, 0xe6, 0x0c, 0xaf, 0x33, 0xc6, 0x76, 0x04, + 0x37, 0xe7, 0x94, 0x5f, 0x43, 0xd4, 0x22, 0x68, 0xed, 0x7b, 0x9c, 0xf9, 0xe3, 0x60, 0x35, 0xed, + 0x6e, 0x01, 0x7d, 0x79, 0x11, 0xbd, 0xfd, 0x7b, 0x03, 0xd6, 0xa7, 0x5a, 0xaf, 0xa1, 0xfd, 0x2d, + 0x26, 0x56, 0x79, 0x59, 0x62, 0x0d, 0x61, 0x1d, 0x03, 0x50, 0xb0, 0xf7, 0xa7, 0x31, 0x2d, 0xcd, + 0xec, 0xdb, 0xcb, 0xbb, 0xbf, 0x07, 0xed, 0x4c, 0xd9, 0x3b, 0xef, 0x40, 0xbf, 0x35, 0x60, 0x5d, + 0x35, 0x3b, 0x55, 0xa9, 0x8b, 0x60, 0xbb, 0x0b, 0x8d, 0x11, 0xbb, 0x98, 0x4b, 0x69, 0x18, 0xb1, + 0x8b, 0x34, 0xa1, 0x73, 0xe7, 0x81, 0xf2, 0x65, 0xe7, 0x01, 0x73, 0xe6, 0x3c, 0x60, 0xff, 0xc1, + 0x80, 0x76, 0x66, 0xd3, 0x35, 0xa4, 0xc1, 0x7d, 0xa8, 0xa8, 0x66, 0x93, 0xec, 0xba, 0xa5, 0x77, + 0xab, 0x84, 0x6f, 0x7f, 0x02, 0xb5, 0x93, 0x8b, 0xe4, 0xfc, 0xdc, 0x86, 0xb2, 0xbc, 0xf0, 0xf5, + 0xfd, 0x48, 0x7d, 0x92, 0xf7, 0xa1, 0x1a, 0x49, 0x26, 0xc7, 0x91, 0xf6, 0x82, 0x1e, 0xd9, 0x7f, + 0x37, 0x80, 0x50, 0x1e, 0x09, 0x6f, 0xc2, 0x8b, 0x7a, 0xf9, 0x4a, 0xa5, 0xe3, 0x6a, 0xc9, 0x4c, + 0xbe, 0x05, 0x75, 0x79, 0xe1, 0x77, 0x5d, 0xff, 0x4c, 0x44, 0x1d, 0x13, 0x01, 0x67, 0x9a, 0x35, + 0x3a, 0x6a, 0xc9, 0xe4, 0x23, 0xb2, 0x5f, 0xc3, 0x46, 0xce, 0xf8, 0x6b, 0x28, 0x3d, 0x2f, 0xa1, + 0xfe, 0x74, 0xbf, 0x88, 0x9b, 0x3e, 0x04, 0x88, 0xd8, 0x19, 0xef, 0x06, 0xc2, 0xf5, 0xa5, 0xf6, + 0x51, 0x5d, 0x51, 0x8e, 0x14, 0xc1, 0x1e, 0x00, 0xa8, 0x75, 0xaf, 0x01, 0xc1, 0x21, 0xac, 0x51, + 0x76, 0xbe, 0xaa, 0xab, 0x82, 0x2d, 0xa0, 0x95, 0x2e, 0xb7, 0xe2, 0xc6, 0x76, 0xc9, 0x31, 0xd2, + 0x41, 0xfb, 0x8f, 0xc6, 0x2b, 0xba, 0xea, 0x2c, 0x57, 0xf2, 0x0a, 0x51, 0xa1, 0x92, 0x55, 0xb7, + 0xeb, 0x3e, 0x10, 0xca, 0xce, 0xb1, 0x8c, 0x16, 0x04, 0x71, 0xc5, 0xf2, 0xf9, 0x63, 0xd8, 0xc8, + 0x29, 0x5a, 0x35, 0x90, 0x93, 0x0c, 0xc8, 0xea, 0x9a, 0x8f, 0x2d, 0x32, 0xab, 0xaf, 0xa7, 0xcb, + 0x1c, 0x41, 0x9b, 0xb2, 0xf3, 0xc7, 0xdc, 0xe3, 0xc5, 0xae, 0x25, 0x8b, 0x5b, 0xe2, 0x87, 0xf0, + 0xde, 0xcc, 0x8a, 0xab, 0x76, 0xfb, 0x97, 0x70, 0x33, 0x75, 0x50, 0x71, 0xa3, 0x97, 0x79, 0x9e, + 0xc1, 0xfb, 0xf3, 0x0b, 0xaf, 0xda, 0xf6, 0x09, 0x10, 0xbd, 0x34, 0xf3, 0xfb, 0x7c, 0xe5, 0x57, + 0xf8, 0x5b, 0x50, 0xe3, 0x7e, 0x6f, 0xa6, 0x9b, 0x57, 0xb9, 0xdf, 0x7b, 0xce, 0x63, 0xb5, 0x15, + 0x72, 0x7a, 0x57, 0x8d, 0x2b, 0xc6, 0x98, 0xfc, 0x5f, 0xa0, 0x25, 0x51, 0x7b, 0xa7, 0xe8, 0x7e, + 0x63, 0x60, 0x35, 0xbc, 0xc6, 0x57, 0x97, 0xd9, 0xb7, 0x15, 0x33, 0xff, 0xb6, 0xd2, 0x87, 0xf5, + 0xa9, 0x2d, 0xc5, 0x81, 0x7e, 0x04, 0xe5, 0xe1, 0xe4, 0xd2, 0xca, 0xa0, 0x78, 0xf6, 0x67, 0xf8, + 0xc2, 0x8d, 0x2e, 0xcd, 0x43, 0x30, 0x2e, 0x0f, 0x4d, 0x29, 0x17, 0x9a, 0xbf, 0x1a, 0x59, 0x2d, + 0x2b, 0xea, 0xbc, 0x07, 0x50, 0x0d, 0x95, 0x09, 0x4b, 0x6f, 0xf9, 0x49, 0xbc, 0xb5, 0x80, 0x3a, + 0x61, 0x70, 0xe6, 0x0c, 0xba, 0xb3, 0xfe, 0xac, 0x2b, 0xca, 0xc1, 0xdb, 0x7c, 0xea, 0xc1, 0x66, + 0xde, 0xce, 0x77, 0xea, 0xd8, 0xaf, 0x0c, 0xa8, 0x1f, 0x4e, 0x1c, 0xe7, 0x4b, 0x7c, 0x7f, 0xbd, + 0x0b, 0xa6, 0x8c, 0x03, 0xbe, 0xec, 0x71, 0x09, 0x19, 0xf8, 0x9c, 0x8d, 0xbe, 0x97, 0xe9, 0xe1, + 0xb5, 0x86, 0xe3, 0x93, 0x48, 0x85, 0x45, 0x9f, 0x2a, 0x65, 0xa4, 0x0f, 0x94, 0x56, 0x42, 0x38, + 0x89, 0xf0, 0x29, 0x7b, 0x20, 0xd4, 0xb9, 0x14, 0xdb, 0xbb, 0x89, 0xa1, 0x01, 0x24, 0xbd, 0xc4, + 0x1e, 0xff, 0xdd, 0xc4, 0x0c, 0x1c, 0xe4, 0xb4, 0x18, 0x79, 0x2d, 0xd3, 0x13, 0x42, 0x69, 0xf6, + 0x84, 0xf0, 0x0b, 0x03, 0x2c, 0x35, 0x1d, 0x9f, 0x91, 0xfe, 0x17, 0x10, 0x1d, 0xa8, 0xe9, 0x17, + 0x29, 0xbd, 0xb3, 0xd3, 0xe1, 0xdb, 0x11, 0xfc, 0x5a, 0xdb, 0x80, 0x87, 0xfe, 0xf4, 0x21, 0x7b, + 0xfe, 0xa5, 0x31, 0x35, 0x52, 0x3f, 0x64, 0xef, 0x40, 0x15, 0x9f, 0x9e, 0xd2, 0x18, 0x91, 0x9c, + 0x20, 0xc6, 0x84, 0x6a, 0x09, 0x25, 0x8b, 0xaa, 0xd3, 0xcb, 0x47, 0x5e, 0x16, 0x6d, 0xa0, 0x5a, + 0xc2, 0x3e, 0x86, 0x0d, 0x45, 0x7c, 0xca, 0xe5, 0xa3, 0x58, 0xa5, 0xe6, 0x4a, 0x3a, 0xe9, 0xaf, + 0x0c, 0xd8, 0xcc, 0xaf, 0xba, 0xea, 0x33, 0xe6, 0x3d, 0x30, 0xd5, 0x6d, 0x63, 0xe1, 0x5d, 0x3f, + 0x75, 0x2b, 0x45, 0xb6, 0xfd, 0x13, 0xb8, 0x35, 0xb5, 0xe3, 0x38, 0x09, 0x5c, 0x11, 0x84, 0x97, + 0xa7, 0x81, 0xfd, 0x27, 0x03, 0x3a, 0x8b, 0x2a, 0x56, 0x0d, 0x77, 0xf1, 0x3f, 0xbe, 0xd4, 0x01, + 0xe6, 0x7f, 0x77, 0xc0, 0x8f, 0x80, 0x1c, 0x07, 0x9e, 0x2b, 0x93, 0x3f, 0xd1, 0x8a, 0xb6, 0x01, + 0xb5, 0x42, 0xae, 0x0d, 0x28, 0x82, 0x2a, 0x95, 0xbf, 0x33, 0x60, 0x23, 0xb7, 0x7e, 0x71, 0xe0, + 0x36, 0x98, 0x1e, 0x3f, 0x93, 0xfa, 0x1e, 0xd4, 0xca, 0xff, 0xfb, 0x47, 0x91, 0x47, 0x3e, 0x86, + 0x4a, 0xe8, 0xf6, 0x07, 0x52, 0x87, 0x7d, 0x5e, 0x28, 0x61, 0xee, 0x7c, 0x13, 0x20, 0xfb, 0x23, + 0x8f, 0x00, 0x54, 0x7f, 0x20, 0xc2, 0x11, 0xf3, 0xda, 0x37, 0x48, 0x0d, 0xca, 0x07, 0xe2, 0xbc, + 0x6d, 0x10, 0x0b, 0xcc, 0xcf, 0xdd, 0xfe, 0xa0, 0x5d, 0xda, 0xd9, 0x82, 0x56, 0xfe, 0xdf, 0x3b, + 0x52, 0x85, 0xd2, 0xf1, 0xb3, 0xf6, 0x0d, 0xf5, 0x4b, 0xf7, 0xdb, 0xc6, 0xce, 0x2e, 0x94, 0x5e, + 0x04, 0x6a, 0xea, 0xd1, 0x58, 0x26, 0x6b, 0x3c, 0xe6, 0x5e, 0xb2, 0x86, 0xda, 0x9f, 0xed, 0x12, + 0x69, 0x82, 0x95, 0xbe, 0xa3, 0xb5, 0xcb, 0x8f, 0x76, 0xfe, 0xf1, 0xe6, 0x8e, 0xf1, 0xcf, 0x37, + 0x77, 0x8c, 0x7f, 0xbd, 0xb9, 0x63, 0xfc, 0xf1, 0xdf, 0x77, 0x6e, 0x40, 0xc7, 0x11, 0xa3, 0xdd, + 0xc0, 0xf5, 0xfb, 0x0e, 0x0b, 0x76, 0xa5, 0x3b, 0x9c, 0xec, 0x0e, 0x27, 0xf8, 0x6f, 0xf0, 0x69, + 0x15, 0x7f, 0x3e, 0xf9, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xe2, 0x8f, 0x9a, 0x77, 0x61, 0x1e, + 0x00, 0x00, } diff --git a/vendor/github.com/pingcap/kvproto/pkg/metapb/metapb.pb.go b/vendor/github.com/pingcap/kvproto/pkg/metapb/metapb.pb.go index 62687c687210d..3da13857558b5 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/metapb/metapb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/metapb/metapb.pb.go @@ -1,5 +1,6 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. +// Code generated by protoc-gen-gogo. // source: metapb.proto +// DO NOT EDIT! /* Package metapb is a generated protocol buffer package. @@ -476,6 +477,24 @@ func (m *Peer) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func encodeFixed64Metapb(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + dAtA[offset+4] = uint8(v >> 32) + dAtA[offset+5] = uint8(v >> 40) + dAtA[offset+6] = uint8(v >> 48) + dAtA[offset+7] = uint8(v >> 56) + return offset + 8 +} +func encodeFixed32Metapb(dAtA []byte, offset int, v uint32) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + return offset + 4 +} func encodeVarintMetapb(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) diff --git a/vendor/github.com/pingcap/kvproto/pkg/pdpb/pdpb.pb.go b/vendor/github.com/pingcap/kvproto/pkg/pdpb/pdpb.pb.go index 100873d258157..10c758ee88429 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/pdpb/pdpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/pdpb/pdpb.pb.go @@ -1,5 +1,6 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. +// Code generated by protoc-gen-gogo. // source: pdpb.proto +// DO NOT EDIT! /* Package pdpb is a generated protocol buffer package. @@ -24,6 +25,8 @@ GetStoreResponse PutStoreRequest PutStoreResponse + GetAllStoresRequest + GetAllStoresResponse GetRegionRequest GetRegionResponse GetRegionByIDRequest @@ -38,6 +41,8 @@ RegionHeartbeatRequest ChangePeer TransferLeader + Merge + SplitRegion RegionHeartbeatResponse AskSplitRequest AskSplitResponse @@ -46,6 +51,8 @@ StoreStats StoreHeartbeatRequest StoreHeartbeatResponse + ScatterRegionRequest + ScatterRegionResponse */ package pdpb @@ -468,6 +475,46 @@ func (m *PutStoreResponse) GetHeader() *ResponseHeader { return nil } +type GetAllStoresRequest struct { + Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` +} + +func (m *GetAllStoresRequest) Reset() { *m = GetAllStoresRequest{} } +func (m *GetAllStoresRequest) String() string { return proto.CompactTextString(m) } +func (*GetAllStoresRequest) ProtoMessage() {} +func (*GetAllStoresRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{16} } + +func (m *GetAllStoresRequest) GetHeader() *RequestHeader { + if m != nil { + return m.Header + } + return nil +} + +type GetAllStoresResponse struct { + Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + Stores []*metapb.Store `protobuf:"bytes,2,rep,name=stores" json:"stores,omitempty"` +} + +func (m *GetAllStoresResponse) Reset() { *m = GetAllStoresResponse{} } +func (m *GetAllStoresResponse) String() string { return proto.CompactTextString(m) } +func (*GetAllStoresResponse) ProtoMessage() {} +func (*GetAllStoresResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{17} } + +func (m *GetAllStoresResponse) GetHeader() *ResponseHeader { + if m != nil { + return m.Header + } + return nil +} + +func (m *GetAllStoresResponse) GetStores() []*metapb.Store { + if m != nil { + return m.Stores + } + return nil +} + type GetRegionRequest struct { Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` RegionKey []byte `protobuf:"bytes,2,opt,name=region_key,json=regionKey,proto3" json:"region_key,omitempty"` @@ -476,7 +523,7 @@ type GetRegionRequest struct { func (m *GetRegionRequest) Reset() { *m = GetRegionRequest{} } func (m *GetRegionRequest) String() string { return proto.CompactTextString(m) } func (*GetRegionRequest) ProtoMessage() {} -func (*GetRegionRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{16} } +func (*GetRegionRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{18} } func (m *GetRegionRequest) GetHeader() *RequestHeader { if m != nil { @@ -501,7 +548,7 @@ type GetRegionResponse struct { func (m *GetRegionResponse) Reset() { *m = GetRegionResponse{} } func (m *GetRegionResponse) String() string { return proto.CompactTextString(m) } func (*GetRegionResponse) ProtoMessage() {} -func (*GetRegionResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{17} } +func (*GetRegionResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{19} } func (m *GetRegionResponse) GetHeader() *ResponseHeader { if m != nil { @@ -532,7 +579,7 @@ type GetRegionByIDRequest struct { func (m *GetRegionByIDRequest) Reset() { *m = GetRegionByIDRequest{} } func (m *GetRegionByIDRequest) String() string { return proto.CompactTextString(m) } func (*GetRegionByIDRequest) ProtoMessage() {} -func (*GetRegionByIDRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{18} } +func (*GetRegionByIDRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{20} } func (m *GetRegionByIDRequest) GetHeader() *RequestHeader { if m != nil { @@ -555,7 +602,7 @@ type GetClusterConfigRequest struct { func (m *GetClusterConfigRequest) Reset() { *m = GetClusterConfigRequest{} } func (m *GetClusterConfigRequest) String() string { return proto.CompactTextString(m) } func (*GetClusterConfigRequest) ProtoMessage() {} -func (*GetClusterConfigRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{19} } +func (*GetClusterConfigRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{21} } func (m *GetClusterConfigRequest) GetHeader() *RequestHeader { if m != nil { @@ -572,7 +619,7 @@ type GetClusterConfigResponse struct { func (m *GetClusterConfigResponse) Reset() { *m = GetClusterConfigResponse{} } func (m *GetClusterConfigResponse) String() string { return proto.CompactTextString(m) } func (*GetClusterConfigResponse) ProtoMessage() {} -func (*GetClusterConfigResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{20} } +func (*GetClusterConfigResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{22} } func (m *GetClusterConfigResponse) GetHeader() *ResponseHeader { if m != nil { @@ -596,7 +643,7 @@ type PutClusterConfigRequest struct { func (m *PutClusterConfigRequest) Reset() { *m = PutClusterConfigRequest{} } func (m *PutClusterConfigRequest) String() string { return proto.CompactTextString(m) } func (*PutClusterConfigRequest) ProtoMessage() {} -func (*PutClusterConfigRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{21} } +func (*PutClusterConfigRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{23} } func (m *PutClusterConfigRequest) GetHeader() *RequestHeader { if m != nil { @@ -619,7 +666,7 @@ type PutClusterConfigResponse struct { func (m *PutClusterConfigResponse) Reset() { *m = PutClusterConfigResponse{} } func (m *PutClusterConfigResponse) String() string { return proto.CompactTextString(m) } func (*PutClusterConfigResponse) ProtoMessage() {} -func (*PutClusterConfigResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{22} } +func (*PutClusterConfigResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{24} } func (m *PutClusterConfigResponse) GetHeader() *ResponseHeader { if m != nil { @@ -632,15 +679,16 @@ type Member struct { // name is the name of the PD member. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` // member_id is the unique id of the PD member. - MemberId uint64 `protobuf:"varint,2,opt,name=member_id,json=memberId,proto3" json:"member_id,omitempty"` - PeerUrls []string `protobuf:"bytes,3,rep,name=peer_urls,json=peerUrls" json:"peer_urls,omitempty"` - ClientUrls []string `protobuf:"bytes,4,rep,name=client_urls,json=clientUrls" json:"client_urls,omitempty"` + MemberId uint64 `protobuf:"varint,2,opt,name=member_id,json=memberId,proto3" json:"member_id,omitempty"` + PeerUrls []string `protobuf:"bytes,3,rep,name=peer_urls,json=peerUrls" json:"peer_urls,omitempty"` + ClientUrls []string `protobuf:"bytes,4,rep,name=client_urls,json=clientUrls" json:"client_urls,omitempty"` + LeaderPriority int32 `protobuf:"varint,5,opt,name=leader_priority,json=leaderPriority,proto3" json:"leader_priority,omitempty"` } func (m *Member) Reset() { *m = Member{} } func (m *Member) String() string { return proto.CompactTextString(m) } func (*Member) ProtoMessage() {} -func (*Member) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{23} } +func (*Member) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{25} } func (m *Member) GetName() string { if m != nil { @@ -670,6 +718,13 @@ func (m *Member) GetClientUrls() []string { return nil } +func (m *Member) GetLeaderPriority() int32 { + if m != nil { + return m.LeaderPriority + } + return 0 +} + type GetMembersRequest struct { Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` } @@ -677,7 +732,7 @@ type GetMembersRequest struct { func (m *GetMembersRequest) Reset() { *m = GetMembersRequest{} } func (m *GetMembersRequest) String() string { return proto.CompactTextString(m) } func (*GetMembersRequest) ProtoMessage() {} -func (*GetMembersRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{24} } +func (*GetMembersRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{26} } func (m *GetMembersRequest) GetHeader() *RequestHeader { if m != nil { @@ -687,15 +742,16 @@ func (m *GetMembersRequest) GetHeader() *RequestHeader { } type GetMembersResponse struct { - Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` - Members []*Member `protobuf:"bytes,2,rep,name=members" json:"members,omitempty"` - Leader *Member `protobuf:"bytes,3,opt,name=leader" json:"leader,omitempty"` + Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + Members []*Member `protobuf:"bytes,2,rep,name=members" json:"members,omitempty"` + Leader *Member `protobuf:"bytes,3,opt,name=leader" json:"leader,omitempty"` + EtcdLeader *Member `protobuf:"bytes,4,opt,name=etcd_leader,json=etcdLeader" json:"etcd_leader,omitempty"` } func (m *GetMembersResponse) Reset() { *m = GetMembersResponse{} } func (m *GetMembersResponse) String() string { return proto.CompactTextString(m) } func (*GetMembersResponse) ProtoMessage() {} -func (*GetMembersResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{25} } +func (*GetMembersResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{27} } func (m *GetMembersResponse) GetHeader() *ResponseHeader { if m != nil { @@ -718,6 +774,13 @@ func (m *GetMembersResponse) GetLeader() *Member { return nil } +func (m *GetMembersResponse) GetEtcdLeader() *Member { + if m != nil { + return m.EtcdLeader + } + return nil +} + type PeerStats struct { Peer *metapb.Peer `protobuf:"bytes,1,opt,name=peer" json:"peer,omitempty"` DownSeconds uint64 `protobuf:"varint,2,opt,name=down_seconds,json=downSeconds,proto3" json:"down_seconds,omitempty"` @@ -726,7 +789,7 @@ type PeerStats struct { func (m *PeerStats) Reset() { *m = PeerStats{} } func (m *PeerStats) String() string { return proto.CompactTextString(m) } func (*PeerStats) ProtoMessage() {} -func (*PeerStats) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{26} } +func (*PeerStats) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{28} } func (m *PeerStats) GetPeer() *metapb.Peer { if m != nil { @@ -760,12 +823,14 @@ type RegionHeartbeatRequest struct { KeysRead uint64 `protobuf:"varint,9,opt,name=keys_read,json=keysRead,proto3" json:"keys_read,omitempty"` // Approximate region size. ApproximateSize uint64 `protobuf:"varint,10,opt,name=approximate_size,json=approximateSize,proto3" json:"approximate_size,omitempty"` + // Unix timestamp when send the heartbeat. + Timestamp uint64 `protobuf:"varint,11,opt,name=timestamp,proto3" json:"timestamp,omitempty"` } func (m *RegionHeartbeatRequest) Reset() { *m = RegionHeartbeatRequest{} } func (m *RegionHeartbeatRequest) String() string { return proto.CompactTextString(m) } func (*RegionHeartbeatRequest) ProtoMessage() {} -func (*RegionHeartbeatRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{27} } +func (*RegionHeartbeatRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{29} } func (m *RegionHeartbeatRequest) GetHeader() *RequestHeader { if m != nil { @@ -837,6 +902,13 @@ func (m *RegionHeartbeatRequest) GetApproximateSize() uint64 { return 0 } +func (m *RegionHeartbeatRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + type ChangePeer struct { Peer *metapb.Peer `protobuf:"bytes,1,opt,name=peer" json:"peer,omitempty"` ChangeType eraftpb.ConfChangeType `protobuf:"varint,2,opt,name=change_type,json=changeType,proto3,enum=eraftpb.ConfChangeType" json:"change_type,omitempty"` @@ -845,7 +917,7 @@ type ChangePeer struct { func (m *ChangePeer) Reset() { *m = ChangePeer{} } func (m *ChangePeer) String() string { return proto.CompactTextString(m) } func (*ChangePeer) ProtoMessage() {} -func (*ChangePeer) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{28} } +func (*ChangePeer) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{30} } func (m *ChangePeer) GetPeer() *metapb.Peer { if m != nil { @@ -868,7 +940,7 @@ type TransferLeader struct { func (m *TransferLeader) Reset() { *m = TransferLeader{} } func (m *TransferLeader) String() string { return proto.CompactTextString(m) } func (*TransferLeader) ProtoMessage() {} -func (*TransferLeader) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{29} } +func (*TransferLeader) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{31} } func (m *TransferLeader) GetPeer() *metapb.Peer { if m != nil { @@ -877,6 +949,30 @@ func (m *TransferLeader) GetPeer() *metapb.Peer { return nil } +type Merge struct { + Target *metapb.Region `protobuf:"bytes,1,opt,name=target" json:"target,omitempty"` +} + +func (m *Merge) Reset() { *m = Merge{} } +func (m *Merge) String() string { return proto.CompactTextString(m) } +func (*Merge) ProtoMessage() {} +func (*Merge) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{32} } + +func (m *Merge) GetTarget() *metapb.Region { + if m != nil { + return m.Target + } + return nil +} + +type SplitRegion struct { +} + +func (m *SplitRegion) Reset() { *m = SplitRegion{} } +func (m *SplitRegion) String() string { return proto.CompactTextString(m) } +func (*SplitRegion) ProtoMessage() {} +func (*SplitRegion) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{33} } + type RegionHeartbeatResponse struct { Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` // Notice, Pd only allows handling reported epoch >= current pd's. @@ -901,12 +997,15 @@ type RegionHeartbeatResponse struct { RegionEpoch *metapb.RegionEpoch `protobuf:"bytes,5,opt,name=region_epoch,json=regionEpoch" json:"region_epoch,omitempty"` // Leader of the region at the moment of the corresponding request was made. TargetPeer *metapb.Peer `protobuf:"bytes,6,opt,name=target_peer,json=targetPeer" json:"target_peer,omitempty"` + Merge *Merge `protobuf:"bytes,7,opt,name=merge" json:"merge,omitempty"` + // PD sends split_region to let TiKV split a region into two regions. + SplitRegion *SplitRegion `protobuf:"bytes,8,opt,name=split_region,json=splitRegion" json:"split_region,omitempty"` } func (m *RegionHeartbeatResponse) Reset() { *m = RegionHeartbeatResponse{} } func (m *RegionHeartbeatResponse) String() string { return proto.CompactTextString(m) } func (*RegionHeartbeatResponse) ProtoMessage() {} -func (*RegionHeartbeatResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{30} } +func (*RegionHeartbeatResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{34} } func (m *RegionHeartbeatResponse) GetHeader() *ResponseHeader { if m != nil { @@ -950,6 +1049,20 @@ func (m *RegionHeartbeatResponse) GetTargetPeer() *metapb.Peer { return nil } +func (m *RegionHeartbeatResponse) GetMerge() *Merge { + if m != nil { + return m.Merge + } + return nil +} + +func (m *RegionHeartbeatResponse) GetSplitRegion() *SplitRegion { + if m != nil { + return m.SplitRegion + } + return nil +} + type AskSplitRequest struct { Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` Region *metapb.Region `protobuf:"bytes,2,opt,name=region" json:"region,omitempty"` @@ -958,7 +1071,7 @@ type AskSplitRequest struct { func (m *AskSplitRequest) Reset() { *m = AskSplitRequest{} } func (m *AskSplitRequest) String() string { return proto.CompactTextString(m) } func (*AskSplitRequest) ProtoMessage() {} -func (*AskSplitRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{31} } +func (*AskSplitRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{35} } func (m *AskSplitRequest) GetHeader() *RequestHeader { if m != nil { @@ -987,7 +1100,7 @@ type AskSplitResponse struct { func (m *AskSplitResponse) Reset() { *m = AskSplitResponse{} } func (m *AskSplitResponse) String() string { return proto.CompactTextString(m) } func (*AskSplitResponse) ProtoMessage() {} -func (*AskSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{32} } +func (*AskSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{36} } func (m *AskSplitResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1019,7 +1132,7 @@ type ReportSplitRequest struct { func (m *ReportSplitRequest) Reset() { *m = ReportSplitRequest{} } func (m *ReportSplitRequest) String() string { return proto.CompactTextString(m) } func (*ReportSplitRequest) ProtoMessage() {} -func (*ReportSplitRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{33} } +func (*ReportSplitRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{37} } func (m *ReportSplitRequest) GetHeader() *RequestHeader { if m != nil { @@ -1049,7 +1162,7 @@ type ReportSplitResponse struct { func (m *ReportSplitResponse) Reset() { *m = ReportSplitResponse{} } func (m *ReportSplitResponse) String() string { return proto.CompactTextString(m) } func (*ReportSplitResponse) ProtoMessage() {} -func (*ReportSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{34} } +func (*ReportSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{38} } func (m *ReportSplitResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1091,7 +1204,7 @@ type StoreStats struct { func (m *StoreStats) Reset() { *m = StoreStats{} } func (m *StoreStats) String() string { return proto.CompactTextString(m) } func (*StoreStats) ProtoMessage() {} -func (*StoreStats) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{35} } +func (*StoreStats) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{39} } func (m *StoreStats) GetStoreId() uint64 { if m != nil { @@ -1199,7 +1312,7 @@ type StoreHeartbeatRequest struct { func (m *StoreHeartbeatRequest) Reset() { *m = StoreHeartbeatRequest{} } func (m *StoreHeartbeatRequest) String() string { return proto.CompactTextString(m) } func (*StoreHeartbeatRequest) ProtoMessage() {} -func (*StoreHeartbeatRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{36} } +func (*StoreHeartbeatRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{40} } func (m *StoreHeartbeatRequest) GetHeader() *RequestHeader { if m != nil { @@ -1222,7 +1335,7 @@ type StoreHeartbeatResponse struct { func (m *StoreHeartbeatResponse) Reset() { *m = StoreHeartbeatResponse{} } func (m *StoreHeartbeatResponse) String() string { return proto.CompactTextString(m) } func (*StoreHeartbeatResponse) ProtoMessage() {} -func (*StoreHeartbeatResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{37} } +func (*StoreHeartbeatResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{41} } func (m *StoreHeartbeatResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1231,6 +1344,64 @@ func (m *StoreHeartbeatResponse) GetHeader() *ResponseHeader { return nil } +type ScatterRegionRequest struct { + Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + RegionId uint64 `protobuf:"varint,2,opt,name=region_id,json=regionId,proto3" json:"region_id,omitempty"` + // PD will use these region information if it can't find the region. + // For example, the region is just split and hasn't report to PD yet. + Region *metapb.Region `protobuf:"bytes,3,opt,name=region" json:"region,omitempty"` + Leader *metapb.Peer `protobuf:"bytes,4,opt,name=leader" json:"leader,omitempty"` +} + +func (m *ScatterRegionRequest) Reset() { *m = ScatterRegionRequest{} } +func (m *ScatterRegionRequest) String() string { return proto.CompactTextString(m) } +func (*ScatterRegionRequest) ProtoMessage() {} +func (*ScatterRegionRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{42} } + +func (m *ScatterRegionRequest) GetHeader() *RequestHeader { + if m != nil { + return m.Header + } + return nil +} + +func (m *ScatterRegionRequest) GetRegionId() uint64 { + if m != nil { + return m.RegionId + } + return 0 +} + +func (m *ScatterRegionRequest) GetRegion() *metapb.Region { + if m != nil { + return m.Region + } + return nil +} + +func (m *ScatterRegionRequest) GetLeader() *metapb.Peer { + if m != nil { + return m.Leader + } + return nil +} + +type ScatterRegionResponse struct { + Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` +} + +func (m *ScatterRegionResponse) Reset() { *m = ScatterRegionResponse{} } +func (m *ScatterRegionResponse) String() string { return proto.CompactTextString(m) } +func (*ScatterRegionResponse) ProtoMessage() {} +func (*ScatterRegionResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{43} } + +func (m *ScatterRegionResponse) GetHeader() *ResponseHeader { + if m != nil { + return m.Header + } + return nil +} + func init() { proto.RegisterType((*RequestHeader)(nil), "pdpb.RequestHeader") proto.RegisterType((*ResponseHeader)(nil), "pdpb.ResponseHeader") @@ -1248,6 +1419,8 @@ func init() { proto.RegisterType((*GetStoreResponse)(nil), "pdpb.GetStoreResponse") proto.RegisterType((*PutStoreRequest)(nil), "pdpb.PutStoreRequest") proto.RegisterType((*PutStoreResponse)(nil), "pdpb.PutStoreResponse") + proto.RegisterType((*GetAllStoresRequest)(nil), "pdpb.GetAllStoresRequest") + proto.RegisterType((*GetAllStoresResponse)(nil), "pdpb.GetAllStoresResponse") proto.RegisterType((*GetRegionRequest)(nil), "pdpb.GetRegionRequest") proto.RegisterType((*GetRegionResponse)(nil), "pdpb.GetRegionResponse") proto.RegisterType((*GetRegionByIDRequest)(nil), "pdpb.GetRegionByIDRequest") @@ -1262,6 +1435,8 @@ func init() { proto.RegisterType((*RegionHeartbeatRequest)(nil), "pdpb.RegionHeartbeatRequest") proto.RegisterType((*ChangePeer)(nil), "pdpb.ChangePeer") proto.RegisterType((*TransferLeader)(nil), "pdpb.TransferLeader") + proto.RegisterType((*Merge)(nil), "pdpb.Merge") + proto.RegisterType((*SplitRegion)(nil), "pdpb.SplitRegion") proto.RegisterType((*RegionHeartbeatResponse)(nil), "pdpb.RegionHeartbeatResponse") proto.RegisterType((*AskSplitRequest)(nil), "pdpb.AskSplitRequest") proto.RegisterType((*AskSplitResponse)(nil), "pdpb.AskSplitResponse") @@ -1270,6 +1445,8 @@ func init() { proto.RegisterType((*StoreStats)(nil), "pdpb.StoreStats") proto.RegisterType((*StoreHeartbeatRequest)(nil), "pdpb.StoreHeartbeatRequest") proto.RegisterType((*StoreHeartbeatResponse)(nil), "pdpb.StoreHeartbeatResponse") + proto.RegisterType((*ScatterRegionRequest)(nil), "pdpb.ScatterRegionRequest") + proto.RegisterType((*ScatterRegionResponse)(nil), "pdpb.ScatterRegionResponse") proto.RegisterEnum("pdpb.ErrorType", ErrorType_name, ErrorType_value) } @@ -1293,6 +1470,7 @@ type PDClient interface { AllocID(ctx context.Context, in *AllocIDRequest, opts ...grpc.CallOption) (*AllocIDResponse, error) GetStore(ctx context.Context, in *GetStoreRequest, opts ...grpc.CallOption) (*GetStoreResponse, error) PutStore(ctx context.Context, in *PutStoreRequest, opts ...grpc.CallOption) (*PutStoreResponse, error) + GetAllStores(ctx context.Context, in *GetAllStoresRequest, opts ...grpc.CallOption) (*GetAllStoresResponse, error) StoreHeartbeat(ctx context.Context, in *StoreHeartbeatRequest, opts ...grpc.CallOption) (*StoreHeartbeatResponse, error) RegionHeartbeat(ctx context.Context, opts ...grpc.CallOption) (PD_RegionHeartbeatClient, error) GetRegion(ctx context.Context, in *GetRegionRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) @@ -1301,6 +1479,7 @@ type PDClient interface { ReportSplit(ctx context.Context, in *ReportSplitRequest, opts ...grpc.CallOption) (*ReportSplitResponse, error) GetClusterConfig(ctx context.Context, in *GetClusterConfigRequest, opts ...grpc.CallOption) (*GetClusterConfigResponse, error) PutClusterConfig(ctx context.Context, in *PutClusterConfigRequest, opts ...grpc.CallOption) (*PutClusterConfigResponse, error) + ScatterRegion(ctx context.Context, in *ScatterRegionRequest, opts ...grpc.CallOption) (*ScatterRegionResponse, error) } type pDClient struct { @@ -1396,6 +1575,15 @@ func (c *pDClient) PutStore(ctx context.Context, in *PutStoreRequest, opts ...gr return out, nil } +func (c *pDClient) GetAllStores(ctx context.Context, in *GetAllStoresRequest, opts ...grpc.CallOption) (*GetAllStoresResponse, error) { + out := new(GetAllStoresResponse) + err := grpc.Invoke(ctx, "/pdpb.PD/GetAllStores", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *pDClient) StoreHeartbeat(ctx context.Context, in *StoreHeartbeatRequest, opts ...grpc.CallOption) (*StoreHeartbeatResponse, error) { out := new(StoreHeartbeatResponse) err := grpc.Invoke(ctx, "/pdpb.PD/StoreHeartbeat", in, out, c.cc, opts...) @@ -1490,6 +1678,15 @@ func (c *pDClient) PutClusterConfig(ctx context.Context, in *PutClusterConfigReq return out, nil } +func (c *pDClient) ScatterRegion(ctx context.Context, in *ScatterRegionRequest, opts ...grpc.CallOption) (*ScatterRegionResponse, error) { + out := new(ScatterRegionResponse) + err := grpc.Invoke(ctx, "/pdpb.PD/ScatterRegion", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // Server API for PD service type PDServer interface { @@ -1502,6 +1699,7 @@ type PDServer interface { AllocID(context.Context, *AllocIDRequest) (*AllocIDResponse, error) GetStore(context.Context, *GetStoreRequest) (*GetStoreResponse, error) PutStore(context.Context, *PutStoreRequest) (*PutStoreResponse, error) + GetAllStores(context.Context, *GetAllStoresRequest) (*GetAllStoresResponse, error) StoreHeartbeat(context.Context, *StoreHeartbeatRequest) (*StoreHeartbeatResponse, error) RegionHeartbeat(PD_RegionHeartbeatServer) error GetRegion(context.Context, *GetRegionRequest) (*GetRegionResponse, error) @@ -1510,6 +1708,7 @@ type PDServer interface { ReportSplit(context.Context, *ReportSplitRequest) (*ReportSplitResponse, error) GetClusterConfig(context.Context, *GetClusterConfigRequest) (*GetClusterConfigResponse, error) PutClusterConfig(context.Context, *PutClusterConfigRequest) (*PutClusterConfigResponse, error) + ScatterRegion(context.Context, *ScatterRegionRequest) (*ScatterRegionResponse, error) } func RegisterPDServer(s *grpc.Server, srv PDServer) { @@ -1650,6 +1849,24 @@ func _PD_PutStore_Handler(srv interface{}, ctx context.Context, dec func(interfa return interceptor(ctx, in, info, handler) } +func _PD_GetAllStores_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetAllStoresRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PDServer).GetAllStores(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pdpb.PD/GetAllStores", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PDServer).GetAllStores(ctx, req.(*GetAllStoresRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _PD_StoreHeartbeat_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(StoreHeartbeatRequest) if err := dec(in); err != nil { @@ -1802,6 +2019,24 @@ func _PD_PutClusterConfig_Handler(srv interface{}, ctx context.Context, dec func return interceptor(ctx, in, info, handler) } +func _PD_ScatterRegion_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ScatterRegionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PDServer).ScatterRegion(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pdpb.PD/ScatterRegion", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PDServer).ScatterRegion(ctx, req.(*ScatterRegionRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _PD_serviceDesc = grpc.ServiceDesc{ ServiceName: "pdpb.PD", HandlerType: (*PDServer)(nil), @@ -1830,6 +2065,10 @@ var _PD_serviceDesc = grpc.ServiceDesc{ MethodName: "PutStore", Handler: _PD_PutStore_Handler, }, + { + MethodName: "GetAllStores", + Handler: _PD_GetAllStores_Handler, + }, { MethodName: "StoreHeartbeat", Handler: _PD_StoreHeartbeat_Handler, @@ -1858,6 +2097,10 @@ var _PD_serviceDesc = grpc.ServiceDesc{ MethodName: "PutClusterConfig", Handler: _PD_PutClusterConfig_Handler, }, + { + MethodName: "ScatterRegion", + Handler: _PD_ScatterRegion_Handler, + }, }, Streams: []grpc.StreamDesc{ { @@ -2405,7 +2648,7 @@ func (m *PutStoreResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *GetRegionRequest) Marshal() (dAtA []byte, err error) { +func (m *GetAllStoresRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -2415,7 +2658,7 @@ func (m *GetRegionRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *GetRegionRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *GetAllStoresRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -2430,6 +2673,74 @@ func (m *GetRegionRequest) MarshalTo(dAtA []byte) (int, error) { } i += n19 } + return i, nil +} + +func (m *GetAllStoresResponse) 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 *GetAllStoresResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Header != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) + n20, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n20 + } + if len(m.Stores) > 0 { + for _, msg := range m.Stores { + dAtA[i] = 0x12 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *GetRegionRequest) 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 *GetRegionRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Header != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) + n21, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n21 + } if len(m.RegionKey) > 0 { dAtA[i] = 0x12 i++ @@ -2458,31 +2769,31 @@ func (m *GetRegionResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n20, err := m.Header.MarshalTo(dAtA[i:]) + n22, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n20 + i += n22 } if m.Region != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Region.Size())) - n21, err := m.Region.MarshalTo(dAtA[i:]) + n23, err := m.Region.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n23 } if m.Leader != nil { dAtA[i] = 0x1a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Leader.Size())) - n22, err := m.Leader.MarshalTo(dAtA[i:]) + n24, err := m.Leader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n24 } return i, nil } @@ -2506,11 +2817,11 @@ func (m *GetRegionByIDRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n23, err := m.Header.MarshalTo(dAtA[i:]) + n25, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n25 } if m.RegionId != 0 { dAtA[i] = 0x10 @@ -2539,11 +2850,11 @@ func (m *GetClusterConfigRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n24, err := m.Header.MarshalTo(dAtA[i:]) + n26, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n26 } return i, nil } @@ -2567,21 +2878,21 @@ func (m *GetClusterConfigResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n25, err := m.Header.MarshalTo(dAtA[i:]) + n27, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n27 } if m.Cluster != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Cluster.Size())) - n26, err := m.Cluster.MarshalTo(dAtA[i:]) + n28, err := m.Cluster.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n28 } return i, nil } @@ -2605,21 +2916,21 @@ func (m *PutClusterConfigRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n27, err := m.Header.MarshalTo(dAtA[i:]) + n29, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n29 } if m.Cluster != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Cluster.Size())) - n28, err := m.Cluster.MarshalTo(dAtA[i:]) + n30, err := m.Cluster.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n30 } return i, nil } @@ -2643,11 +2954,11 @@ func (m *PutClusterConfigResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n29, err := m.Header.MarshalTo(dAtA[i:]) + n31, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n31 } return i, nil } @@ -2708,6 +3019,11 @@ func (m *Member) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], s) } } + if m.LeaderPriority != 0 { + dAtA[i] = 0x28 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.LeaderPriority)) + } return i, nil } @@ -2730,11 +3046,11 @@ func (m *GetMembersRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n30, err := m.Header.MarshalTo(dAtA[i:]) + n32, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n30 + i += n32 } return i, nil } @@ -2758,11 +3074,11 @@ func (m *GetMembersResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n31, err := m.Header.MarshalTo(dAtA[i:]) + n33, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n33 } if len(m.Members) > 0 { for _, msg := range m.Members { @@ -2780,11 +3096,21 @@ func (m *GetMembersResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Leader.Size())) - n32, err := m.Leader.MarshalTo(dAtA[i:]) + n34, err := m.Leader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n34 + } + if m.EtcdLeader != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.EtcdLeader.Size())) + n35, err := m.EtcdLeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n35 } return i, nil } @@ -2808,11 +3134,11 @@ func (m *PeerStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Peer.Size())) - n33, err := m.Peer.MarshalTo(dAtA[i:]) + n36, err := m.Peer.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n33 + i += n36 } if m.DownSeconds != 0 { dAtA[i] = 0x10 @@ -2841,31 +3167,31 @@ func (m *RegionHeartbeatRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n34, err := m.Header.MarshalTo(dAtA[i:]) + n37, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n34 + i += n37 } if m.Region != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Region.Size())) - n35, err := m.Region.MarshalTo(dAtA[i:]) + n38, err := m.Region.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n35 + i += n38 } if m.Leader != nil { dAtA[i] = 0x1a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Leader.Size())) - n36, err := m.Leader.MarshalTo(dAtA[i:]) + n39, err := m.Leader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n36 + i += n39 } if len(m.DownPeers) > 0 { for _, msg := range m.DownPeers { @@ -2916,6 +3242,11 @@ func (m *RegionHeartbeatRequest) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintPdpb(dAtA, i, uint64(m.ApproximateSize)) } + if m.Timestamp != 0 { + dAtA[i] = 0x58 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Timestamp)) + } return i, nil } @@ -2938,11 +3269,11 @@ func (m *ChangePeer) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Peer.Size())) - n37, err := m.Peer.MarshalTo(dAtA[i:]) + n40, err := m.Peer.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n37 + i += n40 } if m.ChangeType != 0 { dAtA[i] = 0x10 @@ -2971,12 +3302,58 @@ func (m *TransferLeader) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Peer.Size())) - n38, err := m.Peer.MarshalTo(dAtA[i:]) + n41, err := m.Peer.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n38 + i += n41 + } + return i, nil +} + +func (m *Merge) 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 *Merge) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Target != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Target.Size())) + n42, err := m.Target.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n42 + } + return i, nil +} + +func (m *SplitRegion) 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 *SplitRegion) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l return i, nil } @@ -2999,31 +3376,31 @@ func (m *RegionHeartbeatResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n39, err := m.Header.MarshalTo(dAtA[i:]) + n43, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n43 } if m.ChangePeer != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.ChangePeer.Size())) - n40, err := m.ChangePeer.MarshalTo(dAtA[i:]) + n44, err := m.ChangePeer.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n44 } if m.TransferLeader != nil { dAtA[i] = 0x1a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.TransferLeader.Size())) - n41, err := m.TransferLeader.MarshalTo(dAtA[i:]) + n45, err := m.TransferLeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n45 } if m.RegionId != 0 { dAtA[i] = 0x20 @@ -3034,21 +3411,41 @@ func (m *RegionHeartbeatResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.RegionEpoch.Size())) - n42, err := m.RegionEpoch.MarshalTo(dAtA[i:]) + n46, err := m.RegionEpoch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n46 } if m.TargetPeer != nil { dAtA[i] = 0x32 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.TargetPeer.Size())) - n43, err := m.TargetPeer.MarshalTo(dAtA[i:]) + n47, err := m.TargetPeer.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n43 + i += n47 + } + if m.Merge != nil { + dAtA[i] = 0x3a + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Merge.Size())) + n48, err := m.Merge.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n48 + } + if m.SplitRegion != nil { + dAtA[i] = 0x42 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.SplitRegion.Size())) + n49, err := m.SplitRegion.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n49 } return i, nil } @@ -3072,21 +3469,21 @@ func (m *AskSplitRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n44, err := m.Header.MarshalTo(dAtA[i:]) + n50, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n44 + i += n50 } if m.Region != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Region.Size())) - n45, err := m.Region.MarshalTo(dAtA[i:]) + n51, err := m.Region.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n45 + i += n51 } return i, nil } @@ -3110,11 +3507,11 @@ func (m *AskSplitResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n46, err := m.Header.MarshalTo(dAtA[i:]) + n52, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n46 + i += n52 } if m.NewRegionId != 0 { dAtA[i] = 0x10 @@ -3122,21 +3519,21 @@ func (m *AskSplitResponse) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintPdpb(dAtA, i, uint64(m.NewRegionId)) } if len(m.NewPeerIds) > 0 { - dAtA48 := make([]byte, len(m.NewPeerIds)*10) - var j47 int + dAtA54 := make([]byte, len(m.NewPeerIds)*10) + var j53 int for _, num := range m.NewPeerIds { for num >= 1<<7 { - dAtA48[j47] = uint8(uint64(num)&0x7f | 0x80) + dAtA54[j53] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j47++ + j53++ } - dAtA48[j47] = uint8(num) - j47++ + dAtA54[j53] = uint8(num) + j53++ } dAtA[i] = 0x1a i++ - i = encodeVarintPdpb(dAtA, i, uint64(j47)) - i += copy(dAtA[i:], dAtA48[:j47]) + i = encodeVarintPdpb(dAtA, i, uint64(j53)) + i += copy(dAtA[i:], dAtA54[:j53]) } return i, nil } @@ -3160,31 +3557,31 @@ func (m *ReportSplitRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n49, err := m.Header.MarshalTo(dAtA[i:]) + n55, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n55 } if m.Left != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Left.Size())) - n50, err := m.Left.MarshalTo(dAtA[i:]) + n56, err := m.Left.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n56 } if m.Right != nil { dAtA[i] = 0x1a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Right.Size())) - n51, err := m.Right.MarshalTo(dAtA[i:]) + n57, err := m.Right.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n57 } return i, nil } @@ -3208,11 +3605,11 @@ func (m *ReportSplitResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n52, err := m.Header.MarshalTo(dAtA[i:]) + n58, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n58 } return i, nil } @@ -3329,21 +3726,21 @@ func (m *StoreHeartbeatRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n53, err := m.Header.MarshalTo(dAtA[i:]) + n59, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n59 } if m.Stats != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Stats.Size())) - n54, err := m.Stats.MarshalTo(dAtA[i:]) + n60, err := m.Stats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n60 } return i, nil } @@ -3367,15 +3764,114 @@ func (m *StoreHeartbeatResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n55, err := m.Header.MarshalTo(dAtA[i:]) + n61, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n61 + } + return i, nil +} + +func (m *ScatterRegionRequest) 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 *ScatterRegionRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Header != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) + n62, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n62 + } + if m.RegionId != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.RegionId)) + } + if m.Region != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Region.Size())) + n63, err := m.Region.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n63 + } + if m.Leader != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Leader.Size())) + n64, err := m.Leader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n64 + } + return i, nil +} + +func (m *ScatterRegionResponse) 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 *ScatterRegionResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Header != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) + n65, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n65 } return i, nil } +func encodeFixed64Pdpb(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + dAtA[offset+4] = uint8(v >> 32) + dAtA[offset+5] = uint8(v >> 40) + dAtA[offset+6] = uint8(v >> 48) + dAtA[offset+7] = uint8(v >> 56) + return offset + 8 +} +func encodeFixed32Pdpb(dAtA []byte, offset int, v uint32) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + return offset + 4 +} func encodeVarintPdpb(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -3587,6 +4083,32 @@ func (m *PutStoreResponse) Size() (n int) { return n } +func (m *GetAllStoresRequest) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + return n +} + +func (m *GetAllStoresResponse) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if len(m.Stores) > 0 { + for _, e := range m.Stores { + l = e.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + } + return n +} + func (m *GetRegionRequest) Size() (n int) { var l int _ = l @@ -3702,6 +4224,9 @@ func (m *Member) Size() (n int) { n += 1 + l + sovPdpb(uint64(l)) } } + if m.LeaderPriority != 0 { + n += 1 + sovPdpb(uint64(m.LeaderPriority)) + } return n } @@ -3732,6 +4257,10 @@ func (m *GetMembersResponse) Size() (n int) { l = m.Leader.Size() n += 1 + l + sovPdpb(uint64(l)) } + if m.EtcdLeader != nil { + l = m.EtcdLeader.Size() + n += 1 + l + sovPdpb(uint64(l)) + } return n } @@ -3790,6 +4319,9 @@ func (m *RegionHeartbeatRequest) Size() (n int) { if m.ApproximateSize != 0 { n += 1 + sovPdpb(uint64(m.ApproximateSize)) } + if m.Timestamp != 0 { + n += 1 + sovPdpb(uint64(m.Timestamp)) + } return n } @@ -3816,6 +4348,22 @@ func (m *TransferLeader) Size() (n int) { return n } +func (m *Merge) Size() (n int) { + var l int + _ = l + if m.Target != nil { + l = m.Target.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + return n +} + +func (m *SplitRegion) Size() (n int) { + var l int + _ = l + return n +} + func (m *RegionHeartbeatResponse) Size() (n int) { var l int _ = l @@ -3842,6 +4390,14 @@ func (m *RegionHeartbeatResponse) Size() (n int) { l = m.TargetPeer.Size() n += 1 + l + sovPdpb(uint64(l)) } + if m.Merge != nil { + l = m.Merge.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if m.SplitRegion != nil { + l = m.SplitRegion.Size() + n += 1 + l + sovPdpb(uint64(l)) + } return n } @@ -3979,6 +4535,37 @@ func (m *StoreHeartbeatResponse) Size() (n int) { return n } +func (m *ScatterRegionRequest) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if m.RegionId != 0 { + n += 1 + sovPdpb(uint64(m.RegionId)) + } + if m.Region != nil { + l = m.Region.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if m.Leader != nil { + l = m.Leader.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + return n +} + +func (m *ScatterRegionResponse) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + return n +} + func sovPdpb(x uint64) (n int) { for { n++ @@ -5606,7 +6193,7 @@ func (m *PutStoreResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *GetRegionRequest) Unmarshal(dAtA []byte) error { +func (m *GetAllStoresRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5629,10 +6216,10 @@ func (m *GetRegionRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetRegionRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetAllStoresRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetRegionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetAllStoresRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -5668,23 +6255,220 @@ func (m *GetRegionRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RegionKey", wireType) + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPdpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break + if skippy < 0 { + return ErrInvalidLengthPdpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetAllStoresResponse) 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 ErrIntOverflowPdpb + } + 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: GetAllStoresResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetAllStoresResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Header == nil { + m.Header = &ResponseHeader{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Stores", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Stores = append(m.Stores, &metapb.Store{}) + if err := m.Stores[len(m.Stores)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPdpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GetRegionRequest) 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 ErrIntOverflowPdpb + } + 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: GetRegionRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetRegionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Header == nil { + m.Header = &RequestHeader{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionKey", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break } } if byteLen < 0 { @@ -6504,6 +7288,25 @@ func (m *Member) Unmarshal(dAtA []byte) error { } m.ClientUrls = append(m.ClientUrls, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LeaderPriority", wireType) + } + m.LeaderPriority = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LeaderPriority |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) @@ -6734,6 +7537,39 @@ func (m *GetMembersResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EtcdLeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EtcdLeader == nil { + m.EtcdLeader = &Member{} + } + if err := m.EtcdLeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) @@ -7142,13 +7978,32 @@ func (m *RegionHeartbeatRequest) Unmarshal(dAtA []byte) error { break } } - default: - iNdEx = preIndex - skippy, err := skipPdpb(dAtA[iNdEx:]) - if err != nil { - return err + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) } - if skippy < 0 { + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { return ErrInvalidLengthPdpb } if (iNdEx + skippy) > l { @@ -7348,6 +8203,139 @@ func (m *TransferLeader) Unmarshal(dAtA []byte) error { } return nil } +func (m *Merge) 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 ErrIntOverflowPdpb + } + 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: Merge: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Merge: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &metapb.Region{} + } + if err := m.Target.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPdpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SplitRegion) 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 ErrIntOverflowPdpb + } + 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: SplitRegion: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SplitRegion: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPdpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *RegionHeartbeatResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -7561,6 +8549,72 @@ func (m *RegionHeartbeatResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Merge", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Merge == nil { + m.Merge = &Merge{} + } + if err := m.Merge.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SplitRegion", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.SplitRegion == nil { + m.SplitRegion = &SplitRegion{} + } + if err := m.SplitRegion.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) @@ -7780,24 +8834,7 @@ func (m *AskSplitResponse) Unmarshal(dAtA []byte) error { } } case 3: - if wireType == 0 { - var v uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPdpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.NewPeerIds = append(m.NewPeerIds, v) - } else if wireType == 2 { + if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { @@ -7838,6 +8875,23 @@ func (m *AskSplitResponse) Unmarshal(dAtA []byte) error { } m.NewPeerIds = append(m.NewPeerIds, v) } + } else if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.NewPeerIds = append(m.NewPeerIds, v) } else { return fmt.Errorf("proto: wrong wireType = %d for field NewPeerIds", wireType) } @@ -8610,6 +9664,257 @@ func (m *StoreHeartbeatResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *ScatterRegionRequest) 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 ErrIntOverflowPdpb + } + 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: ScatterRegionRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ScatterRegionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Header == nil { + m.Header = &RequestHeader{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionId", wireType) + } + m.RegionId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RegionId |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Region", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Region == nil { + m.Region = &metapb.Region{} + } + if err := m.Region.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Leader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Leader == nil { + m.Leader = &metapb.Peer{} + } + if err := m.Leader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPdpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ScatterRegionResponse) 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 ErrIntOverflowPdpb + } + 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: ScatterRegionResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ScatterRegionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Header == nil { + m.Header = &ResponseHeader{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPdpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipPdpb(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 @@ -8718,114 +10023,127 @@ var ( func init() { proto.RegisterFile("pdpb.proto", fileDescriptorPdpb) } var fileDescriptorPdpb = []byte{ - // 1741 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0xcd, 0x6e, 0x23, 0xc7, - 0x11, 0xd6, 0xf0, 0x4f, 0x64, 0xf1, 0x77, 0x7b, 0xb5, 0xd2, 0x98, 0xbb, 0x52, 0xe4, 0x5e, 0x23, - 0x58, 0x3b, 0x36, 0xb3, 0x56, 0x80, 0xc0, 0x80, 0xe1, 0xc0, 0xfa, 0x5b, 0xaf, 0xb0, 0x5e, 0x51, - 0x68, 0xd2, 0x30, 0x7c, 0x09, 0x33, 0xe4, 0xb4, 0xa8, 0x89, 0xc8, 0x99, 0xf1, 0x74, 0x73, 0x15, - 0xee, 0x29, 0xa7, 0x5c, 0x12, 0x20, 0x39, 0x05, 0x79, 0x81, 0x3c, 0x46, 0xee, 0x39, 0xe6, 0x11, - 0x82, 0xcd, 0x39, 0xef, 0x10, 0x74, 0xf7, 0xf4, 0xfc, 0x91, 0xda, 0x28, 0xb3, 0xf1, 0x89, 0xec, - 0xaa, 0xea, 0xaf, 0xbf, 0xae, 0xae, 0xaa, 0xee, 0x1a, 0x00, 0xdf, 0xf6, 0xc7, 0x3d, 0x3f, 0xf0, - 0xb8, 0x87, 0x4a, 0xe2, 0x7f, 0xb7, 0x31, 0xa7, 0xdc, 0xd2, 0xb2, 0x6e, 0x93, 0x06, 0xd6, 0x25, - 0x8f, 0x86, 0x5b, 0x53, 0x6f, 0xea, 0xc9, 0xbf, 0x3f, 0x15, 0xff, 0x94, 0x14, 0xf7, 0xa0, 0x49, - 0xe8, 0xf7, 0x0b, 0xca, 0xf8, 0x73, 0x6a, 0xd9, 0x34, 0x40, 0xbb, 0x00, 0x93, 0xd9, 0x82, 0x71, - 0x1a, 0x8c, 0x1c, 0xdb, 0x34, 0xf6, 0x8d, 0x27, 0x25, 0x52, 0x0b, 0x25, 0x67, 0x36, 0x26, 0xd0, - 0x22, 0x94, 0xf9, 0x9e, 0xcb, 0xe8, 0x9d, 0x26, 0xa0, 0xf7, 0xa1, 0x4c, 0x83, 0xc0, 0x0b, 0xcc, - 0xc2, 0xbe, 0xf1, 0xa4, 0x7e, 0x50, 0xef, 0x49, 0xd6, 0xa7, 0x42, 0x44, 0x94, 0x06, 0x3f, 0x83, - 0xb2, 0x1c, 0xa3, 0xc7, 0x50, 0xe2, 0x4b, 0x9f, 0x4a, 0x90, 0xd6, 0x41, 0x3b, 0x61, 0x3a, 0x5c, - 0xfa, 0x94, 0x48, 0x25, 0x32, 0x61, 0x73, 0x4e, 0x19, 0xb3, 0xa6, 0x54, 0x42, 0xd6, 0x88, 0x1e, - 0xe2, 0x3e, 0xc0, 0x90, 0x79, 0xe1, 0x76, 0xd0, 0x4f, 0xa0, 0x72, 0x25, 0x19, 0x4a, 0xb8, 0xfa, - 0xc1, 0x7d, 0x05, 0x97, 0xda, 0x2d, 0x09, 0x4d, 0xd0, 0x16, 0x94, 0x27, 0xde, 0xc2, 0xe5, 0x12, - 0xb2, 0x49, 0xd4, 0x00, 0x1f, 0x42, 0x6d, 0xe8, 0xcc, 0x29, 0xe3, 0xd6, 0xdc, 0x47, 0x5d, 0xa8, - 0xfa, 0x57, 0x4b, 0xe6, 0x4c, 0xac, 0x99, 0x44, 0x2c, 0x92, 0x68, 0x2c, 0x38, 0xcd, 0xbc, 0xa9, - 0x54, 0x15, 0xa4, 0x4a, 0x0f, 0xf1, 0x6f, 0x0d, 0xa8, 0x4b, 0x52, 0xca, 0x67, 0xe8, 0xe3, 0x0c, - 0xab, 0x2d, 0xcd, 0x2a, 0xe9, 0xd3, 0xb7, 0xd3, 0x42, 0x9f, 0x40, 0x8d, 0x6b, 0x5a, 0x66, 0x51, - 0xc2, 0x84, 0xbe, 0x8a, 0xd8, 0x92, 0xd8, 0x02, 0xff, 0xc1, 0x80, 0xce, 0x91, 0xe7, 0x71, 0xc6, - 0x03, 0xcb, 0xcf, 0xe5, 0x9d, 0xc7, 0x50, 0x66, 0xdc, 0x0b, 0x68, 0x78, 0x86, 0xcd, 0x5e, 0x18, - 0x67, 0x03, 0x21, 0x24, 0x4a, 0x87, 0x7e, 0x0c, 0x95, 0x80, 0x4e, 0x1d, 0xcf, 0x0d, 0x29, 0xb5, - 0xb4, 0x15, 0x91, 0x52, 0x12, 0x6a, 0xf1, 0x21, 0xdc, 0x4b, 0xb0, 0xc9, 0xe3, 0x16, 0x7c, 0x02, - 0x0f, 0xce, 0x58, 0x04, 0xe2, 0x53, 0x3b, 0xcf, 0xae, 0xf0, 0xaf, 0x61, 0x3b, 0x8b, 0x92, 0xeb, - 0x90, 0x30, 0x34, 0xc6, 0x09, 0x14, 0xe9, 0xa4, 0x2a, 0x49, 0xc9, 0xf0, 0x17, 0xd0, 0x3a, 0x9c, - 0xcd, 0xbc, 0xc9, 0xd9, 0x49, 0x2e, 0xaa, 0x7d, 0x68, 0x47, 0xd3, 0x73, 0x71, 0x6c, 0x41, 0xc1, - 0x51, 0xcc, 0x4a, 0xa4, 0xe0, 0xd8, 0xf8, 0x3b, 0x68, 0x7f, 0x45, 0xb9, 0x3a, 0xbf, 0x3c, 0x11, - 0xf1, 0x1e, 0x54, 0xe5, 0xa9, 0x8f, 0x22, 0xd4, 0x4d, 0x39, 0x3e, 0xb3, 0x31, 0x85, 0x4e, 0x0c, - 0x9d, 0x8b, 0xec, 0x5d, 0xc2, 0x0d, 0x4f, 0xa0, 0x7d, 0xb1, 0x78, 0x87, 0x1d, 0xdc, 0x69, 0x91, - 0x2f, 0xa1, 0x13, 0x2f, 0x92, 0x2b, 0x54, 0x7f, 0x29, 0xbd, 0x11, 0xa6, 0x40, 0x1e, 0x9e, 0xbb, - 0x00, 0x2a, 0x71, 0x46, 0xd7, 0x74, 0x29, 0xc9, 0x36, 0x48, 0x4d, 0x49, 0x5e, 0xd0, 0x25, 0xfe, - 0xa3, 0x01, 0xf7, 0x12, 0x0b, 0xe4, 0xf2, 0x77, 0x9c, 0xb9, 0x85, 0xb7, 0x65, 0x2e, 0xfa, 0x00, - 0x2a, 0x33, 0x85, 0xaa, 0x32, 0xbc, 0xa1, 0xed, 0x2e, 0xa8, 0x40, 0x53, 0x3a, 0xfc, 0x2b, 0xd8, - 0x8a, 0x08, 0x1d, 0x2d, 0xf3, 0x05, 0x3c, 0x7a, 0x08, 0xe1, 0x1e, 0xe3, 0x00, 0xab, 0x2a, 0xc1, - 0x99, 0x8d, 0x9f, 0xc1, 0xce, 0x57, 0x94, 0x1f, 0xab, 0x2b, 0xe6, 0xd8, 0x73, 0x2f, 0x9d, 0x69, - 0xae, 0xac, 0x62, 0x60, 0xae, 0xe2, 0xe4, 0xf2, 0xe0, 0x87, 0xb0, 0x19, 0xde, 0x78, 0xa1, 0x0b, - 0xdb, 0xda, 0x35, 0x21, 0x3a, 0xd1, 0x7a, 0xfc, 0x3d, 0xec, 0x5c, 0x2c, 0xde, 0x9d, 0xfc, 0xff, - 0xb2, 0xe4, 0x73, 0x30, 0x57, 0x97, 0xcc, 0x15, 0xcd, 0x37, 0x50, 0x79, 0x49, 0xe7, 0x63, 0x1a, - 0x20, 0x04, 0x25, 0xd7, 0x9a, 0xab, 0xab, 0xba, 0x46, 0xe4, 0x7f, 0x71, 0x68, 0x73, 0xa9, 0x4d, - 0x1c, 0x9a, 0x12, 0x9c, 0xd9, 0x42, 0xe9, 0x53, 0x1a, 0x8c, 0x16, 0xc1, 0x8c, 0x99, 0xc5, 0xfd, - 0xe2, 0x93, 0x1a, 0xa9, 0x0a, 0xc1, 0x37, 0xc1, 0x8c, 0xa1, 0x1f, 0x41, 0x7d, 0x32, 0x73, 0xa8, - 0xcb, 0x95, 0xba, 0x24, 0xd5, 0xa0, 0x44, 0xc2, 0x00, 0x7f, 0x29, 0xa3, 0x5c, 0xad, 0xcd, 0x72, - 0x1d, 0xf6, 0x9f, 0x0c, 0x40, 0x49, 0x88, 0x9c, 0x99, 0xb2, 0xa9, 0x36, 0xc4, 0xcc, 0xc2, 0x7e, - 0x51, 0xa6, 0x80, 0x34, 0x57, 0xa8, 0x44, 0x2b, 0xd7, 0x64, 0x4a, 0xd2, 0x4c, 0x67, 0xca, 0x05, - 0xd4, 0x44, 0xe6, 0x0c, 0xb8, 0xc5, 0x19, 0xda, 0x87, 0x92, 0x70, 0x47, 0x48, 0x23, 0x9d, 0x5a, - 0x52, 0x83, 0xde, 0x87, 0x86, 0xed, 0xdd, 0xb8, 0x23, 0x46, 0x27, 0x9e, 0x6b, 0xb3, 0xd0, 0xc3, - 0x75, 0x21, 0x1b, 0x28, 0x11, 0xfe, 0x6b, 0x11, 0xb6, 0x55, 0xe6, 0x3d, 0xa7, 0x56, 0xc0, 0xc7, - 0xd4, 0xe2, 0xb9, 0x82, 0xeb, 0xff, 0x5a, 0x11, 0x50, 0x0f, 0x40, 0x12, 0x17, 0xbb, 0x50, 0x87, - 0x1b, 0x3d, 0x58, 0xa2, 0xfd, 0x93, 0x9a, 0x30, 0x11, 0x43, 0x86, 0x3e, 0x85, 0xa6, 0x4f, 0x5d, - 0xdb, 0x71, 0xa7, 0xe1, 0x94, 0x72, 0xe8, 0xeb, 0x24, 0x78, 0x23, 0x34, 0x51, 0x53, 0x1e, 0x43, - 0x73, 0xbc, 0xe4, 0x94, 0x8d, 0x6e, 0x02, 0x87, 0x73, 0xea, 0x9a, 0x15, 0xe9, 0x9c, 0x86, 0x14, - 0x7e, 0xab, 0x64, 0xa2, 0x94, 0x2a, 0xa3, 0x80, 0x5a, 0xb6, 0xb9, 0xa9, 0x5e, 0xaa, 0x52, 0x42, - 0xa8, 0x25, 0x5e, 0xaa, 0x8d, 0x6b, 0xba, 0x8c, 0x21, 0xaa, 0xca, 0xbf, 0x42, 0xa6, 0x11, 0x1e, - 0x42, 0x4d, 0x9a, 0x48, 0x80, 0x9a, 0x8a, 0x70, 0x21, 0x90, 0xf3, 0x3f, 0x84, 0x8e, 0xe5, 0xfb, - 0x81, 0xf7, 0x1b, 0x67, 0x6e, 0x71, 0x3a, 0x62, 0xce, 0x6b, 0x6a, 0x82, 0xb4, 0x69, 0x27, 0xe4, - 0x03, 0xe7, 0x35, 0xc5, 0x57, 0x00, 0xc7, 0x57, 0x96, 0x3b, 0xa5, 0x82, 0xfd, 0x1d, 0x8e, 0xfe, - 0x33, 0xa8, 0x4f, 0xa4, 0xfd, 0x48, 0xbe, 0x8f, 0x0b, 0xf2, 0x7d, 0xbc, 0xd3, 0xd3, 0x0f, 0x7c, - 0x91, 0xd3, 0x0a, 0x4f, 0xbe, 0x93, 0x61, 0x12, 0xfd, 0xc7, 0x07, 0xd0, 0x1a, 0x06, 0x96, 0xcb, - 0x2e, 0x69, 0xf0, 0xb5, 0x3a, 0x8d, 0xff, 0xba, 0x1a, 0xfe, 0x5b, 0x01, 0x76, 0x56, 0xa2, 0x28, - 0x57, 0xbe, 0x7c, 0x1a, 0xf1, 0x96, 0x4b, 0xaa, 0x60, 0xea, 0xa8, 0x29, 0xb1, 0x03, 0x34, 0x61, - 0xe9, 0x8c, 0x2f, 0xa0, 0xcd, 0x43, 0xc2, 0xa3, 0x54, 0x6c, 0x85, 0x2b, 0xa5, 0x77, 0x43, 0x5a, - 0x3c, 0xbd, 0xbb, 0xd4, 0xc5, 0x51, 0x4a, 0x5f, 0x1c, 0xe8, 0xe7, 0xd0, 0x08, 0x95, 0xd4, 0xf7, - 0x26, 0x57, 0x66, 0x39, 0xcc, 0x84, 0x54, 0x70, 0x9f, 0x0a, 0x15, 0xa9, 0x07, 0xf1, 0x00, 0x7d, - 0x02, 0x75, 0x6e, 0x05, 0x53, 0xca, 0xd5, 0x36, 0x2a, 0x6b, 0x3c, 0x07, 0xca, 0x40, 0xfc, 0xc7, - 0x97, 0xd0, 0x3e, 0x64, 0xd7, 0x03, 0x7f, 0xe6, 0xfc, 0xa0, 0xd9, 0x87, 0x7f, 0x67, 0x40, 0x27, - 0x5e, 0x28, 0xe7, 0xdb, 0xb5, 0xe9, 0xd2, 0x9b, 0x51, 0xf6, 0xae, 0xad, 0xbb, 0xf4, 0x86, 0x68, - 0xaf, 0xed, 0x43, 0x43, 0xd8, 0xc8, 0xea, 0xed, 0xd8, 0xaa, 0x78, 0x97, 0x08, 0xb8, 0xf4, 0x46, - 0xec, 0xf6, 0xcc, 0x66, 0xf8, 0xf7, 0x06, 0x20, 0x42, 0x7d, 0x2f, 0xe0, 0xf9, 0x37, 0x8d, 0xa1, - 0x34, 0xa3, 0x97, 0xfc, 0x96, 0x2d, 0x4b, 0x1d, 0xfa, 0x00, 0xca, 0x81, 0x33, 0xbd, 0xe2, 0xb7, - 0x74, 0x18, 0x4a, 0x89, 0x8f, 0xe1, 0x7e, 0x8a, 0x4c, 0xae, 0x9b, 0xee, 0xdf, 0x45, 0x00, 0xf9, - 0xee, 0x53, 0xd5, 0x39, 0xf9, 0xde, 0x35, 0x52, 0xef, 0x5d, 0xd1, 0x17, 0x4e, 0x2c, 0xdf, 0x9a, - 0x38, 0x7c, 0xa9, 0x2f, 0x3d, 0x3d, 0x46, 0x8f, 0xa0, 0x66, 0xbd, 0xb2, 0x9c, 0x99, 0x35, 0x9e, - 0x51, 0x49, 0xba, 0x44, 0x62, 0x81, 0x28, 0x38, 0xa1, 0xe3, 0x55, 0x93, 0x57, 0x92, 0x4d, 0x5e, - 0x18, 0x79, 0xc7, 0xb2, 0xd5, 0xfb, 0x18, 0x10, 0x0b, 0x4b, 0x21, 0x73, 0x2d, 0x3f, 0x34, 0x2c, - 0x4b, 0xc3, 0x4e, 0xa8, 0x19, 0xb8, 0x96, 0xaf, 0xac, 0x9f, 0xc2, 0x56, 0x40, 0x27, 0xd4, 0x79, - 0x95, 0xb1, 0xaf, 0x48, 0x7b, 0x14, 0xe9, 0xe2, 0x19, 0xbb, 0x00, 0x8c, 0x5b, 0x01, 0x1f, 0x89, - 0x76, 0x51, 0x96, 0xc4, 0x26, 0xa9, 0x49, 0x89, 0x68, 0x25, 0x51, 0x0f, 0xee, 0x5b, 0xbe, 0x3f, - 0x5b, 0x66, 0xf0, 0xaa, 0xd2, 0xee, 0x9e, 0x56, 0xc5, 0x70, 0x3b, 0xb0, 0xe9, 0xb0, 0xd1, 0x78, - 0xc1, 0x96, 0xb2, 0x3a, 0x56, 0x49, 0xc5, 0x61, 0x47, 0x0b, 0xb6, 0x14, 0x69, 0xb9, 0x60, 0xd4, - 0x4e, 0x16, 0xc5, 0xaa, 0x10, 0x88, 0x6a, 0xb8, 0x5a, 0xbc, 0xeb, 0x6b, 0x8a, 0x77, 0xb6, 0x3a, - 0x37, 0x56, 0xab, 0x73, 0xba, 0xbe, 0x37, 0xb3, 0xf5, 0x3d, 0x55, 0xbc, 0x5b, 0xe9, 0xe2, 0x8d, - 0x67, 0xf0, 0x40, 0x1e, 0xf7, 0xbb, 0xde, 0x9b, 0x65, 0x26, 0xe2, 0x25, 0x5d, 0xe9, 0xe2, 0x38, - 0x22, 0x4a, 0x8d, 0x9f, 0xc1, 0x76, 0x76, 0xb5, 0x3c, 0x51, 0xfa, 0x11, 0x85, 0x5a, 0xf4, 0x79, - 0x04, 0x55, 0xa0, 0xd0, 0x7f, 0xd1, 0xd9, 0x40, 0x75, 0xd8, 0xfc, 0xe6, 0xfc, 0xc5, 0x79, 0xff, - 0xdb, 0xf3, 0x8e, 0x81, 0xb6, 0xa0, 0x73, 0xde, 0x1f, 0x8e, 0x8e, 0xfa, 0xfd, 0xe1, 0x60, 0x48, - 0x0e, 0x2f, 0x2e, 0x4e, 0x4f, 0x3a, 0x05, 0x74, 0x1f, 0xda, 0x83, 0x61, 0x9f, 0x9c, 0x8e, 0x86, - 0xfd, 0x97, 0x47, 0x83, 0x61, 0xff, 0xfc, 0xb4, 0x53, 0x44, 0x26, 0x6c, 0x1d, 0x7e, 0x4d, 0x4e, - 0x0f, 0x4f, 0xbe, 0x4b, 0x9b, 0x97, 0x0e, 0xfe, 0x5c, 0x85, 0xc2, 0xc5, 0x09, 0x3a, 0x04, 0x88, - 0x5f, 0x50, 0x68, 0x47, 0x31, 0x5b, 0x79, 0x96, 0x75, 0xcd, 0x55, 0x85, 0x22, 0x8f, 0x37, 0xd0, - 0x53, 0x28, 0x0e, 0x99, 0x87, 0x42, 0xc7, 0xc4, 0x5f, 0x6b, 0xba, 0xf7, 0x12, 0x12, 0x6d, 0xfd, - 0xc4, 0x78, 0x6a, 0xa0, 0x5f, 0x40, 0x2d, 0xea, 0xd1, 0xd1, 0xb6, 0xb2, 0xca, 0x7e, 0xcd, 0xe8, - 0xee, 0xac, 0xc8, 0xa3, 0x15, 0x5f, 0x42, 0x2b, 0xdd, 0xe5, 0xa3, 0x87, 0xca, 0x78, 0xed, 0x17, - 0x84, 0xee, 0xa3, 0xf5, 0xca, 0x08, 0xee, 0x33, 0xd8, 0x0c, 0x3b, 0x71, 0x14, 0x1e, 0x4d, 0xba, - 0xaf, 0xef, 0x3e, 0xc8, 0x48, 0xa3, 0x99, 0x9f, 0x43, 0x55, 0xf7, 0xc5, 0xe8, 0x41, 0xe4, 0xa2, - 0x64, 0x03, 0xdb, 0xdd, 0xce, 0x8a, 0x93, 0x93, 0x75, 0x23, 0xaa, 0x27, 0x67, 0xba, 0x5f, 0x3d, - 0x39, 0xdb, 0xaf, 0x2a, 0x17, 0xa4, 0xa3, 0x4d, 0xbb, 0x60, 0x6d, 0xc4, 0x6b, 0x17, 0xac, 0x0f, - 0x50, 0xbc, 0x81, 0x86, 0xd0, 0xce, 0xbc, 0x0e, 0xd0, 0x23, 0x1d, 0xa5, 0xeb, 0x9e, 0x9e, 0xdd, - 0xdd, 0x5b, 0xb4, 0xd9, 0x73, 0x8e, 0xda, 0x46, 0x14, 0x3b, 0x22, 0xd5, 0x39, 0x77, 0x77, 0x56, - 0xe4, 0x11, 0xab, 0x67, 0xd0, 0x4c, 0xb5, 0x9d, 0xa8, 0x9b, 0xb1, 0x4d, 0xf4, 0xa2, 0x6f, 0xc3, - 0xf9, 0x1c, 0xaa, 0xfa, 0x4e, 0xd5, 0x9e, 0xce, 0x5c, 0xe6, 0xda, 0xd3, 0xd9, 0xab, 0x17, 0x6f, - 0xa0, 0x13, 0xa8, 0x27, 0xae, 0x1e, 0x64, 0xea, 0x8d, 0x67, 0xaf, 0xc6, 0xee, 0x7b, 0x6b, 0x34, - 0x11, 0xca, 0x40, 0x7e, 0x33, 0x48, 0xf5, 0x6b, 0x68, 0x37, 0x62, 0xbc, 0xae, 0x75, 0xec, 0xee, - 0xdd, 0xa6, 0x4e, 0x82, 0x66, 0x9b, 0x40, 0x0d, 0x7a, 0x4b, 0x3f, 0xaa, 0x41, 0x6f, 0xeb, 0x1d, - 0xf1, 0xc6, 0xd1, 0x47, 0x7f, 0x7f, 0xb3, 0x67, 0xfc, 0xe3, 0xcd, 0x9e, 0xf1, 0xcf, 0x37, 0x7b, - 0xc6, 0x5f, 0xfe, 0xb5, 0xb7, 0x01, 0xe6, 0xc4, 0x9b, 0xf7, 0x7c, 0xc7, 0x9d, 0x4e, 0x2c, 0xbf, - 0xc7, 0x9d, 0xeb, 0x57, 0xbd, 0xeb, 0x57, 0xf2, 0x4b, 0xf3, 0xb8, 0x22, 0x7f, 0x7e, 0xf6, 0x9f, - 0x00, 0x00, 0x00, 0xff, 0xff, 0xc6, 0xc3, 0x58, 0x2b, 0xb7, 0x16, 0x00, 0x00, + // 1946 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x19, 0xcb, 0x72, 0x23, 0x49, + 0xd1, 0xad, 0x97, 0xa5, 0xd4, 0x73, 0xca, 0x2f, 0x8d, 0x66, 0x6c, 0xbc, 0x35, 0x0b, 0x78, 0x97, + 0x5d, 0xed, 0xac, 0x21, 0x88, 0x8d, 0xd8, 0x58, 0x62, 0xe5, 0xc7, 0xcc, 0x98, 0xd9, 0xb1, 0x14, + 0x25, 0x6d, 0x6c, 0xec, 0x05, 0xd1, 0x96, 0xca, 0x72, 0x63, 0xa9, 0xbb, 0xb7, 0xab, 0x34, 0x46, + 0x7b, 0xe2, 0xc4, 0x05, 0x22, 0xb8, 0x11, 0x1c, 0xb9, 0x13, 0xc1, 0x2f, 0x70, 0xe0, 0xc2, 0x91, + 0x4f, 0x20, 0x86, 0x33, 0xff, 0x40, 0xd4, 0xa3, 0x9f, 0x92, 0x8d, 0xe9, 0x81, 0x93, 0xd5, 0x99, + 0x59, 0x59, 0xf9, 0xce, 0xac, 0x34, 0x80, 0x3b, 0x76, 0x2f, 0xda, 0xae, 0xe7, 0x70, 0x07, 0xe5, + 0xc4, 0xef, 0x56, 0x65, 0x46, 0xb9, 0xe9, 0xc3, 0x5a, 0x55, 0xea, 0x99, 0x97, 0x3c, 0xf8, 0xdc, + 0x9c, 0x38, 0x13, 0x47, 0xfe, 0xfc, 0x48, 0xfc, 0x52, 0x50, 0xdc, 0x86, 0x2a, 0xa1, 0xdf, 0xcc, + 0x29, 0xe3, 0x2f, 0xa8, 0x39, 0xa6, 0x1e, 0xda, 0x05, 0x18, 0x4d, 0xe7, 0x8c, 0x53, 0x6f, 0x68, + 0x8d, 0x9b, 0xc6, 0xbe, 0x71, 0x90, 0x23, 0x25, 0x0d, 0x39, 0x1b, 0x63, 0x02, 0x35, 0x42, 0x99, + 0xeb, 0xd8, 0x8c, 0xde, 0xeb, 0x00, 0x7a, 0x07, 0xf2, 0xd4, 0xf3, 0x1c, 0xaf, 0x99, 0xd9, 0x37, + 0x0e, 0xca, 0x87, 0xe5, 0xb6, 0x94, 0xfa, 0x54, 0x80, 0x88, 0xc2, 0xe0, 0x67, 0x90, 0x97, 0xdf, + 0xe8, 0x09, 0xe4, 0xf8, 0xc2, 0xa5, 0x92, 0x49, 0xed, 0xb0, 0x1e, 0x21, 0x1d, 0x2c, 0x5c, 0x4a, + 0x24, 0x12, 0x35, 0x61, 0x7d, 0x46, 0x19, 0x33, 0x27, 0x54, 0xb2, 0x2c, 0x11, 0xff, 0x13, 0x77, + 0x01, 0x06, 0xcc, 0xd1, 0xea, 0xa0, 0x1f, 0x40, 0xe1, 0x4a, 0x4a, 0x28, 0xd9, 0x95, 0x0f, 0x37, + 0x14, 0xbb, 0x98, 0xb6, 0x44, 0x93, 0xa0, 0x4d, 0xc8, 0x8f, 0x9c, 0xb9, 0xcd, 0x25, 0xcb, 0x2a, + 0x51, 0x1f, 0xb8, 0x03, 0xa5, 0x81, 0x35, 0xa3, 0x8c, 0x9b, 0x33, 0x17, 0xb5, 0xa0, 0xe8, 0x5e, + 0x2d, 0x98, 0x35, 0x32, 0xa7, 0x92, 0x63, 0x96, 0x04, 0xdf, 0x42, 0xa6, 0xa9, 0x33, 0x91, 0xa8, + 0x8c, 0x44, 0xf9, 0x9f, 0xf8, 0x57, 0x06, 0x94, 0xa5, 0x50, 0xca, 0x66, 0xe8, 0x83, 0x84, 0x54, + 0x9b, 0xbe, 0x54, 0x51, 0x9b, 0xde, 0x2d, 0x16, 0xfa, 0x10, 0x4a, 0xdc, 0x17, 0xab, 0x99, 0x95, + 0x6c, 0xb4, 0xad, 0x02, 0x69, 0x49, 0x48, 0x81, 0x7f, 0x6b, 0x40, 0xe3, 0xc8, 0x71, 0x38, 0xe3, + 0x9e, 0xe9, 0xa6, 0xb2, 0xce, 0x13, 0xc8, 0x33, 0xee, 0x78, 0x54, 0xfb, 0xb0, 0xda, 0xd6, 0x71, + 0xd6, 0x17, 0x40, 0xa2, 0x70, 0xe8, 0x7b, 0x50, 0xf0, 0xe8, 0xc4, 0x72, 0x6c, 0x2d, 0x52, 0xcd, + 0xa7, 0x22, 0x12, 0x4a, 0x34, 0x16, 0x77, 0xe0, 0x41, 0x44, 0x9a, 0x34, 0x66, 0xc1, 0x27, 0xb0, + 0x75, 0xc6, 0x02, 0x26, 0x2e, 0x1d, 0xa7, 0xd1, 0x0a, 0xff, 0x02, 0xb6, 0x93, 0x5c, 0x52, 0x39, + 0x09, 0x43, 0xe5, 0x22, 0xc2, 0x45, 0x1a, 0xa9, 0x48, 0x62, 0x30, 0xfc, 0x19, 0xd4, 0x3a, 0xd3, + 0xa9, 0x33, 0x3a, 0x3b, 0x49, 0x25, 0x6a, 0x17, 0xea, 0xc1, 0xf1, 0x54, 0x32, 0xd6, 0x20, 0x63, + 0x29, 0xc9, 0x72, 0x24, 0x63, 0x8d, 0xf1, 0xd7, 0x50, 0x7f, 0x4e, 0xb9, 0xf2, 0x5f, 0x9a, 0x88, + 0x78, 0x08, 0x45, 0xe9, 0xf5, 0x61, 0xc0, 0x75, 0x5d, 0x7e, 0x9f, 0x8d, 0x31, 0x85, 0x46, 0xc8, + 0x3a, 0x95, 0xb0, 0xf7, 0x09, 0x37, 0x3c, 0x82, 0x7a, 0x6f, 0xfe, 0x16, 0x1a, 0xdc, 0xeb, 0x92, + 0xcf, 0xa1, 0x11, 0x5e, 0x92, 0x2a, 0x54, 0x8f, 0x60, 0xe3, 0x39, 0xe5, 0x9d, 0xe9, 0x54, 0x32, + 0x61, 0xa9, 0xbc, 0x7f, 0x0d, 0x9b, 0x71, 0x1e, 0xa9, 0xac, 0xfa, 0x5d, 0x28, 0x48, 0xa5, 0x58, + 0x33, 0xb3, 0x9f, 0x5d, 0xd6, 0x58, 0x23, 0xf1, 0xcf, 0xa4, 0xfb, 0x74, 0xce, 0xa6, 0x31, 0xec, + 0x2e, 0x80, 0xca, 0xf4, 0xe1, 0x35, 0x5d, 0x48, 0xeb, 0x56, 0x48, 0x49, 0x41, 0x5e, 0xd2, 0x05, + 0xfe, 0x9d, 0x01, 0x0f, 0x22, 0x17, 0xa4, 0x52, 0x25, 0x2c, 0x35, 0x99, 0xbb, 0x4a, 0x0d, 0x7a, + 0x17, 0x0a, 0x53, 0xc5, 0x55, 0x95, 0xa4, 0x8a, 0x4f, 0xd7, 0xa3, 0x82, 0x9b, 0xc2, 0xe1, 0x9f, + 0x4b, 0xf3, 0xaa, 0xa3, 0x47, 0x8b, 0x74, 0x19, 0x8a, 0x1e, 0x81, 0xd6, 0x31, 0xcc, 0x88, 0xa2, + 0x02, 0x9c, 0x8d, 0xf1, 0x33, 0xd8, 0x79, 0x4e, 0xf9, 0xb1, 0xea, 0x89, 0xc7, 0x8e, 0x7d, 0x69, + 0x4d, 0x52, 0x05, 0x02, 0x83, 0xe6, 0x32, 0x9f, 0x54, 0x16, 0x7c, 0x0f, 0xd6, 0x75, 0x8b, 0xd6, + 0x26, 0xac, 0xfb, 0xa6, 0xd1, 0xdc, 0x89, 0x8f, 0xc7, 0xdf, 0xc0, 0x4e, 0x6f, 0xfe, 0xf6, 0xc2, + 0xff, 0x37, 0x57, 0xbe, 0x80, 0xe6, 0xf2, 0x95, 0xa9, 0xd2, 0xef, 0x8f, 0x06, 0x14, 0x5e, 0xd1, + 0xd9, 0x05, 0xf5, 0x10, 0x82, 0x9c, 0x6d, 0xce, 0xd4, 0x70, 0x51, 0x22, 0xf2, 0xb7, 0xf0, 0xda, + 0x4c, 0x62, 0x23, 0x5e, 0x53, 0x80, 0xb3, 0xb1, 0x40, 0xba, 0x94, 0x7a, 0xc3, 0xb9, 0x37, 0x65, + 0xcd, 0xec, 0x7e, 0xf6, 0xa0, 0x44, 0x8a, 0x02, 0xf0, 0xa5, 0x37, 0x65, 0xe8, 0x3b, 0x50, 0x1e, + 0x4d, 0x2d, 0x6a, 0x73, 0x85, 0xce, 0x49, 0x34, 0x28, 0x90, 0x24, 0xf8, 0x3e, 0xd4, 0x55, 0x7c, + 0x0d, 0x5d, 0xcf, 0x72, 0x3c, 0x8b, 0x2f, 0x9a, 0xf9, 0x7d, 0xe3, 0x20, 0x4f, 0x6a, 0x0a, 0xdc, + 0xd3, 0x50, 0xfc, 0xb9, 0xcc, 0x07, 0x25, 0x64, 0xba, 0xfa, 0xf0, 0x17, 0x03, 0x50, 0x94, 0x45, + 0xca, 0x9c, 0x5a, 0x57, 0x9a, 0xfb, 0xf5, 0xa1, 0xa2, 0xc8, 0x15, 0x57, 0xe2, 0x23, 0x57, 0xe4, + 0x54, 0x94, 0x4c, 0xe3, 0xd0, 0x87, 0x50, 0xa6, 0x7c, 0x34, 0x1e, 0x6a, 0xd2, 0xdc, 0x0a, 0x52, + 0x10, 0x04, 0x5f, 0x28, 0x0d, 0x7a, 0x50, 0x12, 0x29, 0xd9, 0xe7, 0x26, 0x67, 0x68, 0x1f, 0x72, + 0xc2, 0xcc, 0x5a, 0xea, 0x78, 0xce, 0x4a, 0x0c, 0x7a, 0x07, 0x2a, 0x63, 0xe7, 0xc6, 0x1e, 0x32, + 0x3a, 0x72, 0xec, 0x31, 0xd3, 0x9e, 0x2b, 0x0b, 0x58, 0x5f, 0x81, 0xf0, 0x5f, 0xb3, 0xb0, 0xad, + 0x52, 0xfa, 0x05, 0x35, 0x3d, 0x7e, 0x41, 0x4d, 0x9e, 0x2a, 0x6a, 0xff, 0xa7, 0xa5, 0x06, 0xb5, + 0x01, 0xa4, 0xe0, 0x42, 0x0b, 0x15, 0x34, 0xc1, 0xe8, 0x16, 0xe8, 0x4f, 0x4a, 0x82, 0x44, 0x7c, + 0x32, 0xf4, 0x31, 0x54, 0x5d, 0x6a, 0x8f, 0x2d, 0x7b, 0xa2, 0x8f, 0xe4, 0xb5, 0x6b, 0xa2, 0xcc, + 0x2b, 0x9a, 0x44, 0x1d, 0x79, 0x02, 0xd5, 0x8b, 0x05, 0xa7, 0x6c, 0x78, 0xe3, 0x59, 0x9c, 0x53, + 0xbb, 0x59, 0x90, 0xc6, 0xa9, 0x48, 0xe0, 0x57, 0x0a, 0x26, 0x6a, 0xb4, 0x22, 0xf2, 0xa8, 0x39, + 0x6e, 0xae, 0xab, 0x99, 0x5d, 0x42, 0x08, 0x35, 0xc5, 0xcc, 0x5e, 0xb9, 0xa6, 0x8b, 0x90, 0x45, + 0x51, 0xd9, 0x57, 0xc0, 0x7c, 0x0e, 0x8f, 0xa0, 0x24, 0x49, 0x24, 0x83, 0x92, 0xca, 0x1c, 0x01, + 0x90, 0xe7, 0xdf, 0x83, 0x86, 0xe9, 0xba, 0x9e, 0xf3, 0x4b, 0x6b, 0x66, 0x72, 0x3a, 0x64, 0xd6, + 0xb7, 0xb4, 0x09, 0x92, 0xa6, 0x1e, 0x81, 0xf7, 0xad, 0x6f, 0x29, 0x7a, 0x1c, 0x9d, 0x65, 0xcb, + 0x4a, 0x90, 0x70, 0x74, 0xbd, 0x02, 0x38, 0xbe, 0x32, 0xed, 0x09, 0x15, 0xba, 0xdd, 0x23, 0x30, + 0x3e, 0x81, 0xf2, 0x48, 0xd2, 0x0f, 0xe5, 0x3b, 0x22, 0x23, 0xdf, 0x11, 0x3b, 0x6d, 0xff, 0x21, + 0x24, 0x4a, 0x89, 0xe2, 0x27, 0xdf, 0x13, 0x30, 0x0a, 0x7e, 0xe3, 0x43, 0xa8, 0x0d, 0x3c, 0xd3, + 0x66, 0x97, 0xd4, 0x53, 0x31, 0xf9, 0x9f, 0x6f, 0xc3, 0x1f, 0x41, 0xfe, 0x15, 0xf5, 0x26, 0x72, + 0xf4, 0xe5, 0xa6, 0x37, 0xa1, 0x5c, 0x13, 0x2f, 0x05, 0x89, 0xc2, 0xe2, 0x2a, 0x94, 0xfb, 0xee, + 0xd4, 0xd2, 0xbd, 0x06, 0xff, 0x3e, 0x0b, 0x3b, 0x4b, 0x31, 0x9a, 0x2a, 0x79, 0x3f, 0x0e, 0xf4, + 0x96, 0x22, 0xab, 0x50, 0x6d, 0xa8, 0x23, 0xa1, 0x01, 0x7d, 0x85, 0xa5, 0x31, 0x3f, 0x83, 0x3a, + 0xd7, 0x0a, 0x0f, 0x63, 0x91, 0xab, 0x6f, 0x8a, 0x5b, 0x83, 0xd4, 0x78, 0xdc, 0x3a, 0xb1, 0x7e, + 0x97, 0x8b, 0xf7, 0x3b, 0xf4, 0x63, 0xa8, 0x68, 0x24, 0x75, 0x9d, 0xd1, 0x95, 0x2c, 0x7c, 0x22, + 0xcf, 0x62, 0x56, 0x39, 0x15, 0x28, 0x52, 0xf6, 0xc2, 0x0f, 0x51, 0x35, 0x94, 0xa5, 0x94, 0x1a, + 0x85, 0x15, 0x96, 0x07, 0x45, 0xd0, 0x53, 0x65, 0x20, 0x3f, 0x13, 0xf6, 0x97, 0x01, 0x1c, 0x3c, + 0x2d, 0xa5, 0x4b, 0x88, 0xc2, 0xa0, 0x1f, 0x41, 0x85, 0x09, 0x8b, 0x0f, 0x75, 0x12, 0x17, 0x25, + 0xe5, 0x03, 0x45, 0x19, 0xf1, 0x05, 0x29, 0xb3, 0x88, 0x63, 0x2e, 0xa1, 0xde, 0x61, 0xd7, 0x1a, + 0xfd, 0xff, 0x2b, 0x1a, 0xf8, 0xd7, 0x06, 0x34, 0xc2, 0x8b, 0x52, 0x3e, 0x3e, 0xaa, 0x36, 0xbd, + 0x19, 0x26, 0x67, 0x8f, 0xb2, 0x4d, 0x6f, 0x88, 0xef, 0x8e, 0x7d, 0xa8, 0x08, 0x1a, 0xd9, 0xcc, + 0xac, 0xb1, 0xea, 0x65, 0x39, 0x02, 0x36, 0xbd, 0x11, 0x66, 0x3c, 0x1b, 0x33, 0xfc, 0x1b, 0x03, + 0x10, 0xa1, 0xae, 0xe3, 0xf1, 0xf4, 0x4a, 0x63, 0xc8, 0x4d, 0xe9, 0x25, 0xbf, 0x45, 0x65, 0x89, + 0x43, 0xef, 0x42, 0xde, 0xb3, 0x26, 0x57, 0xfc, 0x96, 0x27, 0xa2, 0x42, 0xe2, 0x63, 0xd8, 0x88, + 0x09, 0x93, 0xaa, 0xf3, 0xff, 0x2b, 0x0b, 0x20, 0x27, 0x5b, 0xd5, 0x54, 0xa2, 0x0f, 0x16, 0x23, + 0xf6, 0x60, 0x11, 0x0f, 0xfb, 0x91, 0xe9, 0x9a, 0x23, 0xd1, 0xa2, 0xf5, 0x0c, 0xe0, 0x7f, 0x8b, + 0xf2, 0x64, 0xbe, 0x36, 0xad, 0xa9, 0x79, 0x31, 0xa5, 0x52, 0xe8, 0x1c, 0x09, 0x01, 0xa2, 0x4e, + 0x6a, 0xc3, 0xab, 0x57, 0x7a, 0x4e, 0xbe, 0xd2, 0x75, 0x48, 0x1f, 0xcb, 0xb7, 0xfa, 0x07, 0x80, + 0x98, 0xae, 0xe0, 0xcc, 0x36, 0x5d, 0x4d, 0x98, 0x97, 0x84, 0x0d, 0x8d, 0xe9, 0xdb, 0xa6, 0xab, + 0xa8, 0x9f, 0xc2, 0xa6, 0x47, 0x47, 0xd4, 0x7a, 0x9d, 0xa0, 0x2f, 0x48, 0x7a, 0x14, 0xe0, 0xc2, + 0x13, 0xbb, 0x00, 0x8c, 0x9b, 0x1e, 0x1f, 0x8a, 0xa2, 0x29, 0x13, 0xa1, 0x4a, 0x4a, 0x12, 0x32, + 0xb0, 0x66, 0x14, 0xb5, 0x61, 0xc3, 0x74, 0xdd, 0xe9, 0x22, 0xc1, 0xaf, 0x28, 0xe9, 0x1e, 0xf8, + 0xa8, 0x90, 0xdd, 0x0e, 0xac, 0x5b, 0x6c, 0x78, 0x31, 0x67, 0x0b, 0x59, 0xd4, 0x8b, 0xa4, 0x60, + 0xb1, 0xa3, 0x39, 0x5b, 0x88, 0x7c, 0x9f, 0x33, 0x3a, 0x8e, 0xd6, 0xf2, 0xa2, 0x00, 0xc8, 0x22, + 0xbe, 0xd4, 0x73, 0xca, 0x2b, 0x7a, 0x4e, 0xb2, 0xa9, 0x54, 0x96, 0x9b, 0x4a, 0xbc, 0x2d, 0x55, + 0x93, 0x6d, 0x29, 0xd6, 0x73, 0x6a, 0xf1, 0x9e, 0x83, 0xa7, 0xb0, 0x25, 0xdd, 0xfd, 0xb6, 0xed, + 0x3e, 0xcf, 0x44, 0xbc, 0xc4, 0x4b, 0x68, 0x18, 0x47, 0x44, 0xa1, 0xf1, 0x33, 0xd8, 0x4e, 0xde, + 0x96, 0x2a, 0x4a, 0xff, 0x6c, 0xc0, 0x66, 0x7f, 0x64, 0x72, 0x31, 0xfe, 0xa6, 0x7f, 0x72, 0xdd, + 0xf5, 0xf8, 0xb8, 0xef, 0x5e, 0x26, 0x32, 0xc1, 0xe4, 0xee, 0x78, 0x2c, 0x9d, 0xc2, 0x56, 0x42, + 0xde, 0x34, 0x7a, 0xbf, 0x4f, 0xa1, 0x14, 0xec, 0xf5, 0x50, 0x01, 0x32, 0xdd, 0x97, 0x8d, 0x35, + 0x54, 0x86, 0xf5, 0x2f, 0xcf, 0x5f, 0x9e, 0x77, 0xbf, 0x3a, 0x6f, 0x18, 0x68, 0x13, 0x1a, 0xe7, + 0xdd, 0xc1, 0xf0, 0xa8, 0xdb, 0x1d, 0xf4, 0x07, 0xa4, 0xd3, 0xeb, 0x9d, 0x9e, 0x34, 0x32, 0x68, + 0x03, 0xea, 0xfd, 0x41, 0x97, 0x9c, 0x0e, 0x07, 0xdd, 0x57, 0x47, 0xfd, 0x41, 0xf7, 0xfc, 0xb4, + 0x91, 0x45, 0x4d, 0xd8, 0xec, 0x7c, 0x41, 0x4e, 0x3b, 0x27, 0x5f, 0xc7, 0xc9, 0x73, 0x87, 0x7f, + 0x2a, 0x41, 0xa6, 0x77, 0x82, 0x3a, 0x00, 0xe1, 0x7c, 0x8c, 0x76, 0x94, 0x64, 0x4b, 0x43, 0x77, + 0xab, 0xb9, 0x8c, 0x50, 0xc2, 0xe3, 0x35, 0xf4, 0x14, 0xb2, 0x03, 0xe6, 0x20, 0x1d, 0x10, 0xe1, + 0x9a, 0xb1, 0xf5, 0x20, 0x02, 0xf1, 0xa9, 0x0f, 0x8c, 0xa7, 0x06, 0xfa, 0x09, 0x94, 0x82, 0xe5, + 0x12, 0xda, 0x56, 0x54, 0xc9, 0x35, 0x5c, 0x6b, 0x67, 0x09, 0x1e, 0xdc, 0xf8, 0x0a, 0x6a, 0xf1, + 0xf5, 0x14, 0x7a, 0xa4, 0x88, 0x57, 0xae, 0xbe, 0x5a, 0x8f, 0x57, 0x23, 0x03, 0x76, 0x9f, 0xc0, + 0xba, 0x5e, 0x21, 0x21, 0xed, 0x9a, 0xf8, 0x42, 0xaa, 0xb5, 0x95, 0x80, 0x06, 0x27, 0x3f, 0x85, + 0xa2, 0xbf, 0xd0, 0x41, 0x5b, 0x81, 0x89, 0xa2, 0x9b, 0x97, 0xd6, 0x76, 0x12, 0x1c, 0x3d, 0xec, + 0x6f, 0x50, 0xfc, 0xc3, 0x89, 0xb5, 0x8d, 0x7f, 0x38, 0xb9, 0x68, 0xc1, 0x6b, 0xe8, 0x39, 0x54, + 0xa2, 0x8b, 0x0f, 0xf4, 0x30, 0xb8, 0x26, 0xb9, 0x50, 0x69, 0xb5, 0x56, 0xa1, 0xa2, 0xb6, 0x8c, + 0xa7, 0xab, 0x6f, 0xcb, 0x95, 0x25, 0xc3, 0xb7, 0xe5, 0xea, 0x0c, 0xc7, 0x6b, 0x68, 0x00, 0xf5, + 0xc4, 0xdc, 0x86, 0x1e, 0xfb, 0xe1, 0xbe, 0xea, 0xc9, 0xd1, 0xda, 0xbd, 0x05, 0x9b, 0x0c, 0x98, + 0x60, 0x0f, 0x81, 0x42, 0x8b, 0xc6, 0xea, 0x42, 0x6b, 0x67, 0x09, 0x1e, 0x48, 0xf5, 0x0c, 0xaa, + 0xb1, 0x3d, 0x06, 0x6a, 0x25, 0x68, 0x23, 0xcb, 0x8d, 0xbb, 0xf8, 0x7c, 0x0a, 0x45, 0x7f, 0x28, + 0xf1, 0x5d, 0x96, 0x98, 0x86, 0x7c, 0x97, 0x25, 0x67, 0x17, 0xbc, 0x86, 0x4e, 0xa0, 0x1c, 0xe9, + 0xdd, 0xa8, 0xe9, 0x2b, 0x9e, 0x9c, 0x2d, 0x5a, 0x0f, 0x57, 0x60, 0x02, 0x2e, 0x7d, 0xb9, 0x84, + 0x8a, 0x2d, 0x00, 0xd0, 0x6e, 0x20, 0xf1, 0xaa, 0x5d, 0x44, 0x6b, 0xef, 0x36, 0x74, 0x94, 0x69, + 0x72, 0xab, 0xe0, 0x33, 0xbd, 0x65, 0xc1, 0xe1, 0x33, 0xbd, 0x6d, 0x19, 0x81, 0xd7, 0xd0, 0x4f, + 0xa1, 0x1a, 0xab, 0x87, 0xbe, 0xd1, 0x57, 0x15, 0xf5, 0xd6, 0xa3, 0x95, 0x38, 0x9f, 0xd7, 0xd1, + 0xfb, 0x7f, 0x7b, 0xb3, 0x67, 0xfc, 0xfd, 0xcd, 0x9e, 0xf1, 0x8f, 0x37, 0x7b, 0xc6, 0x1f, 0xfe, + 0xb9, 0xb7, 0x06, 0xcd, 0x91, 0x33, 0x6b, 0xbb, 0x96, 0x3d, 0x19, 0x99, 0x6e, 0x9b, 0x5b, 0xd7, + 0xaf, 0xdb, 0xd7, 0xaf, 0xe5, 0xff, 0x6d, 0x2e, 0x0a, 0xf2, 0xcf, 0x0f, 0xff, 0x1d, 0x00, 0x00, + 0xff, 0xff, 0x93, 0x54, 0x35, 0xa7, 0x05, 0x1a, 0x00, 0x00, } diff --git a/vendor/github.com/pingcap/kvproto/pkg/raft_serverpb/raft_serverpb.pb.go b/vendor/github.com/pingcap/kvproto/pkg/raft_serverpb/raft_serverpb.pb.go index 7b9ca0d98f270..3f99653ca02a6 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/raft_serverpb/raft_serverpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/raft_serverpb/raft_serverpb.pb.go @@ -1,5 +1,6 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. +// Code generated by protoc-gen-gogo. // source: raft_serverpb.proto +// DO NOT EDIT! /* Package raft_serverpb is a generated protocol buffer package. @@ -19,6 +20,7 @@ StoreIdent RaftLocalState RaftApplyState + MergeState RegionLocalState */ package raft_serverpb @@ -52,17 +54,20 @@ const ( PeerState_Normal PeerState = 0 PeerState_Applying PeerState = 1 PeerState_Tombstone PeerState = 2 + PeerState_Merging PeerState = 3 ) var PeerState_name = map[int32]string{ 0: "Normal", 1: "Applying", 2: "Tombstone", + 3: "Merging", } var PeerState_value = map[string]int32{ "Normal": 0, "Applying": 1, "Tombstone": 2, + "Merging": 3, } func (x PeerState) String() string { @@ -81,6 +86,8 @@ type RaftMessage struct { // Region key range [start_key, end_key). StartKey []byte `protobuf:"bytes,7,opt,name=start_key,json=startKey,proto3" json:"start_key,omitempty"` EndKey []byte `protobuf:"bytes,8,opt,name=end_key,json=endKey,proto3" json:"end_key,omitempty"` + // If it has value, to_peer should be removed if merge is never going to complete. + MergeTarget *metapb.Region `protobuf:"bytes,9,opt,name=merge_target,json=mergeTarget" json:"merge_target,omitempty"` } func (m *RaftMessage) Reset() { *m = RaftMessage{} } @@ -144,6 +151,13 @@ func (m *RaftMessage) GetEndKey() []byte { return nil } +func (m *RaftMessage) GetMergeTarget() *metapb.Region { + if m != nil { + return m.MergeTarget + } + return nil +} + type RaftTruncatedState struct { Index uint64 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"` Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` @@ -392,15 +406,48 @@ func (m *RaftApplyState) GetTruncatedState() *RaftTruncatedState { return nil } +type MergeState struct { + MinIndex uint64 `protobuf:"varint,1,opt,name=min_index,json=minIndex,proto3" json:"min_index,omitempty"` + Target *metapb.Region `protobuf:"bytes,2,opt,name=target" json:"target,omitempty"` + Commit uint64 `protobuf:"varint,3,opt,name=commit,proto3" json:"commit,omitempty"` +} + +func (m *MergeState) Reset() { *m = MergeState{} } +func (m *MergeState) String() string { return proto.CompactTextString(m) } +func (*MergeState) ProtoMessage() {} +func (*MergeState) Descriptor() ([]byte, []int) { return fileDescriptorRaftServerpb, []int{11} } + +func (m *MergeState) GetMinIndex() uint64 { + if m != nil { + return m.MinIndex + } + return 0 +} + +func (m *MergeState) GetTarget() *metapb.Region { + if m != nil { + return m.Target + } + return nil +} + +func (m *MergeState) GetCommit() uint64 { + if m != nil { + return m.Commit + } + return 0 +} + type RegionLocalState struct { - State PeerState `protobuf:"varint,1,opt,name=state,proto3,enum=raft_serverpb.PeerState" json:"state,omitempty"` - Region *metapb.Region `protobuf:"bytes,2,opt,name=region" json:"region,omitempty"` + State PeerState `protobuf:"varint,1,opt,name=state,proto3,enum=raft_serverpb.PeerState" json:"state,omitempty"` + Region *metapb.Region `protobuf:"bytes,2,opt,name=region" json:"region,omitempty"` + MergeState *MergeState `protobuf:"bytes,3,opt,name=merge_state,json=mergeState" json:"merge_state,omitempty"` } func (m *RegionLocalState) Reset() { *m = RegionLocalState{} } func (m *RegionLocalState) String() string { return proto.CompactTextString(m) } func (*RegionLocalState) ProtoMessage() {} -func (*RegionLocalState) Descriptor() ([]byte, []int) { return fileDescriptorRaftServerpb, []int{11} } +func (*RegionLocalState) Descriptor() ([]byte, []int) { return fileDescriptorRaftServerpb, []int{12} } func (m *RegionLocalState) GetState() PeerState { if m != nil { @@ -416,6 +463,13 @@ func (m *RegionLocalState) GetRegion() *metapb.Region { return nil } +func (m *RegionLocalState) GetMergeState() *MergeState { + if m != nil { + return m.MergeState + } + return nil +} + func init() { proto.RegisterType((*RaftMessage)(nil), "raft_serverpb.RaftMessage") proto.RegisterType((*RaftTruncatedState)(nil), "raft_serverpb.RaftTruncatedState") @@ -428,6 +482,7 @@ func init() { proto.RegisterType((*StoreIdent)(nil), "raft_serverpb.StoreIdent") proto.RegisterType((*RaftLocalState)(nil), "raft_serverpb.RaftLocalState") proto.RegisterType((*RaftApplyState)(nil), "raft_serverpb.RaftApplyState") + proto.RegisterType((*MergeState)(nil), "raft_serverpb.MergeState") proto.RegisterType((*RegionLocalState)(nil), "raft_serverpb.RegionLocalState") proto.RegisterEnum("raft_serverpb.PeerState", PeerState_name, PeerState_value) } @@ -513,6 +568,16 @@ func (m *RaftMessage) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintRaftServerpb(dAtA, i, uint64(len(m.EndKey))) i += copy(dAtA[i:], m.EndKey) } + if m.MergeTarget != nil { + dAtA[i] = 0x4a + i++ + i = encodeVarintRaftServerpb(dAtA, i, uint64(m.MergeTarget.Size())) + n5, err := m.MergeTarget.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n5 + } return i, nil } @@ -627,11 +692,11 @@ func (m *SnapshotChunk) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRaftServerpb(dAtA, i, uint64(m.Message.Size())) - n5, err := m.Message.MarshalTo(dAtA[i:]) + n6, err := m.Message.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n5 + i += n6 } if len(m.Data) > 0 { dAtA[i] = 0x12 @@ -709,11 +774,11 @@ func (m *RaftSnapshotData) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRaftServerpb(dAtA, i, uint64(m.Region.Size())) - n6, err := m.Region.MarshalTo(dAtA[i:]) + n7, err := m.Region.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n6 + i += n7 } if m.FileSize != 0 { dAtA[i] = 0x10 @@ -741,11 +806,11 @@ func (m *RaftSnapshotData) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintRaftServerpb(dAtA, i, uint64(m.Meta.Size())) - n7, err := m.Meta.MarshalTo(dAtA[i:]) + n8, err := m.Meta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n7 + i += n8 } return i, nil } @@ -797,11 +862,11 @@ func (m *RaftLocalState) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintRaftServerpb(dAtA, i, uint64(m.HardState.Size())) - n8, err := m.HardState.MarshalTo(dAtA[i:]) + n9, err := m.HardState.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n8 + i += n9 } if m.LastIndex != 0 { dAtA[i] = 0x10 @@ -835,11 +900,49 @@ func (m *RaftApplyState) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintRaftServerpb(dAtA, i, uint64(m.TruncatedState.Size())) - n9, err := m.TruncatedState.MarshalTo(dAtA[i:]) + n10, err := m.TruncatedState.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n10 + } + return i, nil +} + +func (m *MergeState) 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 *MergeState) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.MinIndex != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintRaftServerpb(dAtA, i, uint64(m.MinIndex)) + } + if m.Target != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintRaftServerpb(dAtA, i, uint64(m.Target.Size())) + n11, err := m.Target.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n11 + } + if m.Commit != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintRaftServerpb(dAtA, i, uint64(m.Commit)) } return i, nil } @@ -868,15 +971,43 @@ func (m *RegionLocalState) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintRaftServerpb(dAtA, i, uint64(m.Region.Size())) - n10, err := m.Region.MarshalTo(dAtA[i:]) + n12, err := m.Region.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n10 + i += n12 + } + if m.MergeState != nil { + dAtA[i] = 0x1a + i++ + i = encodeVarintRaftServerpb(dAtA, i, uint64(m.MergeState.Size())) + n13, err := m.MergeState.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n13 } return i, nil } +func encodeFixed64RaftServerpb(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + dAtA[offset+4] = uint8(v >> 32) + dAtA[offset+5] = uint8(v >> 40) + dAtA[offset+6] = uint8(v >> 48) + dAtA[offset+7] = uint8(v >> 56) + return offset + 8 +} +func encodeFixed32RaftServerpb(dAtA []byte, offset int, v uint32) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + return offset + 4 +} func encodeVarintRaftServerpb(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -919,6 +1050,10 @@ func (m *RaftMessage) Size() (n int) { if l > 0 { n += 1 + l + sovRaftServerpb(uint64(l)) } + if m.MergeTarget != nil { + l = m.MergeTarget.Size() + n += 1 + l + sovRaftServerpb(uint64(l)) + } return n } @@ -1060,6 +1195,22 @@ func (m *RaftApplyState) Size() (n int) { return n } +func (m *MergeState) Size() (n int) { + var l int + _ = l + if m.MinIndex != 0 { + n += 1 + sovRaftServerpb(uint64(m.MinIndex)) + } + if m.Target != nil { + l = m.Target.Size() + n += 1 + l + sovRaftServerpb(uint64(l)) + } + if m.Commit != 0 { + n += 1 + sovRaftServerpb(uint64(m.Commit)) + } + return n +} + func (m *RegionLocalState) Size() (n int) { var l int _ = l @@ -1070,6 +1221,10 @@ func (m *RegionLocalState) Size() (n int) { l = m.Region.Size() n += 1 + l + sovRaftServerpb(uint64(l)) } + if m.MergeState != nil { + l = m.MergeState.Size() + n += 1 + l + sovRaftServerpb(uint64(l)) + } return n } @@ -1348,6 +1503,39 @@ func (m *RaftMessage) Unmarshal(dAtA []byte) error { m.EndKey = []byte{} } iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MergeTarget", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaftServerpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRaftServerpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MergeTarget == nil { + m.MergeTarget = &metapb.Region{} + } + if err := m.MergeTarget.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRaftServerpb(dAtA[iNdEx:]) @@ -2408,6 +2596,127 @@ func (m *RaftApplyState) Unmarshal(dAtA []byte) error { } return nil } +func (m *MergeState) 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 ErrIntOverflowRaftServerpb + } + 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: MergeState: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MergeState: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MinIndex", wireType) + } + m.MinIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaftServerpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MinIndex |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Target", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaftServerpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRaftServerpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Target == nil { + m.Target = &metapb.Region{} + } + if err := m.Target.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Commit", wireType) + } + m.Commit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaftServerpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Commit |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipRaftServerpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRaftServerpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *RegionLocalState) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -2489,6 +2798,39 @@ func (m *RegionLocalState) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MergeState", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRaftServerpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRaftServerpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.MergeState == nil { + m.MergeState = &MergeState{} + } + if err := m.MergeState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRaftServerpb(dAtA[iNdEx:]) @@ -2618,54 +2960,60 @@ var ( func init() { proto.RegisterFile("raft_serverpb.proto", fileDescriptorRaftServerpb) } var fileDescriptorRaftServerpb = []byte{ - // 774 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x54, 0xdb, 0x8e, 0xdb, 0x44, - 0x18, 0xae, 0xb3, 0xde, 0xc4, 0xfe, 0xe3, 0x84, 0x68, 0x8a, 0x54, 0xb3, 0x55, 0xa3, 0xd4, 0x08, - 0x14, 0x16, 0xc9, 0x88, 0x50, 0x21, 0xae, 0x90, 0x80, 0xb2, 0x6a, 0x58, 0x8a, 0xaa, 0x49, 0x85, - 0xc4, 0x95, 0x35, 0xb1, 0x7f, 0x27, 0x26, 0x3e, 0x69, 0x66, 0x12, 0x11, 0xee, 0x78, 0x0b, 0x5e, - 0x85, 0x37, 0xe0, 0x0e, 0x1e, 0x01, 0x2d, 0x2f, 0x82, 0x66, 0xc6, 0xce, 0x61, 0x55, 0x7a, 0xe5, - 0xff, 0x7c, 0xf8, 0xfe, 0xcf, 0x03, 0x0f, 0x39, 0x4b, 0x65, 0x24, 0x90, 0xef, 0x90, 0xd7, 0xcb, - 0xb0, 0xe6, 0x95, 0xac, 0xc8, 0xe0, 0xcc, 0x78, 0x35, 0x40, 0xa5, 0xb7, 0xde, 0x2b, 0xaf, 0x40, - 0xc9, 0x5a, 0x2d, 0xf8, 0xa3, 0x03, 0x7d, 0xca, 0x52, 0xf9, 0x12, 0x85, 0x60, 0x2b, 0x24, 0x8f, - 0xc1, 0xe5, 0xb8, 0xca, 0xaa, 0x32, 0xca, 0x12, 0xdf, 0x9a, 0x58, 0x53, 0x9b, 0x3a, 0xc6, 0x30, - 0x4f, 0xc8, 0x47, 0xe0, 0xa6, 0xbc, 0x2a, 0xa2, 0x1a, 0x91, 0xfb, 0x9d, 0x89, 0x35, 0xed, 0xcf, - 0xbc, 0xb0, 0x29, 0xf7, 0x0a, 0x91, 0x53, 0x47, 0xb9, 0x95, 0x44, 0x3e, 0x80, 0x9e, 0xac, 0x4c, - 0xe0, 0xc5, 0x1b, 0x02, 0xbb, 0xb2, 0xd2, 0x61, 0xd7, 0xd0, 0x2b, 0x4c, 0x67, 0xdf, 0xd6, 0x61, - 0xa3, 0xb0, 0x9d, 0xb6, 0x99, 0x88, 0xb6, 0x01, 0xe4, 0x73, 0xf0, 0x9a, 0xd1, 0xb0, 0xae, 0xe2, - 0xb5, 0x7f, 0xa9, 0x13, 0x1e, 0xb6, 0x75, 0xa9, 0xf6, 0x7d, 0xab, 0x5c, 0xb4, 0xcf, 0x8f, 0x0a, - 0x79, 0x0a, 0x5e, 0x26, 0x22, 0x59, 0x15, 0x4b, 0x21, 0xab, 0x12, 0xfd, 0xee, 0xc4, 0x9a, 0x3a, - 0xb4, 0x9f, 0x89, 0xd7, 0xad, 0x49, 0x6d, 0x2d, 0x24, 0xe3, 0x32, 0xda, 0xe0, 0xde, 0xef, 0x4d, - 0xac, 0xa9, 0x47, 0x1d, 0x6d, 0xb8, 0xc5, 0x3d, 0x79, 0x04, 0x3d, 0x2c, 0x13, 0xed, 0x72, 0xb4, - 0xab, 0x8b, 0x65, 0x72, 0x8b, 0xfb, 0xe0, 0x4b, 0x20, 0x0a, 0xba, 0xd7, 0x7c, 0x5b, 0xc6, 0x4c, - 0x62, 0xb2, 0x90, 0x4c, 0x22, 0x79, 0x17, 0x2e, 0xb3, 0x32, 0xc1, 0x5f, 0x1a, 0xf4, 0x8c, 0x42, - 0x08, 0xd8, 0x12, 0x79, 0xa1, 0x51, 0xb3, 0xa9, 0x96, 0x83, 0x57, 0x30, 0x5c, 0x94, 0xac, 0x16, - 0xeb, 0x4a, 0x7e, 0x73, 0x73, 0x93, 0xe5, 0x48, 0x86, 0xd0, 0x89, 0x53, 0x9d, 0xe8, 0xd2, 0x4e, - 0x9c, 0xaa, 0x2c, 0x91, 0xfd, 0x8a, 0x6d, 0x96, 0x92, 0xc9, 0x15, 0x38, 0xf1, 0x1a, 0xe3, 0x8d, - 0xd8, 0x16, 0x1a, 0xda, 0x01, 0x3d, 0xe8, 0xc1, 0x0b, 0xf0, 0xda, 0x8a, 0x2f, 0x51, 0x32, 0xf2, - 0x05, 0x38, 0x71, 0x1a, 0xa5, 0x59, 0x8e, 0xc2, 0xb7, 0x26, 0x17, 0xd3, 0xfe, 0xec, 0x49, 0x78, - 0xce, 0x98, 0xf3, 0x01, 0x68, 0x2f, 0x4e, 0xd5, 0x57, 0x04, 0x3f, 0xc1, 0xe0, 0xe0, 0x5a, 0x6f, - 0xcb, 0x0d, 0x79, 0x76, 0xbc, 0x94, 0xa5, 0x81, 0xbf, 0xba, 0x57, 0xe9, 0x84, 0x45, 0xc7, 0x9b, - 0x11, 0xb0, 0x13, 0x26, 0x99, 0x5e, 0xc0, 0xa3, 0x5a, 0x0e, 0xba, 0x60, 0x3f, 0xaf, 0x4a, 0x0c, - 0x66, 0xe0, 0xdc, 0xe2, 0xfe, 0x47, 0x96, 0x6f, 0x91, 0x8c, 0xe0, 0x42, 0xe1, 0x6b, 0xe9, 0x30, - 0x25, 0x2a, 0x18, 0x77, 0xca, 0xd5, 0xa4, 0x1a, 0x25, 0xf8, 0xcb, 0x82, 0x91, 0x6a, 0xd4, 0xce, - 0xf6, 0x9c, 0x49, 0x46, 0x3e, 0x84, 0xae, 0xb9, 0x77, 0x33, 0xd9, 0xf0, 0x9c, 0x12, 0xb4, 0xf1, - 0xaa, 0x2b, 0x2b, 0x28, 0xa2, 0x13, 0x48, 0x1d, 0x65, 0x58, 0x28, 0x58, 0x3f, 0x6e, 0x26, 0xbd, - 0xd0, 0x30, 0x3d, 0xba, 0xb7, 0x5c, 0x3b, 0xa8, 0x59, 0x81, 0xf8, 0xd0, 0xdb, 0x21, 0x17, 0xaa, - 0xa5, 0xad, 0xeb, 0xb4, 0x2a, 0xf9, 0x04, 0x6c, 0xd5, 0xbc, 0x21, 0xe7, 0xe3, 0xff, 0x41, 0x5b, - 0x1d, 0x87, 0xea, 0xc0, 0xe0, 0x06, 0x60, 0x21, 0x2b, 0x8e, 0xf3, 0x04, 0x4b, 0x49, 0x9e, 0x00, - 0xc4, 0xf9, 0x56, 0x48, 0xe4, 0xc7, 0xff, 0xcf, 0x6d, 0x2c, 0xf3, 0x84, 0xbc, 0x07, 0x8e, 0x50, - 0xc1, 0xca, 0x69, 0x16, 0xe8, 0x09, 0x93, 0x1c, 0x2c, 0x61, 0xa8, 0x80, 0xf9, 0xbe, 0x8a, 0x59, - 0x6e, 0x88, 0xf8, 0x29, 0xc0, 0x9a, 0xf1, 0x24, 0x12, 0x4a, 0x6b, 0xa0, 0x21, 0x87, 0xdf, 0xeb, - 0x05, 0xe3, 0x86, 0xb0, 0xd4, 0x5d, 0xb7, 0xa2, 0x6a, 0x9f, 0x33, 0x21, 0x23, 0x43, 0x60, 0xd3, - 0xc1, 0x55, 0x96, 0xb9, 0x32, 0x04, 0xbf, 0x59, 0xa6, 0xc9, 0x57, 0x75, 0x9d, 0xef, 0x4d, 0xc6, - 0xfb, 0x30, 0x60, 0x75, 0x9d, 0x67, 0x98, 0x44, 0xa7, 0xac, 0xf7, 0x1a, 0xa3, 0xce, 0x23, 0xdf, - 0xc1, 0x3b, 0xb2, 0xfd, 0x49, 0x9a, 0x71, 0xcc, 0xeb, 0xf1, 0xf4, 0x0d, 0x1c, 0x3a, 0xff, 0x9d, - 0xe8, 0x50, 0x9e, 0xe9, 0xc1, 0xcf, 0x30, 0x32, 0x67, 0x3d, 0xd9, 0x34, 0x84, 0xcb, 0xe3, 0x92, - 0xc3, 0x99, 0x7f, 0xaf, 0xaa, 0x7a, 0x69, 0x4c, 0x31, 0x13, 0x76, 0x42, 0x98, 0xce, 0xdb, 0x08, - 0x73, 0xfd, 0x0c, 0xdc, 0x43, 0x2e, 0x01, 0xe8, 0xfe, 0x50, 0xf1, 0x82, 0xe5, 0xa3, 0x07, 0xc4, - 0x03, 0x47, 0x63, 0x90, 0x95, 0xab, 0x91, 0x45, 0x06, 0xe0, 0x1e, 0x9e, 0x92, 0x51, 0xe7, 0xeb, - 0xeb, 0x3f, 0xef, 0xc6, 0xd6, 0xdf, 0x77, 0x63, 0xeb, 0x9f, 0xbb, 0xb1, 0xf5, 0xfb, 0xbf, 0xe3, - 0x07, 0xe0, 0xc7, 0x55, 0x11, 0xd6, 0x59, 0xb9, 0x8a, 0x59, 0x1d, 0xca, 0x6c, 0xb3, 0x0b, 0x37, - 0x3b, 0xfd, 0xfc, 0x2e, 0xbb, 0xfa, 0xf3, 0xd9, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x13, 0x81, - 0xd5, 0xd2, 0xc8, 0x05, 0x00, 0x00, + // 866 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x55, 0xdd, 0x6e, 0xe3, 0x44, + 0x14, 0x5e, 0x27, 0xae, 0x63, 0x9f, 0x38, 0x21, 0x9a, 0x45, 0xac, 0xb7, 0xd5, 0x56, 0x59, 0x23, + 0x56, 0xa1, 0x48, 0x41, 0x5b, 0x10, 0x42, 0x5c, 0x20, 0x2d, 0x2c, 0xd5, 0x96, 0xa5, 0xa8, 0x9a, + 0x56, 0x48, 0x5c, 0x59, 0x53, 0xfb, 0x38, 0xb1, 0xe2, 0x3f, 0xcd, 0x4c, 0x22, 0xc2, 0x1d, 0x6f, + 0xc1, 0x13, 0xf0, 0x2c, 0xdc, 0xc1, 0x23, 0xa0, 0x72, 0xcb, 0x43, 0xa0, 0x99, 0xb1, 0xf3, 0xa7, + 0xc2, 0x95, 0xe7, 0xfc, 0xcd, 0x39, 0xe7, 0x3b, 0xdf, 0x19, 0xc3, 0x63, 0xce, 0x52, 0x19, 0x09, + 0xe4, 0x2b, 0xe4, 0xf5, 0xdd, 0xb4, 0xe6, 0x95, 0xac, 0xc8, 0x60, 0x4f, 0x79, 0x3c, 0x40, 0x25, + 0xb7, 0xd6, 0x63, 0xbf, 0x40, 0xc9, 0x5a, 0x29, 0xfc, 0xa7, 0x03, 0x7d, 0xca, 0x52, 0x79, 0x85, + 0x42, 0xb0, 0x19, 0x92, 0x13, 0xf0, 0x38, 0xce, 0xb2, 0xaa, 0x8c, 0xb2, 0x24, 0xb0, 0xc6, 0xd6, + 0xc4, 0xa6, 0xae, 0x51, 0x5c, 0x26, 0xe4, 0x43, 0xf0, 0x52, 0x5e, 0x15, 0x51, 0x8d, 0xc8, 0x83, + 0xce, 0xd8, 0x9a, 0xf4, 0xcf, 0xfd, 0x69, 0x73, 0xdd, 0x35, 0x22, 0xa7, 0xae, 0x32, 0xab, 0x13, + 0xf9, 0x00, 0x7a, 0xb2, 0x32, 0x8e, 0xdd, 0x07, 0x1c, 0x1d, 0x59, 0x69, 0xb7, 0x33, 0xe8, 0x15, + 0x26, 0x73, 0x60, 0x6b, 0xb7, 0xd1, 0xb4, 0xad, 0xb6, 0xa9, 0x88, 0xb6, 0x0e, 0xe4, 0x33, 0xf0, + 0x9b, 0xd2, 0xb0, 0xae, 0xe2, 0x79, 0x70, 0xa4, 0x03, 0x1e, 0xb7, 0xf7, 0x52, 0x6d, 0xfb, 0x46, + 0x99, 0x68, 0x9f, 0x6f, 0x05, 0xf2, 0x1c, 0xfc, 0x4c, 0x44, 0xb2, 0x2a, 0xee, 0x84, 0xac, 0x4a, + 0x0c, 0x9c, 0xb1, 0x35, 0x71, 0x69, 0x3f, 0x13, 0xb7, 0xad, 0x4a, 0x75, 0x2d, 0x24, 0xe3, 0x32, + 0x5a, 0xe0, 0x3a, 0xe8, 0x8d, 0xad, 0x89, 0x4f, 0x5d, 0xad, 0x78, 0x8b, 0x6b, 0xf2, 0x04, 0x7a, + 0x58, 0x26, 0xda, 0xe4, 0x6a, 0x93, 0x83, 0x65, 0xa2, 0x0c, 0x2f, 0xc1, 0x2f, 0x90, 0xcf, 0x30, + 0x92, 0x8c, 0xcf, 0x50, 0x06, 0x9e, 0x2e, 0x68, 0xb8, 0x5f, 0x10, 0xed, 0x6b, 0x9f, 0x5b, 0xed, + 0x12, 0x7e, 0x09, 0x44, 0xa1, 0x7d, 0xcb, 0x97, 0x65, 0xcc, 0x24, 0x26, 0x37, 0x92, 0x49, 0x24, + 0xef, 0xc2, 0x51, 0x56, 0x26, 0xf8, 0x53, 0x03, 0xb8, 0x11, 0x08, 0x01, 0x5b, 0x22, 0x2f, 0x34, + 0xd0, 0x36, 0xd5, 0xe7, 0xf0, 0x1a, 0x86, 0x37, 0x25, 0xab, 0xc5, 0xbc, 0x92, 0x5f, 0x5f, 0x5c, + 0x64, 0x39, 0x92, 0x21, 0x74, 0xe2, 0x54, 0x07, 0x7a, 0xb4, 0x13, 0xa7, 0x2a, 0x4a, 0x64, 0x3f, + 0x63, 0x1b, 0xa5, 0xce, 0xe4, 0x18, 0xdc, 0x78, 0x8e, 0xf1, 0x42, 0x2c, 0x0b, 0x3d, 0x8d, 0x01, + 0xdd, 0xc8, 0xe1, 0x1b, 0xf0, 0xdb, 0x1b, 0xaf, 0x50, 0x32, 0xf2, 0x39, 0xb8, 0x71, 0x1a, 0xa5, + 0x59, 0x8e, 0x22, 0xb0, 0xc6, 0xdd, 0x49, 0xff, 0xfc, 0xd9, 0x74, 0x9f, 0x64, 0xfb, 0x05, 0xd0, + 0x5e, 0x9c, 0xaa, 0xaf, 0x08, 0x7f, 0x84, 0xc1, 0xc6, 0x34, 0x5f, 0x96, 0x0b, 0xf2, 0xe9, 0x76, + 0xb8, 0x96, 0x86, 0xe6, 0xf8, 0xe0, 0xa6, 0x1d, 0xe2, 0x6d, 0xc7, 0x4c, 0xc0, 0x4e, 0x98, 0x64, + 0xba, 0x01, 0x9f, 0xea, 0x73, 0xe8, 0x80, 0xfd, 0xba, 0x2a, 0x31, 0x3c, 0x07, 0xf7, 0x2d, 0xae, + 0x7f, 0x60, 0xf9, 0x12, 0xc9, 0x08, 0xba, 0x6a, 0x24, 0x96, 0x76, 0x53, 0x47, 0x05, 0xe3, 0x4a, + 0x99, 0x9a, 0x50, 0x23, 0x84, 0x7f, 0x58, 0x30, 0x52, 0x89, 0xda, 0xda, 0x5e, 0x33, 0xc9, 0xc8, + 0x0b, 0x70, 0x0c, 0x45, 0x9a, 0xca, 0x0e, 0x87, 0xd6, 0x58, 0x15, 0x31, 0x14, 0x14, 0xd1, 0x0e, + 0xa4, 0xae, 0x52, 0xdc, 0x28, 0x58, 0x3f, 0x6a, 0x2a, 0xed, 0x6a, 0x98, 0x9e, 0x1c, 0x34, 0xd7, + 0x16, 0x6a, 0x5a, 0x20, 0x01, 0xf4, 0x56, 0xc8, 0x85, 0x4a, 0x69, 0xeb, 0x7b, 0x5a, 0x91, 0x7c, + 0x0c, 0xb6, 0x4a, 0xde, 0xf0, 0xf9, 0xe4, 0x3f, 0xd0, 0x56, 0xc3, 0xa1, 0xda, 0x31, 0xbc, 0x00, + 0xb8, 0x91, 0x15, 0xc7, 0xcb, 0x04, 0x4b, 0x49, 0x9e, 0x01, 0xc4, 0xf9, 0x52, 0x48, 0xe4, 0xdb, + 0x95, 0xf5, 0x1a, 0xcd, 0x65, 0x42, 0x9e, 0x82, 0x2b, 0x94, 0xb3, 0x32, 0x9a, 0x06, 0x7a, 0xc2, + 0x04, 0x87, 0x77, 0x30, 0x54, 0xc0, 0x7c, 0x57, 0xc5, 0x2c, 0x37, 0x44, 0x7c, 0x09, 0x30, 0x67, + 0x3c, 0x89, 0x84, 0x92, 0x1a, 0x68, 0xc8, 0x66, 0x23, 0xdf, 0x30, 0x6e, 0x08, 0x4b, 0xbd, 0x79, + 0x7b, 0x54, 0xe9, 0x73, 0x26, 0x64, 0x64, 0x08, 0x6c, 0x32, 0x78, 0x4a, 0x73, 0xa9, 0x14, 0xe1, + 0x2f, 0x96, 0x49, 0xf2, 0xaa, 0xae, 0xf3, 0xb5, 0x89, 0x78, 0x1f, 0x06, 0xac, 0xae, 0xf3, 0x0c, + 0x93, 0x68, 0x97, 0xf5, 0x7e, 0xa3, 0xd4, 0x71, 0xe4, 0x5b, 0x78, 0x47, 0xb6, 0x4b, 0xd2, 0x94, + 0x63, 0x1e, 0x9c, 0xe7, 0x0f, 0x70, 0x68, 0x7f, 0x9d, 0xe8, 0x50, 0xee, 0xc9, 0x61, 0x06, 0x70, + 0xa5, 0x76, 0xd0, 0xa4, 0x3f, 0x01, 0xaf, 0xc8, 0xca, 0xbd, 0xd4, 0x6e, 0x91, 0x95, 0x26, 0xed, + 0x0b, 0x70, 0x9a, 0x65, 0xee, 0x3c, 0xcc, 0x0b, 0x63, 0x25, 0xef, 0x81, 0x13, 0x57, 0x45, 0x91, + 0x49, 0xbd, 0x4f, 0x36, 0x6d, 0xa4, 0xf0, 0x37, 0x45, 0x36, 0xed, 0xba, 0x83, 0xea, 0x14, 0x8e, + 0xb6, 0x80, 0x0e, 0xcf, 0x83, 0x83, 0x0e, 0xd4, 0x43, 0x68, 0x0a, 0x37, 0x6e, 0x3b, 0xe4, 0xec, + 0xfc, 0x2f, 0x39, 0xbf, 0x00, 0xf3, 0xb6, 0x34, 0xf8, 0x98, 0x77, 0xf6, 0xe9, 0xc1, 0xed, 0xdb, + 0xce, 0x29, 0x14, 0x9b, 0xf3, 0xd9, 0x2b, 0xf0, 0x36, 0x79, 0x09, 0x80, 0xf3, 0x7d, 0xc5, 0x0b, + 0x96, 0x8f, 0x1e, 0x11, 0x1f, 0x5c, 0x3d, 0xab, 0xac, 0x9c, 0x8d, 0x2c, 0x32, 0x00, 0x6f, 0xf3, + 0x4a, 0x8e, 0x3a, 0xa4, 0x0f, 0x3d, 0x75, 0x9f, 0xb2, 0x75, 0xbf, 0x3a, 0xfb, 0xfd, 0xfe, 0xd4, + 0xfa, 0xf3, 0xfe, 0xd4, 0xfa, 0xeb, 0xfe, 0xd4, 0xfa, 0xf5, 0xef, 0xd3, 0x47, 0x10, 0xc4, 0x55, + 0x31, 0xad, 0xb3, 0x72, 0x16, 0xb3, 0x7a, 0x2a, 0xb3, 0xc5, 0x6a, 0xba, 0x58, 0xe9, 0xdf, 0xcc, + 0x9d, 0xa3, 0x3f, 0x9f, 0xfc, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x6d, 0xac, 0x3c, 0x58, 0xb0, 0x06, + 0x00, 0x00, } diff --git a/vendor/github.com/pingcap/kvproto/pkg/tikvpb/tikvpb.pb.go b/vendor/github.com/pingcap/kvproto/pkg/tikvpb/tikvpb.pb.go index 3927104221ffa..05dd50e6906d9 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/tikvpb/tikvpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/tikvpb/tikvpb.pb.go @@ -1,5 +1,6 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. +// Code generated by protoc-gen-gogo. // source: tikvpb.proto +// DO NOT EDIT! /* Package tikvpb is a generated protocol buffer package. @@ -65,9 +66,14 @@ type TikvClient interface { KvDeleteRange(ctx context.Context, in *kvrpcpb.DeleteRangeRequest, opts ...grpc.CallOption) (*kvrpcpb.DeleteRangeResponse, error) // RawKV commands. RawGet(ctx context.Context, in *kvrpcpb.RawGetRequest, opts ...grpc.CallOption) (*kvrpcpb.RawGetResponse, error) + RawBatchGet(ctx context.Context, in *kvrpcpb.RawBatchGetRequest, opts ...grpc.CallOption) (*kvrpcpb.RawBatchGetResponse, error) RawPut(ctx context.Context, in *kvrpcpb.RawPutRequest, opts ...grpc.CallOption) (*kvrpcpb.RawPutResponse, error) + RawBatchPut(ctx context.Context, in *kvrpcpb.RawBatchPutRequest, opts ...grpc.CallOption) (*kvrpcpb.RawBatchPutResponse, error) RawDelete(ctx context.Context, in *kvrpcpb.RawDeleteRequest, opts ...grpc.CallOption) (*kvrpcpb.RawDeleteResponse, error) + RawBatchDelete(ctx context.Context, in *kvrpcpb.RawBatchDeleteRequest, opts ...grpc.CallOption) (*kvrpcpb.RawBatchDeleteResponse, error) RawScan(ctx context.Context, in *kvrpcpb.RawScanRequest, opts ...grpc.CallOption) (*kvrpcpb.RawScanResponse, error) + RawDeleteRange(ctx context.Context, in *kvrpcpb.RawDeleteRangeRequest, opts ...grpc.CallOption) (*kvrpcpb.RawDeleteRangeResponse, error) + RawBatchScan(ctx context.Context, in *kvrpcpb.RawBatchScanRequest, opts ...grpc.CallOption) (*kvrpcpb.RawBatchScanResponse, error) // SQL push down commands. Coprocessor(ctx context.Context, in *coprocessor.Request, opts ...grpc.CallOption) (*coprocessor.Response, error) CoprocessorStream(ctx context.Context, in *coprocessor.Request, opts ...grpc.CallOption) (Tikv_CoprocessorStreamClient, error) @@ -206,6 +212,15 @@ func (c *tikvClient) RawGet(ctx context.Context, in *kvrpcpb.RawGetRequest, opts return out, nil } +func (c *tikvClient) RawBatchGet(ctx context.Context, in *kvrpcpb.RawBatchGetRequest, opts ...grpc.CallOption) (*kvrpcpb.RawBatchGetResponse, error) { + out := new(kvrpcpb.RawBatchGetResponse) + err := grpc.Invoke(ctx, "/tikvpb.Tikv/RawBatchGet", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *tikvClient) RawPut(ctx context.Context, in *kvrpcpb.RawPutRequest, opts ...grpc.CallOption) (*kvrpcpb.RawPutResponse, error) { out := new(kvrpcpb.RawPutResponse) err := grpc.Invoke(ctx, "/tikvpb.Tikv/RawPut", in, out, c.cc, opts...) @@ -215,6 +230,15 @@ func (c *tikvClient) RawPut(ctx context.Context, in *kvrpcpb.RawPutRequest, opts return out, nil } +func (c *tikvClient) RawBatchPut(ctx context.Context, in *kvrpcpb.RawBatchPutRequest, opts ...grpc.CallOption) (*kvrpcpb.RawBatchPutResponse, error) { + out := new(kvrpcpb.RawBatchPutResponse) + err := grpc.Invoke(ctx, "/tikvpb.Tikv/RawBatchPut", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *tikvClient) RawDelete(ctx context.Context, in *kvrpcpb.RawDeleteRequest, opts ...grpc.CallOption) (*kvrpcpb.RawDeleteResponse, error) { out := new(kvrpcpb.RawDeleteResponse) err := grpc.Invoke(ctx, "/tikvpb.Tikv/RawDelete", in, out, c.cc, opts...) @@ -224,6 +248,15 @@ func (c *tikvClient) RawDelete(ctx context.Context, in *kvrpcpb.RawDeleteRequest return out, nil } +func (c *tikvClient) RawBatchDelete(ctx context.Context, in *kvrpcpb.RawBatchDeleteRequest, opts ...grpc.CallOption) (*kvrpcpb.RawBatchDeleteResponse, error) { + out := new(kvrpcpb.RawBatchDeleteResponse) + err := grpc.Invoke(ctx, "/tikvpb.Tikv/RawBatchDelete", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *tikvClient) RawScan(ctx context.Context, in *kvrpcpb.RawScanRequest, opts ...grpc.CallOption) (*kvrpcpb.RawScanResponse, error) { out := new(kvrpcpb.RawScanResponse) err := grpc.Invoke(ctx, "/tikvpb.Tikv/RawScan", in, out, c.cc, opts...) @@ -233,6 +266,24 @@ func (c *tikvClient) RawScan(ctx context.Context, in *kvrpcpb.RawScanRequest, op return out, nil } +func (c *tikvClient) RawDeleteRange(ctx context.Context, in *kvrpcpb.RawDeleteRangeRequest, opts ...grpc.CallOption) (*kvrpcpb.RawDeleteRangeResponse, error) { + out := new(kvrpcpb.RawDeleteRangeResponse) + err := grpc.Invoke(ctx, "/tikvpb.Tikv/RawDeleteRange", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *tikvClient) RawBatchScan(ctx context.Context, in *kvrpcpb.RawBatchScanRequest, opts ...grpc.CallOption) (*kvrpcpb.RawBatchScanResponse, error) { + out := new(kvrpcpb.RawBatchScanResponse) + err := grpc.Invoke(ctx, "/tikvpb.Tikv/RawBatchScan", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *tikvClient) Coprocessor(ctx context.Context, in *coprocessor.Request, opts ...grpc.CallOption) (*coprocessor.Response, error) { out := new(coprocessor.Response) err := grpc.Invoke(ctx, "/tikvpb.Tikv/Coprocessor", in, out, c.cc, opts...) @@ -387,9 +438,14 @@ type TikvServer interface { KvDeleteRange(context.Context, *kvrpcpb.DeleteRangeRequest) (*kvrpcpb.DeleteRangeResponse, error) // RawKV commands. RawGet(context.Context, *kvrpcpb.RawGetRequest) (*kvrpcpb.RawGetResponse, error) + RawBatchGet(context.Context, *kvrpcpb.RawBatchGetRequest) (*kvrpcpb.RawBatchGetResponse, error) RawPut(context.Context, *kvrpcpb.RawPutRequest) (*kvrpcpb.RawPutResponse, error) + RawBatchPut(context.Context, *kvrpcpb.RawBatchPutRequest) (*kvrpcpb.RawBatchPutResponse, error) RawDelete(context.Context, *kvrpcpb.RawDeleteRequest) (*kvrpcpb.RawDeleteResponse, error) + RawBatchDelete(context.Context, *kvrpcpb.RawBatchDeleteRequest) (*kvrpcpb.RawBatchDeleteResponse, error) RawScan(context.Context, *kvrpcpb.RawScanRequest) (*kvrpcpb.RawScanResponse, error) + RawDeleteRange(context.Context, *kvrpcpb.RawDeleteRangeRequest) (*kvrpcpb.RawDeleteRangeResponse, error) + RawBatchScan(context.Context, *kvrpcpb.RawBatchScanRequest) (*kvrpcpb.RawBatchScanResponse, error) // SQL push down commands. Coprocessor(context.Context, *coprocessor.Request) (*coprocessor.Response, error) CoprocessorStream(*coprocessor.Request, Tikv_CoprocessorStreamServer) error @@ -641,6 +697,24 @@ func _Tikv_RawGet_Handler(srv interface{}, ctx context.Context, dec func(interfa return interceptor(ctx, in, info, handler) } +func _Tikv_RawBatchGet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(kvrpcpb.RawBatchGetRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TikvServer).RawBatchGet(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tikvpb.Tikv/RawBatchGet", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TikvServer).RawBatchGet(ctx, req.(*kvrpcpb.RawBatchGetRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Tikv_RawPut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(kvrpcpb.RawPutRequest) if err := dec(in); err != nil { @@ -659,6 +733,24 @@ func _Tikv_RawPut_Handler(srv interface{}, ctx context.Context, dec func(interfa return interceptor(ctx, in, info, handler) } +func _Tikv_RawBatchPut_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(kvrpcpb.RawBatchPutRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TikvServer).RawBatchPut(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tikvpb.Tikv/RawBatchPut", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TikvServer).RawBatchPut(ctx, req.(*kvrpcpb.RawBatchPutRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Tikv_RawDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(kvrpcpb.RawDeleteRequest) if err := dec(in); err != nil { @@ -677,6 +769,24 @@ func _Tikv_RawDelete_Handler(srv interface{}, ctx context.Context, dec func(inte return interceptor(ctx, in, info, handler) } +func _Tikv_RawBatchDelete_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(kvrpcpb.RawBatchDeleteRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TikvServer).RawBatchDelete(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tikvpb.Tikv/RawBatchDelete", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TikvServer).RawBatchDelete(ctx, req.(*kvrpcpb.RawBatchDeleteRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Tikv_RawScan_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(kvrpcpb.RawScanRequest) if err := dec(in); err != nil { @@ -695,6 +805,42 @@ func _Tikv_RawScan_Handler(srv interface{}, ctx context.Context, dec func(interf return interceptor(ctx, in, info, handler) } +func _Tikv_RawDeleteRange_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(kvrpcpb.RawDeleteRangeRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TikvServer).RawDeleteRange(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tikvpb.Tikv/RawDeleteRange", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TikvServer).RawDeleteRange(ctx, req.(*kvrpcpb.RawDeleteRangeRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Tikv_RawBatchScan_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(kvrpcpb.RawBatchScanRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TikvServer).RawBatchScan(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tikvpb.Tikv/RawBatchScan", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TikvServer).RawBatchScan(ctx, req.(*kvrpcpb.RawBatchScanRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _Tikv_Coprocessor_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(coprocessor.Request) if err := dec(in); err != nil { @@ -896,18 +1042,38 @@ var _Tikv_serviceDesc = grpc.ServiceDesc{ MethodName: "RawGet", Handler: _Tikv_RawGet_Handler, }, + { + MethodName: "RawBatchGet", + Handler: _Tikv_RawBatchGet_Handler, + }, { MethodName: "RawPut", Handler: _Tikv_RawPut_Handler, }, + { + MethodName: "RawBatchPut", + Handler: _Tikv_RawBatchPut_Handler, + }, { MethodName: "RawDelete", Handler: _Tikv_RawDelete_Handler, }, + { + MethodName: "RawBatchDelete", + Handler: _Tikv_RawBatchDelete_Handler, + }, { MethodName: "RawScan", Handler: _Tikv_RawScan_Handler, }, + { + MethodName: "RawDeleteRange", + Handler: _Tikv_RawDeleteRange_Handler, + }, + { + MethodName: "RawBatchScan", + Handler: _Tikv_RawBatchScan_Handler, + }, { MethodName: "Coprocessor", Handler: _Tikv_Coprocessor_Handler, @@ -948,44 +1114,48 @@ var _Tikv_serviceDesc = grpc.ServiceDesc{ func init() { proto.RegisterFile("tikvpb.proto", fileDescriptorTikvpb) } var fileDescriptorTikvpb = []byte{ - // 612 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x95, 0x4f, 0x6f, 0xd3, 0x30, - 0x18, 0xc6, 0x57, 0x69, 0x94, 0xcd, 0xdb, 0x04, 0x73, 0x0b, 0x74, 0x61, 0xab, 0x60, 0x27, 0xc4, - 0x21, 0xfc, 0x95, 0x38, 0x0c, 0x10, 0x34, 0x95, 0x2a, 0xc8, 0x2a, 0x55, 0xe9, 0x2e, 0x9c, 0x90, - 0x1b, 0xbd, 0x4b, 0xa3, 0xa4, 0x71, 0x70, 0x5c, 0x57, 0xfb, 0x26, 0x5c, 0xf9, 0x36, 0x1c, 0xf9, - 0x08, 0xa8, 0x7c, 0x11, 0x94, 0xa4, 0x76, 0xec, 0xa6, 0x45, 0xe2, 0xd4, 0xe4, 0xf7, 0xbc, 0xcf, - 0x93, 0xb7, 0xce, 0x6b, 0x07, 0x1d, 0xf2, 0x30, 0x12, 0xe9, 0xc4, 0x4e, 0x19, 0xe5, 0x14, 0x37, - 0xcb, 0x3b, 0xeb, 0xd8, 0xa7, 0x29, 0xa3, 0x3e, 0x64, 0x19, 0x65, 0xa5, 0x64, 0x1d, 0x45, 0x82, - 0xa5, 0xbe, 0xac, 0xb4, 0x5a, 0x8c, 0x5c, 0xf3, 0xaf, 0x19, 0x30, 0x01, 0x4c, 0xc1, 0x76, 0x40, - 0x03, 0x5a, 0x5c, 0x3e, 0xcb, 0xaf, 0x4a, 0xfa, 0xf2, 0xc7, 0x21, 0xda, 0xbd, 0x0a, 0x23, 0x81, - 0x5f, 0xa3, 0x5b, 0xae, 0x18, 0x00, 0xc7, 0x2d, 0x5b, 0x86, 0x0d, 0x80, 0x7b, 0xf0, 0x6d, 0x0e, - 0x19, 0xb7, 0xda, 0x26, 0xcc, 0x52, 0x9a, 0x64, 0x70, 0xbe, 0x83, 0xdf, 0xa0, 0xa6, 0x2b, 0xc6, - 0x3e, 0x49, 0x70, 0x55, 0x91, 0xdf, 0x4a, 0xdf, 0xbd, 0x35, 0xaa, 0x8c, 0x0e, 0x42, 0xae, 0x18, - 0x31, 0x58, 0xb0, 0x90, 0x03, 0xee, 0xa8, 0x32, 0x89, 0x64, 0xc0, 0xc9, 0x06, 0x45, 0x85, 0xbc, - 0x43, 0x7b, 0xae, 0x70, 0xe8, 0x6c, 0x16, 0x72, 0x7c, 0x5f, 0x15, 0x96, 0x40, 0x06, 0x3c, 0xa8, - 0x71, 0xd3, 0xfe, 0x69, 0x96, 0x52, 0xa6, 0xdb, 0x4b, 0x50, 0xb7, 0x4b, 0xae, 0xec, 0x1f, 0xd0, - 0xbe, 0x2b, 0x9c, 0x18, 0x48, 0x32, 0x4f, 0xb1, 0xf6, 0x98, 0x92, 0xc8, 0x80, 0x4e, 0x5d, 0x30, - 0x17, 0xa1, 0x47, 0xb8, 0x3f, 0xcd, 0x17, 0xbe, 0xaa, 0x94, 0xa8, 0xbe, 0x08, 0x95, 0xa2, 0x42, - 0x3c, 0x74, 0x67, 0x15, 0xe2, 0xd1, 0x38, 0x9e, 0x10, 0x3f, 0xc2, 0x67, 0x66, 0xbd, 0xe4, 0x32, - 0xae, 0xbb, 0x4d, 0x36, 0x1b, 0xcb, 0xdf, 0xd8, 0x25, 0xf5, 0x23, 0xad, 0x31, 0x89, 0xea, 0x8d, - 0x55, 0x8a, 0x0a, 0xb9, 0x44, 0x47, 0xae, 0xf0, 0x20, 0xa3, 0xb1, 0x80, 0x22, 0xe7, 0xa1, 0xaa, - 0xd6, 0xa8, 0x8c, 0x3a, 0xdd, 0x2c, 0xaa, 0xb4, 0x17, 0x68, 0xd7, 0x15, 0x03, 0x07, 0xe3, 0x6a, - 0x12, 0x1d, 0xe9, 0x6d, 0x19, 0xcc, 0x6c, 0xa0, 0x0f, 0x31, 0x70, 0xf0, 0x48, 0x12, 0x80, 0xd6, - 0x80, 0x46, 0xeb, 0x0d, 0x18, 0xa2, 0x4a, 0xbb, 0x40, 0x4d, 0x8f, 0x2c, 0xf2, 0x17, 0x55, 0xcd, - 0x4a, 0x09, 0xea, 0xb3, 0x22, 0xf9, 0x9a, 0x79, 0x34, 0x5f, 0x33, 0x8f, 0xe6, 0x9b, 0xcd, 0x05, - 0x57, 0xe6, 0x3e, 0xda, 0xf7, 0xc8, 0xa2, 0xec, 0x0a, 0x9f, 0xe8, 0x75, 0xab, 0x4e, 0x57, 0x11, - 0xd6, 0x26, 0x49, 0xa5, 0xbc, 0x47, 0xb7, 0x3d, 0xb2, 0x28, 0xf6, 0xaa, 0xf1, 0x2c, 0x7d, 0xbb, - 0x76, 0xea, 0x82, 0xf2, 0xbf, 0x45, 0x07, 0x4e, 0x75, 0xf0, 0xe0, 0xb6, 0xad, 0x1f, 0x43, 0xd5, - 0x7e, 0x37, 0xa9, 0xf6, 0x1f, 0x8e, 0x35, 0xf7, 0x98, 0x33, 0x20, 0xb3, 0xff, 0xcc, 0x78, 0xde, - 0xc0, 0x17, 0x68, 0xd7, 0x23, 0xd7, 0x1c, 0x5b, 0xb6, 0x79, 0xc2, 0xe5, 0x70, 0x08, 0x59, 0x46, - 0x02, 0xb0, 0x5a, 0x6b, 0x5a, 0x9f, 0x26, 0x70, 0xbe, 0xf3, 0xa4, 0x81, 0x3f, 0xa2, 0xbd, 0x71, - 0x42, 0xd2, 0x6c, 0x4a, 0x39, 0x3e, 0x5d, 0x2b, 0x92, 0x82, 0x33, 0x9d, 0x27, 0xd1, 0xf6, 0x88, - 0xcf, 0xe8, 0x60, 0x9c, 0xc6, 0xf9, 0x21, 0x12, 0x84, 0x34, 0xd1, 0xe6, 0x49, 0xa3, 0xf5, 0x79, - 0x32, 0x44, 0xb5, 0x22, 0x43, 0x74, 0x38, 0x14, 0xbe, 0x3f, 0x00, 0xde, 0xbb, 0x71, 0xe1, 0x06, - 0x57, 0xf5, 0x3a, 0x96, 0x69, 0x67, 0x5b, 0x54, 0x15, 0xf7, 0x05, 0xdd, 0x55, 0xca, 0x98, 0x13, - 0xc6, 0xaf, 0x32, 0xfc, 0xa8, 0x6e, 0x5a, 0x49, 0x32, 0xf6, 0xf1, 0x3f, 0x2a, 0x64, 0x74, 0xef, - 0xe9, 0xcf, 0x65, 0xb7, 0xf1, 0x6b, 0xd9, 0x6d, 0xfc, 0x5e, 0x76, 0x1b, 0xdf, 0xff, 0x74, 0x77, - 0x50, 0xc7, 0xa7, 0x33, 0x3b, 0x0d, 0x93, 0xc0, 0x27, 0xa9, 0x9d, 0x7f, 0x96, 0xec, 0x48, 0x14, - 0xdf, 0x93, 0x49, 0xb3, 0xf8, 0x79, 0xf5, 0x37, 0x00, 0x00, 0xff, 0xff, 0xea, 0xa8, 0xe1, 0xc4, - 0xbb, 0x06, 0x00, 0x00, + // 682 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x96, 0xdf, 0x4e, 0xd4, 0x40, + 0x14, 0xc6, 0xd9, 0x04, 0x57, 0x18, 0x40, 0x64, 0x16, 0x75, 0x29, 0x4b, 0x51, 0xae, 0x8c, 0x17, + 0xf5, 0x6f, 0xe2, 0x05, 0x6a, 0x94, 0x92, 0x6c, 0xa4, 0x90, 0x90, 0x96, 0x1b, 0xaf, 0xcc, 0xd0, + 0x0c, 0xa5, 0x69, 0xb7, 0x53, 0xa7, 0xb3, 0xb3, 0xe1, 0x4d, 0x7c, 0x0e, 0x9f, 0xc2, 0x4b, 0x1f, + 0xc1, 0xe0, 0x8b, 0x98, 0xb6, 0x3b, 0xd3, 0x99, 0x4e, 0xd7, 0xc4, 0x2b, 0xda, 0xef, 0x3b, 0xe7, + 0xd7, 0x8f, 0x39, 0x93, 0x99, 0x05, 0xeb, 0x2c, 0x4e, 0x78, 0x7e, 0xe9, 0xe4, 0x94, 0x30, 0x02, + 0xfb, 0xf5, 0x9b, 0xb5, 0x15, 0x92, 0x9c, 0x92, 0x10, 0x17, 0x05, 0xa1, 0xb5, 0x65, 0x6d, 0x24, + 0x9c, 0xe6, 0xa1, 0xa8, 0xb4, 0x06, 0x14, 0x5d, 0xb1, 0xaf, 0x05, 0xa6, 0x1c, 0x53, 0x29, 0x6e, + 0x47, 0x24, 0x22, 0xd5, 0xe3, 0xf3, 0xf2, 0xa9, 0x56, 0x5f, 0xfd, 0xd8, 0x04, 0xcb, 0x17, 0x71, + 0xc2, 0xe1, 0x1b, 0x70, 0xc7, 0xe3, 0x63, 0xcc, 0xe0, 0xc0, 0x11, 0xb0, 0x31, 0x66, 0x3e, 0xfe, + 0x36, 0xc5, 0x05, 0xb3, 0xb6, 0x75, 0xb1, 0xc8, 0x49, 0x56, 0xe0, 0x83, 0x25, 0xf8, 0x16, 0xf4, + 0x3d, 0x1e, 0x84, 0x28, 0x83, 0x4d, 0x45, 0xf9, 0x2a, 0xfa, 0x1e, 0xb4, 0x54, 0xd9, 0xe8, 0x02, + 0xe0, 0xf1, 0x73, 0x8a, 0x67, 0x34, 0x66, 0x18, 0x0e, 0x65, 0x99, 0x90, 0x04, 0x60, 0xa7, 0xc3, + 0x91, 0x90, 0xf7, 0x60, 0xc5, 0xe3, 0x2e, 0x99, 0x4c, 0x62, 0x06, 0x1f, 0xca, 0xc2, 0x5a, 0x10, + 0x80, 0x47, 0x86, 0xae, 0xb7, 0x7f, 0x9e, 0xe4, 0x84, 0xaa, 0xed, 0xb5, 0x60, 0xb6, 0x0b, 0x5d, + 0xb6, 0x7f, 0x04, 0xab, 0x1e, 0x77, 0x53, 0x8c, 0xb2, 0x69, 0x0e, 0x95, 0xcf, 0xd4, 0x8a, 0x00, + 0x0c, 0x4d, 0x43, 0x5f, 0x84, 0x23, 0xc4, 0xc2, 0xeb, 0x72, 0xe1, 0x9b, 0x4a, 0x21, 0x99, 0x8b, + 0xd0, 0x38, 0x12, 0xe2, 0x83, 0xcd, 0x39, 0xc4, 0x27, 0x69, 0x7a, 0x89, 0xc2, 0x04, 0xee, 0xe9, + 0xf5, 0x42, 0x17, 0x38, 0x7b, 0x91, 0xad, 0x07, 0x2b, 0x27, 0x76, 0x4a, 0xc2, 0x44, 0x09, 0x26, + 0x24, 0x33, 0x58, 0xe3, 0x48, 0xc8, 0x29, 0xd8, 0xf0, 0xb8, 0x8f, 0x0b, 0x92, 0x72, 0x5c, 0x71, + 0x76, 0x65, 0xb5, 0xa2, 0x0a, 0xd4, 0xa8, 0xdb, 0x94, 0xb4, 0x97, 0x60, 0xd9, 0xe3, 0x63, 0x17, + 0xc2, 0x66, 0x27, 0xba, 0xa2, 0x77, 0xa0, 0x69, 0x7a, 0x80, 0x63, 0x9c, 0x62, 0x86, 0x7d, 0x94, + 0x45, 0x58, 0x09, 0xa0, 0xa8, 0x66, 0x00, 0xcd, 0x94, 0xb4, 0x43, 0xd0, 0xf7, 0xd1, 0xac, 0x1c, + 0x54, 0xb3, 0x57, 0x6a, 0xc1, 0xdc, 0x2b, 0x42, 0x97, 0xcd, 0x27, 0x60, 0xcd, 0x47, 0x33, 0x39, + 0xea, 0x5d, 0xb5, 0xb2, 0x3d, 0xed, 0x51, 0xb7, 0xd9, 0x0a, 0x72, 0x3e, 0x6d, 0x05, 0x39, 0x9f, + 0x76, 0x07, 0xa9, 0xf4, 0xae, 0x20, 0x25, 0xc1, 0x0c, 0xa2, 0x60, 0x46, 0xdd, 0xa6, 0x64, 0x1d, + 0x83, 0x55, 0x1f, 0xcd, 0xea, 0xd5, 0x82, 0x3b, 0x6a, 0xf1, 0x7c, 0x05, 0xe7, 0x1c, 0xab, 0xcb, + 0x92, 0x94, 0x00, 0xdc, 0x13, 0xf8, 0x39, 0xca, 0x36, 0xbe, 0xab, 0xf3, 0xf6, 0x17, 0xfa, 0x12, + 0xfa, 0x01, 0xdc, 0xf5, 0xd1, 0xac, 0x3a, 0x98, 0xb4, 0xc5, 0x50, 0xcf, 0xa6, 0xa1, 0x69, 0xb4, + 0x42, 0xa9, 0x7b, 0xc7, 0xee, 0xf8, 0x27, 0xd4, 0xed, 0xb3, 0xbf, 0xd0, 0x97, 0xd0, 0x33, 0xb0, + 0x2e, 0x02, 0x57, 0xc9, 0xcc, 0xf5, 0x55, 0xe3, 0xed, 0x2d, 0x70, 0x25, 0xee, 0x1d, 0x58, 0x73, + 0x9b, 0x9b, 0x00, 0x6e, 0x3b, 0xea, 0xbd, 0xd0, 0x1c, 0xc0, 0xba, 0xaa, 0x0c, 0x6f, 0x4b, 0xe9, + 0x0e, 0x18, 0xc5, 0x68, 0xf2, 0x9f, 0x8c, 0x17, 0x3d, 0x78, 0x08, 0x96, 0x7d, 0x74, 0xc5, 0xa0, + 0xe5, 0xe8, 0x57, 0x4e, 0x29, 0x9e, 0xe1, 0xa2, 0x40, 0x11, 0xb6, 0x06, 0x2d, 0xef, 0x98, 0x64, + 0xf8, 0x60, 0xe9, 0x69, 0x0f, 0x7e, 0x02, 0x2b, 0x41, 0x86, 0xf2, 0xe2, 0x9a, 0x30, 0x38, 0x6a, + 0x15, 0x09, 0xc3, 0xbd, 0x9e, 0x66, 0xc9, 0x62, 0xc4, 0x09, 0x58, 0x0b, 0xf2, 0xb4, 0x3c, 0xd5, + 0xa3, 0x98, 0x64, 0xca, 0x76, 0x56, 0x54, 0x73, 0x3b, 0x6b, 0xa6, 0x3a, 0x9e, 0x33, 0x1e, 0x86, + 0x63, 0xcc, 0x8e, 0x6e, 0x3c, 0x7c, 0xa3, 0x8c, 0x47, 0x95, 0xcd, 0xf1, 0xe8, 0xae, 0xc4, 0x7d, + 0x01, 0xf7, 0xa5, 0x13, 0x30, 0x44, 0xd9, 0x45, 0x01, 0x1f, 0x9b, 0x4d, 0x73, 0x4b, 0x60, 0x9f, + 0xfc, 0xa3, 0x42, 0xa0, 0x8f, 0x9e, 0xfd, 0xbc, 0xb5, 0x7b, 0xbf, 0x6e, 0xed, 0xde, 0xef, 0x5b, + 0xbb, 0xf7, 0xfd, 0x8f, 0xbd, 0x04, 0x86, 0x21, 0x99, 0x38, 0x79, 0x9c, 0x45, 0x21, 0xca, 0x9d, + 0xf2, 0x77, 0x82, 0x93, 0xf0, 0xea, 0x82, 0xbf, 0xec, 0x57, 0x7f, 0x5e, 0xff, 0x0d, 0x00, 0x00, + 0xff, 0xff, 0x1e, 0x48, 0x64, 0x89, 0x4c, 0x08, 0x00, 0x00, }