diff --git a/Gopkg.lock b/Gopkg.lock index 87fff41338351..85b2e80bfb92e 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -210,7 +210,7 @@ "pkg/raft_serverpb", "pkg/tikvpb" ] - revision = "7b013aefd7210f4f4dbfca3330b462ddce5dd496" + revision = "187ef904091a0381594224b321d284b0cee3a4f4" [[projects]] name = "github.com/pingcap/pd" 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..388b73de73aed 100644 --- a/ast/ddl.go +++ b/ast/ddl.go @@ -745,6 +745,7 @@ type AlterTableSpec struct { OldColumnName *ColumnName Position *ColumnPosition LockType LockType + Comment string } // Accept implements Node Accept interface. 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.toml.example b/config/config.toml.example index cb230446d2921..6b082406e0c87 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -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" diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d707b8eb0c4bf..1ab38a1f383a5 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -934,6 +934,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) @@ -1448,6 +1451,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_test.go b/ddl/ddl_db_test.go index c96aa4cfb73dc..045c09a0221df 100644 --- a/ddl/ddl_db_test.go +++ b/ddl/ddl_db_test.go @@ -167,6 +167,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 +326,7 @@ func (s *testDBSuite) TestCancelAddIndex(c *C) { } var checkErr error + var c3IdxInfo *model.IndexInfo hook := &ddl.TestDDLCallback{} first := true oldReorgWaitTimeout := ddl.ReorgWaitTimeout @@ -335,6 +339,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 +421,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 } @@ -713,6 +731,11 @@ 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) { f := func() map[int64]struct{} { handles := make(map[int64]struct{}) @@ -745,8 +768,6 @@ LOOP: } } c.Assert(handles, HasLen, 0) - - s.tk.MustExec("drop table test_drop_index") } func (s *testDBSuite) TestAddIndexWithDupCols(c *C) { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 46689a2ef573d..a7882f7183ec4 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -147,6 +147,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 +164,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() @@ -254,9 +267,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() { d.waitSchemaChanged(nil, waitTime, schemaVer) } if job.IsSynced() { @@ -329,6 +342,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 +432,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() { return } // TODO: Make ctx exits when the d is close. diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 34b2ae98da71f..18694eb696d8a 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -234,6 +234,13 @@ 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 + indexID := job.Args[1].(int64) + 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/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/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..a33b76532cb47 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -42,12 +42,10 @@ var ( // 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. @@ -121,23 +119,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 +130,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) { 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/stream.go b/distsql/stream.go index bdbf241efb130..e14edb4c4a29c 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 { 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..5e3243f7182ec 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++ @@ -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 := txn.GetSnapshot().BatchGet(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/analyze.go b/executor/analyze.go index acfaba16b5e0a..c7d50fa5b8f7d 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -52,8 +52,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) 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..29830223316d6 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 } 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 81e688295b04a..465e6746226a3 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -78,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 } @@ -219,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() @@ -357,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() } @@ -588,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() @@ -691,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 } @@ -789,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) @@ -880,7 +880,7 @@ func (tr *tableResultHandler) open(optionalResult, result distsql.SelectResult) 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) } @@ -889,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 65f7bc068b6bc..cbc56398b0fc2 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -157,8 +157,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 } @@ -187,9 +187,9 @@ type Executor interface { Schema() *expression.Schema retTypes() []*types.FieldType newChunk() *chunk.Chunk - // NextChunk fills a chunk with multiple rows + // Next fills a chunk with multiple rows // NOTE: chunk has to call Reset() method before any use. - NextChunk(ctx context.Context, chk *chunk.Chunk) error + Next(ctx context.Context, chk *chunk.Chunk) error } // CancelDDLJobsExec represents a cancel DDL jobs executor. @@ -201,8 +201,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 @@ -230,8 +230,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 @@ -289,8 +289,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 @@ -330,8 +330,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 @@ -365,8 +365,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 } @@ -419,8 +419,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 } @@ -432,7 +432,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) } @@ -470,10 +470,10 @@ func (e *SelectLockExec) Open(ctx context.Context) error { 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) } @@ -512,14 +512,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) } @@ -540,7 +540,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) } @@ -589,7 +589,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) } @@ -628,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)) @@ -652,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 @@ -703,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 { @@ -721,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) } @@ -737,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) @@ -753,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) } @@ -780,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)) @@ -921,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) } @@ -956,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) } @@ -1075,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 } @@ -1087,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 b8fb231e45627..95f547ae68039 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -89,13 +89,13 @@ func (s *testExecSuite) TestShowProcessList(c *C) { it := chunk.NewIterator4Chunk(chk) // Run test and check results. for _, p := range ps { - err = e.NextChunk(context.Background(), chk) + err = e.Next(context.Background(), chk) c.Assert(err, IsNil) for row := it.Begin(); row != it.End(); row = it.Next() { c.Assert(row.GetUint64(0), Equals, p.ID) } } - err = e.NextChunk(context.Background(), chk) + err = e.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(chk.NumRows(), Equals, 0) err = e.Close() diff --git a/executor/executor_test.go b/executor/executor_test.go index 35015ce24a0a4..c801d2267e6d4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -138,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) @@ -148,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) @@ -163,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() @@ -173,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) @@ -756,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") } @@ -2155,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") @@ -2297,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)) @@ -2462,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() } @@ -2474,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() } @@ -2640,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/index_lookup_join.go b/executor/index_lookup_join.go index 970ea14c4c3f3..409ac572fd0cf 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -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) } @@ -514,7 +514,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/join.go b/executor/join.go index 12b8932ed9e65..3f2979350221d 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,6 +473,39 @@ 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 @@ -498,28 +531,47 @@ 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 +590,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 +608,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 +628,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 +648,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..a11e7487a7aa8 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,13 +43,10 @@ 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, @@ -57,8 +54,7 @@ func newJoinResultGenerator(ctx sessionctx.Context, joinType plan.JoinType, 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 Row) { 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..e036b81bbbd46 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -742,7 +742,7 @@ 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() 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 f3ab0a5fed40a..b869ce2449b1f 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -132,7 +132,7 @@ func (t *mergeJoinInnerTable) nextRow() (chunk.Row, error) { if t.curRow == t.curIter.End() { t.reallocReaderResult() oldMemUsage := t.curResult.MemoryUsage() - err := t.reader.NextChunk(t.ctx, t.curResult) + 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() @@ -234,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 { @@ -274,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) } @@ -287,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) } @@ -319,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 e718b081c7aa4..3a4f9a9b1c6cd 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -26,7 +26,7 @@ type MockExec struct { 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++ { @@ -97,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..43cfa8c74df53 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 diff --git a/executor/show.go b/executor/show.go index 8ca2463ac9eec..6797d18f2eacc 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() diff --git a/executor/show_stats.go b/executor/show_stats.go index cf4f0676e5dab..ae8f1b6357ae5 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,7 +51,7 @@ 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()) @@ -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/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..d85d5229ce43d 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -113,8 +113,8 @@ type UnionScanExec struct { 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++ { @@ -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) } diff --git a/executor/write.go b/executor/write.go index 1143e931b9f65..1f8cb2f8da01f 100644 --- a/executor/write.go +++ b/executor/write.go @@ -182,8 +182,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 @@ -259,7 +259,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) } @@ -339,7 +339,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) } @@ -711,8 +711,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 { @@ -995,8 +995,8 @@ func batchMarkDupRows(ctx sessionctx.Context, t table.Table, rows [][]types.Datu 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 +1268,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) } @@ -1580,8 +1580,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 @@ -1673,8 +1673,8 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema) (Row, return Row{}, 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 +1714,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) } diff --git a/model/ddl.go b/model/ddl.go index 0c4fb4a5b36ee..3bacb3aac794d 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 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..a0da3659fc617 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" @@ -2532,7 +2533,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" @@ -4823,6 +4824,14 @@ AdminStmt: Index: string($5), } } +| "ADMIN" "CLEANUP" "INDEX" TableName Identifier + { + $$ = &ast.AdminStmt{ + Tp: ast.AdminCleanupIndex, + Tables: []*ast.TableName{$4.(*ast.TableName)}, + Index: string($5), + } + } | "ADMIN" "CHECK" "INDEX" TableName Identifier HandleRangeList { $$ = &ast.AdminStmt{ diff --git a/parser/parser_test.go b/parser/parser_test.go index aea081687d92e..4d429bb82edbc 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -416,6 +416,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) { {"admin cancel ddl jobs 1", true}, {"admin cancel ddl jobs 1, 2", true}, {"admin recover index t1 idx_a", true}, + {"admin cleanup index t1 idx_a", true}, // for on duplicate key update {"INSERT INTO t (a,b,c) VALUES (1,2,3),(4,5,6) ON DUPLICATE KEY UPDATE c=VALUES(a)+VALUES(b);", true}, diff --git a/plan/common_plans.go b/plan/common_plans.go index c4f70c98b6ca1..cfefc40c68d9e 100644 --- a/plan/common_plans.go +++ b/plan/common_plans.go @@ -65,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 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/physical_plan_test.go b/plan/physical_plan_test.go index 150d49a6ad9ef..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. { diff --git a/plan/planbuilder.go b/plan/planbuilder.go index a40f4f932ca42..2209b10fad146 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)) 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/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..06b05084c5514 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, @@ -232,6 +232,7 @@ const ( version16 = 16 version17 = 17 version18 = 18 + version19 = 19 ) func checkBootstrapped(s Session) (bool, error) { @@ -274,7 +275,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 +363,10 @@ func upgrade(s Session) { upgradeToVer18(s) } + if ver < version19 { + upgradeToVer19(s) + } + updateBootstrapVer(s) _, err = s.Execute(context.Background(), "COMMIT") @@ -484,7 +489,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 +501,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 +583,12 @@ 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)") +} + // updateBootstrapVer updates bootstrap version variable in mysql.TiDB table. func updateBootstrapVer(s Session) { // Update bootstrap version. diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 66cb14bbad793..2d6a7cbcb7f42 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -51,7 +51,7 @@ 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()) @@ -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,7 +148,7 @@ 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) @@ -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/session.go b/session/session.go index b32bc90714e9e..e209bae05baf1 100644 --- a/session/session.go +++ b/session/session.go @@ -610,7 +610,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) } @@ -860,7 +860,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 @@ -1197,7 +1197,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er const ( notBootstrapped = 0 - currentBootstrapVersion = 18 + currentBootstrapVersion = 19 ) func getStoreBootstrapVersion(store kv.Storage) int64 { diff --git a/session/session_test.go b/session/session_test.go index 755bac23b7689..8cbff9ce28aef 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -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) @@ -1655,7 +1655,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 +1688,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 +1722,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 +1757,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 +1789,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 +1809,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 +1834,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 +1854,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 +1975,11 @@ func (s *testSessionSuite) TestRollbackOnCompileError(c *C) { } c.Assert(recoverErr, IsTrue) } + +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/sessionctx/variable/session.go b/sessionctx/variable/session.go index fba74fac657ea..1c9d6fb7d77c0 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -293,6 +293,8 @@ 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 // EnableStreaming indicates whether the coprocessor request can use streaming API. // TODO: remove this after tidb-server configuration "enable-streaming' removed. @@ -330,6 +332,7 @@ func NewSessionVars() *SessionVars { MemQuotaTopn: DefTiDBMemQuotaTopn, MemQuotaIndexLookupReader: DefTiDBMemQuotaIndexLookupReader, MemQuotaIndexLookupJoin: DefTiDBMemQuotaIndexLookupJoin, + MemQuotaNestedLoopApply: DefTiDBMemQuotaNestedLoopApply, } var enableStreaming string if config.GetGlobalConfig().EnableStreaming { @@ -514,6 +517,8 @@ 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))) case TiDBEnableStreaming: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 361b9c483cc2d..c3cbf0916579f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -629,6 +629,7 @@ var defaultSysVars = []*SysVar{ {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"}, /* The following variable is defined as session scope but is actually server scope. */ {ScopeSession, TiDBGeneralLog, strconv.Itoa(DefTiDBGeneralLog)}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 5bba9931742b2..3d4d6df29bd04 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -121,6 +121,7 @@ const ( // "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. @@ -128,6 +129,7 @@ const ( 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" @@ -160,6 +162,7 @@ const ( DefTiDBMemQuotaTopn = 32 << 30 // 32GB. DefTiDBMemQuotaIndexLookupReader = 32 << 30 // 32GB. DefTiDBMemQuotaIndexLookupJoin = 32 << 30 // 32GB. + DefTiDBMemQuotaNestedLoopApply = 32 << 30 // 32GB. DefTiDBGeneralLog = 0 ) diff --git a/statistics/boostrap.go b/statistics/boostrap.go index 8df521565221e..7b5f3d5fe21f2 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) } @@ -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/ddl.go b/statistics/ddl.go index 145cb0b282906..8f4905e7677c8 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) } diff --git a/statistics/ddl_test.go b/statistics/ddl_test.go index 46ee0f6ed12a1..51c0c2e68a3ee 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,7 +104,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) c.Assert(statsTbl.ColumnIsInvalid(sc, tableInfo.Columns[2].ID), IsTrue) @@ -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,7 +148,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) c.Check(statsTbl.Columns[tableInfo.Columns[5].ID].AvgColSize(), Equals, 3.0) 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/handle.go b/statistics/handle.go index 1a687224b14a7..f886e6e2e1d4d 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" @@ -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..e30c474b981fe 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,7 +219,7 @@ 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) + statsTbl := do.StatsHandle().GetTableStats(tableInfo) c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(), Equals, 8.0) // The size of varchar type is LEN + BYTE, here is 1 + 7 = 8 @@ -228,7 +228,7 @@ func (s *testStatsCacheSuite) TestAvgColLen(c *C) { c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(), 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) + statsTbl = do.StatsHandle().GetTableStats(tableInfo) 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) @@ -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) @@ -333,13 +333,13 @@ func (s *testStatsCacheSuite) TestLoadHist(c *C) { 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)") } 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. @@ -355,7 +355,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 +381,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 +406,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 +422,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/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..9def5f69c6e35 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -161,7 +161,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)), @@ -325,7 +325,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 +337,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 +370,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..57e2da921d8e1 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -228,7 +228,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..e1c0c22f1bdbe 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,7 +134,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(0)) rs := testKit.MustQuery("select modify_count from mysql.stats_meta") @@ -159,7 +159,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 +193,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 +213,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 +242,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 +257,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)) } @@ -287,7 +287,7 @@ 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)") @@ -297,7 +297,7 @@ 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(1)) c.Assert(stats.ModifyCount, Equals, int64(0)) @@ -312,7 +312,7 @@ 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)) @@ -325,7 +325,7 @@ 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] 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/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/region_request_test.go b/store/tikv/region_request_test.go index fb65358ae79c3..f663ada863770 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -206,6 +206,9 @@ func (s *mockTikvGrpcServer) RawPut(context.Context, *kvrpcpb.RawPutRequest) (*k func (s *mockTikvGrpcServer) RawDelete(context.Context, *kvrpcpb.RawDeleteRequest) (*kvrpcpb.RawDeleteResponse, 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) RawScan(context.Context, *kvrpcpb.RawScanRequest) (*kvrpcpb.RawScanResponse, 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/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/vendor/github.com/pingcap/kvproto/pkg/coprocessor/coprocessor.pb.go b/vendor/github.com/pingcap/kvproto/pkg/coprocessor/coprocessor.pb.go index 427fbe792cd08..1109103555ee0 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, + // 419 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x74, 0x52, 0xcd, 0x6e, 0xd3, 0x40, + 0x10, 0xae, 0xe3, 0xc4, 0x2d, 0xe3, 0xb4, 0x6a, 0x57, 0x41, 0xb2, 0x7a, 0x68, 0xaa, 0x9c, 0xa0, + 0x08, 0x5b, 0x98, 0x43, 0xef, 0x81, 0x1e, 0x50, 0x39, 0x2d, 0x0f, 0x50, 0xf9, 0x67, 0x70, 0xac, + 0x14, 0xef, 0xb2, 0xbb, 0xad, 0x92, 0x37, 0xe0, 0x11, 0x78, 0x18, 0x1e, 0x20, 0x47, 0xce, 0x1c, + 0x22, 0x04, 0x2f, 0xc2, 0x7a, 0xd6, 0x0e, 0xb9, 0xf4, 0x60, 0xed, 0x37, 0xdf, 0x7c, 0xfb, 0xed, + 0xcc, 0x27, 0xc3, 0x59, 0x21, 0xa4, 0x12, 0x05, 0x6a, 0x2d, 0x54, 0x6c, 0x91, 0x11, 0x2c, 0xdc, + 0xa3, 0xce, 0x8f, 0x51, 0x29, 0xa1, 0x64, 0xee, 0x7a, 0xe7, 0xc7, 0xcb, 0x47, 0x25, 0x8b, 0x5d, + 0x39, 0xa9, 0x44, 0x25, 0x08, 0x26, 0x2d, 0x72, 0xec, 0x2c, 0x85, 0xa3, 0x5b, 0x5c, 0xf3, 0xac, + 0xa9, 0x90, 0x4d, 0x60, 0xa4, 0x4d, 0xa6, 0x4c, 0xe4, 0x5d, 0x7a, 0x2f, 0xc6, 0xdc, 0x15, 0xec, + 0x14, 0x7c, 0x6c, 0xca, 0x68, 0x40, 0x5c, 0x0b, 0x67, 0xdf, 0x3c, 0x38, 0xe4, 0xf8, 0xf5, 0x01, + 0xb5, 0x61, 0x57, 0x70, 0x58, 0x88, 0xc6, 0xe0, 0xca, 0xdd, 0x0a, 0xd3, 0xd3, 0xb8, 0x7f, 0xf6, + 0x9d, 0xe3, 0x79, 0x2f, 0x60, 0x27, 0x30, 0x30, 0x92, 0x8c, 0x7c, 0x6e, 0x11, 0x63, 0x30, 0x2c, + 0x33, 0x93, 0x45, 0x3e, 0x59, 0x13, 0x66, 0xaf, 0x21, 0x50, 0xed, 0x30, 0x3a, 0x1a, 0x5e, 0xfa, + 0xd6, 0xee, 0x79, 0xbc, 0xbf, 0x74, 0x3f, 0x2a, 0xef, 0x44, 0xb3, 0x1f, 0x03, 0x38, 0xe2, 0xa8, + 0xa5, 0x68, 0x34, 0xb2, 0xdb, 0xce, 0x8f, 0xc6, 0x9f, 0x5f, 0x6f, 0xb6, 0xd3, 0x83, 0x5f, 0xdb, + 0x69, 0x52, 0xd5, 0x66, 0xf1, 0x90, 0x5b, 0x9f, 0x2f, 0x89, 0xac, 0x9b, 0xaa, 0xc8, 0x64, 0x62, + 0x6a, 0x99, 0x27, 0x7a, 0x91, 0x29, 0x2c, 0xf3, 0xb5, 0x41, 0x1d, 0x7f, 0x22, 0x3c, 0x6f, 0x71, + 0x37, 0xc8, 0x1b, 0x18, 0x2b, 0xac, 0x6a, 0xd1, 0xdc, 0x51, 0xaa, 0x34, 0x76, 0x98, 0x9e, 0xc4, + 0x7d, 0xc6, 0x37, 0xed, 0xc9, 0x43, 0xa7, 0xa1, 0x82, 0xbd, 0x84, 0xe0, 0x5e, 0x14, 0x4b, 0x2c, + 0x69, 0xa3, 0x30, 0x3d, 0xdb, 0x45, 0xf1, 0xd1, 0xd2, 0x1f, 0x9a, 0xcf, 0x82, 0x77, 0x02, 0x36, + 0x85, 0x50, 0x98, 0x05, 0xaa, 0xce, 0x7c, 0x68, 0xf5, 0xcf, 0x38, 0x10, 0xe5, 0xbc, 0x5e, 0xc1, + 0x88, 0x56, 0x8c, 0x46, 0x64, 0xf5, 0x44, 0x0c, 0x4e, 0xc3, 0xae, 0x61, 0x8c, 0x2b, 0x2c, 0xee, + 0x4a, 0x34, 0x59, 0x7d, 0xaf, 0xa3, 0x80, 0xee, 0x4c, 0x76, 0xcf, 0xdf, 0xd8, 0xe6, 0x7b, 0xd7, + 0xe3, 0x21, 0xfe, 0x2f, 0xe6, 0x57, 0x9b, 0x3f, 0x17, 0xde, 0x4f, 0xfb, 0xfd, 0xb6, 0xdf, 0xf7, + 0xbf, 0x17, 0x07, 0x10, 0xd9, 0x98, 0xe2, 0x2e, 0xa6, 0xd8, 0xd4, 0xcb, 0x47, 0x6b, 0x42, 0x7f, + 0x4a, 0x1e, 0xd0, 0xf1, 0xf6, 0x5f, 0x00, 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..c02ee1105d931 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,55 @@ 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, + // 798 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x64, 0x55, 0xcd, 0x4e, 0xdc, 0x48, + 0x10, 0xc6, 0xf3, 0xe7, 0x71, 0x79, 0x66, 0xe8, 0xe9, 0x65, 0xc1, 0xac, 0xb4, 0x2c, 0x9a, 0x13, + 0x42, 0x5a, 0x56, 0xb0, 0x5a, 0x69, 0x2f, 0x7b, 0x80, 0xd1, 0x4a, 0xa0, 0x00, 0x22, 0x66, 0xc8, + 0x75, 0xe4, 0xb1, 0x7b, 0x7e, 0x12, 0xec, 0x76, 0xda, 0x3d, 0x04, 0x1e, 0x21, 0x52, 0x6e, 0xb9, + 0xe4, 0x91, 0x72, 0xcc, 0x23, 0x44, 0xc9, 0x21, 0xaf, 0x91, 0xea, 0x76, 0xdb, 0x78, 0xe0, 0x30, + 0x52, 0xd5, 0xd7, 0xd5, 0xd5, 0x5f, 0x7d, 0x5f, 0x59, 0x03, 0x5d, 0x26, 0x82, 0xa9, 0x4c, 0x27, + 0x07, 0xa9, 0xe0, 0x92, 0x53, 0xdb, 0xa4, 0x83, 0x8f, 0x16, 0x34, 0xff, 0x4f, 0xa4, 0x78, 0xa0, + 0x87, 0x00, 0x4c, 0x05, 0x63, 0xf9, 0x90, 0x32, 0xcf, 0xda, 0xb5, 0xf6, 0x7a, 0x47, 0xf4, 0xa0, + 0xb8, 0xa6, 0x6b, 0x46, 0x78, 0xe2, 0x3b, 0xac, 0x08, 0x29, 0x85, 0x86, 0x64, 0x22, 0xf6, 0x6a, + 0x58, 0xdc, 0xf0, 0x75, 0x4c, 0x37, 0xa0, 0xb9, 0x48, 0x22, 0x76, 0xef, 0xd5, 0x35, 0x98, 0x27, + 0xaa, 0x32, 0x0a, 0x64, 0xe0, 0x35, 0x10, 0xec, 0xf8, 0x3a, 0xa6, 0xdb, 0xd0, 0xce, 0x1e, 0x92, + 0x70, 0x7c, 0xcb, 0x67, 0x5e, 0x13, 0xf1, 0xb6, 0x6f, 0xab, 0xfc, 0x9c, 0xcf, 0x06, 0x1c, 0xc8, + 0x75, 0x12, 0xa4, 0xd9, 0x9c, 0xcb, 0x0b, 0x26, 0x03, 0x5d, 0x8e, 0xfc, 0x42, 0x9e, 0x4c, 0xc7, + 0x99, 0x0c, 0x64, 0xce, 0xcf, 0xad, 0xf0, 0x1b, 0xe2, 0xd1, 0xb5, 0x3a, 0xf1, 0x9d, 0xb0, 0x08, + 0x1f, 0xb9, 0xd4, 0x9e, 0x70, 0xd1, 0xac, 0xeb, 0x8f, 0xac, 0x07, 0x37, 0xd0, 0x2e, 0x1e, 0x2c, + 0xb9, 0x5a, 0x15, 0xae, 0xff, 0x40, 0x3b, 0x36, 0x44, 0x74, 0x33, 0xf7, 0x68, 0xbb, 0x7c, 0xfa, + 0x29, 0x53, 0xbf, 0x2c, 0x1d, 0xfc, 0xa8, 0x81, 0x7d, 0xc1, 0xb2, 0x2c, 0x98, 0x31, 0xfa, 0x17, + 0xb6, 0xc8, 0x66, 0x55, 0x75, 0x37, 0xca, 0x16, 0xa6, 0x46, 0xeb, 0x6b, 0x63, 0x95, 0x56, 0xb7, + 0x07, 0x35, 0xc9, 0x0d, 0x75, 0x8c, 0x14, 0xaf, 0xa9, 0xe0, 0x25, 0x6f, 0x15, 0x97, 0xb3, 0x34, + 0x2a, 0x0e, 0xa0, 0xae, 0x28, 0xe9, 0x58, 0xe3, 0x4d, 0x8d, 0xdb, 0x98, 0x8f, 0x56, 0xcc, 0x69, + 0x55, 0x05, 0xd9, 0x03, 0x5b, 0x79, 0xba, 0x60, 0x99, 0x67, 0xef, 0xd6, 0x71, 0xb6, 0xde, 0xaa, + 0xed, 0x7e, 0x71, 0x4c, 0x37, 0xa1, 0x15, 0xf2, 0x38, 0x5e, 0x48, 0xaf, 0xad, 0x1b, 0x98, 0x8c, + 0xfe, 0x89, 0x56, 0x1a, 0x15, 0x3c, 0x47, 0xcb, 0xd3, 0x7f, 0x26, 0x8f, 0x5f, 0x96, 0xa8, 0x36, + 0x82, 0xbd, 0x66, 0xa1, 0xf4, 0x40, 0xfb, 0x6e, 0x32, 0xfa, 0x07, 0xb8, 0x79, 0x34, 0x9e, 0x2f, + 0x12, 0xe9, 0xb9, 0xfa, 0x0d, 0xc8, 0xa1, 0x53, 0x44, 0xa8, 0x07, 0x36, 0xba, 0x2b, 0xd9, 0xbd, + 0xf4, 0x3a, 0xda, 0x9d, 0x22, 0x1d, 0xbc, 0x00, 0xe7, 0x34, 0x10, 0x51, 0xee, 0x7b, 0xa1, 0x8a, + 0x55, 0x51, 0x05, 0xb1, 0x3b, 0x8e, 0x8b, 0x63, 0x76, 0x55, 0xc5, 0x95, 0x71, 0xea, 0xd5, 0x71, + 0x06, 0xff, 0x81, 0x33, 0xac, 0x2e, 0x51, 0xc2, 0x23, 0xd4, 0xc6, 0x42, 0x6d, 0x50, 0x33, 0x9d, + 0xd0, 0xdf, 0x50, 0x64, 0x16, 0x88, 0x84, 0x89, 0x0c, 0x5b, 0xaa, 0x83, 0x32, 0x1f, 0xbc, 0xb7, + 0x00, 0xd4, 0xfd, 0xe1, 0x3c, 0x48, 0x66, 0xda, 0xc7, 0x45, 0x64, 0xb8, 0x60, 0x44, 0xff, 0x05, + 0x37, 0xd4, 0x27, 0xf9, 0x2e, 0xd4, 0xf4, 0x2e, 0x6c, 0xad, 0x6c, 0x72, 0x7e, 0x53, 0xaf, 0x03, + 0x84, 0x65, 0x4c, 0xb7, 0xc0, 0x56, 0xaf, 0x8f, 0xb1, 0x9d, 0x21, 0xac, 0xd2, 0xb3, 0xa8, 0xaa, + 0x4b, 0x63, 0x45, 0x97, 0xfd, 0x43, 0x70, 0xca, 0x4f, 0x97, 0xae, 0x83, 0xab, 0x93, 0x4b, 0x2e, + 0xe2, 0xe0, 0x96, 0xac, 0xd1, 0x5f, 0x60, 0x5d, 0x03, 0x8f, 0x6f, 0x12, 0x6b, 0xff, 0x43, 0x1d, + 0xdc, 0xca, 0x42, 0x52, 0x80, 0xd6, 0x45, 0x36, 0x3b, 0x5d, 0xa6, 0x78, 0xc1, 0xc5, 0x7d, 0xce, + 0x66, 0x27, 0x2c, 0x90, 0xc4, 0xc2, 0xc1, 0x00, 0x93, 0x2b, 0xc1, 0x53, 0x9e, 0x31, 0x52, 0xa3, + 0x5d, 0x70, 0x30, 0x3f, 0x4e, 0x53, 0x96, 0x44, 0xa4, 0x4e, 0x7f, 0x85, 0x7e, 0x99, 0xfa, 0x2c, + 0x4b, 0x79, 0x82, 0x55, 0x0d, 0x34, 0xa2, 0x87, 0xb0, 0xcf, 0xde, 0x2e, 0x59, 0x26, 0x5f, 0xa1, + 0x0d, 0xa4, 0x89, 0x6a, 0x6e, 0xae, 0x62, 0x65, 0x7d, 0x4b, 0x91, 0xc6, 0xb3, 0x62, 0x8b, 0x88, + 0x4d, 0x09, 0x74, 0x14, 0x1f, 0x54, 0x5b, 0x4e, 0x14, 0x91, 0x36, 0x8e, 0xbf, 0x51, 0x45, 0xca, + 0xcb, 0x8e, 0x79, 0xec, 0x26, 0x11, 0x2c, 0x40, 0x1d, 0x27, 0xb7, 0x8c, 0x00, 0xed, 0x43, 0xd7, + 0x34, 0x54, 0x06, 0x2f, 0x33, 0xe2, 0x9a, 0xb2, 0xe1, 0x9c, 0x85, 0x6f, 0x5e, 0x2e, 0xb9, 0x58, + 0xc6, 0xa4, 0x63, 0xe8, 0x8f, 0x44, 0x90, 0x64, 0x53, 0x26, 0xce, 0x59, 0x10, 0x31, 0x41, 0xba, + 0xe6, 0xf6, 0x68, 0x11, 0x33, 0xbe, 0x94, 0x97, 0xfc, 0x1d, 0xe9, 0x19, 0x42, 0x3e, 0x56, 0x9c, + 0xa9, 0xef, 0x89, 0xac, 0xe3, 0xce, 0x90, 0x2a, 0xa2, 0x08, 0x11, 0x62, 0x3a, 0x9a, 0x29, 0xaf, + 0x04, 0xd3, 0xc3, 0xf7, 0xe9, 0xef, 0xb0, 0xfd, 0x0c, 0x2e, 0x47, 0xa0, 0xfb, 0xc7, 0xd0, 0x5b, + 0x5d, 0x09, 0x65, 0xc2, 0x71, 0x14, 0x5d, 0xa2, 0xf5, 0xe8, 0x08, 0x9a, 0xe0, 0xb3, 0x98, 0xdf, + 0x31, 0x9d, 0x5b, 0x6a, 0x14, 0x3c, 0x3c, 0xcf, 0x77, 0x51, 0x63, 0xb5, 0x13, 0xf2, 0xf9, 0xdb, + 0x8e, 0xf5, 0x05, 0x7f, 0x5f, 0xf1, 0xf7, 0xe9, 0xfb, 0xce, 0xda, 0xa4, 0xa5, 0xff, 0x06, 0xfe, + 0xfe, 0x19, 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..e32f260594085 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) @@ -1923,43 +1942,43 @@ var ( func init() { proto.RegisterFile("errorpb.proto", fileDescriptorErrorpb) } var fileDescriptorErrorpb = []byte{ - // 604 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x54, 0xdd, 0x6e, 0xd3, 0x30, - 0x14, 0x26, 0xfb, 0x69, 0x97, 0xd3, 0x34, 0xdb, 0xcc, 0xd8, 0xa2, 0x4d, 0xab, 0xa6, 0x88, 0x8b, - 0x09, 0x89, 0x4e, 0x0c, 0xae, 0x40, 0x20, 0x31, 0x54, 0xa4, 0xa9, 0x5b, 0x41, 0x2e, 0xf7, 0x91, - 0xdb, 0x9c, 0x76, 0x51, 0x17, 0xbb, 0xb2, 0xdd, 0x4d, 0xd9, 0x8b, 0xc0, 0x23, 0x71, 0xc9, 0x23, - 0xa0, 0xf1, 0x22, 0xc8, 0x4e, 0xda, 0x26, 0xbd, 0xd8, 0x55, 0xce, 0x39, 0xf6, 0x77, 0xfc, 0xf9, - 0x7c, 0x9f, 0x03, 0x4d, 0x94, 0x52, 0xc8, 0xe9, 0xa0, 0x3d, 0x95, 0x42, 0x0b, 0x52, 0x2f, 0xd2, - 0x43, 0x2f, 0x45, 0xcd, 0xe6, 0xe5, 0xc3, 0xbd, 0xb1, 0x18, 0x0b, 0x1b, 0x9e, 0x99, 0x28, 0xaf, - 0x86, 0x3d, 0x70, 0x7b, 0x42, 0x5f, 0x21, 0x8b, 0x51, 0x92, 0x23, 0x70, 0x25, 0x8e, 0x13, 0xc1, - 0xa3, 0x24, 0x0e, 0x9c, 0x13, 0xe7, 0x74, 0x83, 0x6e, 0xe5, 0x85, 0xcb, 0x98, 0xbc, 0x84, 0xda, - 0xad, 0xdd, 0x16, 0xac, 0x9d, 0x38, 0xa7, 0x8d, 0x73, 0xaf, 0x5d, 0xb4, 0xff, 0x8e, 0x28, 0x69, - 0xb1, 0x16, 0x6e, 0x43, 0xb3, 0xaf, 0x85, 0xc4, 0x9e, 0xd0, 0xd7, 0x4c, 0x0f, 0x6f, 0xc2, 0xd7, - 0xe0, 0x53, 0xdb, 0xa2, 0x27, 0xf4, 0x57, 0x31, 0xe3, 0xf1, 0x93, 0xa7, 0x84, 0x33, 0xf0, 0xbb, - 0x98, 0xf5, 0x84, 0xbe, 0xe4, 0x39, 0x8c, 0xec, 0xc0, 0xfa, 0x04, 0x33, 0xbb, 0xd1, 0xa3, 0x26, - 0xac, 0x36, 0x58, 0x5b, 0xa1, 0x79, 0x04, 0xae, 0xd2, 0x4c, 0xea, 0xc8, 0x80, 0xd6, 0x2d, 0x68, - 0xcb, 0x16, 0xba, 0x98, 0x91, 0x03, 0xa8, 0x23, 0x8f, 0xed, 0xd2, 0x86, 0x5d, 0xaa, 0x21, 0x8f, - 0xbb, 0x98, 0x85, 0x1f, 0x01, 0xfa, 0x9a, 0xdd, 0x62, 0x67, 0x2a, 0x86, 0x37, 0xe4, 0x0c, 0x1a, - 0x1c, 0xef, 0xa3, 0xbc, 0xa7, 0x0a, 0x9c, 0x93, 0xf5, 0xd3, 0xc6, 0xb9, 0x3f, 0xbf, 0x6f, 0xce, - 0x8b, 0x02, 0xc7, 0xfb, 0x3c, 0x54, 0x61, 0x07, 0xbc, 0x3e, 0xca, 0x3b, 0x94, 0x97, 0xea, 0x62, - 0xa6, 0x32, 0xb2, 0x0f, 0x35, 0x89, 0x4c, 0x09, 0x6e, 0x69, 0xbb, 0xb4, 0xc8, 0xc8, 0x31, 0xc0, - 0x80, 0x0d, 0x27, 0x62, 0x34, 0x8a, 0x52, 0x55, 0x50, 0x77, 0x8b, 0xca, 0xb5, 0x0a, 0x7d, 0xf0, - 0x2c, 0x8b, 0x2f, 0x22, 0x4d, 0x19, 0x8f, 0xc3, 0x6f, 0xb0, 0x4b, 0xd9, 0x48, 0x77, 0xb8, 0x96, - 0xd9, 0x0f, 0x21, 0xae, 0x98, 0x1c, 0xe3, 0xd3, 0x22, 0x1d, 0x03, 0xa0, 0xd9, 0x1d, 0xa9, 0xe4, - 0x01, 0xe7, 0x07, 0xd8, 0x4a, 0x3f, 0x79, 0xc0, 0xf0, 0xe7, 0x06, 0x6c, 0x76, 0x8c, 0x3b, 0x48, - 0x00, 0xf5, 0x14, 0x95, 0x62, 0x63, 0x2c, 0x28, 0xce, 0x53, 0xf2, 0x06, 0x80, 0x0b, 0x1d, 0x55, - 0xb4, 0x26, 0xed, 0xb9, 0xc5, 0x16, 0x66, 0xa1, 0x2e, 0x5f, 0xf8, 0xe6, 0x33, 0xec, 0x14, 0x94, - 0x0c, 0x72, 0x64, 0x54, 0xb6, 0xa3, 0x6f, 0x9c, 0x1f, 0x2c, 0x80, 0x55, 0x13, 0x50, 0x5f, 0x56, - 0x4d, 0x71, 0x01, 0xbb, 0x13, 0xcc, 0x2c, 0x3e, 0xe1, 0xc5, 0xe4, 0xad, 0x46, 0xe5, 0x1e, 0x55, - 0x67, 0x50, 0x7f, 0x52, 0x75, 0xca, 0x3b, 0x68, 0x28, 0x33, 0xbe, 0x08, 0x8d, 0x8a, 0xc1, 0xa6, - 0x45, 0x3f, 0x5f, 0xa0, 0x97, 0x02, 0x53, 0x50, 0x4b, 0xb1, 0x3f, 0x80, 0xaf, 0xac, 0x76, 0x51, - 0xa2, 0xa2, 0xc1, 0x4c, 0x65, 0x41, 0xcd, 0x02, 0x5f, 0x2c, 0x81, 0x25, 0x69, 0xa9, 0xa7, 0xca, - 0x42, 0xbf, 0x87, 0x66, 0x7e, 0xe4, 0x30, 0x97, 0x2c, 0xa8, 0xaf, 0x62, 0x4b, 0x7a, 0x52, 0x4f, - 0x95, 0x32, 0xf2, 0x09, 0xb6, 0x95, 0x79, 0x2a, 0xf6, 0xd2, 0xa9, 0x79, 0x2c, 0xc1, 0x96, 0x45, - 0xef, 0x97, 0xd0, 0xa5, 0xa7, 0x44, 0x9b, 0xaa, 0x9c, 0x92, 0x2e, 0xec, 0x49, 0x36, 0xd2, 0x51, - 0x2e, 0xb8, 0x16, 0x22, 0xba, 0x35, 0x06, 0x09, 0x5c, 0xdb, 0xe4, 0x70, 0x39, 0xf9, 0x55, 0x0b, - 0xd1, 0x5d, 0xb9, 0x5a, 0xba, 0x78, 0xf5, 0xfb, 0xb1, 0xe5, 0xfc, 0x79, 0x6c, 0x39, 0x7f, 0x1f, - 0x5b, 0xce, 0xaf, 0x7f, 0xad, 0x67, 0x10, 0x0c, 0x45, 0xda, 0x9e, 0x26, 0x7c, 0x3c, 0x64, 0xd3, - 0xb6, 0x4e, 0x26, 0x77, 0xed, 0xc9, 0x9d, 0xfd, 0x67, 0x0c, 0x6a, 0xf6, 0xf3, 0xf6, 0x7f, 0x00, - 0x00, 0x00, 0xff, 0xff, 0x6f, 0xa9, 0xa3, 0xe6, 0x78, 0x04, 0x00, 0x00, + // 594 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x7c, 0x54, 0xdf, 0x8e, 0x12, 0x3f, + 0x14, 0xfe, 0xb1, 0xb0, 0xc0, 0x1c, 0x60, 0x76, 0xe9, 0x6f, 0xdd, 0x9d, 0x60, 0xdc, 0x98, 0x89, + 0x17, 0xc6, 0xc4, 0xd9, 0xb8, 0x7a, 0xa5, 0xd1, 0x44, 0x0c, 0x26, 0x64, 0xff, 0x68, 0x8a, 0xf7, + 0x93, 0xc2, 0x94, 0xd9, 0x09, 0x4c, 0x4b, 0xda, 0x82, 0xc1, 0x17, 0xd1, 0x47, 0xf2, 0xd2, 0x47, + 0x30, 0xfa, 0x22, 0xb6, 0x9d, 0x01, 0x66, 0xb8, 0xd8, 0x8b, 0x09, 0x3d, 0xe7, 0xf4, 0xfb, 0xce, + 0x69, 0xbf, 0xaf, 0x40, 0x87, 0x0a, 0xc1, 0xc5, 0x62, 0x1c, 0x2c, 0x04, 0x57, 0x1c, 0x35, 0xf2, + 0xb0, 0xd7, 0x4e, 0xa9, 0x22, 0x9b, 0x74, 0xef, 0x24, 0xe6, 0x31, 0xb7, 0xcb, 0x0b, 0xb3, 0xca, + 0xb2, 0xfe, 0x2d, 0x38, 0xb7, 0x5c, 0x5d, 0x53, 0x12, 0x51, 0x81, 0x1e, 0x82, 0x23, 0x68, 0x9c, + 0x70, 0x16, 0x26, 0x91, 0x57, 0x79, 0x5c, 0x79, 0x5a, 0xc3, 0xcd, 0x2c, 0x31, 0x8c, 0xd0, 0x13, + 0xa8, 0xcf, 0xed, 0x36, 0xef, 0x40, 0x57, 0x5a, 0x97, 0xed, 0x20, 0xa7, 0xff, 0x4c, 0xa9, 0xc0, + 0x79, 0xcd, 0x3f, 0x82, 0xce, 0x48, 0x71, 0x41, 0x35, 0xe9, 0x0d, 0x51, 0x93, 0x3b, 0xff, 0x39, + 0xb8, 0xd8, 0x52, 0xe8, 0xcc, 0x47, 0xbe, 0x64, 0xd1, 0xbd, 0x5d, 0xfc, 0x25, 0xb8, 0x57, 0x74, + 0xad, 0xf7, 0x0e, 0x59, 0x06, 0x43, 0xc7, 0x50, 0x9d, 0xd1, 0xb5, 0xdd, 0xd8, 0xc6, 0x66, 0x59, + 0x26, 0x38, 0xd8, 0x1b, 0x53, 0x17, 0xa5, 0x22, 0x42, 0x85, 0x06, 0x54, 0xb5, 0xa0, 0xa6, 0x4d, + 0x68, 0x5a, 0x74, 0x06, 0x0d, 0xca, 0x22, 0x5b, 0xaa, 0xd9, 0x52, 0x5d, 0x87, 0xba, 0xe0, 0xbf, + 0x05, 0x18, 0x29, 0x32, 0xa7, 0x83, 0x05, 0x9f, 0xdc, 0xa1, 0x0b, 0x68, 0x31, 0xfa, 0x35, 0xcc, + 0x38, 0xa5, 0x6e, 0x5d, 0xd5, 0xe7, 0x75, 0x37, 0xe7, 0xcd, 0xe6, 0xc2, 0xa0, 0xb7, 0x64, 0x4b, + 0xe9, 0x0f, 0xa0, 0x3d, 0xa2, 0x62, 0x45, 0xc5, 0x50, 0xf6, 0x97, 0x72, 0x8d, 0x4e, 0xa1, 0x2e, + 0x28, 0x91, 0x9c, 0xd9, 0xb1, 0x1d, 0x9c, 0x47, 0xe8, 0x11, 0xc0, 0x98, 0x4c, 0x66, 0x7c, 0x3a, + 0x0d, 0x53, 0x99, 0x8f, 0xee, 0xe4, 0x99, 0x1b, 0xe9, 0xbb, 0x9a, 0xc6, 0x4c, 0xf1, 0x81, 0xa7, + 0x29, 0x61, 0x91, 0xff, 0x09, 0xba, 0x98, 0x4c, 0xd5, 0x80, 0x29, 0xb1, 0xfe, 0xc2, 0xf9, 0x35, + 0x11, 0x31, 0xbd, 0x5f, 0x24, 0xdd, 0x80, 0x9a, 0xdd, 0xa1, 0x4c, 0xbe, 0xd1, 0x4d, 0x03, 0x9b, + 0x19, 0xe9, 0x84, 0xff, 0xbd, 0x06, 0x87, 0x03, 0xe3, 0x0e, 0xe4, 0x41, 0x23, 0xa5, 0x52, 0x92, + 0x98, 0xe6, 0x23, 0x6e, 0x42, 0xf4, 0x02, 0x80, 0x71, 0x15, 0x96, 0xb4, 0x46, 0xc1, 0xc6, 0x62, + 0x5b, 0xb3, 0x60, 0x87, 0x6d, 0x7d, 0xf3, 0x1e, 0x8e, 0xf3, 0x91, 0x0c, 0x72, 0x6a, 0x54, 0xb6, + 0x57, 0xdf, 0xba, 0x3c, 0xdb, 0x02, 0xcb, 0x26, 0xc0, 0xae, 0x28, 0x9b, 0xa2, 0x0f, 0x5d, 0xad, + 0x8a, 0xc5, 0x27, 0x2c, 0xbf, 0x79, 0xab, 0x51, 0x91, 0xa3, 0xec, 0x0c, 0xec, 0xce, 0xca, 0x4e, + 0x79, 0x05, 0x2d, 0x69, 0xae, 0x2f, 0xa4, 0x46, 0x45, 0xef, 0xd0, 0xa2, 0xff, 0xdf, 0xa2, 0x77, + 0x02, 0x63, 0x90, 0x3b, 0xb1, 0xdf, 0x80, 0x2b, 0xad, 0x76, 0x61, 0x22, 0xc3, 0xb1, 0x56, 0xcf, + 0xab, 0x5b, 0xe0, 0x83, 0x1d, 0xb0, 0x20, 0x2d, 0x6e, 0xcb, 0xa2, 0xd0, 0xaf, 0xa1, 0x93, 0xb5, + 0x9c, 0x64, 0x92, 0x79, 0x8d, 0x7d, 0x6c, 0x41, 0x4f, 0x8d, 0x2d, 0x44, 0xe8, 0x1d, 0x1c, 0x49, + 0xf3, 0x54, 0xec, 0xa1, 0x53, 0xf3, 0x58, 0xbc, 0xa6, 0x45, 0x9f, 0x16, 0xd0, 0x85, 0xa7, 0x84, + 0x3b, 0xb2, 0x18, 0xa2, 0x2b, 0x38, 0x11, 0xda, 0x1d, 0x61, 0x26, 0xb8, 0xe2, 0x3c, 0x9c, 0x1b, + 0x83, 0x78, 0x8e, 0x25, 0xe9, 0xed, 0x6e, 0x7e, 0xdf, 0x42, 0xb8, 0x2b, 0xf6, 0x53, 0xfd, 0x67, + 0x3f, 0xff, 0x9c, 0x57, 0x7e, 0xe9, 0xef, 0xb7, 0xfe, 0x7e, 0xfc, 0x3d, 0xff, 0x0f, 0x3c, 0x7d, + 0xa4, 0x60, 0x91, 0xb0, 0x78, 0x42, 0x16, 0x81, 0x4a, 0x66, 0xab, 0x60, 0xb6, 0xb2, 0xff, 0x19, + 0xe3, 0xba, 0xfd, 0x79, 0xf9, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x6f, 0xa9, 0xa3, 0xe6, 0x78, 0x04, + 0x00, 0x00, } 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..39e92c50cd869 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go @@ -49,6 +49,8 @@ RawDeleteResponse DeleteRangeRequest DeleteRangeResponse + RawDeleteRangeRequest + RawDeleteRangeResponse RawScanRequest RawScanResponse WriteInfo @@ -1416,6 +1418,62 @@ 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{40} } + +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{41} } + +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"` @@ -1425,7 +1483,7 @@ type RawScanRequest struct { 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{42} } func (m *RawScanRequest) GetContext() *Context { if m != nil { @@ -1456,7 +1514,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{43} } func (m *RawScanResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1481,7 +1539,7 @@ type WriteInfo struct { 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 (*WriteInfo) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{44} } func (m *WriteInfo) GetStartTs() uint64 { if m != nil { @@ -1513,7 +1571,7 @@ type ValueInfo struct { 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 (*ValueInfo) Descriptor() ([]byte, []int) { return fileDescriptorKvrpcpb, []int{45} } func (m *ValueInfo) GetValue() []byte { if m != nil { @@ -1545,7 +1603,7 @@ type MvccInfo struct { 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{46} } func (m *MvccInfo) GetLock() *LockInfo { if m != nil { @@ -1576,7 +1634,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{47} } func (m *MvccGetByKeyRequest) GetContext() *Context { if m != nil { @@ -1601,7 +1659,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{48} } func (m *MvccGetByKeyResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1632,7 +1690,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{49} } func (m *MvccGetByStartTsRequest) GetContext() *Context { if m != nil { @@ -1658,7 +1716,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{50} } func (m *MvccGetByStartTsResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1696,7 +1754,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{51} } func (m *SplitRegionRequest) GetContext() *Context { if m != nil { @@ -1721,7 +1779,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{52} } func (m *SplitRegionResponse) GetRegionError() *errorpb.Error { if m != nil { @@ -1785,6 +1843,8 @@ func init() { proto.RegisterType((*RawDeleteResponse)(nil), "kvrpcpb.RawDeleteResponse") 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") @@ -3444,7 +3504,7 @@ func (m *DeleteRangeResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *RawScanRequest) 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) @@ -3454,7 +3514,7 @@ func (m *RawScanRequest) Marshal() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *RawScanRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *RawDeleteRangeRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -3475,6 +3535,80 @@ func (m *RawScanRequest) MarshalTo(dAtA []byte) (int, error) { 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 *RawDeleteRangeResponse) 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 *RawDeleteRangeResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.RegionError != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) + n47, err := m.RegionError.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n47 + } + 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 *RawScanRequest) 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 *RawScanRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Context != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) + n48, err := m.Context.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n48 + } + if len(m.StartKey) > 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++ @@ -3502,11 +3636,11 @@ func (m *RawScanResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) - n47, err := m.RegionError.MarshalTo(dAtA[i:]) + n49, err := m.RegionError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n47 + i += n49 } if len(m.Kvs) > 0 { for _, msg := range m.Kvs { @@ -3614,11 +3748,11 @@ func (m *MvccInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Lock.Size())) - n48, err := m.Lock.MarshalTo(dAtA[i:]) + n50, err := m.Lock.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n48 + i += n50 } if len(m.Writes) > 0 { for _, msg := range m.Writes { @@ -3666,11 +3800,11 @@ 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:]) + n51, err := m.Context.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n51 } if len(m.Key) > 0 { dAtA[i] = 0x12 @@ -3700,11 +3834,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:]) + n52, err := m.RegionError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n52 } if len(m.Error) > 0 { dAtA[i] = 0x12 @@ -3716,11 +3850,11 @@ func (m *MvccGetByKeyResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Info.Size())) - n51, err := m.Info.MarshalTo(dAtA[i:]) + n53, err := m.Info.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n53 } return i, nil } @@ -3744,11 +3878,11 @@ func (m *MvccGetByStartTsRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Context.Size())) - n52, err := m.Context.MarshalTo(dAtA[i:]) + n54, err := m.Context.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n54 } if m.StartTs != 0 { dAtA[i] = 0x10 @@ -3777,11 +3911,11 @@ func (m *MvccGetByStartTsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.RegionError.Size())) - n53, err := m.RegionError.MarshalTo(dAtA[i:]) + n55, err := m.RegionError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n55 } if len(m.Error) > 0 { dAtA[i] = 0x12 @@ -3799,11 +3933,11 @@ func (m *MvccGetByStartTsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Info.Size())) - n54, err := m.Info.MarshalTo(dAtA[i:]) + n56, err := m.Info.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n56 } return i, nil } @@ -3827,11 +3961,11 @@ 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:]) + n57, err := m.Context.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n57 } if len(m.SplitKey) > 0 { dAtA[i] = 0x12 @@ -3861,31 +3995,31 @@ 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:]) + n58, err := m.RegionError.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n56 + i += n58 } if m.Left != nil { dAtA[i] = 0x12 i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Left.Size())) - n57, err := m.Left.MarshalTo(dAtA[i:]) + n59, err := m.Left.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n57 + i += n59 } if m.Right != nil { dAtA[i] = 0x1a i++ i = encodeVarintKvrpcpb(dAtA, i, uint64(m.Right.Size())) - n58, err := m.Right.MarshalTo(dAtA[i:]) + n60, err := m.Right.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n58 + i += n60 } return i, nil } @@ -4609,6 +4743,38 @@ func (m *DeleteRangeResponse) Size() (n int) { 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 @@ -10078,6 +10244,263 @@ func (m *DeleteRangeResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *RawDeleteRangeRequest) 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: RawDeleteRangeRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RawDeleteRangeRequest: 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) + } + 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 StartKey", 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.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: + 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 *RawDeleteRangeResponse) 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: RawDeleteRangeResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RawDeleteRangeResponse: 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 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.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) + } + 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.Error = 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 *RawScanRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -11605,117 +12028,116 @@ 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, + // 1773 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xbc, 0x59, 0x5f, 0x6f, 0x1b, 0x45, + 0x10, 0xef, 0x9d, 0xff, 0x9d, 0xc7, 0x7f, 0xe2, 0x6e, 0xd2, 0xc6, 0x14, 0x28, 0xed, 0x41, 0x05, + 0x04, 0x91, 0x8a, 0xb4, 0xe2, 0x11, 0xa1, 0xa6, 0xa5, 0x0d, 0x4d, 0x48, 0x74, 0x89, 0x52, 0x55, + 0x02, 0x8e, 0xcb, 0x79, 0xe3, 0x9c, 0x72, 0xbe, 0xbb, 0xde, 0xad, 0x9d, 0x58, 0x08, 0xf1, 0x80, + 0x40, 0xe2, 0x11, 0x84, 0x04, 0x0f, 0xf0, 0xc0, 0x23, 0x9f, 0x81, 0x2f, 0xc0, 0x23, 0x1f, 0x01, + 0xc1, 0x17, 0x61, 0x76, 0xf7, 0xfe, 0xf8, 0xec, 0x04, 0x22, 0xcb, 0xf5, 0x43, 0xe4, 0xdb, 0x99, + 0xd9, 0x9d, 0x99, 0xdf, 0xcc, 0xce, 0xec, 0x6e, 0xa0, 0x71, 0x3c, 0x08, 0x03, 0x3b, 0x38, 0x58, + 0x0d, 0x42, 0x9f, 0xf9, 0xa4, 0x12, 0x0f, 0xaf, 0xd5, 0x7b, 0x94, 0x59, 0x09, 0xf9, 0x5a, 0x83, + 0x86, 0xa1, 0x1f, 0xa6, 0xc3, 0xa5, 0xae, 0xdf, 0xf5, 0xc5, 0xe7, 0x6d, 0xfe, 0x25, 0xa9, 0xfa, + 0x17, 0xa0, 0x6d, 0xfa, 0xf6, 0xf1, 0x86, 0x77, 0xe8, 0x93, 0x9b, 0x50, 0x0f, 0x42, 0xa7, 0x67, + 0x85, 0x43, 0xd3, 0x45, 0x5a, 0x5b, 0xb9, 0xa1, 0xbc, 0x51, 0x37, 0x6a, 0x31, 0x8d, 0x8b, 0x71, + 0x11, 0xce, 0x32, 0x07, 0x34, 0x8c, 0x1c, 0xdf, 0x6b, 0xab, 0x28, 0x52, 0x34, 0x6a, 0x9c, 0xb6, + 0x2f, 0x49, 0xa4, 0x05, 0x85, 0x63, 0x3a, 0x6c, 0x17, 0xc4, 0x64, 0xfe, 0x49, 0x5e, 0x00, 0x4d, + 0x4c, 0x62, 0xcc, 0x6d, 0x17, 0xc5, 0x84, 0x0a, 0x1f, 0xef, 0x31, 0x57, 0x77, 0x40, 0x7b, 0x4c, + 0x87, 0x0f, 0xb8, 0xa1, 0xe4, 0x4d, 0x28, 0x73, 0x32, 0xed, 0x08, 0xc5, 0xb5, 0xb5, 0xcb, 0xab, + 0x89, 0x9b, 0x89, 0x85, 0x46, 0x2c, 0x40, 0x5e, 0x82, 0x6a, 0x48, 0x59, 0x38, 0xb4, 0x0e, 0x5c, + 0x2a, 0x6c, 0xa8, 0x1a, 0x19, 0x81, 0x2c, 0x41, 0xc9, 0x3a, 0xf0, 0x43, 0x26, 0x6c, 0xa8, 0x1a, + 0x72, 0xa0, 0xff, 0x52, 0x80, 0xca, 0xba, 0xef, 0x31, 0x7a, 0xca, 0xc8, 0x8b, 0x7c, 0x7e, 0x17, + 0xad, 0x35, 0x1d, 0xa9, 0xad, 0x68, 0x68, 0x92, 0xb0, 0xd1, 0x21, 0xef, 0x42, 0x3d, 0x66, 0xd2, + 0xc0, 0xb7, 0x8f, 0xc4, 0xfa, 0xb5, 0xb5, 0xc5, 0xd5, 0x18, 0x5c, 0x43, 0xf0, 0x1e, 0x70, 0x96, + 0x51, 0x0b, 0xb3, 0x01, 0xb9, 0x01, 0xc5, 0x80, 0xd2, 0x50, 0x68, 0xad, 0xad, 0xd5, 0x13, 0xf9, + 0x1d, 0xa4, 0x19, 0x82, 0x43, 0x08, 0x14, 0x19, 0x0d, 0x7b, 0xed, 0x92, 0xd0, 0x28, 0xbe, 0xc9, + 0x6d, 0xd0, 0x10, 0x60, 0x3f, 0x74, 0xd8, 0xb0, 0x5d, 0x46, 0x7a, 0x13, 0x35, 0x25, 0x7e, 0xaf, + 0xfb, 0xbd, 0x9e, 0xe5, 0x75, 0x76, 0x42, 0xc7, 0x48, 0x85, 0xc8, 0xfb, 0xb0, 0xe0, 0x44, 0xbe, + 0x6b, 0x31, 0x6e, 0xa1, 0x4b, 0x07, 0xd4, 0x6d, 0x57, 0xc4, 0xbc, 0xe5, 0x74, 0xde, 0x46, 0xc2, + 0xdf, 0xe4, 0x6c, 0xa3, 0xe9, 0xe4, 0xc6, 0xe4, 0x35, 0x68, 0x7a, 0x3e, 0x33, 0x0f, 0x1d, 0xd7, + 0x35, 0x6d, 0xcb, 0x3e, 0xa2, 0x6d, 0x0d, 0x17, 0xd0, 0x8c, 0x3a, 0x52, 0x3f, 0x40, 0xe2, 0x3a, + 0xa7, 0xf1, 0xa8, 0x45, 0x43, 0xcf, 0xc6, 0x54, 0xe8, 0xb6, 0xab, 0x82, 0x5f, 0xe1, 0xe3, 0x4d, + 0xbf, 0x4b, 0x5e, 0x81, 0xda, 0x11, 0xda, 0xe5, 0x52, 0x93, 0x39, 0x3d, 0xda, 0x06, 0xc1, 0x05, + 0x49, 0xda, 0x43, 0x0a, 0x17, 0x88, 0x6c, 0xcb, 0x33, 0x3b, 0x08, 0x81, 0xe3, 0xb6, 0x6b, 0x52, + 0x80, 0x93, 0xee, 0x0b, 0xca, 0x87, 0x45, 0xad, 0xd8, 0x2a, 0x71, 0xf8, 0xac, 0x8e, 0xf9, 0xac, + 0xef, 0x87, 0xfd, 0x9e, 0x7e, 0x1f, 0xe0, 0x51, 0xb6, 0xc2, 0x32, 0x54, 0x4e, 0x2c, 0x87, 0x99, + 0xbd, 0x48, 0xc4, 0xa7, 0x60, 0x94, 0xf9, 0x70, 0x2b, 0x22, 0x2f, 0x03, 0x60, 0xe6, 0xda, 0x34, + 0x8a, 0x38, 0x4f, 0x15, 0xbc, 0x6a, 0x4c, 0xd9, 0x8a, 0xf4, 0xf7, 0x40, 0xdb, 0x45, 0x35, 0x22, + 0x9f, 0x31, 0x0f, 0x98, 0xcf, 0x2c, 0x37, 0x5e, 0x41, 0x0e, 0x78, 0xee, 0xc4, 0xe2, 0x98, 0x69, + 0xf9, 0xf9, 0xb4, 0xa3, 0x7f, 0xa5, 0x00, 0xec, 0xa6, 0x76, 0x92, 0xd7, 0xa1, 0x74, 0x82, 0xa8, + 0xd3, 0x89, 0x94, 0x4c, 0x94, 0x18, 0x92, 0x4f, 0x6e, 0x41, 0x51, 0xec, 0x19, 0xf5, 0x3c, 0x39, + 0xc1, 0xe6, 0x62, 0x1d, 0x8b, 0x59, 0x71, 0x8e, 0x9c, 0x25, 0xc6, 0xd9, 0xfa, 0x10, 0x6a, 0x0f, + 0x4e, 0xa9, 0x2d, 0x8d, 0x88, 0xc8, 0xdd, 0x3c, 0xde, 0x4a, 0x9c, 0x90, 0xc9, 0xe4, 0x0c, 0xb6, + 0x5c, 0x10, 0xee, 0xe6, 0x83, 0xa0, 0x8e, 0xcd, 0xca, 0xbc, 0x1c, 0x8d, 0x8c, 0xde, 0x01, 0x78, + 0x48, 0x99, 0x41, 0x9f, 0xf5, 0x69, 0xc4, 0xc8, 0x0a, 0x54, 0x6c, 0xb9, 0x67, 0x62, 0xad, 0xad, + 0x91, 0xe4, 0x14, 0x74, 0x23, 0x11, 0x48, 0x36, 0xbe, 0x9a, 0x6d, 0xfc, 0x36, 0x54, 0x92, 0x42, + 0x51, 0x90, 0xfb, 0x3e, 0x1e, 0xea, 0x5f, 0x42, 0x4d, 0x68, 0x89, 0x02, 0xdf, 0x8b, 0x28, 0x79, + 0x27, 0xdb, 0x72, 0xbc, 0x14, 0xc4, 0xba, 0x9a, 0xab, 0x49, 0x05, 0x13, 0x05, 0x22, 0xdd, 0x6d, + 0xa2, 0x5a, 0x60, 0x64, 0xa4, 0xec, 0x38, 0xe2, 0x49, 0x3d, 0x31, 0x24, 0x9f, 0x67, 0xc1, 0xc0, + 0x72, 0xfb, 0x34, 0xae, 0x48, 0x72, 0xa0, 0xff, 0xaa, 0x40, 0x8d, 0x23, 0x30, 0x8d, 0xa3, 0x58, + 0x3d, 0x22, 0x66, 0x85, 0xcc, 0xcc, 0xdc, 0xd5, 0x04, 0x01, 0x55, 0x73, 0x75, 0xae, 0xd3, 0x73, + 0x64, 0xf1, 0x69, 0x18, 0x72, 0x30, 0x8a, 0x44, 0x31, 0x87, 0x04, 0xdf, 0x66, 0xb8, 0x8c, 0xe9, + 0x7b, 0xee, 0x50, 0xd4, 0x05, 0xdc, 0x66, 0x38, 0xde, 0xc6, 0xa1, 0xfe, 0x14, 0xca, 0x8f, 0x07, + 0x3b, 0x96, 0x33, 0xe2, 0xac, 0xf2, 0x3f, 0xce, 0x4e, 0xc6, 0xe0, 0x6c, 0xf7, 0x8f, 0xa0, 0x2e, + 0xbd, 0x9f, 0x3e, 0x00, 0xb7, 0xa0, 0x14, 0xa0, 0x6d, 0x7c, 0x0f, 0x16, 0x50, 0x76, 0x21, 0xb3, + 0x49, 0xd8, 0x6c, 0x48, 0xae, 0xbe, 0x0d, 0xda, 0x56, 0x9f, 0x89, 0xea, 0x83, 0xc0, 0xa9, 0x7e, + 0x20, 0xd6, 0x6e, 0xae, 0xd5, 0x52, 0xf9, 0xed, 0xc0, 0x40, 0xf2, 0x85, 0x4d, 0xff, 0x5a, 0x85, + 0x85, 0x9d, 0x90, 0x8a, 0x6d, 0x37, 0x4d, 0xf4, 0x6e, 0x43, 0xb5, 0x17, 0x1b, 0x94, 0xd8, 0x9e, + 0xe1, 0x99, 0x98, 0x6a, 0x64, 0x32, 0x13, 0x6d, 0xb1, 0x30, 0xd9, 0x16, 0x5f, 0x85, 0x86, 0xcc, + 0x88, 0x7c, 0x90, 0xeb, 0x82, 0xb8, 0x9f, 0x45, 0x3a, 0x6d, 0x83, 0xa5, 0x5c, 0x1b, 0x24, 0x6b, + 0x70, 0x25, 0x3a, 0x76, 0x02, 0x13, 0x6d, 0x8c, 0x58, 0x68, 0x39, 0x1e, 0x33, 0xb1, 0x04, 0xa3, + 0xae, 0xb2, 0xc8, 0x88, 0x45, 0xce, 0x5c, 0x4f, 0x79, 0xeb, 0x9c, 0xa5, 0x07, 0xd0, 0xca, 0x60, + 0x98, 0x3e, 0x8c, 0xd8, 0x75, 0x05, 0x77, 0x12, 0x8b, 0x34, 0xb7, 0x62, 0x01, 0xfd, 0x37, 0x05, + 0x1a, 0xbc, 0x25, 0x39, 0x53, 0x95, 0x87, 0x09, 0x8c, 0xd4, 0x33, 0x30, 0xc2, 0x0e, 0x89, 0x91, + 0x8f, 0x10, 0xe3, 0x02, 0x62, 0x2c, 0xbe, 0x31, 0xd1, 0x9a, 0xb6, 0xd0, 0x3a, 0x86, 0x6e, 0x43, + 0x52, 0xe3, 0xa9, 0xd8, 0x52, 0x4a, 0xad, 0xb2, 0x51, 0x3e, 0x70, 0x3c, 0xec, 0x58, 0xba, 0x0b, + 0xcd, 0xc4, 0xd4, 0xe7, 0x5f, 0x63, 0xf4, 0x2e, 0x34, 0x36, 0x7a, 0x01, 0x9e, 0x32, 0x12, 0x60, + 0x72, 0x49, 0xa6, 0x5c, 0x20, 0xc9, 0x26, 0x9d, 0x54, 0xcf, 0x70, 0x12, 0x4b, 0x42, 0x33, 0x51, + 0x34, 0xbd, 0x5b, 0x4b, 0xa3, 0x6e, 0x55, 0x13, 0x1f, 0x3e, 0x87, 0xa5, 0x7b, 0x16, 0xc3, 0x43, + 0x8d, 0xef, 0xba, 0x07, 0x96, 0x7d, 0x3c, 0xcf, 0x18, 0xeb, 0x11, 0x5c, 0x19, 0x53, 0x3e, 0x87, + 0xa8, 0x45, 0x98, 0x23, 0x2e, 0xb5, 0xbc, 0x7e, 0x30, 0x9b, 0x76, 0x37, 0xe1, 0x7d, 0x61, 0xd2, + 0x7b, 0xfd, 0x07, 0x05, 0x16, 0x52, 0xad, 0x73, 0x68, 0x7f, 0x93, 0x89, 0x55, 0x38, 0x2b, 0xb1, + 0x8e, 0x61, 0x41, 0x04, 0x60, 0xca, 0xde, 0x9f, 0xc4, 0x54, 0x1d, 0xd9, 0xb7, 0xe7, 0x77, 0x7f, + 0x17, 0x5a, 0x99, 0xb2, 0xe7, 0xde, 0x81, 0xbe, 0x43, 0xc4, 0x79, 0xb3, 0xe3, 0x95, 0x7a, 0x1a, + 0xdf, 0xf0, 0x30, 0xdb, 0xb3, 0x4e, 0xc7, 0x52, 0x1a, 0x90, 0x94, 0x24, 0x74, 0xee, 0x3c, 0x50, + 0x38, 0xef, 0x3c, 0x50, 0x1c, 0x39, 0x0f, 0xe8, 0x3f, 0x2a, 0xd0, 0xca, 0x6c, 0x9a, 0x43, 0x1a, + 0xa0, 0x20, 0x6f, 0x36, 0x72, 0xd7, 0x9d, 0x79, 0xb7, 0x92, 0x7c, 0xfd, 0x0e, 0x54, 0xf6, 0x4e, + 0xe5, 0xf9, 0x19, 0x33, 0x9c, 0x9d, 0x7a, 0xf1, 0xfd, 0x88, 0x7f, 0x92, 0xab, 0x50, 0x46, 0xc7, + 0x58, 0x3f, 0x8a, 0x51, 0x88, 0x47, 0xfa, 0xef, 0x0a, 0x10, 0x74, 0xc3, 0x77, 0x07, 0x74, 0x5a, + 0x94, 0x2f, 0x54, 0x3a, 0x2e, 0x96, 0xcc, 0xe4, 0x6d, 0xa8, 0xa2, 0xb5, 0xa6, 0x83, 0x4e, 0x44, + 0x88, 0x7b, 0x21, 0xa7, 0x39, 0xf6, 0xce, 0xd0, 0x98, 0xfc, 0x88, 0xf4, 0x67, 0xb0, 0x98, 0x33, + 0x7e, 0x0e, 0xa5, 0x67, 0x1f, 0xaa, 0x0f, 0xd7, 0xa7, 0x81, 0x09, 0xaf, 0x3f, 0x91, 0x75, 0x48, + 0xcd, 0xc0, 0xc7, 0x83, 0x40, 0x8c, 0x51, 0x95, 0x53, 0x76, 0x38, 0x01, 0xcf, 0x75, 0xc0, 0xd7, + 0x9d, 0x83, 0x07, 0x5b, 0xd0, 0x30, 0xac, 0x93, 0x59, 0x5d, 0x15, 0x74, 0x1f, 0x9a, 0xc9, 0x72, + 0x33, 0x6e, 0x6c, 0xe7, 0x1c, 0x23, 0x6d, 0x61, 0xff, 0x4e, 0x7f, 0x46, 0x57, 0x9d, 0xb3, 0x95, + 0x3c, 0x15, 0x5e, 0x09, 0x25, 0xb3, 0x6e, 0xd7, 0x3b, 0xd0, 0xc2, 0xa5, 0xef, 0x53, 0x97, 0x4e, + 0x77, 0x0c, 0x9e, 0x0c, 0xc1, 0xc7, 0x70, 0x79, 0x64, 0xc5, 0x59, 0xdb, 0x3b, 0x00, 0x12, 0x2f, + 0x6d, 0x79, 0x5d, 0x3a, 0xf3, 0x6b, 0xd7, 0x32, 0x54, 0xa8, 0xd7, 0x19, 0xa9, 0xc0, 0x65, 0x1c, + 0x22, 0x43, 0xff, 0x14, 0x16, 0x73, 0x7a, 0x67, 0xed, 0xd7, 0x10, 0xae, 0x64, 0xa8, 0xcd, 0xd7, + 0x35, 0x0b, 0xae, 0x8e, 0xab, 0x9e, 0xb5, 0x77, 0x72, 0x5b, 0xce, 0xef, 0xa2, 0x8c, 0x27, 0xe9, + 0x85, 0x54, 0xe1, 0xf4, 0xce, 0xdc, 0xc4, 0xe4, 0x1e, 0x9c, 0x7b, 0x2e, 0xe0, 0x3c, 0xbd, 0x03, + 0xd5, 0x27, 0xfc, 0xee, 0x24, 0x3a, 0x1d, 0x7f, 0xeb, 0x12, 0x86, 0xb2, 0x28, 0x6e, 0x77, 0x15, + 0x31, 0xde, 0x8b, 0xb0, 0xfb, 0x17, 0xd9, 0x30, 0x90, 0xaf, 0x8c, 0x63, 0xb7, 0x56, 0xc1, 0xe0, + 0x4e, 0xc6, 0x3d, 0x89, 0x45, 0x71, 0x3b, 0xd2, 0x24, 0x61, 0x2f, 0xd2, 0x9f, 0x40, 0x75, 0x9f, + 0x57, 0x82, 0xe4, 0x3d, 0x4a, 0xd6, 0x08, 0x65, 0xa4, 0x46, 0x90, 0x26, 0xa8, 0x2c, 0xe9, 0xa7, + 0xf8, 0xc5, 0x5f, 0xe7, 0x9c, 0xc8, 0x8c, 0x8e, 0x7c, 0xde, 0x0b, 0xd3, 0x92, 0xa2, 0x19, 0x75, + 0x27, 0xda, 0xe5, 0x44, 0xb1, 0x9e, 0xfe, 0xad, 0x82, 0xf7, 0xea, 0x81, 0x6d, 0x8b, 0x85, 0x93, + 0xc7, 0xa7, 0x73, 0xdf, 0x4d, 0xe5, 0xe3, 0xd3, 0x0a, 0x94, 0xc5, 0x75, 0x31, 0x01, 0x86, 0xa4, + 0x82, 0x29, 0x12, 0x46, 0x2c, 0xc1, 0x65, 0x85, 0xf2, 0xe4, 0xc0, 0x90, 0xc9, 0xa6, 0xfe, 0x18, + 0xb1, 0x84, 0xbe, 0x0b, 0x8b, 0xdc, 0x14, 0x2c, 0xde, 0xf7, 0x86, 0x18, 0xd9, 0xd9, 0x54, 0xa3, + 0x6f, 0x14, 0x58, 0xca, 0xaf, 0x3a, 0xeb, 0xbe, 0x80, 0xa8, 0xf1, 0x13, 0xc2, 0xc4, 0x5b, 0x5c, + 0x02, 0xab, 0x21, 0xd8, 0xfa, 0x67, 0xb0, 0x9c, 0xda, 0xb1, 0x2b, 0x93, 0x62, 0x1a, 0x0f, 0x47, + 0x53, 0x4c, 0xcd, 0xa5, 0x98, 0xfe, 0xb3, 0x02, 0xed, 0x49, 0x15, 0xb3, 0x76, 0x77, 0xf2, 0x5d, + 0x3e, 0x01, 0xa0, 0xf8, 0xdf, 0x00, 0x7c, 0x02, 0x64, 0x37, 0x70, 0xf9, 0x4d, 0x9a, 0xab, 0x98, + 0xb6, 0x0e, 0xf0, 0x15, 0x72, 0x75, 0x80, 0x13, 0x78, 0x15, 0xfb, 0x5e, 0x81, 0xc5, 0xdc, 0xfa, + 0xd3, 0x3b, 0xae, 0xe3, 0x3e, 0xa0, 0x87, 0x2c, 0x3e, 0xbb, 0x34, 0xf3, 0x2f, 0xf6, 0x86, 0xe0, + 0xe1, 0xf6, 0x2a, 0x85, 0x4e, 0xf7, 0x88, 0xc5, 0x61, 0x1f, 0x17, 0x92, 0xcc, 0x95, 0xb7, 0x00, + 0xb2, 0xc7, 0x77, 0x02, 0x50, 0xfe, 0xc8, 0x0f, 0x7b, 0x96, 0xdb, 0xba, 0x44, 0x2a, 0x50, 0xd8, + 0xf4, 0x4f, 0x5a, 0x0a, 0xd1, 0xa0, 0xf8, 0x08, 0x65, 0x5b, 0xea, 0xca, 0x0d, 0xbc, 0x94, 0xe7, + 0x5f, 0xd8, 0xcb, 0xa0, 0xee, 0x6e, 0xa0, 0x30, 0xfe, 0x1a, 0xeb, 0x2d, 0x65, 0x65, 0x15, 0xd4, + 0xed, 0x80, 0x4f, 0xc5, 0xa3, 0x80, 0x5c, 0x03, 0xab, 0xb6, 0x5c, 0x83, 0xef, 0xd1, 0x96, 0x4a, + 0xea, 0xa0, 0x25, 0x77, 0xdf, 0x56, 0xe1, 0xde, 0xca, 0x1f, 0x7f, 0x5f, 0x57, 0xfe, 0xc4, 0xbf, + 0xbf, 0xf0, 0xef, 0xa7, 0x7f, 0xae, 0x5f, 0x82, 0x36, 0x96, 0x94, 0xd5, 0xc0, 0xf1, 0xba, 0xb6, + 0x15, 0xac, 0x32, 0xe7, 0x78, 0x80, 0x68, 0x8b, 0xff, 0xe0, 0x1c, 0x94, 0xc5, 0xcf, 0x9d, 0x7f, + 0x03, 0x00, 0x00, 0xff, 0xff, 0x08, 0xac, 0x24, 0x9c, 0x15, 0x1a, 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..72d3eb96eaec3 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) @@ -1420,34 +1439,33 @@ var ( func init() { proto.RegisterFile("metapb.proto", fileDescriptorMetapb) } var fileDescriptorMetapb = []byte{ - // 456 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x52, 0x4d, 0x8e, 0xd3, 0x4c, - 0x10, 0x9d, 0x4e, 0xfc, 0x93, 0x94, 0x3d, 0x51, 0xd4, 0xdf, 0x48, 0x9f, 0x07, 0xa4, 0xc8, 0xb2, - 0x58, 0x58, 0x59, 0x18, 0x18, 0x10, 0x5b, 0xa4, 0x19, 0xb1, 0x40, 0x20, 0x81, 0x7a, 0x80, 0xad, - 0xe5, 0xd8, 0x95, 0x60, 0xc5, 0x76, 0x5b, 0xdd, 0x1d, 0x6b, 0xe6, 0x04, 0x5c, 0x81, 0x33, 0x70, - 0x12, 0x96, 0x1c, 0x01, 0x85, 0x8b, 0xa0, 0x6e, 0xdb, 0xd2, 0x88, 0xac, 0xd2, 0xaf, 0x5e, 0xbd, - 0xca, 0x7b, 0x55, 0x06, 0xbf, 0x46, 0x95, 0xb5, 0x9b, 0xa4, 0x15, 0x5c, 0x71, 0xea, 0xf4, 0xe8, - 0xd1, 0xc5, 0x8e, 0xef, 0xb8, 0x29, 0x3d, 0xd5, 0xaf, 0x9e, 0x8d, 0x5e, 0x83, 0x7b, 0x53, 0x1d, - 0xa4, 0x42, 0x41, 0x17, 0x30, 0x29, 0x8b, 0x80, 0x84, 0x24, 0xb6, 0xd8, 0xa4, 0x2c, 0xe8, 0x13, - 0x58, 0xd4, 0xd9, 0x5d, 0xda, 0x22, 0x8a, 0x34, 0xe7, 0x87, 0x46, 0x05, 0x93, 0x90, 0xc4, 0xe7, - 0xcc, 0xaf, 0xb3, 0xbb, 0x8f, 0x88, 0xe2, 0x46, 0xd7, 0xa2, 0x97, 0x00, 0xb7, 0x8a, 0x0b, 0x7c, - 0x9f, 0x6d, 0xb0, 0xa2, 0x4b, 0x98, 0xee, 0xf1, 0xde, 0x0c, 0x99, 0x33, 0xfd, 0xa4, 0x17, 0x60, - 0x77, 0x59, 0x75, 0x40, 0x23, 0x9e, 0xb3, 0x1e, 0x44, 0xdf, 0x08, 0xd8, 0x46, 0x76, 0xf2, 0xaf, - 0x01, 0xb8, 0x59, 0x51, 0x08, 0x94, 0x72, 0x50, 0x8c, 0x90, 0xc6, 0x60, 0x4b, 0x95, 0x29, 0x0c, - 0xa6, 0x21, 0x89, 0x17, 0x57, 0x34, 0x19, 0x62, 0x9a, 0x39, 0xb7, 0x9a, 0x61, 0x7d, 0x03, 0x5d, - 0x83, 0x53, 0x69, 0x3b, 0x32, 0xb0, 0xc2, 0x69, 0xec, 0xfd, 0xd3, 0x6a, 0x9c, 0xb2, 0xa1, 0x23, - 0xba, 0x06, 0x8f, 0xe1, 0xae, 0xe4, 0xcd, 0x9b, 0x96, 0xe7, 0x5f, 0xe9, 0x25, 0xcc, 0x72, 0xde, - 0x6c, 0xd3, 0x0e, 0xc5, 0x60, 0xca, 0xd5, 0xf8, 0x0b, 0x0a, 0xed, 0xac, 0x43, 0x21, 0x4b, 0xde, - 0x18, 0x67, 0x16, 0x1b, 0x61, 0xf4, 0x83, 0x80, 0xd3, 0x0f, 0x39, 0x89, 0xf3, 0x18, 0xe6, 0x52, - 0x65, 0x42, 0xa5, 0x7a, 0x2d, 0x5a, 0xe6, 0xb3, 0x99, 0x29, 0xbc, 0xc3, 0x7b, 0xfa, 0x3f, 0xb8, - 0xd8, 0x14, 0x86, 0x9a, 0x1a, 0xca, 0xc1, 0xa6, 0xd0, 0xc4, 0x2b, 0xf0, 0x85, 0x99, 0x97, 0xa2, - 0x76, 0x15, 0x58, 0x21, 0x89, 0xbd, 0xab, 0xff, 0xc6, 0x18, 0x0f, 0x0c, 0x33, 0x4f, 0x3c, 0x70, - 0x1f, 0x81, 0xad, 0xcf, 0x25, 0x03, 0xdb, 0xe4, 0xf6, 0x47, 0x81, 0x3e, 0x17, 0xeb, 0xa9, 0xe8, - 0x39, 0x58, 0x1a, 0x9e, 0x38, 0xbd, 0x84, 0x99, 0xd4, 0xeb, 0x49, 0xcb, 0x62, 0xcc, 0x67, 0xf0, - 0xdb, 0x62, 0xfd, 0x6c, 0xb8, 0xb1, 0x59, 0x32, 0x75, 0x60, 0xf2, 0xb9, 0x5d, 0x9e, 0x51, 0x0f, - 0xdc, 0x0f, 0xdb, 0x6d, 0x55, 0x36, 0xb8, 0x24, 0xf4, 0x1c, 0xe6, 0x9f, 0x78, 0xbd, 0x91, 0x8a, - 0x37, 0xb8, 0x9c, 0x5c, 0xaf, 0x7f, 0x1e, 0x57, 0xe4, 0xd7, 0x71, 0x45, 0x7e, 0x1f, 0x57, 0xe4, - 0xfb, 0x9f, 0xd5, 0x19, 0x04, 0x39, 0xaf, 0x93, 0xb6, 0x6c, 0x76, 0x79, 0xd6, 0x26, 0xaa, 0xdc, - 0x77, 0xc9, 0xbe, 0x33, 0x9f, 0xe0, 0xc6, 0x31, 0x3f, 0x2f, 0xfe, 0x06, 0x00, 0x00, 0xff, 0xff, - 0x50, 0x3f, 0x60, 0xb0, 0xb7, 0x02, 0x00, 0x00, + // 445 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x64, 0x52, 0xdd, 0x8a, 0xd3, 0x40, + 0x14, 0xde, 0xb4, 0x4d, 0xb2, 0x3d, 0xcd, 0x96, 0x32, 0x2e, 0x18, 0x15, 0x96, 0x25, 0x78, 0x51, + 0x7a, 0x11, 0x7f, 0xf1, 0x56, 0xd8, 0xc5, 0x0b, 0x51, 0x50, 0x66, 0xd5, 0xdb, 0x90, 0x9f, 0x69, + 0x0c, 0x4d, 0x32, 0x61, 0x66, 0x1a, 0xf4, 0x09, 0x7c, 0x05, 0x9f, 0xc1, 0x27, 0xf1, 0xd2, 0x47, + 0x10, 0x7d, 0x11, 0xcf, 0x9c, 0xa4, 0xb0, 0xd8, 0x8b, 0xa1, 0xf3, 0x7d, 0xdf, 0x39, 0xa7, 0xdf, + 0x77, 0x26, 0x10, 0x34, 0xc2, 0xa4, 0x5d, 0x16, 0x77, 0x4a, 0x1a, 0xc9, 0xbc, 0x01, 0xdd, 0x3f, + 0x2f, 0x65, 0x29, 0x89, 0x7a, 0x64, 0x6f, 0x83, 0x1a, 0xbd, 0x04, 0xff, 0xba, 0xde, 0x6b, 0x23, + 0x14, 0x5b, 0xc2, 0xa4, 0x2a, 0x42, 0xe7, 0xd2, 0x59, 0xcf, 0x38, 0xde, 0xd8, 0x43, 0x58, 0x36, + 0xe9, 0x97, 0xa4, 0x13, 0x42, 0x25, 0xb9, 0xdc, 0xb7, 0x26, 0x9c, 0xa0, 0x76, 0xc6, 0x03, 0x64, + 0xdf, 0x23, 0x79, 0x6d, 0xb9, 0xe8, 0x39, 0xc0, 0x8d, 0x91, 0x4a, 0xbc, 0x4d, 0x33, 0x51, 0xb3, + 0x15, 0x4c, 0x77, 0xe2, 0x2b, 0x0d, 0x99, 0x73, 0x7b, 0x65, 0xe7, 0xe0, 0xf6, 0x69, 0xbd, 0x17, + 0xd4, 0x3c, 0xe7, 0x03, 0x88, 0xbe, 0x39, 0xe0, 0x52, 0xdb, 0xd1, 0xbf, 0x86, 0xe0, 0xa7, 0x45, + 0xa1, 0x84, 0xd6, 0x63, 0xc7, 0x01, 0xb2, 0x35, 0xb8, 0xda, 0xa4, 0x46, 0x84, 0x53, 0xe4, 0x97, + 0x4f, 0x59, 0x3c, 0xc6, 0xa4, 0x39, 0x37, 0x56, 0xe1, 0x43, 0x01, 0xdb, 0x80, 0x57, 0x5b, 0x3b, + 0x3a, 0x9c, 0x5d, 0x4e, 0xd7, 0x8b, 0xff, 0x4a, 0xc9, 0x29, 0x1f, 0x2b, 0xa2, 0x2b, 0x58, 0x70, + 0x51, 0x56, 0xb2, 0x7d, 0xd5, 0xc9, 0xfc, 0x33, 0xbb, 0x07, 0xa7, 0xb9, 0x6c, 0xb7, 0x49, 0x2f, + 0xd4, 0x68, 0xca, 0xb7, 0xf8, 0x13, 0xee, 0x07, 0x9d, 0x21, 0xab, 0xb1, 0x94, 0x9c, 0xa1, 0x32, + 0xc2, 0xe8, 0x87, 0x03, 0xde, 0x30, 0xe4, 0x28, 0xce, 0x03, 0x98, 0xa3, 0x27, 0x65, 0x12, 0xbb, + 0x16, 0xdb, 0x16, 0xf0, 0x53, 0x22, 0xde, 0xe0, 0x6e, 0xee, 0x82, 0x2f, 0xda, 0x82, 0xa4, 0x29, + 0x49, 0x1e, 0x42, 0x2b, 0xbc, 0x80, 0x40, 0xd1, 0xbc, 0x44, 0x58, 0x57, 0x18, 0xc3, 0xc1, 0x18, + 0x77, 0x0e, 0x31, 0x6e, 0x19, 0xe6, 0x0b, 0x75, 0xcb, 0x7d, 0x04, 0xae, 0x7d, 0x2e, 0x1d, 0xba, + 0x94, 0x3b, 0x38, 0x34, 0xd8, 0xe7, 0xe2, 0x83, 0x14, 0x3d, 0x81, 0x99, 0x85, 0x47, 0x4e, 0x31, + 0xb9, 0xb6, 0xeb, 0x49, 0x90, 0x1d, 0xf3, 0x11, 0x7e, 0x5d, 0x6c, 0x1e, 0x8f, 0x6f, 0x4c, 0x4b, + 0x66, 0x1e, 0x4c, 0x3e, 0x76, 0xab, 0x13, 0xb6, 0x00, 0xff, 0xdd, 0x76, 0x5b, 0x57, 0xad, 0x58, + 0x39, 0xec, 0x0c, 0xe6, 0x1f, 0x64, 0x93, 0x61, 0x07, 0xc2, 0xc9, 0xd5, 0xe6, 0xe7, 0x9f, 0x0b, + 0xe7, 0x17, 0x9e, 0xdf, 0x78, 0xbe, 0xff, 0xbd, 0x38, 0x81, 0x30, 0x97, 0x4d, 0xdc, 0x55, 0x6d, + 0x99, 0xa7, 0x5d, 0x6c, 0xaa, 0x5d, 0x1f, 0xef, 0x7a, 0xfa, 0x04, 0x33, 0x8f, 0x7e, 0x9e, 0xfd, + 0x0b, 0x00, 0x00, 0xff, 0xff, 0x50, 0x3f, 0x60, 0xb0, 0xb7, 0x02, 0x00, 0x00, } 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..5fc41ce00c046 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 @@ -46,6 +49,8 @@ StoreStats StoreHeartbeatRequest StoreHeartbeatResponse + ScatterRegionRequest + ScatterRegionResponse */ package pdpb @@ -468,6 +473,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 +521,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 +546,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 +577,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 +600,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 +617,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 +641,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 +664,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 +677,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 +716,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 +730,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 +740,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 +772,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 +787,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 +821,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 +900,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 +915,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 +938,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 { @@ -906,7 +976,7 @@ type RegionHeartbeatResponse struct { 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{32} } func (m *RegionHeartbeatResponse) GetHeader() *ResponseHeader { if m != nil { @@ -958,7 +1028,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{33} } func (m *AskSplitRequest) GetHeader() *RequestHeader { if m != nil { @@ -987,7 +1057,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{34} } func (m *AskSplitResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1019,7 +1089,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{35} } func (m *ReportSplitRequest) GetHeader() *RequestHeader { if m != nil { @@ -1049,7 +1119,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{36} } func (m *ReportSplitResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1091,7 +1161,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{37} } func (m *StoreStats) GetStoreId() uint64 { if m != nil { @@ -1199,7 +1269,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{38} } func (m *StoreHeartbeatRequest) GetHeader() *RequestHeader { if m != nil { @@ -1222,7 +1292,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{39} } func (m *StoreHeartbeatResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1231,6 +1301,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{40} } + +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{41} } + +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 +1376,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") @@ -1270,6 +1400,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 +1425,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 +1434,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 +1530,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 +1633,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 +1654,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 +1663,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 +1804,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 +1974,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 +2020,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 +2052,10 @@ var _PD_serviceDesc = grpc.ServiceDesc{ MethodName: "PutClusterConfig", Handler: _PD_PutClusterConfig_Handler, }, + { + MethodName: "ScatterRegion", + Handler: _PD_ScatterRegion_Handler, + }, }, Streams: []grpc.StreamDesc{ { @@ -2405,7 +2603,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 +2613,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 +2628,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 +2724,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 +2772,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 +2805,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 +2833,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 +2871,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 +2909,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 +2974,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 +3001,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 +3029,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 +3051,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 +3089,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 +3122,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 +3197,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 +3224,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,11 +3257,11 @@ 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 } @@ -2999,31 +3285,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:]) + n42, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n39 + i += n42 } if m.ChangePeer != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.ChangePeer.Size())) - n40, err := m.ChangePeer.MarshalTo(dAtA[i:]) + n43, err := m.ChangePeer.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n40 + i += n43 } if m.TransferLeader != nil { dAtA[i] = 0x1a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.TransferLeader.Size())) - n41, err := m.TransferLeader.MarshalTo(dAtA[i:]) + n44, err := m.TransferLeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n41 + i += n44 } if m.RegionId != 0 { dAtA[i] = 0x20 @@ -3034,21 +3320,21 @@ 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:]) + n45, err := m.RegionEpoch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n42 + i += n45 } if m.TargetPeer != nil { dAtA[i] = 0x32 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.TargetPeer.Size())) - n43, err := m.TargetPeer.MarshalTo(dAtA[i:]) + n46, err := m.TargetPeer.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n43 + i += n46 } return i, nil } @@ -3072,21 +3358,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:]) + n47, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n44 + i += n47 } if m.Region != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Region.Size())) - n45, err := m.Region.MarshalTo(dAtA[i:]) + n48, err := m.Region.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n45 + i += n48 } return i, nil } @@ -3110,11 +3396,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:]) + n49, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n46 + i += n49 } if m.NewRegionId != 0 { dAtA[i] = 0x10 @@ -3122,21 +3408,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 + dAtA51 := make([]byte, len(m.NewPeerIds)*10) + var j50 int for _, num := range m.NewPeerIds { for num >= 1<<7 { - dAtA48[j47] = uint8(uint64(num)&0x7f | 0x80) + dAtA51[j50] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j47++ + j50++ } - dAtA48[j47] = uint8(num) - j47++ + dAtA51[j50] = uint8(num) + j50++ } dAtA[i] = 0x1a i++ - i = encodeVarintPdpb(dAtA, i, uint64(j47)) - i += copy(dAtA[i:], dAtA48[:j47]) + i = encodeVarintPdpb(dAtA, i, uint64(j50)) + i += copy(dAtA[i:], dAtA51[:j50]) } return i, nil } @@ -3160,31 +3446,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:]) + n52, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n49 + i += n52 } if m.Left != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Left.Size())) - n50, err := m.Left.MarshalTo(dAtA[i:]) + n53, err := m.Left.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n50 + i += n53 } if m.Right != nil { dAtA[i] = 0x1a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Right.Size())) - n51, err := m.Right.MarshalTo(dAtA[i:]) + n54, err := m.Right.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n51 + i += n54 } return i, nil } @@ -3208,11 +3494,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:]) + n55, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n52 + i += n55 } return i, nil } @@ -3329,21 +3615,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:]) + n56, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n53 + i += n56 } if m.Stats != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Stats.Size())) - n54, err := m.Stats.MarshalTo(dAtA[i:]) + n57, err := m.Stats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n54 + i += n57 } return i, nil } @@ -3367,15 +3653,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:]) + n58, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n55 + i += n58 + } + 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())) + n59, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n59 + } + 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())) + n60, err := m.Region.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n60 + } + if m.Leader != nil { + dAtA[i] = 0x22 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Leader.Size())) + n61, err := m.Leader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n61 + } + 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())) + n62, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n62 } 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 +3972,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 +4113,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 +4146,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 +4208,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 } @@ -3979,6 +4400,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 +6058,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 +6081,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,37 +6120,6 @@ 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) - } - 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 { - return ErrInvalidLengthPdpb - } - postIndex := iNdEx + byteLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.RegionKey = append(m.RegionKey[:0], dAtA[iNdEx:postIndex]...) - if m.RegionKey == nil { - m.RegionKey = []byte{} - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) @@ -5720,7 +6141,7 @@ func (m *GetRegionRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *GetRegionResponse) Unmarshal(dAtA []byte) error { +func (m *GetAllStoresResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5743,10 +6164,10 @@ func (m *GetRegionResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetRegionResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetAllStoresResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetRegionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetAllStoresResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -5784,40 +6205,7 @@ func (m *GetRegionResponse) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: 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 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Leader", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Stores", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -5841,10 +6229,8 @@ func (m *GetRegionResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Leader == nil { - m.Leader = &metapb.Peer{} - } - if err := m.Leader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + 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 @@ -5869,7 +6255,7 @@ func (m *GetRegionResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *GetRegionByIDRequest) Unmarshal(dAtA []byte) error { +func (m *GetRegionRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5892,10 +6278,10 @@ func (m *GetRegionByIDRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetRegionByIDRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetRegionRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetRegionByIDRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetRegionRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -5932,10 +6318,10 @@ func (m *GetRegionByIDRequest) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 2: - 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 RegionKey", wireType) } - m.RegionId = 0 + var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -5945,11 +6331,23 @@ func (m *GetRegionByIDRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RegionId |= (uint64(b) & 0x7F) << shift + byteLen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if byteLen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RegionKey = append(m.RegionKey[:0], dAtA[iNdEx:postIndex]...) + if m.RegionKey == nil { + m.RegionKey = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) @@ -5971,7 +6369,7 @@ func (m *GetRegionByIDRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *GetClusterConfigRequest) Unmarshal(dAtA []byte) error { +func (m *GetRegionResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -5994,10 +6392,261 @@ func (m *GetClusterConfigRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: GetClusterConfigRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetRegionResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: GetClusterConfigRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetRegionResponse: 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 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 3: + 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 *GetRegionByIDRequest) 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: GetRegionByIDRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetRegionByIDRequest: 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 + } + } + 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 *GetClusterConfigRequest) 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: GetClusterConfigRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GetClusterConfigRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -6504,6 +7153,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 +7402,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,6 +7843,25 @@ func (m *RegionHeartbeatRequest) Unmarshal(dAtA []byte) error { break } } + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + 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:]) @@ -7780,24 +8500,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 { @@ -7836,8 +8539,25 @@ func (m *AskSplitResponse) Unmarshal(dAtA []byte) error { break } } - m.NewPeerIds = append(m.NewPeerIds, v) + 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 +9330,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 +9689,123 @@ 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, + // 1875 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x59, 0x4b, 0x73, 0x23, 0x49, + 0x11, 0x1e, 0xbd, 0xa5, 0xd4, 0x73, 0xca, 0x1a, 0x4b, 0xd3, 0xf3, 0x60, 0xe8, 0xe1, 0xb1, 0xbb, + 0x2c, 0x62, 0xd7, 0x44, 0x10, 0x1b, 0x41, 0x2c, 0x81, 0x5f, 0xb3, 0x6b, 0x66, 0xc7, 0x72, 0x94, + 0xb4, 0xb1, 0xb1, 0x17, 0x9a, 0x96, 0x54, 0x96, 0x1b, 0x4b, 0xea, 0x9e, 0xee, 0xb2, 0x07, 0x71, + 0xe2, 0xc4, 0x05, 0x22, 0xb8, 0x72, 0xe4, 0x4e, 0x04, 0xff, 0x80, 0xe0, 0xc0, 0x85, 0x23, 0x3f, + 0x81, 0x80, 0x33, 0xff, 0x81, 0x7a, 0xf6, 0x4b, 0xb2, 0x31, 0x3d, 0xec, 0x41, 0xe1, 0xae, 0xcc, + 0xac, 0xac, 0xcc, 0xac, 0xcc, 0xaf, 0xb2, 0xca, 0x00, 0xde, 0xcc, 0x9b, 0x0c, 0x3c, 0xdf, 0xa5, + 0x2e, 0x2a, 0xf2, 0x6f, 0xa3, 0xb1, 0x24, 0xd4, 0xd6, 0x34, 0xa3, 0x49, 0x7c, 0xfb, 0x9c, 0x86, + 0xc3, 0xee, 0xdc, 0x9d, 0xbb, 0xe2, 0xf3, 0x7b, 0xfc, 0x4b, 0x52, 0xcd, 0x01, 0x34, 0x31, 0x79, + 0x7d, 0x45, 0x02, 0xfa, 0x29, 0xb1, 0x67, 0xc4, 0x47, 0x4f, 0x00, 0xa6, 0x8b, 0xab, 0x80, 0x12, + 0xdf, 0x72, 0x66, 0xfd, 0xdc, 0xb3, 0xdc, 0x3b, 0x45, 0x5c, 0x53, 0x94, 0x93, 0x99, 0x89, 0xa1, + 0x85, 0x49, 0xe0, 0xb9, 0xab, 0x80, 0xdc, 0x69, 0x02, 0xfa, 0x3a, 0x94, 0x88, 0xef, 0xbb, 0x7e, + 0x3f, 0xcf, 0x38, 0xf5, 0xbd, 0xfa, 0x40, 0x58, 0x7d, 0xcc, 0x49, 0x58, 0x72, 0xcc, 0x17, 0x50, + 0x12, 0x63, 0xf4, 0x1c, 0x8a, 0x74, 0xed, 0x11, 0xa1, 0xa4, 0xb5, 0xd7, 0x8e, 0x89, 0x8e, 0x19, + 0x19, 0x0b, 0x26, 0xea, 0x43, 0x65, 0x49, 0x82, 0xc0, 0x9e, 0x13, 0xa1, 0xb2, 0x86, 0xf5, 0xd0, + 0x1c, 0x02, 0x8c, 0x03, 0x57, 0xb9, 0x83, 0xbe, 0x03, 0xe5, 0x0b, 0x61, 0xa1, 0x50, 0x57, 0xdf, + 0xdb, 0x91, 0xea, 0x12, 0xde, 0x62, 0x25, 0x82, 0xba, 0x50, 0x9a, 0xba, 0x57, 0x2b, 0x2a, 0x54, + 0x36, 0xb1, 0x1c, 0x98, 0xfb, 0x50, 0x1b, 0x3b, 0x4c, 0x3b, 0xb5, 0x97, 0x1e, 0x32, 0xa0, 0xea, + 0x5d, 0xac, 0x03, 0x67, 0x6a, 0x2f, 0x84, 0xc6, 0x02, 0x0e, 0xc7, 0xdc, 0xa6, 0x85, 0x3b, 0x17, + 0xac, 0xbc, 0x60, 0xe9, 0xa1, 0xf9, 0xab, 0x1c, 0xd4, 0x85, 0x51, 0x32, 0x66, 0xe8, 0xfd, 0x94, + 0x55, 0x5d, 0x6d, 0x55, 0x3c, 0xa6, 0xb7, 0x9b, 0x85, 0xbe, 0x0b, 0x35, 0xaa, 0xcd, 0xea, 0x17, + 0x84, 0x1a, 0x15, 0xab, 0xd0, 0x5a, 0x1c, 0x49, 0x98, 0xbf, 0xcd, 0x41, 0xe7, 0xc0, 0x75, 0x69, + 0x40, 0x7d, 0xdb, 0xcb, 0x14, 0x9d, 0xe7, 0x50, 0x0a, 0xa8, 0xeb, 0x13, 0xb5, 0x87, 0xcd, 0x81, + 0xca, 0xb3, 0x11, 0x27, 0x62, 0xc9, 0x43, 0xdf, 0x82, 0xb2, 0x4f, 0xe6, 0x8e, 0xbb, 0x52, 0x26, + 0xb5, 0xb4, 0x14, 0x16, 0x54, 0xac, 0xb8, 0x2c, 0xa8, 0xf7, 0x63, 0xd6, 0x64, 0x09, 0x8b, 0x79, + 0x04, 0x0f, 0x4e, 0x82, 0x50, 0x89, 0x47, 0x66, 0x59, 0xbc, 0x32, 0x7f, 0x0e, 0xbb, 0x69, 0x2d, + 0x99, 0x36, 0xc9, 0x84, 0xc6, 0x24, 0xa6, 0x45, 0x04, 0xa9, 0x8a, 0x13, 0x34, 0xf3, 0x63, 0x68, + 0xed, 0x2f, 0x16, 0xee, 0xf4, 0xe4, 0x28, 0x93, 0xa9, 0x43, 0x68, 0x87, 0xd3, 0x33, 0xd9, 0xd8, + 0x82, 0xbc, 0x23, 0x2d, 0x2b, 0x62, 0xf6, 0x65, 0x7e, 0x09, 0xed, 0x4f, 0x08, 0x95, 0xfb, 0x97, + 0x25, 0x23, 0x1e, 0x42, 0x55, 0xec, 0xba, 0x15, 0x6a, 0xad, 0x88, 0x31, 0x43, 0x08, 0x02, 0x9d, + 0x48, 0x75, 0x26, 0x63, 0xef, 0x92, 0x6e, 0xe6, 0x14, 0xda, 0x67, 0x57, 0x6f, 0xe1, 0xc1, 0x9d, + 0x16, 0xf9, 0x31, 0x74, 0xa2, 0x45, 0x32, 0xa5, 0xea, 0x01, 0xec, 0xb0, 0x68, 0xb0, 0xcd, 0x13, + 0x4a, 0x82, 0x4c, 0xbb, 0x7f, 0x09, 0xdd, 0xa4, 0x8e, 0x4c, 0x51, 0xfd, 0x26, 0x94, 0x85, 0x53, + 0x01, 0xf3, 0xb8, 0xb0, 0xe9, 0xb1, 0x62, 0x9a, 0x3f, 0x15, 0xdb, 0xa7, 0x6a, 0x36, 0x4b, 0x60, + 0xd9, 0x79, 0x20, 0x2b, 0xdd, 0xba, 0x24, 0x6b, 0x11, 0xdd, 0x06, 0xae, 0x49, 0xca, 0x4b, 0xb2, + 0x36, 0x7f, 0x97, 0x83, 0xfb, 0xb1, 0x05, 0x32, 0xb9, 0x12, 0x41, 0x4d, 0xfe, 0x36, 0xa8, 0x41, + 0xdf, 0x80, 0xf2, 0x42, 0x6a, 0x95, 0x90, 0xd4, 0xd0, 0x72, 0x67, 0x84, 0x6b, 0x93, 0x3c, 0xf3, + 0x67, 0x22, 0xbc, 0x72, 0xea, 0xc1, 0x3a, 0x5b, 0x85, 0xa2, 0x47, 0xa0, 0x7c, 0x8c, 0x2a, 0xa2, + 0x2a, 0x09, 0xac, 0x24, 0x5e, 0x40, 0x8f, 0xad, 0x70, 0x28, 0xcf, 0xc4, 0x43, 0x77, 0x75, 0xee, + 0xcc, 0x33, 0x25, 0x42, 0x00, 0xfd, 0x4d, 0x3d, 0x99, 0x22, 0xf8, 0x2e, 0x54, 0xd4, 0x11, 0xad, + 0x42, 0xd8, 0xd6, 0xa1, 0x51, 0xda, 0xb1, 0xe6, 0x9b, 0xaf, 0xa1, 0xc7, 0x6a, 0xe0, 0xad, 0x8d, + 0xff, 0x5f, 0x96, 0xfc, 0x14, 0xfa, 0x9b, 0x4b, 0x66, 0x2a, 0xbf, 0x3f, 0xe4, 0xa0, 0xfc, 0x8a, + 0x2c, 0x27, 0x6c, 0x7d, 0x04, 0xc5, 0x95, 0xbd, 0x94, 0xcd, 0x45, 0x0d, 0x8b, 0x6f, 0xbe, 0x6b, + 0x4b, 0xc1, 0x8d, 0xed, 0x9a, 0x24, 0xb0, 0xce, 0x85, 0x31, 0x3d, 0x96, 0x27, 0xd6, 0x95, 0xbf, + 0x08, 0x58, 0x02, 0x15, 0xd8, 0xac, 0x2a, 0x27, 0x7c, 0xce, 0xc6, 0xe8, 0x6b, 0x50, 0x9f, 0x2e, + 0x1c, 0xb2, 0xa2, 0x92, 0x5d, 0x14, 0x6c, 0x90, 0x24, 0x21, 0xf0, 0x6d, 0x68, 0xcb, 0xfc, 0xb2, + 0x3c, 0xdf, 0x71, 0x7d, 0x87, 0xae, 0xfb, 0x25, 0xb6, 0x40, 0x09, 0xb7, 0x24, 0xf9, 0x4c, 0x51, + 0x19, 0xc6, 0xf0, 0x7a, 0x90, 0x46, 0x66, 0xc3, 0x87, 0xbf, 0xe4, 0x00, 0xc5, 0x55, 0x64, 0xac, + 0xa9, 0x8a, 0xf4, 0x5c, 0xe3, 0x43, 0x43, 0x8a, 0x4b, 0xad, 0x58, 0x33, 0xb7, 0xd4, 0x54, 0x5c, + 0x4c, 0xf1, 0x58, 0x8b, 0x52, 0x27, 0x74, 0x3a, 0xb3, 0x94, 0x68, 0x71, 0x8b, 0x28, 0x70, 0x81, + 0xcf, 0xa4, 0x07, 0x67, 0x50, 0xe3, 0x25, 0x39, 0xa2, 0x36, 0x0d, 0xd0, 0x33, 0x28, 0xf2, 0x30, + 0x2b, 0xab, 0x93, 0x35, 0x2b, 0x38, 0xac, 0xa7, 0x6c, 0xcc, 0xdc, 0x37, 0x2b, 0x2b, 0x20, 0x53, + 0x77, 0x35, 0x0b, 0xd4, 0xce, 0xd5, 0x39, 0x6d, 0x24, 0x49, 0xe6, 0x5f, 0x0b, 0xb0, 0x2b, 0x4b, + 0x9a, 0xf9, 0xe9, 0xd3, 0x09, 0xb1, 0x69, 0xa6, 0xac, 0xfd, 0xbf, 0x42, 0x0d, 0x1a, 0x00, 0x08, + 0xc3, 0xb9, 0x17, 0x32, 0x69, 0xc2, 0xd6, 0x2d, 0xf4, 0x1f, 0xd7, 0xb8, 0x08, 0x1f, 0x06, 0xe8, + 0x43, 0x68, 0x7a, 0x64, 0x35, 0x73, 0x56, 0x73, 0x35, 0xa5, 0xa4, 0xb6, 0x26, 0xae, 0xbc, 0xa1, + 0x44, 0xe4, 0x94, 0xe7, 0xd0, 0x9c, 0xac, 0x29, 0x09, 0xac, 0x37, 0x2c, 0xbb, 0x28, 0x59, 0xf5, + 0xcb, 0x22, 0x38, 0x0d, 0x41, 0xfc, 0x42, 0xd2, 0x38, 0x46, 0x4b, 0x21, 0x9f, 0xd9, 0xd5, 0xaf, + 0xc8, 0x9e, 0x5d, 0x50, 0x30, 0x23, 0xf0, 0xf8, 0x32, 0xec, 0x8e, 0x54, 0x54, 0x65, 0x7c, 0x39, + 0x4d, 0x6b, 0x60, 0xc5, 0x21, 0x44, 0x84, 0x82, 0x9a, 0xac, 0x1c, 0x4e, 0x10, 0xf3, 0xdf, 0x85, + 0x0e, 0x6b, 0x7b, 0x7c, 0xf7, 0x17, 0xce, 0xd2, 0xa6, 0xc4, 0x0a, 0x9c, 0x5f, 0x92, 0x3e, 0x08, + 0x99, 0x76, 0x8c, 0x3e, 0x62, 0x64, 0xf4, 0x38, 0xde, 0xcb, 0xd6, 0xa5, 0x21, 0x51, 0xeb, 0x7a, + 0x01, 0x70, 0x78, 0x61, 0xaf, 0xe6, 0x84, 0xfb, 0x76, 0x87, 0xc4, 0xf8, 0x88, 0x55, 0xa5, 0x90, + 0xb7, 0xc4, 0x3d, 0x22, 0x2f, 0xee, 0x11, 0xbd, 0x81, 0xbe, 0x08, 0x71, 0x28, 0x91, 0xfa, 0xc4, + 0x7d, 0x02, 0xa6, 0xe1, 0xb7, 0xb9, 0x07, 0xad, 0xb1, 0x6f, 0xaf, 0x82, 0x73, 0xe2, 0xcb, 0x9c, + 0xfc, 0xef, 0xab, 0x99, 0x7f, 0xce, 0x43, 0x6f, 0x23, 0xc7, 0x32, 0x15, 0xdf, 0x87, 0xa1, 0xdd, + 0x62, 0x49, 0x99, 0x6a, 0x1d, 0x39, 0x25, 0x0a, 0x80, 0x36, 0x58, 0x04, 0xe3, 0x63, 0x68, 0x53, + 0x65, 0xb0, 0x95, 0xc8, 0x3c, 0xb5, 0x52, 0xd2, 0x1b, 0xdc, 0xa2, 0x49, 0xef, 0x12, 0xe7, 0x55, + 0x31, 0x79, 0x5e, 0xa1, 0x1f, 0x40, 0x43, 0x31, 0x89, 0xe7, 0x4e, 0x2f, 0x04, 0x70, 0xf1, 0x3a, + 0x49, 0xa4, 0xfe, 0x31, 0x67, 0xe1, 0xba, 0x1f, 0x0d, 0x78, 0xd5, 0x53, 0xdb, 0x9f, 0x13, 0x2a, + 0xdd, 0x28, 0x6f, 0x89, 0x1c, 0x48, 0x01, 0xfe, 0x6d, 0x9e, 0xb3, 0xae, 0x36, 0xb8, 0x1c, 0x79, + 0x0b, 0xe7, 0x2b, 0xad, 0x4d, 0xf3, 0xd7, 0xec, 0x02, 0x14, 0x2d, 0x94, 0xb1, 0xc7, 0x6f, 0xae, + 0xc8, 0x1b, 0x2b, 0x7d, 0xc4, 0xd7, 0x19, 0x11, 0xeb, 0xa8, 0x3d, 0x83, 0x06, 0x97, 0x11, 0x67, + 0x86, 0x33, 0x93, 0x47, 0x46, 0x11, 0x03, 0xa3, 0x71, 0x6f, 0x4f, 0x18, 0x28, 0xfd, 0x86, 0x01, + 0x35, 0x66, 0x21, 0xf5, 0x69, 0x76, 0xa7, 0x4d, 0x28, 0x2e, 0xc8, 0x39, 0xbd, 0xc1, 0x65, 0xc1, + 0x63, 0x60, 0x54, 0xf2, 0x9d, 0xf9, 0x05, 0xbd, 0xe1, 0x26, 0x26, 0x99, 0xe6, 0x21, 0xec, 0x24, + 0x8c, 0xc9, 0x74, 0xc0, 0xfe, 0xbb, 0x00, 0x20, 0x1a, 0x48, 0x89, 0xdd, 0xf1, 0x7b, 0x41, 0x2e, + 0x71, 0x2f, 0xe0, 0xf7, 0xe7, 0xa9, 0xed, 0xd9, 0x53, 0x7e, 0x12, 0xaa, 0xa3, 0x56, 0x8f, 0x39, + 0x0a, 0xd8, 0xd7, 0xb6, 0xb3, 0xb0, 0x27, 0x0b, 0x22, 0x8c, 0x66, 0x28, 0x10, 0x12, 0x38, 0x1c, + 0xa9, 0xc0, 0xcb, 0xcb, 0x70, 0x51, 0x5c, 0x86, 0x55, 0xe6, 0x1d, 0x8a, 0x2b, 0xf1, 0xfb, 0x80, + 0x02, 0x05, 0x94, 0xc1, 0xca, 0xf6, 0x94, 0x60, 0x49, 0x08, 0x76, 0x14, 0x67, 0xc4, 0x18, 0x52, + 0xfa, 0x03, 0xe8, 0xfa, 0x64, 0x4a, 0x9c, 0xeb, 0x94, 0x7c, 0x59, 0xc8, 0xa3, 0x90, 0x17, 0xcd, + 0x60, 0x80, 0xc9, 0x10, 0xc9, 0xa7, 0x16, 0xc7, 0x26, 0x01, 0x98, 0x4d, 0x5c, 0x13, 0x14, 0x7e, + 0xe5, 0x66, 0xb8, 0xbe, 0xc3, 0x80, 0x6d, 0xb1, 0x4e, 0xe9, 0xab, 0x0a, 0xb9, 0xfb, 0x9a, 0x15, + 0xa9, 0xeb, 0x41, 0xc5, 0x09, 0xac, 0xc9, 0x55, 0xb0, 0x16, 0xd8, 0x59, 0xc5, 0x65, 0x27, 0x38, + 0x60, 0x23, 0x5e, 0x96, 0x57, 0x01, 0x99, 0xc5, 0x21, 0xb3, 0xca, 0x09, 0x02, 0x2b, 0x37, 0xa0, + 0xbd, 0xbe, 0x05, 0xda, 0xd3, 0xd8, 0xdd, 0xd8, 0xc4, 0xee, 0x24, 0xfa, 0x37, 0xd3, 0xe8, 0x9f, + 0x80, 0xf6, 0x56, 0x12, 0xda, 0xcd, 0x05, 0x3c, 0x10, 0xdb, 0xfd, 0xb6, 0xa7, 0x2a, 0xbb, 0x60, + 0xb1, 0x7c, 0x49, 0x22, 0x5d, 0x94, 0x47, 0x58, 0xb2, 0x59, 0xe3, 0xbc, 0x9b, 0x5e, 0x2d, 0x53, + 0x96, 0xfe, 0x29, 0x07, 0xdd, 0xd1, 0xd4, 0xa6, 0xbc, 0xcb, 0xcc, 0x7e, 0xb3, 0xb9, 0xad, 0xc7, + 0xbf, 0xeb, 0xf3, 0x47, 0xac, 0x51, 0x28, 0xde, 0x72, 0x27, 0x39, 0x66, 0x61, 0x4e, 0xda, 0x9b, + 0xc5, 0xef, 0xf7, 0x08, 0xd4, 0xc2, 0xe7, 0x33, 0x54, 0x86, 0xfc, 0xf0, 0x65, 0xe7, 0x1e, 0xaa, + 0x43, 0xe5, 0xf3, 0xd3, 0x97, 0xa7, 0xc3, 0x2f, 0x4e, 0x3b, 0x39, 0xd4, 0x85, 0xce, 0xe9, 0x70, + 0x6c, 0x1d, 0x0c, 0x87, 0xe3, 0xd1, 0x18, 0xef, 0x9f, 0x9d, 0x1d, 0x1f, 0x75, 0xf2, 0x68, 0x07, + 0xda, 0xa3, 0xf1, 0x10, 0x1f, 0x5b, 0xe3, 0xe1, 0xab, 0x03, 0xf6, 0x75, 0x7a, 0xdc, 0x29, 0xa0, + 0x3e, 0x74, 0xf7, 0x3f, 0xc3, 0xc7, 0xfb, 0x47, 0x5f, 0x26, 0xc5, 0x8b, 0x7b, 0x7f, 0xac, 0x41, + 0xfe, 0xec, 0x08, 0xed, 0x03, 0x44, 0x6d, 0x28, 0xea, 0x49, 0xcb, 0x36, 0x7a, 0x5b, 0xa3, 0xbf, + 0xc9, 0x90, 0xc6, 0x9b, 0xf7, 0x58, 0x65, 0x16, 0xc6, 0x81, 0x8b, 0x54, 0x42, 0x44, 0xaf, 0x79, + 0xc6, 0xfd, 0x18, 0x45, 0x4b, 0xbf, 0x93, 0xfb, 0x20, 0x87, 0x7e, 0x04, 0xb5, 0xf0, 0x0d, 0x07, + 0xed, 0x4a, 0xa9, 0xf4, 0x6b, 0x97, 0xd1, 0xdb, 0xa0, 0x87, 0x2b, 0xbe, 0x82, 0x56, 0xf2, 0x15, + 0x08, 0x3d, 0x92, 0xc2, 0x5b, 0x5f, 0x98, 0x8c, 0xc7, 0xdb, 0x99, 0xa1, 0xba, 0x8f, 0xa0, 0xa2, + 0x5e, 0x6a, 0x90, 0xda, 0x9a, 0xe4, 0xbb, 0x8f, 0xf1, 0x20, 0x45, 0x0d, 0x67, 0xfe, 0x10, 0xaa, + 0xfa, 0xdd, 0x04, 0x3d, 0x08, 0x43, 0x14, 0x7f, 0xe0, 0x30, 0x76, 0xd3, 0xe4, 0xf8, 0x64, 0xfd, + 0x50, 0xa1, 0x27, 0xa7, 0x5e, 0x47, 0xf4, 0xe4, 0xf4, 0x7b, 0x06, 0x9b, 0xfc, 0x09, 0x34, 0xe2, + 0xef, 0x0b, 0xe8, 0x61, 0xb8, 0x4c, 0xfa, 0xdd, 0xc2, 0x30, 0xb6, 0xb1, 0xe2, 0xb1, 0x4c, 0x96, + 0xab, 0x8e, 0xe5, 0x56, 0xc8, 0xd0, 0xb1, 0xdc, 0x5e, 0xe1, 0x4c, 0xdd, 0x18, 0xda, 0xa9, 0xf6, + 0x0a, 0x3d, 0xd6, 0xe9, 0xbe, 0xad, 0xb3, 0x37, 0x9e, 0xdc, 0xc0, 0x4d, 0x27, 0x4c, 0x78, 0xdd, + 0x47, 0x51, 0x44, 0x13, 0xb8, 0x60, 0xf4, 0x36, 0xe8, 0xa1, 0x55, 0x2f, 0xa0, 0x99, 0x78, 0x2e, + 0x40, 0x46, 0x4a, 0x36, 0xf6, 0x86, 0x70, 0x9b, 0x1e, 0xb6, 0x65, 0xba, 0x29, 0xd1, 0x5b, 0x96, + 0xea, 0x86, 0xf4, 0x96, 0xa5, 0x7b, 0x17, 0x36, 0xf9, 0x08, 0xea, 0xb1, 0xb3, 0x1b, 0xf5, 0xb5, + 0xe3, 0xe9, 0xde, 0xc2, 0x78, 0xb8, 0x85, 0x13, 0x6a, 0x19, 0x89, 0xb7, 0x9e, 0xc4, 0x3d, 0x1b, + 0x3d, 0x09, 0x2d, 0xde, 0x76, 0xe5, 0x37, 0x9e, 0xde, 0xc4, 0x8e, 0x2b, 0x4d, 0x5f, 0xde, 0xb5, + 0xd2, 0x1b, 0xde, 0x11, 0xb4, 0xd2, 0x9b, 0xee, 0xfc, 0x4c, 0xe9, 0x4f, 0xa0, 0x99, 0xc0, 0x43, + 0x1d, 0xf4, 0x6d, 0xa0, 0x6e, 0x3c, 0xda, 0xca, 0xd3, 0xba, 0x0e, 0xde, 0xfb, 0xdb, 0x3f, 0x9f, + 0xe6, 0xfe, 0xce, 0x7e, 0xff, 0x60, 0xbf, 0xdf, 0xff, 0xeb, 0xe9, 0x3d, 0xe8, 0x4f, 0xdd, 0xe5, + 0xc0, 0x63, 0x27, 0x34, 0x6b, 0x48, 0x06, 0xd4, 0xb9, 0xbc, 0x1e, 0x5c, 0x5e, 0x8b, 0x7f, 0x8f, + 0x4c, 0xca, 0xe2, 0xcf, 0xf7, 0xff, 0x13, 0x00, 0x00, 0xff, 0xff, 0xab, 0x35, 0x0b, 0x39, 0x6c, + 0x19, 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..5ab294181037c 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. @@ -877,6 +878,24 @@ func (m *RegionLocalState) MarshalTo(dAtA []byte) (int, error) { 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) @@ -2618,54 +2637,53 @@ 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, + // 762 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x7c, 0x54, 0xdb, 0x6e, 0xd3, 0x4c, + 0x10, 0xae, 0x93, 0x34, 0xb1, 0x27, 0x4e, 0xfe, 0x68, 0xfb, 0x4b, 0x35, 0xa9, 0x5a, 0x51, 0x23, + 0x10, 0x14, 0xc9, 0x88, 0x52, 0x21, 0xae, 0x90, 0x80, 0x52, 0xb5, 0x40, 0x51, 0xb5, 0xa9, 0x90, + 0xb8, 0xb2, 0x36, 0xf6, 0xa6, 0x31, 0xf1, 0x49, 0xde, 0x4d, 0x44, 0xb9, 0xe3, 0x2d, 0x78, 0x15, + 0xde, 0x80, 0x3b, 0x78, 0x04, 0x04, 0x2f, 0xc2, 0x1e, 0xec, 0x1c, 0xaa, 0xc2, 0x45, 0xe2, 0x99, + 0xf9, 0xe6, 0xf8, 0xed, 0xec, 0xc2, 0x46, 0x41, 0x46, 0xdc, 0x67, 0xb4, 0x98, 0xd1, 0x22, 0x1f, + 0x7a, 0x79, 0x91, 0xf1, 0x0c, 0x75, 0x56, 0x8c, 0xfd, 0x0e, 0x95, 0x7a, 0x85, 0xf6, 0xed, 0x84, + 0x72, 0x52, 0x69, 0xee, 0xd7, 0x1a, 0xb4, 0xb1, 0x80, 0x4f, 0x29, 0x63, 0xe4, 0x82, 0xa2, 0x2d, + 0xb0, 0x0a, 0x7a, 0x11, 0x65, 0xa9, 0x1f, 0x85, 0x8e, 0x71, 0xd3, 0xb8, 0xdb, 0xc0, 0xa6, 0x36, + 0x9c, 0x84, 0xe8, 0x1e, 0x58, 0xa3, 0x22, 0x4b, 0xfc, 0x9c, 0xd2, 0xc2, 0xa9, 0x09, 0xb0, 0xbd, + 0x6f, 0x7b, 0x65, 0xba, 0x33, 0x61, 0xc3, 0xa6, 0x84, 0xa5, 0x84, 0x6e, 0x43, 0x8b, 0x67, 0xda, + 0xb1, 0x7e, 0x8d, 0x63, 0x93, 0x67, 0xca, 0x6d, 0x0f, 0x5a, 0x89, 0xae, 0xec, 0x34, 0x94, 0x5b, + 0xcf, 0xab, 0xba, 0x2d, 0x3b, 0xc2, 0x95, 0x03, 0x7a, 0x0c, 0x76, 0xd9, 0x1a, 0xcd, 0xb3, 0x60, + 0xec, 0xac, 0xab, 0x80, 0x8d, 0x2a, 0x2f, 0x56, 0xd8, 0x4b, 0x09, 0xe1, 0x76, 0xb1, 0x50, 0xd0, + 0x2e, 0xd8, 0x11, 0xf3, 0x79, 0x96, 0x0c, 0x19, 0xcf, 0x52, 0xea, 0x34, 0x45, 0x9c, 0x89, 0xdb, + 0x11, 0x3b, 0xaf, 0x4c, 0x72, 0x6a, 0xc6, 0x49, 0xc1, 0xfd, 0x09, 0xbd, 0x74, 0x5a, 0x02, 0xb7, + 0xb1, 0xa9, 0x0c, 0xaf, 0xe9, 0x25, 0xda, 0x84, 0x16, 0x4d, 0x43, 0x05, 0x99, 0x0a, 0x6a, 0x0a, + 0x55, 0x00, 0xee, 0x53, 0x40, 0x92, 0xba, 0xf3, 0x62, 0x9a, 0x06, 0x84, 0xd3, 0x70, 0xc0, 0xc5, + 0x3f, 0xfa, 0x1f, 0xd6, 0xa3, 0x34, 0xa4, 0x1f, 0x4b, 0xf6, 0xb4, 0x82, 0x10, 0x34, 0x38, 0x2d, + 0x12, 0xc5, 0x5a, 0x03, 0x2b, 0xd9, 0x3d, 0x83, 0xee, 0x20, 0x25, 0x39, 0x1b, 0x67, 0xfc, 0xc5, + 0xd1, 0x51, 0x14, 0x53, 0xd4, 0x85, 0x5a, 0x30, 0x52, 0x81, 0x16, 0x16, 0x92, 0x8c, 0x62, 0xd1, + 0x27, 0x5a, 0x45, 0x49, 0x19, 0xf5, 0xc1, 0x0c, 0xc6, 0x34, 0x98, 0xb0, 0x69, 0xa2, 0xa8, 0xed, + 0xe0, 0xb9, 0xee, 0x1e, 0x83, 0x5d, 0x65, 0x3c, 0x15, 0xac, 0xa0, 0x27, 0xc2, 0x77, 0xe4, 0x8f, + 0x44, 0x6a, 0x26, 0xb2, 0xd6, 0x05, 0x5d, 0xdb, 0xde, 0xea, 0xc6, 0xac, 0x36, 0x80, 0x5b, 0xc1, + 0x48, 0x7e, 0x99, 0xfb, 0x1e, 0x3a, 0x73, 0x68, 0x3c, 0x4d, 0x27, 0xe8, 0x60, 0x71, 0x52, 0x86, + 0x22, 0xbe, 0x7f, 0x25, 0xd3, 0xd2, 0x16, 0x2d, 0xce, 0x4c, 0x0c, 0x10, 0x12, 0x4e, 0xd4, 0x00, + 0x36, 0x56, 0xb2, 0xdb, 0x84, 0xc6, 0xa1, 0x20, 0xdd, 0xdd, 0x07, 0x53, 0xb0, 0xf8, 0x8e, 0xc4, + 0x53, 0x8a, 0x7a, 0x50, 0x97, 0xfc, 0x1a, 0xca, 0x4d, 0x8a, 0x92, 0xc6, 0x99, 0x84, 0xca, 0x50, + 0xad, 0xb8, 0xdf, 0x0d, 0xe8, 0xc9, 0x42, 0x55, 0x6f, 0x87, 0x22, 0x21, 0xba, 0x03, 0x4d, 0x7d, + 0xde, 0x65, 0x67, 0xdd, 0xd5, 0x95, 0xc0, 0x25, 0x2a, 0x4f, 0x59, 0x52, 0xe1, 0x2f, 0x51, 0x6a, + 0x4a, 0xc3, 0x40, 0xd2, 0x7a, 0xbf, 0xec, 0xb4, 0xae, 0x68, 0xda, 0xbc, 0x32, 0x5c, 0xd5, 0xa8, + 0x1e, 0x01, 0x39, 0xd0, 0x12, 0x76, 0x26, 0x4b, 0x36, 0x54, 0x9e, 0x4a, 0x45, 0x0f, 0xa0, 0x21, + 0x8b, 0x97, 0xcb, 0xb9, 0xf5, 0x17, 0xb6, 0xe5, 0xe1, 0x60, 0xe5, 0xe8, 0x1e, 0x01, 0x0c, 0x78, + 0x56, 0xd0, 0x93, 0x90, 0xa6, 0x1c, 0x6d, 0x03, 0x04, 0xf1, 0x94, 0x89, 0xf5, 0x58, 0xdc, 0x3f, + 0xab, 0xb4, 0x88, 0x0b, 0x78, 0x03, 0xc4, 0x5a, 0x0a, 0x67, 0x09, 0xea, 0x01, 0x5a, 0x4c, 0x07, + 0xbb, 0x43, 0xe8, 0x4a, 0x62, 0xde, 0x64, 0x01, 0x89, 0xf5, 0x22, 0x3e, 0x04, 0x18, 0x93, 0x22, + 0xf4, 0x99, 0xd4, 0x4a, 0x6a, 0xd0, 0xfc, 0x7a, 0x1d, 0x0b, 0x48, 0xf9, 0x61, 0x6b, 0x5c, 0x89, + 0xb2, 0x7c, 0x4c, 0x18, 0xf7, 0xf5, 0x02, 0xeb, 0x0a, 0x96, 0xb4, 0x9c, 0x48, 0x83, 0xfb, 0xd9, + 0xd0, 0x45, 0x9e, 0xe5, 0x79, 0x7c, 0xa9, 0x23, 0x6e, 0x41, 0x87, 0x08, 0x2d, 0xa2, 0xa1, 0xbf, + 0xbc, 0xf5, 0x76, 0x69, 0x54, 0x71, 0xe8, 0x15, 0xfc, 0xc7, 0xab, 0x4b, 0x52, 0xb6, 0xa3, 0x5f, + 0x8f, 0xdd, 0x6b, 0x76, 0x68, 0xf5, 0x3a, 0xe1, 0x2e, 0x5f, 0xd1, 0xdd, 0x0f, 0x62, 0x01, 0xd4, + 0x71, 0x2e, 0x4d, 0xea, 0xc1, 0xfa, 0x62, 0xc8, 0xee, 0xbe, 0x73, 0x25, 0xab, 0x7c, 0x69, 0x74, + 0x32, 0xed, 0xb6, 0xb4, 0x30, 0xb5, 0x7f, 0x2d, 0xcc, 0xde, 0x01, 0x58, 0xf3, 0x58, 0x04, 0xd0, + 0x7c, 0x9b, 0x15, 0x09, 0x89, 0x7b, 0x6b, 0xc8, 0x06, 0x53, 0x71, 0x10, 0xa5, 0x17, 0x3d, 0x03, + 0x75, 0xc0, 0x9a, 0x3f, 0x25, 0xbd, 0xda, 0xf3, 0xbd, 0x6f, 0xbf, 0x76, 0x8c, 0x1f, 0xe2, 0xf7, + 0x53, 0xfc, 0xbe, 0xfc, 0xde, 0x59, 0x03, 0x27, 0xc8, 0x12, 0x2f, 0x17, 0xce, 0x01, 0xc9, 0x3d, + 0x1e, 0x4d, 0x66, 0xde, 0x64, 0xa6, 0x9e, 0xdf, 0x61, 0x53, 0x7d, 0x1e, 0xfd, 0x09, 0x00, 0x00, + 0xff, 0xff, 0x13, 0x81, 0xd5, 0xd2, 0xc8, 0x05, 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..47239e8a9ce28 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. @@ -68,6 +69,7 @@ type TikvClient interface { RawPut(ctx context.Context, in *kvrpcpb.RawPutRequest, opts ...grpc.CallOption) (*kvrpcpb.RawPutResponse, error) RawDelete(ctx context.Context, in *kvrpcpb.RawDeleteRequest, opts ...grpc.CallOption) (*kvrpcpb.RawDeleteResponse, 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) // 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) @@ -233,6 +235,15 @@ 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) 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...) @@ -390,6 +401,7 @@ type TikvServer interface { RawPut(context.Context, *kvrpcpb.RawPutRequest) (*kvrpcpb.RawPutResponse, error) RawDelete(context.Context, *kvrpcpb.RawDeleteRequest) (*kvrpcpb.RawDeleteResponse, error) RawScan(context.Context, *kvrpcpb.RawScanRequest) (*kvrpcpb.RawScanResponse, error) + RawDeleteRange(context.Context, *kvrpcpb.RawDeleteRangeRequest) (*kvrpcpb.RawDeleteRangeResponse, error) // SQL push down commands. Coprocessor(context.Context, *coprocessor.Request) (*coprocessor.Response, error) CoprocessorStream(*coprocessor.Request, Tikv_CoprocessorStreamServer) error @@ -695,6 +707,24 @@ 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_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 { @@ -908,6 +938,10 @@ var _Tikv_serviceDesc = grpc.ServiceDesc{ MethodName: "RawScan", Handler: _Tikv_RawScan_Handler, }, + { + MethodName: "RawDeleteRange", + Handler: _Tikv_RawDeleteRange_Handler, + }, { MethodName: "Coprocessor", Handler: _Tikv_Coprocessor_Handler, @@ -948,44 +982,45 @@ 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, + // 626 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x94, 0x95, 0xdd, 0x6e, 0xd3, 0x3e, + 0x18, 0xc6, 0x37, 0xa9, 0xff, 0xfe, 0xb7, 0x77, 0x1d, 0x30, 0x77, 0x40, 0x17, 0xb6, 0xf2, 0x71, + 0x84, 0x38, 0x28, 0x9f, 0x12, 0x07, 0x7c, 0x08, 0x9a, 0x4a, 0x08, 0xb2, 0x4a, 0x53, 0xb3, 0x13, + 0x8e, 0x90, 0x1b, 0x99, 0xb4, 0x6a, 0x1a, 0x07, 0xc7, 0xcd, 0xb4, 0x3b, 0xe1, 0x92, 0x38, 0xe4, + 0x12, 0x10, 0xdc, 0x03, 0xc7, 0xbc, 0x49, 0x6a, 0xc7, 0x6e, 0x5a, 0x24, 0x0e, 0xaa, 0xc6, 0xbf, + 0xe7, 0x7d, 0x1e, 0x5b, 0xf6, 0x1b, 0x07, 0x5a, 0x72, 0x3a, 0xcb, 0x92, 0x71, 0x2f, 0x11, 0x5c, + 0x72, 0xd2, 0x2c, 0x47, 0xce, 0x41, 0xc0, 0x11, 0x04, 0x2c, 0x4d, 0xb9, 0x28, 0x25, 0x67, 0x7f, + 0x96, 0x89, 0x24, 0x50, 0x95, 0x4e, 0x5b, 0xd0, 0xcf, 0xf2, 0x53, 0xca, 0x44, 0xc6, 0x84, 0x86, + 0x87, 0x21, 0x0f, 0x79, 0xf1, 0xf8, 0x30, 0x7f, 0x2a, 0xe9, 0x93, 0xdf, 0x2d, 0x68, 0x9c, 0x63, + 0x2e, 0x79, 0x06, 0xff, 0x79, 0xd9, 0x3b, 0x26, 0x49, 0xbb, 0xa7, 0xc2, 0x70, 0x34, 0x62, 0x5f, + 0x16, 0x2c, 0x95, 0xce, 0xa1, 0x0d, 0xd3, 0x84, 0xc7, 0x29, 0xbb, 0xb7, 0x45, 0x9e, 0x43, 0xd3, + 0xcb, 0xfc, 0x80, 0xc6, 0xa4, 0xaa, 0xc8, 0x87, 0xca, 0x77, 0x7d, 0x85, 0x6a, 0xa3, 0x0b, 0xe0, + 0x65, 0x67, 0x82, 0x5d, 0x88, 0xa9, 0x64, 0xa4, 0xa3, 0xcb, 0x14, 0x52, 0x01, 0x47, 0x6b, 0x14, + 0x1d, 0xf2, 0x0a, 0x76, 0xbc, 0xcc, 0xe5, 0xf3, 0xf9, 0x54, 0x92, 0x1b, 0xba, 0xb0, 0x04, 0x2a, + 0xe0, 0x66, 0x8d, 0xdb, 0xf6, 0xf7, 0xf3, 0x84, 0x0b, 0xd3, 0x5e, 0x82, 0xba, 0x5d, 0x71, 0x6d, + 0x7f, 0x03, 0xbb, 0x38, 0x7b, 0xc4, 0x68, 0xbc, 0x48, 0x88, 0x31, 0x4d, 0x49, 0x54, 0x40, 0xa7, + 0x2e, 0xd8, 0x9b, 0xd0, 0xa7, 0x32, 0x98, 0xe4, 0x1b, 0x5f, 0x55, 0x2a, 0x54, 0xdf, 0x84, 0x4a, + 0xd1, 0x21, 0x23, 0xb8, 0xba, 0x0c, 0x19, 0xf1, 0x28, 0x1a, 0xd3, 0x60, 0x46, 0x4e, 0xec, 0x7a, + 0xc5, 0x55, 0x5c, 0x77, 0x93, 0x6c, 0x2f, 0x2c, 0x3f, 0xb1, 0x53, 0x8e, 0x71, 0x1d, 0xeb, 0x10, + 0x73, 0x54, 0x5f, 0x58, 0xa5, 0xe8, 0x90, 0x53, 0xd8, 0xf7, 0x32, 0x1c, 0xf3, 0x28, 0x63, 0x45, + 0xce, 0x2d, 0x5d, 0x6d, 0x50, 0x15, 0x75, 0xbc, 0x5e, 0xd4, 0x69, 0x8f, 0xa1, 0x81, 0xfd, 0xe9, + 0x12, 0x52, 0x75, 0xa2, 0xab, 0xbc, 0x6d, 0x8b, 0xd9, 0x0b, 0x18, 0xb0, 0x88, 0x61, 0xd3, 0xd0, + 0x38, 0x64, 0xc6, 0x02, 0x0c, 0x5a, 0x5f, 0x80, 0x25, 0xea, 0xb4, 0x17, 0xd0, 0x1c, 0xd1, 0x8b, + 0xfc, 0xa0, 0xaa, 0x5e, 0x29, 0x41, 0xbd, 0x57, 0x14, 0x5f, 0x31, 0x9f, 0x2d, 0x56, 0xcc, 0x08, + 0xd6, 0x9a, 0x0b, 0xae, 0xcd, 0x03, 0xd8, 0x45, 0x56, 0xae, 0x8a, 0x1c, 0x99, 0x75, 0xcb, 0x95, + 0x2e, 0x23, 0x9c, 0x75, 0x92, 0x4e, 0x79, 0x0d, 0xff, 0x23, 0x2e, 0xde, 0x55, 0x6b, 0x2e, 0xf3, + 0x75, 0xed, 0xd4, 0x05, 0xed, 0xf7, 0xe1, 0x4a, 0x15, 0x5b, 0x6c, 0x67, 0x77, 0xcd, 0x7c, 0xe6, + 0x8e, 0xde, 0xde, 0xa8, 0xeb, 0xd0, 0x97, 0xb0, 0xe7, 0x56, 0xb7, 0x19, 0x5e, 0x22, 0xe6, 0xdd, + 0x56, 0x5d, 0x22, 0x36, 0x35, 0x36, 0xe6, 0xc0, 0x70, 0xfb, 0x52, 0x30, 0x3a, 0xff, 0xc7, 0x8c, + 0x47, 0xdb, 0x78, 0x36, 0x8d, 0x11, 0xde, 0x97, 0xc4, 0xe9, 0xd9, 0xd7, 0x66, 0x0e, 0x87, 0x58, + 0x4f, 0x43, 0x86, 0x1d, 0x66, 0x6b, 0x03, 0x1e, 0xa3, 0xf9, 0xfe, 0x36, 0x79, 0x0b, 0x3b, 0x7e, + 0x4c, 0x93, 0x74, 0xc2, 0x25, 0x39, 0x5e, 0x29, 0x52, 0x82, 0x3b, 0x59, 0xc4, 0xb3, 0xcd, 0x11, + 0x1f, 0x60, 0xcf, 0x4f, 0xa2, 0xfc, 0x66, 0x0a, 0xa7, 0x3c, 0x36, 0x9a, 0xd4, 0xa0, 0xf5, 0x26, + 0xb5, 0x44, 0xbd, 0x23, 0x43, 0x68, 0x0d, 0xb3, 0x20, 0xc0, 0xe6, 0xeb, 0x5f, 0x7a, 0xec, 0x92, + 0x54, 0xf5, 0x26, 0x56, 0x69, 0x27, 0x1b, 0x54, 0x1d, 0xf7, 0x11, 0xae, 0x69, 0xc5, 0x97, 0x54, + 0xc8, 0xf3, 0x94, 0xdc, 0xa9, 0x9b, 0x96, 0x92, 0x8a, 0xbd, 0xfb, 0x97, 0x0a, 0x15, 0xdd, 0x7f, + 0xf0, 0xed, 0x67, 0x77, 0xfb, 0x3b, 0xfe, 0x7e, 0xe0, 0xef, 0xeb, 0xaf, 0xee, 0x16, 0x74, 0x02, + 0x3e, 0xef, 0x25, 0xd3, 0x38, 0x0c, 0x68, 0xd2, 0xcb, 0xbf, 0x75, 0x18, 0x53, 0x7c, 0xa4, 0xc6, + 0xcd, 0xe2, 0xef, 0xe9, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x42, 0x1e, 0x29, 0x02, 0x10, 0x07, + 0x00, 0x00, }