From d9f7ffe539fef04b516c9326bbff0d850191eba2 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 24 Aug 2018 16:52:40 +0800 Subject: [PATCH 01/41] plan: fix a bug when using correlated column as index (#7357) --- expression/column.go | 15 +++ expression/simple_rewriter.go | 27 +++++ plan/cbo_test.go | 15 ++- plan/logical_plans.go | 5 +- plan/logical_plans_test.go | 195 ++++++++++++++++++++++++++++++++++ 5 files changed, 254 insertions(+), 3 deletions(-) create mode 100644 plan/logical_plans_test.go diff --git a/expression/column.go b/expression/column.go index 52557377c34d0..f63cf44a117a5 100644 --- a/expression/column.go +++ b/expression/column.go @@ -375,3 +375,18 @@ func IndexInfo2Cols(cols []*Column, index *model.IndexInfo) ([]*Column, []int) { } return retCols, lengths } + +// FindColumnsByUniqueIDs will find columns by checking the unique id. +// Note: `ids` must be a subset of the column slice. +func FindColumnsByUniqueIDs(cols []*Column, ids []int) []*Column { + retCols := make([]*Column, 0, len(ids)) + for _, id := range ids { + for _, col := range cols { + if col.UniqueID == id { + retCols = append(retCols, col) + break + } + } + } + return retCols +} diff --git a/expression/simple_rewriter.go b/expression/simple_rewriter.go index 5a5b4450b5c1e..6d752fca78ff6 100644 --- a/expression/simple_rewriter.go +++ b/expression/simple_rewriter.go @@ -56,6 +56,33 @@ func RewriteSimpleExprWithTableInfo(ctx sessionctx.Context, tbl *model.TableInfo return rewriter.pop(), nil } +func ParseSimpleExprsWithSchema(ctx sessionctx.Context, exprStr string, schema *Schema) ([]Expression, error) { + exprStr = "select " + exprStr + stmts, err := parser.New().Parse(exprStr, "", "") + if err != nil { + return nil, errors.Trace(err) + } + fields := stmts[0].(*ast.SelectStmt).Fields.Fields + exprs := make([]Expression, 0, len(fields)) + for _, field := range fields { + expr, err := RewriteSimpleExprWithSchema(ctx, field.Expr, schema) + if err != nil { + return nil, errors.Trace(err) + } + exprs = append(exprs, expr) + } + return exprs, nil +} + +func RewriteSimpleExprWithSchema(ctx sessionctx.Context, expr ast.ExprNode, schema *Schema) (Expression, error) { + rewriter := &simpleRewriter{ctx: ctx, schema: schema} + expr.Accept(rewriter) + if rewriter.err != nil { + return nil, errors.Trace(rewriter.err) + } + return rewriter.pop(), nil +} + func (sr *simpleRewriter) rewriteColumn(nodeColName *ast.ColumnNameExpr) (*Column, error) { col := sr.schema.FindColumnByName(nodeColName.Name.Name.L) if col != nil { diff --git a/plan/cbo_test.go b/plan/cbo_test.go index 875709e3c54e4..832ac06ede355 100644 --- a/plan/cbo_test.go +++ b/plan/cbo_test.go @@ -623,7 +623,7 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { store.Close() }() tk.MustExec("use test") - tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("create table t(a int, b int, c int, index idx(c))") tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5), (6,6,6), (7,7,7), (8,8,8), (9,9,9),(10,10,10)") tk.MustExec("analyze table t") tk.MustQuery("explain select t.c in (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t;"). @@ -640,6 +640,19 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { " └─TableReader_27 10.00 root data:TableScan_26", " └─TableScan_26 10.00 cop table:t1, range:[-inf,+inf], keep order:false", )) + tk.MustQuery("explain select (select concat(t1.a, \",\", t1.b) from t t1 where t1.a=t.a and t1.c=t.c) from t"). + Check(testkit.Rows( + "Projection_8 10.00 root concat(t1.a, \",\", t1.b)", + "└─Apply_10 10.00 root left outer join, inner:MaxOneRow_13", + " ├─TableReader_12 10.00 root data:TableScan_11", + " │ └─TableScan_11 10.00 cop table:t, range:[-inf,+inf], keep order:false", + " └─MaxOneRow_13 1.00 root ", + " └─Projection_14 0.80 root concat(cast(t1.a), \",\", cast(t1.b))", + " └─IndexLookUp_21 0.80 root ", + " ├─IndexScan_18 1.00 cop table:t1, index:c, range: decided by [eq(t1.c, test.t.c)], keep order:false", + " └─Selection_20 0.80 cop eq(t1.a, test.t.a)", + " └─TableScan_19 1.00 cop table:t, keep order:false", + )) } func (s *testAnalyzeSuite) TestInconsistentEstimation(c *C) { diff --git a/plan/logical_plans.go b/plan/logical_plans.go index 1e4ad4a859292..12d2ff433106d 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -484,14 +484,15 @@ func (path *accessPath) splitCorColAccessCondFromFilters() (access, remained []e for i := path.eqCondCount; i < len(path.idxCols); i++ { matched := false for j, filter := range path.tableFilters { - if !isColEqCorColOrConstant(filter, path.idxCols[i]) { - break + if used[j] || !isColEqCorColOrConstant(filter, path.idxCols[i]) { + continue } matched = true access[i-path.eqCondCount] = filter if path.idxColLens[i] == types.UnspecifiedLength { used[j] = true } + break } if !matched { access = access[:i-path.eqCondCount] diff --git a/plan/logical_plans_test.go b/plan/logical_plans_test.go new file mode 100644 index 0000000000000..6ec6894d51381 --- /dev/null +++ b/plan/logical_plans_test.go @@ -0,0 +1,195 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain col1 copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package plan + +import ( + "fmt" + + "github.com/juju/errors" + . "github.com/pingcap/check" + "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/testleak" +) + +var _ = Suite(&testUnitTestSuit{}) + +type testUnitTestSuit struct { + ctx sessionctx.Context +} + +func (s *testUnitTestSuit) SetUpSuite(c *C) { + s.ctx = mockContext() +} + +func (s *testUnitTestSuit) newTypeWithFlen(typeByte byte, flen int) *types.FieldType { + tp := types.NewFieldType(typeByte) + tp.Flen = flen + return tp +} + +func (s *testUnitTestSuit) SubstituteCol2CorCol(expr expression.Expression, colIDs map[int]struct{}) (expression.Expression, error) { + switch x := expr.(type) { + case *expression.ScalarFunction: + newArgs := make([]expression.Expression, 0, len(x.GetArgs())) + for _, arg := range x.GetArgs() { + newArg, err := s.SubstituteCol2CorCol(arg, colIDs) + if err != nil { + return nil, errors.Trace(err) + } + newArgs = append(newArgs, newArg) + } + newSf, err := expression.NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + return newSf, errors.Trace(err) + case *expression.Column: + if _, ok := colIDs[x.UniqueID]; ok { + return &expression.CorrelatedColumn{Column: *x}, nil + } + } + return expr, nil +} + +func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { + defer testleak.AfterTest(c)() + totalSchema := expression.NewSchema() + totalSchema.Append(&expression.Column{ + ColName: model.NewCIStr("col1"), + UniqueID: 1, + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + totalSchema.Append(&expression.Column{ + ColName: model.NewCIStr("col2"), + UniqueID: 2, + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + totalSchema.Append(&expression.Column{ + ColName: model.NewCIStr("col3"), + UniqueID: 3, + RetType: s.newTypeWithFlen(mysql.TypeVarchar, 10), + }) + totalSchema.Append(&expression.Column{ + ColName: model.NewCIStr("col4"), + UniqueID: 4, + RetType: s.newTypeWithFlen(mysql.TypeVarchar, 10), + }) + totalSchema.Append(&expression.Column{ + ColName: model.NewCIStr("col5"), + UniqueID: 5, + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + testCases := []struct { + expr string + corColIDs []int + idxColIDs []int + idxColLens []int + access string + remained string + }{ + { + expr: "col1 = col2", + corColIDs: []int{2}, + idxColIDs: []int{1}, + idxColLens: []int{types.UnspecifiedLength}, + access: "[eq(col1, col2)]", + remained: "[]", + }, + { + expr: "col1 = col5 and col2 = 1", + corColIDs: []int{5}, + idxColIDs: []int{1, 2}, + idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, + access: "[eq(col1, col5) eq(col2, 1)]", + remained: "[]", + }, + { + expr: "col1 = col5 and col2 = 1", + corColIDs: []int{5}, + idxColIDs: []int{2, 1}, + idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, + access: "[eq(col2, 1) eq(col1, col5)]", + remained: "[]", + }, + { + expr: "col1 = col5 and col2 = 1", + corColIDs: []int{5}, + idxColIDs: []int{1}, + idxColLens: []int{types.UnspecifiedLength}, + access: "[eq(col1, col5)]", + remained: "[eq(col2, 1)]", + }, + { + expr: "col2 = 1 and col1 = col5", + corColIDs: []int{5}, + idxColIDs: []int{1}, + idxColLens: []int{types.UnspecifiedLength}, + access: "[eq(col1, col5)]", + remained: "[eq(col2, 1)]", + }, + { + expr: "col1 = col2 and col3 = col4 and col5 = 1", + corColIDs: []int{2, 4}, + idxColIDs: []int{1, 3}, + idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, + access: "[eq(col1, col2) eq(col3, col4)]", + remained: "[eq(col5, 1)]", + }, + { + expr: "col1 = col2 and col3 = col4 and col5 = 1", + corColIDs: []int{2, 4}, + idxColIDs: []int{1, 3}, + idxColLens: []int{types.UnspecifiedLength, 2}, + access: "[eq(col1, col2) eq(col3, col4)]", + remained: "[eq(col3, col4) eq(col5, 1)]", + }, + { + expr: `col1 = col5 and col3 = "col1" and col2 = col5`, + corColIDs: []int{5}, + idxColIDs: []int{1, 2, 3}, + idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength, types.UnspecifiedLength}, + access: "[eq(col1, col5) eq(col2, col5) eq(col3, col1)]", + remained: "[]", + }, + } + for _, tt := range testCases { + comment := Commentf("failed at case:\nexpr: %v\ncorColIDs: %v\nidxColIDs: %v\nidxColLens: %v\naccess: %v\nremained: %v\n", tt.expr, tt.corColIDs, tt.idxColIDs, tt.idxColLens, tt.access, tt.remained) + filters, err := expression.ParseSimpleExprsWithSchema(s.ctx, tt.expr, totalSchema) + if sf, ok := filters[0].(*expression.ScalarFunction); ok && sf.FuncName.L == ast.LogicAnd { + filters = expression.FlattenCNFConditions(sf) + } + c.Assert(err, IsNil, comment) + trueFilters := make([]expression.Expression, 0, len(filters)) + idMap := make(map[int]struct{}) + for _, id := range tt.corColIDs { + idMap[id] = struct{}{} + } + for _, filter := range filters { + trueFilter, err := s.SubstituteCol2CorCol(filter, idMap) + c.Assert(err, IsNil, comment) + trueFilters = append(trueFilters, trueFilter) + } + path := accessPath{ + eqCondCount: 0, + tableFilters: trueFilters, + idxCols: expression.FindColumnsByUniqueIDs(totalSchema.Columns, tt.idxColIDs), + idxColLens: tt.idxColLens, + } + access, remained := path.splitCorColAccessCondFromFilters() + c.Assert(fmt.Sprintf("%s", access), Equals, tt.access, comment) + c.Assert(fmt.Sprintf("%s", remained), Equals, tt.remained, comment) + } +} From 01c6bd820c752d6ee739020fd425caff94b96368 Mon Sep 17 00:00:00 2001 From: winkyao Date: Fri, 24 Aug 2018 21:54:31 +0800 Subject: [PATCH 02/41] privilege: execute admin command must have Super_priv. (#7486) --- plan/planbuilder.go | 3 +++ privilege/privileges/privileges_test.go | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/plan/planbuilder.go b/plan/planbuilder.go index bdacf33edea5f..1508cb535a5b4 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -498,6 +498,9 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { default: return nil, ErrUnsupportedType.Gen("Unsupported ast.AdminStmt(%T) for buildAdmin", as) } + + // Admin command can only be executed by administrator. + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "") return ret, nil } diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 8f0389f97dfed..359114f48570a 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -15,6 +15,7 @@ package privileges_test import ( "fmt" + "strings" "testing" . "github.com/pingcap/check" @@ -301,6 +302,24 @@ func (s *testPrivilegeSuite) TestInformationSchema(c *C) { mustExec(c, se, `select * from information_schema.key_column_usage`) } +func (s *testPrivilegeSuite) TestAdminCommand(c *C) { + se := newSession(c, s.store, s.dbName) + c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil), IsTrue) + mustExec(c, se, `CREATE USER 'test_admin'@'localhost';`) + mustExec(c, se, `FLUSH PRIVILEGES;`) + mustExec(c, se, `CREATE TABLE t(a int)`) + + c.Assert(se.Auth(&auth.UserIdentity{Username: "test_admin", Hostname: "localhost"}, nil, nil), IsTrue) + _, err := se.Execute(context.Background(), "ADMIN SHOW DDL JOBS") + c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + _, err = se.Execute(context.Background(), "ADMIN CHECK TABLE t") + c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + + c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil), IsTrue) + _, err = se.Execute(context.Background(), "ADMIN SHOW DDL JOBS") + c.Assert(err, IsNil) +} + func mustExec(c *C, se session.Session, sql string) { _, err := se.Execute(context.Background(), sql) c.Assert(err, IsNil) From 09fb68ae3bd5d93f5d8c2c4d3efc50575769338c Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Mon, 27 Aug 2018 13:53:28 +0800 Subject: [PATCH 03/41] server: fix ComStmtSendLongData when data length is 0 (#7485) --- server/driver_tidb.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 0967ca5eebc04..739e253e7422e 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -85,7 +85,12 @@ func (ts *TiDBStatement) AppendParam(paramID int, data []byte) error { if paramID >= len(ts.boundParams) { return mysql.NewErr(mysql.ErrWrongArguments, "stmt_send_longdata") } - ts.boundParams[paramID] = append(ts.boundParams[paramID], data...) + // If len(data) is 0, append an empty byte slice to the end to distinguish no data and no parameter. + if len(data) == 0 { + ts.boundParams[paramID] = []byte{} + } else { + ts.boundParams[paramID] = append(ts.boundParams[paramID], data...) + } return nil } From 3af92ab2085501641eeac7feab6ece80cfd1254d Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 27 Aug 2018 15:05:31 +0800 Subject: [PATCH 04/41] ddl, metrics: update metrics (#7472) --- ddl/ddl_worker.go | 11 +++++++---- ddl/syncer.go | 18 +++++++++++------- metrics/ddl.go | 21 +++++++++------------ 3 files changed, 27 insertions(+), 23 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 326c843734e18..283e70220e4f9 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -209,7 +209,7 @@ func (d *ddl) addDDLJob(ctx sessionctx.Context, job *model.Job) error { return errors.Trace(err) }) - metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerAddDDLJob, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerAddDDLJob, job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) return errors.Trace(err) } @@ -278,7 +278,7 @@ func (w *worker) deleteRange(job *model.Job) error { func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) { startTime := time.Now() defer func() { - metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerFinishDDLJob, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerFinishDDLJob, job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() switch job.Type { @@ -396,7 +396,6 @@ func (w *worker) handleDDLJobQueue(d *ddlCtx) error { // wait a while to retry again. If we don't wait here, DDL will retry this job immediately, // which may act like a deadlock. log.Infof("[ddl-%s] run DDL job error, sleeps a while:%v then retries it.", w, WaitTimeWhenErrorOccured) - metrics.DDLJobErrCounter.Inc() time.Sleep(WaitTimeWhenErrorOccured) } @@ -447,6 +446,10 @@ func chooseLeaseTime(t, max time.Duration) time.Duration { // runDDLJob runs a DDL job. It returns the current schema version in this transaction and the error. func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { log.Infof("[ddl-%s] run DDL job %s", w, job) + timeStart := time.Now() + defer func() { + metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerRunDDLJob, job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) + }() if job.IsFinished() { return } @@ -554,7 +557,7 @@ func (w *worker) waitSchemaChanged(ctx context.Context, d *ddlCtx, waitTime time timeStart := time.Now() var err error defer func() { - metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerWaitSchemaChanged, metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) + metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerWaitSchemaChanged, job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) }() if latestSchemaVersion == 0 { diff --git a/ddl/syncer.go b/ddl/syncer.go index 50b1c97e5a7b2..f077346183042 100644 --- a/ddl/syncer.go +++ b/ddl/syncer.go @@ -221,7 +221,7 @@ func (s *schemaVersionSyncer) UpdateSelfVersion(ctx context.Context, version int err := PutKVToEtcd(ctx, s.etcdCli, putKeyNoRetry, s.selfSchemaVerPath, ver, clientv3.WithLease(s.session.Lease())) - metrics.UpdateSelfVersionHistogram.WithLabelValues(metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + metrics.UpdateSelfVersionHistogram.WithLabelValues(ver, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) return errors.Trace(err) } @@ -232,7 +232,7 @@ func (s *schemaVersionSyncer) OwnerUpdateGlobalVersion(ctx context.Context, vers // TODO: If the version is larger than the original global version, we need set the version. // Otherwise, we'd better set the original global version. err := PutKVToEtcd(ctx, s.etcdCli, putKeyRetryUnlimited, DDLGlobalSchemaVersion, ver) - metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerUpdateGlobalVersion, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerUpdateGlobalVersion, ver, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) return errors.Trace(err) } @@ -267,13 +267,17 @@ func DeleteKeyFromEtcd(key string, etcdCli *clientv3.Client, retryCnt int, timeo // MustGetGlobalVersion implements SchemaSyncer.MustGetGlobalVersion interface. func (s *schemaVersionSyncer) MustGetGlobalVersion(ctx context.Context) (int64, error) { startTime := time.Now() - var err error - var resp *clientv3.GetResponse + var ( + err error + ver int + resp *clientv3.GetResponse + ) failedCnt := 0 intervalCnt := int(time.Second / keyOpRetryInterval) defer func() { - metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerGetGlobalVersion, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + gVer := strconv.FormatInt(int64(ver), 10) + metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerGetGlobalVersion, gVer, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() for { if err != nil { @@ -294,7 +298,6 @@ func (s *schemaVersionSyncer) MustGetGlobalVersion(ctx context.Context) (int64, continue } if len(resp.Kvs) > 0 { - var ver int ver, err = strconv.Atoi(string(resp.Kvs[0].Value)) if err == nil { return int64(ver), nil @@ -322,7 +325,8 @@ func (s *schemaVersionSyncer) OwnerCheckAllVersions(ctx context.Context, latestV var err error defer func() { - metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerGetGlobalVersion, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + ver := strconv.FormatInt(latestVer, 10) + metrics.OwnerHandleSyncerHistogram.WithLabelValues(metrics.OwnerGetGlobalVersion, ver, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() for { if isContextDone(ctx) { diff --git a/metrics/ddl.go b/metrics/ddl.go index 4666690f46609..d404ab6fd2d7e 100644 --- a/metrics/ddl.go +++ b/metrics/ddl.go @@ -63,7 +63,7 @@ var ( Name: "update_self_ver_duration_seconds", Help: "Bucketed histogram of processing time (s) of update self version", Buckets: prometheus.ExponentialBuckets(0.01, 2, 20), - }, []string{LblResult}) + }, []string{LblVersion, LblResult}) OwnerUpdateGlobalVersion = "update_global_version" OwnerGetGlobalVersion = "get_global_version" @@ -75,10 +75,11 @@ var ( Name: "owner_handle_syncer_duration_seconds", Help: "Bucketed histogram of processing time (s) of handle syncer", Buckets: prometheus.ExponentialBuckets(0.01, 2, 20), - }, []string{LblType, LblResult}) + }, []string{LblType, LblVersion, LblResult}) // Metrics for ddl_worker.go. WorkerAddDDLJob = "add_job" + WorkerRunDDLJob = "run_job" WorkerFinishDDLJob = "finish_job" WorkerWaitSchemaChanged = "wait_schema_changed" DDLWorkerHistogram = prometheus.NewHistogramVec( @@ -88,7 +89,7 @@ var ( Name: "worker_operation_duration_seconds", Help: "Bucketed histogram of processing time (s) of ddl worker operations", Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), - }, []string{LblType, LblResult}) + }, []string{LblType, LblAction, LblResult}) CreateDDLInstance = "create_ddl_instance" CreateDDL = "create_ddl" @@ -100,15 +101,12 @@ var ( Name: "worker_operation_total", Help: "Counter of creating ddl/worker and isowner.", }, []string{LblType}) +) - // DDLJobErrCounter is the counter of error occurred in ddl job. - DDLJobErrCounter = prometheus.NewCounter( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "ddl", - Name: "job_error_total", - Help: "Counter of error occurred in ddl job.", - }) +// Label constants. +const ( + LblAction = "action" + LblVersion = "version" ) func init() { @@ -120,5 +118,4 @@ func init() { prometheus.MustRegister(OwnerHandleSyncerHistogram) prometheus.MustRegister(DDLWorkerHistogram) prometheus.MustRegister(DDLCounter) - prometheus.MustRegister(DDLJobErrCounter) } From a481f2a41e64b12cd2cfe14eec9f945eaefcb08c Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 27 Aug 2018 16:39:43 +0800 Subject: [PATCH 05/41] ddl: make TestTableSplit stable (#7487) --- ddl/table_split_test.go | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/ddl/table_split_test.go b/ddl/table_split_test.go index b4a2d67e4b6d3..01a4bcd6def75 100644 --- a/ddl/table_split_test.go +++ b/ddl/table_split_test.go @@ -14,6 +14,9 @@ package ddl_test import ( + "bytes" + "time" + . "github.com/pingcap/check" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/model" @@ -48,8 +51,18 @@ func (s *testDDLTableSplitSuite) TestTableSplit(c *C) { type kvStore interface { GetRegionCache() *tikv.RegionCache } - cache := store.(kvStore).GetRegionCache() - loc, err := cache.LocateKey(tikv.NewBackoffer(context.Background(), 5000), regionStartKey) - c.Assert(err, IsNil) + var loc *tikv.KeyLocation + for i := 0; i < 10; i++ { + cache := store.(kvStore).GetRegionCache() + loc, err = cache.LocateKey(tikv.NewBackoffer(context.Background(), 5000), regionStartKey) + c.Assert(err, IsNil) + + // Region cache may be out of date, so we need to drop this expired region and load it again. + cache.DropRegion(loc.Region) + if bytes.Compare(loc.StartKey, []byte(regionStartKey)) == 0 { + return + } + time.Sleep(3 * time.Millisecond) + } c.Assert(loc.StartKey, BytesEquals, []byte(regionStartKey)) } From 2d1fae8e56f066140b9aba42f04c3a6bb6e41aba Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Mon, 27 Aug 2018 18:08:08 +0800 Subject: [PATCH 06/41] plan: fix insert onduplicate key update (#7406) --- executor/insert_test.go | 14 ++++++++++++++ plan/planbuilder.go | 18 +++++++++++++++++- 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/executor/insert_test.go b/executor/insert_test.go index 6d928ae1420c5..d0cf9d00608dc 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -81,6 +81,20 @@ func (s *testSuite) TestInsertOnDuplicateKey(c *C) { tk.MustQuery(`select * from t1`).Check(testkit.Rows("1 400")) tk.MustExec(`insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2;`) tk.MustQuery(`select * from t1`).Check(testkit.Rows("1 400")) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(k1 bigint, k2 bigint, val bigint, primary key(k1, k2));`) + tk.MustExec(`insert into t (val, k1, k2) values (3, 1, 2);`) + tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 3`)) + tk.MustExec(`insert into t (val, k1, k2) select c, a, b from (select 1 as a, 2 as b, 4 as c) tmp on duplicate key update val = tmp.c;`) + tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 4`)) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(k1 double, k2 double, v double, primary key(k1, k2));`) + tk.MustExec(`insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c;`) + tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 3`)) + tk.MustExec(`insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c;`) + tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 3`)) } func (s *testSuite) TestInsertWrongValueForField(c *C) { diff --git a/plan/planbuilder.go b/plan/planbuilder.go index 1508cb535a5b4..53b6897e064fa 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -1190,7 +1190,23 @@ func (b *planBuilder) buildSelectPlanOfInsert(insert *ast.InsertStmt, insertPlan return errors.Trace(err) } - insertPlan.Schema4OnDuplicate = expression.MergeSchema(insertPlan.tableSchema, insertPlan.SelectPlan.Schema()) + // schema4NewRow is the schema for the newly created data record based on + // the result of the select statement. + schema4NewRow := expression.NewSchema(make([]*expression.Column, len(insertPlan.Table.Cols()))...) + for i, selCol := range insertPlan.SelectPlan.Schema().Columns { + ordinal := affectedValuesCols[i].Offset + schema4NewRow.Columns[ordinal] = &expression.Column{} + *schema4NewRow.Columns[ordinal] = *selCol + + schema4NewRow.Columns[ordinal].RetType = &types.FieldType{} + *schema4NewRow.Columns[ordinal].RetType = affectedValuesCols[i].FieldType + } + for i := range schema4NewRow.Columns { + if schema4NewRow.Columns[i] == nil { + schema4NewRow.Columns[i] = &expression.Column{UniqueID: insertPlan.ctx.GetSessionVars().AllocPlanColumnID()} + } + } + insertPlan.Schema4OnDuplicate = expression.MergeSchema(insertPlan.tableSchema, schema4NewRow) return nil } From 1d0d0a6666d545acdb9b13b60de91f432ec8235e Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 27 Aug 2018 18:46:51 +0800 Subject: [PATCH 07/41] distsql: let analyze use RC level and low priority (#7496) --- distsql/request_builder.go | 2 ++ distsql/request_builder_test.go | 6 +----- executor/analyze.go | 4 ++-- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 5e5261bfb9a61..07899b0158084 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -90,6 +90,8 @@ func (builder *RequestBuilder) SetAnalyzeRequest(ana *tipb.AnalyzeReq) *RequestB builder.Request.StartTs = ana.StartTs builder.Request.Data, builder.err = ana.Marshal() builder.Request.NotFillCache = true + builder.Request.IsolationLevel = kv.RC + builder.Request.Priority = kv.PriorityLow return builder } diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index bfd233318c5d9..10d319b9c3e65 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -19,7 +19,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -490,13 +489,10 @@ func (s *testSuite) TestRequestBuilder5(c *C) { }, } - sv := variable.NewSessionVars() - sv.StmtCtx.Priority = mysql.LowPriority - sv.StmtCtx.NotFillCache = true actual, err := (&RequestBuilder{}).SetKeyRanges(keyRanges). SetAnalyzeRequest(&tipb.AnalyzeReq{}). SetKeepOrder(true). - SetFromSessionVars(sv). + SetConcurrency(15). Build() c.Assert(err, IsNil) expect := &kv.Request{ diff --git a/executor/analyze.go b/executor/analyze.go index 853a004c421c2..06b66ba9a05e5 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -174,8 +174,8 @@ func (e *AnalyzeIndexExec) open() error { kvReq, err := builder.SetIndexRanges(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.idxInfo.ID, ranger.FullRange()). SetAnalyzeRequest(e.analyzePB). SetKeepOrder(true). + SetConcurrency(e.concurrency). Build() - kvReq.Concurrency = e.concurrency ctx := context.TODO() e.result, err = distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars) if err != nil { @@ -289,8 +289,8 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe kvReq, err := builder.SetTableRanges(e.physicalTableID, ranges, nil). SetAnalyzeRequest(e.analyzePB). SetKeepOrder(e.keepOrder). + SetConcurrency(e.concurrency). Build() - kvReq.Concurrency = e.concurrency if err != nil { return nil, errors.Trace(err) } From 38c939f4f95c802d85a5b6fd098a7a33daf0937f Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 27 Aug 2018 21:52:04 +0800 Subject: [PATCH 08/41] *: add comment and fix lint tool (#7503) --- expression/simple_rewriter.go | 3 +++ revive.toml | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/expression/simple_rewriter.go b/expression/simple_rewriter.go index 6d752fca78ff6..f5d7386e74f54 100644 --- a/expression/simple_rewriter.go +++ b/expression/simple_rewriter.go @@ -56,6 +56,8 @@ func RewriteSimpleExprWithTableInfo(ctx sessionctx.Context, tbl *model.TableInfo return rewriter.pop(), nil } +// ParseSimpleExprsWithSchema parses simple expression string to Expression. +// The expression string must only reference the column in the given schema. func ParseSimpleExprsWithSchema(ctx sessionctx.Context, exprStr string, schema *Schema) ([]Expression, error) { exprStr = "select " + exprStr stmts, err := parser.New().Parse(exprStr, "", "") @@ -74,6 +76,7 @@ func ParseSimpleExprsWithSchema(ctx sessionctx.Context, exprStr string, schema * return exprs, nil } +// RewriteSimpleExprWithSchema rewrites simple ast.ExprNode to expression.Expression. func RewriteSimpleExprWithSchema(ctx sessionctx.Context, expr ast.ExprNode, schema *Schema) (Expression, error) { rewriter := &simpleRewriter{ctx: ctx, schema: schema} expr.Accept(rewriter) diff --git a/revive.toml b/revive.toml index 4cf298a8cbd42..19193c23cd1b4 100644 --- a/revive.toml +++ b/revive.toml @@ -1,8 +1,8 @@ ignoreGeneratedHeader = false severity = "error" confidence = 0.8 -errorCode = 0 -warningCode = 0 +errorCode = -1 +warningCode = -1 [rule.blank-imports] [rule.context-as-argument] From 55565f1c938f47ab09bae8b04dc4edc29ce4d8e3 Mon Sep 17 00:00:00 2001 From: xiaojian cai Date: Tue, 28 Aug 2018 11:24:48 +0800 Subject: [PATCH 09/41] expression: fix out of range error for intdiv (#7492) --- expression/builtin_arithmetic.go | 36 ++++++++++++++++++++++---------- expression/errors.go | 2 ++ expression/integration_test.go | 7 +++++++ 3 files changed, 34 insertions(+), 11 deletions(-) diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index 93cd217986ab8..4e7117b637e10 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -729,27 +729,41 @@ func (s *builtinArithmeticIntDivideIntSig) evalInt(row chunk.Row) (int64, bool, return ret, err != nil, errors.Trace(err) } -func (s *builtinArithmeticIntDivideDecimalSig) evalInt(row chunk.Row) (int64, bool, error) { - a, isNull, err := s.args[0].EvalDecimal(s.ctx, row) - if isNull || err != nil { - return 0, isNull, errors.Trace(err) - } - - b, isNull, err := s.args[1].EvalDecimal(s.ctx, row) - if isNull || err != nil { - return 0, isNull, errors.Trace(err) +func (s *builtinArithmeticIntDivideDecimalSig) evalInt(row chunk.Row) (ret int64, isNull bool, err error) { + sc := s.ctx.GetSessionVars().StmtCtx + var num [2]*types.MyDecimal + for i, arg := range s.args { + num[i], isNull, err = arg.EvalDecimal(s.ctx, row) + // Its behavior is consistent with MySQL. + if terror.ErrorEqual(err, types.ErrTruncated) { + err = nil + } + if terror.ErrorEqual(err, types.ErrOverflow) { + newErr := errTruncatedWrongValue.GenByArgs("DECIMAL", arg) + err = sc.HandleOverflow(newErr, newErr) + } + if isNull || err != nil { + return 0, isNull, errors.Trace(err) + } } c := &types.MyDecimal{} - err = types.DecimalDiv(a, b, c, types.DivFracIncr) + err = types.DecimalDiv(num[0], num[1], c, types.DivFracIncr) if err == types.ErrDivByZero { return 0, true, errors.Trace(handleDivisionByZeroError(s.ctx)) } + if err == types.ErrTruncated { + err = sc.HandleTruncate(errTruncatedWrongValue.GenByArgs("DECIMAL", c)) + } + if err == types.ErrOverflow { + newErr := errTruncatedWrongValue.GenByArgs("DECIMAL", c) + err = sc.HandleOverflow(newErr, newErr) + } if err != nil { return 0, true, errors.Trace(err) } - ret, err := c.ToInt() + ret, err = c.ToInt() // err returned by ToInt may be ErrTruncated or ErrOverflow, only handle ErrOverflow, ignore ErrTruncated. if err == types.ErrOverflow { return 0, true, types.ErrOverflow.GenByArgs("BIGINT", fmt.Sprintf("(%s DIV %s)", s.args[0].String(), s.args[1].String())) diff --git a/expression/errors.go b/expression/errors.go index f342bf42916ae..92be6e1487054 100644 --- a/expression/errors.go +++ b/expression/errors.go @@ -38,6 +38,7 @@ var ( errDeprecatedSyntaxNoReplacement = terror.ClassExpression.New(mysql.ErrWarnDeprecatedSyntaxNoReplacement, mysql.MySQLErrName[mysql.ErrWarnDeprecatedSyntaxNoReplacement]) errBadField = terror.ClassExpression.New(mysql.ErrBadField, mysql.MySQLErrName[mysql.ErrBadField]) errWarnAllowedPacketOverflowed = terror.ClassExpression.New(mysql.ErrWarnAllowedPacketOverflowed, mysql.MySQLErrName[mysql.ErrWarnAllowedPacketOverflowed]) + errTruncatedWrongValue = terror.ClassExpression.New(mysql.ErrTruncatedWrongValue, mysql.MySQLErrName[mysql.ErrTruncatedWrongValue]) ) func init() { @@ -53,6 +54,7 @@ func init() { mysql.ErrOperandColumns: mysql.ErrOperandColumns, mysql.ErrRegexp: mysql.ErrRegexp, mysql.ErrWarnAllowedPacketOverflowed: mysql.ErrWarnAllowedPacketOverflowed, + mysql.ErrTruncatedWrongValue: mysql.ErrTruncatedWrongValue, } terror.ErrClassToMySQLCodes[terror.ClassExpression] = expressionMySQLErrCodes } diff --git a/expression/integration_test.go b/expression/integration_test.go index 20369d6038ab3..c37b478e5453e 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -2515,6 +2515,13 @@ func (s *testIntegrationSuite) TestArithmeticBuiltin(c *C) { result.Check(testkit.Rows("1 1300 -6 ")) result = tk.MustQuery("SELECT 2.4 div 1.1, 2.4 div 1.2, 2.4 div 1.3;") result.Check(testkit.Rows("2 2 1")) + result = tk.MustQuery("SELECT 1.175494351E-37 div 1.7976931348623157E+308, 1.7976931348623157E+308 div -1.7976931348623157E+307, 1 div 1e-82;") + result.Check(testkit.Rows("0 -1 ")) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", + "Warning|1292|Truncated incorrect DECIMAL value: 'cast(1.7976931348623157e+308)'", + "Warning|1292|Truncated incorrect DECIMAL value: 'cast(1.7976931348623157e+308)'", + "Warning|1292|Truncated incorrect DECIMAL value: 'cast(-1.7976931348623158e+307)'", + "Warning|1365|Division by 0")) rs, err = tk.Exec("select 1e300 DIV 1.5") c.Assert(err, IsNil) _, err = session.GetRows4Test(ctx, tk.Se, rs) From c97c81e180b59dcdc59074c6b1abd37b3fd5894e Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Tue, 28 Aug 2018 12:13:08 +0800 Subject: [PATCH 10/41] tidb-server: enable mutex profiling (#7510) --- tidb-server/main.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tidb-server/main.go b/tidb-server/main.go index 6e9d9072e3414..57b0941338983 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -455,6 +455,8 @@ func setupSignalHandler() { } func setupMetrics() { + // Enable the mutex profile, 1/10 of mutex blocking event sampling. + runtime.SetMutexProfileFraction(10) systimeErrHandler := func() { metrics.TimeJumpBackCounter.Inc() } From bc97e2d6163f18f5ceab4a718a90fca1f762ce11 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 28 Aug 2018 12:42:55 +0800 Subject: [PATCH 11/41] plan, util/admin: fix admin check table error when a column of index is virtual generated (#6817) --- executor/admin_test.go | 5 ++ executor/builder.go | 1 + executor/executor.go | 5 +- model/model.go | 6 +++ plan/common_plans.go | 2 + plan/planbuilder.go | 49 ++++++++++++++++-- util/admin/admin.go | 107 ++++++++++++++++++++++++++++++++++----- util/admin/admin_test.go | 12 ++--- 8 files changed, 162 insertions(+), 25 deletions(-) diff --git a/executor/admin_test.go b/executor/admin_test.go index 87f62784a3676..427aecdf6ce26 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -462,6 +462,11 @@ func (s *testSuite) TestAdminCheckTable(c *C) { } tk.MustExec(`admin check table test;`) + // Test index in virtual generated column. + tk.MustExec(`drop table if exists test`) + tk.MustExec(`create table test ( b json , c int as (JSON_EXTRACT(b,'$.d')) , index idxc(c));`) + tk.MustExec(`INSERT INTO test set b='{"d": 100}';`) + tk.MustExec(`admin check table test;`) // Test prefix index. tk.MustExec(`drop table if exists t`) tk.MustExec(`CREATE TABLE t ( diff --git a/executor/builder.go b/executor/builder.go index 5056b236c5f0e..1992791ba99c8 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -248,6 +248,7 @@ func (b *executorBuilder) buildCheckTable(v *plan.CheckTable) Executor { baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), tables: v.Tables, is: b.is, + genExprs: v.GenExprs, } return e } diff --git a/executor/executor.go b/executor/executor.go index 552e7a3a7ba7d..7932c0a90a2aa 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -352,6 +352,8 @@ type CheckTableExec struct { tables []*ast.TableName done bool is infoschema.InfoSchema + + genExprs map[string]expression.Expression } // Open implements the Executor Open interface. @@ -375,7 +377,6 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { if err != nil { return errors.Trace(err) } - if tb.Meta().GetPartitionInfo() != nil { err = e.doCheckPartitionedTable(tb.(table.PartitionedTable)) } else { @@ -408,7 +409,7 @@ func (e *CheckTableExec) doCheckPartitionedTable(tbl table.PartitionedTable) err func (e *CheckTableExec) doCheckTable(tbl table.Table) error { for _, idx := range tbl.Indices() { txn := e.ctx.Txn() - err := admin.CompareIndexData(e.ctx, txn, tbl, idx) + err := admin.CompareIndexData(e.ctx, txn, tbl, idx, e.genExprs) if err != nil { return errors.Trace(err) } diff --git a/model/model.go b/model/model.go index a7a641234a2ee..bc03eac4d817b 100644 --- a/model/model.go +++ b/model/model.go @@ -15,6 +15,7 @@ package model import ( "encoding/json" + "fmt" "strings" "time" @@ -522,3 +523,8 @@ func collationToProto(c string) int32 { // For the data created when we didn't enforce utf8_bin collation in create table. return int32(mysql.DefaultCollationID) } + +// GetTableColumnID gets a ID of a column with table ID +func GetTableColumnID(tableInfo *TableInfo, col *ColumnInfo) string { + return fmt.Sprintf("%d_%d", tableInfo.ID, col.ID) +} diff --git a/plan/common_plans.go b/plan/common_plans.go index 15a351197bfb8..fafd419a3b62d 100644 --- a/plan/common_plans.go +++ b/plan/common_plans.go @@ -56,6 +56,8 @@ type CheckTable struct { baseSchemaProducer Tables []*ast.TableName + + GenExprs map[string]expression.Expression } // RecoverIndex is used for backfilling corrupted index data. diff --git a/plan/planbuilder.go b/plan/planbuilder.go index 53b6897e064fa..ed0e45b36779a 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -441,11 +441,13 @@ func (b *planBuilder) buildCheckIndex(dbName model.CIStr, as *ast.AdminStmt) (Pl func (b *planBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { var ret Plan - + var err error switch as.Tp { case ast.AdminCheckTable: - p := &CheckTable{Tables: as.Tables} - ret = p + ret, err = b.buildAdminCheckTable(as) + if err != nil { + return ret, errors.Trace(err) + } case ast.AdminCheckIndex: dbName := as.Tables[0].Schema readerPlan, err := b.buildCheckIndex(dbName, as) @@ -504,6 +506,47 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) { return ret, nil } +func (b *planBuilder) buildAdminCheckTable(as *ast.AdminStmt) (*CheckTable, error) { + p := &CheckTable{Tables: as.Tables} + p.GenExprs = make(map[string]expression.Expression) + + mockTablePlan := LogicalTableDual{}.init(b.ctx) + for _, tbl := range p.Tables { + tableInfo := tbl.TableInfo + schema := expression.TableInfo2SchemaWithDBName(b.ctx, tbl.Schema, tableInfo) + table, ok := b.is.TableByID(tableInfo.ID) + if !ok { + return nil, infoschema.ErrTableNotExists.GenByArgs(tbl.DBInfo.Name.O, tableInfo.Name.O) + } + + mockTablePlan.SetSchema(schema) + + // Calculate generated columns. + columns := table.Cols() + for _, column := range columns { + if !column.IsGenerated() { + continue + } + columnName := &ast.ColumnName{Name: column.Name} + columnName.SetText(column.Name.O) + + colExpr, _, err := mockTablePlan.findColumn(columnName) + if err != nil { + return nil, errors.Trace(err) + } + + expr, _, err := b.rewrite(column.GeneratedExpr, mockTablePlan, nil, true) + if err != nil { + return nil, errors.Trace(err) + } + expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType()) + genColumnName := model.GetTableColumnID(tableInfo, column.ColumnInfo) + p.GenExprs[genColumnName] = expr + } + } + return p, nil +} + func (b *planBuilder) buildCheckIndexSchema(tn *ast.TableName, indexName string) (*expression.Schema, error) { schema := expression.NewSchema() indexName = strings.ToLower(indexName) diff --git a/util/admin/admin.go b/util/admin/admin.go index c25cb27d8c2eb..97a0e6d2dac3d 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -20,6 +20,7 @@ import ( "sort" "github.com/juju/errors" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/model" @@ -32,6 +33,7 @@ import ( "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" log "github.com/sirupsen/logrus" ) @@ -292,13 +294,14 @@ func ScanIndexData(sc *stmtctx.StatementContext, txn kv.Transaction, kvIndex tab // CompareIndexData compares index data one by one. // It returns nil if the data from the index is equal to the data from the table columns, // otherwise it returns an error with a different set of records. -func CompareIndexData(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error { - err := checkIndexAndRecord(sessCtx, txn, t, idx) +// genExprs is use to calculate the virtual generate column. +func CompareIndexData(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index, genExprs map[string]expression.Expression) error { + err := checkIndexAndRecord(sessCtx, txn, t, idx, genExprs) if err != nil { return errors.Trace(err) } - return CheckRecordAndIndex(sessCtx, txn, t, idx) + return CheckRecordAndIndex(sessCtx, txn, t, idx, genExprs) } func getIndexFieldTypes(t table.Table, idx table.Index) ([]*types.FieldType, error) { @@ -334,7 +337,7 @@ func adjustDatumKind(vals1, vals2 []types.Datum) { } } -func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error { +func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index, genExprs map[string]expression.Expression) error { it, err := idx.SeekFirst(txn) if err != nil { return errors.Trace(err) @@ -362,7 +365,7 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table if err != nil { return errors.Trace(err) } - vals2, err := rowWithCols(sessCtx, txn, t, h, cols) + vals2, err := rowWithCols(sessCtx, txn, t, h, cols, genExprs) vals2 = tables.TruncateIndexValuesIfNeeded(t.Meta(), idx.Meta(), vals2) if kv.ErrNotExist.Equal(err) { record := &RecordData{Handle: h, Values: vals1} @@ -383,7 +386,7 @@ func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table } // CheckRecordAndIndex is exported for testing. -func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error { +func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index, genExprs map[string]expression.Expression) error { sc := sessCtx.GetSessionVars().StmtCtx cols := make([]*table.Column, len(idx.Meta().Columns)) for i, col := range idx.Meta().Columns { @@ -422,7 +425,7 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table return true, nil } - err := iterRecords(sessCtx, txn, t, startKey, cols, filterFunc) + err := iterRecords(sessCtx, txn, t, startKey, cols, filterFunc, genExprs) if err != nil { return errors.Trace(err) @@ -446,10 +449,9 @@ func scanTableData(sessCtx sessionctx.Context, retriever kv.Retriever, t table.T limit-- return true, nil } - return false, nil } - err := iterRecords(sessCtx, retriever, t, startKey, cols, filterFunc) + err := iterRecords(sessCtx, retriever, t, startKey, cols, filterFunc, nil) if err != nil { return nil, 0, errors.Trace(err) } @@ -522,7 +524,7 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table. return true, nil } - err := iterRecords(sessCtx, txn, t, startKey, t.Cols(), filterFunc) + err := iterRecords(sessCtx, txn, t, startKey, t.Cols(), filterFunc, nil) if err != nil { return errors.Trace(err) } @@ -535,9 +537,11 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table. return nil } -func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h int64, cols []*table.Column) ([]types.Datum, error) { +// genExprs use to calculate generated column value. +func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h int64, cols []*table.Column, genExprs map[string]expression.Expression) ([]types.Datum, error) { key := t.RecordKey(h) value, err := txn.Get(key) + genColFlag := false if err != nil { return nil, errors.Trace(err) } @@ -558,12 +562,34 @@ func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h } continue } + // If have virtual generate column , decode all columns. + if col.IsGenerated() && col.GeneratedStored == false { + genColFlag = true + } colTps[col.ID] = &col.FieldType } - row, err := tablecodec.DecodeRow(value, colTps, sessCtx.GetSessionVars().Location()) + // if have virtual generate column, decode all columns + if genColFlag { + for _, c := range t.Cols() { + if c.State != model.StatePublic { + continue + } + colTps[c.ID] = &c.FieldType + } + } + + rowMap, err := tablecodec.DecodeRow(value, colTps, sessCtx.GetSessionVars().Location()) if err != nil { return nil, errors.Trace(err) } + + if genColFlag && genExprs != nil { + err = fillGenColData(sessCtx, rowMap, t, cols, genExprs) + if err != nil { + return v, errors.Trace(err) + } + } + for i, col := range cols { if col == nil { continue @@ -575,7 +601,7 @@ func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h if col.IsPKHandleColumn(t.Meta()) { continue } - ri, ok := row[col.ID] + ri, ok := rowMap[col.ID] if !ok { if mysql.HasNotNullFlag(col.Flag) { return nil, errors.New("Miss") @@ -593,8 +619,9 @@ func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h return v, nil } +// genExprs use to calculate generated column value. func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Table, startKey kv.Key, cols []*table.Column, - fn table.RecordIterFunc) error { + fn table.RecordIterFunc, genExprs map[string]expression.Expression) error { it, err := retriever.Seek(startKey) if err != nil { return errors.Trace(err) @@ -607,10 +634,21 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab log.Debugf("startKey:%q, key:%q, value:%q", startKey, it.Key(), it.Value()) + genColFlag := false colMap := make(map[int64]*types.FieldType, len(cols)) for _, col := range cols { + if col.IsGenerated() && col.GeneratedStored == false { + genColFlag = true + break + } colMap[col.ID] = &col.FieldType } + if genColFlag { + for _, col := range t.Cols() { + colMap[col.ID] = &col.FieldType + } + } + prefix := t.RecordPrefix() for it.Valid() && it.Key().HasPrefix(prefix) { // first kv pair is row lock information. @@ -625,6 +663,14 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab if err != nil { return errors.Trace(err) } + + if genColFlag && genExprs != nil { + err = fillGenColData(sessCtx, rowMap, t, cols, genExprs) + if err != nil { + return errors.Trace(err) + } + } + data := make([]types.Datum, 0, len(cols)) for _, col := range cols { if col.IsPKHandleColumn(t.Meta()) { @@ -652,6 +698,39 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab return nil } +// genExprs use to calculate generated column value. +func fillGenColData(sessCtx sessionctx.Context, rowMap map[int64]types.Datum, t table.Table, cols []*table.Column, genExprs map[string]expression.Expression) error { + tableInfo := t.Meta() + row := make([]types.Datum, len(t.Cols())) + for _, col := range t.Cols() { + ri, ok := rowMap[col.ID] + if ok { + row[col.Offset] = ri + } + } + + var err error + for _, col := range cols { + if !col.IsGenerated() || col.GeneratedStored == true { + continue + } + genColumnName := model.GetTableColumnID(tableInfo, col.ColumnInfo) + if expr, ok := genExprs[genColumnName]; ok { + var val types.Datum + val, err = expr.Eval(chunk.MutRowFromDatums(row).ToRow()) + if err != nil { + return errors.Trace(err) + } + val, err = table.CastValue(sessCtx, val, col.ToInfo()) + if err != nil { + return errors.Trace(err) + } + rowMap[col.ID] = val + } + } + return nil +} + // admin error codes. const ( codeDataNotEqual terror.ErrCode = 1 diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index ed58353ac894b..1e88fe08a67bc 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -414,7 +414,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta txn, err := s.store.Begin() c.Assert(err, IsNil) sc := &stmtctx.StatementContext{TimeZone: time.Local} - err = CompareIndexData(ctx, txn, tb, idx) + err = CompareIndexData(ctx, txn, tb, idx, nil) c.Assert(err, IsNil) idxNames := []string{idx.Meta().Name.L} @@ -434,7 +434,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta txn, err = s.store.Begin() c.Assert(err, IsNil) - err = CompareIndexData(ctx, txn, tb, idx) + err = CompareIndexData(ctx, txn, tb, idx, nil) c.Assert(err, NotNil) record1 := &RecordData{Handle: int64(3), Values: types.MakeDatums(int64(30))} diffMsg := newDiffRetError("index", record1, nil) @@ -455,7 +455,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta txn, err = s.store.Begin() c.Assert(err, IsNil) - err = CompareIndexData(ctx, txn, tb, idx) + err = CompareIndexData(ctx, txn, tb, idx, nil) c.Assert(err, NotNil) record2 := &RecordData{Handle: int64(3), Values: types.MakeDatums(int64(31))} diffMsg = newDiffRetError("index", record1, record2) @@ -473,7 +473,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta txn, err = s.store.Begin() c.Assert(err, IsNil) - err = CheckRecordAndIndex(ctx, txn, tb, idx) + err = CheckRecordAndIndex(ctx, txn, tb, idx, nil) c.Assert(err, NotNil) record2 = &RecordData{Handle: int64(5), Values: types.MakeDatums(int64(30))} diffMsg = newDiffRetError("index", record1, record2) @@ -493,7 +493,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta txn, err = s.store.Begin() c.Assert(err, IsNil) - err = CompareIndexData(ctx, txn, tb, idx) + err = CompareIndexData(ctx, txn, tb, idx, nil) c.Assert(err, NotNil) record1 = &RecordData{Handle: int64(4), Values: types.MakeDatums(int64(40))} diffMsg = newDiffRetError("index", record1, nil) @@ -514,7 +514,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta txn, err = s.store.Begin() c.Assert(err, IsNil) - err = CompareIndexData(ctx, txn, tb, idx) + err = CompareIndexData(ctx, txn, tb, idx, nil) c.Assert(err, NotNil) diffMsg = newDiffRetError("index", nil, record1) c.Assert(err.Error(), DeepEquals, diffMsg) From 5b689e8c5bd43af8692a5f18b68a0e05b164b648 Mon Sep 17 00:00:00 2001 From: lysu Date: Tue, 28 Aug 2018 13:36:36 +0800 Subject: [PATCH 12/41] server: fix insert zero timestamp bug with prepared statement (#7506) --- server/conn_stmt.go | 3 ++- server/conn_stmt_test.go | 3 ++- session/session_test.go | 13 +++++++++++++ 3 files changed, 17 insertions(+), 2 deletions(-) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 9ab74ec2cb72e..d874d8ea3d182 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -42,6 +42,7 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/hack" "golang.org/x/net/context" ) @@ -354,7 +355,7 @@ func parseStmtArgs(args []interface{}, boundParams [][]byte, nullBitmap, paramTy pos++ switch length { case 0: - args[i] = "0" + args[i] = types.ZeroDatetimeStr case 4: pos, args[i] = parseBinaryDate(pos, paramValues) case 7: diff --git a/server/conn_stmt_test.go b/server/conn_stmt_test.go index fdfce13a338fe..d131e39d6f22c 100644 --- a/server/conn_stmt_test.go +++ b/server/conn_stmt_test.go @@ -17,6 +17,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/terror" + "github.com/pingcap/tidb/types" ) func (ts ConnTestSuite) TestParseStmtArgs(c *C) { @@ -120,7 +121,7 @@ func (ts ConnTestSuite) TestParseStmtArgs(c *C) { []byte{0x00}, }, nil, - "0", + types.ZeroDatetimeStr, }, // Tests for time { diff --git a/session/session_test.go b/session/session_test.go index 88f921afc7391..d95d7e1a58032 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -630,6 +630,19 @@ func (s *testSessionSuite) TestLastInsertID(c *C) { c.Assert(lastInsertID+2, Equals, currLastInsertID) } +func (s *testSessionSuite) TestPrepareZero(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(v timestamp)") + tk.MustExec("prepare s1 from 'insert into t (v) values (?)'") + tk.MustExec("set @v1='0'") + _, rs := tk.Exec("execute s1 using @v1") + c.Assert(rs, NotNil) + tk.MustExec("set @v2='" + types.ZeroDatetimeStr + "'") + tk.MustExec("execute s1 using @v2") + tk.MustQuery("select v from t").Check(testkit.Rows("0000-00-00 00:00:00")) +} + func (s *testSessionSuite) TestPrimaryKeyAutoIncrement(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t") From 364cd97ea37ff399774404b61431d43cc94a779f Mon Sep 17 00:00:00 2001 From: disksing Date: Tue, 28 Aug 2018 15:01:13 +0800 Subject: [PATCH 13/41] *: update pd and kvproto vendor (#7513) --- Gopkg.lock | 35 +- store/mockstore/mocktikv/pd.go | 8 + store/tikv/store_test.go | 8 + .../github.com/etcd-io/gofail/code/binding.go | 52 - .../etcd-io/gofail/code/failpoint.go | 95 - .../github.com/etcd-io/gofail/code/rewrite.go | 167 - vendor/github.com/etcd-io/gofail/gofail.go | 195 -- vendor/github.com/gogo/protobuf/LICENSE | 36 - .../github.com/gogo/protobuf/proto/clone.go | 234 -- .../github.com/gogo/protobuf/proto/decode.go | 978 ------ .../gogo/protobuf/proto/decode_gogo.go | 172 - .../github.com/gogo/protobuf/proto/discard.go | 151 - .../gogo/protobuf/proto/duration.go | 100 - .../gogo/protobuf/proto/duration_gogo.go | 203 -- .../github.com/gogo/protobuf/proto/encode.go | 1362 -------- .../gogo/protobuf/proto/encode_gogo.go | 350 --- .../github.com/gogo/protobuf/proto/equal.go | 300 -- .../gogo/protobuf/proto/extensions.go | 693 ----- .../gogo/protobuf/proto/extensions_gogo.go | 294 -- vendor/github.com/gogo/protobuf/proto/lib.go | 897 ------ .../gogo/protobuf/proto/lib_gogo.go | 42 - .../gogo/protobuf/proto/message_set.go | 311 -- .../gogo/protobuf/proto/pointer_reflect.go | 484 --- .../protobuf/proto/pointer_reflect_gogo.go | 85 - .../gogo/protobuf/proto/pointer_unsafe.go | 270 -- .../protobuf/proto/pointer_unsafe_gogo.go | 128 - .../gogo/protobuf/proto/properties.go | 971 ------ .../gogo/protobuf/proto/properties_gogo.go | 111 - .../gogo/protobuf/proto/skip_gogo.go | 119 - vendor/github.com/gogo/protobuf/proto/text.go | 939 ------ .../gogo/protobuf/proto/text_gogo.go | 57 - .../gogo/protobuf/proto/text_parser.go | 1013 ------ .../gogo/protobuf/proto/timestamp.go | 113 - .../gogo/protobuf/proto/timestamp_gogo.go | 229 -- .../opentracing/basictracer-go/LICENSE | 21 - .../opentracing/basictracer-go/context.go | 42 - .../opentracing/basictracer-go/debug.go | 78 - .../opentracing/basictracer-go/event.go | 62 - .../opentracing/basictracer-go/propagation.go | 61 - .../basictracer-go/propagation_ot.go | 180 -- .../opentracing/basictracer-go/raw.go | 34 - .../opentracing/basictracer-go/recorder.go | 60 - .../opentracing/basictracer-go/span.go | 274 -- .../opentracing/basictracer-go/tracer.go | 262 -- .../opentracing/basictracer-go/util.go | 25 - .../basictracer-go/wire/carrier.go | 40 - .../opentracing/basictracer-go/wire/gen.go | 6 - .../basictracer-go/wire/wire.pb.go | 508 --- vendor/github.com/pingcap/kvproto/LICENSE | 201 ++ .../pingcap/kvproto/pkg/eraftpb/eraftpb.pb.go | 150 +- .../pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go | 316 +- .../pingcap/kvproto/pkg/metapb/metapb.pb.go | 109 +- .../pingcap/kvproto/pkg/pdpb/pdpb.pb.go | 2755 ++++++++++++++--- .../github.com/pingcap/pd/pd-client/client.go | 143 +- .../pingcap/tipb/go-tipb/expression.pb.go | 547 ++-- vendor/github.com/pkg/errors/LICENSE | 23 + vendor/github.com/pkg/errors/errors.go | 269 ++ vendor/github.com/pkg/errors/stack.go | 178 ++ 58 files changed, 3788 insertions(+), 13758 deletions(-) delete mode 100644 vendor/github.com/etcd-io/gofail/code/binding.go delete mode 100644 vendor/github.com/etcd-io/gofail/code/failpoint.go delete mode 100644 vendor/github.com/etcd-io/gofail/code/rewrite.go delete mode 100644 vendor/github.com/etcd-io/gofail/gofail.go delete mode 100644 vendor/github.com/gogo/protobuf/LICENSE delete mode 100644 vendor/github.com/gogo/protobuf/proto/clone.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/decode.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/decode_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/discard.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/duration.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/duration_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/encode.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/encode_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/equal.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/extensions.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/extensions_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/lib.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/lib_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/message_set.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/pointer_reflect.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/pointer_reflect_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/pointer_unsafe.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/pointer_unsafe_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/properties.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/properties_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/skip_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/text.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/text_gogo.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/text_parser.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/timestamp.go delete mode 100644 vendor/github.com/gogo/protobuf/proto/timestamp_gogo.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/LICENSE delete mode 100644 vendor/github.com/opentracing/basictracer-go/context.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/debug.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/event.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/propagation.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/propagation_ot.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/raw.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/recorder.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/span.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/tracer.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/util.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/wire/carrier.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/wire/gen.go delete mode 100644 vendor/github.com/opentracing/basictracer-go/wire/wire.pb.go create mode 100644 vendor/github.com/pingcap/kvproto/LICENSE create mode 100644 vendor/github.com/pkg/errors/LICENSE create mode 100644 vendor/github.com/pkg/errors/errors.go create mode 100644 vendor/github.com/pkg/errors/stack.go diff --git a/Gopkg.lock b/Gopkg.lock index 52d6fb862b56f..2a9b55205d3d3 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -78,11 +78,7 @@ [[projects]] branch = "master" name = "github.com/etcd-io/gofail" - packages = [ - ".", - "code", - "runtime" - ] + packages = ["runtime"] revision = "51ce9a71510a58bad5ae66ddd278ef28762a1550" [[projects]] @@ -90,12 +86,6 @@ packages = ["."] revision = "3955978caca48c1658a4bb7a9c6a0f084e326af3" -[[projects]] - name = "github.com/gogo/protobuf" - packages = ["proto"] - revision = "1adfc126b41513cc696b209667c8656ea7aac67c" - version = "v1.0.0" - [[projects]] name = "github.com/golang/protobuf" packages = [ @@ -172,15 +162,6 @@ packages = ["."] revision = "7a24ed77b2efb460c1468b7dc917821c66e80e55" -[[projects]] - name = "github.com/opentracing/basictracer-go" - packages = [ - ".", - "wire" - ] - revision = "1b32af207119a14b1b231d451df3ed04a72efebf" - version = "v1.0.0" - [[projects]] name = "github.com/opentracing/opentracing-go" packages = [ @@ -234,13 +215,13 @@ "pkg/raft_serverpb", "pkg/tikvpb" ] - revision = "5ef8fa297ea3547b015ef5069cb500bba67b0d94" + revision = "279515615485b0f2d12f1421cc412fe2784e0190" [[projects]] branch = "master" name = "github.com/pingcap/pd" packages = ["pd-client"] - revision = "ea6597638eb7f3d7d50248b90e60888352ac8e85" + revision = "47a950e6649fa0151a6c5d0fc48a97c080012e3d" [[projects]] branch = "master" @@ -260,7 +241,13 @@ "go-tipb", "sharedbytes" ] - revision = "015341a07dc30de16a7cca0fc77231ccacc4b397" + revision = "5e46e0ec647edfee8fe001c5b51609460fb655bf" + +[[projects]] + name = "github.com/pkg/errors" + packages = ["."] + revision = "645ef00459ed84a119197bfb8d8205042c6df63d" + version = "v0.8.0" [[projects]] name = "github.com/prometheus/client_golang" @@ -424,6 +411,6 @@ [solve-meta] analyzer-name = "dep" analyzer-version = 1 - inputs-digest = "472d8f62cb1733b34466708b19eaa0f7d80c01045ad4de7d790cfc34797c991a" + inputs-digest = "9c66d4c8409973a2286ef3018c5de331324953b79c1e323881a8caa5d2a21d2c" solver-name = "gps-cdcl" solver-version = 1 diff --git a/store/mockstore/mocktikv/pd.go b/store/mockstore/mocktikv/pd.go index 1a15a64c78982..60075d1e28207 100644 --- a/store/mockstore/mocktikv/pd.go +++ b/store/mockstore/mocktikv/pd.go @@ -77,6 +77,10 @@ func (c *pdClient) GetRegion(ctx context.Context, key []byte) (*metapb.Region, * return region, peer, nil } +func (c *pdClient) GetPrevRegion(context.Context, []byte) (*metapb.Region, *metapb.Peer, error) { + panic("unimplemented") +} + func (c *pdClient) GetRegionByID(ctx context.Context, regionID uint64) (*metapb.Region, *metapb.Peer, error) { region, peer := c.cluster.GetRegionByID(regionID) return region, peer, nil @@ -92,5 +96,9 @@ func (c *pdClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, return store, nil } +func (c *pdClient) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) { + panic("unimplemented") +} + func (c *pdClient) Close() { } diff --git a/store/tikv/store_test.go b/store/tikv/store_test.go index c4f7b60521b9f..1d466dc7055f8 100644 --- a/store/tikv/store_test.go +++ b/store/tikv/store_test.go @@ -138,6 +138,10 @@ func (c *mockPDClient) GetRegion(ctx context.Context, key []byte) (*metapb.Regio return c.client.GetRegion(ctx, key) } +func (c *mockPDClient) GetPrevRegion(context.Context, []byte) (*metapb.Region, *metapb.Peer, error) { + panic("unimplemented") +} + func (c *mockPDClient) GetRegionByID(ctx context.Context, regionID uint64) (*metapb.Region, *metapb.Peer, error) { c.RLock() defer c.RUnlock() @@ -158,6 +162,10 @@ func (c *mockPDClient) GetStore(ctx context.Context, storeID uint64) (*metapb.St return c.client.GetStore(ctx, storeID) } +func (c *mockPDClient) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) { + panic("unimplemented") +} + func (c *mockPDClient) Close() {} type checkRequestClient struct { diff --git a/vendor/github.com/etcd-io/gofail/code/binding.go b/vendor/github.com/etcd-io/gofail/code/binding.go deleted file mode 100644 index eb595b5f4202e..0000000000000 --- a/vendor/github.com/etcd-io/gofail/code/binding.go +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright 2016 CoreOS, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package code - -import ( - "fmt" - "io" -) - -type Binding struct { - pkg string - fppath string - fps []*Failpoint -} - -func NewBinding(pkg string, fppath string, fps []*Failpoint) *Binding { - return &Binding{pkg, fppath, fps} -} - -// Write writes the fp.generated.go file for a package. -func (b *Binding) Write(dst io.Writer) error { - hdr := "// GENERATED BY GOFAIL. DO NOT EDIT.\n\n" + - "package " + b.pkg + - "\n\nimport \"github.com/etcd-io/gofail/runtime\"\n\n" - if _, err := fmt.Fprint(dst, hdr); err != nil { - return err - } - for _, fp := range b.fps { - _, err := fmt.Fprintf( - dst, - "var %s *runtime.Failpoint = runtime.NewFailpoint(%q, %q)\n", - fp.Runtime(), - b.fppath, - fp.Name()) - if err != nil { - return err - } - } - return nil -} diff --git a/vendor/github.com/etcd-io/gofail/code/failpoint.go b/vendor/github.com/etcd-io/gofail/code/failpoint.go deleted file mode 100644 index 0992ea0281fce..0000000000000 --- a/vendor/github.com/etcd-io/gofail/code/failpoint.go +++ /dev/null @@ -1,95 +0,0 @@ -// Copyright 2016 CoreOS, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package code - -import ( - "fmt" - "io" - "strings" -) - -type Failpoint struct { - name string - varType string - code []string - - // whitespace for padding - ws string -} - -// newFailpoint makes a new failpoint based on the a line containing a -// failpoint comment header. -func newFailpoint(l string) (*Failpoint, error) { - if !strings.HasPrefix(strings.TrimSpace(l), "// gofail:") { - // not a failpoint - return nil, nil - } - cmd := strings.SplitAfter(l, "// gofail:")[1] - fields := strings.Fields(cmd) - if len(fields) != 3 || fields[0] != "var" { - return nil, fmt.Errorf("failpoint: malformed comment header %q", l) - } - return &Failpoint{name: fields[1], varType: fields[2], ws: strings.Split(l, "//")[0]}, nil -} - -// flush writes the failpoint code to a buffer -func (fp *Failpoint) flush(dst io.Writer) error { - if len(fp.code) == 0 { - return fp.flushSingle(dst) - } - return fp.flushMulti(dst) -} - -func (fp *Failpoint) hdr(varname string) string { - hdr := fp.ws + "if v" + fp.name + ", __fpErr := " + fp.Runtime() + ".Acquire(); __fpErr == nil { " - hdr = hdr + "defer " + fp.Runtime() + ".Release(); " - if fp.varType == "struct{}" { - // unused - varname = "_" - } - return hdr + varname + ", __fpTypeOK := v" + fp.name + - ".(" + fp.varType + "); if !__fpTypeOK { goto __badType" + fp.name + "} " -} - -func (fp *Failpoint) footer() string { - return "; __badType" + fp.name + ": " + - fp.Runtime() + ".BadType(v" + fp.name + ", \"" + fp.varType + "\"); };" -} - -func (fp *Failpoint) flushSingle(dst io.Writer) error { - if _, err := io.WriteString(dst, fp.hdr("_")); err != nil { - return err - } - _, err := io.WriteString(dst, fp.footer()+"\n") - return err -} - -func (fp *Failpoint) flushMulti(dst io.Writer) error { - hdr := fp.hdr(fp.name) + "\n" - if _, err := io.WriteString(dst, hdr); err != nil { - return err - } - for _, code := range fp.code[:len(fp.code)-1] { - if _, err := io.WriteString(dst, code+"\n"); err != nil { - return err - } - } - code := fp.code[len(fp.code)-1] - _, err := io.WriteString(dst, code+fp.footer()+"\n") - return err -} - -func (fp *Failpoint) Name() string { return fp.name } -func (fp *Failpoint) Runtime() string { return "__fp_" + fp.name } diff --git a/vendor/github.com/etcd-io/gofail/code/rewrite.go b/vendor/github.com/etcd-io/gofail/code/rewrite.go deleted file mode 100644 index d5f41299f191f..0000000000000 --- a/vendor/github.com/etcd-io/gofail/code/rewrite.go +++ /dev/null @@ -1,167 +0,0 @@ -// Copyright 2016 CoreOS, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package code - -import ( - "bufio" - "io" - "strings" - "unicode" -) - -// ToFailpoints turns all gofail comments into failpoint code. Returns a list of -// all failpoints it activated. -func ToFailpoints(wdst io.Writer, rsrc io.Reader) (fps []*Failpoint, err error) { - var curfp *Failpoint - - dst := bufio.NewWriter(wdst) - defer func() { - if err == nil && curfp != nil { - err = curfp.flush(dst) - } - if err == nil { - err = dst.Flush() - } - }() - - src := bufio.NewReader(rsrc) - for err == nil { - l, rerr := src.ReadString('\n') - if curfp != nil { - if strings.HasPrefix(strings.TrimSpace(l), "//") { - if len(l) > 0 && l[len(l)-1] == '\n' { - l = l[:len(l)-1] - } - curfp.code = append(curfp.code, strings.Replace(l, "//", "\t", 1)) - continue - } else { - curfp.flush(dst) - fps = append(fps, curfp) - curfp = nil - } - } else if label := gofailLabel(l); label != "" { - // expose gofail label - l = label - } else if curfp, err = newFailpoint(l); err != nil { - return - } else if curfp != nil { - // found a new failpoint - continue - } - if _, err = dst.WriteString(l); err != nil { - return - } - if rerr == io.EOF { - break - } - } - return -} - -// ToComments turns all failpoint code into GOFAIL comments. It returns -// a list of all failpoints it deactivated. -func ToComments(wdst io.Writer, rsrc io.Reader) (fps []*Failpoint, err error) { - src := bufio.NewReader(rsrc) - dst := bufio.NewWriter(wdst) - ws := "" - unmatchedBraces := 0 - for err == nil { - l, rerr := src.ReadString('\n') - err = rerr - lTrim := strings.TrimSpace(l) - - if unmatchedBraces > 0 { - opening, closing := numBraces(l) - unmatchedBraces += opening - closing - if unmatchedBraces == 0 { - // strip off badType footer - lTrim = strings.Split(lTrim, "; __badType")[0] - } - s := ws + "//" + wsPrefix(l, ws)[1:] + lTrim + "\n" - dst.WriteString(s) - continue - } - - isHdr := strings.Contains(l, ", __fpErr := __fp_") && strings.HasPrefix(lTrim, "if") - if isHdr { - ws = strings.Split(l, "i")[0] - n := strings.Split(strings.Split(l, "__fp_")[1], ".")[0] - t := strings.Split(strings.Split(l, ".(")[1], ")")[0] - dst.WriteString(ws + "// gofail: var " + n + " " + t + "\n") - if !strings.Contains(l, "; __badType") { - // not single liner - unmatchedBraces = 1 - } - fps = append(fps, &Failpoint{name: n, varType: t}) - continue - } - - if isLabel := strings.Contains(l, "\t/* gofail-label */"); isLabel { - l = strings.Replace(l, "/* gofail-label */", "// gofail:", 1) - } - - if _, werr := dst.WriteString(l); werr != nil { - return fps, werr - } - } - if err == io.EOF { - err = nil - } - dst.Flush() - return -} - -func gofailLabel(l string) string { - if !strings.HasPrefix(strings.TrimSpace(l), "// gofail:") { - return "" - } - label := strings.SplitAfter(l, "// gofail:")[1] - if len(label) == 0 || !strings.Contains(label, ":") { - return "" - } - return strings.Replace(l, "// gofail:", "/* gofail-label */", 1) -} - -func numBraces(l string) (opening int, closing int) { - for i := 0; i < len(l); i++ { - switch l[i] { - case '{': - opening++ - case '}': - closing++ - } - } - return -} - -// wsPrefix computes the left padding of a line given a whitespace prefix. -func wsPrefix(l, wsPfx string) string { - lws := "" - if len(wsPfx) == 0 { - lws = l - } else { - wsSplit := strings.SplitAfter(l, wsPfx) - if len(wsSplit) < 2 { - return "" - } - lws = strings.Join(wsSplit[1:], "") - } - for i, c := range lws { - if !unicode.IsSpace(c) { - return lws[:i] - } - } - return lws -} diff --git a/vendor/github.com/etcd-io/gofail/gofail.go b/vendor/github.com/etcd-io/gofail/gofail.go deleted file mode 100644 index a1798fc8377fb..0000000000000 --- a/vendor/github.com/etcd-io/gofail/gofail.go +++ /dev/null @@ -1,195 +0,0 @@ -// Copyright 2016 CoreOS, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -// gofail is a tool for enabling/disabling failpoints in go code. -package main - -import ( - "fmt" - "go/build" - "io" - "os" - "path" - "path/filepath" - "strings" - - "github.com/etcd-io/gofail/code" -) - -type xfrmFunc func(io.Writer, io.Reader) ([]*code.Failpoint, error) - -func xfrmFile(xfrm xfrmFunc, path string) ([]*code.Failpoint, error) { - src, serr := os.Open(path) - if serr != nil { - return nil, serr - } - defer src.Close() - - dst, derr := os.OpenFile(path+".tmp", os.O_WRONLY|os.O_CREATE|os.O_EXCL, 0600) - if derr != nil { - return nil, derr - } - defer dst.Close() - - fps, xerr := xfrm(dst, src) - if xerr != nil || len(fps) == 0 { - os.Remove(dst.Name()) - return nil, xerr - } - - rerr := os.Rename(dst.Name(), path) - if rerr != nil { - os.Remove(dst.Name()) - return nil, rerr - } - - return fps, nil -} - -func dir2files(dir, ext string) (ret []string, err error) { - if dir, err = filepath.Abs(dir); err != nil { - return nil, err - } - - f, ferr := os.Open(dir) - if ferr != nil { - return nil, ferr - } - defer f.Close() - - names, rerr := f.Readdirnames(0) - if rerr != nil { - return nil, rerr - } - for _, f := range names { - if path.Ext(f) != ext { - continue - } - ret = append(ret, path.Join(dir, f)) - } - return ret, nil -} - -func paths2files(paths []string) (files []string) { - // no paths => use cwd - if len(paths) == 0 { - wd, gerr := os.Getwd() - if gerr != nil { - fmt.Println(gerr) - os.Exit(1) - } - return paths2files([]string{wd}) - } - for _, p := range paths { - s, serr := os.Stat(p) - if serr != nil { - fmt.Println(serr) - os.Exit(1) - } - if s.IsDir() { - fs, err := dir2files(p, ".go") - if err != nil { - fmt.Println(err) - os.Exit(1) - } - files = append(files, fs...) - } else if path.Ext(s.Name()) == ".go" { - abs, err := filepath.Abs(p) - if err != nil { - fmt.Println(err) - os.Exit(1) - } - files = append(files, abs) - } - } - return files -} - -func writeBinding(file string, fps []*code.Failpoint) { - if len(fps) == 0 { - return - } - fname := strings.Split(path.Base(file), ".go")[0] + ".fail.go" - out, err := os.Create(path.Join(path.Dir(file), fname)) - if err != nil { - fmt.Println(err) - os.Exit(1) - } - // XXX: support "package main" - pkgAbsDir := path.Dir(file) - pkg := path.Base(pkgAbsDir) - pkgDir := "" - for _, srcdir := range build.Default.SrcDirs() { - if strings.HasPrefix(pkgAbsDir, srcdir) { - pkgDir = strings.Replace(pkgAbsDir, srcdir, "", 1) - break - } - } - fppath := pkg - if pkgDir == "" { - fmt.Fprintf( - os.Stderr, - "missing package for %q; using %q as failpoint path\n", - pkgAbsDir, - pkg) - } else { - fppath = pkgDir[1:] - } - code.NewBinding(pkg, fppath, fps).Write(out) - out.Close() -} - -func main() { - if len(os.Args) < 2 { - fmt.Println("not enough arguments") - os.Exit(1) - } - - var xfrm xfrmFunc - enable := false - switch os.Args[1] { - case "enable": - xfrm = code.ToFailpoints - enable = true - case "disable": - xfrm = code.ToComments - default: - fmt.Println("expected enable or disable") - os.Exit(1) - } - - files := paths2files(os.Args[2:]) - fps := [][]*code.Failpoint{} - for _, path := range files { - curfps, err := xfrmFile(xfrm, path) - if err != nil { - fmt.Println(err) - os.Exit(1) - } - fps = append(fps, curfps) - } - - if enable { - // build runtime bindings .fail.go - for i := range files { - writeBinding(files[i], fps[i]) - } - } else { - // remove all runtime bindings - for i := range files { - fname := strings.Split(path.Base(files[i]), ".go")[0] + ".fail.go" - os.Remove(path.Join(path.Dir(files[i]), fname)) - } - } -} diff --git a/vendor/github.com/gogo/protobuf/LICENSE b/vendor/github.com/gogo/protobuf/LICENSE deleted file mode 100644 index 7be0cc7b62cf0..0000000000000 --- a/vendor/github.com/gogo/protobuf/LICENSE +++ /dev/null @@ -1,36 +0,0 @@ -Protocol Buffers for Go with Gadgets - -Copyright (c) 2013, The GoGo Authors. All rights reserved. -http://github.com/gogo/protobuf - -Go support for Protocol Buffers - Google's data interchange format - -Copyright 2010 The Go Authors. All rights reserved. -https://github.com/golang/protobuf - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - - * Redistributions of source code must retain the above copyright -notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above -copyright notice, this list of conditions and the following disclaimer -in the documentation and/or other materials provided with the -distribution. - * Neither the name of Google Inc. nor the names of its -contributors may be used to endorse or promote products derived from -this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - diff --git a/vendor/github.com/gogo/protobuf/proto/clone.go b/vendor/github.com/gogo/protobuf/proto/clone.go deleted file mode 100644 index 5d4cba4b51c6c..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/clone.go +++ /dev/null @@ -1,234 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2011 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -// Protocol buffer deep copy and merge. -// TODO: RawMessage. - -package proto - -import ( - "log" - "reflect" - "strings" -) - -// Clone returns a deep copy of a protocol buffer. -func Clone(pb Message) Message { - in := reflect.ValueOf(pb) - if in.IsNil() { - return pb - } - - out := reflect.New(in.Type().Elem()) - // out is empty so a merge is a deep copy. - mergeStruct(out.Elem(), in.Elem()) - return out.Interface().(Message) -} - -// Merge merges src into dst. -// Required and optional fields that are set in src will be set to that value in dst. -// Elements of repeated fields will be appended. -// Merge panics if src and dst are not the same type, or if dst is nil. -func Merge(dst, src Message) { - in := reflect.ValueOf(src) - out := reflect.ValueOf(dst) - if out.IsNil() { - panic("proto: nil destination") - } - if in.Type() != out.Type() { - // Explicit test prior to mergeStruct so that mistyped nils will fail - panic("proto: type mismatch") - } - if in.IsNil() { - // Merging nil into non-nil is a quiet no-op - return - } - mergeStruct(out.Elem(), in.Elem()) -} - -func mergeStruct(out, in reflect.Value) { - sprop := GetProperties(in.Type()) - for i := 0; i < in.NumField(); i++ { - f := in.Type().Field(i) - if strings.HasPrefix(f.Name, "XXX_") { - continue - } - mergeAny(out.Field(i), in.Field(i), false, sprop.Prop[i]) - } - - if emIn, ok := in.Addr().Interface().(extensionsBytes); ok { - emOut := out.Addr().Interface().(extensionsBytes) - bIn := emIn.GetExtensions() - bOut := emOut.GetExtensions() - *bOut = append(*bOut, *bIn...) - } else if emIn, ok := extendable(in.Addr().Interface()); ok { - emOut, _ := extendable(out.Addr().Interface()) - mIn, muIn := emIn.extensionsRead() - if mIn != nil { - mOut := emOut.extensionsWrite() - muIn.Lock() - mergeExtension(mOut, mIn) - muIn.Unlock() - } - } - - uf := in.FieldByName("XXX_unrecognized") - if !uf.IsValid() { - return - } - uin := uf.Bytes() - if len(uin) > 0 { - out.FieldByName("XXX_unrecognized").SetBytes(append([]byte(nil), uin...)) - } -} - -// mergeAny performs a merge between two values of the same type. -// viaPtr indicates whether the values were indirected through a pointer (implying proto2). -// prop is set if this is a struct field (it may be nil). -func mergeAny(out, in reflect.Value, viaPtr bool, prop *Properties) { - if in.Type() == protoMessageType { - if !in.IsNil() { - if out.IsNil() { - out.Set(reflect.ValueOf(Clone(in.Interface().(Message)))) - } else { - Merge(out.Interface().(Message), in.Interface().(Message)) - } - } - return - } - switch in.Kind() { - case reflect.Bool, reflect.Float32, reflect.Float64, reflect.Int32, reflect.Int64, - reflect.String, reflect.Uint32, reflect.Uint64: - if !viaPtr && isProto3Zero(in) { - return - } - out.Set(in) - case reflect.Interface: - // Probably a oneof field; copy non-nil values. - if in.IsNil() { - return - } - // Allocate destination if it is not set, or set to a different type. - // Otherwise we will merge as normal. - if out.IsNil() || out.Elem().Type() != in.Elem().Type() { - out.Set(reflect.New(in.Elem().Elem().Type())) // interface -> *T -> T -> new(T) - } - mergeAny(out.Elem(), in.Elem(), false, nil) - case reflect.Map: - if in.Len() == 0 { - return - } - if out.IsNil() { - out.Set(reflect.MakeMap(in.Type())) - } - // For maps with value types of *T or []byte we need to deep copy each value. - elemKind := in.Type().Elem().Kind() - for _, key := range in.MapKeys() { - var val reflect.Value - switch elemKind { - case reflect.Ptr: - val = reflect.New(in.Type().Elem().Elem()) - mergeAny(val, in.MapIndex(key), false, nil) - case reflect.Slice: - val = in.MapIndex(key) - val = reflect.ValueOf(append([]byte{}, val.Bytes()...)) - default: - val = in.MapIndex(key) - } - out.SetMapIndex(key, val) - } - case reflect.Ptr: - if in.IsNil() { - return - } - if out.IsNil() { - out.Set(reflect.New(in.Elem().Type())) - } - mergeAny(out.Elem(), in.Elem(), true, nil) - case reflect.Slice: - if in.IsNil() { - return - } - if in.Type().Elem().Kind() == reflect.Uint8 { - // []byte is a scalar bytes field, not a repeated field. - - // Edge case: if this is in a proto3 message, a zero length - // bytes field is considered the zero value, and should not - // be merged. - if prop != nil && prop.proto3 && in.Len() == 0 { - return - } - - // Make a deep copy. - // Append to []byte{} instead of []byte(nil) so that we never end up - // with a nil result. - out.SetBytes(append([]byte{}, in.Bytes()...)) - return - } - n := in.Len() - if out.IsNil() { - out.Set(reflect.MakeSlice(in.Type(), 0, n)) - } - switch in.Type().Elem().Kind() { - case reflect.Bool, reflect.Float32, reflect.Float64, reflect.Int32, reflect.Int64, - reflect.String, reflect.Uint32, reflect.Uint64: - out.Set(reflect.AppendSlice(out, in)) - default: - for i := 0; i < n; i++ { - x := reflect.Indirect(reflect.New(in.Type().Elem())) - mergeAny(x, in.Index(i), false, nil) - out.Set(reflect.Append(out, x)) - } - } - case reflect.Struct: - mergeStruct(out, in) - default: - // unknown type, so not a protocol buffer - log.Printf("proto: don't know how to copy %v", in) - } -} - -func mergeExtension(out, in map[int32]Extension) { - for extNum, eIn := range in { - eOut := Extension{desc: eIn.desc} - if eIn.value != nil { - v := reflect.New(reflect.TypeOf(eIn.value)).Elem() - mergeAny(v, reflect.ValueOf(eIn.value), false, nil) - eOut.value = v.Interface() - } - if eIn.enc != nil { - eOut.enc = make([]byte, len(eIn.enc)) - copy(eOut.enc, eIn.enc) - } - - out[extNum] = eOut - } -} diff --git a/vendor/github.com/gogo/protobuf/proto/decode.go b/vendor/github.com/gogo/protobuf/proto/decode.go deleted file mode 100644 index 737f2731d45dc..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/decode.go +++ /dev/null @@ -1,978 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -/* - * Routines for decoding protocol buffer data to construct in-memory representations. - */ - -import ( - "errors" - "fmt" - "io" - "os" - "reflect" -) - -// errOverflow is returned when an integer is too large to be represented. -var errOverflow = errors.New("proto: integer overflow") - -// ErrInternalBadWireType is returned by generated code when an incorrect -// wire type is encountered. It does not get returned to user code. -var ErrInternalBadWireType = errors.New("proto: internal error: bad wiretype for oneof") - -// The fundamental decoders that interpret bytes on the wire. -// Those that take integer types all return uint64 and are -// therefore of type valueDecoder. - -// DecodeVarint reads a varint-encoded integer from the slice. -// It returns the integer and the number of bytes consumed, or -// zero if there is not enough. -// This is the format for the -// int32, int64, uint32, uint64, bool, and enum -// protocol buffer types. -func DecodeVarint(buf []byte) (x uint64, n int) { - for shift := uint(0); shift < 64; shift += 7 { - if n >= len(buf) { - return 0, 0 - } - b := uint64(buf[n]) - n++ - x |= (b & 0x7F) << shift - if (b & 0x80) == 0 { - return x, n - } - } - - // The number is too large to represent in a 64-bit value. - return 0, 0 -} - -func (p *Buffer) decodeVarintSlow() (x uint64, err error) { - i := p.index - l := len(p.buf) - - for shift := uint(0); shift < 64; shift += 7 { - if i >= l { - err = io.ErrUnexpectedEOF - return - } - b := p.buf[i] - i++ - x |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - p.index = i - return - } - } - - // The number is too large to represent in a 64-bit value. - err = errOverflow - return -} - -// DecodeVarint reads a varint-encoded integer from the Buffer. -// This is the format for the -// int32, int64, uint32, uint64, bool, and enum -// protocol buffer types. -func (p *Buffer) DecodeVarint() (x uint64, err error) { - i := p.index - buf := p.buf - - if i >= len(buf) { - return 0, io.ErrUnexpectedEOF - } else if buf[i] < 0x80 { - p.index++ - return uint64(buf[i]), nil - } else if len(buf)-i < 10 { - return p.decodeVarintSlow() - } - - var b uint64 - // we already checked the first byte - x = uint64(buf[i]) - 0x80 - i++ - - b = uint64(buf[i]) - i++ - x += b << 7 - if b&0x80 == 0 { - goto done - } - x -= 0x80 << 7 - - b = uint64(buf[i]) - i++ - x += b << 14 - if b&0x80 == 0 { - goto done - } - x -= 0x80 << 14 - - b = uint64(buf[i]) - i++ - x += b << 21 - if b&0x80 == 0 { - goto done - } - x -= 0x80 << 21 - - b = uint64(buf[i]) - i++ - x += b << 28 - if b&0x80 == 0 { - goto done - } - x -= 0x80 << 28 - - b = uint64(buf[i]) - i++ - x += b << 35 - if b&0x80 == 0 { - goto done - } - x -= 0x80 << 35 - - b = uint64(buf[i]) - i++ - x += b << 42 - if b&0x80 == 0 { - goto done - } - x -= 0x80 << 42 - - b = uint64(buf[i]) - i++ - x += b << 49 - if b&0x80 == 0 { - goto done - } - x -= 0x80 << 49 - - b = uint64(buf[i]) - i++ - x += b << 56 - if b&0x80 == 0 { - goto done - } - x -= 0x80 << 56 - - b = uint64(buf[i]) - i++ - x += b << 63 - if b&0x80 == 0 { - goto done - } - // x -= 0x80 << 63 // Always zero. - - return 0, errOverflow - -done: - p.index = i - return x, nil -} - -// DecodeFixed64 reads a 64-bit integer from the Buffer. -// This is the format for the -// fixed64, sfixed64, and double protocol buffer types. -func (p *Buffer) DecodeFixed64() (x uint64, err error) { - // x, err already 0 - i := p.index + 8 - if i < 0 || i > len(p.buf) { - err = io.ErrUnexpectedEOF - return - } - p.index = i - - x = uint64(p.buf[i-8]) - x |= uint64(p.buf[i-7]) << 8 - x |= uint64(p.buf[i-6]) << 16 - x |= uint64(p.buf[i-5]) << 24 - x |= uint64(p.buf[i-4]) << 32 - x |= uint64(p.buf[i-3]) << 40 - x |= uint64(p.buf[i-2]) << 48 - x |= uint64(p.buf[i-1]) << 56 - return -} - -// DecodeFixed32 reads a 32-bit integer from the Buffer. -// This is the format for the -// fixed32, sfixed32, and float protocol buffer types. -func (p *Buffer) DecodeFixed32() (x uint64, err error) { - // x, err already 0 - i := p.index + 4 - if i < 0 || i > len(p.buf) { - err = io.ErrUnexpectedEOF - return - } - p.index = i - - x = uint64(p.buf[i-4]) - x |= uint64(p.buf[i-3]) << 8 - x |= uint64(p.buf[i-2]) << 16 - x |= uint64(p.buf[i-1]) << 24 - return -} - -// DecodeZigzag64 reads a zigzag-encoded 64-bit integer -// from the Buffer. -// This is the format used for the sint64 protocol buffer type. -func (p *Buffer) DecodeZigzag64() (x uint64, err error) { - x, err = p.DecodeVarint() - if err != nil { - return - } - x = (x >> 1) ^ uint64((int64(x&1)<<63)>>63) - return -} - -// DecodeZigzag32 reads a zigzag-encoded 32-bit integer -// from the Buffer. -// This is the format used for the sint32 protocol buffer type. -func (p *Buffer) DecodeZigzag32() (x uint64, err error) { - x, err = p.DecodeVarint() - if err != nil { - return - } - x = uint64((uint32(x) >> 1) ^ uint32((int32(x&1)<<31)>>31)) - return -} - -// These are not ValueDecoders: they produce an array of bytes or a string. -// bytes, embedded messages - -// DecodeRawBytes reads a count-delimited byte buffer from the Buffer. -// This is the format used for the bytes protocol buffer -// type and for embedded messages. -func (p *Buffer) DecodeRawBytes(alloc bool) (buf []byte, err error) { - n, err := p.DecodeVarint() - if err != nil { - return nil, err - } - - nb := int(n) - if nb < 0 { - return nil, fmt.Errorf("proto: bad byte length %d", nb) - } - end := p.index + nb - if end < p.index || end > len(p.buf) { - return nil, io.ErrUnexpectedEOF - } - - if !alloc { - // todo: check if can get more uses of alloc=false - buf = p.buf[p.index:end] - p.index += nb - return - } - - buf = make([]byte, nb) - copy(buf, p.buf[p.index:]) - p.index += nb - return -} - -// DecodeStringBytes reads an encoded string from the Buffer. -// This is the format used for the proto2 string type. -func (p *Buffer) DecodeStringBytes() (s string, err error) { - buf, err := p.DecodeRawBytes(false) - if err != nil { - return - } - return string(buf), nil -} - -// Skip the next item in the buffer. Its wire type is decoded and presented as an argument. -// If the protocol buffer has extensions, and the field matches, add it as an extension. -// Otherwise, if the XXX_unrecognized field exists, append the skipped data there. -func (o *Buffer) skipAndSave(t reflect.Type, tag, wire int, base structPointer, unrecField field) error { - oi := o.index - - err := o.skip(t, tag, wire) - if err != nil { - return err - } - - if !unrecField.IsValid() { - return nil - } - - ptr := structPointer_Bytes(base, unrecField) - - // Add the skipped field to struct field - obuf := o.buf - - o.buf = *ptr - o.EncodeVarint(uint64(tag<<3 | wire)) - *ptr = append(o.buf, obuf[oi:o.index]...) - - o.buf = obuf - - return nil -} - -// Skip the next item in the buffer. Its wire type is decoded and presented as an argument. -func (o *Buffer) skip(t reflect.Type, tag, wire int) error { - - var u uint64 - var err error - - switch wire { - case WireVarint: - _, err = o.DecodeVarint() - case WireFixed64: - _, err = o.DecodeFixed64() - case WireBytes: - _, err = o.DecodeRawBytes(false) - case WireFixed32: - _, err = o.DecodeFixed32() - case WireStartGroup: - for { - u, err = o.DecodeVarint() - if err != nil { - break - } - fwire := int(u & 0x7) - if fwire == WireEndGroup { - break - } - ftag := int(u >> 3) - err = o.skip(t, ftag, fwire) - if err != nil { - break - } - } - default: - err = fmt.Errorf("proto: can't skip unknown wire type %d for %s", wire, t) - } - return err -} - -// Unmarshaler is the interface representing objects that can -// unmarshal themselves. The method should reset the receiver before -// decoding starts. The argument points to data that may be -// overwritten, so implementations should not keep references to the -// buffer. -type Unmarshaler interface { - Unmarshal([]byte) error -} - -// Unmarshal parses the protocol buffer representation in buf and places the -// decoded result in pb. If the struct underlying pb does not match -// the data in buf, the results can be unpredictable. -// -// Unmarshal resets pb before starting to unmarshal, so any -// existing data in pb is always removed. Use UnmarshalMerge -// to preserve and append to existing data. -func Unmarshal(buf []byte, pb Message) error { - pb.Reset() - return UnmarshalMerge(buf, pb) -} - -// UnmarshalMerge parses the protocol buffer representation in buf and -// writes the decoded result to pb. If the struct underlying pb does not match -// the data in buf, the results can be unpredictable. -// -// UnmarshalMerge merges into existing data in pb. -// Most code should use Unmarshal instead. -func UnmarshalMerge(buf []byte, pb Message) error { - // If the object can unmarshal itself, let it. - if u, ok := pb.(Unmarshaler); ok { - return u.Unmarshal(buf) - } - return NewBuffer(buf).Unmarshal(pb) -} - -// DecodeMessage reads a count-delimited message from the Buffer. -func (p *Buffer) DecodeMessage(pb Message) error { - enc, err := p.DecodeRawBytes(false) - if err != nil { - return err - } - return NewBuffer(enc).Unmarshal(pb) -} - -// DecodeGroup reads a tag-delimited group from the Buffer. -func (p *Buffer) DecodeGroup(pb Message) error { - typ, base, err := getbase(pb) - if err != nil { - return err - } - return p.unmarshalType(typ.Elem(), GetProperties(typ.Elem()), true, base) -} - -// Unmarshal parses the protocol buffer representation in the -// Buffer and places the decoded result in pb. If the struct -// underlying pb does not match the data in the buffer, the results can be -// unpredictable. -// -// Unlike proto.Unmarshal, this does not reset pb before starting to unmarshal. -func (p *Buffer) Unmarshal(pb Message) error { - // If the object can unmarshal itself, let it. - if u, ok := pb.(Unmarshaler); ok { - err := u.Unmarshal(p.buf[p.index:]) - p.index = len(p.buf) - return err - } - - typ, base, err := getbase(pb) - if err != nil { - return err - } - - err = p.unmarshalType(typ.Elem(), GetProperties(typ.Elem()), false, base) - - if collectStats { - stats.Decode++ - } - - return err -} - -// unmarshalType does the work of unmarshaling a structure. -func (o *Buffer) unmarshalType(st reflect.Type, prop *StructProperties, is_group bool, base structPointer) error { - var state errorState - required, reqFields := prop.reqCount, uint64(0) - - var err error - for err == nil && o.index < len(o.buf) { - oi := o.index - var u uint64 - u, err = o.DecodeVarint() - if err != nil { - break - } - wire := int(u & 0x7) - if wire == WireEndGroup { - if is_group { - if required > 0 { - // Not enough information to determine the exact field. - // (See below.) - return &RequiredNotSetError{"{Unknown}"} - } - return nil // input is satisfied - } - return fmt.Errorf("proto: %s: wiretype end group for non-group", st) - } - tag := int(u >> 3) - if tag <= 0 { - return fmt.Errorf("proto: %s: illegal tag %d (wire type %d)", st, tag, wire) - } - fieldnum, ok := prop.decoderTags.get(tag) - if !ok { - // Maybe it's an extension? - if prop.extendable { - if e, eok := structPointer_Interface(base, st).(extensionsBytes); eok { - if isExtensionField(e, int32(tag)) { - if err = o.skip(st, tag, wire); err == nil { - ext := e.GetExtensions() - *ext = append(*ext, o.buf[oi:o.index]...) - } - continue - } - } else if e, _ := extendable(structPointer_Interface(base, st)); isExtensionField(e, int32(tag)) { - if err = o.skip(st, tag, wire); err == nil { - extmap := e.extensionsWrite() - ext := extmap[int32(tag)] // may be missing - ext.enc = append(ext.enc, o.buf[oi:o.index]...) - extmap[int32(tag)] = ext - } - continue - } - } - // Maybe it's a oneof? - if prop.oneofUnmarshaler != nil { - m := structPointer_Interface(base, st).(Message) - // First return value indicates whether tag is a oneof field. - ok, err = prop.oneofUnmarshaler(m, tag, wire, o) - if err == ErrInternalBadWireType { - // Map the error to something more descriptive. - // Do the formatting here to save generated code space. - err = fmt.Errorf("bad wiretype for oneof field in %T", m) - } - if ok { - continue - } - } - err = o.skipAndSave(st, tag, wire, base, prop.unrecField) - continue - } - p := prop.Prop[fieldnum] - - if p.dec == nil { - fmt.Fprintf(os.Stderr, "proto: no protobuf decoder for %s.%s\n", st, st.Field(fieldnum).Name) - continue - } - dec := p.dec - if wire != WireStartGroup && wire != p.WireType { - if wire == WireBytes && p.packedDec != nil { - // a packable field - dec = p.packedDec - } else { - err = fmt.Errorf("proto: bad wiretype for field %s.%s: got wiretype %d, want %d", st, st.Field(fieldnum).Name, wire, p.WireType) - continue - } - } - decErr := dec(o, p, base) - if decErr != nil && !state.shouldContinue(decErr, p) { - err = decErr - } - if err == nil && p.Required { - // Successfully decoded a required field. - if tag <= 64 { - // use bitmap for fields 1-64 to catch field reuse. - var mask uint64 = 1 << uint64(tag-1) - if reqFields&mask == 0 { - // new required field - reqFields |= mask - required-- - } - } else { - // This is imprecise. It can be fooled by a required field - // with a tag > 64 that is encoded twice; that's very rare. - // A fully correct implementation would require allocating - // a data structure, which we would like to avoid. - required-- - } - } - } - if err == nil { - if is_group { - return io.ErrUnexpectedEOF - } - if state.err != nil { - return state.err - } - if required > 0 { - // Not enough information to determine the exact field. If we use extra - // CPU, we could determine the field only if the missing required field - // has a tag <= 64 and we check reqFields. - return &RequiredNotSetError{"{Unknown}"} - } - } - return err -} - -// Individual type decoders -// For each, -// u is the decoded value, -// v is a pointer to the field (pointer) in the struct - -// Sizes of the pools to allocate inside the Buffer. -// The goal is modest amortization and allocation -// on at least 16-byte boundaries. -const ( - boolPoolSize = 16 - uint32PoolSize = 8 - uint64PoolSize = 4 -) - -// Decode a bool. -func (o *Buffer) dec_bool(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - if len(o.bools) == 0 { - o.bools = make([]bool, boolPoolSize) - } - o.bools[0] = u != 0 - *structPointer_Bool(base, p.field) = &o.bools[0] - o.bools = o.bools[1:] - return nil -} - -func (o *Buffer) dec_proto3_bool(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - *structPointer_BoolVal(base, p.field) = u != 0 - return nil -} - -// Decode an int32. -func (o *Buffer) dec_int32(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - word32_Set(structPointer_Word32(base, p.field), o, uint32(u)) - return nil -} - -func (o *Buffer) dec_proto3_int32(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - word32Val_Set(structPointer_Word32Val(base, p.field), uint32(u)) - return nil -} - -// Decode an int64. -func (o *Buffer) dec_int64(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - word64_Set(structPointer_Word64(base, p.field), o, u) - return nil -} - -func (o *Buffer) dec_proto3_int64(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - word64Val_Set(structPointer_Word64Val(base, p.field), o, u) - return nil -} - -// Decode a string. -func (o *Buffer) dec_string(p *Properties, base structPointer) error { - s, err := o.DecodeStringBytes() - if err != nil { - return err - } - *structPointer_String(base, p.field) = &s - return nil -} - -func (o *Buffer) dec_proto3_string(p *Properties, base structPointer) error { - s, err := o.DecodeStringBytes() - if err != nil { - return err - } - *structPointer_StringVal(base, p.field) = s - return nil -} - -// Decode a slice of bytes ([]byte). -func (o *Buffer) dec_slice_byte(p *Properties, base structPointer) error { - b, err := o.DecodeRawBytes(true) - if err != nil { - return err - } - *structPointer_Bytes(base, p.field) = b - return nil -} - -// Decode a slice of bools ([]bool). -func (o *Buffer) dec_slice_bool(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - v := structPointer_BoolSlice(base, p.field) - *v = append(*v, u != 0) - return nil -} - -// Decode a slice of bools ([]bool) in packed format. -func (o *Buffer) dec_slice_packed_bool(p *Properties, base structPointer) error { - v := structPointer_BoolSlice(base, p.field) - - nn, err := o.DecodeVarint() - if err != nil { - return err - } - nb := int(nn) // number of bytes of encoded bools - fin := o.index + nb - if fin < o.index { - return errOverflow - } - - y := *v - for o.index < fin { - u, err := p.valDec(o) - if err != nil { - return err - } - y = append(y, u != 0) - } - - *v = y - return nil -} - -// Decode a slice of int32s ([]int32). -func (o *Buffer) dec_slice_int32(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - structPointer_Word32Slice(base, p.field).Append(uint32(u)) - return nil -} - -// Decode a slice of int32s ([]int32) in packed format. -func (o *Buffer) dec_slice_packed_int32(p *Properties, base structPointer) error { - v := structPointer_Word32Slice(base, p.field) - - nn, err := o.DecodeVarint() - if err != nil { - return err - } - nb := int(nn) // number of bytes of encoded int32s - - fin := o.index + nb - if fin < o.index { - return errOverflow - } - for o.index < fin { - u, err := p.valDec(o) - if err != nil { - return err - } - v.Append(uint32(u)) - } - return nil -} - -// Decode a slice of int64s ([]int64). -func (o *Buffer) dec_slice_int64(p *Properties, base structPointer) error { - u, err := p.valDec(o) - if err != nil { - return err - } - - structPointer_Word64Slice(base, p.field).Append(u) - return nil -} - -// Decode a slice of int64s ([]int64) in packed format. -func (o *Buffer) dec_slice_packed_int64(p *Properties, base structPointer) error { - v := structPointer_Word64Slice(base, p.field) - - nn, err := o.DecodeVarint() - if err != nil { - return err - } - nb := int(nn) // number of bytes of encoded int64s - - fin := o.index + nb - if fin < o.index { - return errOverflow - } - for o.index < fin { - u, err := p.valDec(o) - if err != nil { - return err - } - v.Append(u) - } - return nil -} - -// Decode a slice of strings ([]string). -func (o *Buffer) dec_slice_string(p *Properties, base structPointer) error { - s, err := o.DecodeStringBytes() - if err != nil { - return err - } - v := structPointer_StringSlice(base, p.field) - *v = append(*v, s) - return nil -} - -// Decode a slice of slice of bytes ([][]byte). -func (o *Buffer) dec_slice_slice_byte(p *Properties, base structPointer) error { - b, err := o.DecodeRawBytes(true) - if err != nil { - return err - } - v := structPointer_BytesSlice(base, p.field) - *v = append(*v, b) - return nil -} - -// Decode a map field. -func (o *Buffer) dec_new_map(p *Properties, base structPointer) error { - raw, err := o.DecodeRawBytes(false) - if err != nil { - return err - } - oi := o.index // index at the end of this map entry - o.index -= len(raw) // move buffer back to start of map entry - - mptr := structPointer_NewAt(base, p.field, p.mtype) // *map[K]V - if mptr.Elem().IsNil() { - mptr.Elem().Set(reflect.MakeMap(mptr.Type().Elem())) - } - v := mptr.Elem() // map[K]V - - // Prepare addressable doubly-indirect placeholders for the key and value types. - // See enc_new_map for why. - keyptr := reflect.New(reflect.PtrTo(p.mtype.Key())).Elem() // addressable *K - keybase := toStructPointer(keyptr.Addr()) // **K - - var valbase structPointer - var valptr reflect.Value - switch p.mtype.Elem().Kind() { - case reflect.Slice: - // []byte - var dummy []byte - valptr = reflect.ValueOf(&dummy) // *[]byte - valbase = toStructPointer(valptr) // *[]byte - case reflect.Ptr: - // message; valptr is **Msg; need to allocate the intermediate pointer - valptr = reflect.New(reflect.PtrTo(p.mtype.Elem())).Elem() // addressable *V - valptr.Set(reflect.New(valptr.Type().Elem())) - valbase = toStructPointer(valptr) - default: - // everything else - valptr = reflect.New(reflect.PtrTo(p.mtype.Elem())).Elem() // addressable *V - valbase = toStructPointer(valptr.Addr()) // **V - } - - // Decode. - // This parses a restricted wire format, namely the encoding of a message - // with two fields. See enc_new_map for the format. - for o.index < oi { - // tagcode for key and value properties are always a single byte - // because they have tags 1 and 2. - tagcode := o.buf[o.index] - o.index++ - switch tagcode { - case p.mkeyprop.tagcode[0]: - if err := p.mkeyprop.dec(o, p.mkeyprop, keybase); err != nil { - return err - } - case p.mvalprop.tagcode[0]: - if err := p.mvalprop.dec(o, p.mvalprop, valbase); err != nil { - return err - } - default: - // TODO: Should we silently skip this instead? - return fmt.Errorf("proto: bad map data tag %d", raw[0]) - } - } - keyelem, valelem := keyptr.Elem(), valptr.Elem() - if !keyelem.IsValid() { - keyelem = reflect.Zero(p.mtype.Key()) - } - if !valelem.IsValid() { - valelem = reflect.Zero(p.mtype.Elem()) - } - - v.SetMapIndex(keyelem, valelem) - return nil -} - -// Decode a group. -func (o *Buffer) dec_struct_group(p *Properties, base structPointer) error { - bas := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(bas) { - // allocate new nested message - bas = toStructPointer(reflect.New(p.stype)) - structPointer_SetStructPointer(base, p.field, bas) - } - return o.unmarshalType(p.stype, p.sprop, true, bas) -} - -// Decode an embedded message. -func (o *Buffer) dec_struct_message(p *Properties, base structPointer) (err error) { - raw, e := o.DecodeRawBytes(false) - if e != nil { - return e - } - - bas := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(bas) { - // allocate new nested message - bas = toStructPointer(reflect.New(p.stype)) - structPointer_SetStructPointer(base, p.field, bas) - } - - // If the object can unmarshal itself, let it. - if p.isUnmarshaler { - iv := structPointer_Interface(bas, p.stype) - return iv.(Unmarshaler).Unmarshal(raw) - } - - obuf := o.buf - oi := o.index - o.buf = raw - o.index = 0 - - err = o.unmarshalType(p.stype, p.sprop, false, bas) - o.buf = obuf - o.index = oi - - return err -} - -// Decode a slice of embedded messages. -func (o *Buffer) dec_slice_struct_message(p *Properties, base structPointer) error { - return o.dec_slice_struct(p, false, base) -} - -// Decode a slice of embedded groups. -func (o *Buffer) dec_slice_struct_group(p *Properties, base structPointer) error { - return o.dec_slice_struct(p, true, base) -} - -// Decode a slice of structs ([]*struct). -func (o *Buffer) dec_slice_struct(p *Properties, is_group bool, base structPointer) error { - v := reflect.New(p.stype) - bas := toStructPointer(v) - structPointer_StructPointerSlice(base, p.field).Append(bas) - - if is_group { - err := o.unmarshalType(p.stype, p.sprop, is_group, bas) - return err - } - - raw, err := o.DecodeRawBytes(false) - if err != nil { - return err - } - - // If the object can unmarshal itself, let it. - if p.isUnmarshaler { - iv := v.Interface() - return iv.(Unmarshaler).Unmarshal(raw) - } - - obuf := o.buf - oi := o.index - o.buf = raw - o.index = 0 - - err = o.unmarshalType(p.stype, p.sprop, is_group, bas) - - o.buf = obuf - o.index = oi - - return err -} diff --git a/vendor/github.com/gogo/protobuf/proto/decode_gogo.go b/vendor/github.com/gogo/protobuf/proto/decode_gogo.go deleted file mode 100644 index 6fb74de4cc94c..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/decode_gogo.go +++ /dev/null @@ -1,172 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "reflect" -) - -// Decode a reference to a struct pointer. -func (o *Buffer) dec_ref_struct_message(p *Properties, base structPointer) (err error) { - raw, e := o.DecodeRawBytes(false) - if e != nil { - return e - } - - // If the object can unmarshal itself, let it. - if p.isUnmarshaler { - panic("not supported, since this is a pointer receiver") - } - - obuf := o.buf - oi := o.index - o.buf = raw - o.index = 0 - - bas := structPointer_FieldPointer(base, p.field) - - err = o.unmarshalType(p.stype, p.sprop, false, bas) - o.buf = obuf - o.index = oi - - return err -} - -// Decode a slice of references to struct pointers ([]struct). -func (o *Buffer) dec_slice_ref_struct(p *Properties, is_group bool, base structPointer) error { - newBas := appendStructPointer(base, p.field, p.sstype) - - if is_group { - panic("not supported, maybe in future, if requested.") - } - - raw, err := o.DecodeRawBytes(false) - if err != nil { - return err - } - - // If the object can unmarshal itself, let it. - if p.isUnmarshaler { - panic("not supported, since this is not a pointer receiver.") - } - - obuf := o.buf - oi := o.index - o.buf = raw - o.index = 0 - - err = o.unmarshalType(p.stype, p.sprop, is_group, newBas) - - o.buf = obuf - o.index = oi - - return err -} - -// Decode a slice of references to struct pointers. -func (o *Buffer) dec_slice_ref_struct_message(p *Properties, base structPointer) error { - return o.dec_slice_ref_struct(p, false, base) -} - -func setPtrCustomType(base structPointer, f field, v interface{}) { - if v == nil { - return - } - structPointer_SetStructPointer(base, f, toStructPointer(reflect.ValueOf(v))) -} - -func setCustomType(base structPointer, f field, value interface{}) { - if value == nil { - return - } - v := reflect.ValueOf(value).Elem() - t := reflect.TypeOf(value).Elem() - kind := t.Kind() - switch kind { - case reflect.Slice: - slice := reflect.MakeSlice(t, v.Len(), v.Cap()) - reflect.Copy(slice, v) - oldHeader := structPointer_GetSliceHeader(base, f) - oldHeader.Data = slice.Pointer() - oldHeader.Len = v.Len() - oldHeader.Cap = v.Cap() - default: - size := reflect.TypeOf(value).Elem().Size() - structPointer_Copy(toStructPointer(reflect.ValueOf(value)), structPointer_Add(base, f), int(size)) - } -} - -func (o *Buffer) dec_custom_bytes(p *Properties, base structPointer) error { - b, err := o.DecodeRawBytes(true) - if err != nil { - return err - } - i := reflect.New(p.ctype.Elem()).Interface() - custom := (i).(Unmarshaler) - if err := custom.Unmarshal(b); err != nil { - return err - } - setPtrCustomType(base, p.field, custom) - return nil -} - -func (o *Buffer) dec_custom_ref_bytes(p *Properties, base structPointer) error { - b, err := o.DecodeRawBytes(true) - if err != nil { - return err - } - i := reflect.New(p.ctype).Interface() - custom := (i).(Unmarshaler) - if err := custom.Unmarshal(b); err != nil { - return err - } - if custom != nil { - setCustomType(base, p.field, custom) - } - return nil -} - -// Decode a slice of bytes ([]byte) into a slice of custom types. -func (o *Buffer) dec_custom_slice_bytes(p *Properties, base structPointer) error { - b, err := o.DecodeRawBytes(true) - if err != nil { - return err - } - i := reflect.New(p.ctype.Elem()).Interface() - custom := (i).(Unmarshaler) - if err := custom.Unmarshal(b); err != nil { - return err - } - newBas := appendStructPointer(base, p.field, p.ctype) - - var zero field - setCustomType(newBas, zero, custom) - - return nil -} diff --git a/vendor/github.com/gogo/protobuf/proto/discard.go b/vendor/github.com/gogo/protobuf/proto/discard.go deleted file mode 100644 index bd0e3bb4c8510..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/discard.go +++ /dev/null @@ -1,151 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2017 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "fmt" - "reflect" - "strings" -) - -// DiscardUnknown recursively discards all unknown fields from this message -// and all embedded messages. -// -// When unmarshaling a message with unrecognized fields, the tags and values -// of such fields are preserved in the Message. This allows a later call to -// marshal to be able to produce a message that continues to have those -// unrecognized fields. To avoid this, DiscardUnknown is used to -// explicitly clear the unknown fields after unmarshaling. -// -// For proto2 messages, the unknown fields of message extensions are only -// discarded from messages that have been accessed via GetExtension. -func DiscardUnknown(m Message) { - discardLegacy(m) -} - -func discardLegacy(m Message) { - v := reflect.ValueOf(m) - if v.Kind() != reflect.Ptr || v.IsNil() { - return - } - v = v.Elem() - if v.Kind() != reflect.Struct { - return - } - t := v.Type() - - for i := 0; i < v.NumField(); i++ { - f := t.Field(i) - if strings.HasPrefix(f.Name, "XXX_") { - continue - } - vf := v.Field(i) - tf := f.Type - - // Unwrap tf to get its most basic type. - var isPointer, isSlice bool - if tf.Kind() == reflect.Slice && tf.Elem().Kind() != reflect.Uint8 { - isSlice = true - tf = tf.Elem() - } - if tf.Kind() == reflect.Ptr { - isPointer = true - tf = tf.Elem() - } - if isPointer && isSlice && tf.Kind() != reflect.Struct { - panic(fmt.Sprintf("%T.%s cannot be a slice of pointers to primitive types", m, f.Name)) - } - - switch tf.Kind() { - case reflect.Struct: - switch { - case !isPointer: - panic(fmt.Sprintf("%T.%s cannot be a direct struct value", m, f.Name)) - case isSlice: // E.g., []*pb.T - for j := 0; j < vf.Len(); j++ { - discardLegacy(vf.Index(j).Interface().(Message)) - } - default: // E.g., *pb.T - discardLegacy(vf.Interface().(Message)) - } - case reflect.Map: - switch { - case isPointer || isSlice: - panic(fmt.Sprintf("%T.%s cannot be a pointer to a map or a slice of map values", m, f.Name)) - default: // E.g., map[K]V - tv := vf.Type().Elem() - if tv.Kind() == reflect.Ptr && tv.Implements(protoMessageType) { // Proto struct (e.g., *T) - for _, key := range vf.MapKeys() { - val := vf.MapIndex(key) - discardLegacy(val.Interface().(Message)) - } - } - } - case reflect.Interface: - // Must be oneof field. - switch { - case isPointer || isSlice: - panic(fmt.Sprintf("%T.%s cannot be a pointer to a interface or a slice of interface values", m, f.Name)) - default: // E.g., test_proto.isCommunique_Union interface - if !vf.IsNil() && f.Tag.Get("protobuf_oneof") != "" { - vf = vf.Elem() // E.g., *test_proto.Communique_Msg - if !vf.IsNil() { - vf = vf.Elem() // E.g., test_proto.Communique_Msg - vf = vf.Field(0) // E.g., Proto struct (e.g., *T) or primitive value - if vf.Kind() == reflect.Ptr { - discardLegacy(vf.Interface().(Message)) - } - } - } - } - } - } - - if vf := v.FieldByName("XXX_unrecognized"); vf.IsValid() { - if vf.Type() != reflect.TypeOf([]byte{}) { - panic("expected XXX_unrecognized to be of type []byte") - } - vf.Set(reflect.ValueOf([]byte(nil))) - } - - // For proto2 messages, only discard unknown fields in message extensions - // that have been accessed via GetExtension. - if em, ok := extendable(m); ok { - // Ignore lock since discardLegacy is not concurrency safe. - emm, _ := em.extensionsRead() - for _, mx := range emm { - if m, ok := mx.value.(Message); ok { - discardLegacy(m) - } - } - } -} diff --git a/vendor/github.com/gogo/protobuf/proto/duration.go b/vendor/github.com/gogo/protobuf/proto/duration.go deleted file mode 100644 index 93464c91cffb5..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/duration.go +++ /dev/null @@ -1,100 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2016 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -// This file implements conversions between google.protobuf.Duration -// and time.Duration. - -import ( - "errors" - "fmt" - "time" -) - -const ( - // Range of a Duration in seconds, as specified in - // google/protobuf/duration.proto. This is about 10,000 years in seconds. - maxSeconds = int64(10000 * 365.25 * 24 * 60 * 60) - minSeconds = -maxSeconds -) - -// validateDuration determines whether the Duration is valid according to the -// definition in google/protobuf/duration.proto. A valid Duration -// may still be too large to fit into a time.Duration (the range of Duration -// is about 10,000 years, and the range of time.Duration is about 290). -func validateDuration(d *duration) error { - if d == nil { - return errors.New("duration: nil Duration") - } - if d.Seconds < minSeconds || d.Seconds > maxSeconds { - return fmt.Errorf("duration: %#v: seconds out of range", d) - } - if d.Nanos <= -1e9 || d.Nanos >= 1e9 { - return fmt.Errorf("duration: %#v: nanos out of range", d) - } - // Seconds and Nanos must have the same sign, unless d.Nanos is zero. - if (d.Seconds < 0 && d.Nanos > 0) || (d.Seconds > 0 && d.Nanos < 0) { - return fmt.Errorf("duration: %#v: seconds and nanos have different signs", d) - } - return nil -} - -// DurationFromProto converts a Duration to a time.Duration. DurationFromProto -// returns an error if the Duration is invalid or is too large to be -// represented in a time.Duration. -func durationFromProto(p *duration) (time.Duration, error) { - if err := validateDuration(p); err != nil { - return 0, err - } - d := time.Duration(p.Seconds) * time.Second - if int64(d/time.Second) != p.Seconds { - return 0, fmt.Errorf("duration: %#v is out of range for time.Duration", p) - } - if p.Nanos != 0 { - d += time.Duration(p.Nanos) - if (d < 0) != (p.Nanos < 0) { - return 0, fmt.Errorf("duration: %#v is out of range for time.Duration", p) - } - } - return d, nil -} - -// DurationProto converts a time.Duration to a Duration. -func durationProto(d time.Duration) *duration { - nanos := d.Nanoseconds() - secs := nanos / 1e9 - nanos -= secs * 1e9 - return &duration{ - Seconds: secs, - Nanos: int32(nanos), - } -} diff --git a/vendor/github.com/gogo/protobuf/proto/duration_gogo.go b/vendor/github.com/gogo/protobuf/proto/duration_gogo.go deleted file mode 100644 index 18e2a5f77654f..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/duration_gogo.go +++ /dev/null @@ -1,203 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2016, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "reflect" - "time" -) - -var durationType = reflect.TypeOf((*time.Duration)(nil)).Elem() - -type duration struct { - Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"` - Nanos int32 `protobuf:"varint,2,opt,name=nanos,proto3" json:"nanos,omitempty"` -} - -func (m *duration) Reset() { *m = duration{} } -func (*duration) ProtoMessage() {} -func (*duration) String() string { return "duration" } - -func init() { - RegisterType((*duration)(nil), "gogo.protobuf.proto.duration") -} - -func (o *Buffer) decDuration() (time.Duration, error) { - b, err := o.DecodeRawBytes(true) - if err != nil { - return 0, err - } - dproto := &duration{} - if err := Unmarshal(b, dproto); err != nil { - return 0, err - } - return durationFromProto(dproto) -} - -func (o *Buffer) dec_duration(p *Properties, base structPointer) error { - d, err := o.decDuration() - if err != nil { - return err - } - word64_Set(structPointer_Word64(base, p.field), o, uint64(d)) - return nil -} - -func (o *Buffer) dec_ref_duration(p *Properties, base structPointer) error { - d, err := o.decDuration() - if err != nil { - return err - } - word64Val_Set(structPointer_Word64Val(base, p.field), o, uint64(d)) - return nil -} - -func (o *Buffer) dec_slice_duration(p *Properties, base structPointer) error { - d, err := o.decDuration() - if err != nil { - return err - } - newBas := appendStructPointer(base, p.field, reflect.SliceOf(reflect.PtrTo(durationType))) - var zero field - setPtrCustomType(newBas, zero, &d) - return nil -} - -func (o *Buffer) dec_slice_ref_duration(p *Properties, base structPointer) error { - d, err := o.decDuration() - if err != nil { - return err - } - structPointer_Word64Slice(base, p.field).Append(uint64(d)) - return nil -} - -func size_duration(p *Properties, base structPointer) (n int) { - structp := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(structp) { - return 0 - } - dur := structPointer_Interface(structp, durationType).(*time.Duration) - d := durationProto(*dur) - size := Size(d) - return size + sizeVarint(uint64(size)) + len(p.tagcode) -} - -func (o *Buffer) enc_duration(p *Properties, base structPointer) error { - structp := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(structp) { - return ErrNil - } - dur := structPointer_Interface(structp, durationType).(*time.Duration) - d := durationProto(*dur) - data, err := Marshal(d) - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - return nil -} - -func size_ref_duration(p *Properties, base structPointer) (n int) { - dur := structPointer_InterfaceAt(base, p.field, durationType).(*time.Duration) - d := durationProto(*dur) - size := Size(d) - return size + sizeVarint(uint64(size)) + len(p.tagcode) -} - -func (o *Buffer) enc_ref_duration(p *Properties, base structPointer) error { - dur := structPointer_InterfaceAt(base, p.field, durationType).(*time.Duration) - d := durationProto(*dur) - data, err := Marshal(d) - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - return nil -} - -func size_slice_duration(p *Properties, base structPointer) (n int) { - pdurs := structPointer_InterfaceAt(base, p.field, reflect.SliceOf(reflect.PtrTo(durationType))).(*[]*time.Duration) - durs := *pdurs - for i := 0; i < len(durs); i++ { - if durs[i] == nil { - return 0 - } - dproto := durationProto(*durs[i]) - size := Size(dproto) - n += len(p.tagcode) + size + sizeVarint(uint64(size)) - } - return n -} - -func (o *Buffer) enc_slice_duration(p *Properties, base structPointer) error { - pdurs := structPointer_InterfaceAt(base, p.field, reflect.SliceOf(reflect.PtrTo(durationType))).(*[]*time.Duration) - durs := *pdurs - for i := 0; i < len(durs); i++ { - if durs[i] == nil { - return errRepeatedHasNil - } - dproto := durationProto(*durs[i]) - data, err := Marshal(dproto) - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - } - return nil -} - -func size_slice_ref_duration(p *Properties, base structPointer) (n int) { - pdurs := structPointer_InterfaceAt(base, p.field, reflect.SliceOf(durationType)).(*[]time.Duration) - durs := *pdurs - for i := 0; i < len(durs); i++ { - dproto := durationProto(durs[i]) - size := Size(dproto) - n += len(p.tagcode) + size + sizeVarint(uint64(size)) - } - return n -} - -func (o *Buffer) enc_slice_ref_duration(p *Properties, base structPointer) error { - pdurs := structPointer_InterfaceAt(base, p.field, reflect.SliceOf(durationType)).(*[]time.Duration) - durs := *pdurs - for i := 0; i < len(durs); i++ { - dproto := durationProto(durs[i]) - data, err := Marshal(dproto) - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - } - return nil -} diff --git a/vendor/github.com/gogo/protobuf/proto/encode.go b/vendor/github.com/gogo/protobuf/proto/encode.go deleted file mode 100644 index 8b84d1b22d4c0..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/encode.go +++ /dev/null @@ -1,1362 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -/* - * Routines for encoding data into the wire format for protocol buffers. - */ - -import ( - "errors" - "fmt" - "reflect" - "sort" -) - -// RequiredNotSetError is the error returned if Marshal is called with -// a protocol buffer struct whose required fields have not -// all been initialized. It is also the error returned if Unmarshal is -// called with an encoded protocol buffer that does not include all the -// required fields. -// -// When printed, RequiredNotSetError reports the first unset required field in a -// message. If the field cannot be precisely determined, it is reported as -// "{Unknown}". -type RequiredNotSetError struct { - field string -} - -func (e *RequiredNotSetError) Error() string { - return fmt.Sprintf("proto: required field %q not set", e.field) -} - -var ( - // errRepeatedHasNil is the error returned if Marshal is called with - // a struct with a repeated field containing a nil element. - errRepeatedHasNil = errors.New("proto: repeated field has nil element") - - // errOneofHasNil is the error returned if Marshal is called with - // a struct with a oneof field containing a nil element. - errOneofHasNil = errors.New("proto: oneof field has nil value") - - // ErrNil is the error returned if Marshal is called with nil. - ErrNil = errors.New("proto: Marshal called with nil") - - // ErrTooLarge is the error returned if Marshal is called with a - // message that encodes to >2GB. - ErrTooLarge = errors.New("proto: message encodes to over 2 GB") -) - -// The fundamental encoders that put bytes on the wire. -// Those that take integer types all accept uint64 and are -// therefore of type valueEncoder. - -const maxVarintBytes = 10 // maximum length of a varint - -// maxMarshalSize is the largest allowed size of an encoded protobuf, -// since C++ and Java use signed int32s for the size. -const maxMarshalSize = 1<<31 - 1 - -// EncodeVarint returns the varint encoding of x. -// This is the format for the -// int32, int64, uint32, uint64, bool, and enum -// protocol buffer types. -// Not used by the package itself, but helpful to clients -// wishing to use the same encoding. -func EncodeVarint(x uint64) []byte { - var buf [maxVarintBytes]byte - var n int - for n = 0; x > 127; n++ { - buf[n] = 0x80 | uint8(x&0x7F) - x >>= 7 - } - buf[n] = uint8(x) - n++ - return buf[0:n] -} - -// EncodeVarint writes a varint-encoded integer to the Buffer. -// This is the format for the -// int32, int64, uint32, uint64, bool, and enum -// protocol buffer types. -func (p *Buffer) EncodeVarint(x uint64) error { - for x >= 1<<7 { - p.buf = append(p.buf, uint8(x&0x7f|0x80)) - x >>= 7 - } - p.buf = append(p.buf, uint8(x)) - return nil -} - -// SizeVarint returns the varint encoding size of an integer. -func SizeVarint(x uint64) int { - return sizeVarint(x) -} - -func sizeVarint(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { - break - } - } - return n -} - -// EncodeFixed64 writes a 64-bit integer to the Buffer. -// This is the format for the -// fixed64, sfixed64, and double protocol buffer types. -func (p *Buffer) EncodeFixed64(x uint64) error { - p.buf = append(p.buf, - uint8(x), - uint8(x>>8), - uint8(x>>16), - uint8(x>>24), - uint8(x>>32), - uint8(x>>40), - uint8(x>>48), - uint8(x>>56)) - return nil -} - -func sizeFixed64(x uint64) int { - return 8 -} - -// EncodeFixed32 writes a 32-bit integer to the Buffer. -// This is the format for the -// fixed32, sfixed32, and float protocol buffer types. -func (p *Buffer) EncodeFixed32(x uint64) error { - p.buf = append(p.buf, - uint8(x), - uint8(x>>8), - uint8(x>>16), - uint8(x>>24)) - return nil -} - -func sizeFixed32(x uint64) int { - return 4 -} - -// EncodeZigzag64 writes a zigzag-encoded 64-bit integer -// to the Buffer. -// This is the format used for the sint64 protocol buffer type. -func (p *Buffer) EncodeZigzag64(x uint64) error { - // use signed number to get arithmetic right shift. - return p.EncodeVarint((x << 1) ^ uint64((int64(x) >> 63))) -} - -func sizeZigzag64(x uint64) int { - return sizeVarint((x << 1) ^ uint64((int64(x) >> 63))) -} - -// EncodeZigzag32 writes a zigzag-encoded 32-bit integer -// to the Buffer. -// This is the format used for the sint32 protocol buffer type. -func (p *Buffer) EncodeZigzag32(x uint64) error { - // use signed number to get arithmetic right shift. - return p.EncodeVarint(uint64((uint32(x) << 1) ^ uint32((int32(x) >> 31)))) -} - -func sizeZigzag32(x uint64) int { - return sizeVarint(uint64((uint32(x) << 1) ^ uint32((int32(x) >> 31)))) -} - -// EncodeRawBytes writes a count-delimited byte buffer to the Buffer. -// This is the format used for the bytes protocol buffer -// type and for embedded messages. -func (p *Buffer) EncodeRawBytes(b []byte) error { - p.EncodeVarint(uint64(len(b))) - p.buf = append(p.buf, b...) - return nil -} - -func sizeRawBytes(b []byte) int { - return sizeVarint(uint64(len(b))) + - len(b) -} - -// EncodeStringBytes writes an encoded string to the Buffer. -// This is the format used for the proto2 string type. -func (p *Buffer) EncodeStringBytes(s string) error { - p.EncodeVarint(uint64(len(s))) - p.buf = append(p.buf, s...) - return nil -} - -func sizeStringBytes(s string) int { - return sizeVarint(uint64(len(s))) + - len(s) -} - -// Marshaler is the interface representing objects that can marshal themselves. -type Marshaler interface { - Marshal() ([]byte, error) -} - -// Marshal takes the protocol buffer -// and encodes it into the wire format, returning the data. -func Marshal(pb Message) ([]byte, error) { - // Can the object marshal itself? - if m, ok := pb.(Marshaler); ok { - return m.Marshal() - } - p := NewBuffer(nil) - err := p.Marshal(pb) - if p.buf == nil && err == nil { - // Return a non-nil slice on success. - return []byte{}, nil - } - return p.buf, err -} - -// EncodeMessage writes the protocol buffer to the Buffer, -// prefixed by a varint-encoded length. -func (p *Buffer) EncodeMessage(pb Message) error { - t, base, err := getbase(pb) - if structPointer_IsNil(base) { - return ErrNil - } - if err == nil { - var state errorState - err = p.enc_len_struct(GetProperties(t.Elem()), base, &state) - } - return err -} - -// Marshal takes the protocol buffer -// and encodes it into the wire format, writing the result to the -// Buffer. -func (p *Buffer) Marshal(pb Message) error { - // Can the object marshal itself? - if m, ok := pb.(Marshaler); ok { - data, err := m.Marshal() - p.buf = append(p.buf, data...) - return err - } - - t, base, err := getbase(pb) - if structPointer_IsNil(base) { - return ErrNil - } - if err == nil { - err = p.enc_struct(GetProperties(t.Elem()), base) - } - - if collectStats { - (stats).Encode++ // Parens are to work around a goimports bug. - } - - if len(p.buf) > maxMarshalSize { - return ErrTooLarge - } - return err -} - -// Size returns the encoded size of a protocol buffer. -func Size(pb Message) (n int) { - // Can the object marshal itself? If so, Size is slow. - // TODO: add Size to Marshaler, or add a Sizer interface. - if m, ok := pb.(Marshaler); ok { - b, _ := m.Marshal() - return len(b) - } - - t, base, err := getbase(pb) - if structPointer_IsNil(base) { - return 0 - } - if err == nil { - n = size_struct(GetProperties(t.Elem()), base) - } - - if collectStats { - (stats).Size++ // Parens are to work around a goimports bug. - } - - return -} - -// Individual type encoders. - -// Encode a bool. -func (o *Buffer) enc_bool(p *Properties, base structPointer) error { - v := *structPointer_Bool(base, p.field) - if v == nil { - return ErrNil - } - x := 0 - if *v { - x = 1 - } - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, uint64(x)) - return nil -} - -func (o *Buffer) enc_proto3_bool(p *Properties, base structPointer) error { - v := *structPointer_BoolVal(base, p.field) - if !v { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, 1) - return nil -} - -func size_bool(p *Properties, base structPointer) int { - v := *structPointer_Bool(base, p.field) - if v == nil { - return 0 - } - return len(p.tagcode) + 1 // each bool takes exactly one byte -} - -func size_proto3_bool(p *Properties, base structPointer) int { - v := *structPointer_BoolVal(base, p.field) - if !v && !p.oneof { - return 0 - } - return len(p.tagcode) + 1 // each bool takes exactly one byte -} - -// Encode an int32. -func (o *Buffer) enc_int32(p *Properties, base structPointer) error { - v := structPointer_Word32(base, p.field) - if word32_IsNil(v) { - return ErrNil - } - x := int32(word32_Get(v)) // permit sign extension to use full 64-bit range - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, uint64(x)) - return nil -} - -func (o *Buffer) enc_proto3_int32(p *Properties, base structPointer) error { - v := structPointer_Word32Val(base, p.field) - x := int32(word32Val_Get(v)) // permit sign extension to use full 64-bit range - if x == 0 { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, uint64(x)) - return nil -} - -func size_int32(p *Properties, base structPointer) (n int) { - v := structPointer_Word32(base, p.field) - if word32_IsNil(v) { - return 0 - } - x := int32(word32_Get(v)) // permit sign extension to use full 64-bit range - n += len(p.tagcode) - n += p.valSize(uint64(x)) - return -} - -func size_proto3_int32(p *Properties, base structPointer) (n int) { - v := structPointer_Word32Val(base, p.field) - x := int32(word32Val_Get(v)) // permit sign extension to use full 64-bit range - if x == 0 && !p.oneof { - return 0 - } - n += len(p.tagcode) - n += p.valSize(uint64(x)) - return -} - -// Encode a uint32. -// Exactly the same as int32, except for no sign extension. -func (o *Buffer) enc_uint32(p *Properties, base structPointer) error { - v := structPointer_Word32(base, p.field) - if word32_IsNil(v) { - return ErrNil - } - x := word32_Get(v) - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, uint64(x)) - return nil -} - -func (o *Buffer) enc_proto3_uint32(p *Properties, base structPointer) error { - v := structPointer_Word32Val(base, p.field) - x := word32Val_Get(v) - if x == 0 { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, uint64(x)) - return nil -} - -func size_uint32(p *Properties, base structPointer) (n int) { - v := structPointer_Word32(base, p.field) - if word32_IsNil(v) { - return 0 - } - x := word32_Get(v) - n += len(p.tagcode) - n += p.valSize(uint64(x)) - return -} - -func size_proto3_uint32(p *Properties, base structPointer) (n int) { - v := structPointer_Word32Val(base, p.field) - x := word32Val_Get(v) - if x == 0 && !p.oneof { - return 0 - } - n += len(p.tagcode) - n += p.valSize(uint64(x)) - return -} - -// Encode an int64. -func (o *Buffer) enc_int64(p *Properties, base structPointer) error { - v := structPointer_Word64(base, p.field) - if word64_IsNil(v) { - return ErrNil - } - x := word64_Get(v) - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, x) - return nil -} - -func (o *Buffer) enc_proto3_int64(p *Properties, base structPointer) error { - v := structPointer_Word64Val(base, p.field) - x := word64Val_Get(v) - if x == 0 { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, x) - return nil -} - -func size_int64(p *Properties, base structPointer) (n int) { - v := structPointer_Word64(base, p.field) - if word64_IsNil(v) { - return 0 - } - x := word64_Get(v) - n += len(p.tagcode) - n += p.valSize(x) - return -} - -func size_proto3_int64(p *Properties, base structPointer) (n int) { - v := structPointer_Word64Val(base, p.field) - x := word64Val_Get(v) - if x == 0 && !p.oneof { - return 0 - } - n += len(p.tagcode) - n += p.valSize(x) - return -} - -// Encode a string. -func (o *Buffer) enc_string(p *Properties, base structPointer) error { - v := *structPointer_String(base, p.field) - if v == nil { - return ErrNil - } - x := *v - o.buf = append(o.buf, p.tagcode...) - o.EncodeStringBytes(x) - return nil -} - -func (o *Buffer) enc_proto3_string(p *Properties, base structPointer) error { - v := *structPointer_StringVal(base, p.field) - if v == "" { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeStringBytes(v) - return nil -} - -func size_string(p *Properties, base structPointer) (n int) { - v := *structPointer_String(base, p.field) - if v == nil { - return 0 - } - x := *v - n += len(p.tagcode) - n += sizeStringBytes(x) - return -} - -func size_proto3_string(p *Properties, base structPointer) (n int) { - v := *structPointer_StringVal(base, p.field) - if v == "" && !p.oneof { - return 0 - } - n += len(p.tagcode) - n += sizeStringBytes(v) - return -} - -// All protocol buffer fields are nillable, but be careful. -func isNil(v reflect.Value) bool { - switch v.Kind() { - case reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice: - return v.IsNil() - } - return false -} - -// Encode a message struct. -func (o *Buffer) enc_struct_message(p *Properties, base structPointer) error { - var state errorState - structp := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(structp) { - return ErrNil - } - - // Can the object marshal itself? - if p.isMarshaler { - m := structPointer_Interface(structp, p.stype).(Marshaler) - data, err := m.Marshal() - if err != nil && !state.shouldContinue(err, nil) { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - return state.err - } - - o.buf = append(o.buf, p.tagcode...) - return o.enc_len_struct(p.sprop, structp, &state) -} - -func size_struct_message(p *Properties, base structPointer) int { - structp := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(structp) { - return 0 - } - - // Can the object marshal itself? - if p.isMarshaler { - m := structPointer_Interface(structp, p.stype).(Marshaler) - data, _ := m.Marshal() - n0 := len(p.tagcode) - n1 := sizeRawBytes(data) - return n0 + n1 - } - - n0 := len(p.tagcode) - n1 := size_struct(p.sprop, structp) - n2 := sizeVarint(uint64(n1)) // size of encoded length - return n0 + n1 + n2 -} - -// Encode a group struct. -func (o *Buffer) enc_struct_group(p *Properties, base structPointer) error { - var state errorState - b := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(b) { - return ErrNil - } - - o.EncodeVarint(uint64((p.Tag << 3) | WireStartGroup)) - err := o.enc_struct(p.sprop, b) - if err != nil && !state.shouldContinue(err, nil) { - return err - } - o.EncodeVarint(uint64((p.Tag << 3) | WireEndGroup)) - return state.err -} - -func size_struct_group(p *Properties, base structPointer) (n int) { - b := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(b) { - return 0 - } - - n += sizeVarint(uint64((p.Tag << 3) | WireStartGroup)) - n += size_struct(p.sprop, b) - n += sizeVarint(uint64((p.Tag << 3) | WireEndGroup)) - return -} - -// Encode a slice of bools ([]bool). -func (o *Buffer) enc_slice_bool(p *Properties, base structPointer) error { - s := *structPointer_BoolSlice(base, p.field) - l := len(s) - if l == 0 { - return ErrNil - } - for _, x := range s { - o.buf = append(o.buf, p.tagcode...) - v := uint64(0) - if x { - v = 1 - } - p.valEnc(o, v) - } - return nil -} - -func size_slice_bool(p *Properties, base structPointer) int { - s := *structPointer_BoolSlice(base, p.field) - l := len(s) - if l == 0 { - return 0 - } - return l * (len(p.tagcode) + 1) // each bool takes exactly one byte -} - -// Encode a slice of bools ([]bool) in packed format. -func (o *Buffer) enc_slice_packed_bool(p *Properties, base structPointer) error { - s := *structPointer_BoolSlice(base, p.field) - l := len(s) - if l == 0 { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeVarint(uint64(l)) // each bool takes exactly one byte - for _, x := range s { - v := uint64(0) - if x { - v = 1 - } - p.valEnc(o, v) - } - return nil -} - -func size_slice_packed_bool(p *Properties, base structPointer) (n int) { - s := *structPointer_BoolSlice(base, p.field) - l := len(s) - if l == 0 { - return 0 - } - n += len(p.tagcode) - n += sizeVarint(uint64(l)) - n += l // each bool takes exactly one byte - return -} - -// Encode a slice of bytes ([]byte). -func (o *Buffer) enc_slice_byte(p *Properties, base structPointer) error { - s := *structPointer_Bytes(base, p.field) - if s == nil { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(s) - return nil -} - -func (o *Buffer) enc_proto3_slice_byte(p *Properties, base structPointer) error { - s := *structPointer_Bytes(base, p.field) - if len(s) == 0 { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(s) - return nil -} - -func size_slice_byte(p *Properties, base structPointer) (n int) { - s := *structPointer_Bytes(base, p.field) - if s == nil && !p.oneof { - return 0 - } - n += len(p.tagcode) - n += sizeRawBytes(s) - return -} - -func size_proto3_slice_byte(p *Properties, base structPointer) (n int) { - s := *structPointer_Bytes(base, p.field) - if len(s) == 0 && !p.oneof { - return 0 - } - n += len(p.tagcode) - n += sizeRawBytes(s) - return -} - -// Encode a slice of int32s ([]int32). -func (o *Buffer) enc_slice_int32(p *Properties, base structPointer) error { - s := structPointer_Word32Slice(base, p.field) - l := s.Len() - if l == 0 { - return ErrNil - } - for i := 0; i < l; i++ { - o.buf = append(o.buf, p.tagcode...) - x := int32(s.Index(i)) // permit sign extension to use full 64-bit range - p.valEnc(o, uint64(x)) - } - return nil -} - -func size_slice_int32(p *Properties, base structPointer) (n int) { - s := structPointer_Word32Slice(base, p.field) - l := s.Len() - if l == 0 { - return 0 - } - for i := 0; i < l; i++ { - n += len(p.tagcode) - x := int32(s.Index(i)) // permit sign extension to use full 64-bit range - n += p.valSize(uint64(x)) - } - return -} - -// Encode a slice of int32s ([]int32) in packed format. -func (o *Buffer) enc_slice_packed_int32(p *Properties, base structPointer) error { - s := structPointer_Word32Slice(base, p.field) - l := s.Len() - if l == 0 { - return ErrNil - } - // TODO: Reuse a Buffer. - buf := NewBuffer(nil) - for i := 0; i < l; i++ { - x := int32(s.Index(i)) // permit sign extension to use full 64-bit range - p.valEnc(buf, uint64(x)) - } - - o.buf = append(o.buf, p.tagcode...) - o.EncodeVarint(uint64(len(buf.buf))) - o.buf = append(o.buf, buf.buf...) - return nil -} - -func size_slice_packed_int32(p *Properties, base structPointer) (n int) { - s := structPointer_Word32Slice(base, p.field) - l := s.Len() - if l == 0 { - return 0 - } - var bufSize int - for i := 0; i < l; i++ { - x := int32(s.Index(i)) // permit sign extension to use full 64-bit range - bufSize += p.valSize(uint64(x)) - } - - n += len(p.tagcode) - n += sizeVarint(uint64(bufSize)) - n += bufSize - return -} - -// Encode a slice of uint32s ([]uint32). -// Exactly the same as int32, except for no sign extension. -func (o *Buffer) enc_slice_uint32(p *Properties, base structPointer) error { - s := structPointer_Word32Slice(base, p.field) - l := s.Len() - if l == 0 { - return ErrNil - } - for i := 0; i < l; i++ { - o.buf = append(o.buf, p.tagcode...) - x := s.Index(i) - p.valEnc(o, uint64(x)) - } - return nil -} - -func size_slice_uint32(p *Properties, base structPointer) (n int) { - s := structPointer_Word32Slice(base, p.field) - l := s.Len() - if l == 0 { - return 0 - } - for i := 0; i < l; i++ { - n += len(p.tagcode) - x := s.Index(i) - n += p.valSize(uint64(x)) - } - return -} - -// Encode a slice of uint32s ([]uint32) in packed format. -// Exactly the same as int32, except for no sign extension. -func (o *Buffer) enc_slice_packed_uint32(p *Properties, base structPointer) error { - s := structPointer_Word32Slice(base, p.field) - l := s.Len() - if l == 0 { - return ErrNil - } - // TODO: Reuse a Buffer. - buf := NewBuffer(nil) - for i := 0; i < l; i++ { - p.valEnc(buf, uint64(s.Index(i))) - } - - o.buf = append(o.buf, p.tagcode...) - o.EncodeVarint(uint64(len(buf.buf))) - o.buf = append(o.buf, buf.buf...) - return nil -} - -func size_slice_packed_uint32(p *Properties, base structPointer) (n int) { - s := structPointer_Word32Slice(base, p.field) - l := s.Len() - if l == 0 { - return 0 - } - var bufSize int - for i := 0; i < l; i++ { - bufSize += p.valSize(uint64(s.Index(i))) - } - - n += len(p.tagcode) - n += sizeVarint(uint64(bufSize)) - n += bufSize - return -} - -// Encode a slice of int64s ([]int64). -func (o *Buffer) enc_slice_int64(p *Properties, base structPointer) error { - s := structPointer_Word64Slice(base, p.field) - l := s.Len() - if l == 0 { - return ErrNil - } - for i := 0; i < l; i++ { - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, s.Index(i)) - } - return nil -} - -func size_slice_int64(p *Properties, base structPointer) (n int) { - s := structPointer_Word64Slice(base, p.field) - l := s.Len() - if l == 0 { - return 0 - } - for i := 0; i < l; i++ { - n += len(p.tagcode) - n += p.valSize(s.Index(i)) - } - return -} - -// Encode a slice of int64s ([]int64) in packed format. -func (o *Buffer) enc_slice_packed_int64(p *Properties, base structPointer) error { - s := structPointer_Word64Slice(base, p.field) - l := s.Len() - if l == 0 { - return ErrNil - } - // TODO: Reuse a Buffer. - buf := NewBuffer(nil) - for i := 0; i < l; i++ { - p.valEnc(buf, s.Index(i)) - } - - o.buf = append(o.buf, p.tagcode...) - o.EncodeVarint(uint64(len(buf.buf))) - o.buf = append(o.buf, buf.buf...) - return nil -} - -func size_slice_packed_int64(p *Properties, base structPointer) (n int) { - s := structPointer_Word64Slice(base, p.field) - l := s.Len() - if l == 0 { - return 0 - } - var bufSize int - for i := 0; i < l; i++ { - bufSize += p.valSize(s.Index(i)) - } - - n += len(p.tagcode) - n += sizeVarint(uint64(bufSize)) - n += bufSize - return -} - -// Encode a slice of slice of bytes ([][]byte). -func (o *Buffer) enc_slice_slice_byte(p *Properties, base structPointer) error { - ss := *structPointer_BytesSlice(base, p.field) - l := len(ss) - if l == 0 { - return ErrNil - } - for i := 0; i < l; i++ { - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(ss[i]) - } - return nil -} - -func size_slice_slice_byte(p *Properties, base structPointer) (n int) { - ss := *structPointer_BytesSlice(base, p.field) - l := len(ss) - if l == 0 { - return 0 - } - n += l * len(p.tagcode) - for i := 0; i < l; i++ { - n += sizeRawBytes(ss[i]) - } - return -} - -// Encode a slice of strings ([]string). -func (o *Buffer) enc_slice_string(p *Properties, base structPointer) error { - ss := *structPointer_StringSlice(base, p.field) - l := len(ss) - for i := 0; i < l; i++ { - o.buf = append(o.buf, p.tagcode...) - o.EncodeStringBytes(ss[i]) - } - return nil -} - -func size_slice_string(p *Properties, base structPointer) (n int) { - ss := *structPointer_StringSlice(base, p.field) - l := len(ss) - n += l * len(p.tagcode) - for i := 0; i < l; i++ { - n += sizeStringBytes(ss[i]) - } - return -} - -// Encode a slice of message structs ([]*struct). -func (o *Buffer) enc_slice_struct_message(p *Properties, base structPointer) error { - var state errorState - s := structPointer_StructPointerSlice(base, p.field) - l := s.Len() - - for i := 0; i < l; i++ { - structp := s.Index(i) - if structPointer_IsNil(structp) { - return errRepeatedHasNil - } - - // Can the object marshal itself? - if p.isMarshaler { - m := structPointer_Interface(structp, p.stype).(Marshaler) - data, err := m.Marshal() - if err != nil && !state.shouldContinue(err, nil) { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - continue - } - - o.buf = append(o.buf, p.tagcode...) - err := o.enc_len_struct(p.sprop, structp, &state) - if err != nil && !state.shouldContinue(err, nil) { - if err == ErrNil { - return errRepeatedHasNil - } - return err - } - } - return state.err -} - -func size_slice_struct_message(p *Properties, base structPointer) (n int) { - s := structPointer_StructPointerSlice(base, p.field) - l := s.Len() - n += l * len(p.tagcode) - for i := 0; i < l; i++ { - structp := s.Index(i) - if structPointer_IsNil(structp) { - return // return the size up to this point - } - - // Can the object marshal itself? - if p.isMarshaler { - m := structPointer_Interface(structp, p.stype).(Marshaler) - data, _ := m.Marshal() - n += sizeRawBytes(data) - continue - } - - n0 := size_struct(p.sprop, structp) - n1 := sizeVarint(uint64(n0)) // size of encoded length - n += n0 + n1 - } - return -} - -// Encode a slice of group structs ([]*struct). -func (o *Buffer) enc_slice_struct_group(p *Properties, base structPointer) error { - var state errorState - s := structPointer_StructPointerSlice(base, p.field) - l := s.Len() - - for i := 0; i < l; i++ { - b := s.Index(i) - if structPointer_IsNil(b) { - return errRepeatedHasNil - } - - o.EncodeVarint(uint64((p.Tag << 3) | WireStartGroup)) - - err := o.enc_struct(p.sprop, b) - - if err != nil && !state.shouldContinue(err, nil) { - if err == ErrNil { - return errRepeatedHasNil - } - return err - } - - o.EncodeVarint(uint64((p.Tag << 3) | WireEndGroup)) - } - return state.err -} - -func size_slice_struct_group(p *Properties, base structPointer) (n int) { - s := structPointer_StructPointerSlice(base, p.field) - l := s.Len() - - n += l * sizeVarint(uint64((p.Tag<<3)|WireStartGroup)) - n += l * sizeVarint(uint64((p.Tag<<3)|WireEndGroup)) - for i := 0; i < l; i++ { - b := s.Index(i) - if structPointer_IsNil(b) { - return // return size up to this point - } - - n += size_struct(p.sprop, b) - } - return -} - -// Encode an extension map. -func (o *Buffer) enc_map(p *Properties, base structPointer) error { - exts := structPointer_ExtMap(base, p.field) - if err := encodeExtensionsMap(*exts); err != nil { - return err - } - - return o.enc_map_body(*exts) -} - -func (o *Buffer) enc_exts(p *Properties, base structPointer) error { - exts := structPointer_Extensions(base, p.field) - - v, mu := exts.extensionsRead() - if v == nil { - return nil - } - - mu.Lock() - defer mu.Unlock() - if err := encodeExtensionsMap(v); err != nil { - return err - } - - return o.enc_map_body(v) -} - -func (o *Buffer) enc_map_body(v map[int32]Extension) error { - // Fast-path for common cases: zero or one extensions. - if len(v) <= 1 { - for _, e := range v { - o.buf = append(o.buf, e.enc...) - } - return nil - } - - // Sort keys to provide a deterministic encoding. - keys := make([]int, 0, len(v)) - for k := range v { - keys = append(keys, int(k)) - } - sort.Ints(keys) - - for _, k := range keys { - o.buf = append(o.buf, v[int32(k)].enc...) - } - return nil -} - -func size_map(p *Properties, base structPointer) int { - v := structPointer_ExtMap(base, p.field) - return extensionsMapSize(*v) -} - -func size_exts(p *Properties, base structPointer) int { - v := structPointer_Extensions(base, p.field) - return extensionsSize(v) -} - -// Encode a map field. -func (o *Buffer) enc_new_map(p *Properties, base structPointer) error { - var state errorState // XXX: or do we need to plumb this through? - - /* - A map defined as - map map_field = N; - is encoded in the same way as - message MapFieldEntry { - key_type key = 1; - value_type value = 2; - } - repeated MapFieldEntry map_field = N; - */ - - v := structPointer_NewAt(base, p.field, p.mtype).Elem() // map[K]V - if v.Len() == 0 { - return nil - } - - keycopy, valcopy, keybase, valbase := mapEncodeScratch(p.mtype) - - enc := func() error { - if err := p.mkeyprop.enc(o, p.mkeyprop, keybase); err != nil { - return err - } - if err := p.mvalprop.enc(o, p.mvalprop, valbase); err != nil && err != ErrNil { - return err - } - return nil - } - - // Don't sort map keys. It is not required by the spec, and C++ doesn't do it. - for _, key := range v.MapKeys() { - val := v.MapIndex(key) - - keycopy.Set(key) - valcopy.Set(val) - - o.buf = append(o.buf, p.tagcode...) - if err := o.enc_len_thing(enc, &state); err != nil { - return err - } - } - return nil -} - -func size_new_map(p *Properties, base structPointer) int { - v := structPointer_NewAt(base, p.field, p.mtype).Elem() // map[K]V - - keycopy, valcopy, keybase, valbase := mapEncodeScratch(p.mtype) - - n := 0 - for _, key := range v.MapKeys() { - val := v.MapIndex(key) - keycopy.Set(key) - valcopy.Set(val) - - // Tag codes for key and val are the responsibility of the sub-sizer. - keysize := p.mkeyprop.size(p.mkeyprop, keybase) - valsize := p.mvalprop.size(p.mvalprop, valbase) - entry := keysize + valsize - // Add on tag code and length of map entry itself. - n += len(p.tagcode) + sizeVarint(uint64(entry)) + entry - } - return n -} - -// mapEncodeScratch returns a new reflect.Value matching the map's value type, -// and a structPointer suitable for passing to an encoder or sizer. -func mapEncodeScratch(mapType reflect.Type) (keycopy, valcopy reflect.Value, keybase, valbase structPointer) { - // Prepare addressable doubly-indirect placeholders for the key and value types. - // This is needed because the element-type encoders expect **T, but the map iteration produces T. - - keycopy = reflect.New(mapType.Key()).Elem() // addressable K - keyptr := reflect.New(reflect.PtrTo(keycopy.Type())).Elem() // addressable *K - keyptr.Set(keycopy.Addr()) // - keybase = toStructPointer(keyptr.Addr()) // **K - - // Value types are more varied and require special handling. - switch mapType.Elem().Kind() { - case reflect.Slice: - // []byte - var dummy []byte - valcopy = reflect.ValueOf(&dummy).Elem() // addressable []byte - valbase = toStructPointer(valcopy.Addr()) - case reflect.Ptr: - // message; the generated field type is map[K]*Msg (so V is *Msg), - // so we only need one level of indirection. - valcopy = reflect.New(mapType.Elem()).Elem() // addressable V - valbase = toStructPointer(valcopy.Addr()) - default: - // everything else - valcopy = reflect.New(mapType.Elem()).Elem() // addressable V - valptr := reflect.New(reflect.PtrTo(valcopy.Type())).Elem() // addressable *V - valptr.Set(valcopy.Addr()) // - valbase = toStructPointer(valptr.Addr()) // **V - } - return -} - -// Encode a struct. -func (o *Buffer) enc_struct(prop *StructProperties, base structPointer) error { - var state errorState - // Encode fields in tag order so that decoders may use optimizations - // that depend on the ordering. - // https://developers.google.com/protocol-buffers/docs/encoding#order - for _, i := range prop.order { - p := prop.Prop[i] - if p.enc != nil { - err := p.enc(o, p, base) - if err != nil { - if err == ErrNil { - if p.Required && state.err == nil { - state.err = &RequiredNotSetError{p.Name} - } - } else if err == errRepeatedHasNil { - // Give more context to nil values in repeated fields. - return errors.New("repeated field " + p.OrigName + " has nil element") - } else if !state.shouldContinue(err, p) { - return err - } - } - if len(o.buf) > maxMarshalSize { - return ErrTooLarge - } - } - } - - // Do oneof fields. - if prop.oneofMarshaler != nil { - m := structPointer_Interface(base, prop.stype).(Message) - if err := prop.oneofMarshaler(m, o); err == ErrNil { - return errOneofHasNil - } else if err != nil { - return err - } - } - - // Add unrecognized fields at the end. - if prop.unrecField.IsValid() { - v := *structPointer_Bytes(base, prop.unrecField) - if len(o.buf)+len(v) > maxMarshalSize { - return ErrTooLarge - } - if len(v) > 0 { - o.buf = append(o.buf, v...) - } - } - - return state.err -} - -func size_struct(prop *StructProperties, base structPointer) (n int) { - for _, i := range prop.order { - p := prop.Prop[i] - if p.size != nil { - n += p.size(p, base) - } - } - - // Add unrecognized fields at the end. - if prop.unrecField.IsValid() { - v := *structPointer_Bytes(base, prop.unrecField) - n += len(v) - } - - // Factor in any oneof fields. - if prop.oneofSizer != nil { - m := structPointer_Interface(base, prop.stype).(Message) - n += prop.oneofSizer(m) - } - - return -} - -var zeroes [20]byte // longer than any conceivable sizeVarint - -// Encode a struct, preceded by its encoded length (as a varint). -func (o *Buffer) enc_len_struct(prop *StructProperties, base structPointer, state *errorState) error { - return o.enc_len_thing(func() error { return o.enc_struct(prop, base) }, state) -} - -// Encode something, preceded by its encoded length (as a varint). -func (o *Buffer) enc_len_thing(enc func() error, state *errorState) error { - iLen := len(o.buf) - o.buf = append(o.buf, 0, 0, 0, 0) // reserve four bytes for length - iMsg := len(o.buf) - err := enc() - if err != nil && !state.shouldContinue(err, nil) { - return err - } - lMsg := len(o.buf) - iMsg - lLen := sizeVarint(uint64(lMsg)) - switch x := lLen - (iMsg - iLen); { - case x > 0: // actual length is x bytes larger than the space we reserved - // Move msg x bytes right. - o.buf = append(o.buf, zeroes[:x]...) - copy(o.buf[iMsg+x:], o.buf[iMsg:iMsg+lMsg]) - case x < 0: // actual length is x bytes smaller than the space we reserved - // Move msg x bytes left. - copy(o.buf[iMsg+x:], o.buf[iMsg:iMsg+lMsg]) - o.buf = o.buf[:len(o.buf)+x] // x is negative - } - // Encode the length in the reserved space. - o.buf = o.buf[:iLen] - o.EncodeVarint(uint64(lMsg)) - o.buf = o.buf[:len(o.buf)+lMsg] - return state.err -} - -// errorState maintains the first error that occurs and updates that error -// with additional context. -type errorState struct { - err error -} - -// shouldContinue reports whether encoding should continue upon encountering the -// given error. If the error is RequiredNotSetError, shouldContinue returns true -// and, if this is the first appearance of that error, remembers it for future -// reporting. -// -// If prop is not nil, it may update any error with additional context about the -// field with the error. -func (s *errorState) shouldContinue(err error, prop *Properties) bool { - // Ignore unset required fields. - reqNotSet, ok := err.(*RequiredNotSetError) - if !ok { - return false - } - if s.err == nil { - if prop != nil { - err = &RequiredNotSetError{prop.Name + "." + reqNotSet.field} - } - s.err = err - } - return true -} diff --git a/vendor/github.com/gogo/protobuf/proto/encode_gogo.go b/vendor/github.com/gogo/protobuf/proto/encode_gogo.go deleted file mode 100644 index 32111b7f41d79..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/encode_gogo.go +++ /dev/null @@ -1,350 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// http://github.com/golang/protobuf/ -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "reflect" -) - -func NewRequiredNotSetError(field string) *RequiredNotSetError { - return &RequiredNotSetError{field} -} - -type Sizer interface { - Size() int -} - -func (o *Buffer) enc_ext_slice_byte(p *Properties, base structPointer) error { - s := *structPointer_Bytes(base, p.field) - if s == nil { - return ErrNil - } - o.buf = append(o.buf, s...) - return nil -} - -func size_ext_slice_byte(p *Properties, base structPointer) (n int) { - s := *structPointer_Bytes(base, p.field) - if s == nil { - return 0 - } - n += len(s) - return -} - -// Encode a reference to bool pointer. -func (o *Buffer) enc_ref_bool(p *Properties, base structPointer) error { - v := *structPointer_BoolVal(base, p.field) - x := 0 - if v { - x = 1 - } - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, uint64(x)) - return nil -} - -func size_ref_bool(p *Properties, base structPointer) int { - return len(p.tagcode) + 1 // each bool takes exactly one byte -} - -// Encode a reference to int32 pointer. -func (o *Buffer) enc_ref_int32(p *Properties, base structPointer) error { - v := structPointer_Word32Val(base, p.field) - x := int32(word32Val_Get(v)) - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, uint64(x)) - return nil -} - -func size_ref_int32(p *Properties, base structPointer) (n int) { - v := structPointer_Word32Val(base, p.field) - x := int32(word32Val_Get(v)) - n += len(p.tagcode) - n += p.valSize(uint64(x)) - return -} - -func (o *Buffer) enc_ref_uint32(p *Properties, base structPointer) error { - v := structPointer_Word32Val(base, p.field) - x := word32Val_Get(v) - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, uint64(x)) - return nil -} - -func size_ref_uint32(p *Properties, base structPointer) (n int) { - v := structPointer_Word32Val(base, p.field) - x := word32Val_Get(v) - n += len(p.tagcode) - n += p.valSize(uint64(x)) - return -} - -// Encode a reference to an int64 pointer. -func (o *Buffer) enc_ref_int64(p *Properties, base structPointer) error { - v := structPointer_Word64Val(base, p.field) - x := word64Val_Get(v) - o.buf = append(o.buf, p.tagcode...) - p.valEnc(o, x) - return nil -} - -func size_ref_int64(p *Properties, base structPointer) (n int) { - v := structPointer_Word64Val(base, p.field) - x := word64Val_Get(v) - n += len(p.tagcode) - n += p.valSize(x) - return -} - -// Encode a reference to a string pointer. -func (o *Buffer) enc_ref_string(p *Properties, base structPointer) error { - v := *structPointer_StringVal(base, p.field) - o.buf = append(o.buf, p.tagcode...) - o.EncodeStringBytes(v) - return nil -} - -func size_ref_string(p *Properties, base structPointer) (n int) { - v := *structPointer_StringVal(base, p.field) - n += len(p.tagcode) - n += sizeStringBytes(v) - return -} - -// Encode a reference to a message struct. -func (o *Buffer) enc_ref_struct_message(p *Properties, base structPointer) error { - var state errorState - structp := structPointer_GetRefStructPointer(base, p.field) - if structPointer_IsNil(structp) { - return ErrNil - } - - // Can the object marshal itself? - if p.isMarshaler { - m := structPointer_Interface(structp, p.stype).(Marshaler) - data, err := m.Marshal() - if err != nil && !state.shouldContinue(err, nil) { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - return nil - } - - o.buf = append(o.buf, p.tagcode...) - return o.enc_len_struct(p.sprop, structp, &state) -} - -//TODO this is only copied, please fix this -func size_ref_struct_message(p *Properties, base structPointer) int { - structp := structPointer_GetRefStructPointer(base, p.field) - if structPointer_IsNil(structp) { - return 0 - } - - // Can the object marshal itself? - if p.isMarshaler { - m := structPointer_Interface(structp, p.stype).(Marshaler) - data, _ := m.Marshal() - n0 := len(p.tagcode) - n1 := sizeRawBytes(data) - return n0 + n1 - } - - n0 := len(p.tagcode) - n1 := size_struct(p.sprop, structp) - n2 := sizeVarint(uint64(n1)) // size of encoded length - return n0 + n1 + n2 -} - -// Encode a slice of references to message struct pointers ([]struct). -func (o *Buffer) enc_slice_ref_struct_message(p *Properties, base structPointer) error { - var state errorState - ss := structPointer_StructRefSlice(base, p.field, p.stype.Size()) - l := ss.Len() - for i := 0; i < l; i++ { - structp := ss.Index(i) - if structPointer_IsNil(structp) { - return errRepeatedHasNil - } - - // Can the object marshal itself? - if p.isMarshaler { - m := structPointer_Interface(structp, p.stype).(Marshaler) - data, err := m.Marshal() - if err != nil && !state.shouldContinue(err, nil) { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - continue - } - - o.buf = append(o.buf, p.tagcode...) - err := o.enc_len_struct(p.sprop, structp, &state) - if err != nil && !state.shouldContinue(err, nil) { - if err == ErrNil { - return errRepeatedHasNil - } - return err - } - - } - return state.err -} - -//TODO this is only copied, please fix this -func size_slice_ref_struct_message(p *Properties, base structPointer) (n int) { - ss := structPointer_StructRefSlice(base, p.field, p.stype.Size()) - l := ss.Len() - n += l * len(p.tagcode) - for i := 0; i < l; i++ { - structp := ss.Index(i) - if structPointer_IsNil(structp) { - return // return the size up to this point - } - - // Can the object marshal itself? - if p.isMarshaler { - m := structPointer_Interface(structp, p.stype).(Marshaler) - data, _ := m.Marshal() - n += len(p.tagcode) - n += sizeRawBytes(data) - continue - } - - n0 := size_struct(p.sprop, structp) - n1 := sizeVarint(uint64(n0)) // size of encoded length - n += n0 + n1 - } - return -} - -func (o *Buffer) enc_custom_bytes(p *Properties, base structPointer) error { - i := structPointer_InterfaceRef(base, p.field, p.ctype) - if i == nil { - return ErrNil - } - custom := i.(Marshaler) - data, err := custom.Marshal() - if err != nil { - return err - } - if data == nil { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - return nil -} - -func size_custom_bytes(p *Properties, base structPointer) (n int) { - n += len(p.tagcode) - i := structPointer_InterfaceRef(base, p.field, p.ctype) - if i == nil { - return 0 - } - custom := i.(Marshaler) - data, _ := custom.Marshal() - n += sizeRawBytes(data) - return -} - -func (o *Buffer) enc_custom_ref_bytes(p *Properties, base structPointer) error { - custom := structPointer_InterfaceAt(base, p.field, p.ctype).(Marshaler) - data, err := custom.Marshal() - if err != nil { - return err - } - if data == nil { - return ErrNil - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - return nil -} - -func size_custom_ref_bytes(p *Properties, base structPointer) (n int) { - n += len(p.tagcode) - i := structPointer_InterfaceAt(base, p.field, p.ctype) - if i == nil { - return 0 - } - custom := i.(Marshaler) - data, _ := custom.Marshal() - n += sizeRawBytes(data) - return -} - -func (o *Buffer) enc_custom_slice_bytes(p *Properties, base structPointer) error { - inter := structPointer_InterfaceRef(base, p.field, p.ctype) - if inter == nil { - return ErrNil - } - slice := reflect.ValueOf(inter) - l := slice.Len() - for i := 0; i < l; i++ { - v := slice.Index(i) - custom := v.Interface().(Marshaler) - data, err := custom.Marshal() - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - } - return nil -} - -func size_custom_slice_bytes(p *Properties, base structPointer) (n int) { - inter := structPointer_InterfaceRef(base, p.field, p.ctype) - if inter == nil { - return 0 - } - slice := reflect.ValueOf(inter) - l := slice.Len() - n += l * len(p.tagcode) - for i := 0; i < l; i++ { - v := slice.Index(i) - custom := v.Interface().(Marshaler) - data, _ := custom.Marshal() - n += sizeRawBytes(data) - } - return -} diff --git a/vendor/github.com/gogo/protobuf/proto/equal.go b/vendor/github.com/gogo/protobuf/proto/equal.go deleted file mode 100644 index 2ed1cf596664d..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/equal.go +++ /dev/null @@ -1,300 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2011 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -// Protocol buffer comparison. - -package proto - -import ( - "bytes" - "log" - "reflect" - "strings" -) - -/* -Equal returns true iff protocol buffers a and b are equal. -The arguments must both be pointers to protocol buffer structs. - -Equality is defined in this way: - - Two messages are equal iff they are the same type, - corresponding fields are equal, unknown field sets - are equal, and extensions sets are equal. - - Two set scalar fields are equal iff their values are equal. - If the fields are of a floating-point type, remember that - NaN != x for all x, including NaN. If the message is defined - in a proto3 .proto file, fields are not "set"; specifically, - zero length proto3 "bytes" fields are equal (nil == {}). - - Two repeated fields are equal iff their lengths are the same, - and their corresponding elements are equal. Note a "bytes" field, - although represented by []byte, is not a repeated field and the - rule for the scalar fields described above applies. - - Two unset fields are equal. - - Two unknown field sets are equal if their current - encoded state is equal. - - Two extension sets are equal iff they have corresponding - elements that are pairwise equal. - - Two map fields are equal iff their lengths are the same, - and they contain the same set of elements. Zero-length map - fields are equal. - - Every other combination of things are not equal. - -The return value is undefined if a and b are not protocol buffers. -*/ -func Equal(a, b Message) bool { - if a == nil || b == nil { - return a == b - } - v1, v2 := reflect.ValueOf(a), reflect.ValueOf(b) - if v1.Type() != v2.Type() { - return false - } - if v1.Kind() == reflect.Ptr { - if v1.IsNil() { - return v2.IsNil() - } - if v2.IsNil() { - return false - } - v1, v2 = v1.Elem(), v2.Elem() - } - if v1.Kind() != reflect.Struct { - return false - } - return equalStruct(v1, v2) -} - -// v1 and v2 are known to have the same type. -func equalStruct(v1, v2 reflect.Value) bool { - sprop := GetProperties(v1.Type()) - for i := 0; i < v1.NumField(); i++ { - f := v1.Type().Field(i) - if strings.HasPrefix(f.Name, "XXX_") { - continue - } - f1, f2 := v1.Field(i), v2.Field(i) - if f.Type.Kind() == reflect.Ptr { - if n1, n2 := f1.IsNil(), f2.IsNil(); n1 && n2 { - // both unset - continue - } else if n1 != n2 { - // set/unset mismatch - return false - } - b1, ok := f1.Interface().(raw) - if ok { - b2 := f2.Interface().(raw) - // RawMessage - if !bytes.Equal(b1.Bytes(), b2.Bytes()) { - return false - } - continue - } - f1, f2 = f1.Elem(), f2.Elem() - } - if !equalAny(f1, f2, sprop.Prop[i]) { - return false - } - } - - if em1 := v1.FieldByName("XXX_InternalExtensions"); em1.IsValid() { - em2 := v2.FieldByName("XXX_InternalExtensions") - if !equalExtensions(v1.Type(), em1.Interface().(XXX_InternalExtensions), em2.Interface().(XXX_InternalExtensions)) { - return false - } - } - - if em1 := v1.FieldByName("XXX_extensions"); em1.IsValid() { - em2 := v2.FieldByName("XXX_extensions") - if !equalExtMap(v1.Type(), em1.Interface().(map[int32]Extension), em2.Interface().(map[int32]Extension)) { - return false - } - } - - uf := v1.FieldByName("XXX_unrecognized") - if !uf.IsValid() { - return true - } - - u1 := uf.Bytes() - u2 := v2.FieldByName("XXX_unrecognized").Bytes() - if !bytes.Equal(u1, u2) { - return false - } - - return true -} - -// v1 and v2 are known to have the same type. -// prop may be nil. -func equalAny(v1, v2 reflect.Value, prop *Properties) bool { - if v1.Type() == protoMessageType { - m1, _ := v1.Interface().(Message) - m2, _ := v2.Interface().(Message) - return Equal(m1, m2) - } - switch v1.Kind() { - case reflect.Bool: - return v1.Bool() == v2.Bool() - case reflect.Float32, reflect.Float64: - return v1.Float() == v2.Float() - case reflect.Int32, reflect.Int64: - return v1.Int() == v2.Int() - case reflect.Interface: - // Probably a oneof field; compare the inner values. - n1, n2 := v1.IsNil(), v2.IsNil() - if n1 || n2 { - return n1 == n2 - } - e1, e2 := v1.Elem(), v2.Elem() - if e1.Type() != e2.Type() { - return false - } - return equalAny(e1, e2, nil) - case reflect.Map: - if v1.Len() != v2.Len() { - return false - } - for _, key := range v1.MapKeys() { - val2 := v2.MapIndex(key) - if !val2.IsValid() { - // This key was not found in the second map. - return false - } - if !equalAny(v1.MapIndex(key), val2, nil) { - return false - } - } - return true - case reflect.Ptr: - // Maps may have nil values in them, so check for nil. - if v1.IsNil() && v2.IsNil() { - return true - } - if v1.IsNil() != v2.IsNil() { - return false - } - return equalAny(v1.Elem(), v2.Elem(), prop) - case reflect.Slice: - if v1.Type().Elem().Kind() == reflect.Uint8 { - // short circuit: []byte - - // Edge case: if this is in a proto3 message, a zero length - // bytes field is considered the zero value. - if prop != nil && prop.proto3 && v1.Len() == 0 && v2.Len() == 0 { - return true - } - if v1.IsNil() != v2.IsNil() { - return false - } - return bytes.Equal(v1.Interface().([]byte), v2.Interface().([]byte)) - } - - if v1.Len() != v2.Len() { - return false - } - for i := 0; i < v1.Len(); i++ { - if !equalAny(v1.Index(i), v2.Index(i), prop) { - return false - } - } - return true - case reflect.String: - return v1.Interface().(string) == v2.Interface().(string) - case reflect.Struct: - return equalStruct(v1, v2) - case reflect.Uint32, reflect.Uint64: - return v1.Uint() == v2.Uint() - } - - // unknown type, so not a protocol buffer - log.Printf("proto: don't know how to compare %v", v1) - return false -} - -// base is the struct type that the extensions are based on. -// x1 and x2 are InternalExtensions. -func equalExtensions(base reflect.Type, x1, x2 XXX_InternalExtensions) bool { - em1, _ := x1.extensionsRead() - em2, _ := x2.extensionsRead() - return equalExtMap(base, em1, em2) -} - -func equalExtMap(base reflect.Type, em1, em2 map[int32]Extension) bool { - if len(em1) != len(em2) { - return false - } - - for extNum, e1 := range em1 { - e2, ok := em2[extNum] - if !ok { - return false - } - - m1, m2 := e1.value, e2.value - - if m1 != nil && m2 != nil { - // Both are unencoded. - if !equalAny(reflect.ValueOf(m1), reflect.ValueOf(m2), nil) { - return false - } - continue - } - - // At least one is encoded. To do a semantically correct comparison - // we need to unmarshal them first. - var desc *ExtensionDesc - if m := extensionMaps[base]; m != nil { - desc = m[extNum] - } - if desc == nil { - log.Printf("proto: don't know how to compare extension %d of %v", extNum, base) - continue - } - var err error - if m1 == nil { - m1, err = decodeExtension(e1.enc, desc) - } - if m2 == nil && err == nil { - m2, err = decodeExtension(e2.enc, desc) - } - if err != nil { - // The encoded form is invalid. - log.Printf("proto: badly encoded extension %d of %v: %v", extNum, base, err) - return false - } - if !equalAny(reflect.ValueOf(m1), reflect.ValueOf(m2), nil) { - return false - } - } - - return true -} diff --git a/vendor/github.com/gogo/protobuf/proto/extensions.go b/vendor/github.com/gogo/protobuf/proto/extensions.go deleted file mode 100644 index 0dfcb538e8f1b..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/extensions.go +++ /dev/null @@ -1,693 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -/* - * Types and routines for supporting protocol buffer extensions. - */ - -import ( - "errors" - "fmt" - "reflect" - "strconv" - "sync" -) - -// ErrMissingExtension is the error returned by GetExtension if the named extension is not in the message. -var ErrMissingExtension = errors.New("proto: missing extension") - -// ExtensionRange represents a range of message extensions for a protocol buffer. -// Used in code generated by the protocol compiler. -type ExtensionRange struct { - Start, End int32 // both inclusive -} - -// extendableProto is an interface implemented by any protocol buffer generated by the current -// proto compiler that may be extended. -type extendableProto interface { - Message - ExtensionRangeArray() []ExtensionRange - extensionsWrite() map[int32]Extension - extensionsRead() (map[int32]Extension, sync.Locker) -} - -// extendableProtoV1 is an interface implemented by a protocol buffer generated by the previous -// version of the proto compiler that may be extended. -type extendableProtoV1 interface { - Message - ExtensionRangeArray() []ExtensionRange - ExtensionMap() map[int32]Extension -} - -type extensionsBytes interface { - Message - ExtensionRangeArray() []ExtensionRange - GetExtensions() *[]byte -} - -// extensionAdapter is a wrapper around extendableProtoV1 that implements extendableProto. -type extensionAdapter struct { - extendableProtoV1 -} - -func (e extensionAdapter) extensionsWrite() map[int32]Extension { - return e.ExtensionMap() -} - -func (e extensionAdapter) extensionsRead() (map[int32]Extension, sync.Locker) { - return e.ExtensionMap(), notLocker{} -} - -// notLocker is a sync.Locker whose Lock and Unlock methods are nops. -type notLocker struct{} - -func (n notLocker) Lock() {} -func (n notLocker) Unlock() {} - -// extendable returns the extendableProto interface for the given generated proto message. -// If the proto message has the old extension format, it returns a wrapper that implements -// the extendableProto interface. -func extendable(p interface{}) (extendableProto, bool) { - if ep, ok := p.(extendableProto); ok { - return ep, ok - } - if ep, ok := p.(extendableProtoV1); ok { - return extensionAdapter{ep}, ok - } - return nil, false -} - -// XXX_InternalExtensions is an internal representation of proto extensions. -// -// Each generated message struct type embeds an anonymous XXX_InternalExtensions field, -// thus gaining the unexported 'extensions' method, which can be called only from the proto package. -// -// The methods of XXX_InternalExtensions are not concurrency safe in general, -// but calls to logically read-only methods such as has and get may be executed concurrently. -type XXX_InternalExtensions struct { - // The struct must be indirect so that if a user inadvertently copies a - // generated message and its embedded XXX_InternalExtensions, they - // avoid the mayhem of a copied mutex. - // - // The mutex serializes all logically read-only operations to p.extensionMap. - // It is up to the client to ensure that write operations to p.extensionMap are - // mutually exclusive with other accesses. - p *struct { - mu sync.Mutex - extensionMap map[int32]Extension - } -} - -// extensionsWrite returns the extension map, creating it on first use. -func (e *XXX_InternalExtensions) extensionsWrite() map[int32]Extension { - if e.p == nil { - e.p = new(struct { - mu sync.Mutex - extensionMap map[int32]Extension - }) - e.p.extensionMap = make(map[int32]Extension) - } - return e.p.extensionMap -} - -// extensionsRead returns the extensions map for read-only use. It may be nil. -// The caller must hold the returned mutex's lock when accessing Elements within the map. -func (e *XXX_InternalExtensions) extensionsRead() (map[int32]Extension, sync.Locker) { - if e.p == nil { - return nil, nil - } - return e.p.extensionMap, &e.p.mu -} - -type extensionRange interface { - Message - ExtensionRangeArray() []ExtensionRange -} - -var extendableProtoType = reflect.TypeOf((*extendableProto)(nil)).Elem() -var extendableProtoV1Type = reflect.TypeOf((*extendableProtoV1)(nil)).Elem() -var extendableBytesType = reflect.TypeOf((*extensionsBytes)(nil)).Elem() -var extensionRangeType = reflect.TypeOf((*extensionRange)(nil)).Elem() - -// ExtensionDesc represents an extension specification. -// Used in generated code from the protocol compiler. -type ExtensionDesc struct { - ExtendedType Message // nil pointer to the type that is being extended - ExtensionType interface{} // nil pointer to the extension type - Field int32 // field number - Name string // fully-qualified name of extension, for text formatting - Tag string // protobuf tag style - Filename string // name of the file in which the extension is defined -} - -func (ed *ExtensionDesc) repeated() bool { - t := reflect.TypeOf(ed.ExtensionType) - return t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8 -} - -// Extension represents an extension in a message. -type Extension struct { - // When an extension is stored in a message using SetExtension - // only desc and value are set. When the message is marshaled - // enc will be set to the encoded form of the message. - // - // When a message is unmarshaled and contains extensions, each - // extension will have only enc set. When such an extension is - // accessed using GetExtension (or GetExtensions) desc and value - // will be set. - desc *ExtensionDesc - value interface{} - enc []byte -} - -// SetRawExtension is for testing only. -func SetRawExtension(base Message, id int32, b []byte) { - if ebase, ok := base.(extensionsBytes); ok { - clearExtension(base, id) - ext := ebase.GetExtensions() - *ext = append(*ext, b...) - return - } - epb, ok := extendable(base) - if !ok { - return - } - extmap := epb.extensionsWrite() - extmap[id] = Extension{enc: b} -} - -// isExtensionField returns true iff the given field number is in an extension range. -func isExtensionField(pb extensionRange, field int32) bool { - for _, er := range pb.ExtensionRangeArray() { - if er.Start <= field && field <= er.End { - return true - } - } - return false -} - -// checkExtensionTypes checks that the given extension is valid for pb. -func checkExtensionTypes(pb extendableProto, extension *ExtensionDesc) error { - var pbi interface{} = pb - // Check the extended type. - if ea, ok := pbi.(extensionAdapter); ok { - pbi = ea.extendableProtoV1 - } - if a, b := reflect.TypeOf(pbi), reflect.TypeOf(extension.ExtendedType); a != b { - return errors.New("proto: bad extended type; " + b.String() + " does not extend " + a.String()) - } - // Check the range. - if !isExtensionField(pb, extension.Field) { - return errors.New("proto: bad extension number; not in declared ranges") - } - return nil -} - -// extPropKey is sufficient to uniquely identify an extension. -type extPropKey struct { - base reflect.Type - field int32 -} - -var extProp = struct { - sync.RWMutex - m map[extPropKey]*Properties -}{ - m: make(map[extPropKey]*Properties), -} - -func extensionProperties(ed *ExtensionDesc) *Properties { - key := extPropKey{base: reflect.TypeOf(ed.ExtendedType), field: ed.Field} - - extProp.RLock() - if prop, ok := extProp.m[key]; ok { - extProp.RUnlock() - return prop - } - extProp.RUnlock() - - extProp.Lock() - defer extProp.Unlock() - // Check again. - if prop, ok := extProp.m[key]; ok { - return prop - } - - prop := new(Properties) - prop.Init(reflect.TypeOf(ed.ExtensionType), "unknown_name", ed.Tag, nil) - extProp.m[key] = prop - return prop -} - -// encode encodes any unmarshaled (unencoded) extensions in e. -func encodeExtensions(e *XXX_InternalExtensions) error { - m, mu := e.extensionsRead() - if m == nil { - return nil // fast path - } - mu.Lock() - defer mu.Unlock() - return encodeExtensionsMap(m) -} - -// encode encodes any unmarshaled (unencoded) extensions in e. -func encodeExtensionsMap(m map[int32]Extension) error { - for k, e := range m { - if e.value == nil || e.desc == nil { - // Extension is only in its encoded form. - continue - } - - // We don't skip extensions that have an encoded form set, - // because the extension value may have been mutated after - // the last time this function was called. - - et := reflect.TypeOf(e.desc.ExtensionType) - props := extensionProperties(e.desc) - - p := NewBuffer(nil) - // If e.value has type T, the encoder expects a *struct{ X T }. - // Pass a *T with a zero field and hope it all works out. - x := reflect.New(et) - x.Elem().Set(reflect.ValueOf(e.value)) - if err := props.enc(p, props, toStructPointer(x)); err != nil { - return err - } - e.enc = p.buf - m[k] = e - } - return nil -} - -func extensionsSize(e *XXX_InternalExtensions) (n int) { - m, mu := e.extensionsRead() - if m == nil { - return 0 - } - mu.Lock() - defer mu.Unlock() - return extensionsMapSize(m) -} - -func extensionsMapSize(m map[int32]Extension) (n int) { - for _, e := range m { - if e.value == nil || e.desc == nil { - // Extension is only in its encoded form. - n += len(e.enc) - continue - } - - // We don't skip extensions that have an encoded form set, - // because the extension value may have been mutated after - // the last time this function was called. - - et := reflect.TypeOf(e.desc.ExtensionType) - props := extensionProperties(e.desc) - - // If e.value has type T, the encoder expects a *struct{ X T }. - // Pass a *T with a zero field and hope it all works out. - x := reflect.New(et) - x.Elem().Set(reflect.ValueOf(e.value)) - n += props.size(props, toStructPointer(x)) - } - return -} - -// HasExtension returns whether the given extension is present in pb. -func HasExtension(pb Message, extension *ExtensionDesc) bool { - if epb, doki := pb.(extensionsBytes); doki { - ext := epb.GetExtensions() - buf := *ext - o := 0 - for o < len(buf) { - tag, n := DecodeVarint(buf[o:]) - fieldNum := int32(tag >> 3) - if int32(fieldNum) == extension.Field { - return true - } - wireType := int(tag & 0x7) - o += n - l, err := size(buf[o:], wireType) - if err != nil { - return false - } - o += l - } - return false - } - // TODO: Check types, field numbers, etc.? - epb, ok := extendable(pb) - if !ok { - return false - } - extmap, mu := epb.extensionsRead() - if extmap == nil { - return false - } - mu.Lock() - _, ok = extmap[extension.Field] - mu.Unlock() - return ok -} - -func deleteExtension(pb extensionsBytes, theFieldNum int32, offset int) int { - ext := pb.GetExtensions() - for offset < len(*ext) { - tag, n1 := DecodeVarint((*ext)[offset:]) - fieldNum := int32(tag >> 3) - wireType := int(tag & 0x7) - n2, err := size((*ext)[offset+n1:], wireType) - if err != nil { - panic(err) - } - newOffset := offset + n1 + n2 - if fieldNum == theFieldNum { - *ext = append((*ext)[:offset], (*ext)[newOffset:]...) - return offset - } - offset = newOffset - } - return -1 -} - -// ClearExtension removes the given extension from pb. -func ClearExtension(pb Message, extension *ExtensionDesc) { - clearExtension(pb, extension.Field) -} - -func clearExtension(pb Message, fieldNum int32) { - if epb, doki := pb.(extensionsBytes); doki { - offset := 0 - for offset != -1 { - offset = deleteExtension(epb, fieldNum, offset) - } - return - } - epb, ok := extendable(pb) - if !ok { - return - } - // TODO: Check types, field numbers, etc.? - extmap := epb.extensionsWrite() - delete(extmap, fieldNum) -} - -// GetExtension parses and returns the given extension of pb. -// If the extension is not present and has no default value it returns ErrMissingExtension. -func GetExtension(pb Message, extension *ExtensionDesc) (interface{}, error) { - if epb, doki := pb.(extensionsBytes); doki { - ext := epb.GetExtensions() - o := 0 - for o < len(*ext) { - tag, n := DecodeVarint((*ext)[o:]) - fieldNum := int32(tag >> 3) - wireType := int(tag & 0x7) - l, err := size((*ext)[o+n:], wireType) - if err != nil { - return nil, err - } - if int32(fieldNum) == extension.Field { - v, err := decodeExtension((*ext)[o:o+n+l], extension) - if err != nil { - return nil, err - } - return v, nil - } - o += n + l - } - return defaultExtensionValue(extension) - } - epb, ok := extendable(pb) - if !ok { - return nil, errors.New("proto: not an extendable proto") - } - if err := checkExtensionTypes(epb, extension); err != nil { - return nil, err - } - - emap, mu := epb.extensionsRead() - if emap == nil { - return defaultExtensionValue(extension) - } - mu.Lock() - defer mu.Unlock() - e, ok := emap[extension.Field] - if !ok { - // defaultExtensionValue returns the default value or - // ErrMissingExtension if there is no default. - return defaultExtensionValue(extension) - } - - if e.value != nil { - // Already decoded. Check the descriptor, though. - if e.desc != extension { - // This shouldn't happen. If it does, it means that - // GetExtension was called twice with two different - // descriptors with the same field number. - return nil, errors.New("proto: descriptor conflict") - } - return e.value, nil - } - - v, err := decodeExtension(e.enc, extension) - if err != nil { - return nil, err - } - - // Remember the decoded version and drop the encoded version. - // That way it is safe to mutate what we return. - e.value = v - e.desc = extension - e.enc = nil - emap[extension.Field] = e - return e.value, nil -} - -// defaultExtensionValue returns the default value for extension. -// If no default for an extension is defined ErrMissingExtension is returned. -func defaultExtensionValue(extension *ExtensionDesc) (interface{}, error) { - t := reflect.TypeOf(extension.ExtensionType) - props := extensionProperties(extension) - - sf, _, err := fieldDefault(t, props) - if err != nil { - return nil, err - } - - if sf == nil || sf.value == nil { - // There is no default value. - return nil, ErrMissingExtension - } - - if t.Kind() != reflect.Ptr { - // We do not need to return a Ptr, we can directly return sf.value. - return sf.value, nil - } - - // We need to return an interface{} that is a pointer to sf.value. - value := reflect.New(t).Elem() - value.Set(reflect.New(value.Type().Elem())) - if sf.kind == reflect.Int32 { - // We may have an int32 or an enum, but the underlying data is int32. - // Since we can't set an int32 into a non int32 reflect.value directly - // set it as a int32. - value.Elem().SetInt(int64(sf.value.(int32))) - } else { - value.Elem().Set(reflect.ValueOf(sf.value)) - } - return value.Interface(), nil -} - -// decodeExtension decodes an extension encoded in b. -func decodeExtension(b []byte, extension *ExtensionDesc) (interface{}, error) { - o := NewBuffer(b) - - t := reflect.TypeOf(extension.ExtensionType) - - props := extensionProperties(extension) - - // t is a pointer to a struct, pointer to basic type or a slice. - // Allocate a "field" to store the pointer/slice itself; the - // pointer/slice will be stored here. We pass - // the address of this field to props.dec. - // This passes a zero field and a *t and lets props.dec - // interpret it as a *struct{ x t }. - value := reflect.New(t).Elem() - - for { - // Discard wire type and field number varint. It isn't needed. - if _, err := o.DecodeVarint(); err != nil { - return nil, err - } - - if err := props.dec(o, props, toStructPointer(value.Addr())); err != nil { - return nil, err - } - - if o.index >= len(o.buf) { - break - } - } - return value.Interface(), nil -} - -// GetExtensions returns a slice of the extensions present in pb that are also listed in es. -// The returned slice has the same length as es; missing extensions will appear as nil elements. -func GetExtensions(pb Message, es []*ExtensionDesc) (extensions []interface{}, err error) { - extensions = make([]interface{}, len(es)) - for i, e := range es { - extensions[i], err = GetExtension(pb, e) - if err == ErrMissingExtension { - err = nil - } - if err != nil { - return - } - } - return -} - -// ExtensionDescs returns a new slice containing pb's extension descriptors, in undefined order. -// For non-registered extensions, ExtensionDescs returns an incomplete descriptor containing -// just the Field field, which defines the extension's field number. -func ExtensionDescs(pb Message) ([]*ExtensionDesc, error) { - epb, ok := extendable(pb) - if !ok { - return nil, fmt.Errorf("proto: %T is not an extendable proto.Message", pb) - } - registeredExtensions := RegisteredExtensions(pb) - - emap, mu := epb.extensionsRead() - if emap == nil { - return nil, nil - } - mu.Lock() - defer mu.Unlock() - extensions := make([]*ExtensionDesc, 0, len(emap)) - for extid, e := range emap { - desc := e.desc - if desc == nil { - desc = registeredExtensions[extid] - if desc == nil { - desc = &ExtensionDesc{Field: extid} - } - } - - extensions = append(extensions, desc) - } - return extensions, nil -} - -// SetExtension sets the specified extension of pb to the specified value. -func SetExtension(pb Message, extension *ExtensionDesc, value interface{}) error { - if epb, doki := pb.(extensionsBytes); doki { - ClearExtension(pb, extension) - ext := epb.GetExtensions() - et := reflect.TypeOf(extension.ExtensionType) - props := extensionProperties(extension) - p := NewBuffer(nil) - x := reflect.New(et) - x.Elem().Set(reflect.ValueOf(value)) - if err := props.enc(p, props, toStructPointer(x)); err != nil { - return err - } - *ext = append(*ext, p.buf...) - return nil - } - epb, ok := extendable(pb) - if !ok { - return errors.New("proto: not an extendable proto") - } - if err := checkExtensionTypes(epb, extension); err != nil { - return err - } - typ := reflect.TypeOf(extension.ExtensionType) - if typ != reflect.TypeOf(value) { - return errors.New("proto: bad extension value type") - } - // nil extension values need to be caught early, because the - // encoder can't distinguish an ErrNil due to a nil extension - // from an ErrNil due to a missing field. Extensions are - // always optional, so the encoder would just swallow the error - // and drop all the extensions from the encoded message. - if reflect.ValueOf(value).IsNil() { - return fmt.Errorf("proto: SetExtension called with nil value of type %T", value) - } - - extmap := epb.extensionsWrite() - extmap[extension.Field] = Extension{desc: extension, value: value} - return nil -} - -// ClearAllExtensions clears all extensions from pb. -func ClearAllExtensions(pb Message) { - if epb, doki := pb.(extensionsBytes); doki { - ext := epb.GetExtensions() - *ext = []byte{} - return - } - epb, ok := extendable(pb) - if !ok { - return - } - m := epb.extensionsWrite() - for k := range m { - delete(m, k) - } -} - -// A global registry of extensions. -// The generated code will register the generated descriptors by calling RegisterExtension. - -var extensionMaps = make(map[reflect.Type]map[int32]*ExtensionDesc) - -// RegisterExtension is called from the generated code. -func RegisterExtension(desc *ExtensionDesc) { - st := reflect.TypeOf(desc.ExtendedType).Elem() - m := extensionMaps[st] - if m == nil { - m = make(map[int32]*ExtensionDesc) - extensionMaps[st] = m - } - if _, ok := m[desc.Field]; ok { - panic("proto: duplicate extension registered: " + st.String() + " " + strconv.Itoa(int(desc.Field))) - } - m[desc.Field] = desc -} - -// RegisteredExtensions returns a map of the registered extensions of a -// protocol buffer struct, indexed by the extension number. -// The argument pb should be a nil pointer to the struct type. -func RegisteredExtensions(pb Message) map[int32]*ExtensionDesc { - return extensionMaps[reflect.TypeOf(pb).Elem()] -} diff --git a/vendor/github.com/gogo/protobuf/proto/extensions_gogo.go b/vendor/github.com/gogo/protobuf/proto/extensions_gogo.go deleted file mode 100644 index ea6478f009d7f..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/extensions_gogo.go +++ /dev/null @@ -1,294 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "bytes" - "errors" - "fmt" - "reflect" - "sort" - "strings" - "sync" -) - -func GetBoolExtension(pb Message, extension *ExtensionDesc, ifnotset bool) bool { - if reflect.ValueOf(pb).IsNil() { - return ifnotset - } - value, err := GetExtension(pb, extension) - if err != nil { - return ifnotset - } - if value == nil { - return ifnotset - } - if value.(*bool) == nil { - return ifnotset - } - return *(value.(*bool)) -} - -func (this *Extension) Equal(that *Extension) bool { - return bytes.Equal(this.enc, that.enc) -} - -func (this *Extension) Compare(that *Extension) int { - return bytes.Compare(this.enc, that.enc) -} - -func SizeOfInternalExtension(m extendableProto) (n int) { - return SizeOfExtensionMap(m.extensionsWrite()) -} - -func SizeOfExtensionMap(m map[int32]Extension) (n int) { - return extensionsMapSize(m) -} - -type sortableMapElem struct { - field int32 - ext Extension -} - -func newSortableExtensionsFromMap(m map[int32]Extension) sortableExtensions { - s := make(sortableExtensions, 0, len(m)) - for k, v := range m { - s = append(s, &sortableMapElem{field: k, ext: v}) - } - return s -} - -type sortableExtensions []*sortableMapElem - -func (this sortableExtensions) Len() int { return len(this) } - -func (this sortableExtensions) Swap(i, j int) { this[i], this[j] = this[j], this[i] } - -func (this sortableExtensions) Less(i, j int) bool { return this[i].field < this[j].field } - -func (this sortableExtensions) String() string { - sort.Sort(this) - ss := make([]string, len(this)) - for i := range this { - ss[i] = fmt.Sprintf("%d: %v", this[i].field, this[i].ext) - } - return "map[" + strings.Join(ss, ",") + "]" -} - -func StringFromInternalExtension(m extendableProto) string { - return StringFromExtensionsMap(m.extensionsWrite()) -} - -func StringFromExtensionsMap(m map[int32]Extension) string { - return newSortableExtensionsFromMap(m).String() -} - -func StringFromExtensionsBytes(ext []byte) string { - m, err := BytesToExtensionsMap(ext) - if err != nil { - panic(err) - } - return StringFromExtensionsMap(m) -} - -func EncodeInternalExtension(m extendableProto, data []byte) (n int, err error) { - return EncodeExtensionMap(m.extensionsWrite(), data) -} - -func EncodeExtensionMap(m map[int32]Extension, data []byte) (n int, err error) { - if err := encodeExtensionsMap(m); err != nil { - return 0, err - } - keys := make([]int, 0, len(m)) - for k := range m { - keys = append(keys, int(k)) - } - sort.Ints(keys) - for _, k := range keys { - n += copy(data[n:], m[int32(k)].enc) - } - return n, nil -} - -func GetRawExtension(m map[int32]Extension, id int32) ([]byte, error) { - if m[id].value == nil || m[id].desc == nil { - return m[id].enc, nil - } - if err := encodeExtensionsMap(m); err != nil { - return nil, err - } - return m[id].enc, nil -} - -func size(buf []byte, wire int) (int, error) { - switch wire { - case WireVarint: - _, n := DecodeVarint(buf) - return n, nil - case WireFixed64: - return 8, nil - case WireBytes: - v, n := DecodeVarint(buf) - return int(v) + n, nil - case WireFixed32: - return 4, nil - case WireStartGroup: - offset := 0 - for { - u, n := DecodeVarint(buf[offset:]) - fwire := int(u & 0x7) - offset += n - if fwire == WireEndGroup { - return offset, nil - } - s, err := size(buf[offset:], wire) - if err != nil { - return 0, err - } - offset += s - } - } - return 0, fmt.Errorf("proto: can't get size for unknown wire type %d", wire) -} - -func BytesToExtensionsMap(buf []byte) (map[int32]Extension, error) { - m := make(map[int32]Extension) - i := 0 - for i < len(buf) { - tag, n := DecodeVarint(buf[i:]) - if n <= 0 { - return nil, fmt.Errorf("unable to decode varint") - } - fieldNum := int32(tag >> 3) - wireType := int(tag & 0x7) - l, err := size(buf[i+n:], wireType) - if err != nil { - return nil, err - } - end := i + int(l) + n - m[int32(fieldNum)] = Extension{enc: buf[i:end]} - i = end - } - return m, nil -} - -func NewExtension(e []byte) Extension { - ee := Extension{enc: make([]byte, len(e))} - copy(ee.enc, e) - return ee -} - -func AppendExtension(e Message, tag int32, buf []byte) { - if ee, eok := e.(extensionsBytes); eok { - ext := ee.GetExtensions() - *ext = append(*ext, buf...) - return - } - if ee, eok := e.(extendableProto); eok { - m := ee.extensionsWrite() - ext := m[int32(tag)] // may be missing - ext.enc = append(ext.enc, buf...) - m[int32(tag)] = ext - } -} - -func encodeExtension(e *Extension) error { - if e.value == nil || e.desc == nil { - // Extension is only in its encoded form. - return nil - } - // We don't skip extensions that have an encoded form set, - // because the extension value may have been mutated after - // the last time this function was called. - - et := reflect.TypeOf(e.desc.ExtensionType) - props := extensionProperties(e.desc) - - p := NewBuffer(nil) - // If e.value has type T, the encoder expects a *struct{ X T }. - // Pass a *T with a zero field and hope it all works out. - x := reflect.New(et) - x.Elem().Set(reflect.ValueOf(e.value)) - if err := props.enc(p, props, toStructPointer(x)); err != nil { - return err - } - e.enc = p.buf - return nil -} - -func (this Extension) GoString() string { - if this.enc == nil { - if err := encodeExtension(&this); err != nil { - panic(err) - } - } - return fmt.Sprintf("proto.NewExtension(%#v)", this.enc) -} - -func SetUnsafeExtension(pb Message, fieldNum int32, value interface{}) error { - typ := reflect.TypeOf(pb).Elem() - ext, ok := extensionMaps[typ] - if !ok { - return fmt.Errorf("proto: bad extended type; %s is not extendable", typ.String()) - } - desc, ok := ext[fieldNum] - if !ok { - return errors.New("proto: bad extension number; not in declared ranges") - } - return SetExtension(pb, desc, value) -} - -func GetUnsafeExtension(pb Message, fieldNum int32) (interface{}, error) { - typ := reflect.TypeOf(pb).Elem() - ext, ok := extensionMaps[typ] - if !ok { - return nil, fmt.Errorf("proto: bad extended type; %s is not extendable", typ.String()) - } - desc, ok := ext[fieldNum] - if !ok { - return nil, fmt.Errorf("unregistered field number %d", fieldNum) - } - return GetExtension(pb, desc) -} - -func NewUnsafeXXX_InternalExtensions(m map[int32]Extension) XXX_InternalExtensions { - x := &XXX_InternalExtensions{ - p: new(struct { - mu sync.Mutex - extensionMap map[int32]Extension - }), - } - x.p.extensionMap = m - return *x -} - -func GetUnsafeExtensionsMap(extendable Message) map[int32]Extension { - pb := extendable.(extendableProto) - return pb.extensionsWrite() -} diff --git a/vendor/github.com/gogo/protobuf/proto/lib.go b/vendor/github.com/gogo/protobuf/proto/lib.go deleted file mode 100644 index c98d73da49ea2..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/lib.go +++ /dev/null @@ -1,897 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -/* -Package proto converts data structures to and from the wire format of -protocol buffers. It works in concert with the Go source code generated -for .proto files by the protocol compiler. - -A summary of the properties of the protocol buffer interface -for a protocol buffer variable v: - - - Names are turned from camel_case to CamelCase for export. - - There are no methods on v to set fields; just treat - them as structure fields. - - There are getters that return a field's value if set, - and return the field's default value if unset. - The getters work even if the receiver is a nil message. - - The zero value for a struct is its correct initialization state. - All desired fields must be set before marshaling. - - A Reset() method will restore a protobuf struct to its zero state. - - Non-repeated fields are pointers to the values; nil means unset. - That is, optional or required field int32 f becomes F *int32. - - Repeated fields are slices. - - Helper functions are available to aid the setting of fields. - msg.Foo = proto.String("hello") // set field - - Constants are defined to hold the default values of all fields that - have them. They have the form Default_StructName_FieldName. - Because the getter methods handle defaulted values, - direct use of these constants should be rare. - - Enums are given type names and maps from names to values. - Enum values are prefixed by the enclosing message's name, or by the - enum's type name if it is a top-level enum. Enum types have a String - method, and a Enum method to assist in message construction. - - Nested messages, groups and enums have type names prefixed with the name of - the surrounding message type. - - Extensions are given descriptor names that start with E_, - followed by an underscore-delimited list of the nested messages - that contain it (if any) followed by the CamelCased name of the - extension field itself. HasExtension, ClearExtension, GetExtension - and SetExtension are functions for manipulating extensions. - - Oneof field sets are given a single field in their message, - with distinguished wrapper types for each possible field value. - - Marshal and Unmarshal are functions to encode and decode the wire format. - -When the .proto file specifies `syntax="proto3"`, there are some differences: - - - Non-repeated fields of non-message type are values instead of pointers. - - Enum types do not get an Enum method. - -The simplest way to describe this is to see an example. -Given file test.proto, containing - - package example; - - enum FOO { X = 17; } - - message Test { - required string label = 1; - optional int32 type = 2 [default=77]; - repeated int64 reps = 3; - optional group OptionalGroup = 4 { - required string RequiredField = 5; - } - oneof union { - int32 number = 6; - string name = 7; - } - } - -The resulting file, test.pb.go, is: - - package example - - import proto "github.com/gogo/protobuf/proto" - import math "math" - - type FOO int32 - const ( - FOO_X FOO = 17 - ) - var FOO_name = map[int32]string{ - 17: "X", - } - var FOO_value = map[string]int32{ - "X": 17, - } - - func (x FOO) Enum() *FOO { - p := new(FOO) - *p = x - return p - } - func (x FOO) String() string { - return proto.EnumName(FOO_name, int32(x)) - } - func (x *FOO) UnmarshalJSON(data []byte) error { - value, err := proto.UnmarshalJSONEnum(FOO_value, data) - if err != nil { - return err - } - *x = FOO(value) - return nil - } - - type Test struct { - Label *string `protobuf:"bytes,1,req,name=label" json:"label,omitempty"` - Type *int32 `protobuf:"varint,2,opt,name=type,def=77" json:"type,omitempty"` - Reps []int64 `protobuf:"varint,3,rep,name=reps" json:"reps,omitempty"` - Optionalgroup *Test_OptionalGroup `protobuf:"group,4,opt,name=OptionalGroup" json:"optionalgroup,omitempty"` - // Types that are valid to be assigned to Union: - // *Test_Number - // *Test_Name - Union isTest_Union `protobuf_oneof:"union"` - XXX_unrecognized []byte `json:"-"` - } - func (m *Test) Reset() { *m = Test{} } - func (m *Test) String() string { return proto.CompactTextString(m) } - func (*Test) ProtoMessage() {} - - type isTest_Union interface { - isTest_Union() - } - - type Test_Number struct { - Number int32 `protobuf:"varint,6,opt,name=number"` - } - type Test_Name struct { - Name string `protobuf:"bytes,7,opt,name=name"` - } - - func (*Test_Number) isTest_Union() {} - func (*Test_Name) isTest_Union() {} - - func (m *Test) GetUnion() isTest_Union { - if m != nil { - return m.Union - } - return nil - } - const Default_Test_Type int32 = 77 - - func (m *Test) GetLabel() string { - if m != nil && m.Label != nil { - return *m.Label - } - return "" - } - - func (m *Test) GetType() int32 { - if m != nil && m.Type != nil { - return *m.Type - } - return Default_Test_Type - } - - func (m *Test) GetOptionalgroup() *Test_OptionalGroup { - if m != nil { - return m.Optionalgroup - } - return nil - } - - type Test_OptionalGroup struct { - RequiredField *string `protobuf:"bytes,5,req" json:"RequiredField,omitempty"` - } - func (m *Test_OptionalGroup) Reset() { *m = Test_OptionalGroup{} } - func (m *Test_OptionalGroup) String() string { return proto.CompactTextString(m) } - - func (m *Test_OptionalGroup) GetRequiredField() string { - if m != nil && m.RequiredField != nil { - return *m.RequiredField - } - return "" - } - - func (m *Test) GetNumber() int32 { - if x, ok := m.GetUnion().(*Test_Number); ok { - return x.Number - } - return 0 - } - - func (m *Test) GetName() string { - if x, ok := m.GetUnion().(*Test_Name); ok { - return x.Name - } - return "" - } - - func init() { - proto.RegisterEnum("example.FOO", FOO_name, FOO_value) - } - -To create and play with a Test object: - - package main - - import ( - "log" - - "github.com/gogo/protobuf/proto" - pb "./example.pb" - ) - - func main() { - test := &pb.Test{ - Label: proto.String("hello"), - Type: proto.Int32(17), - Reps: []int64{1, 2, 3}, - Optionalgroup: &pb.Test_OptionalGroup{ - RequiredField: proto.String("good bye"), - }, - Union: &pb.Test_Name{"fred"}, - } - data, err := proto.Marshal(test) - if err != nil { - log.Fatal("marshaling error: ", err) - } - newTest := &pb.Test{} - err = proto.Unmarshal(data, newTest) - if err != nil { - log.Fatal("unmarshaling error: ", err) - } - // Now test and newTest contain the same data. - if test.GetLabel() != newTest.GetLabel() { - log.Fatalf("data mismatch %q != %q", test.GetLabel(), newTest.GetLabel()) - } - // Use a type switch to determine which oneof was set. - switch u := test.Union.(type) { - case *pb.Test_Number: // u.Number contains the number. - case *pb.Test_Name: // u.Name contains the string. - } - // etc. - } -*/ -package proto - -import ( - "encoding/json" - "fmt" - "log" - "reflect" - "sort" - "strconv" - "sync" -) - -// Message is implemented by generated protocol buffer messages. -type Message interface { - Reset() - String() string - ProtoMessage() -} - -// Stats records allocation details about the protocol buffer encoders -// and decoders. Useful for tuning the library itself. -type Stats struct { - Emalloc uint64 // mallocs in encode - Dmalloc uint64 // mallocs in decode - Encode uint64 // number of encodes - Decode uint64 // number of decodes - Chit uint64 // number of cache hits - Cmiss uint64 // number of cache misses - Size uint64 // number of sizes -} - -// Set to true to enable stats collection. -const collectStats = false - -var stats Stats - -// GetStats returns a copy of the global Stats structure. -func GetStats() Stats { return stats } - -// A Buffer is a buffer manager for marshaling and unmarshaling -// protocol buffers. It may be reused between invocations to -// reduce memory usage. It is not necessary to use a Buffer; -// the global functions Marshal and Unmarshal create a -// temporary Buffer and are fine for most applications. -type Buffer struct { - buf []byte // encode/decode byte stream - index int // read point - - // pools of basic types to amortize allocation. - bools []bool - uint32s []uint32 - uint64s []uint64 - - // extra pools, only used with pointer_reflect.go - int32s []int32 - int64s []int64 - float32s []float32 - float64s []float64 -} - -// NewBuffer allocates a new Buffer and initializes its internal data to -// the contents of the argument slice. -func NewBuffer(e []byte) *Buffer { - return &Buffer{buf: e} -} - -// Reset resets the Buffer, ready for marshaling a new protocol buffer. -func (p *Buffer) Reset() { - p.buf = p.buf[0:0] // for reading/writing - p.index = 0 // for reading -} - -// SetBuf replaces the internal buffer with the slice, -// ready for unmarshaling the contents of the slice. -func (p *Buffer) SetBuf(s []byte) { - p.buf = s - p.index = 0 -} - -// Bytes returns the contents of the Buffer. -func (p *Buffer) Bytes() []byte { return p.buf } - -/* - * Helper routines for simplifying the creation of optional fields of basic type. - */ - -// Bool is a helper routine that allocates a new bool value -// to store v and returns a pointer to it. -func Bool(v bool) *bool { - return &v -} - -// Int32 is a helper routine that allocates a new int32 value -// to store v and returns a pointer to it. -func Int32(v int32) *int32 { - return &v -} - -// Int is a helper routine that allocates a new int32 value -// to store v and returns a pointer to it, but unlike Int32 -// its argument value is an int. -func Int(v int) *int32 { - p := new(int32) - *p = int32(v) - return p -} - -// Int64 is a helper routine that allocates a new int64 value -// to store v and returns a pointer to it. -func Int64(v int64) *int64 { - return &v -} - -// Float32 is a helper routine that allocates a new float32 value -// to store v and returns a pointer to it. -func Float32(v float32) *float32 { - return &v -} - -// Float64 is a helper routine that allocates a new float64 value -// to store v and returns a pointer to it. -func Float64(v float64) *float64 { - return &v -} - -// Uint32 is a helper routine that allocates a new uint32 value -// to store v and returns a pointer to it. -func Uint32(v uint32) *uint32 { - return &v -} - -// Uint64 is a helper routine that allocates a new uint64 value -// to store v and returns a pointer to it. -func Uint64(v uint64) *uint64 { - return &v -} - -// String is a helper routine that allocates a new string value -// to store v and returns a pointer to it. -func String(v string) *string { - return &v -} - -// EnumName is a helper function to simplify printing protocol buffer enums -// by name. Given an enum map and a value, it returns a useful string. -func EnumName(m map[int32]string, v int32) string { - s, ok := m[v] - if ok { - return s - } - return strconv.Itoa(int(v)) -} - -// UnmarshalJSONEnum is a helper function to simplify recovering enum int values -// from their JSON-encoded representation. Given a map from the enum's symbolic -// names to its int values, and a byte buffer containing the JSON-encoded -// value, it returns an int32 that can be cast to the enum type by the caller. -// -// The function can deal with both JSON representations, numeric and symbolic. -func UnmarshalJSONEnum(m map[string]int32, data []byte, enumName string) (int32, error) { - if data[0] == '"' { - // New style: enums are strings. - var repr string - if err := json.Unmarshal(data, &repr); err != nil { - return -1, err - } - val, ok := m[repr] - if !ok { - return 0, fmt.Errorf("unrecognized enum %s value %q", enumName, repr) - } - return val, nil - } - // Old style: enums are ints. - var val int32 - if err := json.Unmarshal(data, &val); err != nil { - return 0, fmt.Errorf("cannot unmarshal %#q into enum %s", data, enumName) - } - return val, nil -} - -// DebugPrint dumps the encoded data in b in a debugging format with a header -// including the string s. Used in testing but made available for general debugging. -func (p *Buffer) DebugPrint(s string, b []byte) { - var u uint64 - - obuf := p.buf - sindex := p.index - p.buf = b - p.index = 0 - depth := 0 - - fmt.Printf("\n--- %s ---\n", s) - -out: - for { - for i := 0; i < depth; i++ { - fmt.Print(" ") - } - - index := p.index - if index == len(p.buf) { - break - } - - op, err := p.DecodeVarint() - if err != nil { - fmt.Printf("%3d: fetching op err %v\n", index, err) - break out - } - tag := op >> 3 - wire := op & 7 - - switch wire { - default: - fmt.Printf("%3d: t=%3d unknown wire=%d\n", - index, tag, wire) - break out - - case WireBytes: - var r []byte - - r, err = p.DecodeRawBytes(false) - if err != nil { - break out - } - fmt.Printf("%3d: t=%3d bytes [%d]", index, tag, len(r)) - if len(r) <= 6 { - for i := 0; i < len(r); i++ { - fmt.Printf(" %.2x", r[i]) - } - } else { - for i := 0; i < 3; i++ { - fmt.Printf(" %.2x", r[i]) - } - fmt.Printf(" ..") - for i := len(r) - 3; i < len(r); i++ { - fmt.Printf(" %.2x", r[i]) - } - } - fmt.Printf("\n") - - case WireFixed32: - u, err = p.DecodeFixed32() - if err != nil { - fmt.Printf("%3d: t=%3d fix32 err %v\n", index, tag, err) - break out - } - fmt.Printf("%3d: t=%3d fix32 %d\n", index, tag, u) - - case WireFixed64: - u, err = p.DecodeFixed64() - if err != nil { - fmt.Printf("%3d: t=%3d fix64 err %v\n", index, tag, err) - break out - } - fmt.Printf("%3d: t=%3d fix64 %d\n", index, tag, u) - - case WireVarint: - u, err = p.DecodeVarint() - if err != nil { - fmt.Printf("%3d: t=%3d varint err %v\n", index, tag, err) - break out - } - fmt.Printf("%3d: t=%3d varint %d\n", index, tag, u) - - case WireStartGroup: - fmt.Printf("%3d: t=%3d start\n", index, tag) - depth++ - - case WireEndGroup: - depth-- - fmt.Printf("%3d: t=%3d end\n", index, tag) - } - } - - if depth != 0 { - fmt.Printf("%3d: start-end not balanced %d\n", p.index, depth) - } - fmt.Printf("\n") - - p.buf = obuf - p.index = sindex -} - -// SetDefaults sets unset protocol buffer fields to their default values. -// It only modifies fields that are both unset and have defined defaults. -// It recursively sets default values in any non-nil sub-messages. -func SetDefaults(pb Message) { - setDefaults(reflect.ValueOf(pb), true, false) -} - -// v is a pointer to a struct. -func setDefaults(v reflect.Value, recur, zeros bool) { - v = v.Elem() - - defaultMu.RLock() - dm, ok := defaults[v.Type()] - defaultMu.RUnlock() - if !ok { - dm = buildDefaultMessage(v.Type()) - defaultMu.Lock() - defaults[v.Type()] = dm - defaultMu.Unlock() - } - - for _, sf := range dm.scalars { - f := v.Field(sf.index) - if !f.IsNil() { - // field already set - continue - } - dv := sf.value - if dv == nil && !zeros { - // no explicit default, and don't want to set zeros - continue - } - fptr := f.Addr().Interface() // **T - // TODO: Consider batching the allocations we do here. - switch sf.kind { - case reflect.Bool: - b := new(bool) - if dv != nil { - *b = dv.(bool) - } - *(fptr.(**bool)) = b - case reflect.Float32: - f := new(float32) - if dv != nil { - *f = dv.(float32) - } - *(fptr.(**float32)) = f - case reflect.Float64: - f := new(float64) - if dv != nil { - *f = dv.(float64) - } - *(fptr.(**float64)) = f - case reflect.Int32: - // might be an enum - if ft := f.Type(); ft != int32PtrType { - // enum - f.Set(reflect.New(ft.Elem())) - if dv != nil { - f.Elem().SetInt(int64(dv.(int32))) - } - } else { - // int32 field - i := new(int32) - if dv != nil { - *i = dv.(int32) - } - *(fptr.(**int32)) = i - } - case reflect.Int64: - i := new(int64) - if dv != nil { - *i = dv.(int64) - } - *(fptr.(**int64)) = i - case reflect.String: - s := new(string) - if dv != nil { - *s = dv.(string) - } - *(fptr.(**string)) = s - case reflect.Uint8: - // exceptional case: []byte - var b []byte - if dv != nil { - db := dv.([]byte) - b = make([]byte, len(db)) - copy(b, db) - } else { - b = []byte{} - } - *(fptr.(*[]byte)) = b - case reflect.Uint32: - u := new(uint32) - if dv != nil { - *u = dv.(uint32) - } - *(fptr.(**uint32)) = u - case reflect.Uint64: - u := new(uint64) - if dv != nil { - *u = dv.(uint64) - } - *(fptr.(**uint64)) = u - default: - log.Printf("proto: can't set default for field %v (sf.kind=%v)", f, sf.kind) - } - } - - for _, ni := range dm.nested { - f := v.Field(ni) - // f is *T or []*T or map[T]*T - switch f.Kind() { - case reflect.Ptr: - if f.IsNil() { - continue - } - setDefaults(f, recur, zeros) - - case reflect.Slice: - for i := 0; i < f.Len(); i++ { - e := f.Index(i) - if e.IsNil() { - continue - } - setDefaults(e, recur, zeros) - } - - case reflect.Map: - for _, k := range f.MapKeys() { - e := f.MapIndex(k) - if e.IsNil() { - continue - } - setDefaults(e, recur, zeros) - } - } - } -} - -var ( - // defaults maps a protocol buffer struct type to a slice of the fields, - // with its scalar fields set to their proto-declared non-zero default values. - defaultMu sync.RWMutex - defaults = make(map[reflect.Type]defaultMessage) - - int32PtrType = reflect.TypeOf((*int32)(nil)) -) - -// defaultMessage represents information about the default values of a message. -type defaultMessage struct { - scalars []scalarField - nested []int // struct field index of nested messages -} - -type scalarField struct { - index int // struct field index - kind reflect.Kind // element type (the T in *T or []T) - value interface{} // the proto-declared default value, or nil -} - -// t is a struct type. -func buildDefaultMessage(t reflect.Type) (dm defaultMessage) { - sprop := GetProperties(t) - for _, prop := range sprop.Prop { - fi, ok := sprop.decoderTags.get(prop.Tag) - if !ok { - // XXX_unrecognized - continue - } - ft := t.Field(fi).Type - - sf, nested, err := fieldDefault(ft, prop) - switch { - case err != nil: - log.Print(err) - case nested: - dm.nested = append(dm.nested, fi) - case sf != nil: - sf.index = fi - dm.scalars = append(dm.scalars, *sf) - } - } - - return dm -} - -// fieldDefault returns the scalarField for field type ft. -// sf will be nil if the field can not have a default. -// nestedMessage will be true if this is a nested message. -// Note that sf.index is not set on return. -func fieldDefault(ft reflect.Type, prop *Properties) (sf *scalarField, nestedMessage bool, err error) { - var canHaveDefault bool - switch ft.Kind() { - case reflect.Ptr: - if ft.Elem().Kind() == reflect.Struct { - nestedMessage = true - } else { - canHaveDefault = true // proto2 scalar field - } - - case reflect.Slice: - switch ft.Elem().Kind() { - case reflect.Ptr: - nestedMessage = true // repeated message - case reflect.Uint8: - canHaveDefault = true // bytes field - } - - case reflect.Map: - if ft.Elem().Kind() == reflect.Ptr { - nestedMessage = true // map with message values - } - } - - if !canHaveDefault { - if nestedMessage { - return nil, true, nil - } - return nil, false, nil - } - - // We now know that ft is a pointer or slice. - sf = &scalarField{kind: ft.Elem().Kind()} - - // scalar fields without defaults - if !prop.HasDefault { - return sf, false, nil - } - - // a scalar field: either *T or []byte - switch ft.Elem().Kind() { - case reflect.Bool: - x, err := strconv.ParseBool(prop.Default) - if err != nil { - return nil, false, fmt.Errorf("proto: bad default bool %q: %v", prop.Default, err) - } - sf.value = x - case reflect.Float32: - x, err := strconv.ParseFloat(prop.Default, 32) - if err != nil { - return nil, false, fmt.Errorf("proto: bad default float32 %q: %v", prop.Default, err) - } - sf.value = float32(x) - case reflect.Float64: - x, err := strconv.ParseFloat(prop.Default, 64) - if err != nil { - return nil, false, fmt.Errorf("proto: bad default float64 %q: %v", prop.Default, err) - } - sf.value = x - case reflect.Int32: - x, err := strconv.ParseInt(prop.Default, 10, 32) - if err != nil { - return nil, false, fmt.Errorf("proto: bad default int32 %q: %v", prop.Default, err) - } - sf.value = int32(x) - case reflect.Int64: - x, err := strconv.ParseInt(prop.Default, 10, 64) - if err != nil { - return nil, false, fmt.Errorf("proto: bad default int64 %q: %v", prop.Default, err) - } - sf.value = x - case reflect.String: - sf.value = prop.Default - case reflect.Uint8: - // []byte (not *uint8) - sf.value = []byte(prop.Default) - case reflect.Uint32: - x, err := strconv.ParseUint(prop.Default, 10, 32) - if err != nil { - return nil, false, fmt.Errorf("proto: bad default uint32 %q: %v", prop.Default, err) - } - sf.value = uint32(x) - case reflect.Uint64: - x, err := strconv.ParseUint(prop.Default, 10, 64) - if err != nil { - return nil, false, fmt.Errorf("proto: bad default uint64 %q: %v", prop.Default, err) - } - sf.value = x - default: - return nil, false, fmt.Errorf("proto: unhandled def kind %v", ft.Elem().Kind()) - } - - return sf, false, nil -} - -// Map fields may have key types of non-float scalars, strings and enums. -// The easiest way to sort them in some deterministic order is to use fmt. -// If this turns out to be inefficient we can always consider other options, -// such as doing a Schwartzian transform. - -func mapKeys(vs []reflect.Value) sort.Interface { - s := mapKeySorter{ - vs: vs, - // default Less function: textual comparison - less: func(a, b reflect.Value) bool { - return fmt.Sprint(a.Interface()) < fmt.Sprint(b.Interface()) - }, - } - - // Type specialization per https://developers.google.com/protocol-buffers/docs/proto#maps; - // numeric keys are sorted numerically. - if len(vs) == 0 { - return s - } - switch vs[0].Kind() { - case reflect.Int32, reflect.Int64: - s.less = func(a, b reflect.Value) bool { return a.Int() < b.Int() } - case reflect.Uint32, reflect.Uint64: - s.less = func(a, b reflect.Value) bool { return a.Uint() < b.Uint() } - } - - return s -} - -type mapKeySorter struct { - vs []reflect.Value - less func(a, b reflect.Value) bool -} - -func (s mapKeySorter) Len() int { return len(s.vs) } -func (s mapKeySorter) Swap(i, j int) { s.vs[i], s.vs[j] = s.vs[j], s.vs[i] } -func (s mapKeySorter) Less(i, j int) bool { - return s.less(s.vs[i], s.vs[j]) -} - -// isProto3Zero reports whether v is a zero proto3 value. -func isProto3Zero(v reflect.Value) bool { - switch v.Kind() { - case reflect.Bool: - return !v.Bool() - case reflect.Int32, reflect.Int64: - return v.Int() == 0 - case reflect.Uint32, reflect.Uint64: - return v.Uint() == 0 - case reflect.Float32, reflect.Float64: - return v.Float() == 0 - case reflect.String: - return v.String() == "" - } - return false -} - -// ProtoPackageIsVersion2 is referenced from generated protocol buffer files -// to assert that that code is compatible with this version of the proto package. -const GoGoProtoPackageIsVersion2 = true - -// ProtoPackageIsVersion1 is referenced from generated protocol buffer files -// to assert that that code is compatible with this version of the proto package. -const GoGoProtoPackageIsVersion1 = true diff --git a/vendor/github.com/gogo/protobuf/proto/lib_gogo.go b/vendor/github.com/gogo/protobuf/proto/lib_gogo.go deleted file mode 100644 index 4b4f7c909e6ba..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/lib_gogo.go +++ /dev/null @@ -1,42 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "encoding/json" - "strconv" -) - -func MarshalJSONEnum(m map[int32]string, value int32) ([]byte, error) { - s, ok := m[value] - if !ok { - s = strconv.Itoa(int(value)) - } - return json.Marshal(s) -} diff --git a/vendor/github.com/gogo/protobuf/proto/message_set.go b/vendor/github.com/gogo/protobuf/proto/message_set.go deleted file mode 100644 index fd982decd66e4..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/message_set.go +++ /dev/null @@ -1,311 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -/* - * Support for message sets. - */ - -import ( - "bytes" - "encoding/json" - "errors" - "fmt" - "reflect" - "sort" -) - -// errNoMessageTypeID occurs when a protocol buffer does not have a message type ID. -// A message type ID is required for storing a protocol buffer in a message set. -var errNoMessageTypeID = errors.New("proto does not have a message type ID") - -// The first two types (_MessageSet_Item and messageSet) -// model what the protocol compiler produces for the following protocol message: -// message MessageSet { -// repeated group Item = 1 { -// required int32 type_id = 2; -// required string message = 3; -// }; -// } -// That is the MessageSet wire format. We can't use a proto to generate these -// because that would introduce a circular dependency between it and this package. - -type _MessageSet_Item struct { - TypeId *int32 `protobuf:"varint,2,req,name=type_id"` - Message []byte `protobuf:"bytes,3,req,name=message"` -} - -type messageSet struct { - Item []*_MessageSet_Item `protobuf:"group,1,rep"` - XXX_unrecognized []byte - // TODO: caching? -} - -// Make sure messageSet is a Message. -var _ Message = (*messageSet)(nil) - -// messageTypeIder is an interface satisfied by a protocol buffer type -// that may be stored in a MessageSet. -type messageTypeIder interface { - MessageTypeId() int32 -} - -func (ms *messageSet) find(pb Message) *_MessageSet_Item { - mti, ok := pb.(messageTypeIder) - if !ok { - return nil - } - id := mti.MessageTypeId() - for _, item := range ms.Item { - if *item.TypeId == id { - return item - } - } - return nil -} - -func (ms *messageSet) Has(pb Message) bool { - if ms.find(pb) != nil { - return true - } - return false -} - -func (ms *messageSet) Unmarshal(pb Message) error { - if item := ms.find(pb); item != nil { - return Unmarshal(item.Message, pb) - } - if _, ok := pb.(messageTypeIder); !ok { - return errNoMessageTypeID - } - return nil // TODO: return error instead? -} - -func (ms *messageSet) Marshal(pb Message) error { - msg, err := Marshal(pb) - if err != nil { - return err - } - if item := ms.find(pb); item != nil { - // reuse existing item - item.Message = msg - return nil - } - - mti, ok := pb.(messageTypeIder) - if !ok { - return errNoMessageTypeID - } - - mtid := mti.MessageTypeId() - ms.Item = append(ms.Item, &_MessageSet_Item{ - TypeId: &mtid, - Message: msg, - }) - return nil -} - -func (ms *messageSet) Reset() { *ms = messageSet{} } -func (ms *messageSet) String() string { return CompactTextString(ms) } -func (*messageSet) ProtoMessage() {} - -// Support for the message_set_wire_format message option. - -func skipVarint(buf []byte) []byte { - i := 0 - for ; buf[i]&0x80 != 0; i++ { - } - return buf[i+1:] -} - -// MarshalMessageSet encodes the extension map represented by m in the message set wire format. -// It is called by generated Marshal methods on protocol buffer messages with the message_set_wire_format option. -func MarshalMessageSet(exts interface{}) ([]byte, error) { - var m map[int32]Extension - switch exts := exts.(type) { - case *XXX_InternalExtensions: - if err := encodeExtensions(exts); err != nil { - return nil, err - } - m, _ = exts.extensionsRead() - case map[int32]Extension: - if err := encodeExtensionsMap(exts); err != nil { - return nil, err - } - m = exts - default: - return nil, errors.New("proto: not an extension map") - } - - // Sort extension IDs to provide a deterministic encoding. - // See also enc_map in encode.go. - ids := make([]int, 0, len(m)) - for id := range m { - ids = append(ids, int(id)) - } - sort.Ints(ids) - - ms := &messageSet{Item: make([]*_MessageSet_Item, 0, len(m))} - for _, id := range ids { - e := m[int32(id)] - // Remove the wire type and field number varint, as well as the length varint. - msg := skipVarint(skipVarint(e.enc)) - - ms.Item = append(ms.Item, &_MessageSet_Item{ - TypeId: Int32(int32(id)), - Message: msg, - }) - } - return Marshal(ms) -} - -// UnmarshalMessageSet decodes the extension map encoded in buf in the message set wire format. -// It is called by generated Unmarshal methods on protocol buffer messages with the message_set_wire_format option. -func UnmarshalMessageSet(buf []byte, exts interface{}) error { - var m map[int32]Extension - switch exts := exts.(type) { - case *XXX_InternalExtensions: - m = exts.extensionsWrite() - case map[int32]Extension: - m = exts - default: - return errors.New("proto: not an extension map") - } - - ms := new(messageSet) - if err := Unmarshal(buf, ms); err != nil { - return err - } - for _, item := range ms.Item { - id := *item.TypeId - msg := item.Message - - // Restore wire type and field number varint, plus length varint. - // Be careful to preserve duplicate items. - b := EncodeVarint(uint64(id)<<3 | WireBytes) - if ext, ok := m[id]; ok { - // Existing data; rip off the tag and length varint - // so we join the new data correctly. - // We can assume that ext.enc is set because we are unmarshaling. - o := ext.enc[len(b):] // skip wire type and field number - _, n := DecodeVarint(o) // calculate length of length varint - o = o[n:] // skip length varint - msg = append(o, msg...) // join old data and new data - } - b = append(b, EncodeVarint(uint64(len(msg)))...) - b = append(b, msg...) - - m[id] = Extension{enc: b} - } - return nil -} - -// MarshalMessageSetJSON encodes the extension map represented by m in JSON format. -// It is called by generated MarshalJSON methods on protocol buffer messages with the message_set_wire_format option. -func MarshalMessageSetJSON(exts interface{}) ([]byte, error) { - var m map[int32]Extension - switch exts := exts.(type) { - case *XXX_InternalExtensions: - m, _ = exts.extensionsRead() - case map[int32]Extension: - m = exts - default: - return nil, errors.New("proto: not an extension map") - } - var b bytes.Buffer - b.WriteByte('{') - - // Process the map in key order for deterministic output. - ids := make([]int32, 0, len(m)) - for id := range m { - ids = append(ids, id) - } - sort.Sort(int32Slice(ids)) // int32Slice defined in text.go - - for i, id := range ids { - ext := m[id] - if i > 0 { - b.WriteByte(',') - } - - msd, ok := messageSetMap[id] - if !ok { - // Unknown type; we can't render it, so skip it. - continue - } - fmt.Fprintf(&b, `"[%s]":`, msd.name) - - x := ext.value - if x == nil { - x = reflect.New(msd.t.Elem()).Interface() - if err := Unmarshal(ext.enc, x.(Message)); err != nil { - return nil, err - } - } - d, err := json.Marshal(x) - if err != nil { - return nil, err - } - b.Write(d) - } - b.WriteByte('}') - return b.Bytes(), nil -} - -// UnmarshalMessageSetJSON decodes the extension map encoded in buf in JSON format. -// It is called by generated UnmarshalJSON methods on protocol buffer messages with the message_set_wire_format option. -func UnmarshalMessageSetJSON(buf []byte, exts interface{}) error { - // Common-case fast path. - if len(buf) == 0 || bytes.Equal(buf, []byte("{}")) { - return nil - } - - // This is fairly tricky, and it's not clear that it is needed. - return errors.New("TODO: UnmarshalMessageSetJSON not yet implemented") -} - -// A global registry of types that can be used in a MessageSet. - -var messageSetMap = make(map[int32]messageSetDesc) - -type messageSetDesc struct { - t reflect.Type // pointer to struct - name string -} - -// RegisterMessageSetType is called from the generated code. -func RegisterMessageSetType(m Message, fieldNum int32, name string) { - messageSetMap[fieldNum] = messageSetDesc{ - t: reflect.TypeOf(m), - name: name, - } -} diff --git a/vendor/github.com/gogo/protobuf/proto/pointer_reflect.go b/vendor/github.com/gogo/protobuf/proto/pointer_reflect.go deleted file mode 100644 index fb512e2e16dce..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/pointer_reflect.go +++ /dev/null @@ -1,484 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2012 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -// +build appengine js - -// This file contains an implementation of proto field accesses using package reflect. -// It is slower than the code in pointer_unsafe.go but it avoids package unsafe and can -// be used on App Engine. - -package proto - -import ( - "math" - "reflect" -) - -// A structPointer is a pointer to a struct. -type structPointer struct { - v reflect.Value -} - -// toStructPointer returns a structPointer equivalent to the given reflect value. -// The reflect value must itself be a pointer to a struct. -func toStructPointer(v reflect.Value) structPointer { - return structPointer{v} -} - -// IsNil reports whether p is nil. -func structPointer_IsNil(p structPointer) bool { - return p.v.IsNil() -} - -// Interface returns the struct pointer as an interface value. -func structPointer_Interface(p structPointer, _ reflect.Type) interface{} { - return p.v.Interface() -} - -// A field identifies a field in a struct, accessible from a structPointer. -// In this implementation, a field is identified by the sequence of field indices -// passed to reflect's FieldByIndex. -type field []int - -// toField returns a field equivalent to the given reflect field. -func toField(f *reflect.StructField) field { - return f.Index -} - -// invalidField is an invalid field identifier. -var invalidField = field(nil) - -// IsValid reports whether the field identifier is valid. -func (f field) IsValid() bool { return f != nil } - -// field returns the given field in the struct as a reflect value. -func structPointer_field(p structPointer, f field) reflect.Value { - // Special case: an extension map entry with a value of type T - // passes a *T to the struct-handling code with a zero field, - // expecting that it will be treated as equivalent to *struct{ X T }, - // which has the same memory layout. We have to handle that case - // specially, because reflect will panic if we call FieldByIndex on a - // non-struct. - if f == nil { - return p.v.Elem() - } - - return p.v.Elem().FieldByIndex(f) -} - -// ifield returns the given field in the struct as an interface value. -func structPointer_ifield(p structPointer, f field) interface{} { - return structPointer_field(p, f).Addr().Interface() -} - -// Bytes returns the address of a []byte field in the struct. -func structPointer_Bytes(p structPointer, f field) *[]byte { - return structPointer_ifield(p, f).(*[]byte) -} - -// BytesSlice returns the address of a [][]byte field in the struct. -func structPointer_BytesSlice(p structPointer, f field) *[][]byte { - return structPointer_ifield(p, f).(*[][]byte) -} - -// Bool returns the address of a *bool field in the struct. -func structPointer_Bool(p structPointer, f field) **bool { - return structPointer_ifield(p, f).(**bool) -} - -// BoolVal returns the address of a bool field in the struct. -func structPointer_BoolVal(p structPointer, f field) *bool { - return structPointer_ifield(p, f).(*bool) -} - -// BoolSlice returns the address of a []bool field in the struct. -func structPointer_BoolSlice(p structPointer, f field) *[]bool { - return structPointer_ifield(p, f).(*[]bool) -} - -// String returns the address of a *string field in the struct. -func structPointer_String(p structPointer, f field) **string { - return structPointer_ifield(p, f).(**string) -} - -// StringVal returns the address of a string field in the struct. -func structPointer_StringVal(p structPointer, f field) *string { - return structPointer_ifield(p, f).(*string) -} - -// StringSlice returns the address of a []string field in the struct. -func structPointer_StringSlice(p structPointer, f field) *[]string { - return structPointer_ifield(p, f).(*[]string) -} - -// Extensions returns the address of an extension map field in the struct. -func structPointer_Extensions(p structPointer, f field) *XXX_InternalExtensions { - return structPointer_ifield(p, f).(*XXX_InternalExtensions) -} - -// ExtMap returns the address of an extension map field in the struct. -func structPointer_ExtMap(p structPointer, f field) *map[int32]Extension { - return structPointer_ifield(p, f).(*map[int32]Extension) -} - -// NewAt returns the reflect.Value for a pointer to a field in the struct. -func structPointer_NewAt(p structPointer, f field, typ reflect.Type) reflect.Value { - return structPointer_field(p, f).Addr() -} - -// SetStructPointer writes a *struct field in the struct. -func structPointer_SetStructPointer(p structPointer, f field, q structPointer) { - structPointer_field(p, f).Set(q.v) -} - -// GetStructPointer reads a *struct field in the struct. -func structPointer_GetStructPointer(p structPointer, f field) structPointer { - return structPointer{structPointer_field(p, f)} -} - -// StructPointerSlice the address of a []*struct field in the struct. -func structPointer_StructPointerSlice(p structPointer, f field) structPointerSlice { - return structPointerSlice{structPointer_field(p, f)} -} - -// A structPointerSlice represents the address of a slice of pointers to structs -// (themselves messages or groups). That is, v.Type() is *[]*struct{...}. -type structPointerSlice struct { - v reflect.Value -} - -func (p structPointerSlice) Len() int { return p.v.Len() } -func (p structPointerSlice) Index(i int) structPointer { return structPointer{p.v.Index(i)} } -func (p structPointerSlice) Append(q structPointer) { - p.v.Set(reflect.Append(p.v, q.v)) -} - -var ( - int32Type = reflect.TypeOf(int32(0)) - uint32Type = reflect.TypeOf(uint32(0)) - float32Type = reflect.TypeOf(float32(0)) - int64Type = reflect.TypeOf(int64(0)) - uint64Type = reflect.TypeOf(uint64(0)) - float64Type = reflect.TypeOf(float64(0)) -) - -// A word32 represents a field of type *int32, *uint32, *float32, or *enum. -// That is, v.Type() is *int32, *uint32, *float32, or *enum and v is assignable. -type word32 struct { - v reflect.Value -} - -// IsNil reports whether p is nil. -func word32_IsNil(p word32) bool { - return p.v.IsNil() -} - -// Set sets p to point at a newly allocated word with bits set to x. -func word32_Set(p word32, o *Buffer, x uint32) { - t := p.v.Type().Elem() - switch t { - case int32Type: - if len(o.int32s) == 0 { - o.int32s = make([]int32, uint32PoolSize) - } - o.int32s[0] = int32(x) - p.v.Set(reflect.ValueOf(&o.int32s[0])) - o.int32s = o.int32s[1:] - return - case uint32Type: - if len(o.uint32s) == 0 { - o.uint32s = make([]uint32, uint32PoolSize) - } - o.uint32s[0] = x - p.v.Set(reflect.ValueOf(&o.uint32s[0])) - o.uint32s = o.uint32s[1:] - return - case float32Type: - if len(o.float32s) == 0 { - o.float32s = make([]float32, uint32PoolSize) - } - o.float32s[0] = math.Float32frombits(x) - p.v.Set(reflect.ValueOf(&o.float32s[0])) - o.float32s = o.float32s[1:] - return - } - - // must be enum - p.v.Set(reflect.New(t)) - p.v.Elem().SetInt(int64(int32(x))) -} - -// Get gets the bits pointed at by p, as a uint32. -func word32_Get(p word32) uint32 { - elem := p.v.Elem() - switch elem.Kind() { - case reflect.Int32: - return uint32(elem.Int()) - case reflect.Uint32: - return uint32(elem.Uint()) - case reflect.Float32: - return math.Float32bits(float32(elem.Float())) - } - panic("unreachable") -} - -// Word32 returns a reference to a *int32, *uint32, *float32, or *enum field in the struct. -func structPointer_Word32(p structPointer, f field) word32 { - return word32{structPointer_field(p, f)} -} - -// A word32Val represents a field of type int32, uint32, float32, or enum. -// That is, v.Type() is int32, uint32, float32, or enum and v is assignable. -type word32Val struct { - v reflect.Value -} - -// Set sets *p to x. -func word32Val_Set(p word32Val, x uint32) { - switch p.v.Type() { - case int32Type: - p.v.SetInt(int64(x)) - return - case uint32Type: - p.v.SetUint(uint64(x)) - return - case float32Type: - p.v.SetFloat(float64(math.Float32frombits(x))) - return - } - - // must be enum - p.v.SetInt(int64(int32(x))) -} - -// Get gets the bits pointed at by p, as a uint32. -func word32Val_Get(p word32Val) uint32 { - elem := p.v - switch elem.Kind() { - case reflect.Int32: - return uint32(elem.Int()) - case reflect.Uint32: - return uint32(elem.Uint()) - case reflect.Float32: - return math.Float32bits(float32(elem.Float())) - } - panic("unreachable") -} - -// Word32Val returns a reference to a int32, uint32, float32, or enum field in the struct. -func structPointer_Word32Val(p structPointer, f field) word32Val { - return word32Val{structPointer_field(p, f)} -} - -// A word32Slice is a slice of 32-bit values. -// That is, v.Type() is []int32, []uint32, []float32, or []enum. -type word32Slice struct { - v reflect.Value -} - -func (p word32Slice) Append(x uint32) { - n, m := p.v.Len(), p.v.Cap() - if n < m { - p.v.SetLen(n + 1) - } else { - t := p.v.Type().Elem() - p.v.Set(reflect.Append(p.v, reflect.Zero(t))) - } - elem := p.v.Index(n) - switch elem.Kind() { - case reflect.Int32: - elem.SetInt(int64(int32(x))) - case reflect.Uint32: - elem.SetUint(uint64(x)) - case reflect.Float32: - elem.SetFloat(float64(math.Float32frombits(x))) - } -} - -func (p word32Slice) Len() int { - return p.v.Len() -} - -func (p word32Slice) Index(i int) uint32 { - elem := p.v.Index(i) - switch elem.Kind() { - case reflect.Int32: - return uint32(elem.Int()) - case reflect.Uint32: - return uint32(elem.Uint()) - case reflect.Float32: - return math.Float32bits(float32(elem.Float())) - } - panic("unreachable") -} - -// Word32Slice returns a reference to a []int32, []uint32, []float32, or []enum field in the struct. -func structPointer_Word32Slice(p structPointer, f field) word32Slice { - return word32Slice{structPointer_field(p, f)} -} - -// word64 is like word32 but for 64-bit values. -type word64 struct { - v reflect.Value -} - -func word64_Set(p word64, o *Buffer, x uint64) { - t := p.v.Type().Elem() - switch t { - case int64Type: - if len(o.int64s) == 0 { - o.int64s = make([]int64, uint64PoolSize) - } - o.int64s[0] = int64(x) - p.v.Set(reflect.ValueOf(&o.int64s[0])) - o.int64s = o.int64s[1:] - return - case uint64Type: - if len(o.uint64s) == 0 { - o.uint64s = make([]uint64, uint64PoolSize) - } - o.uint64s[0] = x - p.v.Set(reflect.ValueOf(&o.uint64s[0])) - o.uint64s = o.uint64s[1:] - return - case float64Type: - if len(o.float64s) == 0 { - o.float64s = make([]float64, uint64PoolSize) - } - o.float64s[0] = math.Float64frombits(x) - p.v.Set(reflect.ValueOf(&o.float64s[0])) - o.float64s = o.float64s[1:] - return - } - panic("unreachable") -} - -func word64_IsNil(p word64) bool { - return p.v.IsNil() -} - -func word64_Get(p word64) uint64 { - elem := p.v.Elem() - switch elem.Kind() { - case reflect.Int64: - return uint64(elem.Int()) - case reflect.Uint64: - return elem.Uint() - case reflect.Float64: - return math.Float64bits(elem.Float()) - } - panic("unreachable") -} - -func structPointer_Word64(p structPointer, f field) word64 { - return word64{structPointer_field(p, f)} -} - -// word64Val is like word32Val but for 64-bit values. -type word64Val struct { - v reflect.Value -} - -func word64Val_Set(p word64Val, o *Buffer, x uint64) { - switch p.v.Type() { - case int64Type: - p.v.SetInt(int64(x)) - return - case uint64Type: - p.v.SetUint(x) - return - case float64Type: - p.v.SetFloat(math.Float64frombits(x)) - return - } - panic("unreachable") -} - -func word64Val_Get(p word64Val) uint64 { - elem := p.v - switch elem.Kind() { - case reflect.Int64: - return uint64(elem.Int()) - case reflect.Uint64: - return elem.Uint() - case reflect.Float64: - return math.Float64bits(elem.Float()) - } - panic("unreachable") -} - -func structPointer_Word64Val(p structPointer, f field) word64Val { - return word64Val{structPointer_field(p, f)} -} - -type word64Slice struct { - v reflect.Value -} - -func (p word64Slice) Append(x uint64) { - n, m := p.v.Len(), p.v.Cap() - if n < m { - p.v.SetLen(n + 1) - } else { - t := p.v.Type().Elem() - p.v.Set(reflect.Append(p.v, reflect.Zero(t))) - } - elem := p.v.Index(n) - switch elem.Kind() { - case reflect.Int64: - elem.SetInt(int64(int64(x))) - case reflect.Uint64: - elem.SetUint(uint64(x)) - case reflect.Float64: - elem.SetFloat(float64(math.Float64frombits(x))) - } -} - -func (p word64Slice) Len() int { - return p.v.Len() -} - -func (p word64Slice) Index(i int) uint64 { - elem := p.v.Index(i) - switch elem.Kind() { - case reflect.Int64: - return uint64(elem.Int()) - case reflect.Uint64: - return uint64(elem.Uint()) - case reflect.Float64: - return math.Float64bits(float64(elem.Float())) - } - panic("unreachable") -} - -func structPointer_Word64Slice(p structPointer, f field) word64Slice { - return word64Slice{structPointer_field(p, f)} -} diff --git a/vendor/github.com/gogo/protobuf/proto/pointer_reflect_gogo.go b/vendor/github.com/gogo/protobuf/proto/pointer_reflect_gogo.go deleted file mode 100644 index 1763a5f227a0c..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/pointer_reflect_gogo.go +++ /dev/null @@ -1,85 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2016, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -// +build appengine js - -package proto - -import ( - "reflect" -) - -func structPointer_FieldPointer(p structPointer, f field) structPointer { - panic("not implemented") -} - -func appendStructPointer(base structPointer, f field, typ reflect.Type) structPointer { - panic("not implemented") -} - -func structPointer_InterfaceAt(p structPointer, f field, t reflect.Type) interface{} { - panic("not implemented") -} - -func structPointer_InterfaceRef(p structPointer, f field, t reflect.Type) interface{} { - panic("not implemented") -} - -func structPointer_GetRefStructPointer(p structPointer, f field) structPointer { - panic("not implemented") -} - -func structPointer_Add(p structPointer, size field) structPointer { - panic("not implemented") -} - -func structPointer_Len(p structPointer, f field) int { - panic("not implemented") -} - -func structPointer_GetSliceHeader(p structPointer, f field) *reflect.SliceHeader { - panic("not implemented") -} - -func structPointer_Copy(oldptr structPointer, newptr structPointer, size int) { - panic("not implemented") -} - -func structPointer_StructRefSlice(p structPointer, f field, size uintptr) *structRefSlice { - panic("not implemented") -} - -type structRefSlice struct{} - -func (v *structRefSlice) Len() int { - panic("not implemented") -} - -func (v *structRefSlice) Index(i int) structPointer { - panic("not implemented") -} diff --git a/vendor/github.com/gogo/protobuf/proto/pointer_unsafe.go b/vendor/github.com/gogo/protobuf/proto/pointer_unsafe.go deleted file mode 100644 index 6b5567d47cd39..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/pointer_unsafe.go +++ /dev/null @@ -1,270 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2012 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -// +build !appengine,!js - -// This file contains the implementation of the proto field accesses using package unsafe. - -package proto - -import ( - "reflect" - "unsafe" -) - -// NOTE: These type_Foo functions would more idiomatically be methods, -// but Go does not allow methods on pointer types, and we must preserve -// some pointer type for the garbage collector. We use these -// funcs with clunky names as our poor approximation to methods. -// -// An alternative would be -// type structPointer struct { p unsafe.Pointer } -// but that does not registerize as well. - -// A structPointer is a pointer to a struct. -type structPointer unsafe.Pointer - -// toStructPointer returns a structPointer equivalent to the given reflect value. -func toStructPointer(v reflect.Value) structPointer { - return structPointer(unsafe.Pointer(v.Pointer())) -} - -// IsNil reports whether p is nil. -func structPointer_IsNil(p structPointer) bool { - return p == nil -} - -// Interface returns the struct pointer, assumed to have element type t, -// as an interface value. -func structPointer_Interface(p structPointer, t reflect.Type) interface{} { - return reflect.NewAt(t, unsafe.Pointer(p)).Interface() -} - -// A field identifies a field in a struct, accessible from a structPointer. -// In this implementation, a field is identified by its byte offset from the start of the struct. -type field uintptr - -// toField returns a field equivalent to the given reflect field. -func toField(f *reflect.StructField) field { - return field(f.Offset) -} - -// invalidField is an invalid field identifier. -const invalidField = ^field(0) - -// IsValid reports whether the field identifier is valid. -func (f field) IsValid() bool { - return f != ^field(0) -} - -// Bytes returns the address of a []byte field in the struct. -func structPointer_Bytes(p structPointer, f field) *[]byte { - return (*[]byte)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// BytesSlice returns the address of a [][]byte field in the struct. -func structPointer_BytesSlice(p structPointer, f field) *[][]byte { - return (*[][]byte)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// Bool returns the address of a *bool field in the struct. -func structPointer_Bool(p structPointer, f field) **bool { - return (**bool)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// BoolVal returns the address of a bool field in the struct. -func structPointer_BoolVal(p structPointer, f field) *bool { - return (*bool)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// BoolSlice returns the address of a []bool field in the struct. -func structPointer_BoolSlice(p structPointer, f field) *[]bool { - return (*[]bool)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// String returns the address of a *string field in the struct. -func structPointer_String(p structPointer, f field) **string { - return (**string)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// StringVal returns the address of a string field in the struct. -func structPointer_StringVal(p structPointer, f field) *string { - return (*string)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// StringSlice returns the address of a []string field in the struct. -func structPointer_StringSlice(p structPointer, f field) *[]string { - return (*[]string)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// ExtMap returns the address of an extension map field in the struct. -func structPointer_Extensions(p structPointer, f field) *XXX_InternalExtensions { - return (*XXX_InternalExtensions)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -func structPointer_ExtMap(p structPointer, f field) *map[int32]Extension { - return (*map[int32]Extension)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// NewAt returns the reflect.Value for a pointer to a field in the struct. -func structPointer_NewAt(p structPointer, f field, typ reflect.Type) reflect.Value { - return reflect.NewAt(typ, unsafe.Pointer(uintptr(p)+uintptr(f))) -} - -// SetStructPointer writes a *struct field in the struct. -func structPointer_SetStructPointer(p structPointer, f field, q structPointer) { - *(*structPointer)(unsafe.Pointer(uintptr(p) + uintptr(f))) = q -} - -// GetStructPointer reads a *struct field in the struct. -func structPointer_GetStructPointer(p structPointer, f field) structPointer { - return *(*structPointer)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// StructPointerSlice the address of a []*struct field in the struct. -func structPointer_StructPointerSlice(p structPointer, f field) *structPointerSlice { - return (*structPointerSlice)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// A structPointerSlice represents a slice of pointers to structs (themselves submessages or groups). -type structPointerSlice []structPointer - -func (v *structPointerSlice) Len() int { return len(*v) } -func (v *structPointerSlice) Index(i int) structPointer { return (*v)[i] } -func (v *structPointerSlice) Append(p structPointer) { *v = append(*v, p) } - -// A word32 is the address of a "pointer to 32-bit value" field. -type word32 **uint32 - -// IsNil reports whether *v is nil. -func word32_IsNil(p word32) bool { - return *p == nil -} - -// Set sets *v to point at a newly allocated word set to x. -func word32_Set(p word32, o *Buffer, x uint32) { - if len(o.uint32s) == 0 { - o.uint32s = make([]uint32, uint32PoolSize) - } - o.uint32s[0] = x - *p = &o.uint32s[0] - o.uint32s = o.uint32s[1:] -} - -// Get gets the value pointed at by *v. -func word32_Get(p word32) uint32 { - return **p -} - -// Word32 returns the address of a *int32, *uint32, *float32, or *enum field in the struct. -func structPointer_Word32(p structPointer, f field) word32 { - return word32((**uint32)(unsafe.Pointer(uintptr(p) + uintptr(f)))) -} - -// A word32Val is the address of a 32-bit value field. -type word32Val *uint32 - -// Set sets *p to x. -func word32Val_Set(p word32Val, x uint32) { - *p = x -} - -// Get gets the value pointed at by p. -func word32Val_Get(p word32Val) uint32 { - return *p -} - -// Word32Val returns the address of a *int32, *uint32, *float32, or *enum field in the struct. -func structPointer_Word32Val(p structPointer, f field) word32Val { - return word32Val((*uint32)(unsafe.Pointer(uintptr(p) + uintptr(f)))) -} - -// A word32Slice is a slice of 32-bit values. -type word32Slice []uint32 - -func (v *word32Slice) Append(x uint32) { *v = append(*v, x) } -func (v *word32Slice) Len() int { return len(*v) } -func (v *word32Slice) Index(i int) uint32 { return (*v)[i] } - -// Word32Slice returns the address of a []int32, []uint32, []float32, or []enum field in the struct. -func structPointer_Word32Slice(p structPointer, f field) *word32Slice { - return (*word32Slice)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -// word64 is like word32 but for 64-bit values. -type word64 **uint64 - -func word64_Set(p word64, o *Buffer, x uint64) { - if len(o.uint64s) == 0 { - o.uint64s = make([]uint64, uint64PoolSize) - } - o.uint64s[0] = x - *p = &o.uint64s[0] - o.uint64s = o.uint64s[1:] -} - -func word64_IsNil(p word64) bool { - return *p == nil -} - -func word64_Get(p word64) uint64 { - return **p -} - -func structPointer_Word64(p structPointer, f field) word64 { - return word64((**uint64)(unsafe.Pointer(uintptr(p) + uintptr(f)))) -} - -// word64Val is like word32Val but for 64-bit values. -type word64Val *uint64 - -func word64Val_Set(p word64Val, o *Buffer, x uint64) { - *p = x -} - -func word64Val_Get(p word64Val) uint64 { - return *p -} - -func structPointer_Word64Val(p structPointer, f field) word64Val { - return word64Val((*uint64)(unsafe.Pointer(uintptr(p) + uintptr(f)))) -} - -// word64Slice is like word32Slice but for 64-bit values. -type word64Slice []uint64 - -func (v *word64Slice) Append(x uint64) { *v = append(*v, x) } -func (v *word64Slice) Len() int { return len(*v) } -func (v *word64Slice) Index(i int) uint64 { return (*v)[i] } - -func structPointer_Word64Slice(p structPointer, f field) *word64Slice { - return (*word64Slice)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} diff --git a/vendor/github.com/gogo/protobuf/proto/pointer_unsafe_gogo.go b/vendor/github.com/gogo/protobuf/proto/pointer_unsafe_gogo.go deleted file mode 100644 index f156a29f0e830..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/pointer_unsafe_gogo.go +++ /dev/null @@ -1,128 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -// +build !appengine,!js - -// This file contains the implementation of the proto field accesses using package unsafe. - -package proto - -import ( - "reflect" - "unsafe" -) - -func structPointer_InterfaceAt(p structPointer, f field, t reflect.Type) interface{} { - point := unsafe.Pointer(uintptr(p) + uintptr(f)) - r := reflect.NewAt(t, point) - return r.Interface() -} - -func structPointer_InterfaceRef(p structPointer, f field, t reflect.Type) interface{} { - point := unsafe.Pointer(uintptr(p) + uintptr(f)) - r := reflect.NewAt(t, point) - if r.Elem().IsNil() { - return nil - } - return r.Elem().Interface() -} - -func copyUintPtr(oldptr, newptr uintptr, size int) { - oldbytes := make([]byte, 0) - oldslice := (*reflect.SliceHeader)(unsafe.Pointer(&oldbytes)) - oldslice.Data = oldptr - oldslice.Len = size - oldslice.Cap = size - newbytes := make([]byte, 0) - newslice := (*reflect.SliceHeader)(unsafe.Pointer(&newbytes)) - newslice.Data = newptr - newslice.Len = size - newslice.Cap = size - copy(newbytes, oldbytes) -} - -func structPointer_Copy(oldptr structPointer, newptr structPointer, size int) { - copyUintPtr(uintptr(oldptr), uintptr(newptr), size) -} - -func appendStructPointer(base structPointer, f field, typ reflect.Type) structPointer { - size := typ.Elem().Size() - - oldHeader := structPointer_GetSliceHeader(base, f) - oldSlice := reflect.NewAt(typ, unsafe.Pointer(oldHeader)).Elem() - newLen := oldHeader.Len + 1 - newSlice := reflect.MakeSlice(typ, newLen, newLen) - reflect.Copy(newSlice, oldSlice) - bas := toStructPointer(newSlice) - oldHeader.Data = uintptr(bas) - oldHeader.Len = newLen - oldHeader.Cap = newLen - - return structPointer(unsafe.Pointer(uintptr(unsafe.Pointer(bas)) + uintptr(uintptr(newLen-1)*size))) -} - -func structPointer_FieldPointer(p structPointer, f field) structPointer { - return structPointer(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -func structPointer_GetRefStructPointer(p structPointer, f field) structPointer { - return structPointer((*structPointer)(unsafe.Pointer(uintptr(p) + uintptr(f)))) -} - -func structPointer_GetSliceHeader(p structPointer, f field) *reflect.SliceHeader { - return (*reflect.SliceHeader)(unsafe.Pointer(uintptr(p) + uintptr(f))) -} - -func structPointer_Add(p structPointer, size field) structPointer { - return structPointer(unsafe.Pointer(uintptr(p) + uintptr(size))) -} - -func structPointer_Len(p structPointer, f field) int { - return len(*(*[]interface{})(unsafe.Pointer(structPointer_GetRefStructPointer(p, f)))) -} - -func structPointer_StructRefSlice(p structPointer, f field, size uintptr) *structRefSlice { - return &structRefSlice{p: p, f: f, size: size} -} - -// A structRefSlice represents a slice of structs (themselves submessages or groups). -type structRefSlice struct { - p structPointer - f field - size uintptr -} - -func (v *structRefSlice) Len() int { - return structPointer_Len(v.p, v.f) -} - -func (v *structRefSlice) Index(i int) structPointer { - ss := structPointer_GetStructPointer(v.p, v.f) - ss1 := structPointer_GetRefStructPointer(ss, 0) - return structPointer_Add(ss1, field(uintptr(i)*v.size)) -} diff --git a/vendor/github.com/gogo/protobuf/proto/properties.go b/vendor/github.com/gogo/protobuf/proto/properties.go deleted file mode 100644 index 2a69e8862d564..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/properties.go +++ /dev/null @@ -1,971 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -/* - * Routines for encoding data into the wire format for protocol buffers. - */ - -import ( - "fmt" - "log" - "os" - "reflect" - "sort" - "strconv" - "strings" - "sync" -) - -const debug bool = false - -// Constants that identify the encoding of a value on the wire. -const ( - WireVarint = 0 - WireFixed64 = 1 - WireBytes = 2 - WireStartGroup = 3 - WireEndGroup = 4 - WireFixed32 = 5 -) - -const startSize = 10 // initial slice/string sizes - -// Encoders are defined in encode.go -// An encoder outputs the full representation of a field, including its -// tag and encoder type. -type encoder func(p *Buffer, prop *Properties, base structPointer) error - -// A valueEncoder encodes a single integer in a particular encoding. -type valueEncoder func(o *Buffer, x uint64) error - -// Sizers are defined in encode.go -// A sizer returns the encoded size of a field, including its tag and encoder -// type. -type sizer func(prop *Properties, base structPointer) int - -// A valueSizer returns the encoded size of a single integer in a particular -// encoding. -type valueSizer func(x uint64) int - -// Decoders are defined in decode.go -// A decoder creates a value from its wire representation. -// Unrecognized subelements are saved in unrec. -type decoder func(p *Buffer, prop *Properties, base structPointer) error - -// A valueDecoder decodes a single integer in a particular encoding. -type valueDecoder func(o *Buffer) (x uint64, err error) - -// A oneofMarshaler does the marshaling for all oneof fields in a message. -type oneofMarshaler func(Message, *Buffer) error - -// A oneofUnmarshaler does the unmarshaling for a oneof field in a message. -type oneofUnmarshaler func(Message, int, int, *Buffer) (bool, error) - -// A oneofSizer does the sizing for all oneof fields in a message. -type oneofSizer func(Message) int - -// tagMap is an optimization over map[int]int for typical protocol buffer -// use-cases. Encoded protocol buffers are often in tag order with small tag -// numbers. -type tagMap struct { - fastTags []int - slowTags map[int]int -} - -// tagMapFastLimit is the upper bound on the tag number that will be stored in -// the tagMap slice rather than its map. -const tagMapFastLimit = 1024 - -func (p *tagMap) get(t int) (int, bool) { - if t > 0 && t < tagMapFastLimit { - if t >= len(p.fastTags) { - return 0, false - } - fi := p.fastTags[t] - return fi, fi >= 0 - } - fi, ok := p.slowTags[t] - return fi, ok -} - -func (p *tagMap) put(t int, fi int) { - if t > 0 && t < tagMapFastLimit { - for len(p.fastTags) < t+1 { - p.fastTags = append(p.fastTags, -1) - } - p.fastTags[t] = fi - return - } - if p.slowTags == nil { - p.slowTags = make(map[int]int) - } - p.slowTags[t] = fi -} - -// StructProperties represents properties for all the fields of a struct. -// decoderTags and decoderOrigNames should only be used by the decoder. -type StructProperties struct { - Prop []*Properties // properties for each field - reqCount int // required count - decoderTags tagMap // map from proto tag to struct field number - decoderOrigNames map[string]int // map from original name to struct field number - order []int // list of struct field numbers in tag order - unrecField field // field id of the XXX_unrecognized []byte field - extendable bool // is this an extendable proto - - oneofMarshaler oneofMarshaler - oneofUnmarshaler oneofUnmarshaler - oneofSizer oneofSizer - stype reflect.Type - - // OneofTypes contains information about the oneof fields in this message. - // It is keyed by the original name of a field. - OneofTypes map[string]*OneofProperties -} - -// OneofProperties represents information about a specific field in a oneof. -type OneofProperties struct { - Type reflect.Type // pointer to generated struct type for this oneof field - Field int // struct field number of the containing oneof in the message - Prop *Properties -} - -// Implement the sorting interface so we can sort the fields in tag order, as recommended by the spec. -// See encode.go, (*Buffer).enc_struct. - -func (sp *StructProperties) Len() int { return len(sp.order) } -func (sp *StructProperties) Less(i, j int) bool { - return sp.Prop[sp.order[i]].Tag < sp.Prop[sp.order[j]].Tag -} -func (sp *StructProperties) Swap(i, j int) { sp.order[i], sp.order[j] = sp.order[j], sp.order[i] } - -// Properties represents the protocol-specific behavior of a single struct field. -type Properties struct { - Name string // name of the field, for error messages - OrigName string // original name before protocol compiler (always set) - JSONName string // name to use for JSON; determined by protoc - Wire string - WireType int - Tag int - Required bool - Optional bool - Repeated bool - Packed bool // relevant for repeated primitives only - Enum string // set for enum types only - proto3 bool // whether this is known to be a proto3 field; set for []byte only - oneof bool // whether this is a oneof field - - Default string // default value - HasDefault bool // whether an explicit default was provided - CustomType string - CastType string - StdTime bool - StdDuration bool - - enc encoder - valEnc valueEncoder // set for bool and numeric types only - field field - tagcode []byte // encoding of EncodeVarint((Tag<<3)|WireType) - tagbuf [8]byte - stype reflect.Type // set for struct types only - sstype reflect.Type // set for slices of structs types only - ctype reflect.Type // set for custom types only - sprop *StructProperties // set for struct types only - isMarshaler bool - isUnmarshaler bool - - mtype reflect.Type // set for map types only - mkeyprop *Properties // set for map types only - mvalprop *Properties // set for map types only - - size sizer - valSize valueSizer // set for bool and numeric types only - - dec decoder - valDec valueDecoder // set for bool and numeric types only - - // If this is a packable field, this will be the decoder for the packed version of the field. - packedDec decoder -} - -// String formats the properties in the protobuf struct field tag style. -func (p *Properties) String() string { - s := p.Wire - s = "," - s += strconv.Itoa(p.Tag) - if p.Required { - s += ",req" - } - if p.Optional { - s += ",opt" - } - if p.Repeated { - s += ",rep" - } - if p.Packed { - s += ",packed" - } - s += ",name=" + p.OrigName - if p.JSONName != p.OrigName { - s += ",json=" + p.JSONName - } - if p.proto3 { - s += ",proto3" - } - if p.oneof { - s += ",oneof" - } - if len(p.Enum) > 0 { - s += ",enum=" + p.Enum - } - if p.HasDefault { - s += ",def=" + p.Default - } - return s -} - -// Parse populates p by parsing a string in the protobuf struct field tag style. -func (p *Properties) Parse(s string) { - // "bytes,49,opt,name=foo,def=hello!" - fields := strings.Split(s, ",") // breaks def=, but handled below. - if len(fields) < 2 { - fmt.Fprintf(os.Stderr, "proto: tag has too few fields: %q\n", s) - return - } - - p.Wire = fields[0] - switch p.Wire { - case "varint": - p.WireType = WireVarint - p.valEnc = (*Buffer).EncodeVarint - p.valDec = (*Buffer).DecodeVarint - p.valSize = sizeVarint - case "fixed32": - p.WireType = WireFixed32 - p.valEnc = (*Buffer).EncodeFixed32 - p.valDec = (*Buffer).DecodeFixed32 - p.valSize = sizeFixed32 - case "fixed64": - p.WireType = WireFixed64 - p.valEnc = (*Buffer).EncodeFixed64 - p.valDec = (*Buffer).DecodeFixed64 - p.valSize = sizeFixed64 - case "zigzag32": - p.WireType = WireVarint - p.valEnc = (*Buffer).EncodeZigzag32 - p.valDec = (*Buffer).DecodeZigzag32 - p.valSize = sizeZigzag32 - case "zigzag64": - p.WireType = WireVarint - p.valEnc = (*Buffer).EncodeZigzag64 - p.valDec = (*Buffer).DecodeZigzag64 - p.valSize = sizeZigzag64 - case "bytes", "group": - p.WireType = WireBytes - // no numeric converter for non-numeric types - default: - fmt.Fprintf(os.Stderr, "proto: tag has unknown wire type: %q\n", s) - return - } - - var err error - p.Tag, err = strconv.Atoi(fields[1]) - if err != nil { - return - } - - for i := 2; i < len(fields); i++ { - f := fields[i] - switch { - case f == "req": - p.Required = true - case f == "opt": - p.Optional = true - case f == "rep": - p.Repeated = true - case f == "packed": - p.Packed = true - case strings.HasPrefix(f, "name="): - p.OrigName = f[5:] - case strings.HasPrefix(f, "json="): - p.JSONName = f[5:] - case strings.HasPrefix(f, "enum="): - p.Enum = f[5:] - case f == "proto3": - p.proto3 = true - case f == "oneof": - p.oneof = true - case strings.HasPrefix(f, "def="): - p.HasDefault = true - p.Default = f[4:] // rest of string - if i+1 < len(fields) { - // Commas aren't escaped, and def is always last. - p.Default += "," + strings.Join(fields[i+1:], ",") - break - } - case strings.HasPrefix(f, "embedded="): - p.OrigName = strings.Split(f, "=")[1] - case strings.HasPrefix(f, "customtype="): - p.CustomType = strings.Split(f, "=")[1] - case strings.HasPrefix(f, "casttype="): - p.CastType = strings.Split(f, "=")[1] - case f == "stdtime": - p.StdTime = true - case f == "stdduration": - p.StdDuration = true - } - } -} - -func logNoSliceEnc(t1, t2 reflect.Type) { - fmt.Fprintf(os.Stderr, "proto: no slice oenc for %T = []%T\n", t1, t2) -} - -var protoMessageType = reflect.TypeOf((*Message)(nil)).Elem() - -// Initialize the fields for encoding and decoding. -func (p *Properties) setEncAndDec(typ reflect.Type, f *reflect.StructField, lockGetProp bool) { - p.enc = nil - p.dec = nil - p.size = nil - isMap := typ.Kind() == reflect.Map - if len(p.CustomType) > 0 && !isMap { - p.setCustomEncAndDec(typ) - p.setTag(lockGetProp) - return - } - if p.StdTime && !isMap { - p.setTimeEncAndDec(typ) - p.setTag(lockGetProp) - return - } - if p.StdDuration && !isMap { - p.setDurationEncAndDec(typ) - p.setTag(lockGetProp) - return - } - switch t1 := typ; t1.Kind() { - default: - fmt.Fprintf(os.Stderr, "proto: no coders for %v\n", t1) - - // proto3 scalar types - - case reflect.Bool: - if p.proto3 { - p.enc = (*Buffer).enc_proto3_bool - p.dec = (*Buffer).dec_proto3_bool - p.size = size_proto3_bool - } else { - p.enc = (*Buffer).enc_ref_bool - p.dec = (*Buffer).dec_proto3_bool - p.size = size_ref_bool - } - case reflect.Int32: - if p.proto3 { - p.enc = (*Buffer).enc_proto3_int32 - p.dec = (*Buffer).dec_proto3_int32 - p.size = size_proto3_int32 - } else { - p.enc = (*Buffer).enc_ref_int32 - p.dec = (*Buffer).dec_proto3_int32 - p.size = size_ref_int32 - } - case reflect.Uint32: - if p.proto3 { - p.enc = (*Buffer).enc_proto3_uint32 - p.dec = (*Buffer).dec_proto3_int32 // can reuse - p.size = size_proto3_uint32 - } else { - p.enc = (*Buffer).enc_ref_uint32 - p.dec = (*Buffer).dec_proto3_int32 // can reuse - p.size = size_ref_uint32 - } - case reflect.Int64, reflect.Uint64: - if p.proto3 { - p.enc = (*Buffer).enc_proto3_int64 - p.dec = (*Buffer).dec_proto3_int64 - p.size = size_proto3_int64 - } else { - p.enc = (*Buffer).enc_ref_int64 - p.dec = (*Buffer).dec_proto3_int64 - p.size = size_ref_int64 - } - case reflect.Float32: - if p.proto3 { - p.enc = (*Buffer).enc_proto3_uint32 // can just treat them as bits - p.dec = (*Buffer).dec_proto3_int32 - p.size = size_proto3_uint32 - } else { - p.enc = (*Buffer).enc_ref_uint32 // can just treat them as bits - p.dec = (*Buffer).dec_proto3_int32 - p.size = size_ref_uint32 - } - case reflect.Float64: - if p.proto3 { - p.enc = (*Buffer).enc_proto3_int64 // can just treat them as bits - p.dec = (*Buffer).dec_proto3_int64 - p.size = size_proto3_int64 - } else { - p.enc = (*Buffer).enc_ref_int64 // can just treat them as bits - p.dec = (*Buffer).dec_proto3_int64 - p.size = size_ref_int64 - } - case reflect.String: - if p.proto3 { - p.enc = (*Buffer).enc_proto3_string - p.dec = (*Buffer).dec_proto3_string - p.size = size_proto3_string - } else { - p.enc = (*Buffer).enc_ref_string - p.dec = (*Buffer).dec_proto3_string - p.size = size_ref_string - } - case reflect.Struct: - p.stype = typ - p.isMarshaler = isMarshaler(typ) - p.isUnmarshaler = isUnmarshaler(typ) - if p.Wire == "bytes" { - p.enc = (*Buffer).enc_ref_struct_message - p.dec = (*Buffer).dec_ref_struct_message - p.size = size_ref_struct_message - } else { - fmt.Fprintf(os.Stderr, "proto: no coders for struct %T\n", typ) - } - - case reflect.Ptr: - switch t2 := t1.Elem(); t2.Kind() { - default: - fmt.Fprintf(os.Stderr, "proto: no encoder function for %v -> %v\n", t1, t2) - break - case reflect.Bool: - p.enc = (*Buffer).enc_bool - p.dec = (*Buffer).dec_bool - p.size = size_bool - case reflect.Int32: - p.enc = (*Buffer).enc_int32 - p.dec = (*Buffer).dec_int32 - p.size = size_int32 - case reflect.Uint32: - p.enc = (*Buffer).enc_uint32 - p.dec = (*Buffer).dec_int32 // can reuse - p.size = size_uint32 - case reflect.Int64, reflect.Uint64: - p.enc = (*Buffer).enc_int64 - p.dec = (*Buffer).dec_int64 - p.size = size_int64 - case reflect.Float32: - p.enc = (*Buffer).enc_uint32 // can just treat them as bits - p.dec = (*Buffer).dec_int32 - p.size = size_uint32 - case reflect.Float64: - p.enc = (*Buffer).enc_int64 // can just treat them as bits - p.dec = (*Buffer).dec_int64 - p.size = size_int64 - case reflect.String: - p.enc = (*Buffer).enc_string - p.dec = (*Buffer).dec_string - p.size = size_string - case reflect.Struct: - p.stype = t1.Elem() - p.isMarshaler = isMarshaler(t1) - p.isUnmarshaler = isUnmarshaler(t1) - if p.Wire == "bytes" { - p.enc = (*Buffer).enc_struct_message - p.dec = (*Buffer).dec_struct_message - p.size = size_struct_message - } else { - p.enc = (*Buffer).enc_struct_group - p.dec = (*Buffer).dec_struct_group - p.size = size_struct_group - } - } - - case reflect.Slice: - switch t2 := t1.Elem(); t2.Kind() { - default: - logNoSliceEnc(t1, t2) - break - case reflect.Bool: - if p.Packed { - p.enc = (*Buffer).enc_slice_packed_bool - p.size = size_slice_packed_bool - } else { - p.enc = (*Buffer).enc_slice_bool - p.size = size_slice_bool - } - p.dec = (*Buffer).dec_slice_bool - p.packedDec = (*Buffer).dec_slice_packed_bool - case reflect.Int32: - if p.Packed { - p.enc = (*Buffer).enc_slice_packed_int32 - p.size = size_slice_packed_int32 - } else { - p.enc = (*Buffer).enc_slice_int32 - p.size = size_slice_int32 - } - p.dec = (*Buffer).dec_slice_int32 - p.packedDec = (*Buffer).dec_slice_packed_int32 - case reflect.Uint32: - if p.Packed { - p.enc = (*Buffer).enc_slice_packed_uint32 - p.size = size_slice_packed_uint32 - } else { - p.enc = (*Buffer).enc_slice_uint32 - p.size = size_slice_uint32 - } - p.dec = (*Buffer).dec_slice_int32 - p.packedDec = (*Buffer).dec_slice_packed_int32 - case reflect.Int64, reflect.Uint64: - if p.Packed { - p.enc = (*Buffer).enc_slice_packed_int64 - p.size = size_slice_packed_int64 - } else { - p.enc = (*Buffer).enc_slice_int64 - p.size = size_slice_int64 - } - p.dec = (*Buffer).dec_slice_int64 - p.packedDec = (*Buffer).dec_slice_packed_int64 - case reflect.Uint8: - p.dec = (*Buffer).dec_slice_byte - if p.proto3 { - p.enc = (*Buffer).enc_proto3_slice_byte - p.size = size_proto3_slice_byte - } else { - p.enc = (*Buffer).enc_slice_byte - p.size = size_slice_byte - } - case reflect.Float32, reflect.Float64: - switch t2.Bits() { - case 32: - // can just treat them as bits - if p.Packed { - p.enc = (*Buffer).enc_slice_packed_uint32 - p.size = size_slice_packed_uint32 - } else { - p.enc = (*Buffer).enc_slice_uint32 - p.size = size_slice_uint32 - } - p.dec = (*Buffer).dec_slice_int32 - p.packedDec = (*Buffer).dec_slice_packed_int32 - case 64: - // can just treat them as bits - if p.Packed { - p.enc = (*Buffer).enc_slice_packed_int64 - p.size = size_slice_packed_int64 - } else { - p.enc = (*Buffer).enc_slice_int64 - p.size = size_slice_int64 - } - p.dec = (*Buffer).dec_slice_int64 - p.packedDec = (*Buffer).dec_slice_packed_int64 - default: - logNoSliceEnc(t1, t2) - break - } - case reflect.String: - p.enc = (*Buffer).enc_slice_string - p.dec = (*Buffer).dec_slice_string - p.size = size_slice_string - case reflect.Ptr: - switch t3 := t2.Elem(); t3.Kind() { - default: - fmt.Fprintf(os.Stderr, "proto: no ptr oenc for %T -> %T -> %T\n", t1, t2, t3) - break - case reflect.Struct: - p.stype = t2.Elem() - p.isMarshaler = isMarshaler(t2) - p.isUnmarshaler = isUnmarshaler(t2) - if p.Wire == "bytes" { - p.enc = (*Buffer).enc_slice_struct_message - p.dec = (*Buffer).dec_slice_struct_message - p.size = size_slice_struct_message - } else { - p.enc = (*Buffer).enc_slice_struct_group - p.dec = (*Buffer).dec_slice_struct_group - p.size = size_slice_struct_group - } - } - case reflect.Slice: - switch t2.Elem().Kind() { - default: - fmt.Fprintf(os.Stderr, "proto: no slice elem oenc for %T -> %T -> %T\n", t1, t2, t2.Elem()) - break - case reflect.Uint8: - p.enc = (*Buffer).enc_slice_slice_byte - p.dec = (*Buffer).dec_slice_slice_byte - p.size = size_slice_slice_byte - } - case reflect.Struct: - p.setSliceOfNonPointerStructs(t1) - } - - case reflect.Map: - p.enc = (*Buffer).enc_new_map - p.dec = (*Buffer).dec_new_map - p.size = size_new_map - - p.mtype = t1 - p.mkeyprop = &Properties{} - p.mkeyprop.init(reflect.PtrTo(p.mtype.Key()), "Key", f.Tag.Get("protobuf_key"), nil, lockGetProp) - p.mvalprop = &Properties{} - vtype := p.mtype.Elem() - if vtype.Kind() != reflect.Ptr && vtype.Kind() != reflect.Slice { - // The value type is not a message (*T) or bytes ([]byte), - // so we need encoders for the pointer to this type. - vtype = reflect.PtrTo(vtype) - } - - p.mvalprop.CustomType = p.CustomType - p.mvalprop.StdDuration = p.StdDuration - p.mvalprop.StdTime = p.StdTime - p.mvalprop.init(vtype, "Value", f.Tag.Get("protobuf_val"), nil, lockGetProp) - } - p.setTag(lockGetProp) -} - -func (p *Properties) setTag(lockGetProp bool) { - // precalculate tag code - wire := p.WireType - if p.Packed { - wire = WireBytes - } - x := uint32(p.Tag)<<3 | uint32(wire) - i := 0 - for i = 0; x > 127; i++ { - p.tagbuf[i] = 0x80 | uint8(x&0x7F) - x >>= 7 - } - p.tagbuf[i] = uint8(x) - p.tagcode = p.tagbuf[0 : i+1] - - if p.stype != nil { - if lockGetProp { - p.sprop = GetProperties(p.stype) - } else { - p.sprop = getPropertiesLocked(p.stype) - } - } -} - -var ( - marshalerType = reflect.TypeOf((*Marshaler)(nil)).Elem() - unmarshalerType = reflect.TypeOf((*Unmarshaler)(nil)).Elem() -) - -// isMarshaler reports whether type t implements Marshaler. -func isMarshaler(t reflect.Type) bool { - return t.Implements(marshalerType) -} - -// isUnmarshaler reports whether type t implements Unmarshaler. -func isUnmarshaler(t reflect.Type) bool { - return t.Implements(unmarshalerType) -} - -// Init populates the properties from a protocol buffer struct tag. -func (p *Properties) Init(typ reflect.Type, name, tag string, f *reflect.StructField) { - p.init(typ, name, tag, f, true) -} - -func (p *Properties) init(typ reflect.Type, name, tag string, f *reflect.StructField, lockGetProp bool) { - // "bytes,49,opt,def=hello!" - p.Name = name - p.OrigName = name - if f != nil { - p.field = toField(f) - } - if tag == "" { - return - } - p.Parse(tag) - p.setEncAndDec(typ, f, lockGetProp) -} - -var ( - propertiesMu sync.RWMutex - propertiesMap = make(map[reflect.Type]*StructProperties) -) - -// GetProperties returns the list of properties for the type represented by t. -// t must represent a generated struct type of a protocol message. -func GetProperties(t reflect.Type) *StructProperties { - if t.Kind() != reflect.Struct { - panic("proto: type must have kind struct") - } - - // Most calls to GetProperties in a long-running program will be - // retrieving details for types we have seen before. - propertiesMu.RLock() - sprop, ok := propertiesMap[t] - propertiesMu.RUnlock() - if ok { - if collectStats { - stats.Chit++ - } - return sprop - } - - propertiesMu.Lock() - sprop = getPropertiesLocked(t) - propertiesMu.Unlock() - return sprop -} - -// getPropertiesLocked requires that propertiesMu is held. -func getPropertiesLocked(t reflect.Type) *StructProperties { - if prop, ok := propertiesMap[t]; ok { - if collectStats { - stats.Chit++ - } - return prop - } - if collectStats { - stats.Cmiss++ - } - - prop := new(StructProperties) - // in case of recursive protos, fill this in now. - propertiesMap[t] = prop - - // build properties - prop.extendable = reflect.PtrTo(t).Implements(extendableProtoType) || - reflect.PtrTo(t).Implements(extendableProtoV1Type) || - reflect.PtrTo(t).Implements(extendableBytesType) - prop.unrecField = invalidField - prop.Prop = make([]*Properties, t.NumField()) - prop.order = make([]int, t.NumField()) - - isOneofMessage := false - for i := 0; i < t.NumField(); i++ { - f := t.Field(i) - p := new(Properties) - name := f.Name - p.init(f.Type, name, f.Tag.Get("protobuf"), &f, false) - - if f.Name == "XXX_InternalExtensions" { // special case - p.enc = (*Buffer).enc_exts - p.dec = nil // not needed - p.size = size_exts - } else if f.Name == "XXX_extensions" { // special case - if len(f.Tag.Get("protobuf")) > 0 { - p.enc = (*Buffer).enc_ext_slice_byte - p.dec = nil // not needed - p.size = size_ext_slice_byte - } else { - p.enc = (*Buffer).enc_map - p.dec = nil // not needed - p.size = size_map - } - } else if f.Name == "XXX_unrecognized" { // special case - prop.unrecField = toField(&f) - } - oneof := f.Tag.Get("protobuf_oneof") // special case - if oneof != "" { - isOneofMessage = true - // Oneof fields don't use the traditional protobuf tag. - p.OrigName = oneof - } - prop.Prop[i] = p - prop.order[i] = i - if debug { - print(i, " ", f.Name, " ", t.String(), " ") - if p.Tag > 0 { - print(p.String()) - } - print("\n") - } - if p.enc == nil && !strings.HasPrefix(f.Name, "XXX_") && oneof == "" { - fmt.Fprintln(os.Stderr, "proto: no encoder for", f.Name, f.Type.String(), "[GetProperties]") - } - } - - // Re-order prop.order. - sort.Sort(prop) - - type oneofMessage interface { - XXX_OneofFuncs() (func(Message, *Buffer) error, func(Message, int, int, *Buffer) (bool, error), func(Message) int, []interface{}) - } - if om, ok := reflect.Zero(reflect.PtrTo(t)).Interface().(oneofMessage); isOneofMessage && ok { - var oots []interface{} - prop.oneofMarshaler, prop.oneofUnmarshaler, prop.oneofSizer, oots = om.XXX_OneofFuncs() - prop.stype = t - - // Interpret oneof metadata. - prop.OneofTypes = make(map[string]*OneofProperties) - for _, oot := range oots { - oop := &OneofProperties{ - Type: reflect.ValueOf(oot).Type(), // *T - Prop: new(Properties), - } - sft := oop.Type.Elem().Field(0) - oop.Prop.Name = sft.Name - oop.Prop.Parse(sft.Tag.Get("protobuf")) - // There will be exactly one interface field that - // this new value is assignable to. - for i := 0; i < t.NumField(); i++ { - f := t.Field(i) - if f.Type.Kind() != reflect.Interface { - continue - } - if !oop.Type.AssignableTo(f.Type) { - continue - } - oop.Field = i - break - } - prop.OneofTypes[oop.Prop.OrigName] = oop - } - } - - // build required counts - // build tags - reqCount := 0 - prop.decoderOrigNames = make(map[string]int) - for i, p := range prop.Prop { - if strings.HasPrefix(p.Name, "XXX_") { - // Internal fields should not appear in tags/origNames maps. - // They are handled specially when encoding and decoding. - continue - } - if p.Required { - reqCount++ - } - prop.decoderTags.put(p.Tag, i) - prop.decoderOrigNames[p.OrigName] = i - } - prop.reqCount = reqCount - - return prop -} - -// Return the Properties object for the x[0]'th field of the structure. -func propByIndex(t reflect.Type, x []int) *Properties { - if len(x) != 1 { - fmt.Fprintf(os.Stderr, "proto: field index dimension %d (not 1) for type %s\n", len(x), t) - return nil - } - prop := GetProperties(t) - return prop.Prop[x[0]] -} - -// Get the address and type of a pointer to a struct from an interface. -func getbase(pb Message) (t reflect.Type, b structPointer, err error) { - if pb == nil { - err = ErrNil - return - } - // get the reflect type of the pointer to the struct. - t = reflect.TypeOf(pb) - // get the address of the struct. - value := reflect.ValueOf(pb) - b = toStructPointer(value) - return -} - -// A global registry of enum types. -// The generated code will register the generated maps by calling RegisterEnum. - -var enumValueMaps = make(map[string]map[string]int32) -var enumStringMaps = make(map[string]map[int32]string) - -// RegisterEnum is called from the generated code to install the enum descriptor -// maps into the global table to aid parsing text format protocol buffers. -func RegisterEnum(typeName string, unusedNameMap map[int32]string, valueMap map[string]int32) { - if _, ok := enumValueMaps[typeName]; ok { - panic("proto: duplicate enum registered: " + typeName) - } - enumValueMaps[typeName] = valueMap - if _, ok := enumStringMaps[typeName]; ok { - panic("proto: duplicate enum registered: " + typeName) - } - enumStringMaps[typeName] = unusedNameMap -} - -// EnumValueMap returns the mapping from names to integers of the -// enum type enumType, or a nil if not found. -func EnumValueMap(enumType string) map[string]int32 { - return enumValueMaps[enumType] -} - -// A registry of all linked message types. -// The string is a fully-qualified proto name ("pkg.Message"). -var ( - protoTypes = make(map[string]reflect.Type) - revProtoTypes = make(map[reflect.Type]string) -) - -// RegisterType is called from generated code and maps from the fully qualified -// proto name to the type (pointer to struct) of the protocol buffer. -func RegisterType(x Message, name string) { - if _, ok := protoTypes[name]; ok { - // TODO: Some day, make this a panic. - log.Printf("proto: duplicate proto type registered: %s", name) - return - } - t := reflect.TypeOf(x) - protoTypes[name] = t - revProtoTypes[t] = name -} - -// MessageName returns the fully-qualified proto name for the given message type. -func MessageName(x Message) string { - type xname interface { - XXX_MessageName() string - } - if m, ok := x.(xname); ok { - return m.XXX_MessageName() - } - return revProtoTypes[reflect.TypeOf(x)] -} - -// MessageType returns the message type (pointer to struct) for a named message. -func MessageType(name string) reflect.Type { return protoTypes[name] } - -// A registry of all linked proto files. -var ( - protoFiles = make(map[string][]byte) // file name => fileDescriptor -) - -// RegisterFile is called from generated code and maps from the -// full file name of a .proto file to its compressed FileDescriptorProto. -func RegisterFile(filename string, fileDescriptor []byte) { - protoFiles[filename] = fileDescriptor -} - -// FileDescriptor returns the compressed FileDescriptorProto for a .proto file. -func FileDescriptor(filename string) []byte { return protoFiles[filename] } diff --git a/vendor/github.com/gogo/protobuf/proto/properties_gogo.go b/vendor/github.com/gogo/protobuf/proto/properties_gogo.go deleted file mode 100644 index b6b7176c5656f..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/properties_gogo.go +++ /dev/null @@ -1,111 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "fmt" - "os" - "reflect" -) - -func (p *Properties) setCustomEncAndDec(typ reflect.Type) { - p.ctype = typ - if p.Repeated { - p.enc = (*Buffer).enc_custom_slice_bytes - p.dec = (*Buffer).dec_custom_slice_bytes - p.size = size_custom_slice_bytes - } else if typ.Kind() == reflect.Ptr { - p.enc = (*Buffer).enc_custom_bytes - p.dec = (*Buffer).dec_custom_bytes - p.size = size_custom_bytes - } else { - p.enc = (*Buffer).enc_custom_ref_bytes - p.dec = (*Buffer).dec_custom_ref_bytes - p.size = size_custom_ref_bytes - } -} - -func (p *Properties) setDurationEncAndDec(typ reflect.Type) { - if p.Repeated { - if typ.Elem().Kind() == reflect.Ptr { - p.enc = (*Buffer).enc_slice_duration - p.dec = (*Buffer).dec_slice_duration - p.size = size_slice_duration - } else { - p.enc = (*Buffer).enc_slice_ref_duration - p.dec = (*Buffer).dec_slice_ref_duration - p.size = size_slice_ref_duration - } - } else if typ.Kind() == reflect.Ptr { - p.enc = (*Buffer).enc_duration - p.dec = (*Buffer).dec_duration - p.size = size_duration - } else { - p.enc = (*Buffer).enc_ref_duration - p.dec = (*Buffer).dec_ref_duration - p.size = size_ref_duration - } -} - -func (p *Properties) setTimeEncAndDec(typ reflect.Type) { - if p.Repeated { - if typ.Elem().Kind() == reflect.Ptr { - p.enc = (*Buffer).enc_slice_time - p.dec = (*Buffer).dec_slice_time - p.size = size_slice_time - } else { - p.enc = (*Buffer).enc_slice_ref_time - p.dec = (*Buffer).dec_slice_ref_time - p.size = size_slice_ref_time - } - } else if typ.Kind() == reflect.Ptr { - p.enc = (*Buffer).enc_time - p.dec = (*Buffer).dec_time - p.size = size_time - } else { - p.enc = (*Buffer).enc_ref_time - p.dec = (*Buffer).dec_ref_time - p.size = size_ref_time - } - -} - -func (p *Properties) setSliceOfNonPointerStructs(typ reflect.Type) { - t2 := typ.Elem() - p.sstype = typ - p.stype = t2 - p.isMarshaler = isMarshaler(t2) - p.isUnmarshaler = isUnmarshaler(t2) - p.enc = (*Buffer).enc_slice_ref_struct_message - p.dec = (*Buffer).dec_slice_ref_struct_message - p.size = size_slice_ref_struct_message - if p.Wire != "bytes" { - fmt.Fprintf(os.Stderr, "proto: no ptr oenc for %T -> %T \n", typ, t2) - } -} diff --git a/vendor/github.com/gogo/protobuf/proto/skip_gogo.go b/vendor/github.com/gogo/protobuf/proto/skip_gogo.go deleted file mode 100644 index 5a5fd93f7c196..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/skip_gogo.go +++ /dev/null @@ -1,119 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "fmt" - "io" -) - -func Skip(data []byte) (n int, err error) { - l := len(data) - index := 0 - for index < l { - var wire uint64 - for shift := uint(0); ; shift += 7 { - if index >= l { - return 0, io.ErrUnexpectedEOF - } - b := data[index] - index++ - wire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - wireType := int(wire & 0x7) - switch wireType { - case 0: - for { - if index >= l { - return 0, io.ErrUnexpectedEOF - } - index++ - if data[index-1] < 0x80 { - break - } - } - return index, nil - case 1: - index += 8 - return index, nil - case 2: - var length int - for shift := uint(0); ; shift += 7 { - if index >= l { - return 0, io.ErrUnexpectedEOF - } - b := data[index] - index++ - length |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - index += length - return index, nil - case 3: - for { - var innerWire uint64 - var start int = index - for shift := uint(0); ; shift += 7 { - if index >= l { - return 0, io.ErrUnexpectedEOF - } - b := data[index] - index++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := Skip(data[start:]) - if err != nil { - return 0, err - } - index = start + next - } - return index, nil - case 4: - return index, nil - case 5: - index += 4 - return index, nil - default: - return 0, fmt.Errorf("proto: illegal wireType %d", wireType) - } - } - panic("unreachable") -} diff --git a/vendor/github.com/gogo/protobuf/proto/text.go b/vendor/github.com/gogo/protobuf/proto/text.go deleted file mode 100644 index f609d1d453b04..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/text.go +++ /dev/null @@ -1,939 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -// Functions for writing the text protocol buffer format. - -import ( - "bufio" - "bytes" - "encoding" - "errors" - "fmt" - "io" - "log" - "math" - "reflect" - "sort" - "strings" - "sync" - "time" -) - -var ( - newline = []byte("\n") - spaces = []byte(" ") - gtNewline = []byte(">\n") - endBraceNewline = []byte("}\n") - backslashN = []byte{'\\', 'n'} - backslashR = []byte{'\\', 'r'} - backslashT = []byte{'\\', 't'} - backslashDQ = []byte{'\\', '"'} - backslashBS = []byte{'\\', '\\'} - posInf = []byte("inf") - negInf = []byte("-inf") - nan = []byte("nan") -) - -type writer interface { - io.Writer - WriteByte(byte) error -} - -// textWriter is an io.Writer that tracks its indentation level. -type textWriter struct { - ind int - complete bool // if the current position is a complete line - compact bool // whether to write out as a one-liner - w writer -} - -func (w *textWriter) WriteString(s string) (n int, err error) { - if !strings.Contains(s, "\n") { - if !w.compact && w.complete { - w.writeIndent() - } - w.complete = false - return io.WriteString(w.w, s) - } - // WriteString is typically called without newlines, so this - // codepath and its copy are rare. We copy to avoid - // duplicating all of Write's logic here. - return w.Write([]byte(s)) -} - -func (w *textWriter) Write(p []byte) (n int, err error) { - newlines := bytes.Count(p, newline) - if newlines == 0 { - if !w.compact && w.complete { - w.writeIndent() - } - n, err = w.w.Write(p) - w.complete = false - return n, err - } - - frags := bytes.SplitN(p, newline, newlines+1) - if w.compact { - for i, frag := range frags { - if i > 0 { - if err := w.w.WriteByte(' '); err != nil { - return n, err - } - n++ - } - nn, err := w.w.Write(frag) - n += nn - if err != nil { - return n, err - } - } - return n, nil - } - - for i, frag := range frags { - if w.complete { - w.writeIndent() - } - nn, err := w.w.Write(frag) - n += nn - if err != nil { - return n, err - } - if i+1 < len(frags) { - if err := w.w.WriteByte('\n'); err != nil { - return n, err - } - n++ - } - } - w.complete = len(frags[len(frags)-1]) == 0 - return n, nil -} - -func (w *textWriter) WriteByte(c byte) error { - if w.compact && c == '\n' { - c = ' ' - } - if !w.compact && w.complete { - w.writeIndent() - } - err := w.w.WriteByte(c) - w.complete = c == '\n' - return err -} - -func (w *textWriter) indent() { w.ind++ } - -func (w *textWriter) unindent() { - if w.ind == 0 { - log.Print("proto: textWriter unindented too far") - return - } - w.ind-- -} - -func writeName(w *textWriter, props *Properties) error { - if _, err := w.WriteString(props.OrigName); err != nil { - return err - } - if props.Wire != "group" { - return w.WriteByte(':') - } - return nil -} - -// raw is the interface satisfied by RawMessage. -type raw interface { - Bytes() []byte -} - -func requiresQuotes(u string) bool { - // When type URL contains any characters except [0-9A-Za-z./\-]*, it must be quoted. - for _, ch := range u { - switch { - case ch == '.' || ch == '/' || ch == '_': - continue - case '0' <= ch && ch <= '9': - continue - case 'A' <= ch && ch <= 'Z': - continue - case 'a' <= ch && ch <= 'z': - continue - default: - return true - } - } - return false -} - -// isAny reports whether sv is a google.protobuf.Any message -func isAny(sv reflect.Value) bool { - type wkt interface { - XXX_WellKnownType() string - } - t, ok := sv.Addr().Interface().(wkt) - return ok && t.XXX_WellKnownType() == "Any" -} - -// writeProto3Any writes an expanded google.protobuf.Any message. -// -// It returns (false, nil) if sv value can't be unmarshaled (e.g. because -// required messages are not linked in). -// -// It returns (true, error) when sv was written in expanded format or an error -// was encountered. -func (tm *TextMarshaler) writeProto3Any(w *textWriter, sv reflect.Value) (bool, error) { - turl := sv.FieldByName("TypeUrl") - val := sv.FieldByName("Value") - if !turl.IsValid() || !val.IsValid() { - return true, errors.New("proto: invalid google.protobuf.Any message") - } - - b, ok := val.Interface().([]byte) - if !ok { - return true, errors.New("proto: invalid google.protobuf.Any message") - } - - parts := strings.Split(turl.String(), "/") - mt := MessageType(parts[len(parts)-1]) - if mt == nil { - return false, nil - } - m := reflect.New(mt.Elem()) - if err := Unmarshal(b, m.Interface().(Message)); err != nil { - return false, nil - } - w.Write([]byte("[")) - u := turl.String() - if requiresQuotes(u) { - writeString(w, u) - } else { - w.Write([]byte(u)) - } - if w.compact { - w.Write([]byte("]:<")) - } else { - w.Write([]byte("]: <\n")) - w.ind++ - } - if err := tm.writeStruct(w, m.Elem()); err != nil { - return true, err - } - if w.compact { - w.Write([]byte("> ")) - } else { - w.ind-- - w.Write([]byte(">\n")) - } - return true, nil -} - -func (tm *TextMarshaler) writeStruct(w *textWriter, sv reflect.Value) error { - if tm.ExpandAny && isAny(sv) { - if canExpand, err := tm.writeProto3Any(w, sv); canExpand { - return err - } - } - st := sv.Type() - sprops := GetProperties(st) - for i := 0; i < sv.NumField(); i++ { - fv := sv.Field(i) - props := sprops.Prop[i] - name := st.Field(i).Name - - if strings.HasPrefix(name, "XXX_") { - // There are two XXX_ fields: - // XXX_unrecognized []byte - // XXX_extensions map[int32]proto.Extension - // The first is handled here; - // the second is handled at the bottom of this function. - if name == "XXX_unrecognized" && !fv.IsNil() { - if err := writeUnknownStruct(w, fv.Interface().([]byte)); err != nil { - return err - } - } - continue - } - if fv.Kind() == reflect.Ptr && fv.IsNil() { - // Field not filled in. This could be an optional field or - // a required field that wasn't filled in. Either way, there - // isn't anything we can show for it. - continue - } - if fv.Kind() == reflect.Slice && fv.IsNil() { - // Repeated field that is empty, or a bytes field that is unused. - continue - } - - if props.Repeated && fv.Kind() == reflect.Slice { - // Repeated field. - for j := 0; j < fv.Len(); j++ { - if err := writeName(w, props); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte(' '); err != nil { - return err - } - } - v := fv.Index(j) - if v.Kind() == reflect.Ptr && v.IsNil() { - // A nil message in a repeated field is not valid, - // but we can handle that more gracefully than panicking. - if _, err := w.Write([]byte("\n")); err != nil { - return err - } - continue - } - if len(props.Enum) > 0 { - if err := tm.writeEnum(w, v, props); err != nil { - return err - } - } else if err := tm.writeAny(w, v, props); err != nil { - return err - } - if err := w.WriteByte('\n'); err != nil { - return err - } - } - continue - } - if fv.Kind() == reflect.Map { - // Map fields are rendered as a repeated struct with key/value fields. - keys := fv.MapKeys() - sort.Sort(mapKeys(keys)) - for _, key := range keys { - val := fv.MapIndex(key) - if err := writeName(w, props); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte(' '); err != nil { - return err - } - } - // open struct - if err := w.WriteByte('<'); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte('\n'); err != nil { - return err - } - } - w.indent() - // key - if _, err := w.WriteString("key:"); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte(' '); err != nil { - return err - } - } - if err := tm.writeAny(w, key, props.mkeyprop); err != nil { - return err - } - if err := w.WriteByte('\n'); err != nil { - return err - } - // nil values aren't legal, but we can avoid panicking because of them. - if val.Kind() != reflect.Ptr || !val.IsNil() { - // value - if _, err := w.WriteString("value:"); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte(' '); err != nil { - return err - } - } - if err := tm.writeAny(w, val, props.mvalprop); err != nil { - return err - } - if err := w.WriteByte('\n'); err != nil { - return err - } - } - // close struct - w.unindent() - if err := w.WriteByte('>'); err != nil { - return err - } - if err := w.WriteByte('\n'); err != nil { - return err - } - } - continue - } - if props.proto3 && fv.Kind() == reflect.Slice && fv.Len() == 0 { - // empty bytes field - continue - } - if props.proto3 && fv.Kind() != reflect.Ptr && fv.Kind() != reflect.Slice { - // proto3 non-repeated scalar field; skip if zero value - if isProto3Zero(fv) { - continue - } - } - - if fv.Kind() == reflect.Interface { - // Check if it is a oneof. - if st.Field(i).Tag.Get("protobuf_oneof") != "" { - // fv is nil, or holds a pointer to generated struct. - // That generated struct has exactly one field, - // which has a protobuf struct tag. - if fv.IsNil() { - continue - } - inner := fv.Elem().Elem() // interface -> *T -> T - tag := inner.Type().Field(0).Tag.Get("protobuf") - props = new(Properties) // Overwrite the outer props var, but not its pointee. - props.Parse(tag) - // Write the value in the oneof, not the oneof itself. - fv = inner.Field(0) - - // Special case to cope with malformed messages gracefully: - // If the value in the oneof is a nil pointer, don't panic - // in writeAny. - if fv.Kind() == reflect.Ptr && fv.IsNil() { - // Use errors.New so writeAny won't render quotes. - msg := errors.New("/* nil */") - fv = reflect.ValueOf(&msg).Elem() - } - } - } - - if err := writeName(w, props); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte(' '); err != nil { - return err - } - } - if b, ok := fv.Interface().(raw); ok { - if err := writeRaw(w, b.Bytes()); err != nil { - return err - } - continue - } - - if len(props.Enum) > 0 { - if err := tm.writeEnum(w, fv, props); err != nil { - return err - } - } else if err := tm.writeAny(w, fv, props); err != nil { - return err - } - - if err := w.WriteByte('\n'); err != nil { - return err - } - } - - // Extensions (the XXX_extensions field). - pv := sv - if pv.CanAddr() { - pv = sv.Addr() - } else { - pv = reflect.New(sv.Type()) - pv.Elem().Set(sv) - } - if pv.Type().Implements(extensionRangeType) { - if err := tm.writeExtensions(w, pv); err != nil { - return err - } - } - - return nil -} - -// writeRaw writes an uninterpreted raw message. -func writeRaw(w *textWriter, b []byte) error { - if err := w.WriteByte('<'); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte('\n'); err != nil { - return err - } - } - w.indent() - if err := writeUnknownStruct(w, b); err != nil { - return err - } - w.unindent() - if err := w.WriteByte('>'); err != nil { - return err - } - return nil -} - -// writeAny writes an arbitrary field. -func (tm *TextMarshaler) writeAny(w *textWriter, v reflect.Value, props *Properties) error { - v = reflect.Indirect(v) - - if props != nil { - if len(props.CustomType) > 0 { - custom, ok := v.Interface().(Marshaler) - if ok { - data, err := custom.Marshal() - if err != nil { - return err - } - if err := writeString(w, string(data)); err != nil { - return err - } - return nil - } - } else if len(props.CastType) > 0 { - if _, ok := v.Interface().(interface { - String() string - }); ok { - switch v.Kind() { - case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64, - reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64: - _, err := fmt.Fprintf(w, "%d", v.Interface()) - return err - } - } - } else if props.StdTime { - t, ok := v.Interface().(time.Time) - if !ok { - return fmt.Errorf("stdtime is not time.Time, but %T", v.Interface()) - } - tproto, err := timestampProto(t) - if err != nil { - return err - } - propsCopy := *props // Make a copy so that this is goroutine-safe - propsCopy.StdTime = false - err = tm.writeAny(w, reflect.ValueOf(tproto), &propsCopy) - return err - } else if props.StdDuration { - d, ok := v.Interface().(time.Duration) - if !ok { - return fmt.Errorf("stdtime is not time.Duration, but %T", v.Interface()) - } - dproto := durationProto(d) - propsCopy := *props // Make a copy so that this is goroutine-safe - propsCopy.StdDuration = false - err := tm.writeAny(w, reflect.ValueOf(dproto), &propsCopy) - return err - } - } - - // Floats have special cases. - if v.Kind() == reflect.Float32 || v.Kind() == reflect.Float64 { - x := v.Float() - var b []byte - switch { - case math.IsInf(x, 1): - b = posInf - case math.IsInf(x, -1): - b = negInf - case math.IsNaN(x): - b = nan - } - if b != nil { - _, err := w.Write(b) - return err - } - // Other values are handled below. - } - - // We don't attempt to serialise every possible value type; only those - // that can occur in protocol buffers. - switch v.Kind() { - case reflect.Slice: - // Should only be a []byte; repeated fields are handled in writeStruct. - if err := writeString(w, string(v.Bytes())); err != nil { - return err - } - case reflect.String: - if err := writeString(w, v.String()); err != nil { - return err - } - case reflect.Struct: - // Required/optional group/message. - var bra, ket byte = '<', '>' - if props != nil && props.Wire == "group" { - bra, ket = '{', '}' - } - if err := w.WriteByte(bra); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte('\n'); err != nil { - return err - } - } - w.indent() - if etm, ok := v.Interface().(encoding.TextMarshaler); ok { - text, err := etm.MarshalText() - if err != nil { - return err - } - if _, err = w.Write(text); err != nil { - return err - } - } else if err := tm.writeStruct(w, v); err != nil { - return err - } - w.unindent() - if err := w.WriteByte(ket); err != nil { - return err - } - default: - _, err := fmt.Fprint(w, v.Interface()) - return err - } - return nil -} - -// equivalent to C's isprint. -func isprint(c byte) bool { - return c >= 0x20 && c < 0x7f -} - -// writeString writes a string in the protocol buffer text format. -// It is similar to strconv.Quote except we don't use Go escape sequences, -// we treat the string as a byte sequence, and we use octal escapes. -// These differences are to maintain interoperability with the other -// languages' implementations of the text format. -func writeString(w *textWriter, s string) error { - // use WriteByte here to get any needed indent - if err := w.WriteByte('"'); err != nil { - return err - } - // Loop over the bytes, not the runes. - for i := 0; i < len(s); i++ { - var err error - // Divergence from C++: we don't escape apostrophes. - // There's no need to escape them, and the C++ parser - // copes with a naked apostrophe. - switch c := s[i]; c { - case '\n': - _, err = w.w.Write(backslashN) - case '\r': - _, err = w.w.Write(backslashR) - case '\t': - _, err = w.w.Write(backslashT) - case '"': - _, err = w.w.Write(backslashDQ) - case '\\': - _, err = w.w.Write(backslashBS) - default: - if isprint(c) { - err = w.w.WriteByte(c) - } else { - _, err = fmt.Fprintf(w.w, "\\%03o", c) - } - } - if err != nil { - return err - } - } - return w.WriteByte('"') -} - -func writeUnknownStruct(w *textWriter, data []byte) (err error) { - if !w.compact { - if _, err := fmt.Fprintf(w, "/* %d unknown bytes */\n", len(data)); err != nil { - return err - } - } - b := NewBuffer(data) - for b.index < len(b.buf) { - x, err := b.DecodeVarint() - if err != nil { - _, ferr := fmt.Fprintf(w, "/* %v */\n", err) - return ferr - } - wire, tag := x&7, x>>3 - if wire == WireEndGroup { - w.unindent() - if _, werr := w.Write(endBraceNewline); werr != nil { - return werr - } - continue - } - if _, ferr := fmt.Fprint(w, tag); ferr != nil { - return ferr - } - if wire != WireStartGroup { - if err = w.WriteByte(':'); err != nil { - return err - } - } - if !w.compact || wire == WireStartGroup { - if err = w.WriteByte(' '); err != nil { - return err - } - } - switch wire { - case WireBytes: - buf, e := b.DecodeRawBytes(false) - if e == nil { - _, err = fmt.Fprintf(w, "%q", buf) - } else { - _, err = fmt.Fprintf(w, "/* %v */", e) - } - case WireFixed32: - x, err = b.DecodeFixed32() - err = writeUnknownInt(w, x, err) - case WireFixed64: - x, err = b.DecodeFixed64() - err = writeUnknownInt(w, x, err) - case WireStartGroup: - err = w.WriteByte('{') - w.indent() - case WireVarint: - x, err = b.DecodeVarint() - err = writeUnknownInt(w, x, err) - default: - _, err = fmt.Fprintf(w, "/* unknown wire type %d */", wire) - } - if err != nil { - return err - } - if err := w.WriteByte('\n'); err != nil { - return err - } - } - return nil -} - -func writeUnknownInt(w *textWriter, x uint64, err error) error { - if err == nil { - _, err = fmt.Fprint(w, x) - } else { - _, err = fmt.Fprintf(w, "/* %v */", err) - } - return err -} - -type int32Slice []int32 - -func (s int32Slice) Len() int { return len(s) } -func (s int32Slice) Less(i, j int) bool { return s[i] < s[j] } -func (s int32Slice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } - -// writeExtensions writes all the extensions in pv. -// pv is assumed to be a pointer to a protocol message struct that is extendable. -func (tm *TextMarshaler) writeExtensions(w *textWriter, pv reflect.Value) error { - emap := extensionMaps[pv.Type().Elem()] - e := pv.Interface().(Message) - - var m map[int32]Extension - var mu sync.Locker - if em, ok := e.(extensionsBytes); ok { - eb := em.GetExtensions() - var err error - m, err = BytesToExtensionsMap(*eb) - if err != nil { - return err - } - mu = notLocker{} - } else if _, ok := e.(extendableProto); ok { - ep, _ := extendable(e) - m, mu = ep.extensionsRead() - if m == nil { - return nil - } - } - - // Order the extensions by ID. - // This isn't strictly necessary, but it will give us - // canonical output, which will also make testing easier. - - mu.Lock() - ids := make([]int32, 0, len(m)) - for id := range m { - ids = append(ids, id) - } - sort.Sort(int32Slice(ids)) - mu.Unlock() - - for _, extNum := range ids { - ext := m[extNum] - var desc *ExtensionDesc - if emap != nil { - desc = emap[extNum] - } - if desc == nil { - // Unknown extension. - if err := writeUnknownStruct(w, ext.enc); err != nil { - return err - } - continue - } - - pb, err := GetExtension(e, desc) - if err != nil { - return fmt.Errorf("failed getting extension: %v", err) - } - - // Repeated extensions will appear as a slice. - if !desc.repeated() { - if err := tm.writeExtension(w, desc.Name, pb); err != nil { - return err - } - } else { - v := reflect.ValueOf(pb) - for i := 0; i < v.Len(); i++ { - if err := tm.writeExtension(w, desc.Name, v.Index(i).Interface()); err != nil { - return err - } - } - } - } - return nil -} - -func (tm *TextMarshaler) writeExtension(w *textWriter, name string, pb interface{}) error { - if _, err := fmt.Fprintf(w, "[%s]:", name); err != nil { - return err - } - if !w.compact { - if err := w.WriteByte(' '); err != nil { - return err - } - } - if err := tm.writeAny(w, reflect.ValueOf(pb), nil); err != nil { - return err - } - if err := w.WriteByte('\n'); err != nil { - return err - } - return nil -} - -func (w *textWriter) writeIndent() { - if !w.complete { - return - } - remain := w.ind * 2 - for remain > 0 { - n := remain - if n > len(spaces) { - n = len(spaces) - } - w.w.Write(spaces[:n]) - remain -= n - } - w.complete = false -} - -// TextMarshaler is a configurable text format marshaler. -type TextMarshaler struct { - Compact bool // use compact text format (one line). - ExpandAny bool // expand google.protobuf.Any messages of known types -} - -// Marshal writes a given protocol buffer in text format. -// The only errors returned are from w. -func (tm *TextMarshaler) Marshal(w io.Writer, pb Message) error { - val := reflect.ValueOf(pb) - if pb == nil || val.IsNil() { - w.Write([]byte("")) - return nil - } - var bw *bufio.Writer - ww, ok := w.(writer) - if !ok { - bw = bufio.NewWriter(w) - ww = bw - } - aw := &textWriter{ - w: ww, - complete: true, - compact: tm.Compact, - } - - if etm, ok := pb.(encoding.TextMarshaler); ok { - text, err := etm.MarshalText() - if err != nil { - return err - } - if _, err = aw.Write(text); err != nil { - return err - } - if bw != nil { - return bw.Flush() - } - return nil - } - // Dereference the received pointer so we don't have outer < and >. - v := reflect.Indirect(val) - if err := tm.writeStruct(aw, v); err != nil { - return err - } - if bw != nil { - return bw.Flush() - } - return nil -} - -// Text is the same as Marshal, but returns the string directly. -func (tm *TextMarshaler) Text(pb Message) string { - var buf bytes.Buffer - tm.Marshal(&buf, pb) - return buf.String() -} - -var ( - defaultTextMarshaler = TextMarshaler{} - compactTextMarshaler = TextMarshaler{Compact: true} -) - -// TODO: consider removing some of the Marshal functions below. - -// MarshalText writes a given protocol buffer in text format. -// The only errors returned are from w. -func MarshalText(w io.Writer, pb Message) error { return defaultTextMarshaler.Marshal(w, pb) } - -// MarshalTextString is the same as MarshalText, but returns the string directly. -func MarshalTextString(pb Message) string { return defaultTextMarshaler.Text(pb) } - -// CompactText writes a given protocol buffer in compact text format (one line). -func CompactText(w io.Writer, pb Message) error { return compactTextMarshaler.Marshal(w, pb) } - -// CompactTextString is the same as CompactText, but returns the string directly. -func CompactTextString(pb Message) string { return compactTextMarshaler.Text(pb) } diff --git a/vendor/github.com/gogo/protobuf/proto/text_gogo.go b/vendor/github.com/gogo/protobuf/proto/text_gogo.go deleted file mode 100644 index 1d6c6aa0e41bb..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/text_gogo.go +++ /dev/null @@ -1,57 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "fmt" - "reflect" -) - -func (tm *TextMarshaler) writeEnum(w *textWriter, v reflect.Value, props *Properties) error { - m, ok := enumStringMaps[props.Enum] - if !ok { - if err := tm.writeAny(w, v, props); err != nil { - return err - } - } - key := int32(0) - if v.Kind() == reflect.Ptr { - key = int32(v.Elem().Int()) - } else { - key = int32(v.Int()) - } - s, ok := m[key] - if !ok { - if err := tm.writeAny(w, v, props); err != nil { - return err - } - } - _, err := fmt.Fprint(w, s) - return err -} diff --git a/vendor/github.com/gogo/protobuf/proto/text_parser.go b/vendor/github.com/gogo/protobuf/proto/text_parser.go deleted file mode 100644 index f1276729a3543..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/text_parser.go +++ /dev/null @@ -1,1013 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2013, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2010 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -// Functions for parsing the Text protocol buffer format. -// TODO: message sets. - -import ( - "encoding" - "errors" - "fmt" - "reflect" - "strconv" - "strings" - "time" - "unicode/utf8" -) - -// Error string emitted when deserializing Any and fields are already set -const anyRepeatedlyUnpacked = "Any message unpacked multiple times, or %q already set" - -type ParseError struct { - Message string - Line int // 1-based line number - Offset int // 0-based byte offset from start of input -} - -func (p *ParseError) Error() string { - if p.Line == 1 { - // show offset only for first line - return fmt.Sprintf("line 1.%d: %v", p.Offset, p.Message) - } - return fmt.Sprintf("line %d: %v", p.Line, p.Message) -} - -type token struct { - value string - err *ParseError - line int // line number - offset int // byte number from start of input, not start of line - unquoted string // the unquoted version of value, if it was a quoted string -} - -func (t *token) String() string { - if t.err == nil { - return fmt.Sprintf("%q (line=%d, offset=%d)", t.value, t.line, t.offset) - } - return fmt.Sprintf("parse error: %v", t.err) -} - -type textParser struct { - s string // remaining input - done bool // whether the parsing is finished (success or error) - backed bool // whether back() was called - offset, line int - cur token -} - -func newTextParser(s string) *textParser { - p := new(textParser) - p.s = s - p.line = 1 - p.cur.line = 1 - return p -} - -func (p *textParser) errorf(format string, a ...interface{}) *ParseError { - pe := &ParseError{fmt.Sprintf(format, a...), p.cur.line, p.cur.offset} - p.cur.err = pe - p.done = true - return pe -} - -// Numbers and identifiers are matched by [-+._A-Za-z0-9] -func isIdentOrNumberChar(c byte) bool { - switch { - case 'A' <= c && c <= 'Z', 'a' <= c && c <= 'z': - return true - case '0' <= c && c <= '9': - return true - } - switch c { - case '-', '+', '.', '_': - return true - } - return false -} - -func isWhitespace(c byte) bool { - switch c { - case ' ', '\t', '\n', '\r': - return true - } - return false -} - -func isQuote(c byte) bool { - switch c { - case '"', '\'': - return true - } - return false -} - -func (p *textParser) skipWhitespace() { - i := 0 - for i < len(p.s) && (isWhitespace(p.s[i]) || p.s[i] == '#') { - if p.s[i] == '#' { - // comment; skip to end of line or input - for i < len(p.s) && p.s[i] != '\n' { - i++ - } - if i == len(p.s) { - break - } - } - if p.s[i] == '\n' { - p.line++ - } - i++ - } - p.offset += i - p.s = p.s[i:len(p.s)] - if len(p.s) == 0 { - p.done = true - } -} - -func (p *textParser) advance() { - // Skip whitespace - p.skipWhitespace() - if p.done { - return - } - - // Start of non-whitespace - p.cur.err = nil - p.cur.offset, p.cur.line = p.offset, p.line - p.cur.unquoted = "" - switch p.s[0] { - case '<', '>', '{', '}', ':', '[', ']', ';', ',', '/': - // Single symbol - p.cur.value, p.s = p.s[0:1], p.s[1:len(p.s)] - case '"', '\'': - // Quoted string - i := 1 - for i < len(p.s) && p.s[i] != p.s[0] && p.s[i] != '\n' { - if p.s[i] == '\\' && i+1 < len(p.s) { - // skip escaped char - i++ - } - i++ - } - if i >= len(p.s) || p.s[i] != p.s[0] { - p.errorf("unmatched quote") - return - } - unq, err := unquoteC(p.s[1:i], rune(p.s[0])) - if err != nil { - p.errorf("invalid quoted string %s: %v", p.s[0:i+1], err) - return - } - p.cur.value, p.s = p.s[0:i+1], p.s[i+1:len(p.s)] - p.cur.unquoted = unq - default: - i := 0 - for i < len(p.s) && isIdentOrNumberChar(p.s[i]) { - i++ - } - if i == 0 { - p.errorf("unexpected byte %#x", p.s[0]) - return - } - p.cur.value, p.s = p.s[0:i], p.s[i:len(p.s)] - } - p.offset += len(p.cur.value) -} - -var ( - errBadUTF8 = errors.New("proto: bad UTF-8") - errBadHex = errors.New("proto: bad hexadecimal") -) - -func unquoteC(s string, quote rune) (string, error) { - // This is based on C++'s tokenizer.cc. - // Despite its name, this is *not* parsing C syntax. - // For instance, "\0" is an invalid quoted string. - - // Avoid allocation in trivial cases. - simple := true - for _, r := range s { - if r == '\\' || r == quote { - simple = false - break - } - } - if simple { - return s, nil - } - - buf := make([]byte, 0, 3*len(s)/2) - for len(s) > 0 { - r, n := utf8.DecodeRuneInString(s) - if r == utf8.RuneError && n == 1 { - return "", errBadUTF8 - } - s = s[n:] - if r != '\\' { - if r < utf8.RuneSelf { - buf = append(buf, byte(r)) - } else { - buf = append(buf, string(r)...) - } - continue - } - - ch, tail, err := unescape(s) - if err != nil { - return "", err - } - buf = append(buf, ch...) - s = tail - } - return string(buf), nil -} - -func unescape(s string) (ch string, tail string, err error) { - r, n := utf8.DecodeRuneInString(s) - if r == utf8.RuneError && n == 1 { - return "", "", errBadUTF8 - } - s = s[n:] - switch r { - case 'a': - return "\a", s, nil - case 'b': - return "\b", s, nil - case 'f': - return "\f", s, nil - case 'n': - return "\n", s, nil - case 'r': - return "\r", s, nil - case 't': - return "\t", s, nil - case 'v': - return "\v", s, nil - case '?': - return "?", s, nil // trigraph workaround - case '\'', '"', '\\': - return string(r), s, nil - case '0', '1', '2', '3', '4', '5', '6', '7', 'x', 'X': - if len(s) < 2 { - return "", "", fmt.Errorf(`\%c requires 2 following digits`, r) - } - base := 8 - ss := s[:2] - s = s[2:] - if r == 'x' || r == 'X' { - base = 16 - } else { - ss = string(r) + ss - } - i, err := strconv.ParseUint(ss, base, 8) - if err != nil { - return "", "", err - } - return string([]byte{byte(i)}), s, nil - case 'u', 'U': - n := 4 - if r == 'U' { - n = 8 - } - if len(s) < n { - return "", "", fmt.Errorf(`\%c requires %d digits`, r, n) - } - - bs := make([]byte, n/2) - for i := 0; i < n; i += 2 { - a, ok1 := unhex(s[i]) - b, ok2 := unhex(s[i+1]) - if !ok1 || !ok2 { - return "", "", errBadHex - } - bs[i/2] = a<<4 | b - } - s = s[n:] - return string(bs), s, nil - } - return "", "", fmt.Errorf(`unknown escape \%c`, r) -} - -// Adapted from src/pkg/strconv/quote.go. -func unhex(b byte) (v byte, ok bool) { - switch { - case '0' <= b && b <= '9': - return b - '0', true - case 'a' <= b && b <= 'f': - return b - 'a' + 10, true - case 'A' <= b && b <= 'F': - return b - 'A' + 10, true - } - return 0, false -} - -// Back off the parser by one token. Can only be done between calls to next(). -// It makes the next advance() a no-op. -func (p *textParser) back() { p.backed = true } - -// Advances the parser and returns the new current token. -func (p *textParser) next() *token { - if p.backed || p.done { - p.backed = false - return &p.cur - } - p.advance() - if p.done { - p.cur.value = "" - } else if len(p.cur.value) > 0 && isQuote(p.cur.value[0]) { - // Look for multiple quoted strings separated by whitespace, - // and concatenate them. - cat := p.cur - for { - p.skipWhitespace() - if p.done || !isQuote(p.s[0]) { - break - } - p.advance() - if p.cur.err != nil { - return &p.cur - } - cat.value += " " + p.cur.value - cat.unquoted += p.cur.unquoted - } - p.done = false // parser may have seen EOF, but we want to return cat - p.cur = cat - } - return &p.cur -} - -func (p *textParser) consumeToken(s string) error { - tok := p.next() - if tok.err != nil { - return tok.err - } - if tok.value != s { - p.back() - return p.errorf("expected %q, found %q", s, tok.value) - } - return nil -} - -// Return a RequiredNotSetError indicating which required field was not set. -func (p *textParser) missingRequiredFieldError(sv reflect.Value) *RequiredNotSetError { - st := sv.Type() - sprops := GetProperties(st) - for i := 0; i < st.NumField(); i++ { - if !isNil(sv.Field(i)) { - continue - } - - props := sprops.Prop[i] - if props.Required { - return &RequiredNotSetError{fmt.Sprintf("%v.%v", st, props.OrigName)} - } - } - return &RequiredNotSetError{fmt.Sprintf("%v.", st)} // should not happen -} - -// Returns the index in the struct for the named field, as well as the parsed tag properties. -func structFieldByName(sprops *StructProperties, name string) (int, *Properties, bool) { - i, ok := sprops.decoderOrigNames[name] - if ok { - return i, sprops.Prop[i], true - } - return -1, nil, false -} - -// Consume a ':' from the input stream (if the next token is a colon), -// returning an error if a colon is needed but not present. -func (p *textParser) checkForColon(props *Properties, typ reflect.Type) *ParseError { - tok := p.next() - if tok.err != nil { - return tok.err - } - if tok.value != ":" { - // Colon is optional when the field is a group or message. - needColon := true - switch props.Wire { - case "group": - needColon = false - case "bytes": - // A "bytes" field is either a message, a string, or a repeated field; - // those three become *T, *string and []T respectively, so we can check for - // this field being a pointer to a non-string. - if typ.Kind() == reflect.Ptr { - // *T or *string - if typ.Elem().Kind() == reflect.String { - break - } - } else if typ.Kind() == reflect.Slice { - // []T or []*T - if typ.Elem().Kind() != reflect.Ptr { - break - } - } else if typ.Kind() == reflect.String { - // The proto3 exception is for a string field, - // which requires a colon. - break - } - needColon = false - } - if needColon { - return p.errorf("expected ':', found %q", tok.value) - } - p.back() - } - return nil -} - -func (p *textParser) readStruct(sv reflect.Value, terminator string) error { - st := sv.Type() - sprops := GetProperties(st) - reqCount := sprops.reqCount - var reqFieldErr error - fieldSet := make(map[string]bool) - // A struct is a sequence of "name: value", terminated by one of - // '>' or '}', or the end of the input. A name may also be - // "[extension]" or "[type/url]". - // - // The whole struct can also be an expanded Any message, like: - // [type/url] < ... struct contents ... > - for { - tok := p.next() - if tok.err != nil { - return tok.err - } - if tok.value == terminator { - break - } - if tok.value == "[" { - // Looks like an extension or an Any. - // - // TODO: Check whether we need to handle - // namespace rooted names (e.g. ".something.Foo"). - extName, err := p.consumeExtName() - if err != nil { - return err - } - - if s := strings.LastIndex(extName, "/"); s >= 0 { - // If it contains a slash, it's an Any type URL. - messageName := extName[s+1:] - mt := MessageType(messageName) - if mt == nil { - return p.errorf("unrecognized message %q in google.protobuf.Any", messageName) - } - tok = p.next() - if tok.err != nil { - return tok.err - } - // consume an optional colon - if tok.value == ":" { - tok = p.next() - if tok.err != nil { - return tok.err - } - } - var terminator string - switch tok.value { - case "<": - terminator = ">" - case "{": - terminator = "}" - default: - return p.errorf("expected '{' or '<', found %q", tok.value) - } - v := reflect.New(mt.Elem()) - if pe := p.readStruct(v.Elem(), terminator); pe != nil { - return pe - } - b, err := Marshal(v.Interface().(Message)) - if err != nil { - return p.errorf("failed to marshal message of type %q: %v", messageName, err) - } - if fieldSet["type_url"] { - return p.errorf(anyRepeatedlyUnpacked, "type_url") - } - if fieldSet["value"] { - return p.errorf(anyRepeatedlyUnpacked, "value") - } - sv.FieldByName("TypeUrl").SetString(extName) - sv.FieldByName("Value").SetBytes(b) - fieldSet["type_url"] = true - fieldSet["value"] = true - continue - } - - var desc *ExtensionDesc - // This could be faster, but it's functional. - // TODO: Do something smarter than a linear scan. - for _, d := range RegisteredExtensions(reflect.New(st).Interface().(Message)) { - if d.Name == extName { - desc = d - break - } - } - if desc == nil { - return p.errorf("unrecognized extension %q", extName) - } - - props := &Properties{} - props.Parse(desc.Tag) - - typ := reflect.TypeOf(desc.ExtensionType) - if err := p.checkForColon(props, typ); err != nil { - return err - } - - rep := desc.repeated() - - // Read the extension structure, and set it in - // the value we're constructing. - var ext reflect.Value - if !rep { - ext = reflect.New(typ).Elem() - } else { - ext = reflect.New(typ.Elem()).Elem() - } - if err := p.readAny(ext, props); err != nil { - if _, ok := err.(*RequiredNotSetError); !ok { - return err - } - reqFieldErr = err - } - ep := sv.Addr().Interface().(Message) - if !rep { - SetExtension(ep, desc, ext.Interface()) - } else { - old, err := GetExtension(ep, desc) - var sl reflect.Value - if err == nil { - sl = reflect.ValueOf(old) // existing slice - } else { - sl = reflect.MakeSlice(typ, 0, 1) - } - sl = reflect.Append(sl, ext) - SetExtension(ep, desc, sl.Interface()) - } - if err := p.consumeOptionalSeparator(); err != nil { - return err - } - continue - } - - // This is a normal, non-extension field. - name := tok.value - var dst reflect.Value - fi, props, ok := structFieldByName(sprops, name) - if ok { - dst = sv.Field(fi) - } else if oop, ok := sprops.OneofTypes[name]; ok { - // It is a oneof. - props = oop.Prop - nv := reflect.New(oop.Type.Elem()) - dst = nv.Elem().Field(0) - field := sv.Field(oop.Field) - if !field.IsNil() { - return p.errorf("field '%s' would overwrite already parsed oneof '%s'", name, sv.Type().Field(oop.Field).Name) - } - field.Set(nv) - } - if !dst.IsValid() { - return p.errorf("unknown field name %q in %v", name, st) - } - - if dst.Kind() == reflect.Map { - // Consume any colon. - if err := p.checkForColon(props, dst.Type()); err != nil { - return err - } - - // Construct the map if it doesn't already exist. - if dst.IsNil() { - dst.Set(reflect.MakeMap(dst.Type())) - } - key := reflect.New(dst.Type().Key()).Elem() - val := reflect.New(dst.Type().Elem()).Elem() - - // The map entry should be this sequence of tokens: - // < key : KEY value : VALUE > - // However, implementations may omit key or value, and technically - // we should support them in any order. See b/28924776 for a time - // this went wrong. - - tok := p.next() - var terminator string - switch tok.value { - case "<": - terminator = ">" - case "{": - terminator = "}" - default: - return p.errorf("expected '{' or '<', found %q", tok.value) - } - for { - tok := p.next() - if tok.err != nil { - return tok.err - } - if tok.value == terminator { - break - } - switch tok.value { - case "key": - if err := p.consumeToken(":"); err != nil { - return err - } - if err := p.readAny(key, props.mkeyprop); err != nil { - return err - } - if err := p.consumeOptionalSeparator(); err != nil { - return err - } - case "value": - if err := p.checkForColon(props.mvalprop, dst.Type().Elem()); err != nil { - return err - } - if err := p.readAny(val, props.mvalprop); err != nil { - return err - } - if err := p.consumeOptionalSeparator(); err != nil { - return err - } - default: - p.back() - return p.errorf(`expected "key", "value", or %q, found %q`, terminator, tok.value) - } - } - - dst.SetMapIndex(key, val) - continue - } - - // Check that it's not already set if it's not a repeated field. - if !props.Repeated && fieldSet[name] { - return p.errorf("non-repeated field %q was repeated", name) - } - - if err := p.checkForColon(props, dst.Type()); err != nil { - return err - } - - // Parse into the field. - fieldSet[name] = true - if err := p.readAny(dst, props); err != nil { - if _, ok := err.(*RequiredNotSetError); !ok { - return err - } - reqFieldErr = err - } - if props.Required { - reqCount-- - } - - if err := p.consumeOptionalSeparator(); err != nil { - return err - } - - } - - if reqCount > 0 { - return p.missingRequiredFieldError(sv) - } - return reqFieldErr -} - -// consumeExtName consumes extension name or expanded Any type URL and the -// following ']'. It returns the name or URL consumed. -func (p *textParser) consumeExtName() (string, error) { - tok := p.next() - if tok.err != nil { - return "", tok.err - } - - // If extension name or type url is quoted, it's a single token. - if len(tok.value) > 2 && isQuote(tok.value[0]) && tok.value[len(tok.value)-1] == tok.value[0] { - name, err := unquoteC(tok.value[1:len(tok.value)-1], rune(tok.value[0])) - if err != nil { - return "", err - } - return name, p.consumeToken("]") - } - - // Consume everything up to "]" - var parts []string - for tok.value != "]" { - parts = append(parts, tok.value) - tok = p.next() - if tok.err != nil { - return "", p.errorf("unrecognized type_url or extension name: %s", tok.err) - } - } - return strings.Join(parts, ""), nil -} - -// consumeOptionalSeparator consumes an optional semicolon or comma. -// It is used in readStruct to provide backward compatibility. -func (p *textParser) consumeOptionalSeparator() error { - tok := p.next() - if tok.err != nil { - return tok.err - } - if tok.value != ";" && tok.value != "," { - p.back() - } - return nil -} - -func (p *textParser) readAny(v reflect.Value, props *Properties) error { - tok := p.next() - if tok.err != nil { - return tok.err - } - if tok.value == "" { - return p.errorf("unexpected EOF") - } - if len(props.CustomType) > 0 { - if props.Repeated { - t := reflect.TypeOf(v.Interface()) - if t.Kind() == reflect.Slice { - tc := reflect.TypeOf(new(Marshaler)) - ok := t.Elem().Implements(tc.Elem()) - if ok { - fv := v - flen := fv.Len() - if flen == fv.Cap() { - nav := reflect.MakeSlice(v.Type(), flen, 2*flen+1) - reflect.Copy(nav, fv) - fv.Set(nav) - } - fv.SetLen(flen + 1) - - // Read one. - p.back() - return p.readAny(fv.Index(flen), props) - } - } - } - if reflect.TypeOf(v.Interface()).Kind() == reflect.Ptr { - custom := reflect.New(props.ctype.Elem()).Interface().(Unmarshaler) - err := custom.Unmarshal([]byte(tok.unquoted)) - if err != nil { - return p.errorf("%v %v: %v", err, v.Type(), tok.value) - } - v.Set(reflect.ValueOf(custom)) - } else { - custom := reflect.New(reflect.TypeOf(v.Interface())).Interface().(Unmarshaler) - err := custom.Unmarshal([]byte(tok.unquoted)) - if err != nil { - return p.errorf("%v %v: %v", err, v.Type(), tok.value) - } - v.Set(reflect.Indirect(reflect.ValueOf(custom))) - } - return nil - } - if props.StdTime { - fv := v - p.back() - props.StdTime = false - tproto := ×tamp{} - err := p.readAny(reflect.ValueOf(tproto).Elem(), props) - props.StdTime = true - if err != nil { - return err - } - tim, err := timestampFromProto(tproto) - if err != nil { - return err - } - if props.Repeated { - t := reflect.TypeOf(v.Interface()) - if t.Kind() == reflect.Slice { - if t.Elem().Kind() == reflect.Ptr { - ts := fv.Interface().([]*time.Time) - ts = append(ts, &tim) - fv.Set(reflect.ValueOf(ts)) - return nil - } else { - ts := fv.Interface().([]time.Time) - ts = append(ts, tim) - fv.Set(reflect.ValueOf(ts)) - return nil - } - } - } - if reflect.TypeOf(v.Interface()).Kind() == reflect.Ptr { - v.Set(reflect.ValueOf(&tim)) - } else { - v.Set(reflect.Indirect(reflect.ValueOf(&tim))) - } - return nil - } - if props.StdDuration { - fv := v - p.back() - props.StdDuration = false - dproto := &duration{} - err := p.readAny(reflect.ValueOf(dproto).Elem(), props) - props.StdDuration = true - if err != nil { - return err - } - dur, err := durationFromProto(dproto) - if err != nil { - return err - } - if props.Repeated { - t := reflect.TypeOf(v.Interface()) - if t.Kind() == reflect.Slice { - if t.Elem().Kind() == reflect.Ptr { - ds := fv.Interface().([]*time.Duration) - ds = append(ds, &dur) - fv.Set(reflect.ValueOf(ds)) - return nil - } else { - ds := fv.Interface().([]time.Duration) - ds = append(ds, dur) - fv.Set(reflect.ValueOf(ds)) - return nil - } - } - } - if reflect.TypeOf(v.Interface()).Kind() == reflect.Ptr { - v.Set(reflect.ValueOf(&dur)) - } else { - v.Set(reflect.Indirect(reflect.ValueOf(&dur))) - } - return nil - } - switch fv := v; fv.Kind() { - case reflect.Slice: - at := v.Type() - if at.Elem().Kind() == reflect.Uint8 { - // Special case for []byte - if tok.value[0] != '"' && tok.value[0] != '\'' { - // Deliberately written out here, as the error after - // this switch statement would write "invalid []byte: ...", - // which is not as user-friendly. - return p.errorf("invalid string: %v", tok.value) - } - bytes := []byte(tok.unquoted) - fv.Set(reflect.ValueOf(bytes)) - return nil - } - // Repeated field. - if tok.value == "[" { - // Repeated field with list notation, like [1,2,3]. - for { - fv.Set(reflect.Append(fv, reflect.New(at.Elem()).Elem())) - err := p.readAny(fv.Index(fv.Len()-1), props) - if err != nil { - return err - } - ntok := p.next() - if ntok.err != nil { - return ntok.err - } - if ntok.value == "]" { - break - } - if ntok.value != "," { - return p.errorf("Expected ']' or ',' found %q", ntok.value) - } - } - return nil - } - // One value of the repeated field. - p.back() - fv.Set(reflect.Append(fv, reflect.New(at.Elem()).Elem())) - return p.readAny(fv.Index(fv.Len()-1), props) - case reflect.Bool: - // true/1/t/True or false/f/0/False. - switch tok.value { - case "true", "1", "t", "True": - fv.SetBool(true) - return nil - case "false", "0", "f", "False": - fv.SetBool(false) - return nil - } - case reflect.Float32, reflect.Float64: - v := tok.value - // Ignore 'f' for compatibility with output generated by C++, but don't - // remove 'f' when the value is "-inf" or "inf". - if strings.HasSuffix(v, "f") && tok.value != "-inf" && tok.value != "inf" { - v = v[:len(v)-1] - } - if f, err := strconv.ParseFloat(v, fv.Type().Bits()); err == nil { - fv.SetFloat(f) - return nil - } - case reflect.Int32: - if x, err := strconv.ParseInt(tok.value, 0, 32); err == nil { - fv.SetInt(x) - return nil - } - - if len(props.Enum) == 0 { - break - } - m, ok := enumValueMaps[props.Enum] - if !ok { - break - } - x, ok := m[tok.value] - if !ok { - break - } - fv.SetInt(int64(x)) - return nil - case reflect.Int64: - if x, err := strconv.ParseInt(tok.value, 0, 64); err == nil { - fv.SetInt(x) - return nil - } - - case reflect.Ptr: - // A basic field (indirected through pointer), or a repeated message/group - p.back() - fv.Set(reflect.New(fv.Type().Elem())) - return p.readAny(fv.Elem(), props) - case reflect.String: - if tok.value[0] == '"' || tok.value[0] == '\'' { - fv.SetString(tok.unquoted) - return nil - } - case reflect.Struct: - var terminator string - switch tok.value { - case "{": - terminator = "}" - case "<": - terminator = ">" - default: - return p.errorf("expected '{' or '<', found %q", tok.value) - } - // TODO: Handle nested messages which implement encoding.TextUnmarshaler. - return p.readStruct(fv, terminator) - case reflect.Uint32: - if x, err := strconv.ParseUint(tok.value, 0, 32); err == nil { - fv.SetUint(x) - return nil - } - case reflect.Uint64: - if x, err := strconv.ParseUint(tok.value, 0, 64); err == nil { - fv.SetUint(x) - return nil - } - } - return p.errorf("invalid %v: %v", v.Type(), tok.value) -} - -// UnmarshalText reads a protocol buffer in Text format. UnmarshalText resets pb -// before starting to unmarshal, so any existing data in pb is always removed. -// If a required field is not set and no other error occurs, -// UnmarshalText returns *RequiredNotSetError. -func UnmarshalText(s string, pb Message) error { - if um, ok := pb.(encoding.TextUnmarshaler); ok { - err := um.UnmarshalText([]byte(s)) - return err - } - pb.Reset() - v := reflect.ValueOf(pb) - if pe := newTextParser(s).readStruct(v.Elem(), ""); pe != nil { - return pe - } - return nil -} diff --git a/vendor/github.com/gogo/protobuf/proto/timestamp.go b/vendor/github.com/gogo/protobuf/proto/timestamp.go deleted file mode 100644 index 9324f6542bcf7..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/timestamp.go +++ /dev/null @@ -1,113 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2016 The Go Authors. All rights reserved. -// https://github.com/golang/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -// This file implements operations on google.protobuf.Timestamp. - -import ( - "errors" - "fmt" - "time" -) - -const ( - // Seconds field of the earliest valid Timestamp. - // This is time.Date(1, 1, 1, 0, 0, 0, 0, time.UTC).Unix(). - minValidSeconds = -62135596800 - // Seconds field just after the latest valid Timestamp. - // This is time.Date(10000, 1, 1, 0, 0, 0, 0, time.UTC).Unix(). - maxValidSeconds = 253402300800 -) - -// validateTimestamp determines whether a Timestamp is valid. -// A valid timestamp represents a time in the range -// [0001-01-01, 10000-01-01) and has a Nanos field -// in the range [0, 1e9). -// -// If the Timestamp is valid, validateTimestamp returns nil. -// Otherwise, it returns an error that describes -// the problem. -// -// Every valid Timestamp can be represented by a time.Time, but the converse is not true. -func validateTimestamp(ts *timestamp) error { - if ts == nil { - return errors.New("timestamp: nil Timestamp") - } - if ts.Seconds < minValidSeconds { - return fmt.Errorf("timestamp: %#v before 0001-01-01", ts) - } - if ts.Seconds >= maxValidSeconds { - return fmt.Errorf("timestamp: %#v after 10000-01-01", ts) - } - if ts.Nanos < 0 || ts.Nanos >= 1e9 { - return fmt.Errorf("timestamp: %#v: nanos not in range [0, 1e9)", ts) - } - return nil -} - -// TimestampFromProto converts a google.protobuf.Timestamp proto to a time.Time. -// It returns an error if the argument is invalid. -// -// Unlike most Go functions, if Timestamp returns an error, the first return value -// is not the zero time.Time. Instead, it is the value obtained from the -// time.Unix function when passed the contents of the Timestamp, in the UTC -// locale. This may or may not be a meaningful time; many invalid Timestamps -// do map to valid time.Times. -// -// A nil Timestamp returns an error. The first return value in that case is -// undefined. -func timestampFromProto(ts *timestamp) (time.Time, error) { - // Don't return the zero value on error, because corresponds to a valid - // timestamp. Instead return whatever time.Unix gives us. - var t time.Time - if ts == nil { - t = time.Unix(0, 0).UTC() // treat nil like the empty Timestamp - } else { - t = time.Unix(ts.Seconds, int64(ts.Nanos)).UTC() - } - return t, validateTimestamp(ts) -} - -// TimestampProto converts the time.Time to a google.protobuf.Timestamp proto. -// It returns an error if the resulting Timestamp is invalid. -func timestampProto(t time.Time) (*timestamp, error) { - seconds := t.Unix() - nanos := int32(t.Sub(time.Unix(seconds, 0))) - ts := ×tamp{ - Seconds: seconds, - Nanos: nanos, - } - if err := validateTimestamp(ts); err != nil { - return nil, err - } - return ts, nil -} diff --git a/vendor/github.com/gogo/protobuf/proto/timestamp_gogo.go b/vendor/github.com/gogo/protobuf/proto/timestamp_gogo.go deleted file mode 100644 index d427647436066..0000000000000 --- a/vendor/github.com/gogo/protobuf/proto/timestamp_gogo.go +++ /dev/null @@ -1,229 +0,0 @@ -// Protocol Buffers for Go with Gadgets -// -// Copyright (c) 2016, The GoGo Authors. All rights reserved. -// http://github.com/gogo/protobuf -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -package proto - -import ( - "reflect" - "time" -) - -var timeType = reflect.TypeOf((*time.Time)(nil)).Elem() - -type timestamp struct { - Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"` - Nanos int32 `protobuf:"varint,2,opt,name=nanos,proto3" json:"nanos,omitempty"` -} - -func (m *timestamp) Reset() { *m = timestamp{} } -func (*timestamp) ProtoMessage() {} -func (*timestamp) String() string { return "timestamp" } - -func init() { - RegisterType((*timestamp)(nil), "gogo.protobuf.proto.timestamp") -} - -func (o *Buffer) decTimestamp() (time.Time, error) { - b, err := o.DecodeRawBytes(true) - if err != nil { - return time.Time{}, err - } - tproto := ×tamp{} - if err := Unmarshal(b, tproto); err != nil { - return time.Time{}, err - } - return timestampFromProto(tproto) -} - -func (o *Buffer) dec_time(p *Properties, base structPointer) error { - t, err := o.decTimestamp() - if err != nil { - return err - } - setPtrCustomType(base, p.field, &t) - return nil -} - -func (o *Buffer) dec_ref_time(p *Properties, base structPointer) error { - t, err := o.decTimestamp() - if err != nil { - return err - } - setCustomType(base, p.field, &t) - return nil -} - -func (o *Buffer) dec_slice_time(p *Properties, base structPointer) error { - t, err := o.decTimestamp() - if err != nil { - return err - } - newBas := appendStructPointer(base, p.field, reflect.SliceOf(reflect.PtrTo(timeType))) - var zero field - setPtrCustomType(newBas, zero, &t) - return nil -} - -func (o *Buffer) dec_slice_ref_time(p *Properties, base structPointer) error { - t, err := o.decTimestamp() - if err != nil { - return err - } - newBas := appendStructPointer(base, p.field, reflect.SliceOf(timeType)) - var zero field - setCustomType(newBas, zero, &t) - return nil -} - -func size_time(p *Properties, base structPointer) (n int) { - structp := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(structp) { - return 0 - } - tim := structPointer_Interface(structp, timeType).(*time.Time) - t, err := timestampProto(*tim) - if err != nil { - return 0 - } - size := Size(t) - return size + sizeVarint(uint64(size)) + len(p.tagcode) -} - -func (o *Buffer) enc_time(p *Properties, base structPointer) error { - structp := structPointer_GetStructPointer(base, p.field) - if structPointer_IsNil(structp) { - return ErrNil - } - tim := structPointer_Interface(structp, timeType).(*time.Time) - t, err := timestampProto(*tim) - if err != nil { - return err - } - data, err := Marshal(t) - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - return nil -} - -func size_ref_time(p *Properties, base structPointer) (n int) { - tim := structPointer_InterfaceAt(base, p.field, timeType).(*time.Time) - t, err := timestampProto(*tim) - if err != nil { - return 0 - } - size := Size(t) - return size + sizeVarint(uint64(size)) + len(p.tagcode) -} - -func (o *Buffer) enc_ref_time(p *Properties, base structPointer) error { - tim := structPointer_InterfaceAt(base, p.field, timeType).(*time.Time) - t, err := timestampProto(*tim) - if err != nil { - return err - } - data, err := Marshal(t) - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - return nil -} - -func size_slice_time(p *Properties, base structPointer) (n int) { - ptims := structPointer_InterfaceAt(base, p.field, reflect.SliceOf(reflect.PtrTo(timeType))).(*[]*time.Time) - tims := *ptims - for i := 0; i < len(tims); i++ { - if tims[i] == nil { - return 0 - } - tproto, err := timestampProto(*tims[i]) - if err != nil { - return 0 - } - size := Size(tproto) - n += len(p.tagcode) + size + sizeVarint(uint64(size)) - } - return n -} - -func (o *Buffer) enc_slice_time(p *Properties, base structPointer) error { - ptims := structPointer_InterfaceAt(base, p.field, reflect.SliceOf(reflect.PtrTo(timeType))).(*[]*time.Time) - tims := *ptims - for i := 0; i < len(tims); i++ { - if tims[i] == nil { - return errRepeatedHasNil - } - tproto, err := timestampProto(*tims[i]) - if err != nil { - return err - } - data, err := Marshal(tproto) - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - } - return nil -} - -func size_slice_ref_time(p *Properties, base structPointer) (n int) { - ptims := structPointer_InterfaceAt(base, p.field, reflect.SliceOf(timeType)).(*[]time.Time) - tims := *ptims - for i := 0; i < len(tims); i++ { - tproto, err := timestampProto(tims[i]) - if err != nil { - return 0 - } - size := Size(tproto) - n += len(p.tagcode) + size + sizeVarint(uint64(size)) - } - return n -} - -func (o *Buffer) enc_slice_ref_time(p *Properties, base structPointer) error { - ptims := structPointer_InterfaceAt(base, p.field, reflect.SliceOf(timeType)).(*[]time.Time) - tims := *ptims - for i := 0; i < len(tims); i++ { - tproto, err := timestampProto(tims[i]) - if err != nil { - return err - } - data, err := Marshal(tproto) - if err != nil { - return err - } - o.buf = append(o.buf, p.tagcode...) - o.EncodeRawBytes(data) - } - return nil -} diff --git a/vendor/github.com/opentracing/basictracer-go/LICENSE b/vendor/github.com/opentracing/basictracer-go/LICENSE deleted file mode 100644 index 148509a4035a9..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/LICENSE +++ /dev/null @@ -1,21 +0,0 @@ -The MIT License (MIT) - -Copyright (c) 2016 The OpenTracing Authors - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all -copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -SOFTWARE. diff --git a/vendor/github.com/opentracing/basictracer-go/context.go b/vendor/github.com/opentracing/basictracer-go/context.go deleted file mode 100644 index a2db9ffe889f3..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/context.go +++ /dev/null @@ -1,42 +0,0 @@ -package basictracer - -// SpanContext holds the basic Span metadata. -type SpanContext struct { - // A probabilistically unique identifier for a [multi-span] trace. - TraceID uint64 - - // A probabilistically unique identifier for a span. - SpanID uint64 - - // Whether the trace is sampled. - Sampled bool - - // The span's associated baggage. - Baggage map[string]string // initialized on first use -} - -// ForeachBaggageItem belongs to the opentracing.SpanContext interface -func (c SpanContext) ForeachBaggageItem(handler func(k, v string) bool) { - for k, v := range c.Baggage { - if !handler(k, v) { - break - } - } -} - -// WithBaggageItem returns an entirely new basictracer SpanContext with the -// given key:value baggage pair set. -func (c SpanContext) WithBaggageItem(key, val string) SpanContext { - var newBaggage map[string]string - if c.Baggage == nil { - newBaggage = map[string]string{key: val} - } else { - newBaggage = make(map[string]string, len(c.Baggage)+1) - for k, v := range c.Baggage { - newBaggage[k] = v - } - newBaggage[key] = val - } - // Use positional parameters so the compiler will help catch new fields. - return SpanContext{c.TraceID, c.SpanID, c.Sampled, newBaggage} -} diff --git a/vendor/github.com/opentracing/basictracer-go/debug.go b/vendor/github.com/opentracing/basictracer-go/debug.go deleted file mode 100644 index 817061927f5e6..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/debug.go +++ /dev/null @@ -1,78 +0,0 @@ -package basictracer - -import ( - "bytes" - "fmt" - "runtime" - "strconv" - "sync" -) - -const debugGoroutineIDTag = "_initial_goroutine" - -type errAssertionFailed struct { - span *spanImpl - msg string -} - -// Error implements the error interface. -func (err *errAssertionFailed) Error() string { - return fmt.Sprintf("%s:\n%+v", err.msg, err.span) -} - -func (s *spanImpl) Lock() { - s.Mutex.Lock() - s.maybeAssertSanityLocked() -} - -func (s *spanImpl) maybeAssertSanityLocked() { - if s.tracer == nil { - s.Mutex.Unlock() - panic(&errAssertionFailed{span: s, msg: "span used after call to Finish()"}) - } - if s.tracer.options.DebugAssertSingleGoroutine { - startID := curGoroutineID() - curID, ok := s.raw.Tags[debugGoroutineIDTag].(uint64) - if !ok { - // This is likely invoked in the context of the SetTag which sets - // debugGoroutineTag. - return - } - if startID != curID { - s.Mutex.Unlock() - panic(&errAssertionFailed{ - span: s, - msg: fmt.Sprintf("span started on goroutine %d, but now running on %d", startID, curID), - }) - } - } -} - -var goroutineSpace = []byte("goroutine ") -var littleBuf = sync.Pool{ - New: func() interface{} { - buf := make([]byte, 64) - return &buf - }, -} - -// Credit to @bradfitz: -// https://github.com/golang/net/blob/master/http2/gotrack.go#L51 -func curGoroutineID() uint64 { - bp := littleBuf.Get().(*[]byte) - defer littleBuf.Put(bp) - b := *bp - b = b[:runtime.Stack(b, false)] - // Parse the 4707 out of "goroutine 4707 [" - b = bytes.TrimPrefix(b, goroutineSpace) - i := bytes.IndexByte(b, ' ') - if i < 0 { - panic(fmt.Sprintf("No space found in %q", b)) - } - b = b[:i] - n, err := strconv.ParseUint(string(b), 10, 64) - if err != nil { - panic(fmt.Sprintf("Failed to parse goroutine ID out of %q: %v", b, err)) - } - return n -} diff --git a/vendor/github.com/opentracing/basictracer-go/event.go b/vendor/github.com/opentracing/basictracer-go/event.go deleted file mode 100644 index e074fcc7d86ad..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/event.go +++ /dev/null @@ -1,62 +0,0 @@ -package basictracer - -import "github.com/opentracing/opentracing-go" - -// A SpanEvent is emitted when a mutating command is called on a Span. -type SpanEvent interface{} - -// EventCreate is emitted when a Span is created. -type EventCreate struct{ OperationName string } - -// EventTag is received when SetTag is called. -type EventTag struct { - Key string - Value interface{} -} - -// EventBaggage is received when SetBaggageItem is called. -type EventBaggage struct { - Key, Value string -} - -// EventLogFields is received when LogFields or LogKV is called. -type EventLogFields opentracing.LogRecord - -// EventLog is received when Log (or one of its derivatives) is called. -// -// DEPRECATED -type EventLog opentracing.LogData - -// EventFinish is received when Finish is called. -type EventFinish RawSpan - -func (s *spanImpl) onCreate(opName string) { - if s.event != nil { - s.event(EventCreate{OperationName: opName}) - } -} -func (s *spanImpl) onTag(key string, value interface{}) { - if s.event != nil { - s.event(EventTag{Key: key, Value: value}) - } -} -func (s *spanImpl) onLog(ld opentracing.LogData) { - if s.event != nil { - s.event(EventLog(ld)) - } -} -func (s *spanImpl) onLogFields(lr opentracing.LogRecord) { - if s.event != nil { - s.event(EventLogFields(lr)) - } -} -func (s *spanImpl) onBaggage(key, value string) { - if s.event != nil { - s.event(EventBaggage{Key: key, Value: value}) - } -} -func (s *spanImpl) onFinish(sp RawSpan) { - if s.event != nil { - s.event(EventFinish(sp)) - } -} diff --git a/vendor/github.com/opentracing/basictracer-go/propagation.go b/vendor/github.com/opentracing/basictracer-go/propagation.go deleted file mode 100644 index 15970c696486c..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/propagation.go +++ /dev/null @@ -1,61 +0,0 @@ -package basictracer - -import opentracing "github.com/opentracing/opentracing-go" - -type accessorPropagator struct { - tracer *tracerImpl -} - -// DelegatingCarrier is a flexible carrier interface which can be implemented -// by types which have a means of storing the trace metadata and already know -// how to serialize themselves (for example, protocol buffers). -type DelegatingCarrier interface { - SetState(traceID, spanID uint64, sampled bool) - State() (traceID, spanID uint64, sampled bool) - SetBaggageItem(key, value string) - GetBaggage(func(key, value string)) -} - -func (p *accessorPropagator) Inject( - spanContext opentracing.SpanContext, - carrier interface{}, -) error { - dc, ok := carrier.(DelegatingCarrier) - if !ok || dc == nil { - return opentracing.ErrInvalidCarrier - } - sc, ok := spanContext.(SpanContext) - if !ok { - return opentracing.ErrInvalidSpanContext - } - dc.SetState(sc.TraceID, sc.SpanID, sc.Sampled) - for k, v := range sc.Baggage { - dc.SetBaggageItem(k, v) - } - return nil -} - -func (p *accessorPropagator) Extract( - carrier interface{}, -) (opentracing.SpanContext, error) { - dc, ok := carrier.(DelegatingCarrier) - if !ok || dc == nil { - return nil, opentracing.ErrInvalidCarrier - } - - traceID, spanID, sampled := dc.State() - sc := SpanContext{ - TraceID: traceID, - SpanID: spanID, - Sampled: sampled, - Baggage: nil, - } - dc.GetBaggage(func(k, v string) { - if sc.Baggage == nil { - sc.Baggage = map[string]string{} - } - sc.Baggage[k] = v - }) - - return sc, nil -} diff --git a/vendor/github.com/opentracing/basictracer-go/propagation_ot.go b/vendor/github.com/opentracing/basictracer-go/propagation_ot.go deleted file mode 100644 index 61362ceecaab9..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/propagation_ot.go +++ /dev/null @@ -1,180 +0,0 @@ -package basictracer - -import ( - "encoding/binary" - "io" - "strconv" - "strings" - - "github.com/gogo/protobuf/proto" - "github.com/opentracing/basictracer-go/wire" - opentracing "github.com/opentracing/opentracing-go" -) - -type textMapPropagator struct { - tracer *tracerImpl -} -type binaryPropagator struct { - tracer *tracerImpl -} - -const ( - prefixTracerState = "ot-tracer-" - prefixBaggage = "ot-baggage-" - - tracerStateFieldCount = 3 - fieldNameTraceID = prefixTracerState + "traceid" - fieldNameSpanID = prefixTracerState + "spanid" - fieldNameSampled = prefixTracerState + "sampled" -) - -func (p *textMapPropagator) Inject( - spanContext opentracing.SpanContext, - opaqueCarrier interface{}, -) error { - sc, ok := spanContext.(SpanContext) - if !ok { - return opentracing.ErrInvalidSpanContext - } - carrier, ok := opaqueCarrier.(opentracing.TextMapWriter) - if !ok { - return opentracing.ErrInvalidCarrier - } - carrier.Set(fieldNameTraceID, strconv.FormatUint(sc.TraceID, 16)) - carrier.Set(fieldNameSpanID, strconv.FormatUint(sc.SpanID, 16)) - carrier.Set(fieldNameSampled, strconv.FormatBool(sc.Sampled)) - - for k, v := range sc.Baggage { - carrier.Set(prefixBaggage+k, v) - } - return nil -} - -func (p *textMapPropagator) Extract( - opaqueCarrier interface{}, -) (opentracing.SpanContext, error) { - carrier, ok := opaqueCarrier.(opentracing.TextMapReader) - if !ok { - return nil, opentracing.ErrInvalidCarrier - } - requiredFieldCount := 0 - var traceID, spanID uint64 - var sampled bool - var err error - decodedBaggage := make(map[string]string) - err = carrier.ForeachKey(func(k, v string) error { - switch strings.ToLower(k) { - case fieldNameTraceID: - traceID, err = strconv.ParseUint(v, 16, 64) - if err != nil { - return opentracing.ErrSpanContextCorrupted - } - case fieldNameSpanID: - spanID, err = strconv.ParseUint(v, 16, 64) - if err != nil { - return opentracing.ErrSpanContextCorrupted - } - case fieldNameSampled: - sampled, err = strconv.ParseBool(v) - if err != nil { - return opentracing.ErrSpanContextCorrupted - } - default: - lowercaseK := strings.ToLower(k) - if strings.HasPrefix(lowercaseK, prefixBaggage) { - decodedBaggage[strings.TrimPrefix(lowercaseK, prefixBaggage)] = v - } - // Balance off the requiredFieldCount++ just below... - requiredFieldCount-- - } - requiredFieldCount++ - return nil - }) - if err != nil { - return nil, err - } - if requiredFieldCount < tracerStateFieldCount { - if requiredFieldCount == 0 { - return nil, opentracing.ErrSpanContextNotFound - } - return nil, opentracing.ErrSpanContextCorrupted - } - - return SpanContext{ - TraceID: traceID, - SpanID: spanID, - Sampled: sampled, - Baggage: decodedBaggage, - }, nil -} - -func (p *binaryPropagator) Inject( - spanContext opentracing.SpanContext, - opaqueCarrier interface{}, -) error { - sc, ok := spanContext.(SpanContext) - if !ok { - return opentracing.ErrInvalidSpanContext - } - carrier, ok := opaqueCarrier.(io.Writer) - if !ok { - return opentracing.ErrInvalidCarrier - } - - state := wire.TracerState{} - state.TraceId = sc.TraceID - state.SpanId = sc.SpanID - state.Sampled = sc.Sampled - state.BaggageItems = sc.Baggage - - b, err := proto.Marshal(&state) - if err != nil { - return err - } - - // Write the length of the marshalled binary to the writer. - length := uint32(len(b)) - if err := binary.Write(carrier, binary.BigEndian, &length); err != nil { - return err - } - - _, err = carrier.Write(b) - return err -} - -func (p *binaryPropagator) Extract( - opaqueCarrier interface{}, -) (opentracing.SpanContext, error) { - carrier, ok := opaqueCarrier.(io.Reader) - if !ok { - return nil, opentracing.ErrInvalidCarrier - } - - // Read the length of marshalled binary. io.ReadAll isn't that performant - // since it keeps resizing the underlying buffer as it encounters more bytes - // to read. By reading the length, we can allocate a fixed sized buf and read - // the exact amount of bytes into it. - var length uint32 - if err := binary.Read(carrier, binary.BigEndian, &length); err != nil { - return nil, opentracing.ErrSpanContextCorrupted - } - buf := make([]byte, length) - if n, err := carrier.Read(buf); err != nil { - if n > 0 { - return nil, opentracing.ErrSpanContextCorrupted - } - return nil, opentracing.ErrSpanContextNotFound - } - - ctx := wire.TracerState{} - if err := proto.Unmarshal(buf, &ctx); err != nil { - return nil, opentracing.ErrSpanContextCorrupted - } - - return SpanContext{ - TraceID: ctx.TraceId, - SpanID: ctx.SpanId, - Sampled: ctx.Sampled, - Baggage: ctx.BaggageItems, - }, nil -} diff --git a/vendor/github.com/opentracing/basictracer-go/raw.go b/vendor/github.com/opentracing/basictracer-go/raw.go deleted file mode 100644 index e038b3694c103..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/raw.go +++ /dev/null @@ -1,34 +0,0 @@ -package basictracer - -import ( - "time" - - opentracing "github.com/opentracing/opentracing-go" -) - -// RawSpan encapsulates all state associated with a (finished) Span. -type RawSpan struct { - // Those recording the RawSpan should also record the contents of its - // SpanContext. - Context SpanContext - - // The SpanID of this SpanContext's first intra-trace reference (i.e., - // "parent"), or 0 if there is no parent. - ParentSpanID uint64 - - // The name of the "operation" this span is an instance of. (Called a "span - // name" in some implementations) - Operation string - - // We store rather than so that only - // one of the timestamps has global clock uncertainty issues. - Start time.Time - Duration time.Duration - - // Essentially an extension mechanism. Can be used for many purposes, - // not to be enumerated here. - Tags opentracing.Tags - - // The span's "microlog". - Logs []opentracing.LogRecord -} diff --git a/vendor/github.com/opentracing/basictracer-go/recorder.go b/vendor/github.com/opentracing/basictracer-go/recorder.go deleted file mode 100644 index d6a0856c33a71..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/recorder.go +++ /dev/null @@ -1,60 +0,0 @@ -package basictracer - -import "sync" - -// A SpanRecorder handles all of the `RawSpan` data generated via an -// associated `Tracer` (see `NewStandardTracer`) instance. It also names -// the containing process and provides access to a straightforward tag map. -type SpanRecorder interface { - // Implementations must determine whether and where to store `span`. - RecordSpan(span RawSpan) -} - -// InMemorySpanRecorder is a simple thread-safe implementation of -// SpanRecorder that stores all reported spans in memory, accessible -// via reporter.GetSpans(). It is primarily intended for testing purposes. -type InMemorySpanRecorder struct { - sync.RWMutex - spans []RawSpan -} - -// NewInMemoryRecorder creates new InMemorySpanRecorder -func NewInMemoryRecorder() *InMemorySpanRecorder { - return new(InMemorySpanRecorder) -} - -// RecordSpan implements the respective method of SpanRecorder. -func (r *InMemorySpanRecorder) RecordSpan(span RawSpan) { - r.Lock() - defer r.Unlock() - r.spans = append(r.spans, span) -} - -// GetSpans returns a copy of the array of spans accumulated so far. -func (r *InMemorySpanRecorder) GetSpans() []RawSpan { - r.RLock() - defer r.RUnlock() - spans := make([]RawSpan, len(r.spans)) - copy(spans, r.spans) - return spans -} - -// GetSampledSpans returns a slice of spans accumulated so far which were sampled. -func (r *InMemorySpanRecorder) GetSampledSpans() []RawSpan { - r.RLock() - defer r.RUnlock() - spans := make([]RawSpan, 0, len(r.spans)) - for _, span := range r.spans { - if span.Context.Sampled { - spans = append(spans, span) - } - } - return spans -} - -// Reset clears the internal array of spans. -func (r *InMemorySpanRecorder) Reset() { - r.Lock() - defer r.Unlock() - r.spans = nil -} diff --git a/vendor/github.com/opentracing/basictracer-go/span.go b/vendor/github.com/opentracing/basictracer-go/span.go deleted file mode 100644 index fdceba5bcbdb4..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/span.go +++ /dev/null @@ -1,274 +0,0 @@ -package basictracer - -import ( - "sync" - "time" - - opentracing "github.com/opentracing/opentracing-go" - "github.com/opentracing/opentracing-go/ext" - "github.com/opentracing/opentracing-go/log" -) - -// Span provides access to the essential details of the span, for use -// by basictracer consumers. These methods may only be called prior -// to (*opentracing.Span).Finish(). -type Span interface { - opentracing.Span - - // Operation names the work done by this span instance - Operation() string - - // Start indicates when the span began - Start() time.Time -} - -// Implements the `Span` interface. Created via tracerImpl (see -// `basictracer.New()`). -type spanImpl struct { - tracer *tracerImpl - event func(SpanEvent) - sync.Mutex // protects the fields below - raw RawSpan - // The number of logs dropped because of MaxLogsPerSpan. - numDroppedLogs int -} - -var spanPool = &sync.Pool{New: func() interface{} { - return &spanImpl{} -}} - -func (s *spanImpl) reset() { - s.tracer, s.event = nil, nil - // Note: Would like to do the following, but then the consumer of RawSpan - // (the recorder) needs to make sure that they're not holding on to the - // baggage or logs when they return (i.e. they need to copy if they care): - // - // logs, baggage := s.raw.Logs[:0], s.raw.Baggage - // for k := range baggage { - // delete(baggage, k) - // } - // s.raw.Logs, s.raw.Baggage = logs, baggage - // - // That's likely too much to ask for. But there is some magic we should - // be able to do with `runtime.SetFinalizer` to reclaim that memory into - // a buffer pool when GC considers them unreachable, which should ease - // some of the load. Hard to say how quickly that would be in practice - // though. - s.raw = RawSpan{ - Context: SpanContext{}, - } -} - -func (s *spanImpl) SetOperationName(operationName string) opentracing.Span { - s.Lock() - defer s.Unlock() - s.raw.Operation = operationName - return s -} - -func (s *spanImpl) trim() bool { - return !s.raw.Context.Sampled && s.tracer.options.TrimUnsampledSpans -} - -func (s *spanImpl) SetTag(key string, value interface{}) opentracing.Span { - defer s.onTag(key, value) - s.Lock() - defer s.Unlock() - if key == string(ext.SamplingPriority) { - if v, ok := value.(uint16); ok { - s.raw.Context.Sampled = v != 0 - return s - } - } - if s.trim() { - return s - } - - if s.raw.Tags == nil { - s.raw.Tags = opentracing.Tags{} - } - s.raw.Tags[key] = value - return s -} - -func (s *spanImpl) LogKV(keyValues ...interface{}) { - fields, err := log.InterleavedKVToFields(keyValues...) - if err != nil { - s.LogFields(log.Error(err), log.String("function", "LogKV")) - return - } - s.LogFields(fields...) -} - -func (s *spanImpl) appendLog(lr opentracing.LogRecord) { - maxLogs := s.tracer.options.MaxLogsPerSpan - if maxLogs == 0 || len(s.raw.Logs) < maxLogs { - s.raw.Logs = append(s.raw.Logs, lr) - return - } - - // We have too many logs. We don't touch the first numOld logs; we treat the - // rest as a circular buffer and overwrite the oldest log among those. - numOld := (maxLogs - 1) / 2 - numNew := maxLogs - numOld - s.raw.Logs[numOld+s.numDroppedLogs%numNew] = lr - s.numDroppedLogs++ -} - -func (s *spanImpl) LogFields(fields ...log.Field) { - lr := opentracing.LogRecord{ - Fields: fields, - } - defer s.onLogFields(lr) - s.Lock() - defer s.Unlock() - if s.trim() || s.tracer.options.DropAllLogs { - return - } - if lr.Timestamp.IsZero() { - lr.Timestamp = time.Now() - } - s.appendLog(lr) -} - -func (s *spanImpl) LogEvent(event string) { - s.Log(opentracing.LogData{ - Event: event, - }) -} - -func (s *spanImpl) LogEventWithPayload(event string, payload interface{}) { - s.Log(opentracing.LogData{ - Event: event, - Payload: payload, - }) -} - -func (s *spanImpl) Log(ld opentracing.LogData) { - defer s.onLog(ld) - s.Lock() - defer s.Unlock() - if s.trim() || s.tracer.options.DropAllLogs { - return - } - - if ld.Timestamp.IsZero() { - ld.Timestamp = time.Now() - } - - s.appendLog(ld.ToLogRecord()) -} - -func (s *spanImpl) Finish() { - s.FinishWithOptions(opentracing.FinishOptions{}) -} - -// rotateLogBuffer rotates the records in the buffer: records 0 to pos-1 move at -// the end (i.e. pos circular left shifts). -func rotateLogBuffer(buf []opentracing.LogRecord, pos int) { - // This algorithm is described in: - // http://www.cplusplus.com/reference/algorithm/rotate - for first, middle, next := 0, pos, pos; first != middle; { - buf[first], buf[next] = buf[next], buf[first] - first++ - next++ - if next == len(buf) { - next = middle - } else if first == middle { - middle = next - } - } -} - -func (s *spanImpl) FinishWithOptions(opts opentracing.FinishOptions) { - finishTime := opts.FinishTime - if finishTime.IsZero() { - finishTime = time.Now() - } - duration := finishTime.Sub(s.raw.Start) - - s.Lock() - defer s.Unlock() - - for _, lr := range opts.LogRecords { - s.appendLog(lr) - } - for _, ld := range opts.BulkLogData { - s.appendLog(ld.ToLogRecord()) - } - - if s.numDroppedLogs > 0 { - // We dropped some log events, which means that we used part of Logs as a - // circular buffer (see appendLog). De-circularize it. - numOld := (len(s.raw.Logs) - 1) / 2 - numNew := len(s.raw.Logs) - numOld - rotateLogBuffer(s.raw.Logs[numOld:], s.numDroppedLogs%numNew) - - // Replace the log in the middle (the oldest "new" log) with information - // about the dropped logs. This means that we are effectively dropping one - // more "new" log. - numDropped := s.numDroppedLogs + 1 - s.raw.Logs[numOld] = opentracing.LogRecord{ - // Keep the timestamp of the last dropped event. - Timestamp: s.raw.Logs[numOld].Timestamp, - Fields: []log.Field{ - log.String("event", "dropped Span logs"), - log.Int("dropped_log_count", numDropped), - log.String("component", "basictracer"), - }, - } - } - - s.raw.Duration = duration - - s.onFinish(s.raw) - s.tracer.options.Recorder.RecordSpan(s.raw) - - // Last chance to get options before the span is possibly reset. - poolEnabled := s.tracer.options.EnableSpanPool - if s.tracer.options.DebugAssertUseAfterFinish { - // This makes it much more likely to catch a panic on any subsequent - // operation since s.tracer is accessed on every call to `Lock`. - // We don't call `reset()` here to preserve the logs in the Span - // which are printed when the assertion triggers. - s.tracer = nil - } - - if poolEnabled { - spanPool.Put(s) - } -} - -func (s *spanImpl) Tracer() opentracing.Tracer { - return s.tracer -} - -func (s *spanImpl) Context() opentracing.SpanContext { - return s.raw.Context -} - -func (s *spanImpl) SetBaggageItem(key, val string) opentracing.Span { - s.onBaggage(key, val) - if s.trim() { - return s - } - - s.Lock() - defer s.Unlock() - s.raw.Context = s.raw.Context.WithBaggageItem(key, val) - return s -} - -func (s *spanImpl) BaggageItem(key string) string { - s.Lock() - defer s.Unlock() - return s.raw.Context.Baggage[key] -} - -func (s *spanImpl) Operation() string { - return s.raw.Operation -} - -func (s *spanImpl) Start() time.Time { - return s.raw.Start -} diff --git a/vendor/github.com/opentracing/basictracer-go/tracer.go b/vendor/github.com/opentracing/basictracer-go/tracer.go deleted file mode 100644 index 3e03696e68244..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/tracer.go +++ /dev/null @@ -1,262 +0,0 @@ -package basictracer - -import ( - "time" - - opentracing "github.com/opentracing/opentracing-go" -) - -// Tracer extends the opentracing.Tracer interface with methods to -// probe implementation state, for use by basictracer consumers. -type Tracer interface { - opentracing.Tracer - - // Options gets the Options used in New() or NewWithOptions(). - Options() Options -} - -// Options allows creating a customized Tracer via NewWithOptions. The object -// must not be updated when there is an active tracer using it. -type Options struct { - // ShouldSample is a function which is called when creating a new Span and - // determines whether that Span is sampled. The randomized TraceID is supplied - // to allow deterministic sampling decisions to be made across different nodes. - // For example, - // - // func(traceID uint64) { return traceID % 64 == 0 } - // - // samples every 64th trace on average. - ShouldSample func(traceID uint64) bool - // TrimUnsampledSpans turns potentially expensive operations on unsampled - // Spans into no-ops. More precisely, tags and log events are silently - // discarded. If NewSpanEventListener is set, the callbacks will still fire. - TrimUnsampledSpans bool - // Recorder receives Spans which have been finished. - Recorder SpanRecorder - // NewSpanEventListener can be used to enhance the tracer by effectively - // attaching external code to trace events. See NetTraceIntegrator for a - // practical example, and event.go for the list of possible events. - NewSpanEventListener func() func(SpanEvent) - // DropAllLogs turns log events on all Spans into no-ops. - // If NewSpanEventListener is set, the callbacks will still fire. - DropAllLogs bool - // MaxLogsPerSpan limits the number of Logs in a span (if set to a nonzero - // value). If a span has more logs than this value, logs are dropped as - // necessary (and replaced with a log describing how many were dropped). - // - // About half of the MaxLogPerSpan logs kept are the oldest logs, and about - // half are the newest logs. - // - // If NewSpanEventListener is set, the callbacks will still fire for all log - // events. This value is ignored if DropAllLogs is true. - MaxLogsPerSpan int - // DebugAssertSingleGoroutine internally records the ID of the goroutine - // creating each Span and verifies that no operation is carried out on - // it on a different goroutine. - // Provided strictly for development purposes. - // Passing Spans between goroutine without proper synchronization often - // results in use-after-Finish() errors. For a simple example, consider the - // following pseudocode: - // - // func (s *Server) Handle(req http.Request) error { - // sp := s.StartSpan("server") - // defer sp.Finish() - // wait := s.queueProcessing(opentracing.ContextWithSpan(context.Background(), sp), req) - // select { - // case resp := <-wait: - // return resp.Error - // case <-time.After(10*time.Second): - // sp.LogEvent("timed out waiting for processing") - // return ErrTimedOut - // } - // } - // - // This looks reasonable at first, but a request which spends more than ten - // seconds in the queue is abandoned by the main goroutine and its trace - // finished, leading to use-after-finish when the request is finally - // processed. Note also that even joining on to a finished Span via - // StartSpanWithOptions constitutes an illegal operation. - // - // Code bases which do not require (or decide they do not want) Spans to - // be passed across goroutine boundaries can run with this flag enabled in - // tests to increase their chances of spotting wrong-doers. - DebugAssertSingleGoroutine bool - // DebugAssertUseAfterFinish is provided strictly for development purposes. - // When set, it attempts to exacerbate issues emanating from use of Spans - // after calling Finish by running additional assertions. - DebugAssertUseAfterFinish bool - // EnableSpanPool enables the use of a pool, so that the tracer reuses spans - // after Finish has been called on it. Adds a slight performance gain as it - // reduces allocations. However, if you have any use-after-finish race - // conditions the code may panic. - EnableSpanPool bool -} - -// DefaultOptions returns an Options object with a 1 in 64 sampling rate and -// all options disabled. A Recorder needs to be set manually before using the -// returned object with a Tracer. -func DefaultOptions() Options { - return Options{ - ShouldSample: func(traceID uint64) bool { return traceID%64 == 0 }, - MaxLogsPerSpan: 100, - } -} - -// NewWithOptions creates a customized Tracer. -func NewWithOptions(opts Options) opentracing.Tracer { - rval := &tracerImpl{options: opts} - rval.textPropagator = &textMapPropagator{rval} - rval.binaryPropagator = &binaryPropagator{rval} - rval.accessorPropagator = &accessorPropagator{rval} - return rval -} - -// New creates and returns a standard Tracer which defers completed Spans to -// `recorder`. -// Spans created by this Tracer support the ext.SamplingPriority tag: Setting -// ext.SamplingPriority causes the Span to be Sampled from that point on. -func New(recorder SpanRecorder) opentracing.Tracer { - opts := DefaultOptions() - opts.Recorder = recorder - return NewWithOptions(opts) -} - -// Implements the `Tracer` interface. -type tracerImpl struct { - options Options - textPropagator *textMapPropagator - binaryPropagator *binaryPropagator - accessorPropagator *accessorPropagator -} - -func (t *tracerImpl) StartSpan( - operationName string, - opts ...opentracing.StartSpanOption, -) opentracing.Span { - sso := opentracing.StartSpanOptions{} - for _, o := range opts { - o.Apply(&sso) - } - return t.StartSpanWithOptions(operationName, sso) -} - -func (t *tracerImpl) getSpan() *spanImpl { - if t.options.EnableSpanPool { - sp := spanPool.Get().(*spanImpl) - sp.reset() - return sp - } - return &spanImpl{} -} - -func (t *tracerImpl) StartSpanWithOptions( - operationName string, - opts opentracing.StartSpanOptions, -) opentracing.Span { - // Start time. - startTime := opts.StartTime - if startTime.IsZero() { - startTime = time.Now() - } - - // Tags. - tags := opts.Tags - - // Build the new span. This is the only allocation: We'll return this as - // an opentracing.Span. - sp := t.getSpan() - // Look for a parent in the list of References. - // - // TODO: would be nice if basictracer did something with all - // References, not just the first one. -ReferencesLoop: - for _, ref := range opts.References { - switch ref.Type { - case opentracing.ChildOfRef, - opentracing.FollowsFromRef: - - refCtx := ref.ReferencedContext.(SpanContext) - sp.raw.Context.TraceID = refCtx.TraceID - sp.raw.Context.SpanID = randomID() - sp.raw.Context.Sampled = refCtx.Sampled - sp.raw.ParentSpanID = refCtx.SpanID - - if l := len(refCtx.Baggage); l > 0 { - sp.raw.Context.Baggage = make(map[string]string, l) - for k, v := range refCtx.Baggage { - sp.raw.Context.Baggage[k] = v - } - } - break ReferencesLoop - } - } - if sp.raw.Context.TraceID == 0 { - // No parent Span found; allocate new trace and span ids and determine - // the Sampled status. - sp.raw.Context.TraceID, sp.raw.Context.SpanID = randomID2() - sp.raw.Context.Sampled = t.options.ShouldSample(sp.raw.Context.TraceID) - } - - return t.startSpanInternal( - sp, - operationName, - startTime, - tags, - ) -} - -func (t *tracerImpl) startSpanInternal( - sp *spanImpl, - operationName string, - startTime time.Time, - tags opentracing.Tags, -) opentracing.Span { - sp.tracer = t - if t.options.NewSpanEventListener != nil { - sp.event = t.options.NewSpanEventListener() - } - sp.raw.Operation = operationName - sp.raw.Start = startTime - sp.raw.Duration = -1 - sp.raw.Tags = tags - if t.options.DebugAssertSingleGoroutine { - sp.SetTag(debugGoroutineIDTag, curGoroutineID()) - } - defer sp.onCreate(operationName) - return sp -} - -type delegatorType struct{} - -// Delegator is the format to use for DelegatingCarrier. -var Delegator delegatorType - -func (t *tracerImpl) Inject(sc opentracing.SpanContext, format interface{}, carrier interface{}) error { - switch format { - case opentracing.TextMap, opentracing.HTTPHeaders: - return t.textPropagator.Inject(sc, carrier) - case opentracing.Binary: - return t.binaryPropagator.Inject(sc, carrier) - } - if _, ok := format.(delegatorType); ok { - return t.accessorPropagator.Inject(sc, carrier) - } - return opentracing.ErrUnsupportedFormat -} - -func (t *tracerImpl) Extract(format interface{}, carrier interface{}) (opentracing.SpanContext, error) { - switch format { - case opentracing.TextMap, opentracing.HTTPHeaders: - return t.textPropagator.Extract(carrier) - case opentracing.Binary: - return t.binaryPropagator.Extract(carrier) - } - if _, ok := format.(delegatorType); ok { - return t.accessorPropagator.Extract(carrier) - } - return nil, opentracing.ErrUnsupportedFormat -} - -func (t *tracerImpl) Options() Options { - return t.options -} diff --git a/vendor/github.com/opentracing/basictracer-go/util.go b/vendor/github.com/opentracing/basictracer-go/util.go deleted file mode 100644 index 0f4ed8ac9a23b..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/util.go +++ /dev/null @@ -1,25 +0,0 @@ -package basictracer - -import ( - "math/rand" - "sync" - "time" -) - -var ( - seededIDGen = rand.New(rand.NewSource(time.Now().UnixNano())) - // The golang rand generators are *not* intrinsically thread-safe. - seededIDLock sync.Mutex -) - -func randomID() uint64 { - seededIDLock.Lock() - defer seededIDLock.Unlock() - return uint64(seededIDGen.Int63()) -} - -func randomID2() (uint64, uint64) { - seededIDLock.Lock() - defer seededIDLock.Unlock() - return uint64(seededIDGen.Int63()), uint64(seededIDGen.Int63()) -} diff --git a/vendor/github.com/opentracing/basictracer-go/wire/carrier.go b/vendor/github.com/opentracing/basictracer-go/wire/carrier.go deleted file mode 100644 index 12ec98e906b0e..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/wire/carrier.go +++ /dev/null @@ -1,40 +0,0 @@ -package wire - -// ProtobufCarrier is a DelegatingCarrier that uses protocol buffers as the -// the underlying datastructure. The reason for implementing DelagatingCarrier -// is to allow for end users to serialize the underlying protocol buffers using -// jsonpb or any other serialization forms they want. -type ProtobufCarrier TracerState - -// SetState set's the tracer state. -func (p *ProtobufCarrier) SetState(traceID, spanID uint64, sampled bool) { - p.TraceId = traceID - p.SpanId = spanID - p.Sampled = sampled -} - -// State returns the tracer state. -func (p *ProtobufCarrier) State() (traceID, spanID uint64, sampled bool) { - traceID = p.TraceId - spanID = p.SpanId - sampled = p.Sampled - return traceID, spanID, sampled -} - -// SetBaggageItem sets a baggage item. -func (p *ProtobufCarrier) SetBaggageItem(key, value string) { - if p.BaggageItems == nil { - p.BaggageItems = map[string]string{key: value} - return - } - - p.BaggageItems[key] = value -} - -// GetBaggage iterates over each baggage item and executes the callback with -// the key:value pair. -func (p *ProtobufCarrier) GetBaggage(f func(k, v string)) { - for k, v := range p.BaggageItems { - f(k, v) - } -} diff --git a/vendor/github.com/opentracing/basictracer-go/wire/gen.go b/vendor/github.com/opentracing/basictracer-go/wire/gen.go deleted file mode 100644 index ce48e8b8d0935..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/wire/gen.go +++ /dev/null @@ -1,6 +0,0 @@ -package wire - -//go:generate protoc --gogofaster_out=$GOPATH/src/github.com/opentracing/basictracer-go/wire wire.proto - -// Run `go get github.com/gogo/protobuf/protoc-gen-gogofaster` to install the -// gogofaster generator binary. diff --git a/vendor/github.com/opentracing/basictracer-go/wire/wire.pb.go b/vendor/github.com/opentracing/basictracer-go/wire/wire.pb.go deleted file mode 100644 index b78d7ea9199a8..0000000000000 --- a/vendor/github.com/opentracing/basictracer-go/wire/wire.pb.go +++ /dev/null @@ -1,508 +0,0 @@ -// Code generated by protoc-gen-gogo. -// source: wire.proto -// DO NOT EDIT! - -/* - Package wire is a generated protocol buffer package. - - It is generated from these files: - wire.proto - - It has these top-level messages: - TracerState -*/ -package wire - -import proto "github.com/gogo/protobuf/proto" -import fmt "fmt" -import math "math" - -import io "io" - -// Reference imports to suppress errors if they are not otherwise used. -var _ = proto.Marshal -var _ = fmt.Errorf -var _ = math.Inf - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the proto package it is being compiled against. -const _ = proto.GoGoProtoPackageIsVersion1 - -type TracerState struct { - TraceId uint64 `protobuf:"fixed64,1,opt,name=trace_id,json=traceId,proto3" json:"trace_id,omitempty"` - SpanId uint64 `protobuf:"fixed64,2,opt,name=span_id,json=spanId,proto3" json:"span_id,omitempty"` - Sampled bool `protobuf:"varint,3,opt,name=sampled,proto3" json:"sampled,omitempty"` - BaggageItems map[string]string `protobuf:"bytes,4,rep,name=baggage_items,json=baggageItems" json:"baggage_items,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` -} - -func (m *TracerState) Reset() { *m = TracerState{} } -func (m *TracerState) String() string { return proto.CompactTextString(m) } -func (*TracerState) ProtoMessage() {} -func (*TracerState) Descriptor() ([]byte, []int) { return fileDescriptorWire, []int{0} } - -func (m *TracerState) GetBaggageItems() map[string]string { - if m != nil { - return m.BaggageItems - } - return nil -} - -func init() { - proto.RegisterType((*TracerState)(nil), "basictracer_go.wire.TracerState") -} -func (m *TracerState) 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 *TracerState) MarshalTo(data []byte) (int, error) { - var i int - _ = i - var l int - _ = l - if m.TraceId != 0 { - data[i] = 0x9 - i++ - i = encodeFixed64Wire(data, i, uint64(m.TraceId)) - } - if m.SpanId != 0 { - data[i] = 0x11 - i++ - i = encodeFixed64Wire(data, i, uint64(m.SpanId)) - } - if m.Sampled { - data[i] = 0x18 - i++ - if m.Sampled { - data[i] = 1 - } else { - data[i] = 0 - } - i++ - } - if len(m.BaggageItems) > 0 { - for k, _ := range m.BaggageItems { - data[i] = 0x22 - i++ - v := m.BaggageItems[k] - mapSize := 1 + len(k) + sovWire(uint64(len(k))) + 1 + len(v) + sovWire(uint64(len(v))) - i = encodeVarintWire(data, i, uint64(mapSize)) - data[i] = 0xa - i++ - i = encodeVarintWire(data, i, uint64(len(k))) - i += copy(data[i:], k) - data[i] = 0x12 - i++ - i = encodeVarintWire(data, i, uint64(len(v))) - i += copy(data[i:], v) - } - } - return i, nil -} - -func encodeFixed64Wire(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 encodeFixed32Wire(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 encodeVarintWire(data []byte, offset int, v uint64) int { - for v >= 1<<7 { - data[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ - } - data[offset] = uint8(v) - return offset + 1 -} -func (m *TracerState) Size() (n int) { - var l int - _ = l - if m.TraceId != 0 { - n += 9 - } - if m.SpanId != 0 { - n += 9 - } - if m.Sampled { - n += 2 - } - if len(m.BaggageItems) > 0 { - for k, v := range m.BaggageItems { - _ = k - _ = v - mapEntrySize := 1 + len(k) + sovWire(uint64(len(k))) + 1 + len(v) + sovWire(uint64(len(v))) - n += mapEntrySize + 1 + sovWire(uint64(mapEntrySize)) - } - } - return n -} - -func sovWire(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { - break - } - } - return n -} -func sozWire(x uint64) (n int) { - return sovWire(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} -func (m *TracerState) 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 ErrIntOverflowWire - } - 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: TracerState: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: TracerState: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field TraceId", wireType) - } - m.TraceId = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - iNdEx += 8 - m.TraceId = uint64(data[iNdEx-8]) - m.TraceId |= uint64(data[iNdEx-7]) << 8 - m.TraceId |= uint64(data[iNdEx-6]) << 16 - m.TraceId |= uint64(data[iNdEx-5]) << 24 - m.TraceId |= uint64(data[iNdEx-4]) << 32 - m.TraceId |= uint64(data[iNdEx-3]) << 40 - m.TraceId |= uint64(data[iNdEx-2]) << 48 - m.TraceId |= uint64(data[iNdEx-1]) << 56 - case 2: - if wireType != 1 { - return fmt.Errorf("proto: wrong wireType = %d for field SpanId", wireType) - } - m.SpanId = 0 - if (iNdEx + 8) > l { - return io.ErrUnexpectedEOF - } - iNdEx += 8 - m.SpanId = uint64(data[iNdEx-8]) - m.SpanId |= uint64(data[iNdEx-7]) << 8 - m.SpanId |= uint64(data[iNdEx-6]) << 16 - m.SpanId |= uint64(data[iNdEx-5]) << 24 - m.SpanId |= uint64(data[iNdEx-4]) << 32 - m.SpanId |= uint64(data[iNdEx-3]) << 40 - m.SpanId |= uint64(data[iNdEx-2]) << 48 - m.SpanId |= uint64(data[iNdEx-1]) << 56 - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Sampled", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowWire - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - v |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - m.Sampled = bool(v != 0) - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field BaggageItems", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowWire - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - msglen |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthWire - } - postIndex := iNdEx + msglen - if postIndex > l { - return io.ErrUnexpectedEOF - } - var keykey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowWire - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - keykey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - var stringLenmapkey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowWire - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - stringLenmapkey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapkey := int(stringLenmapkey) - if intStringLenmapkey < 0 { - return ErrInvalidLengthWire - } - postStringIndexmapkey := iNdEx + intStringLenmapkey - if postStringIndexmapkey > l { - return io.ErrUnexpectedEOF - } - mapkey := string(data[iNdEx:postStringIndexmapkey]) - iNdEx = postStringIndexmapkey - var valuekey uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowWire - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - valuekey |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - var stringLenmapvalue uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowWire - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - stringLenmapvalue |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - intStringLenmapvalue := int(stringLenmapvalue) - if intStringLenmapvalue < 0 { - return ErrInvalidLengthWire - } - postStringIndexmapvalue := iNdEx + intStringLenmapvalue - if postStringIndexmapvalue > l { - return io.ErrUnexpectedEOF - } - mapvalue := string(data[iNdEx:postStringIndexmapvalue]) - iNdEx = postStringIndexmapvalue - if m.BaggageItems == nil { - m.BaggageItems = make(map[string]string) - } - m.BaggageItems[mapkey] = mapvalue - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipWire(data[iNdEx:]) - if err != nil { - return err - } - if skippy < 0 { - return ErrInvalidLengthWire - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func skipWire(data []byte) (n int, err error) { - l := len(data) - iNdEx := 0 - for iNdEx < l { - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowWire - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - wire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - wireType := int(wire & 0x7) - switch wireType { - case 0: - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowWire - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - iNdEx++ - if data[iNdEx-1] < 0x80 { - break - } - } - return iNdEx, nil - case 1: - iNdEx += 8 - return iNdEx, nil - case 2: - var length int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowWire - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - length |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - iNdEx += length - if length < 0 { - return 0, ErrInvalidLengthWire - } - return iNdEx, nil - case 3: - for { - var innerWire uint64 - var start int = iNdEx - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return 0, ErrIntOverflowWire - } - if iNdEx >= l { - return 0, io.ErrUnexpectedEOF - } - b := data[iNdEx] - iNdEx++ - innerWire |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - innerWireType := int(innerWire & 0x7) - if innerWireType == 4 { - break - } - next, err := skipWire(data[start:]) - if err != nil { - return 0, err - } - iNdEx = start + next - } - return iNdEx, nil - case 4: - return iNdEx, nil - case 5: - iNdEx += 4 - return iNdEx, nil - default: - return 0, fmt.Errorf("proto: illegal wireType %d", wireType) - } - } - panic("unreachable") -} - -var ( - ErrInvalidLengthWire = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowWire = fmt.Errorf("proto: integer overflow") -) - -var fileDescriptorWire = []byte{ - // 234 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xe2, 0xe2, 0x2a, 0xcf, 0x2c, 0x4a, - 0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, 0x4e, 0x4a, 0x2c, 0xce, 0x4c, 0x2e, 0x29, 0x4a, - 0x4c, 0x4e, 0x2d, 0x8a, 0x4f, 0xcf, 0xd7, 0x03, 0x49, 0x29, 0x7d, 0x65, 0xe4, 0xe2, 0x0e, 0x01, - 0x0b, 0x05, 0x97, 0x24, 0x96, 0xa4, 0x0a, 0x49, 0x72, 0x71, 0x80, 0x55, 0xc4, 0x67, 0xa6, 0x48, - 0x30, 0x2a, 0x30, 0x6a, 0xb0, 0x05, 0xb1, 0x83, 0xf9, 0x9e, 0x29, 0x42, 0xe2, 0x5c, 0xec, 0xc5, - 0x05, 0x89, 0x79, 0x20, 0x19, 0x26, 0xb0, 0x0c, 0x1b, 0x88, 0x0b, 0x94, 0x90, 0x00, 0x4a, 0x24, - 0xe6, 0x16, 0xe4, 0xa4, 0xa6, 0x48, 0x30, 0x03, 0x25, 0x38, 0x82, 0x60, 0x5c, 0xa1, 0x70, 0x2e, - 0xde, 0xa4, 0xc4, 0xf4, 0xf4, 0xc4, 0x74, 0xa0, 0x79, 0x25, 0xa9, 0xb9, 0xc5, 0x12, 0x2c, 0x0a, - 0xcc, 0x1a, 0xdc, 0x46, 0x46, 0x7a, 0x58, 0x9c, 0xa2, 0x87, 0xe4, 0x0c, 0x3d, 0x27, 0x88, 0x2e, - 0x4f, 0x90, 0x26, 0xd7, 0xbc, 0x92, 0xa2, 0xca, 0x20, 0x9e, 0x24, 0x24, 0x21, 0x29, 0x7b, 0x2e, - 0x41, 0x0c, 0x25, 0x42, 0x02, 0x5c, 0xcc, 0xd9, 0xa9, 0x95, 0x60, 0x67, 0x73, 0x06, 0x81, 0x98, - 0x42, 0x22, 0x5c, 0xac, 0x65, 0x89, 0x39, 0xa5, 0xa9, 0x60, 0x07, 0x73, 0x06, 0x41, 0x38, 0x56, - 0x4c, 0x16, 0x8c, 0x4e, 0x62, 0x27, 0x1e, 0xc9, 0x31, 0x5e, 0x00, 0xe2, 0x07, 0x40, 0x3c, 0xe1, - 0xb1, 0x1c, 0x43, 0x14, 0x0b, 0xc8, 0x11, 0x49, 0x6c, 0xe0, 0xb0, 0x32, 0x06, 0x04, 0x00, 0x00, - 0xff, 0xff, 0x0a, 0x20, 0x89, 0x38, 0x39, 0x01, 0x00, 0x00, -} diff --git a/vendor/github.com/pingcap/kvproto/LICENSE b/vendor/github.com/pingcap/kvproto/LICENSE new file mode 100644 index 0000000000000..b67d9091009d1 --- /dev/null +++ b/vendor/github.com/pingcap/kvproto/LICENSE @@ -0,0 +1,201 @@ +Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {} + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. 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 f3876fc9e1c5b..5d35067351034 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/eraftpb/eraftpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/eraftpb/eraftpb.pb.go @@ -160,7 +160,10 @@ type Entry struct { Term uint64 `protobuf:"varint,2,opt,name=term,proto3" json:"term,omitempty"` Index uint64 `protobuf:"varint,3,opt,name=index,proto3" json:"index,omitempty"` Data []byte `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"` - SyncLog bool `protobuf:"varint,5,opt,name=sync_log,json=syncLog,proto3" json:"sync_log,omitempty"` + Context []byte `protobuf:"bytes,6,opt,name=context,proto3" json:"context,omitempty"` + // Deprecated! It is kept for backward compatibility. + // TODO: remove it in the next major release. + SyncLog bool `protobuf:"varint,5,opt,name=sync_log,json=syncLog,proto3" json:"sync_log,omitempty"` } func (m *Entry) Reset() { *m = Entry{} } @@ -196,6 +199,13 @@ func (m *Entry) GetData() []byte { return nil } +func (m *Entry) GetContext() []byte { + if m != nil { + return m.Context + } + return nil +} + func (m *Entry) GetSyncLog() bool { if m != nil { return m.SyncLog @@ -517,6 +527,12 @@ func (m *Entry) MarshalTo(dAtA []byte) (int, error) { } i++ } + if len(m.Context) > 0 { + dAtA[i] = 0x32 + i++ + i = encodeVarintEraftpb(dAtA, i, uint64(len(m.Context))) + i += copy(dAtA[i:], m.Context) + } return i, nil } @@ -858,6 +874,10 @@ func (m *Entry) Size() (n int) { if m.SyncLog { n += 2 } + l = len(m.Context) + if l > 0 { + n += 1 + l + sovEraftpb(uint64(l)) + } return n } @@ -1142,6 +1162,37 @@ func (m *Entry) Unmarshal(dAtA []byte) error { } } m.SyncLog = bool(v != 0) + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEraftpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthEraftpb + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Context = append(m.Context[:0], dAtA[iNdEx:postIndex]...) + if m.Context == nil { + m.Context = []byte{} + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipEraftpb(dAtA[iNdEx:]) @@ -2242,56 +2293,57 @@ var ( func init() { proto.RegisterFile("eraftpb.proto", fileDescriptorEraftpb) } var fileDescriptorEraftpb = []byte{ - // 815 bytes of a gzipped FileDescriptorProto + // 821 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x55, 0xcd, 0x6e, 0xe4, 0x44, 0x10, 0x8e, 0x3d, 0x3f, 0xf6, 0x94, 0x93, 0x49, 0xa7, 0x08, 0xbb, 0xce, 0x4a, 0x84, 0xd1, 0x9c, 0x46, 0x91, 0x58, 0x94, 0x20, 0x24, 0x2e, 0x1c, 0xb2, 0x11, 0x52, 0x56, 0x64, 0xa2, 0xc5, 0x9b, 0xe5, 0x3a, 0xea, 0xd8, 0x35, 0x1e, 0xc3, 0xd8, 0x6d, 0xba, 0x7b, 0x96, 0x9d, 0x47, 0x40, 0xe2, - 0xc6, 0x85, 0x47, 0xe2, 0xc8, 0x23, 0xa0, 0x70, 0xe0, 0x35, 0x50, 0xb7, 0x7f, 0xe2, 0xd9, 0xdc, - 0xea, 0xab, 0xae, 0xee, 0xfa, 0xea, 0xfb, 0xca, 0x32, 0x1c, 0x90, 0xe4, 0x4b, 0x5d, 0xde, 0xbf, - 0x2c, 0xa5, 0xd0, 0x02, 0xbd, 0x1a, 0x4e, 0xff, 0x70, 0x60, 0xf0, 0x5d, 0xa1, 0xe5, 0x16, 0xcf, - 0x01, 0xc8, 0x04, 0x0b, 0xbd, 0x2d, 0x29, 0x74, 0x26, 0xce, 0x6c, 0x7c, 0x81, 0x2f, 0x9b, 0x6b, - 0xb6, 0xe6, 0x6e, 0x5b, 0x52, 0x34, 0xa2, 0x26, 0x44, 0x84, 0xbe, 0x26, 0x99, 0x87, 0xee, 0xc4, - 0x99, 0xf5, 0x23, 0x1b, 0xe3, 0x31, 0x0c, 0xb2, 0x22, 0xa1, 0x0f, 0x61, 0xcf, 0x26, 0x2b, 0x60, - 0x2a, 0x13, 0xae, 0x79, 0xd8, 0x9f, 0x38, 0xb3, 0xfd, 0xc8, 0xc6, 0x78, 0x02, 0xbe, 0xda, 0x16, - 0xf1, 0x62, 0x2d, 0xd2, 0x70, 0x30, 0x71, 0x66, 0x7e, 0xe4, 0x19, 0x7c, 0x23, 0xd2, 0xa9, 0x00, - 0xf6, 0xb6, 0xe0, 0xa5, 0x5a, 0x09, 0x3d, 0x27, 0xcd, 0x6d, 0xf9, 0x39, 0x40, 0x2c, 0x8a, 0xe5, - 0x42, 0x69, 0xae, 0x2b, 0x7e, 0x41, 0x87, 0xdf, 0x95, 0x28, 0x96, 0x6f, 0xcd, 0x49, 0x34, 0x8a, - 0x9b, 0xf0, 0x91, 0x8b, 0xfb, 0x11, 0x17, 0xcb, 0xba, 0xf7, 0xc8, 0x7a, 0xfa, 0x0e, 0xfc, 0xa6, - 0x61, 0xcb, 0xd5, 0xe9, 0x70, 0xfd, 0x1a, 0xfc, 0xbc, 0x26, 0x62, 0x1f, 0x0b, 0x2e, 0x4e, 0xda, - 0xd6, 0x1f, 0x33, 0x8d, 0xda, 0xd2, 0xe9, 0x7f, 0x2e, 0x78, 0x73, 0x52, 0x8a, 0xa7, 0x84, 0x5f, - 0x82, 0x9f, 0xab, 0xb4, 0xab, 0xee, 0x71, 0xfb, 0x44, 0x5d, 0x63, 0xf5, 0xf5, 0x72, 0x95, 0x5a, - 0x75, 0xc7, 0xe0, 0x6a, 0x51, 0x53, 0x77, 0xb5, 0x30, 0xbc, 0x96, 0x52, 0xb4, 0xbc, 0x4d, 0xdc, - 0xce, 0xd2, 0xef, 0x38, 0x70, 0x02, 0xfe, 0x5a, 0xa4, 0x0b, 0x9b, 0x1f, 0xd8, 0xbc, 0xb7, 0x16, - 0xe9, 0xdd, 0x8e, 0x39, 0xc3, 0xae, 0x20, 0x33, 0xf0, 0x8c, 0xa7, 0x19, 0xa9, 0xd0, 0x9b, 0xf4, - 0x66, 0xc1, 0xc5, 0x78, 0xd7, 0xf6, 0xa8, 0x39, 0xc6, 0x67, 0x30, 0x8c, 0x45, 0x9e, 0x67, 0x3a, - 0xf4, 0xed, 0x03, 0x35, 0xc2, 0x2f, 0xc0, 0x57, 0xb5, 0x0a, 0xe1, 0xc8, 0xca, 0x73, 0xf4, 0x44, - 0x9e, 0xa8, 0x2d, 0x31, 0xcf, 0x48, 0xfa, 0x89, 0x62, 0x1d, 0x82, 0xf5, 0xbd, 0x46, 0xf8, 0x39, - 0x04, 0x55, 0xb4, 0x58, 0x65, 0x85, 0x0e, 0x03, 0xdb, 0x03, 0xaa, 0xd4, 0x75, 0x56, 0x68, 0x0c, - 0xc1, 0x8b, 0x45, 0xa1, 0xe9, 0x83, 0x0e, 0xf7, 0xad, 0x3b, 0x0d, 0x9c, 0x7e, 0x0f, 0xa3, 0x6b, - 0x2e, 0x93, 0xca, 0xf7, 0x46, 0x15, 0xa7, 0xa3, 0x0a, 0x42, 0xff, 0xbd, 0xd0, 0xd4, 0xec, 0xaa, - 0x89, 0x3b, 0xe3, 0xf4, 0xba, 0xe3, 0x4c, 0xbf, 0x85, 0xd1, 0x55, 0x77, 0x89, 0x0a, 0x91, 0x90, - 0x0a, 0x9d, 0x49, 0xcf, 0x68, 0x66, 0x01, 0xbe, 0x00, 0x7f, 0x4d, 0x5c, 0x16, 0x24, 0x55, 0xe8, - 0xda, 0x83, 0x16, 0x4f, 0x7f, 0x73, 0x00, 0xcc, 0xfd, 0xab, 0x15, 0x2f, 0x52, 0xeb, 0x63, 0x96, - 0xd4, 0x5c, 0xdc, 0x2c, 0xc1, 0x6f, 0x20, 0x88, 0xed, 0x49, 0xb5, 0x0b, 0xae, 0xdd, 0x85, 0xe7, - 0x3b, 0x9b, 0x5c, 0xdd, 0xb4, 0xeb, 0x00, 0x71, 0x1b, 0xe3, 0x73, 0xf0, 0x4c, 0xf7, 0x45, 0x96, - 0x34, 0x84, 0x0d, 0x7c, 0x9d, 0x74, 0x75, 0xe9, 0xef, 0xe8, 0x72, 0x76, 0x0e, 0xa3, 0xf6, 0xd3, - 0xc5, 0x43, 0x08, 0x2c, 0xb8, 0x15, 0x32, 0xe7, 0x6b, 0xb6, 0x87, 0x9f, 0xc0, 0xa1, 0x4d, 0x3c, - 0xf6, 0x64, 0xce, 0xd9, 0xef, 0x3d, 0x08, 0x3a, 0x0b, 0x89, 0x00, 0xc3, 0xb9, 0x4a, 0xaf, 0x37, - 0x25, 0xdb, 0xc3, 0x00, 0xbc, 0xb9, 0x4a, 0x5f, 0x11, 0xd7, 0xcc, 0xc1, 0x31, 0xc0, 0x5c, 0xa5, - 0x6f, 0xa4, 0x28, 0x85, 0x22, 0xe6, 0xe2, 0x01, 0x8c, 0xe6, 0x2a, 0xbd, 0x2c, 0x4b, 0x2a, 0x12, - 0xd6, 0xc3, 0x4f, 0xe1, 0xa8, 0x85, 0x11, 0xa9, 0x52, 0x14, 0x8a, 0x58, 0x1f, 0x11, 0xc6, 0x73, - 0x95, 0x46, 0xf4, 0xcb, 0x86, 0x94, 0xfe, 0x51, 0x68, 0x62, 0x03, 0x7c, 0x01, 0xcf, 0x76, 0x73, - 0x6d, 0xfd, 0xd0, 0x90, 0x9e, 0xab, 0xb4, 0xd9, 0x22, 0xe6, 0x21, 0x83, 0x7d, 0xc3, 0x87, 0xb8, - 0xd4, 0xf7, 0x86, 0x88, 0x8f, 0x21, 0x1c, 0x77, 0x33, 0xed, 0xe5, 0x51, 0xdd, 0xec, 0x5d, 0x21, - 0x89, 0xc7, 0x2b, 0x7e, 0xbf, 0x26, 0x06, 0x78, 0x04, 0x07, 0xf5, 0x83, 0xc6, 0xe0, 0x8d, 0x62, - 0x41, 0x5d, 0x76, 0xb5, 0xa2, 0xf8, 0xe7, 0x1f, 0x36, 0x42, 0x6e, 0x72, 0xb6, 0x5f, 0xd3, 0xbf, - 0x93, 0xbc, 0x50, 0x4b, 0x92, 0x37, 0xc4, 0x13, 0x92, 0xec, 0xa0, 0xbe, 0x7d, 0x97, 0xe5, 0x24, - 0x36, 0xfa, 0x56, 0xfc, 0xca, 0xc6, 0x35, 0xa1, 0x88, 0x78, 0xf2, 0xda, 0x7c, 0x4f, 0xec, 0x10, - 0x8f, 0x81, 0x75, 0x33, 0x86, 0x10, 0x63, 0xf5, 0x8b, 0xf5, 0x94, 0x6f, 0x24, 0xd9, 0xe1, 0x8f, - 0xf0, 0x33, 0x38, 0x79, 0x92, 0x6e, 0x47, 0xc0, 0xb3, 0x4b, 0x18, 0xef, 0xae, 0x84, 0x31, 0xe1, - 0x32, 0x49, 0x6e, 0x45, 0x42, 0x6c, 0xcf, 0x98, 0x10, 0x51, 0x2e, 0xde, 0x93, 0xc5, 0x8e, 0x19, - 0xe5, 0x32, 0x49, 0x6e, 0xaa, 0x5d, 0xb4, 0x39, 0xf7, 0x15, 0xfb, 0xeb, 0xe1, 0xd4, 0xf9, 0xfb, - 0xe1, 0xd4, 0xf9, 0xe7, 0xe1, 0xd4, 0xf9, 0xf3, 0xdf, 0xd3, 0xbd, 0xfb, 0xa1, 0xfd, 0x0d, 0x7c, - 0xf5, 0x7f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xf4, 0xcf, 0x7f, 0x2b, 0x17, 0x06, 0x00, 0x00, + 0x01, 0x78, 0x09, 0xde, 0x83, 0x23, 0x8f, 0x80, 0xc2, 0x81, 0xd7, 0x40, 0xdd, 0xfe, 0x89, 0x67, + 0x73, 0xab, 0xaf, 0xba, 0xba, 0xfa, 0xab, 0xaf, 0x3e, 0xcb, 0x70, 0x40, 0x92, 0x2f, 0x75, 0x79, + 0xff, 0xb2, 0x94, 0x42, 0x0b, 0xf4, 0x6a, 0x38, 0xfd, 0xd3, 0x81, 0xc1, 0x77, 0x85, 0x96, 0x5b, + 0x3c, 0x07, 0x20, 0x13, 0x2c, 0xf4, 0xb6, 0xa4, 0xd0, 0x99, 0x38, 0xb3, 0xf1, 0x05, 0xbe, 0x6c, + 0xae, 0xd9, 0x9a, 0xbb, 0x6d, 0x49, 0xd1, 0x88, 0x9a, 0x10, 0x11, 0xfa, 0x9a, 0x64, 0x1e, 0xba, + 0x13, 0x67, 0xd6, 0x8f, 0x6c, 0x8c, 0xc7, 0x30, 0xc8, 0x8a, 0x84, 0x3e, 0x84, 0x3d, 0x9b, 0xac, + 0x80, 0xa9, 0x4c, 0xb8, 0xe6, 0x61, 0x7f, 0xe2, 0xcc, 0xf6, 0x23, 0x1b, 0xe3, 0x09, 0xf8, 0x6a, + 0x5b, 0xc4, 0x8b, 0xb5, 0x48, 0xc3, 0xc1, 0xc4, 0x99, 0xf9, 0x91, 0x67, 0xf0, 0x8d, 0x48, 0x31, + 0x04, 0x2f, 0x16, 0x85, 0xa6, 0x0f, 0x3a, 0x1c, 0xda, 0x1b, 0x0d, 0x9c, 0x0a, 0x60, 0x6f, 0x0b, + 0x5e, 0xaa, 0x95, 0xd0, 0x73, 0xd2, 0xdc, 0x36, 0x3a, 0x07, 0x88, 0x45, 0xb1, 0x5c, 0x28, 0xcd, + 0x75, 0xc5, 0x3c, 0xe8, 0x30, 0xbf, 0x12, 0xc5, 0xf2, 0xad, 0x39, 0x89, 0x46, 0x71, 0x13, 0x3e, + 0xb2, 0x74, 0x3f, 0x62, 0x69, 0xe7, 0xe9, 0x3d, 0xce, 0x33, 0x7d, 0x07, 0x7e, 0xf3, 0x60, 0x3b, + 0x85, 0xd3, 0x99, 0xe2, 0x6b, 0xf0, 0xf3, 0x9a, 0x88, 0x6d, 0x16, 0x5c, 0x9c, 0xb4, 0x4f, 0x7f, + 0xcc, 0x34, 0x6a, 0x4b, 0xa7, 0xff, 0xb9, 0xe0, 0xcd, 0x49, 0x29, 0x9e, 0x12, 0x7e, 0x09, 0x7e, + 0xae, 0xd2, 0xae, 0xee, 0xc7, 0x6d, 0x8b, 0xba, 0xc6, 0x2a, 0xef, 0xe5, 0x2a, 0xb5, 0xba, 0x8f, + 0xc1, 0xd5, 0xa2, 0xa6, 0xee, 0x6a, 0x61, 0x78, 0x2d, 0xa5, 0x68, 0x79, 0x9b, 0xb8, 0x9d, 0xa5, + 0xdf, 0xd9, 0xcd, 0x09, 0xf8, 0x6b, 0x91, 0x2e, 0x6c, 0x7e, 0x60, 0xf3, 0xde, 0x5a, 0xa4, 0x77, + 0x3b, 0x6b, 0x1b, 0x76, 0x05, 0x99, 0x81, 0x67, 0xb6, 0x9d, 0x91, 0x0a, 0xbd, 0x49, 0x6f, 0x16, + 0x5c, 0x8c, 0x77, 0x0d, 0x11, 0x35, 0xc7, 0xf8, 0x0c, 0x86, 0xb1, 0xc8, 0xf3, 0x4c, 0x87, 0xbe, + 0x6d, 0x50, 0x23, 0xfc, 0x02, 0x7c, 0x55, 0xab, 0x10, 0x8e, 0xac, 0x3c, 0x47, 0x4f, 0xe4, 0x89, + 0xda, 0x12, 0xd3, 0x46, 0xd2, 0x4f, 0x14, 0xeb, 0x10, 0xac, 0x23, 0x6a, 0x84, 0x9f, 0x43, 0x50, + 0x45, 0x8b, 0x55, 0x56, 0xe8, 0x30, 0xb0, 0x6f, 0x40, 0x95, 0xba, 0xce, 0x0a, 0xdd, 0x75, 0xcc, + 0xfe, 0xae, 0x63, 0xbe, 0x87, 0xd1, 0x35, 0x97, 0x49, 0xb5, 0xf7, 0x46, 0x15, 0xa7, 0xa3, 0x0a, + 0x42, 0xff, 0xbd, 0xd0, 0xd4, 0xb8, 0xd8, 0xc4, 0x9d, 0x71, 0x7a, 0xdd, 0x71, 0xa6, 0xdf, 0xc2, + 0xe8, 0xaa, 0x6b, 0xa2, 0x42, 0x24, 0xa4, 0x42, 0x67, 0xd2, 0x33, 0x9a, 0x59, 0x80, 0x2f, 0xc0, + 0x5f, 0x13, 0x97, 0x05, 0x49, 0x15, 0xba, 0xf6, 0xa0, 0xc5, 0xd3, 0xdf, 0x1c, 0x00, 0x73, 0xff, + 0x6a, 0xc5, 0x8b, 0xd4, 0xee, 0x31, 0x4b, 0x6a, 0x2e, 0x6e, 0x96, 0xe0, 0x37, 0x10, 0xc4, 0xf6, + 0xa4, 0xf2, 0x82, 0x6b, 0xbd, 0xf0, 0x7c, 0xc7, 0xc9, 0xd5, 0x4d, 0x6b, 0x07, 0x88, 0xdb, 0x18, + 0x9f, 0x83, 0x67, 0x5e, 0x5f, 0x64, 0x49, 0x43, 0xd8, 0xc0, 0xd7, 0x49, 0x57, 0x97, 0xfe, 0x8e, + 0x2e, 0x67, 0xe7, 0x30, 0x6a, 0x3f, 0x6a, 0x3c, 0x84, 0xc0, 0x82, 0x5b, 0x21, 0x73, 0xbe, 0x66, + 0x7b, 0xf8, 0x09, 0x1c, 0xda, 0xc4, 0xe3, 0x9b, 0xcc, 0x39, 0xfb, 0xbd, 0x07, 0x41, 0xc7, 0x90, + 0x08, 0x30, 0x9c, 0xab, 0xf4, 0x7a, 0x53, 0xb2, 0x3d, 0x0c, 0xc0, 0x9b, 0xab, 0xf4, 0x15, 0x71, + 0xcd, 0x1c, 0x1c, 0x03, 0xcc, 0x55, 0xfa, 0x46, 0x8a, 0x52, 0x28, 0x62, 0x2e, 0x1e, 0xc0, 0x68, + 0xae, 0xd2, 0xcb, 0xb2, 0xa4, 0x22, 0x61, 0x3d, 0xfc, 0x14, 0x8e, 0x5a, 0x18, 0x91, 0x2a, 0x45, + 0xa1, 0x88, 0xf5, 0x11, 0x61, 0x3c, 0x57, 0x69, 0x44, 0xbf, 0x6c, 0x48, 0xe9, 0x1f, 0x85, 0x26, + 0x36, 0xc0, 0x17, 0xf0, 0x6c, 0x37, 0xd7, 0xd6, 0x0f, 0x0d, 0xe9, 0xb9, 0x4a, 0x1b, 0x17, 0x31, + 0x0f, 0x19, 0xec, 0x1b, 0x3e, 0xc4, 0xa5, 0xbe, 0x37, 0x44, 0x7c, 0x0c, 0xe1, 0xb8, 0x9b, 0x69, + 0x2f, 0x8f, 0xea, 0xc7, 0xde, 0x15, 0x92, 0x78, 0xbc, 0xe2, 0xf7, 0x6b, 0x62, 0x80, 0x47, 0x70, + 0x50, 0x37, 0x34, 0x0b, 0xde, 0x28, 0x16, 0xd4, 0x65, 0x57, 0x2b, 0x8a, 0x7f, 0xfe, 0x61, 0x23, + 0xe4, 0x26, 0x67, 0xfb, 0x35, 0xfd, 0x3b, 0xc9, 0x0b, 0xb5, 0x24, 0x79, 0x43, 0x3c, 0x21, 0xc9, + 0x0e, 0xea, 0xdb, 0x77, 0x59, 0x4e, 0x62, 0xa3, 0x6f, 0xc5, 0xaf, 0x6c, 0x5c, 0x13, 0x8a, 0x88, + 0x27, 0xaf, 0xcd, 0xf7, 0xc4, 0x0e, 0xf1, 0x18, 0x58, 0x37, 0x63, 0x08, 0x31, 0x56, 0x77, 0xac, + 0xa7, 0x7c, 0x23, 0xc9, 0x0e, 0x7f, 0x84, 0x9f, 0xc1, 0xc9, 0x93, 0x74, 0x3b, 0x02, 0x9e, 0x5d, + 0xc2, 0x78, 0xd7, 0x12, 0x66, 0x09, 0x97, 0x49, 0x72, 0x2b, 0x12, 0x62, 0x7b, 0x66, 0x09, 0x11, + 0xe5, 0xe2, 0x3d, 0x59, 0xec, 0x98, 0x51, 0x2e, 0x93, 0xe4, 0xa6, 0xf2, 0xa2, 0xcd, 0xb9, 0xaf, + 0xd8, 0x5f, 0x0f, 0xa7, 0xce, 0xdf, 0x0f, 0xa7, 0xce, 0x3f, 0x0f, 0xa7, 0xce, 0x1f, 0xff, 0x9e, + 0xee, 0xdd, 0x0f, 0xed, 0x0f, 0xe2, 0xab, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xcc, 0xa3, 0xf2, + 0xc5, 0x31, 0x06, 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 43430cc6ec573..b3a7a1c1e0a0a 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/kvrpcpb/kvrpcpb.pb.go @@ -1,5 +1,6 @@ -// Code generated by protoc-gen-gogo. DO NOT EDIT. +// Code generated by protoc-gen-gogo. // source: kvrpcpb.proto +// DO NOT EDIT! /* Package kvrpcpb is a generated protocol buffer package. @@ -552,6 +553,7 @@ type ScanRequest struct { Limit uint32 `protobuf:"varint,3,opt,name=limit,proto3" json:"limit,omitempty"` Version uint64 `protobuf:"varint,4,opt,name=version,proto3" json:"version,omitempty"` KeyOnly bool `protobuf:"varint,5,opt,name=key_only,json=keyOnly,proto3" json:"key_only,omitempty"` + Reverse bool `protobuf:"varint,6,opt,name=reverse,proto3" json:"reverse,omitempty"` } func (m *ScanRequest) Reset() { *m = ScanRequest{} } @@ -594,6 +596,13 @@ func (m *ScanRequest) GetKeyOnly() bool { return false } +func (m *ScanRequest) GetReverse() bool { + if m != nil { + return m.Reverse + } + return false +} + type KvPair struct { Error *KeyError `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` @@ -2785,6 +2794,16 @@ func (m *ScanRequest) MarshalTo(dAtA []byte) (int, error) { } i++ } + if m.Reverse { + dAtA[i] = 0x30 + i++ + if m.Reverse { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } return i, nil } @@ -4940,6 +4959,24 @@ func (m *SplitRegionResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func encodeFixed64Kvrpcpb(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + dAtA[offset+4] = uint8(v >> 32) + dAtA[offset+5] = uint8(v >> 40) + dAtA[offset+6] = uint8(v >> 48) + dAtA[offset+7] = uint8(v >> 56) + return offset + 8 +} +func encodeFixed32Kvrpcpb(dAtA []byte, offset int, v uint32) int { + dAtA[offset] = uint8(v) + dAtA[offset+1] = uint8(v >> 8) + dAtA[offset+2] = uint8(v >> 16) + dAtA[offset+3] = uint8(v >> 24) + return offset + 4 +} func encodeVarintKvrpcpb(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -5160,6 +5197,9 @@ func (m *ScanRequest) Size() (n int) { if m.KeyOnly { n += 2 } + if m.Reverse { + n += 2 + } return n } @@ -7717,6 +7757,26 @@ func (m *ScanRequest) Unmarshal(dAtA []byte) error { } } m.KeyOnly = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Reverse", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowKvrpcpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Reverse = bool(v != 0) default: iNdEx = preIndex skippy, err := skipKvrpcpb(dAtA[iNdEx:]) @@ -14834,131 +14894,131 @@ var ( func init() { proto.RegisterFile("kvrpcpb.proto", fileDescriptorKvrpcpb) } var fileDescriptorKvrpcpb = []byte{ - // 2003 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0x5f, 0x6f, 0xdb, 0xc8, - 0x11, 0x0f, 0x29, 0x4a, 0xa2, 0x46, 0xb2, 0xac, 0x5b, 0x3b, 0x89, 0x7a, 0xe9, 0x25, 0x3e, 0xf6, - 0x82, 0x38, 0x2e, 0xea, 0xa0, 0xbe, 0x43, 0x9f, 0x8a, 0xe2, 0x10, 0x25, 0xcd, 0xe5, 0xe2, 0x34, - 0xc6, 0xda, 0x48, 0x11, 0xa0, 0x57, 0xdd, 0x9a, 0x5a, 0x4b, 0x84, 0x28, 0x2e, 0x43, 0xae, 0x64, - 0x0b, 0x45, 0x51, 0x14, 0xc5, 0x15, 0xb8, 0xc7, 0x16, 0x05, 0x5a, 0x14, 0xbd, 0x87, 0x3e, 0xf6, - 0x0b, 0xf4, 0xa5, 0x5f, 0xa0, 0x8f, 0xfd, 0x08, 0x45, 0xfa, 0x45, 0x8a, 0xfd, 0x43, 0x52, 0x94, - 0xe4, 0x8b, 0xcb, 0xca, 0xbe, 0x27, 0x69, 0x67, 0x66, 0x77, 0xe6, 0x37, 0x33, 0x3b, 0xb3, 0xdc, - 0x85, 0xb5, 0xe1, 0x24, 0x0a, 0xdd, 0xf0, 0x78, 0x37, 0x8c, 0x18, 0x67, 0xa8, 0xaa, 0x87, 0xef, - 0x36, 0x46, 0x94, 0x93, 0x84, 0xfc, 0xee, 0x1a, 0x8d, 0x22, 0x16, 0xa5, 0xc3, 0xcd, 0x3e, 0xeb, - 0x33, 0xf9, 0xf7, 0x81, 0xf8, 0xa7, 0xa8, 0xce, 0x2f, 0xc1, 0xde, 0x67, 0xee, 0xf0, 0x69, 0x70, - 0xc2, 0xd0, 0xfb, 0xd0, 0x08, 0x23, 0x6f, 0x44, 0xa2, 0x69, 0xd7, 0x67, 0xee, 0xb0, 0x6d, 0x6c, - 0x19, 0xdb, 0x0d, 0x5c, 0xd7, 0x34, 0x21, 0x26, 0x44, 0x04, 0xab, 0x3b, 0xa1, 0x51, 0xec, 0xb1, - 0xa0, 0x6d, 0x6e, 0x19, 0xdb, 0x16, 0xae, 0x0b, 0xda, 0x4b, 0x45, 0x42, 0x2d, 0x28, 0x0d, 0xe9, - 0xb4, 0x5d, 0x92, 0x93, 0xc5, 0x5f, 0xf4, 0x2d, 0xb0, 0xe5, 0x24, 0xce, 0xfd, 0xb6, 0x25, 0x27, - 0x54, 0xc5, 0xf8, 0x88, 0xfb, 0xce, 0x57, 0x06, 0xd8, 0xcf, 0xe8, 0xf4, 0xb1, 0xb0, 0x14, 0xdd, - 0x87, 0x8a, 0xa0, 0xd3, 0x9e, 0xd4, 0x5c, 0xdf, 0x7b, 0x67, 0x37, 0xc1, 0x99, 0x98, 0x88, 0xb5, - 0x00, 0xfa, 0x36, 0xd4, 0x22, 0xca, 0xa3, 0x29, 0x39, 0xf6, 0xa9, 0x34, 0xa2, 0x86, 0x33, 0x02, - 0xda, 0x84, 0x32, 0x39, 0x66, 0x11, 0x97, 0x46, 0xd4, 0xb0, 0x1a, 0xa0, 0x3d, 0xb0, 0x5d, 0x16, - 0x9c, 0xf8, 0x9e, 0xcb, 0xa5, 0x19, 0xf5, 0xbd, 0x1b, 0xa9, 0x82, 0x9f, 0x46, 0x1e, 0xa7, 0x1d, - 0xcd, 0xc5, 0xa9, 0x9c, 0x73, 0x0a, 0x6b, 0x39, 0x96, 0xc0, 0x12, 0x73, 0x12, 0xf1, 0x2e, 0x8f, - 0xa5, 0x95, 0x16, 0xae, 0xca, 0xf1, 0x51, 0x8c, 0xee, 0x40, 0x3d, 0x99, 0x27, 0xb8, 0xca, 0x35, - 0x90, 0x90, 0x8e, 0xe2, 0x25, 0x9e, 0x69, 0x43, 0x55, 0x7b, 0x57, 0x5a, 0xd4, 0xc0, 0xc9, 0xd0, - 0xf9, 0xaa, 0x04, 0xd5, 0x0e, 0x0b, 0x38, 0x3d, 0xe3, 0xe8, 0x96, 0x00, 0xdb, 0xf7, 0x58, 0xd0, - 0xf5, 0x7a, 0x5a, 0xa9, 0xad, 0x08, 0x4f, 0x7b, 0xe8, 0x07, 0xd0, 0xd0, 0x4c, 0x1a, 0x32, 0x77, - 0x20, 0xd5, 0xd6, 0xf7, 0x36, 0x76, 0x75, 0x2a, 0x60, 0xc9, 0x7b, 0x2c, 0x58, 0xb8, 0x1e, 0x65, - 0x03, 0xb4, 0x05, 0x56, 0x48, 0x69, 0x24, 0xad, 0xa9, 0xef, 0x35, 0x12, 0xf9, 0x03, 0x4a, 0x23, - 0x2c, 0x39, 0x08, 0x81, 0xc5, 0x69, 0x34, 0x6a, 0x97, 0xa5, 0x46, 0xf9, 0x1f, 0x3d, 0x00, 0x3b, - 0x8c, 0x3c, 0x16, 0x79, 0x7c, 0xda, 0xae, 0x6c, 0x19, 0xdb, 0xcd, 0xbd, 0x8d, 0xd4, 0x87, 0x1d, - 0x36, 0x1a, 0x91, 0xa0, 0x77, 0x10, 0x79, 0x38, 0x15, 0x42, 0x1f, 0xc3, 0xba, 0x17, 0x33, 0x9f, - 0x70, 0x61, 0xa1, 0x4f, 0x27, 0xd4, 0x6f, 0x57, 0xe5, 0xbc, 0x9b, 0xe9, 0xbc, 0xa7, 0x09, 0x7f, - 0x5f, 0xb0, 0x71, 0xd3, 0xcb, 0x8d, 0xd1, 0x07, 0xd0, 0x0c, 0x18, 0xef, 0x9e, 0x78, 0xbe, 0xdf, - 0x75, 0x89, 0x3b, 0xa0, 0x6d, 0x7b, 0xcb, 0xd8, 0xb6, 0x71, 0x23, 0x60, 0xfc, 0xc7, 0x9e, 0xef, - 0x77, 0x04, 0x4d, 0xc6, 0x65, 0x1a, 0xb8, 0x5d, 0x9f, 0xf5, 0xdb, 0x35, 0xc9, 0xaf, 0x8a, 0xf1, - 0x3e, 0xeb, 0x8b, 0xb8, 0x0c, 0x48, 0xd0, 0xf3, 0x69, 0x97, 0x7b, 0x23, 0xda, 0x06, 0xc9, 0x05, - 0x45, 0x3a, 0xf2, 0x46, 0x54, 0x08, 0xc4, 0x2e, 0x09, 0xba, 0x3d, 0xca, 0x89, 0xe7, 0xb7, 0xeb, - 0x4a, 0x40, 0x90, 0x1e, 0x49, 0xca, 0xa7, 0x96, 0x6d, 0xb5, 0xca, 0xc2, 0x7d, 0xa4, 0xd7, 0x7d, - 0x3d, 0x66, 0xd1, 0x78, 0xe4, 0x3c, 0x02, 0xf8, 0x24, 0x5b, 0xe1, 0x26, 0x54, 0x4f, 0x89, 0xc7, - 0xbb, 0x23, 0x95, 0x14, 0x25, 0x5c, 0x11, 0xc3, 0xe7, 0x31, 0x7a, 0x0f, 0x20, 0x8c, 0x98, 0x4b, - 0xe3, 0x58, 0xf0, 0x4c, 0xc9, 0xab, 0x69, 0xca, 0xf3, 0xd8, 0xf9, 0x11, 0xd8, 0x87, 0x2e, 0x09, - 0xe4, 0xee, 0xdb, 0x84, 0x32, 0x67, 0x9c, 0xf8, 0x7a, 0x05, 0x35, 0x10, 0x89, 0xae, 0xc5, 0x69, - 0x6f, 0x6e, 0x3e, 0xed, 0x39, 0xbf, 0x31, 0x00, 0x0e, 0x53, 0x3b, 0xd1, 0x3d, 0x28, 0x9f, 0x8a, - 0x6c, 0x5d, 0xd8, 0x3f, 0x89, 0x12, 0xac, 0xf8, 0xe8, 0x2e, 0x58, 0x72, 0x87, 0x9b, 0xe7, 0xc9, - 0x49, 0xb6, 0x10, 0xeb, 0x11, 0x4e, 0x74, 0x8e, 0x2c, 0x13, 0x13, 0x6c, 0x67, 0x0a, 0xf5, 0xc7, - 0x67, 0xd4, 0x55, 0x46, 0xc4, 0xe8, 0xa3, 0xbc, 0xbf, 0x0d, 0x9d, 0x90, 0xc9, 0xe4, 0xcc, 0x6d, - 0xb9, 0x20, 0x7c, 0x94, 0x0f, 0x82, 0x39, 0x37, 0x2b, 0x43, 0x39, 0x1b, 0x19, 0xa7, 0x07, 0xf0, - 0x84, 0x72, 0x4c, 0x5f, 0x8f, 0x69, 0xcc, 0xd1, 0x0e, 0x54, 0x5d, 0xb5, 0x67, 0xb4, 0xd6, 0xd6, - 0x4c, 0x72, 0x4a, 0x3a, 0x4e, 0x04, 0x92, 0xcd, 0x68, 0xe6, 0x36, 0x63, 0x52, 0xd6, 0x4a, 0x6a, - 0x67, 0xeb, 0xa1, 0xf3, 0x2b, 0xa8, 0x4b, 0x2d, 0x71, 0xc8, 0x82, 0x98, 0xa2, 0xef, 0x67, 0x5b, - 0x4e, 0xd4, 0x2d, 0xad, 0xab, 0xb9, 0x9b, 0xd4, 0x5b, 0x59, 0xcd, 0xd2, 0xdd, 0x26, 0x4b, 0xdb, - 0x3d, 0x28, 0x2b, 0xd9, 0x79, 0x8f, 0x27, 0xc5, 0x0f, 0x2b, 0xbe, 0xc8, 0x82, 0x09, 0xf1, 0xc7, - 0x54, 0x57, 0x09, 0x35, 0x70, 0xfe, 0x6a, 0x40, 0x5d, 0x78, 0xa0, 0x08, 0xd0, 0x5b, 0x50, 0x53, - 0x15, 0x2b, 0x83, 0xab, 0x4a, 0xd8, 0x33, 0x3a, 0x15, 0xea, 0x7c, 0x6f, 0xe4, 0xa9, 0x4a, 0xb9, - 0x86, 0xd5, 0x60, 0xd6, 0x13, 0x56, 0xce, 0x13, 0x62, 0x9b, 0x0d, 0xe9, 0xb4, 0xcb, 0x02, 0x7f, - 0x2a, 0xeb, 0x82, 0x8d, 0xab, 0x43, 0x3a, 0x7d, 0x11, 0xf8, 0x53, 0xe7, 0x15, 0x54, 0x9e, 0x4d, - 0x0e, 0x88, 0x37, 0x03, 0xd6, 0x78, 0x0b, 0xd8, 0xc5, 0x18, 0x2c, 0x87, 0x3f, 0x80, 0x86, 0x42, - 0x5f, 0x3c, 0x00, 0x77, 0xa1, 0x1c, 0x12, 0x2f, 0x12, 0x7b, 0xb0, 0xb4, 0x5d, 0xdf, 0x5b, 0xcf, - 0x6c, 0x92, 0x36, 0x63, 0xc5, 0x75, 0x5e, 0x80, 0xfd, 0x7c, 0xcc, 0x65, 0xf5, 0x41, 0xb7, 0xc0, - 0x64, 0xa1, 0x5c, 0xbb, 0xb9, 0x57, 0x4f, 0xe5, 0x5f, 0x84, 0xd8, 0x64, 0xe1, 0x85, 0x4d, 0xff, - 0xc2, 0x84, 0xf5, 0x83, 0x88, 0xca, 0x6d, 0x57, 0x24, 0x7a, 0x0f, 0xa0, 0x36, 0xd2, 0x06, 0x25, - 0xb6, 0x67, 0xfe, 0x4c, 0x4c, 0xc5, 0x99, 0xcc, 0x42, 0x13, 0x2f, 0x2d, 0x36, 0xf1, 0xef, 0xc0, - 0x9a, 0xca, 0x88, 0x7c, 0x90, 0x1b, 0x92, 0xf8, 0x32, 0x8b, 0x74, 0xda, 0xb4, 0xcb, 0xb9, 0xa6, - 0x8d, 0xf6, 0xe0, 0x7a, 0x3c, 0xf4, 0xc2, 0xae, 0xcb, 0x82, 0x98, 0x47, 0xc4, 0x0b, 0x78, 0xd7, - 0x1d, 0x50, 0x77, 0x28, 0x3b, 0x82, 0x8d, 0x37, 0x04, 0xb3, 0x93, 0xf2, 0x3a, 0x82, 0xe5, 0x84, - 0xd0, 0xca, 0xdc, 0x50, 0x3c, 0x8c, 0xf7, 0xa1, 0x22, 0xb9, 0x8b, 0xbe, 0x48, 0x73, 0x4b, 0x0b, - 0x38, 0x7f, 0x33, 0x60, 0x4d, 0xb4, 0x24, 0xaf, 0x50, 0x79, 0x58, 0xf0, 0x91, 0xb9, 0xc4, 0x47, - 0x08, 0xac, 0x21, 0x9d, 0xc6, 0xed, 0xd2, 0x56, 0x69, 0xbb, 0x81, 0xe5, 0x7f, 0x74, 0x17, 0x9a, - 0xae, 0xd4, 0x3a, 0xe7, 0xdd, 0x35, 0x45, 0xd5, 0x53, 0x3f, 0xb5, 0xec, 0x72, 0xab, 0x82, 0x2b, - 0xc7, 0x5e, 0xe0, 0xb3, 0xbe, 0xe3, 0x43, 0x33, 0x31, 0xf5, 0xf2, 0x6b, 0x8c, 0xd3, 0x87, 0xb5, - 0xa7, 0xa3, 0x90, 0x45, 0xa9, 0x63, 0x72, 0x49, 0x66, 0x5c, 0x20, 0xc9, 0x16, 0x41, 0x9a, 0x4b, - 0x40, 0x3a, 0xaf, 0xa0, 0x99, 0x28, 0x2a, 0x0e, 0x6b, 0x73, 0x16, 0x56, 0x2d, 0xc1, 0xf0, 0x0b, - 0xd8, 0x7c, 0x48, 0xb8, 0x3b, 0xc0, 0xcc, 0xf7, 0x8f, 0x89, 0x3b, 0xbc, 0xca, 0x18, 0x3b, 0x31, - 0x5c, 0x9f, 0x53, 0x7e, 0x05, 0x51, 0x8b, 0xa1, 0xd9, 0xf1, 0x29, 0x09, 0xc6, 0xe1, 0x6a, 0xda, - 0xdd, 0x02, 0xfa, 0xd2, 0x22, 0x7a, 0xe7, 0x0f, 0x06, 0xac, 0xa7, 0x5a, 0xaf, 0xa0, 0xfd, 0x2d, - 0x26, 0x56, 0x69, 0x59, 0x62, 0x0d, 0x61, 0x5d, 0x06, 0xa0, 0x60, 0xef, 0x4f, 0x62, 0x6a, 0xce, - 0xec, 0xdb, 0xf3, 0xbb, 0xbf, 0x0f, 0xad, 0x4c, 0xd9, 0xa5, 0x77, 0xa0, 0xdf, 0x19, 0xb0, 0x2e, - 0x9a, 0x9d, 0xa8, 0xd4, 0x45, 0xb0, 0xdd, 0x81, 0xfa, 0x88, 0x9c, 0xcd, 0xa5, 0x34, 0x8c, 0xc8, - 0x59, 0x92, 0xd0, 0xb9, 0xf3, 0x40, 0xe9, 0xbc, 0xf3, 0x80, 0x35, 0x73, 0x1e, 0x70, 0xfe, 0x68, - 0x40, 0x2b, 0xb3, 0xe9, 0x0a, 0xd2, 0xe0, 0x1e, 0x94, 0x45, 0xb3, 0x51, 0xbb, 0x6e, 0xe9, 0x87, - 0xa0, 0xe2, 0x3b, 0x1f, 0x42, 0xf5, 0xe8, 0x4c, 0x9d, 0x9f, 0x5b, 0x50, 0xe2, 0x67, 0x81, 0xfe, - 0x3e, 0x12, 0x7f, 0xd1, 0x0d, 0xa8, 0xc4, 0x9c, 0xf0, 0x71, 0xf2, 0x2d, 0xa6, 0x47, 0xce, 0x3f, - 0x0c, 0x40, 0x98, 0xc6, 0xcc, 0x9f, 0xd0, 0xa2, 0x5e, 0xbe, 0x50, 0xe9, 0xb8, 0x58, 0x32, 0xa3, - 0xef, 0x41, 0x8d, 0x9f, 0x05, 0x5d, 0x2f, 0x38, 0x61, 0x71, 0xdb, 0x92, 0x80, 0x33, 0xcd, 0x1a, - 0x1d, 0xb6, 0xb9, 0xfa, 0x13, 0x3b, 0xaf, 0x61, 0x23, 0x67, 0xfc, 0x15, 0x94, 0x9e, 0x97, 0x50, - 0x7b, 0xd2, 0x29, 0xe2, 0xa6, 0xf7, 0x00, 0x62, 0x72, 0x42, 0xbb, 0x21, 0xf3, 0x02, 0xae, 0x7d, - 0x54, 0x13, 0x94, 0x03, 0x41, 0x70, 0x06, 0x00, 0x62, 0xdd, 0x2b, 0x40, 0xf0, 0x19, 0xac, 0x61, - 0x72, 0xba, 0xb2, 0x4f, 0x85, 0x26, 0x98, 0xee, 0x89, 0xbe, 0x5d, 0x30, 0xdd, 0x13, 0x87, 0x41, - 0x33, 0x59, 0x7e, 0xc5, 0x8d, 0xee, 0x9c, 0x63, 0x65, 0x2c, 0xf1, 0x1c, 0x8c, 0x57, 0x84, 0x67, - 0xa9, 0x12, 0x8d, 0xd2, 0x4a, 0x51, 0xbe, 0x92, 0x28, 0xa5, 0xd2, 0x55, 0xb7, 0xf3, 0x53, 0x40, - 0x98, 0x9c, 0xca, 0x32, 0x5b, 0x10, 0xd4, 0xc5, 0xca, 0xeb, 0x42, 0xe4, 0x7e, 0x0e, 0x1b, 0x39, - 0xc5, 0xab, 0x06, 0xd6, 0xcb, 0x80, 0xad, 0xb0, 0x59, 0x2d, 0xe6, 0xdf, 0x46, 0x4e, 0xcb, 0xa5, - 0x77, 0xa9, 0xcf, 0xa1, 0x85, 0xc9, 0xe9, 0x23, 0xea, 0xd3, 0x62, 0x9f, 0x35, 0x6f, 0xdf, 0x52, - 0x3f, 0x83, 0x77, 0x66, 0x34, 0xac, 0x3a, 0x2c, 0x7d, 0xb8, 0x9e, 0x38, 0xac, 0x38, 0x88, 0x8b, - 0x44, 0x86, 0xc0, 0x8d, 0x79, 0x45, 0xab, 0xc6, 0x32, 0x01, 0xa4, 0x97, 0x26, 0x41, 0x9f, 0xae, - 0xfc, 0x8a, 0xe0, 0x26, 0x54, 0x69, 0xd0, 0x9b, 0x39, 0x2d, 0x54, 0x68, 0xd0, 0x7b, 0x46, 0xa7, - 0x62, 0xeb, 0xe4, 0xf4, 0xae, 0x1a, 0xd7, 0x97, 0x86, 0x0c, 0xd2, 0x37, 0x81, 0x6d, 0xa1, 0xf4, - 0xa9, 0x30, 0x5e, 0x2a, 0xdc, 0x3f, 0x1b, 0xb2, 0xbc, 0x5e, 0xe1, 0x35, 0xcf, 0xec, 0x65, 0x8e, - 0x95, 0xbb, 0xcc, 0xd1, 0xf8, 0xcb, 0x29, 0xfe, 0x3e, 0xac, 0xa7, 0xb6, 0x15, 0x07, 0xfe, 0x3e, - 0x94, 0x86, 0x93, 0x73, 0x4b, 0x8b, 0xe0, 0x39, 0x1f, 0xcb, 0xf7, 0x00, 0xe9, 0xe2, 0x3c, 0x24, - 0xe3, 0xfc, 0xd0, 0x99, 0xb9, 0xb4, 0xfc, 0xbb, 0x91, 0x15, 0xc3, 0xa2, 0xce, 0xbc, 0x0f, 0x95, - 0x48, 0x98, 0xb0, 0xf4, 0x9a, 0x41, 0xc5, 0x5f, 0x0b, 0x88, 0x23, 0x0e, 0x25, 0xee, 0xa0, 0x3b, - 0xeb, 0xdf, 0x9a, 0xa0, 0xec, 0xff, 0xaf, 0x3e, 0xf6, 0x61, 0x33, 0x6f, 0xf7, 0xa5, 0x3a, 0xfa, - 0x0b, 0x03, 0x6a, 0xcf, 0x27, 0xae, 0x2b, 0x9f, 0x37, 0xd0, 0x1d, 0xb0, 0xf8, 0x34, 0xa4, 0xcb, - 0x6e, 0xbb, 0x24, 0x23, 0xf7, 0xee, 0x61, 0xe6, 0xdf, 0x3d, 0x6e, 0x41, 0x4d, 0x1f, 0x73, 0x79, - 0xac, 0x4f, 0xb8, 0xb6, 0x22, 0xa8, 0x47, 0x91, 0x78, 0xc0, 0xc4, 0x41, 0x59, 0x9e, 0x2f, 0xd4, - 0x2b, 0x07, 0x48, 0xd2, 0x4b, 0x79, 0x92, 0xf9, 0xa1, 0x32, 0x43, 0x0e, 0xbe, 0xee, 0x75, 0x25, - 0x3d, 0xa2, 0x98, 0xb3, 0xe7, 0xa0, 0x5f, 0x1b, 0x60, 0x8b, 0xe9, 0xf2, 0x5e, 0xeb, 0xff, 0x01, - 0x31, 0xf3, 0x12, 0x53, 0xca, 0xbd, 0xc4, 0xbc, 0x1d, 0xc1, 0x97, 0xda, 0x06, 0xf9, 0x15, 0x92, - 0xdc, 0xac, 0xcf, 0x5f, 0x7d, 0x26, 0x46, 0xea, 0x9b, 0xf5, 0x1d, 0xa8, 0xc8, 0xbb, 0xb0, 0x24, - 0x46, 0x28, 0x27, 0x28, 0x63, 0x82, 0xb5, 0x84, 0x90, 0x95, 0xaa, 0x93, 0xaf, 0xa1, 0xbc, 0xac, - 0xb4, 0x01, 0x6b, 0x09, 0xe7, 0x10, 0x36, 0x04, 0xf1, 0x09, 0xe5, 0x0f, 0xa7, 0x22, 0x55, 0x57, - 0xd1, 0x9a, 0x9d, 0xdf, 0x1a, 0xb0, 0x99, 0x5f, 0x75, 0xd5, 0x87, 0xdc, 0xbb, 0x60, 0x89, 0xcf, - 0x9f, 0x85, 0x87, 0x86, 0xc4, 0xad, 0x58, 0xb2, 0x9d, 0xcf, 0xe1, 0x66, 0x6a, 0xc7, 0xa1, 0x0a, - 0x5c, 0x11, 0x84, 0xe7, 0xa7, 0x81, 0xf3, 0x17, 0x03, 0xda, 0x8b, 0x2a, 0x56, 0x0d, 0x77, 0xf1, - 0x21, 0x30, 0x71, 0x80, 0xf5, 0xf5, 0x0e, 0xf8, 0x0c, 0xd0, 0x61, 0xe8, 0x7b, 0x5c, 0xbd, 0xea, - 0x15, 0x6d, 0x13, 0x62, 0x85, 0x5c, 0x9b, 0x10, 0x04, 0x51, 0x3a, 0x7f, 0x6f, 0xc0, 0x46, 0x6e, - 0xfd, 0xe2, 0xc0, 0x1d, 0xb0, 0x7c, 0x7a, 0xc2, 0xf5, 0x87, 0x59, 0x33, 0xff, 0x1c, 0x89, 0x25, - 0x0f, 0x7d, 0x00, 0xe5, 0xc8, 0xeb, 0x0f, 0xb8, 0x0e, 0xfb, 0xbc, 0x90, 0x62, 0xee, 0x7c, 0x17, - 0x20, 0x7b, 0x59, 0x44, 0x00, 0x95, 0x9f, 0xb0, 0x68, 0x44, 0xfc, 0xd6, 0x35, 0x54, 0x85, 0xd2, - 0x3e, 0x3b, 0x6d, 0x19, 0xc8, 0x06, 0xeb, 0x13, 0xaf, 0x3f, 0x68, 0x99, 0x3b, 0x5b, 0xd0, 0xcc, - 0x3f, 0x27, 0xa2, 0x0a, 0x98, 0x87, 0x4f, 0x5b, 0xd7, 0xc4, 0x2f, 0xee, 0xb4, 0x8c, 0x9d, 0x5d, - 0x30, 0x5f, 0x84, 0x62, 0xea, 0xc1, 0x98, 0xab, 0x35, 0x1e, 0x51, 0x5f, 0xad, 0x21, 0xf6, 0x67, - 0xcb, 0x44, 0x0d, 0xb0, 0x93, 0x8b, 0xbd, 0x56, 0xe9, 0xe1, 0xce, 0x3f, 0xdf, 0xdc, 0x36, 0xfe, - 0xf5, 0xe6, 0xb6, 0xf1, 0xef, 0x37, 0xb7, 0x8d, 0x3f, 0xfd, 0xe7, 0xf6, 0x35, 0x68, 0xbb, 0x6c, - 0xb4, 0x1b, 0x7a, 0x41, 0xdf, 0x25, 0xe1, 0x2e, 0xf7, 0x86, 0x93, 0xdd, 0xe1, 0x44, 0x3e, 0xa6, - 0x1f, 0x57, 0xe4, 0xcf, 0x87, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0xc5, 0xfa, 0x72, 0x7b, 0xa0, - 0x1f, 0x00, 0x00, + // 2011 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0xef, 0x6e, 0xdb, 0xc8, + 0x11, 0x3f, 0x52, 0x94, 0x44, 0x8d, 0x64, 0x59, 0xb7, 0x76, 0x12, 0xf5, 0xd2, 0x4b, 0x7c, 0xec, + 0x05, 0x71, 0x5c, 0xd4, 0x41, 0x7d, 0x87, 0x7e, 0x2a, 0x8a, 0x43, 0x94, 0x34, 0xe7, 0x8b, 0xd3, + 0x18, 0x6b, 0x23, 0x45, 0x80, 0x5e, 0x75, 0x6b, 0x6a, 0x2d, 0x11, 0xa2, 0xb8, 0x0c, 0xb9, 0x92, + 0x2d, 0x14, 0x45, 0x51, 0x14, 0x57, 0xe0, 0x3e, 0xb6, 0x28, 0xd0, 0xa2, 0xe8, 0x3d, 0x40, 0x5f, + 0xa0, 0x5f, 0xfa, 0x02, 0xfd, 0xd8, 0x47, 0x28, 0xd2, 0x17, 0x29, 0xf6, 0x0f, 0x49, 0x51, 0x92, + 0x2f, 0x2e, 0x2b, 0xbb, 0x9f, 0xa4, 0x9d, 0x99, 0xdd, 0x99, 0xdf, 0xcc, 0xec, 0xec, 0x70, 0x17, + 0xd6, 0x86, 0x93, 0x28, 0x74, 0xc3, 0x93, 0xdd, 0x30, 0x62, 0x9c, 0xa1, 0xaa, 0x1e, 0xbe, 0xd7, + 0x18, 0x51, 0x4e, 0x12, 0xf2, 0x7b, 0x6b, 0x34, 0x8a, 0x58, 0x94, 0x0e, 0x37, 0xfb, 0xac, 0xcf, + 0xe4, 0xdf, 0x87, 0xe2, 0x9f, 0xa2, 0x3a, 0xbf, 0x04, 0xfb, 0x80, 0xb9, 0xc3, 0xfd, 0xe0, 0x94, + 0xa1, 0x0f, 0xa0, 0x11, 0x46, 0xde, 0x88, 0x44, 0xd3, 0xae, 0xcf, 0xdc, 0x61, 0xdb, 0xd8, 0x32, + 0xb6, 0x1b, 0xb8, 0xae, 0x69, 0x42, 0x4c, 0x88, 0x08, 0x56, 0x77, 0x42, 0xa3, 0xd8, 0x63, 0x41, + 0xdb, 0xdc, 0x32, 0xb6, 0x2d, 0x5c, 0x17, 0xb4, 0x97, 0x8a, 0x84, 0x5a, 0x50, 0x1a, 0xd2, 0x69, + 0xbb, 0x24, 0x27, 0x8b, 0xbf, 0xe8, 0x5b, 0x60, 0xcb, 0x49, 0x9c, 0xfb, 0x6d, 0x4b, 0x4e, 0xa8, + 0x8a, 0xf1, 0x31, 0xf7, 0x9d, 0xaf, 0x0d, 0xb0, 0x9f, 0xd1, 0xe9, 0x13, 0x61, 0x29, 0x7a, 0x00, + 0x15, 0x41, 0xa7, 0x3d, 0xa9, 0xb9, 0xbe, 0xf7, 0xee, 0x6e, 0x82, 0x33, 0x31, 0x11, 0x6b, 0x01, + 0xf4, 0x6d, 0xa8, 0x45, 0x94, 0x47, 0x53, 0x72, 0xe2, 0x53, 0x69, 0x44, 0x0d, 0x67, 0x04, 0xb4, + 0x09, 0x65, 0x72, 0xc2, 0x22, 0x2e, 0x8d, 0xa8, 0x61, 0x35, 0x40, 0x7b, 0x60, 0xbb, 0x2c, 0x38, + 0xf5, 0x3d, 0x97, 0x4b, 0x33, 0xea, 0x7b, 0x37, 0x53, 0x05, 0x3f, 0x8d, 0x3c, 0x4e, 0x3b, 0x9a, + 0x8b, 0x53, 0x39, 0xe7, 0x0c, 0xd6, 0x72, 0x2c, 0x81, 0x25, 0xe6, 0x24, 0xe2, 0x5d, 0x1e, 0x4b, + 0x2b, 0x2d, 0x5c, 0x95, 0xe3, 0xe3, 0x18, 0xdd, 0x85, 0x7a, 0x32, 0x4f, 0x70, 0x95, 0x6b, 0x20, + 0x21, 0x1d, 0xc7, 0x4b, 0x3c, 0xd3, 0x86, 0xaa, 0xf6, 0xae, 0xb4, 0xa8, 0x81, 0x93, 0xa1, 0xf3, + 0x75, 0x09, 0xaa, 0x1d, 0x16, 0x70, 0x7a, 0xce, 0xd1, 0x6d, 0x01, 0xb6, 0xef, 0xb1, 0xa0, 0xeb, + 0xf5, 0xb4, 0x52, 0x5b, 0x11, 0xf6, 0x7b, 0xe8, 0x07, 0xd0, 0xd0, 0x4c, 0x1a, 0x32, 0x77, 0x20, + 0xd5, 0xd6, 0xf7, 0x36, 0x76, 0x75, 0x2a, 0x60, 0xc9, 0x7b, 0x22, 0x58, 0xb8, 0x1e, 0x65, 0x03, + 0xb4, 0x05, 0x56, 0x48, 0x69, 0x24, 0xad, 0xa9, 0xef, 0x35, 0x12, 0xf9, 0x43, 0x4a, 0x23, 0x2c, + 0x39, 0x08, 0x81, 0xc5, 0x69, 0x34, 0x6a, 0x97, 0xa5, 0x46, 0xf9, 0x1f, 0x3d, 0x04, 0x3b, 0x8c, + 0x3c, 0x16, 0x79, 0x7c, 0xda, 0xae, 0x6c, 0x19, 0xdb, 0xcd, 0xbd, 0x8d, 0xd4, 0x87, 0x1d, 0x36, + 0x1a, 0x91, 0xa0, 0x77, 0x18, 0x79, 0x38, 0x15, 0x42, 0x9f, 0xc0, 0xba, 0x17, 0x33, 0x9f, 0x70, + 0x61, 0xa1, 0x4f, 0x27, 0xd4, 0x6f, 0x57, 0xe5, 0xbc, 0x5b, 0xe9, 0xbc, 0xfd, 0x84, 0x7f, 0x20, + 0xd8, 0xb8, 0xe9, 0xe5, 0xc6, 0xe8, 0x43, 0x68, 0x06, 0x8c, 0x77, 0x4f, 0x3d, 0xdf, 0xef, 0xba, + 0xc4, 0x1d, 0xd0, 0xb6, 0xbd, 0x65, 0x6c, 0xdb, 0xb8, 0x11, 0x30, 0xfe, 0x63, 0xcf, 0xf7, 0x3b, + 0x82, 0x26, 0xe3, 0x32, 0x0d, 0xdc, 0xae, 0xcf, 0xfa, 0xed, 0x9a, 0xe4, 0x57, 0xc5, 0xf8, 0x80, + 0xf5, 0x45, 0x5c, 0x06, 0x24, 0xe8, 0xf9, 0xb4, 0xcb, 0xbd, 0x11, 0x6d, 0x83, 0xe4, 0x82, 0x22, + 0x1d, 0x7b, 0x23, 0x2a, 0x04, 0x62, 0x97, 0x04, 0xdd, 0x1e, 0xe5, 0xc4, 0xf3, 0xdb, 0x75, 0x25, + 0x20, 0x48, 0x8f, 0x25, 0xe5, 0x33, 0xcb, 0xb6, 0x5a, 0x65, 0xe1, 0x3e, 0xd2, 0xeb, 0xbe, 0x1e, + 0xb3, 0x68, 0x3c, 0x72, 0x1e, 0x03, 0x7c, 0x9a, 0xad, 0x70, 0x0b, 0xaa, 0x67, 0xc4, 0xe3, 0xdd, + 0x91, 0x4a, 0x8a, 0x12, 0xae, 0x88, 0xe1, 0xf3, 0x18, 0xbd, 0x0f, 0x10, 0x46, 0xcc, 0xa5, 0x71, + 0x2c, 0x78, 0xa6, 0xe4, 0xd5, 0x34, 0xe5, 0x79, 0xec, 0xfc, 0x08, 0xec, 0x23, 0x97, 0x04, 0x72, + 0xf7, 0x6d, 0x42, 0x99, 0x33, 0x4e, 0x7c, 0xbd, 0x82, 0x1a, 0x88, 0x44, 0xd7, 0xe2, 0xb4, 0x37, + 0x37, 0x9f, 0xf6, 0x9c, 0xdf, 0x18, 0x00, 0x47, 0xa9, 0x9d, 0xe8, 0x3e, 0x94, 0xcf, 0x44, 0xb6, + 0x2e, 0xec, 0x9f, 0x44, 0x09, 0x56, 0x7c, 0x74, 0x0f, 0x2c, 0xb9, 0xc3, 0xcd, 0x8b, 0xe4, 0x24, + 0x5b, 0x88, 0xf5, 0x08, 0x27, 0x3a, 0x47, 0x96, 0x89, 0x09, 0xb6, 0x33, 0x85, 0xfa, 0x93, 0x73, + 0xea, 0x2a, 0x23, 0x62, 0xf4, 0x71, 0xde, 0xdf, 0x86, 0x4e, 0xc8, 0x64, 0x72, 0xe6, 0xb6, 0x5c, + 0x10, 0x3e, 0xce, 0x07, 0xc1, 0x9c, 0x9b, 0x95, 0xa1, 0x9c, 0x8d, 0x8c, 0xd3, 0x03, 0x78, 0x4a, + 0x39, 0xa6, 0xaf, 0xc7, 0x34, 0xe6, 0x68, 0x07, 0xaa, 0xae, 0xda, 0x33, 0x5a, 0x6b, 0x6b, 0x26, + 0x39, 0x25, 0x1d, 0x27, 0x02, 0xc9, 0x66, 0x34, 0x73, 0x9b, 0x31, 0x29, 0x6b, 0x25, 0xb5, 0xb3, + 0xf5, 0xd0, 0xf9, 0x15, 0xd4, 0xa5, 0x96, 0x38, 0x64, 0x41, 0x4c, 0xd1, 0xf7, 0xb3, 0x2d, 0x27, + 0xea, 0x96, 0xd6, 0xd5, 0xdc, 0x4d, 0xea, 0xad, 0xac, 0x66, 0xe9, 0x6e, 0x93, 0xa5, 0xed, 0x3e, + 0x94, 0x95, 0xec, 0xbc, 0xc7, 0x93, 0xe2, 0x87, 0x15, 0x5f, 0x64, 0xc1, 0x84, 0xf8, 0x63, 0xaa, + 0xab, 0x84, 0x1a, 0x38, 0x7f, 0x37, 0xa0, 0x2e, 0x3c, 0x50, 0x04, 0xe8, 0x6d, 0xa8, 0xa9, 0x8a, + 0x95, 0xc1, 0x55, 0x25, 0xec, 0x19, 0x9d, 0x0a, 0x75, 0xbe, 0x37, 0xf2, 0x54, 0xa5, 0x5c, 0xc3, + 0x6a, 0x30, 0xeb, 0x09, 0x2b, 0xe7, 0x09, 0xb1, 0xcd, 0x86, 0x74, 0xda, 0x65, 0x81, 0x3f, 0x95, + 0x75, 0xc1, 0xc6, 0xd5, 0x21, 0x9d, 0xbe, 0x08, 0x7c, 0xe9, 0xbe, 0x88, 0x0a, 0x39, 0x2a, 0x2b, + 0x83, 0x8d, 0x93, 0xa1, 0xf3, 0x0a, 0x2a, 0xcf, 0x26, 0x87, 0xc4, 0x9b, 0x71, 0x83, 0xf1, 0x16, + 0x37, 0x2c, 0x46, 0x67, 0xb9, 0x63, 0x06, 0xd0, 0x50, 0x7e, 0x29, 0x1e, 0x9a, 0x7b, 0x50, 0x0e, + 0x89, 0x17, 0x89, 0xdd, 0x59, 0xda, 0xae, 0xef, 0xad, 0x67, 0x36, 0x49, 0x9b, 0xb1, 0xe2, 0x3a, + 0x2f, 0xc0, 0x7e, 0x3e, 0xe6, 0xb2, 0x2e, 0xa1, 0xdb, 0x60, 0xb2, 0x50, 0xae, 0xdd, 0xdc, 0xab, + 0xa7, 0xf2, 0x2f, 0x42, 0x6c, 0xb2, 0xf0, 0xd2, 0xa6, 0x7f, 0x69, 0xc2, 0xfa, 0x61, 0x44, 0xe5, + 0x86, 0x2c, 0x12, 0xd7, 0x87, 0x50, 0x1b, 0x69, 0x83, 0x12, 0xdb, 0x33, 0x7f, 0x26, 0xa6, 0xe2, + 0x4c, 0x66, 0xe1, 0x78, 0x2f, 0x2d, 0x1e, 0xef, 0xdf, 0x81, 0x35, 0x95, 0x2b, 0xf9, 0xf0, 0x37, + 0x24, 0xf1, 0x65, 0x96, 0x03, 0xe9, 0x71, 0x5e, 0xce, 0x1d, 0xe7, 0x68, 0x0f, 0x6e, 0xc4, 0x43, + 0x2f, 0xec, 0xba, 0x2c, 0x88, 0x79, 0x44, 0xbc, 0x80, 0x77, 0xdd, 0x01, 0x75, 0x87, 0x3a, 0x23, + 0x36, 0x04, 0xb3, 0x93, 0xf2, 0x3a, 0x82, 0xe5, 0x84, 0xd0, 0xca, 0xdc, 0x50, 0x3c, 0x8c, 0x0f, + 0xa0, 0x22, 0xb9, 0x8b, 0xbe, 0x48, 0x73, 0x4b, 0x0b, 0x38, 0x7f, 0x35, 0x60, 0x4d, 0x1c, 0x56, + 0x5e, 0xa1, 0xc2, 0xb1, 0xe0, 0x23, 0x73, 0x89, 0x8f, 0x10, 0x58, 0x43, 0x3a, 0x8d, 0xdb, 0xa5, + 0xad, 0xd2, 0x76, 0x03, 0xcb, 0xff, 0xe8, 0x1e, 0x34, 0x5d, 0xa9, 0x75, 0xce, 0xbb, 0x6b, 0x8a, + 0xaa, 0xa7, 0x7e, 0x66, 0xd9, 0xe5, 0x56, 0x05, 0x57, 0x4e, 0xbc, 0xc0, 0x67, 0x7d, 0xc7, 0x87, + 0x66, 0x62, 0xea, 0xd5, 0x57, 0x1f, 0xa7, 0x0f, 0x6b, 0xfb, 0xa3, 0x90, 0x45, 0xa9, 0x63, 0x72, + 0x49, 0x66, 0x5c, 0x22, 0xc9, 0x16, 0x41, 0x9a, 0x4b, 0x40, 0x3a, 0xaf, 0xa0, 0x99, 0x28, 0x2a, + 0x0e, 0x6b, 0x73, 0x16, 0x56, 0x2d, 0xc1, 0xf0, 0x0b, 0xd8, 0x7c, 0x44, 0xb8, 0x3b, 0xc0, 0xcc, + 0xf7, 0x4f, 0x88, 0x3b, 0xbc, 0xce, 0x18, 0x3b, 0x31, 0xdc, 0x98, 0x53, 0x7e, 0x0d, 0x51, 0x8b, + 0xa1, 0xd9, 0xf1, 0x29, 0x09, 0xc6, 0xe1, 0x6a, 0x0e, 0xc2, 0x05, 0xf4, 0xa5, 0x45, 0xf4, 0xce, + 0x1f, 0x0c, 0x58, 0x4f, 0xb5, 0x5e, 0xc3, 0xc1, 0xb8, 0x98, 0x58, 0xa5, 0x65, 0x89, 0x35, 0x84, + 0x75, 0x19, 0x80, 0x82, 0x5d, 0x41, 0x12, 0x53, 0x73, 0x66, 0xdf, 0x5e, 0xdc, 0x17, 0xf8, 0xd0, + 0xca, 0x94, 0x5d, 0xf9, 0x09, 0xf4, 0x3b, 0x03, 0xd6, 0xc5, 0x61, 0x27, 0x2a, 0x75, 0x11, 0x6c, + 0x77, 0xa1, 0x3e, 0x22, 0xe7, 0x73, 0x29, 0x0d, 0x23, 0x72, 0x9e, 0x24, 0x74, 0xae, 0x53, 0x28, + 0x5d, 0xd4, 0x29, 0x58, 0x33, 0x9d, 0x82, 0xf3, 0x47, 0x03, 0x5a, 0x99, 0x4d, 0xd7, 0x90, 0x06, + 0xf7, 0xa1, 0x2c, 0x0e, 0x1b, 0xb5, 0xeb, 0x96, 0x7e, 0x22, 0x2a, 0xbe, 0xf3, 0x11, 0x54, 0x8f, + 0xcf, 0x55, 0x67, 0xdd, 0x82, 0x12, 0x3f, 0x0f, 0xf4, 0x97, 0x93, 0xf8, 0x8b, 0x6e, 0x42, 0x25, + 0xe6, 0x84, 0x8f, 0x93, 0xaf, 0x34, 0x3d, 0x12, 0x7d, 0x16, 0xc2, 0x34, 0x66, 0xfe, 0x84, 0x16, + 0xf5, 0xf2, 0xa5, 0x4a, 0xc7, 0xe5, 0x92, 0x19, 0x7d, 0x0f, 0x6a, 0xfc, 0x3c, 0xe8, 0x7a, 0xc1, + 0x29, 0x8b, 0xdb, 0x96, 0x04, 0x9c, 0x69, 0xd6, 0xe8, 0xb0, 0xcd, 0xd5, 0x9f, 0xd8, 0x79, 0x0d, + 0x1b, 0x39, 0xe3, 0xaf, 0xa1, 0xf4, 0xbc, 0x84, 0xda, 0xd3, 0x4e, 0x11, 0x37, 0xbd, 0x0f, 0x10, + 0x93, 0x53, 0xda, 0x0d, 0x99, 0x17, 0x70, 0xed, 0xa3, 0x9a, 0xa0, 0x1c, 0x0a, 0x82, 0x33, 0x00, + 0x10, 0xeb, 0x5e, 0x03, 0x82, 0xcf, 0x61, 0x0d, 0x93, 0xb3, 0x95, 0x7d, 0x44, 0x34, 0xc1, 0x74, + 0x4f, 0xf5, 0xbd, 0x83, 0xe9, 0x9e, 0x3a, 0x0c, 0x9a, 0xc9, 0xf2, 0x2b, 0x3e, 0xe8, 0x2e, 0x68, + 0x2b, 0x63, 0x89, 0xe7, 0x70, 0xbc, 0x22, 0x3c, 0x4b, 0x95, 0x68, 0x94, 0x56, 0x8a, 0xf2, 0x95, + 0x44, 0x29, 0x95, 0xae, 0xfa, 0x38, 0x3f, 0x03, 0x84, 0xc9, 0x99, 0x2c, 0xb3, 0x05, 0x41, 0x5d, + 0xae, 0xbc, 0x2e, 0x44, 0xee, 0xe7, 0xb0, 0x91, 0x53, 0xbc, 0x6a, 0x60, 0xbd, 0x0c, 0xd8, 0x0a, + 0x0f, 0xab, 0xc5, 0xfc, 0xdb, 0xc8, 0x69, 0xb9, 0xf2, 0x53, 0xea, 0x0b, 0x68, 0x61, 0x72, 0xf6, + 0x98, 0xfa, 0xb4, 0xd8, 0x67, 0xcd, 0xdb, 0xb7, 0xd4, 0xcf, 0xe0, 0xdd, 0x19, 0x0d, 0xab, 0x0e, + 0x4b, 0x1f, 0x6e, 0x24, 0x0e, 0x2b, 0x0e, 0xe2, 0x32, 0x91, 0x21, 0x70, 0x73, 0x5e, 0xd1, 0xaa, + 0xb1, 0x4c, 0x00, 0xe9, 0xa5, 0x49, 0xd0, 0xa7, 0x2b, 0xbf, 0x3c, 0xb8, 0x05, 0x55, 0x1a, 0xf4, + 0x66, 0xba, 0x85, 0x0a, 0x0d, 0x7a, 0xcf, 0xe8, 0x54, 0x6c, 0x9d, 0x9c, 0xde, 0x55, 0xe3, 0xfa, + 0xca, 0x90, 0x41, 0xfa, 0x7f, 0x60, 0x5b, 0x28, 0x7d, 0x2a, 0x8c, 0x57, 0x0a, 0xf7, 0xcf, 0x86, + 0x2c, 0xaf, 0xd7, 0x78, 0x01, 0x34, 0x7b, 0xcd, 0x63, 0xe5, 0xaf, 0x79, 0x14, 0xfe, 0x72, 0x8a, + 0xbf, 0x0f, 0xeb, 0xa9, 0x6d, 0xc5, 0x81, 0x7f, 0x00, 0xa5, 0xe1, 0xe4, 0xc2, 0xd2, 0x22, 0x78, + 0xce, 0x27, 0xf2, 0xa5, 0x40, 0xba, 0x38, 0x0f, 0xc9, 0xb8, 0x38, 0x74, 0x66, 0x2e, 0x2d, 0xff, + 0x66, 0x64, 0xc5, 0xb0, 0xa8, 0x33, 0x1f, 0x40, 0x25, 0x12, 0x26, 0x2c, 0xbd, 0x66, 0x50, 0xf1, + 0xd7, 0x02, 0xa2, 0xc5, 0xa1, 0xc4, 0x1d, 0x74, 0x67, 0xfd, 0x5b, 0x13, 0x94, 0x83, 0xff, 0xd6, + 0xc7, 0x3e, 0x6c, 0xe6, 0xed, 0xbe, 0x52, 0x47, 0x7f, 0x69, 0x40, 0xed, 0xf9, 0xc4, 0x75, 0xe5, + 0xc3, 0x07, 0xba, 0x0b, 0x16, 0x9f, 0x86, 0x74, 0xd9, 0x6d, 0x97, 0x64, 0xe4, 0x5e, 0x44, 0xcc, + 0xfc, 0x8b, 0xc8, 0x6d, 0xa8, 0xe9, 0x36, 0x97, 0xc7, 0xba, 0xc3, 0xb5, 0x15, 0x41, 0x3d, 0x97, + 0xc4, 0x03, 0x26, 0x1a, 0x65, 0xd9, 0x5f, 0xa8, 0xf7, 0x0f, 0x90, 0xa4, 0x97, 0xb2, 0x93, 0xf9, + 0xa1, 0x32, 0x43, 0x0e, 0xbe, 0xe9, 0xdd, 0x25, 0x6d, 0x51, 0xcc, 0xd9, 0x3e, 0xe8, 0xd7, 0x06, + 0xd8, 0x62, 0xba, 0xbc, 0xd7, 0xfa, 0x5f, 0x40, 0xcc, 0xbc, 0xd1, 0x94, 0x72, 0x6f, 0x34, 0x6f, + 0x47, 0xf0, 0x95, 0xb6, 0x41, 0x7e, 0x85, 0x24, 0x77, 0xee, 0xf3, 0x57, 0x9f, 0x89, 0x91, 0xfa, + 0xce, 0x7d, 0x07, 0x2a, 0xf2, 0x2e, 0x2c, 0x89, 0x11, 0xca, 0x09, 0xca, 0x98, 0x60, 0x2d, 0x21, + 0x64, 0xa5, 0xea, 0xe4, 0x6b, 0x28, 0x2f, 0x2b, 0x6d, 0xc0, 0x5a, 0xc2, 0x39, 0x82, 0x0d, 0x41, + 0x7c, 0x4a, 0xf9, 0xa3, 0xa9, 0x48, 0xd5, 0x55, 0x1c, 0xcd, 0xce, 0x6f, 0x0d, 0xd8, 0xcc, 0xaf, + 0xba, 0xea, 0x26, 0xf7, 0x1e, 0x58, 0xe2, 0xf3, 0x67, 0xe1, 0x09, 0x22, 0x71, 0x2b, 0x96, 0x6c, + 0xe7, 0x0b, 0xb8, 0x95, 0xda, 0x71, 0xa4, 0x02, 0x57, 0x04, 0xe1, 0xc5, 0x69, 0xe0, 0xfc, 0xc5, + 0x80, 0xf6, 0xa2, 0x8a, 0x55, 0xc3, 0x5d, 0x7c, 0x22, 0x4c, 0x1c, 0x60, 0x7d, 0xb3, 0x03, 0x3e, + 0x07, 0x74, 0x14, 0xfa, 0x1e, 0x57, 0xef, 0x7d, 0x45, 0x8f, 0x09, 0xb1, 0x42, 0xee, 0x98, 0x10, + 0x04, 0x51, 0x3a, 0x7f, 0x6f, 0xc0, 0x46, 0x6e, 0xfd, 0xe2, 0xc0, 0x1d, 0xb0, 0x7c, 0x7a, 0xca, + 0xf5, 0x87, 0x59, 0x33, 0xff, 0x50, 0x89, 0x25, 0x0f, 0x7d, 0x08, 0xe5, 0xc8, 0xeb, 0x0f, 0xb8, + 0x0e, 0xfb, 0xbc, 0x90, 0x62, 0xee, 0x7c, 0x17, 0x20, 0x7b, 0x73, 0x44, 0x00, 0x95, 0x9f, 0xb0, + 0x68, 0x44, 0xfc, 0xd6, 0x3b, 0xa8, 0x0a, 0xa5, 0x03, 0x76, 0xd6, 0x32, 0x90, 0x0d, 0xd6, 0xa7, + 0x5e, 0x7f, 0xd0, 0x32, 0x77, 0xb6, 0xa0, 0x99, 0x7f, 0x68, 0x44, 0x15, 0x30, 0x8f, 0xf6, 0x5b, + 0xef, 0x88, 0x5f, 0xdc, 0x69, 0x19, 0x3b, 0xbb, 0x60, 0xbe, 0x08, 0xc5, 0xd4, 0xc3, 0x31, 0x57, + 0x6b, 0x3c, 0xa6, 0xbe, 0x5a, 0x43, 0xec, 0xcf, 0x96, 0x89, 0x1a, 0x60, 0x27, 0x17, 0x7b, 0xad, + 0xd2, 0xa3, 0x9d, 0x7f, 0xbc, 0xb9, 0x63, 0xfc, 0xf3, 0xcd, 0x1d, 0xe3, 0x5f, 0x6f, 0xee, 0x18, + 0x7f, 0xfa, 0xf7, 0x9d, 0x77, 0xa0, 0xed, 0xb2, 0xd1, 0x6e, 0xe8, 0x05, 0x7d, 0x97, 0x84, 0xbb, + 0xdc, 0x1b, 0x4e, 0x76, 0x87, 0x13, 0xf9, 0xcc, 0x7e, 0x52, 0x91, 0x3f, 0x1f, 0xfd, 0x27, 0x00, + 0x00, 0xff, 0xff, 0x30, 0x25, 0x4a, 0xcb, 0xba, 0x1f, 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 497f52d332b3b..3479ceed1358c 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/metapb/metapb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/metapb/metapb.pb.go @@ -117,6 +117,7 @@ type Store struct { Address string `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` State StoreState `protobuf:"varint,3,opt,name=state,proto3,enum=metapb.StoreState" json:"state,omitempty"` Labels []*StoreLabel `protobuf:"bytes,4,rep,name=labels" json:"labels,omitempty"` + Version string `protobuf:"bytes,5,opt,name=version,proto3" json:"version,omitempty"` } func (m *Store) Reset() { *m = Store{} } @@ -152,6 +153,13 @@ func (m *Store) GetLabels() []*StoreLabel { return nil } +func (m *Store) GetVersion() string { + if m != nil { + return m.Version + } + return "" +} + type RegionEpoch struct { // Conf change version, auto increment when add or remove peer ConfVer uint64 `protobuf:"varint,1,opt,name=conf_ver,json=confVer,proto3" json:"conf_ver,omitempty"` @@ -369,6 +377,12 @@ func (m *Store) MarshalTo(dAtA []byte) (int, error) { i += n } } + if len(m.Version) > 0 { + dAtA[i] = 0x2a + i++ + i = encodeVarintMetapb(dAtA, i, uint64(len(m.Version))) + i += copy(dAtA[i:], m.Version) + } return i, nil } @@ -567,6 +581,10 @@ func (m *Store) Size() (n int) { n += 1 + l + sovMetapb(uint64(l)) } } + l = len(m.Version) + if l > 0 { + n += 1 + l + sovMetapb(uint64(l)) + } return n } @@ -960,6 +978,35 @@ func (m *Store) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMetapb + } + 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 ErrInvalidLengthMetapb + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Version = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipMetapb(dAtA[iNdEx:]) @@ -1480,35 +1527,35 @@ var ( func init() { proto.RegisterFile("metapb.proto", fileDescriptorMetapb) } var fileDescriptorMetapb = []byte{ - // 474 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x92, 0xcf, 0x8e, 0xd3, 0x30, - 0x10, 0xc6, 0xd7, 0xfd, 0x93, 0xb6, 0xd3, 0x6e, 0x55, 0x99, 0x95, 0xc8, 0x82, 0xa8, 0xaa, 0x88, - 0x43, 0xd4, 0x43, 0x41, 0x0b, 0xe2, 0x8a, 0xb4, 0x2b, 0x0e, 0x88, 0x95, 0x58, 0x79, 0x81, 0x6b, - 0xe4, 0x26, 0xd3, 0x62, 0x35, 0xb1, 0x23, 0xdb, 0x8d, 0x76, 0x9f, 0x80, 0x57, 0xe0, 0x19, 0x78, - 0x12, 0x8e, 0x3c, 0x02, 0x2a, 0x2f, 0x82, 0xec, 0x24, 0xd2, 0x6a, 0x7b, 0x8a, 0xbf, 0xf9, 0x66, - 0x26, 0xbf, 0xf1, 0x18, 0x26, 0x05, 0x5a, 0x5e, 0xae, 0x57, 0xa5, 0x56, 0x56, 0xd1, 0xa0, 0x56, - 0xcf, 0xce, 0xb6, 0x6a, 0xab, 0x7c, 0xe8, 0x95, 0x3b, 0xd5, 0x6e, 0xf4, 0x1e, 0x06, 0x57, 0xf9, - 0xde, 0x58, 0xd4, 0x74, 0x0a, 0x1d, 0x91, 0x85, 0x64, 0x41, 0xe2, 0x1e, 0xeb, 0x88, 0x8c, 0xbe, - 0x84, 0x69, 0xc1, 0xef, 0x92, 0x12, 0x51, 0x27, 0xa9, 0xda, 0x4b, 0x1b, 0x76, 0x16, 0x24, 0x3e, - 0x65, 0x93, 0x82, 0xdf, 0xdd, 0x20, 0xea, 0x2b, 0x17, 0x8b, 0xde, 0x02, 0xdc, 0x5a, 0xa5, 0xf1, - 0x9a, 0xaf, 0x31, 0xa7, 0x33, 0xe8, 0xee, 0xf0, 0xde, 0x37, 0x19, 0x31, 0x77, 0xa4, 0x67, 0xd0, - 0xaf, 0x78, 0xbe, 0x47, 0x5f, 0x3c, 0x62, 0xb5, 0x88, 0x7e, 0x10, 0xe8, 0xfb, 0xb2, 0xa3, 0xbf, - 0x86, 0x30, 0xe0, 0x59, 0xa6, 0xd1, 0x98, 0xa6, 0xa2, 0x95, 0x34, 0x86, 0xbe, 0xb1, 0xdc, 0x62, - 0xd8, 0x5d, 0x90, 0x78, 0x7a, 0x41, 0x57, 0xcd, 0x98, 0xbe, 0xcf, 0xad, 0x73, 0x58, 0x9d, 0x40, - 0x97, 0x10, 0xe4, 0x0e, 0xc7, 0x84, 0xbd, 0x45, 0x37, 0x1e, 0x3f, 0x4a, 0xf5, 0xa4, 0xac, 0xc9, - 0x88, 0x2e, 0x61, 0xcc, 0x70, 0x2b, 0x94, 0xfc, 0x50, 0xaa, 0xf4, 0x3b, 0x3d, 0x87, 0x61, 0xaa, - 0xe4, 0x26, 0xa9, 0x50, 0x37, 0x50, 0x03, 0xa7, 0xbf, 0xa1, 0x76, 0x64, 0x15, 0x6a, 0x23, 0x94, - 0xf4, 0x64, 0x3d, 0xd6, 0xca, 0xe8, 0x17, 0x81, 0xa0, 0x6e, 0x72, 0x34, 0xce, 0x73, 0x18, 0x19, - 0xcb, 0xb5, 0x4d, 0xdc, 0xb5, 0xb8, 0xb2, 0x09, 0x1b, 0xfa, 0xc0, 0x27, 0xbc, 0xa7, 0x4f, 0x61, - 0x80, 0x32, 0xf3, 0x56, 0xd7, 0x5b, 0x01, 0xca, 0xcc, 0x19, 0xef, 0x60, 0xa2, 0x7d, 0xbf, 0x04, - 0x1d, 0x55, 0xd8, 0x5b, 0x90, 0x78, 0x7c, 0xf1, 0xa4, 0x1d, 0xe3, 0x01, 0x30, 0x1b, 0xeb, 0x07, - 0xf4, 0x11, 0xf4, 0xdd, 0xba, 0x4c, 0xd8, 0xf7, 0x73, 0x4f, 0xda, 0x02, 0xb7, 0x2e, 0x56, 0x5b, - 0xd1, 0x0d, 0xf4, 0x9c, 0x3c, 0x22, 0x3d, 0x87, 0xa1, 0x71, 0xd7, 0x93, 0x88, 0xac, 0x9d, 0xcf, - 0xeb, 0x8f, 0x19, 0x7d, 0x01, 0x20, 0x4c, 0x92, 0x23, 0xd7, 0x12, 0xb5, 0x47, 0x1d, 0xb2, 0x91, - 0x30, 0xd7, 0x75, 0x60, 0xf9, 0xba, 0x79, 0x02, 0x7e, 0x07, 0x34, 0x80, 0xce, 0xd7, 0x72, 0x76, - 0x42, 0xc7, 0x30, 0xf8, 0xbc, 0xd9, 0xe4, 0x42, 0xe2, 0x8c, 0xd0, 0x53, 0x18, 0x7d, 0x51, 0xc5, - 0xda, 0x58, 0x25, 0x71, 0xd6, 0xb9, 0x5c, 0xfe, 0x3e, 0xcc, 0xc9, 0x9f, 0xc3, 0x9c, 0xfc, 0x3d, - 0xcc, 0xc9, 0xcf, 0x7f, 0xf3, 0x13, 0x08, 0x53, 0x55, 0xac, 0x4a, 0x21, 0xb7, 0x29, 0x2f, 0x57, - 0x56, 0xec, 0xaa, 0xd5, 0xae, 0xf2, 0x2f, 0x74, 0x1d, 0xf8, 0xcf, 0x9b, 0xff, 0x01, 0x00, 0x00, - 0xff, 0xff, 0x2a, 0x28, 0xbc, 0x30, 0xd6, 0x02, 0x00, 0x00, + // 479 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x52, 0xc1, 0x8e, 0xd3, 0x3c, + 0x10, 0x5e, 0xa7, 0x4d, 0xda, 0x4e, 0xbb, 0x55, 0xe5, 0x7f, 0xa5, 0x3f, 0x0b, 0xa2, 0xaa, 0x22, + 0x0e, 0x51, 0x0f, 0x05, 0x2d, 0x88, 0x2b, 0xd2, 0xae, 0x38, 0x20, 0x56, 0x62, 0xe5, 0x05, 0xae, + 0x91, 0xdb, 0x4c, 0x8b, 0xd5, 0xd4, 0x8e, 0x6c, 0x37, 0xda, 0x7d, 0x13, 0xae, 0x5c, 0x79, 0x12, + 0x8e, 0x3c, 0x02, 0x2a, 0x2f, 0x82, 0xec, 0x24, 0x52, 0x45, 0x4f, 0xc9, 0x37, 0xdf, 0xcc, 0x37, + 0xdf, 0xcc, 0x18, 0x46, 0x3b, 0xb4, 0xbc, 0x5c, 0x2e, 0x4a, 0xad, 0xac, 0xa2, 0x51, 0x8d, 0x9e, + 0x5c, 0x6c, 0xd4, 0x46, 0xf9, 0xd0, 0x0b, 0xf7, 0x57, 0xb3, 0xc9, 0x5b, 0xe8, 0xdd, 0x14, 0x7b, + 0x63, 0x51, 0xd3, 0x31, 0x04, 0x22, 0x8f, 0xc9, 0x8c, 0xa4, 0x5d, 0x16, 0x88, 0x9c, 0x3e, 0x87, + 0xf1, 0x8e, 0x3f, 0x64, 0x25, 0xa2, 0xce, 0x56, 0x6a, 0x2f, 0x6d, 0x1c, 0xcc, 0x48, 0x7a, 0xce, + 0x46, 0x3b, 0xfe, 0x70, 0x87, 0xa8, 0x6f, 0x5c, 0x2c, 0x79, 0x0d, 0x70, 0x6f, 0x95, 0xc6, 0x5b, + 0xbe, 0xc4, 0x82, 0x4e, 0xa0, 0xb3, 0xc5, 0x47, 0x2f, 0x32, 0x60, 0xee, 0x97, 0x5e, 0x40, 0x58, + 0xf1, 0x62, 0x8f, 0xbe, 0x78, 0xc0, 0x6a, 0x90, 0x7c, 0x27, 0x10, 0xfa, 0xb2, 0x93, 0xae, 0x31, + 0xf4, 0x78, 0x9e, 0x6b, 0x34, 0xa6, 0xa9, 0x68, 0x21, 0x4d, 0x21, 0x34, 0x96, 0x5b, 0x8c, 0x3b, + 0x33, 0x92, 0x8e, 0xaf, 0xe8, 0xa2, 0x19, 0xd3, 0xeb, 0xdc, 0x3b, 0x86, 0xd5, 0x09, 0x74, 0x0e, + 0x51, 0xe1, 0xec, 0x98, 0xb8, 0x3b, 0xeb, 0xa4, 0xc3, 0x7f, 0x52, 0xbd, 0x53, 0xd6, 0x64, 0xb8, + 0x7e, 0x15, 0x6a, 0x23, 0x94, 0x8c, 0xc3, 0xba, 0x5f, 0x03, 0x93, 0x6b, 0x18, 0x32, 0xdc, 0x08, + 0x25, 0xdf, 0x95, 0x6a, 0xf5, 0x95, 0x5e, 0x42, 0x7f, 0xa5, 0xe4, 0x3a, 0xab, 0x50, 0x37, 0x76, + 0x7b, 0x0e, 0x7f, 0x41, 0x7d, 0xac, 0x11, 0xd4, 0x4c, 0xab, 0xf1, 0x83, 0x40, 0x54, 0x8b, 0x9c, + 0x0c, 0xfa, 0x14, 0x06, 0xc6, 0x72, 0x6d, 0x33, 0xb7, 0x30, 0x57, 0x36, 0x62, 0x7d, 0x1f, 0xf8, + 0x80, 0x8f, 0xf4, 0x7f, 0xe8, 0xa1, 0xcc, 0x3d, 0xd5, 0xf1, 0x54, 0x84, 0x32, 0x77, 0xc4, 0x1b, + 0x18, 0x69, 0xaf, 0x97, 0xa1, 0x73, 0x15, 0x77, 0x67, 0x24, 0x1d, 0x5e, 0xfd, 0xd7, 0x0e, 0x78, + 0x64, 0x98, 0x0d, 0xf5, 0x91, 0xfb, 0x04, 0x42, 0x77, 0x48, 0x13, 0x87, 0x7e, 0x23, 0xa3, 0xb6, + 0xc0, 0x1d, 0x92, 0xd5, 0x54, 0x72, 0x07, 0x5d, 0x07, 0x4f, 0x9c, 0x5e, 0x42, 0xdf, 0xb8, 0xc5, + 0x65, 0x22, 0x6f, 0xe7, 0xf3, 0xf8, 0x7d, 0x4e, 0x9f, 0x01, 0x08, 0x93, 0x15, 0xc8, 0xb5, 0x44, + 0xed, 0xad, 0xf6, 0xd9, 0x40, 0x98, 0xdb, 0x3a, 0x30, 0x7f, 0xd9, 0x3c, 0x0e, 0x7f, 0x1d, 0x1a, + 0x41, 0xf0, 0xb9, 0x9c, 0x9c, 0xd1, 0x21, 0xf4, 0x3e, 0xae, 0xd7, 0x85, 0x90, 0x38, 0x21, 0xf4, + 0x1c, 0x06, 0x9f, 0xd4, 0x6e, 0x69, 0xac, 0x92, 0x38, 0x09, 0xae, 0xe7, 0x3f, 0x0f, 0x53, 0xf2, + 0xeb, 0x30, 0x25, 0xbf, 0x0f, 0x53, 0xf2, 0xed, 0xcf, 0xf4, 0x0c, 0xe2, 0x95, 0xda, 0x2d, 0x4a, + 0x21, 0x37, 0x2b, 0x5e, 0x2e, 0xac, 0xd8, 0x56, 0x8b, 0x6d, 0xe5, 0xdf, 0xee, 0x32, 0xf2, 0x9f, + 0x57, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x0b, 0x59, 0x52, 0x48, 0xf0, 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 861c6d970a79b..e0a6dce3dca23 100644 --- a/vendor/github.com/pingcap/kvproto/pkg/pdpb/pdpb.pb.go +++ b/vendor/github.com/pingcap/kvproto/pkg/pdpb/pdpb.pb.go @@ -48,12 +48,21 @@ AskSplitResponse ReportSplitRequest ReportSplitResponse + AskBatchSplitRequest + SplitID + AskBatchSplitResponse + ReportBatchSplitRequest + ReportBatchSplitResponse TimeInterval StoreStats StoreHeartbeatRequest StoreHeartbeatResponse ScatterRegionRequest ScatterRegionResponse + GetGCSafePointRequest + GetGCSafePointResponse + UpdateGCSafePointRequest + UpdateGCSafePointResponse */ package pdpb @@ -92,6 +101,7 @@ const ( ErrorType_NOT_BOOTSTRAPPED ErrorType = 2 ErrorType_STORE_TOMBSTONE ErrorType = 3 ErrorType_ALREADY_BOOTSTRAPPED ErrorType = 4 + ErrorType_INCOMPATIBLE_VERSION ErrorType = 5 ) var ErrorType_name = map[int32]string{ @@ -100,6 +110,7 @@ var ErrorType_name = map[int32]string{ 2: "NOT_BOOTSTRAPPED", 3: "STORE_TOMBSTONE", 4: "ALREADY_BOOTSTRAPPED", + 5: "INCOMPATIBLE_VERSION", } var ErrorType_value = map[string]int32{ "OK": 0, @@ -107,6 +118,7 @@ var ErrorType_value = map[string]int32{ "NOT_BOOTSTRAPPED": 2, "STORE_TOMBSTONE": 3, "ALREADY_BOOTSTRAPPED": 4, + "INCOMPATIBLE_VERSION": 5, } func (x ErrorType) String() string { @@ -114,6 +126,27 @@ func (x ErrorType) String() string { } func (ErrorType) EnumDescriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{0} } +type CheckPolicy int32 + +const ( + CheckPolicy_SCAN CheckPolicy = 0 + CheckPolicy_APPROXIMATE CheckPolicy = 1 +) + +var CheckPolicy_name = map[int32]string{ + 0: "SCAN", + 1: "APPROXIMATE", +} +var CheckPolicy_value = map[string]int32{ + "SCAN": 0, + "APPROXIMATE": 1, +} + +func (x CheckPolicy) String() string { + return proto.EnumName(CheckPolicy_name, int32(x)) +} +func (CheckPolicy) EnumDescriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{1} } + type RequestHeader struct { // cluster_id is the ID of the cluster which be sent to. ClusterId uint64 `protobuf:"varint,1,opt,name=cluster_id,json=clusterId,proto3" json:"cluster_id,omitempty"` @@ -826,6 +859,8 @@ type RegionHeartbeatRequest struct { ApproximateSize uint64 `protobuf:"varint,10,opt,name=approximate_size,json=approximateSize,proto3" json:"approximate_size,omitempty"` // Actually reported time interval Interval *TimeInterval `protobuf:"bytes,12,opt,name=interval" json:"interval,omitempty"` + // Approximate number of keys. + ApproximateKeys uint64 `protobuf:"varint,13,opt,name=approximate_keys,json=approximateKeys,proto3" json:"approximate_keys,omitempty"` } func (m *RegionHeartbeatRequest) Reset() { *m = RegionHeartbeatRequest{} } @@ -910,6 +945,13 @@ func (m *RegionHeartbeatRequest) GetInterval() *TimeInterval { return nil } +func (m *RegionHeartbeatRequest) GetApproximateKeys() uint64 { + if m != nil { + return m.ApproximateKeys + } + 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"` @@ -967,6 +1009,7 @@ func (m *Merge) GetTarget() *metapb.Region { } type SplitRegion struct { + Policy CheckPolicy `protobuf:"varint,1,opt,name=policy,proto3,enum=pdpb.CheckPolicy" json:"policy,omitempty"` } func (m *SplitRegion) Reset() { *m = SplitRegion{} } @@ -974,6 +1017,13 @@ func (m *SplitRegion) String() string { return proto.CompactTextStrin func (*SplitRegion) ProtoMessage() {} func (*SplitRegion) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{33} } +func (m *SplitRegion) GetPolicy() CheckPolicy { + if m != nil { + return m.Policy + } + return CheckPolicy_SCAN +} + type RegionHeartbeatResponse struct { Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` // Notice, Pd only allows handling reported epoch >= current pd's. @@ -1172,6 +1222,126 @@ func (m *ReportSplitResponse) GetHeader() *ResponseHeader { return nil } +type AskBatchSplitRequest struct { + Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + Region *metapb.Region `protobuf:"bytes,2,opt,name=region" json:"region,omitempty"` + SplitCount uint32 `protobuf:"varint,3,opt,name=split_count,json=splitCount,proto3" json:"split_count,omitempty"` +} + +func (m *AskBatchSplitRequest) Reset() { *m = AskBatchSplitRequest{} } +func (m *AskBatchSplitRequest) String() string { return proto.CompactTextString(m) } +func (*AskBatchSplitRequest) ProtoMessage() {} +func (*AskBatchSplitRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{39} } + +func (m *AskBatchSplitRequest) GetHeader() *RequestHeader { + if m != nil { + return m.Header + } + return nil +} + +func (m *AskBatchSplitRequest) GetRegion() *metapb.Region { + if m != nil { + return m.Region + } + return nil +} + +func (m *AskBatchSplitRequest) GetSplitCount() uint32 { + if m != nil { + return m.SplitCount + } + return 0 +} + +type SplitID struct { + NewRegionId uint64 `protobuf:"varint,1,opt,name=new_region_id,json=newRegionId,proto3" json:"new_region_id,omitempty"` + NewPeerIds []uint64 `protobuf:"varint,2,rep,packed,name=new_peer_ids,json=newPeerIds" json:"new_peer_ids,omitempty"` +} + +func (m *SplitID) Reset() { *m = SplitID{} } +func (m *SplitID) String() string { return proto.CompactTextString(m) } +func (*SplitID) ProtoMessage() {} +func (*SplitID) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{40} } + +func (m *SplitID) GetNewRegionId() uint64 { + if m != nil { + return m.NewRegionId + } + return 0 +} + +func (m *SplitID) GetNewPeerIds() []uint64 { + if m != nil { + return m.NewPeerIds + } + return nil +} + +type AskBatchSplitResponse struct { + Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + Ids []*SplitID `protobuf:"bytes,2,rep,name=ids" json:"ids,omitempty"` +} + +func (m *AskBatchSplitResponse) Reset() { *m = AskBatchSplitResponse{} } +func (m *AskBatchSplitResponse) String() string { return proto.CompactTextString(m) } +func (*AskBatchSplitResponse) ProtoMessage() {} +func (*AskBatchSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{41} } + +func (m *AskBatchSplitResponse) GetHeader() *ResponseHeader { + if m != nil { + return m.Header + } + return nil +} + +func (m *AskBatchSplitResponse) GetIds() []*SplitID { + if m != nil { + return m.Ids + } + return nil +} + +type ReportBatchSplitRequest struct { + Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + Regions []*metapb.Region `protobuf:"bytes,2,rep,name=regions" json:"regions,omitempty"` +} + +func (m *ReportBatchSplitRequest) Reset() { *m = ReportBatchSplitRequest{} } +func (m *ReportBatchSplitRequest) String() string { return proto.CompactTextString(m) } +func (*ReportBatchSplitRequest) ProtoMessage() {} +func (*ReportBatchSplitRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{42} } + +func (m *ReportBatchSplitRequest) GetHeader() *RequestHeader { + if m != nil { + return m.Header + } + return nil +} + +func (m *ReportBatchSplitRequest) GetRegions() []*metapb.Region { + if m != nil { + return m.Regions + } + return nil +} + +type ReportBatchSplitResponse struct { + Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` +} + +func (m *ReportBatchSplitResponse) Reset() { *m = ReportBatchSplitResponse{} } +func (m *ReportBatchSplitResponse) String() string { return proto.CompactTextString(m) } +func (*ReportBatchSplitResponse) ProtoMessage() {} +func (*ReportBatchSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{43} } + +func (m *ReportBatchSplitResponse) GetHeader() *ResponseHeader { + if m != nil { + return m.Header + } + return nil +} + type TimeInterval struct { // The unix timestamp in seconds of the start of this period. StartTimestamp uint64 `protobuf:"varint,1,opt,name=start_timestamp,json=startTimestamp,proto3" json:"start_timestamp,omitempty"` @@ -1182,7 +1352,7 @@ type TimeInterval struct { func (m *TimeInterval) Reset() { *m = TimeInterval{} } func (m *TimeInterval) String() string { return proto.CompactTextString(m) } func (*TimeInterval) ProtoMessage() {} -func (*TimeInterval) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{39} } +func (*TimeInterval) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{44} } func (m *TimeInterval) GetStartTimestamp() uint64 { if m != nil { @@ -1233,7 +1403,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{40} } +func (*StoreStats) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{45} } func (m *StoreStats) GetStoreId() uint64 { if m != nil { @@ -1348,7 +1518,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{41} } +func (*StoreHeartbeatRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{46} } func (m *StoreHeartbeatRequest) GetHeader() *RequestHeader { if m != nil { @@ -1371,7 +1541,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{42} } +func (*StoreHeartbeatResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{47} } func (m *StoreHeartbeatResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1392,7 +1562,7 @@ type ScatterRegionRequest struct { 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{43} } +func (*ScatterRegionRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{48} } func (m *ScatterRegionRequest) GetHeader() *RequestHeader { if m != nil { @@ -1429,7 +1599,7 @@ type ScatterRegionResponse struct { 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{44} } +func (*ScatterRegionResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{49} } func (m *ScatterRegionResponse) GetHeader() *ResponseHeader { if m != nil { @@ -1438,6 +1608,94 @@ func (m *ScatterRegionResponse) GetHeader() *ResponseHeader { return nil } +type GetGCSafePointRequest struct { + Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` +} + +func (m *GetGCSafePointRequest) Reset() { *m = GetGCSafePointRequest{} } +func (m *GetGCSafePointRequest) String() string { return proto.CompactTextString(m) } +func (*GetGCSafePointRequest) ProtoMessage() {} +func (*GetGCSafePointRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{50} } + +func (m *GetGCSafePointRequest) GetHeader() *RequestHeader { + if m != nil { + return m.Header + } + return nil +} + +type GetGCSafePointResponse struct { + Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + SafePoint uint64 `protobuf:"varint,2,opt,name=safe_point,json=safePoint,proto3" json:"safe_point,omitempty"` +} + +func (m *GetGCSafePointResponse) Reset() { *m = GetGCSafePointResponse{} } +func (m *GetGCSafePointResponse) String() string { return proto.CompactTextString(m) } +func (*GetGCSafePointResponse) ProtoMessage() {} +func (*GetGCSafePointResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{51} } + +func (m *GetGCSafePointResponse) GetHeader() *ResponseHeader { + if m != nil { + return m.Header + } + return nil +} + +func (m *GetGCSafePointResponse) GetSafePoint() uint64 { + if m != nil { + return m.SafePoint + } + return 0 +} + +type UpdateGCSafePointRequest struct { + Header *RequestHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + SafePoint uint64 `protobuf:"varint,2,opt,name=safe_point,json=safePoint,proto3" json:"safe_point,omitempty"` +} + +func (m *UpdateGCSafePointRequest) Reset() { *m = UpdateGCSafePointRequest{} } +func (m *UpdateGCSafePointRequest) String() string { return proto.CompactTextString(m) } +func (*UpdateGCSafePointRequest) ProtoMessage() {} +func (*UpdateGCSafePointRequest) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{52} } + +func (m *UpdateGCSafePointRequest) GetHeader() *RequestHeader { + if m != nil { + return m.Header + } + return nil +} + +func (m *UpdateGCSafePointRequest) GetSafePoint() uint64 { + if m != nil { + return m.SafePoint + } + return 0 +} + +type UpdateGCSafePointResponse struct { + Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` + NewSafePoint uint64 `protobuf:"varint,2,opt,name=new_safe_point,json=newSafePoint,proto3" json:"new_safe_point,omitempty"` +} + +func (m *UpdateGCSafePointResponse) Reset() { *m = UpdateGCSafePointResponse{} } +func (m *UpdateGCSafePointResponse) String() string { return proto.CompactTextString(m) } +func (*UpdateGCSafePointResponse) ProtoMessage() {} +func (*UpdateGCSafePointResponse) Descriptor() ([]byte, []int) { return fileDescriptorPdpb, []int{53} } + +func (m *UpdateGCSafePointResponse) GetHeader() *ResponseHeader { + if m != nil { + return m.Header + } + return nil +} + +func (m *UpdateGCSafePointResponse) GetNewSafePoint() uint64 { + if m != nil { + return m.NewSafePoint + } + return 0 +} + func init() { proto.RegisterType((*RequestHeader)(nil), "pdpb.RequestHeader") proto.RegisterType((*ResponseHeader)(nil), "pdpb.ResponseHeader") @@ -1478,13 +1736,23 @@ func init() { proto.RegisterType((*AskSplitResponse)(nil), "pdpb.AskSplitResponse") proto.RegisterType((*ReportSplitRequest)(nil), "pdpb.ReportSplitRequest") proto.RegisterType((*ReportSplitResponse)(nil), "pdpb.ReportSplitResponse") + proto.RegisterType((*AskBatchSplitRequest)(nil), "pdpb.AskBatchSplitRequest") + proto.RegisterType((*SplitID)(nil), "pdpb.SplitID") + proto.RegisterType((*AskBatchSplitResponse)(nil), "pdpb.AskBatchSplitResponse") + proto.RegisterType((*ReportBatchSplitRequest)(nil), "pdpb.ReportBatchSplitRequest") + proto.RegisterType((*ReportBatchSplitResponse)(nil), "pdpb.ReportBatchSplitResponse") proto.RegisterType((*TimeInterval)(nil), "pdpb.TimeInterval") 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.RegisterType((*GetGCSafePointRequest)(nil), "pdpb.GetGCSafePointRequest") + proto.RegisterType((*GetGCSafePointResponse)(nil), "pdpb.GetGCSafePointResponse") + proto.RegisterType((*UpdateGCSafePointRequest)(nil), "pdpb.UpdateGCSafePointRequest") + proto.RegisterType((*UpdateGCSafePointResponse)(nil), "pdpb.UpdateGCSafePointResponse") proto.RegisterEnum("pdpb.ErrorType", ErrorType_name, ErrorType_value) + proto.RegisterEnum("pdpb.CheckPolicy", CheckPolicy_name, CheckPolicy_value) } // Reference imports to suppress errors if they are not otherwise used. @@ -1511,12 +1779,17 @@ type PDClient interface { 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) + GetPrevRegion(ctx context.Context, in *GetRegionRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) GetRegionByID(ctx context.Context, in *GetRegionByIDRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) AskSplit(ctx context.Context, in *AskSplitRequest, opts ...grpc.CallOption) (*AskSplitResponse, error) ReportSplit(ctx context.Context, in *ReportSplitRequest, opts ...grpc.CallOption) (*ReportSplitResponse, error) + AskBatchSplit(ctx context.Context, in *AskBatchSplitRequest, opts ...grpc.CallOption) (*AskBatchSplitResponse, error) + ReportBatchSplit(ctx context.Context, in *ReportBatchSplitRequest, opts ...grpc.CallOption) (*ReportBatchSplitResponse, 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) + GetGCSafePoint(ctx context.Context, in *GetGCSafePointRequest, opts ...grpc.CallOption) (*GetGCSafePointResponse, error) + UpdateGCSafePoint(ctx context.Context, in *UpdateGCSafePointRequest, opts ...grpc.CallOption) (*UpdateGCSafePointResponse, error) } type pDClient struct { @@ -1670,6 +1943,15 @@ func (c *pDClient) GetRegion(ctx context.Context, in *GetRegionRequest, opts ... return out, nil } +func (c *pDClient) GetPrevRegion(ctx context.Context, in *GetRegionRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) { + out := new(GetRegionResponse) + err := grpc.Invoke(ctx, "/pdpb.PD/GetPrevRegion", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *pDClient) GetRegionByID(ctx context.Context, in *GetRegionByIDRequest, opts ...grpc.CallOption) (*GetRegionResponse, error) { out := new(GetRegionResponse) err := grpc.Invoke(ctx, "/pdpb.PD/GetRegionByID", in, out, c.cc, opts...) @@ -1697,6 +1979,24 @@ func (c *pDClient) ReportSplit(ctx context.Context, in *ReportSplitRequest, opts return out, nil } +func (c *pDClient) AskBatchSplit(ctx context.Context, in *AskBatchSplitRequest, opts ...grpc.CallOption) (*AskBatchSplitResponse, error) { + out := new(AskBatchSplitResponse) + err := grpc.Invoke(ctx, "/pdpb.PD/AskBatchSplit", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *pDClient) ReportBatchSplit(ctx context.Context, in *ReportBatchSplitRequest, opts ...grpc.CallOption) (*ReportBatchSplitResponse, error) { + out := new(ReportBatchSplitResponse) + err := grpc.Invoke(ctx, "/pdpb.PD/ReportBatchSplit", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *pDClient) GetClusterConfig(ctx context.Context, in *GetClusterConfigRequest, opts ...grpc.CallOption) (*GetClusterConfigResponse, error) { out := new(GetClusterConfigResponse) err := grpc.Invoke(ctx, "/pdpb.PD/GetClusterConfig", in, out, c.cc, opts...) @@ -1724,6 +2024,24 @@ func (c *pDClient) ScatterRegion(ctx context.Context, in *ScatterRegionRequest, return out, nil } +func (c *pDClient) GetGCSafePoint(ctx context.Context, in *GetGCSafePointRequest, opts ...grpc.CallOption) (*GetGCSafePointResponse, error) { + out := new(GetGCSafePointResponse) + err := grpc.Invoke(ctx, "/pdpb.PD/GetGCSafePoint", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *pDClient) UpdateGCSafePoint(ctx context.Context, in *UpdateGCSafePointRequest, opts ...grpc.CallOption) (*UpdateGCSafePointResponse, error) { + out := new(UpdateGCSafePointResponse) + err := grpc.Invoke(ctx, "/pdpb.PD/UpdateGCSafePoint", in, out, c.cc, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // Server API for PD service type PDServer interface { @@ -1740,12 +2058,17 @@ type PDServer interface { StoreHeartbeat(context.Context, *StoreHeartbeatRequest) (*StoreHeartbeatResponse, error) RegionHeartbeat(PD_RegionHeartbeatServer) error GetRegion(context.Context, *GetRegionRequest) (*GetRegionResponse, error) + GetPrevRegion(context.Context, *GetRegionRequest) (*GetRegionResponse, error) GetRegionByID(context.Context, *GetRegionByIDRequest) (*GetRegionResponse, error) AskSplit(context.Context, *AskSplitRequest) (*AskSplitResponse, error) ReportSplit(context.Context, *ReportSplitRequest) (*ReportSplitResponse, error) + AskBatchSplit(context.Context, *AskBatchSplitRequest) (*AskBatchSplitResponse, error) + ReportBatchSplit(context.Context, *ReportBatchSplitRequest) (*ReportBatchSplitResponse, error) GetClusterConfig(context.Context, *GetClusterConfigRequest) (*GetClusterConfigResponse, error) PutClusterConfig(context.Context, *PutClusterConfigRequest) (*PutClusterConfigResponse, error) ScatterRegion(context.Context, *ScatterRegionRequest) (*ScatterRegionResponse, error) + GetGCSafePoint(context.Context, *GetGCSafePointRequest) (*GetGCSafePointResponse, error) + UpdateGCSafePoint(context.Context, *UpdateGCSafePointRequest) (*UpdateGCSafePointResponse, error) } func RegisterPDServer(s *grpc.Server, srv PDServer) { @@ -1966,6 +2289,24 @@ func _PD_GetRegion_Handler(srv interface{}, ctx context.Context, dec func(interf return interceptor(ctx, in, info, handler) } +func _PD_GetPrevRegion_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetRegionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PDServer).GetPrevRegion(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pdpb.PD/GetPrevRegion", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PDServer).GetPrevRegion(ctx, req.(*GetRegionRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _PD_GetRegionByID_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(GetRegionByIDRequest) if err := dec(in); err != nil { @@ -2020,6 +2361,42 @@ func _PD_ReportSplit_Handler(srv interface{}, ctx context.Context, dec func(inte return interceptor(ctx, in, info, handler) } +func _PD_AskBatchSplit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AskBatchSplitRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PDServer).AskBatchSplit(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pdpb.PD/AskBatchSplit", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PDServer).AskBatchSplit(ctx, req.(*AskBatchSplitRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _PD_ReportBatchSplit_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReportBatchSplitRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PDServer).ReportBatchSplit(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pdpb.PD/ReportBatchSplit", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PDServer).ReportBatchSplit(ctx, req.(*ReportBatchSplitRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _PD_GetClusterConfig_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(GetClusterConfigRequest) if err := dec(in); err != nil { @@ -2074,6 +2451,42 @@ func _PD_ScatterRegion_Handler(srv interface{}, ctx context.Context, dec func(in return interceptor(ctx, in, info, handler) } +func _PD_GetGCSafePoint_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetGCSafePointRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PDServer).GetGCSafePoint(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pdpb.PD/GetGCSafePoint", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PDServer).GetGCSafePoint(ctx, req.(*GetGCSafePointRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _PD_UpdateGCSafePoint_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateGCSafePointRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PDServer).UpdateGCSafePoint(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/pdpb.PD/UpdateGCSafePoint", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PDServer).UpdateGCSafePoint(ctx, req.(*UpdateGCSafePointRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _PD_serviceDesc = grpc.ServiceDesc{ ServiceName: "pdpb.PD", HandlerType: (*PDServer)(nil), @@ -2114,6 +2527,10 @@ var _PD_serviceDesc = grpc.ServiceDesc{ MethodName: "GetRegion", Handler: _PD_GetRegion_Handler, }, + { + MethodName: "GetPrevRegion", + Handler: _PD_GetPrevRegion_Handler, + }, { MethodName: "GetRegionByID", Handler: _PD_GetRegionByID_Handler, @@ -2126,6 +2543,14 @@ var _PD_serviceDesc = grpc.ServiceDesc{ MethodName: "ReportSplit", Handler: _PD_ReportSplit_Handler, }, + { + MethodName: "AskBatchSplit", + Handler: _PD_AskBatchSplit_Handler, + }, + { + MethodName: "ReportBatchSplit", + Handler: _PD_ReportBatchSplit_Handler, + }, { MethodName: "GetClusterConfig", Handler: _PD_GetClusterConfig_Handler, @@ -2138,6 +2563,14 @@ var _PD_serviceDesc = grpc.ServiceDesc{ MethodName: "ScatterRegion", Handler: _PD_ScatterRegion_Handler, }, + { + MethodName: "GetGCSafePoint", + Handler: _PD_GetGCSafePoint_Handler, + }, + { + MethodName: "UpdateGCSafePoint", + Handler: _PD_UpdateGCSafePoint_Handler, + }, }, Streams: []grpc.StreamDesc{ { @@ -3289,8 +3722,13 @@ func (m *RegionHeartbeatRequest) MarshalTo(dAtA []byte) (int, error) { } i += n40 } - return i, nil -} + if m.ApproximateKeys != 0 { + dAtA[i] = 0x68 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.ApproximateKeys)) + } + return i, nil +} func (m *ChangePeer) Marshal() (dAtA []byte, err error) { size := m.Size() @@ -3396,6 +3834,11 @@ func (m *SplitRegion) MarshalTo(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Policy != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Policy)) + } return i, nil } @@ -3656,6 +4099,197 @@ func (m *ReportSplitResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *AskBatchSplitRequest) 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 *AskBatchSplitRequest) 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())) + n60, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n60 + } + if m.Region != nil { + dAtA[i] = 0x12 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Region.Size())) + n61, err := m.Region.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n61 + } + if m.SplitCount != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.SplitCount)) + } + return i, nil +} + +func (m *SplitID) 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 *SplitID) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.NewRegionId != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.NewRegionId)) + } + if len(m.NewPeerIds) > 0 { + dAtA63 := make([]byte, len(m.NewPeerIds)*10) + var j62 int + for _, num := range m.NewPeerIds { + for num >= 1<<7 { + dAtA63[j62] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j62++ + } + dAtA63[j62] = uint8(num) + j62++ + } + dAtA[i] = 0x12 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(j62)) + i += copy(dAtA[i:], dAtA63[:j62]) + } + return i, nil +} + +func (m *AskBatchSplitResponse) 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 *AskBatchSplitResponse) 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())) + n64, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n64 + } + if len(m.Ids) > 0 { + for _, msg := range m.Ids { + 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 *ReportBatchSplitRequest) 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 *ReportBatchSplitRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Header != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) + n65, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n65 + } + if len(m.Regions) > 0 { + for _, msg := range m.Regions { + 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 *ReportBatchSplitResponse) 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 *ReportBatchSplitResponse) 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())) + n66, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n66 + } + return i, nil +} + func (m *TimeInterval) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -3778,11 +4412,11 @@ func (m *StoreStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x7a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Interval.Size())) - n60, err := m.Interval.MarshalTo(dAtA[i:]) + n67, err := m.Interval.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n60 + i += n67 } return i, nil } @@ -3806,21 +4440,21 @@ func (m *StoreHeartbeatRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n61, err := m.Header.MarshalTo(dAtA[i:]) + n68, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n61 + i += n68 } if m.Stats != nil { dAtA[i] = 0x12 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Stats.Size())) - n62, err := m.Stats.MarshalTo(dAtA[i:]) + n69, err := m.Stats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n62 + i += n69 } return i, nil } @@ -3844,11 +4478,11 @@ func (m *StoreHeartbeatResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n63, err := m.Header.MarshalTo(dAtA[i:]) + n70, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n63 + i += n70 } return i, nil } @@ -3872,11 +4506,11 @@ func (m *ScatterRegionRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n64, err := m.Header.MarshalTo(dAtA[i:]) + n71, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n64 + i += n71 } if m.RegionId != 0 { dAtA[i] = 0x10 @@ -3887,21 +4521,21 @@ func (m *ScatterRegionRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Region.Size())) - n65, err := m.Region.MarshalTo(dAtA[i:]) + n72, err := m.Region.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n65 + i += n72 } if m.Leader != nil { dAtA[i] = 0x22 i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Leader.Size())) - n66, err := m.Leader.MarshalTo(dAtA[i:]) + n73, err := m.Leader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n66 + i += n73 } return i, nil } @@ -3925,82 +4559,209 @@ func (m *ScatterRegionResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) - n67, err := m.Header.MarshalTo(dAtA[i:]) + n74, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n67 + i += n74 } 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) - v >>= 7 - offset++ - } - dAtA[offset] = uint8(v) - return offset + 1 -} -func (m *RequestHeader) Size() (n int) { - var l int - _ = l - if m.ClusterId != 0 { - n += 1 + sovPdpb(uint64(m.ClusterId)) +func (m *GetGCSafePointRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *ResponseHeader) Size() (n int) { +func (m *GetGCSafePointRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l - if m.ClusterId != 0 { - n += 1 + sovPdpb(uint64(m.ClusterId)) - } - if m.Error != nil { - l = m.Error.Size() - n += 1 + l + sovPdpb(uint64(l)) + if m.Header != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) + n75, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n75 } - return n + return i, nil } -func (m *Error) Size() (n int) { - var l int - _ = l - if m.Type != 0 { - n += 1 + sovPdpb(uint64(m.Type)) - } - l = len(m.Message) - if l > 0 { - n += 1 + l + sovPdpb(uint64(l)) +func (m *GetGCSafePointResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err } - return n + return dAtA[:n], nil } -func (m *TsoRequest) Size() (n int) { +func (m *GetGCSafePointResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i var l int _ = l if m.Header != nil { - l = m.Header.Size() + dAtA[i] = 0xa + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.Header.Size())) + n76, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n76 + } + if m.SafePoint != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.SafePoint)) + } + return i, nil +} + +func (m *UpdateGCSafePointRequest) 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 *UpdateGCSafePointRequest) 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())) + n77, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n77 + } + if m.SafePoint != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.SafePoint)) + } + return i, nil +} + +func (m *UpdateGCSafePointResponse) 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 *UpdateGCSafePointResponse) 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())) + n78, err := m.Header.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n78 + } + if m.NewSafePoint != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintPdpb(dAtA, i, uint64(m.NewSafePoint)) + } + 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) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *RequestHeader) Size() (n int) { + var l int + _ = l + if m.ClusterId != 0 { + n += 1 + sovPdpb(uint64(m.ClusterId)) + } + return n +} + +func (m *ResponseHeader) Size() (n int) { + var l int + _ = l + if m.ClusterId != 0 { + n += 1 + sovPdpb(uint64(m.ClusterId)) + } + if m.Error != nil { + l = m.Error.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + return n +} + +func (m *Error) Size() (n int) { + var l int + _ = l + if m.Type != 0 { + n += 1 + sovPdpb(uint64(m.Type)) + } + l = len(m.Message) + if l > 0 { + n += 1 + l + sovPdpb(uint64(l)) + } + return n +} + +func (m *TsoRequest) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() n += 1 + l + sovPdpb(uint64(l)) } if m.Count != 0 { @@ -4403,6 +5164,9 @@ func (m *RegionHeartbeatRequest) Size() (n int) { l = m.Interval.Size() n += 1 + l + sovPdpb(uint64(l)) } + if m.ApproximateKeys != 0 { + n += 1 + sovPdpb(uint64(m.ApproximateKeys)) + } return n } @@ -4442,6 +5206,9 @@ func (m *Merge) Size() (n int) { func (m *SplitRegion) Size() (n int) { var l int _ = l + if m.Policy != 0 { + n += 1 + sovPdpb(uint64(m.Policy)) + } return n } @@ -4544,6 +5311,81 @@ func (m *ReportSplitResponse) Size() (n int) { return n } +func (m *AskBatchSplitRequest) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if m.Region != nil { + l = m.Region.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if m.SplitCount != 0 { + n += 1 + sovPdpb(uint64(m.SplitCount)) + } + return n +} + +func (m *SplitID) Size() (n int) { + var l int + _ = l + if m.NewRegionId != 0 { + n += 1 + sovPdpb(uint64(m.NewRegionId)) + } + if len(m.NewPeerIds) > 0 { + l = 0 + for _, e := range m.NewPeerIds { + l += sovPdpb(uint64(e)) + } + n += 1 + sovPdpb(uint64(l)) + l + } + return n +} + +func (m *AskBatchSplitResponse) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if len(m.Ids) > 0 { + for _, e := range m.Ids { + l = e.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + } + return n +} + +func (m *ReportBatchSplitRequest) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if len(m.Regions) > 0 { + for _, e := range m.Regions { + l = e.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + } + return n +} + +func (m *ReportBatchSplitResponse) 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 *TimeInterval) Size() (n int) { var l int _ = l @@ -4663,6 +5505,55 @@ func (m *ScatterRegionResponse) Size() (n int) { return n } +func (m *GetGCSafePointRequest) 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 *GetGCSafePointResponse) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if m.SafePoint != 0 { + n += 1 + sovPdpb(uint64(m.SafePoint)) + } + return n +} + +func (m *UpdateGCSafePointRequest) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if m.SafePoint != 0 { + n += 1 + sovPdpb(uint64(m.SafePoint)) + } + return n +} + +func (m *UpdateGCSafePointResponse) Size() (n int) { + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovPdpb(uint64(l)) + } + if m.NewSafePoint != 0 { + n += 1 + sovPdpb(uint64(m.NewSafePoint)) + } + return n +} + func sovPdpb(x uint64) (n int) { for { n++ @@ -8108,6 +8999,25 @@ func (m *RegionHeartbeatRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ApproximateKeys", wireType) + } + m.ApproximateKeys = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ApproximateKeys |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) @@ -8426,7 +9336,26 @@ func (m *SplitRegion) Unmarshal(dAtA []byte) error { return fmt.Errorf("proto: SplitRegion: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { - default: + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Policy", wireType) + } + m.Policy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Policy |= (CheckPolicy(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) if err != nil { @@ -9259,7 +10188,7 @@ func (m *ReportSplitResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *TimeInterval) Unmarshal(dAtA []byte) error { +func (m *AskBatchSplitRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9282,17 +10211,17 @@ func (m *TimeInterval) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TimeInterval: wiretype end group for non-group") + return fmt.Errorf("proto: AskBatchSplitRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: TimeInterval: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: AskBatchSplitRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StartTimestamp", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) } - m.StartTimestamp = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9302,16 +10231,63 @@ func (m *TimeInterval) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.StartTimestamp |= (uint64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Header == nil { + m.Header = &RequestHeader{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field 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 != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field EndTimestamp", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SplitCount", wireType) } - m.EndTimestamp = 0 + m.SplitCount = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9321,7 +10297,7 @@ func (m *TimeInterval) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.EndTimestamp |= (uint64(b) & 0x7F) << shift + m.SplitCount |= (uint32(b) & 0x7F) << shift if b < 0x80 { break } @@ -9347,7 +10323,7 @@ func (m *TimeInterval) Unmarshal(dAtA []byte) error { } return nil } -func (m *StoreStats) Unmarshal(dAtA []byte) error { +func (m *SplitID) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9370,17 +10346,17 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StoreStats: wiretype end group for non-group") + return fmt.Errorf("proto: SplitID: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StoreStats: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SplitID: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StoreId", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field NewRegionId", wireType) } - m.StoreId = 0 + m.NewRegionId = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9390,54 +10366,128 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.StoreId |= (uint64(b) & 0x7F) << shift + m.NewRegionId |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Capacity", wireType) - } - m.Capacity = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPdpb + if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { + if packedLen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + packedLen + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - m.Capacity |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break + for iNdEx < postIndex { + 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 == 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) } - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Available", wireType) + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err } - m.Available = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPdpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Available |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } + if skippy < 0 { + return ErrInvalidLengthPdpb } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field RegionCount", wireType) + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF } - m.RegionCount = 0 + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *AskBatchSplitResponse) 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: AskBatchSplitResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AskBatchSplitResponse: 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 @@ -9447,16 +10497,30 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RegionCount |= (uint32(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SendingSnapCount", wireType) + if msglen < 0 { + return ErrInvalidLengthPdpb } - m.SendingSnapCount = 0 + 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 Ids", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9466,16 +10530,78 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.SendingSnapCount |= (uint32(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ReceivingSnapCount", wireType) + if msglen < 0 { + return ErrInvalidLengthPdpb } - m.ReceivingSnapCount = 0 + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Ids = append(m.Ids, &SplitID{}) + if err := m.Ids[len(m.Ids)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPdpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ReportBatchSplitRequest) 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: ReportBatchSplitRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReportBatchSplitRequest: 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 @@ -9485,16 +10611,30 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ReceivingSnapCount |= (uint32(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 7: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field StartTime", wireType) + if msglen < 0 { + return ErrInvalidLengthPdpb } - m.StartTime = 0 + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Header == nil { + m.Header = &RequestHeader{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Regions", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9504,16 +10644,161 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.StartTime |= (uint32(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 8: + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Regions = append(m.Regions, &metapb.Region{}) + if err := m.Regions[len(m.Regions)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipPdpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthPdpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ReportBatchSplitResponse) 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: ReportBatchSplitResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReportBatchSplitResponse: 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 (m *TimeInterval) 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: TimeInterval: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TimeInterval: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ApplyingSnapCount", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StartTimestamp", wireType) } - m.ApplyingSnapCount = 0 + m.StartTimestamp = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9523,16 +10808,634 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ApplyingSnapCount |= (uint32(b) & 0x7F) << shift + m.StartTimestamp |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - case 9: + case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IsBusy", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field EndTimestamp", wireType) } - var v int + m.EndTimestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.EndTimestamp |= (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 *StoreStats) 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: StoreStats: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StoreStats: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StoreId", wireType) + } + m.StoreId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StoreId |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Capacity", wireType) + } + m.Capacity = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Capacity |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Available", wireType) + } + m.Available = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Available |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RegionCount", wireType) + } + m.RegionCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RegionCount |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SendingSnapCount", wireType) + } + m.SendingSnapCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SendingSnapCount |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceivingSnapCount", wireType) + } + m.ReceivingSnapCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ReceivingSnapCount |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartTime", wireType) + } + m.StartTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StartTime |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ApplyingSnapCount", wireType) + } + m.ApplyingSnapCount = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ApplyingSnapCount |= (uint32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsBusy", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.IsBusy = bool(v != 0) + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UsedSize", wireType) + } + m.UsedSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.UsedSize |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BytesWritten", wireType) + } + m.BytesWritten = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BytesWritten |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 12: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeysWritten", wireType) + } + m.KeysWritten = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.KeysWritten |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BytesRead", wireType) + } + m.BytesRead = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BytesRead |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field KeysRead", wireType) + } + m.KeysRead = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.KeysRead |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Interval", 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.Interval == nil { + m.Interval = &TimeInterval{} + } + if err := m.Interval.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 *StoreHeartbeatRequest) 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: StoreHeartbeatRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StoreHeartbeatRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPdpb + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Header == nil { + m.Header = &RequestHeader{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Stats", 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.Stats == nil { + m.Stats = &StoreStats{} + } + if err := m.Stats.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 *StoreHeartbeatResponse) 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: StoreHeartbeatResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StoreHeartbeatResponse: 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 (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 @@ -9542,55 +11445,30 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (int(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - m.IsBusy = bool(v != 0) - case 10: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field UsedSize", wireType) + if msglen < 0 { + return ErrInvalidLengthPdpb } - m.UsedSize = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPdpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.UsedSize |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF } - case 11: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field BytesWritten", wireType) + if m.Header == nil { + m.Header = &RequestHeader{} } - m.BytesWritten = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPdpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.BytesWritten |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - case 12: + iNdEx = postIndex + case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field KeysWritten", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field RegionId", wireType) } - m.KeysWritten = 0 + m.RegionId = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9600,16 +11478,16 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.KeysWritten |= (uint64(b) & 0x7F) << shift + m.RegionId |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - case 13: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field BytesRead", wireType) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Region", wireType) } - m.BytesRead = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9619,33 +11497,28 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.BytesRead |= (uint64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - case 14: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field KeysRead", wireType) + if msglen < 0 { + return ErrInvalidLengthPdpb } - m.KeysRead = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPdpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.KeysRead |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF } - case 15: + 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 Interval", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Leader", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -9669,10 +11542,10 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Interval == nil { - m.Interval = &TimeInterval{} + if m.Leader == nil { + m.Leader = &metapb.Peer{} } - if err := m.Interval.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Leader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -9697,7 +11570,7 @@ func (m *StoreStats) Unmarshal(dAtA []byte) error { } return nil } -func (m *StoreHeartbeatRequest) Unmarshal(dAtA []byte) error { +func (m *ScatterRegionResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9720,10 +11593,10 @@ func (m *StoreHeartbeatRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StoreHeartbeatRequest: wiretype end group for non-group") + return fmt.Errorf("proto: ScatterRegionResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StoreHeartbeatRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ScatterRegionResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9753,45 +11626,12 @@ func (m *StoreHeartbeatRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Header == nil { - m.Header = &RequestHeader{} + 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 Stats", 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.Stats == nil { - m.Stats = &StoreStats{} - } - if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) @@ -9813,7 +11653,7 @@ func (m *StoreHeartbeatRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *StoreHeartbeatResponse) Unmarshal(dAtA []byte) error { +func (m *GetGCSafePointRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9836,10 +11676,10 @@ func (m *StoreHeartbeatResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: StoreHeartbeatResponse: wiretype end group for non-group") + return fmt.Errorf("proto: GetGCSafePointRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: StoreHeartbeatResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetGCSafePointRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9869,7 +11709,7 @@ func (m *StoreHeartbeatResponse) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Header == nil { - m.Header = &ResponseHeader{} + m.Header = &RequestHeader{} } if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9896,7 +11736,7 @@ func (m *StoreHeartbeatResponse) Unmarshal(dAtA []byte) error { } return nil } -func (m *ScatterRegionRequest) Unmarshal(dAtA []byte) error { +func (m *GetGCSafePointResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9919,10 +11759,10 @@ func (m *ScatterRegionRequest) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ScatterRegionRequest: wiretype end group for non-group") + return fmt.Errorf("proto: GetGCSafePointResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ScatterRegionRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: GetGCSafePointResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -9952,7 +11792,7 @@ func (m *ScatterRegionRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Header == nil { - m.Header = &RequestHeader{} + m.Header = &ResponseHeader{} } if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9960,9 +11800,9 @@ func (m *ScatterRegionRequest) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field RegionId", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SafePoint", wireType) } - m.RegionId = 0 + m.SafePoint = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -9972,14 +11812,64 @@ func (m *ScatterRegionRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.RegionId |= (uint64(b) & 0x7F) << shift + m.SafePoint |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } } - case 3: + 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 *UpdateGCSafePointRequest) 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: UpdateGCSafePointRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: UpdateGCSafePointRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Region", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -10003,18 +11893,18 @@ func (m *ScatterRegionRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Region == nil { - m.Region = &metapb.Region{} + if m.Header == nil { + m.Header = &RequestHeader{} } - if err := m.Region.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Header.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) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SafePoint", wireType) } - var msglen int + m.SafePoint = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowPdpb @@ -10024,25 +11914,11 @@ func (m *ScatterRegionRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= (int(b) & 0x7F) << shift + m.SafePoint |= (uint64(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:]) @@ -10064,7 +11940,7 @@ func (m *ScatterRegionRequest) Unmarshal(dAtA []byte) error { } return nil } -func (m *ScatterRegionResponse) Unmarshal(dAtA []byte) error { +func (m *UpdateGCSafePointResponse) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -10087,10 +11963,10 @@ func (m *ScatterRegionResponse) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ScatterRegionResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateGCSafePointResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ScatterRegionResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateGCSafePointResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -10126,6 +12002,25 @@ func (m *ScatterRegionResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NewSafePoint", wireType) + } + m.NewSafePoint = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPdpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NewSafePoint |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipPdpb(dAtA[iNdEx:]) @@ -10255,131 +12150,151 @@ var ( func init() { proto.RegisterFile("pdpb.proto", fileDescriptorPdpb) } var fileDescriptorPdpb = []byte{ - // 2002 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x19, 0xcb, 0x6e, 0x23, 0xc7, - 0x51, 0x43, 0x91, 0x14, 0x59, 0x7c, 0x6e, 0xeb, 0xc5, 0xe5, 0xae, 0x14, 0xb9, 0xd7, 0x49, 0x64, - 0xc7, 0xa6, 0xd7, 0x4a, 0x10, 0x18, 0x30, 0x1c, 0x58, 0xaf, 0xdd, 0x95, 0xd7, 0x2b, 0x0a, 0x4d, - 0x1a, 0x86, 0x81, 0x20, 0xcc, 0x88, 0xd3, 0xa2, 0x26, 0x22, 0x67, 0xc6, 0xd3, 0x4d, 0x29, 0xf4, - 0x29, 0xa7, 0x5c, 0x12, 0x20, 0xb7, 0x20, 0x97, 0x00, 0xb9, 0x07, 0xc8, 0x2f, 0xe4, 0x9a, 0x63, - 0xee, 0xb9, 0x04, 0x9b, 0x1f, 0x09, 0xfa, 0x31, 0x4f, 0x52, 0xb2, 0x32, 0x1b, 0x9f, 0xc4, 0xa9, - 0xaa, 0xae, 0xae, 0x77, 0x55, 0x97, 0x00, 0x3c, 0xcb, 0x3b, 0xef, 0x78, 0xbe, 0xcb, 0x5d, 0x94, - 0x17, 0xbf, 0xdb, 0xd5, 0x09, 0xe5, 0x66, 0x00, 0x6b, 0xd7, 0xa8, 0x6f, 0x5e, 0xf0, 0xf0, 0x73, - 0x6d, 0xe4, 0x8e, 0x5c, 0xf9, 0xf3, 0x03, 0xf1, 0x4b, 0x41, 0x71, 0x07, 0x6a, 0x84, 0x7e, 0x3d, - 0xa5, 0x8c, 0xbf, 0xa0, 0xa6, 0x45, 0x7d, 0xb4, 0x05, 0x30, 0x1c, 0x4f, 0x19, 0xa7, 0xfe, 0xc0, - 0xb6, 0x5a, 0xc6, 0x8e, 0xb1, 0x9b, 0x27, 0x65, 0x0d, 0x39, 0xb1, 0x30, 0x81, 0x3a, 0xa1, 0xcc, - 0x73, 0x1d, 0x46, 0xef, 0x75, 0x00, 0xbd, 0x05, 0x05, 0xea, 0xfb, 0xae, 0xdf, 0xca, 0xed, 0x18, - 0xbb, 0x95, 0xbd, 0x4a, 0x47, 0x4a, 0x7d, 0x2c, 0x40, 0x44, 0x61, 0xf0, 0x33, 0x28, 0xc8, 0x6f, - 0xf4, 0x04, 0xf2, 0x7c, 0xe6, 0x51, 0xc9, 0xa4, 0xbe, 0xd7, 0x88, 0x91, 0xf6, 0x67, 0x1e, 0x25, - 0x12, 0x89, 0x5a, 0xb0, 0x32, 0xa1, 0x8c, 0x99, 0x23, 0x2a, 0x59, 0x96, 0x49, 0xf0, 0x89, 0xbb, - 0x00, 0x7d, 0xe6, 0x6a, 0x75, 0xd0, 0x8f, 0xa0, 0x78, 0x29, 0x25, 0x94, 0xec, 0x2a, 0x7b, 0xab, - 0x8a, 0x5d, 0x42, 0x5b, 0xa2, 0x49, 0xd0, 0x1a, 0x14, 0x86, 0xee, 0xd4, 0xe1, 0x92, 0x65, 0x8d, - 0xa8, 0x0f, 0xbc, 0x0f, 0xe5, 0xbe, 0x3d, 0xa1, 0x8c, 0x9b, 0x13, 0x0f, 0xb5, 0xa1, 0xe4, 0x5d, - 0xce, 0x98, 0x3d, 0x34, 0xc7, 0x92, 0xe3, 0x32, 0x09, 0xbf, 0x85, 0x4c, 0x63, 0x77, 0x24, 0x51, - 0x39, 0x89, 0x0a, 0x3e, 0xf1, 0x6f, 0x0c, 0xa8, 0x48, 0xa1, 0x94, 0xcd, 0xd0, 0x7b, 0x29, 0xa9, - 0xd6, 0x02, 0xa9, 0xe2, 0x36, 0xbd, 0x5b, 0x2c, 0xf4, 0x3e, 0x94, 0x79, 0x20, 0x56, 0x6b, 0x59, - 0xb2, 0xd1, 0xb6, 0x0a, 0xa5, 0x25, 0x11, 0x05, 0xfe, 0xbd, 0x01, 0xcd, 0x03, 0xd7, 0xe5, 0x8c, - 0xfb, 0xa6, 0x97, 0xc9, 0x3a, 0x4f, 0xa0, 0xc0, 0xb8, 0xeb, 0x53, 0xed, 0xc3, 0x5a, 0x47, 0xc7, - 0x59, 0x4f, 0x00, 0x89, 0xc2, 0xa1, 0x1f, 0x40, 0xd1, 0xa7, 0x23, 0xdb, 0x75, 0xb4, 0x48, 0xf5, - 0x80, 0x8a, 0x48, 0x28, 0xd1, 0x58, 0xbc, 0x0f, 0x0f, 0x62, 0xd2, 0x64, 0x31, 0x0b, 0x3e, 0x82, - 0xf5, 0x13, 0x16, 0x32, 0xf1, 0xa8, 0x95, 0x45, 0x2b, 0xfc, 0x2b, 0xd8, 0x48, 0x73, 0xc9, 0xe4, - 0x24, 0x0c, 0xd5, 0xf3, 0x18, 0x17, 0x69, 0xa4, 0x12, 0x49, 0xc0, 0xf0, 0x27, 0x50, 0xdf, 0x1f, - 0x8f, 0xdd, 0xe1, 0xc9, 0x51, 0x26, 0x51, 0xbb, 0xd0, 0x08, 0x8f, 0x67, 0x92, 0xb1, 0x0e, 0x39, - 0x5b, 0x49, 0x96, 0x27, 0x39, 0xdb, 0xc2, 0x5f, 0x41, 0xe3, 0x39, 0xe5, 0xca, 0x7f, 0x59, 0x22, - 0xe2, 0x21, 0x94, 0xa4, 0xd7, 0x07, 0x21, 0xd7, 0x15, 0xf9, 0x7d, 0x62, 0x61, 0x0a, 0xcd, 0x88, - 0x75, 0x26, 0x61, 0xef, 0x13, 0x6e, 0x78, 0x08, 0x8d, 0xb3, 0xe9, 0x1b, 0x68, 0x70, 0xaf, 0x4b, - 0x3e, 0x85, 0x66, 0x74, 0x49, 0xa6, 0x50, 0x3d, 0x80, 0xd5, 0xe7, 0x94, 0xef, 0x8f, 0xc7, 0x92, - 0x09, 0xcb, 0xe4, 0xfd, 0x2b, 0x58, 0x4b, 0xf2, 0xc8, 0x64, 0xd5, 0xef, 0x43, 0x51, 0x2a, 0xc5, - 0x5a, 0xb9, 0x9d, 0xe5, 0x79, 0x8d, 0x35, 0x12, 0xff, 0x42, 0xba, 0x4f, 0xe7, 0x6c, 0x16, 0xc3, - 0x6e, 0x01, 0xa8, 0x4c, 0x1f, 0x5c, 0xd1, 0x99, 0xb4, 0x6e, 0x95, 0x94, 0x15, 0xe4, 0x25, 0x9d, - 0xe1, 0x3f, 0x18, 0xf0, 0x20, 0x76, 0x41, 0x26, 0x55, 0xa2, 0x52, 0x93, 0xbb, 0xab, 0xd4, 0xa0, - 0xb7, 0xa1, 0x38, 0x56, 0x5c, 0x55, 0x49, 0xaa, 0x06, 0x74, 0x67, 0x54, 0x70, 0x53, 0x38, 0xfc, - 0x4b, 0x69, 0x5e, 0x75, 0xf4, 0x60, 0x96, 0x2d, 0x43, 0xd1, 0x23, 0xd0, 0x3a, 0x46, 0x19, 0x51, - 0x52, 0x80, 0x13, 0x0b, 0x3f, 0x83, 0xcd, 0xe7, 0x94, 0x1f, 0xaa, 0x9e, 0x78, 0xe8, 0x3a, 0x17, - 0xf6, 0x28, 0x53, 0x20, 0x30, 0x68, 0xcd, 0xf3, 0xc9, 0x64, 0xc1, 0x77, 0x60, 0x45, 0xb7, 0x68, - 0x6d, 0xc2, 0x46, 0x60, 0x1a, 0xcd, 0x9d, 0x04, 0x78, 0xfc, 0x35, 0x6c, 0x9e, 0x4d, 0xdf, 0x5c, - 0xf8, 0xff, 0xe5, 0xca, 0x17, 0xd0, 0x9a, 0xbf, 0x32, 0x53, 0xfa, 0xfd, 0xc5, 0x80, 0xe2, 0x2b, - 0x3a, 0x39, 0xa7, 0x3e, 0x42, 0x90, 0x77, 0xcc, 0x89, 0x1a, 0x2e, 0xca, 0x44, 0xfe, 0x16, 0x5e, - 0x9b, 0x48, 0x6c, 0xcc, 0x6b, 0x0a, 0x70, 0x62, 0x09, 0xa4, 0x47, 0xa9, 0x3f, 0x98, 0xfa, 0x63, - 0xd6, 0x5a, 0xde, 0x59, 0xde, 0x2d, 0x93, 0x92, 0x00, 0x7c, 0xe1, 0x8f, 0x19, 0xfa, 0x1e, 0x54, - 0x86, 0x63, 0x9b, 0x3a, 0x5c, 0xa1, 0xf3, 0x12, 0x0d, 0x0a, 0x24, 0x09, 0x7e, 0x08, 0x0d, 0x15, - 0x5f, 0x03, 0xcf, 0xb7, 0x5d, 0xdf, 0xe6, 0xb3, 0x56, 0x61, 0xc7, 0xd8, 0x2d, 0x90, 0xba, 0x02, - 0x9f, 0x69, 0x28, 0xfe, 0x54, 0xe6, 0x83, 0x12, 0x32, 0x5b, 0x7d, 0xf8, 0xbb, 0x01, 0x28, 0xce, - 0x22, 0x63, 0x4e, 0xad, 0x28, 0xcd, 0x83, 0xfa, 0x50, 0x55, 0xe4, 0x8a, 0x2b, 0x09, 0x90, 0x0b, - 0x72, 0x2a, 0x4e, 0xa6, 0x71, 0xe8, 0x7d, 0xa8, 0x50, 0x3e, 0xb4, 0x06, 0x9a, 0x34, 0xbf, 0x80, - 0x14, 0x04, 0xc1, 0xe7, 0x4a, 0x83, 0x33, 0x28, 0x8b, 0x94, 0xec, 0x71, 0x93, 0x33, 0xb4, 0x03, - 0x79, 0x61, 0x66, 0x2d, 0x75, 0x32, 0x67, 0x25, 0x06, 0xbd, 0x05, 0x55, 0xcb, 0xbd, 0x71, 0x06, - 0x8c, 0x0e, 0x5d, 0xc7, 0x62, 0xda, 0x73, 0x15, 0x01, 0xeb, 0x29, 0x10, 0xfe, 0xd7, 0x32, 0x6c, - 0xa8, 0x94, 0x7e, 0x41, 0x4d, 0x9f, 0x9f, 0x53, 0x93, 0x67, 0x8a, 0xda, 0xff, 0x6b, 0xa9, 0x41, - 0x1d, 0x00, 0x29, 0xb8, 0xd0, 0x42, 0x05, 0x4d, 0x38, 0xba, 0x85, 0xfa, 0x93, 0xb2, 0x20, 0x11, - 0x9f, 0x0c, 0x7d, 0x08, 0x35, 0x8f, 0x3a, 0x96, 0xed, 0x8c, 0xf4, 0x91, 0x82, 0x76, 0x4d, 0x9c, - 0x79, 0x55, 0x93, 0xa8, 0x23, 0x4f, 0xa0, 0x76, 0x3e, 0xe3, 0x94, 0x0d, 0x6e, 0x7c, 0x9b, 0x73, - 0xea, 0xb4, 0x8a, 0xd2, 0x38, 0x55, 0x09, 0xfc, 0x52, 0xc1, 0x44, 0x8d, 0x56, 0x44, 0x3e, 0x35, - 0xad, 0xd6, 0x8a, 0x9a, 0xd9, 0x25, 0x84, 0x50, 0x53, 0xcc, 0xec, 0xd5, 0x2b, 0x3a, 0x8b, 0x58, - 0x94, 0x94, 0x7d, 0x05, 0x2c, 0xe0, 0xf0, 0x08, 0xca, 0x92, 0x44, 0x32, 0x28, 0xab, 0xcc, 0x11, - 0x00, 0x79, 0xfe, 0x1d, 0x68, 0x9a, 0x9e, 0xe7, 0xbb, 0xbf, 0xb6, 0x27, 0x26, 0xa7, 0x03, 0x66, - 0x7f, 0x43, 0x5b, 0x20, 0x69, 0x1a, 0x31, 0x78, 0xcf, 0xfe, 0x86, 0xa2, 0x0e, 0x94, 0x6c, 0x87, - 0x53, 0xff, 0xda, 0x1c, 0xb7, 0xaa, 0xd2, 0x72, 0x28, 0x1a, 0x65, 0x4f, 0x34, 0x86, 0x84, 0x34, - 0x9f, 0xe5, 0x4b, 0x95, 0x66, 0x15, 0x5f, 0x02, 0x1c, 0x5e, 0x9a, 0xce, 0x88, 0x0a, 0x9d, 0xef, - 0x11, 0x30, 0x1f, 0x41, 0x65, 0x28, 0xe9, 0x07, 0xf2, 0x7d, 0x91, 0x93, 0xef, 0x8b, 0xcd, 0x4e, - 0xf0, 0x40, 0x12, 0x25, 0x46, 0xf1, 0x93, 0xef, 0x0c, 0x18, 0x86, 0xbf, 0xf1, 0x1e, 0xd4, 0xfb, - 0xbe, 0xe9, 0xb0, 0x0b, 0xea, 0xab, 0x58, 0xfd, 0xf6, 0xdb, 0xf0, 0x07, 0x50, 0x78, 0x45, 0xfd, - 0x91, 0x1c, 0x89, 0xb9, 0xe9, 0x8f, 0x28, 0xd7, 0xc4, 0x73, 0xc1, 0xa3, 0xb0, 0xb8, 0x06, 0x95, - 0x9e, 0x37, 0xb6, 0x75, 0x0f, 0xc2, 0x7f, 0x5c, 0x86, 0xcd, 0xb9, 0xd8, 0xcd, 0x94, 0xd4, 0x1f, - 0x86, 0x7a, 0x4b, 0x91, 0x55, 0x08, 0x37, 0xd5, 0x91, 0xc8, 0x80, 0x81, 0xc2, 0xd2, 0x98, 0x9f, - 0x40, 0x83, 0x6b, 0x85, 0x07, 0x89, 0x88, 0xd6, 0x37, 0x25, 0xad, 0x41, 0xea, 0x3c, 0x69, 0x9d, - 0x44, 0x1f, 0xcc, 0x27, 0xfb, 0x20, 0xfa, 0x29, 0x54, 0x35, 0x92, 0x7a, 0xee, 0xf0, 0x52, 0x16, - 0x44, 0x91, 0x7f, 0x09, 0xab, 0x1c, 0x0b, 0x14, 0xa9, 0xf8, 0xd1, 0x87, 0xa8, 0x26, 0xca, 0x52, - 0x4a, 0x8d, 0xe2, 0x02, 0xcb, 0x83, 0x22, 0x38, 0x53, 0xe5, 0xa1, 0x30, 0x11, 0xf6, 0x97, 0x81, - 0x1d, 0x3e, 0x39, 0xa5, 0x4b, 0x88, 0xc2, 0xa0, 0x9f, 0x40, 0x95, 0x09, 0x8b, 0x0f, 0x74, 0x72, - 0x97, 0x24, 0xe5, 0x03, 0x45, 0x19, 0xf3, 0x05, 0xa9, 0xb0, 0x98, 0x63, 0x2e, 0xa0, 0xb1, 0xcf, - 0xae, 0x34, 0xfa, 0xbb, 0x2b, 0x26, 0xf8, 0xb7, 0x06, 0x34, 0xa3, 0x8b, 0x32, 0x3e, 0x4a, 0x6a, - 0x0e, 0xbd, 0x19, 0xa4, 0x67, 0x92, 0x8a, 0x43, 0x6f, 0x48, 0xe0, 0x8e, 0x1d, 0xa8, 0x0a, 0x1a, - 0xd9, 0xe4, 0x6c, 0x4b, 0xf5, 0xb8, 0x3c, 0x01, 0x87, 0xde, 0x08, 0x33, 0x9e, 0x58, 0x0c, 0xff, - 0xce, 0x00, 0x44, 0xa8, 0xe7, 0xfa, 0x3c, 0xbb, 0xd2, 0x18, 0xf2, 0x63, 0x7a, 0xc1, 0x6f, 0x51, - 0x59, 0xe2, 0xd0, 0xdb, 0x50, 0xf0, 0xed, 0xd1, 0x25, 0xbf, 0xe5, 0xe9, 0xa8, 0x90, 0xf8, 0x10, - 0x56, 0x13, 0xc2, 0x64, 0x9a, 0x08, 0x7e, 0x0e, 0xd5, 0x78, 0x69, 0x11, 0x7d, 0x9a, 0x71, 0xd3, - 0xe7, 0x83, 0xe8, 0x49, 0xad, 0x76, 0x18, 0x75, 0x09, 0x8e, 0xde, 0xff, 0x4f, 0xa0, 0x46, 0x1d, - 0x2b, 0x46, 0xa6, 0x2c, 0x5a, 0xa5, 0x8e, 0x15, 0x12, 0xe1, 0x3f, 0xe7, 0x01, 0xe4, 0x3c, 0xad, - 0x5a, 0x59, 0xfc, 0x99, 0x64, 0x24, 0x9e, 0x49, 0xa8, 0x0d, 0xa5, 0xa1, 0xe9, 0x99, 0x43, 0x31, - 0x18, 0xe8, 0xc9, 0x23, 0xf8, 0x46, 0x8f, 0xa1, 0x6c, 0x5e, 0x9b, 0xf6, 0xd8, 0x3c, 0x1f, 0x53, - 0x69, 0x92, 0x3c, 0x89, 0x00, 0xa2, 0x3a, 0x6b, 0xb7, 0xaa, 0xdd, 0x40, 0x5e, 0xee, 0x06, 0x74, - 0xc2, 0x1c, 0xca, 0x0d, 0xc1, 0x7b, 0x80, 0x98, 0xee, 0x1b, 0xcc, 0x31, 0x3d, 0x4d, 0x58, 0x90, - 0x84, 0x4d, 0x8d, 0xe9, 0x39, 0xa6, 0xa7, 0xa8, 0x9f, 0xc2, 0x9a, 0x4f, 0x87, 0xd4, 0xbe, 0x4e, - 0xd1, 0x17, 0x25, 0x3d, 0x0a, 0x71, 0xd1, 0x89, 0x2d, 0x80, 0xc8, 0x68, 0x32, 0xcd, 0x6a, 0xa4, - 0x1c, 0xda, 0x0b, 0x75, 0x60, 0xd5, 0xf4, 0xbc, 0xf1, 0x2c, 0xc5, 0xaf, 0x24, 0xe9, 0x1e, 0x04, - 0xa8, 0x88, 0xdd, 0x26, 0xac, 0xd8, 0x6c, 0x70, 0x3e, 0x65, 0x33, 0xd9, 0x4a, 0x4a, 0xa4, 0x68, - 0xb3, 0x83, 0x29, 0x9b, 0x89, 0x6a, 0x32, 0x65, 0xd4, 0x8a, 0x77, 0x90, 0x92, 0x00, 0xc8, 0xd6, - 0x31, 0xd7, 0xe9, 0x2a, 0x0b, 0x3a, 0x5d, 0xba, 0x95, 0x55, 0xe7, 0x5b, 0x59, 0xb2, 0x19, 0xd6, - 0xd2, 0xcd, 0x30, 0xd1, 0xe9, 0xea, 0xa9, 0x4e, 0x17, 0x6f, 0x5f, 0x8d, 0x6f, 0x6f, 0x5f, 0x78, - 0x0c, 0xeb, 0x32, 0x3c, 0xde, 0x74, 0x28, 0x29, 0x30, 0x11, 0x5f, 0xc9, 0x82, 0x1e, 0xc5, 0x1d, - 0x51, 0x68, 0xfc, 0x0c, 0x36, 0xd2, 0xb7, 0x65, 0xca, 0x99, 0xbf, 0x19, 0xb0, 0xd6, 0x1b, 0x9a, - 0x5c, 0x0c, 0xe9, 0xd9, 0x1f, 0x86, 0x77, 0x3d, 0x91, 0xee, 0xbb, 0x3d, 0x8a, 0xcd, 0x59, 0xf9, - 0x3b, 0x9e, 0x74, 0xc7, 0xb0, 0x9e, 0x92, 0x37, 0x8b, 0xde, 0xef, 0x52, 0x28, 0x87, 0xdb, 0x47, - 0x54, 0x84, 0x5c, 0xf7, 0x65, 0x73, 0x09, 0x55, 0x60, 0xe5, 0x8b, 0xd3, 0x97, 0xa7, 0xdd, 0x2f, - 0x4f, 0x9b, 0x06, 0x5a, 0x83, 0xe6, 0x69, 0xb7, 0x3f, 0x38, 0xe8, 0x76, 0xfb, 0xbd, 0x3e, 0xd9, - 0x3f, 0x3b, 0x3b, 0x3e, 0x6a, 0xe6, 0xd0, 0x2a, 0x34, 0x7a, 0xfd, 0x2e, 0x39, 0x1e, 0xf4, 0xbb, - 0xaf, 0x0e, 0x7a, 0xfd, 0xee, 0xe9, 0x71, 0x73, 0x19, 0xb5, 0x60, 0x6d, 0xff, 0x73, 0x72, 0xbc, - 0x7f, 0xf4, 0x55, 0x92, 0x3c, 0xbf, 0xf7, 0xd7, 0x32, 0xe4, 0xce, 0x8e, 0xd0, 0x3e, 0x40, 0x34, - 0xc5, 0xa3, 0x4d, 0x25, 0xd9, 0xdc, 0xd3, 0xa0, 0xdd, 0x9a, 0x47, 0x28, 0xe1, 0xf1, 0x12, 0x7a, - 0x0a, 0xcb, 0x7d, 0xe6, 0x22, 0x1d, 0x10, 0xd1, 0x32, 0xb4, 0xfd, 0x20, 0x06, 0x09, 0xa8, 0x77, - 0x8d, 0xa7, 0x06, 0xfa, 0x19, 0x94, 0xc3, 0x15, 0x18, 0xda, 0x50, 0x54, 0xe9, 0x65, 0x61, 0x7b, - 0x73, 0x0e, 0x1e, 0xde, 0xf8, 0x0a, 0xea, 0xc9, 0x25, 0x1a, 0x7a, 0xa4, 0x88, 0x17, 0x2e, 0xe8, - 0xda, 0x8f, 0x17, 0x23, 0x43, 0x76, 0x1f, 0xc1, 0x8a, 0x5e, 0x74, 0x21, 0xed, 0x9a, 0xe4, 0xda, - 0xac, 0xbd, 0x9e, 0x82, 0x86, 0x27, 0x3f, 0x86, 0x52, 0xb0, 0x76, 0x42, 0xeb, 0xa1, 0x89, 0xe2, - 0xfb, 0xa1, 0xf6, 0x46, 0x1a, 0x1c, 0x3f, 0x1c, 0xec, 0x79, 0x82, 0xc3, 0xa9, 0xe5, 0x52, 0x70, - 0x38, 0xbd, 0x0e, 0xc2, 0x4b, 0xe8, 0x39, 0x54, 0xe3, 0xeb, 0x19, 0xf4, 0x30, 0xbc, 0x26, 0xbd, - 0xf6, 0x69, 0xb7, 0x17, 0xa1, 0xe2, 0xb6, 0x4c, 0xa6, 0x6b, 0x60, 0xcb, 0x85, 0x25, 0x23, 0xb0, - 0xe5, 0xe2, 0x0c, 0xc7, 0x4b, 0xa8, 0x0f, 0x8d, 0xd4, 0x14, 0x89, 0x1e, 0x07, 0xe1, 0xbe, 0xe8, - 0x61, 0xd4, 0xde, 0xba, 0x05, 0x9b, 0x0e, 0x98, 0x70, 0x5b, 0x82, 0x22, 0x8b, 0x26, 0xea, 0x42, - 0x7b, 0x73, 0x0e, 0x1e, 0x4a, 0xf5, 0x0c, 0x6a, 0x89, 0x6d, 0x0b, 0x6a, 0xa7, 0x68, 0x63, 0x2b, - 0x98, 0xbb, 0xf8, 0x7c, 0x0c, 0xa5, 0x60, 0x44, 0x0a, 0x5c, 0x96, 0x9a, 0xcd, 0x02, 0x97, 0xa5, - 0x27, 0x29, 0xbc, 0x84, 0x8e, 0xa0, 0x12, 0x9b, 0x24, 0x50, 0x2b, 0x50, 0x3c, 0x3d, 0xe9, 0xb4, - 0x1f, 0x2e, 0xc0, 0x84, 0x5c, 0x7a, 0x72, 0x55, 0x96, 0x58, 0x53, 0xa0, 0xad, 0x50, 0xe2, 0x45, - 0x1b, 0x93, 0xf6, 0xf6, 0x6d, 0xe8, 0x38, 0xd3, 0xf4, 0xee, 0x23, 0x60, 0x7a, 0xcb, 0x1a, 0x26, - 0x60, 0x7a, 0xdb, 0xca, 0x04, 0x2f, 0xa1, 0xcf, 0xa0, 0x96, 0xa8, 0x87, 0x81, 0xd1, 0x17, 0x15, - 0xf5, 0xf6, 0xa3, 0x85, 0xb8, 0x80, 0xd7, 0xc1, 0xbb, 0xff, 0x78, 0xbd, 0x6d, 0xfc, 0xf3, 0xf5, - 0xb6, 0xf1, 0xef, 0xd7, 0xdb, 0xc6, 0x9f, 0xfe, 0xb3, 0xbd, 0x04, 0xad, 0xa1, 0x3b, 0xe9, 0x78, - 0xb6, 0x33, 0x1a, 0x9a, 0x5e, 0x87, 0xdb, 0x57, 0xd7, 0x9d, 0xab, 0x6b, 0xf9, 0xdf, 0xa5, 0xf3, - 0xa2, 0xfc, 0xf3, 0xe3, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0xdc, 0x50, 0x39, 0xcf, 0xab, 0x1a, - 0x00, 0x00, + // 2328 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x5a, 0x5b, 0x6f, 0x1b, 0xc7, + 0x15, 0xd6, 0x52, 0x24, 0x45, 0x1e, 0x5e, 0x35, 0xd6, 0x85, 0xa6, 0x6f, 0xca, 0xd8, 0x6d, 0x65, + 0x37, 0x61, 0x1c, 0xb7, 0x28, 0x0c, 0x14, 0x09, 0x42, 0x5d, 0x2c, 0x33, 0xb6, 0x44, 0x62, 0x48, + 0x27, 0x0d, 0x50, 0x94, 0x5d, 0x91, 0x23, 0x6a, 0x2b, 0x6a, 0x77, 0xb3, 0x3b, 0x92, 0xcb, 0xbc, + 0xb4, 0x4f, 0x41, 0x81, 0x06, 0xe8, 0x5b, 0xd1, 0x97, 0x02, 0xfd, 0x05, 0xfd, 0x07, 0x45, 0x5f, + 0xfb, 0xd8, 0x9f, 0x50, 0xb8, 0x7f, 0xa4, 0x98, 0xdb, 0xde, 0x48, 0xd9, 0xee, 0x3a, 0x79, 0x32, + 0xf7, 0x9c, 0x33, 0xdf, 0xcc, 0x7c, 0xe7, 0x9c, 0x99, 0x33, 0x47, 0x06, 0x70, 0xc7, 0xee, 0x71, + 0xcb, 0xf5, 0x1c, 0xe6, 0xa0, 0x2c, 0xff, 0xdd, 0x2c, 0x9f, 0x53, 0x66, 0x6a, 0x59, 0xb3, 0x42, + 0x3d, 0xf3, 0x84, 0x05, 0x9f, 0x6b, 0x13, 0x67, 0xe2, 0x88, 0x9f, 0x1f, 0xf2, 0x5f, 0x52, 0x8a, + 0x5b, 0x50, 0x21, 0xf4, 0xab, 0x0b, 0xea, 0xb3, 0xa7, 0xd4, 0x1c, 0x53, 0x0f, 0xdd, 0x02, 0x18, + 0x4d, 0x2f, 0x7c, 0x46, 0xbd, 0xa1, 0x35, 0x6e, 0x18, 0x5b, 0xc6, 0x76, 0x96, 0x14, 0x95, 0xa4, + 0x33, 0xc6, 0x04, 0xaa, 0x84, 0xfa, 0xae, 0x63, 0xfb, 0xf4, 0xad, 0x06, 0xa0, 0xf7, 0x20, 0x47, + 0x3d, 0xcf, 0xf1, 0x1a, 0x99, 0x2d, 0x63, 0xbb, 0xf4, 0xa8, 0xd4, 0x12, 0xab, 0xde, 0xe7, 0x22, + 0x22, 0x35, 0xf8, 0x09, 0xe4, 0xc4, 0x37, 0xba, 0x0b, 0x59, 0x36, 0x73, 0xa9, 0x00, 0xa9, 0x3e, + 0xaa, 0x45, 0x4c, 0x07, 0x33, 0x97, 0x12, 0xa1, 0x44, 0x0d, 0x58, 0x39, 0xa7, 0xbe, 0x6f, 0x4e, + 0xa8, 0x80, 0x2c, 0x12, 0xfd, 0x89, 0xbb, 0x00, 0x03, 0xdf, 0x51, 0xdb, 0x41, 0x3f, 0x86, 0xfc, + 0xa9, 0x58, 0xa1, 0x80, 0x2b, 0x3d, 0xba, 0x26, 0xe1, 0x62, 0xbb, 0x25, 0xca, 0x04, 0xad, 0x41, + 0x6e, 0xe4, 0x5c, 0xd8, 0x4c, 0x40, 0x56, 0x88, 0xfc, 0xc0, 0x6d, 0x28, 0x0e, 0xac, 0x73, 0xea, + 0x33, 0xf3, 0xdc, 0x45, 0x4d, 0x28, 0xb8, 0xa7, 0x33, 0xdf, 0x1a, 0x99, 0x53, 0x81, 0xb8, 0x4c, + 0x82, 0x6f, 0xbe, 0xa6, 0xa9, 0x33, 0x11, 0xaa, 0x8c, 0x50, 0xe9, 0x4f, 0xfc, 0x7b, 0x03, 0x4a, + 0x62, 0x51, 0x92, 0x33, 0xf4, 0x7e, 0x62, 0x55, 0x6b, 0x7a, 0x55, 0x51, 0x4e, 0x5f, 0xbf, 0x2c, + 0xf4, 0x01, 0x14, 0x99, 0x5e, 0x56, 0x63, 0x59, 0xc0, 0x28, 0xae, 0x82, 0xd5, 0x92, 0xd0, 0x02, + 0x7f, 0x6b, 0x40, 0x7d, 0xc7, 0x71, 0x98, 0xcf, 0x3c, 0xd3, 0x4d, 0xc5, 0xce, 0x5d, 0xc8, 0xf9, + 0xcc, 0xf1, 0xa8, 0xf2, 0x61, 0xa5, 0xa5, 0xe2, 0xac, 0xcf, 0x85, 0x44, 0xea, 0xd0, 0x0f, 0x21, + 0xef, 0xd1, 0x89, 0xe5, 0xd8, 0x6a, 0x49, 0x55, 0x6d, 0x45, 0x84, 0x94, 0x28, 0x2d, 0x6e, 0xc3, + 0x6a, 0x64, 0x35, 0x69, 0x68, 0xc1, 0x7b, 0xb0, 0xde, 0xf1, 0x03, 0x10, 0x97, 0x8e, 0xd3, 0xec, + 0x0a, 0xff, 0x06, 0x36, 0x92, 0x28, 0xa9, 0x9c, 0x84, 0xa1, 0x7c, 0x1c, 0x41, 0x11, 0x24, 0x15, + 0x48, 0x4c, 0x86, 0x3f, 0x86, 0x6a, 0x7b, 0x3a, 0x75, 0x46, 0x9d, 0xbd, 0x54, 0x4b, 0xed, 0x42, + 0x2d, 0x18, 0x9e, 0x6a, 0x8d, 0x55, 0xc8, 0x58, 0x72, 0x65, 0x59, 0x92, 0xb1, 0xc6, 0xf8, 0x4b, + 0xa8, 0x1d, 0x50, 0x26, 0xfd, 0x97, 0x26, 0x22, 0xae, 0x43, 0x41, 0x78, 0x7d, 0x18, 0xa0, 0xae, + 0x88, 0xef, 0xce, 0x18, 0x53, 0xa8, 0x87, 0xd0, 0xa9, 0x16, 0xfb, 0x36, 0xe1, 0x86, 0x47, 0x50, + 0xeb, 0x5d, 0xbc, 0xc3, 0x0e, 0xde, 0x6a, 0x92, 0x4f, 0xa1, 0x1e, 0x4e, 0x92, 0x2a, 0x54, 0x77, + 0xe0, 0xda, 0x01, 0x65, 0xed, 0xe9, 0x54, 0x80, 0xf8, 0xa9, 0xbc, 0x7f, 0x06, 0x6b, 0x71, 0x8c, + 0x54, 0xac, 0xfe, 0x00, 0xf2, 0x62, 0x53, 0x7e, 0x23, 0xb3, 0xb5, 0x3c, 0xbf, 0x63, 0xa5, 0xc4, + 0xbf, 0x12, 0xee, 0x53, 0x39, 0x9b, 0x86, 0xd8, 0x5b, 0x00, 0x32, 0xd3, 0x87, 0x67, 0x74, 0x26, + 0xd8, 0x2d, 0x93, 0xa2, 0x94, 0x3c, 0xa3, 0x33, 0xfc, 0x27, 0x03, 0x56, 0x23, 0x13, 0xa4, 0xda, + 0x4a, 0x78, 0xd4, 0x64, 0x5e, 0x77, 0xd4, 0xa0, 0x7b, 0x90, 0x9f, 0x4a, 0x54, 0x79, 0x24, 0x95, + 0xb5, 0x5d, 0x8f, 0x72, 0x34, 0xa9, 0xc3, 0xbf, 0x16, 0xf4, 0xca, 0xa1, 0x3b, 0xb3, 0x74, 0x19, + 0x8a, 0x6e, 0x80, 0xda, 0x63, 0x98, 0x11, 0x05, 0x29, 0xe8, 0x8c, 0xf1, 0x13, 0xd8, 0x3c, 0xa0, + 0x6c, 0x57, 0xde, 0x89, 0xbb, 0x8e, 0x7d, 0x62, 0x4d, 0x52, 0x05, 0x82, 0x0f, 0x8d, 0x79, 0x9c, + 0x54, 0x0c, 0xde, 0x87, 0x15, 0x75, 0x45, 0x2b, 0x0a, 0x6b, 0x9a, 0x1a, 0x85, 0x4e, 0xb4, 0x1e, + 0x7f, 0x05, 0x9b, 0xbd, 0x8b, 0x77, 0x5f, 0xfc, 0xff, 0x33, 0xe5, 0x53, 0x68, 0xcc, 0x4f, 0x99, + 0x2a, 0xfd, 0xfe, 0x66, 0x40, 0xfe, 0x90, 0x9e, 0x1f, 0x53, 0x0f, 0x21, 0xc8, 0xda, 0xe6, 0xb9, + 0x2c, 0x2e, 0x8a, 0x44, 0xfc, 0xe6, 0x5e, 0x3b, 0x17, 0xda, 0x88, 0xd7, 0xa4, 0xa0, 0x33, 0xe6, + 0x4a, 0x97, 0x52, 0x6f, 0x78, 0xe1, 0x4d, 0xfd, 0xc6, 0xf2, 0xd6, 0xf2, 0x76, 0x91, 0x14, 0xb8, + 0xe0, 0x85, 0x37, 0xf5, 0xd1, 0x1d, 0x28, 0x8d, 0xa6, 0x16, 0xb5, 0x99, 0x54, 0x67, 0x85, 0x1a, + 0xa4, 0x48, 0x18, 0xfc, 0x08, 0x6a, 0x32, 0xbe, 0x86, 0xae, 0x67, 0x39, 0x9e, 0xc5, 0x66, 0x8d, + 0xdc, 0x96, 0xb1, 0x9d, 0x23, 0x55, 0x29, 0xee, 0x29, 0x29, 0xfe, 0x54, 0xe4, 0x83, 0x5c, 0x64, + 0xba, 0xf3, 0xe1, 0x9f, 0x06, 0xa0, 0x28, 0x44, 0xca, 0x9c, 0x5a, 0x91, 0x3b, 0xd7, 0xe7, 0x43, + 0x59, 0x9a, 0x4b, 0x54, 0xa2, 0x95, 0x0b, 0x72, 0x2a, 0x6a, 0xa6, 0x74, 0xe8, 0x03, 0x28, 0x51, + 0x36, 0x1a, 0x0f, 0x95, 0x69, 0x76, 0x81, 0x29, 0x70, 0x83, 0xe7, 0x72, 0x07, 0x3d, 0x28, 0xf2, + 0x94, 0xec, 0x33, 0x93, 0xf9, 0x68, 0x0b, 0xb2, 0x9c, 0x66, 0xb5, 0xea, 0x78, 0xce, 0x0a, 0x0d, + 0x7a, 0x0f, 0xca, 0x63, 0xe7, 0xa5, 0x3d, 0xf4, 0xe9, 0xc8, 0xb1, 0xc7, 0xbe, 0xf2, 0x5c, 0x89, + 0xcb, 0xfa, 0x52, 0x84, 0xbf, 0xc9, 0xc2, 0x86, 0x4c, 0xe9, 0xa7, 0xd4, 0xf4, 0xd8, 0x31, 0x35, + 0x59, 0xaa, 0xa8, 0xfd, 0x4e, 0x8f, 0x1a, 0xd4, 0x02, 0x10, 0x0b, 0xe7, 0xbb, 0x90, 0x41, 0x13, + 0x94, 0x6e, 0xc1, 0xfe, 0x49, 0x91, 0x9b, 0xf0, 0x4f, 0x1f, 0x7d, 0x04, 0x15, 0x97, 0xda, 0x63, + 0xcb, 0x9e, 0xa8, 0x21, 0x39, 0xe5, 0x9a, 0x28, 0x78, 0x59, 0x99, 0xc8, 0x21, 0x77, 0xa1, 0x72, + 0x3c, 0x63, 0xd4, 0x1f, 0xbe, 0xf4, 0x2c, 0xc6, 0xa8, 0xdd, 0xc8, 0x0b, 0x72, 0xca, 0x42, 0xf8, + 0x85, 0x94, 0xf1, 0x33, 0x5a, 0x1a, 0x79, 0xd4, 0x1c, 0x37, 0x56, 0x64, 0xcd, 0x2e, 0x24, 0x84, + 0x9a, 0xbc, 0x66, 0x2f, 0x9f, 0xd1, 0x59, 0x08, 0x51, 0x90, 0xfc, 0x72, 0x99, 0x46, 0xb8, 0x01, + 0x45, 0x61, 0x22, 0x00, 0x8a, 0x32, 0x73, 0xb8, 0x40, 0x8c, 0xbf, 0x0f, 0x75, 0xd3, 0x75, 0x3d, + 0xe7, 0xb7, 0xd6, 0xb9, 0xc9, 0xe8, 0xd0, 0xb7, 0xbe, 0xa6, 0x0d, 0x10, 0x36, 0xb5, 0x88, 0xbc, + 0x6f, 0x7d, 0x4d, 0x51, 0x0b, 0x0a, 0x96, 0xcd, 0xa8, 0x77, 0x69, 0x4e, 0x1b, 0x65, 0xc1, 0x1c, + 0x0a, 0x4b, 0xd9, 0x8e, 0xd2, 0x90, 0xc0, 0x26, 0x09, 0xcd, 0xa7, 0x6c, 0x54, 0xe6, 0xa0, 0x9f, + 0xd1, 0x99, 0xff, 0x59, 0xb6, 0x50, 0xaa, 0x97, 0xf1, 0x29, 0xc0, 0xee, 0xa9, 0x69, 0x4f, 0x28, + 0xa7, 0xe7, 0x2d, 0x62, 0xeb, 0x31, 0x94, 0x46, 0xc2, 0x7e, 0x28, 0x9e, 0x22, 0x19, 0xf1, 0x14, + 0xd9, 0x6c, 0xe9, 0xb7, 0x14, 0x3f, 0x8d, 0x24, 0x9e, 0x78, 0x92, 0xc0, 0x28, 0xf8, 0x8d, 0x1f, + 0x41, 0x75, 0xe0, 0x99, 0xb6, 0x7f, 0x42, 0x3d, 0x19, 0xd6, 0x6f, 0x9e, 0x0d, 0x7f, 0x08, 0xb9, + 0x43, 0xea, 0x4d, 0x44, 0xf5, 0xcc, 0x4c, 0x6f, 0x42, 0x99, 0x32, 0x9e, 0x8b, 0x33, 0xa9, 0xc5, + 0x8f, 0xa1, 0xd4, 0x77, 0xa7, 0x96, 0xba, 0xae, 0xd0, 0x7d, 0xc8, 0xbb, 0xce, 0xd4, 0x1a, 0xcd, + 0xd4, 0x9b, 0x69, 0x55, 0x92, 0xb7, 0x7b, 0x4a, 0x47, 0x67, 0x3d, 0xa1, 0x20, 0xca, 0x00, 0xff, + 0x79, 0x19, 0x36, 0xe7, 0x32, 0x22, 0xd5, 0x51, 0xf1, 0x51, 0x40, 0x91, 0xd8, 0x9d, 0x4c, 0x8c, + 0xba, 0x9e, 0x59, 0x73, 0xad, 0xb9, 0x11, 0xbc, 0x7f, 0x0c, 0x35, 0xa6, 0xb8, 0x19, 0xc6, 0xf2, + 0x44, 0xcd, 0x14, 0x27, 0x8e, 0x54, 0x59, 0x9c, 0xc8, 0xd8, 0xed, 0x9a, 0x8d, 0xdf, 0xae, 0xe8, + 0x67, 0x50, 0x56, 0x4a, 0xea, 0x3a, 0xa3, 0x53, 0x71, 0xcc, 0xf2, 0xac, 0x8e, 0x11, 0xb8, 0xcf, + 0x55, 0xa4, 0xe4, 0x85, 0x1f, 0xfc, 0x8c, 0x92, 0xa4, 0xca, 0x6d, 0xe4, 0x17, 0x38, 0x09, 0xa4, + 0x41, 0x4f, 0x1e, 0x3a, 0xb9, 0x73, 0xee, 0x2a, 0x91, 0x2e, 0xc1, 0x43, 0x56, 0x78, 0x8f, 0x48, + 0x0d, 0xfa, 0x29, 0x94, 0x7d, 0xee, 0x9c, 0xa1, 0x3a, 0x32, 0x0a, 0xc2, 0x52, 0xf9, 0x24, 0xe2, + 0x36, 0x52, 0xf2, 0xc3, 0x0f, 0x7c, 0x02, 0xb5, 0xb6, 0x7f, 0xa6, 0xd4, 0xdf, 0xdf, 0x11, 0x85, + 0xbf, 0x31, 0xa0, 0x1e, 0x4e, 0x94, 0xf2, 0xa9, 0x53, 0xb1, 0xe9, 0xcb, 0x61, 0xb2, 0xd2, 0x29, + 0xd9, 0xf4, 0x25, 0xd1, 0xee, 0xd8, 0x82, 0x32, 0xb7, 0x11, 0x57, 0xa7, 0x35, 0x96, 0x37, 0x67, + 0x96, 0x80, 0x4d, 0x5f, 0x72, 0x1a, 0x3b, 0x63, 0x1f, 0xff, 0xd1, 0x00, 0x44, 0xa8, 0xeb, 0x78, + 0x2c, 0xfd, 0xa6, 0x31, 0x64, 0xa7, 0xf4, 0x84, 0x5d, 0xb1, 0x65, 0xa1, 0x43, 0xf7, 0x20, 0xe7, + 0x59, 0x93, 0x53, 0x76, 0xc5, 0x83, 0x54, 0x2a, 0xf1, 0x2e, 0x5c, 0x8b, 0x2d, 0x26, 0x55, 0x9d, + 0xf1, 0xad, 0x01, 0x6b, 0x6d, 0xff, 0x6c, 0xc7, 0x64, 0xa3, 0xd3, 0xef, 0xdd, 0x93, 0xbc, 0xf8, + 0x90, 0x71, 0x26, 0x9b, 0x03, 0xcb, 0xa2, 0x39, 0x00, 0x42, 0xb4, 0x2b, 0x1a, 0x17, 0x5d, 0x58, + 0x11, 0xab, 0xe8, 0xec, 0xcd, 0xbb, 0xcc, 0x78, 0xb3, 0xcb, 0x32, 0x73, 0x2e, 0x3b, 0x81, 0xf5, + 0xc4, 0xf6, 0x52, 0xc5, 0xcf, 0x1d, 0x58, 0xd6, 0xf8, 0xfc, 0x01, 0x12, 0xe6, 0x45, 0x67, 0x8f, + 0x70, 0x0d, 0x76, 0xf9, 0x19, 0xc5, 0x9d, 0xf1, 0x8e, 0x4c, 0x6e, 0xc3, 0x8a, 0xdc, 0xb1, 0x9e, + 0x2c, 0x49, 0xa5, 0x56, 0xf3, 0x5a, 0x73, 0x7e, 0xc6, 0x54, 0x31, 0xf0, 0x4b, 0x28, 0x47, 0x2f, + 0x2d, 0x5e, 0x01, 0xfa, 0xcc, 0xf4, 0xd8, 0x30, 0x6c, 0xd6, 0x48, 0xee, 0xab, 0x42, 0x1c, 0x76, + 0x96, 0xee, 0x42, 0x85, 0xda, 0xe3, 0x88, 0x99, 0xcc, 0xaa, 0x32, 0xb5, 0xc7, 0x81, 0x11, 0xfe, + 0x6b, 0x16, 0x40, 0xbc, 0xd4, 0x64, 0x91, 0x14, 0x7d, 0x80, 0x1b, 0xb1, 0x07, 0x38, 0x6a, 0x42, + 0x61, 0x64, 0xba, 0xe6, 0x88, 0x97, 0x9c, 0xaa, 0xa6, 0xd5, 0xdf, 0xe8, 0x26, 0x14, 0xcd, 0x4b, + 0xd3, 0x9a, 0x9a, 0xc7, 0x53, 0x2a, 0xe2, 0x26, 0x4b, 0x42, 0x01, 0xbf, 0xf7, 0x55, 0x9c, 0xc8, + 0xc0, 0xca, 0x8a, 0xc0, 0x52, 0x87, 0xa6, 0x88, 0x2c, 0xf4, 0x3e, 0x20, 0x5f, 0x55, 0x24, 0xbe, + 0x6d, 0xba, 0xca, 0x30, 0x27, 0x0c, 0xeb, 0x4a, 0xd3, 0xb7, 0x4d, 0x57, 0x5a, 0x3f, 0x84, 0x35, + 0x8f, 0x8e, 0xa8, 0x75, 0x99, 0xb0, 0xcf, 0x0b, 0x7b, 0x14, 0xe8, 0xc2, 0x11, 0xb7, 0x00, 0x42, + 0xd2, 0xc4, 0x51, 0x5b, 0x21, 0xc5, 0x80, 0x2f, 0xd4, 0x82, 0x6b, 0xa6, 0xeb, 0x4e, 0x67, 0x09, + 0xbc, 0x82, 0xb0, 0x5b, 0xd5, 0xaa, 0x10, 0x6e, 0x13, 0x56, 0x2c, 0x7f, 0x78, 0x7c, 0xe1, 0xcf, + 0x44, 0x91, 0x52, 0x20, 0x79, 0xcb, 0xdf, 0xb9, 0xf0, 0x67, 0xfc, 0x46, 0xb9, 0xf0, 0xe9, 0x38, + 0x5a, 0x9b, 0x14, 0xb8, 0x40, 0x14, 0x25, 0x73, 0x35, 0x54, 0x69, 0x41, 0x0d, 0x95, 0x2c, 0x92, + 0xca, 0xf3, 0x45, 0x52, 0xbc, 0xcc, 0xaa, 0x24, 0xcb, 0xac, 0x58, 0x0d, 0x55, 0x4d, 0xd4, 0x50, + 0xd1, 0xc2, 0xa8, 0xf6, 0xe6, 0xc2, 0x08, 0x4f, 0x61, 0x5d, 0x84, 0xc7, 0xbb, 0x96, 0xbb, 0x39, + 0x9f, 0xc7, 0x57, 0xfc, 0x52, 0x0f, 0xe3, 0x8e, 0x48, 0x35, 0x7e, 0x02, 0x1b, 0xc9, 0xd9, 0x52, + 0xe5, 0xcc, 0xdf, 0x0d, 0x58, 0xeb, 0x8f, 0x4c, 0xc6, 0x9f, 0x7f, 0xe9, 0x5b, 0x0e, 0xaf, 0x7b, + 0x7c, 0xbf, 0x6d, 0x5f, 0x32, 0x52, 0xc1, 0x67, 0x5f, 0xd3, 0x2c, 0xd8, 0x87, 0xf5, 0xc4, 0x7a, + 0xd3, 0x76, 0x30, 0x0f, 0x28, 0x3b, 0xd8, 0xed, 0x9b, 0x27, 0xb4, 0xe7, 0x58, 0x76, 0x2a, 0x6f, + 0x61, 0x0a, 0x1b, 0x49, 0x94, 0x54, 0xc7, 0x32, 0x4f, 0x3a, 0xf3, 0x84, 0x0e, 0x5d, 0x8e, 0xa1, + 0x08, 0x2c, 0xfa, 0x1a, 0x14, 0x9f, 0x40, 0xe3, 0x85, 0x3b, 0x36, 0x19, 0x7d, 0xc7, 0xf5, 0xbe, + 0x69, 0x1e, 0x07, 0xae, 0x2f, 0x98, 0x27, 0xd5, 0x8e, 0xee, 0x41, 0x95, 0xdf, 0x68, 0x73, 0xb3, + 0xf1, 0x7b, 0x2e, 0xc0, 0x7e, 0xf0, 0x3b, 0x28, 0x06, 0x7f, 0x5d, 0x40, 0x79, 0xc8, 0x74, 0x9f, + 0xd5, 0x97, 0x50, 0x09, 0x56, 0x5e, 0x1c, 0x3d, 0x3b, 0xea, 0x7e, 0x71, 0x54, 0x37, 0xd0, 0x1a, + 0xd4, 0x8f, 0xba, 0x83, 0xe1, 0x4e, 0xb7, 0x3b, 0xe8, 0x0f, 0x48, 0xbb, 0xd7, 0xdb, 0xdf, 0xab, + 0x67, 0xd0, 0x35, 0xa8, 0xf5, 0x07, 0x5d, 0xb2, 0x3f, 0x1c, 0x74, 0x0f, 0x77, 0xfa, 0x83, 0xee, + 0xd1, 0x7e, 0x7d, 0x19, 0x35, 0x60, 0xad, 0xfd, 0x9c, 0xec, 0xb7, 0xf7, 0xbe, 0x8c, 0x9b, 0x67, + 0xb9, 0xa6, 0x73, 0xb4, 0xdb, 0x3d, 0xec, 0xb5, 0x07, 0x9d, 0x9d, 0xe7, 0xfb, 0xc3, 0xcf, 0xf7, + 0x49, 0xbf, 0xd3, 0x3d, 0xaa, 0xe7, 0x1e, 0x6c, 0x43, 0x29, 0x52, 0xaa, 0xa3, 0x02, 0x64, 0xfb, + 0xbb, 0xed, 0xa3, 0xfa, 0x12, 0xaa, 0x41, 0xa9, 0xdd, 0xeb, 0x91, 0xee, 0x2f, 0x3a, 0x87, 0xed, + 0xc1, 0x7e, 0xdd, 0x78, 0xf4, 0x8f, 0x32, 0x64, 0x7a, 0x7b, 0xa8, 0x0d, 0x10, 0xbe, 0xf4, 0xd1, + 0xa6, 0xe4, 0x60, 0xae, 0x7d, 0xd0, 0x6c, 0xcc, 0x2b, 0x24, 0x4d, 0x78, 0x09, 0x3d, 0x84, 0xe5, + 0x81, 0xef, 0x20, 0x95, 0xda, 0xe1, 0x1f, 0x4c, 0x9a, 0xab, 0x11, 0x89, 0xb6, 0xde, 0x36, 0x1e, + 0x1a, 0xe8, 0x13, 0x28, 0x06, 0x6d, 0x72, 0xb4, 0x21, 0xad, 0x92, 0x7f, 0x50, 0x68, 0x6e, 0xce, + 0xc9, 0x83, 0x19, 0x0f, 0xa1, 0x1a, 0x6f, 0xb4, 0xa3, 0x1b, 0xd2, 0x78, 0x61, 0x13, 0xbf, 0x79, + 0x73, 0xb1, 0x32, 0x80, 0x7b, 0x0c, 0x2b, 0xaa, 0x19, 0x8e, 0x54, 0x10, 0xc4, 0x5b, 0xeb, 0xcd, + 0xf5, 0x84, 0x34, 0x18, 0xf9, 0x73, 0x28, 0xe8, 0xd6, 0x34, 0x5a, 0x0f, 0x28, 0x8a, 0xf6, 0x90, + 0x9b, 0x1b, 0x49, 0x71, 0x74, 0xb0, 0xee, 0x05, 0xeb, 0xc1, 0x89, 0x06, 0xb4, 0x1e, 0x9c, 0x6c, + 0x19, 0xe3, 0x25, 0x74, 0x00, 0xe5, 0x68, 0x0b, 0x17, 0x5d, 0x0f, 0xa6, 0x49, 0xb6, 0x86, 0x9b, + 0xcd, 0x45, 0xaa, 0x28, 0x97, 0xf1, 0x83, 0x57, 0x73, 0xb9, 0xf0, 0xf0, 0xd7, 0x5c, 0x2e, 0x3e, + 0xab, 0xf1, 0x12, 0x1a, 0x40, 0x2d, 0xf1, 0x26, 0x44, 0x37, 0x75, 0x62, 0x2d, 0x6a, 0x9e, 0x34, + 0x6f, 0x5d, 0xa1, 0x4d, 0x06, 0x4c, 0xd0, 0x51, 0x45, 0x21, 0xa3, 0xb1, 0x13, 0xbe, 0xb9, 0x39, + 0x27, 0x0f, 0x56, 0xb5, 0x03, 0x95, 0x03, 0xca, 0x7a, 0x1e, 0xbd, 0x4c, 0x8f, 0xf1, 0x44, 0x60, + 0x84, 0x5d, 0x5d, 0xd4, 0x4c, 0xd8, 0x46, 0x5a, 0xbd, 0xaf, 0xc3, 0xf9, 0x04, 0x0a, 0xfa, 0xd1, + 0xa4, 0xdd, 0x9e, 0x78, 0xad, 0x69, 0xb7, 0x27, 0xdf, 0x56, 0x78, 0xf9, 0x0f, 0x19, 0x03, 0x1d, + 0x40, 0x29, 0xf2, 0xbc, 0x40, 0x0d, 0xcd, 0x5f, 0xf2, 0xf9, 0xd3, 0xbc, 0xbe, 0x40, 0x13, 0x05, + 0xfa, 0x0c, 0x2a, 0xb1, 0x12, 0x5c, 0x6f, 0x68, 0xd1, 0xb3, 0xa3, 0x79, 0x63, 0xa1, 0x2e, 0xd8, + 0x54, 0x1f, 0xea, 0xc9, 0xa2, 0x17, 0xdd, 0x8a, 0xce, 0x3f, 0x8f, 0x78, 0xfb, 0x2a, 0x75, 0x14, + 0x34, 0xd9, 0x9d, 0xd6, 0xa0, 0x57, 0x74, 0xbf, 0x35, 0xe8, 0x55, 0x4d, 0x6d, 0x09, 0x9a, 0x6c, + 0x05, 0x6b, 0xd0, 0x2b, 0xba, 0xd2, 0x1a, 0xf4, 0xaa, 0x0e, 0x32, 0x5e, 0xe2, 0x54, 0xc6, 0x2e, + 0x71, 0x4d, 0xe5, 0xa2, 0x4a, 0x44, 0x53, 0xb9, 0xf0, 0xd6, 0x97, 0x09, 0x19, 0xbf, 0x83, 0x75, + 0x42, 0x2e, 0xbc, 0xdf, 0x75, 0x42, 0x2e, 0xbe, 0xb6, 0xf1, 0x12, 0xfa, 0x1c, 0x56, 0xe7, 0xee, + 0x40, 0xa4, 0x76, 0x74, 0xd5, 0x25, 0xdc, 0xbc, 0x73, 0xa5, 0x5e, 0xe3, 0xee, 0x3c, 0xf8, 0xd7, + 0xab, 0xdb, 0xc6, 0xbf, 0x5f, 0xdd, 0x36, 0xfe, 0xf3, 0xea, 0xb6, 0xf1, 0x97, 0xff, 0xde, 0x5e, + 0x82, 0xc6, 0xc8, 0x39, 0x6f, 0xb9, 0x96, 0x3d, 0x19, 0x99, 0x6e, 0x8b, 0x59, 0x67, 0x97, 0xad, + 0xb3, 0x4b, 0xf1, 0x7f, 0x02, 0x8e, 0xf3, 0xe2, 0x9f, 0x9f, 0xfc, 0x2f, 0x00, 0x00, 0xff, 0xff, + 0x64, 0x88, 0xec, 0x1f, 0x61, 0x20, 0x00, 0x00, } diff --git a/vendor/github.com/pingcap/pd/pd-client/client.go b/vendor/github.com/pingcap/pd/pd-client/client.go index e52b606a25580..97b6667d73507 100644 --- a/vendor/github.com/pingcap/pd/pd-client/client.go +++ b/vendor/github.com/pingcap/pd/pd-client/client.go @@ -22,10 +22,10 @@ import ( "sync" "time" - "github.com/juju/errors" "github.com/opentracing/opentracing-go" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" "google.golang.org/grpc" @@ -47,12 +47,18 @@ type Client interface { // Also it may return nil if PD finds no Region for the key temporarily, // client should retry later. GetRegion(ctx context.Context, key []byte) (*metapb.Region, *metapb.Peer, error) + // GetPrevRegion gets the previous region and its leader Peer of the region where the key is located. + GetPrevRegion(ctx context.Context, key []byte) (*metapb.Region, *metapb.Peer, error) // GetRegionByID gets a region and its leader Peer from PD by id. GetRegionByID(ctx context.Context, regionID uint64) (*metapb.Region, *metapb.Peer, error) // GetStore gets a store from PD by store id. // The store may expire later. Caller is responsible for caching and taking care // of store change. GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) + // Update GC safe point. TiKV will check it and do GC themselves if necessary. + // If the given safePoint is less than the current one, it will not be updated. + // Returns the new safePoint after updating. + UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) // Close closes the client. Close() } @@ -125,10 +131,10 @@ func NewClient(pdAddrs []string, security SecurityOption) (Client, error) { c.connMu.clientConns = make(map[string]*grpc.ClientConn) if err := c.initClusterID(); err != nil { - return nil, errors.Trace(err) + return nil, errors.WithStack(err) } if err := c.updateLeader(); err != nil { - return nil, errors.Trace(err) + return nil, errors.WithStack(err) } log.Infof("[pd] init cluster id %v", c.clusterID) @@ -165,7 +171,7 @@ func (c *client) initClusterID() error { time.Sleep(time.Second) } - return errors.Trace(errFailInitClusterID) + return errors.WithStack(errFailInitClusterID) } func (c *client) updateLeader() error { @@ -174,11 +180,16 @@ func (c *client) updateLeader() error { members, err := c.getMembers(ctx, u) cancel() if err != nil || members.GetLeader() == nil || len(members.GetLeader().GetClientUrls()) == 0 { - continue + select { + case <-c.ctx.Done(): + return errors.WithStack(err) + default: + continue + } } c.updateURLs(members.GetMembers()) if err = c.switchLeader(members.GetLeader().GetClientUrls()); err != nil { - return errors.Trace(err) + return errors.WithStack(err) } return nil } @@ -188,11 +199,11 @@ func (c *client) updateLeader() error { func (c *client) getMembers(ctx context.Context, url string) (*pdpb.GetMembersResponse, error) { cc, err := c.getOrCreateGRPCConn(url) if err != nil { - return nil, errors.Trace(err) + return nil, errors.WithStack(err) } members, err := pdpb.NewPDClient(cc).GetMembers(ctx, &pdpb.GetMembersRequest{}) if err != nil { - return nil, errors.Trace(err) + return nil, errors.WithStack(err) } return members, nil } @@ -211,7 +222,7 @@ func (c *client) switchLeader(addrs []string) error { log.Infof("[pd] leader switches to: %v, previous: %v", addr, oldLeader) if _, err := c.getOrCreateGRPCConn(addr); err != nil { - return errors.Trace(err) + return errors.WithStack(err) } c.connMu.Lock() @@ -262,11 +273,11 @@ func (c *client) getOrCreateGRPCConn(addr string) (*grpc.ClientConn, error) { } u, err := url.Parse(addr) if err != nil { - return nil, errors.Trace(err) + return nil, errors.WithStack(err) } cc, err := grpc.Dial(u.Host, opt) if err != nil { - return nil, errors.Trace(err) + return nil, errors.WithStack(err) } c.connMu.Lock() defer c.connMu.Unlock() @@ -344,11 +355,16 @@ func (c *client) tsLoop() { if stream == nil { var ctx context.Context - ctx, cancel = context.WithCancel(c.ctx) + ctx, cancel = context.WithCancel(loopCtx) stream, err = c.leaderClient().Tso(ctx) if err != nil { + select { + case <-loopCtx.Done(): + return + default: + } log.Errorf("[pd] create tso stream error: %v", err) - c.scheduleCheckLeader() + c.ScheduleCheckLeader() cancel() c.revokeTSORequest(err) select { @@ -388,8 +404,13 @@ func (c *client) tsLoop() { } if err != nil { + select { + case <-loopCtx.Done(): + return + default: + } log.Errorf("[pd] getTS error: %v", err) - c.scheduleCheckLeader() + c.ScheduleCheckLeader() cancel() stream, cancel = nil, nil } @@ -419,20 +440,20 @@ func (c *client) processTSORequests(stream pdpb.PD_TsoClient, requests []*tsoReq if err := stream.Send(req); err != nil { c.finishTSORequest(requests, 0, 0, err) - return errors.Trace(err) + return errors.WithStack(err) } resp, err := stream.Recv() if err != nil { - c.finishTSORequest(requests, 0, 0, errors.Trace(err)) - return errors.Trace(err) + c.finishTSORequest(requests, 0, 0, errors.WithStack(err)) + return errors.WithStack(err) } requestDuration.WithLabelValues("tso").Observe(time.Since(start).Seconds()) if err == nil && resp.GetCount() != uint32(len(requests)) { err = errTSOLength } if err != nil { - c.finishTSORequest(requests, 0, 0, errors.Trace(err)) - return errors.Trace(err) + c.finishTSORequest(requests, 0, 0, errors.WithStack(err)) + return errors.WithStack(err) } physical, logical := resp.GetTimestamp().GetPhysical(), resp.GetTimestamp().GetLogical() @@ -456,7 +477,7 @@ func (c *client) revokeTSORequest(err error) { n := len(c.tsoRequests) for i := 0; i < n; i++ { req := <-c.tsoRequests - req.done <- errors.Trace(err) + req.done <- errors.WithStack(err) } } @@ -482,7 +503,7 @@ func (c *client) leaderClient() pdpb.PDClient { return pdpb.NewPDClient(c.connMu.clientConns[c.connMu.leader]) } -func (c *client) scheduleCheckLeader() { +func (c *client) ScheduleCheckLeader() { select { case c.checkLeaderCh <- struct{}{}: default: @@ -493,6 +514,18 @@ func (c *client) GetClusterID(context.Context) uint64 { return c.clusterID } +// For testing use. +func (c *client) GetLeaderAddr() string { + c.connMu.RLock() + defer c.connMu.RUnlock() + return c.connMu.leader +} + +// For testing use. It should only be called when the client is closed. +func (c *client) GetURLs() []string { + return c.urls +} + var tsoReqPool = sync.Pool{ New: func() interface{} { return &tsoRequest{ @@ -523,18 +556,21 @@ type TSFuture interface { } func (req *tsoRequest) Wait() (int64, int64, error) { + // If tso command duration is observed very high, the reason could be it + // takes too long for Wait() be called. + cmdDuration.WithLabelValues("tso_async_wait").Observe(time.Since(req.start).Seconds()) select { case err := <-req.done: defer tsoReqPool.Put(req) if err != nil { cmdFailedDuration.WithLabelValues("tso").Observe(time.Since(req.start).Seconds()) - return 0, 0, errors.Trace(err) + return 0, 0, errors.WithStack(err) } physical, logical := req.physical, req.logical cmdDuration.WithLabelValues("tso").Observe(time.Since(req.start).Seconds()) return physical, logical, err case <-req.ctx.Done(): - return 0, 0, errors.Trace(req.ctx.Err()) + return 0, 0, errors.WithStack(req.ctx.Err()) } } @@ -556,13 +592,35 @@ func (c *client) GetRegion(ctx context.Context, key []byte) (*metapb.Region, *me Header: c.requestHeader(), RegionKey: key, }) - requestDuration.WithLabelValues("get_region").Observe(time.Since(start).Seconds()) cancel() if err != nil { cmdFailedDuration.WithLabelValues("get_region").Observe(time.Since(start).Seconds()) - c.scheduleCheckLeader() - return nil, nil, errors.Trace(err) + c.ScheduleCheckLeader() + return nil, nil, errors.WithStack(err) + } + return resp.GetRegion(), resp.GetLeader(), nil +} + +func (c *client) GetPrevRegion(ctx context.Context, key []byte) (*metapb.Region, *metapb.Peer, error) { + if span := opentracing.SpanFromContext(ctx); span != nil { + span = opentracing.StartSpan("pdclient.GetPrevRegion", opentracing.ChildOf(span.Context())) + defer span.Finish() + } + start := time.Now() + defer func() { cmdDuration.WithLabelValues("get_prev_region").Observe(time.Since(start).Seconds()) }() + + ctx, cancel := context.WithTimeout(ctx, pdTimeout) + resp, err := c.leaderClient().GetPrevRegion(ctx, &pdpb.GetRegionRequest{ + Header: c.requestHeader(), + RegionKey: key, + }) + cancel() + + if err != nil { + cmdFailedDuration.WithLabelValues("get_prev_region").Observe(time.Since(start).Seconds()) + c.ScheduleCheckLeader() + return nil, nil, errors.WithStack(err) } return resp.GetRegion(), resp.GetLeader(), nil } @@ -580,13 +638,12 @@ func (c *client) GetRegionByID(ctx context.Context, regionID uint64) (*metapb.Re Header: c.requestHeader(), RegionId: regionID, }) - requestDuration.WithLabelValues("get_region_byid").Observe(time.Since(start).Seconds()) cancel() if err != nil { cmdFailedDuration.WithLabelValues("get_region_byid").Observe(time.Since(start).Seconds()) - c.scheduleCheckLeader() - return nil, nil, errors.Trace(err) + c.ScheduleCheckLeader() + return nil, nil, errors.WithStack(err) } return resp.GetRegion(), resp.GetLeader(), nil } @@ -604,13 +661,12 @@ func (c *client) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, e Header: c.requestHeader(), StoreId: storeID, }) - requestDuration.WithLabelValues("get_store").Observe(time.Since(start).Seconds()) cancel() if err != nil { cmdFailedDuration.WithLabelValues("get_store").Observe(time.Since(start).Seconds()) - c.scheduleCheckLeader() - return nil, errors.Trace(err) + c.ScheduleCheckLeader() + return nil, errors.WithStack(err) } store := resp.GetStore() if store == nil { @@ -622,6 +678,29 @@ func (c *client) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, e return store, nil } +func (c *client) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uint64, error) { + if span := opentracing.SpanFromContext(ctx); span != nil { + span = opentracing.StartSpan("pdclient.UpdateGCSafePoint", opentracing.ChildOf(span.Context())) + defer span.Finish() + } + start := time.Now() + defer func() { cmdDuration.WithLabelValues("update_gc_safe_point").Observe(time.Since(start).Seconds()) }() + + ctx, cancel := context.WithTimeout(ctx, pdTimeout) + resp, err := c.leaderClient().UpdateGCSafePoint(ctx, &pdpb.UpdateGCSafePointRequest{ + Header: c.requestHeader(), + SafePoint: safePoint, + }) + cancel() + + if err != nil { + cmdFailedDuration.WithLabelValues("update_gc_safe_point").Observe(time.Since(start).Seconds()) + c.ScheduleCheckLeader() + return 0, errors.WithStack(err) + } + return resp.GetNewSafePoint(), nil +} + func (c *client) requestHeader() *pdpb.RequestHeader { return &pdpb.RequestHeader{ ClusterId: c.clusterID, diff --git a/vendor/github.com/pingcap/tipb/go-tipb/expression.pb.go b/vendor/github.com/pingcap/tipb/go-tipb/expression.pb.go index d3ed63f8fa436..8e85c353ae9c2 100644 --- a/vendor/github.com/pingcap/tipb/go-tipb/expression.pb.go +++ b/vendor/github.com/pingcap/tipb/go-tipb/expression.pb.go @@ -313,7 +313,7 @@ const ( ScalarFuncSig_FloorDecToInt ScalarFuncSig = 2112 ScalarFuncSig_FloorDecToDec ScalarFuncSig = 2113 ScalarFuncSig_FloorReal ScalarFuncSig = 2114 - ScalarFuncSig_RouldReal ScalarFuncSig = 2121 + ScalarFuncSig_RoundReal ScalarFuncSig = 2121 ScalarFuncSig_RoundInt ScalarFuncSig = 2122 ScalarFuncSig_RoundDec ScalarFuncSig = 2123 ScalarFuncSig_RoundWithFracReal ScalarFuncSig = 2124 @@ -463,16 +463,18 @@ const ( ScalarFuncSig_RegexpBinarySig ScalarFuncSig = 4311 ScalarFuncSig_RegexpSig ScalarFuncSig = 4312 // json - ScalarFuncSig_JsonExtractSig ScalarFuncSig = 5001 - ScalarFuncSig_JsonUnquoteSig ScalarFuncSig = 5002 - ScalarFuncSig_JsonTypeSig ScalarFuncSig = 5003 - ScalarFuncSig_JsonSetSig ScalarFuncSig = 5004 - ScalarFuncSig_JsonInsertSig ScalarFuncSig = 5005 - ScalarFuncSig_JsonReplaceSig ScalarFuncSig = 5006 - ScalarFuncSig_JsonRemoveSig ScalarFuncSig = 5007 - ScalarFuncSig_JsonMergeSig ScalarFuncSig = 5008 - ScalarFuncSig_JsonObjectSig ScalarFuncSig = 5009 - ScalarFuncSig_JsonArraySig ScalarFuncSig = 5010 + ScalarFuncSig_JsonExtractSig ScalarFuncSig = 5001 + ScalarFuncSig_JsonUnquoteSig ScalarFuncSig = 5002 + ScalarFuncSig_JsonTypeSig ScalarFuncSig = 5003 + ScalarFuncSig_JsonSetSig ScalarFuncSig = 5004 + ScalarFuncSig_JsonInsertSig ScalarFuncSig = 5005 + ScalarFuncSig_JsonReplaceSig ScalarFuncSig = 5006 + ScalarFuncSig_JsonRemoveSig ScalarFuncSig = 5007 + ScalarFuncSig_JsonMergeSig ScalarFuncSig = 5008 + ScalarFuncSig_JsonObjectSig ScalarFuncSig = 5009 + ScalarFuncSig_JsonArraySig ScalarFuncSig = 5010 + ScalarFuncSig_JsonValidSig ScalarFuncSig = 5011 + ScalarFuncSig_JsonContainsSig ScalarFuncSig = 5012 // time ScalarFuncSig_DateFormatSig ScalarFuncSig = 6001 ScalarFuncSig_DateLiteral ScalarFuncSig = 6002 @@ -797,7 +799,7 @@ var ScalarFuncSig_name = map[int32]string{ 2112: "FloorDecToInt", 2113: "FloorDecToDec", 2114: "FloorReal", - 2121: "RouldReal", + 2121: "RoundReal", 2122: "RoundInt", 2123: "RoundDec", 2124: "RoundWithFracReal", @@ -947,6 +949,8 @@ var ScalarFuncSig_name = map[int32]string{ 5008: "JsonMergeSig", 5009: "JsonObjectSig", 5010: "JsonArraySig", + 5011: "JsonValidSig", + 5012: "JsonContainsSig", 6001: "DateFormatSig", 6002: "DateLiteral", 6003: "DateDiff", @@ -1268,7 +1272,7 @@ var ScalarFuncSig_value = map[string]int32{ "FloorDecToInt": 2112, "FloorDecToDec": 2113, "FloorReal": 2114, - "RouldReal": 2121, + "RoundReal": 2121, "RoundInt": 2122, "RoundDec": 2123, "RoundWithFracReal": 2124, @@ -1418,6 +1422,8 @@ var ScalarFuncSig_value = map[string]int32{ "JsonMergeSig": 5008, "JsonObjectSig": 5009, "JsonArraySig": 5010, + "JsonValidSig": 5011, + "JsonContainsSig": 5012, "DateFormatSig": 6001, "DateLiteral": 6002, "DateDiff": 6003, @@ -2531,264 +2537,265 @@ var ( func init() { proto.RegisterFile("expression.proto", fileDescriptorExpression) } var fileDescriptorExpression = []byte{ - // 4141 bytes of a gzipped FileDescriptorProto + // 4160 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x5a, 0x67, 0x74, 0x5c, 0xd5, - 0xb5, 0x66, 0x6c, 0x19, 0xec, 0xeb, 0xb6, 0xb9, 0x2e, 0x1a, 0x78, 0xeb, 0xd9, 0x32, 0x93, 0xf5, - 0xec, 0x90, 0xf7, 0x34, 0x89, 0x6d, 0x46, 0xef, 0xc7, 0x6b, 0xa3, 0x62, 0x79, 0xb2, 0x24, 0xd9, - 0xd6, 0x8c, 0x4c, 0xde, 0xaf, 0xac, 0xab, 0x99, 0xa3, 0xd1, 0xc5, 0x33, 0xf7, 0x0e, 0xf7, 0x9e, - 0x91, 0xa5, 0xbc, 0xb2, 0xe8, 0xd5, 0x95, 0x84, 0x1a, 0x20, 0x74, 0x42, 0x2f, 0x09, 0xbd, 0xb7, - 0x60, 0xd3, 0x09, 0xe0, 0x02, 0x84, 0x10, 0x93, 0x46, 0x49, 0x08, 0xa1, 0xd7, 0x64, 0xed, 0xef, - 0x9c, 0x73, 0xe7, 0x8e, 0xc4, 0x2f, 0xdf, 0xfd, 0x9d, 0xbd, 0xbf, 0xfd, 0x9d, 0x7d, 0xfa, 0x58, - 0x16, 0x89, 0xf1, 0x5a, 0x20, 0xc2, 0xd0, 0xf5, 0xbd, 0xf6, 0x5a, 0xe0, 0x4b, 0xdf, 0x6e, 0x91, - 0x6e, 0x6d, 0xf8, 0xf0, 0x85, 0x65, 0xbf, 0xec, 0x03, 0x48, 0xf3, 0x97, 0x6a, 0x3b, 0xe2, 0xb6, - 0x84, 0x35, 0x6b, 0x8d, 0x2b, 0x2a, 0xa5, 0xc2, 0x44, 0x4d, 0xd8, 0x0b, 0xad, 0x69, 0xb2, 0x96, - 0x4c, 0xb4, 0x25, 0x56, 0xcc, 0xe8, 0x6c, 0xd9, 0xf5, 0xda, 0xd2, 0x83, 0x06, 0xa7, 0xc9, 0x9a, - 0x9d, 0xb4, 0x5a, 0x46, 0x2a, 0x4e, 0x39, 0x39, 0xad, 0x2d, 0xb1, 0x62, 0xae, 0xc6, 0x81, 0xa8, - 0x16, 0xe1, 0x25, 0xa7, 0xc7, 0x22, 0x80, 0xd8, 0x4b, 0xac, 0x43, 0x4a, 0xa2, 0xe8, 0x56, 0x9d, - 0x4a, 0xb2, 0x25, 0xd6, 0x68, 0x40, 0x6e, 0x2f, 0xfa, 0x95, 0x8a, 0x23, 0x45, 0x72, 0x46, 0xbc, - 0x5d, 0x83, 0x68, 0x1f, 0x75, 0x82, 0x50, 0xc8, 0xe4, 0xc1, 0x6d, 0x89, 0x15, 0xb3, 0xa2, 0x76, - 0x05, 0x1e, 0xf1, 0x48, 0xc2, 0x6a, 0xe9, 0x19, 0xaf, 0x05, 0xf6, 0x37, 0x22, 0xc9, 0xf3, 0x56, - 0xce, 0x6b, 0xe7, 0x9e, 0xb6, 0x33, 0xce, 0xdd, 0x89, 0x75, 0x81, 0xac, 0xe9, 0x63, 0x4e, 0x05, - 0x3d, 0x98, 0x33, 0xc8, 0x9f, 0xf6, 0x3f, 0x59, 0x33, 0x8b, 0xa3, 0x6e, 0xa5, 0x14, 0x40, 0xfe, - 0xf4, 0x15, 0xb3, 0x57, 0x5a, 0x8d, 0xe8, 0xc1, 0xa8, 0xcd, 0xfe, 0x96, 0x35, 0x3d, 0x74, 0xcb, - 0xe8, 0xc4, 0xbc, 0x95, 0x0b, 0x94, 0x4b, 0xbe, 0xe8, 0x54, 0x9c, 0x60, 0x4d, 0xdd, 0x2b, 0xe6, - 0xdd, 0xb2, 0xce, 0xc2, 0x5e, 0x76, 0xbb, 0x65, 0x8d, 0x70, 0x31, 0xbf, 0x2f, 0x27, 0x6a, 0xaa, - 0x63, 0xb3, 0x57, 0xce, 0x57, 0x31, 0x51, 0x91, 0x07, 0x67, 0x8d, 0x98, 0xcf, 0x23, 0x3a, 0xad, - 0x83, 0x3b, 0x27, 0x72, 0x52, 0x54, 0xed, 0x25, 0x56, 0x0b, 0x8f, 0x1b, 0x3a, 0xd2, 0x2c, 0x05, - 0x38, 0x57, 0xba, 0x24, 0xc2, 0x22, 0x7a, 0x30, 0xd3, 0x54, 0x9a, 0x91, 0x23, 0xdf, 0x9d, 0x6e, - 0xcd, 0x34, 0x3d, 0xb6, 0x67, 0x5a, 0x2d, 0x03, 0xf5, 0x4a, 0x85, 0x0e, 0xb2, 0x67, 0x59, 0x33, - 0x72, 0x9e, 0xcc, 0xac, 0xa6, 0x84, 0x6d, 0x59, 0x07, 0x0f, 0xb9, 0xf8, 0x9e, 0x66, 0xcf, 0xb6, - 0x0e, 0x59, 0x53, 0xf1, 0x1d, 0xb9, 0x6a, 0x25, 0x4d, 0x8f, 0x8c, 0xcc, 0x6a, 0x6a, 0x61, 0xaf, - 0xbc, 0x0c, 0x5c, 0xaf, 0x4c, 0x33, 0x38, 0xb8, 0x73, 0x42, 0x8a, 0x90, 0x0e, 0xb6, 0xe7, 0x58, + 0xb5, 0x66, 0x6c, 0x19, 0xec, 0xeb, 0xb6, 0xb9, 0xd8, 0x68, 0xe0, 0xad, 0x67, 0xcb, 0x4c, 0xd6, + 0xb3, 0x43, 0xde, 0xd3, 0x24, 0xb6, 0x19, 0xbd, 0x1f, 0xaf, 0x8d, 0x8a, 0xe5, 0xc9, 0x92, 0x64, + 0x5b, 0x33, 0x32, 0x79, 0xbf, 0xb2, 0xae, 0x66, 0x8e, 0x46, 0x17, 0xcf, 0xdc, 0x3b, 0xdc, 0x7b, + 0x46, 0x96, 0xf2, 0xca, 0xa2, 0x57, 0x57, 0x3a, 0x04, 0x08, 0x9d, 0xd0, 0x4b, 0x42, 0xef, 0x2d, + 0xd8, 0x74, 0x82, 0x71, 0x01, 0x42, 0x88, 0x49, 0xa3, 0x24, 0x84, 0xd0, 0x6b, 0xb2, 0xf6, 0x77, + 0xce, 0xb9, 0x73, 0x47, 0xe2, 0x97, 0xef, 0xfe, 0xce, 0xde, 0xdf, 0xfe, 0xce, 0x3e, 0x7d, 0x2c, + 0x8b, 0xc4, 0x78, 0x2d, 0x10, 0x61, 0xe8, 0xfa, 0x5e, 0x7b, 0x2d, 0xf0, 0xa5, 0x6f, 0xb7, 0x48, + 0xb7, 0x36, 0x7c, 0xf8, 0x82, 0xb2, 0x5f, 0xf6, 0x01, 0xa4, 0xf9, 0x4b, 0xb5, 0x1d, 0x71, 0x47, + 0xc2, 0x9a, 0xb5, 0xca, 0x15, 0x95, 0x52, 0x61, 0xa2, 0x26, 0xec, 0x05, 0xd6, 0x34, 0x59, 0x4b, + 0x26, 0xda, 0x12, 0xcb, 0x66, 0x74, 0xb6, 0xec, 0x78, 0x7d, 0xf1, 0x01, 0x83, 0xd3, 0x64, 0xcd, + 0x4e, 0x5a, 0x2d, 0x23, 0x15, 0xa7, 0x9c, 0x9c, 0xd6, 0x96, 0x58, 0x36, 0x57, 0xe3, 0x40, 0x54, + 0x8b, 0xf0, 0x92, 0xd3, 0x63, 0x11, 0x40, 0xec, 0x45, 0xd6, 0x41, 0x25, 0x51, 0x74, 0xab, 0x4e, + 0x25, 0xd9, 0x12, 0x6b, 0x34, 0x20, 0xb7, 0x17, 0xfd, 0x4a, 0xc5, 0x91, 0x22, 0x39, 0x23, 0xde, + 0xae, 0x41, 0xb4, 0x8f, 0x3a, 0x41, 0x28, 0x64, 0xf2, 0xc0, 0xb6, 0xc4, 0xb2, 0x59, 0x51, 0xbb, + 0x02, 0x8f, 0x78, 0x2c, 0x61, 0xb5, 0xf4, 0x8c, 0xd7, 0x02, 0xfb, 0x5b, 0x91, 0xe4, 0x79, 0xcb, + 0xe7, 0xb5, 0x73, 0x4f, 0xdb, 0x19, 0xe7, 0xee, 0xc4, 0xba, 0x40, 0xd6, 0xf4, 0x31, 0xa7, 0x82, + 0x1e, 0xcc, 0x19, 0xe4, 0x4f, 0xfb, 0x9f, 0xac, 0x99, 0xc5, 0x51, 0xb7, 0x52, 0x0a, 0x20, 0x7f, + 0xfa, 0xb2, 0xd9, 0xcb, 0xad, 0x46, 0xf4, 0x60, 0xd4, 0x66, 0x7f, 0xc7, 0x9a, 0x1e, 0xba, 0x65, + 0x74, 0x62, 0xde, 0xf2, 0x43, 0x94, 0x4b, 0xbe, 0xe8, 0x54, 0x9c, 0x60, 0x55, 0xdd, 0x2b, 0xe6, + 0xdd, 0xb2, 0xce, 0xc2, 0x5e, 0x76, 0xbb, 0x65, 0x8d, 0x70, 0x31, 0x7f, 0x28, 0x27, 0x6a, 0xaa, + 0x63, 0xb3, 0x97, 0xcf, 0x57, 0x31, 0x51, 0x91, 0x07, 0x67, 0x8d, 0x98, 0xcf, 0x23, 0x3a, 0xad, + 0x03, 0x3b, 0x27, 0x72, 0x52, 0x54, 0xed, 0x45, 0x56, 0x0b, 0x8f, 0x1b, 0x3a, 0xd2, 0x2c, 0x05, + 0x38, 0x57, 0xba, 0x24, 0xc2, 0x22, 0x7a, 0x30, 0xd3, 0x54, 0x9a, 0x91, 0x23, 0xdf, 0x9b, 0x6e, + 0xcd, 0x34, 0x3d, 0xb6, 0x67, 0x5a, 0x2d, 0x03, 0xf5, 0x4a, 0x85, 0x0e, 0xb0, 0x67, 0x59, 0x33, + 0x72, 0x9e, 0xcc, 0xac, 0xa4, 0x84, 0x6d, 0x59, 0x07, 0x0e, 0xb9, 0xf8, 0x9e, 0x66, 0xcf, 0xb6, + 0x0e, 0x5a, 0x55, 0xf1, 0x1d, 0xb9, 0x62, 0x39, 0x4d, 0x8f, 0x8c, 0xcc, 0x4a, 0x6a, 0x61, 0xaf, + 0xbc, 0x0c, 0x5c, 0xaf, 0x4c, 0x33, 0x38, 0xb8, 0x73, 0x42, 0x8a, 0x90, 0x0e, 0xb4, 0xe7, 0x58, 0x33, 0xfb, 0x27, 0xc2, 0x63, 0x2b, 0x9d, 0xae, 0x24, 0x61, 0x93, 0x35, 0x07, 0x56, 0xb7, 0x1a, - 0x46, 0x1a, 0xb1, 0x0f, 0xb5, 0xe6, 0x2a, 0xa4, 0x1e, 0x38, 0xd2, 0xf5, 0x3d, 0x2a, 0xdb, 0x73, - 0xad, 0x59, 0x80, 0x7a, 0xbc, 0x7a, 0x95, 0x46, 0x23, 0x86, 0xb5, 0x62, 0x9c, 0xdc, 0xc8, 0xca, - 0x0b, 0x49, 0xc7, 0x44, 0xae, 0x05, 0xb7, 0x2a, 0x68, 0x53, 0x64, 0x7e, 0x37, 0xf4, 0x3d, 0xaa, - 0xd8, 0xf3, 0xac, 0x59, 0x1b, 0x9d, 0x4a, 0x5d, 0xf4, 0xb9, 0xa1, 0xa4, 0x73, 0x13, 0x6c, 0x77, - 0xf9, 0x95, 0x7a, 0xd5, 0x1b, 0x14, 0x23, 0xb4, 0x9b, 0x3b, 0x36, 0xa3, 0xcb, 0xaf, 0x7b, 0x92, - 0xee, 0x6a, 0xb5, 0x67, 0x5a, 0xd3, 0xf3, 0xf5, 0x2a, 0xdd, 0x8d, 0xaf, 0xec, 0x58, 0x99, 0xee, - 0xc1, 0x57, 0xbf, 0xeb, 0xd1, 0xbd, 0xea, 0xcb, 0x19, 0xa7, 0xfb, 0x5a, 0x39, 0x66, 0x8d, 0x1b, - 0x84, 0x92, 0xee, 0x6f, 0xb5, 0xc9, 0x9a, 0xdd, 0x1b, 0xf8, 0xf5, 0x5a, 0x97, 0xef, 0x15, 0x1d, - 0x49, 0x0f, 0xb4, 0xda, 0xf3, 0x2d, 0x2b, 0x5b, 0x2e, 0x7f, 0xbf, 0xd3, 0x95, 0x59, 0xaf, 0x44, - 0x0f, 0xb6, 0x72, 0x4a, 0x0d, 0xac, 0x0b, 0xe8, 0xa1, 0xb8, 0xc3, 0xf7, 0xfc, 0x80, 0x1e, 0x56, - 0x79, 0x65, 0x89, 0x1e, 0x69, 0xb5, 0x67, 0x73, 0x01, 0x4b, 0x25, 0x31, 0x46, 0x8f, 0x22, 0x4e, - 0x19, 0xeb, 0xfd, 0x1a, 0x3d, 0x86, 0x38, 0x65, 0xe7, 0x9d, 0x6a, 0x8d, 0x7e, 0x01, 0xef, 0x8d, - 0x4e, 0xc0, 0xad, 0x8f, 0xb7, 0xda, 0x73, 0xac, 0x43, 0x36, 0x3a, 0x01, 0x9a, 0x76, 0xb5, 0xda, - 0x73, 0xad, 0x99, 0x1b, 0x9d, 0xc0, 0x75, 0xbc, 0xa2, 0xa0, 0xdd, 0x2a, 0x34, 0x9a, 0x70, 0xb4, - 0x63, 0xe0, 0xc8, 0xeb, 0x3a, 0xad, 0xb9, 0x4d, 0x53, 0x90, 0x87, 0xa5, 0xcb, 0x09, 0x65, 0xce, - 0x93, 0xd9, 0x30, 0xe7, 0x49, 0x3a, 0x88, 0x87, 0x25, 0x42, 0x06, 0x85, 0x53, 0xa1, 0x84, 0xbd, - 0xc0, 0x9a, 0x1f, 0x41, 0x7a, 0xa4, 0xa7, 0xd9, 0x0b, 0x2d, 0x8a, 0x40, 0x33, 0xa8, 0xd3, 0x9b, - 0xa2, 0x31, 0x34, 0x2d, 0xf6, 0x22, 0xeb, 0xd0, 0x86, 0xa3, 0x19, 0xeb, 0x19, 0x4d, 0x9e, 0x18, - 0xb5, 0x83, 0x0d, 0xc4, 0x59, 0x95, 0x1a, 0xcb, 0xb6, 0xad, 0x79, 0x0d, 0x08, 0x72, 0x66, 0x9b, + 0x46, 0x1a, 0xb1, 0x0f, 0xb6, 0xe6, 0x2a, 0xa4, 0x1e, 0x38, 0xd2, 0xf5, 0x3d, 0x2a, 0xdb, 0x73, + 0xad, 0x59, 0x80, 0x7a, 0xbc, 0x7a, 0x95, 0x46, 0x23, 0x86, 0xd5, 0x62, 0x9c, 0xdc, 0xc8, 0xca, + 0x0b, 0x49, 0xc7, 0x44, 0xae, 0x05, 0xb7, 0x2a, 0x68, 0x43, 0x64, 0x7e, 0x3f, 0xf4, 0x3d, 0xaa, + 0xd8, 0xf3, 0xac, 0x59, 0xeb, 0x9d, 0x4a, 0x5d, 0xf4, 0xb9, 0xa1, 0xa4, 0xf3, 0x13, 0x6c, 0x77, + 0xf9, 0x95, 0x7a, 0xd5, 0x1b, 0x14, 0x23, 0xb4, 0x93, 0x3b, 0x36, 0xa3, 0xcb, 0xaf, 0x7b, 0x92, + 0xee, 0x69, 0xb5, 0x67, 0x5a, 0xd3, 0xf3, 0xf5, 0x2a, 0xdd, 0x8b, 0xaf, 0xec, 0x58, 0x99, 0xee, + 0xc3, 0x57, 0xbf, 0xeb, 0xd1, 0xfd, 0xea, 0xcb, 0x19, 0xa7, 0x07, 0x5a, 0x39, 0x66, 0x95, 0x1b, + 0x84, 0x92, 0x1e, 0x6c, 0xb5, 0xc9, 0x9a, 0xdd, 0x1b, 0xf8, 0xf5, 0x5a, 0x97, 0xef, 0x15, 0x1d, + 0x49, 0x0f, 0xb5, 0xda, 0xf3, 0x2d, 0x2b, 0x5b, 0x2e, 0xff, 0xb0, 0xd3, 0x95, 0x59, 0xaf, 0x44, + 0x0f, 0xb7, 0x72, 0x4a, 0x0d, 0xac, 0x09, 0xe8, 0x91, 0xb8, 0xc3, 0x0f, 0xfc, 0x80, 0x1e, 0x55, + 0x79, 0x65, 0x89, 0x1e, 0x6b, 0xb5, 0x67, 0x73, 0x01, 0x4b, 0x25, 0x31, 0x46, 0x8f, 0x23, 0x4e, + 0x19, 0x6b, 0xfd, 0x1a, 0x3d, 0x81, 0x38, 0x65, 0xe7, 0x9d, 0x6a, 0x8d, 0x7e, 0x01, 0xef, 0xf5, + 0x4e, 0xc0, 0xad, 0x4f, 0xb6, 0xda, 0x73, 0xac, 0x83, 0xd6, 0x3b, 0x01, 0x9a, 0x76, 0xb4, 0xda, + 0x73, 0xad, 0x99, 0xeb, 0x9d, 0xc0, 0x75, 0xbc, 0xa2, 0xa0, 0x9d, 0x2a, 0x34, 0x9a, 0x70, 0xb4, + 0x6d, 0xe0, 0xc8, 0x5d, 0x9d, 0xd6, 0xdc, 0xa6, 0x29, 0xc8, 0xc3, 0xd2, 0xe5, 0x84, 0x32, 0xe7, + 0xc9, 0x6c, 0x98, 0xf3, 0x24, 0x1d, 0xc0, 0xc3, 0x12, 0x21, 0x83, 0xc2, 0xa9, 0x50, 0xc2, 0x3e, + 0xc4, 0x9a, 0x1f, 0x41, 0x7a, 0xa4, 0xa7, 0xd9, 0x0b, 0x2c, 0x8a, 0x40, 0x33, 0xa8, 0xd3, 0x9b, + 0xa2, 0x31, 0x34, 0x2d, 0xf6, 0x42, 0xeb, 0xe0, 0x86, 0xa3, 0x19, 0xeb, 0x19, 0x4d, 0x9e, 0x18, + 0xb5, 0x03, 0x0d, 0xc4, 0x59, 0x95, 0x1a, 0xcb, 0xb6, 0xad, 0x79, 0x0d, 0x08, 0x72, 0x66, 0x9b, 0xcc, 0x0a, 0xd3, 0x7a, 0xe6, 0x98, 0x34, 0x0a, 0x35, 0x82, 0xe6, 0x36, 0x13, 0x40, 0xd1, 0x3c, - 0x7b, 0xb1, 0x65, 0xc7, 0x5c, 0x8d, 0xa4, 0xf9, 0xcd, 0xbe, 0xd0, 0x44, 0x26, 0x99, 0x26, 0x54, - 0xb2, 0x16, 0x9a, 0x64, 0x11, 0x0a, 0x65, 0x8b, 0xec, 0x56, 0x6b, 0x41, 0x13, 0xac, 0xc5, 0x2d, - 0xb6, 0x93, 0xd6, 0xc2, 0xa6, 0x06, 0xa3, 0xaf, 0x75, 0x0a, 0x13, 0x24, 0x26, 0xed, 0xc3, 0xac, - 0x45, 0xcd, 0x01, 0x46, 0xe5, 0x61, 0x53, 0x22, 0x20, 0xf4, 0x70, 0x33, 0x48, 0x2a, 0xa5, 0xd2, - 0xb9, 0xc4, 0xa8, 0x37, 0x20, 0x64, 0x2e, 0x35, 0xfd, 0x37, 0xa8, 0x56, 0xd9, 0x66, 0xe4, 0x1b, - 0xdc, 0x88, 0x5c, 0x36, 0x99, 0x06, 0x1a, 0x8f, 0x30, 0x9d, 0x8a, 0xdc, 0x8d, 0xc4, 0xd4, 0x64, - 0x7f, 0x28, 0xfc, 0x86, 0x19, 0x5e, 0x8e, 0x56, 0xfa, 0x56, 0x98, 0x8a, 0x2b, 0x08, 0xea, 0xbe, - 0x69, 0x82, 0x15, 0xa6, 0xb5, 0x1d, 0x69, 0x7a, 0xad, 0x50, 0xa3, 0xec, 0x5b, 0xcd, 0x04, 0xd0, - 0xf5, 0xcf, 0xa6, 0x7b, 0xda, 0xd5, 0xa8, 0xfa, 0x97, 0x66, 0x5f, 0x68, 0x6a, 0x8f, 0x8a, 0xa9, - 0xbd, 0x94, 0xae, 0x95, 0x86, 0xa2, 0x01, 0x43, 0xdb, 0xaa, 0x68, 0x1c, 0x23, 0x5c, 0xeb, 0x5b, - 0x1d, 0x0d, 0x58, 0xd4, 0x62, 0x34, 0x1e, 0x35, 0x95, 0x0c, 0x3a, 0x33, 0xf6, 0xe1, 0xd6, 0xe2, - 0x49, 0x21, 0x46, 0x6b, 0xc7, 0xd4, 0x18, 0xe8, 0xfd, 0x57, 0x53, 0x43, 0xb6, 0x94, 0xd6, 0x7f, - 0x33, 0xdd, 0x52, 0x10, 0x74, 0xfe, 0xbb, 0xa9, 0xa1, 0xc2, 0xb4, 0xc6, 0xff, 0x30, 0x9d, 0x55, - 0xa8, 0xd1, 0xf7, 0x9f, 0xcd, 0x04, 0xd0, 0xf6, 0x5f, 0x26, 0xbf, 0x76, 0x35, 0xba, 0xb2, 0xcd, - 0xbe, 0xd0, 0xd4, 0xc9, 0x9b, 0x61, 0x97, 0xef, 0x54, 0x44, 0x58, 0x14, 0xac, 0xe8, 0xcd, 0x36, - 0xfb, 0x50, 0x6b, 0x8e, 0x41, 0x20, 0xe8, 0xad, 0x36, 0x7b, 0xa1, 0x35, 0xdf, 0x40, 0x26, 0xf3, - 0xdb, 0x6d, 0xf6, 0x02, 0x6b, 0x9e, 0x41, 0xb5, 0xca, 0x77, 0x9a, 0xa2, 0xa1, 0xe6, 0x4f, 0x6d, - 0xf6, 0x22, 0x8b, 0xa2, 0x68, 0x23, 0xe6, 0xcf, 0x4d, 0x9e, 0xd0, 0xf2, 0x6e, 0x1b, 0x1f, 0x40, - 0x7d, 0x05, 0x96, 0x51, 0xe2, 0x73, 0xa9, 0xaf, 0x80, 0xfc, 0x82, 0xcf, 0x87, 0xbe, 0x42, 0xe3, - 0xec, 0x99, 0x63, 0xcd, 0xec, 0x2b, 0xe8, 0x8c, 0x65, 0xe5, 0x88, 0x54, 0xa3, 0xf6, 0x3c, 0xcb, - 0xea, 0x2b, 0x44, 0x39, 0x5c, 0xd5, 0x06, 0xf2, 0x63, 0xc0, 0xdd, 0xc3, 0xdc, 0x1e, 0xe0, 0x1e, - 0x70, 0xfb, 0xe0, 0xee, 0x31, 0xdc, 0x35, 0x70, 0xf7, 0x68, 0xee, 0x63, 0x95, 0x23, 0xb8, 0x03, - 0x70, 0xf7, 0x44, 0xdc, 0xa1, 0x6a, 0x03, 0xb7, 0x64, 0xee, 0x5e, 0xe8, 0x1e, 0x67, 0xb8, 0x57, - 0xe9, 0x9e, 0x60, 0xee, 0xde, 0x48, 0xf7, 0x0f, 0x98, 0xbb, 0xd7, 0xe8, 0xfe, 0x1f, 0xe5, 0x08, - 0xee, 0xff, 0x65, 0xee, 0xde, 0x86, 0xee, 0xff, 0x53, 0x6d, 0xe0, 0xfe, 0x7f, 0x75, 0x5a, 0x09, - 0x47, 0x0a, 0x6c, 0xb7, 0xf4, 0x31, 0x0a, 0x67, 0x10, 0x24, 0xfa, 0x04, 0x03, 0x64, 0x20, 0x93, - 0xef, 0x53, 0x0c, 0x90, 0x41, 0x75, 0xda, 0xcf, 0x9a, 0xa2, 0x91, 0xfd, 0xf3, 0x36, 0x3e, 0x79, - 0xfa, 0x84, 0xda, 0xce, 0xe9, 0x8b, 0x36, 0x3e, 0xc4, 0x60, 0x82, 0xfc, 0x4b, 0x44, 0xc0, 0x36, - 0xcc, 0x5f, 0xb5, 0xb1, 0x28, 0x40, 0x9a, 0xf6, 0x6f, 0x8d, 0x20, 0x70, 0x1e, 0xb7, 0x8c, 0x3d, - 0x72, 0x9e, 0x14, 0xc1, 0x98, 0x53, 0x61, 0xda, 0xe3, 0x97, 0x31, 0x8d, 0x41, 0xc0, 0x7c, 0xc2, - 0x32, 0x3e, 0x95, 0x7b, 0x31, 0x26, 0x27, 0x24, 0xf8, 0x64, 0xec, 0x55, 0x83, 0x72, 0x22, 0x8e, - 0xfc, 0xde, 0x68, 0x54, 0x4e, 0x4a, 0xb0, 0xc2, 0x5e, 0x33, 0x2c, 0x27, 0x6b, 0x5f, 0x64, 0x3a, - 0x25, 0xc1, 0x07, 0x65, 0x6f, 0x63, 0x60, 0x4e, 0xd5, 0xad, 0xa8, 0xde, 0x69, 0xb8, 0x2c, 0xf4, - 0x6c, 0xe0, 0x14, 0x5b, 0xd0, 0xd0, 0xb3, 0x01, 0x29, 0xb6, 0x22, 0x45, 0xcf, 0x06, 0x93, 0x62, - 0x1b, 0x52, 0xf4, 0x6c, 0xd0, 0x29, 0xb6, 0x6b, 0x5f, 0xa4, 0xd8, 0x81, 0x14, 0x3d, 0x1b, 0xa2, - 0x14, 0x3b, 0x75, 0x2b, 0x52, 0x9c, 0x85, 0x14, 0x03, 0xe8, 0xc5, 0x39, 0x68, 0x18, 0x50, 0xbd, - 0x50, 0x17, 0x97, 0x81, 0xa8, 0x17, 0xe7, 0x21, 0xc5, 0x80, 0xe9, 0xc5, 0xf9, 0xda, 0x17, 0x29, - 0x2e, 0x40, 0x8a, 0x81, 0x46, 0x2f, 0x7e, 0xac, 0x5b, 0x91, 0xe2, 0x42, 0xc5, 0x54, 0xaf, 0x54, - 0x54, 0x4f, 0x2e, 0x51, 0xde, 0xb0, 0x91, 0xea, 0xd2, 0x84, 0x6d, 0x5b, 0x73, 0x15, 0x60, 0xd2, - 0x5d, 0x96, 0xe0, 0x82, 0x2b, 0x4c, 0xa7, 0xbc, 0x3c, 0x16, 0x87, 0xb4, 0x57, 0xf0, 0xed, 0x60, - 0x9e, 0x8e, 0x33, 0xa9, 0xaf, 0x8c, 0x79, 0x21, 0xfd, 0x55, 0x10, 0xbe, 0xbe, 0x52, 0x57, 0xfb, - 0xd3, 0xae, 0x04, 0x8f, 0x2d, 0x9b, 0x26, 0xd5, 0xee, 0x04, 0xdf, 0x64, 0x18, 0x61, 0x75, 0x4f, - 0x42, 0x6d, 0xbf, 0xeb, 0x69, 0xff, 0xa7, 0x20, 0x04, 0xb6, 0x09, 0x78, 0x1a, 0x8c, 0x80, 0x38, - 0xe2, 0x59, 0xe5, 0x51, 0xaf, 0x48, 0xb7, 0x56, 0x99, 0x40, 0xd0, 0x73, 0x09, 0x9e, 0xd2, 0x06, - 0x32, 0x71, 0xcf, 0x23, 0xb5, 0x41, 0x39, 0xf4, 0x97, 0x10, 0xdb, 0xed, 0x8e, 0xb9, 0x25, 0xb5, - 0x59, 0xbd, 0x80, 0x52, 0x28, 0xc0, 0x84, 0xbd, 0x98, 0xd0, 0x73, 0x4f, 0xc1, 0x1c, 0xf7, 0x52, - 0x82, 0x77, 0xa5, 0x08, 0x32, 0x9e, 0x7b, 0xd0, 0x93, 0x7e, 0xbf, 0x04, 0xae, 0xbd, 0x20, 0xef, - 0xf7, 0x4b, 0xa6, 0x79, 0x1f, 0x46, 0xa5, 0xdf, 0x2f, 0x31, 0xc5, 0xfe, 0x84, 0x9d, 0xb4, 0x16, - 0xc4, 0xc4, 0x0c, 0x79, 0xa1, 0x5b, 0xf6, 0x44, 0x89, 0x5e, 0x86, 0x5b, 0x76, 0x18, 0x9d, 0xbb, - 0x8d, 0x98, 0x32, 0x3b, 0x1c, 0x0e, 0xb1, 0x75, 0xbb, 0xb1, 0x90, 0xe0, 0x0e, 0xc2, 0xc5, 0x72, - 0x38, 0x2a, 0xcc, 0x9d, 0x84, 0x5d, 0x51, 0xb8, 0xbc, 0x66, 0x0a, 0x7e, 0xb7, 0x28, 0xd2, 0x5d, - 0xcd, 0x10, 0x93, 0xdc, 0x1d, 0x41, 0xdd, 0xa2, 0xa8, 0xa0, 0x7b, 0x9a, 0x21, 0x0e, 0xbc, 0x97, - 0xb8, 0xc8, 0x0c, 0x21, 0xd7, 0x7d, 0xc4, 0x85, 0x59, 0x53, 0xf1, 0xfd, 0x20, 0xe2, 0xbe, 0x7f, - 0x12, 0xc6, 0x4c, 0x0f, 0x34, 0xb0, 0x88, 0xfd, 0xc1, 0x49, 0x18, 0xc7, 0x3e, 0x44, 0x3c, 0xcc, - 0xc0, 0xc0, 0xff, 0x30, 0xec, 0x41, 0xbf, 0x5e, 0x51, 0xc5, 0xdb, 0x8d, 0xf4, 0x83, 0x7e, 0xdd, - 0x43, 0xb5, 0x9e, 0x68, 0x98, 0x1c, 0xfd, 0x24, 0xd9, 0x8b, 0xad, 0x43, 0x61, 0x1e, 0xed, 0xca, - 0xd1, 0x35, 0x81, 0x53, 0x54, 0x93, 0x85, 0x78, 0x5c, 0x9a, 0x70, 0x8e, 0x7e, 0x7a, 0x2a, 0xcc, - 0x2c, 0xcf, 0xa0, 0x9a, 0x7d, 0x7e, 0xf9, 0x3b, 0xd9, 0xa0, 0x4c, 0x2f, 0x20, 0x45, 0x9f, 0x5f, - 0x5e, 0x99, 0x0d, 0xca, 0x21, 0xbd, 0x48, 0xf6, 0x2c, 0xab, 0x85, 0x4d, 0x7a, 0x89, 0x78, 0x8d, - 0xb2, 0xdf, 0xb7, 0x69, 0x0f, 0xe0, 0x41, 0xc7, 0x2b, 0xd1, 0x5e, 0x14, 0x8d, 0x3f, 0x99, 0x34, - 0x2f, 0x44, 0x89, 0xf6, 0x11, 0xdf, 0xec, 0xd7, 0xfb, 0x9b, 0x69, 0x3f, 0xfc, 0xba, 0x7c, 0x6f, - 0x8c, 0x5e, 0x46, 0x78, 0xd7, 0x60, 0xd7, 0xaa, 0x95, 0xf4, 0x0a, 0xe0, 0xbc, 0x5b, 0xf6, 0xe8, - 0x55, 0xf5, 0x79, 0x6c, 0x20, 0xe9, 0x57, 0xf8, 0xcc, 0x16, 0xfd, 0x90, 0x5e, 0x53, 0x9f, 0xa1, - 0xeb, 0xd1, 0xaf, 0x21, 0x28, 0x2b, 0x1d, 0x0f, 0xfa, 0x5e, 0x47, 0x85, 0xd8, 0x54, 0x02, 0x7f, - 0x83, 0x5c, 0x5d, 0x7e, 0x48, 0x07, 0xf4, 0x97, 0xa4, 0x37, 0xd0, 0xa3, 0x6e, 0x51, 0x0e, 0x84, - 0x08, 0xe9, 0xb7, 0xc0, 0x7b, 0xc6, 0x6b, 0xf4, 0x3b, 0xb2, 0x0f, 0xb1, 0xa6, 0xad, 0xcf, 0xd1, - 0xef, 0xe1, 0x30, 0xe8, 0x94, 0x5c, 0xc7, 0x0b, 0xe9, 0x0f, 0x70, 0xc8, 0xbb, 0x1e, 0xfd, 0x11, - 0x5f, 0x05, 0xc7, 0xa3, 0x37, 0x89, 0x17, 0x49, 0x21, 0xa8, 0xf3, 0xe3, 0x06, 0x93, 0xfd, 0x2d, - 0xf4, 0xd3, 0x20, 0xa8, 0xf3, 0xdb, 0x7c, 0x37, 0x9e, 0x6f, 0x20, 0x33, 0xfd, 0xde, 0xc1, 0x7c, - 0xec, 0xf3, 0xcb, 0x6e, 0xd1, 0xa9, 0xf0, 0x4b, 0xe8, 0xa2, 0x24, 0xf6, 0x75, 0x05, 0xac, 0x0b, - 0xe8, 0xe2, 0x64, 0xcc, 0x81, 0x5f, 0x42, 0x3f, 0x49, 0x72, 0x17, 0x87, 0x3c, 0x27, 0x98, 0x18, - 0xf0, 0x25, 0x5d, 0x92, 0xe4, 0x89, 0x02, 0x33, 0x5a, 0xdd, 0x57, 0x24, 0x79, 0x93, 0x69, 0x60, - 0xc8, 0x7f, 0x65, 0x92, 0xc7, 0xbf, 0x01, 0x1a, 0x05, 0x57, 0x81, 0x40, 0x5b, 0xb9, 0x10, 0x8f, - 0xda, 0xab, 0x41, 0x60, 0xf6, 0x27, 0x0d, 0xfe, 0x14, 0x4a, 0x98, 0x4b, 0x03, 0xd7, 0x24, 0xb9, - 0x93, 0x6a, 0xa7, 0xd3, 0xd0, 0xb5, 0xf0, 0xe1, 0x9d, 0x4e, 0x03, 0xd7, 0xa1, 0x3b, 0x39, 0x4f, - 0x6a, 0xfb, 0x7a, 0x38, 0xf0, 0x26, 0xa7, 0x81, 0x1b, 0xe0, 0xa0, 0x9e, 0x81, 0x79, 0xb7, 0x4c, - 0x37, 0xa2, 0x7b, 0x78, 0x05, 0xb2, 0x79, 0x93, 0x69, 0xfe, 0x9e, 0x0f, 0xfb, 0x66, 0x63, 0x0f, - 0x88, 0x32, 0xdb, 0xb7, 0x34, 0xf8, 0x0b, 0x41, 0x5d, 0xd0, 0xad, 0x31, 0x91, 0x00, 0x7e, 0xd6, - 0xdc, 0x3d, 0x60, 0x3f, 0x87, 0x13, 0x82, 0xd6, 0x38, 0x95, 0x50, 0xd0, 0x6d, 0x49, 0x1e, 0x40, - 0x15, 0xa5, 0x90, 0xdb, 0x55, 0x05, 0x4c, 0x98, 0x02, 0xef, 0x50, 0x63, 0x23, 0x46, 0x64, 0x7e, - 0xd4, 0x1d, 0x91, 0x74, 0x97, 0x4a, 0xe6, 0x96, 0x47, 0x35, 0x70, 0xb7, 0x11, 0xaf, 0x1e, 0xcb, - 0x77, 0x82, 0xa4, 0x57, 0xc8, 0xf5, 0x4e, 0xe0, 0x54, 0xf5, 0x91, 0x70, 0x4f, 0x12, 0xa7, 0xa5, - 0x90, 0x1b, 0x9d, 0x80, 0xee, 0x85, 0x31, 0xe8, 0x6f, 0xe6, 0xbe, 0xdc, 0x07, 0x23, 0xaf, 0x5a, - 0xee, 0x87, 0x6e, 0x3c, 0xca, 0xa3, 0xa1, 0x7a, 0x00, 0x7c, 0x1a, 0x33, 0x87, 0xc7, 0x83, 0xc9, - 0xe8, 0xf5, 0x8e, 0xc1, 0x7f, 0x08, 0xa2, 0x94, 0xfd, 0xdd, 0xfc, 0xba, 0x01, 0x7a, 0x38, 0x06, - 0x60, 0x26, 0x3c, 0x82, 0x71, 0x53, 0x80, 0x16, 0xf5, 0x68, 0xcc, 0x07, 0xe7, 0xd4, 0x63, 0x49, - 0x5e, 0x80, 0x39, 0x8f, 0x19, 0x2f, 0x5d, 0xca, 0xba, 0x72, 0x1e, 0x82, 0x2f, 0x5b, 0xaa, 0x0a, - 0x6e, 0x34, 0x5d, 0xbe, 0x94, 0xbb, 0x9c, 0xf3, 0x34, 0xd1, 0x15, 0xda, 0x17, 0x24, 0x57, 0x2e, - 0x55, 0x75, 0x8e, 0xb4, 0x5e, 0xa5, 0x5b, 0x71, 0xc8, 0x5d, 0xad, 0x98, 0x46, 0x78, 0x1a, 0x70, - 0x9a, 0x93, 0xdb, 0xe0, 0x0d, 0x1b, 0xa9, 0x4e, 0x69, 0xe3, 0x12, 0x28, 0xc0, 0xa4, 0x3b, 0x15, - 0x77, 0x23, 0x85, 0xe9, 0x94, 0xa7, 0xc5, 0xe2, 0x90, 0xf6, 0x74, 0x5c, 0xc3, 0x74, 0x9c, 0x49, - 0x7d, 0x46, 0x1b, 0x3a, 0x34, 0xc2, 0x99, 0xce, 0x6c, 0x83, 0x8c, 0x11, 0x64, 0xd9, 0xd2, 0xa6, - 0x64, 0x98, 0x0c, 0x5b, 0x71, 0x39, 0xcb, 0x8d, 0x68, 0xf6, 0x6d, 0xda, 0x17, 0xcc, 0xdb, 0x75, - 0xaa, 0x88, 0x75, 0x47, 0x2c, 0x37, 0x3a, 0xb5, 0x53, 0xbb, 0xab, 0x8b, 0x0a, 0x6e, 0x6d, 0x5d, - 0x4e, 0x28, 0x8e, 0x1e, 0x15, 0x28, 0xe5, 0x5f, 0xd4, 0x1d, 0x5c, 0x23, 0xc8, 0xff, 0x9e, 0xba, - 0xeb, 0x6b, 0xc8, 0xa8, 0xf8, 0xab, 0xba, 0xeb, 0x6b, 0x54, 0x6b, 0x79, 0xbf, 0x29, 0x1a, 0x8a, - 0x3e, 0x50, 0x77, 0x7d, 0x13, 0x6d, 0x74, 0x7d, 0xd8, 0xe4, 0x09, 0x31, 0x1f, 0x41, 0x6a, 0x16, - 0xb3, 0x29, 0x98, 0xa8, 0x49, 0x3a, 0x3b, 0xa5, 0x81, 0x1e, 0x4f, 0x01, 0xe7, 0xa4, 0x70, 0x7c, - 0xf9, 0x55, 0xfc, 0xaa, 0x49, 0xe7, 0xa6, 0xf0, 0x63, 0x4e, 0xf7, 0x51, 0x74, 0x1e, 0x1a, 0xd6, - 0x3b, 0x61, 0xb8, 0xd9, 0x0f, 0x4a, 0x74, 0x7e, 0x0a, 0xab, 0xc5, 0xf1, 0x4a, 0x7e, 0x55, 0xfd, - 0x78, 0x75, 0x41, 0x0a, 0xfb, 0xf2, 0xda, 0xec, 0x77, 0xe8, 0xc7, 0xe6, 0x73, 0x25, 0x5d, 0x88, - 0x04, 0x43, 0x5e, 0xd1, 0x30, 0x5e, 0x94, 0xb2, 0x5b, 0x2d, 0xbb, 0x01, 0x88, 0x52, 0x9f, 0xf0, - 0xca, 0x72, 0x94, 0x2e, 0x46, 0x82, 0x6e, 0x47, 0x3a, 0xc3, 0x4e, 0x28, 0xe8, 0x9a, 0x14, 0x0e, - 0x3a, 0x3e, 0x7b, 0x06, 0xfd, 0xcd, 0x21, 0x5d, 0x8b, 0x84, 0x5d, 0xf5, 0x20, 0x10, 0x9e, 0x1c, - 0x0a, 0x45, 0x40, 0xd7, 0x21, 0x0b, 0x3e, 0xaf, 0x4f, 0xa9, 0x67, 0x8d, 0xe7, 0x89, 0x22, 0x76, - 0xab, 0x6e, 0xba, 0x01, 0x50, 0x1f, 0xae, 0xd7, 0xa1, 0x08, 0x64, 0xae, 0x9b, 0x6e, 0x44, 0xea, - 0x38, 0xc4, 0x07, 0x50, 0xae, 0x9b, 0x6e, 0x4a, 0xe1, 0x37, 0x21, 0x11, 0x84, 0x5c, 0xb7, 0x9b, - 0x91, 0xa9, 0xe0, 0x76, 0x77, 0x1a, 0xe4, 0x96, 0x94, 0x3a, 0x45, 0x37, 0xab, 0x25, 0x7d, 0x6b, - 0x8a, 0xa7, 0x51, 0xbe, 0x22, 0x44, 0x8d, 0x1e, 0x4f, 0xa9, 0xe3, 0xae, 0xb8, 0x89, 0x76, 0xa9, - 0x92, 0x88, 0x8a, 0x70, 0x42, 0x01, 0x64, 0x37, 0xbf, 0xea, 0xe7, 0x9b, 0x1f, 0x1d, 0xbc, 0x09, - 0xac, 0x2d, 0x7a, 0x22, 0xc5, 0xc3, 0x15, 0xbd, 0x52, 0x0d, 0xfc, 0x64, 0x4a, 0x5f, 0xde, 0x23, - 0xe4, 0x29, 0x74, 0x81, 0xd7, 0x6f, 0x04, 0x3d, 0x0d, 0x08, 0x3f, 0xb6, 0x18, 0xe8, 0x99, 0x14, - 0xcf, 0x12, 0xfd, 0xb3, 0x81, 0x01, 0x9f, 0x85, 0x1f, 0x5e, 0xed, 0x06, 0x7a, 0x2e, 0xa5, 0x16, - 0xa9, 0x90, 0x59, 0xe9, 0x7b, 0xf4, 0x7c, 0x64, 0x0e, 0x48, 0xdf, 0xa1, 0x5f, 0xa6, 0xd4, 0x92, - 0x16, 0x32, 0x83, 0xe6, 0x17, 0x1a, 0x36, 0xda, 0x5f, 0x4c, 0x61, 0x4e, 0x87, 0xb9, 0xf5, 0x63, - 0xab, 0xe9, 0x25, 0xb0, 0x2b, 0x83, 0xa7, 0x8a, 0x23, 0x69, 0x4f, 0x0c, 0xea, 0x77, 0x6a, 0x35, - 0x51, 0xa2, 0xbd, 0x8d, 0x90, 0x0c, 0xed, 0x53, 0x83, 0x35, 0x94, 0xeb, 0xa6, 0xfd, 0xa8, 0x76, - 0x9f, 0xbb, 0x49, 0xf0, 0x86, 0xb7, 0x67, 0x19, 0xd7, 0x68, 0x50, 0x94, 0xc5, 0x78, 0xad, 0xd3, - 0xe5, 0x93, 0x89, 0xd1, 0xbd, 0xcb, 0x70, 0xad, 0x01, 0xca, 0xf6, 0xbe, 0x65, 0xdc, 0x49, 0x9e, - 0xc3, 0x3d, 0xe3, 0x32, 0x70, 0x8a, 0x92, 0xc1, 0xd3, 0x97, 0x1b, 0x70, 0xc8, 0x3b, 0xb6, 0xee, - 0x4b, 0xf0, 0x9d, 0xb1, 0x9c, 0xcb, 0xc8, 0x60, 0x61, 0xa2, 0x06, 0xe4, 0xcc, 0xe5, 0xe6, 0xb8, - 0xc9, 0x0b, 0xc4, 0x6d, 0x59, 0xce, 0x7b, 0x0a, 0xce, 0x1f, 0xcc, 0x03, 0xc6, 0xb6, 0x46, 0x5c, - 0x83, 0xa2, 0x56, 0x71, 0x8a, 0x88, 0xdc, 0x16, 0x39, 0x0e, 0x8a, 0xaa, 0x3f, 0x06, 0x6c, 0xfb, - 0x72, 0x0c, 0x4a, 0xe8, 0x7b, 0xfd, 0x22, 0x28, 0x03, 0xda, 0x11, 0xb9, 0xad, 0x1b, 0x3e, 0x46, - 0x28, 0x6d, 0x3b, 0x23, 0xb7, 0x6c, 0x10, 0x38, 0xe8, 0xd3, 0x59, 0x70, 0xeb, 0x76, 0xa4, 0x58, - 0xe3, 0x07, 0x55, 0x07, 0x6e, 0xef, 0xb5, 0xb3, 0x5a, 0xc6, 0xfa, 0x5c, 0x29, 0x02, 0x5e, 0xf4, - 0xed, 0x7a, 0x19, 0x88, 0x6e, 0x77, 0x64, 0x84, 0xde, 0x6f, 0x37, 0xef, 0x09, 0x1e, 0x4c, 0x40, - 0x1f, 0xb4, 0xf3, 0x34, 0x66, 0x53, 0x0d, 0x7a, 0xd4, 0xf0, 0x61, 0xbb, 0xfd, 0x0f, 0xd6, 0x62, - 0x33, 0xb1, 0x26, 0x35, 0x7e, 0xd4, 0x6e, 0xff, 0xa3, 0x95, 0x34, 0x8d, 0xe6, 0xdf, 0xa8, 0xf9, - 0x63, 0x90, 0x36, 0x62, 0xa2, 0x86, 0x4f, 0x40, 0xaa, 0x1a, 0xa6, 0x44, 0x7d, 0xda, 0x6e, 0x1f, - 0x66, 0x2d, 0x54, 0x8d, 0x93, 0xf2, 0x7d, 0xd6, 0xce, 0xb3, 0xdc, 0x50, 0x45, 0xf0, 0xe7, 0xed, - 0x3c, 0x0f, 0xb8, 0x7b, 0xf4, 0x05, 0x3e, 0xd7, 0xfa, 0xf5, 0x80, 0xbe, 0x6c, 0xc7, 0x0d, 0xdf, - 0xf5, 0xea, 0x52, 0xd0, 0x57, 0xed, 0xea, 0x08, 0x2c, 0xfa, 0x5e, 0x89, 0xfe, 0x86, 0x02, 0xf5, - 0xbb, 0xc5, 0xc0, 0xd7, 0xc8, 0x71, 0x69, 0x5e, 0x7d, 0xfd, 0xbe, 0x27, 0x47, 0xe9, 0xf8, 0x34, - 0x1e, 0x3d, 0xfc, 0x3d, 0xe0, 0x54, 0x05, 0x9d, 0x90, 0xc6, 0x23, 0xca, 0xdf, 0xcc, 0x0b, 0x9b, - 0x2f, 0x7f, 0x27, 0xa6, 0xf1, 0x44, 0x53, 0x80, 0x5f, 0x97, 0x8c, 0x9d, 0x94, 0xc6, 0x65, 0xcf, - 0x99, 0x40, 0xc8, 0xc9, 0x08, 0xe9, 0x76, 0x26, 0xd6, 0x8d, 0x28, 0xce, 0x53, 0xc0, 0x09, 0xe0, - 0x68, 0x21, 0x36, 0xd1, 0xa9, 0x0d, 0xfb, 0xbf, 0x85, 0x13, 0xd0, 0x69, 0x69, 0x1e, 0x11, 0x6e, - 0x62, 0xce, 0x7e, 0xbf, 0x24, 0xe8, 0xf4, 0x34, 0xcf, 0x61, 0x03, 0xf9, 0x75, 0x09, 0xf4, 0x0c, - 0xe4, 0x61, 0xb4, 0xdb, 0x99, 0xa0, 0x33, 0x91, 0x87, 0x2d, 0xcd, 0xb3, 0x25, 0xcd, 0xdd, 0xc7, - 0xe7, 0xd6, 0x34, 0xd7, 0x8a, 0x3f, 0x9b, 0x68, 0xb7, 0xa5, 0xf9, 0xa9, 0x13, 0x87, 0x0d, 0xf5, - 0x76, 0x68, 0xea, 0x15, 0x52, 0xcd, 0x24, 0xda, 0x91, 0xc6, 0x1e, 0x30, 0x11, 0x72, 0x61, 0x71, - 0x55, 0x0f, 0x6b, 0xb4, 0x33, 0xcd, 0x97, 0xbd, 0x18, 0xe8, 0xd7, 0x25, 0xe3, 0x67, 0xa5, 0x63, - 0x3b, 0x29, 0x46, 0xe2, 0x87, 0xd0, 0xaf, 0x11, 0x1e, 0xaa, 0x6f, 0x73, 0x9d, 0x7e, 0x34, 0x19, - 0xc5, 0x75, 0xfa, 0x6c, 0xc8, 0xc6, 0xb9, 0x73, 0x4e, 0x5a, 0x6d, 0x94, 0xd5, 0x68, 0xf2, 0x9e, - 0x8b, 0x2e, 0x0f, 0x15, 0xba, 0x40, 0x7b, 0x1e, 0xf4, 0x0f, 0x15, 0xba, 0xd8, 0x25, 0x94, 0x4e, - 0xb5, 0x66, 0x86, 0xe5, 0xfc, 0x34, 0x4f, 0xaa, 0xc9, 0x2d, 0x7a, 0x7c, 0x2e, 0x40, 0x63, 0xb6, - 0x54, 0x62, 0x12, 0x89, 0x2d, 0xac, 0xd4, 0x78, 0xa1, 0xa7, 0x79, 0xc6, 0x35, 0x37, 0xea, 0x93, - 0xf1, 0x42, 0xa4, 0xcb, 0x96, 0x4a, 0x98, 0x6c, 0x8e, 0xc4, 0xa4, 0xc3, 0xfd, 0xf2, 0x22, 0x13, - 0x64, 0x76, 0xc9, 0x06, 0xdf, 0xc5, 0x69, 0x9e, 0xf7, 0xf1, 0x26, 0xcd, 0xf6, 0x13, 0x54, 0x4f, - 0xb3, 0x29, 0x0c, 0x5c, 0x97, 0x44, 0xea, 0xa2, 0x0d, 0xbc, 0xc1, 0x76, 0x69, 0xa4, 0xae, 0xd1, - 0xa8, 0xf9, 0x2e, 0x6b, 0x52, 0xa7, 0x9b, 0xc1, 0x78, 0xb9, 0x91, 0xc0, 0x9a, 0xe3, 0x6c, 0x57, - 0x60, 0x5a, 0x34, 0x1a, 0x34, 0xd3, 0x95, 0x50, 0x90, 0xaf, 0x0f, 0x7f, 0x5d, 0x7d, 0xae, 0x82, - 0x82, 0xe6, 0x46, 0x1d, 0x77, 0x35, 0x14, 0xe4, 0xeb, 0xc3, 0x53, 0xea, 0xf3, 0x53, 0x13, 0x34, - 0xb5, 0x3e, 0xd7, 0x40, 0x5c, 0xbc, 0x49, 0xb3, 0x5d, 0xab, 0x66, 0x97, 0x62, 0x8b, 0xd5, 0xe7, - 0xba, 0x48, 0xdd, 0xd7, 0xd4, 0xe7, 0xfa, 0x48, 0xdd, 0x94, 0xfa, 0xdc, 0xd0, 0xa4, 0x2e, 0x5e, - 0x9f, 0x1b, 0x8d, 0x84, 0xc9, 0xf5, 0xb9, 0x09, 0xf5, 0x69, 0x34, 0x68, 0xa6, 0x9b, 0x91, 0x64, - 0xc8, 0x73, 0xc7, 0xa3, 0xd9, 0xa5, 0x27, 0x31, 0xdd, 0x82, 0x88, 0xa6, 0x26, 0xbe, 0x91, 0xdd, - 0x3a, 0x15, 0xe6, 0x77, 0xee, 0xcf, 0xd2, 0xea, 0xff, 0xc0, 0xbc, 0x31, 0x11, 0xc8, 0xc2, 0x0f, - 0xe8, 0xe7, 0x69, 0xfc, 0x7e, 0xe2, 0x6c, 0x42, 0xf5, 0xe8, 0xb6, 0xc8, 0xc4, 0x6a, 0xb8, 0x1d, - 0xde, 0xeb, 0x45, 0xe0, 0xfa, 0xa5, 0x6c, 0xa9, 0x44, 0x77, 0x60, 0xc1, 0x2b, 0x1b, 0x5b, 0xdf, - 0x9d, 0x58, 0x1c, 0x1b, 0xea, 0x4e, 0x20, 0x45, 0x40, 0x77, 0xc1, 0x3d, 0xcf, 0xef, 0x7a, 0x84, - 0xdf, 0x0d, 0x1b, 0x7b, 0x25, 0xef, 0x74, 0x74, 0x4f, 0xda, 0x9c, 0xe0, 0xd0, 0xc3, 0x8c, 0xf7, - 0xa6, 0x79, 0x63, 0x2c, 0xf8, 0xdd, 0xce, 0x44, 0x48, 0xf7, 0x29, 0x7f, 0xbd, 0x2b, 0x86, 0x74, - 0x3f, 0xb6, 0x00, 0xbd, 0xa4, 0xcc, 0x3a, 0x7b, 0x00, 0x83, 0x14, 0x03, 0xf5, 0x12, 0x7b, 0x10, - 0xdb, 0x62, 0x44, 0x8e, 0x85, 0xfd, 0x10, 0x08, 0x22, 0x4c, 0x3d, 0x96, 0x1f, 0x4e, 0x9b, 0x5d, - 0x1c, 0xa0, 0x59, 0xe7, 0x8f, 0xa0, 0x2b, 0x7c, 0x93, 0xe2, 0xad, 0xed, 0x51, 0xb0, 0xe5, 0x65, - 0xc0, 0xd2, 0xa4, 0x2a, 0xce, 0x63, 0x6a, 0x7a, 0xc4, 0x31, 0x9e, 0x8d, 0xf4, 0x8b, 0x49, 0xb8, - 0x19, 0xcb, 0xc7, 0x91, 0x68, 0x4d, 0xe0, 0x57, 0xcd, 0x30, 0x40, 0xd4, 0xae, 0x29, 0x30, 0xeb, - 0xa2, 0xdd, 0xd8, 0x9a, 0xf4, 0x55, 0x20, 0xe2, 0x7e, 0xa2, 0x09, 0x35, 0xcc, 0x4f, 0x9a, 0x85, - 0xc7, 0x7e, 0xf1, 0xa3, 0x8a, 0x9e, 0x8a, 0xaf, 0x2f, 0xfd, 0x84, 0xf5, 0x24, 0x3d, 0x1d, 0xdf, - 0x62, 0xf4, 0xc1, 0xa7, 0x6f, 0xe4, 0xcf, 0xc4, 0x23, 0x72, 0x9e, 0xf9, 0x1d, 0xf6, 0x59, 0x14, - 0xa0, 0x01, 0x33, 0xcb, 0x73, 0x69, 0xfb, 0x70, 0x6b, 0x91, 0xc6, 0x8c, 0x44, 0xed, 0xff, 0x7c, - 0x7c, 0xc5, 0x9b, 0x36, 0xfc, 0xae, 0x66, 0x16, 0xc1, 0x14, 0xad, 0x2f, 0xc4, 0xe7, 0x7a, 0x43, - 0xeb, 0x8b, 0xf1, 0xe5, 0xde, 0xac, 0xf5, 0xa5, 0x78, 0x44, 0x43, 0xeb, 0x1e, 0x35, 0x58, 0x11, - 0xcc, 0x2c, 0x7b, 0xa1, 0x55, 0x63, 0x93, 0xb4, 0xee, 0x8b, 0xaf, 0xbe, 0xb8, 0xd6, 0xfd, 0x98, - 0xfe, 0x3c, 0x34, 0x98, 0x8f, 0x2f, 0xa7, 0xcd, 0xab, 0x5f, 0x9f, 0x49, 0xaf, 0x34, 0xcf, 0x31, - 0x2c, 0x81, 0x57, 0xd3, 0xfa, 0xe5, 0xae, 0xef, 0xfc, 0xfb, 0x33, 0xfc, 0xbc, 0xe8, 0x74, 0x3d, - 0x7a, 0x39, 0xc3, 0xa7, 0x7a, 0x36, 0xdf, 0x95, 0xcb, 0xd1, 0x2b, 0x19, 0xfc, 0x06, 0x34, 0xea, - 0x04, 0xf4, 0x6a, 0x86, 0x59, 0xf9, 0x53, 0x47, 0xfc, 0x2a, 0xc3, 0x6b, 0x40, 0xff, 0x17, 0xf2, - 0x6b, 0x19, 0xf5, 0x58, 0x61, 0xe3, 0xe8, 0x3c, 0xfd, 0x3a, 0xc3, 0xb3, 0x52, 0xaf, 0x57, 0x7a, - 0x1d, 0xdc, 0x3d, 0x15, 0x49, 0xbf, 0xc9, 0xb0, 0x92, 0x9e, 0xf1, 0x9a, 0x1f, 0xc8, 0xbc, 0x90, - 0xab, 0x78, 0x06, 0x1d, 0x68, 0xc6, 0x56, 0x33, 0xf6, 0x46, 0x33, 0x76, 0x14, 0x63, 0xbf, 0x45, - 0x0a, 0xfc, 0x39, 0x01, 0x77, 0xfa, 0x77, 0x19, 0xbc, 0x4a, 0xd8, 0xc4, 0xc3, 0xed, 0xf7, 0x19, - 0x3e, 0x02, 0x61, 0xeb, 0x7a, 0xfd, 0x41, 0x7b, 0x78, 0xa5, 0x1c, 0xdf, 0x37, 0xe9, 0x8f, 0x10, - 0xac, 0x6b, 0xf2, 0x66, 0x06, 0xd3, 0x19, 0x06, 0x2f, 0xc7, 0x3e, 0xbf, 0xe8, 0x54, 0x04, 0xbd, - 0x85, 0x5e, 0x72, 0x29, 0x3b, 0x9d, 0x50, 0x64, 0x56, 0xd3, 0xdb, 0x20, 0x59, 0x2b, 0xc6, 0xf9, - 0x39, 0x10, 0x94, 0xe9, 0x1d, 0x63, 0xe7, 0x65, 0xc0, 0xf6, 0x9f, 0x32, 0xea, 0x0d, 0xcd, 0xb7, - 0x57, 0xfa, 0x73, 0x46, 0xfd, 0xce, 0xca, 0x86, 0xba, 0x41, 0xd3, 0xbb, 0x19, 0xf5, 0x72, 0x0f, - 0x65, 0x40, 0x7f, 0xc9, 0xa8, 0x77, 0x45, 0x28, 0x03, 0xdd, 0xfa, 0x1e, 0x5a, 0xfb, 0x0a, 0x81, - 0x5b, 0xa5, 0xbf, 0xa2, 0xd6, 0x7d, 0x62, 0x44, 0xd2, 0xfb, 0x50, 0xc1, 0x9f, 0xda, 0xef, 0x03, - 0x64, 0xd1, 0x85, 0xff, 0x10, 0x34, 0x2c, 0x58, 0x0a, 0xb5, 0x37, 0x7c, 0x14, 0x43, 0x56, 0x01, - 0xf9, 0x38, 0xc3, 0x8b, 0x5b, 0x21, 0x8a, 0x43, 0x79, 0x7e, 0x32, 0x05, 0x57, 0xfe, 0x9f, 0x2a, - 0x21, 0xfe, 0x66, 0x11, 0xd0, 0x67, 0x4a, 0x48, 0xcd, 0x29, 0xd1, 0xe7, 0x4a, 0x48, 0xcd, 0x29, - 0x69, 0x21, 0x5f, 0x60, 0x60, 0x79, 0xa7, 0xe5, 0x8a, 0x7e, 0x09, 0x59, 0xeb, 0x8a, 0x98, 0xba, - 0x5f, 0xa1, 0x32, 0xeb, 0x8a, 0xd1, 0x7f, 0x89, 0x60, 0xd4, 0xd7, 0x05, 0x25, 0x3a, 0xae, 0x83, - 0xc9, 0x37, 0xf0, 0xbb, 0x80, 0x8e, 0xc7, 0xf7, 0x20, 0x7a, 0x7c, 0x42, 0x07, 0x7e, 0x6e, 0x11, - 0x35, 0xe1, 0x48, 0x3a, 0xb1, 0x03, 0xbf, 0xeb, 0xa9, 0x2b, 0x3f, 0x9d, 0xa4, 0xad, 0x31, 0x11, - 0x84, 0x82, 0x4e, 0xee, 0xe0, 0xe9, 0xa0, 0x2d, 0xad, 0xe4, 0x14, 0x45, 0xe4, 0x96, 0x47, 0x25, - 0x9d, 0xda, 0x81, 0xf7, 0x1e, 0x7f, 0xeb, 0xd6, 0xd3, 0x3a, 0xf0, 0x23, 0x27, 0xf7, 0xe1, 0xf4, - 0x0e, 0xfc, 0x2c, 0xd4, 0xe8, 0xc3, 0x19, 0x88, 0xcc, 0xd7, 0x38, 0xcf, 0x99, 0x1d, 0xea, 0xcf, - 0x17, 0x82, 0x62, 0xb5, 0x46, 0x5b, 0x3a, 0x70, 0x77, 0xab, 0x0f, 0x87, 0xe8, 0x81, 0xaa, 0xd8, - 0xd6, 0x66, 0x50, 0x95, 0x6b, 0x5b, 0x87, 0x5e, 0xec, 0x0a, 0x8c, 0x57, 0x78, 0xfb, 0xd7, 0x35, - 0xa9, 0xa8, 0x1d, 0xcd, 0x54, 0x39, 0xaf, 0x24, 0xc6, 0x69, 0x67, 0x07, 0x4f, 0xeb, 0x82, 0xaf, - 0xe7, 0xdb, 0x59, 0xca, 0x0c, 0xdc, 0x2a, 0x76, 0xdd, 0x1f, 0x76, 0xe0, 0x6c, 0x09, 0xdc, 0xaa, - 0x62, 0xff, 0x51, 0x64, 0x2b, 0xca, 0xb3, 0xd1, 0x97, 0x21, 0x6f, 0xad, 0x18, 0xa7, 0x73, 0xd4, - 0x77, 0xad, 0x26, 0x02, 0x3a, 0xb7, 0xa3, 0xf3, 0x9b, 0xbb, 0x0e, 0x2c, 0x49, 0x3c, 0x77, 0x60, - 0x49, 0xe2, 0xf5, 0x03, 0x4b, 0x12, 0xe7, 0xbd, 0xb1, 0xe4, 0x20, 0x6b, 0x51, 0xd1, 0xaf, 0xb6, - 0xd7, 0x5c, 0xaf, 0x5c, 0x74, 0x6a, 0xed, 0xd2, 0x2d, 0x0d, 0xe3, 0xaf, 0x6d, 0xd6, 0x27, 0xfe, - 0x1e, 0x00, 0x00, 0xff, 0xff, 0x3b, 0x25, 0x93, 0x70, 0x3b, 0x25, 0x00, 0x00, + 0xfb, 0x50, 0xcb, 0x8e, 0xb9, 0x1a, 0x49, 0xf3, 0x9b, 0x7d, 0xa1, 0x89, 0x4c, 0x32, 0x4d, 0xa8, + 0x64, 0x2d, 0x30, 0xc9, 0x22, 0x14, 0xca, 0x16, 0xda, 0xad, 0xd6, 0x21, 0x4d, 0xb0, 0x16, 0x77, + 0xa8, 0x9d, 0xb4, 0x16, 0x34, 0x35, 0x18, 0x7d, 0xad, 0x53, 0x98, 0x20, 0x31, 0x69, 0x1f, 0x66, + 0x2d, 0x6c, 0x0e, 0x30, 0x2a, 0x0f, 0x9b, 0x12, 0x01, 0xa1, 0x87, 0x9b, 0x41, 0x52, 0x29, 0x95, + 0xce, 0x45, 0x46, 0xbd, 0x01, 0x21, 0x73, 0xb1, 0xe9, 0xbf, 0x41, 0xb5, 0xca, 0x36, 0x23, 0xdf, + 0xe0, 0x46, 0xe4, 0x92, 0xc9, 0x34, 0xd0, 0x78, 0x84, 0xe9, 0x54, 0xe4, 0x6e, 0x24, 0xa6, 0x26, + 0xfb, 0x43, 0xe1, 0xb7, 0xcc, 0xf0, 0x72, 0xb4, 0xd2, 0xb7, 0xcc, 0x54, 0x5c, 0x41, 0x50, 0xf7, + 0x6d, 0x13, 0xac, 0x30, 0xad, 0xed, 0x48, 0xd3, 0x6b, 0x85, 0x1a, 0x65, 0xdf, 0x69, 0x26, 0x80, + 0xae, 0x7f, 0x36, 0xdd, 0xd3, 0xae, 0x46, 0xd5, 0xbf, 0x34, 0xfb, 0x42, 0x53, 0x7b, 0x54, 0x4c, + 0xed, 0xa5, 0x74, 0x2d, 0x37, 0x14, 0x0d, 0x18, 0xda, 0x56, 0x44, 0xe3, 0x18, 0xe1, 0x5a, 0xdf, + 0xca, 0x68, 0xc0, 0xa2, 0x16, 0xa3, 0xf1, 0xa8, 0xa9, 0x64, 0xd0, 0x99, 0xb1, 0x0f, 0xb7, 0x0e, + 0x9d, 0x14, 0x62, 0xb4, 0x76, 0x4c, 0x8d, 0x81, 0xde, 0x7f, 0x35, 0x35, 0x64, 0x4b, 0x69, 0xfd, + 0x37, 0xd3, 0x2d, 0x05, 0x41, 0xe7, 0xbf, 0x9b, 0x1a, 0x2a, 0x4c, 0x6b, 0xfc, 0x0f, 0xd3, 0x59, + 0x85, 0x1a, 0x7d, 0xff, 0xd9, 0x4c, 0x00, 0x6d, 0xff, 0x65, 0xf2, 0x6b, 0x57, 0xa3, 0x2b, 0xdb, + 0xec, 0x0b, 0x4d, 0x9d, 0xbc, 0x19, 0x76, 0xf9, 0x4e, 0x45, 0x84, 0x45, 0xc1, 0x8a, 0xde, 0x6a, + 0xb3, 0x0f, 0xb6, 0xe6, 0x18, 0x04, 0x82, 0xde, 0x6e, 0xb3, 0x17, 0x58, 0xf3, 0x0d, 0x64, 0x32, + 0xbf, 0xd3, 0x66, 0x1f, 0x62, 0xcd, 0x33, 0xa8, 0x56, 0xf9, 0x6e, 0x53, 0x34, 0xd4, 0xfc, 0xa9, + 0xcd, 0x5e, 0x68, 0x51, 0x14, 0x6d, 0xc4, 0xfc, 0xb9, 0xc9, 0x13, 0x5a, 0xde, 0x6b, 0xe3, 0x03, + 0xa8, 0xaf, 0xc0, 0x32, 0x4a, 0x7c, 0x2e, 0xf5, 0x15, 0x90, 0x5f, 0xf0, 0xf9, 0xd0, 0x57, 0x68, + 0x9c, 0x3d, 0x73, 0xac, 0x99, 0x7d, 0x05, 0x9d, 0xb1, 0xac, 0x1c, 0x91, 0x6a, 0xd4, 0x9e, 0x67, + 0x59, 0x7d, 0x85, 0x28, 0x87, 0xab, 0xda, 0x40, 0x7e, 0x0c, 0xb8, 0x7b, 0x98, 0xdb, 0x03, 0xdc, + 0x03, 0x6e, 0x1f, 0xdc, 0x3d, 0x86, 0xbb, 0x06, 0xee, 0x1e, 0xcd, 0x7d, 0xac, 0x72, 0x04, 0x77, + 0x00, 0xee, 0x9e, 0x88, 0x3b, 0x54, 0x6d, 0xe0, 0x96, 0xcc, 0xdd, 0x0b, 0xdd, 0xe3, 0x0c, 0xf7, + 0x2a, 0xdd, 0x13, 0xcc, 0xdd, 0x1b, 0xe9, 0xfe, 0x11, 0x73, 0xf7, 0x1a, 0xdd, 0xff, 0xa3, 0x1c, + 0xc1, 0xfd, 0xbf, 0xcc, 0xdd, 0xdb, 0xd0, 0xfd, 0x7f, 0xaa, 0x0d, 0xdc, 0xff, 0xaf, 0x4e, 0x2b, + 0xe1, 0x48, 0x81, 0xed, 0x96, 0x3e, 0x41, 0xe1, 0x0c, 0x82, 0x44, 0x9f, 0x62, 0x80, 0x0c, 0x64, + 0xf2, 0x7d, 0x86, 0x01, 0x32, 0xa8, 0x4e, 0xfb, 0x79, 0x53, 0x34, 0xb2, 0x7f, 0xd1, 0xc6, 0x27, + 0x4f, 0x9f, 0x50, 0xdb, 0x39, 0x7d, 0xd9, 0xc6, 0x87, 0x18, 0x4c, 0x90, 0x7f, 0x85, 0x08, 0xd8, + 0x86, 0xf9, 0xeb, 0x36, 0x16, 0x05, 0x48, 0xd3, 0xfe, 0xad, 0x11, 0x04, 0xce, 0xe3, 0x96, 0xb0, + 0x47, 0xce, 0x93, 0x22, 0x18, 0x73, 0x2a, 0x4c, 0x7b, 0xfc, 0x12, 0xa6, 0x31, 0x08, 0x98, 0x4f, + 0x58, 0xc2, 0xa7, 0x72, 0x2f, 0xc6, 0xe4, 0x84, 0x04, 0x9f, 0x8c, 0xbd, 0x6a, 0x50, 0x4e, 0xc4, + 0x91, 0xdf, 0x1b, 0x8d, 0xca, 0x49, 0x09, 0x56, 0xd8, 0x6b, 0x86, 0xe5, 0x64, 0xed, 0x8b, 0x4c, + 0xa7, 0x24, 0xf8, 0xa0, 0xec, 0x6d, 0x0c, 0xcc, 0xa9, 0xba, 0x15, 0xd5, 0x3b, 0x0d, 0x97, 0x85, + 0x9e, 0x75, 0x9c, 0x62, 0x13, 0x1a, 0x7a, 0xd6, 0x21, 0xc5, 0x66, 0xa4, 0xe8, 0x59, 0x67, 0x52, + 0x6c, 0x41, 0x8a, 0x9e, 0x75, 0x3a, 0xc5, 0x56, 0xed, 0x8b, 0x14, 0xdb, 0x90, 0xa2, 0x67, 0x5d, + 0x94, 0x62, 0xbb, 0x6e, 0x45, 0x8a, 0xb3, 0x90, 0x62, 0x00, 0xbd, 0x38, 0x0f, 0x0d, 0x03, 0xaa, + 0x17, 0xea, 0xe2, 0x32, 0x10, 0xf5, 0xe2, 0x02, 0xa4, 0x18, 0x30, 0xbd, 0xb8, 0x50, 0xfb, 0x22, + 0xc5, 0x45, 0x48, 0x31, 0xd0, 0xe8, 0xc5, 0x8f, 0x75, 0x2b, 0x52, 0x5c, 0xac, 0x98, 0xea, 0x95, + 0x8a, 0xea, 0xc9, 0x65, 0xca, 0x1b, 0x36, 0x52, 0x5d, 0x9e, 0xb0, 0x6d, 0x6b, 0xae, 0x02, 0x4c, + 0xba, 0x2b, 0x12, 0x5c, 0x70, 0x85, 0xe9, 0x94, 0x57, 0xc6, 0xe2, 0x90, 0xf6, 0x2a, 0xbe, 0x1d, + 0xcc, 0xd3, 0x71, 0x26, 0xf5, 0xd5, 0x31, 0x2f, 0xa4, 0xbf, 0x06, 0xc2, 0xd7, 0x56, 0xea, 0x6a, + 0x7f, 0xda, 0x91, 0xe0, 0xb1, 0x65, 0xd3, 0xa4, 0xda, 0x99, 0xe0, 0x9b, 0x0c, 0x23, 0xac, 0xee, + 0x69, 0xa8, 0xed, 0x77, 0x3d, 0xed, 0xff, 0x0c, 0x84, 0xc0, 0x36, 0x01, 0xcf, 0x82, 0x11, 0x10, + 0x47, 0x3c, 0xaf, 0x3c, 0xea, 0x15, 0xe9, 0xd6, 0x2a, 0x13, 0x08, 0x7a, 0x21, 0xc1, 0x53, 0xda, + 0x40, 0x26, 0xee, 0x45, 0xa4, 0x36, 0x28, 0x87, 0xfe, 0x12, 0x62, 0xbb, 0xdd, 0x31, 0xb7, 0xa4, + 0x36, 0xab, 0x97, 0x50, 0x0a, 0x05, 0x98, 0xb0, 0x5d, 0x09, 0x3d, 0xf7, 0x14, 0xcc, 0x71, 0x2f, + 0x27, 0x78, 0x57, 0x8a, 0x20, 0xe3, 0xb9, 0x1b, 0x3d, 0xe9, 0xf7, 0x4b, 0xe0, 0xda, 0x03, 0xf2, + 0x7e, 0xbf, 0x64, 0x9a, 0xf7, 0x62, 0x54, 0xfa, 0xfd, 0x12, 0x53, 0xec, 0x4b, 0xd8, 0x49, 0xeb, + 0x90, 0x98, 0x98, 0x21, 0x2f, 0x74, 0xcb, 0x9e, 0x28, 0xd1, 0x2b, 0x70, 0xcb, 0x0e, 0xa3, 0x73, + 0x77, 0x10, 0x53, 0x66, 0x87, 0xc3, 0x21, 0xb6, 0xee, 0x34, 0x16, 0x12, 0xdc, 0x45, 0xb8, 0x58, + 0x0e, 0x47, 0x85, 0xb9, 0x9b, 0xb0, 0x2b, 0x0a, 0x97, 0xd7, 0x4c, 0xc1, 0xef, 0x16, 0x45, 0xba, + 0xa7, 0x19, 0x62, 0x92, 0x7b, 0x23, 0xa8, 0x5b, 0x14, 0x15, 0x74, 0x5f, 0x33, 0xc4, 0x81, 0xf7, + 0x13, 0x17, 0x99, 0x21, 0xe4, 0x7a, 0x80, 0xb8, 0x30, 0xab, 0x2a, 0xbe, 0x1f, 0x44, 0xdc, 0x0f, + 0x4e, 0xc2, 0x98, 0xe9, 0xa1, 0x06, 0x16, 0xb1, 0x3f, 0x3c, 0x09, 0xe3, 0xd8, 0x47, 0x88, 0x87, + 0x19, 0x18, 0xf8, 0x1f, 0x85, 0x3d, 0xe8, 0xd7, 0x3d, 0x55, 0xbc, 0x9d, 0x48, 0x0f, 0x9b, 0x29, + 0x9e, 0x6a, 0x98, 0x1c, 0xfd, 0x34, 0xd9, 0x87, 0x5a, 0x07, 0xc3, 0x3c, 0xda, 0x95, 0xa3, 0xab, + 0x02, 0xa7, 0xa8, 0x26, 0x0b, 0xf1, 0xb8, 0x34, 0xe1, 0x1c, 0xfd, 0xec, 0x54, 0x98, 0x59, 0x9e, + 0x43, 0x35, 0xfb, 0xfc, 0xf2, 0xf7, 0xb2, 0x41, 0x99, 0x5e, 0x42, 0x8a, 0x3e, 0xbf, 0xbc, 0x3c, + 0x1b, 0x94, 0x43, 0xda, 0x45, 0xf6, 0x2c, 0xab, 0x85, 0x4d, 0x7a, 0x99, 0x78, 0x8d, 0xb2, 0xdf, + 0x77, 0x69, 0x37, 0xe0, 0x41, 0xc7, 0x2b, 0xd1, 0x1e, 0x14, 0x8d, 0x3f, 0x99, 0x34, 0x2f, 0x44, + 0x89, 0xf6, 0x12, 0xdf, 0xec, 0xd7, 0xfa, 0x1b, 0x69, 0x1f, 0xfc, 0xba, 0x7c, 0x6f, 0x8c, 0x5e, + 0x41, 0x78, 0xd7, 0x60, 0xd7, 0x8a, 0xe5, 0xf4, 0x2a, 0xe0, 0xbc, 0x5b, 0xf6, 0xe8, 0x35, 0xf5, + 0x79, 0x6c, 0x20, 0xe9, 0x57, 0xf8, 0xcc, 0x16, 0xfd, 0x90, 0x5e, 0x57, 0x9f, 0xa1, 0xeb, 0xd1, + 0xaf, 0x21, 0x28, 0x2b, 0x1d, 0x0f, 0xfa, 0xde, 0x40, 0x85, 0xd8, 0x54, 0x02, 0x7f, 0x83, 0x5c, + 0x5d, 0x7e, 0x48, 0xfb, 0xf5, 0x97, 0xa4, 0x37, 0xd1, 0xa3, 0x6e, 0x51, 0x0e, 0x84, 0x08, 0xe9, + 0xb7, 0xc0, 0x7b, 0xc6, 0x6b, 0xf4, 0x3b, 0xb2, 0x0f, 0xb2, 0xa6, 0xad, 0xcd, 0xd1, 0xef, 0xe1, + 0x30, 0xe8, 0x94, 0x5c, 0xc7, 0x0b, 0xe9, 0x0f, 0x70, 0xc8, 0xbb, 0x1e, 0xfd, 0x11, 0x5f, 0x05, + 0xc7, 0xa3, 0xb7, 0x88, 0x17, 0x49, 0x21, 0xa8, 0xf3, 0xe3, 0x06, 0x93, 0xfd, 0x6d, 0xf4, 0xd3, + 0x20, 0xa8, 0xf3, 0x3b, 0x7c, 0x37, 0x9e, 0x6f, 0x20, 0x33, 0xfd, 0xde, 0xc5, 0x7c, 0xec, 0xf3, + 0xcb, 0x6e, 0xd1, 0xa9, 0xf0, 0x4b, 0xe8, 0x92, 0x24, 0xf6, 0x75, 0x05, 0xac, 0x09, 0xe8, 0xd2, + 0x64, 0xcc, 0x81, 0x5f, 0x42, 0x3f, 0x49, 0x72, 0x17, 0x87, 0x3c, 0x27, 0x98, 0x18, 0xf0, 0x25, + 0x5d, 0x96, 0xe4, 0x89, 0x02, 0x33, 0x5a, 0xdd, 0x57, 0x25, 0x79, 0x93, 0x69, 0x60, 0xc8, 0x7f, + 0x75, 0x92, 0xc7, 0xbf, 0x01, 0x1a, 0x05, 0xd7, 0x80, 0x40, 0x5b, 0xb9, 0x10, 0x8f, 0xda, 0x6b, + 0x41, 0x60, 0xf6, 0x27, 0x0d, 0xfe, 0x14, 0x4a, 0x98, 0x4b, 0x03, 0xd7, 0x25, 0xb9, 0x93, 0x6a, + 0xa7, 0xd3, 0xd0, 0xf5, 0xf0, 0xe1, 0x9d, 0x4e, 0x03, 0x37, 0xa0, 0x3b, 0x39, 0x4f, 0x6a, 0xfb, + 0x46, 0x38, 0xf0, 0x26, 0xa7, 0x81, 0x9b, 0xe0, 0xa0, 0x9e, 0x81, 0x79, 0xb7, 0x4c, 0x37, 0xa3, + 0x7b, 0x78, 0x05, 0xb2, 0x79, 0x8b, 0x69, 0xfe, 0x81, 0x0f, 0xfb, 0x56, 0x63, 0x0f, 0x88, 0x32, + 0xdb, 0xb7, 0x35, 0xf8, 0x0b, 0x41, 0x5d, 0xd0, 0xed, 0x31, 0x91, 0x00, 0x7e, 0xd6, 0xdc, 0x3d, + 0x60, 0x3f, 0x87, 0x13, 0x82, 0x56, 0x39, 0x95, 0x50, 0xd0, 0x1d, 0x49, 0x1e, 0x40, 0x15, 0xa5, + 0x90, 0x3b, 0x55, 0x05, 0x4c, 0x98, 0x02, 0xef, 0x52, 0x63, 0x23, 0x46, 0x64, 0x7e, 0xd4, 0x1d, + 0x91, 0x74, 0x8f, 0x4a, 0xe6, 0x96, 0x47, 0x35, 0x70, 0xaf, 0x11, 0xaf, 0x1e, 0xcb, 0x77, 0x83, + 0xa4, 0x57, 0xc8, 0xb5, 0x4e, 0xe0, 0x54, 0xf5, 0x91, 0x70, 0x5f, 0x12, 0xa7, 0xa5, 0x90, 0xeb, + 0x9d, 0x80, 0xee, 0x87, 0x31, 0xe8, 0x6f, 0xe4, 0xbe, 0x3c, 0x00, 0x23, 0xaf, 0x5a, 0x1e, 0x84, + 0x6e, 0x3c, 0xca, 0xa3, 0xa1, 0x7a, 0x08, 0x7c, 0x1a, 0x33, 0x87, 0xc7, 0xc3, 0xc9, 0xe8, 0xf5, + 0x8e, 0xc1, 0x7f, 0x04, 0xa2, 0x94, 0xfd, 0xfd, 0xfc, 0x9a, 0x01, 0x7a, 0x34, 0x06, 0x60, 0x26, + 0x3c, 0x86, 0x71, 0x53, 0x80, 0x16, 0xf5, 0x78, 0xcc, 0x07, 0xe7, 0xd4, 0x13, 0x49, 0x5e, 0x80, + 0x39, 0x8f, 0x19, 0x2f, 0x5f, 0xcc, 0xba, 0x72, 0x1e, 0x82, 0xaf, 0x58, 0xac, 0x0a, 0x6e, 0x34, + 0x5d, 0xb9, 0x98, 0xbb, 0x9c, 0xf3, 0x34, 0xd1, 0x55, 0xda, 0x17, 0x24, 0x57, 0x2f, 0x56, 0x75, + 0x8e, 0xb4, 0x5e, 0xa3, 0x5b, 0x71, 0xc8, 0x5d, 0xab, 0x98, 0x46, 0x78, 0x1a, 0x70, 0x9a, 0x93, + 0xdb, 0xe0, 0x0d, 0x1b, 0xa9, 0x4e, 0x69, 0xe3, 0x12, 0x28, 0xc0, 0xa4, 0x3b, 0x15, 0x77, 0x23, + 0x85, 0xe9, 0x94, 0xa7, 0xc5, 0xe2, 0x90, 0xf6, 0x74, 0x5c, 0xc3, 0x74, 0x9c, 0x49, 0x7d, 0x46, + 0x1b, 0x3a, 0x34, 0xc2, 0x99, 0xce, 0x6c, 0x83, 0x8c, 0x11, 0x64, 0xd9, 0xd4, 0xa6, 0x64, 0x98, + 0x0c, 0x9b, 0x71, 0x39, 0xcb, 0x8d, 0x68, 0xf6, 0x2d, 0xda, 0x17, 0xcc, 0x5b, 0x75, 0xaa, 0x88, + 0x75, 0x5b, 0x2c, 0x37, 0x3a, 0xb5, 0x5d, 0xbb, 0xab, 0x8b, 0x0a, 0x6e, 0x6d, 0x5d, 0x4e, 0x28, + 0x8e, 0x1e, 0x15, 0x28, 0xe5, 0x5f, 0xd4, 0x1d, 0x5c, 0x23, 0xc8, 0xff, 0xbe, 0xba, 0xeb, 0x6b, + 0xc8, 0xa8, 0xf8, 0xab, 0xba, 0xeb, 0x6b, 0x54, 0x6b, 0xf9, 0xa0, 0x29, 0x1a, 0x8a, 0x3e, 0x54, + 0x77, 0x7d, 0x13, 0x6d, 0x74, 0x7d, 0xd4, 0xe4, 0x09, 0x31, 0x1f, 0x43, 0x6a, 0x16, 0xb3, 0x29, + 0x98, 0xa8, 0x49, 0x3a, 0x37, 0xa5, 0x81, 0x1e, 0x4f, 0x01, 0xe7, 0xa5, 0x70, 0x7c, 0xf9, 0x55, + 0xfc, 0xaa, 0x49, 0xe7, 0xa7, 0xf0, 0x63, 0x4e, 0xf7, 0x51, 0x74, 0x01, 0x1a, 0xd6, 0x3a, 0x61, + 0xb8, 0xd1, 0x0f, 0x4a, 0x74, 0x61, 0x0a, 0xab, 0xc5, 0xf1, 0x4a, 0x7e, 0x55, 0xfd, 0x78, 0x75, + 0x51, 0x0a, 0xfb, 0xf2, 0xea, 0xec, 0xf7, 0xe8, 0xc7, 0xe6, 0x73, 0x39, 0x5d, 0x8c, 0x04, 0x43, + 0x5e, 0xd1, 0x30, 0x5e, 0x92, 0xb2, 0x5b, 0x2d, 0xbb, 0x01, 0x88, 0x52, 0x9f, 0xf0, 0xca, 0x72, + 0x94, 0x2e, 0x45, 0x82, 0x6e, 0x47, 0x3a, 0xc3, 0x4e, 0x28, 0xe8, 0xba, 0x14, 0x0e, 0x3a, 0x1c, + 0x6c, 0xfe, 0xc6, 0x90, 0xae, 0x47, 0xc2, 0xae, 0x7a, 0x10, 0x08, 0x4f, 0x0e, 0x85, 0x22, 0xa0, + 0x1b, 0x90, 0x05, 0x9f, 0x37, 0xa6, 0xd4, 0xb3, 0xc6, 0xf3, 0x44, 0x11, 0xbb, 0x55, 0x37, 0xdd, + 0x04, 0xa8, 0x0f, 0xd7, 0xeb, 0x50, 0x04, 0x32, 0xd7, 0x4d, 0x37, 0x23, 0x75, 0x1c, 0xe2, 0x03, + 0x28, 0xd7, 0x4d, 0xb7, 0xa4, 0xf0, 0x9b, 0x90, 0x08, 0x42, 0xae, 0xdb, 0xad, 0xc8, 0x54, 0x70, + 0xbb, 0x3b, 0x0d, 0x72, 0x5b, 0x4a, 0x9d, 0xa2, 0x1b, 0xd5, 0x92, 0xbe, 0x3d, 0xc5, 0xd3, 0x28, + 0x5f, 0x11, 0xa2, 0x46, 0x4f, 0xa6, 0xd4, 0x71, 0x57, 0xdc, 0x40, 0x3b, 0x54, 0x49, 0x44, 0x45, + 0x38, 0xa1, 0x00, 0xb2, 0x93, 0x5f, 0xf5, 0xf3, 0xcd, 0x8f, 0x0e, 0xde, 0x04, 0xd6, 0x16, 0x3d, + 0x95, 0xe2, 0xe1, 0x8a, 0x5e, 0xa9, 0x06, 0x7e, 0x3a, 0xa5, 0x2f, 0xef, 0x11, 0xf2, 0x0c, 0xba, + 0xc0, 0xeb, 0x37, 0x82, 0x9e, 0x05, 0x84, 0x1f, 0x5b, 0x0c, 0xf4, 0x5c, 0x8a, 0x67, 0x89, 0xfe, + 0xd9, 0xc0, 0x80, 0xcf, 0xc3, 0x0f, 0xaf, 0x76, 0x03, 0xbd, 0x90, 0x52, 0x8b, 0x54, 0xc8, 0xac, + 0xf4, 0x3d, 0x7a, 0x31, 0x32, 0x07, 0xa4, 0xef, 0xd0, 0x2f, 0x53, 0x6a, 0x49, 0x0b, 0x99, 0x41, + 0xf3, 0x4b, 0x0d, 0x1b, 0xed, 0xbb, 0x52, 0x98, 0xd3, 0x61, 0x6e, 0xed, 0xd8, 0x4a, 0x7a, 0x19, + 0xec, 0xca, 0xe0, 0xa9, 0xe2, 0x48, 0xda, 0x1d, 0x83, 0xfa, 0x9d, 0x5a, 0x4d, 0x94, 0x68, 0x4f, + 0x23, 0x24, 0x43, 0x7b, 0xd5, 0x60, 0x0d, 0xe5, 0xba, 0x69, 0x1f, 0xaa, 0xdd, 0xe7, 0x6e, 0x10, + 0xbc, 0xe1, 0xed, 0x5e, 0xc2, 0x35, 0x1a, 0x14, 0x65, 0x31, 0x5e, 0xeb, 0x74, 0xf9, 0x64, 0x62, + 0x74, 0xcf, 0x12, 0x5c, 0x6b, 0x80, 0xb2, 0xbd, 0x77, 0x09, 0x77, 0x92, 0xe7, 0x70, 0xcf, 0xb8, + 0x0c, 0x9c, 0xa2, 0x64, 0xf0, 0xf4, 0xa5, 0x06, 0x1c, 0xf2, 0x8e, 0xad, 0xfb, 0x12, 0x7c, 0x67, + 0x2c, 0xe5, 0x32, 0x32, 0x58, 0x98, 0xa8, 0x01, 0x39, 0x73, 0xa9, 0x39, 0x6e, 0xf2, 0x02, 0x71, + 0x9b, 0x96, 0xf2, 0x9e, 0x82, 0xf3, 0x07, 0xf3, 0x80, 0xb1, 0xcd, 0x11, 0xd7, 0xa0, 0xa8, 0x55, + 0x9c, 0x22, 0x22, 0xb7, 0x44, 0x8e, 0x83, 0xa2, 0xea, 0x8f, 0x01, 0xdb, 0xba, 0x14, 0x83, 0x12, + 0xfa, 0x5e, 0xbf, 0x08, 0xca, 0x80, 0xb6, 0x45, 0x6e, 0x6b, 0x86, 0x8f, 0x11, 0x4a, 0xdb, 0xf6, + 0xc8, 0x2d, 0x1b, 0x04, 0x0e, 0xfa, 0x74, 0x56, 0x04, 0xad, 0x77, 0x2a, 0x2e, 0x0e, 0xba, 0xb3, + 0x97, 0x72, 0xe7, 0x19, 0xea, 0xf2, 0x3d, 0xe9, 0xb8, 0x5e, 0xc8, 0xe8, 0x39, 0xe0, 0xeb, 0x76, + 0xa4, 0x58, 0xe5, 0x07, 0x55, 0x07, 0x7c, 0xef, 0xb7, 0x73, 0xb7, 0x18, 0xeb, 0x73, 0xa5, 0x08, + 0x78, 0x77, 0x68, 0xd7, 0xeb, 0x45, 0x74, 0xbb, 0x23, 0x23, 0xf4, 0x41, 0xbb, 0x79, 0x78, 0xf0, + 0xa8, 0x03, 0xfa, 0xb0, 0x9d, 0xe7, 0x3b, 0x9b, 0x6a, 0x76, 0x44, 0x0d, 0x1f, 0xb5, 0xdb, 0xff, + 0x60, 0x1d, 0x6a, 0x66, 0xe0, 0xa4, 0xc6, 0x8f, 0xdb, 0xed, 0x7f, 0xb4, 0x92, 0xa6, 0xd1, 0xfc, + 0x1b, 0x35, 0x7f, 0x02, 0xd2, 0x46, 0x4c, 0xd4, 0xf0, 0x29, 0x48, 0x55, 0xc3, 0x94, 0xa8, 0xcf, + 0xda, 0xed, 0xc3, 0xac, 0x05, 0xaa, 0x71, 0x52, 0xbe, 0xcf, 0xdb, 0x79, 0x39, 0x18, 0xaa, 0x08, + 0xfe, 0xa2, 0x9d, 0x27, 0x0c, 0x77, 0x8f, 0xbe, 0xc4, 0xe7, 0x6a, 0xbf, 0x1e, 0xd0, 0x57, 0xed, + 0x78, 0x0a, 0xb8, 0x5e, 0x5d, 0x0a, 0xfa, 0xba, 0x5d, 0x9d, 0x95, 0x45, 0xdf, 0x2b, 0xd1, 0xdf, + 0x50, 0xa0, 0x7e, 0xb7, 0x18, 0xf8, 0x1a, 0x39, 0x2e, 0xcd, 0xcb, 0xb4, 0xdf, 0xf7, 0xe4, 0x28, + 0x1d, 0x9f, 0xc6, 0xeb, 0x88, 0xbf, 0x07, 0x9c, 0xaa, 0xa0, 0x13, 0xd2, 0x78, 0x6d, 0xf9, 0x1b, + 0x79, 0x07, 0xe0, 0x5b, 0xe2, 0x89, 0x69, 0xbc, 0xe5, 0x14, 0xe0, 0xd7, 0x25, 0x63, 0x27, 0xa5, + 0x71, 0x2b, 0x74, 0x26, 0x10, 0x72, 0x32, 0x42, 0xba, 0x9d, 0x89, 0x35, 0x23, 0x8a, 0xf3, 0x14, + 0x70, 0x02, 0x38, 0x5a, 0x88, 0x0d, 0x74, 0x6a, 0xc3, 0xfe, 0x6f, 0xe1, 0x04, 0x74, 0x5a, 0x9a, + 0x47, 0x84, 0x9b, 0x98, 0xb3, 0xdf, 0x2f, 0x09, 0x3a, 0x3d, 0xcd, 0xe3, 0x6d, 0x20, 0xbf, 0x2e, + 0x81, 0x9e, 0x81, 0x3c, 0x8c, 0x76, 0x3b, 0x13, 0x74, 0x26, 0xf2, 0xb0, 0xa5, 0x79, 0x36, 0xa5, + 0xb9, 0xfb, 0xf8, 0xdc, 0x9c, 0xe6, 0x5a, 0xf1, 0x67, 0x13, 0xed, 0x96, 0x34, 0xbf, 0x89, 0xe2, + 0xb0, 0xa1, 0xde, 0x0a, 0x4d, 0xbd, 0x42, 0xaa, 0x99, 0x44, 0xdb, 0xd2, 0xd8, 0x2c, 0x26, 0x42, + 0x2e, 0x2c, 0xee, 0xf4, 0x61, 0x8d, 0xb6, 0xa7, 0xf9, 0x56, 0x18, 0x03, 0xfd, 0xba, 0x64, 0xfc, + 0xac, 0x74, 0x6c, 0xcb, 0xc5, 0x48, 0x9c, 0x0d, 0xfd, 0x1a, 0xe1, 0xa1, 0xfa, 0x2e, 0xd7, 0xe9, + 0x9c, 0xc9, 0x28, 0xee, 0xdd, 0xe7, 0x42, 0x36, 0x0e, 0xa8, 0xf3, 0xd2, 0x6a, 0x47, 0xad, 0x46, + 0x93, 0xf7, 0x7c, 0x74, 0x79, 0xa8, 0xd0, 0x05, 0xda, 0x0b, 0xa0, 0x7f, 0xa8, 0xd0, 0xc5, 0x2e, + 0xa1, 0x74, 0xaa, 0x35, 0x33, 0x2c, 0x17, 0xa6, 0x79, 0x52, 0x4d, 0x6e, 0xd1, 0xe3, 0x73, 0x11, + 0x1a, 0xb3, 0xa5, 0x12, 0x93, 0x48, 0xec, 0x75, 0xa5, 0xc6, 0x53, 0x3e, 0xcd, 0x33, 0xae, 0xb9, + 0x51, 0x1f, 0xa1, 0x17, 0x23, 0x5d, 0xb6, 0x54, 0xc2, 0x64, 0x73, 0x24, 0x26, 0x1d, 0x2e, 0xa2, + 0x97, 0x98, 0x20, 0xb3, 0x9d, 0x36, 0xf8, 0x2e, 0x4d, 0xf3, 0xbc, 0x8f, 0x37, 0x69, 0xb6, 0x9f, + 0xa0, 0x7a, 0x9a, 0x4d, 0x61, 0xe0, 0xba, 0x2c, 0x52, 0x17, 0xed, 0xf4, 0x0d, 0xb6, 0xcb, 0x23, + 0x75, 0x8d, 0x46, 0xcd, 0x77, 0x45, 0x93, 0x3a, 0xdd, 0x0c, 0xc6, 0x2b, 0x8d, 0x04, 0xd6, 0x1c, + 0x67, 0xbb, 0x0a, 0xd3, 0xa2, 0xd1, 0xa0, 0x99, 0xae, 0x86, 0x82, 0x7c, 0x7d, 0xf8, 0x9b, 0xea, + 0x73, 0x0d, 0x14, 0x34, 0x37, 0xea, 0xb8, 0x6b, 0xa1, 0x20, 0x5f, 0x1f, 0x9e, 0x52, 0x9f, 0x9f, + 0x9a, 0xa0, 0xa9, 0xf5, 0xb9, 0x0e, 0xe2, 0xe2, 0x4d, 0x9a, 0xed, 0x7a, 0x35, 0xbb, 0x14, 0x5b, + 0xac, 0x3e, 0x37, 0x44, 0xea, 0xbe, 0xa1, 0x3e, 0x37, 0x46, 0xea, 0xa6, 0xd4, 0xe7, 0xa6, 0x26, + 0x75, 0xf1, 0xfa, 0xdc, 0x6c, 0x24, 0x4c, 0xae, 0xcf, 0x2d, 0xa8, 0x4f, 0xa3, 0x41, 0x33, 0xdd, + 0x8a, 0x24, 0x43, 0x9e, 0x3b, 0x1e, 0xcd, 0x2e, 0x3d, 0x89, 0xe9, 0x36, 0x44, 0x34, 0x35, 0xf1, + 0xd5, 0xed, 0xf6, 0xa9, 0x30, 0x3f, 0x88, 0x7f, 0x96, 0x56, 0xff, 0x59, 0xe6, 0x8d, 0x89, 0x40, + 0x16, 0x7e, 0x44, 0x3f, 0x4f, 0xe3, 0x87, 0x16, 0x67, 0x03, 0xaa, 0x47, 0x77, 0x44, 0x26, 0x56, + 0xc3, 0x9d, 0xf0, 0x5e, 0x2b, 0x02, 0xd7, 0x2f, 0x65, 0x4b, 0x25, 0xba, 0x0b, 0x0b, 0x5e, 0xd9, + 0xd8, 0xfa, 0xee, 0xc6, 0xe2, 0x58, 0x57, 0x77, 0x02, 0x29, 0x02, 0xba, 0x07, 0xee, 0x79, 0x51, + 0x2c, 0xf8, 0x08, 0xbf, 0x17, 0x36, 0xf6, 0x4a, 0xde, 0xe9, 0xe8, 0xbe, 0xb4, 0x39, 0xea, 0xa1, + 0x87, 0x19, 0xef, 0x4f, 0xf3, 0xc6, 0x58, 0xf0, 0xbb, 0x9d, 0x89, 0x90, 0x1e, 0x50, 0xfe, 0x7a, + 0x57, 0x0c, 0xe9, 0x41, 0x6c, 0x01, 0x7a, 0x49, 0x99, 0x75, 0xf6, 0x10, 0x06, 0x29, 0x06, 0xea, + 0x25, 0xf6, 0x30, 0xb6, 0xc5, 0x88, 0x1c, 0x0b, 0xfb, 0x11, 0x10, 0x44, 0x98, 0x7a, 0x55, 0x3f, + 0x9a, 0x36, 0xbb, 0x38, 0x40, 0xb3, 0xce, 0x1f, 0x43, 0x57, 0xf8, 0xca, 0xc5, 0x5b, 0xdb, 0xe3, + 0x60, 0xcb, 0xcb, 0x80, 0xa5, 0x49, 0x55, 0x9c, 0x27, 0xd4, 0xf4, 0x88, 0x63, 0x3c, 0x1b, 0xe9, + 0x17, 0x93, 0x70, 0x33, 0x96, 0x4f, 0x22, 0xd1, 0xaa, 0xc0, 0xaf, 0x9a, 0x61, 0x80, 0xa8, 0x1d, + 0x53, 0x60, 0xd6, 0x45, 0x3b, 0xb1, 0x35, 0xe9, 0x3b, 0x43, 0xc4, 0xfd, 0x54, 0x13, 0x6a, 0x98, + 0x9f, 0x36, 0x0b, 0x8f, 0xfd, 0xe2, 0x47, 0x15, 0x3d, 0x13, 0x5f, 0x5f, 0xfa, 0xad, 0xeb, 0x49, + 0x7a, 0x36, 0xbe, 0xc5, 0xe8, 0x83, 0x4f, 0x5f, 0xdd, 0x9f, 0x8b, 0x47, 0xe4, 0x3c, 0xf3, 0x83, + 0xed, 0xf3, 0x28, 0x40, 0x03, 0x66, 0x96, 0x17, 0xd2, 0xf6, 0xe1, 0xd6, 0x42, 0x8d, 0x19, 0x89, + 0xda, 0xff, 0xc5, 0xf8, 0x8a, 0x37, 0x6d, 0xf8, 0x01, 0xce, 0x2c, 0x82, 0x29, 0x5a, 0x5f, 0x8a, + 0xcf, 0xf5, 0x86, 0xd6, 0x5d, 0xf1, 0xe5, 0xde, 0xac, 0xf5, 0xe5, 0x78, 0x44, 0x43, 0xeb, 0x6e, + 0x35, 0x58, 0x11, 0xcc, 0x2c, 0x7b, 0xa0, 0x55, 0x63, 0x93, 0xb4, 0xee, 0x8d, 0xaf, 0xbe, 0xb8, + 0xd6, 0x7d, 0x98, 0xfe, 0x3c, 0x34, 0x98, 0x8f, 0xaf, 0xa4, 0xcd, 0xcf, 0x03, 0xfa, 0x4c, 0x7a, + 0xb5, 0x79, 0x8e, 0x61, 0x09, 0xbc, 0x96, 0xd6, 0x4f, 0x7c, 0xfd, 0x38, 0xd8, 0x97, 0xe1, 0x77, + 0x48, 0xa7, 0xeb, 0xd1, 0x2b, 0x19, 0x3e, 0xd5, 0xb3, 0xf9, 0xae, 0x5c, 0x8e, 0x5e, 0xcd, 0xe0, + 0xc7, 0xa2, 0x51, 0x27, 0xa0, 0xd7, 0x32, 0xcc, 0xca, 0x9f, 0x3a, 0xe2, 0x57, 0x19, 0x5e, 0x03, + 0xfa, 0xff, 0x9a, 0x5f, 0xcf, 0xa8, 0x57, 0x0d, 0x1b, 0x47, 0xe7, 0xe9, 0xd7, 0x19, 0x9e, 0x95, + 0x7a, 0xbd, 0xd2, 0x1b, 0xe0, 0xee, 0xa9, 0x48, 0xfa, 0x4d, 0x86, 0x95, 0xf4, 0x8c, 0xd7, 0xfc, + 0x40, 0xe6, 0x85, 0x5c, 0xc1, 0x33, 0x68, 0x7f, 0x33, 0xb6, 0x92, 0xb1, 0x37, 0x9b, 0xb1, 0xa3, + 0x18, 0xfb, 0x2d, 0x52, 0xe0, 0xef, 0x0e, 0xb8, 0xd3, 0xbf, 0xcb, 0xe0, 0xf9, 0xc2, 0x26, 0x5e, + 0x78, 0xbf, 0xcf, 0xf0, 0x11, 0x08, 0x5b, 0xd7, 0xeb, 0x0f, 0xda, 0xc3, 0x2b, 0xe5, 0xf8, 0x62, + 0x4a, 0x7f, 0x84, 0x60, 0x5d, 0x93, 0xb7, 0x32, 0x98, 0xce, 0x30, 0x78, 0x39, 0xf6, 0xf9, 0x45, + 0xa7, 0x22, 0xe8, 0x6d, 0xf4, 0x92, 0x4b, 0xd9, 0xe9, 0x84, 0x22, 0xb3, 0x92, 0xde, 0x01, 0xc9, + 0x6a, 0x31, 0xce, 0xef, 0x86, 0xa0, 0x4c, 0xef, 0x1a, 0x3b, 0x2f, 0x03, 0xb6, 0xff, 0x94, 0x51, + 0x8f, 0x6d, 0xbe, 0xe6, 0xd2, 0x9f, 0x33, 0xea, 0x07, 0x59, 0x36, 0xd4, 0x55, 0x9b, 0xde, 0xcb, + 0xa8, 0x27, 0x7e, 0x28, 0x03, 0xfa, 0x4b, 0x46, 0x3d, 0x40, 0x42, 0x19, 0xe8, 0xd6, 0xf7, 0xd1, + 0xda, 0x57, 0x08, 0xdc, 0x2a, 0xfd, 0x15, 0xb5, 0xee, 0x13, 0x23, 0x92, 0x3e, 0x80, 0x0a, 0xfe, + 0xd4, 0x7e, 0x1f, 0x22, 0x8b, 0x2e, 0xfc, 0x47, 0xa0, 0x61, 0xc1, 0x52, 0xa8, 0xbd, 0xe1, 0xe3, + 0x18, 0xb2, 0x02, 0xc8, 0x27, 0x19, 0x5e, 0xdc, 0x0a, 0x51, 0x1c, 0xca, 0xf3, 0xd3, 0x29, 0xb8, + 0xf2, 0xff, 0x4c, 0x09, 0xf1, 0x37, 0x8a, 0x80, 0x3e, 0x57, 0x42, 0x6a, 0x4e, 0x89, 0xbe, 0x50, + 0x42, 0x6a, 0x4e, 0x49, 0x0b, 0xf9, 0x12, 0x03, 0xcb, 0x3b, 0x2d, 0x57, 0xf4, 0x2b, 0xc8, 0x5a, + 0x53, 0xc4, 0xd4, 0xfd, 0x1a, 0x95, 0x59, 0x53, 0x8c, 0xfe, 0xef, 0x04, 0xa3, 0xbe, 0x26, 0x28, + 0xd1, 0x71, 0x1d, 0x4c, 0xbe, 0x8e, 0x1f, 0x10, 0x74, 0x3c, 0xbe, 0x07, 0xd1, 0xe3, 0x13, 0x3a, + 0xf0, 0xbb, 0x8c, 0xa8, 0x09, 0x47, 0xd2, 0x89, 0x1d, 0xf8, 0x01, 0x50, 0xbd, 0x0d, 0xe8, 0x24, + 0x6d, 0x8d, 0x89, 0x20, 0x14, 0x74, 0x72, 0x07, 0x4f, 0x07, 0x6d, 0x69, 0x25, 0xa7, 0x28, 0x22, + 0xb7, 0x3c, 0x2a, 0xe9, 0xd4, 0x0e, 0x3c, 0x0c, 0xf9, 0x5b, 0xb7, 0x9e, 0xd6, 0x81, 0x5f, 0x43, + 0xb9, 0x0f, 0xa7, 0x77, 0xe0, 0xf7, 0xa3, 0x46, 0x1f, 0xce, 0x40, 0x64, 0xbe, 0xc6, 0x79, 0xce, + 0xec, 0x50, 0x7f, 0xe7, 0x10, 0x14, 0xab, 0x35, 0xda, 0xd4, 0x81, 0xbb, 0x5b, 0x7d, 0x38, 0x44, + 0x0f, 0x54, 0xc5, 0x36, 0x37, 0x83, 0xaa, 0x5c, 0x5b, 0x3a, 0xf4, 0x62, 0x57, 0x60, 0xbc, 0xc2, + 0x5b, 0xbf, 0xa9, 0x49, 0x45, 0x6d, 0x6b, 0xa6, 0xca, 0x79, 0x25, 0x31, 0x4e, 0xdb, 0x3b, 0x78, + 0x5a, 0x17, 0x7c, 0x3d, 0xdf, 0xce, 0x52, 0x66, 0xe0, 0x56, 0xb1, 0xeb, 0x9e, 0xdd, 0x81, 0xb3, + 0x25, 0x70, 0xab, 0x8a, 0xfd, 0x9c, 0xc8, 0x56, 0x94, 0xe7, 0xa2, 0x2f, 0x43, 0xde, 0x6a, 0x31, + 0x4e, 0xe7, 0xa9, 0xef, 0x5a, 0x4d, 0x04, 0x74, 0x7e, 0x47, 0xe7, 0xb7, 0x77, 0xec, 0x5f, 0x94, + 0x78, 0x61, 0xff, 0xa2, 0xc4, 0x1b, 0xfb, 0x17, 0x25, 0x2e, 0x78, 0x73, 0xd1, 0x01, 0xd6, 0xc2, + 0xa2, 0x5f, 0x6d, 0xaf, 0xb9, 0x5e, 0xb9, 0xe8, 0xd4, 0xda, 0xa5, 0x5b, 0x1a, 0xc6, 0x9f, 0xe5, + 0xac, 0x4d, 0xfc, 0x3d, 0x00, 0x00, 0xff, 0xff, 0x84, 0x9c, 0x23, 0x87, 0x64, 0x25, 0x00, 0x00, } diff --git a/vendor/github.com/pkg/errors/LICENSE b/vendor/github.com/pkg/errors/LICENSE new file mode 100644 index 0000000000000..835ba3e755cef --- /dev/null +++ b/vendor/github.com/pkg/errors/LICENSE @@ -0,0 +1,23 @@ +Copyright (c) 2015, Dave Cheney +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/vendor/github.com/pkg/errors/errors.go b/vendor/github.com/pkg/errors/errors.go new file mode 100644 index 0000000000000..842ee80456dba --- /dev/null +++ b/vendor/github.com/pkg/errors/errors.go @@ -0,0 +1,269 @@ +// Package errors provides simple error handling primitives. +// +// The traditional error handling idiom in Go is roughly akin to +// +// if err != nil { +// return err +// } +// +// which applied recursively up the call stack results in error reports +// without context or debugging information. The errors package allows +// programmers to add context to the failure path in their code in a way +// that does not destroy the original value of the error. +// +// Adding context to an error +// +// The errors.Wrap function returns a new error that adds context to the +// original error by recording a stack trace at the point Wrap is called, +// and the supplied message. For example +// +// _, err := ioutil.ReadAll(r) +// if err != nil { +// return errors.Wrap(err, "read failed") +// } +// +// If additional control is required the errors.WithStack and errors.WithMessage +// functions destructure errors.Wrap into its component operations of annotating +// an error with a stack trace and an a message, respectively. +// +// Retrieving the cause of an error +// +// Using errors.Wrap constructs a stack of errors, adding context to the +// preceding error. Depending on the nature of the error it may be necessary +// to reverse the operation of errors.Wrap to retrieve the original error +// for inspection. Any error value which implements this interface +// +// type causer interface { +// Cause() error +// } +// +// can be inspected by errors.Cause. errors.Cause will recursively retrieve +// the topmost error which does not implement causer, which is assumed to be +// the original cause. For example: +// +// switch err := errors.Cause(err).(type) { +// case *MyError: +// // handle specifically +// default: +// // unknown error +// } +// +// causer interface is not exported by this package, but is considered a part +// of stable public API. +// +// Formatted printing of errors +// +// All error values returned from this package implement fmt.Formatter and can +// be formatted by the fmt package. The following verbs are supported +// +// %s print the error. If the error has a Cause it will be +// printed recursively +// %v see %s +// %+v extended format. Each Frame of the error's StackTrace will +// be printed in detail. +// +// Retrieving the stack trace of an error or wrapper +// +// New, Errorf, Wrap, and Wrapf record a stack trace at the point they are +// invoked. This information can be retrieved with the following interface. +// +// type stackTracer interface { +// StackTrace() errors.StackTrace +// } +// +// Where errors.StackTrace is defined as +// +// type StackTrace []Frame +// +// The Frame type represents a call site in the stack trace. Frame supports +// the fmt.Formatter interface that can be used for printing information about +// the stack trace of this error. For example: +// +// if err, ok := err.(stackTracer); ok { +// for _, f := range err.StackTrace() { +// fmt.Printf("%+s:%d", f) +// } +// } +// +// stackTracer interface is not exported by this package, but is considered a part +// of stable public API. +// +// See the documentation for Frame.Format for more details. +package errors + +import ( + "fmt" + "io" +) + +// New returns an error with the supplied message. +// New also records the stack trace at the point it was called. +func New(message string) error { + return &fundamental{ + msg: message, + stack: callers(), + } +} + +// Errorf formats according to a format specifier and returns the string +// as a value that satisfies error. +// Errorf also records the stack trace at the point it was called. +func Errorf(format string, args ...interface{}) error { + return &fundamental{ + msg: fmt.Sprintf(format, args...), + stack: callers(), + } +} + +// fundamental is an error that has a message and a stack, but no caller. +type fundamental struct { + msg string + *stack +} + +func (f *fundamental) Error() string { return f.msg } + +func (f *fundamental) Format(s fmt.State, verb rune) { + switch verb { + case 'v': + if s.Flag('+') { + io.WriteString(s, f.msg) + f.stack.Format(s, verb) + return + } + fallthrough + case 's': + io.WriteString(s, f.msg) + case 'q': + fmt.Fprintf(s, "%q", f.msg) + } +} + +// WithStack annotates err with a stack trace at the point WithStack was called. +// If err is nil, WithStack returns nil. +func WithStack(err error) error { + if err == nil { + return nil + } + return &withStack{ + err, + callers(), + } +} + +type withStack struct { + error + *stack +} + +func (w *withStack) Cause() error { return w.error } + +func (w *withStack) Format(s fmt.State, verb rune) { + switch verb { + case 'v': + if s.Flag('+') { + fmt.Fprintf(s, "%+v", w.Cause()) + w.stack.Format(s, verb) + return + } + fallthrough + case 's': + io.WriteString(s, w.Error()) + case 'q': + fmt.Fprintf(s, "%q", w.Error()) + } +} + +// Wrap returns an error annotating err with a stack trace +// at the point Wrap is called, and the supplied message. +// If err is nil, Wrap returns nil. +func Wrap(err error, message string) error { + if err == nil { + return nil + } + err = &withMessage{ + cause: err, + msg: message, + } + return &withStack{ + err, + callers(), + } +} + +// Wrapf returns an error annotating err with a stack trace +// at the point Wrapf is call, and the format specifier. +// If err is nil, Wrapf returns nil. +func Wrapf(err error, format string, args ...interface{}) error { + if err == nil { + return nil + } + err = &withMessage{ + cause: err, + msg: fmt.Sprintf(format, args...), + } + return &withStack{ + err, + callers(), + } +} + +// WithMessage annotates err with a new message. +// If err is nil, WithMessage returns nil. +func WithMessage(err error, message string) error { + if err == nil { + return nil + } + return &withMessage{ + cause: err, + msg: message, + } +} + +type withMessage struct { + cause error + msg string +} + +func (w *withMessage) Error() string { return w.msg + ": " + w.cause.Error() } +func (w *withMessage) Cause() error { return w.cause } + +func (w *withMessage) Format(s fmt.State, verb rune) { + switch verb { + case 'v': + if s.Flag('+') { + fmt.Fprintf(s, "%+v\n", w.Cause()) + io.WriteString(s, w.msg) + return + } + fallthrough + case 's', 'q': + io.WriteString(s, w.Error()) + } +} + +// Cause returns the underlying cause of the error, if possible. +// An error value has a cause if it implements the following +// interface: +// +// type causer interface { +// Cause() error +// } +// +// If the error does not implement Cause, the original error will +// be returned. If the error is nil, nil will be returned without further +// investigation. +func Cause(err error) error { + type causer interface { + Cause() error + } + + for err != nil { + cause, ok := err.(causer) + if !ok { + break + } + err = cause.Cause() + } + return err +} diff --git a/vendor/github.com/pkg/errors/stack.go b/vendor/github.com/pkg/errors/stack.go new file mode 100644 index 0000000000000..6b1f2891a5ac0 --- /dev/null +++ b/vendor/github.com/pkg/errors/stack.go @@ -0,0 +1,178 @@ +package errors + +import ( + "fmt" + "io" + "path" + "runtime" + "strings" +) + +// Frame represents a program counter inside a stack frame. +type Frame uintptr + +// pc returns the program counter for this frame; +// multiple frames may have the same PC value. +func (f Frame) pc() uintptr { return uintptr(f) - 1 } + +// file returns the full path to the file that contains the +// function for this Frame's pc. +func (f Frame) file() string { + fn := runtime.FuncForPC(f.pc()) + if fn == nil { + return "unknown" + } + file, _ := fn.FileLine(f.pc()) + return file +} + +// line returns the line number of source code of the +// function for this Frame's pc. +func (f Frame) line() int { + fn := runtime.FuncForPC(f.pc()) + if fn == nil { + return 0 + } + _, line := fn.FileLine(f.pc()) + return line +} + +// Format formats the frame according to the fmt.Formatter interface. +// +// %s source file +// %d source line +// %n function name +// %v equivalent to %s:%d +// +// Format accepts flags that alter the printing of some verbs, as follows: +// +// %+s path of source file relative to the compile time GOPATH +// %+v equivalent to %+s:%d +func (f Frame) Format(s fmt.State, verb rune) { + switch verb { + case 's': + switch { + case s.Flag('+'): + pc := f.pc() + fn := runtime.FuncForPC(pc) + if fn == nil { + io.WriteString(s, "unknown") + } else { + file, _ := fn.FileLine(pc) + fmt.Fprintf(s, "%s\n\t%s", fn.Name(), file) + } + default: + io.WriteString(s, path.Base(f.file())) + } + case 'd': + fmt.Fprintf(s, "%d", f.line()) + case 'n': + name := runtime.FuncForPC(f.pc()).Name() + io.WriteString(s, funcname(name)) + case 'v': + f.Format(s, 's') + io.WriteString(s, ":") + f.Format(s, 'd') + } +} + +// StackTrace is stack of Frames from innermost (newest) to outermost (oldest). +type StackTrace []Frame + +func (st StackTrace) Format(s fmt.State, verb rune) { + switch verb { + case 'v': + switch { + case s.Flag('+'): + for _, f := range st { + fmt.Fprintf(s, "\n%+v", f) + } + case s.Flag('#'): + fmt.Fprintf(s, "%#v", []Frame(st)) + default: + fmt.Fprintf(s, "%v", []Frame(st)) + } + case 's': + fmt.Fprintf(s, "%s", []Frame(st)) + } +} + +// stack represents a stack of program counters. +type stack []uintptr + +func (s *stack) Format(st fmt.State, verb rune) { + switch verb { + case 'v': + switch { + case st.Flag('+'): + for _, pc := range *s { + f := Frame(pc) + fmt.Fprintf(st, "\n%+v", f) + } + } + } +} + +func (s *stack) StackTrace() StackTrace { + f := make([]Frame, len(*s)) + for i := 0; i < len(f); i++ { + f[i] = Frame((*s)[i]) + } + return f +} + +func callers() *stack { + const depth = 32 + var pcs [depth]uintptr + n := runtime.Callers(3, pcs[:]) + var st stack = pcs[0:n] + return &st +} + +// funcname removes the path prefix component of a function's name reported by func.Name(). +func funcname(name string) string { + i := strings.LastIndex(name, "/") + name = name[i+1:] + i = strings.Index(name, ".") + return name[i+1:] +} + +func trimGOPATH(name, file string) string { + // Here we want to get the source file path relative to the compile time + // GOPATH. As of Go 1.6.x there is no direct way to know the compiled + // GOPATH at runtime, but we can infer the number of path segments in the + // GOPATH. We note that fn.Name() returns the function name qualified by + // the import path, which does not include the GOPATH. Thus we can trim + // segments from the beginning of the file path until the number of path + // separators remaining is one more than the number of path separators in + // the function name. For example, given: + // + // GOPATH /home/user + // file /home/user/src/pkg/sub/file.go + // fn.Name() pkg/sub.Type.Method + // + // We want to produce: + // + // pkg/sub/file.go + // + // From this we can easily see that fn.Name() has one less path separator + // than our desired output. We count separators from the end of the file + // path until it finds two more than in the function name and then move + // one character forward to preserve the initial path segment without a + // leading separator. + const sep = "/" + goal := strings.Count(name, sep) + 2 + i := len(file) + for n := 0; n < goal; n++ { + i = strings.LastIndex(file[:i], sep) + if i == -1 { + // not enough separators found, set i so that the slice expression + // below leaves file unmodified + i = -len(sep) + break + } + } + // get back to 0 or trim the leading separator + file = file[i+len(sep):] + return file +} From 1255973764f72575d71920b95fc0861d1677c6c5 Mon Sep 17 00:00:00 2001 From: winkyao Date: Tue, 28 Aug 2018 15:34:39 +0800 Subject: [PATCH 14/41] tables: make duplicate error in update clearer (#7495) --- executor/insert_test.go | 12 ++++++++++++ executor/write_test.go | 2 +- table/tables/tables.go | 10 ++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/executor/insert_test.go b/executor/insert_test.go index d0cf9d00608dc..2b66f80bf31cf 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -97,6 +97,18 @@ func (s *testSuite) TestInsertOnDuplicateKey(c *C) { tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 3`)) } +func (s *testSuite) TestUpdateDuplicateKey(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table c(i int,j int,k int,primary key(i,j,k));`) + tk.MustExec(`insert into c values(1,2,3);`) + tk.MustExec(`insert into c values(1,2,4);`) + _, err := tk.Exec(`update c set i=1,j=2,k=4 where i=1 and j=2 and k=3;`) + c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1-2-4' for key 'PRIMARY'") +} + func (s *testSuite) TestInsertWrongValueForField(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/write_test.go b/executor/write_test.go index c7476e9a8b3ae..902bd5fd2121b 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1000,7 +1000,7 @@ func (s *testSuite) TestUpdate(c *C) { _, err = tk.Exec("update ignore t set a = 1 where a = 2;") c.Assert(err, IsNil) r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 key already exist")) + r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 'I_uniq'")) tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) tk.MustExec("drop table if exists t") diff --git a/table/tables/tables.go b/table/tables/tables.go index 37bd31a5ce788..4cfc7a85303a1 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -748,6 +748,16 @@ func (t *tableCommon) removeRowIndex(sc *stmtctx.StatementContext, rm kv.Retriev // buildIndexForRow implements table.Table BuildIndexForRow interface. func (t *tableCommon) buildIndexForRow(ctx sessionctx.Context, rm kv.RetrieverMutator, h int64, vals []types.Datum, idx table.Index) error { if _, err := idx.Create(ctx, rm, vals, h); err != nil { + if kv.ErrKeyExists.Equal(err) { + // Make error message consistent with MySQL. + entryKey, err1 := t.genIndexKeyStr(vals) + if err1 != nil { + // if genIndexKeyStr failed, return the original error. + return errors.Trace(err) + } + + return kv.ErrKeyExists.FastGen("Duplicate entry '%s' for key '%s'", entryKey, idx.Meta().Name) + } return errors.Trace(err) } return nil From 2fb314537a190b4e9e5af4cb258b540bbc3e400a Mon Sep 17 00:00:00 2001 From: lysu Date: Tue, 28 Aug 2018 20:08:13 +0800 Subject: [PATCH 15/41] chunk: support capacity grow (#7473) --- util/chunk/chunk.go | 124 +++++++++++++++++++++++++++++++-------- util/chunk/chunk_test.go | 98 +++++++++++++++++++++++++++++-- util/chunk/codec.go | 5 +- 3 files changed, 196 insertions(+), 31 deletions(-) diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index ccdcdc7219d8c..a9211cb79a416 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -17,6 +17,7 @@ import ( "encoding/binary" "unsafe" + "github.com/cznic/mathutil" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" ) @@ -30,6 +31,8 @@ type Chunk struct { // numVirtualRows indicates the number of virtual rows, which have zero column. // It is used only when this Chunk doesn't hold any data, i.e. "len(columns)==0". numVirtualRows int + // capacity indicates the max number of rows this chunk can hold. + capacity int } // Capacity constants. @@ -39,15 +42,56 @@ const ( // NewChunkWithCapacity creates a new chunk with field types and capacity. func NewChunkWithCapacity(fields []*types.FieldType, cap int) *Chunk { + return New(fields, cap, cap) //FIXME: in following PR. +} + +// New creates a new chunk. +// cap: the limit for the max number of rows. +// maxChunkSize: the max limit for the number of rows. +func New(fields []*types.FieldType, cap, maxChunkSize int) *Chunk { chk := new(Chunk) chk.columns = make([]*column, 0, len(fields)) - chk.numVirtualRows = 0 + chk.capacity = mathutil.Min(cap, maxChunkSize) for _, f := range fields { - chk.addColumnByFieldType(f, cap) + elemLen := getFixedLen(f) + if elemLen == varElemLen { + chk.columns = append(chk.columns, newVarLenColumn(chk.capacity, nil)) + } else { + chk.columns = append(chk.columns, newFixedLenColumn(elemLen, chk.capacity)) + } } + chk.numVirtualRows = 0 return chk } +// Renew creates a new Chunk based on an existing Chunk. The newly created Chunk +// has the same data schema with the old Chunk. The capacity of the new Chunk +// might be doubled based on the capacity of the old Chunk and the maxChunkSize. +// chk: old chunk(often used in previous call). +// maxChunkSize: the limit for the max number of rows. +func Renew(chk *Chunk, maxChunkSize int) *Chunk { + newCap := reCalcCapacity(chk, maxChunkSize) + newChk := new(Chunk) + newChk.columns = renewColumns(chk.columns, newCap) + newChk.numVirtualRows = 0 + newChk.capacity = newCap + return newChk +} + +// renewColumns creates the columns of a Chunk. The capacity of the newly +// created columns is equal to cap. +func renewColumns(oldCol []*column, cap int) []*column { + columns := make([]*column, 0, len(oldCol)) + for _, col := range oldCol { + if col.isFixed() { + columns = append(columns, newFixedLenColumn(len(col.elemBuf), cap)) + } else { + columns = append(columns, newVarLenColumn(cap, col)) + } + } + return columns +} + // MemoryUsage returns the total memory usage of a Chunk in B. // We ignore the size of column.length and column.nullCount // since they have little effect of the total memory usage. @@ -59,32 +103,29 @@ func (c *Chunk) MemoryUsage() (sum int64) { return } -// addFixedLenColumn adds a fixed length column with elemLen and initial data capacity. -func (c *Chunk) addFixedLenColumn(elemLen, initCap int) { - c.columns = append(c.columns, &column{ +// newFixedLenColumn creates a fixed length column with elemLen and initial data capacity. +func newFixedLenColumn(elemLen, cap int) *column { + return &column{ elemBuf: make([]byte, elemLen), - data: make([]byte, 0, initCap*elemLen), - nullBitmap: make([]byte, 0, initCap>>3), - }) -} - -// addVarLenColumn adds a variable length column with initial data capacity. -func (c *Chunk) addVarLenColumn(initCap int) { - c.columns = append(c.columns, &column{ - offsets: make([]int32, 1, initCap+1), - data: make([]byte, 0, initCap*4), - nullBitmap: make([]byte, 0, initCap>>3), - }) + data: make([]byte, 0, cap*elemLen), + nullBitmap: make([]byte, 0, cap>>3), + } } -// addColumnByFieldType adds a column by field type. -func (c *Chunk) addColumnByFieldType(fieldTp *types.FieldType, initCap int) { - numFixedBytes := getFixedLen(fieldTp) - if numFixedBytes != -1 { - c.addFixedLenColumn(numFixedBytes, initCap) - return +// newVarLenColumn creates a variable length column with initial data capacity. +func newVarLenColumn(cap int, old *column) *column { + estimatedElemLen := 8 + // For varLenColumn (e.g. varchar), the accurate length of an element is unknown. + // Therefore, in the first executor.Next we use an experience value -- 8 (so it may make runtime.growslice) + // but in the following Next call we estimate the length as AVG x 1.125 elemLen of the previous call. + if old != nil && old.length != 0 { + estimatedElemLen = (len(old.data) + len(old.data)/8) / old.length + } + return &column{ + offsets: make([]int32, 1, cap+1), + data: make([]byte, 0, cap*estimatedElemLen), + nullBitmap: make([]byte, 0, cap>>3), } - c.addVarLenColumn(initCap) } // MakeRef makes column in "dstColIdx" reference to column in "srcColIdx". @@ -112,12 +153,43 @@ func (c *Chunk) SetNumVirtualRows(numVirtualRows int) { // Reset resets the chunk, so the memory it allocated can be reused. // Make sure all the data in the chunk is not used anymore before you reuse this chunk. func (c *Chunk) Reset() { - for _, c := range c.columns { - c.reset() + for _, col := range c.columns { + col.reset() } c.numVirtualRows = 0 } +// GrowAndReset resets the Chunk and doubles the capacity of the Chunk. +// The doubled capacity should not be larger than maxChunkSize. +// TODO: this method will be used in following PR. +func (c *Chunk) GrowAndReset(maxChunkSize int) { + if c.columns == nil { + return + } + newCap := reCalcCapacity(c, maxChunkSize) + if newCap <= c.capacity { + c.Reset() + return + } + c.capacity = newCap + c.columns = renewColumns(c.columns, newCap) + c.numVirtualRows = 0 +} + +// reCalcCapacity calculates the capacity for another Chunk based on the current +// Chunk. The new capacity is doubled only when the current Chunk is full. +func reCalcCapacity(c *Chunk, maxChunkSize int) int { + if c.NumRows() < c.capacity { + return c.capacity + } + return mathutil.Min(c.capacity*2, maxChunkSize) +} + +// Capacity returns the capacity of the Chunk. +func (c *Chunk) Capacity() int { + return c.capacity +} + // NumCols returns the number of columns in the chunk. func (c *Chunk) NumCols() int { return len(c.columns) diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index 7c0feea0e18fc..d6d7a1344aa48 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -14,8 +14,10 @@ package chunk import ( + "bytes" "fmt" "math" + "strconv" "testing" "time" "unsafe" @@ -248,9 +250,9 @@ func newChunk(elemLen ...int) *Chunk { chk := &Chunk{} for _, l := range elemLen { if l > 0 { - chk.addFixedLenColumn(l, 0) + chk.columns = append(chk.columns, newFixedLenColumn(l, 0)) } else { - chk.addVarLenColumn(0) + chk.columns = append(chk.columns, newVarLenColumn(0, nil)) } } return chk @@ -416,8 +418,8 @@ func (s *testChunkSuite) TestChunkMemoryUsage(c *check.C) { //cap(c.nullBitmap) + cap(c.offsets)*4 + cap(c.data) + cap(c.elemBuf) colUsage := make([]int, len(fieldTypes)) colUsage[0] = initCap>>3 + 0 + initCap*4 + 4 - colUsage[1] = initCap>>3 + (initCap+1)*4 + initCap*4 + 0 - colUsage[2] = initCap>>3 + (initCap+1)*4 + initCap*4 + 0 + colUsage[1] = initCap>>3 + (initCap+1)*4 + initCap*8 + 0 + colUsage[2] = initCap>>3 + (initCap+1)*4 + initCap*8 + 0 colUsage[3] = initCap>>3 + 0 + initCap*16 + 16 colUsage[4] = initCap>>3 + 0 + initCap*8 + 8 @@ -642,3 +644,91 @@ func BenchmarkChunkMemoryUsage(b *testing.B) { chk.MemoryUsage() } } + +type seqNumberGenerateExec struct { + seq int + genCountSize int +} + +func (x *seqNumberGenerateExec) Next(chk *Chunk, resize bool) { + if resize { + chk.GrowAndReset(1024) + } else { + chk.Reset() + } + for chk.NumRows() < chk.Capacity() { + x.seq++ + if x.seq > x.genCountSize { + break + } + chk.AppendInt64(0, 1) + } +} + +type benchChunkGrowCase struct { + tag string + reuse bool + newReset bool + cntPerCall int + initCap int + maxCap int +} + +func (b *benchChunkGrowCase) String() string { + var buff bytes.Buffer + if b.reuse { + buff.WriteString("renew,") + } else { + buff.WriteString("reset,") + } + buff.WriteString("cntPerCall:" + strconv.Itoa(b.cntPerCall) + ",") + buff.WriteString("cap from:" + strconv.Itoa(b.initCap) + " to " + strconv.Itoa(b.maxCap) + ",") + if b.tag != "" { + buff.WriteString("[" + b.tag + "]") + } + return buff.String() +} + +func BenchmarkChunkGrowSuit(b *testing.B) { + tests := []benchChunkGrowCase{ + {reuse: true, newReset: false, cntPerCall: 10000000, initCap: 1024, maxCap: 1024}, + {reuse: true, newReset: false, cntPerCall: 10000000, initCap: 32, maxCap: 32}, + {reuse: true, newReset: true, cntPerCall: 10000000, initCap: 32, maxCap: 1024, tag: "grow"}, + {reuse: false, newReset: false, cntPerCall: 10000000, initCap: 1024, maxCap: 1024}, + {reuse: false, newReset: false, cntPerCall: 10000000, initCap: 32, maxCap: 32}, + {reuse: false, newReset: true, cntPerCall: 10000000, initCap: 32, maxCap: 1024, tag: "grow"}, + {reuse: true, newReset: false, cntPerCall: 10, initCap: 1024, maxCap: 1024}, + {reuse: true, newReset: false, cntPerCall: 10, initCap: 32, maxCap: 32}, + {reuse: true, newReset: true, cntPerCall: 10, initCap: 32, maxCap: 1024, tag: "grow"}, + {reuse: false, newReset: false, cntPerCall: 10, initCap: 1024, maxCap: 1024}, + {reuse: false, newReset: false, cntPerCall: 10, initCap: 32, maxCap: 32}, + {reuse: false, newReset: true, cntPerCall: 10, initCap: 32, maxCap: 1024, tag: "grow"}, + } + for _, test := range tests { + b.Run(test.String(), benchmarkChunkGrow(test)) + } +} + +func benchmarkChunkGrow(t benchChunkGrowCase) func(b *testing.B) { + return func(b *testing.B) { + b.ReportAllocs() + chk := New([]*types.FieldType{{Tp: mysql.TypeLong}}, t.initCap, t.maxCap) + b.ResetTimer() + for i := 0; i < b.N; i++ { + e := &seqNumberGenerateExec{genCountSize: t.cntPerCall} + for { + e.Next(chk, t.newReset) + if chk.NumRows() == 0 { + break + } + if !t.reuse { + if t.newReset { + chk = Renew(chk, t.maxCap) + } else { + chk = New([]*types.FieldType{{Tp: mysql.TypeLong}}, t.initCap, t.maxCap) + } + } + } + } + } +} diff --git a/util/chunk/codec.go b/util/chunk/codec.go index a9a3df7587194..20401f95f762b 100644 --- a/util/chunk/codec.go +++ b/util/chunk/codec.go @@ -163,6 +163,9 @@ func (c *Codec) bytesToI32Slice(b []byte) (i32s []int32) { return i32s } +// varElemLen indicates this column is a variable length column. +const varElemLen = -1 + func getFixedLen(colType *types.FieldType) int { switch colType.Tp { case mysql.TypeFloat: @@ -175,7 +178,7 @@ func getFixedLen(colType *types.FieldType) int { case mysql.TypeNewDecimal: return types.MyDecimalStructSize default: - return -1 + return varElemLen } } From 3d468b970ec35e43ecd56233614cf756695993d9 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Wed, 29 Aug 2018 11:35:27 +0800 Subject: [PATCH 16/41] ast: remove unused code (#7521) --- ast/expressions.go | 29 +++++------------------------ ast/misc.go | 12 ------------ 2 files changed, 5 insertions(+), 36 deletions(-) diff --git a/ast/expressions.go b/ast/expressions.go index e927b1d9e2ebd..da3b94ab38568 100644 --- a/ast/expressions.go +++ b/ast/expressions.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/parser/opcode" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" ) @@ -319,30 +318,15 @@ func (n *CaseExpr) Accept(v Visitor) (Node, bool) { return v.Leave(n) } -// SubqueryExec represents a subquery executor interface. -// This interface is implemented in executor and used in plan/evaluator. -// It will execute the subselect and get the result. -type SubqueryExec interface { - // EvalRows executes the subquery and returns the multi rows with rowCount. - // rowCount < 0 means no limit. - // If the ColumnCount is 1, we will return a column result like {1, 2, 3}, - // otherwise, we will return a table result like {{1, 1}, {2, 2}}. - EvalRows(ctx sessionctx.Context, rowCount int) ([]types.Datum, error) - - // ColumnCount returns column count for the sub query. - ColumnCount() (int, error) -} - // SubqueryExpr represents a subquery. type SubqueryExpr struct { exprNode // Query is the query SelectNode. - Query ResultSetNode - SubqueryExec SubqueryExec - Evaluated bool - Correlated bool - MultiRows bool - Exists bool + Query ResultSetNode + Evaluated bool + Correlated bool + MultiRows bool + Exists bool } // Format the ExprNode into a Writer. @@ -357,9 +341,6 @@ func (n *SubqueryExpr) Accept(v Visitor) (Node, bool) { return v.Leave(newNode) } n = newNode.(*SubqueryExpr) - if n.SubqueryExec != nil { - return v.Leave(n) - } node, ok := n.Query.Accept(v) if !ok { return n, false diff --git a/ast/misc.go b/ast/misc.go index 49c45dceff105..2e89eeaec6945 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/auth" ) @@ -787,17 +786,6 @@ type Ident struct { Name model.CIStr } -// Full returns an Ident which set schema to the current schema if it is empty. -func (i Ident) Full(ctx sessionctx.Context) (full Ident) { - full.Name = i.Name - if i.Schema.O != "" { - full.Schema = i.Schema - } else { - full.Schema = model.NewCIStr(ctx.GetSessionVars().CurrentDB) - } - return -} - // String implements fmt.Stringer interface. func (i Ident) String() string { if i.Schema.O == "" { From c625c2723914be69327347650db080a0faafe3d3 Mon Sep 17 00:00:00 2001 From: Kenan Yao <6261973+eurekaka@users.noreply.github.com> Date: Wed, 29 Aug 2018 13:52:15 +0800 Subject: [PATCH 17/41] expression: check max_allowed_packet constraint for function insert (#7502) --- expression/builtin_string.go | 34 +++++++++++++++++++---- expression/builtin_string_test.go | 45 +++++++++++++++++++++++++++++++ 2 files changed, 74 insertions(+), 5 deletions(-) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index f29a950fbd9b1..b2c28dee5a772 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -3170,21 +3170,30 @@ func (c *insertFunctionClass) getFunction(ctx sessionctx.Context, args []Express bf.tp.Flen = mysql.MaxBlobWidth SetBinFlagOrBinStr(args[0].GetType(), bf.tp) SetBinFlagOrBinStr(args[3].GetType(), bf.tp) + + valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket) + maxAllowedPacket, err := strconv.ParseUint(valStr, 10, 64) + if err != nil { + return nil, errors.Trace(err) + } + if types.IsBinaryStr(args[0].GetType()) { - sig = &builtinInsertBinarySig{bf} + sig = &builtinInsertBinarySig{bf, maxAllowedPacket} } else { - sig = &builtinInsertSig{bf} + sig = &builtinInsertSig{bf, maxAllowedPacket} } return sig, nil } type builtinInsertBinarySig struct { baseBuiltinFunc + maxAllowedPacket uint64 } func (b *builtinInsertBinarySig) Clone() builtinFunc { newSig := &builtinInsertBinarySig{} newSig.cloneFrom(&b.baseBuiltinFunc) + newSig.maxAllowedPacket = b.maxAllowedPacket return newSig } @@ -3216,18 +3225,26 @@ func (b *builtinInsertBinarySig) evalString(row chunk.Row) (string, bool, error) } if length > strLength-pos+1 || length < 0 { - return str[0:pos-1] + newstr, false, nil + length = strLength - pos + 1 } + + if uint64(strLength-length+int64(len(newstr))) > b.maxAllowedPacket { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errWarnAllowedPacketOverflowed.GenByArgs("insert", b.maxAllowedPacket)) + return "", true, nil + } + return str[0:pos-1] + newstr + str[pos+length-1:], false, nil } type builtinInsertSig struct { baseBuiltinFunc + maxAllowedPacket uint64 } func (b *builtinInsertSig) Clone() builtinFunc { newSig := &builtinInsertSig{} newSig.cloneFrom(&b.baseBuiltinFunc) + newSig.maxAllowedPacket = b.maxAllowedPacket return newSig } @@ -3260,9 +3277,16 @@ func (b *builtinInsertSig) evalString(row chunk.Row) (string, bool, error) { } if length > runeLength-pos+1 || length < 0 { - return string(runes[0:pos-1]) + newstr, false, nil + length = runeLength - pos + 1 + } + + strHead := string(runes[0 : pos-1]) + strTail := string(runes[pos+length-1:]) + if uint64(len(strHead)+len(newstr)+len(strTail)) > b.maxAllowedPacket { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errWarnAllowedPacketOverflowed.GenByArgs("insert", b.maxAllowedPacket)) + return "", true, nil } - return string(runes[0:pos-1]) + newstr + string(runes[pos+length-1:]), false, nil + return strHead + newstr + strTail, false, nil } type instrFunctionClass struct { diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index c2202ca4661dd..dbadbca0ef833 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -1425,6 +1425,51 @@ func (s *testEvaluatorSuite) TestRpadSig(c *C) { c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) } +func (s *testEvaluatorSuite) TestInsertBinarySig(c *C) { + colTypes := []*types.FieldType{ + {Tp: mysql.TypeVarchar}, + {Tp: mysql.TypeLonglong}, + {Tp: mysql.TypeLonglong}, + {Tp: mysql.TypeVarchar}, + } + resultType := &types.FieldType{Tp: mysql.TypeVarchar, Flen: 3} + + args := []Expression{ + &Column{Index: 0, RetType: colTypes[0]}, + &Column{Index: 1, RetType: colTypes[1]}, + &Column{Index: 2, RetType: colTypes[2]}, + &Column{Index: 3, RetType: colTypes[3]}, + } + + base := baseBuiltinFunc{args: args, ctx: s.ctx, tp: resultType} + insert := &builtinInsertBinarySig{base, 3} + + input := chunk.NewChunkWithCapacity(colTypes, 2) + input.AppendString(0, "abc") + input.AppendString(0, "abc") + input.AppendInt64(1, 3) + input.AppendInt64(1, 3) + input.AppendInt64(2, -1) + input.AppendInt64(2, -1) + input.AppendString(3, "d") + input.AppendString(3, "de") + + res, isNull, err := insert.evalString(input.GetRow(0)) + c.Assert(res, Equals, "abd") + c.Assert(isNull, IsFalse) + c.Assert(err, IsNil) + + res, isNull, err = insert.evalString(input.GetRow(1)) + c.Assert(res, Equals, "") + c.Assert(isNull, IsTrue) + c.Assert(err, IsNil) + + warnings := s.ctx.GetSessionVars().StmtCtx.GetWarnings() + c.Assert(len(warnings), Equals, 1) + lastWarn := warnings[len(warnings)-1] + c.Assert(terror.ErrorEqual(errWarnAllowedPacketOverflowed, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) +} + func (s *testEvaluatorSuite) TestInstr(c *C) { defer testleak.AfterTest(c)() tbl := []struct { From 360567b1e880e8df24dbf27fa7d09e5a0484a549 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 29 Aug 2018 15:57:45 +0800 Subject: [PATCH 18/41] executor/join : use shallow copy for semi join. (#7433) --- executor/joiner.go | 47 +++++++++++++++-------------- util/chunk/chunk_test.go | 12 ++++++++ util/chunk/mutrow.go | 23 +++++++++++++++ util/chunk/mutrow_test.go | 62 +++++++++++++++++++++++++++++++++++++++ 4 files changed, 122 insertions(+), 22 deletions(-) diff --git a/executor/joiner.go b/executor/joiner.go index 562afa75dc6be..870669268d064 100644 --- a/executor/joiner.go +++ b/executor/joiner.go @@ -91,7 +91,6 @@ func newJoiner(ctx sessionctx.Context, joinType plan.JoinType, colTypes := make([]*types.FieldType, 0, len(lhsColTypes)+len(rhsColTypes)) colTypes = append(colTypes, lhsColTypes...) colTypes = append(colTypes, rhsColTypes...) - base.chk = chunk.NewChunkWithCapacity(colTypes, ctx.GetSessionVars().MaxChunkSize) base.selected = make([]bool, 0, chunk.InitialCapacity) if joinType == plan.LeftOuterJoin || joinType == plan.RightOuterJoin { innerColTypes := lhsColTypes @@ -102,18 +101,25 @@ func newJoiner(ctx sessionctx.Context, joinType plan.JoinType, } switch joinType { case plan.SemiJoin: + base.shallowRow = chunk.MutRowFromTypes(colTypes) return &semiJoiner{base} case plan.AntiSemiJoin: + base.shallowRow = chunk.MutRowFromTypes(colTypes) return &antiSemiJoiner{base} case plan.LeftOuterSemiJoin: + base.shallowRow = chunk.MutRowFromTypes(colTypes) return &leftOuterSemiJoiner{base} case plan.AntiLeftOuterSemiJoin: + base.shallowRow = chunk.MutRowFromTypes(colTypes) return &antiLeftOuterSemiJoiner{base} case plan.LeftOuterJoin: + base.chk = chunk.NewChunkWithCapacity(colTypes, ctx.GetSessionVars().MaxChunkSize) return &leftOuterJoiner{base} case plan.RightOuterJoin: + base.chk = chunk.NewChunkWithCapacity(colTypes, ctx.GetSessionVars().MaxChunkSize) return &rightOuterJoiner{base} case plan.InnerJoin: + base.chk = chunk.NewChunkWithCapacity(colTypes, ctx.GetSessionVars().MaxChunkSize) return &innerJoiner{base} } panic("unsupported join type in func newJoiner()") @@ -125,6 +131,7 @@ type baseJoiner struct { defaultInner chunk.Row outerIsRight bool chk *chunk.Chunk + shallowRow chunk.MutRow selected []bool maxChunkSize int } @@ -142,6 +149,15 @@ func (j *baseJoiner) makeJoinRowToChunk(chk *chunk.Chunk, lhs, rhs chunk.Row) { chk.AppendPartialRow(lhs.Len(), rhs) } +// makeShallowJoinRow shallow copies `inner` and `outer` into `shallowRow`. +func (j *baseJoiner) makeShallowJoinRow(isRightJoin bool, inner, outer chunk.Row) { + if !isRightJoin { + inner, outer = outer, inner + } + j.shallowRow.ShallowCopyPartialRow(0, inner) + j.shallowRow.ShallowCopyPartialRow(inner.Len(), outer) +} + func (j *baseJoiner) filter(input, output *chunk.Chunk) (matched bool, err error) { j.selected, err = expression.VectorizedFilter(j.ctx, j.conditions, chunk.NewIterator4Chunk(input), j.selected) if err != nil { @@ -173,14 +189,9 @@ func (j *semiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk *chu } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - j.chk.Reset() - if j.outerIsRight { - j.makeJoinRowToChunk(j.chk, inner, outer) - } else { - j.makeJoinRowToChunk(j.chk, outer, inner) - } + j.makeShallowJoinRow(j.outerIsRight, inner, outer) - matched, err = expression.EvalBool(j.ctx, j.conditions, j.chk.GetRow(0)) + matched, err = expression.EvalBool(j.ctx, j.conditions, j.shallowRow.ToRow()) if err != nil { return false, errors.Trace(err) } @@ -212,14 +223,9 @@ func (j *antiSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - j.chk.Reset() - if j.outerIsRight { - j.makeJoinRowToChunk(j.chk, inner, outer) - } else { - j.makeJoinRowToChunk(j.chk, outer, inner) - } + j.makeShallowJoinRow(j.outerIsRight, inner, outer) - matched, err = expression.EvalBool(j.ctx, j.conditions, j.chk.GetRow(0)) + matched, err = expression.EvalBool(j.ctx, j.conditions, j.shallowRow.ToRow()) if err != nil { return false, errors.Trace(err) } @@ -252,10 +258,9 @@ func (j *leftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - j.chk.Reset() - j.makeJoinRowToChunk(j.chk, outer, inner) + j.makeShallowJoinRow(false, inner, outer) - matched, err = expression.EvalBool(j.ctx, j.conditions, j.chk.GetRow(0)) + matched, err = expression.EvalBool(j.ctx, j.conditions, j.shallowRow.ToRow()) if err != nil { return false, errors.Trace(err) } @@ -295,10 +300,9 @@ func (j *antiLeftOuterSemiJoiner) tryToMatch(outer chunk.Row, inners chunk.Itera } for inner := inners.Current(); inner != inners.End(); inner = inners.Next() { - j.chk.Reset() - j.makeJoinRowToChunk(j.chk, outer, inner) - matched, err := expression.EvalBool(j.ctx, j.conditions, j.chk.GetRow(0)) + j.makeShallowJoinRow(false, inner, outer) + matched, err := expression.EvalBool(j.ctx, j.conditions, j.shallowRow.ToRow()) if err != nil { return false, errors.Trace(err) } @@ -330,7 +334,6 @@ func (j *leftOuterJoiner) tryToMatch(outer chunk.Row, inners chunk.Iterator, chk if inners.Len() == 0 { return false, nil } - j.chk.Reset() chkForJoin := j.chk if len(j.conditions) == 0 { diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index d6d7a1344aa48..f5158ca4b4da2 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -258,6 +258,18 @@ func newChunk(elemLen ...int) *Chunk { return chk } +func newChunkWithInitCap(cap int, elemLen ...int) *Chunk { + chk := &Chunk{} + for _, l := range elemLen { + if l > 0 { + chk.columns = append(chk.columns, newFixedLenColumn(l, cap)) + } else { + chk.columns = append(chk.columns, newVarLenColumn(cap, nil)) + } + } + return chk +} + var allTypes = []*types.FieldType{ types.NewFieldType(mysql.TypeTiny), types.NewFieldType(mysql.TypeShort), diff --git a/util/chunk/mutrow.go b/util/chunk/mutrow.go index 7cf1721296475..1eba29a2f9e5c 100644 --- a/util/chunk/mutrow.go +++ b/util/chunk/mutrow.go @@ -346,3 +346,26 @@ func setMutRowJSON(col *column, j json.BinaryJSON) { copy(col.data[1:], j.Value) col.offsets[1] = int32(dataLen) } + +// ShallowCopyPartialRow shallow copies the data of `row` to MutRow. +func (mr MutRow) ShallowCopyPartialRow(colIdx int, row Row) { + for i, srcCol := range row.c.columns { + dstCol := mr.c.columns[colIdx+i] + if !srcCol.isNull(row.idx) { + // MutRow only contains one row, so we can directly set the whole byte. + dstCol.nullBitmap[0] = 1 + } else { + dstCol.nullBitmap[0] = 0 + } + + if srcCol.isFixed() { + elemLen := len(srcCol.elemBuf) + offset := row.idx * elemLen + dstCol.data = srcCol.data[offset : offset+elemLen] + } else { + start, end := srcCol.offsets[row.idx], srcCol.offsets[row.idx+1] + dstCol.data = srcCol.data[start:end] + dstCol.offsets[1] = int32(len(dstCol.data)) + } + } +} diff --git a/util/chunk/mutrow_test.go b/util/chunk/mutrow_test.go index b4264e39a5855..bf2e925c7fb41 100644 --- a/util/chunk/mutrow_test.go +++ b/util/chunk/mutrow_test.go @@ -15,6 +15,7 @@ package chunk import ( "testing" + "time" "github.com/pingcap/check" "github.com/pingcap/tidb/mysql" @@ -134,3 +135,64 @@ func BenchmarkMutRowFromValues(b *testing.B) { MutRowFromValues(values) } } + +func (s *testChunkSuite) TestMutRowShallowCopyPartialRow(c *check.C) { + colTypes := make([]*types.FieldType, 0, 3) + colTypes = append(colTypes, &types.FieldType{Tp: mysql.TypeVarString}) + colTypes = append(colTypes, &types.FieldType{Tp: mysql.TypeLonglong}) + colTypes = append(colTypes, &types.FieldType{Tp: mysql.TypeTimestamp}) + + mutRow := MutRowFromTypes(colTypes) + row := MutRowFromValues("abc", 123, types.ZeroTimestamp).ToRow() + mutRow.ShallowCopyPartialRow(0, row) + c.Assert(row.GetString(0), check.Equals, mutRow.ToRow().GetString(0)) + c.Assert(row.GetInt64(1), check.Equals, mutRow.ToRow().GetInt64(1)) + c.Assert(row.GetTime(2), check.DeepEquals, mutRow.ToRow().GetTime(2)) + + row.c.Reset() + d := types.NewStringDatum("dfg") + row.c.AppendDatum(0, &d) + d = types.NewIntDatum(567) + row.c.AppendDatum(1, &d) + d = types.NewTimeDatum(types.Time{Time: types.FromGoTime(time.Now()), Fsp: 6, Type: mysql.TypeTimestamp}) + row.c.AppendDatum(2, &d) + + c.Assert(d.GetMysqlTime(), check.DeepEquals, mutRow.ToRow().GetTime(2)) + c.Assert(row.GetString(0), check.Equals, mutRow.ToRow().GetString(0)) + c.Assert(row.GetInt64(1), check.Equals, mutRow.ToRow().GetInt64(1)) + c.Assert(row.GetTime(2), check.DeepEquals, mutRow.ToRow().GetTime(2)) +} + +var rowsNum = 1024 + +func BenchmarkMutRowShallowCopyPartialRow(b *testing.B) { + b.ReportAllocs() + colTypes := make([]*types.FieldType, 0, 8) + colTypes = append(colTypes, &types.FieldType{Tp: mysql.TypeVarString}) + colTypes = append(colTypes, &types.FieldType{Tp: mysql.TypeVarString}) + colTypes = append(colTypes, &types.FieldType{Tp: mysql.TypeLonglong}) + colTypes = append(colTypes, &types.FieldType{Tp: mysql.TypeLonglong}) + colTypes = append(colTypes, &types.FieldType{Tp: mysql.TypeDatetime}) + + mutRow := MutRowFromTypes(colTypes) + row := MutRowFromValues("abc", "abcdefg", 123, 456, types.ZeroDatetime).ToRow() + b.ResetTimer() + for i := 0; i < b.N; i++ { + for j := 0; j < rowsNum; j++ { + mutRow.ShallowCopyPartialRow(0, row) + } + } +} + +func BenchmarkChunkAppendPartialRow(b *testing.B) { + b.ReportAllocs() + chk := newChunkWithInitCap(rowsNum, 0, 0, 8, 8, 16) + row := MutRowFromValues("abc", "abcdefg", 123, 456, types.ZeroDatetime).ToRow() + b.ResetTimer() + for i := 0; i < b.N; i++ { + chk.Reset() + for j := 0; j < rowsNum; j++ { + chk.AppendPartialRow(0, row) + } + } +} From eec68995689d5811f4d52846586e969652ed8ffc Mon Sep 17 00:00:00 2001 From: Zhexuan Yang Date: Wed, 29 Aug 2018 18:44:41 +0800 Subject: [PATCH 19/41] vendor: add basictracer-go (#7532) --- Gopkg.lock | 217 +- vendor/github.com/coreos/etcd/cmd/etcd | 1 + vendor/github.com/coreos/etcd/cmd/etcdctl | 1 + vendor/github.com/coreos/etcd/cmd/tools | 1 + vendor/github.com/gogo/protobuf/LICENSE | 36 + .../github.com/gogo/protobuf/proto/clone.go | 258 ++ .../gogo/protobuf/proto/custom_gogo.go | 39 + .../github.com/gogo/protobuf/proto/decode.go | 428 +++ .../github.com/gogo/protobuf/proto/discard.go | 350 +++ .../gogo/protobuf/proto/duration.go | 100 + .../gogo/protobuf/proto/duration_gogo.go | 49 + .../github.com/gogo/protobuf/proto/encode.go | 221 ++ .../gogo/protobuf/proto/encode_gogo.go | 33 + .../github.com/gogo/protobuf/proto/equal.go | 300 ++ .../gogo/protobuf/proto/extensions.go | 604 ++++ .../gogo/protobuf/proto/extensions_gogo.go | 368 +++ vendor/github.com/gogo/protobuf/proto/lib.go | 921 ++++++ .../gogo/protobuf/proto/lib_gogo.go | 50 + .../gogo/protobuf/proto/message_set.go | 314 ++ .../gogo/protobuf/proto/pointer_reflect.go | 357 +++ .../protobuf/proto/pointer_reflect_gogo.go | 59 + .../gogo/protobuf/proto/pointer_unsafe.go | 308 ++ .../protobuf/proto/pointer_unsafe_gogo.go | 56 + .../gogo/protobuf/proto/properties.go | 600 ++++ .../gogo/protobuf/proto/properties_gogo.go | 36 + .../gogo/protobuf/proto/skip_gogo.go | 119 + .../gogo/protobuf/proto/table_marshal.go | 2799 +++++++++++++++++ .../gogo/protobuf/proto/table_marshal_gogo.go | 388 +++ .../gogo/protobuf/proto/table_merge.go | 657 ++++ .../gogo/protobuf/proto/table_unmarshal.go | 2048 ++++++++++++ .../protobuf/proto/table_unmarshal_gogo.go | 385 +++ vendor/github.com/gogo/protobuf/proto/text.go | 928 ++++++ .../gogo/protobuf/proto/text_gogo.go | 57 + .../gogo/protobuf/proto/text_parser.go | 998 ++++++ .../gogo/protobuf/proto/timestamp.go | 113 + .../gogo/protobuf/proto/timestamp_gogo.go | 49 + .../opentracing/basictracer-go/LICENSE | 21 + .../opentracing/basictracer-go/context.go | 42 + .../opentracing/basictracer-go/debug.go | 78 + .../opentracing/basictracer-go/event.go | 62 + .../opentracing/basictracer-go/propagation.go | 61 + .../basictracer-go/propagation_ot.go | 180 ++ .../opentracing/basictracer-go/raw.go | 34 + .../opentracing/basictracer-go/recorder.go | 60 + .../opentracing/basictracer-go/span.go | 274 ++ .../opentracing/basictracer-go/tracer.go | 262 ++ .../opentracing/basictracer-go/util.go | 25 + .../basictracer-go/wire/carrier.go | 40 + .../opentracing/basictracer-go/wire/gen.go | 6 + .../basictracer-go/wire/wire.pb.go | 508 +++ 50 files changed, 15886 insertions(+), 15 deletions(-) create mode 120000 vendor/github.com/coreos/etcd/cmd/etcd create mode 120000 vendor/github.com/coreos/etcd/cmd/etcdctl create mode 120000 vendor/github.com/coreos/etcd/cmd/tools create mode 100644 vendor/github.com/gogo/protobuf/LICENSE create mode 100644 vendor/github.com/gogo/protobuf/proto/clone.go create mode 100644 vendor/github.com/gogo/protobuf/proto/custom_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/decode.go create mode 100644 vendor/github.com/gogo/protobuf/proto/discard.go create mode 100644 vendor/github.com/gogo/protobuf/proto/duration.go create mode 100644 vendor/github.com/gogo/protobuf/proto/duration_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/encode.go create mode 100644 vendor/github.com/gogo/protobuf/proto/encode_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/equal.go create mode 100644 vendor/github.com/gogo/protobuf/proto/extensions.go create mode 100644 vendor/github.com/gogo/protobuf/proto/extensions_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/lib.go create mode 100644 vendor/github.com/gogo/protobuf/proto/lib_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/message_set.go create mode 100644 vendor/github.com/gogo/protobuf/proto/pointer_reflect.go create mode 100644 vendor/github.com/gogo/protobuf/proto/pointer_reflect_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/pointer_unsafe.go create mode 100644 vendor/github.com/gogo/protobuf/proto/pointer_unsafe_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/properties.go create mode 100644 vendor/github.com/gogo/protobuf/proto/properties_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/skip_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/table_marshal.go create mode 100644 vendor/github.com/gogo/protobuf/proto/table_marshal_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/table_merge.go create mode 100644 vendor/github.com/gogo/protobuf/proto/table_unmarshal.go create mode 100644 vendor/github.com/gogo/protobuf/proto/table_unmarshal_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/text.go create mode 100644 vendor/github.com/gogo/protobuf/proto/text_gogo.go create mode 100644 vendor/github.com/gogo/protobuf/proto/text_parser.go create mode 100644 vendor/github.com/gogo/protobuf/proto/timestamp.go create mode 100644 vendor/github.com/gogo/protobuf/proto/timestamp_gogo.go create mode 100644 vendor/github.com/opentracing/basictracer-go/LICENSE create mode 100644 vendor/github.com/opentracing/basictracer-go/context.go create mode 100644 vendor/github.com/opentracing/basictracer-go/debug.go create mode 100644 vendor/github.com/opentracing/basictracer-go/event.go create mode 100644 vendor/github.com/opentracing/basictracer-go/propagation.go create mode 100644 vendor/github.com/opentracing/basictracer-go/propagation_ot.go create mode 100644 vendor/github.com/opentracing/basictracer-go/raw.go create mode 100644 vendor/github.com/opentracing/basictracer-go/recorder.go create mode 100644 vendor/github.com/opentracing/basictracer-go/span.go create mode 100644 vendor/github.com/opentracing/basictracer-go/tracer.go create mode 100644 vendor/github.com/opentracing/basictracer-go/util.go create mode 100644 vendor/github.com/opentracing/basictracer-go/wire/carrier.go create mode 100644 vendor/github.com/opentracing/basictracer-go/wire/gen.go create mode 100644 vendor/github.com/opentracing/basictracer-go/wire/wire.pb.go diff --git a/Gopkg.lock b/Gopkg.lock index 2a9b55205d3d3..960216eab8145 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -3,33 +3,44 @@ [[projects]] branch = "master" + digest = "1:be3ccd9f881604e4dd6d15cccfa126aa309232f0ba075ae5f92d3ef729a62758" name = "github.com/BurntSushi/toml" packages = ["."] + pruneopts = "NUT" revision = "a368813c5e648fee92e5f6c30e3944ff9d5e8895" [[projects]] + digest = "1:9752dad5e89cd779096bf2477a4ded16bea7ac62de453c8d6b4bf841d51a8512" name = "github.com/apache/thrift" packages = ["lib/go/thrift"] + pruneopts = "NUT" revision = "b2a4d4ae21c789b689dd162deb819665567f481c" version = "0.10.0" [[projects]] + digest = "1:75d40fa0c338f4c56056a3985e91fa371e8fcd0293e45b80afa7debecaf56012" name = "github.com/beorn7/perks" packages = ["quantile"] + pruneopts = "NUT" revision = "3ac7bf7a47d159a033b107610db8a1b6575507a4" [[projects]] branch = "master" + digest = "1:7b12da6e82292eb06e24dfe544c628115a3f4316c152f9dcb87d4f60cbf7cd7d" name = "github.com/blacktear23/go-proxyprotocol" packages = ["."] + pruneopts = "NUT" revision = "62e368e1c4700c34b4b6f77afd49b215211574c2" [[projects]] + digest = "1:60142a898f3808b3e6aa604e5f3296bdef921e625ef3223b6019c1a345b8765c" name = "github.com/codahale/hdrhistogram" packages = ["."] + pruneopts = "NUT" revision = "f8ad88b59a584afeee9d334eff879b104439117b" [[projects]] + digest = "1:7a5c43af23a0c21f4f1762d54af95586f9c04836257e66631557d8f8200ef0e1" name = "github.com/coreos/etcd" packages = [ "auth/authpb", @@ -38,55 +49,81 @@ "etcdserver/api/v3rpc/rpctypes", "etcdserver/etcdserverpb", "mvcc/mvccpb", - "pkg/types" + "pkg/types", ] + pruneopts = "NUT" revision = "eddf599c689ec85f4752060edff5a72e81e9106a" version = "v3.2.18" [[projects]] + digest = "1:883ab1d72c3c5851413baad190ebe3ceaf31bb8dece09a943ac07f9e722f811c" name = "github.com/cznic/golex" packages = ["lex"] + pruneopts = "NUT" revision = "da5a7153a51074477ecac5c45a7e5182a0c72448" [[projects]] + digest = "1:f80ed82cae006d02025cd63bd7cbe63a7e593de2714db785ea36d6323cc995eb" name = "github.com/cznic/mathutil" packages = ["."] + pruneopts = "NUT" revision = "78ad7f262603437f0ecfebc835d80094f89c8f54" [[projects]] branch = "master" + digest = "1:a7e4a0d213c4c29e79a0da8fe9d16bbcedce561763ff643bff252ea14260f61c" name = "github.com/cznic/parser" packages = ["yacc"] + pruneopts = "NUT" revision = "31edd927e5b19d1c4a260c41a397e7f81d6694d9" [[projects]] branch = "master" + digest = "1:809006f9378a46bcc70bc4330d14f43c1a7818ae9d93c09cab062e575d7e95a2" name = "github.com/cznic/sortutil" packages = ["."] + pruneopts = "NUT" revision = "4c7342852e65c2088c981288f2c5610d10b9f7f4" [[projects]] + digest = "1:5799bcff29ed827fa851b40b2f650bc777df4b67fd9b8a3041d9d89b9eb2fd86" name = "github.com/cznic/strutil" packages = ["."] + pruneopts = "NUT" revision = "1eb03e3cc9d345307a45ec82bd3016cde4bd4464" [[projects]] + digest = "1:71705901e4a356cde002ae6f0052fb661f59971c12262c12006b25cec40025b1" name = "github.com/cznic/y" packages = ["."] + pruneopts = "NUT" revision = "9fdf92d4aac058959f814606bb729ed50f5e4240" [[projects]] branch = "master" + digest = "1:f18dbc529543fe5fd5294f8385ea1f71681be964c43461f5f45335bb51ba83ae" name = "github.com/etcd-io/gofail" packages = ["runtime"] + pruneopts = "NUT" revision = "51ce9a71510a58bad5ae66ddd278ef28762a1550" [[projects]] + digest = "1:973dbcbbb1be662b61604319582383c315add70906a68d14894f31542ffc3a25" name = "github.com/go-sql-driver/mysql" packages = ["."] + pruneopts = "NUT" revision = "3955978caca48c1658a4bb7a9c6a0f084e326af3" [[projects]] + digest = "1:38e684375ef5b55e812332266d63f9fc5b6329ab303067c4cdda051db6d29ca4" + name = "github.com/gogo/protobuf" + packages = ["proto"] + pruneopts = "NUT" + revision = "636bf0302bc95575d69441b25a2603156ffdddf1" + version = "v1.1.1" + +[[projects]] + digest = "1:6aef947ba53156da1a66ee891d70d61835e0dcfc9f0d728ae1132db651e81c22" name = "github.com/golang/protobuf" packages = [ "jsonpb", @@ -96,95 +133,134 @@ "ptypes/any", "ptypes/duration", "ptypes/struct", - "ptypes/timestamp" + "ptypes/timestamp", ] + pruneopts = "NUT" revision = "b4deda0973fb4c70b50d226b1af49f3da59f5265" version = "v1.1.0" [[projects]] + digest = "1:c6dfb6c55c1989f1d89622b3c45b786127f76f47322c50e487585f823cb12543" name = "github.com/golang/snappy" packages = ["."] + pruneopts = "NUT" revision = "723cc1e459b8eea2dea4583200fd60757d40097a" [[projects]] branch = "master" + digest = "1:c0883bc20a7c1ff552ff53d414f3cdc28fe847d15fab48c4486aa772ab2fb131" name = "github.com/google/btree" packages = ["."] + pruneopts = "NUT" revision = "316fb6d3f031ae8f4d457c6c5186b9e3ded70435" [[projects]] + digest = "1:dbd86d229eacaa86a98b10f8fb3e3fc69a1913e0f4e010e7cc1f92bf12edca92" name = "github.com/gorilla/context" packages = ["."] + pruneopts = "NUT" revision = "1ea25387ff6f684839d82767c1733ff4d4d15d0a" version = "v1.1" [[projects]] + digest = "1:054b11c45900b575d23fa8e0fa3636a784e2c4d1d43d05e8e20fd592ebe0d5db" name = "github.com/gorilla/mux" packages = ["."] + pruneopts = "NUT" revision = "599cba5e7b6137d46ddf58fb1765f5d928e69604" [[projects]] + digest = "1:0aa5274053fdc232896f0835c712f4a39992d959f8e2363f189a7b0df36f136b" name = "github.com/grpc-ecosystem/go-grpc-middleware" packages = [ ".", "tags", "tracing/opentracing", - "util/metautils" + "util/metautils", ] + pruneopts = "NUT" revision = "82921fcf811d228d2fa202bc31238b356bf9f8d5" [[projects]] + digest = "1:96c558cff0532e2e9ffc0b6d7c8c7431c592d781b109343aa51e27f9fd9a6b82" name = "github.com/grpc-ecosystem/go-grpc-prometheus" packages = ["."] + pruneopts = "NUT" revision = "6b7015e65d366bf3f19b2b2a000a831940f0f7e0" version = "v1.1" [[projects]] + digest = "1:e8fd2854e80293d75ff2aab6df2ea0887bb604c2119f343dbd63e25dcf9f10ef" name = "github.com/juju/errors" packages = ["."] + pruneopts = "NUT" revision = "b2c7a7da5b2995941048f60146e67702a292e468" [[projects]] branch = "master" + digest = "1:5985ef4caf91ece5d54817c11ea25f182697534f8ae6521eadcd628c142ac4b6" name = "github.com/matttproud/golang_protobuf_extensions" packages = ["pbutil"] + pruneopts = "NUT" revision = "c12348ce28de40eed0136aa2b644d0ee0650e56c" [[projects]] branch = "master" + digest = "1:b95cb972bfd3eb737d334b1703a51a82d1d1d3e92f7f355798d94796c7120c73" name = "github.com/ngaut/pools" packages = ["."] + pruneopts = "NUT" revision = "b7bc8c42aac787667ba45adea78233f53f548443" [[projects]] branch = "master" + digest = "1:7d5f99346aa63d23681f4d92708469f32384d0c26722a4de5725bd0f22caedac" name = "github.com/ngaut/sync2" packages = ["."] + pruneopts = "NUT" revision = "7a24ed77b2efb460c1468b7dc917821c66e80e55" [[projects]] + digest = "1:cc405544fecfb5a8e0c409127ef67ce3b91d11143a00121e5b822e4f8eabe7d2" + name = "github.com/opentracing/basictracer-go" + packages = [ + ".", + "wire", + ] + pruneopts = "NUT" + revision = "1b32af207119a14b1b231d451df3ed04a72efebf" + version = "v1.0.0" + +[[projects]] + digest = "1:7da29c22bcc5c2ffb308324377dc00b5084650348c2799e573ed226d8cc9faf0" name = "github.com/opentracing/opentracing-go" packages = [ ".", "ext", - "log" + "log", ] + pruneopts = "NUT" revision = "1949ddbfd147afd4d964a9f00b24eb291e0e7c38" version = "v1.0.2" [[projects]] branch = "master" + digest = "1:3bf17a6e6eaa6ad24152148a631d18662f7212e21637c2699bff3369b7f00fa2" name = "github.com/petar/GoLLRB" packages = ["llrb"] + pruneopts = "NUT" revision = "53be0d36a84c2a886ca057d34b6aa4468df9ccb4" [[projects]] branch = "master" + digest = "1:dc85cc12f7f97c2b033157f9d40395b2f0458b7bd3083257f8f661ea6f48f89a" name = "github.com/pingcap/check" packages = ["."] + pruneopts = "NUT" revision = "1c287c953996ab3a0bf535dba9d53d809d3dc0b6" [[projects]] + digest = "1:8fd099a567b1e9b3a7e1f66d8547a0d2b1852427e86a0dae96fa59e9583e13e6" name = "github.com/pingcap/goleveldb" packages = [ "leveldb", @@ -198,12 +274,14 @@ "leveldb/opt", "leveldb/storage", "leveldb/table", - "leveldb/util" + "leveldb/util", ] + pruneopts = "NUT" revision = "8d44bfdf1030639ae7130922c95df12d6d4da3b6" [[projects]] branch = "master" + digest = "1:5d1ffbb5f63904390f71350a6f9d91e9b2e018dbbded1d8be71cd04df6635093" name = "github.com/pingcap/kvproto" packages = [ "pkg/coprocessor", @@ -213,18 +291,22 @@ "pkg/metapb", "pkg/pdpb", "pkg/raft_serverpb", - "pkg/tikvpb" + "pkg/tikvpb", ] + pruneopts = "NUT" revision = "279515615485b0f2d12f1421cc412fe2784e0190" [[projects]] branch = "master" + digest = "1:c9481427c87777d146d7e299c66cf08cf90a2ea2be035bb6581cfa1a78cdd39f" name = "github.com/pingcap/pd" packages = ["pd-client"] + pruneopts = "NUT" revision = "47a950e6649fa0151a6c5d0fc48a97c080012e3d" [[projects]] branch = "master" + digest = "1:5b504d252d8014e70c12a6492878d041a0534b2deace4b8a4d71609bec7d4049" name = "github.com/pingcap/tipb" packages = [ "go-binlog", @@ -239,61 +321,79 @@ "go-mysqlx/Session", "go-mysqlx/Sql", "go-tipb", - "sharedbytes" + "sharedbytes", ] + pruneopts = "NUT" revision = "5e46e0ec647edfee8fe001c5b51609460fb655bf" [[projects]] + digest = "1:5cf3f025cbee5951a4ee961de067c8a89fc95a5adabead774f82822efabab121" name = "github.com/pkg/errors" packages = ["."] + pruneopts = "NUT" revision = "645ef00459ed84a119197bfb8d8205042c6df63d" version = "v0.8.0" [[projects]] + digest = "1:1ef3c4d6e78616bd3d1b5b7d8899febb9aa1b83d3373fbbdc2804408c7977b57" name = "github.com/prometheus/client_golang" packages = [ "prometheus", - "prometheus/push" + "prometheus/push", ] + pruneopts = "NUT" revision = "c5b7fccd204277076155f10851dad72b76a49317" version = "v0.8.0" [[projects]] + digest = "1:9fe8945a11a9f588a9d306b4741cad634da9015a704271b9506810e2cc77fa17" name = "github.com/prometheus/client_model" packages = ["go"] + pruneopts = "NUT" revision = "fa8ad6fec33561be4280a8f0514318c79d7f6cb6" [[projects]] + digest = "1:c90717fa0864d47e19eaa855af60b202b537795f485052c7f48333c679dd7310" name = "github.com/prometheus/common" packages = [ "expfmt", "internal/bitbucket.org/ww/goautoneg", - "model" + "model", ] + pruneopts = "NUT" revision = "4402f4e5ea79ec15f3c574773b6a5198fbea215f" [[projects]] + digest = "1:dcfff2d5e99e01dcb856dd8afb0b509c1d05443f0b523cc5333b33a819829ed9" name = "github.com/prometheus/procfs" packages = ["."] + pruneopts = "NUT" revision = "abf152e5f3e97f2fafac028d2cc06c1feb87ffa5" [[projects]] + digest = "1:4c173651d2deb815a0420aeb1b3f7ca3c4aef2d980ba164a501a53f6abf368ef" name = "github.com/sirupsen/logrus" packages = ["."] + pruneopts = "NUT" revision = "3bcb09397d6d88e7676a9bc8433ca11ba5304837" [[projects]] + digest = "1:0af5ed795eeb9df3e3d32e2c0229b012e2c107945f75a0556733d643c94e55be" name = "github.com/spaolacci/murmur3" packages = ["."] + pruneopts = "NUT" revision = "0d12bf811670bf6a1a63828dfbd003eded177fce" version = "v1.0" [[projects]] + digest = "1:194c26fad062f6b1530720ee1afd6cd6f40d79274b2434caef2693b1da5d2ab2" name = "github.com/twinj/uuid" packages = ["."] + pruneopts = "NUT" revision = "70cac2bcd273ef6a371bb96cde363d28b68734c3" [[projects]] + digest = "1:0e28a98b9579858cb5de885935499fefebc8bc44a652cde08a6d035ee7435603" name = "github.com/uber/jaeger-client-go" packages = [ ".", @@ -305,18 +405,22 @@ "thrift-gen/jaeger", "thrift-gen/sampling", "thrift-gen/zipkincore", - "utils" + "utils", ] + pruneopts = "NUT" revision = "d021e646f5187d77b55592c3efee1a2810e895d7" version = "v2.8.0" [[projects]] + digest = "1:0da2810678a062e0567c3215911869b0423da0e497c56683ff8e87e7a6952597" name = "github.com/uber/jaeger-lib" packages = ["metrics"] + pruneopts = "NUT" revision = "3b2a9ad2a045881ab7a0f81d465be54c8292ee4f" version = "v1.1.0" [[projects]] + digest = "1:686219a880e6ec42870431372756a66c19c1396e9fe203b659179422d3c6bf96" name = "golang.org/x/net" packages = [ "context", @@ -325,17 +429,21 @@ "idna", "internal/timeseries", "lex/httplex", - "trace" + "trace", ] + pruneopts = "NUT" revision = "d1e1b351919c6738fdeb9893d5c998b161464f0c" [[projects]] branch = "master" + digest = "1:2f375ec82e53522eb4a1670f2f24f064f407ef2b32e01bb217f5daa4a4d226d6" name = "golang.org/x/sys" packages = ["unix"] + pruneopts = "NUT" revision = "7dfd1290c7917b7ba22824b9d24954ab3002fe24" [[projects]] + digest = "1:a005696b163ffe1842de27eeb3ccfece9d3c2e70d02b83ea1d8c0eeab597c9e0" name = "golang.org/x/text" packages = [ "encoding", @@ -357,19 +465,23 @@ "unicode/bidi", "unicode/cldr", "unicode/norm", - "unicode/rangetable" + "unicode/rangetable", ] + pruneopts = "NUT" revision = "4ee4af566555f5fbe026368b75596286a312663a" [[projects]] + digest = "1:0efcfe82e59b828eb6f4115bba88ff45c0898c38e823fbe7f450bdffed9e739b" name = "google.golang.org/genproto" packages = [ "googleapis/api/annotations", - "googleapis/rpc/status" + "googleapis/rpc/status", ] + pruneopts = "NUT" revision = "6b7d9516179cd47f4714cfeb0103ad1dede756c4" [[projects]] + digest = "1:a893d24a604ae6f45f2d6e00ae9e817476e110e2dd5455152b35bb720ca2f626" name = "google.golang.org/grpc" packages = [ ".", @@ -396,21 +508,96 @@ "stats", "status", "tap", - "transport" + "transport", ] + pruneopts = "NUT" revision = "41344da2231b913fa3d983840a57a6b1b7b631a1" source = "https://github.com/grpc/grpc-go.git" version = "v1.12.0" [[projects]] + digest = "1:7d95d61ff5828a65cf072a46f3c68d67daffdd11e97d758b0af2176cde717fcd" name = "gopkg.in/natefinch/lumberjack.v2" packages = ["."] + pruneopts = "NUT" revision = "dd45e6a67c53f673bb49ca8a001fd3a63ceb640e" version = "v2.0" [solve-meta] analyzer-name = "dep" analyzer-version = 1 - inputs-digest = "9c66d4c8409973a2286ef3018c5de331324953b79c1e323881a8caa5d2a21d2c" + input-imports = [ + "github.com/BurntSushi/toml", + "github.com/blacktear23/go-proxyprotocol", + "github.com/coreos/etcd/clientv3", + "github.com/coreos/etcd/clientv3/concurrency", + "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes", + "github.com/coreos/etcd/mvcc/mvccpb", + "github.com/cznic/mathutil", + "github.com/cznic/parser/yacc", + "github.com/cznic/sortutil", + "github.com/cznic/strutil", + "github.com/cznic/y", + "github.com/etcd-io/gofail/runtime", + "github.com/go-sql-driver/mysql", + "github.com/golang/protobuf/jsonpb", + "github.com/golang/protobuf/proto", + "github.com/google/btree", + "github.com/gorilla/mux", + "github.com/grpc-ecosystem/go-grpc-middleware", + "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing", + "github.com/grpc-ecosystem/go-grpc-prometheus", + "github.com/juju/errors", + "github.com/ngaut/pools", + "github.com/opentracing/basictracer-go", + "github.com/opentracing/opentracing-go", + "github.com/pingcap/check", + "github.com/pingcap/goleveldb/leveldb", + "github.com/pingcap/goleveldb/leveldb/comparer", + "github.com/pingcap/goleveldb/leveldb/iterator", + "github.com/pingcap/goleveldb/leveldb/memdb", + "github.com/pingcap/goleveldb/leveldb/opt", + "github.com/pingcap/goleveldb/leveldb/storage", + "github.com/pingcap/goleveldb/leveldb/util", + "github.com/pingcap/kvproto/pkg/coprocessor", + "github.com/pingcap/kvproto/pkg/errorpb", + "github.com/pingcap/kvproto/pkg/kvrpcpb", + "github.com/pingcap/kvproto/pkg/metapb", + "github.com/pingcap/kvproto/pkg/tikvpb", + "github.com/pingcap/pd/pd-client", + "github.com/pingcap/tipb/go-binlog", + "github.com/pingcap/tipb/go-mysqlx", + "github.com/pingcap/tipb/go-mysqlx/Connection", + "github.com/pingcap/tipb/go-mysqlx/Crud", + "github.com/pingcap/tipb/go-mysqlx/Datatypes", + "github.com/pingcap/tipb/go-mysqlx/Expect", + "github.com/pingcap/tipb/go-mysqlx/Expr", + "github.com/pingcap/tipb/go-mysqlx/Notice", + "github.com/pingcap/tipb/go-mysqlx/Resultset", + "github.com/pingcap/tipb/go-mysqlx/Session", + "github.com/pingcap/tipb/go-mysqlx/Sql", + "github.com/pingcap/tipb/go-tipb", + "github.com/prometheus/client_golang/prometheus", + "github.com/prometheus/client_golang/prometheus/push", + "github.com/sirupsen/logrus", + "github.com/spaolacci/murmur3", + "github.com/twinj/uuid", + "github.com/uber/jaeger-client-go/config", + "golang.org/x/net/context", + "golang.org/x/text/encoding", + "golang.org/x/text/encoding/charmap", + "golang.org/x/text/encoding/japanese", + "golang.org/x/text/encoding/korean", + "golang.org/x/text/encoding/simplifiedchinese", + "golang.org/x/text/encoding/traditionalchinese", + "golang.org/x/text/encoding/unicode", + "golang.org/x/text/transform", + "google.golang.org/grpc", + "google.golang.org/grpc/codes", + "google.golang.org/grpc/credentials", + "google.golang.org/grpc/keepalive", + "google.golang.org/grpc/metadata", + "gopkg.in/natefinch/lumberjack.v2", + ] solver-name = "gps-cdcl" solver-version = 1 diff --git a/vendor/github.com/coreos/etcd/cmd/etcd b/vendor/github.com/coreos/etcd/cmd/etcd new file mode 120000 index 0000000000000..b870225aa053e --- /dev/null +++ b/vendor/github.com/coreos/etcd/cmd/etcd @@ -0,0 +1 @@ +../ \ No newline at end of file diff --git a/vendor/github.com/coreos/etcd/cmd/etcdctl b/vendor/github.com/coreos/etcd/cmd/etcdctl new file mode 120000 index 0000000000000..05bb269d60967 --- /dev/null +++ b/vendor/github.com/coreos/etcd/cmd/etcdctl @@ -0,0 +1 @@ +../etcdctl \ No newline at end of file diff --git a/vendor/github.com/coreos/etcd/cmd/tools b/vendor/github.com/coreos/etcd/cmd/tools new file mode 120000 index 0000000000000..4887d6e0c92bc --- /dev/null +++ b/vendor/github.com/coreos/etcd/cmd/tools @@ -0,0 +1 @@ +../tools \ No newline at end of file diff --git a/vendor/github.com/gogo/protobuf/LICENSE b/vendor/github.com/gogo/protobuf/LICENSE new file mode 100644 index 0000000000000..7be0cc7b62cf0 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/LICENSE @@ -0,0 +1,36 @@ +Protocol Buffers for Go with Gadgets + +Copyright (c) 2013, The GoGo Authors. All rights reserved. +http://github.com/gogo/protobuf + +Go support for Protocol Buffers - Google's data interchange format + +Copyright 2010 The Go Authors. All rights reserved. +https://github.com/golang/protobuf + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/vendor/github.com/gogo/protobuf/proto/clone.go b/vendor/github.com/gogo/protobuf/proto/clone.go new file mode 100644 index 0000000000000..a26b046d94f1a --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/clone.go @@ -0,0 +1,258 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2011 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// Protocol buffer deep copy and merge. +// TODO: RawMessage. + +package proto + +import ( + "fmt" + "log" + "reflect" + "strings" +) + +// Clone returns a deep copy of a protocol buffer. +func Clone(src Message) Message { + in := reflect.ValueOf(src) + if in.IsNil() { + return src + } + out := reflect.New(in.Type().Elem()) + dst := out.Interface().(Message) + Merge(dst, src) + return dst +} + +// Merger is the interface representing objects that can merge messages of the same type. +type Merger interface { + // Merge merges src into this message. + // Required and optional fields that are set in src will be set to that value in dst. + // Elements of repeated fields will be appended. + // + // Merge may panic if called with a different argument type than the receiver. + Merge(src Message) +} + +// generatedMerger is the custom merge method that generated protos will have. +// We must add this method since a generate Merge method will conflict with +// many existing protos that have a Merge data field already defined. +type generatedMerger interface { + XXX_Merge(src Message) +} + +// Merge merges src into dst. +// Required and optional fields that are set in src will be set to that value in dst. +// Elements of repeated fields will be appended. +// Merge panics if src and dst are not the same type, or if dst is nil. +func Merge(dst, src Message) { + if m, ok := dst.(Merger); ok { + m.Merge(src) + return + } + + in := reflect.ValueOf(src) + out := reflect.ValueOf(dst) + if out.IsNil() { + panic("proto: nil destination") + } + if in.Type() != out.Type() { + panic(fmt.Sprintf("proto.Merge(%T, %T) type mismatch", dst, src)) + } + if in.IsNil() { + return // Merge from nil src is a noop + } + if m, ok := dst.(generatedMerger); ok { + m.XXX_Merge(src) + return + } + mergeStruct(out.Elem(), in.Elem()) +} + +func mergeStruct(out, in reflect.Value) { + sprop := GetProperties(in.Type()) + for i := 0; i < in.NumField(); i++ { + f := in.Type().Field(i) + if strings.HasPrefix(f.Name, "XXX_") { + continue + } + mergeAny(out.Field(i), in.Field(i), false, sprop.Prop[i]) + } + + if emIn, ok := in.Addr().Interface().(extensionsBytes); ok { + emOut := out.Addr().Interface().(extensionsBytes) + bIn := emIn.GetExtensions() + bOut := emOut.GetExtensions() + *bOut = append(*bOut, *bIn...) + } else if emIn, err := extendable(in.Addr().Interface()); err == nil { + emOut, _ := extendable(out.Addr().Interface()) + mIn, muIn := emIn.extensionsRead() + if mIn != nil { + mOut := emOut.extensionsWrite() + muIn.Lock() + mergeExtension(mOut, mIn) + muIn.Unlock() + } + } + + uf := in.FieldByName("XXX_unrecognized") + if !uf.IsValid() { + return + } + uin := uf.Bytes() + if len(uin) > 0 { + out.FieldByName("XXX_unrecognized").SetBytes(append([]byte(nil), uin...)) + } +} + +// mergeAny performs a merge between two values of the same type. +// viaPtr indicates whether the values were indirected through a pointer (implying proto2). +// prop is set if this is a struct field (it may be nil). +func mergeAny(out, in reflect.Value, viaPtr bool, prop *Properties) { + if in.Type() == protoMessageType { + if !in.IsNil() { + if out.IsNil() { + out.Set(reflect.ValueOf(Clone(in.Interface().(Message)))) + } else { + Merge(out.Interface().(Message), in.Interface().(Message)) + } + } + return + } + switch in.Kind() { + case reflect.Bool, reflect.Float32, reflect.Float64, reflect.Int32, reflect.Int64, + reflect.String, reflect.Uint32, reflect.Uint64: + if !viaPtr && isProto3Zero(in) { + return + } + out.Set(in) + case reflect.Interface: + // Probably a oneof field; copy non-nil values. + if in.IsNil() { + return + } + // Allocate destination if it is not set, or set to a different type. + // Otherwise we will merge as normal. + if out.IsNil() || out.Elem().Type() != in.Elem().Type() { + out.Set(reflect.New(in.Elem().Elem().Type())) // interface -> *T -> T -> new(T) + } + mergeAny(out.Elem(), in.Elem(), false, nil) + case reflect.Map: + if in.Len() == 0 { + return + } + if out.IsNil() { + out.Set(reflect.MakeMap(in.Type())) + } + // For maps with value types of *T or []byte we need to deep copy each value. + elemKind := in.Type().Elem().Kind() + for _, key := range in.MapKeys() { + var val reflect.Value + switch elemKind { + case reflect.Ptr: + val = reflect.New(in.Type().Elem().Elem()) + mergeAny(val, in.MapIndex(key), false, nil) + case reflect.Slice: + val = in.MapIndex(key) + val = reflect.ValueOf(append([]byte{}, val.Bytes()...)) + default: + val = in.MapIndex(key) + } + out.SetMapIndex(key, val) + } + case reflect.Ptr: + if in.IsNil() { + return + } + if out.IsNil() { + out.Set(reflect.New(in.Elem().Type())) + } + mergeAny(out.Elem(), in.Elem(), true, nil) + case reflect.Slice: + if in.IsNil() { + return + } + if in.Type().Elem().Kind() == reflect.Uint8 { + // []byte is a scalar bytes field, not a repeated field. + + // Edge case: if this is in a proto3 message, a zero length + // bytes field is considered the zero value, and should not + // be merged. + if prop != nil && prop.proto3 && in.Len() == 0 { + return + } + + // Make a deep copy. + // Append to []byte{} instead of []byte(nil) so that we never end up + // with a nil result. + out.SetBytes(append([]byte{}, in.Bytes()...)) + return + } + n := in.Len() + if out.IsNil() { + out.Set(reflect.MakeSlice(in.Type(), 0, n)) + } + switch in.Type().Elem().Kind() { + case reflect.Bool, reflect.Float32, reflect.Float64, reflect.Int32, reflect.Int64, + reflect.String, reflect.Uint32, reflect.Uint64: + out.Set(reflect.AppendSlice(out, in)) + default: + for i := 0; i < n; i++ { + x := reflect.Indirect(reflect.New(in.Type().Elem())) + mergeAny(x, in.Index(i), false, nil) + out.Set(reflect.Append(out, x)) + } + } + case reflect.Struct: + mergeStruct(out, in) + default: + // unknown type, so not a protocol buffer + log.Printf("proto: don't know how to copy %v", in) + } +} + +func mergeExtension(out, in map[int32]Extension) { + for extNum, eIn := range in { + eOut := Extension{desc: eIn.desc} + if eIn.value != nil { + v := reflect.New(reflect.TypeOf(eIn.value)).Elem() + mergeAny(v, reflect.ValueOf(eIn.value), false, nil) + eOut.value = v.Interface() + } + if eIn.enc != nil { + eOut.enc = make([]byte, len(eIn.enc)) + copy(eOut.enc, eIn.enc) + } + + out[extNum] = eOut + } +} diff --git a/vendor/github.com/gogo/protobuf/proto/custom_gogo.go b/vendor/github.com/gogo/protobuf/proto/custom_gogo.go new file mode 100644 index 0000000000000..24552483c6cea --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/custom_gogo.go @@ -0,0 +1,39 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2018, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import "reflect" + +type custom interface { + Marshal() ([]byte, error) + Unmarshal(data []byte) error + Size() int +} + +var customType = reflect.TypeOf((*custom)(nil)).Elem() diff --git a/vendor/github.com/gogo/protobuf/proto/decode.go b/vendor/github.com/gogo/protobuf/proto/decode.go new file mode 100644 index 0000000000000..d9aa3c42d666e --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/decode.go @@ -0,0 +1,428 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2010 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +/* + * Routines for decoding protocol buffer data to construct in-memory representations. + */ + +import ( + "errors" + "fmt" + "io" +) + +// errOverflow is returned when an integer is too large to be represented. +var errOverflow = errors.New("proto: integer overflow") + +// ErrInternalBadWireType is returned by generated code when an incorrect +// wire type is encountered. It does not get returned to user code. +var ErrInternalBadWireType = errors.New("proto: internal error: bad wiretype for oneof") + +// DecodeVarint reads a varint-encoded integer from the slice. +// It returns the integer and the number of bytes consumed, or +// zero if there is not enough. +// This is the format for the +// int32, int64, uint32, uint64, bool, and enum +// protocol buffer types. +func DecodeVarint(buf []byte) (x uint64, n int) { + for shift := uint(0); shift < 64; shift += 7 { + if n >= len(buf) { + return 0, 0 + } + b := uint64(buf[n]) + n++ + x |= (b & 0x7F) << shift + if (b & 0x80) == 0 { + return x, n + } + } + + // The number is too large to represent in a 64-bit value. + return 0, 0 +} + +func (p *Buffer) decodeVarintSlow() (x uint64, err error) { + i := p.index + l := len(p.buf) + + for shift := uint(0); shift < 64; shift += 7 { + if i >= l { + err = io.ErrUnexpectedEOF + return + } + b := p.buf[i] + i++ + x |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + p.index = i + return + } + } + + // The number is too large to represent in a 64-bit value. + err = errOverflow + return +} + +// DecodeVarint reads a varint-encoded integer from the Buffer. +// This is the format for the +// int32, int64, uint32, uint64, bool, and enum +// protocol buffer types. +func (p *Buffer) DecodeVarint() (x uint64, err error) { + i := p.index + buf := p.buf + + if i >= len(buf) { + return 0, io.ErrUnexpectedEOF + } else if buf[i] < 0x80 { + p.index++ + return uint64(buf[i]), nil + } else if len(buf)-i < 10 { + return p.decodeVarintSlow() + } + + var b uint64 + // we already checked the first byte + x = uint64(buf[i]) - 0x80 + i++ + + b = uint64(buf[i]) + i++ + x += b << 7 + if b&0x80 == 0 { + goto done + } + x -= 0x80 << 7 + + b = uint64(buf[i]) + i++ + x += b << 14 + if b&0x80 == 0 { + goto done + } + x -= 0x80 << 14 + + b = uint64(buf[i]) + i++ + x += b << 21 + if b&0x80 == 0 { + goto done + } + x -= 0x80 << 21 + + b = uint64(buf[i]) + i++ + x += b << 28 + if b&0x80 == 0 { + goto done + } + x -= 0x80 << 28 + + b = uint64(buf[i]) + i++ + x += b << 35 + if b&0x80 == 0 { + goto done + } + x -= 0x80 << 35 + + b = uint64(buf[i]) + i++ + x += b << 42 + if b&0x80 == 0 { + goto done + } + x -= 0x80 << 42 + + b = uint64(buf[i]) + i++ + x += b << 49 + if b&0x80 == 0 { + goto done + } + x -= 0x80 << 49 + + b = uint64(buf[i]) + i++ + x += b << 56 + if b&0x80 == 0 { + goto done + } + x -= 0x80 << 56 + + b = uint64(buf[i]) + i++ + x += b << 63 + if b&0x80 == 0 { + goto done + } + // x -= 0x80 << 63 // Always zero. + + return 0, errOverflow + +done: + p.index = i + return x, nil +} + +// DecodeFixed64 reads a 64-bit integer from the Buffer. +// This is the format for the +// fixed64, sfixed64, and double protocol buffer types. +func (p *Buffer) DecodeFixed64() (x uint64, err error) { + // x, err already 0 + i := p.index + 8 + if i < 0 || i > len(p.buf) { + err = io.ErrUnexpectedEOF + return + } + p.index = i + + x = uint64(p.buf[i-8]) + x |= uint64(p.buf[i-7]) << 8 + x |= uint64(p.buf[i-6]) << 16 + x |= uint64(p.buf[i-5]) << 24 + x |= uint64(p.buf[i-4]) << 32 + x |= uint64(p.buf[i-3]) << 40 + x |= uint64(p.buf[i-2]) << 48 + x |= uint64(p.buf[i-1]) << 56 + return +} + +// DecodeFixed32 reads a 32-bit integer from the Buffer. +// This is the format for the +// fixed32, sfixed32, and float protocol buffer types. +func (p *Buffer) DecodeFixed32() (x uint64, err error) { + // x, err already 0 + i := p.index + 4 + if i < 0 || i > len(p.buf) { + err = io.ErrUnexpectedEOF + return + } + p.index = i + + x = uint64(p.buf[i-4]) + x |= uint64(p.buf[i-3]) << 8 + x |= uint64(p.buf[i-2]) << 16 + x |= uint64(p.buf[i-1]) << 24 + return +} + +// DecodeZigzag64 reads a zigzag-encoded 64-bit integer +// from the Buffer. +// This is the format used for the sint64 protocol buffer type. +func (p *Buffer) DecodeZigzag64() (x uint64, err error) { + x, err = p.DecodeVarint() + if err != nil { + return + } + x = (x >> 1) ^ uint64((int64(x&1)<<63)>>63) + return +} + +// DecodeZigzag32 reads a zigzag-encoded 32-bit integer +// from the Buffer. +// This is the format used for the sint32 protocol buffer type. +func (p *Buffer) DecodeZigzag32() (x uint64, err error) { + x, err = p.DecodeVarint() + if err != nil { + return + } + x = uint64((uint32(x) >> 1) ^ uint32((int32(x&1)<<31)>>31)) + return +} + +// DecodeRawBytes reads a count-delimited byte buffer from the Buffer. +// This is the format used for the bytes protocol buffer +// type and for embedded messages. +func (p *Buffer) DecodeRawBytes(alloc bool) (buf []byte, err error) { + n, err := p.DecodeVarint() + if err != nil { + return nil, err + } + + nb := int(n) + if nb < 0 { + return nil, fmt.Errorf("proto: bad byte length %d", nb) + } + end := p.index + nb + if end < p.index || end > len(p.buf) { + return nil, io.ErrUnexpectedEOF + } + + if !alloc { + // todo: check if can get more uses of alloc=false + buf = p.buf[p.index:end] + p.index += nb + return + } + + buf = make([]byte, nb) + copy(buf, p.buf[p.index:]) + p.index += nb + return +} + +// DecodeStringBytes reads an encoded string from the Buffer. +// This is the format used for the proto2 string type. +func (p *Buffer) DecodeStringBytes() (s string, err error) { + buf, err := p.DecodeRawBytes(false) + if err != nil { + return + } + return string(buf), nil +} + +// Unmarshaler is the interface representing objects that can +// unmarshal themselves. The argument points to data that may be +// overwritten, so implementations should not keep references to the +// buffer. +// Unmarshal implementations should not clear the receiver. +// Any unmarshaled data should be merged into the receiver. +// Callers of Unmarshal that do not want to retain existing data +// should Reset the receiver before calling Unmarshal. +type Unmarshaler interface { + Unmarshal([]byte) error +} + +// newUnmarshaler is the interface representing objects that can +// unmarshal themselves. The semantics are identical to Unmarshaler. +// +// This exists to support protoc-gen-go generated messages. +// The proto package will stop type-asserting to this interface in the future. +// +// DO NOT DEPEND ON THIS. +type newUnmarshaler interface { + XXX_Unmarshal([]byte) error +} + +// Unmarshal parses the protocol buffer representation in buf and places the +// decoded result in pb. If the struct underlying pb does not match +// the data in buf, the results can be unpredictable. +// +// Unmarshal resets pb before starting to unmarshal, so any +// existing data in pb is always removed. Use UnmarshalMerge +// to preserve and append to existing data. +func Unmarshal(buf []byte, pb Message) error { + pb.Reset() + if u, ok := pb.(newUnmarshaler); ok { + return u.XXX_Unmarshal(buf) + } + if u, ok := pb.(Unmarshaler); ok { + return u.Unmarshal(buf) + } + return NewBuffer(buf).Unmarshal(pb) +} + +// UnmarshalMerge parses the protocol buffer representation in buf and +// writes the decoded result to pb. If the struct underlying pb does not match +// the data in buf, the results can be unpredictable. +// +// UnmarshalMerge merges into existing data in pb. +// Most code should use Unmarshal instead. +func UnmarshalMerge(buf []byte, pb Message) error { + if u, ok := pb.(newUnmarshaler); ok { + return u.XXX_Unmarshal(buf) + } + if u, ok := pb.(Unmarshaler); ok { + // NOTE: The history of proto have unfortunately been inconsistent + // whether Unmarshaler should or should not implicitly clear itself. + // Some implementations do, most do not. + // Thus, calling this here may or may not do what people want. + // + // See https://github.com/golang/protobuf/issues/424 + return u.Unmarshal(buf) + } + return NewBuffer(buf).Unmarshal(pb) +} + +// DecodeMessage reads a count-delimited message from the Buffer. +func (p *Buffer) DecodeMessage(pb Message) error { + enc, err := p.DecodeRawBytes(false) + if err != nil { + return err + } + return NewBuffer(enc).Unmarshal(pb) +} + +// DecodeGroup reads a tag-delimited group from the Buffer. +// StartGroup tag is already consumed. This function consumes +// EndGroup tag. +func (p *Buffer) DecodeGroup(pb Message) error { + b := p.buf[p.index:] + x, y := findEndGroup(b) + if x < 0 { + return io.ErrUnexpectedEOF + } + err := Unmarshal(b[:x], pb) + p.index += y + return err +} + +// Unmarshal parses the protocol buffer representation in the +// Buffer and places the decoded result in pb. If the struct +// underlying pb does not match the data in the buffer, the results can be +// unpredictable. +// +// Unlike proto.Unmarshal, this does not reset pb before starting to unmarshal. +func (p *Buffer) Unmarshal(pb Message) error { + // If the object can unmarshal itself, let it. + if u, ok := pb.(newUnmarshaler); ok { + err := u.XXX_Unmarshal(p.buf[p.index:]) + p.index = len(p.buf) + return err + } + if u, ok := pb.(Unmarshaler); ok { + // NOTE: The history of proto have unfortunately been inconsistent + // whether Unmarshaler should or should not implicitly clear itself. + // Some implementations do, most do not. + // Thus, calling this here may or may not do what people want. + // + // See https://github.com/golang/protobuf/issues/424 + err := u.Unmarshal(p.buf[p.index:]) + p.index = len(p.buf) + return err + } + + // Slow workaround for messages that aren't Unmarshalers. + // This includes some hand-coded .pb.go files and + // bootstrap protos. + // TODO: fix all of those and then add Unmarshal to + // the Message interface. Then: + // The cast above and code below can be deleted. + // The old unmarshaler can be deleted. + // Clients can call Unmarshal directly (can already do that, actually). + var info InternalMessageInfo + err := info.Unmarshal(pb, p.buf[p.index:]) + p.index = len(p.buf) + return err +} diff --git a/vendor/github.com/gogo/protobuf/proto/discard.go b/vendor/github.com/gogo/protobuf/proto/discard.go new file mode 100644 index 0000000000000..fe1bd7d904e24 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/discard.go @@ -0,0 +1,350 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2017 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "fmt" + "reflect" + "strings" + "sync" + "sync/atomic" +) + +type generatedDiscarder interface { + XXX_DiscardUnknown() +} + +// DiscardUnknown recursively discards all unknown fields from this message +// and all embedded messages. +// +// When unmarshaling a message with unrecognized fields, the tags and values +// of such fields are preserved in the Message. This allows a later call to +// marshal to be able to produce a message that continues to have those +// unrecognized fields. To avoid this, DiscardUnknown is used to +// explicitly clear the unknown fields after unmarshaling. +// +// For proto2 messages, the unknown fields of message extensions are only +// discarded from messages that have been accessed via GetExtension. +func DiscardUnknown(m Message) { + if m, ok := m.(generatedDiscarder); ok { + m.XXX_DiscardUnknown() + return + } + // TODO: Dynamically populate a InternalMessageInfo for legacy messages, + // but the master branch has no implementation for InternalMessageInfo, + // so it would be more work to replicate that approach. + discardLegacy(m) +} + +// DiscardUnknown recursively discards all unknown fields. +func (a *InternalMessageInfo) DiscardUnknown(m Message) { + di := atomicLoadDiscardInfo(&a.discard) + if di == nil { + di = getDiscardInfo(reflect.TypeOf(m).Elem()) + atomicStoreDiscardInfo(&a.discard, di) + } + di.discard(toPointer(&m)) +} + +type discardInfo struct { + typ reflect.Type + + initialized int32 // 0: only typ is valid, 1: everything is valid + lock sync.Mutex + + fields []discardFieldInfo + unrecognized field +} + +type discardFieldInfo struct { + field field // Offset of field, guaranteed to be valid + discard func(src pointer) +} + +var ( + discardInfoMap = map[reflect.Type]*discardInfo{} + discardInfoLock sync.Mutex +) + +func getDiscardInfo(t reflect.Type) *discardInfo { + discardInfoLock.Lock() + defer discardInfoLock.Unlock() + di := discardInfoMap[t] + if di == nil { + di = &discardInfo{typ: t} + discardInfoMap[t] = di + } + return di +} + +func (di *discardInfo) discard(src pointer) { + if src.isNil() { + return // Nothing to do. + } + + if atomic.LoadInt32(&di.initialized) == 0 { + di.computeDiscardInfo() + } + + for _, fi := range di.fields { + sfp := src.offset(fi.field) + fi.discard(sfp) + } + + // For proto2 messages, only discard unknown fields in message extensions + // that have been accessed via GetExtension. + if em, err := extendable(src.asPointerTo(di.typ).Interface()); err == nil { + // Ignore lock since DiscardUnknown is not concurrency safe. + emm, _ := em.extensionsRead() + for _, mx := range emm { + if m, ok := mx.value.(Message); ok { + DiscardUnknown(m) + } + } + } + + if di.unrecognized.IsValid() { + *src.offset(di.unrecognized).toBytes() = nil + } +} + +func (di *discardInfo) computeDiscardInfo() { + di.lock.Lock() + defer di.lock.Unlock() + if di.initialized != 0 { + return + } + t := di.typ + n := t.NumField() + + for i := 0; i < n; i++ { + f := t.Field(i) + if strings.HasPrefix(f.Name, "XXX_") { + continue + } + + dfi := discardFieldInfo{field: toField(&f)} + tf := f.Type + + // Unwrap tf to get its most basic type. + var isPointer, isSlice bool + if tf.Kind() == reflect.Slice && tf.Elem().Kind() != reflect.Uint8 { + isSlice = true + tf = tf.Elem() + } + if tf.Kind() == reflect.Ptr { + isPointer = true + tf = tf.Elem() + } + if isPointer && isSlice && tf.Kind() != reflect.Struct { + panic(fmt.Sprintf("%v.%s cannot be a slice of pointers to primitive types", t, f.Name)) + } + + switch tf.Kind() { + case reflect.Struct: + switch { + case !isPointer: + panic(fmt.Sprintf("%v.%s cannot be a direct struct value", t, f.Name)) + case isSlice: // E.g., []*pb.T + discardInfo := getDiscardInfo(tf) + dfi.discard = func(src pointer) { + sps := src.getPointerSlice() + for _, sp := range sps { + if !sp.isNil() { + discardInfo.discard(sp) + } + } + } + default: // E.g., *pb.T + discardInfo := getDiscardInfo(tf) + dfi.discard = func(src pointer) { + sp := src.getPointer() + if !sp.isNil() { + discardInfo.discard(sp) + } + } + } + case reflect.Map: + switch { + case isPointer || isSlice: + panic(fmt.Sprintf("%v.%s cannot be a pointer to a map or a slice of map values", t, f.Name)) + default: // E.g., map[K]V + if tf.Elem().Kind() == reflect.Ptr { // Proto struct (e.g., *T) + dfi.discard = func(src pointer) { + sm := src.asPointerTo(tf).Elem() + if sm.Len() == 0 { + return + } + for _, key := range sm.MapKeys() { + val := sm.MapIndex(key) + DiscardUnknown(val.Interface().(Message)) + } + } + } else { + dfi.discard = func(pointer) {} // Noop + } + } + case reflect.Interface: + // Must be oneof field. + switch { + case isPointer || isSlice: + panic(fmt.Sprintf("%v.%s cannot be a pointer to a interface or a slice of interface values", t, f.Name)) + default: // E.g., interface{} + // TODO: Make this faster? + dfi.discard = func(src pointer) { + su := src.asPointerTo(tf).Elem() + if !su.IsNil() { + sv := su.Elem().Elem().Field(0) + if sv.Kind() == reflect.Ptr && sv.IsNil() { + return + } + switch sv.Type().Kind() { + case reflect.Ptr: // Proto struct (e.g., *T) + DiscardUnknown(sv.Interface().(Message)) + } + } + } + } + default: + continue + } + di.fields = append(di.fields, dfi) + } + + di.unrecognized = invalidField + if f, ok := t.FieldByName("XXX_unrecognized"); ok { + if f.Type != reflect.TypeOf([]byte{}) { + panic("expected XXX_unrecognized to be of type []byte") + } + di.unrecognized = toField(&f) + } + + atomic.StoreInt32(&di.initialized, 1) +} + +func discardLegacy(m Message) { + v := reflect.ValueOf(m) + if v.Kind() != reflect.Ptr || v.IsNil() { + return + } + v = v.Elem() + if v.Kind() != reflect.Struct { + return + } + t := v.Type() + + for i := 0; i < v.NumField(); i++ { + f := t.Field(i) + if strings.HasPrefix(f.Name, "XXX_") { + continue + } + vf := v.Field(i) + tf := f.Type + + // Unwrap tf to get its most basic type. + var isPointer, isSlice bool + if tf.Kind() == reflect.Slice && tf.Elem().Kind() != reflect.Uint8 { + isSlice = true + tf = tf.Elem() + } + if tf.Kind() == reflect.Ptr { + isPointer = true + tf = tf.Elem() + } + if isPointer && isSlice && tf.Kind() != reflect.Struct { + panic(fmt.Sprintf("%T.%s cannot be a slice of pointers to primitive types", m, f.Name)) + } + + switch tf.Kind() { + case reflect.Struct: + switch { + case !isPointer: + panic(fmt.Sprintf("%T.%s cannot be a direct struct value", m, f.Name)) + case isSlice: // E.g., []*pb.T + for j := 0; j < vf.Len(); j++ { + discardLegacy(vf.Index(j).Interface().(Message)) + } + default: // E.g., *pb.T + discardLegacy(vf.Interface().(Message)) + } + case reflect.Map: + switch { + case isPointer || isSlice: + panic(fmt.Sprintf("%T.%s cannot be a pointer to a map or a slice of map values", m, f.Name)) + default: // E.g., map[K]V + tv := vf.Type().Elem() + if tv.Kind() == reflect.Ptr && tv.Implements(protoMessageType) { // Proto struct (e.g., *T) + for _, key := range vf.MapKeys() { + val := vf.MapIndex(key) + discardLegacy(val.Interface().(Message)) + } + } + } + case reflect.Interface: + // Must be oneof field. + switch { + case isPointer || isSlice: + panic(fmt.Sprintf("%T.%s cannot be a pointer to a interface or a slice of interface values", m, f.Name)) + default: // E.g., test_proto.isCommunique_Union interface + if !vf.IsNil() && f.Tag.Get("protobuf_oneof") != "" { + vf = vf.Elem() // E.g., *test_proto.Communique_Msg + if !vf.IsNil() { + vf = vf.Elem() // E.g., test_proto.Communique_Msg + vf = vf.Field(0) // E.g., Proto struct (e.g., *T) or primitive value + if vf.Kind() == reflect.Ptr { + discardLegacy(vf.Interface().(Message)) + } + } + } + } + } + } + + if vf := v.FieldByName("XXX_unrecognized"); vf.IsValid() { + if vf.Type() != reflect.TypeOf([]byte{}) { + panic("expected XXX_unrecognized to be of type []byte") + } + vf.Set(reflect.ValueOf([]byte(nil))) + } + + // For proto2 messages, only discard unknown fields in message extensions + // that have been accessed via GetExtension. + if em, err := extendable(m); err == nil { + // Ignore lock since discardLegacy is not concurrency safe. + emm, _ := em.extensionsRead() + for _, mx := range emm { + if m, ok := mx.value.(Message); ok { + discardLegacy(m) + } + } + } +} diff --git a/vendor/github.com/gogo/protobuf/proto/duration.go b/vendor/github.com/gogo/protobuf/proto/duration.go new file mode 100644 index 0000000000000..93464c91cffb5 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/duration.go @@ -0,0 +1,100 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2016 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +// This file implements conversions between google.protobuf.Duration +// and time.Duration. + +import ( + "errors" + "fmt" + "time" +) + +const ( + // Range of a Duration in seconds, as specified in + // google/protobuf/duration.proto. This is about 10,000 years in seconds. + maxSeconds = int64(10000 * 365.25 * 24 * 60 * 60) + minSeconds = -maxSeconds +) + +// validateDuration determines whether the Duration is valid according to the +// definition in google/protobuf/duration.proto. A valid Duration +// may still be too large to fit into a time.Duration (the range of Duration +// is about 10,000 years, and the range of time.Duration is about 290). +func validateDuration(d *duration) error { + if d == nil { + return errors.New("duration: nil Duration") + } + if d.Seconds < minSeconds || d.Seconds > maxSeconds { + return fmt.Errorf("duration: %#v: seconds out of range", d) + } + if d.Nanos <= -1e9 || d.Nanos >= 1e9 { + return fmt.Errorf("duration: %#v: nanos out of range", d) + } + // Seconds and Nanos must have the same sign, unless d.Nanos is zero. + if (d.Seconds < 0 && d.Nanos > 0) || (d.Seconds > 0 && d.Nanos < 0) { + return fmt.Errorf("duration: %#v: seconds and nanos have different signs", d) + } + return nil +} + +// DurationFromProto converts a Duration to a time.Duration. DurationFromProto +// returns an error if the Duration is invalid or is too large to be +// represented in a time.Duration. +func durationFromProto(p *duration) (time.Duration, error) { + if err := validateDuration(p); err != nil { + return 0, err + } + d := time.Duration(p.Seconds) * time.Second + if int64(d/time.Second) != p.Seconds { + return 0, fmt.Errorf("duration: %#v is out of range for time.Duration", p) + } + if p.Nanos != 0 { + d += time.Duration(p.Nanos) + if (d < 0) != (p.Nanos < 0) { + return 0, fmt.Errorf("duration: %#v is out of range for time.Duration", p) + } + } + return d, nil +} + +// DurationProto converts a time.Duration to a Duration. +func durationProto(d time.Duration) *duration { + nanos := d.Nanoseconds() + secs := nanos / 1e9 + nanos -= secs * 1e9 + return &duration{ + Seconds: secs, + Nanos: int32(nanos), + } +} diff --git a/vendor/github.com/gogo/protobuf/proto/duration_gogo.go b/vendor/github.com/gogo/protobuf/proto/duration_gogo.go new file mode 100644 index 0000000000000..e748e1730e1cc --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/duration_gogo.go @@ -0,0 +1,49 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2016, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "reflect" + "time" +) + +var durationType = reflect.TypeOf((*time.Duration)(nil)).Elem() + +type duration struct { + Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"` + Nanos int32 `protobuf:"varint,2,opt,name=nanos,proto3" json:"nanos,omitempty"` +} + +func (m *duration) Reset() { *m = duration{} } +func (*duration) ProtoMessage() {} +func (*duration) String() string { return "duration" } + +func init() { + RegisterType((*duration)(nil), "gogo.protobuf.proto.duration") +} diff --git a/vendor/github.com/gogo/protobuf/proto/encode.go b/vendor/github.com/gogo/protobuf/proto/encode.go new file mode 100644 index 0000000000000..c27d35f866bb0 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/encode.go @@ -0,0 +1,221 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2010 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +/* + * Routines for encoding data into the wire format for protocol buffers. + */ + +import ( + "errors" + "fmt" + "reflect" +) + +// RequiredNotSetError is the error returned if Marshal is called with +// a protocol buffer struct whose required fields have not +// all been initialized. It is also the error returned if Unmarshal is +// called with an encoded protocol buffer that does not include all the +// required fields. +// +// When printed, RequiredNotSetError reports the first unset required field in a +// message. If the field cannot be precisely determined, it is reported as +// "{Unknown}". +type RequiredNotSetError struct { + field string +} + +func (e *RequiredNotSetError) Error() string { + return fmt.Sprintf("proto: required field %q not set", e.field) +} + +var ( + // errRepeatedHasNil is the error returned if Marshal is called with + // a struct with a repeated field containing a nil element. + errRepeatedHasNil = errors.New("proto: repeated field has nil element") + + // errOneofHasNil is the error returned if Marshal is called with + // a struct with a oneof field containing a nil element. + errOneofHasNil = errors.New("proto: oneof field has nil value") + + // ErrNil is the error returned if Marshal is called with nil. + ErrNil = errors.New("proto: Marshal called with nil") + + // ErrTooLarge is the error returned if Marshal is called with a + // message that encodes to >2GB. + ErrTooLarge = errors.New("proto: message encodes to over 2 GB") +) + +// The fundamental encoders that put bytes on the wire. +// Those that take integer types all accept uint64 and are +// therefore of type valueEncoder. + +const maxVarintBytes = 10 // maximum length of a varint + +// EncodeVarint returns the varint encoding of x. +// This is the format for the +// int32, int64, uint32, uint64, bool, and enum +// protocol buffer types. +// Not used by the package itself, but helpful to clients +// wishing to use the same encoding. +func EncodeVarint(x uint64) []byte { + var buf [maxVarintBytes]byte + var n int + for n = 0; x > 127; n++ { + buf[n] = 0x80 | uint8(x&0x7F) + x >>= 7 + } + buf[n] = uint8(x) + n++ + return buf[0:n] +} + +// EncodeVarint writes a varint-encoded integer to the Buffer. +// This is the format for the +// int32, int64, uint32, uint64, bool, and enum +// protocol buffer types. +func (p *Buffer) EncodeVarint(x uint64) error { + for x >= 1<<7 { + p.buf = append(p.buf, uint8(x&0x7f|0x80)) + x >>= 7 + } + p.buf = append(p.buf, uint8(x)) + return nil +} + +// SizeVarint returns the varint encoding size of an integer. +func SizeVarint(x uint64) int { + switch { + case x < 1<<7: + return 1 + case x < 1<<14: + return 2 + case x < 1<<21: + return 3 + case x < 1<<28: + return 4 + case x < 1<<35: + return 5 + case x < 1<<42: + return 6 + case x < 1<<49: + return 7 + case x < 1<<56: + return 8 + case x < 1<<63: + return 9 + } + return 10 +} + +// EncodeFixed64 writes a 64-bit integer to the Buffer. +// This is the format for the +// fixed64, sfixed64, and double protocol buffer types. +func (p *Buffer) EncodeFixed64(x uint64) error { + p.buf = append(p.buf, + uint8(x), + uint8(x>>8), + uint8(x>>16), + uint8(x>>24), + uint8(x>>32), + uint8(x>>40), + uint8(x>>48), + uint8(x>>56)) + return nil +} + +// EncodeFixed32 writes a 32-bit integer to the Buffer. +// This is the format for the +// fixed32, sfixed32, and float protocol buffer types. +func (p *Buffer) EncodeFixed32(x uint64) error { + p.buf = append(p.buf, + uint8(x), + uint8(x>>8), + uint8(x>>16), + uint8(x>>24)) + return nil +} + +// EncodeZigzag64 writes a zigzag-encoded 64-bit integer +// to the Buffer. +// This is the format used for the sint64 protocol buffer type. +func (p *Buffer) EncodeZigzag64(x uint64) error { + // use signed number to get arithmetic right shift. + return p.EncodeVarint(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} + +// EncodeZigzag32 writes a zigzag-encoded 32-bit integer +// to the Buffer. +// This is the format used for the sint32 protocol buffer type. +func (p *Buffer) EncodeZigzag32(x uint64) error { + // use signed number to get arithmetic right shift. + return p.EncodeVarint(uint64((uint32(x) << 1) ^ uint32((int32(x) >> 31)))) +} + +// EncodeRawBytes writes a count-delimited byte buffer to the Buffer. +// This is the format used for the bytes protocol buffer +// type and for embedded messages. +func (p *Buffer) EncodeRawBytes(b []byte) error { + p.EncodeVarint(uint64(len(b))) + p.buf = append(p.buf, b...) + return nil +} + +// EncodeStringBytes writes an encoded string to the Buffer. +// This is the format used for the proto2 string type. +func (p *Buffer) EncodeStringBytes(s string) error { + p.EncodeVarint(uint64(len(s))) + p.buf = append(p.buf, s...) + return nil +} + +// Marshaler is the interface representing objects that can marshal themselves. +type Marshaler interface { + Marshal() ([]byte, error) +} + +// EncodeMessage writes the protocol buffer to the Buffer, +// prefixed by a varint-encoded length. +func (p *Buffer) EncodeMessage(pb Message) error { + siz := Size(pb) + p.EncodeVarint(uint64(siz)) + return p.Marshal(pb) +} + +// All protocol buffer fields are nillable, but be careful. +func isNil(v reflect.Value) bool { + switch v.Kind() { + case reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice: + return v.IsNil() + } + return false +} diff --git a/vendor/github.com/gogo/protobuf/proto/encode_gogo.go b/vendor/github.com/gogo/protobuf/proto/encode_gogo.go new file mode 100644 index 0000000000000..0f5fb173e9fd0 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/encode_gogo.go @@ -0,0 +1,33 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +func NewRequiredNotSetError(field string) *RequiredNotSetError { + return &RequiredNotSetError{field} +} diff --git a/vendor/github.com/gogo/protobuf/proto/equal.go b/vendor/github.com/gogo/protobuf/proto/equal.go new file mode 100644 index 0000000000000..d4db5a1c14577 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/equal.go @@ -0,0 +1,300 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2011 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// Protocol buffer comparison. + +package proto + +import ( + "bytes" + "log" + "reflect" + "strings" +) + +/* +Equal returns true iff protocol buffers a and b are equal. +The arguments must both be pointers to protocol buffer structs. + +Equality is defined in this way: + - Two messages are equal iff they are the same type, + corresponding fields are equal, unknown field sets + are equal, and extensions sets are equal. + - Two set scalar fields are equal iff their values are equal. + If the fields are of a floating-point type, remember that + NaN != x for all x, including NaN. If the message is defined + in a proto3 .proto file, fields are not "set"; specifically, + zero length proto3 "bytes" fields are equal (nil == {}). + - Two repeated fields are equal iff their lengths are the same, + and their corresponding elements are equal. Note a "bytes" field, + although represented by []byte, is not a repeated field and the + rule for the scalar fields described above applies. + - Two unset fields are equal. + - Two unknown field sets are equal if their current + encoded state is equal. + - Two extension sets are equal iff they have corresponding + elements that are pairwise equal. + - Two map fields are equal iff their lengths are the same, + and they contain the same set of elements. Zero-length map + fields are equal. + - Every other combination of things are not equal. + +The return value is undefined if a and b are not protocol buffers. +*/ +func Equal(a, b Message) bool { + if a == nil || b == nil { + return a == b + } + v1, v2 := reflect.ValueOf(a), reflect.ValueOf(b) + if v1.Type() != v2.Type() { + return false + } + if v1.Kind() == reflect.Ptr { + if v1.IsNil() { + return v2.IsNil() + } + if v2.IsNil() { + return false + } + v1, v2 = v1.Elem(), v2.Elem() + } + if v1.Kind() != reflect.Struct { + return false + } + return equalStruct(v1, v2) +} + +// v1 and v2 are known to have the same type. +func equalStruct(v1, v2 reflect.Value) bool { + sprop := GetProperties(v1.Type()) + for i := 0; i < v1.NumField(); i++ { + f := v1.Type().Field(i) + if strings.HasPrefix(f.Name, "XXX_") { + continue + } + f1, f2 := v1.Field(i), v2.Field(i) + if f.Type.Kind() == reflect.Ptr { + if n1, n2 := f1.IsNil(), f2.IsNil(); n1 && n2 { + // both unset + continue + } else if n1 != n2 { + // set/unset mismatch + return false + } + f1, f2 = f1.Elem(), f2.Elem() + } + if !equalAny(f1, f2, sprop.Prop[i]) { + return false + } + } + + if em1 := v1.FieldByName("XXX_InternalExtensions"); em1.IsValid() { + em2 := v2.FieldByName("XXX_InternalExtensions") + if !equalExtensions(v1.Type(), em1.Interface().(XXX_InternalExtensions), em2.Interface().(XXX_InternalExtensions)) { + return false + } + } + + if em1 := v1.FieldByName("XXX_extensions"); em1.IsValid() { + em2 := v2.FieldByName("XXX_extensions") + if !equalExtMap(v1.Type(), em1.Interface().(map[int32]Extension), em2.Interface().(map[int32]Extension)) { + return false + } + } + + uf := v1.FieldByName("XXX_unrecognized") + if !uf.IsValid() { + return true + } + + u1 := uf.Bytes() + u2 := v2.FieldByName("XXX_unrecognized").Bytes() + return bytes.Equal(u1, u2) +} + +// v1 and v2 are known to have the same type. +// prop may be nil. +func equalAny(v1, v2 reflect.Value, prop *Properties) bool { + if v1.Type() == protoMessageType { + m1, _ := v1.Interface().(Message) + m2, _ := v2.Interface().(Message) + return Equal(m1, m2) + } + switch v1.Kind() { + case reflect.Bool: + return v1.Bool() == v2.Bool() + case reflect.Float32, reflect.Float64: + return v1.Float() == v2.Float() + case reflect.Int32, reflect.Int64: + return v1.Int() == v2.Int() + case reflect.Interface: + // Probably a oneof field; compare the inner values. + n1, n2 := v1.IsNil(), v2.IsNil() + if n1 || n2 { + return n1 == n2 + } + e1, e2 := v1.Elem(), v2.Elem() + if e1.Type() != e2.Type() { + return false + } + return equalAny(e1, e2, nil) + case reflect.Map: + if v1.Len() != v2.Len() { + return false + } + for _, key := range v1.MapKeys() { + val2 := v2.MapIndex(key) + if !val2.IsValid() { + // This key was not found in the second map. + return false + } + if !equalAny(v1.MapIndex(key), val2, nil) { + return false + } + } + return true + case reflect.Ptr: + // Maps may have nil values in them, so check for nil. + if v1.IsNil() && v2.IsNil() { + return true + } + if v1.IsNil() != v2.IsNil() { + return false + } + return equalAny(v1.Elem(), v2.Elem(), prop) + case reflect.Slice: + if v1.Type().Elem().Kind() == reflect.Uint8 { + // short circuit: []byte + + // Edge case: if this is in a proto3 message, a zero length + // bytes field is considered the zero value. + if prop != nil && prop.proto3 && v1.Len() == 0 && v2.Len() == 0 { + return true + } + if v1.IsNil() != v2.IsNil() { + return false + } + return bytes.Equal(v1.Interface().([]byte), v2.Interface().([]byte)) + } + + if v1.Len() != v2.Len() { + return false + } + for i := 0; i < v1.Len(); i++ { + if !equalAny(v1.Index(i), v2.Index(i), prop) { + return false + } + } + return true + case reflect.String: + return v1.Interface().(string) == v2.Interface().(string) + case reflect.Struct: + return equalStruct(v1, v2) + case reflect.Uint32, reflect.Uint64: + return v1.Uint() == v2.Uint() + } + + // unknown type, so not a protocol buffer + log.Printf("proto: don't know how to compare %v", v1) + return false +} + +// base is the struct type that the extensions are based on. +// x1 and x2 are InternalExtensions. +func equalExtensions(base reflect.Type, x1, x2 XXX_InternalExtensions) bool { + em1, _ := x1.extensionsRead() + em2, _ := x2.extensionsRead() + return equalExtMap(base, em1, em2) +} + +func equalExtMap(base reflect.Type, em1, em2 map[int32]Extension) bool { + if len(em1) != len(em2) { + return false + } + + for extNum, e1 := range em1 { + e2, ok := em2[extNum] + if !ok { + return false + } + + m1, m2 := e1.value, e2.value + + if m1 == nil && m2 == nil { + // Both have only encoded form. + if bytes.Equal(e1.enc, e2.enc) { + continue + } + // The bytes are different, but the extensions might still be + // equal. We need to decode them to compare. + } + + if m1 != nil && m2 != nil { + // Both are unencoded. + if !equalAny(reflect.ValueOf(m1), reflect.ValueOf(m2), nil) { + return false + } + continue + } + + // At least one is encoded. To do a semantically correct comparison + // we need to unmarshal them first. + var desc *ExtensionDesc + if m := extensionMaps[base]; m != nil { + desc = m[extNum] + } + if desc == nil { + // If both have only encoded form and the bytes are the same, + // it is handled above. We get here when the bytes are different. + // We don't know how to decode it, so just compare them as byte + // slices. + log.Printf("proto: don't know how to compare extension %d of %v", extNum, base) + return false + } + var err error + if m1 == nil { + m1, err = decodeExtension(e1.enc, desc) + } + if m2 == nil && err == nil { + m2, err = decodeExtension(e2.enc, desc) + } + if err != nil { + // The encoded form is invalid. + log.Printf("proto: badly encoded extension %d of %v: %v", extNum, base, err) + return false + } + if !equalAny(reflect.ValueOf(m1), reflect.ValueOf(m2), nil) { + return false + } + } + + return true +} diff --git a/vendor/github.com/gogo/protobuf/proto/extensions.go b/vendor/github.com/gogo/protobuf/proto/extensions.go new file mode 100644 index 0000000000000..44ebd457cf618 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/extensions.go @@ -0,0 +1,604 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2010 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +/* + * Types and routines for supporting protocol buffer extensions. + */ + +import ( + "errors" + "fmt" + "io" + "reflect" + "strconv" + "sync" +) + +// ErrMissingExtension is the error returned by GetExtension if the named extension is not in the message. +var ErrMissingExtension = errors.New("proto: missing extension") + +// ExtensionRange represents a range of message extensions for a protocol buffer. +// Used in code generated by the protocol compiler. +type ExtensionRange struct { + Start, End int32 // both inclusive +} + +// extendableProto is an interface implemented by any protocol buffer generated by the current +// proto compiler that may be extended. +type extendableProto interface { + Message + ExtensionRangeArray() []ExtensionRange + extensionsWrite() map[int32]Extension + extensionsRead() (map[int32]Extension, sync.Locker) +} + +// extendableProtoV1 is an interface implemented by a protocol buffer generated by the previous +// version of the proto compiler that may be extended. +type extendableProtoV1 interface { + Message + ExtensionRangeArray() []ExtensionRange + ExtensionMap() map[int32]Extension +} + +// extensionAdapter is a wrapper around extendableProtoV1 that implements extendableProto. +type extensionAdapter struct { + extendableProtoV1 +} + +func (e extensionAdapter) extensionsWrite() map[int32]Extension { + return e.ExtensionMap() +} + +func (e extensionAdapter) extensionsRead() (map[int32]Extension, sync.Locker) { + return e.ExtensionMap(), notLocker{} +} + +// notLocker is a sync.Locker whose Lock and Unlock methods are nops. +type notLocker struct{} + +func (n notLocker) Lock() {} +func (n notLocker) Unlock() {} + +// extendable returns the extendableProto interface for the given generated proto message. +// If the proto message has the old extension format, it returns a wrapper that implements +// the extendableProto interface. +func extendable(p interface{}) (extendableProto, error) { + switch p := p.(type) { + case extendableProto: + if isNilPtr(p) { + return nil, fmt.Errorf("proto: nil %T is not extendable", p) + } + return p, nil + case extendableProtoV1: + if isNilPtr(p) { + return nil, fmt.Errorf("proto: nil %T is not extendable", p) + } + return extensionAdapter{p}, nil + case extensionsBytes: + return slowExtensionAdapter{p}, nil + } + // Don't allocate a specific error containing %T: + // this is the hot path for Clone and MarshalText. + return nil, errNotExtendable +} + +var errNotExtendable = errors.New("proto: not an extendable proto.Message") + +func isNilPtr(x interface{}) bool { + v := reflect.ValueOf(x) + return v.Kind() == reflect.Ptr && v.IsNil() +} + +// XXX_InternalExtensions is an internal representation of proto extensions. +// +// Each generated message struct type embeds an anonymous XXX_InternalExtensions field, +// thus gaining the unexported 'extensions' method, which can be called only from the proto package. +// +// The methods of XXX_InternalExtensions are not concurrency safe in general, +// but calls to logically read-only methods such as has and get may be executed concurrently. +type XXX_InternalExtensions struct { + // The struct must be indirect so that if a user inadvertently copies a + // generated message and its embedded XXX_InternalExtensions, they + // avoid the mayhem of a copied mutex. + // + // The mutex serializes all logically read-only operations to p.extensionMap. + // It is up to the client to ensure that write operations to p.extensionMap are + // mutually exclusive with other accesses. + p *struct { + mu sync.Mutex + extensionMap map[int32]Extension + } +} + +// extensionsWrite returns the extension map, creating it on first use. +func (e *XXX_InternalExtensions) extensionsWrite() map[int32]Extension { + if e.p == nil { + e.p = new(struct { + mu sync.Mutex + extensionMap map[int32]Extension + }) + e.p.extensionMap = make(map[int32]Extension) + } + return e.p.extensionMap +} + +// extensionsRead returns the extensions map for read-only use. It may be nil. +// The caller must hold the returned mutex's lock when accessing Elements within the map. +func (e *XXX_InternalExtensions) extensionsRead() (map[int32]Extension, sync.Locker) { + if e.p == nil { + return nil, nil + } + return e.p.extensionMap, &e.p.mu +} + +// ExtensionDesc represents an extension specification. +// Used in generated code from the protocol compiler. +type ExtensionDesc struct { + ExtendedType Message // nil pointer to the type that is being extended + ExtensionType interface{} // nil pointer to the extension type + Field int32 // field number + Name string // fully-qualified name of extension, for text formatting + Tag string // protobuf tag style + Filename string // name of the file in which the extension is defined +} + +func (ed *ExtensionDesc) repeated() bool { + t := reflect.TypeOf(ed.ExtensionType) + return t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8 +} + +// Extension represents an extension in a message. +type Extension struct { + // When an extension is stored in a message using SetExtension + // only desc and value are set. When the message is marshaled + // enc will be set to the encoded form of the message. + // + // When a message is unmarshaled and contains extensions, each + // extension will have only enc set. When such an extension is + // accessed using GetExtension (or GetExtensions) desc and value + // will be set. + desc *ExtensionDesc + value interface{} + enc []byte +} + +// SetRawExtension is for testing only. +func SetRawExtension(base Message, id int32, b []byte) { + if ebase, ok := base.(extensionsBytes); ok { + clearExtension(base, id) + ext := ebase.GetExtensions() + *ext = append(*ext, b...) + return + } + epb, err := extendable(base) + if err != nil { + return + } + extmap := epb.extensionsWrite() + extmap[id] = Extension{enc: b} +} + +// isExtensionField returns true iff the given field number is in an extension range. +func isExtensionField(pb extendableProto, field int32) bool { + for _, er := range pb.ExtensionRangeArray() { + if er.Start <= field && field <= er.End { + return true + } + } + return false +} + +// checkExtensionTypes checks that the given extension is valid for pb. +func checkExtensionTypes(pb extendableProto, extension *ExtensionDesc) error { + var pbi interface{} = pb + // Check the extended type. + if ea, ok := pbi.(extensionAdapter); ok { + pbi = ea.extendableProtoV1 + } + if ea, ok := pbi.(slowExtensionAdapter); ok { + pbi = ea.extensionsBytes + } + if a, b := reflect.TypeOf(pbi), reflect.TypeOf(extension.ExtendedType); a != b { + return fmt.Errorf("proto: bad extended type; %v does not extend %v", b, a) + } + // Check the range. + if !isExtensionField(pb, extension.Field) { + return errors.New("proto: bad extension number; not in declared ranges") + } + return nil +} + +// extPropKey is sufficient to uniquely identify an extension. +type extPropKey struct { + base reflect.Type + field int32 +} + +var extProp = struct { + sync.RWMutex + m map[extPropKey]*Properties +}{ + m: make(map[extPropKey]*Properties), +} + +func extensionProperties(ed *ExtensionDesc) *Properties { + key := extPropKey{base: reflect.TypeOf(ed.ExtendedType), field: ed.Field} + + extProp.RLock() + if prop, ok := extProp.m[key]; ok { + extProp.RUnlock() + return prop + } + extProp.RUnlock() + + extProp.Lock() + defer extProp.Unlock() + // Check again. + if prop, ok := extProp.m[key]; ok { + return prop + } + + prop := new(Properties) + prop.Init(reflect.TypeOf(ed.ExtensionType), "unknown_name", ed.Tag, nil) + extProp.m[key] = prop + return prop +} + +// HasExtension returns whether the given extension is present in pb. +func HasExtension(pb Message, extension *ExtensionDesc) bool { + if epb, doki := pb.(extensionsBytes); doki { + ext := epb.GetExtensions() + buf := *ext + o := 0 + for o < len(buf) { + tag, n := DecodeVarint(buf[o:]) + fieldNum := int32(tag >> 3) + if int32(fieldNum) == extension.Field { + return true + } + wireType := int(tag & 0x7) + o += n + l, err := size(buf[o:], wireType) + if err != nil { + return false + } + o += l + } + return false + } + // TODO: Check types, field numbers, etc.? + epb, err := extendable(pb) + if err != nil { + return false + } + extmap, mu := epb.extensionsRead() + if extmap == nil { + return false + } + mu.Lock() + _, ok := extmap[extension.Field] + mu.Unlock() + return ok +} + +// ClearExtension removes the given extension from pb. +func ClearExtension(pb Message, extension *ExtensionDesc) { + clearExtension(pb, extension.Field) +} + +func clearExtension(pb Message, fieldNum int32) { + if epb, ok := pb.(extensionsBytes); ok { + offset := 0 + for offset != -1 { + offset = deleteExtension(epb, fieldNum, offset) + } + return + } + epb, err := extendable(pb) + if err != nil { + return + } + // TODO: Check types, field numbers, etc.? + extmap := epb.extensionsWrite() + delete(extmap, fieldNum) +} + +// GetExtension retrieves a proto2 extended field from pb. +// +// If the descriptor is type complete (i.e., ExtensionDesc.ExtensionType is non-nil), +// then GetExtension parses the encoded field and returns a Go value of the specified type. +// If the field is not present, then the default value is returned (if one is specified), +// otherwise ErrMissingExtension is reported. +// +// If the descriptor is not type complete (i.e., ExtensionDesc.ExtensionType is nil), +// then GetExtension returns the raw encoded bytes of the field extension. +func GetExtension(pb Message, extension *ExtensionDesc) (interface{}, error) { + if epb, doki := pb.(extensionsBytes); doki { + ext := epb.GetExtensions() + return decodeExtensionFromBytes(extension, *ext) + } + + epb, err := extendable(pb) + if err != nil { + return nil, err + } + + if extension.ExtendedType != nil { + // can only check type if this is a complete descriptor + if cerr := checkExtensionTypes(epb, extension); cerr != nil { + return nil, cerr + } + } + + emap, mu := epb.extensionsRead() + if emap == nil { + return defaultExtensionValue(extension) + } + mu.Lock() + defer mu.Unlock() + e, ok := emap[extension.Field] + if !ok { + // defaultExtensionValue returns the default value or + // ErrMissingExtension if there is no default. + return defaultExtensionValue(extension) + } + + if e.value != nil { + // Already decoded. Check the descriptor, though. + if e.desc != extension { + // This shouldn't happen. If it does, it means that + // GetExtension was called twice with two different + // descriptors with the same field number. + return nil, errors.New("proto: descriptor conflict") + } + return e.value, nil + } + + if extension.ExtensionType == nil { + // incomplete descriptor + return e.enc, nil + } + + v, err := decodeExtension(e.enc, extension) + if err != nil { + return nil, err + } + + // Remember the decoded version and drop the encoded version. + // That way it is safe to mutate what we return. + e.value = v + e.desc = extension + e.enc = nil + emap[extension.Field] = e + return e.value, nil +} + +// defaultExtensionValue returns the default value for extension. +// If no default for an extension is defined ErrMissingExtension is returned. +func defaultExtensionValue(extension *ExtensionDesc) (interface{}, error) { + if extension.ExtensionType == nil { + // incomplete descriptor, so no default + return nil, ErrMissingExtension + } + + t := reflect.TypeOf(extension.ExtensionType) + props := extensionProperties(extension) + + sf, _, err := fieldDefault(t, props) + if err != nil { + return nil, err + } + + if sf == nil || sf.value == nil { + // There is no default value. + return nil, ErrMissingExtension + } + + if t.Kind() != reflect.Ptr { + // We do not need to return a Ptr, we can directly return sf.value. + return sf.value, nil + } + + // We need to return an interface{} that is a pointer to sf.value. + value := reflect.New(t).Elem() + value.Set(reflect.New(value.Type().Elem())) + if sf.kind == reflect.Int32 { + // We may have an int32 or an enum, but the underlying data is int32. + // Since we can't set an int32 into a non int32 reflect.value directly + // set it as a int32. + value.Elem().SetInt(int64(sf.value.(int32))) + } else { + value.Elem().Set(reflect.ValueOf(sf.value)) + } + return value.Interface(), nil +} + +// decodeExtension decodes an extension encoded in b. +func decodeExtension(b []byte, extension *ExtensionDesc) (interface{}, error) { + t := reflect.TypeOf(extension.ExtensionType) + unmarshal := typeUnmarshaler(t, extension.Tag) + + // t is a pointer to a struct, pointer to basic type or a slice. + // Allocate space to store the pointer/slice. + value := reflect.New(t).Elem() + + var err error + for { + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + wire := int(x) & 7 + + b, err = unmarshal(b, valToPointer(value.Addr()), wire) + if err != nil { + return nil, err + } + + if len(b) == 0 { + break + } + } + return value.Interface(), nil +} + +// GetExtensions returns a slice of the extensions present in pb that are also listed in es. +// The returned slice has the same length as es; missing extensions will appear as nil elements. +func GetExtensions(pb Message, es []*ExtensionDesc) (extensions []interface{}, err error) { + epb, err := extendable(pb) + if err != nil { + return nil, err + } + extensions = make([]interface{}, len(es)) + for i, e := range es { + extensions[i], err = GetExtension(epb, e) + if err == ErrMissingExtension { + err = nil + } + if err != nil { + return + } + } + return +} + +// ExtensionDescs returns a new slice containing pb's extension descriptors, in undefined order. +// For non-registered extensions, ExtensionDescs returns an incomplete descriptor containing +// just the Field field, which defines the extension's field number. +func ExtensionDescs(pb Message) ([]*ExtensionDesc, error) { + epb, err := extendable(pb) + if err != nil { + return nil, err + } + registeredExtensions := RegisteredExtensions(pb) + + emap, mu := epb.extensionsRead() + if emap == nil { + return nil, nil + } + mu.Lock() + defer mu.Unlock() + extensions := make([]*ExtensionDesc, 0, len(emap)) + for extid, e := range emap { + desc := e.desc + if desc == nil { + desc = registeredExtensions[extid] + if desc == nil { + desc = &ExtensionDesc{Field: extid} + } + } + + extensions = append(extensions, desc) + } + return extensions, nil +} + +// SetExtension sets the specified extension of pb to the specified value. +func SetExtension(pb Message, extension *ExtensionDesc, value interface{}) error { + if epb, ok := pb.(extensionsBytes); ok { + newb, err := encodeExtension(extension, value) + if err != nil { + return err + } + bb := epb.GetExtensions() + *bb = append(*bb, newb...) + return nil + } + epb, err := extendable(pb) + if err != nil { + return err + } + if err := checkExtensionTypes(epb, extension); err != nil { + return err + } + typ := reflect.TypeOf(extension.ExtensionType) + if typ != reflect.TypeOf(value) { + return errors.New("proto: bad extension value type") + } + // nil extension values need to be caught early, because the + // encoder can't distinguish an ErrNil due to a nil extension + // from an ErrNil due to a missing field. Extensions are + // always optional, so the encoder would just swallow the error + // and drop all the extensions from the encoded message. + if reflect.ValueOf(value).IsNil() { + return fmt.Errorf("proto: SetExtension called with nil value of type %T", value) + } + + extmap := epb.extensionsWrite() + extmap[extension.Field] = Extension{desc: extension, value: value} + return nil +} + +// ClearAllExtensions clears all extensions from pb. +func ClearAllExtensions(pb Message) { + if epb, doki := pb.(extensionsBytes); doki { + ext := epb.GetExtensions() + *ext = []byte{} + return + } + epb, err := extendable(pb) + if err != nil { + return + } + m := epb.extensionsWrite() + for k := range m { + delete(m, k) + } +} + +// A global registry of extensions. +// The generated code will register the generated descriptors by calling RegisterExtension. + +var extensionMaps = make(map[reflect.Type]map[int32]*ExtensionDesc) + +// RegisterExtension is called from the generated code. +func RegisterExtension(desc *ExtensionDesc) { + st := reflect.TypeOf(desc.ExtendedType).Elem() + m := extensionMaps[st] + if m == nil { + m = make(map[int32]*ExtensionDesc) + extensionMaps[st] = m + } + if _, ok := m[desc.Field]; ok { + panic("proto: duplicate extension registered: " + st.String() + " " + strconv.Itoa(int(desc.Field))) + } + m[desc.Field] = desc +} + +// RegisteredExtensions returns a map of the registered extensions of a +// protocol buffer struct, indexed by the extension number. +// The argument pb should be a nil pointer to the struct type. +func RegisteredExtensions(pb Message) map[int32]*ExtensionDesc { + return extensionMaps[reflect.TypeOf(pb).Elem()] +} diff --git a/vendor/github.com/gogo/protobuf/proto/extensions_gogo.go b/vendor/github.com/gogo/protobuf/proto/extensions_gogo.go new file mode 100644 index 0000000000000..53ebd8cca01c3 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/extensions_gogo.go @@ -0,0 +1,368 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "bytes" + "errors" + "fmt" + "io" + "reflect" + "sort" + "strings" + "sync" +) + +type extensionsBytes interface { + Message + ExtensionRangeArray() []ExtensionRange + GetExtensions() *[]byte +} + +type slowExtensionAdapter struct { + extensionsBytes +} + +func (s slowExtensionAdapter) extensionsWrite() map[int32]Extension { + panic("Please report a bug to github.com/gogo/protobuf if you see this message: Writing extensions is not supported for extensions stored in a byte slice field.") +} + +func (s slowExtensionAdapter) extensionsRead() (map[int32]Extension, sync.Locker) { + b := s.GetExtensions() + m, err := BytesToExtensionsMap(*b) + if err != nil { + panic(err) + } + return m, notLocker{} +} + +func GetBoolExtension(pb Message, extension *ExtensionDesc, ifnotset bool) bool { + if reflect.ValueOf(pb).IsNil() { + return ifnotset + } + value, err := GetExtension(pb, extension) + if err != nil { + return ifnotset + } + if value == nil { + return ifnotset + } + if value.(*bool) == nil { + return ifnotset + } + return *(value.(*bool)) +} + +func (this *Extension) Equal(that *Extension) bool { + if err := this.Encode(); err != nil { + return false + } + if err := that.Encode(); err != nil { + return false + } + return bytes.Equal(this.enc, that.enc) +} + +func (this *Extension) Compare(that *Extension) int { + if err := this.Encode(); err != nil { + return 1 + } + if err := that.Encode(); err != nil { + return -1 + } + return bytes.Compare(this.enc, that.enc) +} + +func SizeOfInternalExtension(m extendableProto) (n int) { + info := getMarshalInfo(reflect.TypeOf(m)) + return info.sizeV1Extensions(m.extensionsWrite()) +} + +type sortableMapElem struct { + field int32 + ext Extension +} + +func newSortableExtensionsFromMap(m map[int32]Extension) sortableExtensions { + s := make(sortableExtensions, 0, len(m)) + for k, v := range m { + s = append(s, &sortableMapElem{field: k, ext: v}) + } + return s +} + +type sortableExtensions []*sortableMapElem + +func (this sortableExtensions) Len() int { return len(this) } + +func (this sortableExtensions) Swap(i, j int) { this[i], this[j] = this[j], this[i] } + +func (this sortableExtensions) Less(i, j int) bool { return this[i].field < this[j].field } + +func (this sortableExtensions) String() string { + sort.Sort(this) + ss := make([]string, len(this)) + for i := range this { + ss[i] = fmt.Sprintf("%d: %v", this[i].field, this[i].ext) + } + return "map[" + strings.Join(ss, ",") + "]" +} + +func StringFromInternalExtension(m extendableProto) string { + return StringFromExtensionsMap(m.extensionsWrite()) +} + +func StringFromExtensionsMap(m map[int32]Extension) string { + return newSortableExtensionsFromMap(m).String() +} + +func StringFromExtensionsBytes(ext []byte) string { + m, err := BytesToExtensionsMap(ext) + if err != nil { + panic(err) + } + return StringFromExtensionsMap(m) +} + +func EncodeInternalExtension(m extendableProto, data []byte) (n int, err error) { + return EncodeExtensionMap(m.extensionsWrite(), data) +} + +func EncodeExtensionMap(m map[int32]Extension, data []byte) (n int, err error) { + o := 0 + for _, e := range m { + if err := e.Encode(); err != nil { + return 0, err + } + n := copy(data[o:], e.enc) + if n != len(e.enc) { + return 0, io.ErrShortBuffer + } + o += n + } + return o, nil +} + +func GetRawExtension(m map[int32]Extension, id int32) ([]byte, error) { + e := m[id] + if err := e.Encode(); err != nil { + return nil, err + } + return e.enc, nil +} + +func size(buf []byte, wire int) (int, error) { + switch wire { + case WireVarint: + _, n := DecodeVarint(buf) + return n, nil + case WireFixed64: + return 8, nil + case WireBytes: + v, n := DecodeVarint(buf) + return int(v) + n, nil + case WireFixed32: + return 4, nil + case WireStartGroup: + offset := 0 + for { + u, n := DecodeVarint(buf[offset:]) + fwire := int(u & 0x7) + offset += n + if fwire == WireEndGroup { + return offset, nil + } + s, err := size(buf[offset:], wire) + if err != nil { + return 0, err + } + offset += s + } + } + return 0, fmt.Errorf("proto: can't get size for unknown wire type %d", wire) +} + +func BytesToExtensionsMap(buf []byte) (map[int32]Extension, error) { + m := make(map[int32]Extension) + i := 0 + for i < len(buf) { + tag, n := DecodeVarint(buf[i:]) + if n <= 0 { + return nil, fmt.Errorf("unable to decode varint") + } + fieldNum := int32(tag >> 3) + wireType := int(tag & 0x7) + l, err := size(buf[i+n:], wireType) + if err != nil { + return nil, err + } + end := i + int(l) + n + m[int32(fieldNum)] = Extension{enc: buf[i:end]} + i = end + } + return m, nil +} + +func NewExtension(e []byte) Extension { + ee := Extension{enc: make([]byte, len(e))} + copy(ee.enc, e) + return ee +} + +func AppendExtension(e Message, tag int32, buf []byte) { + if ee, eok := e.(extensionsBytes); eok { + ext := ee.GetExtensions() + *ext = append(*ext, buf...) + return + } + if ee, eok := e.(extendableProto); eok { + m := ee.extensionsWrite() + ext := m[int32(tag)] // may be missing + ext.enc = append(ext.enc, buf...) + m[int32(tag)] = ext + } +} + +func encodeExtension(extension *ExtensionDesc, value interface{}) ([]byte, error) { + u := getMarshalInfo(reflect.TypeOf(extension.ExtendedType)) + ei := u.getExtElemInfo(extension) + v := value + p := toAddrPointer(&v, ei.isptr) + siz := ei.sizer(p, SizeVarint(ei.wiretag)) + buf := make([]byte, 0, siz) + return ei.marshaler(buf, p, ei.wiretag, false) +} + +func decodeExtensionFromBytes(extension *ExtensionDesc, buf []byte) (interface{}, error) { + o := 0 + for o < len(buf) { + tag, n := DecodeVarint((buf)[o:]) + fieldNum := int32(tag >> 3) + wireType := int(tag & 0x7) + if o+n > len(buf) { + return nil, fmt.Errorf("unable to decode extension") + } + l, err := size((buf)[o+n:], wireType) + if err != nil { + return nil, err + } + if int32(fieldNum) == extension.Field { + if o+n+l > len(buf) { + return nil, fmt.Errorf("unable to decode extension") + } + v, err := decodeExtension((buf)[o:o+n+l], extension) + if err != nil { + return nil, err + } + return v, nil + } + o += n + l + } + return defaultExtensionValue(extension) +} + +func (this *Extension) Encode() error { + if this.enc == nil { + var err error + this.enc, err = encodeExtension(this.desc, this.value) + if err != nil { + return err + } + } + return nil +} + +func (this Extension) GoString() string { + if err := this.Encode(); err != nil { + return fmt.Sprintf("error encoding extension: %v", err) + } + return fmt.Sprintf("proto.NewExtension(%#v)", this.enc) +} + +func SetUnsafeExtension(pb Message, fieldNum int32, value interface{}) error { + typ := reflect.TypeOf(pb).Elem() + ext, ok := extensionMaps[typ] + if !ok { + return fmt.Errorf("proto: bad extended type; %s is not extendable", typ.String()) + } + desc, ok := ext[fieldNum] + if !ok { + return errors.New("proto: bad extension number; not in declared ranges") + } + return SetExtension(pb, desc, value) +} + +func GetUnsafeExtension(pb Message, fieldNum int32) (interface{}, error) { + typ := reflect.TypeOf(pb).Elem() + ext, ok := extensionMaps[typ] + if !ok { + return nil, fmt.Errorf("proto: bad extended type; %s is not extendable", typ.String()) + } + desc, ok := ext[fieldNum] + if !ok { + return nil, fmt.Errorf("unregistered field number %d", fieldNum) + } + return GetExtension(pb, desc) +} + +func NewUnsafeXXX_InternalExtensions(m map[int32]Extension) XXX_InternalExtensions { + x := &XXX_InternalExtensions{ + p: new(struct { + mu sync.Mutex + extensionMap map[int32]Extension + }), + } + x.p.extensionMap = m + return *x +} + +func GetUnsafeExtensionsMap(extendable Message) map[int32]Extension { + pb := extendable.(extendableProto) + return pb.extensionsWrite() +} + +func deleteExtension(pb extensionsBytes, theFieldNum int32, offset int) int { + ext := pb.GetExtensions() + for offset < len(*ext) { + tag, n1 := DecodeVarint((*ext)[offset:]) + fieldNum := int32(tag >> 3) + wireType := int(tag & 0x7) + n2, err := size((*ext)[offset+n1:], wireType) + if err != nil { + panic(err) + } + newOffset := offset + n1 + n2 + if fieldNum == theFieldNum { + *ext = append((*ext)[:offset], (*ext)[newOffset:]...) + return offset + } + offset = newOffset + } + return -1 +} diff --git a/vendor/github.com/gogo/protobuf/proto/lib.go b/vendor/github.com/gogo/protobuf/proto/lib.go new file mode 100644 index 0000000000000..0f1950c67e485 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/lib.go @@ -0,0 +1,921 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2010 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +/* +Package proto converts data structures to and from the wire format of +protocol buffers. It works in concert with the Go source code generated +for .proto files by the protocol compiler. + +A summary of the properties of the protocol buffer interface +for a protocol buffer variable v: + + - Names are turned from camel_case to CamelCase for export. + - There are no methods on v to set fields; just treat + them as structure fields. + - There are getters that return a field's value if set, + and return the field's default value if unset. + The getters work even if the receiver is a nil message. + - The zero value for a struct is its correct initialization state. + All desired fields must be set before marshaling. + - A Reset() method will restore a protobuf struct to its zero state. + - Non-repeated fields are pointers to the values; nil means unset. + That is, optional or required field int32 f becomes F *int32. + - Repeated fields are slices. + - Helper functions are available to aid the setting of fields. + msg.Foo = proto.String("hello") // set field + - Constants are defined to hold the default values of all fields that + have them. They have the form Default_StructName_FieldName. + Because the getter methods handle defaulted values, + direct use of these constants should be rare. + - Enums are given type names and maps from names to values. + Enum values are prefixed by the enclosing message's name, or by the + enum's type name if it is a top-level enum. Enum types have a String + method, and a Enum method to assist in message construction. + - Nested messages, groups and enums have type names prefixed with the name of + the surrounding message type. + - Extensions are given descriptor names that start with E_, + followed by an underscore-delimited list of the nested messages + that contain it (if any) followed by the CamelCased name of the + extension field itself. HasExtension, ClearExtension, GetExtension + and SetExtension are functions for manipulating extensions. + - Oneof field sets are given a single field in their message, + with distinguished wrapper types for each possible field value. + - Marshal and Unmarshal are functions to encode and decode the wire format. + +When the .proto file specifies `syntax="proto3"`, there are some differences: + + - Non-repeated fields of non-message type are values instead of pointers. + - Enum types do not get an Enum method. + +The simplest way to describe this is to see an example. +Given file test.proto, containing + + package example; + + enum FOO { X = 17; } + + message Test { + required string label = 1; + optional int32 type = 2 [default=77]; + repeated int64 reps = 3; + optional group OptionalGroup = 4 { + required string RequiredField = 5; + } + oneof union { + int32 number = 6; + string name = 7; + } + } + +The resulting file, test.pb.go, is: + + package example + + import proto "github.com/gogo/protobuf/proto" + import math "math" + + type FOO int32 + const ( + FOO_X FOO = 17 + ) + var FOO_name = map[int32]string{ + 17: "X", + } + var FOO_value = map[string]int32{ + "X": 17, + } + + func (x FOO) Enum() *FOO { + p := new(FOO) + *p = x + return p + } + func (x FOO) String() string { + return proto.EnumName(FOO_name, int32(x)) + } + func (x *FOO) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(FOO_value, data) + if err != nil { + return err + } + *x = FOO(value) + return nil + } + + type Test struct { + Label *string `protobuf:"bytes,1,req,name=label" json:"label,omitempty"` + Type *int32 `protobuf:"varint,2,opt,name=type,def=77" json:"type,omitempty"` + Reps []int64 `protobuf:"varint,3,rep,name=reps" json:"reps,omitempty"` + Optionalgroup *Test_OptionalGroup `protobuf:"group,4,opt,name=OptionalGroup" json:"optionalgroup,omitempty"` + // Types that are valid to be assigned to Union: + // *Test_Number + // *Test_Name + Union isTest_Union `protobuf_oneof:"union"` + XXX_unrecognized []byte `json:"-"` + } + func (m *Test) Reset() { *m = Test{} } + func (m *Test) String() string { return proto.CompactTextString(m) } + func (*Test) ProtoMessage() {} + + type isTest_Union interface { + isTest_Union() + } + + type Test_Number struct { + Number int32 `protobuf:"varint,6,opt,name=number"` + } + type Test_Name struct { + Name string `protobuf:"bytes,7,opt,name=name"` + } + + func (*Test_Number) isTest_Union() {} + func (*Test_Name) isTest_Union() {} + + func (m *Test) GetUnion() isTest_Union { + if m != nil { + return m.Union + } + return nil + } + const Default_Test_Type int32 = 77 + + func (m *Test) GetLabel() string { + if m != nil && m.Label != nil { + return *m.Label + } + return "" + } + + func (m *Test) GetType() int32 { + if m != nil && m.Type != nil { + return *m.Type + } + return Default_Test_Type + } + + func (m *Test) GetOptionalgroup() *Test_OptionalGroup { + if m != nil { + return m.Optionalgroup + } + return nil + } + + type Test_OptionalGroup struct { + RequiredField *string `protobuf:"bytes,5,req" json:"RequiredField,omitempty"` + } + func (m *Test_OptionalGroup) Reset() { *m = Test_OptionalGroup{} } + func (m *Test_OptionalGroup) String() string { return proto.CompactTextString(m) } + + func (m *Test_OptionalGroup) GetRequiredField() string { + if m != nil && m.RequiredField != nil { + return *m.RequiredField + } + return "" + } + + func (m *Test) GetNumber() int32 { + if x, ok := m.GetUnion().(*Test_Number); ok { + return x.Number + } + return 0 + } + + func (m *Test) GetName() string { + if x, ok := m.GetUnion().(*Test_Name); ok { + return x.Name + } + return "" + } + + func init() { + proto.RegisterEnum("example.FOO", FOO_name, FOO_value) + } + +To create and play with a Test object: + + package main + + import ( + "log" + + "github.com/gogo/protobuf/proto" + pb "./example.pb" + ) + + func main() { + test := &pb.Test{ + Label: proto.String("hello"), + Type: proto.Int32(17), + Reps: []int64{1, 2, 3}, + Optionalgroup: &pb.Test_OptionalGroup{ + RequiredField: proto.String("good bye"), + }, + Union: &pb.Test_Name{"fred"}, + } + data, err := proto.Marshal(test) + if err != nil { + log.Fatal("marshaling error: ", err) + } + newTest := &pb.Test{} + err = proto.Unmarshal(data, newTest) + if err != nil { + log.Fatal("unmarshaling error: ", err) + } + // Now test and newTest contain the same data. + if test.GetLabel() != newTest.GetLabel() { + log.Fatalf("data mismatch %q != %q", test.GetLabel(), newTest.GetLabel()) + } + // Use a type switch to determine which oneof was set. + switch u := test.Union.(type) { + case *pb.Test_Number: // u.Number contains the number. + case *pb.Test_Name: // u.Name contains the string. + } + // etc. + } +*/ +package proto + +import ( + "encoding/json" + "errors" + "fmt" + "log" + "reflect" + "sort" + "strconv" + "sync" +) + +var errInvalidUTF8 = errors.New("proto: invalid UTF-8 string") + +// Message is implemented by generated protocol buffer messages. +type Message interface { + Reset() + String() string + ProtoMessage() +} + +// Stats records allocation details about the protocol buffer encoders +// and decoders. Useful for tuning the library itself. +type Stats struct { + Emalloc uint64 // mallocs in encode + Dmalloc uint64 // mallocs in decode + Encode uint64 // number of encodes + Decode uint64 // number of decodes + Chit uint64 // number of cache hits + Cmiss uint64 // number of cache misses + Size uint64 // number of sizes +} + +// Set to true to enable stats collection. +const collectStats = false + +var stats Stats + +// GetStats returns a copy of the global Stats structure. +func GetStats() Stats { return stats } + +// A Buffer is a buffer manager for marshaling and unmarshaling +// protocol buffers. It may be reused between invocations to +// reduce memory usage. It is not necessary to use a Buffer; +// the global functions Marshal and Unmarshal create a +// temporary Buffer and are fine for most applications. +type Buffer struct { + buf []byte // encode/decode byte stream + index int // read point + + deterministic bool +} + +// NewBuffer allocates a new Buffer and initializes its internal data to +// the contents of the argument slice. +func NewBuffer(e []byte) *Buffer { + return &Buffer{buf: e} +} + +// Reset resets the Buffer, ready for marshaling a new protocol buffer. +func (p *Buffer) Reset() { + p.buf = p.buf[0:0] // for reading/writing + p.index = 0 // for reading +} + +// SetBuf replaces the internal buffer with the slice, +// ready for unmarshaling the contents of the slice. +func (p *Buffer) SetBuf(s []byte) { + p.buf = s + p.index = 0 +} + +// Bytes returns the contents of the Buffer. +func (p *Buffer) Bytes() []byte { return p.buf } + +// SetDeterministic sets whether to use deterministic serialization. +// +// Deterministic serialization guarantees that for a given binary, equal +// messages will always be serialized to the same bytes. This implies: +// +// - Repeated serialization of a message will return the same bytes. +// - Different processes of the same binary (which may be executing on +// different machines) will serialize equal messages to the same bytes. +// +// Note that the deterministic serialization is NOT canonical across +// languages. It is not guaranteed to remain stable over time. It is unstable +// across different builds with schema changes due to unknown fields. +// Users who need canonical serialization (e.g., persistent storage in a +// canonical form, fingerprinting, etc.) should define their own +// canonicalization specification and implement their own serializer rather +// than relying on this API. +// +// If deterministic serialization is requested, map entries will be sorted +// by keys in lexographical order. This is an implementation detail and +// subject to change. +func (p *Buffer) SetDeterministic(deterministic bool) { + p.deterministic = deterministic +} + +/* + * Helper routines for simplifying the creation of optional fields of basic type. + */ + +// Bool is a helper routine that allocates a new bool value +// to store v and returns a pointer to it. +func Bool(v bool) *bool { + return &v +} + +// Int32 is a helper routine that allocates a new int32 value +// to store v and returns a pointer to it. +func Int32(v int32) *int32 { + return &v +} + +// Int is a helper routine that allocates a new int32 value +// to store v and returns a pointer to it, but unlike Int32 +// its argument value is an int. +func Int(v int) *int32 { + p := new(int32) + *p = int32(v) + return p +} + +// Int64 is a helper routine that allocates a new int64 value +// to store v and returns a pointer to it. +func Int64(v int64) *int64 { + return &v +} + +// Float32 is a helper routine that allocates a new float32 value +// to store v and returns a pointer to it. +func Float32(v float32) *float32 { + return &v +} + +// Float64 is a helper routine that allocates a new float64 value +// to store v and returns a pointer to it. +func Float64(v float64) *float64 { + return &v +} + +// Uint32 is a helper routine that allocates a new uint32 value +// to store v and returns a pointer to it. +func Uint32(v uint32) *uint32 { + return &v +} + +// Uint64 is a helper routine that allocates a new uint64 value +// to store v and returns a pointer to it. +func Uint64(v uint64) *uint64 { + return &v +} + +// String is a helper routine that allocates a new string value +// to store v and returns a pointer to it. +func String(v string) *string { + return &v +} + +// EnumName is a helper function to simplify printing protocol buffer enums +// by name. Given an enum map and a value, it returns a useful string. +func EnumName(m map[int32]string, v int32) string { + s, ok := m[v] + if ok { + return s + } + return strconv.Itoa(int(v)) +} + +// UnmarshalJSONEnum is a helper function to simplify recovering enum int values +// from their JSON-encoded representation. Given a map from the enum's symbolic +// names to its int values, and a byte buffer containing the JSON-encoded +// value, it returns an int32 that can be cast to the enum type by the caller. +// +// The function can deal with both JSON representations, numeric and symbolic. +func UnmarshalJSONEnum(m map[string]int32, data []byte, enumName string) (int32, error) { + if data[0] == '"' { + // New style: enums are strings. + var repr string + if err := json.Unmarshal(data, &repr); err != nil { + return -1, err + } + val, ok := m[repr] + if !ok { + return 0, fmt.Errorf("unrecognized enum %s value %q", enumName, repr) + } + return val, nil + } + // Old style: enums are ints. + var val int32 + if err := json.Unmarshal(data, &val); err != nil { + return 0, fmt.Errorf("cannot unmarshal %#q into enum %s", data, enumName) + } + return val, nil +} + +// DebugPrint dumps the encoded data in b in a debugging format with a header +// including the string s. Used in testing but made available for general debugging. +func (p *Buffer) DebugPrint(s string, b []byte) { + var u uint64 + + obuf := p.buf + sindex := p.index + p.buf = b + p.index = 0 + depth := 0 + + fmt.Printf("\n--- %s ---\n", s) + +out: + for { + for i := 0; i < depth; i++ { + fmt.Print(" ") + } + + index := p.index + if index == len(p.buf) { + break + } + + op, err := p.DecodeVarint() + if err != nil { + fmt.Printf("%3d: fetching op err %v\n", index, err) + break out + } + tag := op >> 3 + wire := op & 7 + + switch wire { + default: + fmt.Printf("%3d: t=%3d unknown wire=%d\n", + index, tag, wire) + break out + + case WireBytes: + var r []byte + + r, err = p.DecodeRawBytes(false) + if err != nil { + break out + } + fmt.Printf("%3d: t=%3d bytes [%d]", index, tag, len(r)) + if len(r) <= 6 { + for i := 0; i < len(r); i++ { + fmt.Printf(" %.2x", r[i]) + } + } else { + for i := 0; i < 3; i++ { + fmt.Printf(" %.2x", r[i]) + } + fmt.Printf(" ..") + for i := len(r) - 3; i < len(r); i++ { + fmt.Printf(" %.2x", r[i]) + } + } + fmt.Printf("\n") + + case WireFixed32: + u, err = p.DecodeFixed32() + if err != nil { + fmt.Printf("%3d: t=%3d fix32 err %v\n", index, tag, err) + break out + } + fmt.Printf("%3d: t=%3d fix32 %d\n", index, tag, u) + + case WireFixed64: + u, err = p.DecodeFixed64() + if err != nil { + fmt.Printf("%3d: t=%3d fix64 err %v\n", index, tag, err) + break out + } + fmt.Printf("%3d: t=%3d fix64 %d\n", index, tag, u) + + case WireVarint: + u, err = p.DecodeVarint() + if err != nil { + fmt.Printf("%3d: t=%3d varint err %v\n", index, tag, err) + break out + } + fmt.Printf("%3d: t=%3d varint %d\n", index, tag, u) + + case WireStartGroup: + fmt.Printf("%3d: t=%3d start\n", index, tag) + depth++ + + case WireEndGroup: + depth-- + fmt.Printf("%3d: t=%3d end\n", index, tag) + } + } + + if depth != 0 { + fmt.Printf("%3d: start-end not balanced %d\n", p.index, depth) + } + fmt.Printf("\n") + + p.buf = obuf + p.index = sindex +} + +// SetDefaults sets unset protocol buffer fields to their default values. +// It only modifies fields that are both unset and have defined defaults. +// It recursively sets default values in any non-nil sub-messages. +func SetDefaults(pb Message) { + setDefaults(reflect.ValueOf(pb), true, false) +} + +// v is a pointer to a struct. +func setDefaults(v reflect.Value, recur, zeros bool) { + v = v.Elem() + + defaultMu.RLock() + dm, ok := defaults[v.Type()] + defaultMu.RUnlock() + if !ok { + dm = buildDefaultMessage(v.Type()) + defaultMu.Lock() + defaults[v.Type()] = dm + defaultMu.Unlock() + } + + for _, sf := range dm.scalars { + f := v.Field(sf.index) + if !f.IsNil() { + // field already set + continue + } + dv := sf.value + if dv == nil && !zeros { + // no explicit default, and don't want to set zeros + continue + } + fptr := f.Addr().Interface() // **T + // TODO: Consider batching the allocations we do here. + switch sf.kind { + case reflect.Bool: + b := new(bool) + if dv != nil { + *b = dv.(bool) + } + *(fptr.(**bool)) = b + case reflect.Float32: + f := new(float32) + if dv != nil { + *f = dv.(float32) + } + *(fptr.(**float32)) = f + case reflect.Float64: + f := new(float64) + if dv != nil { + *f = dv.(float64) + } + *(fptr.(**float64)) = f + case reflect.Int32: + // might be an enum + if ft := f.Type(); ft != int32PtrType { + // enum + f.Set(reflect.New(ft.Elem())) + if dv != nil { + f.Elem().SetInt(int64(dv.(int32))) + } + } else { + // int32 field + i := new(int32) + if dv != nil { + *i = dv.(int32) + } + *(fptr.(**int32)) = i + } + case reflect.Int64: + i := new(int64) + if dv != nil { + *i = dv.(int64) + } + *(fptr.(**int64)) = i + case reflect.String: + s := new(string) + if dv != nil { + *s = dv.(string) + } + *(fptr.(**string)) = s + case reflect.Uint8: + // exceptional case: []byte + var b []byte + if dv != nil { + db := dv.([]byte) + b = make([]byte, len(db)) + copy(b, db) + } else { + b = []byte{} + } + *(fptr.(*[]byte)) = b + case reflect.Uint32: + u := new(uint32) + if dv != nil { + *u = dv.(uint32) + } + *(fptr.(**uint32)) = u + case reflect.Uint64: + u := new(uint64) + if dv != nil { + *u = dv.(uint64) + } + *(fptr.(**uint64)) = u + default: + log.Printf("proto: can't set default for field %v (sf.kind=%v)", f, sf.kind) + } + } + + for _, ni := range dm.nested { + f := v.Field(ni) + // f is *T or []*T or map[T]*T + switch f.Kind() { + case reflect.Ptr: + if f.IsNil() { + continue + } + setDefaults(f, recur, zeros) + + case reflect.Slice: + for i := 0; i < f.Len(); i++ { + e := f.Index(i) + if e.IsNil() { + continue + } + setDefaults(e, recur, zeros) + } + + case reflect.Map: + for _, k := range f.MapKeys() { + e := f.MapIndex(k) + if e.IsNil() { + continue + } + setDefaults(e, recur, zeros) + } + } + } +} + +var ( + // defaults maps a protocol buffer struct type to a slice of the fields, + // with its scalar fields set to their proto-declared non-zero default values. + defaultMu sync.RWMutex + defaults = make(map[reflect.Type]defaultMessage) + + int32PtrType = reflect.TypeOf((*int32)(nil)) +) + +// defaultMessage represents information about the default values of a message. +type defaultMessage struct { + scalars []scalarField + nested []int // struct field index of nested messages +} + +type scalarField struct { + index int // struct field index + kind reflect.Kind // element type (the T in *T or []T) + value interface{} // the proto-declared default value, or nil +} + +// t is a struct type. +func buildDefaultMessage(t reflect.Type) (dm defaultMessage) { + sprop := GetProperties(t) + for _, prop := range sprop.Prop { + fi, ok := sprop.decoderTags.get(prop.Tag) + if !ok { + // XXX_unrecognized + continue + } + ft := t.Field(fi).Type + + sf, nested, err := fieldDefault(ft, prop) + switch { + case err != nil: + log.Print(err) + case nested: + dm.nested = append(dm.nested, fi) + case sf != nil: + sf.index = fi + dm.scalars = append(dm.scalars, *sf) + } + } + + return dm +} + +// fieldDefault returns the scalarField for field type ft. +// sf will be nil if the field can not have a default. +// nestedMessage will be true if this is a nested message. +// Note that sf.index is not set on return. +func fieldDefault(ft reflect.Type, prop *Properties) (sf *scalarField, nestedMessage bool, err error) { + var canHaveDefault bool + switch ft.Kind() { + case reflect.Ptr: + if ft.Elem().Kind() == reflect.Struct { + nestedMessage = true + } else { + canHaveDefault = true // proto2 scalar field + } + + case reflect.Slice: + switch ft.Elem().Kind() { + case reflect.Ptr: + nestedMessage = true // repeated message + case reflect.Uint8: + canHaveDefault = true // bytes field + } + + case reflect.Map: + if ft.Elem().Kind() == reflect.Ptr { + nestedMessage = true // map with message values + } + } + + if !canHaveDefault { + if nestedMessage { + return nil, true, nil + } + return nil, false, nil + } + + // We now know that ft is a pointer or slice. + sf = &scalarField{kind: ft.Elem().Kind()} + + // scalar fields without defaults + if !prop.HasDefault { + return sf, false, nil + } + + // a scalar field: either *T or []byte + switch ft.Elem().Kind() { + case reflect.Bool: + x, err := strconv.ParseBool(prop.Default) + if err != nil { + return nil, false, fmt.Errorf("proto: bad default bool %q: %v", prop.Default, err) + } + sf.value = x + case reflect.Float32: + x, err := strconv.ParseFloat(prop.Default, 32) + if err != nil { + return nil, false, fmt.Errorf("proto: bad default float32 %q: %v", prop.Default, err) + } + sf.value = float32(x) + case reflect.Float64: + x, err := strconv.ParseFloat(prop.Default, 64) + if err != nil { + return nil, false, fmt.Errorf("proto: bad default float64 %q: %v", prop.Default, err) + } + sf.value = x + case reflect.Int32: + x, err := strconv.ParseInt(prop.Default, 10, 32) + if err != nil { + return nil, false, fmt.Errorf("proto: bad default int32 %q: %v", prop.Default, err) + } + sf.value = int32(x) + case reflect.Int64: + x, err := strconv.ParseInt(prop.Default, 10, 64) + if err != nil { + return nil, false, fmt.Errorf("proto: bad default int64 %q: %v", prop.Default, err) + } + sf.value = x + case reflect.String: + sf.value = prop.Default + case reflect.Uint8: + // []byte (not *uint8) + sf.value = []byte(prop.Default) + case reflect.Uint32: + x, err := strconv.ParseUint(prop.Default, 10, 32) + if err != nil { + return nil, false, fmt.Errorf("proto: bad default uint32 %q: %v", prop.Default, err) + } + sf.value = uint32(x) + case reflect.Uint64: + x, err := strconv.ParseUint(prop.Default, 10, 64) + if err != nil { + return nil, false, fmt.Errorf("proto: bad default uint64 %q: %v", prop.Default, err) + } + sf.value = x + default: + return nil, false, fmt.Errorf("proto: unhandled def kind %v", ft.Elem().Kind()) + } + + return sf, false, nil +} + +// mapKeys returns a sort.Interface to be used for sorting the map keys. +// Map fields may have key types of non-float scalars, strings and enums. +func mapKeys(vs []reflect.Value) sort.Interface { + s := mapKeySorter{vs: vs} + + // Type specialization per https://developers.google.com/protocol-buffers/docs/proto#maps. + if len(vs) == 0 { + return s + } + switch vs[0].Kind() { + case reflect.Int32, reflect.Int64: + s.less = func(a, b reflect.Value) bool { return a.Int() < b.Int() } + case reflect.Uint32, reflect.Uint64: + s.less = func(a, b reflect.Value) bool { return a.Uint() < b.Uint() } + case reflect.Bool: + s.less = func(a, b reflect.Value) bool { return !a.Bool() && b.Bool() } // false < true + case reflect.String: + s.less = func(a, b reflect.Value) bool { return a.String() < b.String() } + default: + panic(fmt.Sprintf("unsupported map key type: %v", vs[0].Kind())) + } + + return s +} + +type mapKeySorter struct { + vs []reflect.Value + less func(a, b reflect.Value) bool +} + +func (s mapKeySorter) Len() int { return len(s.vs) } +func (s mapKeySorter) Swap(i, j int) { s.vs[i], s.vs[j] = s.vs[j], s.vs[i] } +func (s mapKeySorter) Less(i, j int) bool { + return s.less(s.vs[i], s.vs[j]) +} + +// isProto3Zero reports whether v is a zero proto3 value. +func isProto3Zero(v reflect.Value) bool { + switch v.Kind() { + case reflect.Bool: + return !v.Bool() + case reflect.Int32, reflect.Int64: + return v.Int() == 0 + case reflect.Uint32, reflect.Uint64: + return v.Uint() == 0 + case reflect.Float32, reflect.Float64: + return v.Float() == 0 + case reflect.String: + return v.String() == "" + } + return false +} + +// ProtoPackageIsVersion2 is referenced from generated protocol buffer files +// to assert that that code is compatible with this version of the proto package. +const GoGoProtoPackageIsVersion2 = true + +// ProtoPackageIsVersion1 is referenced from generated protocol buffer files +// to assert that that code is compatible with this version of the proto package. +const GoGoProtoPackageIsVersion1 = true + +// InternalMessageInfo is a type used internally by generated .pb.go files. +// This type is not intended to be used by non-generated code. +// This type is not subject to any compatibility guarantee. +type InternalMessageInfo struct { + marshal *marshalInfo + unmarshal *unmarshalInfo + merge *mergeInfo + discard *discardInfo +} diff --git a/vendor/github.com/gogo/protobuf/proto/lib_gogo.go b/vendor/github.com/gogo/protobuf/proto/lib_gogo.go new file mode 100644 index 0000000000000..b3aa39190a130 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/lib_gogo.go @@ -0,0 +1,50 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "encoding/json" + "strconv" +) + +type Sizer interface { + Size() int +} + +type ProtoSizer interface { + ProtoSize() int +} + +func MarshalJSONEnum(m map[int32]string, value int32) ([]byte, error) { + s, ok := m[value] + if !ok { + s = strconv.Itoa(int(value)) + } + return json.Marshal(s) +} diff --git a/vendor/github.com/gogo/protobuf/proto/message_set.go b/vendor/github.com/gogo/protobuf/proto/message_set.go new file mode 100644 index 0000000000000..3b6ca41d5e554 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/message_set.go @@ -0,0 +1,314 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2010 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +/* + * Support for message sets. + */ + +import ( + "bytes" + "encoding/json" + "errors" + "fmt" + "reflect" + "sort" + "sync" +) + +// errNoMessageTypeID occurs when a protocol buffer does not have a message type ID. +// A message type ID is required for storing a protocol buffer in a message set. +var errNoMessageTypeID = errors.New("proto does not have a message type ID") + +// The first two types (_MessageSet_Item and messageSet) +// model what the protocol compiler produces for the following protocol message: +// message MessageSet { +// repeated group Item = 1 { +// required int32 type_id = 2; +// required string message = 3; +// }; +// } +// That is the MessageSet wire format. We can't use a proto to generate these +// because that would introduce a circular dependency between it and this package. + +type _MessageSet_Item struct { + TypeId *int32 `protobuf:"varint,2,req,name=type_id"` + Message []byte `protobuf:"bytes,3,req,name=message"` +} + +type messageSet struct { + Item []*_MessageSet_Item `protobuf:"group,1,rep"` + XXX_unrecognized []byte + // TODO: caching? +} + +// Make sure messageSet is a Message. +var _ Message = (*messageSet)(nil) + +// messageTypeIder is an interface satisfied by a protocol buffer type +// that may be stored in a MessageSet. +type messageTypeIder interface { + MessageTypeId() int32 +} + +func (ms *messageSet) find(pb Message) *_MessageSet_Item { + mti, ok := pb.(messageTypeIder) + if !ok { + return nil + } + id := mti.MessageTypeId() + for _, item := range ms.Item { + if *item.TypeId == id { + return item + } + } + return nil +} + +func (ms *messageSet) Has(pb Message) bool { + return ms.find(pb) != nil +} + +func (ms *messageSet) Unmarshal(pb Message) error { + if item := ms.find(pb); item != nil { + return Unmarshal(item.Message, pb) + } + if _, ok := pb.(messageTypeIder); !ok { + return errNoMessageTypeID + } + return nil // TODO: return error instead? +} + +func (ms *messageSet) Marshal(pb Message) error { + msg, err := Marshal(pb) + if err != nil { + return err + } + if item := ms.find(pb); item != nil { + // reuse existing item + item.Message = msg + return nil + } + + mti, ok := pb.(messageTypeIder) + if !ok { + return errNoMessageTypeID + } + + mtid := mti.MessageTypeId() + ms.Item = append(ms.Item, &_MessageSet_Item{ + TypeId: &mtid, + Message: msg, + }) + return nil +} + +func (ms *messageSet) Reset() { *ms = messageSet{} } +func (ms *messageSet) String() string { return CompactTextString(ms) } +func (*messageSet) ProtoMessage() {} + +// Support for the message_set_wire_format message option. + +func skipVarint(buf []byte) []byte { + i := 0 + for ; buf[i]&0x80 != 0; i++ { + } + return buf[i+1:] +} + +// MarshalMessageSet encodes the extension map represented by m in the message set wire format. +// It is called by generated Marshal methods on protocol buffer messages with the message_set_wire_format option. +func MarshalMessageSet(exts interface{}) ([]byte, error) { + return marshalMessageSet(exts, false) +} + +// marshaMessageSet implements above function, with the opt to turn on / off deterministic during Marshal. +func marshalMessageSet(exts interface{}, deterministic bool) ([]byte, error) { + switch exts := exts.(type) { + case *XXX_InternalExtensions: + var u marshalInfo + siz := u.sizeMessageSet(exts) + b := make([]byte, 0, siz) + return u.appendMessageSet(b, exts, deterministic) + + case map[int32]Extension: + // This is an old-style extension map. + // Wrap it in a new-style XXX_InternalExtensions. + ie := XXX_InternalExtensions{ + p: &struct { + mu sync.Mutex + extensionMap map[int32]Extension + }{ + extensionMap: exts, + }, + } + + var u marshalInfo + siz := u.sizeMessageSet(&ie) + b := make([]byte, 0, siz) + return u.appendMessageSet(b, &ie, deterministic) + + default: + return nil, errors.New("proto: not an extension map") + } +} + +// UnmarshalMessageSet decodes the extension map encoded in buf in the message set wire format. +// It is called by Unmarshal methods on protocol buffer messages with the message_set_wire_format option. +func UnmarshalMessageSet(buf []byte, exts interface{}) error { + var m map[int32]Extension + switch exts := exts.(type) { + case *XXX_InternalExtensions: + m = exts.extensionsWrite() + case map[int32]Extension: + m = exts + default: + return errors.New("proto: not an extension map") + } + + ms := new(messageSet) + if err := Unmarshal(buf, ms); err != nil { + return err + } + for _, item := range ms.Item { + id := *item.TypeId + msg := item.Message + + // Restore wire type and field number varint, plus length varint. + // Be careful to preserve duplicate items. + b := EncodeVarint(uint64(id)<<3 | WireBytes) + if ext, ok := m[id]; ok { + // Existing data; rip off the tag and length varint + // so we join the new data correctly. + // We can assume that ext.enc is set because we are unmarshaling. + o := ext.enc[len(b):] // skip wire type and field number + _, n := DecodeVarint(o) // calculate length of length varint + o = o[n:] // skip length varint + msg = append(o, msg...) // join old data and new data + } + b = append(b, EncodeVarint(uint64(len(msg)))...) + b = append(b, msg...) + + m[id] = Extension{enc: b} + } + return nil +} + +// MarshalMessageSetJSON encodes the extension map represented by m in JSON format. +// It is called by generated MarshalJSON methods on protocol buffer messages with the message_set_wire_format option. +func MarshalMessageSetJSON(exts interface{}) ([]byte, error) { + var m map[int32]Extension + switch exts := exts.(type) { + case *XXX_InternalExtensions: + var mu sync.Locker + m, mu = exts.extensionsRead() + if m != nil { + // Keep the extensions map locked until we're done marshaling to prevent + // races between marshaling and unmarshaling the lazily-{en,de}coded + // values. + mu.Lock() + defer mu.Unlock() + } + case map[int32]Extension: + m = exts + default: + return nil, errors.New("proto: not an extension map") + } + var b bytes.Buffer + b.WriteByte('{') + + // Process the map in key order for deterministic output. + ids := make([]int32, 0, len(m)) + for id := range m { + ids = append(ids, id) + } + sort.Sort(int32Slice(ids)) // int32Slice defined in text.go + + for i, id := range ids { + ext := m[id] + msd, ok := messageSetMap[id] + if !ok { + // Unknown type; we can't render it, so skip it. + continue + } + + if i > 0 && b.Len() > 1 { + b.WriteByte(',') + } + + fmt.Fprintf(&b, `"[%s]":`, msd.name) + + x := ext.value + if x == nil { + x = reflect.New(msd.t.Elem()).Interface() + if err := Unmarshal(ext.enc, x.(Message)); err != nil { + return nil, err + } + } + d, err := json.Marshal(x) + if err != nil { + return nil, err + } + b.Write(d) + } + b.WriteByte('}') + return b.Bytes(), nil +} + +// UnmarshalMessageSetJSON decodes the extension map encoded in buf in JSON format. +// It is called by generated UnmarshalJSON methods on protocol buffer messages with the message_set_wire_format option. +func UnmarshalMessageSetJSON(buf []byte, exts interface{}) error { + // Common-case fast path. + if len(buf) == 0 || bytes.Equal(buf, []byte("{}")) { + return nil + } + + // This is fairly tricky, and it's not clear that it is needed. + return errors.New("TODO: UnmarshalMessageSetJSON not yet implemented") +} + +// A global registry of types that can be used in a MessageSet. + +var messageSetMap = make(map[int32]messageSetDesc) + +type messageSetDesc struct { + t reflect.Type // pointer to struct + name string +} + +// RegisterMessageSetType is called from the generated code. +func RegisterMessageSetType(m Message, fieldNum int32, name string) { + messageSetMap[fieldNum] = messageSetDesc{ + t: reflect.TypeOf(m), + name: name, + } +} diff --git a/vendor/github.com/gogo/protobuf/proto/pointer_reflect.go b/vendor/github.com/gogo/protobuf/proto/pointer_reflect.go new file mode 100644 index 0000000000000..b6cad90834b31 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/pointer_reflect.go @@ -0,0 +1,357 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2012 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// +build purego appengine js + +// This file contains an implementation of proto field accesses using package reflect. +// It is slower than the code in pointer_unsafe.go but it avoids package unsafe and can +// be used on App Engine. + +package proto + +import ( + "reflect" + "sync" +) + +const unsafeAllowed = false + +// A field identifies a field in a struct, accessible from a pointer. +// In this implementation, a field is identified by the sequence of field indices +// passed to reflect's FieldByIndex. +type field []int + +// toField returns a field equivalent to the given reflect field. +func toField(f *reflect.StructField) field { + return f.Index +} + +// invalidField is an invalid field identifier. +var invalidField = field(nil) + +// zeroField is a noop when calling pointer.offset. +var zeroField = field([]int{}) + +// IsValid reports whether the field identifier is valid. +func (f field) IsValid() bool { return f != nil } + +// The pointer type is for the table-driven decoder. +// The implementation here uses a reflect.Value of pointer type to +// create a generic pointer. In pointer_unsafe.go we use unsafe +// instead of reflect to implement the same (but faster) interface. +type pointer struct { + v reflect.Value +} + +// toPointer converts an interface of pointer type to a pointer +// that points to the same target. +func toPointer(i *Message) pointer { + return pointer{v: reflect.ValueOf(*i)} +} + +// toAddrPointer converts an interface to a pointer that points to +// the interface data. +func toAddrPointer(i *interface{}, isptr bool) pointer { + v := reflect.ValueOf(*i) + u := reflect.New(v.Type()) + u.Elem().Set(v) + return pointer{v: u} +} + +// valToPointer converts v to a pointer. v must be of pointer type. +func valToPointer(v reflect.Value) pointer { + return pointer{v: v} +} + +// offset converts from a pointer to a structure to a pointer to +// one of its fields. +func (p pointer) offset(f field) pointer { + return pointer{v: p.v.Elem().FieldByIndex(f).Addr()} +} + +func (p pointer) isNil() bool { + return p.v.IsNil() +} + +// grow updates the slice s in place to make it one element longer. +// s must be addressable. +// Returns the (addressable) new element. +func grow(s reflect.Value) reflect.Value { + n, m := s.Len(), s.Cap() + if n < m { + s.SetLen(n + 1) + } else { + s.Set(reflect.Append(s, reflect.Zero(s.Type().Elem()))) + } + return s.Index(n) +} + +func (p pointer) toInt64() *int64 { + return p.v.Interface().(*int64) +} +func (p pointer) toInt64Ptr() **int64 { + return p.v.Interface().(**int64) +} +func (p pointer) toInt64Slice() *[]int64 { + return p.v.Interface().(*[]int64) +} + +var int32ptr = reflect.TypeOf((*int32)(nil)) + +func (p pointer) toInt32() *int32 { + return p.v.Convert(int32ptr).Interface().(*int32) +} + +// The toInt32Ptr/Slice methods don't work because of enums. +// Instead, we must use set/get methods for the int32ptr/slice case. +/* + func (p pointer) toInt32Ptr() **int32 { + return p.v.Interface().(**int32) +} + func (p pointer) toInt32Slice() *[]int32 { + return p.v.Interface().(*[]int32) +} +*/ +func (p pointer) getInt32Ptr() *int32 { + if p.v.Type().Elem().Elem() == reflect.TypeOf(int32(0)) { + // raw int32 type + return p.v.Elem().Interface().(*int32) + } + // an enum + return p.v.Elem().Convert(int32PtrType).Interface().(*int32) +} +func (p pointer) setInt32Ptr(v int32) { + // Allocate value in a *int32. Possibly convert that to a *enum. + // Then assign it to a **int32 or **enum. + // Note: we can convert *int32 to *enum, but we can't convert + // **int32 to **enum! + p.v.Elem().Set(reflect.ValueOf(&v).Convert(p.v.Type().Elem())) +} + +// getInt32Slice copies []int32 from p as a new slice. +// This behavior differs from the implementation in pointer_unsafe.go. +func (p pointer) getInt32Slice() []int32 { + if p.v.Type().Elem().Elem() == reflect.TypeOf(int32(0)) { + // raw int32 type + return p.v.Elem().Interface().([]int32) + } + // an enum + // Allocate a []int32, then assign []enum's values into it. + // Note: we can't convert []enum to []int32. + slice := p.v.Elem() + s := make([]int32, slice.Len()) + for i := 0; i < slice.Len(); i++ { + s[i] = int32(slice.Index(i).Int()) + } + return s +} + +// setInt32Slice copies []int32 into p as a new slice. +// This behavior differs from the implementation in pointer_unsafe.go. +func (p pointer) setInt32Slice(v []int32) { + if p.v.Type().Elem().Elem() == reflect.TypeOf(int32(0)) { + // raw int32 type + p.v.Elem().Set(reflect.ValueOf(v)) + return + } + // an enum + // Allocate a []enum, then assign []int32's values into it. + // Note: we can't convert []enum to []int32. + slice := reflect.MakeSlice(p.v.Type().Elem(), len(v), cap(v)) + for i, x := range v { + slice.Index(i).SetInt(int64(x)) + } + p.v.Elem().Set(slice) +} +func (p pointer) appendInt32Slice(v int32) { + grow(p.v.Elem()).SetInt(int64(v)) +} + +func (p pointer) toUint64() *uint64 { + return p.v.Interface().(*uint64) +} +func (p pointer) toUint64Ptr() **uint64 { + return p.v.Interface().(**uint64) +} +func (p pointer) toUint64Slice() *[]uint64 { + return p.v.Interface().(*[]uint64) +} +func (p pointer) toUint32() *uint32 { + return p.v.Interface().(*uint32) +} +func (p pointer) toUint32Ptr() **uint32 { + return p.v.Interface().(**uint32) +} +func (p pointer) toUint32Slice() *[]uint32 { + return p.v.Interface().(*[]uint32) +} +func (p pointer) toBool() *bool { + return p.v.Interface().(*bool) +} +func (p pointer) toBoolPtr() **bool { + return p.v.Interface().(**bool) +} +func (p pointer) toBoolSlice() *[]bool { + return p.v.Interface().(*[]bool) +} +func (p pointer) toFloat64() *float64 { + return p.v.Interface().(*float64) +} +func (p pointer) toFloat64Ptr() **float64 { + return p.v.Interface().(**float64) +} +func (p pointer) toFloat64Slice() *[]float64 { + return p.v.Interface().(*[]float64) +} +func (p pointer) toFloat32() *float32 { + return p.v.Interface().(*float32) +} +func (p pointer) toFloat32Ptr() **float32 { + return p.v.Interface().(**float32) +} +func (p pointer) toFloat32Slice() *[]float32 { + return p.v.Interface().(*[]float32) +} +func (p pointer) toString() *string { + return p.v.Interface().(*string) +} +func (p pointer) toStringPtr() **string { + return p.v.Interface().(**string) +} +func (p pointer) toStringSlice() *[]string { + return p.v.Interface().(*[]string) +} +func (p pointer) toBytes() *[]byte { + return p.v.Interface().(*[]byte) +} +func (p pointer) toBytesSlice() *[][]byte { + return p.v.Interface().(*[][]byte) +} +func (p pointer) toExtensions() *XXX_InternalExtensions { + return p.v.Interface().(*XXX_InternalExtensions) +} +func (p pointer) toOldExtensions() *map[int32]Extension { + return p.v.Interface().(*map[int32]Extension) +} +func (p pointer) getPointer() pointer { + return pointer{v: p.v.Elem()} +} +func (p pointer) setPointer(q pointer) { + p.v.Elem().Set(q.v) +} +func (p pointer) appendPointer(q pointer) { + grow(p.v.Elem()).Set(q.v) +} + +// getPointerSlice copies []*T from p as a new []pointer. +// This behavior differs from the implementation in pointer_unsafe.go. +func (p pointer) getPointerSlice() []pointer { + if p.v.IsNil() { + return nil + } + n := p.v.Elem().Len() + s := make([]pointer, n) + for i := 0; i < n; i++ { + s[i] = pointer{v: p.v.Elem().Index(i)} + } + return s +} + +// setPointerSlice copies []pointer into p as a new []*T. +// This behavior differs from the implementation in pointer_unsafe.go. +func (p pointer) setPointerSlice(v []pointer) { + if v == nil { + p.v.Elem().Set(reflect.New(p.v.Elem().Type()).Elem()) + return + } + s := reflect.MakeSlice(p.v.Elem().Type(), 0, len(v)) + for _, p := range v { + s = reflect.Append(s, p.v) + } + p.v.Elem().Set(s) +} + +// getInterfacePointer returns a pointer that points to the +// interface data of the interface pointed by p. +func (p pointer) getInterfacePointer() pointer { + if p.v.Elem().IsNil() { + return pointer{v: p.v.Elem()} + } + return pointer{v: p.v.Elem().Elem().Elem().Field(0).Addr()} // *interface -> interface -> *struct -> struct +} + +func (p pointer) asPointerTo(t reflect.Type) reflect.Value { + // TODO: check that p.v.Type().Elem() == t? + return p.v +} + +func atomicLoadUnmarshalInfo(p **unmarshalInfo) *unmarshalInfo { + atomicLock.Lock() + defer atomicLock.Unlock() + return *p +} +func atomicStoreUnmarshalInfo(p **unmarshalInfo, v *unmarshalInfo) { + atomicLock.Lock() + defer atomicLock.Unlock() + *p = v +} +func atomicLoadMarshalInfo(p **marshalInfo) *marshalInfo { + atomicLock.Lock() + defer atomicLock.Unlock() + return *p +} +func atomicStoreMarshalInfo(p **marshalInfo, v *marshalInfo) { + atomicLock.Lock() + defer atomicLock.Unlock() + *p = v +} +func atomicLoadMergeInfo(p **mergeInfo) *mergeInfo { + atomicLock.Lock() + defer atomicLock.Unlock() + return *p +} +func atomicStoreMergeInfo(p **mergeInfo, v *mergeInfo) { + atomicLock.Lock() + defer atomicLock.Unlock() + *p = v +} +func atomicLoadDiscardInfo(p **discardInfo) *discardInfo { + atomicLock.Lock() + defer atomicLock.Unlock() + return *p +} +func atomicStoreDiscardInfo(p **discardInfo, v *discardInfo) { + atomicLock.Lock() + defer atomicLock.Unlock() + *p = v +} + +var atomicLock sync.Mutex diff --git a/vendor/github.com/gogo/protobuf/proto/pointer_reflect_gogo.go b/vendor/github.com/gogo/protobuf/proto/pointer_reflect_gogo.go new file mode 100644 index 0000000000000..7ffd3c29d90c3 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/pointer_reflect_gogo.go @@ -0,0 +1,59 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2018, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// +build purego appengine js + +// This file contains an implementation of proto field accesses using package reflect. +// It is slower than the code in pointer_unsafe.go but it avoids package unsafe and can +// be used on App Engine. + +package proto + +import ( + "reflect" +) + +// TODO: untested, so probably incorrect. + +func (p pointer) getRef() pointer { + return pointer{v: p.v.Addr()} +} + +func (p pointer) appendRef(v pointer, typ reflect.Type) { + slice := p.getSlice(typ) + elem := v.asPointerTo(typ).Elem() + newSlice := reflect.Append(slice, elem) + slice.Set(newSlice) +} + +func (p pointer) getSlice(typ reflect.Type) reflect.Value { + sliceTyp := reflect.SliceOf(typ) + slice := p.asPointerTo(sliceTyp) + slice = slice.Elem() + return slice +} diff --git a/vendor/github.com/gogo/protobuf/proto/pointer_unsafe.go b/vendor/github.com/gogo/protobuf/proto/pointer_unsafe.go new file mode 100644 index 0000000000000..d55a335d94532 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/pointer_unsafe.go @@ -0,0 +1,308 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2012 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// +build !purego,!appengine,!js + +// This file contains the implementation of the proto field accesses using package unsafe. + +package proto + +import ( + "reflect" + "sync/atomic" + "unsafe" +) + +const unsafeAllowed = true + +// A field identifies a field in a struct, accessible from a pointer. +// In this implementation, a field is identified by its byte offset from the start of the struct. +type field uintptr + +// toField returns a field equivalent to the given reflect field. +func toField(f *reflect.StructField) field { + return field(f.Offset) +} + +// invalidField is an invalid field identifier. +const invalidField = ^field(0) + +// zeroField is a noop when calling pointer.offset. +const zeroField = field(0) + +// IsValid reports whether the field identifier is valid. +func (f field) IsValid() bool { + return f != invalidField +} + +// The pointer type below is for the new table-driven encoder/decoder. +// The implementation here uses unsafe.Pointer to create a generic pointer. +// In pointer_reflect.go we use reflect instead of unsafe to implement +// the same (but slower) interface. +type pointer struct { + p unsafe.Pointer +} + +// size of pointer +var ptrSize = unsafe.Sizeof(uintptr(0)) + +// toPointer converts an interface of pointer type to a pointer +// that points to the same target. +func toPointer(i *Message) pointer { + // Super-tricky - read pointer out of data word of interface value. + // Saves ~25ns over the equivalent: + // return valToPointer(reflect.ValueOf(*i)) + return pointer{p: (*[2]unsafe.Pointer)(unsafe.Pointer(i))[1]} +} + +// toAddrPointer converts an interface to a pointer that points to +// the interface data. +func toAddrPointer(i *interface{}, isptr bool) pointer { + // Super-tricky - read or get the address of data word of interface value. + if isptr { + // The interface is of pointer type, thus it is a direct interface. + // The data word is the pointer data itself. We take its address. + return pointer{p: unsafe.Pointer(uintptr(unsafe.Pointer(i)) + ptrSize)} + } + // The interface is not of pointer type. The data word is the pointer + // to the data. + return pointer{p: (*[2]unsafe.Pointer)(unsafe.Pointer(i))[1]} +} + +// valToPointer converts v to a pointer. v must be of pointer type. +func valToPointer(v reflect.Value) pointer { + return pointer{p: unsafe.Pointer(v.Pointer())} +} + +// offset converts from a pointer to a structure to a pointer to +// one of its fields. +func (p pointer) offset(f field) pointer { + // For safety, we should panic if !f.IsValid, however calling panic causes + // this to no longer be inlineable, which is a serious performance cost. + /* + if !f.IsValid() { + panic("invalid field") + } + */ + return pointer{p: unsafe.Pointer(uintptr(p.p) + uintptr(f))} +} + +func (p pointer) isNil() bool { + return p.p == nil +} + +func (p pointer) toInt64() *int64 { + return (*int64)(p.p) +} +func (p pointer) toInt64Ptr() **int64 { + return (**int64)(p.p) +} +func (p pointer) toInt64Slice() *[]int64 { + return (*[]int64)(p.p) +} +func (p pointer) toInt32() *int32 { + return (*int32)(p.p) +} + +// See pointer_reflect.go for why toInt32Ptr/Slice doesn't exist. +/* + func (p pointer) toInt32Ptr() **int32 { + return (**int32)(p.p) + } + func (p pointer) toInt32Slice() *[]int32 { + return (*[]int32)(p.p) + } +*/ +func (p pointer) getInt32Ptr() *int32 { + return *(**int32)(p.p) +} +func (p pointer) setInt32Ptr(v int32) { + *(**int32)(p.p) = &v +} + +// getInt32Slice loads a []int32 from p. +// The value returned is aliased with the original slice. +// This behavior differs from the implementation in pointer_reflect.go. +func (p pointer) getInt32Slice() []int32 { + return *(*[]int32)(p.p) +} + +// setInt32Slice stores a []int32 to p. +// The value set is aliased with the input slice. +// This behavior differs from the implementation in pointer_reflect.go. +func (p pointer) setInt32Slice(v []int32) { + *(*[]int32)(p.p) = v +} + +// TODO: Can we get rid of appendInt32Slice and use setInt32Slice instead? +func (p pointer) appendInt32Slice(v int32) { + s := (*[]int32)(p.p) + *s = append(*s, v) +} + +func (p pointer) toUint64() *uint64 { + return (*uint64)(p.p) +} +func (p pointer) toUint64Ptr() **uint64 { + return (**uint64)(p.p) +} +func (p pointer) toUint64Slice() *[]uint64 { + return (*[]uint64)(p.p) +} +func (p pointer) toUint32() *uint32 { + return (*uint32)(p.p) +} +func (p pointer) toUint32Ptr() **uint32 { + return (**uint32)(p.p) +} +func (p pointer) toUint32Slice() *[]uint32 { + return (*[]uint32)(p.p) +} +func (p pointer) toBool() *bool { + return (*bool)(p.p) +} +func (p pointer) toBoolPtr() **bool { + return (**bool)(p.p) +} +func (p pointer) toBoolSlice() *[]bool { + return (*[]bool)(p.p) +} +func (p pointer) toFloat64() *float64 { + return (*float64)(p.p) +} +func (p pointer) toFloat64Ptr() **float64 { + return (**float64)(p.p) +} +func (p pointer) toFloat64Slice() *[]float64 { + return (*[]float64)(p.p) +} +func (p pointer) toFloat32() *float32 { + return (*float32)(p.p) +} +func (p pointer) toFloat32Ptr() **float32 { + return (**float32)(p.p) +} +func (p pointer) toFloat32Slice() *[]float32 { + return (*[]float32)(p.p) +} +func (p pointer) toString() *string { + return (*string)(p.p) +} +func (p pointer) toStringPtr() **string { + return (**string)(p.p) +} +func (p pointer) toStringSlice() *[]string { + return (*[]string)(p.p) +} +func (p pointer) toBytes() *[]byte { + return (*[]byte)(p.p) +} +func (p pointer) toBytesSlice() *[][]byte { + return (*[][]byte)(p.p) +} +func (p pointer) toExtensions() *XXX_InternalExtensions { + return (*XXX_InternalExtensions)(p.p) +} +func (p pointer) toOldExtensions() *map[int32]Extension { + return (*map[int32]Extension)(p.p) +} + +// getPointerSlice loads []*T from p as a []pointer. +// The value returned is aliased with the original slice. +// This behavior differs from the implementation in pointer_reflect.go. +func (p pointer) getPointerSlice() []pointer { + // Super-tricky - p should point to a []*T where T is a + // message type. We load it as []pointer. + return *(*[]pointer)(p.p) +} + +// setPointerSlice stores []pointer into p as a []*T. +// The value set is aliased with the input slice. +// This behavior differs from the implementation in pointer_reflect.go. +func (p pointer) setPointerSlice(v []pointer) { + // Super-tricky - p should point to a []*T where T is a + // message type. We store it as []pointer. + *(*[]pointer)(p.p) = v +} + +// getPointer loads the pointer at p and returns it. +func (p pointer) getPointer() pointer { + return pointer{p: *(*unsafe.Pointer)(p.p)} +} + +// setPointer stores the pointer q at p. +func (p pointer) setPointer(q pointer) { + *(*unsafe.Pointer)(p.p) = q.p +} + +// append q to the slice pointed to by p. +func (p pointer) appendPointer(q pointer) { + s := (*[]unsafe.Pointer)(p.p) + *s = append(*s, q.p) +} + +// getInterfacePointer returns a pointer that points to the +// interface data of the interface pointed by p. +func (p pointer) getInterfacePointer() pointer { + // Super-tricky - read pointer out of data word of interface value. + return pointer{p: (*(*[2]unsafe.Pointer)(p.p))[1]} +} + +// asPointerTo returns a reflect.Value that is a pointer to an +// object of type t stored at p. +func (p pointer) asPointerTo(t reflect.Type) reflect.Value { + return reflect.NewAt(t, p.p) +} + +func atomicLoadUnmarshalInfo(p **unmarshalInfo) *unmarshalInfo { + return (*unmarshalInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(p)))) +} +func atomicStoreUnmarshalInfo(p **unmarshalInfo, v *unmarshalInfo) { + atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(p)), unsafe.Pointer(v)) +} +func atomicLoadMarshalInfo(p **marshalInfo) *marshalInfo { + return (*marshalInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(p)))) +} +func atomicStoreMarshalInfo(p **marshalInfo, v *marshalInfo) { + atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(p)), unsafe.Pointer(v)) +} +func atomicLoadMergeInfo(p **mergeInfo) *mergeInfo { + return (*mergeInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(p)))) +} +func atomicStoreMergeInfo(p **mergeInfo, v *mergeInfo) { + atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(p)), unsafe.Pointer(v)) +} +func atomicLoadDiscardInfo(p **discardInfo) *discardInfo { + return (*discardInfo)(atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(p)))) +} +func atomicStoreDiscardInfo(p **discardInfo, v *discardInfo) { + atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(p)), unsafe.Pointer(v)) +} diff --git a/vendor/github.com/gogo/protobuf/proto/pointer_unsafe_gogo.go b/vendor/github.com/gogo/protobuf/proto/pointer_unsafe_gogo.go new file mode 100644 index 0000000000000..b354101b9c3d5 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/pointer_unsafe_gogo.go @@ -0,0 +1,56 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2018, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// +build !purego !appengine,!js + +// This file contains the implementation of the proto field accesses using package unsafe. + +package proto + +import ( + "reflect" + "unsafe" +) + +func (p pointer) getRef() pointer { + return pointer{p: (unsafe.Pointer)(&p.p)} +} + +func (p pointer) appendRef(v pointer, typ reflect.Type) { + slice := p.getSlice(typ) + elem := v.asPointerTo(typ).Elem() + newSlice := reflect.Append(slice, elem) + slice.Set(newSlice) +} + +func (p pointer) getSlice(typ reflect.Type) reflect.Value { + sliceTyp := reflect.SliceOf(typ) + slice := p.asPointerTo(sliceTyp) + slice = slice.Elem() + return slice +} diff --git a/vendor/github.com/gogo/protobuf/proto/properties.go b/vendor/github.com/gogo/protobuf/proto/properties.go new file mode 100644 index 0000000000000..7a5e28efe5199 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/properties.go @@ -0,0 +1,600 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2010 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +/* + * Routines for encoding data into the wire format for protocol buffers. + */ + +import ( + "fmt" + "log" + "os" + "reflect" + "sort" + "strconv" + "strings" + "sync" +) + +const debug bool = false + +// Constants that identify the encoding of a value on the wire. +const ( + WireVarint = 0 + WireFixed64 = 1 + WireBytes = 2 + WireStartGroup = 3 + WireEndGroup = 4 + WireFixed32 = 5 +) + +// tagMap is an optimization over map[int]int for typical protocol buffer +// use-cases. Encoded protocol buffers are often in tag order with small tag +// numbers. +type tagMap struct { + fastTags []int + slowTags map[int]int +} + +// tagMapFastLimit is the upper bound on the tag number that will be stored in +// the tagMap slice rather than its map. +const tagMapFastLimit = 1024 + +func (p *tagMap) get(t int) (int, bool) { + if t > 0 && t < tagMapFastLimit { + if t >= len(p.fastTags) { + return 0, false + } + fi := p.fastTags[t] + return fi, fi >= 0 + } + fi, ok := p.slowTags[t] + return fi, ok +} + +func (p *tagMap) put(t int, fi int) { + if t > 0 && t < tagMapFastLimit { + for len(p.fastTags) < t+1 { + p.fastTags = append(p.fastTags, -1) + } + p.fastTags[t] = fi + return + } + if p.slowTags == nil { + p.slowTags = make(map[int]int) + } + p.slowTags[t] = fi +} + +// StructProperties represents properties for all the fields of a struct. +// decoderTags and decoderOrigNames should only be used by the decoder. +type StructProperties struct { + Prop []*Properties // properties for each field + reqCount int // required count + decoderTags tagMap // map from proto tag to struct field number + decoderOrigNames map[string]int // map from original name to struct field number + order []int // list of struct field numbers in tag order + + // OneofTypes contains information about the oneof fields in this message. + // It is keyed by the original name of a field. + OneofTypes map[string]*OneofProperties +} + +// OneofProperties represents information about a specific field in a oneof. +type OneofProperties struct { + Type reflect.Type // pointer to generated struct type for this oneof field + Field int // struct field number of the containing oneof in the message + Prop *Properties +} + +// Implement the sorting interface so we can sort the fields in tag order, as recommended by the spec. +// See encode.go, (*Buffer).enc_struct. + +func (sp *StructProperties) Len() int { return len(sp.order) } +func (sp *StructProperties) Less(i, j int) bool { + return sp.Prop[sp.order[i]].Tag < sp.Prop[sp.order[j]].Tag +} +func (sp *StructProperties) Swap(i, j int) { sp.order[i], sp.order[j] = sp.order[j], sp.order[i] } + +// Properties represents the protocol-specific behavior of a single struct field. +type Properties struct { + Name string // name of the field, for error messages + OrigName string // original name before protocol compiler (always set) + JSONName string // name to use for JSON; determined by protoc + Wire string + WireType int + Tag int + Required bool + Optional bool + Repeated bool + Packed bool // relevant for repeated primitives only + Enum string // set for enum types only + proto3 bool // whether this is known to be a proto3 field; set for []byte only + oneof bool // whether this is a oneof field + + Default string // default value + HasDefault bool // whether an explicit default was provided + CustomType string + CastType string + StdTime bool + StdDuration bool + + stype reflect.Type // set for struct types only + ctype reflect.Type // set for custom types only + sprop *StructProperties // set for struct types only + + mtype reflect.Type // set for map types only + mkeyprop *Properties // set for map types only + mvalprop *Properties // set for map types only +} + +// String formats the properties in the protobuf struct field tag style. +func (p *Properties) String() string { + s := p.Wire + s += "," + s += strconv.Itoa(p.Tag) + if p.Required { + s += ",req" + } + if p.Optional { + s += ",opt" + } + if p.Repeated { + s += ",rep" + } + if p.Packed { + s += ",packed" + } + s += ",name=" + p.OrigName + if p.JSONName != p.OrigName { + s += ",json=" + p.JSONName + } + if p.proto3 { + s += ",proto3" + } + if p.oneof { + s += ",oneof" + } + if len(p.Enum) > 0 { + s += ",enum=" + p.Enum + } + if p.HasDefault { + s += ",def=" + p.Default + } + return s +} + +// Parse populates p by parsing a string in the protobuf struct field tag style. +func (p *Properties) Parse(s string) { + // "bytes,49,opt,name=foo,def=hello!" + fields := strings.Split(s, ",") // breaks def=, but handled below. + if len(fields) < 2 { + fmt.Fprintf(os.Stderr, "proto: tag has too few fields: %q\n", s) + return + } + + p.Wire = fields[0] + switch p.Wire { + case "varint": + p.WireType = WireVarint + case "fixed32": + p.WireType = WireFixed32 + case "fixed64": + p.WireType = WireFixed64 + case "zigzag32": + p.WireType = WireVarint + case "zigzag64": + p.WireType = WireVarint + case "bytes", "group": + p.WireType = WireBytes + // no numeric converter for non-numeric types + default: + fmt.Fprintf(os.Stderr, "proto: tag has unknown wire type: %q\n", s) + return + } + + var err error + p.Tag, err = strconv.Atoi(fields[1]) + if err != nil { + return + } + +outer: + for i := 2; i < len(fields); i++ { + f := fields[i] + switch { + case f == "req": + p.Required = true + case f == "opt": + p.Optional = true + case f == "rep": + p.Repeated = true + case f == "packed": + p.Packed = true + case strings.HasPrefix(f, "name="): + p.OrigName = f[5:] + case strings.HasPrefix(f, "json="): + p.JSONName = f[5:] + case strings.HasPrefix(f, "enum="): + p.Enum = f[5:] + case f == "proto3": + p.proto3 = true + case f == "oneof": + p.oneof = true + case strings.HasPrefix(f, "def="): + p.HasDefault = true + p.Default = f[4:] // rest of string + if i+1 < len(fields) { + // Commas aren't escaped, and def is always last. + p.Default += "," + strings.Join(fields[i+1:], ",") + break outer + } + case strings.HasPrefix(f, "embedded="): + p.OrigName = strings.Split(f, "=")[1] + case strings.HasPrefix(f, "customtype="): + p.CustomType = strings.Split(f, "=")[1] + case strings.HasPrefix(f, "casttype="): + p.CastType = strings.Split(f, "=")[1] + case f == "stdtime": + p.StdTime = true + case f == "stdduration": + p.StdDuration = true + } + } +} + +var protoMessageType = reflect.TypeOf((*Message)(nil)).Elem() + +// setFieldProps initializes the field properties for submessages and maps. +func (p *Properties) setFieldProps(typ reflect.Type, f *reflect.StructField, lockGetProp bool) { + isMap := typ.Kind() == reflect.Map + if len(p.CustomType) > 0 && !isMap { + p.ctype = typ + p.setTag(lockGetProp) + return + } + if p.StdTime && !isMap { + p.setTag(lockGetProp) + return + } + if p.StdDuration && !isMap { + p.setTag(lockGetProp) + return + } + switch t1 := typ; t1.Kind() { + case reflect.Struct: + p.stype = typ + case reflect.Ptr: + if t1.Elem().Kind() == reflect.Struct { + p.stype = t1.Elem() + } + case reflect.Slice: + switch t2 := t1.Elem(); t2.Kind() { + case reflect.Ptr: + switch t3 := t2.Elem(); t3.Kind() { + case reflect.Struct: + p.stype = t3 + } + case reflect.Struct: + p.stype = t2 + } + + case reflect.Map: + + p.mtype = t1 + p.mkeyprop = &Properties{} + p.mkeyprop.init(reflect.PtrTo(p.mtype.Key()), "Key", f.Tag.Get("protobuf_key"), nil, lockGetProp) + p.mvalprop = &Properties{} + vtype := p.mtype.Elem() + if vtype.Kind() != reflect.Ptr && vtype.Kind() != reflect.Slice { + // The value type is not a message (*T) or bytes ([]byte), + // so we need encoders for the pointer to this type. + vtype = reflect.PtrTo(vtype) + } + + p.mvalprop.CustomType = p.CustomType + p.mvalprop.StdDuration = p.StdDuration + p.mvalprop.StdTime = p.StdTime + p.mvalprop.init(vtype, "Value", f.Tag.Get("protobuf_val"), nil, lockGetProp) + } + p.setTag(lockGetProp) +} + +func (p *Properties) setTag(lockGetProp bool) { + if p.stype != nil { + if lockGetProp { + p.sprop = GetProperties(p.stype) + } else { + p.sprop = getPropertiesLocked(p.stype) + } + } +} + +var ( + marshalerType = reflect.TypeOf((*Marshaler)(nil)).Elem() +) + +// Init populates the properties from a protocol buffer struct tag. +func (p *Properties) Init(typ reflect.Type, name, tag string, f *reflect.StructField) { + p.init(typ, name, tag, f, true) +} + +func (p *Properties) init(typ reflect.Type, name, tag string, f *reflect.StructField, lockGetProp bool) { + // "bytes,49,opt,def=hello!" + p.Name = name + p.OrigName = name + if tag == "" { + return + } + p.Parse(tag) + p.setFieldProps(typ, f, lockGetProp) +} + +var ( + propertiesMu sync.RWMutex + propertiesMap = make(map[reflect.Type]*StructProperties) +) + +// GetProperties returns the list of properties for the type represented by t. +// t must represent a generated struct type of a protocol message. +func GetProperties(t reflect.Type) *StructProperties { + if t.Kind() != reflect.Struct { + panic("proto: type must have kind struct") + } + + // Most calls to GetProperties in a long-running program will be + // retrieving details for types we have seen before. + propertiesMu.RLock() + sprop, ok := propertiesMap[t] + propertiesMu.RUnlock() + if ok { + if collectStats { + stats.Chit++ + } + return sprop + } + + propertiesMu.Lock() + sprop = getPropertiesLocked(t) + propertiesMu.Unlock() + return sprop +} + +// getPropertiesLocked requires that propertiesMu is held. +func getPropertiesLocked(t reflect.Type) *StructProperties { + if prop, ok := propertiesMap[t]; ok { + if collectStats { + stats.Chit++ + } + return prop + } + if collectStats { + stats.Cmiss++ + } + + prop := new(StructProperties) + // in case of recursive protos, fill this in now. + propertiesMap[t] = prop + + // build properties + prop.Prop = make([]*Properties, t.NumField()) + prop.order = make([]int, t.NumField()) + + isOneofMessage := false + for i := 0; i < t.NumField(); i++ { + f := t.Field(i) + p := new(Properties) + name := f.Name + p.init(f.Type, name, f.Tag.Get("protobuf"), &f, false) + + oneof := f.Tag.Get("protobuf_oneof") // special case + if oneof != "" { + isOneofMessage = true + // Oneof fields don't use the traditional protobuf tag. + p.OrigName = oneof + } + prop.Prop[i] = p + prop.order[i] = i + if debug { + print(i, " ", f.Name, " ", t.String(), " ") + if p.Tag > 0 { + print(p.String()) + } + print("\n") + } + } + + // Re-order prop.order. + sort.Sort(prop) + + type oneofMessage interface { + XXX_OneofFuncs() (func(Message, *Buffer) error, func(Message, int, int, *Buffer) (bool, error), func(Message) int, []interface{}) + } + if om, ok := reflect.Zero(reflect.PtrTo(t)).Interface().(oneofMessage); isOneofMessage && ok { + var oots []interface{} + _, _, _, oots = om.XXX_OneofFuncs() + + // Interpret oneof metadata. + prop.OneofTypes = make(map[string]*OneofProperties) + for _, oot := range oots { + oop := &OneofProperties{ + Type: reflect.ValueOf(oot).Type(), // *T + Prop: new(Properties), + } + sft := oop.Type.Elem().Field(0) + oop.Prop.Name = sft.Name + oop.Prop.Parse(sft.Tag.Get("protobuf")) + // There will be exactly one interface field that + // this new value is assignable to. + for i := 0; i < t.NumField(); i++ { + f := t.Field(i) + if f.Type.Kind() != reflect.Interface { + continue + } + if !oop.Type.AssignableTo(f.Type) { + continue + } + oop.Field = i + break + } + prop.OneofTypes[oop.Prop.OrigName] = oop + } + } + + // build required counts + // build tags + reqCount := 0 + prop.decoderOrigNames = make(map[string]int) + for i, p := range prop.Prop { + if strings.HasPrefix(p.Name, "XXX_") { + // Internal fields should not appear in tags/origNames maps. + // They are handled specially when encoding and decoding. + continue + } + if p.Required { + reqCount++ + } + prop.decoderTags.put(p.Tag, i) + prop.decoderOrigNames[p.OrigName] = i + } + prop.reqCount = reqCount + + return prop +} + +// A global registry of enum types. +// The generated code will register the generated maps by calling RegisterEnum. + +var enumValueMaps = make(map[string]map[string]int32) +var enumStringMaps = make(map[string]map[int32]string) + +// RegisterEnum is called from the generated code to install the enum descriptor +// maps into the global table to aid parsing text format protocol buffers. +func RegisterEnum(typeName string, unusedNameMap map[int32]string, valueMap map[string]int32) { + if _, ok := enumValueMaps[typeName]; ok { + panic("proto: duplicate enum registered: " + typeName) + } + enumValueMaps[typeName] = valueMap + if _, ok := enumStringMaps[typeName]; ok { + panic("proto: duplicate enum registered: " + typeName) + } + enumStringMaps[typeName] = unusedNameMap +} + +// EnumValueMap returns the mapping from names to integers of the +// enum type enumType, or a nil if not found. +func EnumValueMap(enumType string) map[string]int32 { + return enumValueMaps[enumType] +} + +// A registry of all linked message types. +// The string is a fully-qualified proto name ("pkg.Message"). +var ( + protoTypedNils = make(map[string]Message) // a map from proto names to typed nil pointers + protoMapTypes = make(map[string]reflect.Type) // a map from proto names to map types + revProtoTypes = make(map[reflect.Type]string) +) + +// RegisterType is called from generated code and maps from the fully qualified +// proto name to the type (pointer to struct) of the protocol buffer. +func RegisterType(x Message, name string) { + if _, ok := protoTypedNils[name]; ok { + // TODO: Some day, make this a panic. + log.Printf("proto: duplicate proto type registered: %s", name) + return + } + t := reflect.TypeOf(x) + if v := reflect.ValueOf(x); v.Kind() == reflect.Ptr && v.Pointer() == 0 { + // Generated code always calls RegisterType with nil x. + // This check is just for extra safety. + protoTypedNils[name] = x + } else { + protoTypedNils[name] = reflect.Zero(t).Interface().(Message) + } + revProtoTypes[t] = name +} + +// RegisterMapType is called from generated code and maps from the fully qualified +// proto name to the native map type of the proto map definition. +func RegisterMapType(x interface{}, name string) { + if reflect.TypeOf(x).Kind() != reflect.Map { + panic(fmt.Sprintf("RegisterMapType(%T, %q); want map", x, name)) + } + if _, ok := protoMapTypes[name]; ok { + log.Printf("proto: duplicate proto type registered: %s", name) + return + } + t := reflect.TypeOf(x) + protoMapTypes[name] = t + revProtoTypes[t] = name +} + +// MessageName returns the fully-qualified proto name for the given message type. +func MessageName(x Message) string { + type xname interface { + XXX_MessageName() string + } + if m, ok := x.(xname); ok { + return m.XXX_MessageName() + } + return revProtoTypes[reflect.TypeOf(x)] +} + +// MessageType returns the message type (pointer to struct) for a named message. +// The type is not guaranteed to implement proto.Message if the name refers to a +// map entry. +func MessageType(name string) reflect.Type { + if t, ok := protoTypedNils[name]; ok { + return reflect.TypeOf(t) + } + return protoMapTypes[name] +} + +// A registry of all linked proto files. +var ( + protoFiles = make(map[string][]byte) // file name => fileDescriptor +) + +// RegisterFile is called from generated code and maps from the +// full file name of a .proto file to its compressed FileDescriptorProto. +func RegisterFile(filename string, fileDescriptor []byte) { + protoFiles[filename] = fileDescriptor +} + +// FileDescriptor returns the compressed FileDescriptorProto for a .proto file. +func FileDescriptor(filename string) []byte { return protoFiles[filename] } diff --git a/vendor/github.com/gogo/protobuf/proto/properties_gogo.go b/vendor/github.com/gogo/protobuf/proto/properties_gogo.go new file mode 100644 index 0000000000000..40ea3dd935c27 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/properties_gogo.go @@ -0,0 +1,36 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2018, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "reflect" +) + +var sizerType = reflect.TypeOf((*Sizer)(nil)).Elem() +var protosizerType = reflect.TypeOf((*ProtoSizer)(nil)).Elem() diff --git a/vendor/github.com/gogo/protobuf/proto/skip_gogo.go b/vendor/github.com/gogo/protobuf/proto/skip_gogo.go new file mode 100644 index 0000000000000..5a5fd93f7c196 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/skip_gogo.go @@ -0,0 +1,119 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "fmt" + "io" +) + +func Skip(data []byte) (n int, err error) { + l := len(data) + index := 0 + for index < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if index >= l { + return 0, io.ErrUnexpectedEOF + } + b := data[index] + index++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for { + if index >= l { + return 0, io.ErrUnexpectedEOF + } + index++ + if data[index-1] < 0x80 { + break + } + } + return index, nil + case 1: + index += 8 + return index, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if index >= l { + return 0, io.ErrUnexpectedEOF + } + b := data[index] + index++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + index += length + return index, nil + case 3: + for { + var innerWire uint64 + var start int = index + for shift := uint(0); ; shift += 7 { + if index >= l { + return 0, io.ErrUnexpectedEOF + } + b := data[index] + index++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := Skip(data[start:]) + if err != nil { + return 0, err + } + index = start + next + } + return index, nil + case 4: + return index, nil + case 5: + index += 4 + return index, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} diff --git a/vendor/github.com/gogo/protobuf/proto/table_marshal.go b/vendor/github.com/gogo/protobuf/proto/table_marshal.go new file mode 100644 index 0000000000000..255e7b508860c --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/table_marshal.go @@ -0,0 +1,2799 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2016 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "errors" + "fmt" + "math" + "reflect" + "sort" + "strconv" + "strings" + "sync" + "sync/atomic" + "unicode/utf8" +) + +// a sizer takes a pointer to a field and the size of its tag, computes the size of +// the encoded data. +type sizer func(pointer, int) int + +// a marshaler takes a byte slice, a pointer to a field, and its tag (in wire format), +// marshals the field to the end of the slice, returns the slice and error (if any). +type marshaler func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) + +// marshalInfo is the information used for marshaling a message. +type marshalInfo struct { + typ reflect.Type + fields []*marshalFieldInfo + unrecognized field // offset of XXX_unrecognized + extensions field // offset of XXX_InternalExtensions + v1extensions field // offset of XXX_extensions + sizecache field // offset of XXX_sizecache + initialized int32 // 0 -- only typ is set, 1 -- fully initialized + messageset bool // uses message set wire format + hasmarshaler bool // has custom marshaler + sync.RWMutex // protect extElems map, also for initialization + extElems map[int32]*marshalElemInfo // info of extension elements + + hassizer bool // has custom sizer + hasprotosizer bool // has custom protosizer + + bytesExtensions field // offset of XXX_extensions where the field type is []byte +} + +// marshalFieldInfo is the information used for marshaling a field of a message. +type marshalFieldInfo struct { + field field + wiretag uint64 // tag in wire format + tagsize int // size of tag in wire format + sizer sizer + marshaler marshaler + isPointer bool + required bool // field is required + name string // name of the field, for error reporting + oneofElems map[reflect.Type]*marshalElemInfo // info of oneof elements +} + +// marshalElemInfo is the information used for marshaling an extension or oneof element. +type marshalElemInfo struct { + wiretag uint64 // tag in wire format + tagsize int // size of tag in wire format + sizer sizer + marshaler marshaler + isptr bool // elem is pointer typed, thus interface of this type is a direct interface (extension only) +} + +var ( + marshalInfoMap = map[reflect.Type]*marshalInfo{} + marshalInfoLock sync.Mutex +) + +// getMarshalInfo returns the information to marshal a given type of message. +// The info it returns may not necessarily initialized. +// t is the type of the message (NOT the pointer to it). +func getMarshalInfo(t reflect.Type) *marshalInfo { + marshalInfoLock.Lock() + u, ok := marshalInfoMap[t] + if !ok { + u = &marshalInfo{typ: t} + marshalInfoMap[t] = u + } + marshalInfoLock.Unlock() + return u +} + +// Size is the entry point from generated code, +// and should be ONLY called by generated code. +// It computes the size of encoded data of msg. +// a is a pointer to a place to store cached marshal info. +func (a *InternalMessageInfo) Size(msg Message) int { + u := getMessageMarshalInfo(msg, a) + ptr := toPointer(&msg) + if ptr.isNil() { + // We get here if msg is a typed nil ((*SomeMessage)(nil)), + // so it satisfies the interface, and msg == nil wouldn't + // catch it. We don't want crash in this case. + return 0 + } + return u.size(ptr) +} + +// Marshal is the entry point from generated code, +// and should be ONLY called by generated code. +// It marshals msg to the end of b. +// a is a pointer to a place to store cached marshal info. +func (a *InternalMessageInfo) Marshal(b []byte, msg Message, deterministic bool) ([]byte, error) { + u := getMessageMarshalInfo(msg, a) + ptr := toPointer(&msg) + if ptr.isNil() { + // We get here if msg is a typed nil ((*SomeMessage)(nil)), + // so it satisfies the interface, and msg == nil wouldn't + // catch it. We don't want crash in this case. + return b, ErrNil + } + return u.marshal(b, ptr, deterministic) +} + +func getMessageMarshalInfo(msg interface{}, a *InternalMessageInfo) *marshalInfo { + // u := a.marshal, but atomically. + // We use an atomic here to ensure memory consistency. + u := atomicLoadMarshalInfo(&a.marshal) + if u == nil { + // Get marshal information from type of message. + t := reflect.ValueOf(msg).Type() + if t.Kind() != reflect.Ptr { + panic(fmt.Sprintf("cannot handle non-pointer message type %v", t)) + } + u = getMarshalInfo(t.Elem()) + // Store it in the cache for later users. + // a.marshal = u, but atomically. + atomicStoreMarshalInfo(&a.marshal, u) + } + return u +} + +// size is the main function to compute the size of the encoded data of a message. +// ptr is the pointer to the message. +func (u *marshalInfo) size(ptr pointer) int { + if atomic.LoadInt32(&u.initialized) == 0 { + u.computeMarshalInfo() + } + + // If the message can marshal itself, let it do it, for compatibility. + // NOTE: This is not efficient. + if u.hasmarshaler { + // Uses the message's Size method if available + if u.hassizer { + s := ptr.asPointerTo(u.typ).Interface().(Sizer) + return s.Size() + } + // Uses the message's ProtoSize method if available + if u.hasprotosizer { + s := ptr.asPointerTo(u.typ).Interface().(ProtoSizer) + return s.ProtoSize() + } + + m := ptr.asPointerTo(u.typ).Interface().(Marshaler) + b, _ := m.Marshal() + return len(b) + } + + n := 0 + for _, f := range u.fields { + if f.isPointer && ptr.offset(f.field).getPointer().isNil() { + // nil pointer always marshals to nothing + continue + } + n += f.sizer(ptr.offset(f.field), f.tagsize) + } + if u.extensions.IsValid() { + e := ptr.offset(u.extensions).toExtensions() + if u.messageset { + n += u.sizeMessageSet(e) + } else { + n += u.sizeExtensions(e) + } + } + if u.v1extensions.IsValid() { + m := *ptr.offset(u.v1extensions).toOldExtensions() + n += u.sizeV1Extensions(m) + } + if u.bytesExtensions.IsValid() { + s := *ptr.offset(u.bytesExtensions).toBytes() + n += len(s) + } + if u.unrecognized.IsValid() { + s := *ptr.offset(u.unrecognized).toBytes() + n += len(s) + } + + // cache the result for use in marshal + if u.sizecache.IsValid() { + atomic.StoreInt32(ptr.offset(u.sizecache).toInt32(), int32(n)) + } + return n +} + +// cachedsize gets the size from cache. If there is no cache (i.e. message is not generated), +// fall back to compute the size. +func (u *marshalInfo) cachedsize(ptr pointer) int { + if u.sizecache.IsValid() { + return int(atomic.LoadInt32(ptr.offset(u.sizecache).toInt32())) + } + return u.size(ptr) +} + +// marshal is the main function to marshal a message. It takes a byte slice and appends +// the encoded data to the end of the slice, returns the slice and error (if any). +// ptr is the pointer to the message. +// If deterministic is true, map is marshaled in deterministic order. +func (u *marshalInfo) marshal(b []byte, ptr pointer, deterministic bool) ([]byte, error) { + if atomic.LoadInt32(&u.initialized) == 0 { + u.computeMarshalInfo() + } + + // If the message can marshal itself, let it do it, for compatibility. + // NOTE: This is not efficient. + if u.hasmarshaler { + if deterministic { + return nil, errors.New("proto: deterministic not supported by the Marshal method of " + u.typ.String()) + } + m := ptr.asPointerTo(u.typ).Interface().(Marshaler) + b1, err := m.Marshal() + b = append(b, b1...) + return b, err + } + + var err, errreq error + // The old marshaler encodes extensions at beginning. + if u.extensions.IsValid() { + e := ptr.offset(u.extensions).toExtensions() + if u.messageset { + b, err = u.appendMessageSet(b, e, deterministic) + } else { + b, err = u.appendExtensions(b, e, deterministic) + } + if err != nil { + return b, err + } + } + if u.v1extensions.IsValid() { + m := *ptr.offset(u.v1extensions).toOldExtensions() + b, err = u.appendV1Extensions(b, m, deterministic) + if err != nil { + return b, err + } + } + if u.bytesExtensions.IsValid() { + s := *ptr.offset(u.bytesExtensions).toBytes() + b = append(b, s...) + } + for _, f := range u.fields { + if f.required && errreq == nil { + if ptr.offset(f.field).getPointer().isNil() { + // Required field is not set. + // We record the error but keep going, to give a complete marshaling. + errreq = &RequiredNotSetError{f.name} + continue + } + } + if f.isPointer && ptr.offset(f.field).getPointer().isNil() { + // nil pointer always marshals to nothing + continue + } + b, err = f.marshaler(b, ptr.offset(f.field), f.wiretag, deterministic) + if err != nil { + if err1, ok := err.(*RequiredNotSetError); ok { + // Required field in submessage is not set. + // We record the error but keep going, to give a complete marshaling. + if errreq == nil { + errreq = &RequiredNotSetError{f.name + "." + err1.field} + } + continue + } + if err == errRepeatedHasNil { + err = errors.New("proto: repeated field " + f.name + " has nil element") + } + return b, err + } + } + if u.unrecognized.IsValid() { + s := *ptr.offset(u.unrecognized).toBytes() + b = append(b, s...) + } + return b, errreq +} + +// computeMarshalInfo initializes the marshal info. +func (u *marshalInfo) computeMarshalInfo() { + u.Lock() + defer u.Unlock() + if u.initialized != 0 { // non-atomic read is ok as it is protected by the lock + return + } + + t := u.typ + u.unrecognized = invalidField + u.extensions = invalidField + u.v1extensions = invalidField + u.bytesExtensions = invalidField + u.sizecache = invalidField + isOneofMessage := false + + if reflect.PtrTo(t).Implements(sizerType) { + u.hassizer = true + } + if reflect.PtrTo(t).Implements(protosizerType) { + u.hasprotosizer = true + } + // If the message can marshal itself, let it do it, for compatibility. + // NOTE: This is not efficient. + if reflect.PtrTo(t).Implements(marshalerType) { + u.hasmarshaler = true + atomic.StoreInt32(&u.initialized, 1) + return + } + + n := t.NumField() + + // deal with XXX fields first + for i := 0; i < t.NumField(); i++ { + f := t.Field(i) + if f.Tag.Get("protobuf_oneof") != "" { + isOneofMessage = true + } + if !strings.HasPrefix(f.Name, "XXX_") { + continue + } + switch f.Name { + case "XXX_sizecache": + u.sizecache = toField(&f) + case "XXX_unrecognized": + u.unrecognized = toField(&f) + case "XXX_InternalExtensions": + u.extensions = toField(&f) + u.messageset = f.Tag.Get("protobuf_messageset") == "1" + case "XXX_extensions": + if f.Type.Kind() == reflect.Map { + u.v1extensions = toField(&f) + } else { + u.bytesExtensions = toField(&f) + } + case "XXX_NoUnkeyedLiteral": + // nothing to do + default: + panic("unknown XXX field: " + f.Name) + } + n-- + } + + // get oneof implementers + var oneofImplementers []interface{} + // gogo: isOneofMessage is needed for embedded oneof messages, without a marshaler and unmarshaler + if m, ok := reflect.Zero(reflect.PtrTo(t)).Interface().(oneofMessage); ok && isOneofMessage { + _, _, _, oneofImplementers = m.XXX_OneofFuncs() + } + + // normal fields + fields := make([]marshalFieldInfo, n) // batch allocation + u.fields = make([]*marshalFieldInfo, 0, n) + for i, j := 0, 0; i < t.NumField(); i++ { + f := t.Field(i) + + if strings.HasPrefix(f.Name, "XXX_") { + continue + } + field := &fields[j] + j++ + field.name = f.Name + u.fields = append(u.fields, field) + if f.Tag.Get("protobuf_oneof") != "" { + field.computeOneofFieldInfo(&f, oneofImplementers) + continue + } + if f.Tag.Get("protobuf") == "" { + // field has no tag (not in generated message), ignore it + u.fields = u.fields[:len(u.fields)-1] + j-- + continue + } + field.computeMarshalFieldInfo(&f) + } + + // fields are marshaled in tag order on the wire. + sort.Sort(byTag(u.fields)) + + atomic.StoreInt32(&u.initialized, 1) +} + +// helper for sorting fields by tag +type byTag []*marshalFieldInfo + +func (a byTag) Len() int { return len(a) } +func (a byTag) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a byTag) Less(i, j int) bool { return a[i].wiretag < a[j].wiretag } + +// getExtElemInfo returns the information to marshal an extension element. +// The info it returns is initialized. +func (u *marshalInfo) getExtElemInfo(desc *ExtensionDesc) *marshalElemInfo { + // get from cache first + u.RLock() + e, ok := u.extElems[desc.Field] + u.RUnlock() + if ok { + return e + } + + t := reflect.TypeOf(desc.ExtensionType) // pointer or slice to basic type or struct + tags := strings.Split(desc.Tag, ",") + tag, err := strconv.Atoi(tags[1]) + if err != nil { + panic("tag is not an integer") + } + wt := wiretype(tags[0]) + sizr, marshalr := typeMarshaler(t, tags, false, false) + e = &marshalElemInfo{ + wiretag: uint64(tag)<<3 | wt, + tagsize: SizeVarint(uint64(tag) << 3), + sizer: sizr, + marshaler: marshalr, + isptr: t.Kind() == reflect.Ptr, + } + + // update cache + u.Lock() + if u.extElems == nil { + u.extElems = make(map[int32]*marshalElemInfo) + } + u.extElems[desc.Field] = e + u.Unlock() + return e +} + +// computeMarshalFieldInfo fills up the information to marshal a field. +func (fi *marshalFieldInfo) computeMarshalFieldInfo(f *reflect.StructField) { + // parse protobuf tag of the field. + // tag has format of "bytes,49,opt,name=foo,def=hello!" + tags := strings.Split(f.Tag.Get("protobuf"), ",") + if tags[0] == "" { + return + } + tag, err := strconv.Atoi(tags[1]) + if err != nil { + panic("tag is not an integer") + } + wt := wiretype(tags[0]) + if tags[2] == "req" { + fi.required = true + } + fi.setTag(f, tag, wt) + fi.setMarshaler(f, tags) +} + +func (fi *marshalFieldInfo) computeOneofFieldInfo(f *reflect.StructField, oneofImplementers []interface{}) { + fi.field = toField(f) + fi.wiretag = 1<<31 - 1 // Use a large tag number, make oneofs sorted at the end. This tag will not appear on the wire. + fi.isPointer = true + fi.sizer, fi.marshaler = makeOneOfMarshaler(fi, f) + fi.oneofElems = make(map[reflect.Type]*marshalElemInfo) + + ityp := f.Type // interface type + for _, o := range oneofImplementers { + t := reflect.TypeOf(o) + if !t.Implements(ityp) { + continue + } + sf := t.Elem().Field(0) // oneof implementer is a struct with a single field + tags := strings.Split(sf.Tag.Get("protobuf"), ",") + tag, err := strconv.Atoi(tags[1]) + if err != nil { + panic("tag is not an integer") + } + wt := wiretype(tags[0]) + sizr, marshalr := typeMarshaler(sf.Type, tags, false, true) // oneof should not omit any zero value + fi.oneofElems[t.Elem()] = &marshalElemInfo{ + wiretag: uint64(tag)<<3 | wt, + tagsize: SizeVarint(uint64(tag) << 3), + sizer: sizr, + marshaler: marshalr, + } + } +} + +type oneofMessage interface { + XXX_OneofFuncs() (func(Message, *Buffer) error, func(Message, int, int, *Buffer) (bool, error), func(Message) int, []interface{}) +} + +// wiretype returns the wire encoding of the type. +func wiretype(encoding string) uint64 { + switch encoding { + case "fixed32": + return WireFixed32 + case "fixed64": + return WireFixed64 + case "varint", "zigzag32", "zigzag64": + return WireVarint + case "bytes": + return WireBytes + case "group": + return WireStartGroup + } + panic("unknown wire type " + encoding) +} + +// setTag fills up the tag (in wire format) and its size in the info of a field. +func (fi *marshalFieldInfo) setTag(f *reflect.StructField, tag int, wt uint64) { + fi.field = toField(f) + fi.wiretag = uint64(tag)<<3 | wt + fi.tagsize = SizeVarint(uint64(tag) << 3) +} + +// setMarshaler fills up the sizer and marshaler in the info of a field. +func (fi *marshalFieldInfo) setMarshaler(f *reflect.StructField, tags []string) { + switch f.Type.Kind() { + case reflect.Map: + // map field + fi.isPointer = true + fi.sizer, fi.marshaler = makeMapMarshaler(f) + return + case reflect.Ptr, reflect.Slice: + fi.isPointer = true + } + fi.sizer, fi.marshaler = typeMarshaler(f.Type, tags, true, false) +} + +// typeMarshaler returns the sizer and marshaler of a given field. +// t is the type of the field. +// tags is the generated "protobuf" tag of the field. +// If nozero is true, zero value is not marshaled to the wire. +// If oneof is true, it is a oneof field. +func typeMarshaler(t reflect.Type, tags []string, nozero, oneof bool) (sizer, marshaler) { + encoding := tags[0] + + pointer := false + slice := false + if t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8 { + slice = true + t = t.Elem() + } + if t.Kind() == reflect.Ptr { + pointer = true + t = t.Elem() + } + + packed := false + proto3 := false + ctype := false + isTime := false + isDuration := false + for i := 2; i < len(tags); i++ { + if tags[i] == "packed" { + packed = true + } + if tags[i] == "proto3" { + proto3 = true + } + if strings.HasPrefix(tags[i], "customtype=") { + ctype = true + } + if tags[i] == "stdtime" { + isTime = true + } + if tags[i] == "stdduration" { + isDuration = true + } + } + if !proto3 && !pointer && !slice { + nozero = false + } + + if ctype { + if reflect.PtrTo(t).Implements(customType) { + if slice { + return makeMessageRefSliceMarshaler(getMarshalInfo(t)) + } + if pointer { + return makeCustomPtrMarshaler(getMarshalInfo(t)) + } + return makeCustomMarshaler(getMarshalInfo(t)) + } else { + panic(fmt.Sprintf("custom type: type: %v, does not implement the proto.custom interface", t)) + } + } + + if isTime { + if pointer { + if slice { + return makeTimePtrSliceMarshaler(getMarshalInfo(t)) + } + return makeTimePtrMarshaler(getMarshalInfo(t)) + } + if slice { + return makeTimeSliceMarshaler(getMarshalInfo(t)) + } + return makeTimeMarshaler(getMarshalInfo(t)) + } + + if isDuration { + if pointer { + if slice { + return makeDurationPtrSliceMarshaler(getMarshalInfo(t)) + } + return makeDurationPtrMarshaler(getMarshalInfo(t)) + } + if slice { + return makeDurationSliceMarshaler(getMarshalInfo(t)) + } + return makeDurationMarshaler(getMarshalInfo(t)) + } + + switch t.Kind() { + case reflect.Bool: + if pointer { + return sizeBoolPtr, appendBoolPtr + } + if slice { + if packed { + return sizeBoolPackedSlice, appendBoolPackedSlice + } + return sizeBoolSlice, appendBoolSlice + } + if nozero { + return sizeBoolValueNoZero, appendBoolValueNoZero + } + return sizeBoolValue, appendBoolValue + case reflect.Uint32: + switch encoding { + case "fixed32": + if pointer { + return sizeFixed32Ptr, appendFixed32Ptr + } + if slice { + if packed { + return sizeFixed32PackedSlice, appendFixed32PackedSlice + } + return sizeFixed32Slice, appendFixed32Slice + } + if nozero { + return sizeFixed32ValueNoZero, appendFixed32ValueNoZero + } + return sizeFixed32Value, appendFixed32Value + case "varint": + if pointer { + return sizeVarint32Ptr, appendVarint32Ptr + } + if slice { + if packed { + return sizeVarint32PackedSlice, appendVarint32PackedSlice + } + return sizeVarint32Slice, appendVarint32Slice + } + if nozero { + return sizeVarint32ValueNoZero, appendVarint32ValueNoZero + } + return sizeVarint32Value, appendVarint32Value + } + case reflect.Int32: + switch encoding { + case "fixed32": + if pointer { + return sizeFixedS32Ptr, appendFixedS32Ptr + } + if slice { + if packed { + return sizeFixedS32PackedSlice, appendFixedS32PackedSlice + } + return sizeFixedS32Slice, appendFixedS32Slice + } + if nozero { + return sizeFixedS32ValueNoZero, appendFixedS32ValueNoZero + } + return sizeFixedS32Value, appendFixedS32Value + case "varint": + if pointer { + return sizeVarintS32Ptr, appendVarintS32Ptr + } + if slice { + if packed { + return sizeVarintS32PackedSlice, appendVarintS32PackedSlice + } + return sizeVarintS32Slice, appendVarintS32Slice + } + if nozero { + return sizeVarintS32ValueNoZero, appendVarintS32ValueNoZero + } + return sizeVarintS32Value, appendVarintS32Value + case "zigzag32": + if pointer { + return sizeZigzag32Ptr, appendZigzag32Ptr + } + if slice { + if packed { + return sizeZigzag32PackedSlice, appendZigzag32PackedSlice + } + return sizeZigzag32Slice, appendZigzag32Slice + } + if nozero { + return sizeZigzag32ValueNoZero, appendZigzag32ValueNoZero + } + return sizeZigzag32Value, appendZigzag32Value + } + case reflect.Uint64: + switch encoding { + case "fixed64": + if pointer { + return sizeFixed64Ptr, appendFixed64Ptr + } + if slice { + if packed { + return sizeFixed64PackedSlice, appendFixed64PackedSlice + } + return sizeFixed64Slice, appendFixed64Slice + } + if nozero { + return sizeFixed64ValueNoZero, appendFixed64ValueNoZero + } + return sizeFixed64Value, appendFixed64Value + case "varint": + if pointer { + return sizeVarint64Ptr, appendVarint64Ptr + } + if slice { + if packed { + return sizeVarint64PackedSlice, appendVarint64PackedSlice + } + return sizeVarint64Slice, appendVarint64Slice + } + if nozero { + return sizeVarint64ValueNoZero, appendVarint64ValueNoZero + } + return sizeVarint64Value, appendVarint64Value + } + case reflect.Int64: + switch encoding { + case "fixed64": + if pointer { + return sizeFixedS64Ptr, appendFixedS64Ptr + } + if slice { + if packed { + return sizeFixedS64PackedSlice, appendFixedS64PackedSlice + } + return sizeFixedS64Slice, appendFixedS64Slice + } + if nozero { + return sizeFixedS64ValueNoZero, appendFixedS64ValueNoZero + } + return sizeFixedS64Value, appendFixedS64Value + case "varint": + if pointer { + return sizeVarintS64Ptr, appendVarintS64Ptr + } + if slice { + if packed { + return sizeVarintS64PackedSlice, appendVarintS64PackedSlice + } + return sizeVarintS64Slice, appendVarintS64Slice + } + if nozero { + return sizeVarintS64ValueNoZero, appendVarintS64ValueNoZero + } + return sizeVarintS64Value, appendVarintS64Value + case "zigzag64": + if pointer { + return sizeZigzag64Ptr, appendZigzag64Ptr + } + if slice { + if packed { + return sizeZigzag64PackedSlice, appendZigzag64PackedSlice + } + return sizeZigzag64Slice, appendZigzag64Slice + } + if nozero { + return sizeZigzag64ValueNoZero, appendZigzag64ValueNoZero + } + return sizeZigzag64Value, appendZigzag64Value + } + case reflect.Float32: + if pointer { + return sizeFloat32Ptr, appendFloat32Ptr + } + if slice { + if packed { + return sizeFloat32PackedSlice, appendFloat32PackedSlice + } + return sizeFloat32Slice, appendFloat32Slice + } + if nozero { + return sizeFloat32ValueNoZero, appendFloat32ValueNoZero + } + return sizeFloat32Value, appendFloat32Value + case reflect.Float64: + if pointer { + return sizeFloat64Ptr, appendFloat64Ptr + } + if slice { + if packed { + return sizeFloat64PackedSlice, appendFloat64PackedSlice + } + return sizeFloat64Slice, appendFloat64Slice + } + if nozero { + return sizeFloat64ValueNoZero, appendFloat64ValueNoZero + } + return sizeFloat64Value, appendFloat64Value + case reflect.String: + if pointer { + return sizeStringPtr, appendStringPtr + } + if slice { + return sizeStringSlice, appendStringSlice + } + if nozero { + return sizeStringValueNoZero, appendStringValueNoZero + } + return sizeStringValue, appendStringValue + case reflect.Slice: + if slice { + return sizeBytesSlice, appendBytesSlice + } + if oneof { + // Oneof bytes field may also have "proto3" tag. + // We want to marshal it as a oneof field. Do this + // check before the proto3 check. + return sizeBytesOneof, appendBytesOneof + } + if proto3 { + return sizeBytes3, appendBytes3 + } + return sizeBytes, appendBytes + case reflect.Struct: + switch encoding { + case "group": + if slice { + return makeGroupSliceMarshaler(getMarshalInfo(t)) + } + return makeGroupMarshaler(getMarshalInfo(t)) + case "bytes": + if pointer { + if slice { + return makeMessageSliceMarshaler(getMarshalInfo(t)) + } + return makeMessageMarshaler(getMarshalInfo(t)) + } else { + if slice { + return makeMessageRefSliceMarshaler(getMarshalInfo(t)) + } + return makeMessageRefMarshaler(getMarshalInfo(t)) + } + } + } + panic(fmt.Sprintf("unknown or mismatched type: type: %v, wire type: %v", t, encoding)) +} + +// Below are functions to size/marshal a specific type of a field. +// They are stored in the field's info, and called by function pointers. +// They have type sizer or marshaler. + +func sizeFixed32Value(_ pointer, tagsize int) int { + return 4 + tagsize +} +func sizeFixed32ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toUint32() + if v == 0 { + return 0 + } + return 4 + tagsize +} +func sizeFixed32Ptr(ptr pointer, tagsize int) int { + p := *ptr.toUint32Ptr() + if p == nil { + return 0 + } + return 4 + tagsize +} +func sizeFixed32Slice(ptr pointer, tagsize int) int { + s := *ptr.toUint32Slice() + return (4 + tagsize) * len(s) +} +func sizeFixed32PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toUint32Slice() + if len(s) == 0 { + return 0 + } + return 4*len(s) + SizeVarint(uint64(4*len(s))) + tagsize +} +func sizeFixedS32Value(_ pointer, tagsize int) int { + return 4 + tagsize +} +func sizeFixedS32ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toInt32() + if v == 0 { + return 0 + } + return 4 + tagsize +} +func sizeFixedS32Ptr(ptr pointer, tagsize int) int { + p := ptr.getInt32Ptr() + if p == nil { + return 0 + } + return 4 + tagsize +} +func sizeFixedS32Slice(ptr pointer, tagsize int) int { + s := ptr.getInt32Slice() + return (4 + tagsize) * len(s) +} +func sizeFixedS32PackedSlice(ptr pointer, tagsize int) int { + s := ptr.getInt32Slice() + if len(s) == 0 { + return 0 + } + return 4*len(s) + SizeVarint(uint64(4*len(s))) + tagsize +} +func sizeFloat32Value(_ pointer, tagsize int) int { + return 4 + tagsize +} +func sizeFloat32ValueNoZero(ptr pointer, tagsize int) int { + v := math.Float32bits(*ptr.toFloat32()) + if v == 0 { + return 0 + } + return 4 + tagsize +} +func sizeFloat32Ptr(ptr pointer, tagsize int) int { + p := *ptr.toFloat32Ptr() + if p == nil { + return 0 + } + return 4 + tagsize +} +func sizeFloat32Slice(ptr pointer, tagsize int) int { + s := *ptr.toFloat32Slice() + return (4 + tagsize) * len(s) +} +func sizeFloat32PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toFloat32Slice() + if len(s) == 0 { + return 0 + } + return 4*len(s) + SizeVarint(uint64(4*len(s))) + tagsize +} +func sizeFixed64Value(_ pointer, tagsize int) int { + return 8 + tagsize +} +func sizeFixed64ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toUint64() + if v == 0 { + return 0 + } + return 8 + tagsize +} +func sizeFixed64Ptr(ptr pointer, tagsize int) int { + p := *ptr.toUint64Ptr() + if p == nil { + return 0 + } + return 8 + tagsize +} +func sizeFixed64Slice(ptr pointer, tagsize int) int { + s := *ptr.toUint64Slice() + return (8 + tagsize) * len(s) +} +func sizeFixed64PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toUint64Slice() + if len(s) == 0 { + return 0 + } + return 8*len(s) + SizeVarint(uint64(8*len(s))) + tagsize +} +func sizeFixedS64Value(_ pointer, tagsize int) int { + return 8 + tagsize +} +func sizeFixedS64ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toInt64() + if v == 0 { + return 0 + } + return 8 + tagsize +} +func sizeFixedS64Ptr(ptr pointer, tagsize int) int { + p := *ptr.toInt64Ptr() + if p == nil { + return 0 + } + return 8 + tagsize +} +func sizeFixedS64Slice(ptr pointer, tagsize int) int { + s := *ptr.toInt64Slice() + return (8 + tagsize) * len(s) +} +func sizeFixedS64PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toInt64Slice() + if len(s) == 0 { + return 0 + } + return 8*len(s) + SizeVarint(uint64(8*len(s))) + tagsize +} +func sizeFloat64Value(_ pointer, tagsize int) int { + return 8 + tagsize +} +func sizeFloat64ValueNoZero(ptr pointer, tagsize int) int { + v := math.Float64bits(*ptr.toFloat64()) + if v == 0 { + return 0 + } + return 8 + tagsize +} +func sizeFloat64Ptr(ptr pointer, tagsize int) int { + p := *ptr.toFloat64Ptr() + if p == nil { + return 0 + } + return 8 + tagsize +} +func sizeFloat64Slice(ptr pointer, tagsize int) int { + s := *ptr.toFloat64Slice() + return (8 + tagsize) * len(s) +} +func sizeFloat64PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toFloat64Slice() + if len(s) == 0 { + return 0 + } + return 8*len(s) + SizeVarint(uint64(8*len(s))) + tagsize +} +func sizeVarint32Value(ptr pointer, tagsize int) int { + v := *ptr.toUint32() + return SizeVarint(uint64(v)) + tagsize +} +func sizeVarint32ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toUint32() + if v == 0 { + return 0 + } + return SizeVarint(uint64(v)) + tagsize +} +func sizeVarint32Ptr(ptr pointer, tagsize int) int { + p := *ptr.toUint32Ptr() + if p == nil { + return 0 + } + return SizeVarint(uint64(*p)) + tagsize +} +func sizeVarint32Slice(ptr pointer, tagsize int) int { + s := *ptr.toUint32Slice() + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + tagsize + } + return n +} +func sizeVarint32PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toUint32Slice() + if len(s) == 0 { + return 0 + } + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + } + return n + SizeVarint(uint64(n)) + tagsize +} +func sizeVarintS32Value(ptr pointer, tagsize int) int { + v := *ptr.toInt32() + return SizeVarint(uint64(v)) + tagsize +} +func sizeVarintS32ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toInt32() + if v == 0 { + return 0 + } + return SizeVarint(uint64(v)) + tagsize +} +func sizeVarintS32Ptr(ptr pointer, tagsize int) int { + p := ptr.getInt32Ptr() + if p == nil { + return 0 + } + return SizeVarint(uint64(*p)) + tagsize +} +func sizeVarintS32Slice(ptr pointer, tagsize int) int { + s := ptr.getInt32Slice() + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + tagsize + } + return n +} +func sizeVarintS32PackedSlice(ptr pointer, tagsize int) int { + s := ptr.getInt32Slice() + if len(s) == 0 { + return 0 + } + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + } + return n + SizeVarint(uint64(n)) + tagsize +} +func sizeVarint64Value(ptr pointer, tagsize int) int { + v := *ptr.toUint64() + return SizeVarint(v) + tagsize +} +func sizeVarint64ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toUint64() + if v == 0 { + return 0 + } + return SizeVarint(v) + tagsize +} +func sizeVarint64Ptr(ptr pointer, tagsize int) int { + p := *ptr.toUint64Ptr() + if p == nil { + return 0 + } + return SizeVarint(*p) + tagsize +} +func sizeVarint64Slice(ptr pointer, tagsize int) int { + s := *ptr.toUint64Slice() + n := 0 + for _, v := range s { + n += SizeVarint(v) + tagsize + } + return n +} +func sizeVarint64PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toUint64Slice() + if len(s) == 0 { + return 0 + } + n := 0 + for _, v := range s { + n += SizeVarint(v) + } + return n + SizeVarint(uint64(n)) + tagsize +} +func sizeVarintS64Value(ptr pointer, tagsize int) int { + v := *ptr.toInt64() + return SizeVarint(uint64(v)) + tagsize +} +func sizeVarintS64ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toInt64() + if v == 0 { + return 0 + } + return SizeVarint(uint64(v)) + tagsize +} +func sizeVarintS64Ptr(ptr pointer, tagsize int) int { + p := *ptr.toInt64Ptr() + if p == nil { + return 0 + } + return SizeVarint(uint64(*p)) + tagsize +} +func sizeVarintS64Slice(ptr pointer, tagsize int) int { + s := *ptr.toInt64Slice() + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + tagsize + } + return n +} +func sizeVarintS64PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toInt64Slice() + if len(s) == 0 { + return 0 + } + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + } + return n + SizeVarint(uint64(n)) + tagsize +} +func sizeZigzag32Value(ptr pointer, tagsize int) int { + v := *ptr.toInt32() + return SizeVarint(uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + tagsize +} +func sizeZigzag32ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toInt32() + if v == 0 { + return 0 + } + return SizeVarint(uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + tagsize +} +func sizeZigzag32Ptr(ptr pointer, tagsize int) int { + p := ptr.getInt32Ptr() + if p == nil { + return 0 + } + v := *p + return SizeVarint(uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + tagsize +} +func sizeZigzag32Slice(ptr pointer, tagsize int) int { + s := ptr.getInt32Slice() + n := 0 + for _, v := range s { + n += SizeVarint(uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + tagsize + } + return n +} +func sizeZigzag32PackedSlice(ptr pointer, tagsize int) int { + s := ptr.getInt32Slice() + if len(s) == 0 { + return 0 + } + n := 0 + for _, v := range s { + n += SizeVarint(uint64((uint32(v) << 1) ^ uint32((int32(v) >> 31)))) + } + return n + SizeVarint(uint64(n)) + tagsize +} +func sizeZigzag64Value(ptr pointer, tagsize int) int { + v := *ptr.toInt64() + return SizeVarint(uint64(v<<1)^uint64((int64(v)>>63))) + tagsize +} +func sizeZigzag64ValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toInt64() + if v == 0 { + return 0 + } + return SizeVarint(uint64(v<<1)^uint64((int64(v)>>63))) + tagsize +} +func sizeZigzag64Ptr(ptr pointer, tagsize int) int { + p := *ptr.toInt64Ptr() + if p == nil { + return 0 + } + v := *p + return SizeVarint(uint64(v<<1)^uint64((int64(v)>>63))) + tagsize +} +func sizeZigzag64Slice(ptr pointer, tagsize int) int { + s := *ptr.toInt64Slice() + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v<<1)^uint64((int64(v)>>63))) + tagsize + } + return n +} +func sizeZigzag64PackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toInt64Slice() + if len(s) == 0 { + return 0 + } + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v<<1) ^ uint64((int64(v) >> 63))) + } + return n + SizeVarint(uint64(n)) + tagsize +} +func sizeBoolValue(_ pointer, tagsize int) int { + return 1 + tagsize +} +func sizeBoolValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toBool() + if !v { + return 0 + } + return 1 + tagsize +} +func sizeBoolPtr(ptr pointer, tagsize int) int { + p := *ptr.toBoolPtr() + if p == nil { + return 0 + } + return 1 + tagsize +} +func sizeBoolSlice(ptr pointer, tagsize int) int { + s := *ptr.toBoolSlice() + return (1 + tagsize) * len(s) +} +func sizeBoolPackedSlice(ptr pointer, tagsize int) int { + s := *ptr.toBoolSlice() + if len(s) == 0 { + return 0 + } + return len(s) + SizeVarint(uint64(len(s))) + tagsize +} +func sizeStringValue(ptr pointer, tagsize int) int { + v := *ptr.toString() + return len(v) + SizeVarint(uint64(len(v))) + tagsize +} +func sizeStringValueNoZero(ptr pointer, tagsize int) int { + v := *ptr.toString() + if v == "" { + return 0 + } + return len(v) + SizeVarint(uint64(len(v))) + tagsize +} +func sizeStringPtr(ptr pointer, tagsize int) int { + p := *ptr.toStringPtr() + if p == nil { + return 0 + } + v := *p + return len(v) + SizeVarint(uint64(len(v))) + tagsize +} +func sizeStringSlice(ptr pointer, tagsize int) int { + s := *ptr.toStringSlice() + n := 0 + for _, v := range s { + n += len(v) + SizeVarint(uint64(len(v))) + tagsize + } + return n +} +func sizeBytes(ptr pointer, tagsize int) int { + v := *ptr.toBytes() + if v == nil { + return 0 + } + return len(v) + SizeVarint(uint64(len(v))) + tagsize +} +func sizeBytes3(ptr pointer, tagsize int) int { + v := *ptr.toBytes() + if len(v) == 0 { + return 0 + } + return len(v) + SizeVarint(uint64(len(v))) + tagsize +} +func sizeBytesOneof(ptr pointer, tagsize int) int { + v := *ptr.toBytes() + return len(v) + SizeVarint(uint64(len(v))) + tagsize +} +func sizeBytesSlice(ptr pointer, tagsize int) int { + s := *ptr.toBytesSlice() + n := 0 + for _, v := range s { + n += len(v) + SizeVarint(uint64(len(v))) + tagsize + } + return n +} + +// appendFixed32 appends an encoded fixed32 to b. +func appendFixed32(b []byte, v uint32) []byte { + b = append(b, + byte(v), + byte(v>>8), + byte(v>>16), + byte(v>>24)) + return b +} + +// appendFixed64 appends an encoded fixed64 to b. +func appendFixed64(b []byte, v uint64) []byte { + b = append(b, + byte(v), + byte(v>>8), + byte(v>>16), + byte(v>>24), + byte(v>>32), + byte(v>>40), + byte(v>>48), + byte(v>>56)) + return b +} + +// appendVarint appends an encoded varint to b. +func appendVarint(b []byte, v uint64) []byte { + // TODO: make 1-byte (maybe 2-byte) case inline-able, once we + // have non-leaf inliner. + switch { + case v < 1<<7: + b = append(b, byte(v)) + case v < 1<<14: + b = append(b, + byte(v&0x7f|0x80), + byte(v>>7)) + case v < 1<<21: + b = append(b, + byte(v&0x7f|0x80), + byte((v>>7)&0x7f|0x80), + byte(v>>14)) + case v < 1<<28: + b = append(b, + byte(v&0x7f|0x80), + byte((v>>7)&0x7f|0x80), + byte((v>>14)&0x7f|0x80), + byte(v>>21)) + case v < 1<<35: + b = append(b, + byte(v&0x7f|0x80), + byte((v>>7)&0x7f|0x80), + byte((v>>14)&0x7f|0x80), + byte((v>>21)&0x7f|0x80), + byte(v>>28)) + case v < 1<<42: + b = append(b, + byte(v&0x7f|0x80), + byte((v>>7)&0x7f|0x80), + byte((v>>14)&0x7f|0x80), + byte((v>>21)&0x7f|0x80), + byte((v>>28)&0x7f|0x80), + byte(v>>35)) + case v < 1<<49: + b = append(b, + byte(v&0x7f|0x80), + byte((v>>7)&0x7f|0x80), + byte((v>>14)&0x7f|0x80), + byte((v>>21)&0x7f|0x80), + byte((v>>28)&0x7f|0x80), + byte((v>>35)&0x7f|0x80), + byte(v>>42)) + case v < 1<<56: + b = append(b, + byte(v&0x7f|0x80), + byte((v>>7)&0x7f|0x80), + byte((v>>14)&0x7f|0x80), + byte((v>>21)&0x7f|0x80), + byte((v>>28)&0x7f|0x80), + byte((v>>35)&0x7f|0x80), + byte((v>>42)&0x7f|0x80), + byte(v>>49)) + case v < 1<<63: + b = append(b, + byte(v&0x7f|0x80), + byte((v>>7)&0x7f|0x80), + byte((v>>14)&0x7f|0x80), + byte((v>>21)&0x7f|0x80), + byte((v>>28)&0x7f|0x80), + byte((v>>35)&0x7f|0x80), + byte((v>>42)&0x7f|0x80), + byte((v>>49)&0x7f|0x80), + byte(v>>56)) + default: + b = append(b, + byte(v&0x7f|0x80), + byte((v>>7)&0x7f|0x80), + byte((v>>14)&0x7f|0x80), + byte((v>>21)&0x7f|0x80), + byte((v>>28)&0x7f|0x80), + byte((v>>35)&0x7f|0x80), + byte((v>>42)&0x7f|0x80), + byte((v>>49)&0x7f|0x80), + byte((v>>56)&0x7f|0x80), + 1) + } + return b +} + +func appendFixed32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toUint32() + b = appendVarint(b, wiretag) + b = appendFixed32(b, v) + return b, nil +} +func appendFixed32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toUint32() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed32(b, v) + return b, nil +} +func appendFixed32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toUint32Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed32(b, *p) + return b, nil +} +func appendFixed32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toUint32Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendFixed32(b, v) + } + return b, nil +} +func appendFixed32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toUint32Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + b = appendVarint(b, uint64(4*len(s))) + for _, v := range s { + b = appendFixed32(b, v) + } + return b, nil +} +func appendFixedS32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt32() + b = appendVarint(b, wiretag) + b = appendFixed32(b, uint32(v)) + return b, nil +} +func appendFixedS32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt32() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed32(b, uint32(v)) + return b, nil +} +func appendFixedS32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := ptr.getInt32Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed32(b, uint32(*p)) + return b, nil +} +func appendFixedS32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := ptr.getInt32Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendFixed32(b, uint32(v)) + } + return b, nil +} +func appendFixedS32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := ptr.getInt32Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + b = appendVarint(b, uint64(4*len(s))) + for _, v := range s { + b = appendFixed32(b, uint32(v)) + } + return b, nil +} +func appendFloat32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := math.Float32bits(*ptr.toFloat32()) + b = appendVarint(b, wiretag) + b = appendFixed32(b, v) + return b, nil +} +func appendFloat32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := math.Float32bits(*ptr.toFloat32()) + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed32(b, v) + return b, nil +} +func appendFloat32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toFloat32Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed32(b, math.Float32bits(*p)) + return b, nil +} +func appendFloat32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toFloat32Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendFixed32(b, math.Float32bits(v)) + } + return b, nil +} +func appendFloat32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toFloat32Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + b = appendVarint(b, uint64(4*len(s))) + for _, v := range s { + b = appendFixed32(b, math.Float32bits(v)) + } + return b, nil +} +func appendFixed64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toUint64() + b = appendVarint(b, wiretag) + b = appendFixed64(b, v) + return b, nil +} +func appendFixed64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toUint64() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed64(b, v) + return b, nil +} +func appendFixed64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toUint64Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed64(b, *p) + return b, nil +} +func appendFixed64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toUint64Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendFixed64(b, v) + } + return b, nil +} +func appendFixed64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toUint64Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + b = appendVarint(b, uint64(8*len(s))) + for _, v := range s { + b = appendFixed64(b, v) + } + return b, nil +} +func appendFixedS64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt64() + b = appendVarint(b, wiretag) + b = appendFixed64(b, uint64(v)) + return b, nil +} +func appendFixedS64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt64() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed64(b, uint64(v)) + return b, nil +} +func appendFixedS64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toInt64Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed64(b, uint64(*p)) + return b, nil +} +func appendFixedS64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toInt64Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendFixed64(b, uint64(v)) + } + return b, nil +} +func appendFixedS64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toInt64Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + b = appendVarint(b, uint64(8*len(s))) + for _, v := range s { + b = appendFixed64(b, uint64(v)) + } + return b, nil +} +func appendFloat64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := math.Float64bits(*ptr.toFloat64()) + b = appendVarint(b, wiretag) + b = appendFixed64(b, v) + return b, nil +} +func appendFloat64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := math.Float64bits(*ptr.toFloat64()) + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed64(b, v) + return b, nil +} +func appendFloat64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toFloat64Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendFixed64(b, math.Float64bits(*p)) + return b, nil +} +func appendFloat64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toFloat64Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendFixed64(b, math.Float64bits(v)) + } + return b, nil +} +func appendFloat64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toFloat64Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + b = appendVarint(b, uint64(8*len(s))) + for _, v := range s { + b = appendFixed64(b, math.Float64bits(v)) + } + return b, nil +} +func appendVarint32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toUint32() + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + return b, nil +} +func appendVarint32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toUint32() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + return b, nil +} +func appendVarint32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toUint32Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(*p)) + return b, nil +} +func appendVarint32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toUint32Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + } + return b, nil +} +func appendVarint32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toUint32Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + // compute size + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + } + b = appendVarint(b, uint64(n)) + for _, v := range s { + b = appendVarint(b, uint64(v)) + } + return b, nil +} +func appendVarintS32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt32() + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + return b, nil +} +func appendVarintS32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt32() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + return b, nil +} +func appendVarintS32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := ptr.getInt32Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(*p)) + return b, nil +} +func appendVarintS32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := ptr.getInt32Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + } + return b, nil +} +func appendVarintS32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := ptr.getInt32Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + // compute size + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + } + b = appendVarint(b, uint64(n)) + for _, v := range s { + b = appendVarint(b, uint64(v)) + } + return b, nil +} +func appendVarint64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toUint64() + b = appendVarint(b, wiretag) + b = appendVarint(b, v) + return b, nil +} +func appendVarint64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toUint64() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, v) + return b, nil +} +func appendVarint64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toUint64Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, *p) + return b, nil +} +func appendVarint64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toUint64Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendVarint(b, v) + } + return b, nil +} +func appendVarint64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toUint64Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + // compute size + n := 0 + for _, v := range s { + n += SizeVarint(v) + } + b = appendVarint(b, uint64(n)) + for _, v := range s { + b = appendVarint(b, v) + } + return b, nil +} +func appendVarintS64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt64() + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + return b, nil +} +func appendVarintS64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt64() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + return b, nil +} +func appendVarintS64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toInt64Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(*p)) + return b, nil +} +func appendVarintS64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toInt64Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v)) + } + return b, nil +} +func appendVarintS64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toInt64Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + // compute size + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v)) + } + b = appendVarint(b, uint64(n)) + for _, v := range s { + b = appendVarint(b, uint64(v)) + } + return b, nil +} +func appendZigzag32Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt32() + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + return b, nil +} +func appendZigzag32ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt32() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + return b, nil +} +func appendZigzag32Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := ptr.getInt32Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + v := *p + b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + return b, nil +} +func appendZigzag32Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := ptr.getInt32Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + } + return b, nil +} +func appendZigzag32PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := ptr.getInt32Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + // compute size + n := 0 + for _, v := range s { + n += SizeVarint(uint64((uint32(v) << 1) ^ uint32((int32(v) >> 31)))) + } + b = appendVarint(b, uint64(n)) + for _, v := range s { + b = appendVarint(b, uint64((uint32(v)<<1)^uint32((int32(v)>>31)))) + } + return b, nil +} +func appendZigzag64Value(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt64() + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63))) + return b, nil +} +func appendZigzag64ValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toInt64() + if v == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63))) + return b, nil +} +func appendZigzag64Ptr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toInt64Ptr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + v := *p + b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63))) + return b, nil +} +func appendZigzag64Slice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toInt64Slice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63))) + } + return b, nil +} +func appendZigzag64PackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toInt64Slice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + // compute size + n := 0 + for _, v := range s { + n += SizeVarint(uint64(v<<1) ^ uint64((int64(v) >> 63))) + } + b = appendVarint(b, uint64(n)) + for _, v := range s { + b = appendVarint(b, uint64(v<<1)^uint64((int64(v)>>63))) + } + return b, nil +} +func appendBoolValue(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toBool() + b = appendVarint(b, wiretag) + if v { + b = append(b, 1) + } else { + b = append(b, 0) + } + return b, nil +} +func appendBoolValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toBool() + if !v { + return b, nil + } + b = appendVarint(b, wiretag) + b = append(b, 1) + return b, nil +} + +func appendBoolPtr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toBoolPtr() + if p == nil { + return b, nil + } + b = appendVarint(b, wiretag) + if *p { + b = append(b, 1) + } else { + b = append(b, 0) + } + return b, nil +} +func appendBoolSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toBoolSlice() + for _, v := range s { + b = appendVarint(b, wiretag) + if v { + b = append(b, 1) + } else { + b = append(b, 0) + } + } + return b, nil +} +func appendBoolPackedSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toBoolSlice() + if len(s) == 0 { + return b, nil + } + b = appendVarint(b, wiretag&^7|WireBytes) + b = appendVarint(b, uint64(len(s))) + for _, v := range s { + if v { + b = append(b, 1) + } else { + b = append(b, 0) + } + } + return b, nil +} +func appendStringValue(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toString() + if !utf8.ValidString(v) { + return nil, errInvalidUTF8 + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(v))) + b = append(b, v...) + return b, nil +} +func appendStringValueNoZero(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toString() + if v == "" { + return b, nil + } + if !utf8.ValidString(v) { + return nil, errInvalidUTF8 + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(v))) + b = append(b, v...) + return b, nil +} +func appendStringPtr(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + p := *ptr.toStringPtr() + if p == nil { + return b, nil + } + v := *p + if !utf8.ValidString(v) { + return nil, errInvalidUTF8 + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(v))) + b = append(b, v...) + return b, nil +} +func appendStringSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toStringSlice() + for _, v := range s { + if !utf8.ValidString(v) { + return nil, errInvalidUTF8 + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(v))) + b = append(b, v...) + } + return b, nil +} +func appendBytes(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toBytes() + if v == nil { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(v))) + b = append(b, v...) + return b, nil +} +func appendBytes3(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toBytes() + if len(v) == 0 { + return b, nil + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(v))) + b = append(b, v...) + return b, nil +} +func appendBytesOneof(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + v := *ptr.toBytes() + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(v))) + b = append(b, v...) + return b, nil +} +func appendBytesSlice(b []byte, ptr pointer, wiretag uint64, _ bool) ([]byte, error) { + s := *ptr.toBytesSlice() + for _, v := range s { + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(v))) + b = append(b, v...) + } + return b, nil +} + +// makeGroupMarshaler returns the sizer and marshaler for a group. +// u is the marshal info of the underlying message. +func makeGroupMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + p := ptr.getPointer() + if p.isNil() { + return 0 + } + return u.size(p) + 2*tagsize + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + p := ptr.getPointer() + if p.isNil() { + return b, nil + } + var err error + b = appendVarint(b, wiretag) // start group + b, err = u.marshal(b, p, deterministic) + b = appendVarint(b, wiretag+(WireEndGroup-WireStartGroup)) // end group + return b, err + } +} + +// makeGroupSliceMarshaler returns the sizer and marshaler for a group slice. +// u is the marshal info of the underlying message. +func makeGroupSliceMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + s := ptr.getPointerSlice() + n := 0 + for _, v := range s { + if v.isNil() { + continue + } + n += u.size(v) + 2*tagsize + } + return n + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + s := ptr.getPointerSlice() + var err, errreq error + for _, v := range s { + if v.isNil() { + return b, errRepeatedHasNil + } + b = appendVarint(b, wiretag) // start group + b, err = u.marshal(b, v, deterministic) + b = appendVarint(b, wiretag+(WireEndGroup-WireStartGroup)) // end group + if err != nil { + if _, ok := err.(*RequiredNotSetError); ok { + // Required field in submessage is not set. + // We record the error but keep going, to give a complete marshaling. + if errreq == nil { + errreq = err + } + continue + } + if err == ErrNil { + err = errRepeatedHasNil + } + return b, err + } + } + return b, errreq + } +} + +// makeMessageMarshaler returns the sizer and marshaler for a message field. +// u is the marshal info of the message. +func makeMessageMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + p := ptr.getPointer() + if p.isNil() { + return 0 + } + siz := u.size(p) + return siz + SizeVarint(uint64(siz)) + tagsize + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + p := ptr.getPointer() + if p.isNil() { + return b, nil + } + b = appendVarint(b, wiretag) + siz := u.cachedsize(p) + b = appendVarint(b, uint64(siz)) + return u.marshal(b, p, deterministic) + } +} + +// makeMessageSliceMarshaler returns the sizer and marshaler for a message slice. +// u is the marshal info of the message. +func makeMessageSliceMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + s := ptr.getPointerSlice() + n := 0 + for _, v := range s { + if v.isNil() { + continue + } + siz := u.size(v) + n += siz + SizeVarint(uint64(siz)) + tagsize + } + return n + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + s := ptr.getPointerSlice() + var err, errreq error + for _, v := range s { + if v.isNil() { + return b, errRepeatedHasNil + } + b = appendVarint(b, wiretag) + siz := u.cachedsize(v) + b = appendVarint(b, uint64(siz)) + b, err = u.marshal(b, v, deterministic) + if err != nil { + if _, ok := err.(*RequiredNotSetError); ok { + // Required field in submessage is not set. + // We record the error but keep going, to give a complete marshaling. + if errreq == nil { + errreq = err + } + continue + } + if err == ErrNil { + err = errRepeatedHasNil + } + return b, err + } + } + return b, errreq + } +} + +// makeMapMarshaler returns the sizer and marshaler for a map field. +// f is the pointer to the reflect data structure of the field. +func makeMapMarshaler(f *reflect.StructField) (sizer, marshaler) { + // figure out key and value type + t := f.Type + keyType := t.Key() + valType := t.Elem() + tags := strings.Split(f.Tag.Get("protobuf"), ",") + keyTags := strings.Split(f.Tag.Get("protobuf_key"), ",") + valTags := strings.Split(f.Tag.Get("protobuf_val"), ",") + for _, t := range tags { + if strings.HasPrefix(t, "customtype=") { + valTags = append(valTags, t) + } + if t == "stdtime" { + valTags = append(valTags, t) + } + if t == "stdduration" { + valTags = append(valTags, t) + } + } + keySizer, keyMarshaler := typeMarshaler(keyType, keyTags, false, false) // don't omit zero value in map + valSizer, valMarshaler := typeMarshaler(valType, valTags, false, false) // don't omit zero value in map + keyWireTag := 1<<3 | wiretype(keyTags[0]) + valWireTag := 2<<3 | wiretype(valTags[0]) + + // We create an interface to get the addresses of the map key and value. + // If value is pointer-typed, the interface is a direct interface, the + // idata itself is the value. Otherwise, the idata is the pointer to the + // value. + // Key cannot be pointer-typed. + valIsPtr := valType.Kind() == reflect.Ptr + return func(ptr pointer, tagsize int) int { + m := ptr.asPointerTo(t).Elem() // the map + n := 0 + for _, k := range m.MapKeys() { + ki := k.Interface() + vi := m.MapIndex(k).Interface() + kaddr := toAddrPointer(&ki, false) // pointer to key + vaddr := toAddrPointer(&vi, valIsPtr) // pointer to value + siz := keySizer(kaddr, 1) + valSizer(vaddr, 1) // tag of key = 1 (size=1), tag of val = 2 (size=1) + n += siz + SizeVarint(uint64(siz)) + tagsize + } + return n + }, + func(b []byte, ptr pointer, tag uint64, deterministic bool) ([]byte, error) { + m := ptr.asPointerTo(t).Elem() // the map + var err error + keys := m.MapKeys() + if len(keys) > 1 && deterministic { + sort.Sort(mapKeys(keys)) + } + for _, k := range keys { + ki := k.Interface() + vi := m.MapIndex(k).Interface() + kaddr := toAddrPointer(&ki, false) // pointer to key + vaddr := toAddrPointer(&vi, valIsPtr) // pointer to value + b = appendVarint(b, tag) + siz := keySizer(kaddr, 1) + valSizer(vaddr, 1) // tag of key = 1 (size=1), tag of val = 2 (size=1) + b = appendVarint(b, uint64(siz)) + b, err = keyMarshaler(b, kaddr, keyWireTag, deterministic) + if err != nil { + return b, err + } + b, err = valMarshaler(b, vaddr, valWireTag, deterministic) + if err != nil && err != ErrNil { // allow nil value in map + return b, err + } + } + return b, nil + } +} + +// makeOneOfMarshaler returns the sizer and marshaler for a oneof field. +// fi is the marshal info of the field. +// f is the pointer to the reflect data structure of the field. +func makeOneOfMarshaler(fi *marshalFieldInfo, f *reflect.StructField) (sizer, marshaler) { + // Oneof field is an interface. We need to get the actual data type on the fly. + t := f.Type + return func(ptr pointer, _ int) int { + p := ptr.getInterfacePointer() + if p.isNil() { + return 0 + } + v := ptr.asPointerTo(t).Elem().Elem().Elem() // *interface -> interface -> *struct -> struct + telem := v.Type() + e := fi.oneofElems[telem] + return e.sizer(p, e.tagsize) + }, + func(b []byte, ptr pointer, _ uint64, deterministic bool) ([]byte, error) { + p := ptr.getInterfacePointer() + if p.isNil() { + return b, nil + } + v := ptr.asPointerTo(t).Elem().Elem().Elem() // *interface -> interface -> *struct -> struct + telem := v.Type() + if telem.Field(0).Type.Kind() == reflect.Ptr && p.getPointer().isNil() { + return b, errOneofHasNil + } + e := fi.oneofElems[telem] + return e.marshaler(b, p, e.wiretag, deterministic) + } +} + +// sizeExtensions computes the size of encoded data for a XXX_InternalExtensions field. +func (u *marshalInfo) sizeExtensions(ext *XXX_InternalExtensions) int { + m, mu := ext.extensionsRead() + if m == nil { + return 0 + } + mu.Lock() + + n := 0 + for _, e := range m { + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + n += len(e.enc) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + ei := u.getExtElemInfo(e.desc) + v := e.value + p := toAddrPointer(&v, ei.isptr) + n += ei.sizer(p, ei.tagsize) + } + mu.Unlock() + return n +} + +// appendExtensions marshals a XXX_InternalExtensions field to the end of byte slice b. +func (u *marshalInfo) appendExtensions(b []byte, ext *XXX_InternalExtensions, deterministic bool) ([]byte, error) { + m, mu := ext.extensionsRead() + if m == nil { + return b, nil + } + mu.Lock() + defer mu.Unlock() + + var err error + + // Fast-path for common cases: zero or one extensions. + // Don't bother sorting the keys. + if len(m) <= 1 { + for _, e := range m { + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + b = append(b, e.enc...) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + + ei := u.getExtElemInfo(e.desc) + v := e.value + p := toAddrPointer(&v, ei.isptr) + b, err = ei.marshaler(b, p, ei.wiretag, deterministic) + if err != nil { + return b, err + } + } + return b, nil + } + + // Sort the keys to provide a deterministic encoding. + // Not sure this is required, but the old code does it. + keys := make([]int, 0, len(m)) + for k := range m { + keys = append(keys, int(k)) + } + sort.Ints(keys) + + for _, k := range keys { + e := m[int32(k)] + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + b = append(b, e.enc...) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + + ei := u.getExtElemInfo(e.desc) + v := e.value + p := toAddrPointer(&v, ei.isptr) + b, err = ei.marshaler(b, p, ei.wiretag, deterministic) + if err != nil { + return b, err + } + } + return b, nil +} + +// message set format is: +// message MessageSet { +// repeated group Item = 1 { +// required int32 type_id = 2; +// required string message = 3; +// }; +// } + +// sizeMessageSet computes the size of encoded data for a XXX_InternalExtensions field +// in message set format (above). +func (u *marshalInfo) sizeMessageSet(ext *XXX_InternalExtensions) int { + m, mu := ext.extensionsRead() + if m == nil { + return 0 + } + mu.Lock() + + n := 0 + for id, e := range m { + n += 2 // start group, end group. tag = 1 (size=1) + n += SizeVarint(uint64(id)) + 1 // type_id, tag = 2 (size=1) + + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + msgWithLen := skipVarint(e.enc) // skip old tag, but leave the length varint + siz := len(msgWithLen) + n += siz + 1 // message, tag = 3 (size=1) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + + ei := u.getExtElemInfo(e.desc) + v := e.value + p := toAddrPointer(&v, ei.isptr) + n += ei.sizer(p, 1) // message, tag = 3 (size=1) + } + mu.Unlock() + return n +} + +// appendMessageSet marshals a XXX_InternalExtensions field in message set format (above) +// to the end of byte slice b. +func (u *marshalInfo) appendMessageSet(b []byte, ext *XXX_InternalExtensions, deterministic bool) ([]byte, error) { + m, mu := ext.extensionsRead() + if m == nil { + return b, nil + } + mu.Lock() + defer mu.Unlock() + + var err error + + // Fast-path for common cases: zero or one extensions. + // Don't bother sorting the keys. + if len(m) <= 1 { + for id, e := range m { + b = append(b, 1<<3|WireStartGroup) + b = append(b, 2<<3|WireVarint) + b = appendVarint(b, uint64(id)) + + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + msgWithLen := skipVarint(e.enc) // skip old tag, but leave the length varint + b = append(b, 3<<3|WireBytes) + b = append(b, msgWithLen...) + b = append(b, 1<<3|WireEndGroup) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + + ei := u.getExtElemInfo(e.desc) + v := e.value + p := toAddrPointer(&v, ei.isptr) + b, err = ei.marshaler(b, p, 3<<3|WireBytes, deterministic) + if err != nil { + return b, err + } + b = append(b, 1<<3|WireEndGroup) + } + return b, nil + } + + // Sort the keys to provide a deterministic encoding. + keys := make([]int, 0, len(m)) + for k := range m { + keys = append(keys, int(k)) + } + sort.Ints(keys) + + for _, id := range keys { + e := m[int32(id)] + b = append(b, 1<<3|WireStartGroup) + b = append(b, 2<<3|WireVarint) + b = appendVarint(b, uint64(id)) + + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + msgWithLen := skipVarint(e.enc) // skip old tag, but leave the length varint + b = append(b, 3<<3|WireBytes) + b = append(b, msgWithLen...) + b = append(b, 1<<3|WireEndGroup) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + + ei := u.getExtElemInfo(e.desc) + v := e.value + p := toAddrPointer(&v, ei.isptr) + b, err = ei.marshaler(b, p, 3<<3|WireBytes, deterministic) + b = append(b, 1<<3|WireEndGroup) + if err != nil { + return b, err + } + } + return b, nil +} + +// sizeV1Extensions computes the size of encoded data for a V1-API extension field. +func (u *marshalInfo) sizeV1Extensions(m map[int32]Extension) int { + if m == nil { + return 0 + } + + n := 0 + for _, e := range m { + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + n += len(e.enc) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + + ei := u.getExtElemInfo(e.desc) + v := e.value + p := toAddrPointer(&v, ei.isptr) + n += ei.sizer(p, ei.tagsize) + } + return n +} + +// appendV1Extensions marshals a V1-API extension field to the end of byte slice b. +func (u *marshalInfo) appendV1Extensions(b []byte, m map[int32]Extension, deterministic bool) ([]byte, error) { + if m == nil { + return b, nil + } + + // Sort the keys to provide a deterministic encoding. + keys := make([]int, 0, len(m)) + for k := range m { + keys = append(keys, int(k)) + } + sort.Ints(keys) + + var err error + for _, k := range keys { + e := m[int32(k)] + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + b = append(b, e.enc...) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + + ei := u.getExtElemInfo(e.desc) + v := e.value + p := toAddrPointer(&v, ei.isptr) + b, err = ei.marshaler(b, p, ei.wiretag, deterministic) + if err != nil { + return b, err + } + } + return b, nil +} + +// newMarshaler is the interface representing objects that can marshal themselves. +// +// This exists to support protoc-gen-go generated messages. +// The proto package will stop type-asserting to this interface in the future. +// +// DO NOT DEPEND ON THIS. +type newMarshaler interface { + XXX_Size() int + XXX_Marshal(b []byte, deterministic bool) ([]byte, error) +} + +// Size returns the encoded size of a protocol buffer message. +// This is the main entry point. +func Size(pb Message) int { + if m, ok := pb.(newMarshaler); ok { + return m.XXX_Size() + } + if m, ok := pb.(Marshaler); ok { + // If the message can marshal itself, let it do it, for compatibility. + // NOTE: This is not efficient. + b, _ := m.Marshal() + return len(b) + } + // in case somehow we didn't generate the wrapper + if pb == nil { + return 0 + } + var info InternalMessageInfo + return info.Size(pb) +} + +// Marshal takes a protocol buffer message +// and encodes it into the wire format, returning the data. +// This is the main entry point. +func Marshal(pb Message) ([]byte, error) { + if m, ok := pb.(newMarshaler); ok { + siz := m.XXX_Size() + b := make([]byte, 0, siz) + return m.XXX_Marshal(b, false) + } + if m, ok := pb.(Marshaler); ok { + // If the message can marshal itself, let it do it, for compatibility. + // NOTE: This is not efficient. + return m.Marshal() + } + // in case somehow we didn't generate the wrapper + if pb == nil { + return nil, ErrNil + } + var info InternalMessageInfo + siz := info.Size(pb) + b := make([]byte, 0, siz) + return info.Marshal(b, pb, false) +} + +// Marshal takes a protocol buffer message +// and encodes it into the wire format, writing the result to the +// Buffer. +// This is an alternative entry point. It is not necessary to use +// a Buffer for most applications. +func (p *Buffer) Marshal(pb Message) error { + var err error + if m, ok := pb.(newMarshaler); ok { + siz := m.XXX_Size() + p.grow(siz) // make sure buf has enough capacity + p.buf, err = m.XXX_Marshal(p.buf, p.deterministic) + return err + } + if m, ok := pb.(Marshaler); ok { + // If the message can marshal itself, let it do it, for compatibility. + // NOTE: This is not efficient. + var b []byte + b, err = m.Marshal() + p.buf = append(p.buf, b...) + return err + } + // in case somehow we didn't generate the wrapper + if pb == nil { + return ErrNil + } + var info InternalMessageInfo + siz := info.Size(pb) + p.grow(siz) // make sure buf has enough capacity + p.buf, err = info.Marshal(p.buf, pb, p.deterministic) + return err +} + +// grow grows the buffer's capacity, if necessary, to guarantee space for +// another n bytes. After grow(n), at least n bytes can be written to the +// buffer without another allocation. +func (p *Buffer) grow(n int) { + need := len(p.buf) + n + if need <= cap(p.buf) { + return + } + newCap := len(p.buf) * 2 + if newCap < need { + newCap = need + } + p.buf = append(make([]byte, 0, newCap), p.buf...) +} diff --git a/vendor/github.com/gogo/protobuf/proto/table_marshal_gogo.go b/vendor/github.com/gogo/protobuf/proto/table_marshal_gogo.go new file mode 100644 index 0000000000000..997f57c1e1027 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/table_marshal_gogo.go @@ -0,0 +1,388 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2018, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "reflect" + "time" +) + +// makeMessageRefMarshaler differs a bit from makeMessageMarshaler +// It marshal a message T instead of a *T +func makeMessageRefMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + siz := u.size(ptr) + return siz + SizeVarint(uint64(siz)) + tagsize + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + b = appendVarint(b, wiretag) + siz := u.cachedsize(ptr) + b = appendVarint(b, uint64(siz)) + return u.marshal(b, ptr, deterministic) + } +} + +// makeMessageRefSliceMarshaler differs quite a lot from makeMessageSliceMarshaler +// It marshals a slice of messages []T instead of []*T +func makeMessageRefSliceMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + s := ptr.getSlice(u.typ) + n := 0 + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + e := elem.Interface() + v := toAddrPointer(&e, false) + siz := u.size(v) + n += siz + SizeVarint(uint64(siz)) + tagsize + } + return n + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + s := ptr.getSlice(u.typ) + var err, errreq error + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + e := elem.Interface() + v := toAddrPointer(&e, false) + b = appendVarint(b, wiretag) + siz := u.size(v) + b = appendVarint(b, uint64(siz)) + b, err = u.marshal(b, v, deterministic) + + if err != nil { + if _, ok := err.(*RequiredNotSetError); ok { + // Required field in submessage is not set. + // We record the error but keep going, to give a complete marshaling. + if errreq == nil { + errreq = err + } + continue + } + if err == ErrNil { + err = errRepeatedHasNil + } + return b, err + } + } + + return b, errreq + } +} + +func makeCustomPtrMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + if ptr.isNil() { + return 0 + } + m := ptr.asPointerTo(reflect.PtrTo(u.typ)).Elem().Interface().(custom) + siz := m.Size() + return tagsize + SizeVarint(uint64(siz)) + siz + }, func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + if ptr.isNil() { + return b, nil + } + m := ptr.asPointerTo(reflect.PtrTo(u.typ)).Elem().Interface().(custom) + siz := m.Size() + buf, err := m.Marshal() + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(siz)) + b = append(b, buf...) + return b, nil + } +} + +func makeCustomMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + m := ptr.asPointerTo(u.typ).Interface().(custom) + siz := m.Size() + return tagsize + SizeVarint(uint64(siz)) + siz + }, func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + m := ptr.asPointerTo(u.typ).Interface().(custom) + siz := m.Size() + buf, err := m.Marshal() + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(siz)) + b = append(b, buf...) + return b, nil + } +} + +func makeTimeMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + t := ptr.asPointerTo(u.typ).Interface().(*time.Time) + ts, err := timestampProto(*t) + if err != nil { + return 0 + } + siz := Size(ts) + return tagsize + SizeVarint(uint64(siz)) + siz + }, func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + t := ptr.asPointerTo(u.typ).Interface().(*time.Time) + ts, err := timestampProto(*t) + if err != nil { + return nil, err + } + buf, err := Marshal(ts) + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(buf))) + b = append(b, buf...) + return b, nil + } +} + +func makeTimePtrMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + if ptr.isNil() { + return 0 + } + t := ptr.asPointerTo(reflect.PtrTo(u.typ)).Elem().Interface().(*time.Time) + ts, err := timestampProto(*t) + if err != nil { + return 0 + } + siz := Size(ts) + return tagsize + SizeVarint(uint64(siz)) + siz + }, func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + if ptr.isNil() { + return b, nil + } + t := ptr.asPointerTo(reflect.PtrTo(u.typ)).Elem().Interface().(*time.Time) + ts, err := timestampProto(*t) + if err != nil { + return nil, err + } + buf, err := Marshal(ts) + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(buf))) + b = append(b, buf...) + return b, nil + } +} + +func makeTimeSliceMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + s := ptr.getSlice(u.typ) + n := 0 + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + t := elem.Interface().(time.Time) + ts, err := timestampProto(t) + if err != nil { + return 0 + } + siz := Size(ts) + n += siz + SizeVarint(uint64(siz)) + tagsize + } + return n + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + s := ptr.getSlice(u.typ) + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + t := elem.Interface().(time.Time) + ts, err := timestampProto(t) + if err != nil { + return nil, err + } + siz := Size(ts) + buf, err := Marshal(ts) + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(siz)) + b = append(b, buf...) + } + + return b, nil + } +} + +func makeTimePtrSliceMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + s := ptr.getSlice(reflect.PtrTo(u.typ)) + n := 0 + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + t := elem.Interface().(*time.Time) + ts, err := timestampProto(*t) + if err != nil { + return 0 + } + siz := Size(ts) + n += siz + SizeVarint(uint64(siz)) + tagsize + } + return n + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + s := ptr.getSlice(reflect.PtrTo(u.typ)) + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + t := elem.Interface().(*time.Time) + ts, err := timestampProto(*t) + if err != nil { + return nil, err + } + siz := Size(ts) + buf, err := Marshal(ts) + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(siz)) + b = append(b, buf...) + } + + return b, nil + } +} + +func makeDurationMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + d := ptr.asPointerTo(u.typ).Interface().(*time.Duration) + dur := durationProto(*d) + siz := Size(dur) + return tagsize + SizeVarint(uint64(siz)) + siz + }, func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + d := ptr.asPointerTo(u.typ).Interface().(*time.Duration) + dur := durationProto(*d) + buf, err := Marshal(dur) + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(buf))) + b = append(b, buf...) + return b, nil + } +} + +func makeDurationPtrMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + if ptr.isNil() { + return 0 + } + d := ptr.asPointerTo(reflect.PtrTo(u.typ)).Elem().Interface().(*time.Duration) + dur := durationProto(*d) + siz := Size(dur) + return tagsize + SizeVarint(uint64(siz)) + siz + }, func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + if ptr.isNil() { + return b, nil + } + d := ptr.asPointerTo(reflect.PtrTo(u.typ)).Elem().Interface().(*time.Duration) + dur := durationProto(*d) + buf, err := Marshal(dur) + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(len(buf))) + b = append(b, buf...) + return b, nil + } +} + +func makeDurationSliceMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + s := ptr.getSlice(u.typ) + n := 0 + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + d := elem.Interface().(time.Duration) + dur := durationProto(d) + siz := Size(dur) + n += siz + SizeVarint(uint64(siz)) + tagsize + } + return n + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + s := ptr.getSlice(u.typ) + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + d := elem.Interface().(time.Duration) + dur := durationProto(d) + siz := Size(dur) + buf, err := Marshal(dur) + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(siz)) + b = append(b, buf...) + } + + return b, nil + } +} + +func makeDurationPtrSliceMarshaler(u *marshalInfo) (sizer, marshaler) { + return func(ptr pointer, tagsize int) int { + s := ptr.getSlice(reflect.PtrTo(u.typ)) + n := 0 + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + d := elem.Interface().(*time.Duration) + dur := durationProto(*d) + siz := Size(dur) + n += siz + SizeVarint(uint64(siz)) + tagsize + } + return n + }, + func(b []byte, ptr pointer, wiretag uint64, deterministic bool) ([]byte, error) { + s := ptr.getSlice(reflect.PtrTo(u.typ)) + for i := 0; i < s.Len(); i++ { + elem := s.Index(i) + d := elem.Interface().(*time.Duration) + dur := durationProto(*d) + siz := Size(dur) + buf, err := Marshal(dur) + if err != nil { + return nil, err + } + b = appendVarint(b, wiretag) + b = appendVarint(b, uint64(siz)) + b = append(b, buf...) + } + + return b, nil + } +} diff --git a/vendor/github.com/gogo/protobuf/proto/table_merge.go b/vendor/github.com/gogo/protobuf/proto/table_merge.go new file mode 100644 index 0000000000000..f520106e09f5a --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/table_merge.go @@ -0,0 +1,657 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2016 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "fmt" + "reflect" + "strings" + "sync" + "sync/atomic" +) + +// Merge merges the src message into dst. +// This assumes that dst and src of the same type and are non-nil. +func (a *InternalMessageInfo) Merge(dst, src Message) { + mi := atomicLoadMergeInfo(&a.merge) + if mi == nil { + mi = getMergeInfo(reflect.TypeOf(dst).Elem()) + atomicStoreMergeInfo(&a.merge, mi) + } + mi.merge(toPointer(&dst), toPointer(&src)) +} + +type mergeInfo struct { + typ reflect.Type + + initialized int32 // 0: only typ is valid, 1: everything is valid + lock sync.Mutex + + fields []mergeFieldInfo + unrecognized field // Offset of XXX_unrecognized +} + +type mergeFieldInfo struct { + field field // Offset of field, guaranteed to be valid + + // isPointer reports whether the value in the field is a pointer. + // This is true for the following situations: + // * Pointer to struct + // * Pointer to basic type (proto2 only) + // * Slice (first value in slice header is a pointer) + // * String (first value in string header is a pointer) + isPointer bool + + // basicWidth reports the width of the field assuming that it is directly + // embedded in the struct (as is the case for basic types in proto3). + // The possible values are: + // 0: invalid + // 1: bool + // 4: int32, uint32, float32 + // 8: int64, uint64, float64 + basicWidth int + + // Where dst and src are pointers to the types being merged. + merge func(dst, src pointer) +} + +var ( + mergeInfoMap = map[reflect.Type]*mergeInfo{} + mergeInfoLock sync.Mutex +) + +func getMergeInfo(t reflect.Type) *mergeInfo { + mergeInfoLock.Lock() + defer mergeInfoLock.Unlock() + mi := mergeInfoMap[t] + if mi == nil { + mi = &mergeInfo{typ: t} + mergeInfoMap[t] = mi + } + return mi +} + +// merge merges src into dst assuming they are both of type *mi.typ. +func (mi *mergeInfo) merge(dst, src pointer) { + if dst.isNil() { + panic("proto: nil destination") + } + if src.isNil() { + return // Nothing to do. + } + + if atomic.LoadInt32(&mi.initialized) == 0 { + mi.computeMergeInfo() + } + + for _, fi := range mi.fields { + sfp := src.offset(fi.field) + + // As an optimization, we can avoid the merge function call cost + // if we know for sure that the source will have no effect + // by checking if it is the zero value. + if unsafeAllowed { + if fi.isPointer && sfp.getPointer().isNil() { // Could be slice or string + continue + } + if fi.basicWidth > 0 { + switch { + case fi.basicWidth == 1 && !*sfp.toBool(): + continue + case fi.basicWidth == 4 && *sfp.toUint32() == 0: + continue + case fi.basicWidth == 8 && *sfp.toUint64() == 0: + continue + } + } + } + + dfp := dst.offset(fi.field) + fi.merge(dfp, sfp) + } + + // TODO: Make this faster? + out := dst.asPointerTo(mi.typ).Elem() + in := src.asPointerTo(mi.typ).Elem() + if emIn, err := extendable(in.Addr().Interface()); err == nil { + emOut, _ := extendable(out.Addr().Interface()) + mIn, muIn := emIn.extensionsRead() + if mIn != nil { + mOut := emOut.extensionsWrite() + muIn.Lock() + mergeExtension(mOut, mIn) + muIn.Unlock() + } + } + + if mi.unrecognized.IsValid() { + if b := *src.offset(mi.unrecognized).toBytes(); len(b) > 0 { + *dst.offset(mi.unrecognized).toBytes() = append([]byte(nil), b...) + } + } +} + +func (mi *mergeInfo) computeMergeInfo() { + mi.lock.Lock() + defer mi.lock.Unlock() + if mi.initialized != 0 { + return + } + t := mi.typ + n := t.NumField() + + props := GetProperties(t) + for i := 0; i < n; i++ { + f := t.Field(i) + if strings.HasPrefix(f.Name, "XXX_") { + continue + } + + mfi := mergeFieldInfo{field: toField(&f)} + tf := f.Type + + // As an optimization, we can avoid the merge function call cost + // if we know for sure that the source will have no effect + // by checking if it is the zero value. + if unsafeAllowed { + switch tf.Kind() { + case reflect.Ptr, reflect.Slice, reflect.String: + // As a special case, we assume slices and strings are pointers + // since we know that the first field in the SliceSlice or + // StringHeader is a data pointer. + mfi.isPointer = true + case reflect.Bool: + mfi.basicWidth = 1 + case reflect.Int32, reflect.Uint32, reflect.Float32: + mfi.basicWidth = 4 + case reflect.Int64, reflect.Uint64, reflect.Float64: + mfi.basicWidth = 8 + } + } + + // Unwrap tf to get at its most basic type. + var isPointer, isSlice bool + if tf.Kind() == reflect.Slice && tf.Elem().Kind() != reflect.Uint8 { + isSlice = true + tf = tf.Elem() + } + if tf.Kind() == reflect.Ptr { + isPointer = true + tf = tf.Elem() + } + if isPointer && isSlice && tf.Kind() != reflect.Struct { + panic("both pointer and slice for basic type in " + tf.Name()) + } + + switch tf.Kind() { + case reflect.Int32: + switch { + case isSlice: // E.g., []int32 + mfi.merge = func(dst, src pointer) { + // NOTE: toInt32Slice is not defined (see pointer_reflect.go). + /* + sfsp := src.toInt32Slice() + if *sfsp != nil { + dfsp := dst.toInt32Slice() + *dfsp = append(*dfsp, *sfsp...) + if *dfsp == nil { + *dfsp = []int64{} + } + } + */ + sfs := src.getInt32Slice() + if sfs != nil { + dfs := dst.getInt32Slice() + dfs = append(dfs, sfs...) + if dfs == nil { + dfs = []int32{} + } + dst.setInt32Slice(dfs) + } + } + case isPointer: // E.g., *int32 + mfi.merge = func(dst, src pointer) { + // NOTE: toInt32Ptr is not defined (see pointer_reflect.go). + /* + sfpp := src.toInt32Ptr() + if *sfpp != nil { + dfpp := dst.toInt32Ptr() + if *dfpp == nil { + *dfpp = Int32(**sfpp) + } else { + **dfpp = **sfpp + } + } + */ + sfp := src.getInt32Ptr() + if sfp != nil { + dfp := dst.getInt32Ptr() + if dfp == nil { + dst.setInt32Ptr(*sfp) + } else { + *dfp = *sfp + } + } + } + default: // E.g., int32 + mfi.merge = func(dst, src pointer) { + if v := *src.toInt32(); v != 0 { + *dst.toInt32() = v + } + } + } + case reflect.Int64: + switch { + case isSlice: // E.g., []int64 + mfi.merge = func(dst, src pointer) { + sfsp := src.toInt64Slice() + if *sfsp != nil { + dfsp := dst.toInt64Slice() + *dfsp = append(*dfsp, *sfsp...) + if *dfsp == nil { + *dfsp = []int64{} + } + } + } + case isPointer: // E.g., *int64 + mfi.merge = func(dst, src pointer) { + sfpp := src.toInt64Ptr() + if *sfpp != nil { + dfpp := dst.toInt64Ptr() + if *dfpp == nil { + *dfpp = Int64(**sfpp) + } else { + **dfpp = **sfpp + } + } + } + default: // E.g., int64 + mfi.merge = func(dst, src pointer) { + if v := *src.toInt64(); v != 0 { + *dst.toInt64() = v + } + } + } + case reflect.Uint32: + switch { + case isSlice: // E.g., []uint32 + mfi.merge = func(dst, src pointer) { + sfsp := src.toUint32Slice() + if *sfsp != nil { + dfsp := dst.toUint32Slice() + *dfsp = append(*dfsp, *sfsp...) + if *dfsp == nil { + *dfsp = []uint32{} + } + } + } + case isPointer: // E.g., *uint32 + mfi.merge = func(dst, src pointer) { + sfpp := src.toUint32Ptr() + if *sfpp != nil { + dfpp := dst.toUint32Ptr() + if *dfpp == nil { + *dfpp = Uint32(**sfpp) + } else { + **dfpp = **sfpp + } + } + } + default: // E.g., uint32 + mfi.merge = func(dst, src pointer) { + if v := *src.toUint32(); v != 0 { + *dst.toUint32() = v + } + } + } + case reflect.Uint64: + switch { + case isSlice: // E.g., []uint64 + mfi.merge = func(dst, src pointer) { + sfsp := src.toUint64Slice() + if *sfsp != nil { + dfsp := dst.toUint64Slice() + *dfsp = append(*dfsp, *sfsp...) + if *dfsp == nil { + *dfsp = []uint64{} + } + } + } + case isPointer: // E.g., *uint64 + mfi.merge = func(dst, src pointer) { + sfpp := src.toUint64Ptr() + if *sfpp != nil { + dfpp := dst.toUint64Ptr() + if *dfpp == nil { + *dfpp = Uint64(**sfpp) + } else { + **dfpp = **sfpp + } + } + } + default: // E.g., uint64 + mfi.merge = func(dst, src pointer) { + if v := *src.toUint64(); v != 0 { + *dst.toUint64() = v + } + } + } + case reflect.Float32: + switch { + case isSlice: // E.g., []float32 + mfi.merge = func(dst, src pointer) { + sfsp := src.toFloat32Slice() + if *sfsp != nil { + dfsp := dst.toFloat32Slice() + *dfsp = append(*dfsp, *sfsp...) + if *dfsp == nil { + *dfsp = []float32{} + } + } + } + case isPointer: // E.g., *float32 + mfi.merge = func(dst, src pointer) { + sfpp := src.toFloat32Ptr() + if *sfpp != nil { + dfpp := dst.toFloat32Ptr() + if *dfpp == nil { + *dfpp = Float32(**sfpp) + } else { + **dfpp = **sfpp + } + } + } + default: // E.g., float32 + mfi.merge = func(dst, src pointer) { + if v := *src.toFloat32(); v != 0 { + *dst.toFloat32() = v + } + } + } + case reflect.Float64: + switch { + case isSlice: // E.g., []float64 + mfi.merge = func(dst, src pointer) { + sfsp := src.toFloat64Slice() + if *sfsp != nil { + dfsp := dst.toFloat64Slice() + *dfsp = append(*dfsp, *sfsp...) + if *dfsp == nil { + *dfsp = []float64{} + } + } + } + case isPointer: // E.g., *float64 + mfi.merge = func(dst, src pointer) { + sfpp := src.toFloat64Ptr() + if *sfpp != nil { + dfpp := dst.toFloat64Ptr() + if *dfpp == nil { + *dfpp = Float64(**sfpp) + } else { + **dfpp = **sfpp + } + } + } + default: // E.g., float64 + mfi.merge = func(dst, src pointer) { + if v := *src.toFloat64(); v != 0 { + *dst.toFloat64() = v + } + } + } + case reflect.Bool: + switch { + case isSlice: // E.g., []bool + mfi.merge = func(dst, src pointer) { + sfsp := src.toBoolSlice() + if *sfsp != nil { + dfsp := dst.toBoolSlice() + *dfsp = append(*dfsp, *sfsp...) + if *dfsp == nil { + *dfsp = []bool{} + } + } + } + case isPointer: // E.g., *bool + mfi.merge = func(dst, src pointer) { + sfpp := src.toBoolPtr() + if *sfpp != nil { + dfpp := dst.toBoolPtr() + if *dfpp == nil { + *dfpp = Bool(**sfpp) + } else { + **dfpp = **sfpp + } + } + } + default: // E.g., bool + mfi.merge = func(dst, src pointer) { + if v := *src.toBool(); v { + *dst.toBool() = v + } + } + } + case reflect.String: + switch { + case isSlice: // E.g., []string + mfi.merge = func(dst, src pointer) { + sfsp := src.toStringSlice() + if *sfsp != nil { + dfsp := dst.toStringSlice() + *dfsp = append(*dfsp, *sfsp...) + if *dfsp == nil { + *dfsp = []string{} + } + } + } + case isPointer: // E.g., *string + mfi.merge = func(dst, src pointer) { + sfpp := src.toStringPtr() + if *sfpp != nil { + dfpp := dst.toStringPtr() + if *dfpp == nil { + *dfpp = String(**sfpp) + } else { + **dfpp = **sfpp + } + } + } + default: // E.g., string + mfi.merge = func(dst, src pointer) { + if v := *src.toString(); v != "" { + *dst.toString() = v + } + } + } + case reflect.Slice: + isProto3 := props.Prop[i].proto3 + switch { + case isPointer: + panic("bad pointer in byte slice case in " + tf.Name()) + case tf.Elem().Kind() != reflect.Uint8: + panic("bad element kind in byte slice case in " + tf.Name()) + case isSlice: // E.g., [][]byte + mfi.merge = func(dst, src pointer) { + sbsp := src.toBytesSlice() + if *sbsp != nil { + dbsp := dst.toBytesSlice() + for _, sb := range *sbsp { + if sb == nil { + *dbsp = append(*dbsp, nil) + } else { + *dbsp = append(*dbsp, append([]byte{}, sb...)) + } + } + if *dbsp == nil { + *dbsp = [][]byte{} + } + } + } + default: // E.g., []byte + mfi.merge = func(dst, src pointer) { + sbp := src.toBytes() + if *sbp != nil { + dbp := dst.toBytes() + if !isProto3 || len(*sbp) > 0 { + *dbp = append([]byte{}, *sbp...) + } + } + } + } + case reflect.Struct: + switch { + case !isPointer: + mergeInfo := getMergeInfo(tf) + mfi.merge = func(dst, src pointer) { + mergeInfo.merge(dst, src) + } + case isSlice: // E.g., []*pb.T + mergeInfo := getMergeInfo(tf) + mfi.merge = func(dst, src pointer) { + sps := src.getPointerSlice() + if sps != nil { + dps := dst.getPointerSlice() + for _, sp := range sps { + var dp pointer + if !sp.isNil() { + dp = valToPointer(reflect.New(tf)) + mergeInfo.merge(dp, sp) + } + dps = append(dps, dp) + } + if dps == nil { + dps = []pointer{} + } + dst.setPointerSlice(dps) + } + } + default: // E.g., *pb.T + mergeInfo := getMergeInfo(tf) + mfi.merge = func(dst, src pointer) { + sp := src.getPointer() + if !sp.isNil() { + dp := dst.getPointer() + if dp.isNil() { + dp = valToPointer(reflect.New(tf)) + dst.setPointer(dp) + } + mergeInfo.merge(dp, sp) + } + } + } + case reflect.Map: + switch { + case isPointer || isSlice: + panic("bad pointer or slice in map case in " + tf.Name()) + default: // E.g., map[K]V + mfi.merge = func(dst, src pointer) { + sm := src.asPointerTo(tf).Elem() + if sm.Len() == 0 { + return + } + dm := dst.asPointerTo(tf).Elem() + if dm.IsNil() { + dm.Set(reflect.MakeMap(tf)) + } + + switch tf.Elem().Kind() { + case reflect.Ptr: // Proto struct (e.g., *T) + for _, key := range sm.MapKeys() { + val := sm.MapIndex(key) + val = reflect.ValueOf(Clone(val.Interface().(Message))) + dm.SetMapIndex(key, val) + } + case reflect.Slice: // E.g. Bytes type (e.g., []byte) + for _, key := range sm.MapKeys() { + val := sm.MapIndex(key) + val = reflect.ValueOf(append([]byte{}, val.Bytes()...)) + dm.SetMapIndex(key, val) + } + default: // Basic type (e.g., string) + for _, key := range sm.MapKeys() { + val := sm.MapIndex(key) + dm.SetMapIndex(key, val) + } + } + } + } + case reflect.Interface: + // Must be oneof field. + switch { + case isPointer || isSlice: + panic("bad pointer or slice in interface case in " + tf.Name()) + default: // E.g., interface{} + // TODO: Make this faster? + mfi.merge = func(dst, src pointer) { + su := src.asPointerTo(tf).Elem() + if !su.IsNil() { + du := dst.asPointerTo(tf).Elem() + typ := su.Elem().Type() + if du.IsNil() || du.Elem().Type() != typ { + du.Set(reflect.New(typ.Elem())) // Initialize interface if empty + } + sv := su.Elem().Elem().Field(0) + if sv.Kind() == reflect.Ptr && sv.IsNil() { + return + } + dv := du.Elem().Elem().Field(0) + if dv.Kind() == reflect.Ptr && dv.IsNil() { + dv.Set(reflect.New(sv.Type().Elem())) // Initialize proto message if empty + } + switch sv.Type().Kind() { + case reflect.Ptr: // Proto struct (e.g., *T) + Merge(dv.Interface().(Message), sv.Interface().(Message)) + case reflect.Slice: // E.g. Bytes type (e.g., []byte) + dv.Set(reflect.ValueOf(append([]byte{}, sv.Bytes()...))) + default: // Basic type (e.g., string) + dv.Set(sv) + } + } + } + } + default: + panic(fmt.Sprintf("merger not found for type:%s", tf)) + } + mi.fields = append(mi.fields, mfi) + } + + mi.unrecognized = invalidField + if f, ok := t.FieldByName("XXX_unrecognized"); ok { + if f.Type != reflect.TypeOf([]byte{}) { + panic("expected XXX_unrecognized to be of type []byte") + } + mi.unrecognized = toField(&f) + } + + atomic.StoreInt32(&mi.initialized, 1) +} diff --git a/vendor/github.com/gogo/protobuf/proto/table_unmarshal.go b/vendor/github.com/gogo/protobuf/proto/table_unmarshal.go new file mode 100644 index 0000000000000..910e2dd6ad31d --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/table_unmarshal.go @@ -0,0 +1,2048 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2016 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "errors" + "fmt" + "io" + "math" + "reflect" + "strconv" + "strings" + "sync" + "sync/atomic" + "unicode/utf8" +) + +// Unmarshal is the entry point from the generated .pb.go files. +// This function is not intended to be used by non-generated code. +// This function is not subject to any compatibility guarantee. +// msg contains a pointer to a protocol buffer struct. +// b is the data to be unmarshaled into the protocol buffer. +// a is a pointer to a place to store cached unmarshal information. +func (a *InternalMessageInfo) Unmarshal(msg Message, b []byte) error { + // Load the unmarshal information for this message type. + // The atomic load ensures memory consistency. + u := atomicLoadUnmarshalInfo(&a.unmarshal) + if u == nil { + // Slow path: find unmarshal info for msg, update a with it. + u = getUnmarshalInfo(reflect.TypeOf(msg).Elem()) + atomicStoreUnmarshalInfo(&a.unmarshal, u) + } + // Then do the unmarshaling. + err := u.unmarshal(toPointer(&msg), b) + return err +} + +type unmarshalInfo struct { + typ reflect.Type // type of the protobuf struct + + // 0 = only typ field is initialized + // 1 = completely initialized + initialized int32 + lock sync.Mutex // prevents double initialization + dense []unmarshalFieldInfo // fields indexed by tag # + sparse map[uint64]unmarshalFieldInfo // fields indexed by tag # + reqFields []string // names of required fields + reqMask uint64 // 1< 0 { + // Read tag and wire type. + // Special case 1 and 2 byte varints. + var x uint64 + if b[0] < 128 { + x = uint64(b[0]) + b = b[1:] + } else if len(b) >= 2 && b[1] < 128 { + x = uint64(b[0]&0x7f) + uint64(b[1])<<7 + b = b[2:] + } else { + var n int + x, n = decodeVarint(b) + if n == 0 { + return io.ErrUnexpectedEOF + } + b = b[n:] + } + tag := x >> 3 + wire := int(x) & 7 + + // Dispatch on the tag to one of the unmarshal* functions below. + var f unmarshalFieldInfo + if tag < uint64(len(u.dense)) { + f = u.dense[tag] + } else { + f = u.sparse[tag] + } + if fn := f.unmarshal; fn != nil { + var err error + b, err = fn(b, m.offset(f.field), wire) + if err == nil { + reqMask |= f.reqMask + continue + } + if r, ok := err.(*RequiredNotSetError); ok { + // Remember this error, but keep parsing. We need to produce + // a full parse even if a required field is missing. + rnse = r + reqMask |= f.reqMask + continue + } + if err != errInternalBadWireType { + return err + } + // Fragments with bad wire type are treated as unknown fields. + } + + // Unknown tag. + if !u.unrecognized.IsValid() { + // Don't keep unrecognized data; just skip it. + var err error + b, err = skipField(b, wire) + if err != nil { + return err + } + continue + } + // Keep unrecognized data around. + // maybe in extensions, maybe in the unrecognized field. + z := m.offset(u.unrecognized).toBytes() + var emap map[int32]Extension + var e Extension + for _, r := range u.extensionRanges { + if uint64(r.Start) <= tag && tag <= uint64(r.End) { + if u.extensions.IsValid() { + mp := m.offset(u.extensions).toExtensions() + emap = mp.extensionsWrite() + e = emap[int32(tag)] + z = &e.enc + break + } + if u.oldExtensions.IsValid() { + p := m.offset(u.oldExtensions).toOldExtensions() + emap = *p + if emap == nil { + emap = map[int32]Extension{} + *p = emap + } + e = emap[int32(tag)] + z = &e.enc + break + } + if u.bytesExtensions.IsValid() { + z = m.offset(u.bytesExtensions).toBytes() + break + } + panic("no extensions field available") + } + } + // Use wire type to skip data. + var err error + b0 := b + b, err = skipField(b, wire) + if err != nil { + return err + } + *z = encodeVarint(*z, tag<<3|uint64(wire)) + *z = append(*z, b0[:len(b0)-len(b)]...) + + if emap != nil { + emap[int32(tag)] = e + } + } + if rnse != nil { + // A required field of a submessage/group is missing. Return that error. + return rnse + } + if reqMask != u.reqMask { + // A required field of this message is missing. + for _, n := range u.reqFields { + if reqMask&1 == 0 { + return &RequiredNotSetError{n} + } + reqMask >>= 1 + } + } + return nil +} + +// computeUnmarshalInfo fills in u with information for use +// in unmarshaling protocol buffers of type u.typ. +func (u *unmarshalInfo) computeUnmarshalInfo() { + u.lock.Lock() + defer u.lock.Unlock() + if u.initialized != 0 { + return + } + t := u.typ + n := t.NumField() + + // Set up the "not found" value for the unrecognized byte buffer. + // This is the default for proto3. + u.unrecognized = invalidField + u.extensions = invalidField + u.oldExtensions = invalidField + u.bytesExtensions = invalidField + + // List of the generated type and offset for each oneof field. + type oneofField struct { + ityp reflect.Type // interface type of oneof field + field field // offset in containing message + } + var oneofFields []oneofField + + for i := 0; i < n; i++ { + f := t.Field(i) + if f.Name == "XXX_unrecognized" { + // The byte slice used to hold unrecognized input is special. + if f.Type != reflect.TypeOf(([]byte)(nil)) { + panic("bad type for XXX_unrecognized field: " + f.Type.Name()) + } + u.unrecognized = toField(&f) + continue + } + if f.Name == "XXX_InternalExtensions" { + // Ditto here. + if f.Type != reflect.TypeOf(XXX_InternalExtensions{}) { + panic("bad type for XXX_InternalExtensions field: " + f.Type.Name()) + } + u.extensions = toField(&f) + if f.Tag.Get("protobuf_messageset") == "1" { + u.isMessageSet = true + } + continue + } + if f.Name == "XXX_extensions" { + // An older form of the extensions field. + if f.Type == reflect.TypeOf((map[int32]Extension)(nil)) { + u.oldExtensions = toField(&f) + continue + } else if f.Type == reflect.TypeOf(([]byte)(nil)) { + u.bytesExtensions = toField(&f) + continue + } + panic("bad type for XXX_extensions field: " + f.Type.Name()) + } + if f.Name == "XXX_NoUnkeyedLiteral" || f.Name == "XXX_sizecache" { + continue + } + + oneof := f.Tag.Get("protobuf_oneof") + if oneof != "" { + oneofFields = append(oneofFields, oneofField{f.Type, toField(&f)}) + // The rest of oneof processing happens below. + continue + } + + tags := f.Tag.Get("protobuf") + tagArray := strings.Split(tags, ",") + if len(tagArray) < 2 { + panic("protobuf tag not enough fields in " + t.Name() + "." + f.Name + ": " + tags) + } + tag, err := strconv.Atoi(tagArray[1]) + if err != nil { + panic("protobuf tag field not an integer: " + tagArray[1]) + } + + name := "" + for _, tag := range tagArray[3:] { + if strings.HasPrefix(tag, "name=") { + name = tag[5:] + } + } + + // Extract unmarshaling function from the field (its type and tags). + unmarshal := fieldUnmarshaler(&f) + + // Required field? + var reqMask uint64 + if tagArray[2] == "req" { + bit := len(u.reqFields) + u.reqFields = append(u.reqFields, name) + reqMask = uint64(1) << uint(bit) + // TODO: if we have more than 64 required fields, we end up + // not verifying that all required fields are present. + // Fix this, perhaps using a count of required fields? + } + + // Store the info in the correct slot in the message. + u.setTag(tag, toField(&f), unmarshal, reqMask) + } + + // Find any types associated with oneof fields. + // TODO: XXX_OneofFuncs returns more info than we need. Get rid of some of it? + fn := reflect.Zero(reflect.PtrTo(t)).MethodByName("XXX_OneofFuncs") + // gogo: len(oneofFields) > 0 is needed for embedded oneof messages, without a marshaler and unmarshaler + if fn.IsValid() && len(oneofFields) > 0 { + res := fn.Call(nil)[3] // last return value from XXX_OneofFuncs: []interface{} + for i := res.Len() - 1; i >= 0; i-- { + v := res.Index(i) // interface{} + tptr := reflect.ValueOf(v.Interface()).Type() // *Msg_X + typ := tptr.Elem() // Msg_X + + f := typ.Field(0) // oneof implementers have one field + baseUnmarshal := fieldUnmarshaler(&f) + tagstr := strings.Split(f.Tag.Get("protobuf"), ",")[1] + tag, err := strconv.Atoi(tagstr) + if err != nil { + panic("protobuf tag field not an integer: " + tagstr) + } + + // Find the oneof field that this struct implements. + // Might take O(n^2) to process all of the oneofs, but who cares. + for _, of := range oneofFields { + if tptr.Implements(of.ityp) { + // We have found the corresponding interface for this struct. + // That lets us know where this struct should be stored + // when we encounter it during unmarshaling. + unmarshal := makeUnmarshalOneof(typ, of.ityp, baseUnmarshal) + u.setTag(tag, of.field, unmarshal, 0) + } + } + } + } + + // Get extension ranges, if any. + fn = reflect.Zero(reflect.PtrTo(t)).MethodByName("ExtensionRangeArray") + if fn.IsValid() { + if !u.extensions.IsValid() && !u.oldExtensions.IsValid() && !u.bytesExtensions.IsValid() { + panic("a message with extensions, but no extensions field in " + t.Name()) + } + u.extensionRanges = fn.Call(nil)[0].Interface().([]ExtensionRange) + } + + // Explicitly disallow tag 0. This will ensure we flag an error + // when decoding a buffer of all zeros. Without this code, we + // would decode and skip an all-zero buffer of even length. + // [0 0] is [tag=0/wiretype=varint varint-encoded-0]. + u.setTag(0, zeroField, func(b []byte, f pointer, w int) ([]byte, error) { + return nil, fmt.Errorf("proto: %s: illegal tag 0 (wire type %d)", t, w) + }, 0) + + // Set mask for required field check. + u.reqMask = uint64(1)<= 0 && (tag < 16 || tag < 2*n) { // TODO: what are the right numbers here? + for len(u.dense) <= tag { + u.dense = append(u.dense, unmarshalFieldInfo{}) + } + u.dense[tag] = i + return + } + if u.sparse == nil { + u.sparse = map[uint64]unmarshalFieldInfo{} + } + u.sparse[uint64(tag)] = i +} + +// fieldUnmarshaler returns an unmarshaler for the given field. +func fieldUnmarshaler(f *reflect.StructField) unmarshaler { + if f.Type.Kind() == reflect.Map { + return makeUnmarshalMap(f) + } + return typeUnmarshaler(f.Type, f.Tag.Get("protobuf")) +} + +// typeUnmarshaler returns an unmarshaler for the given field type / field tag pair. +func typeUnmarshaler(t reflect.Type, tags string) unmarshaler { + tagArray := strings.Split(tags, ",") + encoding := tagArray[0] + name := "unknown" + ctype := false + isTime := false + isDuration := false + for _, tag := range tagArray[3:] { + if strings.HasPrefix(tag, "name=") { + name = tag[5:] + } + if strings.HasPrefix(tag, "customtype=") { + ctype = true + } + if tag == "stdtime" { + isTime = true + } + if tag == "stdduration" { + isDuration = true + } + } + + // Figure out packaging (pointer, slice, or both) + slice := false + pointer := false + if t.Kind() == reflect.Slice && t.Elem().Kind() != reflect.Uint8 { + slice = true + t = t.Elem() + } + if t.Kind() == reflect.Ptr { + pointer = true + t = t.Elem() + } + + if ctype { + if reflect.PtrTo(t).Implements(customType) { + if slice { + return makeUnmarshalCustomSlice(getUnmarshalInfo(t), name) + } + if pointer { + return makeUnmarshalCustomPtr(getUnmarshalInfo(t), name) + } + return makeUnmarshalCustom(getUnmarshalInfo(t), name) + } else { + panic(fmt.Sprintf("custom type: type: %v, does not implement the proto.custom interface", t)) + } + } + + if isTime { + if pointer { + if slice { + return makeUnmarshalTimePtrSlice(getUnmarshalInfo(t), name) + } + return makeUnmarshalTimePtr(getUnmarshalInfo(t), name) + } + if slice { + return makeUnmarshalTimeSlice(getUnmarshalInfo(t), name) + } + return makeUnmarshalTime(getUnmarshalInfo(t), name) + } + + if isDuration { + if pointer { + if slice { + return makeUnmarshalDurationPtrSlice(getUnmarshalInfo(t), name) + } + return makeUnmarshalDurationPtr(getUnmarshalInfo(t), name) + } + if slice { + return makeUnmarshalDurationSlice(getUnmarshalInfo(t), name) + } + return makeUnmarshalDuration(getUnmarshalInfo(t), name) + } + + // We'll never have both pointer and slice for basic types. + if pointer && slice && t.Kind() != reflect.Struct { + panic("both pointer and slice for basic type in " + t.Name()) + } + + switch t.Kind() { + case reflect.Bool: + if pointer { + return unmarshalBoolPtr + } + if slice { + return unmarshalBoolSlice + } + return unmarshalBoolValue + case reflect.Int32: + switch encoding { + case "fixed32": + if pointer { + return unmarshalFixedS32Ptr + } + if slice { + return unmarshalFixedS32Slice + } + return unmarshalFixedS32Value + case "varint": + // this could be int32 or enum + if pointer { + return unmarshalInt32Ptr + } + if slice { + return unmarshalInt32Slice + } + return unmarshalInt32Value + case "zigzag32": + if pointer { + return unmarshalSint32Ptr + } + if slice { + return unmarshalSint32Slice + } + return unmarshalSint32Value + } + case reflect.Int64: + switch encoding { + case "fixed64": + if pointer { + return unmarshalFixedS64Ptr + } + if slice { + return unmarshalFixedS64Slice + } + return unmarshalFixedS64Value + case "varint": + if pointer { + return unmarshalInt64Ptr + } + if slice { + return unmarshalInt64Slice + } + return unmarshalInt64Value + case "zigzag64": + if pointer { + return unmarshalSint64Ptr + } + if slice { + return unmarshalSint64Slice + } + return unmarshalSint64Value + } + case reflect.Uint32: + switch encoding { + case "fixed32": + if pointer { + return unmarshalFixed32Ptr + } + if slice { + return unmarshalFixed32Slice + } + return unmarshalFixed32Value + case "varint": + if pointer { + return unmarshalUint32Ptr + } + if slice { + return unmarshalUint32Slice + } + return unmarshalUint32Value + } + case reflect.Uint64: + switch encoding { + case "fixed64": + if pointer { + return unmarshalFixed64Ptr + } + if slice { + return unmarshalFixed64Slice + } + return unmarshalFixed64Value + case "varint": + if pointer { + return unmarshalUint64Ptr + } + if slice { + return unmarshalUint64Slice + } + return unmarshalUint64Value + } + case reflect.Float32: + if pointer { + return unmarshalFloat32Ptr + } + if slice { + return unmarshalFloat32Slice + } + return unmarshalFloat32Value + case reflect.Float64: + if pointer { + return unmarshalFloat64Ptr + } + if slice { + return unmarshalFloat64Slice + } + return unmarshalFloat64Value + case reflect.Map: + panic("map type in typeUnmarshaler in " + t.Name()) + case reflect.Slice: + if pointer { + panic("bad pointer in slice case in " + t.Name()) + } + if slice { + return unmarshalBytesSlice + } + return unmarshalBytesValue + case reflect.String: + if pointer { + return unmarshalStringPtr + } + if slice { + return unmarshalStringSlice + } + return unmarshalStringValue + case reflect.Struct: + // message or group field + if !pointer { + switch encoding { + case "bytes": + if slice { + return makeUnmarshalMessageSlice(getUnmarshalInfo(t), name) + } + return makeUnmarshalMessage(getUnmarshalInfo(t), name) + } + } + switch encoding { + case "bytes": + if slice { + return makeUnmarshalMessageSlicePtr(getUnmarshalInfo(t), name) + } + return makeUnmarshalMessagePtr(getUnmarshalInfo(t), name) + case "group": + if slice { + return makeUnmarshalGroupSlicePtr(getUnmarshalInfo(t), name) + } + return makeUnmarshalGroupPtr(getUnmarshalInfo(t), name) + } + } + panic(fmt.Sprintf("unmarshaler not found type:%s encoding:%s", t, encoding)) +} + +// Below are all the unmarshalers for individual fields of various types. + +func unmarshalInt64Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int64(x) + *f.toInt64() = v + return b, nil +} + +func unmarshalInt64Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int64(x) + *f.toInt64Ptr() = &v + return b, nil +} + +func unmarshalInt64Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + x, n = decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int64(x) + s := f.toInt64Slice() + *s = append(*s, v) + } + return res, nil + } + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int64(x) + s := f.toInt64Slice() + *s = append(*s, v) + return b, nil +} + +func unmarshalSint64Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int64(x>>1) ^ int64(x)<<63>>63 + *f.toInt64() = v + return b, nil +} + +func unmarshalSint64Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int64(x>>1) ^ int64(x)<<63>>63 + *f.toInt64Ptr() = &v + return b, nil +} + +func unmarshalSint64Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + x, n = decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int64(x>>1) ^ int64(x)<<63>>63 + s := f.toInt64Slice() + *s = append(*s, v) + } + return res, nil + } + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int64(x>>1) ^ int64(x)<<63>>63 + s := f.toInt64Slice() + *s = append(*s, v) + return b, nil +} + +func unmarshalUint64Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := uint64(x) + *f.toUint64() = v + return b, nil +} + +func unmarshalUint64Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := uint64(x) + *f.toUint64Ptr() = &v + return b, nil +} + +func unmarshalUint64Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + x, n = decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := uint64(x) + s := f.toUint64Slice() + *s = append(*s, v) + } + return res, nil + } + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := uint64(x) + s := f.toUint64Slice() + *s = append(*s, v) + return b, nil +} + +func unmarshalInt32Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int32(x) + *f.toInt32() = v + return b, nil +} + +func unmarshalInt32Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int32(x) + f.setInt32Ptr(v) + return b, nil +} + +func unmarshalInt32Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + x, n = decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int32(x) + f.appendInt32Slice(v) + } + return res, nil + } + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int32(x) + f.appendInt32Slice(v) + return b, nil +} + +func unmarshalSint32Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int32(x>>1) ^ int32(x)<<31>>31 + *f.toInt32() = v + return b, nil +} + +func unmarshalSint32Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int32(x>>1) ^ int32(x)<<31>>31 + f.setInt32Ptr(v) + return b, nil +} + +func unmarshalSint32Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + x, n = decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int32(x>>1) ^ int32(x)<<31>>31 + f.appendInt32Slice(v) + } + return res, nil + } + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := int32(x>>1) ^ int32(x)<<31>>31 + f.appendInt32Slice(v) + return b, nil +} + +func unmarshalUint32Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := uint32(x) + *f.toUint32() = v + return b, nil +} + +func unmarshalUint32Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := uint32(x) + *f.toUint32Ptr() = &v + return b, nil +} + +func unmarshalUint32Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + x, n = decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := uint32(x) + s := f.toUint32Slice() + *s = append(*s, v) + } + return res, nil + } + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + v := uint32(x) + s := f.toUint32Slice() + *s = append(*s, v) + return b, nil +} + +func unmarshalFixed64Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56 + *f.toUint64() = v + return b[8:], nil +} + +func unmarshalFixed64Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56 + *f.toUint64Ptr() = &v + return b[8:], nil +} + +func unmarshalFixed64Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56 + s := f.toUint64Slice() + *s = append(*s, v) + b = b[8:] + } + return res, nil + } + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56 + s := f.toUint64Slice() + *s = append(*s, v) + return b[8:], nil +} + +func unmarshalFixedS64Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := int64(b[0]) | int64(b[1])<<8 | int64(b[2])<<16 | int64(b[3])<<24 | int64(b[4])<<32 | int64(b[5])<<40 | int64(b[6])<<48 | int64(b[7])<<56 + *f.toInt64() = v + return b[8:], nil +} + +func unmarshalFixedS64Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := int64(b[0]) | int64(b[1])<<8 | int64(b[2])<<16 | int64(b[3])<<24 | int64(b[4])<<32 | int64(b[5])<<40 | int64(b[6])<<48 | int64(b[7])<<56 + *f.toInt64Ptr() = &v + return b[8:], nil +} + +func unmarshalFixedS64Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := int64(b[0]) | int64(b[1])<<8 | int64(b[2])<<16 | int64(b[3])<<24 | int64(b[4])<<32 | int64(b[5])<<40 | int64(b[6])<<48 | int64(b[7])<<56 + s := f.toInt64Slice() + *s = append(*s, v) + b = b[8:] + } + return res, nil + } + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := int64(b[0]) | int64(b[1])<<8 | int64(b[2])<<16 | int64(b[3])<<24 | int64(b[4])<<32 | int64(b[5])<<40 | int64(b[6])<<48 | int64(b[7])<<56 + s := f.toInt64Slice() + *s = append(*s, v) + return b[8:], nil +} + +func unmarshalFixed32Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24 + *f.toUint32() = v + return b[4:], nil +} + +func unmarshalFixed32Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24 + *f.toUint32Ptr() = &v + return b[4:], nil +} + +func unmarshalFixed32Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24 + s := f.toUint32Slice() + *s = append(*s, v) + b = b[4:] + } + return res, nil + } + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24 + s := f.toUint32Slice() + *s = append(*s, v) + return b[4:], nil +} + +func unmarshalFixedS32Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := int32(b[0]) | int32(b[1])<<8 | int32(b[2])<<16 | int32(b[3])<<24 + *f.toInt32() = v + return b[4:], nil +} + +func unmarshalFixedS32Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := int32(b[0]) | int32(b[1])<<8 | int32(b[2])<<16 | int32(b[3])<<24 + f.setInt32Ptr(v) + return b[4:], nil +} + +func unmarshalFixedS32Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := int32(b[0]) | int32(b[1])<<8 | int32(b[2])<<16 | int32(b[3])<<24 + f.appendInt32Slice(v) + b = b[4:] + } + return res, nil + } + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := int32(b[0]) | int32(b[1])<<8 | int32(b[2])<<16 | int32(b[3])<<24 + f.appendInt32Slice(v) + return b[4:], nil +} + +func unmarshalBoolValue(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + // Note: any length varint is allowed, even though any sane + // encoder will use one byte. + // See https://github.com/golang/protobuf/issues/76 + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + // TODO: check if x>1? Tests seem to indicate no. + v := x != 0 + *f.toBool() = v + return b[n:], nil +} + +func unmarshalBoolPtr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + v := x != 0 + *f.toBoolPtr() = &v + return b[n:], nil +} + +func unmarshalBoolSlice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + x, n = decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + v := x != 0 + s := f.toBoolSlice() + *s = append(*s, v) + b = b[n:] + } + return res, nil + } + if w != WireVarint { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + v := x != 0 + s := f.toBoolSlice() + *s = append(*s, v) + return b[n:], nil +} + +func unmarshalFloat64Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := math.Float64frombits(uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56) + *f.toFloat64() = v + return b[8:], nil +} + +func unmarshalFloat64Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := math.Float64frombits(uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56) + *f.toFloat64Ptr() = &v + return b[8:], nil +} + +func unmarshalFloat64Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := math.Float64frombits(uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56) + s := f.toFloat64Slice() + *s = append(*s, v) + b = b[8:] + } + return res, nil + } + if w != WireFixed64 { + return b, errInternalBadWireType + } + if len(b) < 8 { + return nil, io.ErrUnexpectedEOF + } + v := math.Float64frombits(uint64(b[0]) | uint64(b[1])<<8 | uint64(b[2])<<16 | uint64(b[3])<<24 | uint64(b[4])<<32 | uint64(b[5])<<40 | uint64(b[6])<<48 | uint64(b[7])<<56) + s := f.toFloat64Slice() + *s = append(*s, v) + return b[8:], nil +} + +func unmarshalFloat32Value(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := math.Float32frombits(uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24) + *f.toFloat32() = v + return b[4:], nil +} + +func unmarshalFloat32Ptr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := math.Float32frombits(uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24) + *f.toFloat32Ptr() = &v + return b[4:], nil +} + +func unmarshalFloat32Slice(b []byte, f pointer, w int) ([]byte, error) { + if w == WireBytes { // packed + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + res := b[x:] + b = b[:x] + for len(b) > 0 { + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := math.Float32frombits(uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24) + s := f.toFloat32Slice() + *s = append(*s, v) + b = b[4:] + } + return res, nil + } + if w != WireFixed32 { + return b, errInternalBadWireType + } + if len(b) < 4 { + return nil, io.ErrUnexpectedEOF + } + v := math.Float32frombits(uint32(b[0]) | uint32(b[1])<<8 | uint32(b[2])<<16 | uint32(b[3])<<24) + s := f.toFloat32Slice() + *s = append(*s, v) + return b[4:], nil +} + +func unmarshalStringValue(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + v := string(b[:x]) + if !utf8.ValidString(v) { + return nil, errInvalidUTF8 + } + *f.toString() = v + return b[x:], nil +} + +func unmarshalStringPtr(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + v := string(b[:x]) + if !utf8.ValidString(v) { + return nil, errInvalidUTF8 + } + *f.toStringPtr() = &v + return b[x:], nil +} + +func unmarshalStringSlice(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + v := string(b[:x]) + if !utf8.ValidString(v) { + return nil, errInvalidUTF8 + } + s := f.toStringSlice() + *s = append(*s, v) + return b[x:], nil +} + +var emptyBuf [0]byte + +func unmarshalBytesValue(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + // The use of append here is a trick which avoids the zeroing + // that would be required if we used a make/copy pair. + // We append to emptyBuf instead of nil because we want + // a non-nil result even when the length is 0. + v := append(emptyBuf[:], b[:x]...) + *f.toBytes() = v + return b[x:], nil +} + +func unmarshalBytesSlice(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + v := append(emptyBuf[:], b[:x]...) + s := f.toBytesSlice() + *s = append(*s, v) + return b[x:], nil +} + +func makeUnmarshalMessagePtr(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + // First read the message field to see if something is there. + // The semantics of multiple submessages are weird. Instead of + // the last one winning (as it is for all other fields), multiple + // submessages are merged. + v := f.getPointer() + if v.isNil() { + v = valToPointer(reflect.New(sub.typ)) + f.setPointer(v) + } + err := sub.unmarshal(v, b[:x]) + if err != nil { + if r, ok := err.(*RequiredNotSetError); ok { + r.field = name + "." + r.field + } else { + return nil, err + } + } + return b[x:], err + } +} + +func makeUnmarshalMessageSlicePtr(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return b, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + v := valToPointer(reflect.New(sub.typ)) + err := sub.unmarshal(v, b[:x]) + if err != nil { + if r, ok := err.(*RequiredNotSetError); ok { + r.field = name + "." + r.field + } else { + return nil, err + } + } + f.appendPointer(v) + return b[x:], err + } +} + +func makeUnmarshalGroupPtr(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireStartGroup { + return b, errInternalBadWireType + } + x, y := findEndGroup(b) + if x < 0 { + return nil, io.ErrUnexpectedEOF + } + v := f.getPointer() + if v.isNil() { + v = valToPointer(reflect.New(sub.typ)) + f.setPointer(v) + } + err := sub.unmarshal(v, b[:x]) + if err != nil { + if r, ok := err.(*RequiredNotSetError); ok { + r.field = name + "." + r.field + } else { + return nil, err + } + } + return b[y:], err + } +} + +func makeUnmarshalGroupSlicePtr(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireStartGroup { + return b, errInternalBadWireType + } + x, y := findEndGroup(b) + if x < 0 { + return nil, io.ErrUnexpectedEOF + } + v := valToPointer(reflect.New(sub.typ)) + err := sub.unmarshal(v, b[:x]) + if err != nil { + if r, ok := err.(*RequiredNotSetError); ok { + r.field = name + "." + r.field + } else { + return nil, err + } + } + f.appendPointer(v) + return b[y:], err + } +} + +func makeUnmarshalMap(f *reflect.StructField) unmarshaler { + t := f.Type + kt := t.Key() + vt := t.Elem() + tagArray := strings.Split(f.Tag.Get("protobuf"), ",") + valTags := strings.Split(f.Tag.Get("protobuf_val"), ",") + for _, t := range tagArray { + if strings.HasPrefix(t, "customtype=") { + valTags = append(valTags, t) + } + if t == "stdtime" { + valTags = append(valTags, t) + } + if t == "stdduration" { + valTags = append(valTags, t) + } + } + unmarshalKey := typeUnmarshaler(kt, f.Tag.Get("protobuf_key")) + unmarshalVal := typeUnmarshaler(vt, strings.Join(valTags, ",")) + return func(b []byte, f pointer, w int) ([]byte, error) { + // The map entry is a submessage. Figure out how big it is. + if w != WireBytes { + return nil, fmt.Errorf("proto: bad wiretype for map field: got %d want %d", w, WireBytes) + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + r := b[x:] // unused data to return + b = b[:x] // data for map entry + + // Note: we could use #keys * #values ~= 200 functions + // to do map decoding without reflection. Probably not worth it. + // Maps will be somewhat slow. Oh well. + + // Read key and value from data. + k := reflect.New(kt) + v := reflect.New(vt) + for len(b) > 0 { + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + wire := int(x) & 7 + b = b[n:] + + var err error + switch x >> 3 { + case 1: + b, err = unmarshalKey(b, valToPointer(k), wire) + case 2: + b, err = unmarshalVal(b, valToPointer(v), wire) + default: + err = errInternalBadWireType // skip unknown tag + } + + if err == nil { + continue + } + if err != errInternalBadWireType { + return nil, err + } + + // Skip past unknown fields. + b, err = skipField(b, wire) + if err != nil { + return nil, err + } + } + + // Get map, allocate if needed. + m := f.asPointerTo(t).Elem() // an addressable map[K]T + if m.IsNil() { + m.Set(reflect.MakeMap(t)) + } + + // Insert into map. + m.SetMapIndex(k.Elem(), v.Elem()) + + return r, nil + } +} + +// makeUnmarshalOneof makes an unmarshaler for oneof fields. +// for: +// message Msg { +// oneof F { +// int64 X = 1; +// float64 Y = 2; +// } +// } +// typ is the type of the concrete entry for a oneof case (e.g. Msg_X). +// ityp is the interface type of the oneof field (e.g. isMsg_F). +// unmarshal is the unmarshaler for the base type of the oneof case (e.g. int64). +// Note that this function will be called once for each case in the oneof. +func makeUnmarshalOneof(typ, ityp reflect.Type, unmarshal unmarshaler) unmarshaler { + sf := typ.Field(0) + field0 := toField(&sf) + return func(b []byte, f pointer, w int) ([]byte, error) { + // Allocate holder for value. + v := reflect.New(typ) + + // Unmarshal data into holder. + // We unmarshal into the first field of the holder object. + var err error + b, err = unmarshal(b, valToPointer(v).offset(field0), w) + if err != nil { + return nil, err + } + + // Write pointer to holder into target field. + f.asPointerTo(ityp).Elem().Set(v) + + return b, nil + } +} + +// Error used by decode internally. +var errInternalBadWireType = errors.New("proto: internal error: bad wiretype") + +// skipField skips past a field of type wire and returns the remaining bytes. +func skipField(b []byte, wire int) ([]byte, error) { + switch wire { + case WireVarint: + _, k := decodeVarint(b) + if k == 0 { + return b, io.ErrUnexpectedEOF + } + b = b[k:] + case WireFixed32: + if len(b) < 4 { + return b, io.ErrUnexpectedEOF + } + b = b[4:] + case WireFixed64: + if len(b) < 8 { + return b, io.ErrUnexpectedEOF + } + b = b[8:] + case WireBytes: + m, k := decodeVarint(b) + if k == 0 || uint64(len(b)-k) < m { + return b, io.ErrUnexpectedEOF + } + b = b[uint64(k)+m:] + case WireStartGroup: + _, i := findEndGroup(b) + if i == -1 { + return b, io.ErrUnexpectedEOF + } + b = b[i:] + default: + return b, fmt.Errorf("proto: can't skip unknown wire type %d", wire) + } + return b, nil +} + +// findEndGroup finds the index of the next EndGroup tag. +// Groups may be nested, so the "next" EndGroup tag is the first +// unpaired EndGroup. +// findEndGroup returns the indexes of the start and end of the EndGroup tag. +// Returns (-1,-1) if it can't find one. +func findEndGroup(b []byte) (int, int) { + depth := 1 + i := 0 + for { + x, n := decodeVarint(b[i:]) + if n == 0 { + return -1, -1 + } + j := i + i += n + switch x & 7 { + case WireVarint: + _, k := decodeVarint(b[i:]) + if k == 0 { + return -1, -1 + } + i += k + case WireFixed32: + if len(b)-4 < i { + return -1, -1 + } + i += 4 + case WireFixed64: + if len(b)-8 < i { + return -1, -1 + } + i += 8 + case WireBytes: + m, k := decodeVarint(b[i:]) + if k == 0 { + return -1, -1 + } + i += k + if uint64(len(b)-i) < m { + return -1, -1 + } + i += int(m) + case WireStartGroup: + depth++ + case WireEndGroup: + depth-- + if depth == 0 { + return j, i + } + default: + return -1, -1 + } + } +} + +// encodeVarint appends a varint-encoded integer to b and returns the result. +func encodeVarint(b []byte, x uint64) []byte { + for x >= 1<<7 { + b = append(b, byte(x&0x7f|0x80)) + x >>= 7 + } + return append(b, byte(x)) +} + +// decodeVarint reads a varint-encoded integer from b. +// Returns the decoded integer and the number of bytes read. +// If there is an error, it returns 0,0. +func decodeVarint(b []byte) (uint64, int) { + var x, y uint64 + if len(b) <= 0 { + goto bad + } + x = uint64(b[0]) + if x < 0x80 { + return x, 1 + } + x -= 0x80 + + if len(b) <= 1 { + goto bad + } + y = uint64(b[1]) + x += y << 7 + if y < 0x80 { + return x, 2 + } + x -= 0x80 << 7 + + if len(b) <= 2 { + goto bad + } + y = uint64(b[2]) + x += y << 14 + if y < 0x80 { + return x, 3 + } + x -= 0x80 << 14 + + if len(b) <= 3 { + goto bad + } + y = uint64(b[3]) + x += y << 21 + if y < 0x80 { + return x, 4 + } + x -= 0x80 << 21 + + if len(b) <= 4 { + goto bad + } + y = uint64(b[4]) + x += y << 28 + if y < 0x80 { + return x, 5 + } + x -= 0x80 << 28 + + if len(b) <= 5 { + goto bad + } + y = uint64(b[5]) + x += y << 35 + if y < 0x80 { + return x, 6 + } + x -= 0x80 << 35 + + if len(b) <= 6 { + goto bad + } + y = uint64(b[6]) + x += y << 42 + if y < 0x80 { + return x, 7 + } + x -= 0x80 << 42 + + if len(b) <= 7 { + goto bad + } + y = uint64(b[7]) + x += y << 49 + if y < 0x80 { + return x, 8 + } + x -= 0x80 << 49 + + if len(b) <= 8 { + goto bad + } + y = uint64(b[8]) + x += y << 56 + if y < 0x80 { + return x, 9 + } + x -= 0x80 << 56 + + if len(b) <= 9 { + goto bad + } + y = uint64(b[9]) + x += y << 63 + if y < 2 { + return x, 10 + } + +bad: + return 0, 0 +} diff --git a/vendor/github.com/gogo/protobuf/proto/table_unmarshal_gogo.go b/vendor/github.com/gogo/protobuf/proto/table_unmarshal_gogo.go new file mode 100644 index 0000000000000..00d6c7ad9376a --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/table_unmarshal_gogo.go @@ -0,0 +1,385 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2018, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "io" + "reflect" +) + +func makeUnmarshalMessage(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + // First read the message field to see if something is there. + // The semantics of multiple submessages are weird. Instead of + // the last one winning (as it is for all other fields), multiple + // submessages are merged. + v := f // gogo: changed from v := f.getPointer() + if v.isNil() { + v = valToPointer(reflect.New(sub.typ)) + f.setPointer(v) + } + err := sub.unmarshal(v, b[:x]) + if err != nil { + if r, ok := err.(*RequiredNotSetError); ok { + r.field = name + "." + r.field + } else { + return nil, err + } + } + return b[x:], err + } +} + +func makeUnmarshalMessageSlice(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + v := valToPointer(reflect.New(sub.typ)) + err := sub.unmarshal(v, b[:x]) + if err != nil { + if r, ok := err.(*RequiredNotSetError); ok { + r.field = name + "." + r.field + } else { + return nil, err + } + } + f.appendRef(v, sub.typ) // gogo: changed from f.appendPointer(v) + return b[x:], err + } +} + +func makeUnmarshalCustomPtr(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + + s := f.asPointerTo(reflect.PtrTo(sub.typ)).Elem() + s.Set(reflect.New(sub.typ)) + m := s.Interface().(custom) + if err := m.Unmarshal(b[:x]); err != nil { + return nil, err + } + return b[x:], nil + } +} + +func makeUnmarshalCustomSlice(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := reflect.New(sub.typ) + c := m.Interface().(custom) + if err := c.Unmarshal(b[:x]); err != nil { + return nil, err + } + v := valToPointer(m) + f.appendRef(v, sub.typ) + return b[x:], nil + } +} + +func makeUnmarshalCustom(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + + m := f.asPointerTo(sub.typ).Interface().(custom) + if err := m.Unmarshal(b[:x]); err != nil { + return nil, err + } + return b[x:], nil + } +} + +func makeUnmarshalTime(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := ×tamp{} + if err := Unmarshal(b[:x], m); err != nil { + return nil, err + } + t, err := timestampFromProto(m) + if err != nil { + return nil, err + } + s := f.asPointerTo(sub.typ).Elem() + s.Set(reflect.ValueOf(t)) + return b[x:], nil + } +} + +func makeUnmarshalTimePtr(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := ×tamp{} + if err := Unmarshal(b[:x], m); err != nil { + return nil, err + } + t, err := timestampFromProto(m) + if err != nil { + return nil, err + } + s := f.asPointerTo(reflect.PtrTo(sub.typ)).Elem() + s.Set(reflect.ValueOf(&t)) + return b[x:], nil + } +} + +func makeUnmarshalTimePtrSlice(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := ×tamp{} + if err := Unmarshal(b[:x], m); err != nil { + return nil, err + } + t, err := timestampFromProto(m) + if err != nil { + return nil, err + } + slice := f.getSlice(reflect.PtrTo(sub.typ)) + newSlice := reflect.Append(slice, reflect.ValueOf(&t)) + slice.Set(newSlice) + return b[x:], nil + } +} + +func makeUnmarshalTimeSlice(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := ×tamp{} + if err := Unmarshal(b[:x], m); err != nil { + return nil, err + } + t, err := timestampFromProto(m) + if err != nil { + return nil, err + } + slice := f.getSlice(sub.typ) + newSlice := reflect.Append(slice, reflect.ValueOf(t)) + slice.Set(newSlice) + return b[x:], nil + } +} + +func makeUnmarshalDurationPtr(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := &duration{} + if err := Unmarshal(b[:x], m); err != nil { + return nil, err + } + d, err := durationFromProto(m) + if err != nil { + return nil, err + } + s := f.asPointerTo(reflect.PtrTo(sub.typ)).Elem() + s.Set(reflect.ValueOf(&d)) + return b[x:], nil + } +} + +func makeUnmarshalDuration(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := &duration{} + if err := Unmarshal(b[:x], m); err != nil { + return nil, err + } + d, err := durationFromProto(m) + if err != nil { + return nil, err + } + s := f.asPointerTo(sub.typ).Elem() + s.Set(reflect.ValueOf(d)) + return b[x:], nil + } +} + +func makeUnmarshalDurationPtrSlice(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := &duration{} + if err := Unmarshal(b[:x], m); err != nil { + return nil, err + } + d, err := durationFromProto(m) + if err != nil { + return nil, err + } + slice := f.getSlice(reflect.PtrTo(sub.typ)) + newSlice := reflect.Append(slice, reflect.ValueOf(&d)) + slice.Set(newSlice) + return b[x:], nil + } +} + +func makeUnmarshalDurationSlice(sub *unmarshalInfo, name string) unmarshaler { + return func(b []byte, f pointer, w int) ([]byte, error) { + if w != WireBytes { + return nil, errInternalBadWireType + } + x, n := decodeVarint(b) + if n == 0 { + return nil, io.ErrUnexpectedEOF + } + b = b[n:] + if x > uint64(len(b)) { + return nil, io.ErrUnexpectedEOF + } + m := &duration{} + if err := Unmarshal(b[:x], m); err != nil { + return nil, err + } + d, err := durationFromProto(m) + if err != nil { + return nil, err + } + slice := f.getSlice(sub.typ) + newSlice := reflect.Append(slice, reflect.ValueOf(d)) + slice.Set(newSlice) + return b[x:], nil + } +} diff --git a/vendor/github.com/gogo/protobuf/proto/text.go b/vendor/github.com/gogo/protobuf/proto/text.go new file mode 100644 index 0000000000000..4f5706dc5f369 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/text.go @@ -0,0 +1,928 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2010 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +// Functions for writing the text protocol buffer format. + +import ( + "bufio" + "bytes" + "encoding" + "errors" + "fmt" + "io" + "log" + "math" + "reflect" + "sort" + "strings" + "sync" + "time" +) + +var ( + newline = []byte("\n") + spaces = []byte(" ") + endBraceNewline = []byte("}\n") + backslashN = []byte{'\\', 'n'} + backslashR = []byte{'\\', 'r'} + backslashT = []byte{'\\', 't'} + backslashDQ = []byte{'\\', '"'} + backslashBS = []byte{'\\', '\\'} + posInf = []byte("inf") + negInf = []byte("-inf") + nan = []byte("nan") +) + +type writer interface { + io.Writer + WriteByte(byte) error +} + +// textWriter is an io.Writer that tracks its indentation level. +type textWriter struct { + ind int + complete bool // if the current position is a complete line + compact bool // whether to write out as a one-liner + w writer +} + +func (w *textWriter) WriteString(s string) (n int, err error) { + if !strings.Contains(s, "\n") { + if !w.compact && w.complete { + w.writeIndent() + } + w.complete = false + return io.WriteString(w.w, s) + } + // WriteString is typically called without newlines, so this + // codepath and its copy are rare. We copy to avoid + // duplicating all of Write's logic here. + return w.Write([]byte(s)) +} + +func (w *textWriter) Write(p []byte) (n int, err error) { + newlines := bytes.Count(p, newline) + if newlines == 0 { + if !w.compact && w.complete { + w.writeIndent() + } + n, err = w.w.Write(p) + w.complete = false + return n, err + } + + frags := bytes.SplitN(p, newline, newlines+1) + if w.compact { + for i, frag := range frags { + if i > 0 { + if err := w.w.WriteByte(' '); err != nil { + return n, err + } + n++ + } + nn, err := w.w.Write(frag) + n += nn + if err != nil { + return n, err + } + } + return n, nil + } + + for i, frag := range frags { + if w.complete { + w.writeIndent() + } + nn, err := w.w.Write(frag) + n += nn + if err != nil { + return n, err + } + if i+1 < len(frags) { + if err := w.w.WriteByte('\n'); err != nil { + return n, err + } + n++ + } + } + w.complete = len(frags[len(frags)-1]) == 0 + return n, nil +} + +func (w *textWriter) WriteByte(c byte) error { + if w.compact && c == '\n' { + c = ' ' + } + if !w.compact && w.complete { + w.writeIndent() + } + err := w.w.WriteByte(c) + w.complete = c == '\n' + return err +} + +func (w *textWriter) indent() { w.ind++ } + +func (w *textWriter) unindent() { + if w.ind == 0 { + log.Print("proto: textWriter unindented too far") + return + } + w.ind-- +} + +func writeName(w *textWriter, props *Properties) error { + if _, err := w.WriteString(props.OrigName); err != nil { + return err + } + if props.Wire != "group" { + return w.WriteByte(':') + } + return nil +} + +func requiresQuotes(u string) bool { + // When type URL contains any characters except [0-9A-Za-z./\-]*, it must be quoted. + for _, ch := range u { + switch { + case ch == '.' || ch == '/' || ch == '_': + continue + case '0' <= ch && ch <= '9': + continue + case 'A' <= ch && ch <= 'Z': + continue + case 'a' <= ch && ch <= 'z': + continue + default: + return true + } + } + return false +} + +// isAny reports whether sv is a google.protobuf.Any message +func isAny(sv reflect.Value) bool { + type wkt interface { + XXX_WellKnownType() string + } + t, ok := sv.Addr().Interface().(wkt) + return ok && t.XXX_WellKnownType() == "Any" +} + +// writeProto3Any writes an expanded google.protobuf.Any message. +// +// It returns (false, nil) if sv value can't be unmarshaled (e.g. because +// required messages are not linked in). +// +// It returns (true, error) when sv was written in expanded format or an error +// was encountered. +func (tm *TextMarshaler) writeProto3Any(w *textWriter, sv reflect.Value) (bool, error) { + turl := sv.FieldByName("TypeUrl") + val := sv.FieldByName("Value") + if !turl.IsValid() || !val.IsValid() { + return true, errors.New("proto: invalid google.protobuf.Any message") + } + + b, ok := val.Interface().([]byte) + if !ok { + return true, errors.New("proto: invalid google.protobuf.Any message") + } + + parts := strings.Split(turl.String(), "/") + mt := MessageType(parts[len(parts)-1]) + if mt == nil { + return false, nil + } + m := reflect.New(mt.Elem()) + if err := Unmarshal(b, m.Interface().(Message)); err != nil { + return false, nil + } + w.Write([]byte("[")) + u := turl.String() + if requiresQuotes(u) { + writeString(w, u) + } else { + w.Write([]byte(u)) + } + if w.compact { + w.Write([]byte("]:<")) + } else { + w.Write([]byte("]: <\n")) + w.ind++ + } + if err := tm.writeStruct(w, m.Elem()); err != nil { + return true, err + } + if w.compact { + w.Write([]byte("> ")) + } else { + w.ind-- + w.Write([]byte(">\n")) + } + return true, nil +} + +func (tm *TextMarshaler) writeStruct(w *textWriter, sv reflect.Value) error { + if tm.ExpandAny && isAny(sv) { + if canExpand, err := tm.writeProto3Any(w, sv); canExpand { + return err + } + } + st := sv.Type() + sprops := GetProperties(st) + for i := 0; i < sv.NumField(); i++ { + fv := sv.Field(i) + props := sprops.Prop[i] + name := st.Field(i).Name + + if name == "XXX_NoUnkeyedLiteral" { + continue + } + + if strings.HasPrefix(name, "XXX_") { + // There are two XXX_ fields: + // XXX_unrecognized []byte + // XXX_extensions map[int32]proto.Extension + // The first is handled here; + // the second is handled at the bottom of this function. + if name == "XXX_unrecognized" && !fv.IsNil() { + if err := writeUnknownStruct(w, fv.Interface().([]byte)); err != nil { + return err + } + } + continue + } + if fv.Kind() == reflect.Ptr && fv.IsNil() { + // Field not filled in. This could be an optional field or + // a required field that wasn't filled in. Either way, there + // isn't anything we can show for it. + continue + } + if fv.Kind() == reflect.Slice && fv.IsNil() { + // Repeated field that is empty, or a bytes field that is unused. + continue + } + + if props.Repeated && fv.Kind() == reflect.Slice { + // Repeated field. + for j := 0; j < fv.Len(); j++ { + if err := writeName(w, props); err != nil { + return err + } + if !w.compact { + if err := w.WriteByte(' '); err != nil { + return err + } + } + v := fv.Index(j) + if v.Kind() == reflect.Ptr && v.IsNil() { + // A nil message in a repeated field is not valid, + // but we can handle that more gracefully than panicking. + if _, err := w.Write([]byte("\n")); err != nil { + return err + } + continue + } + if len(props.Enum) > 0 { + if err := tm.writeEnum(w, v, props); err != nil { + return err + } + } else if err := tm.writeAny(w, v, props); err != nil { + return err + } + if err := w.WriteByte('\n'); err != nil { + return err + } + } + continue + } + if fv.Kind() == reflect.Map { + // Map fields are rendered as a repeated struct with key/value fields. + keys := fv.MapKeys() + sort.Sort(mapKeys(keys)) + for _, key := range keys { + val := fv.MapIndex(key) + if err := writeName(w, props); err != nil { + return err + } + if !w.compact { + if err := w.WriteByte(' '); err != nil { + return err + } + } + // open struct + if err := w.WriteByte('<'); err != nil { + return err + } + if !w.compact { + if err := w.WriteByte('\n'); err != nil { + return err + } + } + w.indent() + // key + if _, err := w.WriteString("key:"); err != nil { + return err + } + if !w.compact { + if err := w.WriteByte(' '); err != nil { + return err + } + } + if err := tm.writeAny(w, key, props.mkeyprop); err != nil { + return err + } + if err := w.WriteByte('\n'); err != nil { + return err + } + // nil values aren't legal, but we can avoid panicking because of them. + if val.Kind() != reflect.Ptr || !val.IsNil() { + // value + if _, err := w.WriteString("value:"); err != nil { + return err + } + if !w.compact { + if err := w.WriteByte(' '); err != nil { + return err + } + } + if err := tm.writeAny(w, val, props.mvalprop); err != nil { + return err + } + if err := w.WriteByte('\n'); err != nil { + return err + } + } + // close struct + w.unindent() + if err := w.WriteByte('>'); err != nil { + return err + } + if err := w.WriteByte('\n'); err != nil { + return err + } + } + continue + } + if props.proto3 && fv.Kind() == reflect.Slice && fv.Len() == 0 { + // empty bytes field + continue + } + if props.proto3 && fv.Kind() != reflect.Ptr && fv.Kind() != reflect.Slice { + // proto3 non-repeated scalar field; skip if zero value + if isProto3Zero(fv) { + continue + } + } + + if fv.Kind() == reflect.Interface { + // Check if it is a oneof. + if st.Field(i).Tag.Get("protobuf_oneof") != "" { + // fv is nil, or holds a pointer to generated struct. + // That generated struct has exactly one field, + // which has a protobuf struct tag. + if fv.IsNil() { + continue + } + inner := fv.Elem().Elem() // interface -> *T -> T + tag := inner.Type().Field(0).Tag.Get("protobuf") + props = new(Properties) // Overwrite the outer props var, but not its pointee. + props.Parse(tag) + // Write the value in the oneof, not the oneof itself. + fv = inner.Field(0) + + // Special case to cope with malformed messages gracefully: + // If the value in the oneof is a nil pointer, don't panic + // in writeAny. + if fv.Kind() == reflect.Ptr && fv.IsNil() { + // Use errors.New so writeAny won't render quotes. + msg := errors.New("/* nil */") + fv = reflect.ValueOf(&msg).Elem() + } + } + } + + if err := writeName(w, props); err != nil { + return err + } + if !w.compact { + if err := w.WriteByte(' '); err != nil { + return err + } + } + + if len(props.Enum) > 0 { + if err := tm.writeEnum(w, fv, props); err != nil { + return err + } + } else if err := tm.writeAny(w, fv, props); err != nil { + return err + } + + if err := w.WriteByte('\n'); err != nil { + return err + } + } + + // Extensions (the XXX_extensions field). + pv := sv + if pv.CanAddr() { + pv = sv.Addr() + } else { + pv = reflect.New(sv.Type()) + pv.Elem().Set(sv) + } + if _, err := extendable(pv.Interface()); err == nil { + if err := tm.writeExtensions(w, pv); err != nil { + return err + } + } + + return nil +} + +// writeAny writes an arbitrary field. +func (tm *TextMarshaler) writeAny(w *textWriter, v reflect.Value, props *Properties) error { + v = reflect.Indirect(v) + + if props != nil { + if len(props.CustomType) > 0 { + custom, ok := v.Interface().(Marshaler) + if ok { + data, err := custom.Marshal() + if err != nil { + return err + } + if err := writeString(w, string(data)); err != nil { + return err + } + return nil + } + } else if len(props.CastType) > 0 { + if _, ok := v.Interface().(interface { + String() string + }); ok { + switch v.Kind() { + case reflect.Int, reflect.Int8, reflect.Int16, reflect.Int32, reflect.Int64, + reflect.Uint, reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64: + _, err := fmt.Fprintf(w, "%d", v.Interface()) + return err + } + } + } else if props.StdTime { + t, ok := v.Interface().(time.Time) + if !ok { + return fmt.Errorf("stdtime is not time.Time, but %T", v.Interface()) + } + tproto, err := timestampProto(t) + if err != nil { + return err + } + propsCopy := *props // Make a copy so that this is goroutine-safe + propsCopy.StdTime = false + err = tm.writeAny(w, reflect.ValueOf(tproto), &propsCopy) + return err + } else if props.StdDuration { + d, ok := v.Interface().(time.Duration) + if !ok { + return fmt.Errorf("stdtime is not time.Duration, but %T", v.Interface()) + } + dproto := durationProto(d) + propsCopy := *props // Make a copy so that this is goroutine-safe + propsCopy.StdDuration = false + err := tm.writeAny(w, reflect.ValueOf(dproto), &propsCopy) + return err + } + } + + // Floats have special cases. + if v.Kind() == reflect.Float32 || v.Kind() == reflect.Float64 { + x := v.Float() + var b []byte + switch { + case math.IsInf(x, 1): + b = posInf + case math.IsInf(x, -1): + b = negInf + case math.IsNaN(x): + b = nan + } + if b != nil { + _, err := w.Write(b) + return err + } + // Other values are handled below. + } + + // We don't attempt to serialise every possible value type; only those + // that can occur in protocol buffers. + switch v.Kind() { + case reflect.Slice: + // Should only be a []byte; repeated fields are handled in writeStruct. + if err := writeString(w, string(v.Bytes())); err != nil { + return err + } + case reflect.String: + if err := writeString(w, v.String()); err != nil { + return err + } + case reflect.Struct: + // Required/optional group/message. + var bra, ket byte = '<', '>' + if props != nil && props.Wire == "group" { + bra, ket = '{', '}' + } + if err := w.WriteByte(bra); err != nil { + return err + } + if !w.compact { + if err := w.WriteByte('\n'); err != nil { + return err + } + } + w.indent() + if v.CanAddr() { + // Calling v.Interface on a struct causes the reflect package to + // copy the entire struct. This is racy with the new Marshaler + // since we atomically update the XXX_sizecache. + // + // Thus, we retrieve a pointer to the struct if possible to avoid + // a race since v.Interface on the pointer doesn't copy the struct. + // + // If v is not addressable, then we are not worried about a race + // since it implies that the binary Marshaler cannot possibly be + // mutating this value. + v = v.Addr() + } + if etm, ok := v.Interface().(encoding.TextMarshaler); ok { + text, err := etm.MarshalText() + if err != nil { + return err + } + if _, err = w.Write(text); err != nil { + return err + } + } else { + if v.Kind() == reflect.Ptr { + v = v.Elem() + } + if err := tm.writeStruct(w, v); err != nil { + return err + } + } + w.unindent() + if err := w.WriteByte(ket); err != nil { + return err + } + default: + _, err := fmt.Fprint(w, v.Interface()) + return err + } + return nil +} + +// equivalent to C's isprint. +func isprint(c byte) bool { + return c >= 0x20 && c < 0x7f +} + +// writeString writes a string in the protocol buffer text format. +// It is similar to strconv.Quote except we don't use Go escape sequences, +// we treat the string as a byte sequence, and we use octal escapes. +// These differences are to maintain interoperability with the other +// languages' implementations of the text format. +func writeString(w *textWriter, s string) error { + // use WriteByte here to get any needed indent + if err := w.WriteByte('"'); err != nil { + return err + } + // Loop over the bytes, not the runes. + for i := 0; i < len(s); i++ { + var err error + // Divergence from C++: we don't escape apostrophes. + // There's no need to escape them, and the C++ parser + // copes with a naked apostrophe. + switch c := s[i]; c { + case '\n': + _, err = w.w.Write(backslashN) + case '\r': + _, err = w.w.Write(backslashR) + case '\t': + _, err = w.w.Write(backslashT) + case '"': + _, err = w.w.Write(backslashDQ) + case '\\': + _, err = w.w.Write(backslashBS) + default: + if isprint(c) { + err = w.w.WriteByte(c) + } else { + _, err = fmt.Fprintf(w.w, "\\%03o", c) + } + } + if err != nil { + return err + } + } + return w.WriteByte('"') +} + +func writeUnknownStruct(w *textWriter, data []byte) (err error) { + if !w.compact { + if _, err := fmt.Fprintf(w, "/* %d unknown bytes */\n", len(data)); err != nil { + return err + } + } + b := NewBuffer(data) + for b.index < len(b.buf) { + x, err := b.DecodeVarint() + if err != nil { + _, ferr := fmt.Fprintf(w, "/* %v */\n", err) + return ferr + } + wire, tag := x&7, x>>3 + if wire == WireEndGroup { + w.unindent() + if _, werr := w.Write(endBraceNewline); werr != nil { + return werr + } + continue + } + if _, ferr := fmt.Fprint(w, tag); ferr != nil { + return ferr + } + if wire != WireStartGroup { + if err = w.WriteByte(':'); err != nil { + return err + } + } + if !w.compact || wire == WireStartGroup { + if err = w.WriteByte(' '); err != nil { + return err + } + } + switch wire { + case WireBytes: + buf, e := b.DecodeRawBytes(false) + if e == nil { + _, err = fmt.Fprintf(w, "%q", buf) + } else { + _, err = fmt.Fprintf(w, "/* %v */", e) + } + case WireFixed32: + x, err = b.DecodeFixed32() + err = writeUnknownInt(w, x, err) + case WireFixed64: + x, err = b.DecodeFixed64() + err = writeUnknownInt(w, x, err) + case WireStartGroup: + err = w.WriteByte('{') + w.indent() + case WireVarint: + x, err = b.DecodeVarint() + err = writeUnknownInt(w, x, err) + default: + _, err = fmt.Fprintf(w, "/* unknown wire type %d */", wire) + } + if err != nil { + return err + } + if err := w.WriteByte('\n'); err != nil { + return err + } + } + return nil +} + +func writeUnknownInt(w *textWriter, x uint64, err error) error { + if err == nil { + _, err = fmt.Fprint(w, x) + } else { + _, err = fmt.Fprintf(w, "/* %v */", err) + } + return err +} + +type int32Slice []int32 + +func (s int32Slice) Len() int { return len(s) } +func (s int32Slice) Less(i, j int) bool { return s[i] < s[j] } +func (s int32Slice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } + +// writeExtensions writes all the extensions in pv. +// pv is assumed to be a pointer to a protocol message struct that is extendable. +func (tm *TextMarshaler) writeExtensions(w *textWriter, pv reflect.Value) error { + emap := extensionMaps[pv.Type().Elem()] + e := pv.Interface().(Message) + + var m map[int32]Extension + var mu sync.Locker + if em, ok := e.(extensionsBytes); ok { + eb := em.GetExtensions() + var err error + m, err = BytesToExtensionsMap(*eb) + if err != nil { + return err + } + mu = notLocker{} + } else if _, ok := e.(extendableProto); ok { + ep, _ := extendable(e) + m, mu = ep.extensionsRead() + if m == nil { + return nil + } + } + + // Order the extensions by ID. + // This isn't strictly necessary, but it will give us + // canonical output, which will also make testing easier. + + mu.Lock() + ids := make([]int32, 0, len(m)) + for id := range m { + ids = append(ids, id) + } + sort.Sort(int32Slice(ids)) + mu.Unlock() + + for _, extNum := range ids { + ext := m[extNum] + var desc *ExtensionDesc + if emap != nil { + desc = emap[extNum] + } + if desc == nil { + // Unknown extension. + if err := writeUnknownStruct(w, ext.enc); err != nil { + return err + } + continue + } + + pb, err := GetExtension(e, desc) + if err != nil { + return fmt.Errorf("failed getting extension: %v", err) + } + + // Repeated extensions will appear as a slice. + if !desc.repeated() { + if err := tm.writeExtension(w, desc.Name, pb); err != nil { + return err + } + } else { + v := reflect.ValueOf(pb) + for i := 0; i < v.Len(); i++ { + if err := tm.writeExtension(w, desc.Name, v.Index(i).Interface()); err != nil { + return err + } + } + } + } + return nil +} + +func (tm *TextMarshaler) writeExtension(w *textWriter, name string, pb interface{}) error { + if _, err := fmt.Fprintf(w, "[%s]:", name); err != nil { + return err + } + if !w.compact { + if err := w.WriteByte(' '); err != nil { + return err + } + } + if err := tm.writeAny(w, reflect.ValueOf(pb), nil); err != nil { + return err + } + if err := w.WriteByte('\n'); err != nil { + return err + } + return nil +} + +func (w *textWriter) writeIndent() { + if !w.complete { + return + } + remain := w.ind * 2 + for remain > 0 { + n := remain + if n > len(spaces) { + n = len(spaces) + } + w.w.Write(spaces[:n]) + remain -= n + } + w.complete = false +} + +// TextMarshaler is a configurable text format marshaler. +type TextMarshaler struct { + Compact bool // use compact text format (one line). + ExpandAny bool // expand google.protobuf.Any messages of known types +} + +// Marshal writes a given protocol buffer in text format. +// The only errors returned are from w. +func (tm *TextMarshaler) Marshal(w io.Writer, pb Message) error { + val := reflect.ValueOf(pb) + if pb == nil || val.IsNil() { + w.Write([]byte("")) + return nil + } + var bw *bufio.Writer + ww, ok := w.(writer) + if !ok { + bw = bufio.NewWriter(w) + ww = bw + } + aw := &textWriter{ + w: ww, + complete: true, + compact: tm.Compact, + } + + if etm, ok := pb.(encoding.TextMarshaler); ok { + text, err := etm.MarshalText() + if err != nil { + return err + } + if _, err = aw.Write(text); err != nil { + return err + } + if bw != nil { + return bw.Flush() + } + return nil + } + // Dereference the received pointer so we don't have outer < and >. + v := reflect.Indirect(val) + if err := tm.writeStruct(aw, v); err != nil { + return err + } + if bw != nil { + return bw.Flush() + } + return nil +} + +// Text is the same as Marshal, but returns the string directly. +func (tm *TextMarshaler) Text(pb Message) string { + var buf bytes.Buffer + tm.Marshal(&buf, pb) + return buf.String() +} + +var ( + defaultTextMarshaler = TextMarshaler{} + compactTextMarshaler = TextMarshaler{Compact: true} +) + +// TODO: consider removing some of the Marshal functions below. + +// MarshalText writes a given protocol buffer in text format. +// The only errors returned are from w. +func MarshalText(w io.Writer, pb Message) error { return defaultTextMarshaler.Marshal(w, pb) } + +// MarshalTextString is the same as MarshalText, but returns the string directly. +func MarshalTextString(pb Message) string { return defaultTextMarshaler.Text(pb) } + +// CompactText writes a given protocol buffer in compact text format (one line). +func CompactText(w io.Writer, pb Message) error { return compactTextMarshaler.Marshal(w, pb) } + +// CompactTextString is the same as CompactText, but returns the string directly. +func CompactTextString(pb Message) string { return compactTextMarshaler.Text(pb) } diff --git a/vendor/github.com/gogo/protobuf/proto/text_gogo.go b/vendor/github.com/gogo/protobuf/proto/text_gogo.go new file mode 100644 index 0000000000000..1d6c6aa0e41bb --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/text_gogo.go @@ -0,0 +1,57 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "fmt" + "reflect" +) + +func (tm *TextMarshaler) writeEnum(w *textWriter, v reflect.Value, props *Properties) error { + m, ok := enumStringMaps[props.Enum] + if !ok { + if err := tm.writeAny(w, v, props); err != nil { + return err + } + } + key := int32(0) + if v.Kind() == reflect.Ptr { + key = int32(v.Elem().Int()) + } else { + key = int32(v.Int()) + } + s, ok := m[key] + if !ok { + if err := tm.writeAny(w, v, props); err != nil { + return err + } + } + _, err := fmt.Fprint(w, s) + return err +} diff --git a/vendor/github.com/gogo/protobuf/proto/text_parser.go b/vendor/github.com/gogo/protobuf/proto/text_parser.go new file mode 100644 index 0000000000000..fbb000d3742ac --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/text_parser.go @@ -0,0 +1,998 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2013, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2010 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +// Functions for parsing the Text protocol buffer format. +// TODO: message sets. + +import ( + "encoding" + "errors" + "fmt" + "reflect" + "strconv" + "strings" + "time" + "unicode/utf8" +) + +// Error string emitted when deserializing Any and fields are already set +const anyRepeatedlyUnpacked = "Any message unpacked multiple times, or %q already set" + +type ParseError struct { + Message string + Line int // 1-based line number + Offset int // 0-based byte offset from start of input +} + +func (p *ParseError) Error() string { + if p.Line == 1 { + // show offset only for first line + return fmt.Sprintf("line 1.%d: %v", p.Offset, p.Message) + } + return fmt.Sprintf("line %d: %v", p.Line, p.Message) +} + +type token struct { + value string + err *ParseError + line int // line number + offset int // byte number from start of input, not start of line + unquoted string // the unquoted version of value, if it was a quoted string +} + +func (t *token) String() string { + if t.err == nil { + return fmt.Sprintf("%q (line=%d, offset=%d)", t.value, t.line, t.offset) + } + return fmt.Sprintf("parse error: %v", t.err) +} + +type textParser struct { + s string // remaining input + done bool // whether the parsing is finished (success or error) + backed bool // whether back() was called + offset, line int + cur token +} + +func newTextParser(s string) *textParser { + p := new(textParser) + p.s = s + p.line = 1 + p.cur.line = 1 + return p +} + +func (p *textParser) errorf(format string, a ...interface{}) *ParseError { + pe := &ParseError{fmt.Sprintf(format, a...), p.cur.line, p.cur.offset} + p.cur.err = pe + p.done = true + return pe +} + +// Numbers and identifiers are matched by [-+._A-Za-z0-9] +func isIdentOrNumberChar(c byte) bool { + switch { + case 'A' <= c && c <= 'Z', 'a' <= c && c <= 'z': + return true + case '0' <= c && c <= '9': + return true + } + switch c { + case '-', '+', '.', '_': + return true + } + return false +} + +func isWhitespace(c byte) bool { + switch c { + case ' ', '\t', '\n', '\r': + return true + } + return false +} + +func isQuote(c byte) bool { + switch c { + case '"', '\'': + return true + } + return false +} + +func (p *textParser) skipWhitespace() { + i := 0 + for i < len(p.s) && (isWhitespace(p.s[i]) || p.s[i] == '#') { + if p.s[i] == '#' { + // comment; skip to end of line or input + for i < len(p.s) && p.s[i] != '\n' { + i++ + } + if i == len(p.s) { + break + } + } + if p.s[i] == '\n' { + p.line++ + } + i++ + } + p.offset += i + p.s = p.s[i:len(p.s)] + if len(p.s) == 0 { + p.done = true + } +} + +func (p *textParser) advance() { + // Skip whitespace + p.skipWhitespace() + if p.done { + return + } + + // Start of non-whitespace + p.cur.err = nil + p.cur.offset, p.cur.line = p.offset, p.line + p.cur.unquoted = "" + switch p.s[0] { + case '<', '>', '{', '}', ':', '[', ']', ';', ',', '/': + // Single symbol + p.cur.value, p.s = p.s[0:1], p.s[1:len(p.s)] + case '"', '\'': + // Quoted string + i := 1 + for i < len(p.s) && p.s[i] != p.s[0] && p.s[i] != '\n' { + if p.s[i] == '\\' && i+1 < len(p.s) { + // skip escaped char + i++ + } + i++ + } + if i >= len(p.s) || p.s[i] != p.s[0] { + p.errorf("unmatched quote") + return + } + unq, err := unquoteC(p.s[1:i], rune(p.s[0])) + if err != nil { + p.errorf("invalid quoted string %s: %v", p.s[0:i+1], err) + return + } + p.cur.value, p.s = p.s[0:i+1], p.s[i+1:len(p.s)] + p.cur.unquoted = unq + default: + i := 0 + for i < len(p.s) && isIdentOrNumberChar(p.s[i]) { + i++ + } + if i == 0 { + p.errorf("unexpected byte %#x", p.s[0]) + return + } + p.cur.value, p.s = p.s[0:i], p.s[i:len(p.s)] + } + p.offset += len(p.cur.value) +} + +var ( + errBadUTF8 = errors.New("proto: bad UTF-8") +) + +func unquoteC(s string, quote rune) (string, error) { + // This is based on C++'s tokenizer.cc. + // Despite its name, this is *not* parsing C syntax. + // For instance, "\0" is an invalid quoted string. + + // Avoid allocation in trivial cases. + simple := true + for _, r := range s { + if r == '\\' || r == quote { + simple = false + break + } + } + if simple { + return s, nil + } + + buf := make([]byte, 0, 3*len(s)/2) + for len(s) > 0 { + r, n := utf8.DecodeRuneInString(s) + if r == utf8.RuneError && n == 1 { + return "", errBadUTF8 + } + s = s[n:] + if r != '\\' { + if r < utf8.RuneSelf { + buf = append(buf, byte(r)) + } else { + buf = append(buf, string(r)...) + } + continue + } + + ch, tail, err := unescape(s) + if err != nil { + return "", err + } + buf = append(buf, ch...) + s = tail + } + return string(buf), nil +} + +func unescape(s string) (ch string, tail string, err error) { + r, n := utf8.DecodeRuneInString(s) + if r == utf8.RuneError && n == 1 { + return "", "", errBadUTF8 + } + s = s[n:] + switch r { + case 'a': + return "\a", s, nil + case 'b': + return "\b", s, nil + case 'f': + return "\f", s, nil + case 'n': + return "\n", s, nil + case 'r': + return "\r", s, nil + case 't': + return "\t", s, nil + case 'v': + return "\v", s, nil + case '?': + return "?", s, nil // trigraph workaround + case '\'', '"', '\\': + return string(r), s, nil + case '0', '1', '2', '3', '4', '5', '6', '7': + if len(s) < 2 { + return "", "", fmt.Errorf(`\%c requires 2 following digits`, r) + } + ss := string(r) + s[:2] + s = s[2:] + i, err := strconv.ParseUint(ss, 8, 8) + if err != nil { + return "", "", fmt.Errorf(`\%s contains non-octal digits`, ss) + } + return string([]byte{byte(i)}), s, nil + case 'x', 'X', 'u', 'U': + var n int + switch r { + case 'x', 'X': + n = 2 + case 'u': + n = 4 + case 'U': + n = 8 + } + if len(s) < n { + return "", "", fmt.Errorf(`\%c requires %d following digits`, r, n) + } + ss := s[:n] + s = s[n:] + i, err := strconv.ParseUint(ss, 16, 64) + if err != nil { + return "", "", fmt.Errorf(`\%c%s contains non-hexadecimal digits`, r, ss) + } + if r == 'x' || r == 'X' { + return string([]byte{byte(i)}), s, nil + } + if i > utf8.MaxRune { + return "", "", fmt.Errorf(`\%c%s is not a valid Unicode code point`, r, ss) + } + return string(i), s, nil + } + return "", "", fmt.Errorf(`unknown escape \%c`, r) +} + +// Back off the parser by one token. Can only be done between calls to next(). +// It makes the next advance() a no-op. +func (p *textParser) back() { p.backed = true } + +// Advances the parser and returns the new current token. +func (p *textParser) next() *token { + if p.backed || p.done { + p.backed = false + return &p.cur + } + p.advance() + if p.done { + p.cur.value = "" + } else if len(p.cur.value) > 0 && isQuote(p.cur.value[0]) { + // Look for multiple quoted strings separated by whitespace, + // and concatenate them. + cat := p.cur + for { + p.skipWhitespace() + if p.done || !isQuote(p.s[0]) { + break + } + p.advance() + if p.cur.err != nil { + return &p.cur + } + cat.value += " " + p.cur.value + cat.unquoted += p.cur.unquoted + } + p.done = false // parser may have seen EOF, but we want to return cat + p.cur = cat + } + return &p.cur +} + +func (p *textParser) consumeToken(s string) error { + tok := p.next() + if tok.err != nil { + return tok.err + } + if tok.value != s { + p.back() + return p.errorf("expected %q, found %q", s, tok.value) + } + return nil +} + +// Return a RequiredNotSetError indicating which required field was not set. +func (p *textParser) missingRequiredFieldError(sv reflect.Value) *RequiredNotSetError { + st := sv.Type() + sprops := GetProperties(st) + for i := 0; i < st.NumField(); i++ { + if !isNil(sv.Field(i)) { + continue + } + + props := sprops.Prop[i] + if props.Required { + return &RequiredNotSetError{fmt.Sprintf("%v.%v", st, props.OrigName)} + } + } + return &RequiredNotSetError{fmt.Sprintf("%v.", st)} // should not happen +} + +// Returns the index in the struct for the named field, as well as the parsed tag properties. +func structFieldByName(sprops *StructProperties, name string) (int, *Properties, bool) { + i, ok := sprops.decoderOrigNames[name] + if ok { + return i, sprops.Prop[i], true + } + return -1, nil, false +} + +// Consume a ':' from the input stream (if the next token is a colon), +// returning an error if a colon is needed but not present. +func (p *textParser) checkForColon(props *Properties, typ reflect.Type) *ParseError { + tok := p.next() + if tok.err != nil { + return tok.err + } + if tok.value != ":" { + // Colon is optional when the field is a group or message. + needColon := true + switch props.Wire { + case "group": + needColon = false + case "bytes": + // A "bytes" field is either a message, a string, or a repeated field; + // those three become *T, *string and []T respectively, so we can check for + // this field being a pointer to a non-string. + if typ.Kind() == reflect.Ptr { + // *T or *string + if typ.Elem().Kind() == reflect.String { + break + } + } else if typ.Kind() == reflect.Slice { + // []T or []*T + if typ.Elem().Kind() != reflect.Ptr { + break + } + } else if typ.Kind() == reflect.String { + // The proto3 exception is for a string field, + // which requires a colon. + break + } + needColon = false + } + if needColon { + return p.errorf("expected ':', found %q", tok.value) + } + p.back() + } + return nil +} + +func (p *textParser) readStruct(sv reflect.Value, terminator string) error { + st := sv.Type() + sprops := GetProperties(st) + reqCount := sprops.reqCount + var reqFieldErr error + fieldSet := make(map[string]bool) + // A struct is a sequence of "name: value", terminated by one of + // '>' or '}', or the end of the input. A name may also be + // "[extension]" or "[type/url]". + // + // The whole struct can also be an expanded Any message, like: + // [type/url] < ... struct contents ... > + for { + tok := p.next() + if tok.err != nil { + return tok.err + } + if tok.value == terminator { + break + } + if tok.value == "[" { + // Looks like an extension or an Any. + // + // TODO: Check whether we need to handle + // namespace rooted names (e.g. ".something.Foo"). + extName, err := p.consumeExtName() + if err != nil { + return err + } + + if s := strings.LastIndex(extName, "/"); s >= 0 { + // If it contains a slash, it's an Any type URL. + messageName := extName[s+1:] + mt := MessageType(messageName) + if mt == nil { + return p.errorf("unrecognized message %q in google.protobuf.Any", messageName) + } + tok = p.next() + if tok.err != nil { + return tok.err + } + // consume an optional colon + if tok.value == ":" { + tok = p.next() + if tok.err != nil { + return tok.err + } + } + var terminator string + switch tok.value { + case "<": + terminator = ">" + case "{": + terminator = "}" + default: + return p.errorf("expected '{' or '<', found %q", tok.value) + } + v := reflect.New(mt.Elem()) + if pe := p.readStruct(v.Elem(), terminator); pe != nil { + return pe + } + b, err := Marshal(v.Interface().(Message)) + if err != nil { + return p.errorf("failed to marshal message of type %q: %v", messageName, err) + } + if fieldSet["type_url"] { + return p.errorf(anyRepeatedlyUnpacked, "type_url") + } + if fieldSet["value"] { + return p.errorf(anyRepeatedlyUnpacked, "value") + } + sv.FieldByName("TypeUrl").SetString(extName) + sv.FieldByName("Value").SetBytes(b) + fieldSet["type_url"] = true + fieldSet["value"] = true + continue + } + + var desc *ExtensionDesc + // This could be faster, but it's functional. + // TODO: Do something smarter than a linear scan. + for _, d := range RegisteredExtensions(reflect.New(st).Interface().(Message)) { + if d.Name == extName { + desc = d + break + } + } + if desc == nil { + return p.errorf("unrecognized extension %q", extName) + } + + props := &Properties{} + props.Parse(desc.Tag) + + typ := reflect.TypeOf(desc.ExtensionType) + if err := p.checkForColon(props, typ); err != nil { + return err + } + + rep := desc.repeated() + + // Read the extension structure, and set it in + // the value we're constructing. + var ext reflect.Value + if !rep { + ext = reflect.New(typ).Elem() + } else { + ext = reflect.New(typ.Elem()).Elem() + } + if err := p.readAny(ext, props); err != nil { + if _, ok := err.(*RequiredNotSetError); !ok { + return err + } + reqFieldErr = err + } + ep := sv.Addr().Interface().(Message) + if !rep { + SetExtension(ep, desc, ext.Interface()) + } else { + old, err := GetExtension(ep, desc) + var sl reflect.Value + if err == nil { + sl = reflect.ValueOf(old) // existing slice + } else { + sl = reflect.MakeSlice(typ, 0, 1) + } + sl = reflect.Append(sl, ext) + SetExtension(ep, desc, sl.Interface()) + } + if err := p.consumeOptionalSeparator(); err != nil { + return err + } + continue + } + + // This is a normal, non-extension field. + name := tok.value + var dst reflect.Value + fi, props, ok := structFieldByName(sprops, name) + if ok { + dst = sv.Field(fi) + } else if oop, ok := sprops.OneofTypes[name]; ok { + // It is a oneof. + props = oop.Prop + nv := reflect.New(oop.Type.Elem()) + dst = nv.Elem().Field(0) + field := sv.Field(oop.Field) + if !field.IsNil() { + return p.errorf("field '%s' would overwrite already parsed oneof '%s'", name, sv.Type().Field(oop.Field).Name) + } + field.Set(nv) + } + if !dst.IsValid() { + return p.errorf("unknown field name %q in %v", name, st) + } + + if dst.Kind() == reflect.Map { + // Consume any colon. + if err := p.checkForColon(props, dst.Type()); err != nil { + return err + } + + // Construct the map if it doesn't already exist. + if dst.IsNil() { + dst.Set(reflect.MakeMap(dst.Type())) + } + key := reflect.New(dst.Type().Key()).Elem() + val := reflect.New(dst.Type().Elem()).Elem() + + // The map entry should be this sequence of tokens: + // < key : KEY value : VALUE > + // However, implementations may omit key or value, and technically + // we should support them in any order. See b/28924776 for a time + // this went wrong. + + tok := p.next() + var terminator string + switch tok.value { + case "<": + terminator = ">" + case "{": + terminator = "}" + default: + return p.errorf("expected '{' or '<', found %q", tok.value) + } + for { + tok := p.next() + if tok.err != nil { + return tok.err + } + if tok.value == terminator { + break + } + switch tok.value { + case "key": + if err := p.consumeToken(":"); err != nil { + return err + } + if err := p.readAny(key, props.mkeyprop); err != nil { + return err + } + if err := p.consumeOptionalSeparator(); err != nil { + return err + } + case "value": + if err := p.checkForColon(props.mvalprop, dst.Type().Elem()); err != nil { + return err + } + if err := p.readAny(val, props.mvalprop); err != nil { + return err + } + if err := p.consumeOptionalSeparator(); err != nil { + return err + } + default: + p.back() + return p.errorf(`expected "key", "value", or %q, found %q`, terminator, tok.value) + } + } + + dst.SetMapIndex(key, val) + continue + } + + // Check that it's not already set if it's not a repeated field. + if !props.Repeated && fieldSet[name] { + return p.errorf("non-repeated field %q was repeated", name) + } + + if err := p.checkForColon(props, dst.Type()); err != nil { + return err + } + + // Parse into the field. + fieldSet[name] = true + if err := p.readAny(dst, props); err != nil { + if _, ok := err.(*RequiredNotSetError); !ok { + return err + } + reqFieldErr = err + } + if props.Required { + reqCount-- + } + + if err := p.consumeOptionalSeparator(); err != nil { + return err + } + + } + + if reqCount > 0 { + return p.missingRequiredFieldError(sv) + } + return reqFieldErr +} + +// consumeExtName consumes extension name or expanded Any type URL and the +// following ']'. It returns the name or URL consumed. +func (p *textParser) consumeExtName() (string, error) { + tok := p.next() + if tok.err != nil { + return "", tok.err + } + + // If extension name or type url is quoted, it's a single token. + if len(tok.value) > 2 && isQuote(tok.value[0]) && tok.value[len(tok.value)-1] == tok.value[0] { + name, err := unquoteC(tok.value[1:len(tok.value)-1], rune(tok.value[0])) + if err != nil { + return "", err + } + return name, p.consumeToken("]") + } + + // Consume everything up to "]" + var parts []string + for tok.value != "]" { + parts = append(parts, tok.value) + tok = p.next() + if tok.err != nil { + return "", p.errorf("unrecognized type_url or extension name: %s", tok.err) + } + if p.done && tok.value != "]" { + return "", p.errorf("unclosed type_url or extension name") + } + } + return strings.Join(parts, ""), nil +} + +// consumeOptionalSeparator consumes an optional semicolon or comma. +// It is used in readStruct to provide backward compatibility. +func (p *textParser) consumeOptionalSeparator() error { + tok := p.next() + if tok.err != nil { + return tok.err + } + if tok.value != ";" && tok.value != "," { + p.back() + } + return nil +} + +func (p *textParser) readAny(v reflect.Value, props *Properties) error { + tok := p.next() + if tok.err != nil { + return tok.err + } + if tok.value == "" { + return p.errorf("unexpected EOF") + } + if len(props.CustomType) > 0 { + if props.Repeated { + t := reflect.TypeOf(v.Interface()) + if t.Kind() == reflect.Slice { + tc := reflect.TypeOf(new(Marshaler)) + ok := t.Elem().Implements(tc.Elem()) + if ok { + fv := v + flen := fv.Len() + if flen == fv.Cap() { + nav := reflect.MakeSlice(v.Type(), flen, 2*flen+1) + reflect.Copy(nav, fv) + fv.Set(nav) + } + fv.SetLen(flen + 1) + + // Read one. + p.back() + return p.readAny(fv.Index(flen), props) + } + } + } + if reflect.TypeOf(v.Interface()).Kind() == reflect.Ptr { + custom := reflect.New(props.ctype.Elem()).Interface().(Unmarshaler) + err := custom.Unmarshal([]byte(tok.unquoted)) + if err != nil { + return p.errorf("%v %v: %v", err, v.Type(), tok.value) + } + v.Set(reflect.ValueOf(custom)) + } else { + custom := reflect.New(reflect.TypeOf(v.Interface())).Interface().(Unmarshaler) + err := custom.Unmarshal([]byte(tok.unquoted)) + if err != nil { + return p.errorf("%v %v: %v", err, v.Type(), tok.value) + } + v.Set(reflect.Indirect(reflect.ValueOf(custom))) + } + return nil + } + if props.StdTime { + fv := v + p.back() + props.StdTime = false + tproto := ×tamp{} + err := p.readAny(reflect.ValueOf(tproto).Elem(), props) + props.StdTime = true + if err != nil { + return err + } + tim, err := timestampFromProto(tproto) + if err != nil { + return err + } + if props.Repeated { + t := reflect.TypeOf(v.Interface()) + if t.Kind() == reflect.Slice { + if t.Elem().Kind() == reflect.Ptr { + ts := fv.Interface().([]*time.Time) + ts = append(ts, &tim) + fv.Set(reflect.ValueOf(ts)) + return nil + } else { + ts := fv.Interface().([]time.Time) + ts = append(ts, tim) + fv.Set(reflect.ValueOf(ts)) + return nil + } + } + } + if reflect.TypeOf(v.Interface()).Kind() == reflect.Ptr { + v.Set(reflect.ValueOf(&tim)) + } else { + v.Set(reflect.Indirect(reflect.ValueOf(&tim))) + } + return nil + } + if props.StdDuration { + fv := v + p.back() + props.StdDuration = false + dproto := &duration{} + err := p.readAny(reflect.ValueOf(dproto).Elem(), props) + props.StdDuration = true + if err != nil { + return err + } + dur, err := durationFromProto(dproto) + if err != nil { + return err + } + if props.Repeated { + t := reflect.TypeOf(v.Interface()) + if t.Kind() == reflect.Slice { + if t.Elem().Kind() == reflect.Ptr { + ds := fv.Interface().([]*time.Duration) + ds = append(ds, &dur) + fv.Set(reflect.ValueOf(ds)) + return nil + } else { + ds := fv.Interface().([]time.Duration) + ds = append(ds, dur) + fv.Set(reflect.ValueOf(ds)) + return nil + } + } + } + if reflect.TypeOf(v.Interface()).Kind() == reflect.Ptr { + v.Set(reflect.ValueOf(&dur)) + } else { + v.Set(reflect.Indirect(reflect.ValueOf(&dur))) + } + return nil + } + switch fv := v; fv.Kind() { + case reflect.Slice: + at := v.Type() + if at.Elem().Kind() == reflect.Uint8 { + // Special case for []byte + if tok.value[0] != '"' && tok.value[0] != '\'' { + // Deliberately written out here, as the error after + // this switch statement would write "invalid []byte: ...", + // which is not as user-friendly. + return p.errorf("invalid string: %v", tok.value) + } + bytes := []byte(tok.unquoted) + fv.Set(reflect.ValueOf(bytes)) + return nil + } + // Repeated field. + if tok.value == "[" { + // Repeated field with list notation, like [1,2,3]. + for { + fv.Set(reflect.Append(fv, reflect.New(at.Elem()).Elem())) + err := p.readAny(fv.Index(fv.Len()-1), props) + if err != nil { + return err + } + ntok := p.next() + if ntok.err != nil { + return ntok.err + } + if ntok.value == "]" { + break + } + if ntok.value != "," { + return p.errorf("Expected ']' or ',' found %q", ntok.value) + } + } + return nil + } + // One value of the repeated field. + p.back() + fv.Set(reflect.Append(fv, reflect.New(at.Elem()).Elem())) + return p.readAny(fv.Index(fv.Len()-1), props) + case reflect.Bool: + // true/1/t/True or false/f/0/False. + switch tok.value { + case "true", "1", "t", "True": + fv.SetBool(true) + return nil + case "false", "0", "f", "False": + fv.SetBool(false) + return nil + } + case reflect.Float32, reflect.Float64: + v := tok.value + // Ignore 'f' for compatibility with output generated by C++, but don't + // remove 'f' when the value is "-inf" or "inf". + if strings.HasSuffix(v, "f") && tok.value != "-inf" && tok.value != "inf" { + v = v[:len(v)-1] + } + if f, err := strconv.ParseFloat(v, fv.Type().Bits()); err == nil { + fv.SetFloat(f) + return nil + } + case reflect.Int32: + if x, err := strconv.ParseInt(tok.value, 0, 32); err == nil { + fv.SetInt(x) + return nil + } + + if len(props.Enum) == 0 { + break + } + m, ok := enumValueMaps[props.Enum] + if !ok { + break + } + x, ok := m[tok.value] + if !ok { + break + } + fv.SetInt(int64(x)) + return nil + case reflect.Int64: + if x, err := strconv.ParseInt(tok.value, 0, 64); err == nil { + fv.SetInt(x) + return nil + } + + case reflect.Ptr: + // A basic field (indirected through pointer), or a repeated message/group + p.back() + fv.Set(reflect.New(fv.Type().Elem())) + return p.readAny(fv.Elem(), props) + case reflect.String: + if tok.value[0] == '"' || tok.value[0] == '\'' { + fv.SetString(tok.unquoted) + return nil + } + case reflect.Struct: + var terminator string + switch tok.value { + case "{": + terminator = "}" + case "<": + terminator = ">" + default: + return p.errorf("expected '{' or '<', found %q", tok.value) + } + // TODO: Handle nested messages which implement encoding.TextUnmarshaler. + return p.readStruct(fv, terminator) + case reflect.Uint32: + if x, err := strconv.ParseUint(tok.value, 0, 32); err == nil { + fv.SetUint(uint64(x)) + return nil + } + case reflect.Uint64: + if x, err := strconv.ParseUint(tok.value, 0, 64); err == nil { + fv.SetUint(x) + return nil + } + } + return p.errorf("invalid %v: %v", v.Type(), tok.value) +} + +// UnmarshalText reads a protocol buffer in Text format. UnmarshalText resets pb +// before starting to unmarshal, so any existing data in pb is always removed. +// If a required field is not set and no other error occurs, +// UnmarshalText returns *RequiredNotSetError. +func UnmarshalText(s string, pb Message) error { + if um, ok := pb.(encoding.TextUnmarshaler); ok { + return um.UnmarshalText([]byte(s)) + } + pb.Reset() + v := reflect.ValueOf(pb) + return newTextParser(s).readStruct(v.Elem(), "") +} diff --git a/vendor/github.com/gogo/protobuf/proto/timestamp.go b/vendor/github.com/gogo/protobuf/proto/timestamp.go new file mode 100644 index 0000000000000..9324f6542bcf7 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/timestamp.go @@ -0,0 +1,113 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2016 The Go Authors. All rights reserved. +// https://github.com/golang/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +// This file implements operations on google.protobuf.Timestamp. + +import ( + "errors" + "fmt" + "time" +) + +const ( + // Seconds field of the earliest valid Timestamp. + // This is time.Date(1, 1, 1, 0, 0, 0, 0, time.UTC).Unix(). + minValidSeconds = -62135596800 + // Seconds field just after the latest valid Timestamp. + // This is time.Date(10000, 1, 1, 0, 0, 0, 0, time.UTC).Unix(). + maxValidSeconds = 253402300800 +) + +// validateTimestamp determines whether a Timestamp is valid. +// A valid timestamp represents a time in the range +// [0001-01-01, 10000-01-01) and has a Nanos field +// in the range [0, 1e9). +// +// If the Timestamp is valid, validateTimestamp returns nil. +// Otherwise, it returns an error that describes +// the problem. +// +// Every valid Timestamp can be represented by a time.Time, but the converse is not true. +func validateTimestamp(ts *timestamp) error { + if ts == nil { + return errors.New("timestamp: nil Timestamp") + } + if ts.Seconds < minValidSeconds { + return fmt.Errorf("timestamp: %#v before 0001-01-01", ts) + } + if ts.Seconds >= maxValidSeconds { + return fmt.Errorf("timestamp: %#v after 10000-01-01", ts) + } + if ts.Nanos < 0 || ts.Nanos >= 1e9 { + return fmt.Errorf("timestamp: %#v: nanos not in range [0, 1e9)", ts) + } + return nil +} + +// TimestampFromProto converts a google.protobuf.Timestamp proto to a time.Time. +// It returns an error if the argument is invalid. +// +// Unlike most Go functions, if Timestamp returns an error, the first return value +// is not the zero time.Time. Instead, it is the value obtained from the +// time.Unix function when passed the contents of the Timestamp, in the UTC +// locale. This may or may not be a meaningful time; many invalid Timestamps +// do map to valid time.Times. +// +// A nil Timestamp returns an error. The first return value in that case is +// undefined. +func timestampFromProto(ts *timestamp) (time.Time, error) { + // Don't return the zero value on error, because corresponds to a valid + // timestamp. Instead return whatever time.Unix gives us. + var t time.Time + if ts == nil { + t = time.Unix(0, 0).UTC() // treat nil like the empty Timestamp + } else { + t = time.Unix(ts.Seconds, int64(ts.Nanos)).UTC() + } + return t, validateTimestamp(ts) +} + +// TimestampProto converts the time.Time to a google.protobuf.Timestamp proto. +// It returns an error if the resulting Timestamp is invalid. +func timestampProto(t time.Time) (*timestamp, error) { + seconds := t.Unix() + nanos := int32(t.Sub(time.Unix(seconds, 0))) + ts := ×tamp{ + Seconds: seconds, + Nanos: nanos, + } + if err := validateTimestamp(ts); err != nil { + return nil, err + } + return ts, nil +} diff --git a/vendor/github.com/gogo/protobuf/proto/timestamp_gogo.go b/vendor/github.com/gogo/protobuf/proto/timestamp_gogo.go new file mode 100644 index 0000000000000..38439fa990132 --- /dev/null +++ b/vendor/github.com/gogo/protobuf/proto/timestamp_gogo.go @@ -0,0 +1,49 @@ +// Protocol Buffers for Go with Gadgets +// +// Copyright (c) 2016, The GoGo Authors. All rights reserved. +// http://github.com/gogo/protobuf +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package proto + +import ( + "reflect" + "time" +) + +var timeType = reflect.TypeOf((*time.Time)(nil)).Elem() + +type timestamp struct { + Seconds int64 `protobuf:"varint,1,opt,name=seconds,proto3" json:"seconds,omitempty"` + Nanos int32 `protobuf:"varint,2,opt,name=nanos,proto3" json:"nanos,omitempty"` +} + +func (m *timestamp) Reset() { *m = timestamp{} } +func (*timestamp) ProtoMessage() {} +func (*timestamp) String() string { return "timestamp" } + +func init() { + RegisterType((*timestamp)(nil), "gogo.protobuf.proto.timestamp") +} diff --git a/vendor/github.com/opentracing/basictracer-go/LICENSE b/vendor/github.com/opentracing/basictracer-go/LICENSE new file mode 100644 index 0000000000000..148509a4035a9 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/LICENSE @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2016 The OpenTracing Authors + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/vendor/github.com/opentracing/basictracer-go/context.go b/vendor/github.com/opentracing/basictracer-go/context.go new file mode 100644 index 0000000000000..a2db9ffe889f3 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/context.go @@ -0,0 +1,42 @@ +package basictracer + +// SpanContext holds the basic Span metadata. +type SpanContext struct { + // A probabilistically unique identifier for a [multi-span] trace. + TraceID uint64 + + // A probabilistically unique identifier for a span. + SpanID uint64 + + // Whether the trace is sampled. + Sampled bool + + // The span's associated baggage. + Baggage map[string]string // initialized on first use +} + +// ForeachBaggageItem belongs to the opentracing.SpanContext interface +func (c SpanContext) ForeachBaggageItem(handler func(k, v string) bool) { + for k, v := range c.Baggage { + if !handler(k, v) { + break + } + } +} + +// WithBaggageItem returns an entirely new basictracer SpanContext with the +// given key:value baggage pair set. +func (c SpanContext) WithBaggageItem(key, val string) SpanContext { + var newBaggage map[string]string + if c.Baggage == nil { + newBaggage = map[string]string{key: val} + } else { + newBaggage = make(map[string]string, len(c.Baggage)+1) + for k, v := range c.Baggage { + newBaggage[k] = v + } + newBaggage[key] = val + } + // Use positional parameters so the compiler will help catch new fields. + return SpanContext{c.TraceID, c.SpanID, c.Sampled, newBaggage} +} diff --git a/vendor/github.com/opentracing/basictracer-go/debug.go b/vendor/github.com/opentracing/basictracer-go/debug.go new file mode 100644 index 0000000000000..817061927f5e6 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/debug.go @@ -0,0 +1,78 @@ +package basictracer + +import ( + "bytes" + "fmt" + "runtime" + "strconv" + "sync" +) + +const debugGoroutineIDTag = "_initial_goroutine" + +type errAssertionFailed struct { + span *spanImpl + msg string +} + +// Error implements the error interface. +func (err *errAssertionFailed) Error() string { + return fmt.Sprintf("%s:\n%+v", err.msg, err.span) +} + +func (s *spanImpl) Lock() { + s.Mutex.Lock() + s.maybeAssertSanityLocked() +} + +func (s *spanImpl) maybeAssertSanityLocked() { + if s.tracer == nil { + s.Mutex.Unlock() + panic(&errAssertionFailed{span: s, msg: "span used after call to Finish()"}) + } + if s.tracer.options.DebugAssertSingleGoroutine { + startID := curGoroutineID() + curID, ok := s.raw.Tags[debugGoroutineIDTag].(uint64) + if !ok { + // This is likely invoked in the context of the SetTag which sets + // debugGoroutineTag. + return + } + if startID != curID { + s.Mutex.Unlock() + panic(&errAssertionFailed{ + span: s, + msg: fmt.Sprintf("span started on goroutine %d, but now running on %d", startID, curID), + }) + } + } +} + +var goroutineSpace = []byte("goroutine ") +var littleBuf = sync.Pool{ + New: func() interface{} { + buf := make([]byte, 64) + return &buf + }, +} + +// Credit to @bradfitz: +// https://github.com/golang/net/blob/master/http2/gotrack.go#L51 +func curGoroutineID() uint64 { + bp := littleBuf.Get().(*[]byte) + defer littleBuf.Put(bp) + b := *bp + b = b[:runtime.Stack(b, false)] + // Parse the 4707 out of "goroutine 4707 [" + b = bytes.TrimPrefix(b, goroutineSpace) + i := bytes.IndexByte(b, ' ') + if i < 0 { + panic(fmt.Sprintf("No space found in %q", b)) + } + b = b[:i] + n, err := strconv.ParseUint(string(b), 10, 64) + if err != nil { + panic(fmt.Sprintf("Failed to parse goroutine ID out of %q: %v", b, err)) + } + return n +} diff --git a/vendor/github.com/opentracing/basictracer-go/event.go b/vendor/github.com/opentracing/basictracer-go/event.go new file mode 100644 index 0000000000000..e074fcc7d86ad --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/event.go @@ -0,0 +1,62 @@ +package basictracer + +import "github.com/opentracing/opentracing-go" + +// A SpanEvent is emitted when a mutating command is called on a Span. +type SpanEvent interface{} + +// EventCreate is emitted when a Span is created. +type EventCreate struct{ OperationName string } + +// EventTag is received when SetTag is called. +type EventTag struct { + Key string + Value interface{} +} + +// EventBaggage is received when SetBaggageItem is called. +type EventBaggage struct { + Key, Value string +} + +// EventLogFields is received when LogFields or LogKV is called. +type EventLogFields opentracing.LogRecord + +// EventLog is received when Log (or one of its derivatives) is called. +// +// DEPRECATED +type EventLog opentracing.LogData + +// EventFinish is received when Finish is called. +type EventFinish RawSpan + +func (s *spanImpl) onCreate(opName string) { + if s.event != nil { + s.event(EventCreate{OperationName: opName}) + } +} +func (s *spanImpl) onTag(key string, value interface{}) { + if s.event != nil { + s.event(EventTag{Key: key, Value: value}) + } +} +func (s *spanImpl) onLog(ld opentracing.LogData) { + if s.event != nil { + s.event(EventLog(ld)) + } +} +func (s *spanImpl) onLogFields(lr opentracing.LogRecord) { + if s.event != nil { + s.event(EventLogFields(lr)) + } +} +func (s *spanImpl) onBaggage(key, value string) { + if s.event != nil { + s.event(EventBaggage{Key: key, Value: value}) + } +} +func (s *spanImpl) onFinish(sp RawSpan) { + if s.event != nil { + s.event(EventFinish(sp)) + } +} diff --git a/vendor/github.com/opentracing/basictracer-go/propagation.go b/vendor/github.com/opentracing/basictracer-go/propagation.go new file mode 100644 index 0000000000000..15970c696486c --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/propagation.go @@ -0,0 +1,61 @@ +package basictracer + +import opentracing "github.com/opentracing/opentracing-go" + +type accessorPropagator struct { + tracer *tracerImpl +} + +// DelegatingCarrier is a flexible carrier interface which can be implemented +// by types which have a means of storing the trace metadata and already know +// how to serialize themselves (for example, protocol buffers). +type DelegatingCarrier interface { + SetState(traceID, spanID uint64, sampled bool) + State() (traceID, spanID uint64, sampled bool) + SetBaggageItem(key, value string) + GetBaggage(func(key, value string)) +} + +func (p *accessorPropagator) Inject( + spanContext opentracing.SpanContext, + carrier interface{}, +) error { + dc, ok := carrier.(DelegatingCarrier) + if !ok || dc == nil { + return opentracing.ErrInvalidCarrier + } + sc, ok := spanContext.(SpanContext) + if !ok { + return opentracing.ErrInvalidSpanContext + } + dc.SetState(sc.TraceID, sc.SpanID, sc.Sampled) + for k, v := range sc.Baggage { + dc.SetBaggageItem(k, v) + } + return nil +} + +func (p *accessorPropagator) Extract( + carrier interface{}, +) (opentracing.SpanContext, error) { + dc, ok := carrier.(DelegatingCarrier) + if !ok || dc == nil { + return nil, opentracing.ErrInvalidCarrier + } + + traceID, spanID, sampled := dc.State() + sc := SpanContext{ + TraceID: traceID, + SpanID: spanID, + Sampled: sampled, + Baggage: nil, + } + dc.GetBaggage(func(k, v string) { + if sc.Baggage == nil { + sc.Baggage = map[string]string{} + } + sc.Baggage[k] = v + }) + + return sc, nil +} diff --git a/vendor/github.com/opentracing/basictracer-go/propagation_ot.go b/vendor/github.com/opentracing/basictracer-go/propagation_ot.go new file mode 100644 index 0000000000000..61362ceecaab9 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/propagation_ot.go @@ -0,0 +1,180 @@ +package basictracer + +import ( + "encoding/binary" + "io" + "strconv" + "strings" + + "github.com/gogo/protobuf/proto" + "github.com/opentracing/basictracer-go/wire" + opentracing "github.com/opentracing/opentracing-go" +) + +type textMapPropagator struct { + tracer *tracerImpl +} +type binaryPropagator struct { + tracer *tracerImpl +} + +const ( + prefixTracerState = "ot-tracer-" + prefixBaggage = "ot-baggage-" + + tracerStateFieldCount = 3 + fieldNameTraceID = prefixTracerState + "traceid" + fieldNameSpanID = prefixTracerState + "spanid" + fieldNameSampled = prefixTracerState + "sampled" +) + +func (p *textMapPropagator) Inject( + spanContext opentracing.SpanContext, + opaqueCarrier interface{}, +) error { + sc, ok := spanContext.(SpanContext) + if !ok { + return opentracing.ErrInvalidSpanContext + } + carrier, ok := opaqueCarrier.(opentracing.TextMapWriter) + if !ok { + return opentracing.ErrInvalidCarrier + } + carrier.Set(fieldNameTraceID, strconv.FormatUint(sc.TraceID, 16)) + carrier.Set(fieldNameSpanID, strconv.FormatUint(sc.SpanID, 16)) + carrier.Set(fieldNameSampled, strconv.FormatBool(sc.Sampled)) + + for k, v := range sc.Baggage { + carrier.Set(prefixBaggage+k, v) + } + return nil +} + +func (p *textMapPropagator) Extract( + opaqueCarrier interface{}, +) (opentracing.SpanContext, error) { + carrier, ok := opaqueCarrier.(opentracing.TextMapReader) + if !ok { + return nil, opentracing.ErrInvalidCarrier + } + requiredFieldCount := 0 + var traceID, spanID uint64 + var sampled bool + var err error + decodedBaggage := make(map[string]string) + err = carrier.ForeachKey(func(k, v string) error { + switch strings.ToLower(k) { + case fieldNameTraceID: + traceID, err = strconv.ParseUint(v, 16, 64) + if err != nil { + return opentracing.ErrSpanContextCorrupted + } + case fieldNameSpanID: + spanID, err = strconv.ParseUint(v, 16, 64) + if err != nil { + return opentracing.ErrSpanContextCorrupted + } + case fieldNameSampled: + sampled, err = strconv.ParseBool(v) + if err != nil { + return opentracing.ErrSpanContextCorrupted + } + default: + lowercaseK := strings.ToLower(k) + if strings.HasPrefix(lowercaseK, prefixBaggage) { + decodedBaggage[strings.TrimPrefix(lowercaseK, prefixBaggage)] = v + } + // Balance off the requiredFieldCount++ just below... + requiredFieldCount-- + } + requiredFieldCount++ + return nil + }) + if err != nil { + return nil, err + } + if requiredFieldCount < tracerStateFieldCount { + if requiredFieldCount == 0 { + return nil, opentracing.ErrSpanContextNotFound + } + return nil, opentracing.ErrSpanContextCorrupted + } + + return SpanContext{ + TraceID: traceID, + SpanID: spanID, + Sampled: sampled, + Baggage: decodedBaggage, + }, nil +} + +func (p *binaryPropagator) Inject( + spanContext opentracing.SpanContext, + opaqueCarrier interface{}, +) error { + sc, ok := spanContext.(SpanContext) + if !ok { + return opentracing.ErrInvalidSpanContext + } + carrier, ok := opaqueCarrier.(io.Writer) + if !ok { + return opentracing.ErrInvalidCarrier + } + + state := wire.TracerState{} + state.TraceId = sc.TraceID + state.SpanId = sc.SpanID + state.Sampled = sc.Sampled + state.BaggageItems = sc.Baggage + + b, err := proto.Marshal(&state) + if err != nil { + return err + } + + // Write the length of the marshalled binary to the writer. + length := uint32(len(b)) + if err := binary.Write(carrier, binary.BigEndian, &length); err != nil { + return err + } + + _, err = carrier.Write(b) + return err +} + +func (p *binaryPropagator) Extract( + opaqueCarrier interface{}, +) (opentracing.SpanContext, error) { + carrier, ok := opaqueCarrier.(io.Reader) + if !ok { + return nil, opentracing.ErrInvalidCarrier + } + + // Read the length of marshalled binary. io.ReadAll isn't that performant + // since it keeps resizing the underlying buffer as it encounters more bytes + // to read. By reading the length, we can allocate a fixed sized buf and read + // the exact amount of bytes into it. + var length uint32 + if err := binary.Read(carrier, binary.BigEndian, &length); err != nil { + return nil, opentracing.ErrSpanContextCorrupted + } + buf := make([]byte, length) + if n, err := carrier.Read(buf); err != nil { + if n > 0 { + return nil, opentracing.ErrSpanContextCorrupted + } + return nil, opentracing.ErrSpanContextNotFound + } + + ctx := wire.TracerState{} + if err := proto.Unmarshal(buf, &ctx); err != nil { + return nil, opentracing.ErrSpanContextCorrupted + } + + return SpanContext{ + TraceID: ctx.TraceId, + SpanID: ctx.SpanId, + Sampled: ctx.Sampled, + Baggage: ctx.BaggageItems, + }, nil +} diff --git a/vendor/github.com/opentracing/basictracer-go/raw.go b/vendor/github.com/opentracing/basictracer-go/raw.go new file mode 100644 index 0000000000000..e038b3694c103 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/raw.go @@ -0,0 +1,34 @@ +package basictracer + +import ( + "time" + + opentracing "github.com/opentracing/opentracing-go" +) + +// RawSpan encapsulates all state associated with a (finished) Span. +type RawSpan struct { + // Those recording the RawSpan should also record the contents of its + // SpanContext. + Context SpanContext + + // The SpanID of this SpanContext's first intra-trace reference (i.e., + // "parent"), or 0 if there is no parent. + ParentSpanID uint64 + + // The name of the "operation" this span is an instance of. (Called a "span + // name" in some implementations) + Operation string + + // We store rather than so that only + // one of the timestamps has global clock uncertainty issues. + Start time.Time + Duration time.Duration + + // Essentially an extension mechanism. Can be used for many purposes, + // not to be enumerated here. + Tags opentracing.Tags + + // The span's "microlog". + Logs []opentracing.LogRecord +} diff --git a/vendor/github.com/opentracing/basictracer-go/recorder.go b/vendor/github.com/opentracing/basictracer-go/recorder.go new file mode 100644 index 0000000000000..d6a0856c33a71 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/recorder.go @@ -0,0 +1,60 @@ +package basictracer + +import "sync" + +// A SpanRecorder handles all of the `RawSpan` data generated via an +// associated `Tracer` (see `NewStandardTracer`) instance. It also names +// the containing process and provides access to a straightforward tag map. +type SpanRecorder interface { + // Implementations must determine whether and where to store `span`. + RecordSpan(span RawSpan) +} + +// InMemorySpanRecorder is a simple thread-safe implementation of +// SpanRecorder that stores all reported spans in memory, accessible +// via reporter.GetSpans(). It is primarily intended for testing purposes. +type InMemorySpanRecorder struct { + sync.RWMutex + spans []RawSpan +} + +// NewInMemoryRecorder creates new InMemorySpanRecorder +func NewInMemoryRecorder() *InMemorySpanRecorder { + return new(InMemorySpanRecorder) +} + +// RecordSpan implements the respective method of SpanRecorder. +func (r *InMemorySpanRecorder) RecordSpan(span RawSpan) { + r.Lock() + defer r.Unlock() + r.spans = append(r.spans, span) +} + +// GetSpans returns a copy of the array of spans accumulated so far. +func (r *InMemorySpanRecorder) GetSpans() []RawSpan { + r.RLock() + defer r.RUnlock() + spans := make([]RawSpan, len(r.spans)) + copy(spans, r.spans) + return spans +} + +// GetSampledSpans returns a slice of spans accumulated so far which were sampled. +func (r *InMemorySpanRecorder) GetSampledSpans() []RawSpan { + r.RLock() + defer r.RUnlock() + spans := make([]RawSpan, 0, len(r.spans)) + for _, span := range r.spans { + if span.Context.Sampled { + spans = append(spans, span) + } + } + return spans +} + +// Reset clears the internal array of spans. +func (r *InMemorySpanRecorder) Reset() { + r.Lock() + defer r.Unlock() + r.spans = nil +} diff --git a/vendor/github.com/opentracing/basictracer-go/span.go b/vendor/github.com/opentracing/basictracer-go/span.go new file mode 100644 index 0000000000000..fdceba5bcbdb4 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/span.go @@ -0,0 +1,274 @@ +package basictracer + +import ( + "sync" + "time" + + opentracing "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go/ext" + "github.com/opentracing/opentracing-go/log" +) + +// Span provides access to the essential details of the span, for use +// by basictracer consumers. These methods may only be called prior +// to (*opentracing.Span).Finish(). +type Span interface { + opentracing.Span + + // Operation names the work done by this span instance + Operation() string + + // Start indicates when the span began + Start() time.Time +} + +// Implements the `Span` interface. Created via tracerImpl (see +// `basictracer.New()`). +type spanImpl struct { + tracer *tracerImpl + event func(SpanEvent) + sync.Mutex // protects the fields below + raw RawSpan + // The number of logs dropped because of MaxLogsPerSpan. + numDroppedLogs int +} + +var spanPool = &sync.Pool{New: func() interface{} { + return &spanImpl{} +}} + +func (s *spanImpl) reset() { + s.tracer, s.event = nil, nil + // Note: Would like to do the following, but then the consumer of RawSpan + // (the recorder) needs to make sure that they're not holding on to the + // baggage or logs when they return (i.e. they need to copy if they care): + // + // logs, baggage := s.raw.Logs[:0], s.raw.Baggage + // for k := range baggage { + // delete(baggage, k) + // } + // s.raw.Logs, s.raw.Baggage = logs, baggage + // + // That's likely too much to ask for. But there is some magic we should + // be able to do with `runtime.SetFinalizer` to reclaim that memory into + // a buffer pool when GC considers them unreachable, which should ease + // some of the load. Hard to say how quickly that would be in practice + // though. + s.raw = RawSpan{ + Context: SpanContext{}, + } +} + +func (s *spanImpl) SetOperationName(operationName string) opentracing.Span { + s.Lock() + defer s.Unlock() + s.raw.Operation = operationName + return s +} + +func (s *spanImpl) trim() bool { + return !s.raw.Context.Sampled && s.tracer.options.TrimUnsampledSpans +} + +func (s *spanImpl) SetTag(key string, value interface{}) opentracing.Span { + defer s.onTag(key, value) + s.Lock() + defer s.Unlock() + if key == string(ext.SamplingPriority) { + if v, ok := value.(uint16); ok { + s.raw.Context.Sampled = v != 0 + return s + } + } + if s.trim() { + return s + } + + if s.raw.Tags == nil { + s.raw.Tags = opentracing.Tags{} + } + s.raw.Tags[key] = value + return s +} + +func (s *spanImpl) LogKV(keyValues ...interface{}) { + fields, err := log.InterleavedKVToFields(keyValues...) + if err != nil { + s.LogFields(log.Error(err), log.String("function", "LogKV")) + return + } + s.LogFields(fields...) +} + +func (s *spanImpl) appendLog(lr opentracing.LogRecord) { + maxLogs := s.tracer.options.MaxLogsPerSpan + if maxLogs == 0 || len(s.raw.Logs) < maxLogs { + s.raw.Logs = append(s.raw.Logs, lr) + return + } + + // We have too many logs. We don't touch the first numOld logs; we treat the + // rest as a circular buffer and overwrite the oldest log among those. + numOld := (maxLogs - 1) / 2 + numNew := maxLogs - numOld + s.raw.Logs[numOld+s.numDroppedLogs%numNew] = lr + s.numDroppedLogs++ +} + +func (s *spanImpl) LogFields(fields ...log.Field) { + lr := opentracing.LogRecord{ + Fields: fields, + } + defer s.onLogFields(lr) + s.Lock() + defer s.Unlock() + if s.trim() || s.tracer.options.DropAllLogs { + return + } + if lr.Timestamp.IsZero() { + lr.Timestamp = time.Now() + } + s.appendLog(lr) +} + +func (s *spanImpl) LogEvent(event string) { + s.Log(opentracing.LogData{ + Event: event, + }) +} + +func (s *spanImpl) LogEventWithPayload(event string, payload interface{}) { + s.Log(opentracing.LogData{ + Event: event, + Payload: payload, + }) +} + +func (s *spanImpl) Log(ld opentracing.LogData) { + defer s.onLog(ld) + s.Lock() + defer s.Unlock() + if s.trim() || s.tracer.options.DropAllLogs { + return + } + + if ld.Timestamp.IsZero() { + ld.Timestamp = time.Now() + } + + s.appendLog(ld.ToLogRecord()) +} + +func (s *spanImpl) Finish() { + s.FinishWithOptions(opentracing.FinishOptions{}) +} + +// rotateLogBuffer rotates the records in the buffer: records 0 to pos-1 move at +// the end (i.e. pos circular left shifts). +func rotateLogBuffer(buf []opentracing.LogRecord, pos int) { + // This algorithm is described in: + // http://www.cplusplus.com/reference/algorithm/rotate + for first, middle, next := 0, pos, pos; first != middle; { + buf[first], buf[next] = buf[next], buf[first] + first++ + next++ + if next == len(buf) { + next = middle + } else if first == middle { + middle = next + } + } +} + +func (s *spanImpl) FinishWithOptions(opts opentracing.FinishOptions) { + finishTime := opts.FinishTime + if finishTime.IsZero() { + finishTime = time.Now() + } + duration := finishTime.Sub(s.raw.Start) + + s.Lock() + defer s.Unlock() + + for _, lr := range opts.LogRecords { + s.appendLog(lr) + } + for _, ld := range opts.BulkLogData { + s.appendLog(ld.ToLogRecord()) + } + + if s.numDroppedLogs > 0 { + // We dropped some log events, which means that we used part of Logs as a + // circular buffer (see appendLog). De-circularize it. + numOld := (len(s.raw.Logs) - 1) / 2 + numNew := len(s.raw.Logs) - numOld + rotateLogBuffer(s.raw.Logs[numOld:], s.numDroppedLogs%numNew) + + // Replace the log in the middle (the oldest "new" log) with information + // about the dropped logs. This means that we are effectively dropping one + // more "new" log. + numDropped := s.numDroppedLogs + 1 + s.raw.Logs[numOld] = opentracing.LogRecord{ + // Keep the timestamp of the last dropped event. + Timestamp: s.raw.Logs[numOld].Timestamp, + Fields: []log.Field{ + log.String("event", "dropped Span logs"), + log.Int("dropped_log_count", numDropped), + log.String("component", "basictracer"), + }, + } + } + + s.raw.Duration = duration + + s.onFinish(s.raw) + s.tracer.options.Recorder.RecordSpan(s.raw) + + // Last chance to get options before the span is possibly reset. + poolEnabled := s.tracer.options.EnableSpanPool + if s.tracer.options.DebugAssertUseAfterFinish { + // This makes it much more likely to catch a panic on any subsequent + // operation since s.tracer is accessed on every call to `Lock`. + // We don't call `reset()` here to preserve the logs in the Span + // which are printed when the assertion triggers. + s.tracer = nil + } + + if poolEnabled { + spanPool.Put(s) + } +} + +func (s *spanImpl) Tracer() opentracing.Tracer { + return s.tracer +} + +func (s *spanImpl) Context() opentracing.SpanContext { + return s.raw.Context +} + +func (s *spanImpl) SetBaggageItem(key, val string) opentracing.Span { + s.onBaggage(key, val) + if s.trim() { + return s + } + + s.Lock() + defer s.Unlock() + s.raw.Context = s.raw.Context.WithBaggageItem(key, val) + return s +} + +func (s *spanImpl) BaggageItem(key string) string { + s.Lock() + defer s.Unlock() + return s.raw.Context.Baggage[key] +} + +func (s *spanImpl) Operation() string { + return s.raw.Operation +} + +func (s *spanImpl) Start() time.Time { + return s.raw.Start +} diff --git a/vendor/github.com/opentracing/basictracer-go/tracer.go b/vendor/github.com/opentracing/basictracer-go/tracer.go new file mode 100644 index 0000000000000..3e03696e68244 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/tracer.go @@ -0,0 +1,262 @@ +package basictracer + +import ( + "time" + + opentracing "github.com/opentracing/opentracing-go" +) + +// Tracer extends the opentracing.Tracer interface with methods to +// probe implementation state, for use by basictracer consumers. +type Tracer interface { + opentracing.Tracer + + // Options gets the Options used in New() or NewWithOptions(). + Options() Options +} + +// Options allows creating a customized Tracer via NewWithOptions. The object +// must not be updated when there is an active tracer using it. +type Options struct { + // ShouldSample is a function which is called when creating a new Span and + // determines whether that Span is sampled. The randomized TraceID is supplied + // to allow deterministic sampling decisions to be made across different nodes. + // For example, + // + // func(traceID uint64) { return traceID % 64 == 0 } + // + // samples every 64th trace on average. + ShouldSample func(traceID uint64) bool + // TrimUnsampledSpans turns potentially expensive operations on unsampled + // Spans into no-ops. More precisely, tags and log events are silently + // discarded. If NewSpanEventListener is set, the callbacks will still fire. + TrimUnsampledSpans bool + // Recorder receives Spans which have been finished. + Recorder SpanRecorder + // NewSpanEventListener can be used to enhance the tracer by effectively + // attaching external code to trace events. See NetTraceIntegrator for a + // practical example, and event.go for the list of possible events. + NewSpanEventListener func() func(SpanEvent) + // DropAllLogs turns log events on all Spans into no-ops. + // If NewSpanEventListener is set, the callbacks will still fire. + DropAllLogs bool + // MaxLogsPerSpan limits the number of Logs in a span (if set to a nonzero + // value). If a span has more logs than this value, logs are dropped as + // necessary (and replaced with a log describing how many were dropped). + // + // About half of the MaxLogPerSpan logs kept are the oldest logs, and about + // half are the newest logs. + // + // If NewSpanEventListener is set, the callbacks will still fire for all log + // events. This value is ignored if DropAllLogs is true. + MaxLogsPerSpan int + // DebugAssertSingleGoroutine internally records the ID of the goroutine + // creating each Span and verifies that no operation is carried out on + // it on a different goroutine. + // Provided strictly for development purposes. + // Passing Spans between goroutine without proper synchronization often + // results in use-after-Finish() errors. For a simple example, consider the + // following pseudocode: + // + // func (s *Server) Handle(req http.Request) error { + // sp := s.StartSpan("server") + // defer sp.Finish() + // wait := s.queueProcessing(opentracing.ContextWithSpan(context.Background(), sp), req) + // select { + // case resp := <-wait: + // return resp.Error + // case <-time.After(10*time.Second): + // sp.LogEvent("timed out waiting for processing") + // return ErrTimedOut + // } + // } + // + // This looks reasonable at first, but a request which spends more than ten + // seconds in the queue is abandoned by the main goroutine and its trace + // finished, leading to use-after-finish when the request is finally + // processed. Note also that even joining on to a finished Span via + // StartSpanWithOptions constitutes an illegal operation. + // + // Code bases which do not require (or decide they do not want) Spans to + // be passed across goroutine boundaries can run with this flag enabled in + // tests to increase their chances of spotting wrong-doers. + DebugAssertSingleGoroutine bool + // DebugAssertUseAfterFinish is provided strictly for development purposes. + // When set, it attempts to exacerbate issues emanating from use of Spans + // after calling Finish by running additional assertions. + DebugAssertUseAfterFinish bool + // EnableSpanPool enables the use of a pool, so that the tracer reuses spans + // after Finish has been called on it. Adds a slight performance gain as it + // reduces allocations. However, if you have any use-after-finish race + // conditions the code may panic. + EnableSpanPool bool +} + +// DefaultOptions returns an Options object with a 1 in 64 sampling rate and +// all options disabled. A Recorder needs to be set manually before using the +// returned object with a Tracer. +func DefaultOptions() Options { + return Options{ + ShouldSample: func(traceID uint64) bool { return traceID%64 == 0 }, + MaxLogsPerSpan: 100, + } +} + +// NewWithOptions creates a customized Tracer. +func NewWithOptions(opts Options) opentracing.Tracer { + rval := &tracerImpl{options: opts} + rval.textPropagator = &textMapPropagator{rval} + rval.binaryPropagator = &binaryPropagator{rval} + rval.accessorPropagator = &accessorPropagator{rval} + return rval +} + +// New creates and returns a standard Tracer which defers completed Spans to +// `recorder`. +// Spans created by this Tracer support the ext.SamplingPriority tag: Setting +// ext.SamplingPriority causes the Span to be Sampled from that point on. +func New(recorder SpanRecorder) opentracing.Tracer { + opts := DefaultOptions() + opts.Recorder = recorder + return NewWithOptions(opts) +} + +// Implements the `Tracer` interface. +type tracerImpl struct { + options Options + textPropagator *textMapPropagator + binaryPropagator *binaryPropagator + accessorPropagator *accessorPropagator +} + +func (t *tracerImpl) StartSpan( + operationName string, + opts ...opentracing.StartSpanOption, +) opentracing.Span { + sso := opentracing.StartSpanOptions{} + for _, o := range opts { + o.Apply(&sso) + } + return t.StartSpanWithOptions(operationName, sso) +} + +func (t *tracerImpl) getSpan() *spanImpl { + if t.options.EnableSpanPool { + sp := spanPool.Get().(*spanImpl) + sp.reset() + return sp + } + return &spanImpl{} +} + +func (t *tracerImpl) StartSpanWithOptions( + operationName string, + opts opentracing.StartSpanOptions, +) opentracing.Span { + // Start time. + startTime := opts.StartTime + if startTime.IsZero() { + startTime = time.Now() + } + + // Tags. + tags := opts.Tags + + // Build the new span. This is the only allocation: We'll return this as + // an opentracing.Span. + sp := t.getSpan() + // Look for a parent in the list of References. + // + // TODO: would be nice if basictracer did something with all + // References, not just the first one. +ReferencesLoop: + for _, ref := range opts.References { + switch ref.Type { + case opentracing.ChildOfRef, + opentracing.FollowsFromRef: + + refCtx := ref.ReferencedContext.(SpanContext) + sp.raw.Context.TraceID = refCtx.TraceID + sp.raw.Context.SpanID = randomID() + sp.raw.Context.Sampled = refCtx.Sampled + sp.raw.ParentSpanID = refCtx.SpanID + + if l := len(refCtx.Baggage); l > 0 { + sp.raw.Context.Baggage = make(map[string]string, l) + for k, v := range refCtx.Baggage { + sp.raw.Context.Baggage[k] = v + } + } + break ReferencesLoop + } + } + if sp.raw.Context.TraceID == 0 { + // No parent Span found; allocate new trace and span ids and determine + // the Sampled status. + sp.raw.Context.TraceID, sp.raw.Context.SpanID = randomID2() + sp.raw.Context.Sampled = t.options.ShouldSample(sp.raw.Context.TraceID) + } + + return t.startSpanInternal( + sp, + operationName, + startTime, + tags, + ) +} + +func (t *tracerImpl) startSpanInternal( + sp *spanImpl, + operationName string, + startTime time.Time, + tags opentracing.Tags, +) opentracing.Span { + sp.tracer = t + if t.options.NewSpanEventListener != nil { + sp.event = t.options.NewSpanEventListener() + } + sp.raw.Operation = operationName + sp.raw.Start = startTime + sp.raw.Duration = -1 + sp.raw.Tags = tags + if t.options.DebugAssertSingleGoroutine { + sp.SetTag(debugGoroutineIDTag, curGoroutineID()) + } + defer sp.onCreate(operationName) + return sp +} + +type delegatorType struct{} + +// Delegator is the format to use for DelegatingCarrier. +var Delegator delegatorType + +func (t *tracerImpl) Inject(sc opentracing.SpanContext, format interface{}, carrier interface{}) error { + switch format { + case opentracing.TextMap, opentracing.HTTPHeaders: + return t.textPropagator.Inject(sc, carrier) + case opentracing.Binary: + return t.binaryPropagator.Inject(sc, carrier) + } + if _, ok := format.(delegatorType); ok { + return t.accessorPropagator.Inject(sc, carrier) + } + return opentracing.ErrUnsupportedFormat +} + +func (t *tracerImpl) Extract(format interface{}, carrier interface{}) (opentracing.SpanContext, error) { + switch format { + case opentracing.TextMap, opentracing.HTTPHeaders: + return t.textPropagator.Extract(carrier) + case opentracing.Binary: + return t.binaryPropagator.Extract(carrier) + } + if _, ok := format.(delegatorType); ok { + return t.accessorPropagator.Extract(carrier) + } + return nil, opentracing.ErrUnsupportedFormat +} + +func (t *tracerImpl) Options() Options { + return t.options +} diff --git a/vendor/github.com/opentracing/basictracer-go/util.go b/vendor/github.com/opentracing/basictracer-go/util.go new file mode 100644 index 0000000000000..0f4ed8ac9a23b --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/util.go @@ -0,0 +1,25 @@ +package basictracer + +import ( + "math/rand" + "sync" + "time" +) + +var ( + seededIDGen = rand.New(rand.NewSource(time.Now().UnixNano())) + // The golang rand generators are *not* intrinsically thread-safe. + seededIDLock sync.Mutex +) + +func randomID() uint64 { + seededIDLock.Lock() + defer seededIDLock.Unlock() + return uint64(seededIDGen.Int63()) +} + +func randomID2() (uint64, uint64) { + seededIDLock.Lock() + defer seededIDLock.Unlock() + return uint64(seededIDGen.Int63()), uint64(seededIDGen.Int63()) +} diff --git a/vendor/github.com/opentracing/basictracer-go/wire/carrier.go b/vendor/github.com/opentracing/basictracer-go/wire/carrier.go new file mode 100644 index 0000000000000..12ec98e906b0e --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/wire/carrier.go @@ -0,0 +1,40 @@ +package wire + +// ProtobufCarrier is a DelegatingCarrier that uses protocol buffers as the +// the underlying datastructure. The reason for implementing DelagatingCarrier +// is to allow for end users to serialize the underlying protocol buffers using +// jsonpb or any other serialization forms they want. +type ProtobufCarrier TracerState + +// SetState set's the tracer state. +func (p *ProtobufCarrier) SetState(traceID, spanID uint64, sampled bool) { + p.TraceId = traceID + p.SpanId = spanID + p.Sampled = sampled +} + +// State returns the tracer state. +func (p *ProtobufCarrier) State() (traceID, spanID uint64, sampled bool) { + traceID = p.TraceId + spanID = p.SpanId + sampled = p.Sampled + return traceID, spanID, sampled +} + +// SetBaggageItem sets a baggage item. +func (p *ProtobufCarrier) SetBaggageItem(key, value string) { + if p.BaggageItems == nil { + p.BaggageItems = map[string]string{key: value} + return + } + + p.BaggageItems[key] = value +} + +// GetBaggage iterates over each baggage item and executes the callback with +// the key:value pair. +func (p *ProtobufCarrier) GetBaggage(f func(k, v string)) { + for k, v := range p.BaggageItems { + f(k, v) + } +} diff --git a/vendor/github.com/opentracing/basictracer-go/wire/gen.go b/vendor/github.com/opentracing/basictracer-go/wire/gen.go new file mode 100644 index 0000000000000..ce48e8b8d0935 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/wire/gen.go @@ -0,0 +1,6 @@ +package wire + +//go:generate protoc --gogofaster_out=$GOPATH/src/github.com/opentracing/basictracer-go/wire wire.proto + +// Run `go get github.com/gogo/protobuf/protoc-gen-gogofaster` to install the +// gogofaster generator binary. diff --git a/vendor/github.com/opentracing/basictracer-go/wire/wire.pb.go b/vendor/github.com/opentracing/basictracer-go/wire/wire.pb.go new file mode 100644 index 0000000000000..b78d7ea9199a8 --- /dev/null +++ b/vendor/github.com/opentracing/basictracer-go/wire/wire.pb.go @@ -0,0 +1,508 @@ +// Code generated by protoc-gen-gogo. +// source: wire.proto +// DO NOT EDIT! + +/* + Package wire is a generated protocol buffer package. + + It is generated from these files: + wire.proto + + It has these top-level messages: + TracerState +*/ +package wire + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +const _ = proto.GoGoProtoPackageIsVersion1 + +type TracerState struct { + TraceId uint64 `protobuf:"fixed64,1,opt,name=trace_id,json=traceId,proto3" json:"trace_id,omitempty"` + SpanId uint64 `protobuf:"fixed64,2,opt,name=span_id,json=spanId,proto3" json:"span_id,omitempty"` + Sampled bool `protobuf:"varint,3,opt,name=sampled,proto3" json:"sampled,omitempty"` + BaggageItems map[string]string `protobuf:"bytes,4,rep,name=baggage_items,json=baggageItems" json:"baggage_items,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (m *TracerState) Reset() { *m = TracerState{} } +func (m *TracerState) String() string { return proto.CompactTextString(m) } +func (*TracerState) ProtoMessage() {} +func (*TracerState) Descriptor() ([]byte, []int) { return fileDescriptorWire, []int{0} } + +func (m *TracerState) GetBaggageItems() map[string]string { + if m != nil { + return m.BaggageItems + } + return nil +} + +func init() { + proto.RegisterType((*TracerState)(nil), "basictracer_go.wire.TracerState") +} +func (m *TracerState) 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 *TracerState) MarshalTo(data []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.TraceId != 0 { + data[i] = 0x9 + i++ + i = encodeFixed64Wire(data, i, uint64(m.TraceId)) + } + if m.SpanId != 0 { + data[i] = 0x11 + i++ + i = encodeFixed64Wire(data, i, uint64(m.SpanId)) + } + if m.Sampled { + data[i] = 0x18 + i++ + if m.Sampled { + data[i] = 1 + } else { + data[i] = 0 + } + i++ + } + if len(m.BaggageItems) > 0 { + for k, _ := range m.BaggageItems { + data[i] = 0x22 + i++ + v := m.BaggageItems[k] + mapSize := 1 + len(k) + sovWire(uint64(len(k))) + 1 + len(v) + sovWire(uint64(len(v))) + i = encodeVarintWire(data, i, uint64(mapSize)) + data[i] = 0xa + i++ + i = encodeVarintWire(data, i, uint64(len(k))) + i += copy(data[i:], k) + data[i] = 0x12 + i++ + i = encodeVarintWire(data, i, uint64(len(v))) + i += copy(data[i:], v) + } + } + return i, nil +} + +func encodeFixed64Wire(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 encodeFixed32Wire(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 encodeVarintWire(data []byte, offset int, v uint64) int { + for v >= 1<<7 { + data[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + data[offset] = uint8(v) + return offset + 1 +} +func (m *TracerState) Size() (n int) { + var l int + _ = l + if m.TraceId != 0 { + n += 9 + } + if m.SpanId != 0 { + n += 9 + } + if m.Sampled { + n += 2 + } + if len(m.BaggageItems) > 0 { + for k, v := range m.BaggageItems { + _ = k + _ = v + mapEntrySize := 1 + len(k) + sovWire(uint64(len(k))) + 1 + len(v) + sovWire(uint64(len(v))) + n += mapEntrySize + 1 + sovWire(uint64(mapEntrySize)) + } + } + return n +} + +func sovWire(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozWire(x uint64) (n int) { + return sovWire(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *TracerState) 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 ErrIntOverflowWire + } + 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: TracerState: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TracerState: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field TraceId", wireType) + } + m.TraceId = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + iNdEx += 8 + m.TraceId = uint64(data[iNdEx-8]) + m.TraceId |= uint64(data[iNdEx-7]) << 8 + m.TraceId |= uint64(data[iNdEx-6]) << 16 + m.TraceId |= uint64(data[iNdEx-5]) << 24 + m.TraceId |= uint64(data[iNdEx-4]) << 32 + m.TraceId |= uint64(data[iNdEx-3]) << 40 + m.TraceId |= uint64(data[iNdEx-2]) << 48 + m.TraceId |= uint64(data[iNdEx-1]) << 56 + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field SpanId", wireType) + } + m.SpanId = 0 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + iNdEx += 8 + m.SpanId = uint64(data[iNdEx-8]) + m.SpanId |= uint64(data[iNdEx-7]) << 8 + m.SpanId |= uint64(data[iNdEx-6]) << 16 + m.SpanId |= uint64(data[iNdEx-5]) << 24 + m.SpanId |= uint64(data[iNdEx-4]) << 32 + m.SpanId |= uint64(data[iNdEx-3]) << 40 + m.SpanId |= uint64(data[iNdEx-2]) << 48 + m.SpanId |= uint64(data[iNdEx-1]) << 56 + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Sampled", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowWire + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Sampled = bool(v != 0) + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaggageItems", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowWire + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthWire + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var keykey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowWire + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + keykey |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + var stringLenmapkey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowWire + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + stringLenmapkey |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapkey := int(stringLenmapkey) + if intStringLenmapkey < 0 { + return ErrInvalidLengthWire + } + postStringIndexmapkey := iNdEx + intStringLenmapkey + if postStringIndexmapkey > l { + return io.ErrUnexpectedEOF + } + mapkey := string(data[iNdEx:postStringIndexmapkey]) + iNdEx = postStringIndexmapkey + var valuekey uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowWire + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + valuekey |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + var stringLenmapvalue uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowWire + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + stringLenmapvalue |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLenmapvalue := int(stringLenmapvalue) + if intStringLenmapvalue < 0 { + return ErrInvalidLengthWire + } + postStringIndexmapvalue := iNdEx + intStringLenmapvalue + if postStringIndexmapvalue > l { + return io.ErrUnexpectedEOF + } + mapvalue := string(data[iNdEx:postStringIndexmapvalue]) + iNdEx = postStringIndexmapvalue + if m.BaggageItems == nil { + m.BaggageItems = make(map[string]string) + } + m.BaggageItems[mapkey] = mapvalue + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipWire(data[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthWire + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipWire(data []byte) (n int, err error) { + l := len(data) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowWire + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowWire + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if data[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowWire + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthWire + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowWire + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipWire(data[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthWire = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowWire = fmt.Errorf("proto: integer overflow") +) + +var fileDescriptorWire = []byte{ + // 234 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xe2, 0xe2, 0x2a, 0xcf, 0x2c, 0x4a, + 0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, 0x4e, 0x4a, 0x2c, 0xce, 0x4c, 0x2e, 0x29, 0x4a, + 0x4c, 0x4e, 0x2d, 0x8a, 0x4f, 0xcf, 0xd7, 0x03, 0x49, 0x29, 0x7d, 0x65, 0xe4, 0xe2, 0x0e, 0x01, + 0x0b, 0x05, 0x97, 0x24, 0x96, 0xa4, 0x0a, 0x49, 0x72, 0x71, 0x80, 0x55, 0xc4, 0x67, 0xa6, 0x48, + 0x30, 0x2a, 0x30, 0x6a, 0xb0, 0x05, 0xb1, 0x83, 0xf9, 0x9e, 0x29, 0x42, 0xe2, 0x5c, 0xec, 0xc5, + 0x05, 0x89, 0x79, 0x20, 0x19, 0x26, 0xb0, 0x0c, 0x1b, 0x88, 0x0b, 0x94, 0x90, 0x00, 0x4a, 0x24, + 0xe6, 0x16, 0xe4, 0xa4, 0xa6, 0x48, 0x30, 0x03, 0x25, 0x38, 0x82, 0x60, 0x5c, 0xa1, 0x70, 0x2e, + 0xde, 0xa4, 0xc4, 0xf4, 0xf4, 0xc4, 0x74, 0xa0, 0x79, 0x25, 0xa9, 0xb9, 0xc5, 0x12, 0x2c, 0x0a, + 0xcc, 0x1a, 0xdc, 0x46, 0x46, 0x7a, 0x58, 0x9c, 0xa2, 0x87, 0xe4, 0x0c, 0x3d, 0x27, 0x88, 0x2e, + 0x4f, 0x90, 0x26, 0xd7, 0xbc, 0x92, 0xa2, 0xca, 0x20, 0x9e, 0x24, 0x24, 0x21, 0x29, 0x7b, 0x2e, + 0x41, 0x0c, 0x25, 0x42, 0x02, 0x5c, 0xcc, 0xd9, 0xa9, 0x95, 0x60, 0x67, 0x73, 0x06, 0x81, 0x98, + 0x42, 0x22, 0x5c, 0xac, 0x65, 0x89, 0x39, 0xa5, 0xa9, 0x60, 0x07, 0x73, 0x06, 0x41, 0x38, 0x56, + 0x4c, 0x16, 0x8c, 0x4e, 0x62, 0x27, 0x1e, 0xc9, 0x31, 0x5e, 0x00, 0xe2, 0x07, 0x40, 0x3c, 0xe1, + 0xb1, 0x1c, 0x43, 0x14, 0x0b, 0xc8, 0x11, 0x49, 0x6c, 0xe0, 0xb0, 0x32, 0x06, 0x04, 0x00, 0x00, + 0xff, 0xff, 0x0a, 0x20, 0x89, 0x38, 0x39, 0x01, 0x00, 0x00, +} From e17e757b65eb2b93d076b31bad98efb776fbfe7e Mon Sep 17 00:00:00 2001 From: Shen Li Date: Wed, 29 Aug 2018 20:30:20 +0800 Subject: [PATCH 20/41] travis: Remove test coverage task from travis (#7533) It always meet OOM error. We will find another way to calculate coverage. --- .travis.yml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/.travis.yml b/.travis.yml index 59480cf9c816f..5022c9955b976 100644 --- a/.travis.yml +++ b/.travis.yml @@ -5,17 +5,8 @@ go_import_path: github.com/pingcap/tidb go: - "1.10" -# https://docs.travis-ci.com/user/customizing-the-build/#Rows-that-are-Allowed-to-Fail env: - TRAVIS_COVERAGE=0 - - TRAVIS_COVERAGE=1 - -# Run coverage tests. -matrix: - fast_finish: true - allow_failures: - - go: "1.10" - env: TRAVIS_COVERAGE=1 before_install: # create /logs/unit-test for unit test. From 28fc781474ab7a2332cdc50da80752b3d8f27c89 Mon Sep 17 00:00:00 2001 From: xiaojian cai Date: Wed, 29 Aug 2018 21:15:49 +0800 Subject: [PATCH 21/41] make load_data test clean (#7536) --- executor/write_test.go | 39 ++++++++++++--------------------------- 1 file changed, 12 insertions(+), 27 deletions(-) diff --git a/executor/write_test.go b/executor/write_test.go index 902bd5fd2121b..896032f2e8478 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -19,15 +19,12 @@ import ( "sync/atomic" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" @@ -1426,7 +1423,10 @@ func (s *testSuite) TestLoadData(c *C) { c.Assert(err, NotNil) tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test") ctx := tk.Se.(sessionctx.Context) - ld := makeLoadDataInfo(4, nil, ctx, c) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + c.Assert(ok, IsTrue) + defer ctx.SetValue(executor.LoadDataVarKey, nil) + c.Assert(ld, NotNil) deleteSQL := "delete from load_data_test" selectSQL := "select * from load_data_test;" @@ -1582,7 +1582,10 @@ func (s *testSuite) TestLoadDataEscape(c *C) { tk.MustExec("CREATE TABLE load_data_test (id INT NOT NULL PRIMARY KEY, value TEXT NOT NULL) CHARACTER SET utf8") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test") ctx := tk.Se.(sessionctx.Context) - ld := makeLoadDataInfo(2, nil, ctx, c) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + c.Assert(ok, IsTrue) + defer ctx.SetValue(executor.LoadDataVarKey, nil) + c.Assert(ld, NotNil) // test escape tests := []testCase{ // data1 = nil, data2 != nil @@ -1607,7 +1610,10 @@ func (s *testSuite) TestLoadDataSpecifiedColumns(c *C) { tk.MustExec(`create table load_data_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 varchar(255) default "def", c3 int default 0);`) tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test (c1, c2)") ctx := tk.Se.(sessionctx.Context) - ld := makeLoadDataInfo(2, []string{"c1", "c2"}, ctx, c) + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + c.Assert(ok, IsTrue) + defer ctx.SetValue(executor.LoadDataVarKey, nil) + c.Assert(ld, NotNil) // test tests := []testCase{ // data1 = nil, data2 != nil @@ -1624,27 +1630,6 @@ func (s *testSuite) TestLoadDataSpecifiedColumns(c *C) { checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) } -func makeLoadDataInfo(column int, specifiedColumns []string, ctx sessionctx.Context, c *C) (ld *executor.LoadDataInfo) { - dom := domain.GetDomain(ctx) - is := dom.InfoSchema() - c.Assert(is, NotNil) - tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("load_data_test")) - c.Assert(err, IsNil) - columns := tbl.Cols() - // filter specified columns - if len(specifiedColumns) > 0 { - columns, err = table.FindCols(columns, specifiedColumns, true) - c.Assert(err, IsNil) - } - fields := &ast.FieldsClause{Terminated: "\t"} - lines := &ast.LinesClause{Starting: "", Terminated: "\n"} - ld = executor.NewLoadDataInfo(ctx, make([]types.Datum, column), tbl, columns) - ld.SetMaxRowsInBatch(0) - ld.FieldsInfo = fields - ld.LinesInfo = lines - return -} - func (s *testSuite) TestBatchInsertDelete(c *C) { originLimit := atomic.LoadUint64(&kv.TxnEntryCountLimit) defer func() { From 69d5e36ac44d09de32b0b2cc465c44bd4109f848 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 29 Aug 2018 21:48:50 +0800 Subject: [PATCH 22/41] *: bump go version to 1.11 (#7491) --- .travis.yml | 2 +- Makefile | 4 +- circle.yml | 2 +- executor/show.go | 24 +-- infoschema/tables.go | 30 +-- parser/misc.go | 392 ++++++++++++++++++------------------- plan/explain.go | 6 +- plan/plan.go | 2 +- plan/stats.go | 2 +- plan/stringer.go | 4 +- store/tikv/backoff.go | 2 +- util/ranger/ranger_test.go | 2 +- 12 files changed, 236 insertions(+), 236 deletions(-) diff --git a/.travis.yml b/.travis.yml index 5022c9955b976..c9c219f4b8e54 100644 --- a/.travis.yml +++ b/.travis.yml @@ -3,7 +3,7 @@ language: go go_import_path: github.com/pingcap/tidb go: - - "1.10" + - "1.11" env: - TRAVIS_COVERAGE=0 diff --git a/Makefile b/Makefile index d7037427b7f49..a3013863c404d 100644 --- a/Makefile +++ b/Makefile @@ -103,7 +103,7 @@ check-static: --enable misspell \ --enable megacheck \ --enable ineffassign \ - $$($(PACKAGE_DIRECTORIES)) + $$($(PACKAGE_DIRECTORIES)) check-slow: CGO_ENABLED=0 retool do gometalinter.v2 --disable-all \ @@ -117,7 +117,7 @@ lint: vet: @echo "vet" - @retool do govet -all -shadow $$($(PACKAGE_DIRECTORIES)) 2>&1 | $(FAIL_ON_STDOUT) + @go vet -all -shadow $(PACKAGES) 2>&1 | $(FAIL_ON_STDOUT) clean: $(GO) clean -i ./... diff --git a/circle.yml b/circle.yml index 024061a908499..756b93c9fa0c6 100644 --- a/circle.yml +++ b/circle.yml @@ -3,7 +3,7 @@ version: 2 jobs: build: docker: - - image: golang:1.10 + - image: golang:1.11 working_directory: /go/src/github.com/pingcap/tidb steps: - checkout diff --git a/executor/show.go b/executor/show.go index 6f074811dedd9..ac7de894ba0ab 100644 --- a/executor/show.go +++ b/executor/show.go @@ -356,19 +356,19 @@ func (e *ShowExec) fetchShowIndex() error { subPart = col.Length } e.appendRow([]interface{}{ - tb.Meta().Name.O, // Table - nonUniq, // Non_unique - idx.Meta().Name.O, // Key_name - i + 1, // Seq_in_index - col.Name.O, // Column_name - "A", // Collation - 0, // Cardinality - subPart, // Sub_part - nil, // Packed - "YES", // Null + tb.Meta().Name.O, // Table + nonUniq, // Non_unique + idx.Meta().Name.O, // Key_name + i + 1, // Seq_in_index + col.Name.O, // Column_name + "A", // Collation + 0, // Cardinality + subPart, // Sub_part + nil, // Packed + "YES", // Null idx.Meta().Tp.String(), // Index_type - "", // Comment - idx.Meta().Comment, // Index_comment + "", // Comment + idx.Meta().Comment, // Index_comment }) } } diff --git a/infoschema/tables.go b/infoschema/tables.go index f0a88941c76a1..bd2bda00cbd09 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1006,24 +1006,24 @@ func dataForTableConstraints(schemas []*model.DBInfo) [][]types.Datum { func dataForPseudoProfiling() [][]types.Datum { var rows [][]types.Datum row := types.MakeDatums( - 0, // QUERY_ID - 0, // SEQ - "", // STATE + 0, // QUERY_ID + 0, // SEQ + "", // STATE types.NewDecFromInt(0), // DURATION types.NewDecFromInt(0), // CPU_USER types.NewDecFromInt(0), // CPU_SYSTEM - 0, // CONTEXT_VOLUNTARY - 0, // CONTEXT_INVOLUNTARY - 0, // BLOCK_OPS_IN - 0, // BLOCK_OPS_OUT - 0, // MESSAGES_SENT - 0, // MESSAGES_RECEIVED - 0, // PAGE_FAULTS_MAJOR - 0, // PAGE_FAULTS_MINOR - 0, // SWAPS - 0, // SOURCE_FUNCTION - 0, // SOURCE_FILE - 0, // SOURCE_LINE + 0, // CONTEXT_VOLUNTARY + 0, // CONTEXT_INVOLUNTARY + 0, // BLOCK_OPS_IN + 0, // BLOCK_OPS_OUT + 0, // MESSAGES_SENT + 0, // MESSAGES_RECEIVED + 0, // PAGE_FAULTS_MAJOR + 0, // PAGE_FAULTS_MINOR + 0, // SWAPS + 0, // SOURCE_FUNCTION + 0, // SOURCE_FILE + 0, // SOURCE_LINE ) rows = append(rows, row) return rows diff --git a/parser/misc.go b/parser/misc.go index fafd507d72791..e1544382be7d4 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -133,202 +133,202 @@ func init() { } var tokenMap = map[string]int{ - "ACTION": action, - "ADD": add, - "ADDDATE": addDate, - "ADMIN": admin, - "AFTER": after, - "ALL": all, - "ALGORITHM": algorithm, - "ALTER": alter, - "ALWAYS": always, - "ANALYZE": analyze, - "AND": and, - "ANY": any, - "AS": as, - "ASC": asc, - "ASCII": ascii, - "AUTO_INCREMENT": autoIncrement, - "AVG": avg, - "AVG_ROW_LENGTH": avgRowLength, - "BEGIN": begin, - "BETWEEN": between, - "BIGINT": bigIntType, - "BINARY": binaryType, - "BINLOG": binlog, - "BIT": bitType, - "BIT_AND": bitAnd, - "BIT_OR": bitOr, - "BIT_XOR": bitXor, - "BLOB": blobType, - "BOOL": boolType, - "BOOLEAN": booleanType, - "BOTH": both, - "BTREE": btree, - "BY": by, - "BYTE": byteType, - "CANCEL": cancel, - "CASCADE": cascade, - "CASCADED": cascaded, - "CASE": caseKwd, - "CAST": cast, - "CHANGE": change, - "CHAR": charType, - "CHARACTER": character, - "CHARSET": charsetKwd, - "CHECK": check, - "CHECKSUM": checksum, - "CLEANUP": cleanup, - "CLIENT": client, - "COALESCE": coalesce, - "COLLATE": collate, - "COLLATION": collation, - "COLUMN": column, - "COLUMNS": columns, - "COMMENT": comment, - "COMMIT": commit, - "COMMITTED": committed, - "COMPACT": compact, - "COMPRESSED": compressed, - "COMPRESSION": compression, - "CONNECTION": connection, - "CONSISTENT": consistent, - "CONSTRAINT": constraint, - "CONVERT": convert, - "COPY": copyKwd, - "COUNT": count, - "CREATE": create, - "CROSS": cross, - "CURRENT_DATE": currentDate, - "CURRENT_TIME": currentTime, - "CURRENT_TIMESTAMP": currentTs, - "CURRENT_USER": currentUser, - "CURTIME": curTime, - "DATA": data, - "DATABASE": database, - "DATABASES": databases, - "DATE": dateType, - "DATE_ADD": dateAdd, - "DATE_SUB": dateSub, - "DATETIME": datetimeType, - "DAY": day, - "DAY_HOUR": dayHour, - "DAY_MICROSECOND": dayMicrosecond, - "DAY_MINUTE": dayMinute, - "DAY_SECOND": daySecond, - "DDL": ddl, - "DEALLOCATE": deallocate, - "DEC": decimalType, - "DECIMAL": decimalType, - "DEFAULT": defaultKwd, - "DEFINER": definer, - "DELAY_KEY_WRITE": delayKeyWrite, - "DELAYED": delayed, - "DELETE": deleteKwd, - "DESC": desc, - "DESCRIBE": describe, - "DISABLE": disable, - "DISTINCT": distinct, - "DISTINCTROW": distinct, - "DIV": div, - "DO": do, - "DOUBLE": doubleType, - "DROP": drop, - "DUAL": dual, - "DUPLICATE": duplicate, - "DYNAMIC": dynamic, - "ELSE": elseKwd, - "ENABLE": enable, - "ENCLOSED": enclosed, - "END": end, - "ENGINE": engine, - "ENGINES": engines, - "ENUM": enum, - "ESCAPE": escape, - "ESCAPED": escaped, - "EVENT": event, - "EVENTS": events, - "EXCLUSIVE": exclusive, - "EXECUTE": execute, - "EXISTS": exists, - "EXPLAIN": explain, - "EXTRACT": extract, - "FALSE": falseKwd, - "FIELDS": fields, - "FIRST": first, - "FIXED": fixed, - "FLOAT": floatType, - "FLUSH": flush, - "FOR": forKwd, - "FORCE": force, - "FOREIGN": foreign, - "FORMAT": format, - "FROM": from, - "FULL": full, - "FULLTEXT": fulltext, - "FUNCTION": function, - "GENERATED": generated, - "GET_FORMAT": getFormat, - "GLOBAL": global, - "GRANT": grant, - "GRANTS": grants, - "GROUP": group, - "GROUP_CONCAT": groupConcat, - "HASH": hash, - "HAVING": having, - "HIGH_PRIORITY": highPriority, - "HOUR": hour, - "HOUR_MICROSECOND": hourMicrosecond, - "HOUR_MINUTE": hourMinute, - "HOUR_SECOND": hourSecond, - "IDENTIFIED": identified, - "IF": ifKwd, - "IGNORE": ignore, - "IN": in, - "INDEX": index, - "INDEXES": indexes, - "INFILE": infile, - "INNER": inner, - "INPLACE": inplace, - "INSERT": insert, - "INT": intType, - "INT1": int1Type, - "INT2": int2Type, - "INT3": int3Type, - "INT4": int4Type, - "INT8": int8Type, - "INTEGER": integerType, - "INTERVAL": interval, - "INTO": into, - "INVOKER": invoker, - "IS": is, - "ISOLATION": isolation, - "JOBS": jobs, - "JOB": job, - "JOIN": join, - "JSON": jsonType, - "KEY": key, - "KEY_BLOCK_SIZE": keyBlockSize, - "KEYS": keys, - "KILL": kill, - "LEADING": leading, - "LEFT": left, - "LESS": less, - "LEVEL": level, - "LIKE": like, - "LIMIT": limit, - "LINES": lines, - "LOAD": load, - "LOCAL": local, - "LOCALTIME": localTime, - "LOCALTIMESTAMP": localTs, - "LOCK": lock, - "LONG": long, - "LONGBLOB": longblobType, - "LONGTEXT": longtextType, - "LOW_PRIORITY": lowPriority, - "MASTER": master, - "MAX": max, + "ACTION": action, + "ADD": add, + "ADDDATE": addDate, + "ADMIN": admin, + "AFTER": after, + "ALL": all, + "ALGORITHM": algorithm, + "ALTER": alter, + "ALWAYS": always, + "ANALYZE": analyze, + "AND": and, + "ANY": any, + "AS": as, + "ASC": asc, + "ASCII": ascii, + "AUTO_INCREMENT": autoIncrement, + "AVG": avg, + "AVG_ROW_LENGTH": avgRowLength, + "BEGIN": begin, + "BETWEEN": between, + "BIGINT": bigIntType, + "BINARY": binaryType, + "BINLOG": binlog, + "BIT": bitType, + "BIT_AND": bitAnd, + "BIT_OR": bitOr, + "BIT_XOR": bitXor, + "BLOB": blobType, + "BOOL": boolType, + "BOOLEAN": booleanType, + "BOTH": both, + "BTREE": btree, + "BY": by, + "BYTE": byteType, + "CANCEL": cancel, + "CASCADE": cascade, + "CASCADED": cascaded, + "CASE": caseKwd, + "CAST": cast, + "CHANGE": change, + "CHAR": charType, + "CHARACTER": character, + "CHARSET": charsetKwd, + "CHECK": check, + "CHECKSUM": checksum, + "CLEANUP": cleanup, + "CLIENT": client, + "COALESCE": coalesce, + "COLLATE": collate, + "COLLATION": collation, + "COLUMN": column, + "COLUMNS": columns, + "COMMENT": comment, + "COMMIT": commit, + "COMMITTED": committed, + "COMPACT": compact, + "COMPRESSED": compressed, + "COMPRESSION": compression, + "CONNECTION": connection, + "CONSISTENT": consistent, + "CONSTRAINT": constraint, + "CONVERT": convert, + "COPY": copyKwd, + "COUNT": count, + "CREATE": create, + "CROSS": cross, + "CURRENT_DATE": currentDate, + "CURRENT_TIME": currentTime, + "CURRENT_TIMESTAMP": currentTs, + "CURRENT_USER": currentUser, + "CURTIME": curTime, + "DATA": data, + "DATABASE": database, + "DATABASES": databases, + "DATE": dateType, + "DATE_ADD": dateAdd, + "DATE_SUB": dateSub, + "DATETIME": datetimeType, + "DAY": day, + "DAY_HOUR": dayHour, + "DAY_MICROSECOND": dayMicrosecond, + "DAY_MINUTE": dayMinute, + "DAY_SECOND": daySecond, + "DDL": ddl, + "DEALLOCATE": deallocate, + "DEC": decimalType, + "DECIMAL": decimalType, + "DEFAULT": defaultKwd, + "DEFINER": definer, + "DELAY_KEY_WRITE": delayKeyWrite, + "DELAYED": delayed, + "DELETE": deleteKwd, + "DESC": desc, + "DESCRIBE": describe, + "DISABLE": disable, + "DISTINCT": distinct, + "DISTINCTROW": distinct, + "DIV": div, + "DO": do, + "DOUBLE": doubleType, + "DROP": drop, + "DUAL": dual, + "DUPLICATE": duplicate, + "DYNAMIC": dynamic, + "ELSE": elseKwd, + "ENABLE": enable, + "ENCLOSED": enclosed, + "END": end, + "ENGINE": engine, + "ENGINES": engines, + "ENUM": enum, + "ESCAPE": escape, + "ESCAPED": escaped, + "EVENT": event, + "EVENTS": events, + "EXCLUSIVE": exclusive, + "EXECUTE": execute, + "EXISTS": exists, + "EXPLAIN": explain, + "EXTRACT": extract, + "FALSE": falseKwd, + "FIELDS": fields, + "FIRST": first, + "FIXED": fixed, + "FLOAT": floatType, + "FLUSH": flush, + "FOR": forKwd, + "FORCE": force, + "FOREIGN": foreign, + "FORMAT": format, + "FROM": from, + "FULL": full, + "FULLTEXT": fulltext, + "FUNCTION": function, + "GENERATED": generated, + "GET_FORMAT": getFormat, + "GLOBAL": global, + "GRANT": grant, + "GRANTS": grants, + "GROUP": group, + "GROUP_CONCAT": groupConcat, + "HASH": hash, + "HAVING": having, + "HIGH_PRIORITY": highPriority, + "HOUR": hour, + "HOUR_MICROSECOND": hourMicrosecond, + "HOUR_MINUTE": hourMinute, + "HOUR_SECOND": hourSecond, + "IDENTIFIED": identified, + "IF": ifKwd, + "IGNORE": ignore, + "IN": in, + "INDEX": index, + "INDEXES": indexes, + "INFILE": infile, + "INNER": inner, + "INPLACE": inplace, + "INSERT": insert, + "INT": intType, + "INT1": int1Type, + "INT2": int2Type, + "INT3": int3Type, + "INT4": int4Type, + "INT8": int8Type, + "INTEGER": integerType, + "INTERVAL": interval, + "INTO": into, + "INVOKER": invoker, + "IS": is, + "ISOLATION": isolation, + "JOBS": jobs, + "JOB": job, + "JOIN": join, + "JSON": jsonType, + "KEY": key, + "KEY_BLOCK_SIZE": keyBlockSize, + "KEYS": keys, + "KILL": kill, + "LEADING": leading, + "LEFT": left, + "LESS": less, + "LEVEL": level, + "LIKE": like, + "LIMIT": limit, + "LINES": lines, + "LOAD": load, + "LOCAL": local, + "LOCALTIME": localTime, + "LOCALTIMESTAMP": localTs, + "LOCK": lock, + "LONG": long, + "LONGBLOB": longblobType, + "LONGTEXT": longtextType, + "LOW_PRIORITY": lowPriority, + "MASTER": master, + "MAX": max, "MAX_CONNECTIONS_PER_HOUR": maxConnectionsPerHour, "MAX_EXECUTION_TIME": maxExecutionTime, "MAX_QUERIES_PER_HOUR": maxQueriesPerHour, diff --git a/plan/explain.go b/plan/explain.go index 48db5afc93c32..bfaf0635715ea 100644 --- a/plan/explain.go +++ b/plan/explain.go @@ -51,7 +51,7 @@ func (p *PhysicalIndexScan) ExplainInfo() string { } } if len(p.rangeDecidedBy) > 0 { - fmt.Fprintf(buffer, ", range: decided by %s", p.rangeDecidedBy) + fmt.Fprintf(buffer, ", range: decided by %v", p.rangeDecidedBy) } else if haveCorCol { fmt.Fprintf(buffer, ", range: decided by %v", p.AccessCondition) } else if len(p.Ranges) > 0 { @@ -92,7 +92,7 @@ func (p *PhysicalTableScan) ExplainInfo() string { } } if len(p.rangeDecidedBy) > 0 { - fmt.Fprintf(buffer, ", range: decided by %s", p.rangeDecidedBy) + fmt.Fprintf(buffer, ", range: decided by %v", p.rangeDecidedBy) } else if haveCorCol { fmt.Fprintf(buffer, ", range: decided by %v", p.AccessCondition) } else if len(p.Ranges) > 0 { @@ -228,7 +228,7 @@ func (p *PhysicalHashJoin) ExplainInfo() string { buffer := bytes.NewBufferString(p.JoinType.String()) fmt.Fprintf(buffer, ", inner:%s", p.Children()[p.InnerChildIdx].ExplainID()) if len(p.EqualConditions) > 0 { - fmt.Fprintf(buffer, ", equal:%s", p.EqualConditions) + fmt.Fprintf(buffer, ", equal:%v", p.EqualConditions) } if len(p.LeftConditions) > 0 { fmt.Fprintf(buffer, ", left cond:%s", p.LeftConditions) diff --git a/plan/plan.go b/plan/plan.go index 7a25ca60aeb84..248d8f33965f4 100644 --- a/plan/plan.go +++ b/plan/plan.go @@ -156,7 +156,7 @@ func (p *requiredProp) hashCode() []byte { // String implements fmt.Stringer interface. Just for test. func (p *requiredProp) String() string { - return fmt.Sprintf("Prop{cols: %s, desc: %v, taskTp: %s, expectedCount: %v}", p.cols, p.desc, p.taskTp, p.expectedCnt) + return fmt.Sprintf("Prop{cols: %v, desc: %v, taskTp: %s, expectedCount: %v}", p.cols, p.desc, p.taskTp, p.expectedCnt) } // LogicalPlan is a tree of logical operators. diff --git a/plan/stats.go b/plan/stats.go index c8e641cd8550e..b28c6c128225f 100644 --- a/plan/stats.go +++ b/plan/stats.go @@ -226,7 +226,7 @@ func (lt *LogicalTopN) deriveStats() (*statsInfo, error) { func getCardinality(cols []*expression.Column, schema *expression.Schema, profile *statsInfo) float64 { indices := schema.ColumnsIndices(cols) if indices == nil { - log.Errorf("Cannot find column %s indices from schema %s", cols, schema) + log.Errorf("Cannot find column %v indices from schema %s", cols, schema) return 0 } var cardinality = 1.0 diff --git a/plan/stringer.go b/plan/stringer.go index 9b51dcc62ddf5..d6348a68fc32c 100644 --- a/plan/stringer.go +++ b/plan/stringer.go @@ -156,9 +156,9 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { case *LogicalProjection, *PhysicalProjection: str = "Projection" case *LogicalTopN: - str = fmt.Sprintf("TopN(%s,%d,%d)", x.ByItems, x.Offset, x.Count) + str = fmt.Sprintf("TopN(%v,%d,%d)", x.ByItems, x.Offset, x.Count) case *PhysicalTopN: - str = fmt.Sprintf("TopN(%s,%d,%d)", x.ByItems, x.Offset, x.Count) + str = fmt.Sprintf("TopN(%v,%d,%d)", x.ByItems, x.Offset, x.Count) case *LogicalTableDual, *PhysicalTableDual: str = "Dual" case *PhysicalHashAgg: diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 84a4478fa0837..80d8b9ee29207 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -247,7 +247,7 @@ func (b *Backoffer) String() string { if b.totalSleep == 0 { return "" } - return fmt.Sprintf(" backoff(%dms %s)", b.totalSleep, b.types) + return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.types) } // Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 62eeeb39e62fb..7618e866a1b4a 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -916,7 +916,7 @@ func (s *testRangerSuite) TestColumnRange(c *C) { c.Assert(fmt.Sprintf("%s", conds), Equals, tt.accessConds, Commentf("wrong access conditions for expr: %s", tt.exprStr)) result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType) c.Assert(err, IsNil) - got := fmt.Sprintf("%s", result) + got := fmt.Sprintf("%v", result) c.Assert(got, Equals, tt.resultStr, Commentf("different for expr %s, col: %v", tt.exprStr, col)) } } From 55af7e147269dac05a9741aa461e736a264520c9 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 29 Aug 2018 23:40:53 +0800 Subject: [PATCH 23/41] executor: forbid user to drop important system table (#7471) --- executor/ddl.go | 31 +++++++++++++++++++++++++++++++ executor/ddl_test.go | 6 ++++++ 2 files changed, 37 insertions(+) diff --git a/executor/ddl.go b/executor/ddl.go index ced59b52fb302..0f19c49d1a98b 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -156,6 +156,13 @@ func (e *DDLExec) executeCreateIndex(s *ast.CreateIndexStmt) error { func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { dbName := model.NewCIStr(s.Name) + + // Protect important system table from been dropped by a mistake. + // I can hardly find a case that a user really need to do this. + if dbName.L == "mysql" { + return errors.New("Drop 'mysql' database is forbidden") + } + err := domain.GetDomain(e.ctx).DDL().DropSchema(e.ctx, dbName) if infoschema.ErrDatabaseNotExists.Equal(err) { if s.IfExists { @@ -179,6 +186,24 @@ func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { return errors.Trace(err) } +// If one drop those tables by mistake, it's difficult to recover. +// In the worst case, the whole TiDB cluster fails to bootstrap, so we prevent user from dropping them. +var systemTables = map[string]struct{}{ + "tidb": {}, + "gc_delete_range": {}, + "gc_delete_range_done": {}, +} + +func isSystemTable(schema, table string) bool { + if schema != "mysql" { + return false + } + if _, ok := systemTables[table]; ok { + return true + } + return false +} + func (e *DDLExec) executeDropTable(s *ast.DropTableStmt) error { var notExistTables []string for _, tn := range s.Tables { @@ -198,6 +223,12 @@ func (e *DDLExec) executeDropTable(s *ast.DropTableStmt) error { return errors.Trace(err) } + // Protect important system table from been dropped by a mistake. + // I can hardly find a case that a user really need to do this. + if isSystemTable(tn.Schema.L, tn.Name.L) { + return errors.Errorf("Drop tidb system table '%s.%s' is forbidden", tn.Schema.L, tn.Name.L) + } + if config.CheckTableBeforeDrop { log.Warnf("admin check table `%s`.`%s` before drop.", fullti.Schema.O, fullti.Name.O) sql := fmt.Sprintf("admin check table `%s`.`%s`", fullti.Schema.O, fullti.Name.O) diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 87c69c5b95ac2..4d102ffa0cc20 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -128,6 +128,9 @@ func (s *testSuite) TestCreateDropDatabase(c *C) { c.Assert(err.Error(), Equals, plan.ErrNoDB.Error()) _, err = tk.Exec("select * from t;") c.Assert(err.Error(), Equals, plan.ErrNoDB.Error()) + + _, err = tk.Exec("drop database mysql") + c.Assert(err, NotNil) } func (s *testSuite) TestCreateDropTable(c *C) { @@ -137,6 +140,9 @@ func (s *testSuite) TestCreateDropTable(c *C) { tk.MustExec("drop table if exists drop_test") tk.MustExec("create table drop_test (a int)") tk.MustExec("drop table drop_test") + + _, err := tk.Exec("drop table mysql.gc_delete_range") + c.Assert(err, NotNil) } func (s *testSuite) TestCreateDropIndex(c *C) { From 4a3130216fb9b06526073f73089289051e719852 Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Thu, 30 Aug 2018 11:28:34 +0800 Subject: [PATCH 24/41] server: fix type year has too many 0s in prepare/execute (#7525) --- server/driver_tidb.go | 2 +- server/driver_tidb_test.go | 22 ++++++++++++++++++++++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 739e253e7422e..630f311a091fa 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -391,7 +391,7 @@ func convertColumnInfo(fld *ast.ResultField) (ci *ColumnInfo) { // Consider the decimal point. ci.ColumnLength++ } - } else if fld.Column.Tp != mysql.TypeBit && fld.Column.Tp != mysql.TypeTiny { + } else if types.IsString(fld.Column.Tp) { // Fix issue #4540. // The flen is a hint, not a precise value, so most client will not use the value. // But we found in rare MySQL client, like Navicat for MySQL(version before 12) will truncate diff --git a/server/driver_tidb_test.go b/server/driver_tidb_test.go index f1298e083db70..97e7b87b05040 100644 --- a/server/driver_tidb_test.go +++ b/server/driver_tidb_test.go @@ -88,4 +88,26 @@ func (ts tidbResultSetTestSuite) TestConvertColumnInfo(c *C) { } colInfo = convertColumnInfo(&resultField) c.Assert(colInfo, DeepEquals, createColumnByTypeAndLen(mysql.TypeTiny, 1)) + + resultField = ast.ResultField{ + Column: &model.ColumnInfo{ + Name: model.NewCIStr("a"), + ID: 0, + Offset: 0, + FieldType: types.FieldType{ + Tp: mysql.TypeYear, + Flag: mysql.ZerofillFlag, + Flen: 4, + Decimal: 0, + Charset: charset.CharsetBin, + Collate: charset.CollationBin, + }, + Comment: "column a is the first column in table dual", + }, + ColumnAsName: model.NewCIStr("a"), + TableAsName: model.NewCIStr("dual"), + DBName: model.NewCIStr("test"), + } + colInfo = convertColumnInfo(&resultField) + c.Assert(colInfo.ColumnLength, Equals, uint32(4)) } From 9070bb6b86b27e59f35ebc2f984e369d3c5a115d Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Thu, 30 Aug 2018 14:16:30 +0800 Subject: [PATCH 25/41] executer: fix the last_insert_id in insert on duplicate key update (#7534) --- executor/insert.go | 24 ++++-------------------- executor/insert_common.go | 21 +++++++++++++++++---- executor/load_data.go | 10 +++------- executor/replace.go | 18 +----------------- executor/update.go | 2 +- executor/write.go | 33 +++++++++++++++------------------ executor/write_test.go | 5 +++++ server/server_test.go | 4 ++-- 8 files changed, 48 insertions(+), 69 deletions(-) diff --git a/executor/insert.go b/executor/insert.go index 66a91ee65830c..6587b9650e020 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -34,16 +34,6 @@ type InsertExec struct { finished bool } -func (e *InsertExec) insertOneRow(row []types.Datum) (int64, error) { - e.ctx.Txn().SetOption(kv.PresumeKeyNotExists, nil) - h, err := e.Table.AddRecord(e.ctx, row, false) - e.ctx.Txn().DelOption(kv.PresumeKeyNotExists) - if err != nil { - return 0, errors.Trace(err) - } - return h, nil -} - func (e *InsertExec) exec(rows [][]types.Datum) error { // If tidb_batch_insert is ON and not in a transaction, we could use BatchInsert mode. sessVars := e.ctx.GetSessionVars() @@ -67,20 +57,17 @@ func (e *InsertExec) exec(rows [][]types.Datum) error { return errors.Trace(err) } } else if ignoreErr { - err := e.batchCheckAndInsert(rows, e.insertOneRow) + err := e.batchCheckAndInsert(rows, e.addRecord) if err != nil { return errors.Trace(err) } } else { for _, row := range rows { - if _, err := e.insertOneRow(row); err != nil { + if _, err := e.addRecord(row); err != nil { return errors.Trace(err) } } } - if e.lastInsertID != 0 { - sessVars.SetLastInsertID(e.lastInsertID) - } e.finished = true return nil } @@ -131,7 +118,7 @@ func (e *InsertExec) batchUpdateDupRows(newRows [][]types.Datum) error { // and key-values should be filled back to dupOldRowValues for the further row check, // due to there may be duplicate keys inside the insert statement. if newRows[i] != nil { - newHandle, err := e.insertOneRow(newRows[i]) + newHandle, err := e.addRecord(newRows[i]) if err != nil { return errors.Trace(err) } @@ -220,13 +207,10 @@ func (e *InsertExec) doDupRowUpdate(handle int64, oldRow []types.Datum, newRow [ } newData := row4Update[:len(oldRow)] - _, handleChanged, newHandle, lastInsertID, err := updateRecord(e.ctx, handle, oldRow, newData, assignFlag, e.Table, true) + _, handleChanged, newHandle, err := updateRecord(e.ctx, handle, oldRow, newData, assignFlag, e.Table, true) if err != nil { return nil, false, 0, errors.Trace(err) } - if lastInsertID != 0 { - e.lastInsertID = lastInsertID - } return newData, handleChanged, newHandle, nil } diff --git a/executor/insert_common.go b/executor/insert_common.go index 7859cc68da855..c011c1aa6fe37 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -157,7 +157,7 @@ func (e *InsertValues) insertRows(cols []*table.Column, exec func(rows [][]types rows := make([][]types.Datum, len(e.Lists)) for i, list := range e.Lists { - e.rowCount = uint64(i) + e.rowCount++ rows[i], err = e.getRow(cols, list, i) if err != nil { return errors.Trace(err) @@ -445,7 +445,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(row []types.Datum, i int, c *tab return errors.Trace(err) } // It's compatible with mysql. So it sets last insert id to the first row. - if e.rowCount == 0 { + if e.rowCount == 1 { e.lastInsertID = uint64(recordID) } } @@ -474,7 +474,7 @@ func (e *InsertValues) handleWarning(err error, logInfo string) { // batchCheckAndInsert checks rows with duplicate errors. // All duplicate rows will be ignored and appended as duplicate warnings. -func (e *InsertValues) batchCheckAndInsert(rows [][]types.Datum, insertOneRow func(row []types.Datum) (int64, error)) error { +func (e *InsertValues) batchCheckAndInsert(rows [][]types.Datum, addRecord func(row []types.Datum) (int64, error)) error { // all the rows will be checked, so it is safe to set BatchCheck = true e.ctx.GetSessionVars().StmtCtx.BatchCheck = true err := e.batchGetInsertKeys(e.ctx, e.Table, rows) @@ -502,7 +502,7 @@ func (e *InsertValues) batchCheckAndInsert(rows [][]types.Datum, insertOneRow fu // it should be add to values map for the further row check. // There may be duplicate keys inside the insert statement. if rows[i] != nil { - _, err = insertOneRow(rows[i]) + _, err = addRecord(rows[i]) if err != nil { return errors.Trace(err) } @@ -516,3 +516,16 @@ func (e *InsertValues) batchCheckAndInsert(rows [][]types.Datum, insertOneRow fu } return nil } + +func (e *InsertValues) addRecord(row []types.Datum) (int64, error) { + e.ctx.Txn().SetOption(kv.PresumeKeyNotExists, nil) + h, err := e.Table.AddRecord(e.ctx, row, false) + e.ctx.Txn().DelOption(kv.PresumeKeyNotExists) + if err != nil { + return 0, errors.Trace(err) + } + if e.lastInsertID != 0 { + e.ctx.GetSessionVars().SetLastInsertID(e.lastInsertID) + } + return h, nil +} diff --git a/executor/load_data.go b/executor/load_data.go index 10ee2a4260a85..afe7b61f5bce3 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -248,14 +248,10 @@ func (e *LoadDataInfo) InsertData(prevData, curData []byte) ([]byte, bool, error break } } - err := e.batchCheckAndInsert(rows, e.insertData) + err := e.batchCheckAndInsert(rows, e.addRecordLD) if err != nil { return nil, reachLimit, errors.Trace(err) } - if e.lastInsertID != 0 { - e.ctx.GetSessionVars().SetLastInsertID(e.lastInsertID) - } - return curData, reachLimit, nil } @@ -282,11 +278,11 @@ func (e *LoadDataInfo) colsToRow(cols []field) []types.Datum { return row } -func (e *LoadDataInfo) insertData(row []types.Datum) (int64, error) { +func (e *LoadDataInfo) addRecordLD(row []types.Datum) (int64, error) { if row == nil { return 0, nil } - h, err := e.Table.AddRecord(e.ctx, row, false) + h, err := e.addRecord(row) if err != nil { e.handleWarning(err, fmt.Sprintf("Load Data: insert data:%v failed:%v", e.row, errors.ErrorStack(err))) diff --git a/executor/replace.go b/executor/replace.go index a37f22dcc7e79..1debefb3cf9e5 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -15,7 +15,6 @@ package executor import ( "github.com/juju/errors" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -83,18 +82,6 @@ func (e *ReplaceExec) removeRow(handle int64, r toBeCheckedRow) (bool, error) { return false, nil } -// addRow adds a row when all the duplicate key were checked. -func (e *ReplaceExec) addRow(row []types.Datum) (int64, error) { - // Set kv.PresumeKeyNotExists is safe here, because we've already removed all duplicated rows. - e.ctx.Txn().SetOption(kv.PresumeKeyNotExists, nil) - h, err := e.Table.AddRecord(e.ctx, row, false) - e.ctx.Txn().DelOption(kv.PresumeKeyNotExists) - if err != nil { - return 0, errors.Trace(err) - } - return h, nil -} - // replaceRow removes all duplicate rows for one row, then inserts it. func (e *ReplaceExec) replaceRow(r toBeCheckedRow) error { if r.handleKey != nil { @@ -129,7 +116,7 @@ func (e *ReplaceExec) replaceRow(r toBeCheckedRow) error { } // No duplicated rows now, insert the row. - newHandle, err := e.addRow(r.row) + newHandle, err := e.addRecord(r.row) if err != nil { return errors.Trace(err) } @@ -190,9 +177,6 @@ func (e *ReplaceExec) exec(newRows [][]types.Datum) error { return errors.Trace(err) } } - if e.lastInsertID != 0 { - e.ctx.GetSessionVars().SetLastInsertID(e.lastInsertID) - } e.finished = true return nil } diff --git a/executor/update.go b/executor/update.go index ab697d7db3cb4..39be67de247fb 100644 --- a/executor/update.go +++ b/executor/update.go @@ -80,7 +80,7 @@ func (e *UpdateExec) exec(schema *expression.Schema) ([]types.Datum, error) { } // Update row - changed, _, _, _, err1 := updateRecord(e.ctx, handle, oldData, newTableData, flags, tbl, false) + changed, _, _, err1 := updateRecord(e.ctx, handle, oldData, newTableData, flags, tbl, false) if err1 == nil { if changed { e.updatedRowKeys[id][handle] = struct{}{} diff --git a/executor/write.go b/executor/write.go index f1bc187617d85..ae010cf3ae048 100644 --- a/executor/write.go +++ b/executor/write.go @@ -41,17 +41,15 @@ var ( // 1. changed (bool) : does the update really change the row values. e.g. update set i = 1 where i = 1; // 2. handleChanged (bool) : is the handle changed after the update. // 3. newHandle (int64) : if handleChanged == true, the newHandle means the new handle after update. -// 4. lastInsertID (uint64) : the lastInsertID should be set by the newData. -// 5. err (error) : error in the update. +// 4. err (error) : error in the update. func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, modified []bool, t table.Table, - onDup bool) (bool, bool, int64, uint64, error) { + onDup bool) (bool, bool, int64, error) { var sc = ctx.GetSessionVars().StmtCtx var changed, handleChanged = false, false // onUpdateSpecified is for "UPDATE SET ts_field = old_value", the // timestamp field is explicitly set, but not changed in fact. var onUpdateSpecified = make(map[int]bool) var newHandle int64 - var lastInsertID uint64 // We can iterate on public columns not writable columns, // because all of them are sorted by their `Offset`, which @@ -61,7 +59,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu // Cast changed fields with respective columns. v, err := table.CastValue(ctx, newData[i], col.ToInfo()) if err != nil { - return false, handleChanged, newHandle, 0, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } newData[i] = v } @@ -70,27 +68,26 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu var err error newData[i], err = table.GetColDefaultValue(ctx, col.ToInfo()) if err != nil { - return false, handleChanged, newHandle, 0, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } } // Rebase auto increment id if the field is changed. if mysql.HasAutoIncrementFlag(col.Flag) { if newData[i].IsNull() { - return false, handleChanged, newHandle, 0, table.ErrColumnCantNull.GenByArgs(col.Name) + return false, handleChanged, newHandle, table.ErrColumnCantNull.GenByArgs(col.Name) } val, errTI := newData[i].ToInt64(sc) if errTI != nil { - return false, handleChanged, newHandle, 0, errors.Trace(errTI) + return false, handleChanged, newHandle, errors.Trace(errTI) } - lastInsertID = uint64(val) err := t.RebaseAutoID(ctx, val, true) if err != nil { - return false, handleChanged, newHandle, 0, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } } cmp, err := newData[i].CompareDatum(sc, &oldData[i]) if err != nil { - return false, handleChanged, newHandle, 0, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } if cmp != 0 { changed = true @@ -111,7 +108,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu // Check the not-null constraints. err := table.CheckNotNull(t.Cols(), newData) if err != nil { - return false, handleChanged, newHandle, 0, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } if !changed { @@ -119,7 +116,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu if ctx.GetSessionVars().ClientCapability&mysql.ClientFoundRows > 0 { sc.AddAffectedRows(1) } - return false, handleChanged, newHandle, lastInsertID, nil + return false, handleChanged, newHandle, nil } // Fill values into on-update-now fields, only if they are really changed. @@ -127,7 +124,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu if mysql.HasOnUpdateNowFlag(col.Flag) && !modified[i] && !onUpdateSpecified[i] { v, errGT := expression.GetTimeValue(ctx, strings.ToUpper(ast.CurrentTimestamp), col.Tp, col.Decimal) if errGT != nil { - return false, handleChanged, newHandle, 0, errors.Trace(errGT) + return false, handleChanged, newHandle, errors.Trace(errGT) } newData[i] = v modified[i] = true @@ -140,13 +137,13 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu // if the new handle exists. `UPDATE IGNORE` will avoid removing record, and do nothing. err = tables.CheckHandleExists(ctx, t, newHandle, newData) if err != nil { - return false, handleChanged, newHandle, 0, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } skipHandleCheck = true } err = t.RemoveRecord(ctx, h, oldData) if err != nil { - return false, handleChanged, newHandle, 0, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } newHandle, err = t.AddRecord(ctx, newData, skipHandleCheck) } else { @@ -154,7 +151,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu err = t.UpdateRecord(ctx, h, oldData, newData, modified) } if err != nil { - return false, handleChanged, newHandle, 0, errors.Trace(err) + return false, handleChanged, newHandle, errors.Trace(err) } if onDup { @@ -173,7 +170,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu } } ctx.GetSessionVars().TxnCtx.UpdateDeltaForTable(t.Meta().ID, 0, 1, colSize) - return true, handleChanged, newHandle, lastInsertID, nil + return true, handleChanged, newHandle, nil } // resetErrDataTooLong reset ErrDataTooLong error msg. diff --git a/executor/write_test.go b/executor/write_test.go index 896032f2e8478..ff2ef8d552ca8 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -622,6 +622,11 @@ commit;` testSQL = `SELECT LAST_INSERT_ID();` r = tk.MustQuery(testSQL) r.Check(testkit.Rows("1")) + testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = 2;` + tk.MustExec(testSQL) + testSQL = `SELECT LAST_INSERT_ID();` + r = tk.MustQuery(testSQL) + r.Check(testkit.Rows("1")) testSQL = `DROP TABLE IF EXISTS t1; CREATE TABLE t1 (f1 INT); diff --git a/server/server_test.go b/server/server_test.go index 1b15a99056375..2a358e80ba1ff 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -425,7 +425,7 @@ func runTestLoadData(c *C, server *Server) { dbt.Assert(err, IsNil) lastID, err = rs.LastInsertId() dbt.Assert(err, IsNil) - dbt.Assert(lastID, Equals, int64(6)) + dbt.Assert(lastID, Equals, int64(7)) affectedRows, err = rs.RowsAffected() dbt.Assert(err, IsNil) dbt.Assert(affectedRows, Equals, int64(4)) @@ -464,7 +464,7 @@ func runTestLoadData(c *C, server *Server) { dbt.Assert(err, IsNil) lastID, err = rs.LastInsertId() dbt.Assert(err, IsNil) - dbt.Assert(lastID, Equals, int64(10)) + dbt.Assert(lastID, Equals, int64(11)) affectedRows, err = rs.RowsAffected() dbt.Assert(err, IsNil) dbt.Assert(affectedRows, Equals, int64(799)) From 528d95969417b0166538cf730d6dfb87af8bef7d Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 30 Aug 2018 14:30:10 +0800 Subject: [PATCH 26/41] ddl: fix data race (#7548) --- ddl/stat_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/stat_test.go b/ddl/stat_test.go index 2f2147492ea05..ac19d7984daf9 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -84,7 +84,7 @@ LOOP: case <-ticker.C: d.close() c.Assert(s.getDDLSchemaVer(c, d), GreaterEqual, ver) - d.start(context.Background(), nil) + d.restartWorkers(context.Background()) time.Sleep(time.Millisecond * 20) case err := <-done: c.Assert(err, IsNil) From f915575ebed9b14eef27a23801f7fe9e0015b36a Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Thu, 30 Aug 2018 14:49:33 +0800 Subject: [PATCH 27/41] server: add comment for #7252 (#7549) --- server/driver_tidb.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 630f311a091fa..9405f8eadae2a 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -403,6 +403,8 @@ func convertColumnInfo(fld *ast.ResultField) (ci *ColumnInfo) { // * Utf-8, the multiple is 3 // * utf8mb4, the multiple is 4 // So the large enough multiple is 4 in here. + // We used to check non-string types to avoid the truncation problem in some MySQL + // client such as Navicat. Now we only allow string type enter this branch. ci.ColumnLength = ci.ColumnLength * mysql.MaxBytesOfCharacter } From 67d7544651e466bcb788f9fb950c4cfcbaff7e34 Mon Sep 17 00:00:00 2001 From: winkyao Date: Thu, 30 Aug 2018 15:03:53 +0800 Subject: [PATCH 28/41] design: add a proposal to support a command 'ADMIN RESTORE TABLE table_id' to speed up recover faulty dropped table. (#7383) --- .../2018-08-10-restore-dropped-table.md | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) create mode 100644 docs/design/2018-08-10-restore-dropped-table.md diff --git a/docs/design/2018-08-10-restore-dropped-table.md b/docs/design/2018-08-10-restore-dropped-table.md new file mode 100644 index 0000000000000..2210b565920fe --- /dev/null +++ b/docs/design/2018-08-10-restore-dropped-table.md @@ -0,0 +1,49 @@ + +# Proposal: + +- Author(s): [winkyao](https://github.com/winkyao) +- Last updated: 2018-08-10 + +## Abstract + +This proposal proposes to support the `ADMIN RESTORE TABLE table_id` command, to restore the table that is dropped by a faulty operation. + +## Background + +At present, if we drop the table in production environment, we will realize whether the operation is faulty immediately. Before we support the proposed command, we can only [read data from history versions](https://pingcap.com/docs/op-guide/history-read/) to relieve the disaster. But it needs to read all the data in the storage and it takes too much time to just restore the dropped the table. + +## Proposal + +We can add a new command `ADMIN RESTORE TABLE table_id` to just make the dropped table public again. If the data is not deleted by GC worker, this command can work. So it is better to enlarge the GC life time with `update mysql.tidb set variable_value='30h' where variable_name='tikv_gc_life_time';`, before we execute the statement. The table and the original table data can be restored in a few seconds and it is a lot faster than before. It also can reduce the complexity of the operations and dissolve the artificial operation error. + +## Rationale + +Let's take a look at the workflow of the `DROP TABLE` statement. The `DROP TABLE` statement first removes the dropping table meta data from the coresponding database meta data. After the schemas are synced by all the TiDB instances, in `worker.deleteRange`, TiDB will insert a deleted range of the first row key to the end row key of the dropping table into the table `mysql.gc_delete_range`. At most `max(gcDefaultRunInterval, gcLifeTimeKey)` time later, the GC worker will delete the table data finally. + +The meta data of the table is not really deleted. The meta key format is `Table:table_id`. As long as we can find out the ID of the dropped table, we can recover the table information. The `admin show ddl jobs` statement can retrieve the table ID: + +``` ++-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ +| JOBS | STATE | ++-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ +| ID:44, Type:drop table, State:synced, SchemaState:none, SchemaID:1, TableID:39, RowCount:0, ArgLen:0, start time: 2018-08-11 11:23:53.308 +0800 CST, Err:, ErrCount:0, SnapshotVersion:0 | synced | ++-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+ +``` + +As you can see, if we can restore the table before the GC worker deletes the table data, we can restore the table completely. If the table data is deleted, we can only restore an empty table. + +Before we run `ADMIN RESTORE TABLE table_id`, you must: +* Ensure that no GC task is running and then you can figure this by using TiDB logs and metrics. +* Increase `tikv_gc_life_time` to a sufficient value. + +## Compatibility + +It's a new command and will not lead to compatibility issues. + +## Implementation + +1. `ADMIN RESTORE TABLE table_id` will enqueue a new DDL job to TiDB general DDL queue. +2. Before we start to do this job, we need to check if the table name already exists (created a new table with the same table name after you drop it). If it exists, return the `ErrTableExists` error. +3. Find out whether the delete-range of the dropped table is still in `mysql.gc_delete_range`. If not, it means that the GC worker has cleaned up the data. In this situation, we cannot restore the table successfully but return an error to the client. If it is still there, we remove the record in the `mysql.gc_delete_range` table. If we successfully remove the record, continue to Step 4; otherwise, we return an error to the client to indicate the command cannot be executed safely. +4. Use the previous table meta information of the table_id to insert the meta data into the schema meta data, like what `Meta.CreateTable` does. And set the table information state to `model.StatePublic`, then the restoration will be finished after the schema is synced by all the TiDB instances. +5. If the command is canceled or rollbacked, and the delete-range record is already removed, we need to insert it into `mysql.gc_delete_range` again, like what `Drop Table` does in `worker.finishDDLJob`. From 4df2c7eef61780517aa4f3d7985f7ed144ebf8db Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Thu, 30 Aug 2018 17:20:59 +0800 Subject: [PATCH 29/41] stats: fix auto analyze trigger condition (#7550) --- statistics/update.go | 8 ++++---- statistics/update_test.go | 34 ++++++++++++++++++++++++++++++++++ 2 files changed, 38 insertions(+), 4 deletions(-) diff --git a/statistics/update.go b/statistics/update.go index d33100af918aa..4dfc0634decbc 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -586,10 +586,10 @@ const ( // AutoAnalyzeMinCnt means if the count of table is less than this value, we needn't do auto analyze. var AutoAnalyzeMinCnt int64 = 1000 -// tableAnalyzed checks if the table is analyzed. -func tableAnalyzed(tbl *Table) bool { +// TableAnalyzed checks if the table is analyzed. +func TableAnalyzed(tbl *Table) bool { for _, col := range tbl.Columns { - if col.Histogram.Len() > 0 { + if col.Count > 0 { return true } } @@ -607,7 +607,7 @@ func tableAnalyzed(tbl *Table) bool { // 2. If the table had been analyzed before, we need to analyze it when // "tbl.ModifyCount/tbl.Count > autoAnalyzeRatio". func needAnalyzeTable(tbl *Table, limit time.Duration, autoAnalyzeRatio float64) bool { - analyzed := tableAnalyzed(tbl) + analyzed := TableAnalyzed(tbl) if !analyzed { t := time.Unix(0, oracle.ExtractPhysical(tbl.Version)*int64(time.Millisecond)) return time.Since(t) >= limit diff --git a/statistics/update_test.go b/statistics/update_test.go index 4e42383946fc1..f91c130952940 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -405,6 +405,40 @@ func (s *testStatsUpdateSuite) TestAutoUpdate(c *C) { c.Assert(hg.Len(), Equals, 3) } +func (s *testStatsUpdateSuite) TestTableAnalyzed(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("create table t (a int)") + testKit.MustExec("insert into t values (1)") + + is := s.do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + h := s.do.StatsHandle() + + h.Update(is) + statsTbl := h.GetTableStats(tableInfo) + c.Assert(statistics.TableAnalyzed(statsTbl), IsFalse) + + testKit.MustExec("analyze table t") + h.Update(is) + statsTbl = h.GetTableStats(tableInfo) + c.Assert(statistics.TableAnalyzed(statsTbl), IsTrue) + + h.Clear() + oriLease := h.Lease + // set it to non-zero so we will use load by need strategy + h.Lease = 1 + defer func() { + h.Lease = oriLease + }() + h.Update(is) + statsTbl = h.GetTableStats(tableInfo) + c.Assert(statistics.TableAnalyzed(statsTbl), IsTrue) +} + func (s *testStatsUpdateSuite) TestUpdateErrorRate(c *C) { defer cleanEnv(c, s.store, s.do) h := s.do.StatsHandle() From 5823fb5169a1a1456dd35a55b48ef0ff704de5ff Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Thu, 30 Aug 2018 20:31:37 +0800 Subject: [PATCH 30/41] executor: break innerTable fetcher is error happend during fetching outer table (#7554) --- executor/join.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/executor/join.go b/executor/join.go index 8eb1b953c5691..244cf82c675b3 100644 --- a/executor/join.go +++ b/executor/join.go @@ -102,6 +102,10 @@ func (e *HashJoinExec) Close() error { close(e.closeCh) e.finished.Store(true) if e.prepared { + if e.innerFinished != nil { + for range e.innerFinished { + } + } if e.joinResultCh != nil { for range e.joinResultCh { } @@ -264,6 +268,9 @@ func (e *HashJoinExec) fetchInnerRows(ctx context.Context) (err error) { e.innerResult.GetMemTracker().AttachTo(e.memTracker) e.innerResult.GetMemTracker().SetLabel("innerResult") for { + if e.finished.Load().(bool) { + return nil + } chk := e.children[e.innerIdx].newChunk() err = e.innerExec.Next(ctx, chk) if err != nil || chk.NumRows() == 0 { @@ -511,6 +518,10 @@ func (e *HashJoinExec) fetchInnerAndBuildHashTable(ctx context.Context) { return } + if e.finished.Load().(bool) { + return + } + if err := e.buildHashTableForList(); err != nil { e.innerFinished <- errors.Trace(err) return @@ -533,6 +544,9 @@ func (e *HashJoinExec) buildHashTableForList() error { valBuf = make([]byte, 8) ) for i := 0; i < e.innerResult.NumChunks(); i++ { + if e.finished.Load().(bool) { + return nil + } chk := e.innerResult.GetChunk(i) for j := 0; j < chk.NumRows(); j++ { hasNull, keyBuf, err = e.getJoinKeyFromChkRow(false, chk.GetRow(j), keyBuf) From b4fdaf35ac716cfb16620d26cb6e1ad1604a279e Mon Sep 17 00:00:00 2001 From: Shafreeck Sea Date: Thu, 30 Aug 2018 23:37:03 +0800 Subject: [PATCH 31/41] Add keyonly support for seek (#7419) --- kv/kv.go | 2 ++ store/tikv/lock_test.go | 16 ++++++++++++++++ store/tikv/scan.go | 30 ++++++++++++++++++------------ store/tikv/scan_test.go | 37 +++++++++++++++++++++++++++++++++++++ store/tikv/snapshot.go | 1 + store/tikv/txn.go | 2 ++ 6 files changed, 76 insertions(+), 12 deletions(-) diff --git a/kv/kv.go b/kv/kv.go index 237fbbe5f6f75..1c9230ed65a6a 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -46,6 +46,8 @@ const ( // BypassLatch option tells 2PC commit to bypass latches, it would be true when the // transaction is not conflict-retryable, for example: 'select for update', 'load data'. BypassLatch + // KeyOnly retrieve only keys, it can be used in scan now. + KeyOnly ) // Priority value for transaction priority. diff --git a/store/tikv/lock_test.go b/store/tikv/lock_test.go index f544053f6dbb6..20bedba474f10 100644 --- a/store/tikv/lock_test.go +++ b/store/tikv/lock_test.go @@ -126,6 +126,22 @@ func (s *testLockSuite) TestScanLockResolveWithSeek(c *C) { } } +func (s *testLockSuite) TestScanLockResolveWithSeekKeyOnly(c *C) { + s.putAlphabets(c) + s.prepareAlphabetLocks(c) + + txn, err := s.store.Begin() + c.Assert(err, IsNil) + txn.SetOption(kv.KeyOnly, true) + iter, err := txn.Seek([]byte("a")) + c.Assert(err, IsNil) + for ch := byte('a'); ch <= byte('z'); ch++ { + c.Assert(iter.Valid(), IsTrue) + c.Assert([]byte(iter.Key()), BytesEquals, []byte{ch}) + c.Assert(iter.Next(), IsNil) + } +} + func (s *testLockSuite) TestScanLockResolveWithBatchGet(c *C) { s.putAlphabets(c) s.prepareAlphabetLocks(c) diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 0ccf297666ee6..b37de8c9db394 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -94,13 +94,22 @@ func (s *Scanner) Next() error { continue } } - if err := s.resolveCurrentLock(bo); err != nil { - s.Close() - return errors.Trace(err) - } - if len(s.Value()) == 0 { - // nil stands for NotExist, go to next KV pair. - continue + + current := s.cache[s.idx] + // Try to resolve the lock + if current.GetError() != nil { + // 'current' would be modified if the lock being resolved + if err := s.resolveCurrentLock(bo, current); err != nil { + s.Close() + return errors.Trace(err) + } + + // The check here does not violate the KeyOnly semantic, because current's value + // is filled by resolveCurrentLock which fetches the value by snapshot.get, so an empty + // value stands for NotExist + if len(current.Value) == 0 { + continue + } } return nil } @@ -115,11 +124,7 @@ func (s *Scanner) startTS() uint64 { return s.snapshot.version.Ver } -func (s *Scanner) resolveCurrentLock(bo *Backoffer) error { - current := s.cache[s.idx] - if current.GetError() == nil { - return nil - } +func (s *Scanner) resolveCurrentLock(bo *Backoffer, current *pb.KvPair) error { val, err := s.snapshot.get(bo, kv.Key(current.Key)) if err != nil { return errors.Trace(err) @@ -144,6 +149,7 @@ func (s *Scanner) getData(bo *Backoffer) error { StartKey: s.nextStartKey, Limit: uint32(s.batchSize), Version: s.startTS(), + KeyOnly: s.snapshot.keyOnly, }, Context: pb.Context{ Priority: s.snapshot.priority, diff --git a/store/tikv/scan_test.go b/store/tikv/scan_test.go index 8cacd1e1dd26c..e4897bc2de2b7 100644 --- a/store/tikv/scan_test.go +++ b/store/tikv/scan_test.go @@ -18,6 +18,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/tidb/kv" "golang.org/x/net/context" ) @@ -91,5 +92,41 @@ func (s *testScanSuite) TestSeek(c *C) { } scan.Next() c.Assert(scan.Valid(), IsFalse) + + txn3 := s.beginTxn(c) + txn3.SetOption(kv.KeyOnly, true) + scan, err = txn3.Seek(encodeKey(s.prefix, "")) + c.Assert(err, IsNil) + + for i := 0; i < rowNum; i++ { + k := scan.Key() + c.Assert([]byte(k), BytesEquals, encodeKey(s.prefix, s08d("key", i))) + // Because newScan return first item without calling scan.Next() just like go-hbase, + // for-loop count will decrease 1. + if i < rowNum-1 { + scan.Next() + } + } + scan.Next() + c.Assert(scan.Valid(), IsFalse) + + // Restore KeyOnly to false + txn3.SetOption(kv.KeyOnly, false) + scan, err = txn3.Seek(encodeKey(s.prefix, "")) + c.Assert(err, IsNil) + + for i := 0; i < rowNum; i++ { + k := scan.Key() + c.Assert([]byte(k), BytesEquals, encodeKey(s.prefix, s08d("key", i))) + v := scan.Value() + c.Assert(v, BytesEquals, valueBytes(i)) + // Because newScan return first item without calling scan.Next() just like go-hbase, + // for-loop count will decrease 1. + if i < rowNum-1 { + scan.Next() + } + } + scan.Next() + c.Assert(scan.Valid(), IsFalse) } } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 986dd2dd58943..c51983495cde3 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -47,6 +47,7 @@ type tikvSnapshot struct { priority pb.CommandPri notFillCache bool syncLog bool + keyOnly bool vars *kv.Variables } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index a39a0d110ceaa..5fa3cf28d0f96 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -150,6 +150,8 @@ func (txn *tikvTxn) SetOption(opt kv.Option, val interface{}) { txn.snapshot.notFillCache = val.(bool) case kv.SyncLog: txn.snapshot.syncLog = val.(bool) + case kv.KeyOnly: + txn.snapshot.keyOnly = val.(bool) } } From 7f37bad60ff75cc8e7a5c054155aa00eaad769a5 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Fri, 31 Aug 2018 00:11:04 +0800 Subject: [PATCH 32/41] expression: propagate more filters in PropagateConstant (#7276) --- cmd/explaintest/r/explain_easy.result | 11 +- expression/constant_propagation.go | 182 +++++++++++++++++--------- expression/constant_test.go | 36 +++++ plan/cbo_test.go | 9 +- plan/logical_plan_test.go | 2 +- plan/physical_plan_test.go | 2 +- 6 files changed, 169 insertions(+), 73 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index d08e79cde7ce6..ff0e9159abeaa 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -274,11 +274,12 @@ Projection_11 10000.00 root 9_aux_0 ├─TableReader_15 10000.00 root data:TableScan_14 │ └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_20 1.00 root funcs:count(1) - └─IndexJoin_43 10000.00 root inner join, inner:TableReader_42, outer key:s.a, inner key:t1.a - ├─TableReader_51 1.00 root data:TableScan_50 - │ └─TableScan_50 1.00 cop table:s, range: decided by [eq(s.a, test.t.a)], keep order:false, stats:pseudo - └─TableReader_42 10.00 root data:TableScan_41 - └─TableScan_41 10.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo + └─IndexJoin_44 10000.00 root inner join, inner:TableReader_43, outer key:s.a, inner key:t1.a + ├─TableReader_52 1.00 root data:TableScan_51 + │ └─TableScan_51 1.00 cop table:s, range: decided by [eq(s.a, test.t.a)], keep order:false, stats:pseudo + └─TableReader_43 8000.00 root data:Selection_42 + └─Selection_42 8000.00 cop eq(t1.a, test.t.a) + └─TableScan_41 10.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.a = t1.a) from t; id count task operator info Projection_11 10000.00 root 9_aux_0 diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index 232462e026986..c0573b6712b0c 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -27,19 +27,6 @@ import ( // MaxPropagateColsCnt means the max number of columns that can participate propagation. var MaxPropagateColsCnt = 100 -var eqFuncNameMap = map[string]bool{ - ast.EQ: true, -} - -// inEqFuncNameMap stores all the in-equal operators that can be propagated. -var inEqFuncNameMap = map[string]bool{ - ast.LT: true, - ast.GT: true, - ast.LE: true, - ast.GE: true, - ast.NE: true, -} - type multiEqualSet struct { parent []int } @@ -72,51 +59,12 @@ type propagateConstantSolver struct { ctx sessionctx.Context } -// propagateInEQ propagates all in-equal conditions. -// e.g. For expression a = b and b = c and c = d and c < 1 , we can get extra a < 1 and b < 1 and d < 1. -// We maintain a unionSet representing the equivalent for every two columns. -func (s *propagateConstantSolver) propagateInEQ() { - s.unionSet = &multiEqualSet{} - s.unionSet.init(len(s.columns)) - for i := range s.conditions { - if fun, ok := s.conditions[i].(*ScalarFunction); ok && fun.FuncName.L == ast.EQ { - lCol, lOk := fun.GetArgs()[0].(*Column) - rCol, rOk := fun.GetArgs()[1].(*Column) - if lOk && rOk { - lID := s.getColID(lCol) - rID := s.getColID(rCol) - s.unionSet.addRelation(lID, rID) - } - } - } - condsLen := len(s.conditions) - for i := 0; i < condsLen; i++ { - cond := s.conditions[i] - col, con := s.validPropagateCond(cond, inEqFuncNameMap) - if col == nil { - continue - } - id := s.getColID(col) - for j := range s.columns { - if id != j && s.unionSet.findRoot(id) == s.unionSet.findRoot(j) { - funName := cond.(*ScalarFunction).FuncName.L - var newExpr Expression - if _, ok := cond.(*ScalarFunction).GetArgs()[0].(*Column); ok { - newExpr = NewFunctionInternal(s.ctx, funName, cond.GetType(), s.columns[j], con) - } else { - newExpr = NewFunctionInternal(s.ctx, funName, cond.GetType(), con, s.columns[j]) - } - s.conditions = append(s.conditions, newExpr) - } - } - } -} - -// propagateEQ propagates equal expression multiple times. An example runs as following: +// propagateConstantEQ propagates expressions like 'column = constant' by substituting the constant for column, the +// procedure repeats multiple times. An example runs as following: // a = d & b * 2 = c & c = d + 2 & b = 1 & a = 4, we pick eq cond b = 1 and a = 4 // d = 4 & 2 = c & c = d + 2 & b = 1 & a = 4, we propagate b = 1 and a = 4 and pick eq cond c = 2 and d = 4 // d = 4 & 2 = c & false & b = 1 & a = 4, we propagate c = 2 and d = 4, and do constant folding: c = d + 2 will be folded as false. -func (s *propagateConstantSolver) propagateEQ() { +func (s *propagateConstantSolver) propagateConstantEQ() { s.eqList = make([]*Constant, len(s.columns)) visited := make([]bool, len(s.conditions)) for i := 0; i < MaxPropagateColsCnt; i++ { @@ -138,10 +86,72 @@ func (s *propagateConstantSolver) propagateEQ() { } } -// validPropagateCond checks if the cond is an expression like [column op constant] and op is in the funNameMap. -func (s *propagateConstantSolver) validPropagateCond(cond Expression, funNameMap map[string]bool) (*Column, *Constant) { +// propagateColumnEQ propagates expressions like 'column A = column B' by adding extra filters +// 'expression(..., column B, ...)' propagated from 'expression(..., column A, ...)' as long as: +// +// 1. The expression is deterministic +// 2. The expression doesn't have any side effect +// +// e.g. For expression a = b and b = c and c = d and c < 1 , we can get extra a < 1 and b < 1 and d < 1. +// However, for a = b and a < rand(), we cannot propagate a < rand() to b < rand() because rand() is non-deterministic +// +// This propagation may bring redundancies that we need to resolve later, for example: +// for a = b and a < 3 and b < 3, we get new a < 3 and b < 3, which are redundant +// for a = b and a < 3 and 3 > b, we get new b < 3 and 3 > a, which are redundant +// for a = b and a < 3 and b < 4, we get new a < 4 and b < 3 but should expect a < 3 and b < 3 +// for a = b and a in (3) and b in (4), we get b in (3) and a in (4) but should expect 'false' +// +// TODO: remove redundancies later +// +// We maintain a unionSet representing the equivalent for every two columns. +func (s *propagateConstantSolver) propagateColumnEQ() { + visited := make([]bool, len(s.conditions)) + s.unionSet = &multiEqualSet{} + s.unionSet.init(len(s.columns)) + for i := range s.conditions { + if fun, ok := s.conditions[i].(*ScalarFunction); ok && fun.FuncName.L == ast.EQ { + lCol, lOk := fun.GetArgs()[0].(*Column) + rCol, rOk := fun.GetArgs()[1].(*Column) + if lOk && rOk { + lID := s.getColID(lCol) + rID := s.getColID(rCol) + s.unionSet.addRelation(lID, rID) + visited[i] = true + } + } + } + + condsLen := len(s.conditions) + for i, coli := range s.columns { + for j := i + 1; j < len(s.columns); j++ { + // unionSet doesn't have iterate(), we use a two layer loop to iterate col_i = col_j relation + if s.unionSet.findRoot(i) != s.unionSet.findRoot(j) { + continue + } + colj := s.columns[j] + for k := 0; k < condsLen; k++ { + if visited[k] { + // cond_k has been used to retrieve equality relation + continue + } + cond := s.conditions[k] + replaced, _, newExpr := s.tryToReplaceCond(coli, colj, cond) + if replaced { + s.conditions = append(s.conditions, newExpr) + } + replaced, _, newExpr = s.tryToReplaceCond(colj, coli, cond) + if replaced { + s.conditions = append(s.conditions, newExpr) + } + } + } + } +} + +// validEqualCond checks if the cond is an expression like [column eq constant]. +func (s *propagateConstantSolver) validEqualCond(cond Expression) (*Column, *Constant) { if eq, ok := cond.(*ScalarFunction); ok { - if _, ok := funNameMap[eq.FuncName.L]; !ok { + if eq.FuncName.L != ast.EQ { return nil, nil } if col, colOk := eq.GetArgs()[0].(*Column); colOk { @@ -158,6 +168,54 @@ func (s *propagateConstantSolver) validPropagateCond(cond Expression, funNameMap return nil, nil } +// tryToReplaceCond aims to replace all occurrences of column 'src' and try to replace it with 'tgt' in 'cond' +// It returns +// bool: if a replacement happened +// bool: if 'cond' contains non-deterministic expression +// Expression: the replaced expression, or original 'cond' if the replacement didn't happen +// +// For example: +// for 'a, b, a < 3', it returns 'true, false, b < 3' +// for 'a, b, sin(a) + cos(a) = 5', it returns 'true, false, returns sin(b) + cos(b) = 5' +// for 'a, b, cast(a) < rand()', it returns 'false, true, cast(a) < rand()' +func (s *propagateConstantSolver) tryToReplaceCond(src *Column, tgt *Column, cond Expression) (bool, bool, Expression) { + sf, ok := cond.(*ScalarFunction) + if !ok { + return false, false, cond + } + replaced := false + var args []Expression + if _, ok := unFoldableFunctions[sf.FuncName.L]; ok { + return false, true, cond + } + for idx, expr := range sf.GetArgs() { + if src.Equal(nil, expr) { + replaced = true + if args == nil { + args = make([]Expression, len(sf.GetArgs())) + copy(args, sf.GetArgs()) + } + args[idx] = tgt + } else { + subReplaced, isNonDeterminisitic, subExpr := s.tryToReplaceCond(src, tgt, expr) + if isNonDeterminisitic { + return false, true, cond + } else if subReplaced { + replaced = true + if args == nil { + args = make([]Expression, len(sf.GetArgs())) + copy(args, sf.GetArgs()) + } + args[idx] = subExpr + } + } + } + if replaced { + return true, false, NewFunctionInternal(s.ctx, sf.FuncName.L, sf.GetType(), args...) + } + return false, false, cond +} + func (s *propagateConstantSolver) setConds2ConstFalse() { s.conditions = []Expression{&Constant{ Value: types.NewDatum(false), @@ -172,7 +230,7 @@ func (s *propagateConstantSolver) pickNewEQConds(visited []bool) (retMapper map[ if visited[i] { continue } - col, con := s.validPropagateCond(cond, eqFuncNameMap) + col, con := s.validEqualCond(cond) // Then we check if this CNF item is a false constant. If so, we will set the whole condition to false. var ok bool if col == nil { @@ -227,8 +285,8 @@ func (s *propagateConstantSolver) solve(conditions []Expression) []Expression { log.Warnf("[const_propagation]Too many columns in a single CNF: the column count is %d, the max count is %d.", len(s.columns), MaxPropagateColsCnt) return conditions } - s.propagateEQ() - s.propagateInEQ() + s.propagateConstantEQ() + s.propagateColumnEQ() for i, cond := range s.conditions { if dnf, ok := cond.(*ScalarFunction); ok && dnf.FuncName.L == ast.LogicOr { dnfItems := SplitDNFItems(cond) @@ -255,7 +313,7 @@ func (s *propagateConstantSolver) insertCol(col *Column) { } } -// PropagateConstant propagate constant values of equality predicates and inequality predicates in a condition. +// PropagateConstant propagate constant values of deterministic predicates in a condition. func PropagateConstant(ctx sessionctx.Context, conditions []Expression) []Expression { solver := &propagateConstantSolver{ colMapper: make(map[string]int), diff --git a/expression/constant_test.go b/expression/constant_test.go index c36c2d39e2411..cac21ca495b1b 100644 --- a/expression/constant_test.go +++ b/expression/constant_test.go @@ -113,6 +113,42 @@ func (*testExpressionSuite) TestConstantPropagation(c *C) { }, result: "0", }, + { + conditions: []Expression{ + newFunction(ast.EQ, newColumn(0), newColumn(1)), + newFunction(ast.In, newColumn(0), newLonglong(1), newLonglong(2)), + newFunction(ast.In, newColumn(1), newLonglong(3), newLonglong(4)), + }, + result: "eq(test.t.0, test.t.1), in(test.t.0, 1, 2), in(test.t.0, 3, 4), in(test.t.1, 1, 2), in(test.t.1, 3, 4)", + }, + { + conditions: []Expression{ + newFunction(ast.EQ, newColumn(0), newColumn(1)), + newFunction(ast.EQ, newColumn(0), newFunction(ast.BitLength, newColumn(2))), + }, + result: "eq(test.t.0, bit_length(cast(test.t.2))), eq(test.t.0, test.t.1), eq(test.t.1, bit_length(cast(test.t.2)))", + }, + { + conditions: []Expression{ + newFunction(ast.EQ, newColumn(0), newColumn(1)), + newFunction(ast.LE, newFunction(ast.Mul, newColumn(0), newColumn(0)), newLonglong(50)), + }, + result: "eq(test.t.0, test.t.1), le(mul(test.t.0, test.t.0), 50), le(mul(test.t.1, test.t.1), 50)", + }, + { + conditions: []Expression{ + newFunction(ast.EQ, newColumn(0), newColumn(1)), + newFunction(ast.LE, newColumn(0), newFunction(ast.Plus, newColumn(1), newLonglong(1))), + }, + result: "eq(test.t.0, test.t.1), le(test.t.0, plus(test.t.0, 1)), le(test.t.0, plus(test.t.1, 1)), le(test.t.1, plus(test.t.1, 1))", + }, + { + conditions: []Expression{ + newFunction(ast.EQ, newColumn(0), newColumn(1)), + newFunction(ast.LE, newColumn(0), newFunction(ast.Rand)), + }, + result: "eq(test.t.0, test.t.1), le(cast(test.t.0), rand())", + }, } for _, tt := range tests { ctx := mock.NewContext() diff --git a/plan/cbo_test.go b/plan/cbo_test.go index 832ac06ede355..5f90f40e36576 100644 --- a/plan/cbo_test.go +++ b/plan/cbo_test.go @@ -37,7 +37,7 @@ var _ = Suite(&testAnalyzeSuite{}) type testAnalyzeSuite struct { } -// CBOWithoutAnalyze tests the plan with stats that only have count info. +// TestCBOWithoutAnalyze tests the plan with stats that only have count info. func (s *testAnalyzeSuite) TestCBOWithoutAnalyze(c *C) { defer testleak.AfterTest(c)() store, dom, err := newStoreWithBootstrap() @@ -633,12 +633,13 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { " ├─TableReader_15 10.00 root data:TableScan_14", " │ └─TableScan_14 10.00 cop table:t, range:[-inf,+inf], keep order:false", " └─StreamAgg_20 1.00 root funcs:count(1)", - " └─HashRightJoin_22 1.00 root inner join, inner:TableReader_25, equal:[eq(s.a, t1.a)]", + " └─HashLeftJoin_21 1.00 root inner join, inner:TableReader_28, equal:[eq(s.a, t1.a)]", " ├─TableReader_25 1.00 root data:Selection_24", " │ └─Selection_24 1.00 cop eq(s.a, test.t.a)", " │ └─TableScan_23 10.00 cop table:s, range:[-inf,+inf], keep order:false", - " └─TableReader_27 10.00 root data:TableScan_26", - " └─TableScan_26 10.00 cop table:t1, range:[-inf,+inf], keep order:false", + " └─TableReader_28 1.00 root data:Selection_27", + " └─Selection_27 1.00 cop eq(t1.a, test.t.a)", + " └─TableScan_26 10.00 cop table:t1, range:[-inf,+inf], keep order:false", )) tk.MustQuery("explain select (select concat(t1.a, \",\", t1.b) from t t1 where t1.a=t.a and t1.c=t.c) from t"). Check(testkit.Rows( diff --git a/plan/logical_plan_test.go b/plan/logical_plan_test.go index eb80d298edde2..2f70772b0544f 100644 --- a/plan/logical_plan_test.go +++ b/plan/logical_plan_test.go @@ -315,7 +315,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { }, { sql: "select * from t t1, t t2 where t1.a = t2.b and t2.b > 0 and t1.a = t1.c and t1.d like 'abc' and t2.d = t1.d", - best: "Join{DataScan(t2)->DataScan(t1)->Sel([like(cast(t1.d), abc, 92)])}(t2.b,t1.a)(t2.d,t1.d)->Projection", + best: "Join{DataScan(t2)->Sel([like(cast(t2.d), abc, 92)])->DataScan(t1)->Sel([like(cast(t1.d), abc, 92)])}(t2.b,t1.a)(t2.d,t1.d)->Projection", }, { sql: "select * from t ta join t tb on ta.d = tb.d and ta.d > 1 where tb.a = 0", diff --git a/plan/physical_plan_test.go b/plan/physical_plan_test.go index 8fc193f08b381..189ccc23fab72 100644 --- a/plan/physical_plan_test.go +++ b/plan/physical_plan_test.go @@ -467,7 +467,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderSubquery(c *C) { // Test Apply. { sql: "select t.c in (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t", - best: "Apply{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(s.a,t1.a)->StreamAgg}->Projection", + best: "Apply{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->TableReader(Table(t)->Sel([eq(t1.a, test.t.a)]))}(s.a,t1.a)->StreamAgg}->Projection", }, { sql: "select (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t", From 87c54b2610c73bc215f8a48b417810d751015d7c Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 31 Aug 2018 01:14:43 +0800 Subject: [PATCH 33/41] plan, statistics: maintain `HistColl` in `DataSource`'s `StatsInfo` (#7385) --- expression/column.go | 17 ++-- expression/column_test.go | 2 +- expression/constant_test.go | 2 +- expression/distsql_builtin_test.go | 4 +- expression/expression_test.go | 2 +- plan/logical_plans.go | 8 +- plan/logical_plans_test.go | 45 +++++----- plan/point_get_plan.go | 2 +- plan/stats.go | 6 +- sessionctx/variable/session.go | 4 +- statistics/boostrap.go | 3 - statistics/feedback.go | 22 ++--- statistics/selectivity.go | 18 ++-- statistics/selectivity_test.go | 14 ++- statistics/table.go | 136 +++++++++++++++++++++-------- statistics/update.go | 2 +- 16 files changed, 182 insertions(+), 105 deletions(-) diff --git a/expression/column.go b/expression/column.go index f63cf44a117a5..90d6b61811dfe 100644 --- a/expression/column.go +++ b/expression/column.go @@ -151,7 +151,7 @@ type Column struct { // We'll try to remove it in the future. ID int64 // UniqueID is the unique id of this column. - UniqueID int + UniqueID int64 // IsAggOrSubq means if this column is referenced to a Aggregation column or a Subquery column. // If so, this column's name will be the plain sql text. IsAggOrSubq bool @@ -376,17 +376,20 @@ func IndexInfo2Cols(cols []*Column, index *model.IndexInfo) ([]*Column, []int) { return retCols, lengths } -// FindColumnsByUniqueIDs will find columns by checking the unique id. -// Note: `ids` must be a subset of the column slice. -func FindColumnsByUniqueIDs(cols []*Column, ids []int) []*Column { - retCols := make([]*Column, 0, len(ids)) - for _, id := range ids { +// FindPrefixOfIndex will find columns in index by checking the unique id. +// So it will return at once no matching column is found. +func FindPrefixOfIndex(cols []*Column, idxColIDs []int64) []*Column { + retCols := make([]*Column, 0, len(idxColIDs)) +idLoop: + for _, id := range idxColIDs { for _, col := range cols { if col.UniqueID == id { retCols = append(retCols, col) - break + continue idLoop } } + // If no matching column is found, just return. + return retCols } return retCols } diff --git a/expression/column_test.go b/expression/column_test.go index 4093528f36ab2..62a180aab604a 100644 --- a/expression/column_test.go +++ b/expression/column_test.go @@ -112,7 +112,7 @@ func (s *testEvaluatorSuite) TestColumn2Expr(c *C) { cols := make([]*Column, 0, 5) for i := 0; i < 5; i++ { - cols = append(cols, &Column{UniqueID: i}) + cols = append(cols, &Column{UniqueID: int64(i)}) } exprs := Column2Exprs(cols) diff --git a/expression/constant_test.go b/expression/constant_test.go index cac21ca495b1b..a2f4376afeeb2 100644 --- a/expression/constant_test.go +++ b/expression/constant_test.go @@ -33,7 +33,7 @@ type testExpressionSuite struct{} func newColumn(id int) *Column { return &Column{ - UniqueID: id, + UniqueID: int64(id), ColName: model.NewCIStr(fmt.Sprint(id)), TblName: model.NewCIStr("t"), DBName: model.NewCIStr("test"), diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index 27cd7c906cc77..2df78ca40fe9b 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -29,14 +29,14 @@ import ( var _ = Suite(&testEvalSuite{}) type testEvalSuite struct { - colID int + colID int64 } func (s *testEvalSuite) SetUpSuite(c *C) { s.colID = 0 } -func (s *testEvalSuite) allocColID() int { +func (s *testEvalSuite) allocColID() int64 { s.colID++ return s.colID } diff --git a/expression/expression_test.go b/expression/expression_test.go index d6db7f23e1e28..0fe3b7da63bd5 100644 --- a/expression/expression_test.go +++ b/expression/expression_test.go @@ -128,7 +128,7 @@ func tableInfoToSchemaForTest(tableInfo *model.TableInfo) *Schema { schema := NewSchema(make([]*Column, 0, len(columns))...) for i, col := range columns { schema.Append(&Column{ - UniqueID: i, + UniqueID: int64(i), TblName: tableInfo.Name, ColName: col.Name, ID: col.ID, diff --git a/plan/logical_plans.go b/plan/logical_plans.go index 12d2ff433106d..6dc44e7601bfb 100644 --- a/plan/logical_plans.go +++ b/plan/logical_plans.go @@ -417,7 +417,7 @@ func (ds *DataSource) deriveIndexPathStats(path *accessPath) (bool, error) { if err != nil { return false, errors.Trace(err) } - path.countAfterAccess, err = ds.statisticTable.GetRowCountByIndexRanges(sc, path.index.ID, path.ranges) + path.countAfterAccess, err = ds.stats.histColl.GetRowCountByIndexRanges(sc, path.index.ID, path.ranges) if err != nil { return false, errors.Trace(err) } @@ -435,8 +435,8 @@ func (ds *DataSource) deriveIndexPathStats(path *accessPath) (bool, error) { } path.indexFilters, path.tableFilters = splitIndexFilterConditions(path.tableFilters, path.index.Columns, ds.tableInfo) if corColInAccessConds { - idxHist, ok := ds.statisticTable.Indices[path.index.ID] - if ok && !ds.statisticTable.Pseudo { + idxHist, ok := ds.stats.histColl.Indices[path.index.ID] + if ok && !ds.stats.histColl.Pseudo { path.countAfterAccess = idxHist.AvgCountPerValue(ds.statisticTable.Count) } else { path.countAfterAccess = ds.statisticTable.PseudoAvgCountPerValue() @@ -448,7 +448,7 @@ func (ds *DataSource) deriveIndexPathStats(path *accessPath) (bool, error) { path.countAfterAccess = math.Min(ds.stats.count/selectionFactor, float64(ds.statisticTable.Count)) } if path.indexFilters != nil { - selectivity, err := ds.statisticTable.Selectivity(ds.ctx, path.indexFilters) + selectivity, err := ds.stats.histColl.Selectivity(ds.ctx, path.indexFilters) if err != nil { log.Warnf("An error happened: %v, we have to use the default selectivity", err.Error()) selectivity = selectionFactor diff --git a/plan/logical_plans_test.go b/plan/logical_plans_test.go index 6ec6894d51381..9531e8c32a2eb 100644 --- a/plan/logical_plans_test.go +++ b/plan/logical_plans_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testleak" + "github.com/sirupsen/logrus" ) var _ = Suite(&testUnitTestSuit{}) @@ -43,7 +44,7 @@ func (s *testUnitTestSuit) newTypeWithFlen(typeByte byte, flen int) *types.Field return tp } -func (s *testUnitTestSuit) SubstituteCol2CorCol(expr expression.Expression, colIDs map[int]struct{}) (expression.Expression, error) { +func (s *testUnitTestSuit) SubstituteCol2CorCol(expr expression.Expression, colIDs map[int64]struct{}) (expression.Expression, error) { switch x := expr.(type) { case *expression.ScalarFunction: newArgs := make([]expression.Expression, 0, len(x.GetArgs())) @@ -94,72 +95,72 @@ func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { }) testCases := []struct { expr string - corColIDs []int - idxColIDs []int + corColIDs []int64 + idxColIDs []int64 idxColLens []int access string remained string }{ { expr: "col1 = col2", - corColIDs: []int{2}, - idxColIDs: []int{1}, + corColIDs: []int64{2}, + idxColIDs: []int64{1}, idxColLens: []int{types.UnspecifiedLength}, access: "[eq(col1, col2)]", remained: "[]", }, { expr: "col1 = col5 and col2 = 1", - corColIDs: []int{5}, - idxColIDs: []int{1, 2}, + corColIDs: []int64{5}, + idxColIDs: []int64{1, 2}, idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, access: "[eq(col1, col5) eq(col2, 1)]", remained: "[]", }, { expr: "col1 = col5 and col2 = 1", - corColIDs: []int{5}, - idxColIDs: []int{2, 1}, + corColIDs: []int64{5}, + idxColIDs: []int64{2, 1}, idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, access: "[eq(col2, 1) eq(col1, col5)]", remained: "[]", }, { expr: "col1 = col5 and col2 = 1", - corColIDs: []int{5}, - idxColIDs: []int{1}, + corColIDs: []int64{5}, + idxColIDs: []int64{1}, idxColLens: []int{types.UnspecifiedLength}, access: "[eq(col1, col5)]", remained: "[eq(col2, 1)]", }, { expr: "col2 = 1 and col1 = col5", - corColIDs: []int{5}, - idxColIDs: []int{1}, + corColIDs: []int64{5}, + idxColIDs: []int64{1}, idxColLens: []int{types.UnspecifiedLength}, access: "[eq(col1, col5)]", remained: "[eq(col2, 1)]", }, { expr: "col1 = col2 and col3 = col4 and col5 = 1", - corColIDs: []int{2, 4}, - idxColIDs: []int{1, 3}, + corColIDs: []int64{2, 4}, + idxColIDs: []int64{1, 3}, idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength}, access: "[eq(col1, col2) eq(col3, col4)]", remained: "[eq(col5, 1)]", }, { expr: "col1 = col2 and col3 = col4 and col5 = 1", - corColIDs: []int{2, 4}, - idxColIDs: []int{1, 3}, + corColIDs: []int64{2, 4}, + idxColIDs: []int64{1, 3}, idxColLens: []int{types.UnspecifiedLength, 2}, access: "[eq(col1, col2) eq(col3, col4)]", remained: "[eq(col3, col4) eq(col5, 1)]", }, { expr: `col1 = col5 and col3 = "col1" and col2 = col5`, - corColIDs: []int{5}, - idxColIDs: []int{1, 2, 3}, + corColIDs: []int64{5}, + idxColIDs: []int64{1, 2, 3}, idxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength, types.UnspecifiedLength}, access: "[eq(col1, col5) eq(col2, col5) eq(col3, col1)]", remained: "[]", @@ -173,7 +174,7 @@ func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { } c.Assert(err, IsNil, comment) trueFilters := make([]expression.Expression, 0, len(filters)) - idMap := make(map[int]struct{}) + idMap := make(map[int64]struct{}) for _, id := range tt.corColIDs { idMap[id] = struct{}{} } @@ -185,9 +186,11 @@ func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { path := accessPath{ eqCondCount: 0, tableFilters: trueFilters, - idxCols: expression.FindColumnsByUniqueIDs(totalSchema.Columns, tt.idxColIDs), + idxCols: expression.FindPrefixOfIndex(totalSchema.Columns, tt.idxColIDs), idxColLens: tt.idxColLens, } + + logrus.Warnf("idx cols: %v", path.idxCols) access, remained := path.splitCorColAccessCondFromFilters() c.Assert(fmt.Sprintf("%s", access), Equals, tt.access, comment) c.Assert(fmt.Sprintf("%s", remained), Equals, tt.remained, comment) diff --git a/plan/point_get_plan.go b/plan/point_get_plan.go index 43f119bcd49a8..b74eb5995e542 100644 --- a/plan/point_get_plan.go +++ b/plan/point_get_plan.go @@ -424,7 +424,7 @@ func colInfoToColumn(db model.CIStr, tblName model.CIStr, asName model.CIStr, co TblName: tblName, RetType: &col.FieldType, ID: col.ID, - UniqueID: col.Offset, + UniqueID: int64(col.Offset), Index: idx, } } diff --git a/plan/stats.go b/plan/stats.go index b28c6c128225f..32a19a084acc1 100644 --- a/plan/stats.go +++ b/plan/stats.go @@ -19,6 +19,7 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/statistics" log "github.com/sirupsen/logrus" ) @@ -27,6 +28,7 @@ type statsInfo struct { count float64 cardinality []float64 + histColl statistics.HistColl // usePseudoStats indicates whether the statsInfo is calculated using the // pseudo statistics on a table. usePseudoStats bool @@ -45,6 +47,7 @@ func (s *statsInfo) scale(factor float64) *statsInfo { profile := &statsInfo{ count: s.count * factor, cardinality: make([]float64, len(s.cardinality)), + histColl: s.histColl, usePseudoStats: s.usePseudoStats, } for i := range profile.cardinality { @@ -111,6 +114,7 @@ func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) *statsInfo { profile := &statsInfo{ count: float64(ds.statisticTable.Count), cardinality: make([]float64, len(ds.Columns)), + histColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.Columns, ds.schema.Columns), usePseudoStats: ds.statisticTable.Pseudo, } for i, col := range ds.Columns { @@ -123,7 +127,7 @@ func (ds *DataSource) getStatsByFilter(conds expression.CNFExprs) *statsInfo { } } ds.stats = profile - selectivity, err := ds.statisticTable.Selectivity(ds.ctx, conds) + selectivity, err := profile.histColl.Selectivity(ds.ctx, conds) if err != nil { log.Warnf("An error happened: %v, we have to use the default selectivity", err.Error()) selectivity = selectionFactor diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index f15e4cae39c8c..f8c6e1183c43b 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -209,7 +209,7 @@ type SessionVars struct { PlanID int // PlanColumnID is the unique id for column when building plan. - PlanColumnID int + PlanColumnID int64 // User is the user identity with which the session login. User *auth.UserIdentity @@ -364,7 +364,7 @@ func (s *SessionVars) CleanBuffers() { } // AllocPlanColumnID allocates column id for planner. -func (s *SessionVars) AllocPlanColumnID() int { +func (s *SessionVars) AllocPlanColumnID() int64 { s.PlanColumnID++ return s.PlanColumnID } diff --git a/statistics/boostrap.go b/statistics/boostrap.go index fb28b42cda7d1..938ccf0f31296 100644 --- a/statistics/boostrap.go +++ b/statistics/boostrap.go @@ -45,8 +45,6 @@ func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, tables statsCache ModifyCount: row.GetInt64(2), Columns: make(map[int64]*Column, len(tableInfo.Columns)), Indices: make(map[int64]*Index, len(tableInfo.Indices)), - colName2Idx: make(map[string]int64, len(tableInfo.Columns)), - colName2ID: make(map[string]int64, len(tableInfo.Columns)), } tbl := &Table{ HistColl: newHistColl, @@ -238,7 +236,6 @@ func (h *Handle) initStatsBuckets(tables statsCache) error { } col.PreCalculateScalar() } - table.buildColNameMapper() } return nil } diff --git a/statistics/feedback.go b/statistics/feedback.go index 652b02386f242..0e079010f8ff9 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -843,22 +843,18 @@ func logForIndex(prefix string, t *Table, idx *Index, ranges []*ranger.Range, ac HighVal: []types.Datum{ran.HighVal[rangePosition]}, } colName := idx.Info.Columns[rangePosition].Name.L - var rangeString string // prefer index stats over column stats - if idx, ok := t.colName2Idx[colName]; ok { - if t.Indices[idx] == nil { - return - } - rangeString = logForIndexRange(t.Indices[idx], &rang, -1, factor) + if idxHist := t.indexStartWithColumnForDebugLog(colName); idxHist != nil { + rangeString := logForIndexRange(idxHist, &rang, -1, factor) + log.Debugf("%s index: %s, actual: %d, equality: %s, expected equality: %d, %s", prefix, idx.Info.Name.O, + actual[i], equalityString, equalityCount, rangeString) + } else if colHist := t.columnByNameForDebugLog(colName); colHist != nil { + rangeString := colRangeToStr(colHist, &rang, -1, factor) + log.Debugf("%s index: %s, actual: %d, equality: %s, expected equality: %d, %s", prefix, idx.Info.Name.O, + actual[i], equalityString, equalityCount, rangeString) } else { - id := t.colName2ID[colName] - if t.Columns[id] == nil { - return - } - rangeString = colRangeToStr(t.Columns[t.colName2ID[colName]], &rang, -1, factor) + return } - log.Debugf("%s index: %s, actual: %d, equality: %s, expected equality: %d, %s", prefix, idx.Info.Name.O, - actual[i], equalityString, equalityCount, rangeString) } } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 79260584adbba..a854a881b1fc4 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -160,8 +160,8 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp // Deal with the correlated column. for _, expr := range exprs { - if c := isColEqCorCol(expr); c != nil && !coll.ColumnIsInvalid(sc, c.ID) { - colHist := coll.Columns[c.ID] + if c := isColEqCorCol(expr); c != nil && !coll.ColumnIsInvalid(sc, c.UniqueID) { + colHist := coll.Columns[c.UniqueID] if colHist.NDV > 0 { ret *= 1 / float64(colHist.NDV) } @@ -172,27 +172,31 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp extractedCols := make([]*expression.Column, 0, len(coll.Columns)) extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, remainedExprs, nil) - for _, colInfo := range coll.Columns { + for id, colInfo := range coll.Columns { col := expression.ColInfo2Col(extractedCols, colInfo.Info) if col != nil { maskCovered, ranges, err := getMaskAndRanges(ctx, remainedExprs, ranger.ColumnRangeType, nil, col) if err != nil { return 0, errors.Trace(err) } - sets = append(sets, &exprSet{tp: colType, ID: col.ID, mask: maskCovered, ranges: ranges, numCols: 1}) + sets = append(sets, &exprSet{tp: colType, ID: id, mask: maskCovered, ranges: ranges, numCols: 1}) if mysql.HasPriKeyFlag(colInfo.Info.Flag) { sets[len(sets)-1].tp = pkType } } } - for _, idxInfo := range coll.Indices { - idxCols, lengths := expression.IndexInfo2Cols(extractedCols, idxInfo.Info) + for id, idxInfo := range coll.Indices { + idxCols := expression.FindPrefixOfIndex(extractedCols, coll.Idx2ColumnIDs[id]) if len(idxCols) > 0 { + lengths := make([]int, 0, len(idxCols)) + for i := 0; i < len(idxCols); i++ { + lengths = append(lengths, idxInfo.Info.Columns[i].Length) + } maskCovered, ranges, err := getMaskAndRanges(ctx, remainedExprs, ranger.IndexRangeType, lengths, idxCols...) if err != nil { return 0, errors.Trace(err) } - sets = append(sets, &exprSet{tp: indexType, ID: idxInfo.ID, mask: maskCovered, ranges: ranges, numCols: len(idxInfo.Info.Columns)}) + sets = append(sets, &exprSet{tp: indexType, ID: id, mask: maskCovered, ranges: ranges, numCols: len(idxInfo.Info.Columns)}) } } sets = getUsableSetsByGreedy(sets) diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 49744337bbcec..f2ae8b6819f75 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -188,19 +188,25 @@ func (s *testSelectivitySuite) TestSelectivity(c *C) { stmts, err := session.Parse(ctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprs)) c.Assert(stmts, HasLen, 1) + err = plan.Preprocess(ctx, stmts[0], is, false) c.Assert(err, IsNil, comment) p, err := plan.BuildLogicalPlan(ctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, tt.exprs)) - ratio, err := statsTbl.Selectivity(ctx, p.(plan.LogicalPlan).Children()[0].(*plan.LogicalSelection).Conditions) + + sel := p.(plan.LogicalPlan).Children()[0].(*plan.LogicalSelection) + ds := sel.Children()[0].(*plan.DataSource) + + histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) + + ratio, err := histColl.Selectivity(ctx, sel.Conditions) c.Assert(err, IsNil, comment) c.Assert(math.Abs(ratio-tt.selectivity) < eps, IsTrue, Commentf("for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio)) - statsTbl.Count *= 10 - ratio, err = statsTbl.Selectivity(ctx, p.(plan.LogicalPlan).Children()[0].(*plan.LogicalSelection).Conditions) + histColl.Count *= 10 + ratio, err = histColl.Selectivity(ctx, sel.Conditions) c.Assert(err, IsNil, comment) c.Assert(math.Abs(ratio-tt.selectivity) < eps, IsTrue, Commentf("for %s, needed: %v, got: %v", tt.exprs, tt.selectivity, ratio)) - statsTbl.Count /= 10 } } diff --git a/statistics/table.go b/statistics/table.go index e6d23b13a87fd..5b0f4e6d3e213 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -20,6 +20,7 @@ import ( "sync" "github.com/juju/errors" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -51,15 +52,19 @@ type Table struct { // HistColl is a collection of histogram. It collects enough information for plan to calculate the selectivity. type HistColl struct { - PhysicalID int64 // PhysicalID is the partition id for a partitioned table, otherwise, it is the table id. + PhysicalID int64 + // HavePhysicalID is true means this HistColl is from single table and have its ID's information. + // The physical id is used when try to load column stats from storage. HavePhysicalID bool Columns map[int64]*Column Indices map[int64]*Index - colName2Idx map[string]int64 // map column name to index id - colName2ID map[string]int64 // map column name to column id - Pseudo bool - Count int64 - ModifyCount int64 // Total modify count in a table. + // Idx2ColumnIDs maps the index id to its column ids. It's used to calculate the selectivity in planner. + Idx2ColumnIDs map[int64][]int64 + // ColID2IdxID maps the column id to index id whose first column is it. It's used to calculate the selectivity in planner. + ColID2IdxID map[int64]int64 + Pseudo bool + Count int64 + ModifyCount int64 // Total modify count in a table. } func (t *Table) copy() *Table { @@ -69,8 +74,6 @@ func (t *Table) copy() *Table { Count: t.Count, Columns: make(map[int64]*Column), Indices: make(map[int64]*Index), - colName2Idx: make(map[string]int64), - colName2ID: make(map[string]int64), Pseudo: t.Pseudo, ModifyCount: t.ModifyCount, } @@ -80,12 +83,6 @@ func (t *Table) copy() *Table { for id, idx := range t.Indices { newHistColl.Indices[id] = idx } - for name, id := range t.colName2Idx { - newHistColl.colName2Idx[name] = id - } - for name, id := range t.colName2ID { - newHistColl.colName2ID[name] = id - } nt := &Table{ HistColl: newHistColl, Version: t.Version, @@ -94,16 +91,6 @@ func (t *Table) copy() *Table { return nt } -func (t *Table) buildColNameMapper() { - for id, col := range t.Columns { - t.colName2ID[col.Info.Name.L] = id - } - for id, idx := range t.Indices { - // use this index to estimate the column stats - t.colName2Idx[idx.Info.Columns[0].Name.L] = id - } -} - func (h *Handle) cmSketchFromStorage(tblID int64, isIndex, histID int64) (*CMSketch, error) { selSQL := fmt.Sprintf("select cm_sketch from mysql.stats_histograms where table_id = %d and is_index = %d and hist_id = %d", tblID, isIndex, histID) rows, _, err := h.restrictedExec.ExecRestrictedSQL(nil, selSQL) @@ -251,8 +238,6 @@ func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID in HavePhysicalID: true, Columns: make(map[int64]*Column, len(tableInfo.Columns)), Indices: make(map[int64]*Index, len(tableInfo.Indices)), - colName2Idx: make(map[string]int64), - colName2ID: make(map[string]int64), } table = &Table{ HistColl: histColl, @@ -282,7 +267,6 @@ func (h *Handle) tableStatsFromStorage(tableInfo *model.TableInfo, physicalID in } } } - table.buildColNameMapper() return table, nil } @@ -299,6 +283,24 @@ func (t *Table) String() string { return strings.Join(strs, "\n") } +func (t *Table) indexStartWithColumnForDebugLog(colName string) *Index { + for _, index := range t.Indices { + if index.Info.Columns[0].Name.L == colName { + return index + } + } + return nil +} + +func (t *Table) columnByNameForDebugLog(colName string) *Column { + for _, c := range t.Columns { + if c.Info.Name.L == colName { + return c + } + } + return nil +} + type tableColumnID struct { tableID int64 columnID int64 @@ -352,9 +354,9 @@ func (coll *HistColl) ColumnIsInvalid(sc *stmtctx.StatementContext, colID int64) if !ok || coll.Pseudo && col.NotAccurate() { return true } - if col.NDV > 0 && col.Len() == 0 { + if col.NDV > 0 && col.Len() == 0 && coll.HavePhysicalID { sc.SetHistogramsNotLoad() - histogramNeededColumns.insert(tableColumnID{tableID: coll.PhysicalID, columnID: colID}) + histogramNeededColumns.insert(tableColumnID{tableID: coll.PhysicalID, columnID: col.Info.ID}) } return col.totalRowCount() == 0 || (col.NDV > 0 && col.Len() == 0) } @@ -439,7 +441,7 @@ func (coll *HistColl) GetRowCountByIndexRanges(sc *stmtctx.StatementContext, idx var result float64 var err error if idx.CMSketch != nil && idx.statsVer == version1 { - result, err = coll.getIndexRowCount(sc, idx, indexRanges, coll.ModifyCount) + result, err = coll.getIndexRowCount(sc, idxID, indexRanges) } else { result, err = idx.getRowCount(sc, indexRanges, coll.ModifyCount) } @@ -468,13 +470,69 @@ func getOrdinalOfRangeCond(sc *stmtctx.StatementContext, ran *ranger.Range) int return len(ran.LowVal) } -func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idx *Index, indexRanges []*ranger.Range, modifyCount int64) (float64, error) { +// GenerateHistCollFromColumnInfo generates a new HistColl whose ColID2IdxID and IdxID2ColIDs is built from the given parameter. +func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, columns []*expression.Column) HistColl { + newColHistMap := make(map[int64]*Column) + colInfoID2UniqueID := make(map[int64]int64) + colNames2UniqueID := make(map[string]int64) + for _, col := range columns { + colInfoID2UniqueID[col.ID] = col.UniqueID + } + for _, colInfo := range infos { + uniqueID, ok := colInfoID2UniqueID[colInfo.ID] + if ok { + colNames2UniqueID[colInfo.Name.L] = uniqueID + } + } + for id, colHist := range coll.Columns { + uniqueID, ok := colInfoID2UniqueID[id] + // Collect the statistics by the given columns. + if ok { + newColHistMap[uniqueID] = colHist + } + } + newIdxHistMap := make(map[int64]*Index) + idx2Columns := make(map[int64][]int64) + colID2IdxID := make(map[int64]int64) + for _, idxHist := range coll.Indices { + ids := make([]int64, 0, len(idxHist.Info.Columns)) + for _, idxCol := range idxHist.Info.Columns { + uniqueID, ok := colNames2UniqueID[idxCol.Name.L] + if !ok { + break + } + ids = append(ids, uniqueID) + } + // If the length of the id list is 0, this index won't be used in this query. + if len(ids) == 0 { + continue + } + colID2IdxID[ids[0]] = idxHist.ID + newIdxHistMap[idxHist.ID] = idxHist + idx2Columns[idxHist.ID] = ids + } + newColl := HistColl{ + PhysicalID: coll.PhysicalID, + HavePhysicalID: coll.HavePhysicalID, + Pseudo: coll.Pseudo, + Count: coll.Count, + ModifyCount: coll.ModifyCount, + Columns: newColHistMap, + Indices: newIdxHistMap, + ColID2IdxID: colID2IdxID, + Idx2ColumnIDs: idx2Columns, + } + return newColl +} + +func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idxID int64, indexRanges []*ranger.Range) (float64, error) { + idx := coll.Indices[idxID] totalCount := float64(0) for _, ran := range indexRanges { rangePosition := getOrdinalOfRangeCond(sc, ran) // first one is range, just use the previous way to estimate if rangePosition == 0 { - count, err := idx.getRowCount(sc, []*ranger.Range{ran}, modifyCount) + count, err := idx.getRowCount(sc, []*ranger.Range{ran}, coll.ModifyCount) if err != nil { return 0, errors.Trace(err) } @@ -496,7 +554,7 @@ func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idx *Index, selectivity = 1.0 / float64(idx.NDV) } else { // for range queries - selectivity = float64(modifyCount) / outOfRangeBetweenRate / idx.totalRowCount() + selectivity = float64(coll.ModifyCount) / outOfRangeBetweenRate / idx.totalRowCount() } } else { selectivity = float64(idx.CMSketch.QueryBytes(bytes)) / float64(idx.totalRowCount()) @@ -511,12 +569,18 @@ func (coll *HistColl) getIndexRowCount(sc *stmtctx.StatementContext, idx *Index, } var count float64 var err error - colName := idx.Info.Columns[rangePosition].Name.L + colIDs := coll.Idx2ColumnIDs[idxID] + var colID int64 + if rangePosition >= len(colIDs) { + colID = -1 + } else { + colID = colIDs[rangePosition] + } // prefer index stats over column stats - if idx, ok := coll.colName2Idx[colName]; ok { + if idx, ok := coll.ColID2IdxID[colID]; ok { count, err = coll.GetRowCountByIndexRanges(sc, idx, []*ranger.Range{&rang}) } else { - count, err = coll.GetRowCountByColumnRanges(sc, coll.colName2ID[colName], []*ranger.Range{&rang}) + count, err = coll.GetRowCountByColumnRanges(sc, colID, []*ranger.Range{&rang}) } if err != nil { return 0, errors.Trace(err) diff --git a/statistics/update.go b/statistics/update.go index 4dfc0634decbc..87bbb0db22128 100644 --- a/statistics/update.go +++ b/statistics/update.go @@ -192,7 +192,7 @@ func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Hand } else { rate = math.Abs(expected-float64(q.actual)) / float64(q.actual) } - if rate >= MinLogErrorRate && (q.actual >= MinLogScanCount || q.expected >= MinLogScanCount) { + if rate >= MinLogErrorRate && (q.actual >= MinLogScanCount || q.expected >= MinLogScanCount) && log.GetLevel() == log.DebugLevel { q.logDetailedInfo(h) } metrics.StatsInaccuracyRate.Observe(rate) From 341dc1009cc4e67dfe569d1307f495181e948f35 Mon Sep 17 00:00:00 2001 From: Zhexuan Yang Date: Fri, 31 Aug 2018 05:25:16 +0800 Subject: [PATCH 34/41] *: tidb tracing prototype (#7016) --- executor/builder.go | 12 +++ executor/distsql.go | 19 ----- executor/table_reader.go | 11 +-- executor/trace.go | 131 +++++++++++++++++++++++++++++ executor/trace_test.go | 33 ++++++++ plan/planbuilder.go | 22 +++++ plan/trace.go | 12 +++ session/session.go | 26 ------ session/tidb.go | 12 +-- session/txn.go | 6 +- store/tikv/2pc.go | 15 ---- store/tikv/backoff.go | 2 +- store/tikv/coprocessor.go | 6 -- util/tracing/noop_bench_test.go | 53 ++++++++++++ util/tracing/util.go | 66 +++++++++++++++ util/tracing/util_test.go | 142 ++++++++++++++++++++++++++++++++ 16 files changed, 480 insertions(+), 88 deletions(-) create mode 100644 executor/trace.go create mode 100644 executor/trace_test.go create mode 100644 plan/trace.go create mode 100644 util/tracing/noop_bench_test.go create mode 100644 util/tracing/util.go create mode 100644 util/tracing/util_test.go diff --git a/executor/builder.go b/executor/builder.go index 1992791ba99c8..aed3694fdde3a 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -89,6 +89,8 @@ func (b *executorBuilder) build(p plan.Plan) Executor { return b.buildDelete(v) case *plan.Execute: return b.buildExecute(v) + case *plan.Trace: + return b.buildTrace(v) case *plan.Explain: return b.buildExplain(v) case *plan.PointGetPlan: @@ -619,6 +621,16 @@ func (b *executorBuilder) buildDDL(v *plan.DDL) Executor { return e } +// buildTrace builds a TraceExec for future executing. This method will be called +// at build(). +func (b *executorBuilder) buildTrace(v *plan.Trace) Executor { + return &TraceExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + stmtNode: v.StmtNode, + builder: b, + } +} + // buildExplain builds a explain executor. `e.rows` collects final result to `ExplainExec`. func (b *executorBuilder) buildExplain(v *plan.Explain) Executor { e := &ExplainExec{ diff --git a/executor/distsql.go b/executor/distsql.go index 2cb5ca8e4b5da..b16c04aad1901 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -23,7 +23,6 @@ import ( "unsafe" "github.com/juju/errors" - "github.com/opentracing/opentracing-go" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" @@ -210,18 +209,6 @@ func splitRanges(ranges []*ranger.Range, keepOrder bool) ([]*ranger.Range, []*ra return signedRanges, unsignedRanges } -// startSpanFollowContext is similar to opentracing.StartSpanFromContext, but the span reference use FollowsFrom option. -func startSpanFollowsContext(ctx context.Context, operationName string) (opentracing.Span, context.Context) { - span := opentracing.SpanFromContext(ctx) - if span != nil { - span = opentracing.StartSpan(operationName, opentracing.FollowsFrom(span.Context())) - } else { - span = opentracing.StartSpan(operationName) - } - - return span, opentracing.ContextWithSpan(ctx, span) -} - // rebuildIndexRanges will be called if there's correlated column in access conditions. We will rebuild the range // by substitute correlated column with the constant. func rebuildIndexRanges(ctx sessionctx.Context, is *plan.PhysicalIndexScan, idxCols []*expression.Column, colLens []int) (ranges []*ranger.Range, err error) { @@ -298,9 +285,6 @@ func (e *IndexReaderExecutor) Open(ctx context.Context) error { } func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) error { - span, ctx := startSpanFollowsContext(ctx, "executor.IndexReader.Open") - defer span.Finish() - var err error if e.corColInFilter { e.dagPB.Executors, _, err = constructDistExec(e.ctx, e.plans) @@ -403,9 +387,6 @@ func (e *IndexLookUpExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) e.memTracker = memory.NewTracker(e.id, e.ctx.GetSessionVars().MemQuotaIndexLookupReader) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) - span, ctx := startSpanFollowsContext(ctx, "executor.IndexLookUp.Open") - defer span.Finish() - e.finished = make(chan struct{}) e.resultCh = make(chan *lookupTableTask, atomic.LoadInt32(&LookupTableTaskChannelSize)) diff --git a/executor/table_reader.go b/executor/table_reader.go index a2e7abe88b446..b3aa6a1c3f0cf 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -57,9 +57,6 @@ type TableReaderExecutor struct { // Open initialzes necessary variables for using this executor. func (e *TableReaderExecutor) Open(ctx context.Context) error { - span, ctx := startSpanFollowsContext(ctx, "executor.TableReader.Open") - defer span.Finish() - var err error if e.corColInFilter { e.dagPB.Executors, _, err = constructDistExec(e.ctx, e.plans) @@ -101,11 +98,11 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { // Next fills data into the chunk passed by its caller. // The task was actually done by tableReaderHandler. func (e *TableReaderExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { - err := e.resultHandler.nextChunk(ctx, chk) - if err != nil { + if err := e.resultHandler.nextChunk(ctx, chk); err != nil { e.feedback.Invalidate() + return err } - return errors.Trace(err) + return errors.Trace(nil) } // Close implements the Executor Close interface. @@ -115,7 +112,7 @@ func (e *TableReaderExecutor) Close() error { return errors.Trace(err) } -// buildResp first build request and send it to tikv using distsql.Select. It uses SelectResut returned by the callee +// buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResut returned by the callee // to fetch all results. func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) { var builder distsql.RequestBuilder diff --git a/executor/trace.go b/executor/trace.go new file mode 100644 index 0000000000000..23d1cd570e778 --- /dev/null +++ b/executor/trace.go @@ -0,0 +1,131 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor + +import ( + "time" + + "github.com/juju/errors" + "github.com/opentracing/basictracer-go" + opentracing "github.com/opentracing/opentracing-go" + "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/plan" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/tracing" + "golang.org/x/net/context" +) + +// TraceExec represents a root executor of trace query. +type TraceExec struct { + baseExecutor + // CollectedSpans collects all span during execution. Span is appended via + // callback method which passes into tracer implementation. + CollectedSpans []basictracer.RawSpan + // exhausted being true means there is no more result. + exhausted bool + // stmtNode is the real query ast tree and it is used for building real query's plan. + stmtNode ast.StmtNode + // rootTrace represents root span which is father of all other span. + rootTrace opentracing.Span + + builder *executorBuilder +} + +// Next executes real query and collects span later. +func (e *TraceExec) Next(ctx context.Context, chk *chunk.Chunk) error { + chk.Reset() + if e.exhausted { + return nil + } + + // record how much time was spent for optimizeing plan + optimizeSp := e.rootTrace.Tracer().StartSpan("plan_optimize", opentracing.FollowsFrom(e.rootTrace.Context())) + stmtPlan, err := plan.Optimize(e.builder.ctx, e.stmtNode, e.builder.is) + if err != nil { + return err + } + optimizeSp.Finish() + + pp, ok := stmtPlan.(plan.PhysicalPlan) + if !ok { + return errors.New("cannot cast logical plan to physical plan") + } + + // append select executor to trace executor + stmtExec := e.builder.build(pp) + + e.rootTrace = tracing.NewRecordedTrace("trace_exec", func(sp basictracer.RawSpan) { + e.CollectedSpans = append(e.CollectedSpans, sp) + }) + err = stmtExec.Open(ctx) + if err != nil { + return errors.Trace(err) + } + stmtExecChk := stmtExec.newChunk() + + // store span into context + ctx = opentracing.ContextWithSpan(ctx, e.rootTrace) + + for { + if err := stmtExec.Next(ctx, stmtExecChk); err != nil { + return errors.Trace(err) + } + if stmtExecChk.NumRows() == 0 { + break + } + } + + e.rootTrace.LogKV("event", "tracing completed") + e.rootTrace.Finish() + var rootSpan basictracer.RawSpan + + treeSpans := make(map[uint64][]basictracer.RawSpan) + for _, sp := range e.CollectedSpans { + treeSpans[sp.ParentSpanID] = append(treeSpans[sp.ParentSpanID], sp) + // if a span's parentSpanID is 0, then it is root span + // this is by design + if sp.ParentSpanID == 0 { + rootSpan = sp + } + } + + dfsTree(rootSpan, treeSpans, "", false, chk) + e.exhausted = true + return nil +} + +func dfsTree(span basictracer.RawSpan, tree map[uint64][]basictracer.RawSpan, prefix string, isLast bool, chk *chunk.Chunk) { + suffix := "" + spans := tree[span.Context.SpanID] + var newPrefix string + if span.ParentSpanID == 0 { + newPrefix = prefix + } else { + if len(tree[span.ParentSpanID]) > 0 && !isLast { + suffix = "├─" + newPrefix = prefix + "│ " + } else { + suffix = "└─" + newPrefix = prefix + " " + } + } + + chk.AppendString(0, prefix+suffix+span.Operation) + chk.AppendString(1, span.Start.Format(time.StampNano)) + chk.AppendString(2, span.Duration.String()) + + for i, sp := range spans { + dfsTree(sp, tree, newPrefix, i == (len(spans))-1 /*last element of array*/, chk) + } +} diff --git a/executor/trace_test.go b/executor/trace_test.go new file mode 100644 index 0000000000000..9f8d96f5d8537 --- /dev/null +++ b/executor/trace_test.go @@ -0,0 +1,33 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/util/testkit" +) + +type testTraceExec struct{} + +func (s *testTraceExec) SetupSuite(c *C) { +} + +func (s *testSuite) TestTraceExec(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + testSQL := `create table trace (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` + tk.MustExec(testSQL) + // TODO: check result later in another PR. + tk.MustExec("trace select * from trace where id = 0;") +} diff --git a/plan/planbuilder.go b/plan/planbuilder.go index ed0e45b36779a..3b64ffcb3f164 100644 --- a/plan/planbuilder.go +++ b/plan/planbuilder.go @@ -141,6 +141,8 @@ func (b *planBuilder) build(node ast.Node) (Plan, error) { return b.buildExecute(x) case *ast.ExplainStmt: return b.buildExplain(x) + case *ast.TraceStmt: + return b.buildTrace(x) case *ast.InsertStmt: return b.buildInsert(x) case *ast.LoadDataStmt: @@ -1359,6 +1361,26 @@ func (b *planBuilder) buildDDL(node ast.DDLNode) Plan { return p } +// buildTrace builds a trace plan. Inside this method, it first optimize the +// underlying query and then constructs a schema, which will be used to constructs +// rows result. +func (b *planBuilder) buildTrace(trace *ast.TraceStmt) (Plan, error) { + if _, ok := trace.Stmt.(*ast.SelectStmt); !ok { + return nil, errors.New("trace only supports select query") + } + + p := &Trace{StmtNode: trace.Stmt} + + retFields := []string{"operation", "duration", "spanID"} + schema := expression.NewSchema(make([]*expression.Column, 0, len(retFields))...) + schema.Append(buildColumn("", "operation", mysql.TypeString, mysql.MaxBlobWidth)) + + schema.Append(buildColumn("", "startTS", mysql.TypeString, mysql.MaxBlobWidth)) + schema.Append(buildColumn("", "duration", mysql.TypeString, mysql.MaxBlobWidth)) + p.SetSchema(schema) + return p, nil +} + func (b *planBuilder) buildExplain(explain *ast.ExplainStmt) (Plan, error) { if show, ok := explain.Stmt.(*ast.ShowStmt); ok { return b.buildShow(show) diff --git a/plan/trace.go b/plan/trace.go new file mode 100644 index 0000000000000..8b08336d7a509 --- /dev/null +++ b/plan/trace.go @@ -0,0 +1,12 @@ +package plan + +import ( + "github.com/pingcap/tidb/ast" +) + +// Trace represents a trace plan. +type Trace struct { + baseSchemaProducer + + StmtNode ast.StmtNode +} diff --git a/session/session.go b/session/session.go index d76313e581abd..11f5641553d9a 100644 --- a/session/session.go +++ b/session/session.go @@ -29,7 +29,6 @@ import ( "github.com/juju/errors" "github.com/ngaut/pools" - "github.com/opentracing/opentracing-go" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" @@ -375,11 +374,6 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { } func (s *session) CommitTxn(ctx context.Context) error { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("session.CommitTxn", opentracing.ChildOf(span.Context())) - defer span.Finish() - } - err := s.doCommitWithRetry(ctx) label := metrics.LblOK if err != nil { @@ -390,11 +384,6 @@ func (s *session) CommitTxn(ctx context.Context) error { } func (s *session) RollbackTxn(ctx context.Context) error { - if span := opentracing.SpanFromContext(ctx); span != nil { - span = opentracing.StartSpan("session.RollbackTxn", opentracing.ChildOf(span.Context())) - defer span.Finish() - } - var err error if s.txn.Valid() { terror.Log(s.txn.Rollback()) @@ -451,9 +440,6 @@ func (s *session) isRetryableError(err error) bool { } func (s *session) retry(ctx context.Context, maxCnt uint) error { - span, ctx := opentracing.StartSpanFromContext(ctx, "retry") - defer span.Finish() - connID := s.sessionVars.ConnectionID if s.sessionVars.TxnCtx.ForUpdate { return errForUpdateCantRetry.GenByArgs(connID) @@ -545,10 +531,7 @@ func (s *session) sysSessionPool() *pools.ResourcePool { // Unlike normal Exec, it doesn't reset statement status, doesn't commit or rollback the current transaction // and doesn't write binlog. func (s *session) ExecRestrictedSQL(sctx sessionctx.Context, sql string) ([]chunk.Row, []*ast.ResultField, error) { - var span opentracing.Span ctx := context.TODO() - span, ctx = opentracing.StartSpanFromContext(ctx, "session.ExecRestrictedSQL") - defer span.Finish() // Use special session to execute the sql. tmp, err := s.sysSessionPool().Get() @@ -712,10 +695,6 @@ func (s *session) SetGlobalSysVar(name, value string) error { } func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) ([]ast.StmtNode, error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span1 := opentracing.StartSpan("session.ParseSQL", opentracing.ChildOf(span.Context())) - defer span1.Finish() - } s.parser.SetSQLMode(s.sessionVars.SQLMode) return s.parser.Parse(sql, charset, collation) } @@ -770,11 +749,6 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []ast.Rec } func (s *session) execute(ctx context.Context, sql string) (recordSets []ast.RecordSet, err error) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span, ctx = opentracing.StartSpanFromContext(ctx, "session.Execute") - defer span.Finish() - } - s.PrepareTxnCtx(ctx) connID := s.sessionVars.ConnectionID err = s.loadCommonGlobalVariablesIfNeeded() diff --git a/session/tidb.go b/session/tidb.go index df6ce605d2a68..0a50470813ab3 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -24,7 +24,6 @@ import ( "time" "github.com/juju/errors" - "github.com/opentracing/opentracing-go" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" @@ -143,15 +142,10 @@ func Compile(ctx context.Context, sctx sessionctx.Context, stmtNode ast.StmtNode // runStmt executes the ast.Statement and commit or rollback the current transaction. func runStmt(ctx context.Context, sctx sessionctx.Context, s ast.Statement) (ast.RecordSet, error) { - span, ctx1 := opentracing.StartSpanFromContext(ctx, "runStmt") - span.LogKV("sql", s.OriginText()) - defer span.Finish() - var err error var rs ast.RecordSet se := sctx.(*session) rs, err = s.Exec(ctx) - span.SetTag("txn.id", se.sessionVars.TxnCtx.StartTS) // All the history should be added here. se.GetSessionVars().TxnCtx.StatementCount++ if !s.IsReadOnly() { @@ -169,17 +163,17 @@ func runStmt(ctx context.Context, sctx sessionctx.Context, s ast.Statement) (ast if !se.sessionVars.InTxn() { if err != nil { log.Info("RollbackTxn for ddl/autocommit error.") - err1 := se.RollbackTxn(ctx1) + err1 := se.RollbackTxn(ctx) terror.Log(errors.Trace(err1)) } else { - err = se.CommitTxn(ctx1) + err = se.CommitTxn(ctx) } } else { // If the user insert, insert, insert ... but never commit, TiDB would OOM. // So we limit the statement count in a transaction here. history := GetHistory(sctx) if history.Count() > int(config.GetGlobalConfig().Performance.StmtCountLimit) { - err1 := se.RollbackTxn(ctx1) + err1 := se.RollbackTxn(ctx) terror.Log(errors.Trace(err1)) return rs, errors.Errorf("statement count %d exceeds the transaction limitation, autocommit = %t", history.Count(), sctx.GetSessionVars().IsAutocommit()) diff --git a/session/txn.go b/session/txn.go index 85a64cd77836e..575596b09eb89 100644 --- a/session/txn.go +++ b/session/txn.go @@ -15,7 +15,6 @@ package session import ( "github.com/juju/errors" - "github.com/opentracing/opentracing-go" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" @@ -232,12 +231,10 @@ func mergeToDirtyDB(dirtyDB *executor.DirtyDB, op dirtyTableOperation) { type txnFuture struct { future oracle.Future store kv.Storage - span opentracing.Span } func (tf *txnFuture) wait() (kv.Transaction, error) { startTS, err := tf.future.Wait() - tf.span.Finish() if err == nil { return tf.store.BeginWithStartTS(startTS) } @@ -247,10 +244,9 @@ func (tf *txnFuture) wait() (kv.Transaction, error) { } func (s *session) getTxnFuture(ctx context.Context) *txnFuture { - span, ctx := opentracing.StartSpanFromContext(ctx, "session.getTxnFuture") oracleStore := s.store.GetOracle() tsFuture := oracleStore.GetTimestampAsync(ctx) - return &txnFuture{tsFuture, s.store, span} + return &txnFuture{tsFuture, s.store} } // StmtCommit implements the sessionctx.Context interface. diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 71e5cff4cf114..3fa61f1b405aa 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -21,7 +21,6 @@ import ( "time" "github.com/juju/errors" - "github.com/opentracing/opentracing-go" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" @@ -585,20 +584,6 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) error { } }() - span := opentracing.SpanFromContext(ctx) - if span != nil { - span = opentracing.StartSpan("twoPhaseCommit.execute", opentracing.ChildOf(span.Context())) - } else { - // If we lost the trace information, make a new one for 2PC commit. - span = opentracing.StartSpan("twoPhaseCommit.execute") - } - defer span.Finish() - - // I'm not sure is it safe to cancel 2pc commit process at any time, - // So use a new Background() context instead of inherit the ctx, this is by design, - // to avoid the cancel signal from parent context. - ctx = opentracing.ContextWithSpan(context.Background(), span) - binlogChan := c.prewriteBinlog() err := c.prewriteKeys(NewBackoffer(ctx, prewriteMaxBackoff).WithVars(c.txn.vars), c.keys) if binlogChan != nil { diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 80d8b9ee29207..3a55218aef451 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -225,7 +225,7 @@ func (b *Backoffer) Backoff(typ backoffType, err error) error { b.totalSleep += f(b.ctx) b.types = append(b.types, typ) - log.Debugf("%v, retry later(totalSleep %dms, maxSleep %dms)", err, b.totalSleep, b.maxSleep) + log.Debugf("%v, retry later(totalsleep %dms, maxsleep %dms)", err, b.totalSleep, b.maxSleep) b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano))) if b.maxSleep > 0 && b.totalSleep >= b.maxSleep { diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index e5f15c148a283..34153fb9b60f1 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -25,7 +25,6 @@ import ( "github.com/cznic/mathutil" "github.com/juju/errors" - "github.com/opentracing/opentracing-go" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" @@ -417,11 +416,6 @@ const minLogCopTaskTime = 300 * time.Millisecond // run is a worker function that get a copTask from channel, handle it and // send the result back. func (worker *copIteratorWorker) run(ctx context.Context) { - if span := opentracing.SpanFromContext(ctx); span != nil { - span, ctx = opentracing.StartSpanFromContext(ctx, "copIteratorWorker.run") - defer span.Finish() - } - defer worker.wg.Done() for task := range worker.taskCh { respCh := worker.respChan diff --git a/util/tracing/noop_bench_test.go b/util/tracing/noop_bench_test.go new file mode 100644 index 0000000000000..76b940231c195 --- /dev/null +++ b/util/tracing/noop_bench_test.go @@ -0,0 +1,53 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tracing + +import ( + "context" + "fmt" + "testing" +) + +// BenchmarkNoopLogKV benchs the cost of noop's `LogKV`. +func BenchmarkNoopLogKV(b *testing.B) { + sp := noopSpan() + for i := 0; i < b.N; i++ { + sp.LogKV("event", "noop is finished") + } +} + +// BenchmarkNoopLogKVWithF benchs the the cosst of noop's `LogKV` when +// used with `fmt.Sprintf` +func BenchmarkNoopLogKVWithF(b *testing.B) { + sp := noopSpan() + for i := 0; i < b.N; i++ { + sp.LogKV("event", fmt.Sprintf("this is format %s", "noop is finished")) + } +} + +// BenchmarkSpanFromContext benchs the cost of `SpanFromContext`. +func BenchmarkSpanFromContext(b *testing.B) { + ctx := context.TODO() + for i := 0; i < b.N; i++ { + SpanFromContext(ctx) + } +} + +// BenchmarkChildFromContext benchs the cost of `ChildSpanFromContxt`. +func BenchmarkChildFromContext(b *testing.B) { + ctx := context.TODO() + for i := 0; i < b.N; i++ { + ChildSpanFromContxt(ctx, "child") + } +} diff --git a/util/tracing/util.go b/util/tracing/util.go new file mode 100644 index 0000000000000..f22be09d6f67d --- /dev/null +++ b/util/tracing/util.go @@ -0,0 +1,66 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tracing + +import ( + "github.com/opentracing/basictracer-go" + "github.com/opentracing/opentracing-go" + "golang.org/x/net/context" +) + +// TiDBTrace is set as Baggage on traces which are used for tidb tracing. +const TiDBTrace = "tr" + +// A CallbackRecorder immediately invokes itself on received trace spans. +type CallbackRecorder func(sp basictracer.RawSpan) + +// RecordSpan implements basictracer.SpanRecorder. +func (cr CallbackRecorder) RecordSpan(sp basictracer.RawSpan) { + cr(sp) +} + +// NewRecordedTrace returns a Span which records directly via the specified +// callback. +func NewRecordedTrace(opName string, callback func(sp basictracer.RawSpan)) opentracing.Span { + tr := basictracer.New(CallbackRecorder(callback)) + opentracing.SetGlobalTracer(tr) + sp := tr.StartSpan(opName) + sp.SetBaggageItem(TiDBTrace, "1") + return sp +} + +// noopSpan returns a Span which discards all operations. +func noopSpan() opentracing.Span { + return (opentracing.NoopTracer{}).StartSpan("DefaultSpan") +} + +// SpanFromContext returns the span obtained from the context or, if none is found, a new one started through tracer. +func SpanFromContext(ctx context.Context) (sp opentracing.Span) { + if sp = opentracing.SpanFromContext(ctx); sp == nil { + return noopSpan() + } + return sp +} + +// ChildSpanFromContxt return a non-nil span. If span can be got from ctx, then returned span is +// a child of such span. Otherwise, returned span is a noop span. +func ChildSpanFromContxt(ctx context.Context, opName string) (opentracing.Span, context.Context) { + if sp := opentracing.SpanFromContext(ctx); sp != nil { + if _, ok := sp.Tracer().(opentracing.NoopTracer); !ok { + child := opentracing.StartSpan(opName, opentracing.ChildOf(sp.Context())) + return child, opentracing.ContextWithSpan(ctx, child) + } + } + return noopSpan(), ctx +} diff --git a/util/tracing/util_test.go b/util/tracing/util_test.go new file mode 100644 index 0000000000000..4cbac84f1aaa6 --- /dev/null +++ b/util/tracing/util_test.go @@ -0,0 +1,142 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tracing_test + +import ( + "testing" + + . "github.com/pingcap/check" + + basictracer "github.com/opentracing/basictracer-go" + "github.com/opentracing/opentracing-go" + "github.com/pingcap/tidb/util/tracing" + "golang.org/x/net/context" +) + +var _ = Suite(&testTraceSuite{}) + +func TestT(t *testing.T) { + TestingT(t) +} + +type testTraceSuite struct { +} + +func (s *testTraceSuite) TestSpanFromContext(c *C) { + ctx := context.TODO() + noopSp := tracing.SpanFromContext(ctx) + // test noop span + _, ok := noopSp.Tracer().(opentracing.NoopTracer) + c.Assert(ok, IsTrue) + + // test tidb tracing + collectedSpan := make([]basictracer.RawSpan, 1) + sp := tracing.NewRecordedTrace("test", func(sp basictracer.RawSpan) { + collectedSpan = append(collectedSpan, sp) + }) + sp.Finish() + opentracing.ContextWithSpan(ctx, sp) + child := tracing.SpanFromContext(ctx) + child.Finish() + + // verify second span's operation is not nil, this way we can ensure + // callback logic works. + c.Assert(collectedSpan[0].Operation, NotNil) +} + +func (s *testTraceSuite) TestChildSpanFromContext(c *C) { + ctx := context.TODO() + noopSp, _ := tracing.ChildSpanFromContxt(ctx, "") + _, ok := noopSp.Tracer().(opentracing.NoopTracer) + c.Assert(ok, IsTrue) + + // test tidb tracing + collectedSpan := make([]basictracer.RawSpan, 1) + sp := tracing.NewRecordedTrace("test", func(sp basictracer.RawSpan) { + collectedSpan = append(collectedSpan, sp) + }) + sp.Finish() + opentracing.ContextWithSpan(ctx, sp) + child, _ := tracing.ChildSpanFromContxt(ctx, "test_child") + child.Finish() + + // verify second span's operation is not nil, this way we can ensure + // callback logic works. + c.Assert(collectedSpan[1].Operation, NotNil) + +} + +func (s *testTraceSuite) TestFollowFrom(c *C) { + var collectedSpans []basictracer.RawSpan + // first start a root span + sp1 := tracing.NewRecordedTrace("test", func(sp basictracer.RawSpan) { + collectedSpans = append(collectedSpans, sp) + }) + sp2 := sp1.Tracer().StartSpan("follow_from", opentracing.FollowsFrom(sp1.Context())) + sp1.Finish() + sp2.Finish() + c.Assert(collectedSpans[1].Operation, Equals, "follow_from") + c.Assert(collectedSpans[1].ParentSpanID, Not(Equals), uint64(0)) + // only root span has 0 parent id + c.Assert(collectedSpans[0].ParentSpanID, Equals, uint64(0)) +} + +func (s *testTraceSuite) TestCreateSapnBeforeSetupGlobalTracer(c *C) { + var collectedSpans []basictracer.RawSpan + sp := opentracing.StartSpan("before") + sp.Finish() + + // first start a root span + sp1 := tracing.NewRecordedTrace("test", func(sp basictracer.RawSpan) { + collectedSpans = append(collectedSpans, sp) + }) + sp1.Finish() + + // sp is a span started before we setup global tracer; hence such span will be + // droped. + c.Assert(len(collectedSpans), Equals, 1) +} + +func (s *testTraceSuite) TestTreeRelationship(c *C) { + var collectedSpans []basictracer.RawSpan + ctx := context.TODO() + // first start a root span + sp1 := tracing.NewRecordedTrace("test", func(sp basictracer.RawSpan) { + collectedSpans = append(collectedSpans, sp) + }) + + // then store such root span into context + ctx = opentracing.ContextWithSpan(ctx, sp1) + + // create children span from context + sp2, ctx := tracing.ChildSpanFromContxt(ctx, "parent") + sp3, _ := tracing.ChildSpanFromContxt(ctx, "child") + + // notify span that we are about to reach end of journey. + sp1.Finish() + sp2.Finish() + sp3.Finish() + + // ensure length of collectedSpans is greather than 0 + c.Assert(len(collectedSpans), Greater, 0) + if len(collectedSpans) > 0 { + c.Assert(collectedSpans[0].Operation, Equals, "test") + c.Assert(collectedSpans[1].Operation, Equals, "parent") + c.Assert(collectedSpans[2].Operation, Equals, "child") + // check tree relationship + // sp1 is parent of sp2. And sp2 is parent of sp3. + c.Assert(collectedSpans[1].ParentSpanID, Equals, collectedSpans[0].Context.SpanID) + c.Assert(collectedSpans[2].ParentSpanID, Equals, collectedSpans[1].Context.SpanID) + } +} From 8d1acc2407de8321853784fb5b03c7d1314d1a91 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 31 Aug 2018 10:43:02 +0800 Subject: [PATCH 35/41] Fix bit default value bug (#7249) --- ddl/db_test.go | 9 +++++++++ ddl/ddl_api.go | 28 +++++++++++++++++++--------- executor/show.go | 5 +++-- model/model.go | 27 +++++++++++++++++++++++++++ table/column.go | 4 ++-- table/tables/tables.go | 2 +- 6 files changed, 61 insertions(+), 14 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 269954f99f803..fba087f66f092 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1576,6 +1576,15 @@ func (s *testDBSuite) TestCreateTable(c *C) { c.Assert(err, NotNil) } +func (s *testDBSuite) TestBitDefaultValue(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t_bit (c1 bit(10) default 250, c2 int);") + tk.MustExec("insert into t_bit set c2=1;") + tk.MustQuery("select bin(c1),c2 from t_bit").Check(testkit.Rows("11111010 1")) + tk.MustExec("drop table t_bit") +} + func (s *testDBSuite) TestCreateTableWithPartition(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test;") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2ab73110f6a9f..5a1eaf9331768 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -345,7 +345,9 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o if hasDefaultValue, value, err = checkColumnDefaultValue(ctx, col, value); err != nil { return nil, nil, errors.Trace(err) } - col.DefaultValue = value + if err = col.SetDefaultValue(value); err != nil { + return nil, nil, errors.Trace(err) + } removeOnUpdateNowFlag(col) case ast.ColumnOptionOnUpdate: // TODO: Support other time functions. @@ -473,9 +475,9 @@ func setTimestampDefaultValue(c *table.Column, hasDefaultValue bool, setOnUpdate // For timestamp Col, if is not set default value or not set null, use current timestamp. if mysql.HasTimestampFlag(c.Flag) && mysql.HasNotNullFlag(c.Flag) { if setOnUpdateNow { - c.DefaultValue = types.ZeroDatetimeStr + c.SetDefaultValue(types.ZeroDatetimeStr) } else { - c.DefaultValue = strings.ToUpper(ast.CurrentTimestamp) + c.SetDefaultValue(strings.ToUpper(ast.CurrentTimestamp)) } } } @@ -500,7 +502,7 @@ func checkDefaultValue(ctx sessionctx.Context, c *table.Column, hasDefaultValue return nil } - if c.DefaultValue != nil { + if c.GetDefaultValue() != nil { if _, err := table.GetColDefaultValue(ctx, c.ToInfo()); err != nil { return types.ErrInvalidDefault.GenByArgs(c.Name) } @@ -522,7 +524,7 @@ func checkDefaultValue(ctx sessionctx.Context, c *table.Column, hasDefaultValue // checkPriKeyConstraint check all parts of a PRIMARY KEY must be NOT NULL func checkPriKeyConstraint(col *table.Column, hasDefaultValue, hasNullFlag bool, outPriKeyConstraint *ast.Constraint) error { // Primary key should not be null. - if mysql.HasPriKeyFlag(col.Flag) && hasDefaultValue && col.DefaultValue == nil { + if mysql.HasPriKeyFlag(col.Flag) && hasDefaultValue && col.GetDefaultValue() == nil { return types.ErrInvalidDefault.GenByArgs(col.Name) } // Set primary key flag for outer primary key constraint. @@ -1247,7 +1249,7 @@ func (d *ddl) AddColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTab if err != nil { return errors.Trace(err) } - col.OriginDefaultValue = col.DefaultValue + col.OriginDefaultValue = col.GetDefaultValue() if col.OriginDefaultValue == nil && mysql.HasNotNullFlag(col.Flag) { zeroVal := table.GetZeroValue(col.ToInfo()) col.OriginDefaultValue, err = zeroVal.ToString() @@ -1458,7 +1460,10 @@ func setDefaultValue(ctx sessionctx.Context, col *table.Column, option *ast.Colu if err != nil { return ErrColumnBadNull.Gen("invalid default value - %s", err) } - col.DefaultValue = value + err = col.SetDefaultValue(value) + if err != nil { + return errors.Trace(err) + } return errors.Trace(checkDefaultValue(ctx, col, true)) } @@ -1487,7 +1492,9 @@ func setDefaultAndComment(ctx sessionctx.Context, col *table.Column, options []* if hasDefaultValue, value, err = checkColumnDefaultValue(ctx, col, value); err != nil { return errors.Trace(err) } - col.DefaultValue = value + if err = col.SetDefaultValue(value); err != nil { + return errors.Trace(err) + } case ast.ColumnOptionComment: err := setColumnComment(ctx, col, opt) if err != nil { @@ -1709,7 +1716,10 @@ func (d *ddl) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Alt // Clean the NoDefaultValueFlag value. col.Flag &= ^mysql.NoDefaultValueFlag if len(specNewColumn.Options) == 0 { - col.DefaultValue = nil + err = col.SetDefaultValue(nil) + if err != nil { + return errors.Trace(err) + } setNoDefaultValueFlag(col, false) } else { err = setDefaultValue(ctx, col, specNewColumn.Options[0]) diff --git a/executor/show.go b/executor/show.go index ac7de894ba0ab..1b2c21f81549f 100644 --- a/executor/show.go +++ b/executor/show.go @@ -503,7 +503,8 @@ func (e *ShowExec) fetchShowCreateTable() error { buf.WriteString(" NOT NULL") } if !mysql.HasNoDefaultValueFlag(col.Flag) { - switch col.DefaultValue { + defaultValue := col.GetDefaultValue() + switch defaultValue { case nil: if !mysql.HasNotNullFlag(col.Flag) { if col.Tp == mysql.TypeTimestamp { @@ -514,7 +515,7 @@ func (e *ShowExec) fetchShowCreateTable() error { case "CURRENT_TIMESTAMP": buf.WriteString(" DEFAULT CURRENT_TIMESTAMP") default: - defaultValStr := fmt.Sprintf("%v", col.DefaultValue) + defaultValStr := fmt.Sprintf("%v", defaultValue) if col.Tp == mysql.TypeBit { defaultValBinaryLiteral := types.BinaryLiteral(defaultValStr) buf.WriteString(fmt.Sprintf(" DEFAULT %s", defaultValBinaryLiteral.ToBitLiteralString(true))) diff --git a/model/model.go b/model/model.go index bc03eac4d817b..2fe3c7b6ebb00 100644 --- a/model/model.go +++ b/model/model.go @@ -22,6 +22,7 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tipb/go-tipb" ) @@ -69,6 +70,7 @@ type ColumnInfo struct { Offset int `json:"offset"` OriginDefaultValue interface{} `json:"origin_default"` DefaultValue interface{} `json:"default"` + DefaultValueBit []byte `json:"default_bit"` GeneratedExprString string `json:"generated_expr_string"` GeneratedStored bool `json:"generated_stored"` Dependences map[string]struct{} `json:"dependences"` @@ -88,6 +90,31 @@ func (c *ColumnInfo) IsGenerated() bool { return len(c.GeneratedExprString) != 0 } +// SetDefaultValue sets the default value. +func (c *ColumnInfo) SetDefaultValue(value interface{}) error { + c.DefaultValue = value + if c.Tp == mysql.TypeBit { + // For mysql.TypeBit type, the default value storage format must be a string. + // Other value such as int must convert to string format first. + if v, ok := value.(string); ok { + c.DefaultValueBit = []byte(v) + return nil + } + return types.ErrInvalidDefault.GenByArgs(c.Name) + } + return nil +} + +// GetDefaultValue gets the default value of the column. +// Default value use to stored in DefaultValue field, but now, +// bit type default value will store in DefaultValueBit for fix bit default value decode/encode bug. +func (c *ColumnInfo) GetDefaultValue() interface{} { + if c.Tp == mysql.TypeBit && c.DefaultValueBit != nil { + return hack.String(c.DefaultValueBit) + } + return c.DefaultValue +} + // FindColumnInfo finds ColumnInfo in cols by name. func FindColumnInfo(cols []*ColumnInfo, name string) *ColumnInfo { name = strings.ToLower(name) diff --git a/table/column.go b/table/column.go index c111c0fa9e589..e3766a688816e 100644 --- a/table/column.go +++ b/table/column.go @@ -227,7 +227,7 @@ func NewColDesc(col *Column) *ColDesc { } var defaultValue interface{} if !mysql.HasNoDefaultValueFlag(col.Flag) { - defaultValue = col.DefaultValue + defaultValue = col.GetDefaultValue() } extra := "" @@ -310,7 +310,7 @@ func GetColOriginDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo) (ty // GetColDefaultValue gets default value of the column. func GetColDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo) (types.Datum, error) { - return getColDefaultValue(ctx, col, col.DefaultValue) + return getColDefaultValue(ctx, col, col.GetDefaultValue()) } func getColDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo, defaultVal interface{}) (types.Datum, error) { diff --git a/table/tables/tables.go b/table/tables/tables.go index 4cfc7a85303a1..23a58cae54c86 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -936,7 +936,7 @@ func CanSkip(info *model.TableInfo, col *table.Column, value types.Datum) bool { if col.IsPKHandleColumn(info) { return true } - if col.DefaultValue == nil && value.IsNull() { + if col.GetDefaultValue() == nil && value.IsNull() { return true } if col.IsGenerated() && !col.GeneratedStored { From 2b776ac495067b6339552c3300cc41c0923cae76 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 31 Aug 2018 11:23:14 +0800 Subject: [PATCH 36/41] *: remove goroutine pool (#7564) goroutine pool was introduced to handle stack copy cost, Go1.11 has many optimizations for stack copy, after upgrading to Go1.1, goroutine pool is not necessary any more. --- distsql/select_result.go | 9 +-- store/tikv/2pc.go | 15 ++-- store/tikv/coprocessor.go | 9 +-- store/tikv/rawkv.go | 10 +-- store/tikv/snapshot.go | 7 +- util/goroutine_pool/fake.go | 30 -------- util/goroutine_pool/gp.go | 124 ------------------------------ util/goroutine_pool/gp_test.go | 135 --------------------------------- 8 files changed, 15 insertions(+), 324 deletions(-) delete mode 100644 util/goroutine_pool/fake.go delete mode 100644 util/goroutine_pool/gp.go delete mode 100644 util/goroutine_pool/gp_test.go diff --git a/distsql/select_result.go b/distsql/select_result.go index 215b114e7b743..ecdf76a79a3ad 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/goroutine_pool" "github.com/pingcap/tipb/go-tipb" "golang.org/x/net/context" ) @@ -35,10 +34,6 @@ var ( _ SelectResult = (*streamResult)(nil) ) -var ( - selectResultGP = gp.New(time.Minute * 2) -) - // SelectResult is an iterator of coprocessor partial results. type SelectResult interface { // Fetch fetches partial results from client. @@ -75,9 +70,7 @@ type selectResult struct { } func (r *selectResult) Fetch(ctx context.Context) { - selectResultGP.Go(func() { - r.fetch(ctx) - }) + go r.fetch(ctx) } func (r *selectResult) fetch(ctx context.Context) { diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 3fa61f1b405aa..086acb577b0fa 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/terror" - "github.com/pingcap/tidb/util/goroutine_pool" binlog "github.com/pingcap/tipb/go-binlog" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -42,8 +41,6 @@ const ( actionCleanup twoPhaseCommitAction = 3 ) -var twoPhaseCommitGP = gp.New(3 * time.Minute) - func (ca twoPhaseCommitAction) String() string { switch ca { case actionPrewrite: @@ -225,13 +222,13 @@ func (c *twoPhaseCommitter) doActionOnKeys(bo *Backoffer, action twoPhaseCommitA } if action == actionCommit { // Commit secondary batches in background goroutine to reduce latency. - twoPhaseCommitGP.Go(func() { + go func() { e := c.doActionOnBatches(bo, action, batches) if e != nil { log.Debugf("con:%d 2PC async doActionOnBatches %s err: %v", c.connID, action, e) metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("commit").Inc() } - }) + }() } else { err = c.doActionOnBatches(bo, action, batches) } @@ -272,7 +269,7 @@ func (c *twoPhaseCommitter) doActionOnBatches(bo *Backoffer, action twoPhaseComm for _, batch1 := range batches { batch := batch1 - twoPhaseCommitGP.Go(func() { + go func() { if action == actionCommit { // Because the secondary batches of the commit actions are implemented to be // committed asynchronously in background goroutines, we should not @@ -288,7 +285,7 @@ func (c *twoPhaseCommitter) doActionOnBatches(bo *Backoffer, action twoPhaseComm defer singleBatchCancel() ch <- singleBatchActionFunc(singleBatchBackoffer, batch) } - }) + }() } var err error for i := 0; i < len(batches); i++ { @@ -571,7 +568,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) error { c.mu.RUnlock() if !committed && !undetermined { c.cleanWg.Add(1) - twoPhaseCommitGP.Go(func() { + go func() { err := c.cleanupKeys(NewBackoffer(context.Background(), cleanupMaxBackoff).WithVars(c.txn.vars), c.keys) if err != nil { metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("rollback").Inc() @@ -580,7 +577,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) error { log.Infof("con:%d 2PC clean up done, tid: %d", c.connID, c.startTS) } c.cleanWg.Done() - }) + }() } }() diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 34153fb9b60f1..1e9a6b8acf666 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -31,14 +31,11 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/util/execdetails" - "github.com/pingcap/tidb/util/goroutine_pool" "github.com/pingcap/tipb/go-tipb" log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) -var copIteratorGP = gp.New(time.Minute) - // CopClient is coprocessor client. type CopClient struct { store *tikvStore @@ -452,9 +449,7 @@ func (it *copIterator) open(ctx context.Context) { finishCh: it.finishCh, vars: it.vars, } - copIteratorGP.Go(func() { - worker.run(ctx) - }) + go worker.run(ctx) } taskSender := &copIteratorTaskSender{ taskCh: taskCh, @@ -463,7 +458,7 @@ func (it *copIterator) open(ctx context.Context) { finishCh: it.finishCh, } taskSender.respChan = it.respChan - copIteratorGP.Go(taskSender.run) + go taskSender.run() } func (sender *copIteratorTaskSender) run() { diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go index 41b72a1986859..cdaf29cc35a15 100644 --- a/store/tikv/rawkv.go +++ b/store/tikv/rawkv.go @@ -23,12 +23,10 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/util/goroutine_pool" "golang.org/x/net/context" ) var ( - rawKVClientGP = gp.New(3 * time.Minute) // MaxRawKVScanLimit is the maximum scan limit for rawkv Scan. MaxRawKVScanLimit = 10240 // ErrMaxScanLimitExceeded is returned when the limit for rawkv Scan is to large. @@ -349,11 +347,11 @@ func (c *RawKVClient) sendBatchReq(bo *Backoffer, keys [][]byte, cmdType tikvrpc ches := make(chan singleBatchResp, len(batches)) for _, batch := range batches { batch1 := batch - rawKVClientGP.Go(func() { + go func() { singleBatchBackoffer, singleBatchCancel := bo.Fork() defer singleBatchCancel() ches <- c.doBatchReq(singleBatchBackoffer, batch1, cmdType) - }) + }() } var firstError error @@ -507,11 +505,11 @@ func (c *RawKVClient) sendBatchPut(bo *Backoffer, keys, values [][]byte) error { ch := make(chan error, len(batches)) for _, batch := range batches { batch1 := batch - rawKVClientGP.Go(func() { + go func() { singleBatchBackoffer, singleBatchCancel := bo.Fork() defer singleBatchCancel() ch <- c.doBatchPut(singleBatchBackoffer, batch1) - }) + }() } for i := 0; i < len(batches); i++ { diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index c51983495cde3..e4f55d1600a15 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/util/goroutine_pool" log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -51,8 +50,6 @@ type tikvSnapshot struct { vars *kv.Variables } -var snapshotGP = gp.New(time.Minute) - // newTiKVSnapshot creates a snapshot of an TiKV store. func newTiKVSnapshot(store *tikvStore, ver kv.Version) *tikvSnapshot { return &tikvSnapshot{ @@ -123,11 +120,11 @@ func (s *tikvSnapshot) batchGetKeysByRegions(bo *Backoffer, keys [][]byte, colle ch := make(chan error) for _, batch1 := range batches { batch := batch1 - snapshotGP.Go(func() { + go func() { backoffer, cancel := bo.Fork() defer cancel() ch <- s.batchGetSingleRegion(backoffer, batch, collectF) - }) + }() } for i := 0; i < len(batches); i++ { if e := <-ch; e != nil { diff --git a/util/goroutine_pool/fake.go b/util/goroutine_pool/fake.go deleted file mode 100644 index 2465572c142a8..0000000000000 --- a/util/goroutine_pool/fake.go +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2017 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. -// +build leak - -package gp - -import "time" - -type Pool struct{} - -// New returns a new *Pool object. -// When compile with leak flag, goroutine will not be reusing. -func New(idleTimeout time.Duration) *Pool { - return &Pool{} -} - -// Go run f() in a new goroutine. -func (pool *Pool) Go(f func()) { - go f() -} diff --git a/util/goroutine_pool/gp.go b/util/goroutine_pool/gp.go deleted file mode 100644 index 9d29bb5cc7bc3..0000000000000 --- a/util/goroutine_pool/gp.go +++ /dev/null @@ -1,124 +0,0 @@ -// Copyright 2017 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. -// +build !leak - -package gp - -import ( - "sync" - "sync/atomic" - "time" -) - -// Pool is a struct to represent goroutine pool. -type Pool struct { - stack []*goroutine - idleTimeout time.Duration - sync.Mutex -} - -// goroutine is actually a background goroutine, with a channel binded for communication. -type goroutine struct { - ch chan func() - status int32 -} - -const ( - statusIdle int32 = 0 - statusInUse int32 = 1 - statusDead int32 = 2 -) - -// New returns a new *Pool object. -func New(idleTimeout time.Duration) *Pool { - pool := &Pool{ - idleTimeout: idleTimeout, - stack: make([]*goroutine, 0, 64), - } - return pool -} - -// Go works like go func(), but goroutines are pooled for reusing. -// This strategy can avoid runtime.morestack, because pooled goroutine is already enlarged. -func (pool *Pool) Go(f func()) { - for { - g := pool.get() - if atomic.CompareAndSwapInt32(&g.status, statusIdle, statusInUse) { - g.ch <- f - return - } - // Status already changed from statusIdle => statusDead, drop it, find next one. - } -} - -func (pool *Pool) get() *goroutine { - pool.Lock() - if len(pool.stack) == 0 { - pool.Unlock() - return pool.alloc() - } - - ret := pool.stack[len(pool.stack)-1] - pool.stack = pool.stack[:len(pool.stack)-1] - pool.Unlock() - return ret -} - -func (pool *Pool) alloc() *goroutine { - g := &goroutine{ - ch: make(chan func()), - } - go g.workLoop(pool) - return g -} - -func (g *goroutine) put(pool *Pool) { - g.status = statusIdle - pool.Lock() - - // Recycle dead goroutine space. - i := 0 - for ; i < len(pool.stack) && atomic.LoadInt32(&pool.stack[i].status) == statusDead; i++ { - } - pool.stack = append(pool.stack[i:], g) - pool.Unlock() -} - -func (g *goroutine) workLoop(pool *Pool) { - timer := time.NewTimer(pool.idleTimeout) - for { - select { - case <-timer.C: - // Check to avoid a corner case that the goroutine is take out from pool, - // and get this signal at the same time. - succ := atomic.CompareAndSwapInt32(&g.status, statusIdle, statusDead) - if succ { - return - } - case work := <-g.ch: - work() - // Put g back to the pool. - // This is the normal usage for a resource pool: - // - // obj := pool.get() - // use(obj) - // pool.put(obj) - // - // But when goroutine is used as a resource, we can't pool.put() immediately, - // because the resource(goroutine) maybe still in use. - // So, put back resource is done here, when the goroutine finish its work. - g.put(pool) - } - timer.Reset(pool.idleTimeout) - } -} diff --git a/util/goroutine_pool/gp_test.go b/util/goroutine_pool/gp_test.go deleted file mode 100644 index 46cd5f438fa98..0000000000000 --- a/util/goroutine_pool/gp_test.go +++ /dev/null @@ -1,135 +0,0 @@ -// Copyright 2017 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. -// +build !leak - -package gp - -import ( - "sync" - "testing" - "time" -) - -func TestBasicAPI(t *testing.T) { - gp := New(time.Second) - var wg sync.WaitGroup - wg.Add(1) - // cover alloc() - gp.Go(func() { wg.Done() }) - // cover put() - wg.Wait() - // cover get() - gp.Go(func() {}) -} - -func TestGC(t *testing.T) { - gp := New(200 * time.Millisecond) - var wg sync.WaitGroup - wg.Add(100) - for i := 0; i < 100; i++ { - idx := i - gp.Go(func() { - time.Sleep(time.Duration(idx+1) * time.Millisecond) - wg.Done() - }) - } - wg.Wait() - - time.Sleep(300 * time.Millisecond) - gp.Go(func() {}) // To trigger count change. - - gp.Lock() - count := len(gp.stack) - gp.Unlock() - if count > 1 { - t.Error("all goroutines should be recycled", count) - } -} - -func TestRace(t *testing.T) { - gp := New(8 * time.Millisecond) - var wg sync.WaitGroup - begin := make(chan struct{}) - wg.Add(500) - for i := 0; i < 50; i++ { - go func() { - <-begin - for i := 0; i < 10; i++ { - gp.Go(func() { - wg.Done() - }) - time.Sleep(5 * time.Millisecond) - } - }() - } - close(begin) - wg.Wait() -} - -func BenchmarkGoPool(b *testing.B) { - gp := New(20 * time.Second) - for i := 0; i < b.N/2; i++ { - gp.Go(func() {}) - } - - b.ResetTimer() - for i := 0; i < b.N; i++ { - gp.Go(dummy) - } -} - -func BenchmarkGo(b *testing.B) { - b.ResetTimer() - for i := 0; i < b.N; i++ { - go dummy() - } -} - -func dummy() { -} - -func BenchmarkMorestackPool(b *testing.B) { - gp := New(5 * time.Second) - b.ResetTimer() - for i := 0; i < b.N; i++ { - var wg sync.WaitGroup - wg.Add(1) - gp.Go(func() { - morestack(false) - wg.Done() - }) - wg.Wait() - } -} - -func BenchmarkMoreStack(b *testing.B) { - b.ResetTimer() - for i := 0; i < b.N; i++ { - var wg sync.WaitGroup - wg.Add(1) - go func() { - morestack(false) - wg.Done() - }() - wg.Wait() - } -} - -func morestack(f bool) { - var stack [8 * 1024]byte - if f { - for i := 0; i < len(stack); i++ { - stack[i] = 'a' - } - } -} From 65ecdff62212a5695b426fd2b72a068fcbd122b1 Mon Sep 17 00:00:00 2001 From: ciscoxll Date: Fri, 31 Aug 2018 11:46:00 +0800 Subject: [PATCH 37/41] ddl:support add index rollback of partitioned table (#7437) --- ddl/db_test.go | 184 ++++++++++++++++++++++++++++++++------------ ddl/delete_range.go | 19 ++++- ddl/index.go | 2 +- 3 files changed, 150 insertions(+), 55 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index fba087f66f092..e16367f3ab98c 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -403,61 +403,12 @@ func (s *testDBSuite) TestCancelAddIndex(c *C) { var checkErr error var c3IdxInfo *model.IndexInfo hook := &ddl.TestDDLCallback{} - first := true oldReorgWaitTimeout := ddl.ReorgWaitTimeout // let hook.OnJobUpdatedExported has chance to cancel the job. // the hook.OnJobUpdatedExported is called when the job is updated, runReorgJob will wait ddl.ReorgWaitTimeout, then return the ddl.runDDLJob. // After that ddl call d.hook.OnJobUpdated(job), so that we can canceled the job in this test case. ddl.ReorgWaitTimeout = 50 * time.Millisecond - hook.OnJobUpdatedExported = func(job *model.Job) { - addIndexNotFirstReorg := job.Type == model.ActionAddIndex && job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 - // 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. - if first { - first = false - return - } - if checkErr != nil { - return - } - hookCtx := mock.NewContext() - hookCtx.Store = s.store - var err error - err = hookCtx.NewTxn() - if err != nil { - checkErr = errors.Trace(err) - return - } - jobIDs := []int64{job.ID} - errs, err := admin.CancelJobs(hookCtx.Txn(), jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - // It only tests cancel one DDL job. - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - err = hookCtx.Txn().Commit(context.Background()) - if err != nil { - checkErr = errors.Trace(err) - } - } + hook.OnJobUpdatedExported, c3IdxInfo = backgroundExecOnJobUpdatedExported(c, s, hook, checkErr) s.dom.DDL().(ddl.DDLForTest).SetHook(hook) done := make(chan error, 1) go backgroundExec(s.store, "create unique index c3_index on t1 (c3)", done) @@ -3221,6 +3172,139 @@ LOOP: s.tk.MustExec("drop table partition_drop_idx;") } +func (s *testDBSuite) TestPartitionCancelAddIndex(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + s.mustExec(c, "use test_db") + s.mustExec(c, "set @@session.tidb_enable_table_partition=1;") + s.mustExec(c, "drop table if exists t1;") + s.mustExec(c, `create table t1 ( + c1 int, c2 int, c3 int + ) + partition by range( c1 ) ( + partition p0 values less than (1024), + partition p1 values less than (2048), + partition p2 values less than (3072), + partition p3 values less than (4096), + partition p4 values less than (maxvalue) + );`) + base := defaultBatchSize * 2 + count := base + // add some rows + for i := 0; i < count; i++ { + s.mustExec(c, "insert into t1 values (?, ?, ?)", i, i, i) + } + + var checkErr error + var c3IdxInfo *model.IndexInfo + hook := &ddl.TestDDLCallback{} + oldReorgWaitTimeout := ddl.ReorgWaitTimeout + ddl.ReorgWaitTimeout = 10 * time.Millisecond + hook.OnJobUpdatedExported, c3IdxInfo = backgroundExecOnJobUpdatedExported(c, s, hook, checkErr) + s.dom.DDL().(ddl.DDLForTest).SetHook(hook) + done := make(chan error, 1) + go backgroundExec(s.store, "create index c3_index on t1 (c3)", done) + + times := 0 + ticker := time.NewTicker(s.lease / 2) + defer ticker.Stop() +LOOP: + for { + select { + case err := <-done: + c.Assert(checkErr, IsNil) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:12]cancelled DDL job") + break LOOP + case <-ticker.C: + if times >= 10 { + break + } + step := 10 + rand.Seed(time.Now().Unix()) + // delete some rows, and add some data + for i := count; i < count+step; i++ { + n := rand.Intn(count) + s.mustExec(c, "delete from t1 where c1 = ?", n) + s.mustExec(c, "insert into t1 values (?, ?, ?)", i+10, i, i) + } + count += step + times++ + } + } + + t := s.testGetTable(c, "t1") + // Only one partition id test is taken here. + pid := t.Meta().Partition.Definitions[0].ID + for _, tidx := range t.Indices() { + c.Assert(strings.EqualFold(tidx.Meta().Name.L, "c3_index"), IsFalse) + } + + ctx := s.s.(sessionctx.Context) + idx := tables.NewIndex(pid, t.Meta(), c3IdxInfo) + checkDelRangeDone(c, ctx, idx) + + s.mustExec(c, "drop table t1") + ddl.ReorgWaitTimeout = oldReorgWaitTimeout + callback := &ddl.TestDDLCallback{} + s.dom.DDL().(ddl.DDLForTest).SetHook(callback) +} + +func backgroundExecOnJobUpdatedExported(c *C, s *testDBSuite, hook *ddl.TestDDLCallback, checkErr error) (func(*model.Job), *model.IndexInfo) { + first := true + ddl.ReorgWaitTimeout = 10 * time.Millisecond + c3IdxInfo := &model.IndexInfo{} + hook.OnJobUpdatedExported = func(job *model.Job) { + addIndexNotFirstReorg := job.Type == model.ActionAddIndex && job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 + // 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. + if first { + first = false + return + } + if checkErr != nil { + return + } + hookCtx := mock.NewContext() + hookCtx.Store = s.store + var err error + err = hookCtx.NewTxn() + if err != nil { + checkErr = errors.Trace(err) + return + } + jobIDs := []int64{job.ID} + errs, err := admin.CancelJobs(hookCtx.Txn(), jobIDs) + if err != nil { + checkErr = errors.Trace(err) + return + } + // It only tests cancel one DDL job. + if errs[0] != nil { + checkErr = errors.Trace(errs[0]) + return + } + err = hookCtx.Txn().Commit(context.Background()) + if err != nil { + checkErr = errors.Trace(err) + } + } + return hook.OnJobUpdatedExported, c3IdxInfo +} + func (s *testDBSuite) TestPartitionAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 2ad93e5928038..2eaabe3d28070 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -268,12 +268,23 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error case model.ActionAddIndex: tableID := job.TableID var indexID int64 - if err := job.DecodeArgs(&indexID); err != nil { + var partitionIDs []int64 + if err := job.DecodeArgs(&indexID, &partitionIDs); err != nil { return errors.Trace(err) } - startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) - return doInsert(s, job.ID, indexID, startKey, endKey, now) + if len(partitionIDs) > 0 { + for _, pid := range partitionIDs { + startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) + if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { + return errors.Trace(err) + } + } + } else { + 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/index.go b/ddl/index.go index 54a68511c1e54..30b21039123fd 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -357,7 +357,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int func convert2RollbackJob(t *meta.Meta, job *model.Job, tblInfo *model.TableInfo, indexInfo *model.IndexInfo, err error) (int64, error) { job.State = model.JobStateRollingback - job.Args = []interface{}{indexInfo.Name} + job.Args = []interface{}{indexInfo.Name, getPartitionIDs(tblInfo)} // If add index job rollbacks in write reorganization state, its need to delete all keys which has been added. // Its work is the same as drop index job do. // The write reorganization state in add index job that likes write only state in drop index job. From 720e8230e27f501395d536762969578d511853b1 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Fri, 31 Aug 2018 14:24:32 +0800 Subject: [PATCH 38/41] executor: only rebase auto increment ID when needed (#7515) --- executor/write.go | 28 ++++++++++++++-------------- executor/write_test.go | 33 +++++++++++++++++++++++++++++++++ 2 files changed, 47 insertions(+), 14 deletions(-) diff --git a/executor/write.go b/executor/write.go index ae010cf3ae048..8da915e995475 100644 --- a/executor/write.go +++ b/executor/write.go @@ -71,20 +71,6 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu return false, handleChanged, newHandle, errors.Trace(err) } } - // Rebase auto increment id if the field is changed. - if mysql.HasAutoIncrementFlag(col.Flag) { - if newData[i].IsNull() { - return false, handleChanged, newHandle, table.ErrColumnCantNull.GenByArgs(col.Name) - } - val, errTI := newData[i].ToInt64(sc) - if errTI != nil { - return false, handleChanged, newHandle, errors.Trace(errTI) - } - err := t.RebaseAutoID(ctx, val, true) - if err != nil { - return false, handleChanged, newHandle, errors.Trace(err) - } - } cmp, err := newData[i].CompareDatum(sc, &oldData[i]) if err != nil { return false, handleChanged, newHandle, errors.Trace(err) @@ -92,6 +78,20 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu if cmp != 0 { changed = true modified[i] = true + // Rebase auto increment id if the field is changed. + if mysql.HasAutoIncrementFlag(col.Flag) { + if newData[i].IsNull() { + return false, handleChanged, newHandle, table.ErrColumnCantNull.GenByArgs(col.Name) + } + val, errTI := newData[i].ToInt64(sc) + if errTI != nil { + return false, handleChanged, newHandle, errors.Trace(errTI) + } + err := t.RebaseAutoID(ctx, val, true) + if err != nil { + return false, handleChanged, newHandle, errors.Trace(err) + } + } if col.IsPKHandleColumn(t.Meta()) { handleChanged = true newHandle = newData[i].GetInt64() diff --git a/executor/write_test.go b/executor/write_test.go index ff2ef8d552ca8..30fa336e9d296 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -2021,3 +2021,36 @@ func (s *testSuite) TestReplaceLog(c *C) { tk.MustQuery(`admin cleanup index testLog b;`).Check(testkit.Rows("1")) } + +// For issue 7422. +// There is no need to do the rebase when updating a record if the auto-increment ID not changed. +// This could make the auto ID increasing speed slower. +func (s *testSuite) TestRebaseIfNeeded(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`create table t (a int not null primary key auto_increment, b int unique key);`) + tk.MustExec(`insert into t (b) values (1);`) + + s.ctx = mock.NewContext() + s.ctx.Store = s.store + tbl, err := s.domain.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + c.Assert(s.ctx.NewTxn(), IsNil) + // AddRecord directly here will skip to rebase the auto ID in the insert statement, + // which could simulate another TiDB adds a large auto ID. + _, err = tbl.AddRecord(s.ctx, types.MakeDatums(30001, 2), false) + c.Assert(err, IsNil) + c.Assert(s.ctx.Txn().Commit(context.Background()), IsNil) + + tk.MustExec(`update t set b = 3 where a = 30001;`) + tk.MustExec(`insert into t (b) values (4);`) + tk.MustQuery(`select a from t where b = 4;`).Check(testkit.Rows("2")) + + tk.MustExec(`insert into t set b = 3 on duplicate key update a = a;`) + tk.MustExec(`insert into t (b) values (5);`) + tk.MustQuery(`select a from t where b = 5;`).Check(testkit.Rows("4")) + + tk.MustExec(`insert into t set b = 3 on duplicate key update a = a + 1;`) + tk.MustExec(`insert into t (b) values (6);`) + tk.MustQuery(`select a from t where b = 6;`).Check(testkit.Rows("30003")) +} From 40a4a2dfe8a86285502d8d3b7066fa162471f1fe Mon Sep 17 00:00:00 2001 From: Liao Qiang Date: Fri, 31 Aug 2018 14:47:01 +0800 Subject: [PATCH 39/41] fix compatibility bug with convert string to int return wrong result (#7483) --- expression/builtin_cast_test.go | 40 +++++++++++++++++++++++++++++++++ types/convert.go | 12 +++++----- types/convert_test.go | 13 +++++++---- 3 files changed, 56 insertions(+), 9 deletions(-) diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index ca8f63cb353ba..4de94a8be3798 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -127,6 +127,26 @@ func (s *testEvaluatorSuite) TestCast(c *C) { lastWarn = warnings[len(warnings)-1] c.Assert(terror.ErrorEqual(types.ErrTruncatedWrongVal, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) + // cast('125e342.83' as unsigned) + f = BuildCastFunction(ctx, &Constant{Value: types.NewDatum("125e342.83"), RetType: types.NewFieldType(mysql.TypeString)}, tp1) + res, err = f.Eval(chunk.Row{}) + c.Assert(err, IsNil) + c.Assert(res.GetUint64() == 125, IsTrue) + + warnings = sc.GetWarnings() + lastWarn = warnings[len(warnings)-1] + c.Assert(terror.ErrorEqual(types.ErrOverflow, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) + + // cast('1e9223372036854775807' as unsigned) + f = BuildCastFunction(ctx, &Constant{Value: types.NewDatum("1e9223372036854775807"), RetType: types.NewFieldType(mysql.TypeString)}, tp1) + res, err = f.Eval(chunk.Row{}) + c.Assert(err, IsNil) + c.Assert(res.GetUint64() == 1, IsTrue) + + warnings = sc.GetWarnings() + lastWarn = warnings[len(warnings)-1] + c.Assert(terror.ErrorEqual(types.ErrOverflow, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) + // cast('18446744073709551616' as signed); mask := ^mysql.UnsignedFlag tp1.Flag &= uint(mask) @@ -149,6 +169,26 @@ func (s *testEvaluatorSuite) TestCast(c *C) { lastWarn = warnings[len(warnings)-1] c.Assert(terror.ErrorEqual(types.ErrCastAsSignedOverflow, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) + // cast('125e342.83' as signed) + f = BuildCastFunction(ctx, &Constant{Value: types.NewDatum("125e342.83"), RetType: types.NewFieldType(mysql.TypeString)}, tp1) + res, err = f.Eval(chunk.Row{}) + c.Assert(err, IsNil) + c.Assert(res.GetInt64() == 125, IsTrue) + + warnings = sc.GetWarnings() + lastWarn = warnings[len(warnings)-1] + c.Assert(terror.ErrorEqual(types.ErrOverflow, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) + + // cast('1e9223372036854775807' as signed) + f = BuildCastFunction(ctx, &Constant{Value: types.NewDatum("1e9223372036854775807"), RetType: types.NewFieldType(mysql.TypeString)}, tp1) + res, err = f.Eval(chunk.Row{}) + c.Assert(err, IsNil) + c.Assert(res.GetInt64() == 1, IsTrue) + + warnings = sc.GetWarnings() + lastWarn = warnings[len(warnings)-1] + c.Assert(terror.ErrorEqual(types.ErrOverflow, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) + // create table t1(s1 time); // insert into t1 values('11:11:11'); // select cast(s1 as decimal(7, 2)) from t1; diff --git a/types/convert.go b/types/convert.go index 82aac0547b04a..ab727258b76fd 100644 --- a/types/convert.go +++ b/types/convert.go @@ -236,13 +236,13 @@ func getValidIntPrefix(sc *stmtctx.StatementContext, str string) (string, error) if err != nil { return floatPrefix, errors.Trace(err) } - return floatStrToIntStr(floatPrefix) + return floatStrToIntStr(sc, floatPrefix, str) } // floatStrToIntStr converts a valid float string into valid integer string which can be parsed by // strconv.ParseInt, we can't parse float first then convert it to string because precision will // be lost. -func floatStrToIntStr(validFloat string) (string, error) { +func floatStrToIntStr(sc *stmtctx.StatementContext, validFloat string, oriStr string) (string, error) { var dotIdx = -1 var eIdx = -1 for i := 0; i < len(validFloat); i++ { @@ -275,7 +275,8 @@ func floatStrToIntStr(validFloat string) (string, error) { } if exp > 0 && int64(intCnt) > (math.MaxInt64-int64(exp)) { // (exp + incCnt) overflows MaxInt64. - return validFloat, ErrOverflow.GenByArgs("BIGINT", validFloat) + sc.AppendWarning(ErrOverflow.GenByArgs("BIGINT", oriStr)) + return validFloat[:eIdx], nil } intCnt += exp if intCnt <= 0 { @@ -291,8 +292,9 @@ func floatStrToIntStr(validFloat string) (string, error) { // convert scientific notation decimal number extraZeroCount := intCnt - len(digits) if extraZeroCount > 20 { - // Return overflow to avoid allocating too much memory. - return validFloat, ErrOverflow.GenByArgs("BIGINT", validFloat) + // Append overflow warning and return to avoid allocating too much memory. + sc.AppendWarning(ErrOverflow.GenByArgs("BIGINT", oriStr)) + return validFloat[:eIdx], nil } validInt = string(digits) + strings.Repeat("0", extraZeroCount) } diff --git a/types/convert_test.go b/types/convert_test.go index 2a70a0a2d9d9a..08657d627dae8 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -684,10 +684,15 @@ func (s *testTypeConvertSuite) TestGetValidFloat(c *C) { _, err := strconv.ParseFloat(prefix, 64) c.Assert(err, IsNil) } - _, err := floatStrToIntStr("1e9223372036854775807") - c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue, Commentf("err %v", err)) - _, err = floatStrToIntStr("1e21") - c.Assert(terror.ErrorEqual(err, ErrOverflow), IsTrue, Commentf("err %v", err)) + floatStr, err := floatStrToIntStr(sc, "1e9223372036854775807", "1e9223372036854775807") + c.Assert(err, IsNil) + c.Assert(floatStr, Equals, "1") + floatStr, err = floatStrToIntStr(sc, "125e342", "125e342.83") + c.Assert(err, IsNil) + c.Assert(floatStr, Equals, "125") + floatStr, err = floatStrToIntStr(sc, "1e21", "1e21") + c.Assert(err, IsNil) + c.Assert(floatStr, Equals, "1") } // TestConvertTime tests time related conversion. From 1f2841fbd437275a258eecd554df384c59de4b4b Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Fri, 31 Aug 2018 16:45:38 +0800 Subject: [PATCH 40/41] ddl: year type should have an unsigned flag (#7542) --- ddl/db_test.go | 15 ++++++++++----- ddl/ddl_api.go | 6 ++++++ executor/show_test.go | 12 +++++++++++- expression/typeinfer_test.go | 2 +- parser/parser.y | 2 +- parser/parser_test.go | 1 + table/column.go | 3 +++ table/column_test.go | 2 +- 8 files changed, 34 insertions(+), 9 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index e16367f3ab98c..e577ca8dbf5bc 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2117,24 +2117,29 @@ func (s *testDBSuite) TestRebaseAutoID(c *C) { s.testErrorCode(c, "alter table tidb.test2 add column b int auto_increment key, auto_increment=10;", tmysql.ErrUnknown) } -func (s *testDBSuite) TestYearTypeCreateTable(c *C) { +func (s *testDBSuite) TestZeroFillCreateTable(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists abc;") - s.tk.MustExec("create table abc(y year, x int, primary key(y));") + s.tk.MustExec("create table abc(y year, z tinyint(10) zerofill, primary key(y));") is := s.dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("abc")) c.Assert(err, IsNil) - var yearCol *model.ColumnInfo + var yearCol, zCol *model.ColumnInfo for _, col := range tbl.Meta().Columns { if col.Name.String() == "y" { yearCol = col - break + } + if col.Name.String() == "z" { + zCol = col } } c.Assert(yearCol, NotNil) c.Assert(yearCol.Tp, Equals, mysql.TypeYear) - c.Assert(mysql.HasUnsignedFlag(yearCol.Flag), IsFalse) + c.Assert(mysql.HasUnsignedFlag(yearCol.Flag), IsTrue) + + c.Assert(zCol, NotNil) + c.Assert(mysql.HasUnsignedFlag(zCol.Flag), IsTrue) } func (s *testDBSuite) TestCheckColumnDefaultValue(c *C) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 5a1eaf9331768..f3972dac425e1 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -396,6 +396,12 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o col.Flag &= ^mysql.BinaryFlag col.Flag |= mysql.ZerofillFlag } + // If you specify ZEROFILL for a numeric column, MySQL automatically adds the UNSIGNED attribute to the column. + // See https://dev.mysql.com/doc/refman/5.7/en/numeric-type-overview.html for more details. + // But some types like bit and year, won't show its unsigned flag in `show create table`. + if mysql.HasZerofillFlag(col.Flag) { + col.Flag |= mysql.UnsignedFlag + } err := checkPriKeyConstraint(col, hasDefaultValue, hasNullFlag, outPriKeyConstraint) if err != nil { return nil, nil, errors.Trace(err) diff --git a/executor/show_test.go b/executor/show_test.go index 1e16a89096da2..4527afd03bb14 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -362,13 +362,23 @@ func (s *testSuite) TestShow(c *C) { // Test show create table year type tk.MustExec(`drop table if exists t`) - tk.MustExec(`create table t(y year, x int, primary key(y));`) + tk.MustExec(`create table t(y year unsigned signed zerofill zerofill, x int, primary key(y));`) tk.MustQuery(`show create table t`).Check(testutil.RowsWithSep("|", "t CREATE TABLE `t` (\n"+ " `y` year NOT NULL,\n"+ " `x` int(11) DEFAULT NULL,\n"+ " PRIMARY KEY (`y`)\n"+ ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin")) + + // Test show create table with zerofill flag + tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t(id int primary key, val tinyint(10) zerofill);`) + tk.MustQuery(`show create table t`).Check(testutil.RowsWithSep("|", + "t CREATE TABLE `t` (\n"+ + " `id` int(11) NOT NULL,\n"+ + " `val` tinyint(10) UNSIGNED ZEROFILL DEFAULT NULL,\n"+ + " PRIMARY KEY (`id`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin")) } func (s *testSuite) TestShowVisibility(c *C) { diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 2aac3d967a5da..ffb8ed6776790 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -198,7 +198,7 @@ func (s *testInferTypeSuite) createTestCase4Cast() []typeInferTestCase { func (s *testInferTypeSuite) createTestCase4Columns() []typeInferTestCase { return []typeInferTestCase{ {"c_bit ", mysql.TypeBit, charset.CharsetBin, mysql.UnsignedFlag, 10, 0}, - {"c_year ", mysql.TypeYear, charset.CharsetBin, mysql.ZerofillFlag, 4, 0}, + {"c_year ", mysql.TypeYear, charset.CharsetBin, mysql.UnsignedFlag | mysql.ZerofillFlag, 4, 0}, {"c_int_d ", mysql.TypeLong, charset.CharsetBin, 0, 11, 0}, {"c_uint_d ", mysql.TypeLong, charset.CharsetBin, mysql.UnsignedFlag, 10, 0}, {"c_bigint_d ", mysql.TypeLonglong, charset.CharsetBin, 0, 20, 0}, diff --git a/parser/parser.y b/parser/parser.y index 85895fe731f5e..1c7c8ca0e3e45 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -6321,7 +6321,7 @@ DateAndTimeType: } $$ = x } -| "YEAR" OptFieldLen +| "YEAR" OptFieldLen FieldOpts { x := types.NewFieldType(mysql.TypeYear) x.Flen = $2.(int) diff --git a/parser/parser_test.go b/parser/parser_test.go index 0097a57c5f6c9..b74b43f438687 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -1819,6 +1819,7 @@ func (s *testParserSuite) TestType(c *C) { // for year {"create table t (y year(4), y1 year)", true}, + {"create table t (y year(4) unsigned zerofill zerofill, y1 year signed unsigned zerofill)", true}, // for national {"create table t (c1 national char(2), c2 national varchar(2))", true}, diff --git a/table/column.go b/table/column.go index e3766a688816e..86276ea6330ac 100644 --- a/table/column.go +++ b/table/column.go @@ -203,6 +203,9 @@ func (c *Column) GetTypeDesc() string { if mysql.HasUnsignedFlag(c.Flag) && c.Tp != mysql.TypeBit && c.Tp != mysql.TypeYear { desc += " UNSIGNED" } + if mysql.HasZerofillFlag(c.Flag) && c.Tp != mysql.TypeYear { + desc += " ZEROFILL" + } return desc } diff --git a/table/column_test.go b/table/column_test.go index 87e4999cefde7..bf17187a4d9ab 100644 --- a/table/column_test.go +++ b/table/column_test.go @@ -43,7 +43,7 @@ func (t *testTableSuite) TestString(c *C) { col.Collate = mysql.DefaultCollationName col.Flag |= mysql.ZerofillFlag | mysql.UnsignedFlag | mysql.BinaryFlag | mysql.AutoIncrementFlag | mysql.NotNullFlag - c.Assert(col.GetTypeDesc(), Equals, "tinyint(2) UNSIGNED") + c.Assert(col.GetTypeDesc(), Equals, "tinyint(2) UNSIGNED ZEROFILL") col.ToInfo() tbInfo := &model.TableInfo{} c.Assert(col.IsPKHandleColumn(tbInfo), Equals, false) From 621ca431ad8f92a00b7b8349fd891e9ee863f630 Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Mon, 3 Sep 2018 11:05:28 +0800 Subject: [PATCH 41/41] stats, plan: return row count 0 for empty range on handle (#7583) --- statistics/table.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/statistics/table.go b/statistics/table.go index 5b0f4e6d3e213..650893ec8266d 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -403,7 +403,7 @@ func (t *Table) ColumnEqualRowCount(sc *stmtctx.StatementContext, value types.Da func (coll *HistColl) GetRowCountByIntColumnRanges(sc *stmtctx.StatementContext, colID int64, intRanges []*ranger.Range) (float64, error) { if coll.ColumnIsInvalid(sc, colID) { if len(intRanges) == 0 { - return float64(coll.Count), nil + return 0, nil } if intRanges[0].LowVal[0].Kind() == types.KindInt64 { return getPseudoRowCountBySignedIntRanges(intRanges, float64(coll.Count)), nil