From 38ac1981d62fabc05a2823e7c79156e75432661f Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 22 Jun 2022 10:58:36 +0800 Subject: [PATCH 01/21] lightning: allow caller to set a logger (#35528) close pingcap/tidb#35556 --- br/pkg/lightning/backend/backend.go | 16 +-- br/pkg/lightning/backend/backend_test.go | 4 +- br/pkg/lightning/backend/kv/kv2sql.go | 10 +- br/pkg/lightning/backend/kv/session.go | 10 +- br/pkg/lightning/backend/kv/session_test.go | 3 +- br/pkg/lightning/backend/kv/sql2kv.go | 11 +- br/pkg/lightning/backend/kv/sql2kv_test.go | 26 ++--- br/pkg/lightning/backend/local/duplicate.go | 5 +- br/pkg/lightning/backend/local/engine.go | 27 +++-- br/pkg/lightning/backend/local/engine_test.go | 3 + br/pkg/lightning/backend/local/local.go | 105 +++++++++--------- br/pkg/lightning/backend/local/local_test.go | 6 + br/pkg/lightning/backend/local/localhelper.go | 60 +++++----- .../backend/local/localhelper_test.go | 6 +- br/pkg/lightning/backend/noop/noop.go | 2 +- br/pkg/lightning/backend/tidb/tidb.go | 20 ++-- br/pkg/lightning/backend/tidb/tidb_test.go | 48 ++++---- br/pkg/lightning/checkpoints/checkpoints.go | 36 +++--- .../checkpoints/checkpoints_file_test.go | 4 +- .../checkpoints/checkpoints_sql_test.go | 2 +- .../lightning/checkpoints/glue_checkpoint.go | 46 ++++---- br/pkg/lightning/common/conn.go | 10 +- br/pkg/lightning/errormanager/errormanager.go | 14 ++- .../errormanager/errormanager_test.go | 5 +- br/pkg/lightning/lightning.go | 20 ++-- br/pkg/lightning/lightning_serial_test.go | 2 + br/pkg/lightning/log/log.go | 18 +++ br/pkg/lightning/mydump/csv_parser.go | 3 +- br/pkg/lightning/mydump/loader.go | 8 +- br/pkg/lightning/mydump/parquet_parser.go | 10 +- br/pkg/lightning/mydump/parser.go | 12 +- br/pkg/lightning/mydump/reader.go | 2 +- br/pkg/lightning/mydump/region.go | 12 +- br/pkg/lightning/mydump/router.go | 8 +- br/pkg/lightning/mydump/router_test.go | 33 +++--- br/pkg/lightning/restore/check_info.go | 49 ++++---- br/pkg/lightning/restore/check_info_test.go | 8 +- br/pkg/lightning/restore/checksum.go | 14 +-- .../lightning/restore/chunk_restore_test.go | 26 +++-- br/pkg/lightning/restore/meta_manager.go | 44 ++++---- br/pkg/lightning/restore/restore.go | 104 ++++++++--------- br/pkg/lightning/restore/restore_test.go | 6 +- br/pkg/lightning/restore/table_restore.go | 35 ++++-- .../lightning/restore/table_restore_test.go | 20 ++-- br/pkg/lightning/restore/tidb.go | 20 ++-- br/pkg/lightning/run_options.go | 10 ++ br/pkg/mock/backend.go | 10 +- 47 files changed, 539 insertions(+), 414 deletions(-) mode change 100755 => 100644 br/pkg/lightning/backend/local/local.go diff --git a/br/pkg/lightning/backend/backend.go b/br/pkg/lightning/backend/backend.go index 7de6da020c2a1..fcfbb60a5aec7 100644 --- a/br/pkg/lightning/backend/backend.go +++ b/br/pkg/lightning/backend/backend.go @@ -69,8 +69,8 @@ func makeTag(tableName string, engineID int32) string { return fmt.Sprintf("%s:%d", tableName, engineID) } -func makeLogger(tag string, engineUUID uuid.UUID) log.Logger { - return log.With( +func makeLogger(logger log.Logger, tag string, engineUUID uuid.UUID) log.Logger { + return logger.With( zap.String("engineTag", tag), zap.Stringer("engineUUID", engineUUID), ) @@ -143,7 +143,7 @@ type AbstractBackend interface { ShouldPostProcess() bool // NewEncoder creates an encoder of a TiDB table. - NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) + NewEncoder(ctx context.Context, tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) OpenEngine(ctx context.Context, config *EngineConfig, engineUUID uuid.UUID) error @@ -260,8 +260,8 @@ func (be Backend) MakeEmptyRows() kv.Rows { return be.abstract.MakeEmptyRows() } -func (be Backend) NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { - return be.abstract.NewEncoder(tbl, options) +func (be Backend) NewEncoder(ctx context.Context, tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { + return be.abstract.NewEncoder(ctx, tbl, options) } func (be Backend) ShouldPostProcess() bool { @@ -321,7 +321,7 @@ func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID closedEngine := ClosedEngine{ engine: engine{ backend: be.abstract, - logger: makeLogger("", engineUUID), + logger: makeLogger(log.FromContext(ctx), "", engineUUID), uuid: engineUUID, }, } @@ -334,7 +334,7 @@ func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID // OpenEngine opens an engine with the given table name and engine ID. func (be Backend) OpenEngine(ctx context.Context, config *EngineConfig, tableName string, engineID int32) (*OpenedEngine, error) { tag, engineUUID := MakeUUID(tableName, engineID) - logger := makeLogger(tag, engineUUID) + logger := makeLogger(log.FromContext(ctx), tag, engineUUID) if err := be.abstract.OpenEngine(ctx, config, engineUUID); err != nil { return nil, err @@ -437,7 +437,7 @@ func (be Backend) UnsafeCloseEngine(ctx context.Context, cfg *EngineConfig, tabl func (be Backend) UnsafeCloseEngineWithUUID(ctx context.Context, cfg *EngineConfig, tag string, engineUUID uuid.UUID) (*ClosedEngine, error) { return engine{ backend: be.abstract, - logger: makeLogger(tag, engineUUID), + logger: makeLogger(log.FromContext(ctx), tag, engineUUID), uuid: engineUUID, }.unsafeClose(ctx, cfg) } diff --git a/br/pkg/lightning/backend/backend_test.go b/br/pkg/lightning/backend/backend_test.go index d388e74533833..73adc91deff9f 100644 --- a/br/pkg/lightning/backend/backend_test.go +++ b/br/pkg/lightning/backend/backend_test.go @@ -326,9 +326,9 @@ func TestNewEncoder(t *testing.T) { encoder := mock.NewMockEncoder(s.controller) options := &kv.SessionOptions{SQLMode: mysql.ModeANSIQuotes, Timestamp: 1234567890} - s.mockBackend.EXPECT().NewEncoder(nil, options).Return(encoder, nil) + s.mockBackend.EXPECT().NewEncoder(nil, nil, options).Return(encoder, nil) - realEncoder, err := s.mockBackend.NewEncoder(nil, options) + realEncoder, err := s.mockBackend.NewEncoder(nil, nil, options) require.Equal(t, realEncoder, encoder) require.NoError(t, err) } diff --git a/br/pkg/lightning/backend/kv/kv2sql.go b/br/pkg/lightning/backend/kv/kv2sql.go index 47b9aa5393b2d..1a436bbe822ea 100644 --- a/br/pkg/lightning/backend/kv/kv2sql.go +++ b/br/pkg/lightning/backend/kv/kv2sql.go @@ -17,6 +17,7 @@ package kv import ( "fmt" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/table" @@ -104,8 +105,13 @@ func (t *TableKVDecoder) IterRawIndexKeys(h kv.Handle, rawRow []byte, fn func([] return nil } -func NewTableKVDecoder(tbl table.Table, tableName string, options *SessionOptions) (*TableKVDecoder, error) { - se := newSession(options) +func NewTableKVDecoder( + tbl table.Table, + tableName string, + options *SessionOptions, + logger log.Logger, +) (*TableKVDecoder, error) { + se := newSession(options, logger) cols := tbl.Cols() // Set CommonAddRecordCtx to session to reuse the slices and BufStore in AddRecord recordCtx := tables.NewCommonAddRecordCtx(len(cols)) diff --git a/br/pkg/lightning/backend/kv/session.go b/br/pkg/lightning/backend/kv/session.go index 0e2751135c061..c5b623b31fd24 100644 --- a/br/pkg/lightning/backend/kv/session.go +++ b/br/pkg/lightning/backend/kv/session.go @@ -246,11 +246,11 @@ type SessionOptions struct { } // NewSession creates a new trimmed down Session matching the options. -func NewSession(options *SessionOptions) sessionctx.Context { - return newSession(options) +func NewSession(options *SessionOptions, logger log.Logger) sessionctx.Context { + return newSession(options, logger) } -func newSession(options *SessionOptions) *session { +func newSession(options *SessionOptions, logger log.Logger) *session { sqlMode := options.SQLMode vars := variable.NewSessionVars() vars.SkipUTF8Check = true @@ -265,7 +265,7 @@ func newSession(options *SessionOptions) *session { if options.SysVars != nil { for k, v := range options.SysVars { if err := vars.SetSystemVar(k, v); err != nil { - log.L().DPanic("new session: failed to set system var", + logger.DPanic("new session: failed to set system var", log.ShortError(err), zap.String("key", k)) } @@ -273,7 +273,7 @@ func newSession(options *SessionOptions) *session { } vars.StmtCtx.TimeZone = vars.Location() if err := vars.SetSystemVar("timestamp", strconv.FormatInt(options.Timestamp, 10)); err != nil { - log.L().Warn("new session: failed to set timestamp", + logger.Warn("new session: failed to set timestamp", log.ShortError(err)) } vars.TxnCtx = nil diff --git a/br/pkg/lightning/backend/kv/session_test.go b/br/pkg/lightning/backend/kv/session_test.go index 9703390afb2ec..a37f48c190ed8 100644 --- a/br/pkg/lightning/backend/kv/session_test.go +++ b/br/pkg/lightning/backend/kv/session_test.go @@ -17,12 +17,13 @@ package kv import ( "testing" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/parser/mysql" "github.com/stretchr/testify/require" ) func TestSession(t *testing.T) { - session := newSession(&SessionOptions{SQLMode: mysql.ModeNone, Timestamp: 1234567890}) + session := newSession(&SessionOptions{SQLMode: mysql.ModeNone, Timestamp: 1234567890}, log.L()) _, err := session.Txn(true) require.NoError(t, err) } diff --git a/br/pkg/lightning/backend/kv/sql2kv.go b/br/pkg/lightning/backend/kv/sql2kv.go index 54b34979b980e..46edf79d5e719 100644 --- a/br/pkg/lightning/backend/kv/sql2kv.go +++ b/br/pkg/lightning/backend/kv/sql2kv.go @@ -66,13 +66,18 @@ type tableKVEncoder struct { metrics *metric.Metrics } -func NewTableKVEncoder(tbl table.Table, options *SessionOptions, metrics *metric.Metrics) (Encoder, error) { +func NewTableKVEncoder( + tbl table.Table, + options *SessionOptions, + metrics *metric.Metrics, + logger log.Logger, +) (Encoder, error) { if metrics != nil { metrics.KvEncoderCounter.WithLabelValues("open").Inc() } meta := tbl.Meta() cols := tbl.Cols() - se := newSession(options) + se := newSession(options, logger) // Set CommonAddRecordCtx to session to reuse the slices and BufStore in AddRecord recordCtx := tables.NewCommonAddRecordCtx(len(cols)) tables.SetAddRecordCtx(se, recordCtx) @@ -267,7 +272,7 @@ func logKVConvertFailed(logger log.Logger, row []types.Datum, j int, colInfo *mo log.ShortError(err), ) - log.L().Error("failed to covert kv value", logutil.RedactAny("origVal", original.GetValue()), + logger.Error("failed to convert kv value", logutil.RedactAny("origVal", original.GetValue()), zap.Stringer("fieldType", &colInfo.FieldType), zap.String("column", colInfo.Name.O), zap.Int("columnID", j+1)) return errors.Annotatef( diff --git a/br/pkg/lightning/backend/kv/sql2kv_test.go b/br/pkg/lightning/backend/kv/sql2kv_test.go index b7a33e3c6943a..b604942e38756 100644 --- a/br/pkg/lightning/backend/kv/sql2kv_test.go +++ b/br/pkg/lightning/backend/kv/sql2kv_test.go @@ -89,7 +89,7 @@ func TestEncode(t *testing.T) { strictMode, err := NewTableKVEncoder(tbl, &SessionOptions{ SQLMode: mysql.ModeStrictAllTables, Timestamp: 1234567890, - }, nil) + }, nil, logger) require.NoError(t, err) pairs, err := strictMode.Encode(logger, rows, 1, []int{0, 1}, "1.csv", 1234) require.Regexp(t, "failed to cast value as tinyint\\(4\\) for column `c1` \\(#1\\):.*overflows tinyint", err) @@ -121,7 +121,7 @@ func TestEncode(t *testing.T) { mockMode, err := NewTableKVEncoder(mockTbl, &SessionOptions{ SQLMode: mysql.ModeStrictAllTables, Timestamp: 1234567891, - }, nil) + }, nil, logger) require.NoError(t, err) _, err = mockMode.Encode(logger, rowsWithPk2, 2, []int{0, 1}, "1.csv", 1234) require.EqualError(t, err, "mock error") @@ -131,7 +131,7 @@ func TestEncode(t *testing.T) { SQLMode: mysql.ModeNone, Timestamp: 1234567892, SysVars: map[string]string{"tidb_row_format_version": "1"}, - }, nil) + }, nil, logger) require.NoError(t, err) pairs, err = noneMode.Encode(logger, rows, 1, []int{0, 1}, "1.csv", 1234) require.NoError(t, err) @@ -153,7 +153,7 @@ func TestDecode(t *testing.T) { decoder, err := NewTableKVDecoder(tbl, "`test`.`c1`", &SessionOptions{ SQLMode: mysql.ModeStrictAllTables, Timestamp: 1234567890, - }) + }, log.L()) require.NoError(t, err) require.NotNil(t, decoder) require.Equal(t, decoder.Name(), "`test`.`c1`") @@ -208,7 +208,7 @@ func TestDecodeIndex(t *testing.T) { strictMode, err := NewTableKVEncoder(tbl, &SessionOptions{ SQLMode: mysql.ModeStrictAllTables, Timestamp: 1234567890, - }, nil) + }, nil, log.L()) require.NoError(t, err) pairs, err := strictMode.Encode(logger, rows, 1, []int{0, 1, -1}, "1.csv", 123) data := pairs.(*KvPairs) @@ -217,7 +217,7 @@ func TestDecodeIndex(t *testing.T) { decoder, err := NewTableKVDecoder(tbl, "`test`.``", &SessionOptions{ SQLMode: mysql.ModeStrictAllTables, Timestamp: 1234567890, - }) + }, log.L()) require.NoError(t, err) h1, err := decoder.DecodeHandleFromRowKey(data.pairs[0].Key) require.NoError(t, err) @@ -247,7 +247,7 @@ func TestEncodeRowFormatV2(t *testing.T) { SQLMode: mysql.ModeNone, Timestamp: 1234567892, SysVars: map[string]string{"tidb_row_format_version": "2"}, - }, nil) + }, nil, log.L()) require.NoError(t, err) pairs, err := noneMode.Encode(logger, rows, 1, []int{0, 1}, "1.csv", 1234) require.NoError(t, err) @@ -295,7 +295,7 @@ func TestEncodeTimestamp(t *testing.T) { "tidb_row_format_version": "1", "time_zone": "+08:00", }, - }, nil) + }, nil, log.L()) require.NoError(t, err) pairs, err := encoder.Encode(logger, nil, 70, []int{-1, 1}, "1.csv", 1234) require.NoError(t, err) @@ -320,7 +320,7 @@ func TestEncodeDoubleAutoIncrement(t *testing.T) { SysVars: map[string]string{ "tidb_row_format_version": "2", }, - }, nil) + }, nil, log.L()) require.NoError(t, err) strDatumForID := types.NewStringDatum("1") @@ -386,7 +386,7 @@ func TestEncodeMissingAutoValue(t *testing.T) { SysVars: map[string]string{ "tidb_row_format_version": "2", }, - }, nil) + }, nil, log.L()) require.NoError(t, err) realRowID := encoder.(*tableKVEncoder).autoIDFn(rowID) @@ -447,7 +447,7 @@ func TestDefaultAutoRandoms(t *testing.T) { Timestamp: 1234567893, SysVars: map[string]string{"tidb_row_format_version": "2"}, AutoRandomSeed: 456, - }, nil) + }, nil, log.L()) require.NoError(t, err) logger := log.Logger{Logger: zap.NewNop()} pairs, err := encoder.Encode(logger, []types.Datum{types.NewStringDatum("")}, 70, []int{-1, 0}, "1.csv", 1234) @@ -482,7 +482,7 @@ func TestShardRowId(t *testing.T) { Timestamp: 1234567893, SysVars: map[string]string{"tidb_row_format_version": "2"}, AutoRandomSeed: 456, - }, nil) + }, nil, log.L()) require.NoError(t, err) logger := log.Logger{Logger: zap.NewNop()} keyMap := make(map[int64]struct{}, 16) @@ -636,7 +636,7 @@ func SetUpTest(b *testing.B) *benchSQL2KVSuite { // Construct the corresponding KV encoder. tbl, err := tables.TableFromMeta(NewPanickingAllocators(0), tableInfo) require.NoError(b, err) - encoder, err := NewTableKVEncoder(tbl, &SessionOptions{SysVars: map[string]string{"tidb_row_format_version": "2"}}, nil) + encoder, err := NewTableKVEncoder(tbl, &SessionOptions{SysVars: map[string]string{"tidb_row_format_version": "2"}}, nil, log.L()) require.NoError(b, err) logger := log.Logger{Logger: zap.NewNop()} diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index 983ae33fcfd68..37023c48c2179 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -411,12 +411,13 @@ func NewDuplicateManager( sessOpts *kv.SessionOptions, concurrency int, hasDupe *atomic.Bool, + logger log.Logger, ) (*DuplicateManager, error) { - decoder, err := kv.NewTableKVDecoder(tbl, tableName, sessOpts) + logger = logger.With(zap.String("tableName", tableName)) + decoder, err := kv.NewTableKVDecoder(tbl, tableName, sessOpts, logger) if err != nil { return nil, errors.Trace(err) } - logger := log.With(zap.String("tableName", tableName)) return &DuplicateManager{ tbl: tbl, tableName: tableName, diff --git a/br/pkg/lightning/backend/local/engine.go b/br/pkg/lightning/backend/local/engine.go index f82334e0d58d2..04036e57b16ac 100644 --- a/br/pkg/lightning/backend/local/engine.go +++ b/br/pkg/lightning/backend/local/engine.go @@ -135,6 +135,8 @@ type Engine struct { duplicateDetection bool duplicateDB *pebble.DB errorMgr *errormanager.ErrorManager + + logger log.Logger } func (e *Engine) setError(err error) { @@ -145,7 +147,7 @@ func (e *Engine) setError(err error) { } func (e *Engine) Close() error { - log.L().Debug("closing local engine", zap.Stringer("engine", e.UUID), zap.Stack("stack")) + e.logger.Debug("closing local engine", zap.Stringer("engine", e.UUID), zap.Stack("stack")) if e.db == nil { return nil } @@ -774,7 +776,7 @@ func (e *Engine) ingestSSTs(metas []*sstMeta) error { totalCount += m.totalCount fileSize += m.fileSize } - log.L().Info("write data to local DB", + e.logger.Info("write data to local DB", zap.Int64("size", totalSize), zap.Int64("kvs", totalCount), zap.Int("files", len(metas)), @@ -861,7 +863,7 @@ func saveEngineMetaToDB(meta *engineMeta, db *pebble.DB) error { // saveEngineMeta saves the metadata about the DB into the DB itself. // This method should be followed by a Flush to ensure the data is actually synchronized func (e *Engine) saveEngineMeta() error { - log.L().Debug("save engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()), + e.logger.Debug("save engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()), zap.Int64("size", e.TotalSize.Load())) return errors.Trace(saveEngineMetaToDB(&e.engineMeta, e.db)) } @@ -870,7 +872,7 @@ func (e *Engine) loadEngineMeta() error { jsonBytes, closer, err := e.db.Get(engineMetaKey) if err != nil { if err == pebble.ErrNotFound { - log.L().Debug("local db missing engine meta", zap.Stringer("uuid", e.UUID), log.ShortError(err)) + e.logger.Debug("local db missing engine meta", zap.Stringer("uuid", e.UUID), log.ShortError(err)) return nil } return err @@ -878,10 +880,10 @@ func (e *Engine) loadEngineMeta() error { defer closer.Close() if err = json.Unmarshal(jsonBytes, &e.engineMeta); err != nil { - log.L().Warn("local db failed to deserialize meta", zap.Stringer("uuid", e.UUID), zap.ByteString("content", jsonBytes), zap.Error(err)) + e.logger.Warn("local db failed to deserialize meta", zap.Stringer("uuid", e.UUID), zap.ByteString("content", jsonBytes), zap.Error(err)) return err } - log.L().Debug("load engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()), + e.logger.Debug("load engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()), zap.Int64("size", e.TotalSize.Load())) return nil } @@ -961,7 +963,7 @@ func (e *Engine) newKVIter(ctx context.Context, opts *pebble.IterOptions) Iter { if !e.duplicateDetection { return pebbleIter{Iterator: e.db.NewIter(opts)} } - logger := log.With( + logger := log.FromContext(ctx).With( zap.String("table", common.UniqueTable(e.tableInfo.DB, e.tableInfo.Name)), zap.Int64("tableID", e.tableInfo.ID), zap.Stringer("engineUUID", e.UUID)) @@ -1247,7 +1249,7 @@ func (w *Writer) createSSTWriter() (*sstWriter, error) { if err != nil { return nil, err } - sw := &sstWriter{sstMeta: &sstMeta{path: path}, writer: writer} + sw := &sstWriter{sstMeta: &sstMeta{path: path}, writer: writer, logger: w.engine.logger} return sw, nil } @@ -1256,6 +1258,7 @@ var errorUnorderedSSTInsertion = errors.New("inserting KVs into SST without orde type sstWriter struct { *sstMeta writer *sstable.Writer + logger log.Logger } func newSSTWriter(path string) (*sstable.Writer, error) { @@ -1289,7 +1292,7 @@ func (sw *sstWriter) writeKVs(kvs []common.KvPair) error { var lastKey []byte for _, p := range kvs { if bytes.Equal(p.Key, lastKey) { - log.L().Warn("duplicated key found, skip write", logutil.Key("key", p.Key)) + sw.logger.Warn("duplicated key found, skip write", logutil.Key("key", p.Key)) continue } internalKey.UserKey = p.Key @@ -1467,7 +1470,7 @@ func (i dbSSTIngester) mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) lastKey := make([]byte, 0) for { if bytes.Equal(lastKey, key) { - log.L().Warn("duplicated key found, skipped", zap.Binary("key", lastKey)) + i.e.logger.Warn("duplicated key found, skipped", zap.Binary("key", lastKey)) newMeta.totalCount-- newMeta.totalSize -= int64(len(key) + len(val)) @@ -1500,7 +1503,7 @@ func (i dbSSTIngester) mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) newMeta.fileSize = int64(meta.Size) dur := time.Since(start) - log.L().Info("compact sst", zap.Int("fileCount", len(metas)), zap.Int64("size", newMeta.totalSize), + i.e.logger.Info("compact sst", zap.Int("fileCount", len(metas)), zap.Int64("size", newMeta.totalSize), zap.Int64("count", newMeta.totalCount), zap.Duration("cost", dur), zap.String("file", name)) // async clean raw SSTs. @@ -1509,7 +1512,7 @@ func (i dbSSTIngester) mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) for _, m := range metas { totalSize += m.fileSize if err := os.Remove(m.path); err != nil { - log.L().Warn("async cleanup sst file failed", zap.Error(err)) + i.e.logger.Warn("async cleanup sst file failed", zap.Error(err)) } } // decrease the pending size after clean up diff --git a/br/pkg/lightning/backend/local/engine_test.go b/br/pkg/lightning/backend/local/engine_test.go index cb2dbd6fc3a97..13c890c028297 100644 --- a/br/pkg/lightning/backend/local/engine_test.go +++ b/br/pkg/lightning/backend/local/engine_test.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/sstable" "github.com/google/uuid" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/stretchr/testify/require" "github.com/pingcap/failpoint" @@ -59,6 +60,7 @@ func TestIngestSSTWithClosedEngine(t *testing.T) { cancel: cancel, sstMetasChan: make(chan metaOrFlush, 64), keyAdapter: noopKeyAdapter{}, + logger: log.L(), } f.sstIngester = dbSSTIngester{e: f} sstPath := path.Join(tmpPath, uuid.New().String()+".sst") @@ -97,6 +99,7 @@ func TestAutoSplitSST(t *testing.T) { engine: &Engine{ sstDir: dir, keyAdapter: noopKeyAdapter{}, + logger: log.L(), }, isKVSorted: true, isWriteBatchSorted: true, diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go old mode 100755 new mode 100644 index 13773dc6d2ee4..76bdba192bd99 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -235,6 +235,7 @@ type local struct { bufferPool *membuf.Pool metrics *metric.Metrics writeLimiter StoreWriteLimiter + logger log.Logger } func openDuplicateDB(storeDir string) (*pebble.DB, error) { @@ -342,6 +343,7 @@ func NewLocalBackend( importClientFactory: importClientFactory, bufferPool: membuf.NewPool(membuf.WithAllocator(manual.Allocator{})), writeLimiter: writeLimiter, + logger: log.FromContext(ctx), } if m, ok := metric.FromContext(ctx); ok { local.metrics = m @@ -384,7 +386,7 @@ func (local *local) checkMultiIngestSupport(ctx context.Context) error { client, err1 := local.getImportClient(ctx, s.Id) if err1 != nil { err = err1 - log.L().Warn("get import client failed", zap.Error(err), zap.String("store", s.Address)) + log.FromContext(ctx).Warn("get import client failed", zap.Error(err), zap.String("store", s.Address)) continue } _, err = client.MultiIngest(ctx, &sst.MultiIngestRequest{}) @@ -393,12 +395,12 @@ func (local *local) checkMultiIngestSupport(ctx context.Context) error { } if st, ok := status.FromError(err); ok { if st.Code() == codes.Unimplemented { - log.L().Info("multi ingest not support", zap.Any("unsupported store", s)) + log.FromContext(ctx).Info("multi ingest not support", zap.Any("unsupported store", s)) local.supportMultiIngest = false return nil } } - log.L().Warn("check multi ingest support failed", zap.Error(err), zap.String("store", s.Address), + log.FromContext(ctx).Warn("check multi ingest support failed", zap.Error(err), zap.String("store", s.Address), zap.Int("retry", i)) } if err != nil { @@ -407,14 +409,14 @@ func (local *local) checkMultiIngestSupport(ctx context.Context) error { if hasTiFlash { return errors.Trace(err) } - log.L().Warn("check multi failed all retry, fallback to false", log.ShortError(err)) + log.FromContext(ctx).Warn("check multi failed all retry, fallback to false", log.ShortError(err)) local.supportMultiIngest = false return nil } } local.supportMultiIngest = true - log.L().Info("multi ingest support") + log.FromContext(ctx).Info("multi ingest support") return nil } @@ -479,6 +481,7 @@ func (local *local) Close() { engine.Close() engine.unlock() } + local.importClientFactory.Close() local.bufferPool.Destroy() @@ -488,22 +491,22 @@ func (local *local) Close() { hasDuplicates := iter.First() allIsWell := true if err := iter.Error(); err != nil { - log.L().Warn("iterate duplicate db failed", zap.Error(err)) + local.logger.Warn("iterate duplicate db failed", zap.Error(err)) allIsWell = false } if err := iter.Close(); err != nil { - log.L().Warn("close duplicate db iter failed", zap.Error(err)) + local.logger.Warn("close duplicate db iter failed", zap.Error(err)) allIsWell = false } if err := local.duplicateDB.Close(); err != nil { - log.L().Warn("close duplicate db failed", zap.Error(err)) + local.logger.Warn("close duplicate db failed", zap.Error(err)) allIsWell = false } // If checkpoint is disabled, or we don't detect any duplicate, then this duplicate // db dir will be useless, so we clean up this dir. if allIsWell && (!local.checkpointEnabled || !hasDuplicates) { if err := os.RemoveAll(filepath.Join(local.localStoreDir, duplicateDBName)); err != nil { - log.L().Warn("remove duplicate db file failed", zap.Error(err)) + local.logger.Warn("remove duplicate db file failed", zap.Error(err)) } } local.duplicateDB = nil @@ -514,7 +517,7 @@ func (local *local) Close() { if !local.checkpointEnabled || common.IsEmptyDir(local.localStoreDir) { err := os.RemoveAll(local.localStoreDir) if err != nil { - log.L().Warn("remove local db file failed", zap.Error(err)) + local.logger.Warn("remove local db file failed", zap.Error(err)) } } @@ -626,6 +629,7 @@ func (local *local) OpenEngine(ctx context.Context, cfg *backend.EngineConfig, e duplicateDB: local.duplicateDB, errorMgr: local.errorMgr, keyAdapter: local.keyAdapter, + logger: log.FromContext(ctx), }) engine := e.(*Engine) engine.db = db @@ -674,6 +678,7 @@ func (local *local) CloseEngine(ctx context.Context, cfg *backend.EngineConfig, duplicateDetection: local.duplicateDetection, duplicateDB: local.duplicateDB, errorMgr: local.errorMgr, + logger: log.FromContext(ctx), } engine.sstIngester = dbSSTIngester{e: engine} if err = engine.loadEngineMeta(); err != nil { @@ -750,7 +755,7 @@ func (local *local) WriteToTiKV( break } if e != nil { - log.L().Error("failed to get StoreInfo from pd http api", zap.Error(e)) + log.FromContext(ctx).Error("failed to get StoreInfo from pd http api", zap.Error(e)) } } } @@ -767,7 +772,7 @@ func (local *local) WriteToTiKV( return nil, Range{}, stats, errors.Annotate(iter.Error(), "failed to read the first key") } if !iter.Valid() { - log.L().Info("keys within region is empty, skip ingest", logutil.Key("start", start), + log.FromContext(ctx).Info("keys within region is empty, skip ingest", logutil.Key("start", start), logutil.Key("regionStart", region.Region.StartKey), logutil.Key("end", end), logutil.Key("regionEnd", region.Region.EndKey)) return nil, regionRange, stats, nil @@ -908,20 +913,20 @@ func (local *local) WriteToTiKV( } if leaderID == region.Region.Peers[i].GetId() { leaderPeerMetas = resp.Metas - log.L().Debug("get metas after write kv stream to tikv", zap.Reflect("metas", leaderPeerMetas)) + log.FromContext(ctx).Debug("get metas after write kv stream to tikv", zap.Reflect("metas", leaderPeerMetas)) } } // if there is not leader currently, we should directly return an error if len(leaderPeerMetas) == 0 { - log.L().Warn("write to tikv no leader", logutil.Region(region.Region), logutil.Leader(region.Leader), + log.FromContext(ctx).Warn("write to tikv no leader", logutil.Region(region.Region), logutil.Leader(region.Leader), zap.Uint64("leader_id", leaderID), logutil.SSTMeta(meta), zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", size)) return nil, Range{}, stats, errors.Errorf("write to tikv with no leader returned, region '%d', leader: %d", region.Region.Id, leaderID) } - log.L().Debug("write to kv", zap.Reflect("region", region), zap.Uint64("leader", leaderID), + log.FromContext(ctx).Debug("write to kv", zap.Reflect("region", region), zap.Uint64("leader", leaderID), zap.Reflect("meta", meta), zap.Reflect("return metas", leaderPeerMetas), zap.Int64("kv_pairs", totalCount), zap.Int64("total_bytes", size), zap.Int64("buf_size", bytesBuf.TotalSize()), @@ -931,7 +936,7 @@ func (local *local) WriteToTiKV( if iter.Valid() && iter.Next() { firstKey := append([]byte{}, iter.Key()...) finishedRange = Range{start: regionRange.start, end: firstKey} - log.L().Info("write to tikv partial finish", zap.Int64("count", totalCount), + log.FromContext(ctx).Info("write to tikv partial finish", zap.Int64("count", totalCount), zap.Int64("size", size), logutil.Key("startKey", regionRange.start), logutil.Key("endKey", regionRange.end), logutil.Key("remainStart", firstKey), logutil.Key("remainEnd", regionRange.end), logutil.Region(region.Region), logutil.Leader(region.Leader)) @@ -1047,7 +1052,7 @@ func (local *local) readAndSplitIntoRange(ctx context.Context, engine *Engine, r return ranges, nil } - logger := log.With(zap.Stringer("engine", engine.UUID)) + logger := log.FromContext(ctx).With(zap.Stringer("engine", engine.UUID)) sizeProps, err := getSizeProperties(logger, engine.db, local.keyAdapter) if err != nil { return nil, errors.Trace(err) @@ -1084,7 +1089,7 @@ func (local *local) writeAndIngestByRange( return errors.Annotate(iter.Error(), "failed to read the first key") } if !hasKey { - log.L().Info("There is no pairs in iterator", + log.FromContext(ctxt).Info("There is no pairs in iterator", logutil.Key("start", start), logutil.Key("end", end)) engine.finishedRanges.add(Range{start: start, end: end}) @@ -1115,14 +1120,14 @@ WriteAndIngest: endKey := codec.EncodeBytes([]byte{}, nextKey(pairEnd)) regions, err = split.PaginateScanRegion(ctx, local.splitCli, startKey, endKey, scanRegionLimit) if err != nil || len(regions) == 0 { - log.L().Warn("scan region failed", log.ShortError(err), zap.Int("region_len", len(regions)), + log.FromContext(ctx).Warn("scan region failed", log.ShortError(err), zap.Int("region_len", len(regions)), logutil.Key("startKey", startKey), logutil.Key("endKey", endKey), zap.Int("retry", retry)) retry++ continue WriteAndIngest } for _, region := range regions { - log.L().Debug("get region", zap.Int("retry", retry), zap.Binary("startKey", startKey), + log.FromContext(ctx).Debug("get region", zap.Int("retry", retry), zap.Binary("startKey", startKey), zap.Binary("endKey", endKey), zap.Uint64("id", region.Region.GetId()), zap.Stringer("epoch", region.Region.GetRegionEpoch()), zap.Binary("start", region.Region.GetStartKey()), zap.Binary("end", region.Region.GetEndKey()), zap.Reflect("peers", region.Region.GetPeers())) @@ -1141,7 +1146,7 @@ WriteAndIngest: } else { retry++ } - log.L().Info("retry write and ingest kv pairs", logutil.Key("startKey", pairStart), + log.FromContext(ctx).Info("retry write and ingest kv pairs", logutil.Key("startKey", pairStart), logutil.Key("endKey", end), log.ShortError(err), zap.Int("retry", retry)) continue WriteAndIngest } @@ -1182,7 +1187,7 @@ loopWrite: return err } - log.L().Warn("write to tikv failed", log.ShortError(err), zap.Int("retry", i)) + log.FromContext(ctx).Warn("write to tikv failed", log.ShortError(err), zap.Int("retry", i)) continue loopWrite } @@ -1201,7 +1206,7 @@ loopWrite: ingestMetas := metas[start:end] errCnt := 0 for errCnt < maxRetryTimes { - log.L().Debug("ingest meta", zap.Reflect("meta", ingestMetas)) + log.FromContext(ctx).Debug("ingest meta", zap.Reflect("meta", ingestMetas)) var resp *sst.IngestResponse failpoint.Inject("FailIngestMeta", func(val failpoint.Value) { // only inject the error once @@ -1235,7 +1240,7 @@ loopWrite: if common.IsContextCanceledError(err) { return err } - log.L().Warn("ingest failed", log.ShortError(err), logutil.SSTMetas(ingestMetas), + log.FromContext(ctx).Warn("ingest failed", log.ShortError(err), logutil.SSTMetas(ingestMetas), logutil.Region(region.Region), logutil.Leader(region.Leader)) errCnt++ continue @@ -1253,7 +1258,7 @@ loopWrite: } switch retryTy { case retryNone: - log.L().Warn("ingest failed noretry", log.ShortError(err), logutil.SSTMetas(ingestMetas), + log.FromContext(ctx).Warn("ingest failed noretry", log.ShortError(err), logutil.SSTMetas(ingestMetas), logutil.Region(region.Region), logutil.Leader(region.Leader)) // met non-retryable error retry whole Write procedure return err @@ -1268,7 +1273,7 @@ loopWrite: } if err != nil { - log.L().Warn("write and ingest region, will retry import full range", log.ShortError(err), + log.FromContext(ctx).Warn("write and ingest region, will retry import full range", log.ShortError(err), logutil.Region(region.Region), logutil.Key("start", start), logutil.Key("end", end)) } else { @@ -1288,10 +1293,10 @@ loopWrite: func (local *local) writeAndIngestByRanges(ctx context.Context, engine *Engine, ranges []Range, regionSplitSize int64, regionSplitKeys int64) error { if engine.Length.Load() == 0 { // engine is empty, this is likes because it's a index engine but the table contains no index - log.L().Info("engine contains no data", zap.Stringer("uuid", engine.UUID)) + log.FromContext(ctx).Info("engine contains no data", zap.Stringer("uuid", engine.UUID)) return nil } - log.L().Debug("the ranges Length write to tikv", zap.Int("Length", len(ranges))) + log.FromContext(ctx).Debug("the ranges Length write to tikv", zap.Int("Length", len(ranges))) var allErrLock sync.Mutex var allErr error @@ -1324,7 +1329,7 @@ func (local *local) writeAndIngestByRanges(ctx context.Context, engine *Engine, if !common.IsRetryableError(err) { break } - log.L().Warn("write and ingest by range failed", + log.FromContext(ctx).Warn("write and ingest by range failed", zap.Int("retry time", i+1), log.ShortError(err)) backOffTime *= 2 if backOffTime > maxRetryBackoffTime { @@ -1366,7 +1371,7 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi lfTotalSize := lf.TotalSize.Load() lfLength := lf.Length.Load() if lfTotalSize == 0 { - log.L().Info("engine contains no kv, skip import", zap.Stringer("engine", engineUUID)) + log.FromContext(ctx).Info("engine contains no kv, skip import", zap.Stringer("engine", engineUUID)) return nil } kvRegionSplitSize, kvRegionSplitKeys, err := getRegionSplitSizeKeys(ctx, local.pdCtl.GetPDClient(), local.tls) @@ -1378,7 +1383,7 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi regionSplitKeys = kvRegionSplitKeys } } else { - log.L().Warn("fail to get region split keys and size", zap.Error(err)) + log.FromContext(ctx).Warn("fail to get region split keys and size", zap.Error(err)) } // split sorted file into range by 96MB size per file @@ -1387,14 +1392,14 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi return err } - log.L().Info("start import engine", zap.Stringer("uuid", engineUUID), + log.FromContext(ctx).Info("start import engine", zap.Stringer("uuid", engineUUID), zap.Int("ranges", len(ranges)), zap.Int64("count", lfLength), zap.Int64("size", lfTotalSize)) for { unfinishedRanges := lf.unfinishedRanges(ranges) if len(unfinishedRanges) == 0 { break } - log.L().Info("import engine unfinished ranges", zap.Int("count", len(unfinishedRanges))) + log.FromContext(ctx).Info("import engine unfinished ranges", zap.Int("count", len(unfinishedRanges))) // if all the kv can fit in one region, skip split regions. TiDB will split one region for // the table when table is created. @@ -1406,37 +1411,37 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi break } - log.L().Warn("split and scatter failed in retry", zap.Stringer("uuid", engineUUID), + log.FromContext(ctx).Warn("split and scatter failed in retry", zap.Stringer("uuid", engineUUID), log.ShortError(err), zap.Int("retry", i)) } if err != nil { - log.L().Error("split & scatter ranges failed", zap.Stringer("uuid", engineUUID), log.ShortError(err)) + log.FromContext(ctx).Error("split & scatter ranges failed", zap.Stringer("uuid", engineUUID), log.ShortError(err)) return err } // start to write to kv and ingest err = local.writeAndIngestByRanges(ctx, lf, unfinishedRanges, regionSplitSize, regionSplitKeys) if err != nil { - log.L().Error("write and ingest engine failed", log.ShortError(err)) + log.FromContext(ctx).Error("write and ingest engine failed", log.ShortError(err)) return err } } - log.L().Info("import engine success", zap.Stringer("uuid", engineUUID), + log.FromContext(ctx).Info("import engine success", zap.Stringer("uuid", engineUUID), zap.Int64("size", lfTotalSize), zap.Int64("kvs", lfLength), zap.Int64("importedSize", lf.importedKVSize.Load()), zap.Int64("importedCount", lf.importedKVCount.Load())) return nil } func (local *local) CollectLocalDuplicateRows(ctx context.Context, tbl table.Table, tableName string, opts *kv.SessionOptions) (hasDupe bool, err error) { - logger := log.With(zap.String("table", tableName)).Begin(zap.InfoLevel, "[detect-dupe] collect local duplicate keys") + logger := log.FromContext(ctx).With(zap.String("table", tableName)).Begin(zap.InfoLevel, "[detect-dupe] collect local duplicate keys") defer func() { logger.End(zap.ErrorLevel, err) }() atomicHasDupe := atomic.NewBool(false) duplicateManager, err := NewDuplicateManager(tbl, tableName, local.splitCli, local.tikvCli, - local.errorMgr, opts, local.dupeConcurrency, atomicHasDupe) + local.errorMgr, opts, local.dupeConcurrency, atomicHasDupe, log.FromContext(ctx)) if err != nil { return false, errors.Trace(err) } @@ -1447,14 +1452,14 @@ func (local *local) CollectLocalDuplicateRows(ctx context.Context, tbl table.Tab } func (local *local) CollectRemoteDuplicateRows(ctx context.Context, tbl table.Table, tableName string, opts *kv.SessionOptions) (hasDupe bool, err error) { - logger := log.With(zap.String("table", tableName)).Begin(zap.InfoLevel, "[detect-dupe] collect remote duplicate keys") + logger := log.FromContext(ctx).With(zap.String("table", tableName)).Begin(zap.InfoLevel, "[detect-dupe] collect remote duplicate keys") defer func() { logger.End(zap.ErrorLevel, err) }() atomicHasDupe := atomic.NewBool(false) duplicateManager, err := NewDuplicateManager(tbl, tableName, local.splitCli, local.tikvCli, - local.errorMgr, opts, local.dupeConcurrency, atomicHasDupe) + local.errorMgr, opts, local.dupeConcurrency, atomicHasDupe, log.FromContext(ctx)) if err != nil { return false, errors.Trace(err) } @@ -1465,7 +1470,7 @@ func (local *local) CollectRemoteDuplicateRows(ctx context.Context, tbl table.Ta } func (local *local) ResolveDuplicateRows(ctx context.Context, tbl table.Table, tableName string, algorithm config.DuplicateResolutionAlgorithm) (err error) { - logger := log.With(zap.String("table", tableName)).Begin(zap.InfoLevel, "[resolve-dupe] resolve duplicate rows") + logger := log.FromContext(ctx).With(zap.String("table", tableName)).Begin(zap.InfoLevel, "[resolve-dupe] resolve duplicate rows") defer func() { logger.End(zap.ErrorLevel, err) }() @@ -1483,7 +1488,7 @@ func (local *local) ResolveDuplicateRows(ctx context.Context, tbl table.Table, t // TODO: reuse the *kv.SessionOptions from NewEncoder for picking the correct time zone. decoder, err := kv.NewTableKVDecoder(tbl, tableName, &kv.SessionOptions{ SQLMode: mysql.ModeStrictAllTables, - }) + }, log.FromContext(ctx)) if err != nil { return err } @@ -1565,7 +1570,7 @@ func (local *local) ResetEngine(ctx context.Context, engineUUID uuid.UUID) error // the only way to reset the engine + reclaim the space is to delete and reopen it 🤷 localEngine := local.lockEngine(engineUUID, importMutexStateClose) if localEngine == nil { - log.L().Warn("could not find engine in cleanupEngine", zap.Stringer("uuid", engineUUID)) + log.FromContext(ctx).Warn("could not find engine in cleanupEngine", zap.Stringer("uuid", engineUUID)) return nil } defer localEngine.unlock() @@ -1598,7 +1603,7 @@ func (local *local) CleanupEngine(ctx context.Context, engineUUID uuid.UUID) err localEngine := local.lockEngine(engineUUID, importMutexStateClose) // release this engine after import success if localEngine == nil { - log.L().Warn("could not find engine in cleanupEngine", zap.Stringer("uuid", engineUUID)) + log.FromContext(ctx).Warn("could not find engine in cleanupEngine", zap.Stringer("uuid", engineUUID)) return nil } defer localEngine.unlock() @@ -1675,7 +1680,7 @@ func checkTiFlashVersion(ctx context.Context, g glue.Glue, checkCtx *backend.Che return nil } - res, err := g.GetSQLExecutor().QueryStringsWithLog(ctx, tiFlashReplicaQuery, "fetch tiflash replica info", log.L()) + res, err := g.GetSQLExecutor().QueryStringsWithLog(ctx, tiFlashReplicaQuery, "fetch tiflash replica info", log.FromContext(ctx)) if err != nil { return errors.Annotate(err, "fetch tiflash replica info failed") } @@ -1714,8 +1719,8 @@ func (local *local) MakeEmptyRows() kv.Rows { return kv.MakeRowsFromKvPairs(nil) } -func (local *local) NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { - return kv.NewTableKVEncoder(tbl, options, local.metrics) +func (local *local) NewEncoder(ctx context.Context, tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { + return kv.NewTableKVEncoder(tbl, options, local.metrics, log.FromContext(ctx)) } func engineSSTDir(storeDir string, engineUUID uuid.UUID) string { @@ -1767,7 +1772,7 @@ func (local *local) isIngestRetryable( if newRegion != nil { return newRegion, nil } - log.L().Warn("get region by key return nil, will retry", logutil.Region(region.Region), logutil.Leader(region.Leader), + log.FromContext(ctx).Warn("get region by key return nil, will retry", logutil.Region(region.Region), logutil.Leader(region.Leader), zap.Int("retry", i)) select { case <-ctx.Done(): @@ -1919,7 +1924,7 @@ func getRegionSplitSizeKeys(ctx context.Context, cli pd.Client, tls *common.TLS) if err == nil { return regionSplitSize, regionSplitKeys, nil } - log.L().Warn("get region split size and keys failed", zap.Error(err), zap.String("store", serverInfo.StatusAddr)) + log.FromContext(ctx).Warn("get region split size and keys failed", zap.Error(err), zap.String("store", serverInfo.StatusAddr)) } return 0, 0, errors.New("get region split size and keys failed") } diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index 0711bfb1fc463..13112051e59d6 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/membuf" "github.com/pingcap/tidb/br/pkg/mock" @@ -331,6 +332,7 @@ func testLocalWriter(t *testing.T, needSort bool, partitialSort bool) { cancel: cancel, sstMetasChan: make(chan metaOrFlush, 64), keyAdapter: noopKeyAdapter{}, + logger: log.L(), } f.sstIngester = dbSSTIngester{e: f} f.wg.Add(1) @@ -438,6 +440,7 @@ func (c *mockSplitClient) GetRegion(ctx context.Context, key []byte) (*restore.R func TestIsIngestRetryable(t *testing.T) { local := &local{ splitCli: &mockSplitClient{}, + logger: log.L(), } resp := &sst.IngestResponse{ @@ -567,6 +570,7 @@ func TestLocalIngestLoop(t *testing.T) { CompactThreshold: 100, CompactConcurrency: 4, }, + logger: log.L(), } f.sstIngester = testIngester{} f.wg.Add(1) @@ -784,6 +788,7 @@ func testMergeSSTs(t *testing.T, kvs [][]common.KvPair, meta *sstMeta) { CompactThreshold: 100, CompactConcurrency: 4, }, + logger: log.L(), } createSSTWriter := func() (*sstWriter, error) { @@ -1182,6 +1187,7 @@ func TestMultiIngest(t *testing.T) { return importCli }, }, + logger: log.L(), } err := local.checkMultiIngestSupport(context.Background()) if err != nil { diff --git a/br/pkg/lightning/backend/local/localhelper.go b/br/pkg/lightning/backend/local/localhelper.go index c4aaae30db37b..9839e3592d195 100644 --- a/br/pkg/lightning/backend/local/localhelper.go +++ b/br/pkg/lightning/backend/local/localhelper.go @@ -111,7 +111,7 @@ func (local *local) SplitAndScatterRegionByRanges( waitTime := splitRegionBaseBackOffTime skippedKeys := 0 for i := 0; i < splitRetryTimes; i++ { - log.L().Info("split and scatter region", + log.FromContext(ctx).Info("split and scatter region", logutil.Key("minKey", minKey), logutil.Key("maxKey", maxKey), zap.Int("retry", i), @@ -130,15 +130,15 @@ func (local *local) SplitAndScatterRegionByRanges( } var regions []*split.RegionInfo regions, err = split.PaginateScanRegion(ctx, local.splitCli, minKey, maxKey, 128) - log.L().Info("paginate scan regions", zap.Int("count", len(regions)), + log.FromContext(ctx).Info("paginate scan regions", zap.Int("count", len(regions)), logutil.Key("start", minKey), logutil.Key("end", maxKey)) if err != nil { - log.L().Warn("paginate scan region failed", logutil.Key("minKey", minKey), logutil.Key("maxKey", maxKey), + log.FromContext(ctx).Warn("paginate scan region failed", logutil.Key("minKey", minKey), logutil.Key("maxKey", maxKey), log.ShortError(err), zap.Int("retry", i)) continue } - log.L().Info("paginate scan region finished", logutil.Key("minKey", minKey), logutil.Key("maxKey", maxKey), + log.FromContext(ctx).Info("paginate scan region finished", logutil.Key("minKey", minKey), logutil.Key("maxKey", maxKey), zap.Int("regions", len(regions))) if !needSplit { @@ -156,7 +156,7 @@ func (local *local) SplitAndScatterRegionByRanges( return beforeEnd(startKey, regions[i].Region.EndKey) }) if idx < 0 || idx >= len(regions) { - log.L().Error("target region not found", logutil.Key("start_key", startKey), + log.FromContext(ctx).Error("target region not found", logutil.Key("start_key", startKey), logutil.RegionBy("first_region", regions[0].Region), logutil.RegionBy("last_region", regions[len(regions)-1].Region)) return errors.New("target region not found") @@ -167,7 +167,7 @@ func (local *local) SplitAndScatterRegionByRanges( } ranges = needSplitRanges if len(ranges) == 0 { - log.L().Info("no ranges need to be split, skipped.") + log.FromContext(ctx).Info("no ranges need to be split, skipped.") return nil } @@ -175,7 +175,7 @@ func (local *local) SplitAndScatterRegionByRanges( if tableInfo != nil { tableRegionStats, err = fetchTableRegionSizeStats(ctx, db, tableInfo.ID) if err != nil { - log.L().Warn("fetch table region size statistics failed", + log.FromContext(ctx).Warn("fetch table region size statistics failed", zap.String("table", tableInfo.Name), zap.Error(err)) tableRegionStats = make(map[uint64]int64) } @@ -191,16 +191,16 @@ func (local *local) SplitAndScatterRegionByRanges( firstKeyEnc := codec.EncodeBytes([]byte{}, retryKeys[0]) lastKeyEnc := codec.EncodeBytes([]byte{}, retryKeys[len(retryKeys)-1]) if bytes.Compare(firstKeyEnc, regions[0].Region.StartKey) < 0 || !beforeEnd(lastKeyEnc, regions[len(regions)-1].Region.EndKey) { - log.L().Warn("no valid key for split region", + log.FromContext(ctx).Warn("no valid key for split region", logutil.Key("firstKey", firstKeyEnc), logutil.Key("lastKey", lastKeyEnc), logutil.Key("firstRegionStart", regions[0].Region.StartKey), logutil.Key("lastRegionEnd", regions[len(regions)-1].Region.EndKey)) return errors.New("check split keys failed") } - splitKeyMap = getSplitKeys(retryKeys, regions) + splitKeyMap = getSplitKeys(retryKeys, regions, log.FromContext(ctx)) retryKeys = retryKeys[:0] } else { - splitKeyMap = getSplitKeysByRanges(ranges, regions) + splitKeyMap = getSplitKeysByRanges(ranges, regions, log.FromContext(ctx)) } type splitInfo struct { @@ -233,7 +233,7 @@ func (local *local) SplitAndScatterRegionByRanges( splitRegionStart := codec.EncodeBytes([]byte{}, keys[startIdx]) splitRegionEnd := codec.EncodeBytes([]byte{}, keys[endIdx-1]) if bytes.Compare(splitRegionStart, splitRegion.Region.StartKey) < 0 || !beforeEnd(splitRegionEnd, splitRegion.Region.EndKey) { - log.L().Fatal("no valid key in region", + log.FromContext(ctx).Fatal("no valid key in region", logutil.Key("startKey", splitRegionStart), logutil.Key("endKey", splitRegionEnd), logutil.Key("regionStart", splitRegion.Region.StartKey), logutil.Key("regionEnd", splitRegion.Region.EndKey), logutil.Region(splitRegion.Region), logutil.Leader(splitRegion.Leader)) @@ -242,7 +242,7 @@ func (local *local) SplitAndScatterRegionByRanges( if err1 != nil { if strings.Contains(err1.Error(), "no valid key") { for _, key := range keys { - log.L().Warn("no valid key", + log.FromContext(ctx).Warn("no valid key", logutil.Key("startKey", region.Region.StartKey), logutil.Key("endKey", region.Region.EndKey), logutil.Key("key", codec.EncodeBytes([]byte{}, key))) @@ -252,7 +252,7 @@ func (local *local) SplitAndScatterRegionByRanges( // do not retry on context.Canceled error return err1 } - log.L().Warn("split regions", log.ShortError(err1), zap.Int("retry time", i), + log.FromContext(ctx).Warn("split regions", log.ShortError(err1), zap.Int("retry time", i), zap.Uint64("region_id", region.Region.Id)) syncLock.Lock() @@ -262,7 +262,7 @@ func (local *local) SplitAndScatterRegionByRanges( syncLock.Unlock() break } else { - log.L().Info("batch split region", zap.Uint64("region_id", splitRegion.Region.Id), + log.FromContext(ctx).Info("batch split region", zap.Uint64("region_id", splitRegion.Region.Id), zap.Int("keys", endIdx-startIdx), zap.Binary("firstKey", keys[startIdx]), zap.Binary("end", keys[endIdx-1])) sort.Slice(newRegions, func(i, j int) bool { @@ -294,7 +294,7 @@ func (local *local) SplitAndScatterRegionByRanges( // we can skip split it again. regionSize, ok := tableRegionStats[regionID] if !ok { - log.L().Warn("region stats not found", zap.Uint64("region", regionID)) + log.FromContext(ctx).Warn("region stats not found", zap.Uint64("region", regionID)) } if len(keys) == 1 && regionSize < regionSplitSize { skippedKeys++ @@ -341,11 +341,11 @@ func (local *local) SplitAndScatterRegionByRanges( scatterCount++ } if scatterCount == len(scatterRegions) { - log.L().Info("waiting for scattering regions done", + log.FromContext(ctx).Info("waiting for scattering regions done", zap.Int("skipped_keys", skippedKeys), zap.Int("regions", len(scatterRegions)), zap.Duration("take", time.Since(startTime))) } else { - log.L().Info("waiting for scattering regions timeout", + log.FromContext(ctx).Info("waiting for scattering regions timeout", zap.Int("skipped_keys", skippedKeys), zap.Int("scatterCount", scatterCount), zap.Int("regions", len(scatterRegions)), @@ -357,7 +357,7 @@ func (local *local) SplitAndScatterRegionByRanges( func fetchTableRegionSizeStats(ctx context.Context, db *sql.DB, tableID int64) (map[uint64]int64, error) { exec := &common.SQLWithRetry{ DB: db, - Logger: log.L(), + Logger: log.FromContext(ctx), } stats := make(map[uint64]int64) @@ -406,7 +406,7 @@ func (local *local) BatchSplitRegions(ctx context.Context, region *split.RegionI } // the scatter operation likely fails because region replicate not finish yet // pack them to one log to avoid printing a lot warn logs. - log.L().Warn("scatter region failed", zap.Int("regionCount", len(newRegions)), + log.FromContext(ctx).Warn("scatter region failed", zap.Int("regionCount", len(newRegions)), zap.Int("failedCount", len(retryRegions)), zap.Error(failedErr), zap.Int("retry", i)) scatterRegions = retryRegions retryRegions = make([]*split.RegionInfo, 0) @@ -433,7 +433,7 @@ func (local *local) waitForSplit(ctx context.Context, regionID uint64) { for i := 0; i < split.SplitCheckMaxRetryTimes; i++ { ok, err := local.hasRegion(ctx, regionID) if err != nil { - log.L().Info("wait for split failed", log.ShortError(err)) + log.FromContext(ctx).Info("wait for split failed", log.ShortError(err)) return } if ok { @@ -455,10 +455,10 @@ func (local *local) waitForScatterRegion(ctx context.Context, regionInfo *split. } if err != nil { if !common.IsRetryableError(err) { - log.L().Warn("wait for scatter region encountered non-retryable error", logutil.Region(regionInfo.Region), zap.Error(err)) + log.FromContext(ctx).Warn("wait for scatter region encountered non-retryable error", logutil.Region(regionInfo.Region), zap.Error(err)) return } - log.L().Warn("wait for scatter region encountered error, will retry again", logutil.Region(regionInfo.Region), zap.Error(err)) + log.FromContext(ctx).Warn("wait for scatter region encountered error, will retry again", logutil.Region(regionInfo.Region), zap.Error(err)) } select { case <-time.After(time.Second): @@ -497,13 +497,13 @@ func (local *local) checkScatterRegionFinishedOrReScatter(ctx context.Context, r case pdpb.OperatorStatus_SUCCESS: return true, nil default: - log.L().Warn("scatter-region operator status is abnormal, will scatter region again", + log.FromContext(ctx).Warn("scatter-region operator status is abnormal, will scatter region again", logutil.Region(regionInfo.Region), zap.Stringer("status", resp.GetStatus())) return false, local.splitCli.ScatterRegion(ctx, regionInfo) } } -func getSplitKeysByRanges(ranges []Range, regions []*split.RegionInfo) map[uint64][][]byte { +func getSplitKeysByRanges(ranges []Range, regions []*split.RegionInfo, logger log.Logger) map[uint64][][]byte { checkKeys := make([][]byte, 0) var lastEnd []byte for _, rg := range ranges { @@ -513,19 +513,19 @@ func getSplitKeysByRanges(ranges []Range, regions []*split.RegionInfo) map[uint6 checkKeys = append(checkKeys, rg.end) lastEnd = rg.end } - return getSplitKeys(checkKeys, regions) + return getSplitKeys(checkKeys, regions, logger) } -func getSplitKeys(checkKeys [][]byte, regions []*split.RegionInfo) map[uint64][][]byte { +func getSplitKeys(checkKeys [][]byte, regions []*split.RegionInfo, logger log.Logger) map[uint64][][]byte { splitKeyMap := make(map[uint64][][]byte) for _, key := range checkKeys { - if region := needSplit(key, regions); region != nil { + if region := needSplit(key, regions, logger); region != nil { splitKeys, ok := splitKeyMap[region.Region.GetId()] if !ok { splitKeys = make([][]byte, 0, 1) } splitKeyMap[region.Region.GetId()] = append(splitKeys, key) - log.L().Debug("get key for split region", + logger.Debug("get key for split region", zap.Binary("key", key), zap.Binary("startKey", region.Region.StartKey), zap.Binary("endKey", region.Region.EndKey)) @@ -535,7 +535,7 @@ func getSplitKeys(checkKeys [][]byte, regions []*split.RegionInfo) map[uint64][] } // needSplit checks whether a key is necessary to split, if true returns the split region -func needSplit(key []byte, regions []*split.RegionInfo) *split.RegionInfo { +func needSplit(key []byte, regions []*split.RegionInfo, logger log.Logger) *split.RegionInfo { // If splitKey is the max key. if len(key) == 0 { return nil @@ -548,7 +548,7 @@ func needSplit(key []byte, regions []*split.RegionInfo) *split.RegionInfo { if idx < len(regions) { // If splitKey is in a region if bytes.Compare(splitKey, regions[idx].Region.GetStartKey()) > 0 && beforeEnd(splitKey, regions[idx].Region.GetEndKey()) { - log.L().Debug("need split", + logger.Debug("need split", zap.Binary("splitKey", key), zap.Binary("encodedKey", splitKey), zap.Binary("region start", regions[idx].Region.GetStartKey()), diff --git a/br/pkg/lightning/backend/local/localhelper_test.go b/br/pkg/lightning/backend/local/localhelper_test.go index 767829e9c857f..8d3d367443ac8 100644 --- a/br/pkg/lightning/backend/local/localhelper_test.go +++ b/br/pkg/lightning/backend/local/localhelper_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/tidb/br/pkg/lightning/glue" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" @@ -418,6 +419,7 @@ func doTestBatchSplitRegionByRanges(ctx context.Context, t *testing.T, hook clie local := &local{ splitCli: client, g: glue.NewExternalTiDBGlue(nil, mysql.ModeNone), + logger: log.L(), } // current region ranges: [, aay), [aay, bba), [bba, bbh), [bbh, cca), [cca, ) @@ -586,6 +588,7 @@ func TestSplitAndScatterRegionInBatches(t *testing.T) { local := &local{ splitCli: client, g: glue.NewExternalTiDBGlue(nil, mysql.ModeNone), + logger: log.L(), } ctx, cancel := context.WithCancel(context.Background()) @@ -672,6 +675,7 @@ func doTestBatchSplitByRangesWithClusteredIndex(t *testing.T, hook clientHook) { local := &local{ splitCli: client, g: glue.NewExternalTiDBGlue(nil, mysql.ModeNone), + logger: log.L(), } ctx := context.Background() @@ -762,7 +766,7 @@ func TestNeedSplit(t *testing.T) { for hdl, idx := range checkMap { checkKey := tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(hdl)) - res := needSplit(checkKey, regions) + res := needSplit(checkKey, regions, log.L()) if idx < 0 { require.Nil(t, res) } else { diff --git a/br/pkg/lightning/backend/noop/noop.go b/br/pkg/lightning/backend/noop/noop.go index 2ac3e2b346dbb..93332e41b7074 100644 --- a/br/pkg/lightning/backend/noop/noop.go +++ b/br/pkg/lightning/backend/noop/noop.go @@ -67,7 +67,7 @@ func (b noopBackend) ShouldPostProcess() bool { } // NewEncoder creates an encoder of a TiDB table. -func (b noopBackend) NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { +func (b noopBackend) NewEncoder(ctx context.Context, tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { return noopEncoder{}, nil } diff --git a/br/pkg/lightning/backend/tidb/tidb.go b/br/pkg/lightning/backend/tidb/tidb.go index 9ae4564c68a37..1a9d100d39bd5 100644 --- a/br/pkg/lightning/backend/tidb/tidb.go +++ b/br/pkg/lightning/backend/tidb/tidb.go @@ -100,11 +100,11 @@ type tidbBackend struct { // // The backend does not take ownership of `db`. Caller should close `db` // manually after the backend expired. -func NewTiDBBackend(db *sql.DB, onDuplicate string, errorMgr *errormanager.ErrorManager) backend.Backend { +func NewTiDBBackend(ctx context.Context, db *sql.DB, onDuplicate string, errorMgr *errormanager.ErrorManager) backend.Backend { switch onDuplicate { case config.ReplaceOnDup, config.IgnoreOnDup, config.ErrorOnDup: default: - log.L().Warn("unsupported action on duplicate, overwrite with `replace`") + log.FromContext(ctx).Warn("unsupported action on duplicate, overwrite with `replace`") onDuplicate = config.ReplaceOnDup } return backend.MakeBackend(&tidbBackend{db: db, onDuplicate: onDuplicate, errorMgr: errorMgr}) @@ -354,12 +354,12 @@ func (enc *tidbEncoder) Encode(logger log.Logger, row []types.Datum, _ int64, co } // EncodeRowForRecord encodes a row to a string compatible with INSERT statements. -func EncodeRowForRecord(encTable table.Table, sqlMode mysql.SQLMode, row []types.Datum, columnPermutation []int) string { +func EncodeRowForRecord(ctx context.Context, encTable table.Table, sqlMode mysql.SQLMode, row []types.Datum, columnPermutation []int) string { enc := tidbEncoder{ tbl: encTable, mode: sqlMode, } - resRow, err := enc.Encode(log.L(), row, 0, columnPermutation, "", 0) + resRow, err := enc.Encode(log.FromContext(ctx), row, 0, columnPermutation, "", 0) if err != nil { // if encode can't succeed, fallback to record the raw input strings // ignore the error since it can only happen if the datum type is unknown, this can't happen here. @@ -394,12 +394,12 @@ func (be *tidbBackend) ShouldPostProcess() bool { } func (be *tidbBackend) CheckRequirements(ctx context.Context, _ *backend.CheckCtx) error { - log.L().Info("skipping check requirements for tidb backend") + log.FromContext(ctx).Info("skipping check requirements for tidb backend") return nil } -func (be *tidbBackend) NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { - se := kv.NewSession(options) +func (be *tidbBackend) NewEncoder(ctx context.Context, tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) { + se := kv.NewSession(options, log.FromContext(ctx)) if options.SQLMode.HasStrictMode() { se.GetSessionVars().SkipUTF8Check = false se.GetSessionVars().SkipASCIICheck = false @@ -554,7 +554,7 @@ func (be *tidbBackend) execStmts(ctx context.Context, stmtTasks []stmtTask, tabl _, err := be.db.ExecContext(ctx, stmt) if err != nil { if !common.IsContextCanceledError(err) { - log.L().Error("execute statement failed", + log.FromContext(ctx).Error("execute statement failed", zap.Array("rows", stmtTask.rows), zap.String("stmt", redact.String(stmt)), zap.Error(err)) } // It's batch mode, just return the error. @@ -566,7 +566,7 @@ func (be *tidbBackend) execStmts(ctx context.Context, stmtTasks []stmtTask, tabl continue } firstRow := stmtTask.rows[0] - err = be.errorMgr.RecordTypeError(ctx, log.L(), tableName, firstRow.path, firstRow.offset, firstRow.insertStmt, err) + err = be.errorMgr.RecordTypeError(ctx, log.FromContext(ctx), tableName, firstRow.path, firstRow.offset, firstRow.insertStmt, err) if err == nil { // max-error not yet reached (error consumed by errorMgr), proceed to next stmtTask. break @@ -587,7 +587,7 @@ func (be *tidbBackend) execStmts(ctx context.Context, stmtTasks []stmtTask, tabl func (be *tidbBackend) FetchRemoteTableModels(ctx context.Context, schemaName string) (tables []*model.TableInfo, err error) { s := common.SQLWithRetry{ DB: be.db, - Logger: log.L(), + Logger: log.FromContext(ctx), } err = s.Transact(ctx, "fetch table columns", func(c context.Context, tx *sql.Tx) error { diff --git a/br/pkg/lightning/backend/tidb/tidb_test.go b/br/pkg/lightning/backend/tidb/tidb_test.go index b7de642a26746..097c51dfdfe5f 100644 --- a/br/pkg/lightning/backend/tidb/tidb_test.go +++ b/br/pkg/lightning/backend/tidb/tidb_test.go @@ -63,7 +63,7 @@ func createMysqlSuite(t *testing.T) *mysqlSuite { tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic} tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), tblInfo) require.NoError(t, err) - backend := tidb.NewTiDBBackend(db, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig())) + backend := tidb.NewTiDBBackend(context.Background(), db, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig(), log.L())) return &mysqlSuite{dbHandle: db, mockDB: mock, backend: backend, tbl: tbl} } @@ -99,7 +99,7 @@ func TestWriteRowsReplaceOnDup(t *testing.T) { // skip column a,c due to ignore-columns perms[0] = -1 perms[2] = -1 - encoder, err := s.backend.NewEncoder(s.tbl, &kv.SessionOptions{SQLMode: 0, Timestamp: 1234567890}) + encoder, err := s.backend.NewEncoder(context.Background(), s.tbl, &kv.SessionOptions{SQLMode: 0, Timestamp: 1234567890}) require.NoError(t, err) row, err := encoder.Encode(logger, []types.Datum{ types.NewUintDatum(18446744073709551615), @@ -140,7 +140,7 @@ func TestWriteRowsIgnoreOnDup(t *testing.T) { ctx := context.Background() logger := log.L() - ignoreBackend := tidb.NewTiDBBackend(s.dbHandle, config.IgnoreOnDup, errormanager.New(nil, config.NewConfig())) + ignoreBackend := tidb.NewTiDBBackend(ctx, s.dbHandle, config.IgnoreOnDup, errormanager.New(nil, config.NewConfig(), logger)) engine, err := ignoreBackend.OpenEngine(ctx, &backend.EngineConfig{}, "`foo`.`bar`", 1) require.NoError(t, err) @@ -149,7 +149,7 @@ func TestWriteRowsIgnoreOnDup(t *testing.T) { indexRows := ignoreBackend.MakeEmptyRows() indexChecksum := verification.MakeKVChecksum(0, 0, 0) - encoder, err := ignoreBackend.NewEncoder(s.tbl, &kv.SessionOptions{}) + encoder, err := ignoreBackend.NewEncoder(ctx, s.tbl, &kv.SessionOptions{}) require.NoError(t, err) row, err := encoder.Encode(logger, []types.Datum{ types.NewIntDatum(1), @@ -165,7 +165,7 @@ func TestWriteRowsIgnoreOnDup(t *testing.T) { require.NoError(t, err) // test encode rows with _tidb_rowid - encoder, err = ignoreBackend.NewEncoder(s.tbl, &kv.SessionOptions{}) + encoder, err = ignoreBackend.NewEncoder(ctx, s.tbl, &kv.SessionOptions{}) require.NoError(t, err) rowWithID, err := encoder.Encode(logger, []types.Datum{ types.NewIntDatum(1), @@ -186,7 +186,7 @@ func TestWriteRowsErrorOnDup(t *testing.T) { ctx := context.Background() logger := log.L() - ignoreBackend := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) + ignoreBackend := tidb.NewTiDBBackend(ctx, s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig(), logger)) engine, err := ignoreBackend.OpenEngine(ctx, &backend.EngineConfig{}, "`foo`.`bar`", 1) require.NoError(t, err) @@ -195,7 +195,7 @@ func TestWriteRowsErrorOnDup(t *testing.T) { indexRows := ignoreBackend.MakeEmptyRows() indexChecksum := verification.MakeKVChecksum(0, 0, 0) - encoder, err := ignoreBackend.NewEncoder(s.tbl, &kv.SessionOptions{}) + encoder, err := ignoreBackend.NewEncoder(ctx, s.tbl, &kv.SessionOptions{}) require.NoError(t, err) row, err := encoder.Encode(logger, []types.Datum{ types.NewIntDatum(1), @@ -228,8 +228,10 @@ func testStrictMode(t *testing.T) { tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), tblInfo) require.NoError(t, err) - bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) - encoder, err := bk.NewEncoder(tbl, &kv.SessionOptions{SQLMode: mysql.ModeStrictAllTables}) + ctx := context.Background() + + bk := tidb.NewTiDBBackend(ctx, s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig(), log.L())) + encoder, err := bk.NewEncoder(ctx, tbl, &kv.SessionOptions{SQLMode: mysql.ModeStrictAllTables}) require.NoError(t, err) logger := log.L() @@ -245,7 +247,7 @@ func testStrictMode(t *testing.T) { require.Regexp(t, `incorrect utf8 value .* for column s0$`, err.Error()) // oepn a new encode because column count changed. - encoder, err = bk.NewEncoder(tbl, &kv.SessionOptions{SQLMode: mysql.ModeStrictAllTables}) + encoder, err = bk.NewEncoder(ctx, tbl, &kv.SessionOptions{SQLMode: mysql.ModeStrictAllTables}) require.NoError(t, err) _, err = encoder.Encode(logger, []types.Datum{ types.NewStringDatum(""), @@ -267,7 +269,7 @@ func TestFetchRemoteTableModels_3_x(t *testing.T) { AddRow("t", "id", "int(10)", "", "auto_increment")) s.mockDB.ExpectCommit() - bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) + bk := tidb.NewTiDBBackend(context.Background(), s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig(), log.L())) tableInfos, err := bk.FetchRemoteTableModels(context.Background(), "test") require.NoError(t, err) ft := types.FieldType{} @@ -304,7 +306,7 @@ func TestFetchRemoteTableModels_4_0(t *testing.T) { AddRow("test", "t", "id", int64(1))) s.mockDB.ExpectCommit() - bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) + bk := tidb.NewTiDBBackend(context.Background(), s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig(), log.L())) tableInfos, err := bk.FetchRemoteTableModels(context.Background(), "test") require.NoError(t, err) ft := types.FieldType{} @@ -341,7 +343,7 @@ func TestFetchRemoteTableModels_4_x_auto_increment(t *testing.T) { AddRow("test", "t", "id", int64(1), "AUTO_INCREMENT")) s.mockDB.ExpectCommit() - bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) + bk := tidb.NewTiDBBackend(context.Background(), s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig(), log.L())) tableInfos, err := bk.FetchRemoteTableModels(context.Background(), "test") require.NoError(t, err) ft := types.FieldType{} @@ -378,7 +380,7 @@ func TestFetchRemoteTableModels_4_x_auto_random(t *testing.T) { AddRow("test", "t", "id", int64(1), "AUTO_RANDOM")) s.mockDB.ExpectCommit() - bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) + bk := tidb.NewTiDBBackend(context.Background(), s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig(), log.L())) tableInfos, err := bk.FetchRemoteTableModels(context.Background(), "test") require.NoError(t, err) ft := types.FieldType{} @@ -413,8 +415,8 @@ func TestWriteRowsErrorNoRetry(t *testing.T) { WillReturnError(nonRetryableError) // disable error record, should not expect retry statements one by one. - ignoreBackend := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, - errormanager.New(s.dbHandle, &config.Config{}), + ignoreBackend := tidb.NewTiDBBackend(context.Background(), s.dbHandle, config.ErrorOnDup, + errormanager.New(s.dbHandle, &config.Config{}, log.L()), ) dataRows := encodeRowsTiDB(t, ignoreBackend, s.tbl) ctx := context.Background() @@ -473,7 +475,7 @@ func TestWriteRowsErrorDowngradingAll(t *testing.T) { WillReturnResult(driver.ResultNoRows) // disable error record, should not expect retry statements one by one. - ignoreBackend := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, + ignoreBackend := tidb.NewTiDBBackend(context.Background(), s.dbHandle, config.ErrorOnDup, errormanager.New(s.dbHandle, &config.Config{ App: config.Lightning{ TaskInfoSchemaName: "tidb_lightning_errors", @@ -481,7 +483,7 @@ func TestWriteRowsErrorDowngradingAll(t *testing.T) { Type: *atomic.NewInt64(10), }, }, - }), + }, log.L()), ) dataRows := encodeRowsTiDB(t, ignoreBackend, s.tbl) ctx := context.Background() @@ -528,7 +530,7 @@ func TestWriteRowsErrorDowngradingExceedThreshold(t *testing.T) { ExpectExec("\\QINSERT INTO `foo`.`bar`(`a`) VALUES(4)\\E"). WillReturnError(nonRetryableError) - ignoreBackend := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, + ignoreBackend := tidb.NewTiDBBackend(context.Background(), s.dbHandle, config.ErrorOnDup, errormanager.New(s.dbHandle, &config.Config{ App: config.Lightning{ TaskInfoSchemaName: "tidb_lightning_errors", @@ -536,7 +538,7 @@ func TestWriteRowsErrorDowngradingExceedThreshold(t *testing.T) { Type: *atomic.NewInt64(3), }, }, - }), + }, log.L()), ) dataRows := encodeRowsTiDB(t, ignoreBackend, s.tbl) ctx := context.Background() @@ -558,7 +560,7 @@ func encodeRowsTiDB(t *testing.T, b backend.Backend, tbl table.Table) kv.Rows { indexChecksum := verification.MakeKVChecksum(0, 0, 0) logger := log.L() - encoder, err := b.NewEncoder(tbl, &kv.SessionOptions{}) + encoder, err := b.NewEncoder(context.Background(), tbl, &kv.SessionOptions{}) require.NoError(t, err) row, err := encoder.Encode(logger, []types.Datum{ types.NewIntDatum(1), @@ -609,7 +611,7 @@ func TestEncodeRowForRecord(t *testing.T) { s := createMysqlSuite(t) // for a correct row, the will encode a correct result - row := tidb.EncodeRowForRecord(s.tbl, mysql.ModeStrictTransTables, []types.Datum{ + row := tidb.EncodeRowForRecord(context.Background(), s.tbl, mysql.ModeStrictTransTables, []types.Datum{ types.NewIntDatum(5), types.NewStringDatum("test test"), types.NewBinaryLiteralDatum(types.NewBinaryLiteralFromUint(0xabcdef, 6)), @@ -618,7 +620,7 @@ func TestEncodeRowForRecord(t *testing.T) { // the following row will result in column count mismatch error, there for encode // result will fallback to a "," separated string list. - row = tidb.EncodeRowForRecord(s.tbl, mysql.ModeStrictTransTables, []types.Datum{ + row = tidb.EncodeRowForRecord(context.Background(), s.tbl, mysql.ModeStrictTransTables, []types.Datum{ types.NewIntDatum(5), types.NewStringDatum("test test"), types.NewBinaryLiteralDatum(types.NewBinaryLiteralFromUint(0xabcdef, 6)), diff --git a/br/pkg/lightning/checkpoints/checkpoints.go b/br/pkg/lightning/checkpoints/checkpoints.go index 36cefed180ae3..30ab72b0298f0 100644 --- a/br/pkg/lightning/checkpoints/checkpoints.go +++ b/br/pkg/lightning/checkpoints/checkpoints.go @@ -493,7 +493,7 @@ type DB interface { // It assumes the entire table has not been imported before and will fill in // default values for the column permutations and checksums. InsertEngineCheckpoints(ctx context.Context, tableName string, checkpoints map[int32]*EngineCheckpoint) error - Update(checkpointDiffs map[string]*TableCheckpointDiff) error + Update(taskCtx context.Context, checkpointDiffs map[string]*TableCheckpointDiff) error RemoveCheckpoint(ctx context.Context, tableName string) error // MoveCheckpoints renames the checkpoint schema to include a suffix @@ -608,7 +608,7 @@ func (*NullCheckpointsDB) InsertEngineCheckpoints(_ context.Context, _ string, _ return nil } -func (*NullCheckpointsDB) Update(map[string]*TableCheckpointDiff) error { +func (*NullCheckpointsDB) Update(context.Context, map[string]*TableCheckpointDiff) error { return nil } @@ -621,7 +621,7 @@ func NewMySQLCheckpointsDB(ctx context.Context, db *sql.DB, schemaName string) ( schema := common.EscapeIdentifier(schemaName) sql := common.SQLWithRetry{ DB: db, - Logger: log.With(zap.String("schema", schemaName)), + Logger: log.FromContext(ctx).With(zap.String("schema", schemaName)), HideQueryLog: true, } err := sql.Exec(ctx, "create checkpoints database", fmt.Sprintf(CreateDBTemplate, schema)) @@ -658,7 +658,7 @@ func NewMySQLCheckpointsDB(ctx context.Context, db *sql.DB, schemaName string) ( func (cpdb *MySQLCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, dbInfo map[string]*TidbDBInfo) error { // We can have at most 65535 placeholders https://stackoverflow.com/q/4922345/ // Since this step is not performance critical, we just insert the rows one-by-one. - s := common.SQLWithRetry{DB: cpdb.db, Logger: log.L()} + s := common.SQLWithRetry{DB: cpdb.db, Logger: log.FromContext(ctx)} err := s.Transact(ctx, "insert checkpoints", func(c context.Context, tx *sql.Tx) error { taskStmt, err := tx.PrepareContext(c, fmt.Sprintf(InitTaskTemplate, cpdb.schema, CheckpointTableNameTask)) if err != nil { @@ -706,7 +706,7 @@ func (cpdb *MySQLCheckpointsDB) Initialize(ctx context.Context, cfg *config.Conf func (cpdb *MySQLCheckpointsDB) TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error) { s := common.SQLWithRetry{ DB: cpdb.db, - Logger: log.L(), + Logger: log.FromContext(ctx), } taskQuery := fmt.Sprintf(ReadTaskTemplate, cpdb.schema, CheckpointTableNameTask) @@ -735,7 +735,7 @@ func (cpdb *MySQLCheckpointsDB) Get(ctx context.Context, tableName string) (*Tab s := common.SQLWithRetry{ DB: cpdb.db, - Logger: log.With(zap.String("table", tableName)), + Logger: log.FromContext(ctx).With(zap.String("table", tableName)), } err := s.Transact(ctx, "read checkpoint", func(c context.Context, tx *sql.Tx) error { // 1. Populate the engines. @@ -824,7 +824,7 @@ func (cpdb *MySQLCheckpointsDB) Get(ctx context.Context, tableName string) (*Tab func (cpdb *MySQLCheckpointsDB) InsertEngineCheckpoints(ctx context.Context, tableName string, checkpoints map[int32]*EngineCheckpoint) error { s := common.SQLWithRetry{ DB: cpdb.db, - Logger: log.With(zap.String("table", tableName)), + Logger: log.FromContext(ctx).With(zap.String("table", tableName)), } err := s.Transact(ctx, "update engine checkpoints", func(c context.Context, tx *sql.Tx) error { engineStmt, err := tx.PrepareContext(c, fmt.Sprintf(ReplaceEngineTemplate, cpdb.schema, CheckpointTableNameEngine)) @@ -870,15 +870,15 @@ func (cpdb *MySQLCheckpointsDB) InsertEngineCheckpoints(ctx context.Context, tab return nil } -func (cpdb *MySQLCheckpointsDB) Update(checkpointDiffs map[string]*TableCheckpointDiff) error { +func (cpdb *MySQLCheckpointsDB) Update(taskCtx context.Context, checkpointDiffs map[string]*TableCheckpointDiff) error { chunkQuery := fmt.Sprintf(UpdateChunkTemplate, cpdb.schema, CheckpointTableNameChunk) rebaseQuery := fmt.Sprintf(UpdateTableRebaseTemplate, cpdb.schema, CheckpointTableNameTable) tableStatusQuery := fmt.Sprintf(UpdateTableStatusTemplate, cpdb.schema, CheckpointTableNameTable) tableChecksumQuery := fmt.Sprintf(UpdateTableChecksumTemplate, cpdb.schema, CheckpointTableNameTable) engineStatusQuery := fmt.Sprintf(UpdateEngineTemplate, cpdb.schema, CheckpointTableNameEngine) - s := common.SQLWithRetry{DB: cpdb.db, Logger: log.L()} - return s.Transact(context.Background(), "update checkpoints", func(c context.Context, tx *sql.Tx) error { + s := common.SQLWithRetry{DB: cpdb.db, Logger: log.FromContext(taskCtx)} + return s.Transact(taskCtx, "update checkpoints", func(c context.Context, tx *sql.Tx) error { chunkStmt, e := tx.PrepareContext(c, chunkQuery) if e != nil { return errors.Trace(e) @@ -981,7 +981,7 @@ func newFileCheckpointsDB( return nil, errors.Trace(err) } if !exist { - log.L().Info("open checkpoint file failed, going to create a new one", + log.FromContext(ctx).Info("open checkpoint file failed, going to create a new one", zap.String("path", path), log.ShortError(err), ) @@ -993,7 +993,7 @@ func newFileCheckpointsDB( } err = cpdb.checkpoints.Unmarshal(content) if err != nil { - log.L().Error("checkpoint file is broken", zap.String("path", path), zap.Error(err)) + log.FromContext(ctx).Error("checkpoint file is broken", zap.String("path", path), zap.Error(err)) } // FIXME: patch for empty map may need initialize manually, because currently // FIXME: a map of zero size -> marshall -> unmarshall -> become nil, see checkpoint_test.go @@ -1253,7 +1253,7 @@ func (cpdb *FileCheckpointsDB) InsertEngineCheckpoints(_ context.Context, tableN return errors.Trace(cpdb.save()) } -func (cpdb *FileCheckpointsDB) Update(checkpointDiffs map[string]*TableCheckpointDiff) error { +func (cpdb *FileCheckpointsDB) Update(_ context.Context, checkpointDiffs map[string]*TableCheckpointDiff) error { cpdb.lock.Lock() defer cpdb.lock.Unlock() @@ -1330,7 +1330,7 @@ func (*NullCheckpointsDB) DumpChunks(context.Context, io.Writer) error { func (cpdb *MySQLCheckpointsDB) RemoveCheckpoint(ctx context.Context, tableName string) error { s := common.SQLWithRetry{ DB: cpdb.db, - Logger: log.With(zap.String("table", tableName)), + Logger: log.FromContext(ctx).With(zap.String("table", tableName)), } if tableName == allTables { @@ -1362,7 +1362,7 @@ func (cpdb *MySQLCheckpointsDB) MoveCheckpoints(ctx context.Context, taskID int6 newSchema := fmt.Sprintf("`%s.%d.bak`", cpdb.schema[1:len(cpdb.schema)-1], taskID) s := common.SQLWithRetry{ DB: cpdb.db, - Logger: log.With(zap.Int64("taskID", taskID)), + Logger: log.FromContext(ctx).With(zap.Int64("taskID", taskID)), } createSchemaQuery := "CREATE SCHEMA IF NOT EXISTS " + newSchema @@ -1402,7 +1402,7 @@ func (cpdb *MySQLCheckpointsDB) GetLocalStoringTables(ctx context.Context) (map[ CheckpointStatusMaxInvalid, CheckpointStatusIndexImported, CheckpointStatusMaxInvalid, CheckpointStatusImported) - err := common.Retry("get local storing tables", log.L(), func() error { + err := common.Retry("get local storing tables", log.FromContext(ctx), func() error { targetTables = make(map[string][]int32) rows, err := cpdb.db.QueryContext(ctx, query) // #nosec G201 if err != nil { @@ -1453,7 +1453,7 @@ func (cpdb *MySQLCheckpointsDB) IgnoreErrorCheckpoint(ctx context.Context, table s := common.SQLWithRetry{ DB: cpdb.db, - Logger: log.With(zap.String("table", tableName)), + Logger: log.FromContext(ctx).With(zap.String("table", tableName)), } err := s.Transact(ctx, "ignore error checkpoints", func(c context.Context, tx *sql.Tx) error { if _, e := tx.ExecContext(c, engineQuery, tableName); e != nil { @@ -1510,7 +1510,7 @@ func (cpdb *MySQLCheckpointsDB) DestroyErrorCheckpoint(ctx context.Context, tabl s := common.SQLWithRetry{ DB: cpdb.db, - Logger: log.With(zap.String("table", tableName)), + Logger: log.FromContext(ctx).With(zap.String("table", tableName)), } err := s.Transact(ctx, "destroy error checkpoints", func(c context.Context, tx *sql.Tx) error { // Obtain the list of tables diff --git a/br/pkg/lightning/checkpoints/checkpoints_file_test.go b/br/pkg/lightning/checkpoints/checkpoints_file_test.go index 5911b9952a1c2..824434cb78f82 100644 --- a/br/pkg/lightning/checkpoints/checkpoints_file_test.go +++ b/br/pkg/lightning/checkpoints/checkpoints_file_test.go @@ -119,7 +119,7 @@ func newFileCheckpointsDB(t *testing.T) (*checkpoints.FileCheckpointsDB, func()) } ccm.MergeInto(cpd) - cpdb.Update(map[string]*checkpoints.TableCheckpointDiff{"`db1`.`t2`": cpd}) + cpdb.Update(ctx, map[string]*checkpoints.TableCheckpointDiff{"`db1`.`t2`": cpd}) return cpdb, func() { err := cpdb.Close() require.NoError(t, err) @@ -135,7 +135,7 @@ func setInvalidStatus(cpdb *checkpoints.FileCheckpointsDB) { scm.SetInvalid() scm.MergeInto(cpd) - cpdb.Update(map[string]*checkpoints.TableCheckpointDiff{ + cpdb.Update(context.Background(), map[string]*checkpoints.TableCheckpointDiff{ "`db1`.`t2`": cpd, "`db2`.`t3`": cpd, }) diff --git a/br/pkg/lightning/checkpoints/checkpoints_sql_test.go b/br/pkg/lightning/checkpoints/checkpoints_sql_test.go index 85a23f379fec9..a0f59ab771915 100644 --- a/br/pkg/lightning/checkpoints/checkpoints_sql_test.go +++ b/br/pkg/lightning/checkpoints/checkpoints_sql_test.go @@ -221,7 +221,7 @@ func TestNormalOperations(t *testing.T) { s.mock.ExpectCommit() s.mock.MatchExpectationsInOrder(false) - cpdb.Update(map[string]*checkpoints.TableCheckpointDiff{"`db1`.`t2`": cpd}) + cpdb.Update(ctx, map[string]*checkpoints.TableCheckpointDiff{"`db1`.`t2`": cpd}) s.mock.MatchExpectationsInOrder(true) require.Nil(t, s.mock.ExpectationsWereMet()) diff --git a/br/pkg/lightning/checkpoints/glue_checkpoint.go b/br/pkg/lightning/checkpoints/glue_checkpoint.go index 30b540426f2a7..b0f5278c7e89a 100644 --- a/br/pkg/lightning/checkpoints/glue_checkpoint.go +++ b/br/pkg/lightning/checkpoints/glue_checkpoint.go @@ -59,9 +59,9 @@ var _ DB = (*GlueCheckpointsDB)(nil) // dropPreparedStmt drops the statement and when meet an error, // print an error message. -func dropPreparedStmt(session Session, stmtID uint32) { +func dropPreparedStmt(ctx context.Context, session Session, stmtID uint32) { if err := session.DropPreparedStmt(stmtID); err != nil { - log.L().Error("failed to drop prepared statement", log.ShortError(err)) + log.FromContext(ctx).Error("failed to drop prepared statement", log.ShortError(err)) } } @@ -69,7 +69,7 @@ func NewGlueCheckpointsDB(ctx context.Context, se Session, f func() (Session, er var escapedSchemaName strings.Builder common.WriteMySQLIdentifier(&escapedSchemaName, schemaName) schema := escapedSchemaName.String() - logger := log.With(zap.String("schema", schemaName)) + logger := log.FromContext(ctx).With(zap.String("schema", schemaName)) sql := fmt.Sprintf(CreateDBTemplate, schema) err := common.Retry("create checkpoints database", logger, func() error { @@ -123,7 +123,7 @@ func NewGlueCheckpointsDB(ctx context.Context, se Session, f func() (Session, er } func (g GlueCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, dbInfo map[string]*TidbDBInfo) error { - logger := log.L() + logger := log.FromContext(ctx) se, err := g.getSessionFunc() if err != nil { return errors.Trace(err) @@ -135,7 +135,7 @@ func (g GlueCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, d if err != nil { return errors.Trace(err) } - defer dropPreparedStmt(s, stmtID) + defer dropPreparedStmt(ctx, s, stmtID) _, err = s.ExecutePreparedStmt(c, stmtID, []types.Datum{ types.NewIntDatum(cfg.TaskID), types.NewStringDatum(cfg.Mydumper.SourceDir), @@ -155,7 +155,7 @@ func (g GlueCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, d if err != nil { return errors.Trace(err) } - defer dropPreparedStmt(s, stmtID2) + defer dropPreparedStmt(ctx, s, stmtID2) for _, db := range dbInfo { for _, table := range db.Tables { @@ -177,7 +177,7 @@ func (g GlueCheckpointsDB) Initialize(ctx context.Context, cfg *config.Config, d } func (g GlueCheckpointsDB) TaskCheckpoint(ctx context.Context) (*TaskCheckpoint, error) { - logger := log.L() + logger := log.FromContext(ctx) sql := fmt.Sprintf(ReadTaskTemplate, g.schema, CheckpointTableNameTask) se, err := g.getSessionFunc() if err != nil { @@ -225,7 +225,7 @@ func (g GlueCheckpointsDB) Get(ctx context.Context, tableName string) (*TableChe cp := &TableCheckpoint{ Engines: map[int32]*EngineCheckpoint{}, } - logger := log.With(zap.String("table", tableName)) + logger := log.FromContext(ctx).With(zap.String("table", tableName)) se, err := g.getSessionFunc() if err != nil { return nil, errors.Trace(err) @@ -351,7 +351,7 @@ func (g GlueCheckpointsDB) Close() error { } func (g GlueCheckpointsDB) InsertEngineCheckpoints(ctx context.Context, tableName string, checkpointMap map[int32]*EngineCheckpoint) error { - logger := log.With(zap.String("table", tableName)) + logger := log.FromContext(ctx).With(zap.String("table", tableName)) se, err := g.getSessionFunc() if err != nil { return errors.Trace(err) @@ -363,13 +363,13 @@ func (g GlueCheckpointsDB) InsertEngineCheckpoints(ctx context.Context, tableNam if err != nil { return errors.Trace(err) } - defer dropPreparedStmt(s, engineStmt) + defer dropPreparedStmt(ctx, s, engineStmt) chunkStmt, _, _, err := s.PrepareStmt(fmt.Sprintf(ReplaceChunkTemplate, g.schema, CheckpointTableNameChunk)) if err != nil { return errors.Trace(err) } - defer dropPreparedStmt(s, chunkStmt) + defer dropPreparedStmt(ctx, s, chunkStmt) for engineID, engine := range checkpointMap { _, err := s.ExecutePreparedStmt(c, engineStmt, []types.Datum{ @@ -411,11 +411,11 @@ func (g GlueCheckpointsDB) InsertEngineCheckpoints(ctx context.Context, tableNam return errors.Trace(err) } -func (g GlueCheckpointsDB) Update(checkpointDiffs map[string]*TableCheckpointDiff) error { - logger := log.L() +func (g GlueCheckpointsDB) Update(ctx context.Context, checkpointDiffs map[string]*TableCheckpointDiff) error { + logger := log.FromContext(ctx) se, err := g.getSessionFunc() if err != nil { - log.L().Error("can't get a session to update GlueCheckpointsDB", zap.Error(errors.Trace(err))) + log.FromContext(ctx).Error("can't get a session to update GlueCheckpointsDB", zap.Error(errors.Trace(err))) return err } defer se.Close() @@ -429,22 +429,22 @@ func (g GlueCheckpointsDB) Update(checkpointDiffs map[string]*TableCheckpointDif if err != nil { return errors.Trace(err) } - defer dropPreparedStmt(s, chunkStmt) + defer dropPreparedStmt(ctx, s, chunkStmt) rebaseStmt, _, _, err := s.PrepareStmt(rebaseQuery) if err != nil { return errors.Trace(err) } - defer dropPreparedStmt(s, rebaseStmt) + defer dropPreparedStmt(ctx, s, rebaseStmt) tableStatusStmt, _, _, err := s.PrepareStmt(tableStatusQuery) if err != nil { return errors.Trace(err) } - defer dropPreparedStmt(s, tableStatusStmt) + defer dropPreparedStmt(ctx, s, tableStatusStmt) engineStatusStmt, _, _, err := s.PrepareStmt(engineStatusQuery) if err != nil { return errors.Trace(err) } - defer dropPreparedStmt(s, engineStatusStmt) + defer dropPreparedStmt(ctx, s, engineStatusStmt) for tableName, cpd := range checkpointDiffs { if cpd.hasStatus { @@ -504,7 +504,7 @@ func (g GlueCheckpointsDB) Update(checkpointDiffs map[string]*TableCheckpointDif } func (g GlueCheckpointsDB) RemoveCheckpoint(ctx context.Context, tableName string) error { - logger := log.With(zap.String("table", tableName)) + logger := log.FromContext(ctx).With(zap.String("table", tableName)) se, err := g.getSessionFunc() if err != nil { return errors.Trace(err) @@ -541,7 +541,7 @@ func (g GlueCheckpointsDB) RemoveCheckpoint(ctx context.Context, tableName strin func (g GlueCheckpointsDB) MoveCheckpoints(ctx context.Context, taskID int64) error { newSchema := fmt.Sprintf("`%s.%d.bak`", g.schema[1:len(g.schema)-1], taskID) - logger := log.With(zap.Int64("taskID", taskID)) + logger := log.FromContext(ctx).With(zap.Int64("taskID", taskID)) se, err := g.getSessionFunc() if err != nil { return errors.Trace(err) @@ -596,7 +596,7 @@ func (g GlueCheckpointsDB) GetLocalStoringTables(ctx context.Context) (map[strin CheckpointStatusMaxInvalid, CheckpointStatusIndexImported, CheckpointStatusMaxInvalid, CheckpointStatusImported) - err = common.Retry("get local storing tables", log.L(), func() error { + err = common.Retry("get local storing tables", log.FromContext(ctx), func() error { targetTables = make(map[string][]int32) rs, err := se.Execute(ctx, query) if err != nil { @@ -622,7 +622,7 @@ func (g GlueCheckpointsDB) GetLocalStoringTables(ctx context.Context) (map[strin } func (g GlueCheckpointsDB) IgnoreErrorCheckpoint(ctx context.Context, tableName string) error { - logger := log.With(zap.String("table", tableName)) + logger := log.FromContext(ctx).With(zap.String("table", tableName)) se, err := g.getSessionFunc() if err != nil { return errors.Trace(err) @@ -658,7 +658,7 @@ func (g GlueCheckpointsDB) IgnoreErrorCheckpoint(ctx context.Context, tableName } func (g GlueCheckpointsDB) DestroyErrorCheckpoint(ctx context.Context, tableName string) ([]DestroyedTableCheckpoint, error) { - logger := log.With(zap.String("table", tableName)) + logger := log.FromContext(ctx).With(zap.String("table", tableName)) se, err := g.getSessionFunc() if err != nil { return nil, errors.Trace(err) diff --git a/br/pkg/lightning/common/conn.go b/br/pkg/lightning/common/conn.go index eb9b598e64c55..fcf2aaa55fbf3 100644 --- a/br/pkg/lightning/common/conn.go +++ b/br/pkg/lightning/common/conn.go @@ -34,6 +34,7 @@ type ConnPool struct { next int cap int newConn func(ctx context.Context) (*grpc.ClientConn, error) + logger log.Logger } func (p *ConnPool) TakeConns() (conns []*grpc.ClientConn) { @@ -48,7 +49,7 @@ func (p *ConnPool) TakeConns() (conns []*grpc.ClientConn) { func (p *ConnPool) Close() { for _, c := range p.TakeConns() { if err := c.Close(); err != nil { - log.L().Warn("failed to close clientConn", zap.String("target", c.Target()), log.ShortError(err)) + p.logger.Warn("failed to close clientConn", zap.String("target", c.Target()), log.ShortError(err)) } } } @@ -72,13 +73,12 @@ func (p *ConnPool) get(ctx context.Context) (*grpc.ClientConn, error) { } // NewConnPool creates a new connPool by the specified conn factory function and capacity. -func NewConnPool(capacity int, newConn func(ctx context.Context) (*grpc.ClientConn, error)) *ConnPool { +func NewConnPool(capacity int, newConn func(ctx context.Context) (*grpc.ClientConn, error), logger log.Logger) *ConnPool { return &ConnPool{ cap: capacity, conns: make([]*grpc.ClientConn, 0, capacity), newConn: newConn, - - mu: sync.Mutex{}, + logger: logger, } } @@ -100,7 +100,7 @@ func (conns *GRPCConns) GetGrpcConn(ctx context.Context, storeID uint64, tcpConc conns.mu.Lock() defer conns.mu.Unlock() if _, ok := conns.conns[storeID]; !ok { - conns.conns[storeID] = NewConnPool(tcpConcurrency, newConn) + conns.conns[storeID] = NewConnPool(tcpConcurrency, newConn, log.FromContext(ctx)) } return conns.conns[storeID].get(ctx) } diff --git a/br/pkg/lightning/errormanager/errormanager.go b/br/pkg/lightning/errormanager/errormanager.go index 965191373688e..c663839f3cf35 100644 --- a/br/pkg/lightning/errormanager/errormanager.go +++ b/br/pkg/lightning/errormanager/errormanager.go @@ -122,6 +122,7 @@ type ErrorManager struct { configError *config.MaxError remainingError config.MaxError dupResolution config.DuplicateResolutionAlgorithm + logger log.Logger } func (em *ErrorManager) TypeErrorsRemain() int64 { @@ -129,12 +130,13 @@ func (em *ErrorManager) TypeErrorsRemain() int64 { } // New creates a new error manager. -func New(db *sql.DB, cfg *config.Config) *ErrorManager { +func New(db *sql.DB, cfg *config.Config, logger log.Logger) *ErrorManager { em := &ErrorManager{ taskID: cfg.TaskID, configError: &cfg.App.MaxError, remainingError: cfg.App.MaxError, dupResolution: cfg.TikvImporter.DuplicateResolution, + logger: logger, } if len(cfg.App.TaskInfoSchemaName) != 0 { em.db = db @@ -151,7 +153,7 @@ func (em *ErrorManager) Init(ctx context.Context) error { exec := common.SQLWithRetry{ DB: em.db, - Logger: log.L(), + Logger: em.logger, } sqls := make([][2]string, 0) @@ -456,17 +458,17 @@ func (em *ErrorManager) LogErrorDetails() { cnt, errType, em.fmtTableName(tblName)) } if errCnt := em.typeErrors(); errCnt > 0 { - log.L().Warn(fmtErrMsg(errCnt, "data type", typeErrorTableName)) + em.logger.Warn(fmtErrMsg(errCnt, "data type", typeErrorTableName)) } if errCnt := em.syntaxError(); errCnt > 0 { - log.L().Warn(fmtErrMsg(errCnt, "data type", syntaxErrorTableName)) + em.logger.Warn(fmtErrMsg(errCnt, "data type", syntaxErrorTableName)) } if errCnt := em.charsetError(); errCnt > 0 { // TODO: add charset table name - log.L().Warn(fmtErrMsg(errCnt, "data type", "")) + em.logger.Warn(fmtErrMsg(errCnt, "data type", "")) } if errCnt := em.conflictError(); errCnt > 0 { - log.L().Warn(fmtErrMsg(errCnt, "data type", conflictErrorTableName)) + em.logger.Warn(fmtErrMsg(errCnt, "data type", conflictErrorTableName)) } } diff --git a/br/pkg/lightning/errormanager/errormanager_test.go b/br/pkg/lightning/errormanager/errormanager_test.go index 63446cdd4c573..38f81b51f0299 100644 --- a/br/pkg/lightning/errormanager/errormanager_test.go +++ b/br/pkg/lightning/errormanager/errormanager_test.go @@ -25,6 +25,7 @@ import ( "testing" "github.com/DATA-DOG/go-sqlmock" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/stretchr/testify/require" "go.uber.org/atomic" @@ -41,7 +42,7 @@ func TestInit(t *testing.T) { cfg.App.MaxError.Type.Store(10) cfg.App.TaskInfoSchemaName = "lightning_errors" - em := New(db, cfg) + em := New(db, cfg, log.L()) require.Equal(t, cfg.TikvImporter.DuplicateResolution, em.dupResolution) require.Equal(t, cfg.App.MaxError.Type.Load(), em.remainingError.Type.Load()) require.Equal(t, cfg.App.MaxError.Conflict.Load(), em.remainingError.Conflict.Load()) @@ -162,7 +163,7 @@ func TestResolveAllConflictKeys(t *testing.T) { cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgRemove cfg.App.TaskInfoSchemaName = "lightning_errors" - em := New(db, cfg) + em := New(db, cfg, log.L()) ctx := context.Background() err = em.Init(ctx) require.NoError(t, err) diff --git a/br/pkg/lightning/lightning.go b/br/pkg/lightning/lightning.go index 5cf3df9389d21..111b7c93b59b4 100644 --- a/br/pkg/lightning/lightning.go +++ b/br/pkg/lightning/lightning.go @@ -266,6 +266,7 @@ func (l *Lightning) RunOnce(taskCtx context.Context, taskCfg *config.Config, glu glue: glue, promFactory: l.promFactory, promRegistry: l.promRegistry, + logger: log.L(), } return l.run(taskCtx, taskCfg, o) } @@ -287,6 +288,7 @@ func (l *Lightning) RunServer() error { o := &options{ promFactory: l.promFactory, promRegistry: l.promRegistry, + logger: log.L(), } err = l.run(context.Background(), task, o) if err != nil && !common.IsContextCanceledError(err) { @@ -310,6 +312,7 @@ func (l *Lightning) RunOnceWithOptions(taskCtx context.Context, taskCfg *config. o := &options{ promFactory: l.promFactory, promRegistry: l.promRegistry, + logger: log.L(), } for _, opt := range opts { opt(o) @@ -357,7 +360,7 @@ var ( func (l *Lightning) run(taskCtx context.Context, taskCfg *config.Config, o *options) (err error) { build.LogInfo(build.Lightning) - log.L().Info("cfg", zap.Stringer("cfg", taskCfg)) + o.logger.Info("cfg", zap.Stringer("cfg", taskCfg)) utils.LogEnvVariables() @@ -368,6 +371,7 @@ func (l *Lightning) run(taskCtx context.Context, taskCfg *config.Config, o *opti }() ctx := metric.NewContext(taskCtx, metrics) + ctx = log.NewContext(ctx, o.logger) ctx, cancel := context.WithCancel(ctx) l.cancelLock.Lock() l.cancel = cancel @@ -457,7 +461,7 @@ func (l *Lightning) run(taskCtx context.Context, taskCfg *config.Config, o *opti return common.NormalizeOrWrapErr(common.ErrStorageUnknown, walkErr) } - loadTask := log.L().Begin(zap.InfoLevel, "load data source") + loadTask := o.logger.Begin(zap.InfoLevel, "load data source") var mdl *mydump.MDLoader mdl, err = mydump.NewMyDumpLoaderWithStore(ctx, taskCfg, s) loadTask.End(zap.ErrorLevel, err) @@ -466,13 +470,13 @@ func (l *Lightning) run(taskCtx context.Context, taskCfg *config.Config, o *opti } err = checkSystemRequirement(taskCfg, mdl.GetDatabases()) if err != nil { - log.L().Error("check system requirements failed", zap.Error(err)) + o.logger.Error("check system requirements failed", zap.Error(err)) return common.ErrSystemRequirementNotMet.Wrap(err).GenWithStackByArgs() } // check table schema conflicts err = checkSchemaConflict(taskCfg, mdl.GetDatabases()) if err != nil { - log.L().Error("checkpoint schema conflicts with data files", zap.Error(err)) + o.logger.Error("checkpoint schema conflicts with data files", zap.Error(err)) return errors.Trace(err) } @@ -493,7 +497,7 @@ func (l *Lightning) run(taskCtx context.Context, taskCfg *config.Config, o *opti procedure, err = restore.NewRestoreController(ctx, taskCfg, param) if err != nil { - log.L().Error("restore failed", log.ShortError(err)) + o.logger.Error("restore failed", log.ShortError(err)) return errors.Trace(err) } defer procedure.Close() @@ -835,7 +839,9 @@ func handleLogLevel(w http.ResponseWriter, req *http.Request) { return } oldLevel := log.SetLevel(zapcore.InfoLevel) - log.L().Info("changed log level", zap.Stringer("old", oldLevel), zap.Stringer("new", logLevel.Level)) + log.L().Info("changed log level. No effects if task has specified its logger", + zap.Stringer("old", oldLevel), + zap.Stringer("new", logLevel.Level)) log.SetLevel(logLevel.Level) w.WriteHeader(http.StatusOK) _, _ = w.Write([]byte("{}")) @@ -941,7 +947,7 @@ func CleanupMetas(ctx context.Context, cfg *config.Config, tableName string) err if err != nil || !exist { return errors.Trace(err) } - return errors.Trace(restore.MaybeCleanupAllMetas(ctx, db, cfg.App.MetaSchemaName, tableMetaExist)) + return errors.Trace(restore.MaybeCleanupAllMetas(ctx, log.L(), db, cfg.App.MetaSchemaName, tableMetaExist)) } func SwitchMode(ctx context.Context, cfg *config.Config, tls *common.TLS, mode string) error { diff --git a/br/pkg/lightning/lightning_serial_test.go b/br/pkg/lightning/lightning_serial_test.go index cf5e43d19435a..398d4e90eec6f 100644 --- a/br/pkg/lightning/lightning_serial_test.go +++ b/br/pkg/lightning/lightning_serial_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/glue" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/stretchr/testify/require" ) @@ -65,6 +66,7 @@ func TestRun(t *testing.T) { glue: invalidGlue, promRegistry: lightning.promRegistry, promFactory: lightning.promFactory, + logger: log.L(), } err = lightning.run(ctx, &config.Config{ Mydumper: config.MydumperRuntime{ diff --git a/br/pkg/lightning/log/log.go b/br/pkg/lightning/log/log.go index aa61023a29cc9..2dab04f925182 100644 --- a/br/pkg/lightning/log/log.go +++ b/br/pkg/lightning/log/log.go @@ -230,3 +230,21 @@ func (task *Task) End(level zapcore.Level, err error, extraFields ...zap.Field) } return elapsed } + +type ctxKeyType struct{} + +var ctxKey ctxKeyType + +// NewContext returns a new context with the provided logger. +func NewContext(ctx context.Context, logger Logger) context.Context { + return context.WithValue(ctx, ctxKey, logger) +} + +// FromContext returns the logger stored in the context. +func FromContext(ctx context.Context) Logger { + m, ok := ctx.Value(ctxKey).(Logger) + if !ok { + return appLogger + } + return m +} diff --git a/br/pkg/lightning/mydump/csv_parser.go b/br/pkg/lightning/mydump/csv_parser.go index 758092df960b4..5c538269ebdce 100644 --- a/br/pkg/lightning/mydump/csv_parser.go +++ b/br/pkg/lightning/mydump/csv_parser.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/lightning/metric" "github.com/pingcap/tidb/br/pkg/lightning/worker" "github.com/pingcap/tidb/types" @@ -123,7 +124,7 @@ func NewCSVParser( } metrics, _ := metric.FromContext(ctx) return &CSVParser{ - blockParser: makeBlockParser(reader, blockBufSize, ioWorkers, metrics), + blockParser: makeBlockParser(reader, blockBufSize, ioWorkers, metrics, log.FromContext(ctx)), cfg: cfg, charsetConvertor: charsetConvertor, comma: []byte(separator), diff --git a/br/pkg/lightning/mydump/loader.go b/br/pkg/lightning/mydump/loader.go index 09ef6229c45bc..30f4f14c1464c 100644 --- a/br/pkg/lightning/mydump/loader.go +++ b/br/pkg/lightning/mydump/loader.go @@ -41,7 +41,7 @@ type MDDatabaseMeta struct { func (m *MDDatabaseMeta) GetSchema(ctx context.Context, store storage.ExternalStorage) string { schema, err := ExportStatement(ctx, store, m.SchemaFile, m.charSet) if err != nil { - log.L().Warn("failed to extract table schema", + log.FromContext(ctx).Warn("failed to extract table schema", zap.String("Path", m.SchemaFile.FileMeta.Path), log.ShortError(err), ) @@ -78,7 +78,7 @@ type SourceFileMeta struct { func (m *MDTableMeta) GetSchema(ctx context.Context, store storage.ExternalStorage) (string, error) { schema, err := ExportStatement(ctx, store, m.SchemaFile, m.charSet) if err != nil { - log.L().Error("failed to extract table schema", + log.FromContext(ctx).Error("failed to extract table schema", zap.String("Path", m.SchemaFile.FileMeta.Path), log.ShortError(err), ) @@ -157,7 +157,7 @@ func NewMyDumpLoaderWithStore(ctx context.Context, cfg *config.Config, store sto fileRouteRules = append(fileRouteRules, defaultFileRouteRules...) } - fileRouter, err := NewFileRouter(fileRouteRules) + fileRouter, err := NewFileRouter(fileRouteRules, log.FromContext(ctx)) if err != nil { return nil, common.ErrInvalidConfig.Wrap(err).GenWithStack("parse file routing rule failed") } @@ -300,7 +300,7 @@ func (s *mdLoaderSetup) listFiles(ctx context.Context, store storage.ExternalSto // meaning the file and chunk orders will be the same everytime it is called // (as long as the source is immutable). err := store.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { - logger := log.With(zap.String("path", path)) + logger := log.FromContext(ctx).With(zap.String("path", path)) res, err := s.loader.fileRouter.Route(filepath.ToSlash(path)) if err != nil { diff --git a/br/pkg/lightning/mydump/parquet_parser.go b/br/pkg/lightning/mydump/parquet_parser.go index 789163c18bb01..50ae2d9d63960 100644 --- a/br/pkg/lightning/mydump/parquet_parser.go +++ b/br/pkg/lightning/mydump/parquet_parser.go @@ -208,7 +208,7 @@ func NewParquetParser( Reader: reader, columns: columns, columnMetas: columnMetas, - logger: log.L(), + logger: log.FromContext(ctx), }, nil } @@ -377,7 +377,7 @@ func (pp *ParquetParser) ReadRow() error { } for i := 0; i < length; i++ { pp.lastRow.Length += getDatumLen(v.Field(i)) - if err := setDatumValue(&pp.lastRow.Row[i], v.Field(i), pp.columnMetas[i]); err != nil { + if err := setDatumValue(&pp.lastRow.Row[i], v.Field(i), pp.columnMetas[i], pp.logger); err != nil { return err } } @@ -401,7 +401,7 @@ func getDatumLen(v reflect.Value) int { // convert a parquet value to Datum // // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md -func setDatumValue(d *types.Datum, v reflect.Value, meta *parquet.SchemaElement) error { +func setDatumValue(d *types.Datum, v reflect.Value, meta *parquet.SchemaElement, logger log.Logger) error { switch v.Kind() { case reflect.Uint8, reflect.Uint16, reflect.Uint32, reflect.Uint64: d.SetUint64(v.Uint()) @@ -417,10 +417,10 @@ func setDatumValue(d *types.Datum, v reflect.Value, meta *parquet.SchemaElement) if v.IsNil() { d.SetNull() } else { - return setDatumValue(d, v.Elem(), meta) + return setDatumValue(d, v.Elem(), meta, logger) } default: - log.L().Error("unknown value", zap.Stringer("kind", v.Kind()), + logger.Error("unknown value", zap.Stringer("kind", v.Kind()), zap.String("type", v.Type().Name()), zap.Reflect("value", v.Interface())) return errors.Errorf("unknown value: %v", v) } diff --git a/br/pkg/lightning/mydump/parser.go b/br/pkg/lightning/mydump/parser.go index cf303e903be65..5949016dc74b0 100644 --- a/br/pkg/lightning/mydump/parser.go +++ b/br/pkg/lightning/mydump/parser.go @@ -60,13 +60,19 @@ type blockParser struct { metrics *metric.Metrics } -func makeBlockParser(reader ReadSeekCloser, blockBufSize int64, ioWorkers *worker.Pool, metrics *metric.Metrics) blockParser { +func makeBlockParser( + reader ReadSeekCloser, + blockBufSize int64, + ioWorkers *worker.Pool, + metrics *metric.Metrics, + logger log.Logger, +) blockParser { return blockParser{ reader: MakePooledReader(reader, ioWorkers), blockBuf: make([]byte, blockBufSize*config.BufferSizeScale), remainBuf: &bytes.Buffer{}, appendBuf: &bytes.Buffer{}, - Logger: log.L(), + Logger: logger, rowPool: &sync.Pool{ New: func() interface{} { return make([]types.Datum, 0, 16) @@ -147,7 +153,7 @@ func NewChunkParser( } metrics, _ := metric.FromContext(ctx) return &ChunkParser{ - blockParser: makeBlockParser(reader, blockBufSize, ioWorkers, metrics), + blockParser: makeBlockParser(reader, blockBufSize, ioWorkers, metrics, log.FromContext(ctx)), escFlavor: escFlavor, } } diff --git a/br/pkg/lightning/mydump/reader.go b/br/pkg/lightning/mydump/reader.go index 7db6bf2bf8ed5..5ba5cf974ef30 100644 --- a/br/pkg/lightning/mydump/reader.go +++ b/br/pkg/lightning/mydump/reader.go @@ -107,7 +107,7 @@ func ExportStatement(ctx context.Context, store storage.ExternalStorage, sqlFile data, err = decodeCharacterSet(data, characterSet) if err != nil { - log.L().Error("cannot decode input file, please convert to target encoding manually", + log.FromContext(ctx).Error("cannot decode input file, please convert to target encoding manually", zap.String("encoding", characterSet), zap.String("Path", sqlFile.FileMeta.Path), ) diff --git a/br/pkg/lightning/mydump/region.go b/br/pkg/lightning/mydump/region.go index b347d27bb9ab8..04cc75e5567ae 100644 --- a/br/pkg/lightning/mydump/region.go +++ b/br/pkg/lightning/mydump/region.go @@ -170,7 +170,7 @@ func MakeTableRegions( break } if err != nil { - log.L().Error("make source file region error", zap.Error(err), zap.String("file_path", info.FileMeta.Path)) + log.FromContext(ctx).Error("make source file region error", zap.Error(err), zap.String("file_path", info.FileMeta.Path)) break } } @@ -240,7 +240,7 @@ func MakeTableRegions( } } - log.L().Info("makeTableRegions", zap.Int("filesCount", len(meta.DataFiles)), + log.FromContext(ctx).Info("makeTableRegions", zap.Int("filesCount", len(meta.DataFiles)), zap.Int64("MaxRegionSize", int64(cfg.Mydumper.MaxRegionSize)), zap.Int("RegionsCount", len(filesRegions)), zap.Float64("BatchSize", batchSize), @@ -274,10 +274,10 @@ func makeSourceFileRegion( } sizePerRow, err := GetSampledAvgRowSize(&fi, cfg, ioWorkers, store) if err == nil && sizePerRow != 0 { - log.L().Warn("fail to sample file", zap.String("path", fi.FileMeta.Path), zap.Error(err)) + log.FromContext(ctx).Warn("fail to sample file", zap.String("path", fi.FileMeta.Path), zap.Error(err)) divisor = sizePerRow } - log.L().Debug("avg row size", zap.String("path", fi.FileMeta.Path), zap.Int64("size per row", sizePerRow)) + log.FromContext(ctx).Debug("avg row size", zap.String("path", fi.FileMeta.Path), zap.Int64("size per row", sizePerRow)) // If a csv file is overlarge, we need to split it into multiple regions. // Note: We can only split a csv file whose format is strict. // We increase the check threshold by 1/10 of the `max-region-size` because the source file size dumped by tools @@ -305,7 +305,7 @@ func makeSourceFileRegion( }) if tableRegion.Size() > tableRegionSizeWarningThreshold { - log.L().Warn( + log.FromContext(ctx).Warn( "file is too big to be processed efficiently; we suggest splitting it at 256 MB each", zap.String("file", fi.FileMeta.Path), zap.Int64("size", dataFileSize)) @@ -467,7 +467,7 @@ func SplitLargeFile( if !errors.ErrorEqual(err, io.EOF) { return 0, nil, nil, err } - log.L().Warn("file contains no terminator at end", + log.FromContext(ctx).Warn("file contains no terminator at end", zap.String("path", dataFile.FileMeta.Path), zap.String("terminator", cfg.Mydumper.CSV.Terminator)) pos = dataFile.FileMeta.FileSize diff --git a/br/pkg/lightning/mydump/router.go b/br/pkg/lightning/mydump/router.go index 223d82edb90e9..c3a6ff3aae161 100644 --- a/br/pkg/lightning/mydump/router.go +++ b/br/pkg/lightning/mydump/router.go @@ -142,11 +142,11 @@ func (c chainRouters) Route(path string) (*RouteResult, error) { return nil, nil } -func NewFileRouter(cfg []*config.FileRouteRule) (FileRouter, error) { +func NewFileRouter(cfg []*config.FileRouteRule, logger log.Logger) (FileRouter, error) { res := make([]FileRouter, 0, len(cfg)) p := regexRouterParser{} for _, c := range cfg { - rule, err := p.Parse(c) + rule, err := p.Parse(c, logger) if err != nil { return nil, err } @@ -180,7 +180,7 @@ func (r *RegexRouter) Route(path string) (*RouteResult, error) { type regexRouterParser struct{} -func (p regexRouterParser) Parse(r *config.FileRouteRule) (*RegexRouter, error) { +func (p regexRouterParser) Parse(r *config.FileRouteRule, logger log.Logger) (*RegexRouter, error) { rule := &RegexRouter{} if r.Path == "" && r.Pattern == "" { return nil, errors.New("`path` and `pattern` must not be both empty in [[mydumper.files]]") @@ -225,7 +225,7 @@ func (p regexRouterParser) Parse(r *config.FileRouteRule) (*RegexRouter, error) if unescape { val, err := url.PathUnescape(value) if err != nil { - log.L().Warn("unescape string failed, will be ignored", zap.String("value", value), + logger.Warn("unescape string failed, will be ignored", zap.String("value", value), zap.Error(err)) } else { value = val diff --git a/br/pkg/lightning/mydump/router_test.go b/br/pkg/lightning/mydump/router_test.go index 4df92b74f45c5..7401027cfbd36 100644 --- a/br/pkg/lightning/mydump/router_test.go +++ b/br/pkg/lightning/mydump/router_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/util/filter" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -21,7 +22,7 @@ func TestRouteParser(t *testing.T) { {Pattern: `^(?:[^/]*/)*([^/.]+)\.([^./]+)(?:\.[0-9]+)?\.(csv|sql)`, Schema: "$1-schema", Table: "$1-table", Type: "$2"}, } for _, r := range rules { - _, err := NewFileRouter([]*config.FileRouteRule{r}) + _, err := NewFileRouter([]*config.FileRouteRule{r}, log.L()) assert.NoError(t, err) } @@ -32,7 +33,7 @@ func TestRouteParser(t *testing.T) { {Pattern: `^(?:[^/]*/)*([^/.]+)\.([^./]+)(?:\.[0-9]+)?\.(csv|sql)`, Schema: "$1", Table: "$2", Type: "$3", Key: "$4"}, } for _, r := range invalidRules { - _, err := NewFileRouter([]*config.FileRouteRule{r}) + _, err := NewFileRouter([]*config.FileRouteRule{r}, log.L()) assert.Error(t, err) } } @@ -40,31 +41,31 @@ func TestRouteParser(t *testing.T) { func TestInvalidRouteRule(t *testing.T) { rule := &config.FileRouteRule{} rules := []*config.FileRouteRule{rule} - _, err := NewFileRouter(rules) + _, err := NewFileRouter(rules, log.L()) require.Regexp(t, "`path` and `pattern` must not be both empty in \\[\\[mydumper.files\\]\\]", err.Error()) rule.Pattern = `^(?:[^/]*/)*([^/.]+)\.(?P[^./]+)(?:\.(?P[0-9]+))?\.(?Pcsv|sql)(?:\.(?P[A-Za-z0-9]+))?$` - _, err = NewFileRouter(rules) + _, err = NewFileRouter(rules, log.L()) require.Regexp(t, "field 'type' match pattern can't be empty", err.Error()) rule.Type = "$type" - _, err = NewFileRouter(rules) + _, err = NewFileRouter(rules, log.L()) require.Regexp(t, "field 'schema' match pattern can't be empty", err.Error()) rule.Schema = "$schema" - _, err = NewFileRouter(rules) + _, err = NewFileRouter(rules, log.L()) require.Regexp(t, "invalid named capture '\\$schema'", err.Error()) rule.Schema = "$1" - _, err = NewFileRouter(rules) + _, err = NewFileRouter(rules, log.L()) require.Regexp(t, "field 'table' match pattern can't be empty", err.Error()) rule.Table = "$table" - _, err = NewFileRouter(rules) + _, err = NewFileRouter(rules, log.L()) require.NoError(t, err) rule.Path = "/tmp/1.sql" - _, err = NewFileRouter(rules) + _, err = NewFileRouter(rules, log.L()) require.Regexp(t, "can't set both `path` and `pattern` field in \\[\\[mydumper.files\\]\\]", err.Error()) } @@ -73,7 +74,7 @@ func TestSingleRouteRule(t *testing.T) { {Pattern: `^(?:[^/]*/)*([^/.]+)\.(?P
[^./]+)(?:\.(?P[0-9]+))?\.(?Pcsv|sql)(?:\.(?P[A-Za-z0-9]+))?$`, Schema: "$1", Table: "$table", Type: "$type", Key: "$key", Compression: "$cp"}, } - r, err := NewFileRouter(rules) + r, err := NewFileRouter(rules, log.L()) require.NoError(t, err) inputOutputMap := map[string][]string{ @@ -107,7 +108,7 @@ func TestSingleRouteRule(t *testing.T) { } rule := &config.FileRouteRule{Pattern: `^(?:[^/]*/)*([^/.]+)\.(?P
[^./]+)(?:\.(?P[0-9]+))?\.(?P\w+)(?:\.(?P[A-Za-z0-9]+))?$`, Schema: "$1", Table: "$table", Type: "$type", Key: "$key", Compression: "$cp"} - r, err = NewFileRouter([]*config.FileRouteRule{rule}) + r, err = NewFileRouter([]*config.FileRouteRule{rule}, log.L()) require.NoError(t, err) require.NotNil(t, r) invalidMatchPaths := []string{ @@ -131,7 +132,7 @@ func TestMultiRouteRule(t *testing.T) { {Pattern: `^(?:[^/]*/)*(?P[^/.]+)\.(?P
[^./]+)(?:\.(?P[0-9]+))?\.(?Pcsv|sql)(?:\.(?P[A-Za-z0-9]+))?$`, Schema: "$schema", Table: "$table", Type: "$type", Key: "$key", Compression: "$cp"}, } - r, err := NewFileRouter(rules) + r, err := NewFileRouter(rules, log.L()) require.NoError(t, err) inputOutputMap := map[string][]string{ @@ -163,7 +164,7 @@ func TestMultiRouteRule(t *testing.T) { // add another rule that match same pattern with the third rule, the result should be no different p := &config.FileRouteRule{Pattern: `^(?P[^/.]+)\.(?P
[^./]+)(?:\.(?P[0-9]+))?\.(?Pcsv|sql)(?:\.(?P[A-Za-z0-9]+))?$`, Schema: "test_schema", Table: "test_table", Type: "$type", Key: "$key", Compression: "$cp"} rules = append(rules, p) - r, err = NewFileRouter(rules) + r, err = NewFileRouter(rules, log.L()) require.NoError(t, err) for path, fields := range inputOutputMap { res, err := r.Route(path) @@ -209,7 +210,7 @@ func TestRouteExpanding(t *testing.T) { for pat, value := range tablePatternResMap { rule.Table = pat - router, err := NewFileRouter([]*config.FileRouteRule{rule}) + router, err := NewFileRouter([]*config.FileRouteRule{rule}, log.L()) assert.NoError(t, err) res, err := router.Route(path) assert.NoError(t, err) @@ -220,7 +221,7 @@ func TestRouteExpanding(t *testing.T) { invalidPatterns := []string{"$1_$schema", "$schema_$table_name", "$6"} for _, pat := range invalidPatterns { rule.Table = pat - _, err := NewFileRouter([]*config.FileRouteRule{rule}) + _, err := NewFileRouter([]*config.FileRouteRule{rule}, log.L()) assert.Error(t, err) } } @@ -235,7 +236,7 @@ func TestRouteWithPath(t *testing.T) { Key: "$key", } r := *rule - router, err := NewFileRouter([]*config.FileRouteRule{&r}) + router, err := NewFileRouter([]*config.FileRouteRule{&r}, log.L()) require.NoError(t, err) res, err := router.Route(fileName) require.NoError(t, err) diff --git a/br/pkg/lightning/restore/check_info.go b/br/pkg/lightning/restore/check_info.go index 92ff3d2a5385c..442fae5a3e18b 100644 --- a/br/pkg/lightning/restore/check_info.go +++ b/br/pkg/lightning/restore/check_info.go @@ -402,7 +402,7 @@ func (rc *Controller) estimateSourceData(ctx context.Context) (int64, error) { bigTableCount := 0 tableCount := 0 unSortedTableCount := 0 - errMgr := errormanager.New(nil, rc.cfg) + errMgr := errormanager.New(nil, rc.cfg, log.FromContext(ctx)) for _, db := range rc.dbMetas { info, ok := rc.dbInfos[db.Name] if !ok { @@ -455,7 +455,7 @@ func (rc *Controller) estimateSourceData(ctx context.Context) (int64, error) { } // localResource checks the local node has enough resources for this import when local backend enabled; -func (rc *Controller) localResource(sourceSize int64) error { +func (rc *Controller) localResource(ctx context.Context, sourceSize int64) error { if rc.isSourceInLocal() { sourceDir := strings.TrimPrefix(rc.cfg.Mydumper.SourceDir, storage.LocalURIPrefix) same, err := common.SameDisk(sourceDir, rc.cfg.TikvImporter.SortedKVDir) @@ -489,7 +489,7 @@ func (rc *Controller) localResource(sourceSize int64) error { units.BytesSize(float64(sourceSize)), units.BytesSize(float64(localAvailable)), units.BytesSize(float64(localAvailable))) passed = false - log.L().Error(message) + log.FromContext(ctx).Error(message) default: message = fmt.Sprintf("local disk space may not enough to finish import, "+ "estimate sorted data size is %s, but local available is %s,"+ @@ -497,7 +497,7 @@ func (rc *Controller) localResource(sourceSize int64) error { units.BytesSize(float64(sourceSize)), units.BytesSize(float64(localAvailable)), units.BytesSize(float64(rc.cfg.TikvImporter.DiskQuota))) passed = true - log.L().Warn(message) + log.FromContext(ctx).Warn(message) } rc.checkTemplate.Collect(Critical, passed, message) return nil @@ -510,7 +510,7 @@ func (rc *Controller) CheckpointIsValid(ctx context.Context, tableInfo *mydump.M tableCheckPoint, err := rc.checkpointsDB.Get(ctx, uniqueName) if err != nil { // there is no checkpoint - log.L().Debug("no checkpoint detected", zap.String("table", uniqueName)) + log.FromContext(ctx).Debug("no checkpoint detected", zap.String("table", uniqueName)) return nil, true } // if checkpoint enable and not missing, we skip the check table empty progress. @@ -570,12 +570,12 @@ func (rc *Controller) CheckpointIsValid(ctx context.Context, tableInfo *mydump.M } } if len(columns) == 0 { - log.L().Debug("no valid checkpoint detected", zap.String("table", uniqueName)) + log.FromContext(ctx).Debug("no valid checkpoint detected", zap.String("table", uniqueName)) return nil, false } info := rc.dbInfos[tableInfo.DB].Tables[tableInfo.Name] if info != nil { - permFromTiDB, err := parseColumnPermutations(info.Core, columns, nil) + permFromTiDB, err := parseColumnPermutations(info.Core, columns, nil, log.FromContext(ctx)) if err != nil { msgs = append(msgs, fmt.Sprintf("failed to calculate columns %s, table %s's info has changed,"+ "consider remove this checkpoint, and start import again.", err.Error(), uniqueName)) @@ -641,7 +641,7 @@ func (rc *Controller) readFirstRow(ctx context.Context, dataFileMeta mydump.Sour // SchemaIsValid checks the import file and cluster schema is match. func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTableMeta) ([]string, error) { if len(tableInfo.DataFiles) == 0 { - log.L().Info("no data files detected", zap.String("db", tableInfo.DB), zap.String("table", tableInfo.Name)) + log.FromContext(ctx).Info("no data files detected", zap.String("db", tableInfo.DB), zap.String("table", tableInfo.Name)) return nil, nil } @@ -673,7 +673,7 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab // only check the first file of this table. dataFile := tableInfo.DataFiles[0] - log.L().Info("datafile to check", zap.String("db", tableInfo.DB), + log.FromContext(ctx).Info("datafile to check", zap.String("db", tableInfo.DB), zap.String("table", tableInfo.Name), zap.String("path", dataFile.FileMeta.Path)) // get columns name from data file. dataFileMeta := dataFile.FileMeta @@ -687,7 +687,7 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab return nil, errors.Trace(err) } if colsFromDataFile == nil && len(row) == 0 { - log.L().Info("file contains no data, skip checking against schema validity", zap.String("path", dataFileMeta.Path)) + log.FromContext(ctx).Info("file contains no data, skip checking against schema validity", zap.String("path", dataFileMeta.Path)) return msgs, nil } @@ -898,7 +898,7 @@ outer: level := Warn if hasUniqueField && len(rows) > 1 { level = Critical - } else if !checkFieldCompatibility(tableInfo.Core, ignoreColsSet, rows[0]) { + } else if !checkFieldCompatibility(tableInfo.Core, ignoreColsSet, rows[0], log.FromContext(ctx)) { // if there are only 1 csv file or there is not unique key, try to check if all columns are compatible with string value level = Critical } @@ -907,10 +907,15 @@ outer: return nil } -func checkFieldCompatibility(tbl *model.TableInfo, ignoreCols map[string]struct{}, values []types.Datum) bool { +func checkFieldCompatibility( + tbl *model.TableInfo, + ignoreCols map[string]struct{}, + values []types.Datum, + logger log.Logger, +) bool { se := kv.NewSession(&kv.SessionOptions{ SQLMode: mysql.ModeStrictTransTables, - }) + }, logger) for i, col := range tbl.Columns { // do not check ignored columns if _, ok := ignoreCols[col.Name.L]; ok { @@ -921,7 +926,7 @@ func checkFieldCompatibility(tbl *model.TableInfo, ignoreCols map[string]struct{ } _, err := table.CastValue(se, values[i], col, true, false) if err != nil { - log.L().Error("field value is not consistent with column type", zap.String("value", values[i].GetString()), + logger.Error("field value is not consistent with column type", zap.String("value", values[i].GetString()), zap.Any("column_info", col), zap.Error(err)) return false } @@ -956,7 +961,7 @@ func (rc *Controller) sampleDataFromTable( if err != nil { return errors.Trace(err) } - kvEncoder, err := rc.backend.NewEncoder(tbl, &kv.SessionOptions{ + kvEncoder, err := rc.backend.NewEncoder(ctx, tbl, &kv.SessionOptions{ SQLMode: rc.cfg.TiDB.SQLMode, Timestamp: 0, SysVars: rc.sysVars, @@ -991,7 +996,7 @@ func (rc *Controller) sampleDataFromTable( panic(fmt.Sprintf("file '%s' with unknown source type '%s'", sampleFile.Path, sampleFile.Type.String())) } defer parser.Close() - logTask := log.With(zap.String("table", tableMeta.Name)).Begin(zap.InfoLevel, "sample file") + logTask := log.FromContext(ctx).With(zap.String("table", tableMeta.Name)).Begin(zap.InfoLevel, "sample file") igCols, err := rc.cfg.Mydumper.IgnoreColumns.GetIgnoreColumns(dbName, tableMeta.Name, rc.cfg.Mydumper.CaseSensitive) if err != nil { return errors.Trace(err) @@ -1017,7 +1022,11 @@ outloop: case nil: if !initializedColumns { if len(columnPermutation) == 0 { - columnPermutation, err = createColumnPermutation(columnNames, igCols.ColumnsMap(), tableInfo) + columnPermutation, err = createColumnPermutation( + columnNames, + igCols.ColumnsMap(), + tableInfo, + log.FromContext(ctx)) if err != nil { return errors.Trace(err) } @@ -1036,7 +1045,7 @@ outloop: var dataChecksum, indexChecksum verification.KVChecksum kvs, encodeErr := kvEncoder.Encode(logTask.Logger, lastRow.Row, lastRow.RowID, columnPermutation, sampleFile.Path, offset) if encodeErr != nil { - encodeErr = errMgr.RecordTypeError(ctx, log.L(), tableInfo.Name.O, sampleFile.Path, offset, + encodeErr = errMgr.RecordTypeError(ctx, log.FromContext(ctx), tableInfo.Name.O, sampleFile.Path, offset, "" /* use a empty string here because we don't actually record */, encodeErr) if encodeErr != nil { return errors.Annotatef(encodeErr, "in file at offset %d", offset) @@ -1075,7 +1084,7 @@ outloop: if rowSize > 0 && kvSize > rowSize { tableMeta.IndexRatio = float64(kvSize) / float64(rowSize) } - log.L().Info("Sample source data", zap.String("table", tableMeta.Name), zap.Float64("IndexRatio", tableMeta.IndexRatio), zap.Bool("IsSourceOrder", tableMeta.IsRowOrdered)) + log.FromContext(ctx).Info("Sample source data", zap.String("table", tableMeta.Name), zap.Float64("IndexRatio", tableMeta.IndexRatio), zap.Bool("IsSourceOrder", tableMeta.IsRowOrdered)) return nil } @@ -1159,7 +1168,7 @@ func tableContainsData(ctx context.Context, db utils.DBExecutor, tableName strin query := "select 1 from " + tableName + " limit 1" exec := common.SQLWithRetry{ DB: db, - Logger: log.L(), + Logger: log.FromContext(ctx), } var dump int err := exec.QueryRow(ctx, "check table empty", query, &dump) diff --git a/br/pkg/lightning/restore/check_info_test.go b/br/pkg/lightning/restore/check_info_test.go index 42772b64ea6ab..abdfcf232f0a9 100644 --- a/br/pkg/lightning/restore/check_info_test.go +++ b/br/pkg/lightning/restore/check_info_test.go @@ -583,9 +583,11 @@ func TestLocalResource(t *testing.T) { ioWorkers: worker.NewPool(context.Background(), 1, "io"), } + ctx := context.Background() + // 1. source-size is smaller than disk-size, won't trigger error information rc.checkTemplate = NewSimpleTemplate() - err = rc.localResource(1000) + err = rc.localResource(ctx, 1000) require.NoError(t, err) tmpl := rc.checkTemplate.(*SimpleTemplate) require.Equal(t, 1, tmpl.warnFailedCount) @@ -594,7 +596,7 @@ func TestLocalResource(t *testing.T) { // 2. source-size is bigger than disk-size, with default disk-quota will trigger a critical error rc.checkTemplate = NewSimpleTemplate() - err = rc.localResource(4096) + err = rc.localResource(ctx, 4096) require.NoError(t, err) tmpl = rc.checkTemplate.(*SimpleTemplate) require.Equal(t, 1, tmpl.warnFailedCount) @@ -604,7 +606,7 @@ func TestLocalResource(t *testing.T) { // 3. source-size is bigger than disk-size, with a vaild disk-quota will trigger a warning rc.checkTemplate = NewSimpleTemplate() rc.cfg.TikvImporter.DiskQuota = config.ByteSize(1024) - err = rc.localResource(4096) + err = rc.localResource(ctx, 4096) require.NoError(t, err) tmpl = rc.checkTemplate.(*SimpleTemplate) require.Equal(t, 1, tmpl.warnFailedCount) diff --git a/br/pkg/lightning/restore/checksum.go b/br/pkg/lightning/restore/checksum.go index 4a537c10f626a..20fc77462c196 100644 --- a/br/pkg/lightning/restore/checksum.go +++ b/br/pkg/lightning/restore/checksum.go @@ -138,7 +138,7 @@ func (e *tidbChecksumExecutor) Checksum(ctx context.Context, tableInfo *checkpoi tableName := common.UniqueTable(tableInfo.DB, tableInfo.Name) - task := log.With(zap.String("table", tableName)).Begin(zap.InfoLevel, "remote checksum") + task := log.FromContext(ctx).With(zap.String("table", tableName)).Begin(zap.InfoLevel, "remote checksum") // ADMIN CHECKSUM TABLE
,
example. // mysql> admin checksum table test.t; @@ -171,7 +171,7 @@ func DoChecksum(ctx context.Context, table *checkpoints.TidbTableInfo) (*RemoteC return nil, errors.New("No gcLifeTimeManager found in context, check context initialization") } - task := log.With(zap.String("table", table.Name)).Begin(zap.InfoLevel, "remote checksum") + task := log.FromContext(ctx).With(zap.String("table", table.Name)).Begin(zap.InfoLevel, "remote checksum") cs, err := manager.Checksum(ctx, table) dur := task.End(zap.ErrorLevel, err) @@ -232,7 +232,7 @@ func (m *gcLifeTimeManager) removeOneJob(ctx context.Context, db *sql.DB) { "UPDATE mysql.tidb SET VARIABLE_VALUE = '%s' WHERE VARIABLE_NAME = 'tikv_gc_life_time'", m.oriGCLifeTime, ) - log.L().Warn("revert GC lifetime failed, please reset the GC lifetime manually after Lightning completed", + log.FromContext(ctx).Warn("revert GC lifetime failed, please reset the GC lifetime manually after Lightning completed", zap.String("query", query), log.ShortError(err), ) @@ -309,7 +309,7 @@ func (e *tikvChecksumManager) checksumDB(ctx context.Context, tableInfo *checkpo }, nil } - log.L().Warn("remote checksum failed", zap.String("db", tableInfo.DB), + log.FromContext(ctx).Warn("remote checksum failed", zap.String("db", tableInfo.DB), zap.String("table", tableInfo.Name), zap.Error(err), zap.Int("concurrency", distSQLScanConcurrency), zap.Int("retry", i)) @@ -442,7 +442,7 @@ func (m *gcTTLManager) updateGCTTL(ctx context.Context) error { } func (m *gcTTLManager) doUpdateGCTTL(ctx context.Context, ts uint64) error { - log.L().Debug("update PD safePoint limit with TTL", + log.FromContext(ctx).Debug("update PD safePoint limit with TTL", zap.Uint64("currnet_ts", ts)) var err error if ts > 0 { @@ -460,7 +460,7 @@ func (m *gcTTLManager) start(ctx context.Context) { updateGCTTL := func() { if err := m.updateGCTTL(ctx); err != nil { - log.L().Warn("failed to update service safe point, checksum may fail if gc triggered", zap.Error(err)) + log.FromContext(ctx).Warn("failed to update service safe point, checksum may fail if gc triggered", zap.Error(err)) } } @@ -471,7 +471,7 @@ func (m *gcTTLManager) start(ctx context.Context) { for { select { case <-ctx.Done(): - log.L().Info("service safe point keeper exited") + log.FromContext(ctx).Info("service safe point keeper exited") return case <-updateTick.C: updateGCTTL() diff --git a/br/pkg/lightning/restore/chunk_restore_test.go b/br/pkg/lightning/restore/chunk_restore_test.go index 84b0560544896..59d083d85561c 100644 --- a/br/pkg/lightning/restore/chunk_restore_test.go +++ b/br/pkg/lightning/restore/chunk_restore_test.go @@ -253,7 +253,7 @@ func (s *chunkRestoreSuite) TestEncodeLoop() { kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ SQLMode: s.cfg.TiDB.SQLMode, Timestamp: 1234567895, - }, nil) + }, nil, log.L()) require.NoError(s.T(), err) cfg := config.NewConfig() rc := &Controller{pauser: DeliverPauser, cfg: cfg} @@ -280,7 +280,7 @@ func (s *chunkRestoreSuite) TestEncodeLoopCanceled() { kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ SQLMode: s.cfg.TiDB.SQLMode, Timestamp: 1234567896, - }, nil) + }, nil, log.L()) require.NoError(s.T(), err) go cancel() @@ -298,7 +298,7 @@ func (s *chunkRestoreSuite) TestEncodeLoopForcedError() { kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ SQLMode: s.cfg.TiDB.SQLMode, Timestamp: 1234567897, - }, nil) + }, nil, log.L()) require.NoError(s.T(), err) // close the chunk so reading it will result in the "file already closed" error. @@ -318,7 +318,7 @@ func (s *chunkRestoreSuite) TestEncodeLoopDeliverLimit() { kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ SQLMode: s.cfg.TiDB.SQLMode, Timestamp: 1234567898, - }, nil) + }, nil, log.L()) require.NoError(s.T(), err) dir := s.T().TempDir() @@ -375,7 +375,7 @@ func (s *chunkRestoreSuite) TestEncodeLoopDeliverErrored() { kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ SQLMode: s.cfg.TiDB.SQLMode, Timestamp: 1234567898, - }, nil) + }, nil, log.L()) require.NoError(s.T(), err) go func() { @@ -402,7 +402,8 @@ func (s *chunkRestoreSuite) TestEncodeLoopColumnsMismatch() { ctx := context.Background() cfg := config.NewConfig() - errorMgr := errormanager.New(nil, cfg) + logger := log.L() + errorMgr := errormanager.New(nil, cfg, logger) rc := &Controller{pauser: DeliverPauser, cfg: cfg, errorMgr: errorMgr} reader, err := store.Open(ctx, fileName) @@ -417,7 +418,8 @@ func (s *chunkRestoreSuite) TestEncodeLoopColumnsMismatch() { kvsCh := make(chan []deliveredKVs, 2) deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := tidb.NewTiDBBackend(nil, config.ReplaceOnDup, errorMgr).NewEncoder( + kvEncoder, err := tidb.NewTiDBBackend(ctx, nil, config.ReplaceOnDup, errorMgr).NewEncoder( + ctx, s.tr.encTable, &kv.SessionOptions{ SQLMode: s.cfg.TiDB.SQLMode, @@ -511,7 +513,13 @@ func (s *chunkRestoreSuite) testEncodeLoopIgnoreColumnsCSV( kvsCh := make(chan []deliveredKVs, 2) deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := tidb.NewTiDBBackend(nil, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig())).NewEncoder( + kvEncoder, err := tidb.NewTiDBBackend( + ctx, + nil, + config.ReplaceOnDup, + errormanager.New(nil, config.NewConfig(), log.L()), + ).NewEncoder( + ctx, s.tr.encTable, &kv.SessionOptions{ SQLMode: s.cfg.TiDB.SQLMode, @@ -558,7 +566,7 @@ func (s *chunkRestoreSuite) TestRestore() { mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).Times(1) mockWriter := mock.NewMockEngineWriter(controller) mockBackend.EXPECT().LocalWriter(ctx, gomock.Any(), gomock.Any()).Return(mockWriter, nil).AnyTimes() - mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any()).Return(mockEncoder{}, nil).Times(1) + mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockEncoder{}, nil).Times(1) mockWriter.EXPECT().IsSynced().Return(true).AnyTimes() mockWriter.EXPECT().AppendRows(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() diff --git a/br/pkg/lightning/restore/meta_manager.go b/br/pkg/lightning/restore/meta_manager.go index 3476eace1d292..0af04e69feedb 100644 --- a/br/pkg/lightning/restore/meta_manager.go +++ b/br/pkg/lightning/restore/meta_manager.go @@ -37,7 +37,7 @@ type dbMetaMgrBuilder struct { func (b *dbMetaMgrBuilder) Init(ctx context.Context) error { exec := common.SQLWithRetry{ DB: b.db, - Logger: log.L(), + Logger: log.FromContext(ctx), HideQueryLog: redact.NeedRedact(), } metaDBSQL := fmt.Sprintf("CREATE DATABASE IF NOT EXISTS %s", common.EscapeIdentifier(b.schema)) @@ -382,10 +382,10 @@ func (m *dbTableMetaMgr) AllocTableRowIDs(ctx context.Context, rawRowIDMax int64 ck := verify.MakeKVChecksum(baseTotalBytes, baseTotalKvs, baseChecksum) checksum = &ck } - log.L().Info("allocate table row_id base", zap.String("table", m.tr.tableName), + log.FromContext(ctx).Info("allocate table row_id base", zap.String("table", m.tr.tableName), zap.Int64("row_id_base", newRowIDBase)) if checksum != nil { - log.L().Info("checksum base", zap.Any("checksum", checksum)) + log.FromContext(ctx).Info("checksum base", zap.Any("checksum", checksum)) } return checksum, newRowIDBase, nil } @@ -519,7 +519,7 @@ func (m *dbTableMetaMgr) CheckAndUpdateLocalChecksum(ctx context.Context, checks ck := verify.MakeKVChecksum(totalBytes, totalKvs, totalChecksum) baseTotalChecksum = &ck } - log.L().Info("check table checksum", zap.String("table", m.tr.tableName), + log.FromContext(ctx).Info("check table checksum", zap.String("table", m.tr.tableName), zap.Bool("checksum", needChecksum), zap.String("new_status", newStatus.String())) return } @@ -536,7 +536,7 @@ func (m *dbTableMetaMgr) FinishTable(ctx context.Context) error { func RemoveTableMetaByTableName(ctx context.Context, db *sql.DB, metaTable, tableName string) error { exec := &common.SQLWithRetry{ DB: db, - Logger: log.L(), + Logger: log.FromContext(ctx), } query := fmt.Sprintf("DELETE FROM %s", metaTable) var args []interface{} @@ -636,7 +636,7 @@ type storedCfgs struct { func (m *dbTaskMetaMgr) InitTask(ctx context.Context, source int64) error { exec := &common.SQLWithRetry{ DB: m.session, - Logger: log.L(), + Logger: log.FromContext(ctx), } // avoid override existing metadata if the meta is already inserted. stmt := fmt.Sprintf(`INSERT INTO %s (task_id, status, source_bytes) values (?, ?, ?) ON DUPLICATE KEY UPDATE state = ?`, m.tableName) @@ -647,7 +647,7 @@ func (m *dbTaskMetaMgr) InitTask(ctx context.Context, source int64) error { func (m *dbTaskMetaMgr) CheckTaskExist(ctx context.Context) (bool, error) { exec := &common.SQLWithRetry{ DB: m.session, - Logger: log.L(), + Logger: log.FromContext(ctx), } // avoid override existing metadata if the meta is already inserted. exist := false @@ -689,7 +689,7 @@ func (m *dbTaskMetaMgr) CheckTasksExclusively(ctx context.Context, action func(t defer conn.Close() exec := &common.SQLWithRetry{ DB: m.session, - Logger: log.L(), + Logger: log.FromContext(ctx), } err = exec.Exec(ctx, "enable pessimistic transaction", "SET SESSION tidb_txn_mode = 'pessimistic';") if err != nil { @@ -747,7 +747,7 @@ func (m *dbTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdutil.U defer conn.Close() exec := &common.SQLWithRetry{ DB: m.session, - Logger: log.L(), + Logger: log.FromContext(ctx), } err = exec.Exec(ctx, "enable pessimistic transaction", "SET SESSION tidb_txn_mode = 'pessimistic';") if err != nil { @@ -830,7 +830,7 @@ func (m *dbTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdutil.U // try to rollback the stopped schedulers cancelFunc := m.pd.MakeUndoFunctionByConfig(pausedCfg.RestoreCfg) if err1 := cancelFunc(ctx); err1 != nil { - log.L().Warn("undo remove schedulers failed", zap.Error(err1)) + log.FromContext(ctx).Warn("undo remove schedulers failed", zap.Error(err1)) } return errors.Trace(err) } @@ -878,7 +878,7 @@ func (m *dbTaskMetaMgr) CheckAndFinishRestore(ctx context.Context, finished bool defer conn.Close() exec := &common.SQLWithRetry{ DB: m.session, - Logger: log.L(), + Logger: log.FromContext(ctx), } err = exec.Exec(ctx, "enable pessimistic transaction", "SET SESSION tidb_txn_mode = 'pessimistic';") if err != nil { @@ -923,7 +923,7 @@ func (m *dbTaskMetaMgr) CheckAndFinishRestore(ctx context.Context, finished bool allFinished = false // check if other task still running if state == taskStateNormal { - log.L().Info("unfinished task found", zap.Int64("task_id", taskID), + log.FromContext(ctx).Info("unfinished task found", zap.Int64("task_id", taskID), zap.Stringer("status", status)) switchBack = false } @@ -957,7 +957,7 @@ func (m *dbTaskMetaMgr) CheckAndFinishRestore(ctx context.Context, finished bool return nil }) - log.L().Info("check all task finish status", zap.Bool("task_finished", finished), + log.FromContext(ctx).Info("check all task finish status", zap.Bool("task_finished", finished), zap.Bool("all_finished", allFinished), zap.Bool("switch_back", switchBack)) return switchBack, allFinished, err @@ -966,7 +966,7 @@ func (m *dbTaskMetaMgr) CheckAndFinishRestore(ctx context.Context, finished bool func (m *dbTaskMetaMgr) Cleanup(ctx context.Context) error { exec := &common.SQLWithRetry{ DB: m.session, - Logger: log.L(), + Logger: log.FromContext(ctx), } // avoid override existing metadata if the meta is already inserted. stmt := fmt.Sprintf("DROP TABLE %s;", m.tableName) @@ -979,7 +979,7 @@ func (m *dbTaskMetaMgr) Cleanup(ctx context.Context) error { func (m *dbTaskMetaMgr) CleanupTask(ctx context.Context) error { exec := &common.SQLWithRetry{ DB: m.session, - Logger: log.L(), + Logger: log.FromContext(ctx), } stmt := fmt.Sprintf("DELETE FROM %s WHERE task_id = %d;", m.tableName, m.taskID) err := exec.Exec(ctx, "clean up task", stmt) @@ -991,14 +991,20 @@ func (m *dbTaskMetaMgr) Close() { } func (m *dbTaskMetaMgr) CleanupAllMetas(ctx context.Context) error { - return MaybeCleanupAllMetas(ctx, m.session, m.schemaName, true) + return MaybeCleanupAllMetas(ctx, log.FromContext(ctx), m.session, m.schemaName, true) } // MaybeCleanupAllMetas remove the meta schema if there is no unfinished tables -func MaybeCleanupAllMetas(ctx context.Context, db *sql.DB, schemaName string, tableMetaExist bool) error { +func MaybeCleanupAllMetas( + ctx context.Context, + logger log.Logger, + db *sql.DB, + schemaName string, + tableMetaExist bool, +) error { exec := &common.SQLWithRetry{ DB: db, - Logger: log.L(), + Logger: logger, } // check if all tables are finished @@ -1009,7 +1015,7 @@ func MaybeCleanupAllMetas(ctx context.Context, db *sql.DB, schemaName string, ta return errors.Trace(err) } if cnt > 0 { - log.L().Warn("there are unfinished table in table meta table, cleanup skipped.") + logger.Warn("there are unfinished table in table meta table, cleanup skipped.") return nil } } diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 9778c5d4be352..31a48c620846a 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -187,6 +187,7 @@ const ( ) type Controller struct { + taskCtx context.Context cfg *config.Config dbMetas []*mydump.MDDatabaseMeta dbInfos map[string]*checkpoints.TidbDBInfo @@ -301,7 +302,7 @@ func NewRestoreControllerWithPauser( if err != nil { return nil, errors.Trace(err) } - errorMgr := errormanager.New(db, cfg) + errorMgr := errormanager.New(db, cfg, log.FromContext(ctx)) if err := errorMgr.Init(ctx); err != nil { return nil, common.ErrInitErrManager.Wrap(err).GenWithStackByArgs() } @@ -309,7 +310,7 @@ func NewRestoreControllerWithPauser( var backend backend.Backend switch cfg.TikvImporter.Backend { case config.BackendTiDB: - backend = tidb.NewTiDBBackend(db, cfg.TikvImporter.OnDuplicate, errorMgr) + backend = tidb.NewTiDBBackend(ctx, db, cfg.TikvImporter.OnDuplicate, errorMgr) case config.BackendLocal: var rLimit local.Rlim_t rLimit, err = local.GetSystemRLimit() @@ -325,7 +326,7 @@ func NewRestoreControllerWithPauser( if cfg.TikvImporter.DuplicateResolution != config.DupeResAlgNone { if err := tikv.CheckTiKVVersion(ctx, tls, cfg.TiDB.PdAddr, minTiKVVersionForDuplicateResolution, maxTiKVVersionForDuplicateResolution); err != nil { if berrors.Is(err, berrors.ErrVersionMismatch) { - log.L().Warn("TiKV version doesn't support duplicate resolution. The resolution algorithm will fall back to 'none'", zap.Error(err)) + log.FromContext(ctx).Warn("TiKV version doesn't support duplicate resolution. The resolution algorithm will fall back to 'none'", zap.Error(err)) cfg.TikvImporter.DuplicateResolution = config.DupeResAlgNone } else { return nil, common.ErrCheckKVVersion.Wrap(err).GenWithStackByArgs() @@ -364,6 +365,7 @@ func NewRestoreControllerWithPauser( } rc := &Controller{ + taskCtx: ctx, cfg: cfg, dbMetas: p.DBMetas, tableWorkers: nil, @@ -378,7 +380,7 @@ func NewRestoreControllerWithPauser( tls: tls, checkTemplate: NewSimpleTemplate(), - errorSummaries: makeErrorSummaries(log.L()), + errorSummaries: makeErrorSummaries(log.FromContext(ctx)), checkpointsDB: cpdb, saveCpCh: make(chan saveCp), closedEngineLimit: worker.NewPool(ctx, cfg.App.TableConcurrency*2, "closed-engine"), @@ -410,7 +412,7 @@ func (rc *Controller) Run(ctx context.Context) error { rc.cleanCheckpoints, } - task := log.L().Begin(zap.InfoLevel, "the whole procedure") + task := log.FromContext(ctx).Begin(zap.InfoLevel, "the whole procedure") var err error finished := false @@ -474,13 +476,14 @@ type schemaJob struct { } type restoreSchemaWorker struct { - ctx context.Context - quit context.CancelFunc - jobCh chan *schemaJob - errCh chan error - wg sync.WaitGroup - glue glue.Glue - store storage.ExternalStorage + ctx context.Context + quit context.CancelFunc + logger log.Logger + jobCh chan *schemaJob + errCh chan error + wg sync.WaitGroup + glue glue.Glue + store storage.ExternalStorage } func (worker *restoreSchemaWorker) addJob(sqlStr string, job *schemaJob) error { @@ -617,9 +620,9 @@ loop: break loop } } - logger := log.With(zap.String("db", job.dbName), zap.String("table", job.tblName)) + logger := worker.logger.With(zap.String("db", job.dbName), zap.String("table", job.tblName)) sqlWithRetry := common.SQLWithRetry{ - Logger: log.L(), + Logger: worker.logger, DB: session, } for _, stmt := range job.stmts { @@ -694,16 +697,17 @@ func (rc *Controller) restoreSchema(ctx context.Context) error { // create table with schema file // we can handle the duplicated created with createIfNotExist statement // and we will check the schema in TiDB is valid with the datafile in DataCheck later. - logTask := log.L().Begin(zap.InfoLevel, "restore all schema") + logTask := log.FromContext(ctx).Begin(zap.InfoLevel, "restore all schema") concurrency := mathutil.Min(rc.cfg.App.RegionConcurrency, 8) childCtx, cancel := context.WithCancel(ctx) worker := restoreSchemaWorker{ - ctx: childCtx, - quit: cancel, - jobCh: make(chan *schemaJob, concurrency), - errCh: make(chan error), - glue: rc.tidbGlue, - store: rc.store, + ctx: childCtx, + quit: cancel, + logger: log.FromContext(ctx), + jobCh: make(chan *schemaJob, concurrency), + errCh: make(chan error), + glue: rc.tidbGlue, + store: rc.store, } for i := 0; i < concurrency; i++ { go worker.doJob() @@ -740,12 +744,12 @@ func (rc *Controller) initCheckpoint(ctx context.Context) error { return common.ErrInitCheckpoint.Wrap(err).GenWithStackByArgs() } failpoint.Inject("InitializeCheckpointExit", func() { - log.L().Warn("exit triggered", zap.String("failpoint", "InitializeCheckpointExit")) + log.FromContext(ctx).Warn("exit triggered", zap.String("failpoint", "InitializeCheckpointExit")) os.Exit(0) }) rc.checkpointsWg.Add(1) // checkpointsWg will be done in `rc.listenCheckpointUpdates` - go rc.listenCheckpointUpdates() + go rc.listenCheckpointUpdates(log.FromContext(ctx)) // Estimate the number of chunks for progress reporting return rc.estimateChunkCountIntoMetrics(ctx) @@ -815,7 +819,7 @@ func verifyLocalFile(ctx context.Context, cpdb checkpoints.DB, dir string) error file := local.Engine{UUID: eID} err := file.Exist(dir) if err != nil { - log.L().Error("can't find local file", + log.FromContext(ctx).Error("can't find local file", zap.String("table name", tableName), zap.Int32("engine ID", engineID)) if os.IsNotExist(err) { @@ -906,7 +910,7 @@ func firstErr(errors ...error) error { func (rc *Controller) saveStatusCheckpoint(ctx context.Context, tableName string, engineID int32, err error, statusIfSucceed checkpoints.CheckpointStatus) error { merger := &checkpoints.StatusCheckpointMerger{Status: statusIfSucceed, EngineID: engineID} - logger := log.L().With(zap.String("table", tableName), zap.Int32("engine_id", engineID), + logger := log.FromContext(ctx).With(zap.String("table", tableName), zap.Int32("engine_id", engineID), zap.String("new_status", statusIfSucceed.MetricName()), zap.Error(err)) logger.Debug("update checkpoint") @@ -946,7 +950,7 @@ func (rc *Controller) saveStatusCheckpoint(ctx context.Context, tableName string } // listenCheckpointUpdates will combine several checkpoints together to reduce database load. -func (rc *Controller) listenCheckpointUpdates() { +func (rc *Controller) listenCheckpointUpdates(logger log.Logger) { var lock sync.Mutex coalesed := make(map[string]*checkpoints.TableCheckpointDiff) var waiters []chan<- error @@ -967,7 +971,7 @@ func (rc *Controller) listenCheckpointUpdates() { failpoint.Inject("SlowDownCheckpointUpdate", func() {}) if len(cpd) > 0 { - err := rc.checkpointsDB.Update(cpd) + err := rc.checkpointsDB.Update(rc.taskCtx, cpd) for _, w := range ws { w <- common.NormalizeOrWrapErr(common.ErrUpdateCheckpoint, err) } @@ -1031,7 +1035,7 @@ func (rc *Controller) listenCheckpointUpdates() { rc.checkpointsWg.Done() rc.checkpointsWg.Wait() if err := common.KillMySelf(); err != nil { - log.L().Warn("KillMySelf() failed to kill itself", log.ShortError(err)) + logger.Warn("KillMySelf() failed to kill itself", log.ShortError(err)) } for scp := range rc.saveCpCh { if scp.waitCh != nil { @@ -1100,10 +1104,10 @@ func (rc *Controller) buildRunPeriodicActionAndCancelFunc(ctx context.Context, s for { select { case <-ctx.Done(): - log.L().Warn("stopping periodic actions", log.ShortError(ctx.Err())) + log.FromContext(ctx).Warn("stopping periodic actions", log.ShortError(ctx.Err())) return case <-stop: - log.L().Info("everything imported, stopping periodic actions") + log.FromContext(ctx).Info("everything imported, stopping periodic actions") return case <-switchModeChan: @@ -1113,7 +1117,7 @@ func (rc *Controller) buildRunPeriodicActionAndCancelFunc(ctx context.Context, s case <-logProgressChan: metrics, ok := metric.FromContext(ctx) if !ok { - log.L().Warn("couldn't find metrics from context, skip log progress") + log.FromContext(ctx).Warn("couldn't find metrics from context, skip log progress") continue } // log the current progress periodically, so OPS will know that we're still working @@ -1216,7 +1220,7 @@ func (rc *Controller) buildRunPeriodicActionAndCancelFunc(ctx context.Context, s } // Note: a speed of 28 MiB/s roughly corresponds to 100 GiB/hour. - log.L().Info("progress", + log.FromContext(ctx).Info("progress", zap.String("total", fmt.Sprintf("%.1f%%", totalPercent*100)), // zap.String("files", fmt.Sprintf("%.0f/%.0f (%.1f%%)", finished, estimated, finished/estimated*100)), zap.String("tables", fmt.Sprintf("%.0f/%.0f%s", completedTables, totalTables, formatPercent(completedTables, totalTables))), @@ -1241,7 +1245,7 @@ func (rc *Controller) buildRunPeriodicActionAndCancelFunc(ctx context.Context, s } } }, func(do bool) { - log.L().Info("cancel periodic actions", zap.Bool("do", do)) + log.FromContext(ctx).Info("cancel periodic actions", zap.Bool("do", do)) for _, f := range cancelFuncs { f(do) } @@ -1289,7 +1293,7 @@ func (rc *Controller) keepPauseGCForDupeRes(ctx context.Context) (<-chan struct{ paused = true break } - log.L().Warn( + log.FromContext(ctx).Warn( "Failed to register GC safe point because the current minimum safe point is newer"+ " than what we assume, will retry newMinSafePoint next time", zap.Uint64("minSafePoint", minSafePoint), @@ -1312,11 +1316,11 @@ func (rc *Controller) keepPauseGCForDupeRes(ctx context.Context) (<-chan struct{ case <-ticker.C: minSafePoint, err := pdCli.UpdateServiceGCSafePoint(ctx, serviceID, ttl, safePoint) if err != nil { - log.L().Warn("Failed to register GC safe point", zap.Error(err)) + log.FromContext(ctx).Warn("Failed to register GC safe point", zap.Error(err)) continue } if minSafePoint > safePoint { - log.L().Warn("The current minimum safe point is newer than what we hold, duplicate records are at"+ + log.FromContext(ctx).Warn("The current minimum safe point is newer than what we hold, duplicate records are at"+ "risk of being GC and not detectable", zap.Uint64("safePoint", safePoint), zap.Uint64("minSafePoint", minSafePoint), @@ -1326,7 +1330,7 @@ func (rc *Controller) keepPauseGCForDupeRes(ctx context.Context) (<-chan struct{ case <-ctx.Done(): stopCtx, cancelFunc := context.WithTimeout(context.Background(), time.Second*5) if _, err := pdCli.UpdateServiceGCSafePoint(stopCtx, serviceID, 0, safePoint); err != nil { - log.L().Warn("Failed to reset safe point ttl to zero", zap.Error(err)) + log.FromContext(ctx).Warn("Failed to reset safe point ttl to zero", zap.Error(err)) } // just make compiler happy cancelFunc() @@ -1354,7 +1358,7 @@ func (rc *Controller) restoreTables(ctx context.Context) (finalErr error) { }() } - logTask := log.L().Begin(zap.InfoLevel, "restore all tables data") + logTask := log.FromContext(ctx).Begin(zap.InfoLevel, "restore all tables data") if rc.tableWorkers == nil { rc.tableWorkers = worker.NewPool(ctx, rc.cfg.App.TableConcurrency, "table") } @@ -1498,7 +1502,7 @@ func (rc *Controller) restoreTables(ctx context.Context) (finalErr error) { if err != nil { return errors.Trace(err) } - tr, err := NewTableRestore(tableName, tableMeta, dbInfo, tableInfo, cp, igCols.ColumnsMap()) + tr, err := NewTableRestore(tableName, tableMeta, dbInfo, tableInfo, cp, igCols.ColumnsMap(), log.FromContext(ctx)) if err != nil { return errors.Trace(err) } @@ -1684,12 +1688,12 @@ func (rc *Controller) outpuErrorSummary() { // do full compaction for the whole data. func (rc *Controller) fullCompact(ctx context.Context) error { if !rc.cfg.PostRestore.Compact { - log.L().Info("skip full compaction") + log.FromContext(ctx).Info("skip full compaction") return nil } // wait until any existing level-1 compact to complete first. - task := log.L().Begin(zap.InfoLevel, "wait for completion of existing level 1 compaction") + task := log.FromContext(ctx).Begin(zap.InfoLevel, "wait for completion of existing level 1 compaction") for !rc.compactState.CAS(compactStateIdle, compactStateDoing) { time.Sleep(100 * time.Millisecond) } @@ -1724,7 +1728,7 @@ func (rc *Controller) switchTiKVMode(ctx context.Context, mode sstpb.SwitchMode) return } - log.L().Info("switch import mode", zap.Stringer("mode", mode)) + log.FromContext(ctx).Info("switch import mode", zap.Stringer("mode", mode)) // It is fine if we miss some stores which did not switch to Import mode, // since we're running it periodically, so we exclude disconnected stores. @@ -1789,7 +1793,7 @@ func (rc *Controller) enforceDiskQuota(ctx context.Context) { m.LocalStorageUsageBytesGauge.WithLabelValues("mem").Set(float64(totalMemSize)) } - logger := log.With( + logger := log.FromContext(ctx).With( zap.Int64("diskSize", totalDiskSize), zap.Int64("memSize", totalMemSize), zap.Int64("quota", quota), @@ -1850,7 +1854,7 @@ func (rc *Controller) setGlobalVariables(ctx context.Context) error { // we should enable/disable new collation here since in server mode, tidb config // may be different in different tasks collate.SetNewCollationEnabledForTest(enabled) - log.L().Info("new_collation_enabled", zap.Bool("enabled", enabled)) + log.FromContext(ctx).Info("new_collation_enabled", zap.Bool("enabled", enabled)) return nil } @@ -1868,7 +1872,7 @@ func (rc *Controller) cleanCheckpoints(ctx context.Context) error { return nil } - logger := log.With( + logger := log.FromContext(ctx).With( zap.Stringer("keepAfterSuccess", rc.cfg.Checkpoint.KeepAfterSuccess), zap.Int64("taskID", rc.cfg.TaskID), ) @@ -1958,13 +1962,13 @@ func (rc *Controller) preCheckRequirements(ctx context.Context) error { needCheck = taskCheckpoints == nil } if needCheck { - err = rc.localResource(source) + err = rc.localResource(ctx, source) if err != nil { return common.ErrCheckLocalResource.Wrap(err).GenWithStackByArgs() } if err := rc.clusterResource(ctx, source); err != nil { if err1 := rc.taskMgr.CleanupTask(ctx); err1 != nil { - log.L().Warn("cleanup task failed", zap.Error(err1)) + log.FromContext(ctx).Warn("cleanup task failed", zap.Error(err1)) return common.ErrMetaMgrUnknown.Wrap(err).GenWithStackByArgs() } } @@ -1982,7 +1986,7 @@ func (rc *Controller) preCheckRequirements(ctx context.Context) error { if !taskExist && rc.taskMgr != nil { err := rc.taskMgr.CleanupTask(ctx) if err != nil { - log.L().Warn("cleanup task failed", zap.Error(err)) + log.FromContext(ctx).Warn("cleanup task failed", zap.Error(err)) } } return common.ErrPreCheckFailed.GenWithStackByArgs(rc.checkTemplate.FailedMsg()) @@ -2509,7 +2513,7 @@ func (cr *chunkRestore) encodeLoop( hasIgnoredEncodeErr := false if encodeErr != nil { - rowText := tidb.EncodeRowForRecord(t.encTable, rc.cfg.TiDB.SQLMode, lastRow.Row, cr.chunk.ColumnPermutation) + rowText := tidb.EncodeRowForRecord(ctx, t.encTable, rc.cfg.TiDB.SQLMode, lastRow.Row, cr.chunk.ColumnPermutation) encodeErr = rc.errorMgr.RecordTypeError(ctx, logger, t.tableName, cr.chunk.Key.Path, newOffset, rowText, encodeErr) if encodeErr != nil { err = common.ErrEncodeKV.Wrap(encodeErr).GenWithStackByArgs(&cr.chunk.Key, newOffset) @@ -2570,7 +2574,7 @@ func (cr *chunkRestore) restore( rc *Controller, ) error { // Create the encoder. - kvEncoder, err := rc.backend.NewEncoder(t.encTable, &kv.SessionOptions{ + kvEncoder, err := rc.backend.NewEncoder(ctx, t.encTable, &kv.SessionOptions{ SQLMode: rc.cfg.TiDB.SQLMode, Timestamp: cr.chunk.Timestamp, SysVars: rc.sysVars, diff --git a/br/pkg/lightning/restore/restore_test.go b/br/pkg/lightning/restore/restore_test.go index 852a3a4831e73..73996b08d3987 100644 --- a/br/pkg/lightning/restore/restore_test.go +++ b/br/pkg/lightning/restore/restore_test.go @@ -69,7 +69,7 @@ func TestNewTableRestore(t *testing.T) { for _, tc := range testCases { tableInfo := dbInfo.Tables[tc.name] tableName := common.UniqueTable("mockdb", tableInfo.Name) - tr, err := NewTableRestore(tableName, nil, dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil) + tr, err := NewTableRestore(tableName, nil, dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil, log.L()) require.NotNil(t, tr) require.NoError(t, err) } @@ -85,7 +85,7 @@ func TestNewTableRestoreFailure(t *testing.T) { }} tableName := common.UniqueTable("mockdb", "failure") - _, err := NewTableRestore(tableName, nil, dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil) + _, err := NewTableRestore(tableName, nil, dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil, log.L()) require.Regexp(t, `failed to tables\.TableFromMeta.*`, err.Error()) } @@ -218,7 +218,7 @@ func TestPreCheckFailed(t *testing.T) { metaMgrBuilder: failMetaMgrBuilder{}, checkTemplate: NewSimpleTemplate(), tidbGlue: g, - errorMgr: errormanager.New(nil, cfg), + errorMgr: errormanager.New(nil, cfg, log.L()), } mock.ExpectBegin() diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index ef3245bbcd7fb..491a59fa1c33b 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -67,6 +67,7 @@ func NewTableRestore( tableInfo *checkpoints.TidbTableInfo, cp *checkpoints.TableCheckpoint, ignoreColumns map[string]struct{}, + logger log.Logger, ) (*TableRestore, error) { idAlloc := kv.NewPanickingAllocators(cp.AllocBase) tbl, err := tables.TableFromMeta(idAlloc, tableInfo.Core) @@ -81,7 +82,7 @@ func NewTableRestore( tableMeta: tableMeta, encTable: tbl, alloc: idAlloc, - logger: log.With(zap.String("table", tableName)), + logger: logger.With(zap.String("table", tableName)), ignoreColumns: ignoreColumns, }, nil } @@ -118,7 +119,11 @@ func (tr *TableRestore) populateChunks(ctx context.Context, rc *Controller, cp * Timestamp: timestamp, } if len(chunk.Chunk.Columns) > 0 { - perms, err := parseColumnPermutations(tr.tableInfo.Core, chunk.Chunk.Columns, tr.ignoreColumns) + perms, err := parseColumnPermutations( + tr.tableInfo.Core, + chunk.Chunk.Columns, + tr.ignoreColumns, + log.FromContext(ctx)) if err != nil { return errors.Trace(err) } @@ -166,7 +171,7 @@ func (tr *TableRestore) RebaseChunkRowIDs(cp *checkpoints.TableCheckpoint, rowID // // The argument `columns` _must_ be in lower case. func (tr *TableRestore) initializeColumns(columns []string, ccp *checkpoints.ChunkCheckpoint) error { - colPerm, err := createColumnPermutation(columns, tr.ignoreColumns, tr.tableInfo.Core) + colPerm, err := createColumnPermutation(columns, tr.ignoreColumns, tr.tableInfo.Core, tr.logger) if err != nil { return err } @@ -174,7 +179,12 @@ func (tr *TableRestore) initializeColumns(columns []string, ccp *checkpoints.Chu return nil } -func createColumnPermutation(columns []string, ignoreColumns map[string]struct{}, tableInfo *model.TableInfo) ([]int, error) { +func createColumnPermutation( + columns []string, + ignoreColumns map[string]struct{}, + tableInfo *model.TableInfo, + logger log.Logger, +) ([]int, error) { var colPerm []int if len(columns) == 0 { colPerm = make([]int, 0, len(tableInfo.Columns)+1) @@ -195,7 +205,7 @@ func createColumnPermutation(columns []string, ignoreColumns map[string]struct{} } } else { var err error - colPerm, err = parseColumnPermutations(tableInfo, columns, ignoreColumns) + colPerm, err = parseColumnPermutations(tableInfo, columns, ignoreColumns, logger) if err != nil { return nil, errors.Trace(err) } @@ -620,11 +630,11 @@ func (tr *TableRestore) restoreEngine( if rc.isLocalBackend() && common.IsContextCanceledError(err) { // ctx is canceled, so to avoid Close engine failed, we use `context.Background()` here if _, err2 := dataEngine.Close(context.Background(), dataEngineCfg); err2 != nil { - log.L().Warn("flush all chunk checkpoints failed before manually exits", zap.Error(err2)) + log.FromContext(ctx).Warn("flush all chunk checkpoints failed before manually exits", zap.Error(err2)) return nil, errors.Trace(err) } if err2 := trySavePendingChunks(context.Background()); err2 != nil { - log.L().Warn("flush all chunk checkpoints failed before manually exits", zap.Error(err2)) + log.FromContext(ctx).Warn("flush all chunk checkpoints failed before manually exits", zap.Error(err2)) } } return nil, errors.Trace(err) @@ -864,7 +874,12 @@ func (tr *TableRestore) postProcess( return true, nil } -func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignoreColumns map[string]struct{}) ([]int, error) { +func parseColumnPermutations( + tableInfo *model.TableInfo, + columns []string, + ignoreColumns map[string]struct{}, + logger log.Logger, +) ([]int, error) { colPerm := make([]int, 0, len(tableInfo.Columns)+1) columnMap := make(map[string]int) @@ -896,7 +911,7 @@ func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignor if _, ignore := ignoreColumns[colInfo.Name.L]; !ignore { colPerm = append(colPerm, i) } else { - log.L().Debug("column ignored by user requirements", + logger.Debug("column ignored by user requirements", zap.Stringer("table", tableInfo.Name), zap.String("colName", colInfo.Name.O), zap.Stringer("colType", &colInfo.FieldType), @@ -905,7 +920,7 @@ func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignor } } else { if len(colInfo.GeneratedExprString) == 0 { - log.L().Warn("column missing from data file, going to fill with default value", + logger.Warn("column missing from data file, going to fill with default value", zap.Stringer("table", tableInfo.Name), zap.String("colName", colInfo.Name.O), zap.Stringer("colType", &colInfo.FieldType), diff --git a/br/pkg/lightning/restore/table_restore_test.go b/br/pkg/lightning/restore/table_restore_test.go index 1e32846babf77..87aa389c7167b 100644 --- a/br/pkg/lightning/restore/table_restore_test.go +++ b/br/pkg/lightning/restore/table_restore_test.go @@ -163,7 +163,7 @@ func (s *tableRestoreSuiteBase) setupSuite(t *testing.T) { func (s *tableRestoreSuiteBase) setupTest(t *testing.T) { // Collect into the test TableRestore structure var err error - s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, s.tableInfo, &checkpoints.TableCheckpoint{}, nil) + s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, s.tableInfo, &checkpoints.TableCheckpoint{}, nil, log.L()) require.NoError(t, err) s.cfg = config.NewConfig() @@ -358,12 +358,12 @@ func (s *tableRestoreSuite) TestRestoreEngineFailed() { require.NoError(s.T(), err) _, indexUUID := backend.MakeUUID("`db`.`table`", -1) _, dataUUID := backend.MakeUUID("`db`.`table`", 0) - realBackend := tidb.NewTiDBBackend(nil, "replace", nil) + realBackend := tidb.NewTiDBBackend(ctx, nil, "replace", nil) mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) mockBackend.EXPECT().CloseEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any()). - Return(realBackend.NewEncoder(tbl, &kv.SessionOptions{})). + mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any(), gomock.Any()). + Return(realBackend.NewEncoder(ctx, tbl, &kv.SessionOptions{})). AnyTimes() mockBackend.EXPECT().MakeEmptyRows().Return(realBackend.MakeEmptyRows()).AnyTimes() mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), dataUUID).Return(noop.Writer{}, nil) @@ -455,7 +455,7 @@ func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { cfg.Mydumper.StrictFormat = true rc := &Controller{cfg: cfg, ioWorkers: worker.NewPool(context.Background(), 1, "io"), store: store} - tr, err := NewTableRestore("`db`.`table`", tableMeta, s.dbInfo, s.tableInfo, &checkpoints.TableCheckpoint{}, nil) + tr, err := NewTableRestore("`db`.`table`", tableMeta, s.dbInfo, s.tableInfo, &checkpoints.TableCheckpoint{}, nil, log.L()) require.NoError(s.T(), err) require.NoError(s.T(), tr.populateChunks(context.Background(), rc, cp)) @@ -720,7 +720,7 @@ func (s *tableRestoreSuite) TestInitializeColumnsGenerated() { require.NoError(s.T(), err) core.State = model.StatePublic tableInfo := &checkpoints.TidbTableInfo{Name: "table", DB: "db", Core: core} - s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil) + s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil, log.L()) require.NoError(s.T(), err) ccp := &checkpoints.ChunkCheckpoint{} @@ -938,7 +938,7 @@ func (s *tableRestoreSuite) TestTableRestoreMetrics() { closedEngineLimit: worker.NewPool(ctx, 1, "closed_engine"), store: s.store, metaMgrBuilder: noopMetaMgrBuilder{}, - errorMgr: errormanager.New(nil, cfg), + errorMgr: errormanager.New(nil, cfg, log.L()), taskMgr: noopTaskMetaMgr{}, } go func() { @@ -990,7 +990,7 @@ func (s *tableRestoreSuite) TestSaveStatusCheckpoint() { checkpointsDB: checkpoints.NewNullCheckpointsDB(), } rc.checkpointsWg.Add(1) - go rc.listenCheckpointUpdates() + go rc.listenCheckpointUpdates(log.L()) rc.errorSummaries = makeErrorSummaries(log.L()) @@ -1327,11 +1327,11 @@ func (s *tableRestoreSuite) TestEstimate() { require.NoError(s.T(), err) mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).AnyTimes() - mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any()).Return(kv.NewTableKVEncoder(tbl, &kv.SessionOptions{ + mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any(), gomock.Any()).Return(kv.NewTableKVEncoder(tbl, &kv.SessionOptions{ SQLMode: s.cfg.TiDB.SQLMode, Timestamp: 0, AutoRandomSeed: 0, - }, nil)).AnyTimes() + }, nil, log.L())).AnyTimes() importer := backend.MakeBackend(mockBackend) s.cfg.TikvImporter.Backend = config.BackendLocal diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index 7bdbf09c665a7..9cf278a67d1cc 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.go @@ -104,7 +104,7 @@ func DBFromConfig(ctx context.Context, dsn config.DBStore) (*sql.DB, error) { for k, v := range vars { q := fmt.Sprintf("SET SESSION %s = '%s';", k, v) if _, err1 := db.ExecContext(ctx, q); err1 != nil { - log.L().Warn("set session variable failed, will skip this query", zap.String("query", q), + log.FromContext(ctx).Warn("set session variable failed, will skip this query", zap.String("query", q), zap.Error(err1)) delete(vars, k) } @@ -142,7 +142,7 @@ func (timgr *TiDBManager) Close() { } func InitSchema(ctx context.Context, g glue.Glue, database string, tablesSchema map[string]string) error { - logger := log.With(zap.String("db", database)) + logger := log.FromContext(ctx).With(zap.String("db", database)) sqlExecutor := g.GetSQLExecutor() var createDatabase strings.Builder @@ -223,7 +223,7 @@ func createIfNotExistsStmt(p *parser.Parser, createTable, dbName, tblName string func (timgr *TiDBManager) DropTable(ctx context.Context, tableName string) error { sql := common.SQLWithRetry{ DB: timgr.db, - Logger: log.With(zap.String("table", tableName)), + Logger: log.FromContext(ctx).With(zap.String("table", tableName)), } return sql.Exec(ctx, "drop table", "DROP TABLE "+tableName) } @@ -287,7 +287,7 @@ func LoadSchemaInfo( func ObtainGCLifeTime(ctx context.Context, db *sql.DB) (string, error) { var gcLifeTime string - err := common.SQLWithRetry{DB: db, Logger: log.L()}.QueryRow( + err := common.SQLWithRetry{DB: db, Logger: log.FromContext(ctx)}.QueryRow( ctx, "obtain GC lifetime", "SELECT VARIABLE_VALUE FROM mysql.tidb WHERE VARIABLE_NAME = 'tikv_gc_life_time'", @@ -299,7 +299,7 @@ func ObtainGCLifeTime(ctx context.Context, db *sql.DB) (string, error) { func UpdateGCLifeTime(ctx context.Context, db *sql.DB, gcLifeTime string) error { sql := common.SQLWithRetry{ DB: db, - Logger: log.With(zap.String("gcLifeTime", gcLifeTime)), + Logger: log.FromContext(ctx).With(zap.String("gcLifeTime", gcLifeTime)), } return sql.Exec(ctx, "update GC lifetime", "UPDATE mysql.tidb SET VARIABLE_VALUE = ? WHERE VARIABLE_NAME = 'tikv_gc_life_time'", @@ -326,10 +326,10 @@ func ObtainImportantVariables(ctx context.Context, g glue.SQLExecutor, needTiDBV } } query.WriteString("')") - kvs, err := g.QueryStringsWithLog(ctx, query.String(), "obtain system variables", log.L()) + kvs, err := g.QueryStringsWithLog(ctx, query.String(), "obtain system variables", log.FromContext(ctx)) if err != nil { // error is not fatal - log.L().Warn("obtain system variables failed, use default variables instead", log.ShortError(err)) + log.FromContext(ctx).Warn("obtain system variables failed, use default variables instead", log.ShortError(err)) } // convert result into a map. fill in any missing variables with default values. @@ -359,7 +359,7 @@ func ObtainNewCollationEnabled(ctx context.Context, g glue.SQLExecutor) (bool, e ctx, "SELECT variable_value FROM mysql.tidb WHERE variable_name = 'new_collation_enabled'", "obtain new collation enabled", - log.L(), + log.FromContext(ctx), ) if err == nil && newCollationVal == "True" { newCollationEnabled = true @@ -379,7 +379,7 @@ func ObtainNewCollationEnabled(ctx context.Context, g glue.SQLExecutor) (bool, e // See: https://github.com/pingcap/tidb/blob/64698ef9a3358bfd0fdc323996bb7928a56cadca/ddl/ddl_api.go#L2528-L2533 func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName string, incr uint64) error { var query string - logger := log.With(zap.String("table", tableName), zap.Uint64("auto_increment", incr)) + logger := log.FromContext(ctx).With(zap.String("table", tableName), zap.Uint64("auto_increment", incr)) if incr > math.MaxInt64 { // automatically set max value logger.Warn("auto_increment out of the maximum value TiDB supports, automatically set to the max", zap.Uint64("auto_increment", incr)) @@ -401,7 +401,7 @@ func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName strin } func AlterAutoRandom(ctx context.Context, g glue.SQLExecutor, tableName string, randomBase uint64, maxAutoRandom uint64) error { - logger := log.With(zap.String("table", tableName), zap.Uint64("auto_random", randomBase)) + logger := log.FromContext(ctx).With(zap.String("table", tableName), zap.Uint64("auto_random", randomBase)) if randomBase == maxAutoRandom+1 { // insert a tuple with key maxAutoRandom randomBase = maxAutoRandom diff --git a/br/pkg/lightning/run_options.go b/br/pkg/lightning/run_options.go index 2665faddeb40a..a7b5b90770c02 100644 --- a/br/pkg/lightning/run_options.go +++ b/br/pkg/lightning/run_options.go @@ -16,8 +16,10 @@ package lightning import ( "github.com/pingcap/tidb/br/pkg/lightning/glue" + "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/util/promutil" + "go.uber.org/zap" ) type options struct { @@ -27,6 +29,7 @@ type options struct { checkpointName string promFactory promutil.Factory promRegistry promutil.Registry + logger log.Logger } type Option func(*options) @@ -71,3 +74,10 @@ func WithPromRegistry(r promutil.Registry) Option { o.promRegistry = r } } + +// WithLogger sets the logger to a lightning task. +func WithLogger(logger *zap.Logger) Option { + return func(o *options) { + o.logger = log.Logger{Logger: logger} + } +} diff --git a/br/pkg/mock/backend.go b/br/pkg/mock/backend.go index 7eba5180694ac..04896d4a8efd1 100644 --- a/br/pkg/mock/backend.go +++ b/br/pkg/mock/backend.go @@ -1,8 +1,6 @@ // Code generated by MockGen. DO NOT EDIT. // Source: github.com/pingcap/tidb/br/pkg/lightning/backend (interfaces: AbstractBackend,EngineWriter) -// $ mockgen -package mock -mock_names 'AbstractBackend=MockBackend' github.com/pingcap/tidb/br/pkg/lightning/backend AbstractBackend,EngineWriter - // Package mock is a generated GoMock package. package mock @@ -228,18 +226,18 @@ func (mr *MockBackendMockRecorder) MakeEmptyRows() *gomock.Call { } // NewEncoder mocks base method. -func (m *MockBackend) NewEncoder(arg0 table.Table, arg1 *kv.SessionOptions) (kv.Encoder, error) { +func (m *MockBackend) NewEncoder(arg0 context.Context, arg1 table.Table, arg2 *kv.SessionOptions) (kv.Encoder, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "NewEncoder", arg0, arg1) + ret := m.ctrl.Call(m, "NewEncoder", arg0, arg1, arg2) ret0, _ := ret[0].(kv.Encoder) ret1, _ := ret[1].(error) return ret0, ret1 } // NewEncoder indicates an expected call of NewEncoder. -func (mr *MockBackendMockRecorder) NewEncoder(arg0, arg1 interface{}) *gomock.Call { +func (mr *MockBackendMockRecorder) NewEncoder(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NewEncoder", reflect.TypeOf((*MockBackend)(nil).NewEncoder), arg0, arg1) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NewEncoder", reflect.TypeOf((*MockBackend)(nil).NewEncoder), arg0, arg1, arg2) } // OpenEngine mocks base method. From cca81e832eadc7312983548c1f0ff53764238bfc Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 22 Jun 2022 11:46:36 +0800 Subject: [PATCH 02/21] charset: fix data race in the Collation.IsDefault (#35370) close pingcap/tidb#35145 --- executor/point_get_test.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/executor/point_get_test.go b/executor/point_get_test.go index e366c99d1e37c..95b2a4dbe9e4a 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -18,7 +18,6 @@ import ( "context" "fmt" "strings" - "sync" "testing" "time" @@ -783,7 +782,6 @@ func TestPointGetLockExistKey(t *testing.T) { )) } - var wg sync.WaitGroup for i, one := range []struct { rc bool key string @@ -793,14 +791,12 @@ func TestPointGetLockExistKey(t *testing.T) { {rc: true, key: "primary key"}, {rc: true, key: "unique key"}, } { - wg.Add(1) + tableName := fmt.Sprintf("t_%d", i) - go func(rc bool, key string, tableName string) { - defer wg.Done() + func(rc bool, key string, tableName string) { testLock(rc, key, tableName) }(one.rc, one.key, tableName) } - wg.Wait() } func TestWithTiDBSnapshot(t *testing.T) { From 2910dee858bf608751125b1b111f91fef413c70c Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Wed, 22 Jun 2022 13:52:37 +0800 Subject: [PATCH 03/21] executor: fix wrong runtime stats for `ShuffleReceiver` (#35293) close pingcap/tidb#34987 --- executor/builder.go | 24 +++++++++++++++++------- executor/explain_test.go | 9 +++++++++ 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index d86200b573058..736a8204e5a96 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4509,6 +4509,7 @@ func (b *executorBuilder) buildShuffle(v *plannercore.PhysicalShuffle) *ShuffleE concurrency: v.Concurrency, } + // 1. initialize the splitters splitters := make([]partitionSplitter, len(v.ByItemArrays)) switch v.SplitterType { case plannercore.PartitionHashSplitterType: @@ -4524,6 +4525,7 @@ func (b *executorBuilder) buildShuffle(v *plannercore.PhysicalShuffle) *ShuffleE } shuffle.splitters = splitters + // 2. initialize the data sources (build the data sources from physical plan to executors) shuffle.dataSources = make([]Executor, len(v.DataSources)) for i, dataSource := range v.DataSources { shuffle.dataSources[i] = b.build(dataSource) @@ -4532,13 +4534,24 @@ func (b *executorBuilder) buildShuffle(v *plannercore.PhysicalShuffle) *ShuffleE } } + // 3. initialize the workers head := v.Children()[0] + // A `PhysicalShuffleReceiverStub` for every worker have the same `DataSource` but different `Receiver`. + // We preallocate `PhysicalShuffleReceiverStub`s here and reuse them below. + stubs := make([]*plannercore.PhysicalShuffleReceiverStub, 0, len(v.DataSources)) + for _, dataSource := range v.DataSources { + stub := plannercore.PhysicalShuffleReceiverStub{ + DataSource: dataSource, + }.Init(b.ctx, dataSource.Stats(), dataSource.SelectBlockOffset(), nil) + stub.SetSchema(dataSource.Schema()) + stubs = append(stubs, stub) + } shuffle.workers = make([]*shuffleWorker, shuffle.concurrency) for i := range shuffle.workers { receivers := make([]*shuffleReceiver, len(v.DataSources)) for j, dataSource := range v.DataSources { receivers[j] = &shuffleReceiver{ - baseExecutor: newBaseExecutor(b.ctx, dataSource.Schema(), dataSource.ID()), + baseExecutor: newBaseExecutor(b.ctx, dataSource.Schema(), stubs[j].ID()), } } @@ -4546,12 +4559,9 @@ func (b *executorBuilder) buildShuffle(v *plannercore.PhysicalShuffle) *ShuffleE receivers: receivers, } - for j, dataSource := range v.DataSources { - stub := plannercore.PhysicalShuffleReceiverStub{ - Receiver: (unsafe.Pointer)(receivers[j]), - DataSource: dataSource, - }.Init(b.ctx, dataSource.Stats(), dataSource.SelectBlockOffset(), nil) - stub.SetSchema(dataSource.Schema()) + for j := range v.DataSources { + stub := stubs[j] + stub.Receiver = (unsafe.Pointer)(receivers[j]) v.Tails[j].SetChildren(stub) } diff --git a/executor/explain_test.go b/executor/explain_test.go index 01cd918d83a9e..9d0545fb5a6b0 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -310,6 +310,7 @@ func TestCheckActRowsWithUnistore(t *testing.T) { tk.MustExec("create table t_unistore_act_rows(a int, b int, index(a, b))") tk.MustExec("insert into t_unistore_act_rows values (1, 0), (1, 0), (2, 0), (2, 1)") tk.MustExec("analyze table t_unistore_act_rows") + tk.MustExec("set @@tidb_merge_join_concurrency= 5;") type testStruct struct { sql string @@ -353,6 +354,14 @@ func TestCheckActRowsWithUnistore(t *testing.T) { sql: "with cte(a) as (select a from t_unistore_act_rows) select (select 1 from cte limit 1) from cte;", expected: []string{"4", "4", "4", "4", "4"}, }, + { + sql: "select a, row_number() over (partition by b) from t_unistore_act_rows;", + expected: []string{"4", "4", "4", "4", "4", "4", "4"}, + }, + { + sql: "select /*+ merge_join(t1, t2) */ * from t_unistore_act_rows t1 join t_unistore_act_rows t2 on t1.b = t2.b;", + expected: []string{"10", "10", "4", "4", "4", "4", "4", "4", "4", "4", "4", "4"}, + }, } // Default RPC encoding may cause statistics explain result differ and then the test unstable. From 007d64c2e3f74915c71ad417e1f2b5f0857063c3 Mon Sep 17 00:00:00 2001 From: luzizhuo <496521310@qq.com> Date: Wed, 22 Jun 2022 14:22:37 +0800 Subject: [PATCH 04/21] readme: Add the contributor swag form. (#34437) ref pingcap/tidb#34452 --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 11163a7d2edfd..ca96703510e40 100644 --- a/README.md +++ b/README.md @@ -60,7 +60,7 @@ The [community repository](https://github.com/pingcap/community) hosts all infor [contribution-map](https://github.com/pingcap/tidb-map/blob/master/maps/contribution-map.md#tidb-is-an-open-source-distributed-htap-database-compatible-with-the-mysql-protocol) -Contributions are welcomed and greatly appreciated. See [Contribution to TiDB](https://pingcap.github.io/tidb-dev-guide/contribute-to-tidb/introduction.html) for details on typical contribution workflows. For more contributing information, click on the contributor icon above. +Contributions are welcomed and greatly appreciated. All the contributors are welcomed to claim your reward by filing this [form](https://forms.pingcap.com/f/tidb-contribution-swag). See [Contribution to TiDB](https://pingcap.github.io/tidb-dev-guide/contribute-to-tidb/introduction.html) for details on typical contribution workflows. For more contributing information, click on the contributor icon above. ## Adopters From 494759d4aa18437f994c9aa6781fec74c8f9317c Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 22 Jun 2022 15:56:36 +0800 Subject: [PATCH 05/21] *: remove useless gomod (#34107) ref pingcap/tidb#31716 --- tests/globalkilltest/global_kill_test.go | 13 +- tests/globalkilltest/go.mod | 46 - tests/globalkilltest/go.sum | 312 ------ tests/graceshutdown/go.mod | 27 - tests/graceshutdown/go.sum | 87 -- tests/readonlytest/go.mod | 14 - tests/readonlytest/go.sum | 1257 ---------------------- 7 files changed, 8 insertions(+), 1748 deletions(-) delete mode 100644 tests/globalkilltest/go.mod delete mode 100644 tests/globalkilltest/go.sum delete mode 100644 tests/graceshutdown/go.mod delete mode 100644 tests/graceshutdown/go.sum delete mode 100644 tests/readonlytest/go.mod delete mode 100644 tests/readonlytest/go.sum diff --git a/tests/globalkilltest/global_kill_test.go b/tests/globalkilltest/global_kill_test.go index 85738f873970d..0e4caf303074d 100644 --- a/tests/globalkilltest/global_kill_test.go +++ b/tests/globalkilltest/global_kill_test.go @@ -49,7 +49,10 @@ var ( pdClientPath = flag.String("pd", "127.0.0.1:2379", "pd client path") - lostConnectionToPDTimeout = flag.Int("conn_lost", 5, "lost connection to PD timeout, should be the same as TiDB ldflag ") + // nolint: unused, deadcode + lostConnectionToPDTimeout = flag.Int("conn_lost", 5, "lost connection to PD timeout, should be the same as TiDB ldflag ") + + // nolint: unused, deadcode timeToCheckPDConnectionRestored = flag.Int("conn_restored", 1, "time to check PD connection restored, should be the same as TiDB ldflag ") ) @@ -64,7 +67,7 @@ type GlobalKillSuite struct { pdCli *clientv3.Client pdErr error - clusterId string + clusterID string pdProc *exec.Cmd tikvProc *exec.Cmd } @@ -74,7 +77,7 @@ func createGloabalKillSuite(t *testing.T) (s *GlobalKillSuite, clean func()) { err := logutil.InitLogger(&logutil.LogConfig{Config: log.Config{Level: *logLevel}}) require.NoError(t, err) - s.clusterId = time.Now().Format(time.RFC3339Nano) + s.clusterID = time.Now().Format(time.RFC3339Nano) err = s.startCluster() require.NoError(t, err) s.pdCli, s.pdErr = s.connectPD() @@ -157,12 +160,12 @@ func (s *GlobalKillSuite) startPD(dataDir string) (err error) { } func (s *GlobalKillSuite) startCluster() (err error) { - err = s.startPD(s.clusterId) + err = s.startPD(s.clusterID) if err != nil { return } - err = s.startTiKV(s.clusterId) + err = s.startTiKV(s.clusterID) if err != nil { return } diff --git a/tests/globalkilltest/go.mod b/tests/globalkilltest/go.mod deleted file mode 100644 index 85c7fec1f425f..0000000000000 --- a/tests/globalkilltest/go.mod +++ /dev/null @@ -1,46 +0,0 @@ -module github.com/pingcap/tests/globalkilltest - -go 1.18 - -require ( - github.com/go-sql-driver/mysql v1.6.0 - github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c - github.com/pingcap/log v1.1.0 - github.com/pingcap/tidb v2.0.11+incompatible - github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df - go.etcd.io/etcd/client/v3 v3.5.2 - go.uber.org/zap v1.21.0 - google.golang.org/grpc v1.44.0 -) - -require ( - github.com/benbjohnson/clock v1.3.0 // indirect - github.com/coreos/go-semver v0.3.0 // indirect - github.com/coreos/go-systemd/v22 v22.3.2 // indirect - github.com/davecgh/go-spew v1.1.1 // indirect - github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang/protobuf v1.5.2 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect - github.com/opentracing/opentracing-go v1.2.0 // indirect - github.com/pmezard/go-difflib v1.0.0 // indirect - go.etcd.io/etcd/api/v3 v3.5.2 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.2 // indirect - go.uber.org/atomic v1.9.0 // indirect - go.uber.org/multierr v1.8.0 // indirect - golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd // indirect - golang.org/x/sys v0.0.0-20220408201424-a24fb2fb8a0f // indirect - golang.org/x/text v0.3.7 // indirect - google.golang.org/genproto v0.0.0-20220216160803-4663080d8bc8 // indirect - google.golang.org/protobuf v1.27.1 // indirect - gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect - gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect -) - -// fix potential security issue(CVE-2020-26160) introduced by indirect dependency. -replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible - -replace github.com/pingcap/tidb => ../../ - -replace github.com/pingcap/tidb/parser => ../../parser - -replace google.golang.org/grpc => google.golang.org/grpc v1.29.1 diff --git a/tests/globalkilltest/go.sum b/tests/globalkilltest/go.sum deleted file mode 100644 index 10a048e20d254..0000000000000 --- a/tests/globalkilltest/go.sum +++ /dev/null @@ -1,312 +0,0 @@ -cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= -github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= -github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= -github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= -github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= -github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/coocood/freecache v1.2.1 h1:/v1CqMq45NFH9mp/Pt142reundeBM0dVUD3osQBeu/U= -github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= -github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= -github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= -github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 h1:X6mKGhCFOxrKeeHAjv/3UvT6e5RRxW6wRdlqlV6/H4w= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= -github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= -github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= -github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= -github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= -github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.7 h1:81/ik6ipDQS2aGcBfIN5dHDB36BwrStyeAQquSYCV4o= -github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= -github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= -github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= -github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= -github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= -github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= -github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= -github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= -github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= -github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= -github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= -github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTmyFqUwr+jcCvpVkK7sumiz+ko5H9eq4= -github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 h1:kJolJWbyadVeL8RKBlqmXQR7FRKPsIeU85TUYyhbhiQ= -github.com/pingcap/kvproto v0.0.0-20220328072018-6e75c12dbd73 h1:jKixsi6Iw00hL0+o23hmr8BNzlsQP9pShHTOwyuf/Os= -github.com/pingcap/log v1.1.0 h1:ELiPxACz7vdo1qAvvaWJg1NrYFoY6gqAh/+Uo6aXdD8= -github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/tipb v0.0.0-20220215045658-d12dec7a7609 h1:BiCS1ZRnW0szOvTAa3gCqWIhyo+hv83SVaBgrUghXIU= -github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= -github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= -github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= -github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= -github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf3/SASFAGtQ= -github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= -github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= -github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= -github.com/prometheus/common v0.32.1 h1:hWIdL3N2HoUx3B8j3YN9mWor0qhY/NlEKZEaXxuIRh4= -github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= -github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= -github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk= -github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= -github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= -github.com/shirou/gopsutil/v3 v3.21.12 h1:VoGxEW2hpmz0Vt3wUvHIl9fquzYLNpVpgNNB7pGJimA= -github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df h1:rh3VYpfvzXRbJ90ymx1yfhGl/wq8ac2m/cUbao61kwY= -github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/tikv/client-go/v2 v2.0.1-0.20220406091203-f73ec0e675f4 h1:bi/tuV42dQCu7TTTOwHQW6cHVrV1fhet+Hzo5CUODBQ= -github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 h1:jxgmKOscXSjaFEKQGRyY5qOpK8hLqxs2irb/uDJMtwk= -github.com/twmb/murmur3 v1.1.3 h1:D83U0XYKcHRYwYIpBKf3Pks91Z0Byda/9SJ8B6EMRcA= -github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= -github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= -github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= -go.etcd.io/etcd/api/v3 v3.5.2 h1:tXok5yLlKyuQ/SXSjtqHc4uzNaMqZi2XsoSPr/LlJXI= -go.etcd.io/etcd/api/v3 v3.5.2/go.mod h1:5GB2vv4A4AOn3yk7MftYGHkUfGtDHnEraIjym4dYz5A= -go.etcd.io/etcd/client/pkg/v3 v3.5.2 h1:4hzqQ6hIb3blLyQ8usCU4h3NghkqcsohEQ3o3VetYxE= -go.etcd.io/etcd/client/pkg/v3 v3.5.2/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= -go.etcd.io/etcd/client/v3 v3.5.2 h1:WdnejrUtQC4nCxK0/dLTMqKOB+U5TP/2Ya0BJL+1otA= -go.etcd.io/etcd/client/v3 v3.5.2/go.mod h1:kOOaWFFgHygyT0WlSmL8TJiXmMysO/nNUlEsSsN6W4o= -go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= -go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= -go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= -go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= -go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= -go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= -go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.21.0 h1:WefMeulhovoZ2sYXz7st6K0sLj7bBhpiFaud4r4zST8= -go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= -golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd h1:O7DYs+zxREGLKzKoMQrtrEacpb0ZVXA5rIwylE2Xchk= -golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ= -golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220408201424-a24fb2fb8a0f h1:8w7RhxzTVgUzw/AH/9mUV5q0vMgy40SQRursCcfmkCw= -golang.org/x/sys v0.0.0-20220408201424-a24fb2fb8a0f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= -google.golang.org/genproto v0.0.0-20220216160803-4663080d8bc8 h1:divpuJZKgX3Qt7MFDE5v62yu0yQcQbTCD9VJp9leX58= -google.golang.org/genproto v0.0.0-20220216160803-4663080d8bc8/go.mod h1:kGP+zUP2Ddo0ayMi4YuN7C3WZyJvGLZRh8Z5wnAqvEI= -google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ= -google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= -gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= -gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= diff --git a/tests/graceshutdown/go.mod b/tests/graceshutdown/go.mod deleted file mode 100644 index df86a6bebb48a..0000000000000 --- a/tests/graceshutdown/go.mod +++ /dev/null @@ -1,27 +0,0 @@ -module graceshutdown - -go 1.18 - -require ( - github.com/go-sql-driver/mysql v1.6.0 - github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c - github.com/pingcap/log v1.1.0 - github.com/pingcap/tidb v2.0.11+incompatible - github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df - go.uber.org/goleak v1.1.12 - go.uber.org/zap v1.21.0 -) - -require ( - github.com/benbjohnson/clock v1.3.0 // indirect - github.com/davecgh/go-spew v1.1.1 // indirect - github.com/pmezard/go-difflib v1.0.0 // indirect - go.uber.org/atomic v1.9.0 // indirect - go.uber.org/multierr v1.8.0 // indirect - gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect - gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect -) - -replace github.com/pingcap/tidb => ../../ - -replace github.com/pingcap/tidb/parser => ../../parser diff --git a/tests/graceshutdown/go.sum b/tests/graceshutdown/go.sum deleted file mode 100644 index 8f811d384e31c..0000000000000 --- a/tests/graceshutdown/go.sum +++ /dev/null @@ -1,87 +0,0 @@ -github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= -github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= -github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= -github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= -github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= -github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTmyFqUwr+jcCvpVkK7sumiz+ko5H9eq4= -github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/log v1.1.0 h1:ELiPxACz7vdo1qAvvaWJg1NrYFoY6gqAh/+Uo6aXdD8= -github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= -github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df h1:rh3VYpfvzXRbJ90ymx1yfhGl/wq8ac2m/cUbao61kwY= -github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= -go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= -go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= -go.uber.org/multierr v1.8.0 h1:dg6GjLku4EH+249NNmoIciG9N/jURbDG+pFlTkhzIC8= -go.uber.org/multierr v1.8.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= -go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.21.0 h1:WefMeulhovoZ2sYXz7st6K0sLj7bBhpiFaud4r4zST8= -go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.8 h1:P1HhGGuLW4aAclzjtmJdf0mJOjVUZUzOTqkAkWL+l6w= -golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= -gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/tests/readonlytest/go.mod b/tests/readonlytest/go.mod deleted file mode 100644 index cc0b22e3b1343..0000000000000 --- a/tests/readonlytest/go.mod +++ /dev/null @@ -1,14 +0,0 @@ -module github.com/pingcap/tidb/tests/readonlytest - -go 1.16 - -require ( - github.com/go-sql-driver/mysql v1.6.0 - github.com/pingcap/tidb v2.0.11+incompatible - github.com/stretchr/testify v1.7.0 - go.uber.org/goleak v1.1.12 -) - -replace github.com/pingcap/tidb => ../../ - -replace github.com/pingcap/tidb/parser => ../../parser diff --git a/tests/readonlytest/go.sum b/tests/readonlytest/go.sum deleted file mode 100644 index 387cfacb99a28..0000000000000 --- a/tests/readonlytest/go.sum +++ /dev/null @@ -1,1257 +0,0 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= -cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= -cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= -cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= -cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= -cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= -cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= -cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= -cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= -cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= -cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= -cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= -cloud.google.com/go v0.72.0/go.mod h1:M+5Vjvlc2wnp6tjzE102Dw08nGShTscUx2nZMufOKPI= -cloud.google.com/go v0.74.0/go.mod h1:VV1xSbzvo+9QJOxLDaJfTjx5e+MePCpCWwvftOeQmWk= -cloud.google.com/go v0.78.0/go.mod h1:QjdrLG0uq+YwhjoVOLsS1t7TW8fs36kLs4XO5R5ECHg= -cloud.google.com/go v0.79.0/go.mod h1:3bzgcEeQlzbuEAYu4mrWhKqWjmpprinYgKJLgKHnbb8= -cloud.google.com/go v0.81.0/go.mod h1:mk/AM35KwGk/Nm2YSeZbxXdrNK3KZOYHmLkOqC2V6E0= -cloud.google.com/go v0.83.0/go.mod h1:Z7MJUsANfY0pYPdw0lbnivPx4/vhy/e2FEkSkF7vAVY= -cloud.google.com/go v0.84.0/go.mod h1:RazrYuxIK6Kb7YrzzhPoLmCVzl7Sup4NrbKPg8KHSUM= -cloud.google.com/go v0.87.0/go.mod h1:TpDYlFy7vuLzZMMZ+B6iRiELaY7z/gJPaqbMx6mlWcY= -cloud.google.com/go v0.90.0/go.mod h1:kRX0mNRHe0e2rC6oNakvwQqzyDmg57xJ+SZU1eT2aDQ= -cloud.google.com/go v0.93.3/go.mod h1:8utlLll2EF5XMAV15woO4lSbWQlk8rer9aLOfLh7+YI= -cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= -cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= -cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= -cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= -cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= -cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= -cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= -cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= -cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= -cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= -cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= -cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= -cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= -cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= -cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= -cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= -cloud.google.com/go/storage v1.16.1/go.mod h1:LaNorbty3ehnU3rEjXSNV/NRgQA0O8Y+uh6bPe5UOk4= -dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= -github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= -github.com/Azure/azure-sdk-for-go/sdk/azcore v0.20.0/go.mod h1:ZPW/Z0kLCTdDZaDbYTetxc9Cxl/2lNqxYHYNOF2bti0= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v0.12.0/go.mod h1:GJzjM4SR9T0KyX5gKCVyz1ytD8FeWeUPCwtFCt1AyfE= -github.com/Azure/azure-sdk-for-go/sdk/internal v0.8.1/go.mod h1:KLF4gFr6DcKFZwSuH8w8yEK6DpFl3LP5rhdvAb7Yz5I= -github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v0.2.0/go.mod h1:eHWhQKXc1Gv1DvWH//UzgWjWFEo0Pp4pH2vBzjBw8Fc= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= -github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= -github.com/DATA-DOG/go-sqlmock v1.5.0/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM= -github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= -github.com/HdrHistogram/hdrhistogram-go v1.1.0/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= -github.com/Jeffail/gabs/v2 v2.5.1/go.mod h1:xCn81vdHKxFUuWWAaD5jCTQDNPBMh5pPs9IJ+NcziBI= -github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= -github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= -github.com/KyleBanks/depth v1.2.1/go.mod h1:jzSb9d0L43HxTQfT+oSA1EEp2q+ne2uh6XgeJcm8brE= -github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE= -github.com/ReneKroon/ttlcache/v2 v2.3.0/go.mod h1:zbo6Pv/28e21Z8CzzqgYRArQYGYtjONRxaAKGxzQvG4= -github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= -github.com/VividCortex/ewma v1.1.1/go.mod h1:2Tkkvm3sRDVXaiyucHiACn4cqf7DpdyLvmxzcbUokwA= -github.com/VividCortex/mysqlerr v0.0.0-20200629151747-c28746d985dd/go.mod h1:f3HiCrHjHBdcm6E83vGaXh1KomZMA2P6aeo3hKx/wg0= -github.com/Xeoncross/go-aesctr-with-hmac v0.0.0-20200623134604-12b17a7ff502/go.mod h1:pmnBM9bxWSiHvC/gSWunUIyDvGn33EkP2CUjxFKtTTM= -github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= -github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= -github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= -github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT0m65DJ+Wo= -github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= -github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= -github.com/apache/thrift v0.0.0-20181112125854-24918abba929/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= -github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= -github.com/appleboy/gin-jwt/v2 v2.6.3/go.mod h1:MfPYA4ogzvOcVkRwAxT7quHOtQmVKDpTwxyUrC2DNw0= -github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= -github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= -github.com/aws/aws-sdk-go v1.35.3/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48= -github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= -github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= -github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= -github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= -github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mod h1:VKt7CNAQxpFpSDz3sXyj9hY/GbVsQCr0sB3w59nE7lU= -github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= -github.com/carlmjohnson/flagext v0.21.0/go.mod h1:Eenv0epIUAr4NuedNmkzI8WmBmjIxZC239XcKxYS2ac= -github.com/cenkalti/backoff/v4 v4.0.2/go.mod h1:eEew/i+1Q6OrCDZh3WiXYv3+nJwBASZ8Bog/87DQnVg= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= -github.com/cespare/xxhash/v2 v2.1.0/go.mod h1:dgIUBU3pDso/gPgZ1osOZ0iQf77oPR28Tjxl5dIMyVM= -github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cheggaaa/pb/v3 v3.0.8/go.mod h1:UICbiLec/XO6Hw6k+BHEtHeQFzzBH4i2/qk/ow1EJTA= -github.com/cheynewallace/tabby v1.1.1/go.mod h1:Pba/6cUL8uYqvOc9RkyvFbHGrQ9wShyrn6/S/1OYVys= -github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= -github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= -github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= -github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= -github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= -github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= -github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= -github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= -github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= -github.com/cockroachdb/pebble v0.0.0-20210719141320-8c3bd06debb5/go.mod h1:JXfQr3d+XO4bL1pxGwKKo09xylQSdZ/mpZ9b2wfVcPs= -github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= -github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= -github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= -github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= -github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64/go.mod h1:F86k/6c7aDUdwSUevnLpHS/3Q9hzYCE99jGk2xsHnt0= -github.com/coocood/freecache v1.1.1/go.mod h1:OKrEjkGVoxZhyWAJoeFi5BMLUJm2Tit0kpGkIr7NGYY= -github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= -github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= -github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= -github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= -github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/corona10/goimagehash v1.0.2/go.mod h1:/l9umBhvcHQXVtQO1V6Gp1yD20STawkhRnnX0D1bvVI= -github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= -github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= -github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= -github.com/creack/pty v1.1.11/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= -github.com/cznic/golex v0.0.0-20181122101858-9c343928389c/go.mod h1:+bmmJDNmKlhWNG+gwWCkaBoTy39Fs+bzRxVBzoTQbIc= -github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= -github.com/cznic/parser v0.0.0-20160622100904-31edd927e5b1/go.mod h1:2B43mz36vGZNZEwkWi8ayRSSUXLfjL8OkbzwW4NcPMM= -github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8/go.mod h1:q2w6Bg5jeox1B+QkJ6Wp/+Vn0G/bo3f1uY7Fn3vivIQ= -github.com/cznic/strutil v0.0.0-20171016134553-529a34b1c186/go.mod h1:AHHPPPXTw0h6pVabbcbyGRK1DckRn7r/STdZEeIDzZc= -github.com/cznic/y v0.0.0-20170802143616-045f81c6662a/go.mod h1:1rk5VM7oSnA4vjp+hrLQ3HWHa+Y4yPCa3/CsJrcNnvs= -github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37/go.mod h1:DC3JtzuG7kxMvJ6dZmf2ymjNyoXwgtklr7FN+Um2B0U= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= -github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/denisenkom/go-mssqldb v0.0.0-20191124224453-732737034ffd/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU= -github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= -github.com/dgraph-io/ristretto v0.0.1/go.mod h1:T40EBc7CJke8TkpiYfGGKAeFjSaxuFXhuXRyumBd6RE= -github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= -github.com/dnaeon/go-vcr v1.1.0/go.mod h1:M7tiix8f0r6mKKJ3Yq/kqU1OYf3MnfmBWVbPx/yU9ko= -github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= -github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= -github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po= -github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a2zkGnVExMxdzMo3M0Hi/3sEU+cWnZpSni0O6/Yb/P0= -github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= -github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= -github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= -github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= -github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= -github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= -github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= -github.com/felixge/httpsnoop v1.0.1/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= -github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= -github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= -github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= -github.com/form3tech-oss/jwt-go v3.2.5+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= -github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= -github.com/fsouza/fake-gcs-server v1.19.0/go.mod h1:JtXHY/QzHhtyIxsNfIuQ+XgHtRb5B/w8nqbL5O8zqo0= -github.com/fzipp/gocyclo v0.3.1/go.mod h1:DJHO6AUmbdqj2ET4Z9iArSuwWgYDRryYt2wASxc7x3E= -github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= -github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9/go.mod h1:106OIgooyS7OzLDOpUGgm9fA3bQENb/cFSyyBmMoJDs= -github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/gin-contrib/gzip v0.0.1/go.mod h1:fGBJBCdt6qCZuCAOwWuFhBB4OOq9EFqlo5dEaFhhu5w= -github.com/gin-contrib/sse v0.0.0-20170109093832-22d885f9ecc7/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= -github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= -github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= -github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= -github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= -github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmCsR2Do= -github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= -github.com/go-chi/chi v4.0.2+incompatible/go.mod h1:eB3wogJHnLi3x/kFX2A+IbTBlXxmMeXJVKy9tTv1XzQ= -github.com/go-echarts/go-echarts v1.0.0/go.mod h1:qbmyAb/Rl1f2w7wKba1D4LoNq4U164yO4/wedFbcWyo= -github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= -github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= -github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= -github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= -github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas= -github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= -github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= -github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= -github.com/go-openapi/jsonpointer v0.19.2/go.mod h1:3akKfEdA7DF1sugOqz1dVQHBcuDBPKZGEoHC/NkiQRg= -github.com/go-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= -github.com/go-openapi/jsonreference v0.17.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= -github.com/go-openapi/jsonreference v0.19.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= -github.com/go-openapi/jsonreference v0.19.2/go.mod h1:jMjeRr2HHw6nAVajTXJ4eiUwohSTlpa0o73RUL1owJc= -github.com/go-openapi/jsonreference v0.19.3/go.mod h1:rjx6GuL8TTa9VaixXglHmQmIL98+wF9xc8zWvFonSJ8= -github.com/go-openapi/spec v0.19.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI= -github.com/go-openapi/spec v0.19.4/go.mod h1:FpwSN1ksY1eteniUU7X0N/BgJ7a4WvBFVA8Lj9mJglo= -github.com/go-openapi/swag v0.17.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg= -github.com/go-openapi/swag v0.19.2/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= -github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= -github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= -github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= -github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEKwgtJRd2xk99HeFyHw3yid4rvQIY= -github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= -github.com/go-sql-driver/mysql v1.3.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= -github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= -github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= -github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= -github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= -github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= -github.com/goccy/go-graphviz v0.0.5/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= -github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= -github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= -github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= -github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0= -github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= -github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= -github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= -github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= -github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.1/go.mod h1:DopwsBzvsk0Fs44TXzsVbJyPhcCPeIwnvohx4u74HPM= -github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= -github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= -github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= -github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= -github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= -github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= -github.com/google/martian/v3 v3.2.1/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= -github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= -github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200407044318-7d83b28da2e9/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= -github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= -github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gorilla/handlers v1.4.2/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= -github.com/gorilla/handlers v1.5.1/go.mod h1:t8XrUpc4KVXb7HGyJ4/cEnwQiaxrX/hz1Zv/4g96P1Q= -github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= -github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= -github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= -github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= -github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE= -github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= -github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= -github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= -github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= -github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= -github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= -github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= -github.com/hypnoglow/gormzap v0.3.0/go.mod h1:5Wom8B7Jl2oK0Im9hs6KQ+Kl92w4Y7gKCrj66rhyvw0= -github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334/go.mod h1:SK73tn/9oHe+/Y0h39VT4UCxmurVJkR5NA7kMEAOgSE= -github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= -github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= -github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= -github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= -github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= -github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= -github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= -github.com/jcmturner/gofork v0.0.0-20180107083740-2aebee971930/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= -github.com/jedib0t/go-pretty/v6 v6.2.2/go.mod h1:+nE9fyyHGil+PuISTCrp7avEdo6bqoMwqZnuiK2r2a0= -github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZT/Qs= -github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= -github.com/jinzhu/now v1.0.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jinzhu/now v1.1.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= -github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= -github.com/joho/godotenv v1.3.0/go.mod h1:7hK45KPybAkOC6peb+G5yklZfMxEjkZhHbwpqxOKXbg= -github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df/go.mod h1:mAVCUAYtW9NG31eB30umMSLKcDt6mCUWSjoSn5qBh0k= -github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= -github.com/joomcode/errorx v1.0.1/go.mod h1:kgco15ekB6cs+4Xjzo7SPeXzx38PbJzBwbnu9qfVNHQ= -github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= -github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= -github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= -github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= -github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= -github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= -github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= -github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= -github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= -github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= -github.com/kataras/iris/v12 v12.0.1/go.mod h1:udK4vLQKkdDqMGJJVd/msuMtN6hpYJhg/lSzuxjhO+U= -github.com/kataras/neffos v0.0.10/go.mod h1:ZYmJC07hQPW67eKuzlfY7SO3bC0mw83A3j6im82hfqw= -github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiDuX9AhMbDPkGYSPugBOV6yTZB1l2K9Z0= -github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= -github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= -github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.10.5/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/compress v1.11.7/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= -github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= -github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/pty v1.1.5/go.mod h1:9r2w37qlBe7rQ6e1fg1S/9xpWHSnaqNdHD3WcMdbPDA= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= -github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= -github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= -github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= -github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw= -github.com/lib/pq v1.1.1/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= -github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= -github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= -github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= -github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= -github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= -github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= -github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= -github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= -github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= -github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= -github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= -github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= -github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= -github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= -github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= -github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= -github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= -github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vqg+NOMyg4B2o= -github.com/mattn/go-shellwords v1.0.12/go.mod h1:EZzvwXDESEeg03EKmM+RmDnNOPKG4lLtQsUlTZDWQ8Y= -github.com/mattn/go-sqlite3 v1.14.5/go.mod h1:WVKg1VTActs4Qso6iwGbiFih2UIHo0ENGwNd0Lj+XmI= -github.com/mattn/go-sqlite3 v2.0.1+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= -github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= -github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= -github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= -github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= -github.com/mgechev/revive v1.0.2/go.mod h1:rb0dQy1LVAxW9SWy5R3LPUjevzUbUS316U5MFySA2lo= -github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= -github.com/minio/sio v0.3.0/go.mod h1:8b0yPp2avGThviy/+OCJBI6OMpvxoUuiLvE6F1lebhw= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= -github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/modocache/gover v0.0.0-20171022184752-b58185e213c5/go.mod h1:caMODM3PzxT8aQXRPkAt8xlV/e7d7w8GM5g0fa5F0D8= -github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= -github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= -github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= -github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= -github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= -github.com/ncw/directio v1.0.4/go.mod h1:CKGdcN7StAaqjT7Qack3lAXeX4pjnyc46YeqZH1yWVY= -github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5/go.mod h1:jpp1/29i3P1S/RLdc7JQKbRpFeM1dOBd8T9ki5s+AY8= -github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7/go.mod h1:iWMfgwqYW+e8n5lC/jjNEhwcjbRDpl5NT7n2h+4UNcI= -github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8= -github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q= -github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= -github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= -github.com/oleiade/reflections v1.0.1/go.mod h1:rdFxbxq4QXVZWj0F+e9jqjDkc7dbp97vkRixKo2JR60= -github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= -github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= -github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= -github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= -github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= -github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= -github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= -github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= -github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/pelletier/go-toml v1.3.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= -github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= -github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= -github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= -github.com/pingcap/badger v1.5.1-0.20210831093107-2f6cb8008145/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= -github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= -github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= -github.com/pingcap/errors v0.11.5-0.20201029093017-5a7df2af2ac7/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= -github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= -github.com/pingcap/errors v0.11.5-0.20210425183316-da1aaba5fb63/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= -github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= -github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd/go.mod h1:IVF+ijPSMZVtx2oIqxAg7ur6EyixtTYfOHwpfmlhqI4= -github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= -github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210805052247-76981389e818/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210806074406-317f69fb54b4/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210819164333-bd5706b9d9f2/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211109071446-a8b4d34474bc/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211122024046-03abd340988f/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= -github.com/pingcap/log v0.0.0-20210906054005-afc726e70354 h1:SvWCbCPh1YeHd9yQLksvJYAgft6wLTY1aNG81tpyscQ= -github.com/pingcap/log v0.0.0-20210906054005-afc726e70354/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= -github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= -github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= -github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5/go.mod h1:XsOaV712rUk63aOEKYP9PhXTIE3FMNHmC2r1wX5wElY= -github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d/go.mod h1:7j18ezaWTao2LHOyMlsc2Dg1vW+mDY9dEbPzVyOlaeM= -github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= -github.com/pingcap/tidb-dashboard v0.0.0-20210716172320-2226872e3296/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= -github.com/pingcap/tidb-dashboard v0.0.0-20211008050453-a25c25809529/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= -github.com/pingcap/tidb-dashboard v0.0.0-20211107164327-80363dfbe884/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= -github.com/pingcap/tidb-tools v5.0.3+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20210802080519-94b831c6db55/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= -github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= -github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= -github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= -github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= -github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= -github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g= -github.com/prometheus/client_golang v1.2.1/go.mod h1:XMU6Z2MjaRKVu/dC1qupJI9SiNkDYzz3xecMgSW/F+U= -github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= -github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= -github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= -github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc= -github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= -github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= -github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= -github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= -github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= -github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= -github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= -github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= -github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= -github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= -github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= -github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= -github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= -github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= -github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= -github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= -github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= -github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= -github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= -github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= -github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil v3.21.2+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/gopsutil v3.21.3+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= -github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= -github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= -github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= -github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= -github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= -github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= -github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= -github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= -github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= -github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= -github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= -github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= -github.com/spf13/viper v1.4.0/go.mod h1:PTJ7Z/lr49W6bUbkmS1V3by4uWynFiR9p7+dSq/yZzE= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= -github.com/stretchr/testify v1.6.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/swaggo/files v0.0.0-20190704085106-630677cd5c14/go.mod h1:gxQT6pBGRuIGunNf/+tSOB5OHvguWi8Tbt82WOkf35E= -github.com/swaggo/gin-swagger v1.2.0/go.mod h1:qlH2+W7zXGZkczuL+r2nEBR2JTT+/lX05Nn6vPhc7OI= -github.com/swaggo/http-swagger v0.0.0-20200308142732-58ac5e232fba/go.mod h1:O1lAbCgAAX/KZ80LM/OXwtWFI/5TvZlwxSg8Cq08PV0= -github.com/swaggo/swag v1.5.1/go.mod h1:1Bl9F/ZBpVWh22nY0zmYyASPO1lI/zIwRDrpZU+tv8Y= -github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWAUio= -github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= -github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= -github.com/thoas/go-funk v0.7.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= -github.com/thoas/go-funk v0.8.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= -github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= -github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= -github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210926100628-3cc2459779ca/go.mod h1:KwtZXt0JD+bP9bWW2ka0ir3Wp3oTEfZUTh22bs2sI4o= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211229051614-62d6b4a2e8f7/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= -github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d/go.mod h1:Jw9KG11C/23Rr7DW4XWQ7H5xOgGZo6DFL1OKAF4+Igw= -github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d/go.mod h1:rammPjeZgpvfrQRPkijcx8tlxF1XM5+m6kRXrkDzCAA= -github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= -github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= -github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= -github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1gBkr4QyP8= -github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/twmb/murmur3 v1.1.3/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= -github.com/uber/jaeger-client-go v2.22.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= -github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= -github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= -github.com/ugorji/go v1.1.5-pre/go.mod h1:FwP/aQVg39TXzItUBMwnWp9T9gPQnXw4Poh4/oBQZ/0= -github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= -github.com/ugorji/go/codec v0.0.0-20181022190402-e5e69e061d4f/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= -github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= -github.com/ugorji/go/codec v1.1.5-pre/go.mod h1:tULtS6Gy1AE1yCENaw4Vb//HLH5njI2tfCQDUqRd8fI= -github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= -github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= -github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= -github.com/urfave/cli/v2 v2.1.1/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ= -github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= -github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= -github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= -github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= -github.com/vmihailenco/msgpack/v4 v4.3.11/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+NXzzngzBKDPIqw4= -github.com/vmihailenco/msgpack/v5 v5.0.0-beta.1/go.mod h1:xlngVLeyQ/Qi05oQxhQ+oTuqa03RjMwMfk/7/TCs+QI= -github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= -github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sdOQnirw1PrcnTJYkmW1iOHtUmblMmGdUOHyWYycLI= -github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= -github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= -github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= -github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= -github.com/xitongsys/parquet-go v1.5.1/go.mod h1:xUxwM8ELydxh4edHGegYq1pA8NnMKDx0K/GyB0o2bww= -github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457/go.mod h1:pheqtXeHQFzxJk45lRQ0UIGIivKnLXvialZSFWs81A8= -github.com/xitongsys/parquet-go-source v0.0.0-20190524061010-2b72cbee77d5/go.mod h1:xxCx7Wpym/3QCo6JhujJX51dzSXrwmb0oH6FQb39SEA= -github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0/go.mod h1:HYhIKsdns7xz80OgkbgJYrtQY7FjHWHKH6cvN7+czGE= -github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= -github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= -github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= -github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= -github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= -github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= -github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b/go.mod h1:yVHk9ub3CSBatqGNg7GRmsnfLWtoW60w4eDYfh7vHDg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9/go.mod h1:q+i20RPAmay+xq8LJ3VMOhXCNk4YCk3V7QP91meFavw= -go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= -go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= -go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= -go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= -go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= -go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= -go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= -go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/automaxprocs v1.4.0/go.mod h1:/mTEdr7LvHhs0v7mjdxDreTz1OG5zdZGqgOnhWiR/+Q= -go.uber.org/dig v1.8.0/go.mod h1:X34SnWGr8Fyla9zQNO2GSO2D+TIuqB14OS8JhYocIyw= -go.uber.org/fx v1.10.0/go.mod h1:vLRicqpG/qQEzno4SYU86iCwfT95EZza+Eba0ItuxqY= -go.uber.org/goleak v0.10.0/go.mod h1:VCZuO8V8mFPlL0F5J5GK1rtHV3DrFcQ1R8ryq7FK0aI= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/goleak v1.1.11 h1:wy28qYRKZgnJTxGxvye5/wgWr1EKjmUDGYox5mGlRlI= -go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= -go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= -go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/multierr v1.7.0 h1:zaiO/rmgFjbmCXdSYJWQcdvOCsthmdaHfr3Gm2Kx4Ec= -go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= -go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= -go.uber.org/zap v1.8.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= -go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.19.1 h1:ue41HOKd1vGURxrmeKIgELGb3jPW9DMUDGtsinblHwI= -go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= -golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190325154230-a5d413f7728c/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20191205180655-e7c4368fe9dd/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20181106170214-d68db9428509/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= -golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= -golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= -golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= -golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= -golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20200513190911-00229845015e/go.mod h1:4M0jN8W1tt0AVLNr8HDosyJCDCDuyL9N9+3m7wDWgKw= -golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= -golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= -golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= -golang.org/x/image v0.0.0-20200119044424-58c23975cae1/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= -golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20210508222113-6edffad5e616 h1:VLliZ0d+/avPrXXH+OakdXhpJuEoBZuwh1m2j7U6Iug= -golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= -golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= -golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= -golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= -golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190522155817-f3200d17e092/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= -golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= -golang.org/x/net v0.0.0-20190611141213-3f473d35a33a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20191002035440-2ec189313ef0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20201010224723-4f7140c49acb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= -golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.0.0-20200902213428-5d25da1a8d43/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20201109201403-9fd604954f58/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20201208152858-08078c50e5b5/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210218202405-ba52d332ba99/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210220000619-9bb904979d93/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210313182246-cd4f82c27b84/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210628180205-a41e5a781914/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20210805134026-6f1e6394065a/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180816055513-1c9583448a9c/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181228144115-9a3f9b0469bb/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190610200419-93c9922d18ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200501052902-10377860bb8e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210514084401-e8d321eab015/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210603125802-9665404d3644/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= -golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190524210228-3d17549cdc6b/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190606050223-4d9ae51c2468/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190611222205-d73e1c7e250b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190614205625-5aca471b1d59/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191030062658-86caa796c7ab/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191107010934-f79515f33823/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191114200427-caa0b0f7d508/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= -golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= -golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200527183253-8e7acdbce89d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= -golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= -golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210112230658-8b4aab62c064/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= -golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.5 h1:ouewzE6p+/VEB31YYnTbEJdi8pFqKp4P4n85vwo3DHA= -golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= -golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= -gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= -gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= -gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= -gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= -gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= -google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= -google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= -google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= -google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= -google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= -google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= -google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= -google.golang.org/api v0.35.0/go.mod h1:/XrVsuzM0rZmrsbjJutiuftIzeuTQcEeaYcSk/mQ1dg= -google.golang.org/api v0.36.0/go.mod h1:+z5ficQTmoYpPn8LCUNVpK5I7hwkpjbcgqA7I34qYtE= -google.golang.org/api v0.40.0/go.mod h1:fYKFpnQN0DsDSKRVRcQSDQNtqWPfM9i+zNPxepjRCQ8= -google.golang.org/api v0.41.0/go.mod h1:RkxM5lITDfTzmyKFPt+wGrCJbVfniCr2ool8kTBzRTU= -google.golang.org/api v0.43.0/go.mod h1:nQsDGjRXMo4lvh5hP0TKqF244gqhGcr/YSIykhUk/94= -google.golang.org/api v0.47.0/go.mod h1:Wbvgpq1HddcWVtzsVLyfLp8lDg6AA241LmgIL59tHXo= -google.golang.org/api v0.48.0/go.mod h1:71Pr1vy+TAZRPkPs/xlCf5SsU8WjuAWv1Pfjbtukyy4= -google.golang.org/api v0.50.0/go.mod h1:4bNT5pAuq5ji4SRZm+5QIkjny9JAyVD/3gaSihNefaw= -google.golang.org/api v0.51.0/go.mod h1:t4HdrdoNgyN5cbEfm7Lum0lcLDLiise1F8qDKX00sOU= -google.golang.org/api v0.54.0/go.mod h1:7C4bFFOvVDGXjfDTAsgGwDgAxRDeQ4X8NvUedIt6z3k= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= -google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= -google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= -google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= -google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= -google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= -google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= -google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= -google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201109203340-2640f1f9cdfb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201201144952-b05cb90ed32e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201210142538-e3217bee35cc/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20201214200347-8c77b98c765d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210222152913-aa3ee6e6a81c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210303154014-9728d6b83eeb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210310155132-4ce2db91004e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210319143718-93e7006c17a6/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20210402141018-6c239bbf2bb1/go.mod h1:9lPAdzaEmUacj36I+k7YKbEc5CXzPIeORRgDAUOu28A= -google.golang.org/genproto v0.0.0-20210513213006-bf773b8c8384/go.mod h1:P3QM42oQyzQSnHPnZ/vqoCdDmzH28fzWByN9asMeM8A= -google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= -google.golang.org/genproto v0.0.0-20210604141403-392c879c8b08/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= -google.golang.org/genproto v0.0.0-20210608205507-b6d2f5bf0d7d/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= -google.golang.org/genproto v0.0.0-20210624195500-8bfb893ecb84/go.mod h1:SzzZ/N+nwJDaO1kznhnlzqS8ocJICar6hYhVyhi++24= -google.golang.org/genproto v0.0.0-20210713002101-d411969a0d9a/go.mod h1:AxrInvYm1dci+enl5hChSFPOmmUF1+uAa/UsgNRWd7k= -google.golang.org/genproto v0.0.0-20210716133855-ce7ef5c701ea/go.mod h1:AxrInvYm1dci+enl5hChSFPOmmUF1+uAa/UsgNRWd7k= -google.golang.org/genproto v0.0.0-20210728212813-7823e685a01f/go.mod h1:ob2IJxKrgPT52GcgX759i1sleT07tiKowYBGbczaW48= -google.golang.org/genproto v0.0.0-20210805201207-89edb61ffb67/go.mod h1:ob2IJxKrgPT52GcgX759i1sleT07tiKowYBGbczaW48= -google.golang.org/genproto v0.0.0-20210813162853-db860fec028c/go.mod h1:cFeNkxwySK631ADgubI+/XFU/xp8FD5KIVV4rj8UC5w= -google.golang.org/genproto v0.0.0-20210825212027-de86158e7fda/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= -google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= -google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= -google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= -google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= -google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= -google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA51WJ8= -google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= -google.golang.org/grpc v1.37.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= -google.golang.org/grpc v1.37.1/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= -google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= -google.golang.org/grpc v1.39.0/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnDzfrE= -google.golang.org/grpc v1.39.1/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnDzfrE= -google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= -google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= -google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= -google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= -gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= -gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= -gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= -gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= -gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= -gopkg.in/go-playground/validator.v9 v9.29.1/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ= -gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= -gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q= -gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4= -gopkg.in/jcmturner/gokrb5.v7 v7.3.0/go.mod h1:l8VISx+WGYp+Fp7KRbsiUuXTTOnxIc3Tuvyavf11/WM= -gopkg.in/jcmturner/rpc.v1 v1.1.0/go.mod h1:YIdkC4XfD6GXbzje11McwsDuOlZQSb9W4vfLvuNnlv8= -gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= -gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= -gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= -gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= -gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= -gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= -gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gorm.io/driver/mysql v1.0.6/go.mod h1:KdrTanmfLPPyAOeYGyG+UpDys7/7eeWT1zCq+oekYnU= -gorm.io/driver/sqlite v1.1.4/go.mod h1:mJCeTFr7+crvS+TRnWc5Z3UvwxUN1BGBLMrf5LA9DYw= -gorm.io/gorm v1.20.7/go.mod h1:0HFTzE/SqkGTzK6TlDPPQbAYCluiVvhzoA1+aVyzenw= -gorm.io/gorm v1.21.9/go.mod h1:F+OptMscr0P2F2qU97WT1WimdH9GaQPoDW7AYd5i2Y0= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.2.0/go.mod h1:lPVVZ2BS5TfnjLyizF7o7hv7j9/L+8cZY2hLyjP9cGY= -k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= -modernc.org/fileutil v1.0.0/go.mod h1:JHsWpkrk/CnVV1H/eGlFf85BEpfkrp56ro8nojIq9Q8= -modernc.org/golex v1.0.1/go.mod h1:QCA53QtsT1NdGkaZZkF5ezFwk4IXh4BGNafAARTC254= -modernc.org/lex v1.0.0/go.mod h1:G6rxMTy3cH2iA0iXL/HRRv4Znu8MK4higxph/lE7ypk= -modernc.org/lexer v1.0.0/go.mod h1:F/Dld0YKYdZCLQ7bD0USbWL4YKCyTDRDHiDTOs0q0vk= -modernc.org/mathutil v1.0.0/go.mod h1:wU0vUrJsVWBZ4P6e7xtFJEhFSNsfRLJ8H458uRjg03k= -modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= -modernc.org/mathutil v1.4.1/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= -modernc.org/parser v1.0.0/go.mod h1:H20AntYJ2cHHL6MHthJ8LZzXCdDCHMWt1KZXtIMjejA= -modernc.org/parser v1.0.2/go.mod h1:TXNq3HABP3HMaqLK7brD1fLA/LfN0KS6JxZn71QdDqs= -modernc.org/scanner v1.0.1/go.mod h1:OIzD2ZtjYk6yTuyqZr57FmifbM9fIH74SumloSsajuE= -modernc.org/sortutil v1.0.0/go.mod h1:1QO0q8IlIlmjBIwm6t/7sof874+xCfZouyqZMLIAtxM= -modernc.org/strutil v1.0.0/go.mod h1:lstksw84oURvj9y3tn8lGvRxyRC1S2+g5uuIzNfIOBs= -modernc.org/strutil v1.1.0/go.mod h1:lstksw84oURvj9y3tn8lGvRxyRC1S2+g5uuIzNfIOBs= -modernc.org/y v1.0.1/go.mod h1:Ho86I+LVHEI+LYXoUKlmOMAM1JTXOCfj8qi1T8PsClE= -moul.io/zapgorm2 v1.1.0/go.mod h1:emRfKjNqSzVj5lcgasBdovIXY1jSOwFz2GQZn1Rddks= -rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= -rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= -rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= -rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= -sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= -sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= -sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67/go.mod h1:L5q+DGLGOQFpo1snNEkLOJT2d1YTW66rWNzatr3He1k= From e3cb4bdb279cfb0756e4e4a2652ac5a1b1dac6fb Mon Sep 17 00:00:00 2001 From: likzn <1020193211@qq.com> Date: Wed, 22 Jun 2022 17:42:37 +0800 Subject: [PATCH 06/21] expr: fix `unix_timestamp` range check (#35239) close pingcap/tidb#32197 --- expression/builtin_time.go | 6 +++++- expression/integration_serial_test.go | 6 ++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 0150af13191a9..4a8a5bcca13b4 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -4129,9 +4129,13 @@ func (c *unixTimestampFunctionClass) getFunction(ctx sessionctx.Context, args [] // goTimeToMysqlUnixTimestamp converts go time into MySQL's Unix timestamp. // MySQL's Unix timestamp ranges in int32. Values out of range should be rewritten to 0. +// https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_unix-timestamp func goTimeToMysqlUnixTimestamp(t time.Time, decimal int) (*types.MyDecimal, error) { nanoSeconds := t.UnixNano() - if nanoSeconds < 0 || (nanoSeconds/1e3) >= (math.MaxInt32+1)*1e6 { + // Prior to MySQL 8.0.28, the valid range of argument values is the same as for the TIMESTAMP data type: + // '1970-01-01 00:00:01.000000' UTC to '2038-01-19 03:14:07.999999' UTC. + // This is also the case in MySQL 8.0.28 and later for 32-bit platforms. + if nanoSeconds < 1e9 || (nanoSeconds/1e3) >= (math.MaxInt32+1)*1e6 { return new(types.MyDecimal), nil } dec := new(types.MyDecimal) diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go index 0b7b9a58f9054..fc4ecda462208 100644 --- a/expression/integration_serial_test.go +++ b/expression/integration_serial_test.go @@ -2387,6 +2387,12 @@ func TestTimeBuiltin(t *testing.T) { // for unix_timestamp tk.MustExec("SET time_zone = '+00:00';") + tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 00:00:00.000001');").Check(testkit.Rows("0.000000")) + tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 00:00:00.999999');").Check(testkit.Rows("0.000000")) + tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 00:00:01.000000');").Check(testkit.Rows("1.000000")) + tk.MustQuery("SELECT UNIX_TIMESTAMP('2038-01-19 03:14:07.999999');").Check(testkit.Rows("2147483647.999999")) + tk.MustQuery("SELECT UNIX_TIMESTAMP('2038-01-19 03:14:08.000000');").Check(testkit.Rows("0.000000")) + result = tk.MustQuery("SELECT UNIX_TIMESTAMP(151113);") result.Check(testkit.Rows("1447372800")) result = tk.MustQuery("SELECT UNIX_TIMESTAMP(20151113);") From fc833a30b30e3984871692463ca4952e08048dcc Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 22 Jun 2022 18:34:37 +0800 Subject: [PATCH 07/21] planner, executor: add some comments about how we decide sample rate (#34140) ref pingcap/tidb#24182 --- executor/builder.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/executor/builder.go b/executor/builder.go index 736a8204e5a96..630ef3137db58 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2435,6 +2435,15 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeC } // getAdjustedSampleRate calculate the sample rate by the table size. If we cannot get the table size. We use the 0.001 as the default sample rate. +// From the paper "Random sampling for histogram construction: how much is enough?"'s Corollary 1 to Theorem 5, +// for a table size n, histogram size k, maximum relative error in bin size f, and error probability gamma, +// the minimum random sample size is +// r = 4 * k * ln(2*n/gamma) / f^2 +// If we take f = 0.5, gamma = 0.01, n =1e6, we would got r = 305.82* k. +// Since the there's log function over the table size n, the r grows slowly when the n increases. +// If we take n = 1e12, a 300*k sample still gives <= 0.66 bin size error with probability 0.99. +// So if we don't consider the top-n values, we can keep the sample size at 300*256. +// But we may take some top-n before building the histogram, so we increase the sample a little. func (b *executorBuilder) getAdjustedSampleRate(sctx sessionctx.Context, task plannercore.AnalyzeColumnsTask) float64 { statsHandle := domain.GetDomain(sctx).StatsHandle() defaultRate := 0.001 From 95e13afa13958767d107f731274b5afe1679cf49 Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 22 Jun 2022 19:10:38 +0800 Subject: [PATCH 08/21] *: track the memory usage in Insert/Update/Delete executors (#34097) close pingcap/tidb#34096 --- executor/delete.go | 15 ++++-- executor/update.go | 29 +++++++---- kv/key.go | 93 ++++++++++++++++++++++++++++++++-- kv/key_test.go | 81 +++++++++++++++++++++++++++++ statistics/row_sampler.go | 2 +- util/hack/hack.go | 5 ++ util/set/mem_aware_map.go | 72 ++++++++++++++++++++++++++ util/set/mem_aware_map_test.go | 80 +++++++++++++++++++++++++++++ 8 files changed, 357 insertions(+), 20 deletions(-) create mode 100644 util/set/mem_aware_map.go create mode 100644 util/set/mem_aware_map_test.go diff --git a/executor/delete.go b/executor/delete.go index d72eae827d9e2..5e759e12db25c 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -158,20 +158,26 @@ func (e *DeleteExec) doBatchDelete(ctx context.Context) error { } func (e *DeleteExec) composeTblRowMap(tblRowMap tableRowMapType, colPosInfos []plannercore.TblColPosInfo, joinedRow []types.Datum) error { - // iterate all the joined tables, and got the copresonding rows in joinedRow. + // iterate all the joined tables, and got the corresponding rows in joinedRow. for _, info := range colPosInfos { if unmatchedOuterRow(info, joinedRow) { continue } if tblRowMap[info.TblID] == nil { - tblRowMap[info.TblID] = kv.NewHandleMap() + tblRowMap[info.TblID] = kv.NewMemAwareHandleMap[[]types.Datum]() } handle, err := info.HandleCols.BuildHandleByDatums(joinedRow) if err != nil { return err } // tblRowMap[info.TblID][handle] hold the row datas binding to this table and this handle. - tblRowMap[info.TblID].Set(handle, joinedRow[info.Start:info.End]) + _, exist := tblRowMap[info.TblID].Get(handle) + memDelta := tblRowMap[info.TblID].Set(handle, joinedRow[info.Start:info.End]) + if !exist { + memDelta += types.EstimatedMemUsage(joinedRow, 1) + memDelta += int64(handle.ExtraMemSize()) + } + e.memTracker.Consume(memDelta) } return nil } @@ -240,6 +246,7 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h kv.Handl // Close implements the Executor Close interface. func (e *DeleteExec) Close() error { + defer e.memTracker.ReplaceBytesUsed(0) return e.children[0].Close() } @@ -254,4 +261,4 @@ func (e *DeleteExec) Open(ctx context.Context) error { // tableRowMapType is a map for unique (Table, Row) pair. key is the tableID. // the key in map[int64]Row is the joined table handle, which represent a unique reference row. // the value in map[int64]Row is the deleting row. -type tableRowMapType map[int64]*kv.HandleMap +type tableRowMapType map[int64]*kv.MemAwareHandleMap[[]types.Datum] diff --git a/executor/update.go b/executor/update.go index faf5b1d15e1bf..196f737aa057f 100644 --- a/executor/update.go +++ b/executor/update.go @@ -42,11 +42,11 @@ type UpdateExec struct { // updatedRowKeys is a map for unique (TableAlias, handle) pair. // The value is true if the row is changed, or false otherwise - updatedRowKeys map[int]*kv.HandleMap + updatedRowKeys map[int]*kv.MemAwareHandleMap[bool] tblID2table map[int64]table.Table // mergedRowData is a map for unique (Table, handle) pair. // The value is cached table row - mergedRowData map[int64]*kv.HandleMap + mergedRowData map[int64]*kv.MemAwareHandleMap[[]types.Datum] multiUpdateOnSameTable map[int64]bool matched uint64 // a counter of matched rows during update @@ -71,7 +71,7 @@ type UpdateExec struct { // prepare `handles`, `tableUpdatable`, `changed` to avoid re-computations. func (e *UpdateExec) prepare(row []types.Datum) (err error) { if e.updatedRowKeys == nil { - e.updatedRowKeys = make(map[int]*kv.HandleMap) + e.updatedRowKeys = make(map[int]*kv.MemAwareHandleMap[bool]) } e.handles = e.handles[:0] e.tableUpdatable = e.tableUpdatable[:0] @@ -79,7 +79,7 @@ func (e *UpdateExec) prepare(row []types.Datum) (err error) { e.matches = e.matches[:0] for _, content := range e.tblColPosInfos { if e.updatedRowKeys[content.Start] == nil { - e.updatedRowKeys[content.Start] = kv.NewHandleMap() + e.updatedRowKeys[content.Start] = kv.NewMemAwareHandleMap[bool]() } handle, err := content.HandleCols.BuildHandleByDatums(row) if err != nil { @@ -102,7 +102,7 @@ func (e *UpdateExec) prepare(row []types.Datum) (err error) { changed, ok := e.updatedRowKeys[content.Start].Get(handle) if ok { - e.changed = append(e.changed, changed.(bool)) + e.changed = append(e.changed, changed) e.matches = append(e.matches, false) } else { e.changed = append(e.changed, false) @@ -114,7 +114,7 @@ func (e *UpdateExec) prepare(row []types.Datum) (err error) { func (e *UpdateExec) merge(row, newData []types.Datum, mergeGenerated bool) error { if e.mergedRowData == nil { - e.mergedRowData = make(map[int64]*kv.HandleMap) + e.mergedRowData = make(map[int64]*kv.MemAwareHandleMap[[]types.Datum]) } var mergedData []types.Datum // merge updates from and into mergedRowData @@ -135,13 +135,13 @@ func (e *UpdateExec) merge(row, newData []types.Datum, mergeGenerated bool) erro flags := e.assignFlag[content.Start:content.End] if e.mergedRowData[content.TblID] == nil { - e.mergedRowData[content.TblID] = kv.NewHandleMap() + e.mergedRowData[content.TblID] = kv.NewMemAwareHandleMap[[]types.Datum]() } tbl := e.tblID2table[content.TblID] oldData := row[content.Start:content.End] newTableData := newData[content.Start:content.End] if v, ok := e.mergedRowData[content.TblID].Get(handle); ok { - mergedData = v.([]types.Datum) + mergedData = v for i, flag := range flags { if tbl.WritableCols()[i].IsGenerated() != mergeGenerated { continue @@ -156,7 +156,10 @@ func (e *UpdateExec) merge(row, newData []types.Datum, mergeGenerated bool) erro } else { mergedData = append([]types.Datum{}, newTableData...) } - e.mergedRowData[content.TblID].Set(handle, mergedData) + + memDelta := e.mergedRowData[content.TblID].Set(handle, mergedData) + memDelta += types.EstimatedMemUsage(mergedData, 1) + int64(handle.ExtraMemSize()) + e.memTracker.Consume(memDelta) } return nil } @@ -190,7 +193,12 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema, row, n // Update row changed, err1 := updateRecord(ctx, e.ctx, handle, oldData, newTableData, flags, tbl, false, e.memTracker) if err1 == nil { - e.updatedRowKeys[content.Start].Set(handle, changed) + _, exist := e.updatedRowKeys[content.Start].Get(handle) + memDelta := e.updatedRowKeys[content.Start].Set(handle, changed) + if !exist { + memDelta += int64(handle.ExtraMemSize()) + } + e.memTracker.Consume(memDelta) continue } @@ -426,6 +434,7 @@ func (e *UpdateExec) Close() error { txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, nil) } } + defer e.memTracker.ReplaceBytesUsed(0) return e.children[0].Close() } diff --git a/kv/key.go b/kv/key.go index 3e68b5fc80dd7..561cc2a03fd78 100644 --- a/kv/key.go +++ b/kv/key.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/set" ) // Key represents high-level Key type. @@ -158,9 +159,15 @@ type Handle interface { // String implements the fmt.Stringer interface. String() string // MemUsage returns the memory usage of a handle. - MemUsage() int64 + MemUsage() uint64 + // ExtraMemSize returns the memory usage of objects that are pointed to by the Handle. + ExtraMemSize() uint64 } +var _ Handle = IntHandle(0) +var _ Handle = &CommonHandle{} +var _ Handle = PartitionHandle{} + // IntHandle implement the Handle interface for int64 type handle. type IntHandle int64 @@ -231,10 +238,15 @@ func (ih IntHandle) String() string { } // MemUsage implements the Handle interface. -func (ih IntHandle) MemUsage() int64 { +func (ih IntHandle) MemUsage() uint64 { return 8 } +// ExtraMemSize implements the Handle interface. +func (ih IntHandle) ExtraMemSize() uint64 { + return 0 +} + // CommonHandle implements the Handle interface for non-int64 type handle. type CommonHandle struct { encoded []byte @@ -355,8 +367,15 @@ func (ch *CommonHandle) String() string { } // MemUsage implements the Handle interface. -func (ch *CommonHandle) MemUsage() int64 { - return int64(cap(ch.encoded)) + int64(cap(ch.colEndOffsets))*2 +func (ch *CommonHandle) MemUsage() uint64 { + // 48 is used by the 2 slice fields. + return 48 + ch.ExtraMemSize() +} + +// ExtraMemSize implements the Handle interface. +func (ch *CommonHandle) ExtraMemSize() uint64 { + // colEndOffsets is a slice of uint16. + return uint64(cap(ch.encoded) + cap(ch.colEndOffsets)*2) } // HandleMap is the map for Handle. @@ -431,6 +450,65 @@ func (m *HandleMap) Range(fn func(h Handle, val interface{}) bool) { } } +// MemAwareHandleMap is similar to HandleMap, but it's aware of its memory usage and doesn't support delete. +// It only tracks the actual sizes. Objects that are pointed to by the key or value are not tracked. +// Those should be tracked by the caller. +type MemAwareHandleMap[V any] struct { + ints set.MemAwareMap[int64, V] + strs set.MemAwareMap[string, strHandleValue[V]] +} + +type strHandleValue[V any] struct { + h Handle + val V +} + +// NewMemAwareHandleMap creates a new map for handle. +func NewMemAwareHandleMap[V any]() *MemAwareHandleMap[V] { + // Initialize the two maps to avoid checking nil. + return &MemAwareHandleMap[V]{ + ints: set.NewMemAwareMap[int64, V](), + strs: set.NewMemAwareMap[string, strHandleValue[V]](), + } +} + +// Get gets a value by a Handle. +func (m *MemAwareHandleMap[V]) Get(h Handle) (v V, ok bool) { + if h.IsInt() { + v, ok = m.ints.Get(h.IntValue()) + } else { + var strVal strHandleValue[V] + strVal, ok = m.strs.Get(string(h.Encoded())) + v = strVal.val + } + return +} + +// Set sets a value with a Handle. +func (m *MemAwareHandleMap[V]) Set(h Handle, val V) int64 { + if h.IsInt() { + return m.ints.Set(h.IntValue(), val) + } + return m.strs.Set(string(h.Encoded()), strHandleValue[V]{ + h: h, + val: val, + }) +} + +// Range iterates the MemAwareHandleMap with fn, the fn returns true to continue, returns false to stop. +func (m *MemAwareHandleMap[V]) Range(fn func(h Handle, val interface{}) bool) { + for h, val := range m.ints.M { + if !fn(IntHandle(h), val) { + return + } + } + for _, strVal := range m.strs.M { + if !fn(strVal.h, strVal.val) { + return + } + } +} + // PartitionHandle combines a handle and a PartitionID, used to location a row in partitioned table. // Now only used in global index. // TODO: support PartitionHandle in HandleMap. @@ -470,6 +548,11 @@ func (ph PartitionHandle) Compare(h Handle) int { } // MemUsage implements the Handle interface. -func (ph PartitionHandle) MemUsage() int64 { +func (ph PartitionHandle) MemUsage() uint64 { return ph.Handle.MemUsage() + 8 } + +// ExtraMemSize implements the Handle interface. +func (ph PartitionHandle) ExtraMemSize() uint64 { + return ph.Handle.ExtraMemSize() +} diff --git a/kv/key_test.go b/kv/key_test.go index 3d3ee3ce5fb1a..af45999d5e5bc 100644 --- a/kv/key_test.go +++ b/kv/key_test.go @@ -17,6 +17,7 @@ package kv_test import ( "bytes" "errors" + "strconv" "testing" "time" @@ -221,3 +222,83 @@ func BenchmarkIsPoint(b *testing.B) { kr.IsPoint() } } + +var result int + +var inputs = []struct { + input int +}{ + {input: 1}, + {input: 100}, + {input: 10000}, + {input: 1000000}, +} + +func memAwareIntMap(size int, handles []Handle) int { + var x int + m := NewMemAwareHandleMap[int]() + for j := 0; j < size; j++ { + m.Set(handles[j], j) + } + for j := 0; j < size; j++ { + x, _ = m.Get(handles[j]) + } + return x +} + +func nativeIntMap(size int, handles []Handle) int { + var x int + m := make(map[Handle]int) + for j := 0; j < size; j++ { + m[handles[j]] = j + } + + for j := 0; j < size; j++ { + x = m[handles[j]] + } + return x +} + +func BenchmarkMemAwareHandleMap(b *testing.B) { + var sc stmtctx.StatementContext + for _, s := range inputs { + handles := make([]Handle, s.input) + for i := 0; i < s.input; i++ { + if i%2 == 0 { + handles[i] = IntHandle(i) + } else { + handleBytes, _ := codec.EncodeKey(&sc, nil, types.NewIntDatum(int64(i))) + handles[i], _ = NewCommonHandle(handleBytes) + } + } + b.Run("MemAwareIntMap_"+strconv.Itoa(s.input), func(b *testing.B) { + var x int + for i := 0; i < b.N; i++ { + x = memAwareIntMap(s.input, handles) + } + result = x + }) + } +} + +func BenchmarkNativeHandleMap(b *testing.B) { + var sc stmtctx.StatementContext + for _, s := range inputs { + handles := make([]Handle, s.input) + for i := 0; i < s.input; i++ { + if i%2 == 0 { + handles[i] = IntHandle(i) + } else { + handleBytes, _ := codec.EncodeKey(&sc, nil, types.NewIntDatum(int64(i))) + handles[i], _ = NewCommonHandle(handleBytes) + } + } + b.Run("NativeIntMap_"+strconv.Itoa(s.input), func(b *testing.B) { + var x int + for i := 0; i < b.N; i++ { + x = nativeIntMap(s.input, handles) + } + result = x + }) + } +} diff --git a/statistics/row_sampler.go b/statistics/row_sampler.go index d091f2be818fb..c80af0b980c79 100644 --- a/statistics/row_sampler.go +++ b/statistics/row_sampler.go @@ -78,7 +78,7 @@ func (i ReservoirRowSampleItem) MemUsage() (sum int64) { sum += col.MemUsage() } if i.Handle != nil { - sum += i.Handle.MemUsage() + sum += int64(i.Handle.MemUsage()) } return sum } diff --git a/util/hack/hack.go b/util/hack/hack.go index f4a2f79f894ab..8e586485ee9bd 100644 --- a/util/hack/hack.go +++ b/util/hack/hack.go @@ -76,3 +76,8 @@ const ( // DefBucketMemoryUsageForSetInt64 = bucketSize*(1+unsafe.Sizeof(int64) + unsafe.Sizeof(struct{}))+2*ptrSize DefBucketMemoryUsageForSetInt64 = (8*(1+8+0) + 16) / 2 * 3 ) + +// EstimateBucketMemoryUsage returns the estimated memory usage of a bucket in a map. +func EstimateBucketMemoryUsage[K comparable, V any]() uint64 { + return (8*(1+uint64(unsafe.Sizeof(*new(K))+unsafe.Sizeof(*new(V)))) + 16) / 2 * 3 +} diff --git a/util/set/mem_aware_map.go b/util/set/mem_aware_map.go new file mode 100644 index 0000000000000..da1cb227af306 --- /dev/null +++ b/util/set/mem_aware_map.go @@ -0,0 +1,72 @@ +// Copyright 2022 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, +// 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 set + +import ( + "math" + + "github.com/pingcap/tidb/util/hack" +) + +// MemAwareMap is a map which is aware of its memory usage. It's adapted from SetWithMemoryUsage. +// It doesn't support delete. +// The estimate usage of memory is usually smaller than the real usage. +// According to experiments with SetWithMemoryUsage, 2/3 * estimated usage <= real usage <= estimated usage. +type MemAwareMap[K comparable, V any] struct { + M map[K]V // it's public, when callers want to directly access it, e.g. use in a for-range-loop + bInMap int64 + bucketMemoryUsage uint64 +} + +// EstimateMapSize returns the estimated size of the map. It doesn't include the dynamic part, e.g. objects pointed to by pointers in the map. +// len(map) <= load_factor * 2^bInMap. bInMap = ceil(log2(len(map)/load_factor)). +// memory = bucketSize * 2^bInMap +func EstimateMapSize(length int, bucketSize uint64) uint64 { + if length == 0 { + return 0 + } + bInMap := uint64(math.Ceil(math.Log2(float64(length) * hack.LoadFactorDen / hack.LoadFactorNum))) + return bucketSize * uint64(1< (1< Date: Wed, 22 Jun 2022 19:36:38 +0800 Subject: [PATCH 09/21] DM: update contact information of DM (#34249) ref pingcap/tidb#34452 --- dumpling/README.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dumpling/README.md b/dumpling/README.md index 00715370731c6..d90afa808cbf9 100644 --- a/dumpling/README.md +++ b/dumpling/README.md @@ -6,7 +6,6 @@ [![API Docs](https://img.shields.io/badge/go.dev-reference-007d9c?logo=go&logoColor=white)](https://pkg.go.dev/github.com/pingcap/dumpling) [![Go Report Card](https://goreportcard.com/badge/github.com/pingcap/dumpling)](https://goreportcard.com/report/github.com/pingcap/dumpling) [![FOSSA Status](https://app.fossa.com/api/projects/git%2Bgithub.com%2Fpingcap%2Fdumpling.svg?type=shield)](https://app.fossa.com/projects/git%2Bgithub.com%2Fpingcap%2Fdumpling?ref=badge_shield) -[![Discuss in Slack](https://img.shields.io/badge/slack-sig--migrate-4A154B?logo=slack)](https://slack.tidb.io/invite?team=tidb-community&channel=sig-migrate&ref=github_sig) **Dumpling** is a tool and a Go library for creating SQL dump from a MySQL-compatible database. It is intended to replace `mysqldump` and `mydumper` when targeting TiDB. @@ -24,7 +23,7 @@ Features - [ ] Write to cloud storage (S3, GCS) natively - [x] Advanced table filtering -Any questions? Let's discuss in [#sig-migrate in Slack](https://slack.tidb.io/invite?team=tidb-community&channel=sig-migrate&ref=github_sig)! +Any questions? Let's discuss on [TiDB Internals forum](https://internals.tidb.io/)! Building -------- @@ -51,4 +50,4 @@ License Dumpling is under the Apache 2.0 license. See the [LICENSE](./LICENSE) file for details. -[![FOSSA Status](https://app.fossa.com/api/projects/git%2Bgithub.com%2Fpingcap%2Fdumpling.svg?type=large)](https://app.fossa.com/projects/git%2Bgithub.com%2Fpingcap%2Fdumpling?ref=badge_large) \ No newline at end of file +[![FOSSA Status](https://app.fossa.com/api/projects/git%2Bgithub.com%2Fpingcap%2Fdumpling.svg?type=large)](https://app.fossa.com/projects/git%2Bgithub.com%2Fpingcap%2Fdumpling?ref=badge_large) From 56d004e2ea6f4ba1b7d35a90750bfc5ce961ddab Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 22 Jun 2022 19:58:37 +0800 Subject: [PATCH 10/21] planner: introduce new cost formula for MPPJoins (#35643) ref pingcap/tidb#35240 --- planner/core/plan_cost.go | 28 +++++++++++++++++++--------- planner/core/task.go | 8 ++++---- planner/implementation/join.go | 4 ++-- 3 files changed, 25 insertions(+), 15 deletions(-) diff --git a/planner/core/plan_cost.go b/planner/core/plan_cost.go index ee0af71c51149..c0598e74f2801 100644 --- a/planner/core/plan_cost.go +++ b/planner/core/plan_cost.go @@ -735,7 +735,7 @@ func (p *PhysicalApply) GetPlanCost(taskType property.TaskType, costFlag uint64) } // GetCost computes cost of merge join operator itself. -func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64) float64 { +func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64, costFlag uint64) float64 { outerCnt := lCnt innerCnt := rCnt innerKeys := p.RightJoinKeys @@ -766,6 +766,9 @@ func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64) float64 { numPairs = 0 } } + if hasCostFlag(costFlag, CostFlagUseTrueCardinality) { + numPairs = getOperatorActRows(p) + } sessVars := p.ctx.GetSessionVars() probeCost := numPairs * sessVars.GetCPUFactor() // Cost of evaluating outer filters. @@ -795,13 +798,13 @@ func (p *PhysicalMergeJoin) GetPlanCost(taskType property.TaskType, costFlag uin } p.planCost += childCost } - p.planCost += p.GetCost(getCardinality(p.children[0], costFlag), getCardinality(p.children[1], costFlag)) + p.planCost += p.GetCost(getCardinality(p.children[0], costFlag), getCardinality(p.children[1], costFlag), costFlag) p.planCostInit = true return p.planCost, nil } // GetCost computes cost of hash join operator itself. -func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64) float64 { +func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint64) float64 { buildCnt, probeCnt := lCnt, rCnt build := p.children[0] // Taking the right as the inner for right join or using the outer to build a hash table. @@ -815,7 +818,11 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64) float64 { rowSize := getAvgRowSize(build.statsInfo(), build.Schema()) spill := oomUseTmpStorage && memQuota > 0 && rowSize*buildCnt > float64(memQuota) && p.storeTp != kv.TiFlash // Cost of building hash table. - cpuCost := buildCnt * sessVars.GetCPUFactor() + cpuFactor := sessVars.GetCPUFactor() + if isMPP && p.ctx.GetSessionVars().CostModelVersion == modelVer2 { + cpuFactor = sessVars.GetTiFlashCPUFactor() // use the dedicated TiFlash CPU Factor on modelVer2 + } + cpuCost := buildCnt * cpuFactor memoryCost := buildCnt * sessVars.GetMemoryFactor() diskCost := buildCnt * sessVars.GetDiskFactor() * rowSize // Number of matched row pairs regarding the equal join conditions. @@ -845,16 +852,19 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64) float64 { numPairs = 0 } } + if hasCostFlag(costFlag, CostFlagUseTrueCardinality) { + numPairs = getOperatorActRows(p) + } // Cost of querying hash table is cheap actually, so we just compute the cost of // evaluating `OtherConditions` and joining row pairs. - probeCost := numPairs * sessVars.GetCPUFactor() + probeCost := numPairs * cpuFactor probeDiskCost := numPairs * sessVars.GetDiskFactor() * rowSize // Cost of evaluating outer filter. if len(p.LeftConditions)+len(p.RightConditions) > 0 { // Input outer count for the above compution should be adjusted by SelectionFactor. probeCost *= SelectionFactor probeDiskCost *= SelectionFactor - probeCost += probeCnt * sessVars.GetCPUFactor() + probeCost += probeCnt * cpuFactor } diskCost += probeDiskCost probeCost /= float64(p.Concurrency) @@ -864,9 +874,9 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64) float64 { if p.UseOuterToBuild { if spill { // It runs in sequence when build data is on disk. See handleUnmatchedRowsFromHashTableInDisk - cpuCost += buildCnt * sessVars.GetCPUFactor() + cpuCost += buildCnt * cpuFactor } else { - cpuCost += buildCnt * sessVars.GetCPUFactor() / float64(p.Concurrency) + cpuCost += buildCnt * cpuFactor / float64(p.Concurrency) } diskCost += buildCnt * sessVars.GetDiskFactor() * rowSize } @@ -892,7 +902,7 @@ func (p *PhysicalHashJoin) GetPlanCost(taskType property.TaskType, costFlag uint } p.planCost += childCost } - p.planCost += p.GetCost(getCardinality(p.children[0], costFlag), getCardinality(p.children[1], costFlag)) + p.planCost += p.GetCost(getCardinality(p.children[0], costFlag), getCardinality(p.children[1], costFlag), taskType == property.MppTaskType, costFlag) p.planCostInit = true return p.planCost, nil } diff --git a/planner/core/task.go b/planner/core/task.go index fd6cac675f4c6..4ecc67fc67a34 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -326,7 +326,7 @@ func (p *PhysicalHashJoin) attach2Task(tasks ...task) task { p.SetChildren(lTask.plan(), rTask.plan()) task := &rootTask{ p: p, - cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count()), + cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count(), false, 0), } p.cost = task.cost() return task @@ -547,7 +547,7 @@ func (p *PhysicalHashJoin) attach2TaskForMpp(tasks ...task) task { outerTask = rTask } task := &mppTask{ - cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count()), + cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count(), false, 0), p: p, partTp: outerTask.partTp, hashCols: outerTask.hashCols, @@ -578,7 +578,7 @@ func (p *PhysicalHashJoin) attach2TaskForTiFlash(tasks ...task) task { tblColHists: rTask.tblColHists, indexPlanFinished: true, tablePlan: p, - cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count()), + cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count(), false, 0), } p.cost = task.cst return task @@ -590,7 +590,7 @@ func (p *PhysicalMergeJoin) attach2Task(tasks ...task) task { p.SetChildren(lTask.plan(), rTask.plan()) t := &rootTask{ p: p, - cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count()), + cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count(), 0), } p.cost = t.cost() return t diff --git a/planner/implementation/join.go b/planner/implementation/join.go index f24791e9a987e..4b247353c58bf 100644 --- a/planner/implementation/join.go +++ b/planner/implementation/join.go @@ -29,7 +29,7 @@ func (impl *HashJoinImpl) CalcCost(outCount float64, children ...memo.Implementa hashJoin := impl.plan.(*plannercore.PhysicalHashJoin) // The children here are only used to calculate the cost. hashJoin.SetChildren(children[0].GetPlan(), children[1].GetPlan()) - selfCost := hashJoin.GetCost(children[0].GetPlan().StatsCount(), children[1].GetPlan().StatsCount()) + selfCost := hashJoin.GetCost(children[0].GetPlan().StatsCount(), children[1].GetPlan().StatsCount(), false, 0) impl.cost = selfCost + children[0].GetCost() + children[1].GetCost() return impl.cost } @@ -56,7 +56,7 @@ func (impl *MergeJoinImpl) CalcCost(outCount float64, children ...memo.Implement mergeJoin := impl.plan.(*plannercore.PhysicalMergeJoin) // The children here are only used to calculate the cost. mergeJoin.SetChildren(children[0].GetPlan(), children[1].GetPlan()) - selfCost := mergeJoin.GetCost(children[0].GetPlan().StatsCount(), children[1].GetPlan().StatsCount()) + selfCost := mergeJoin.GetCost(children[0].GetPlan().StatsCount(), children[1].GetPlan().StatsCount(), 0) impl.cost = selfCost + children[0].GetCost() + children[1].GetCost() return impl.cost } From 3de73d1be90b5f7522179527d5c129081a7fdc38 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Wed, 22 Jun 2022 06:32:39 -0600 Subject: [PATCH 11/21] session, session_test: Improve read-only for autocommit statements (#35487) close pingcap/tidb#32895 --- session/session.go | 13 ++++++++ session/session_test/session_test.go | 45 ++++++++++++++++++++++++++++ 2 files changed, 58 insertions(+) diff --git a/session/session.go b/session/session.go index 3f163146e8796..8d03ba0870774 100644 --- a/session/session.go +++ b/session/session.go @@ -534,9 +534,22 @@ func (s *session) doCommit(ctx context.Context) error { s.sessionVars.SetInTxn(false) s.ClearDiskFullOpt() }() + // check if the transaction is read-only if s.txn.IsReadOnly() { return nil } + // check if the cluster is read-only + if !s.sessionVars.InRestrictedSQL && variable.RestrictedReadOnly.Load() || variable.VarTiDBSuperReadOnly.Load() { + // It is not internal SQL, and the cluster has one of RestrictedReadOnly or SuperReadOnly + // We need to privilege check again: a privilege check occurred during planning, but we need + // to prevent the case that a long running auto-commit statement is now trying to commit. + pm := privilege.GetPrivilegeManager(s) + roles := s.sessionVars.ActiveRoles + if pm != nil && !pm.HasExplicitlyGrantedDynamicPrivilege(roles, "RESTRICTED_REPLICA_WRITER_ADMIN", false) { + s.RollbackTxn(ctx) + return plannercore.ErrSQLInReadOnlyMode + } + } err := s.checkPlacementPolicyBeforeCommit() if err != nil { return err diff --git a/session/session_test/session_test.go b/session/session_test/session_test.go index 83f4472ded39a..2a0f95fcbc473 100644 --- a/session/session_test/session_test.go +++ b/session/session_test/session_test.go @@ -17,6 +17,7 @@ package session_test import ( "context" "fmt" + "sync" "sync/atomic" "testing" "time" @@ -25,7 +26,9 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/terror" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" @@ -302,6 +305,48 @@ func TestDisableTxnAutoRetry(t *testing.T) { tk2.MustQuery("select * from no_retry").Check(testkit.Rows("13")) } +// The Read-only flags are checked in the planning stage of queries, +// but this test checks we check them again at commit time. +// The main use case for this is a long-running auto-commit statement. +func TestAutoCommitRespectsReadOnly(t *testing.T) { + store, clean := createMockStoreForSchemaTest(t) + defer clean() + var wg sync.WaitGroup + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + require.True(t, tk2.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + tk1.MustExec("create table test.auto_commit_test (a int)") + wg.Add(1) + go func() { + err := tk1.ExecToErr("INSERT INTO test.auto_commit_test VALUES (SLEEP(1))") + require.True(t, terror.ErrorEqual(err, plannercore.ErrSQLInReadOnlyMode), fmt.Sprintf("err %v", err)) + wg.Done() + }() + tk2.MustExec("SET GLOBAL tidb_restricted_read_only = 1") + err := tk2.ExecToErr("INSERT INTO test.auto_commit_test VALUES (0)") // should also be an error + require.True(t, terror.ErrorEqual(err, plannercore.ErrSQLInReadOnlyMode), fmt.Sprintf("err %v", err)) + // Reset and check with the privilege to ignore the readonly flag and continue to insert. + wg.Wait() + tk1.MustExec("SET GLOBAL tidb_restricted_read_only = 0") + tk1.MustExec("SET GLOBAL tidb_super_read_only = 0") + tk1.MustExec("GRANT RESTRICTED_REPLICA_WRITER_ADMIN on *.* to 'root'") + + wg.Add(1) + go func() { + tk1.MustExec("INSERT INTO test.auto_commit_test VALUES (SLEEP(1))") + wg.Done() + }() + tk2.MustExec("SET GLOBAL tidb_restricted_read_only = 1") + tk2.MustExec("INSERT INTO test.auto_commit_test VALUES (0)") + + // wait for go routines + wg.Wait() + tk1.MustExec("SET GLOBAL tidb_restricted_read_only = 0") + tk1.MustExec("SET GLOBAL tidb_super_read_only = 0") +} + func TestLoadSchemaFailed(t *testing.T) { originalRetryTime := domain.SchemaOutOfDateRetryTimes.Load() originalRetryInterval := domain.SchemaOutOfDateRetryInterval.Load() From a2fe74fc92ed6e731e6d513a2d5592a066efc942 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 22 Jun 2022 20:56:36 +0800 Subject: [PATCH 12/21] dumpling: unregister metrics if meet error in NewDumper (#35650) close pingcap/tidb#35649 --- dumpling/export/dump.go | 9 ++++++++- dumpling/export/dump_test.go | 16 ++++++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/dumpling/export/dump.go b/dumpling/export/dump.go index 7f27759211a44..43d801430e6e8 100755 --- a/dumpling/export/dump.go +++ b/dumpling/export/dump.go @@ -81,10 +81,17 @@ func NewDumper(ctx context.Context, conf *Config) (*Dumper, error) { selectTiDBTableRegionFunc: selectTiDBTableRegion, } + var err error + d.metrics = newMetrics(conf.PromFactory, conf.Labels) d.metrics.registerTo(conf.PromRegistry) + defer func() { + if err != nil { + d.metrics.unregisterFrom(conf.PromRegistry) + } + }() - err := adjustConfig(conf, + err = adjustConfig(conf, registerTLSConfig, validateSpecifiedSQL, adjustFileFormat) diff --git a/dumpling/export/dump_test.go b/dumpling/export/dump_test.go index 204826f8e7716..b059a1ae28ac2 100644 --- a/dumpling/export/dump_test.go +++ b/dumpling/export/dump_test.go @@ -9,6 +9,7 @@ import ( "time" "github.com/DATA-DOG/go-sqlmock" + "github.com/pingcap/tidb/util/promutil" "github.com/stretchr/testify/require" "golang.org/x/sync/errgroup" @@ -207,5 +208,20 @@ func TestAdjustTableCollation(t *testing.T) { require.NoError(t, err) require.Equal(t, expectedSQLs[i], newSQL) } +} + +func TestUnregisterMetrics(t *testing.T) { + ctx := context.Background() + conf := &Config{ + SQL: "not empty", + Where: "not empty", + PromFactory: promutil.NewDefaultFactory(), + PromRegistry: promutil.NewDefaultRegistry(), + } + _, err := NewDumper(ctx, conf) + require.Error(t, err) + _, err = NewDumper(ctx, conf) + // should not panic + require.Error(t, err) } From d4dc6b54c684f75c4d4295581e51fbd4da58dcd4 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 22 Jun 2022 22:00:37 +0800 Subject: [PATCH 13/21] *: enable staticcheck and prealloc on nogo (#35486) ref pingcap/tidb#35345 --- BUILD.bazel | 1 + DEPS.bzl | 33 ++- bindinfo/bind_cache.go | 1 + br/cmd/tidb-lightning-ctl/main.go | 2 +- br/pkg/lightning/backend/kv/sql2kv.go | 2 +- br/pkg/lightning/backend/local/duplicate.go | 11 +- br/pkg/lightning/config/config_test.go | 2 +- br/pkg/lightning/errormanager/errormanager.go | 1 + build/BUILD.bazel | 39 +++- build/linter/prealloc/BUILD.bazel | 13 ++ build/linter/prealloc/analyzer.go | 60 ++++++ build/linter/staticcheck/BUILD.bazel | 26 +++ build/linter/staticcheck/analyzer.go | 33 +++ build/linter/staticcheck/def.bzl | 167 +++++++++++++++ build/linter/staticcheck/util.go | 55 +++++ build/linter/util/BUILD.bazel | 12 ++ build/linter/util/util.go | 149 +++++++++++++ build/nogo_config.json | 202 ++++++++++++++++++ cmd/importer/db.go | 2 +- executor/analyze_test.go | 2 +- executor/builder.go | 1 + executor/infoschema_reader.go | 2 +- executor/inspection_result.go | 1 + executor/memtable_reader.go | 9 +- expression/builtin_string.go | 1 + go.mod | 14 +- go.sum | 31 +-- infoschema/perfschema/tables.go | 2 +- infoschema/tables.go | 1 + parser/consistent_test.go | 1 + parser/model/model.go | 6 +- parser/parser_test.go | 1 + parser/yy_parser.go | 3 +- planner/core/rule_column_pruning.go | 1 + planner/core/rule_join_reorder.go | 1 + planner/core/rule_join_reorder_dp.go | 9 +- privilege/privileges/cache.go | 2 +- server/conn.go | 4 +- store/driver/tikv_driver.go | 1 + tools/check/ut.go | 2 + util/importer/db.go | 2 +- util/processinfo.go | 3 +- util/ranger/detacher.go | 2 +- 43 files changed, 855 insertions(+), 58 deletions(-) create mode 100644 build/linter/prealloc/BUILD.bazel create mode 100644 build/linter/prealloc/analyzer.go create mode 100644 build/linter/staticcheck/BUILD.bazel create mode 100644 build/linter/staticcheck/analyzer.go create mode 100644 build/linter/staticcheck/def.bzl create mode 100644 build/linter/staticcheck/util.go create mode 100644 build/linter/util/BUILD.bazel create mode 100644 build/linter/util/util.go diff --git a/BUILD.bazel b/BUILD.bazel index 00f4ed67a67c6..71ab0b44b41e1 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -4,6 +4,7 @@ load("@bazel_gazelle//:def.bzl", "gazelle") # gazelle:prefix github.com/pingcap/tidb # gazelle:exclude tidb-binlog/proto/proto # gazelle:exclude plugin/conn_ip_example +# gazelle:exclude build/linter/staticcheck gazelle(name = "gazelle") package(default_visibility = ["//visibility:public"]) diff --git a/DEPS.bzl b/DEPS.bzl index 32f321378fa33..c9625069192e7 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -5,8 +5,9 @@ def go_deps(): name = "co_honnef_go_tools", build_file_proto_mode = "disable_global", importpath = "honnef.co/go/tools", - sum = "h1:UoveltGrhghAA7ePc+e+QYDHXrBps2PqFZiHkGR/xK8=", - version = "v0.0.1-2020.1.4", + replace = "honnef.co/go/tools", + sum = "h1:ytYb4rOqyp1TSa2EPvNVwtPQJctSELKaMyLfqNP4+34=", + version = "v0.3.2", ) go_repository( name = "com_github_ajg_form", @@ -174,8 +175,8 @@ def go_deps(): name = "com_github_burntsushi_toml", build_file_proto_mode = "disable_global", importpath = "github.com/BurntSushi/toml", - sum = "h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ=", - version = "v0.3.1", + sum = "h1:GaI7EiDXDRfa8VshkTj7Fym7ha+y8/XxIgD2okUIjLw=", + version = "v0.4.1", ) go_repository( name = "com_github_burntsushi_xgb", @@ -918,6 +919,14 @@ def go_deps(): sum = "h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM=", version = "v0.0.4", ) + go_repository( + name = "com_github_golangci_prealloc", + build_file_proto_mode = "disable", + importpath = "github.com/golangci/prealloc", + sum = "h1:leSNB7iYzLYSSx3J/s5sVf4Drkc68W2wm4Ixh/mr0us=", + version = "v0.0.0-20180630174525-215b22d4de21", + ) + go_repository( name = "com_github_gomodule_redigo", build_file_proto_mode = "disable_global", @@ -2928,6 +2937,14 @@ def go_deps(): sum = "h1:rxKZ2gOnYxjfmakvUUqh9Gyb6KXfrj7JWTxORTYqb0E=", version = "v0.0.0-20220426173459-3bcf042a4bf5", ) + go_repository( + name = "org_golang_x_exp_typeparams", + build_file_proto_mode = "disable", + importpath = "golang.org/x/exp/typeparams", + sum = "h1:qyrTQ++p1afMkO4DPEeLGq/3oTsdlvdH4vqZUBWzUKM=", + version = "v0.0.0-20220218215828-6cf2b201936e", + ) + go_repository( name = "org_golang_x_image", build_file_proto_mode = "disable_global", @@ -2953,8 +2970,8 @@ def go_deps(): name = "org_golang_x_mod", build_file_proto_mode = "disable_global", importpath = "golang.org/x/mod", - sum = "h1:LQmS1nU0twXLA96Kt7U9qtHJEbBk3z6Q0V4UXjZkpr4=", - version = "v0.6.0-dev.0.20211013180041-c96bc1413d57", + sum = "h1:6zppjxzCulZykYSLyVDYbneBfbaBIQPYMevg0bEwv2s=", + version = "v0.6.0-dev.0.20220419223038-86c51ed26bb4", ) go_repository( name = "org_golang_x_net", @@ -3009,8 +3026,8 @@ def go_deps(): name = "org_golang_x_tools", build_file_proto_mode = "disable_global", importpath = "golang.org/x/tools", - sum = "h1:P1HhGGuLW4aAclzjtmJdf0mJOjVUZUzOTqkAkWL+l6w=", - version = "v0.1.8", + sum = "h1:OKYpQQVE3DKSc3r3zHVzq46vq5YH7x8xpR3/k9ixmUg=", + version = "v0.1.11-0.20220513221640-090b14e8501f", ) go_repository( name = "org_golang_x_xerrors", diff --git a/bindinfo/bind_cache.go b/bindinfo/bind_cache.go index f1f58f43dae4a..8ce69deedd840 100644 --- a/bindinfo/bind_cache.go +++ b/bindinfo/bind_cache.go @@ -153,6 +153,7 @@ func (c *bindCache) GetAllBindRecords() []*BindRecord { c.lock.Lock() defer c.lock.Unlock() values := c.cache.Values() + //nolint: prealloc var bindRecords []*BindRecord for _, vals := range values { bindRecords = append(bindRecords, vals.([]*BindRecord)...) diff --git a/br/cmd/tidb-lightning-ctl/main.go b/br/cmd/tidb-lightning-ctl/main.go index 77dd52541470c..08f0c080963f3 100644 --- a/br/cmd/tidb-lightning-ctl/main.go +++ b/br/cmd/tidb-lightning-ctl/main.go @@ -262,7 +262,7 @@ func checkpointDump(ctx context.Context, cfg *config.Config, dumpFolder string) } func getLocalStoringTables(ctx context.Context, cfg *config.Config) (err2 error) { - //nolint:prealloc // This is a placeholder. + //nolint: prealloc var tables []string defer func() { if err2 == nil { diff --git a/br/pkg/lightning/backend/kv/sql2kv.go b/br/pkg/lightning/backend/kv/sql2kv.go index 46edf79d5e719..bd13f27e38954 100644 --- a/br/pkg/lightning/backend/kv/sql2kv.go +++ b/br/pkg/lightning/backend/kv/sql2kv.go @@ -357,7 +357,7 @@ func (kvcodec *tableKVEncoder) Encode( var value types.Datum var err error - //nolint:prealloc // This is a placeholder. + //nolint: prealloc var record []types.Datum if kvcodec.recordCache != nil { diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index 37023c48c2179..be446f81b9ba4 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -223,6 +223,7 @@ func tableHandleKeyRanges(tableInfo *model.TableInfo) ([]tidbkv.KeyRange, error) // tableIndexKeyRanges returns all key ranges associated with the tableInfo and indexInfo. func tableIndexKeyRanges(tableInfo *model.TableInfo, indexInfo *model.IndexInfo) ([]tidbkv.KeyRange, error) { tableIDs := physicalTableIDs(tableInfo) + //nolint: prealloc var keyRanges []tidbkv.KeyRange for _, tid := range tableIDs { partitionKeysRanges, err := distsql.IndexRangesToKVRanges(nil, tid, indexInfo.ID, ranger.FullRange(), nil) @@ -606,6 +607,7 @@ func (m *DuplicateManager) buildLocalDupTasks(dupDB *pebble.DB, keyAdapter KeyAd if err != nil { return nil, errors.Trace(err) } + //nolint: prealloc var newTasks []dupTask for _, task := range tasks { // FIXME: Do not hardcode sizeLimit and keysLimit. @@ -706,10 +708,11 @@ func (m *DuplicateManager) processRemoteDupTaskOnce( regionPool *utils.WorkerPool, remainKeyRanges *pendingKeyRanges, ) (madeProgress bool, err error) { - var ( - regions []*restore.RegionInfo - keyRanges []tidbkv.KeyRange - ) + //nolint: prealloc + var regions []*restore.RegionInfo + //nolint: prealloc + var keyRanges []tidbkv.KeyRange + for _, kr := range remainKeyRanges.list() { subRegions, subKeyRanges, err := m.splitKeyRangeByRegions(ctx, kr) if err != nil { diff --git a/br/pkg/lightning/config/config_test.go b/br/pkg/lightning/config/config_test.go index 881f89de3d380..555cbed109f1d 100644 --- a/br/pkg/lightning/config/config_test.go +++ b/br/pkg/lightning/config/config_test.go @@ -517,7 +517,7 @@ func TestInvalidTOML(t *testing.T) { delimiter = '\' backslash-escape = true `)) - require.EqualError(t, err, "Near line 0 (last key parsed ''): bare keys cannot contain '['") + require.EqualError(t, err, "Near line 2 (last key parsed ''): expected '.' or '=', but got '[' instead") } func TestTOMLUnusedKeys(t *testing.T) { diff --git a/br/pkg/lightning/errormanager/errormanager.go b/br/pkg/lightning/errormanager/errormanager.go index c663839f3cf35..b900d19c3a431 100644 --- a/br/pkg/lightning/errormanager/errormanager.go +++ b/br/pkg/lightning/errormanager/errormanager.go @@ -353,6 +353,7 @@ func (em *ErrorManager) ResolveAllConflictKeys( go func() { //nolint:staticcheck + //lint:ignore SA2000 taskWg.Add(1) taskCh <- [2]int64{0, math.MaxInt64} taskWg.Wait() diff --git a/build/BUILD.bazel b/build/BUILD.bazel index 294fe5b131748..22bc4b679c26a 100644 --- a/build/BUILD.bazel +++ b/build/BUILD.bazel @@ -1,6 +1,42 @@ package(default_visibility = ["//visibility:public"]) load("@io_bazel_rules_go//go:def.bzl", "nogo") +load("//build/linter/staticcheck:def.bzl", "staticcheck_analyzers") + +STATICHECK_ANALYZERS = [ + "S1002", + "S1004", + "S1007", + "S1009", + "S1010", + "S1012", + "S1019", + "S1020", + "S1021", + "S1024", + "S1030", + "SA2000", + "SA2001", + "SA2003", + "SA3000", + "SA3001", + "SA4009", + "SA5000", + "SA5001", + "SA5002", + "SA5003", + "SA5004", + "SA5005", + "SA5007", + "SA5008", + "SA5009", + "SA5010", + #"SA5011", + "SA5012", + "SA6000", + "SA6001", + "SA6005", +] nogo( name = "tidb_nogo", @@ -45,5 +81,6 @@ nogo( "@org_golang_x_tools//go/analysis/passes/unreachable:go_default_library", "@org_golang_x_tools//go/analysis/passes/unsafeptr:go_default_library", "@org_golang_x_tools//go/analysis/passes/unusedresult:go_default_library", - ], + "//build/linter/prealloc:prealloc", + ] + staticcheck_analyzers(STATICHECK_ANALYZERS), ) diff --git a/build/linter/prealloc/BUILD.bazel b/build/linter/prealloc/BUILD.bazel new file mode 100644 index 0000000000000..452108a450df5 --- /dev/null +++ b/build/linter/prealloc/BUILD.bazel @@ -0,0 +1,13 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "prealloc", + srcs = ["analyzer.go"], + importpath = "github.com/pingcap/tidb/build/linter/prealloc", + visibility = ["//visibility:public"], + deps = [ + "//build/linter/util", + "@com_github_golangci_prealloc//:prealloc", + "@org_golang_x_tools//go/analysis", + ], +) diff --git a/build/linter/prealloc/analyzer.go b/build/linter/prealloc/analyzer.go new file mode 100644 index 0000000000000..5e9d38aa8daa9 --- /dev/null +++ b/build/linter/prealloc/analyzer.go @@ -0,0 +1,60 @@ +// Copyright 2022 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, +// 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 prealloc + +import ( + "go/ast" + + "github.com/golangci/prealloc" + "github.com/pingcap/tidb/build/linter/util" + "golang.org/x/tools/go/analysis" +) + +// Settings is the settings for preallocation. +type Settings struct { + Simple bool + RangeLoops bool `mapstructure:"range-loops"` + ForLoops bool `mapstructure:"range-loops"` +} + +// Name is the name of the analyzer. +const Name = "prealloc" + +// Analyzer is the analyzer struct of prealloc. +var Analyzer = &analysis.Analyzer{ + Name: Name, + Doc: "Finds slice declarations that could potentially be preallocated", + Run: run, +} + +func run(pass *analysis.Pass) (interface{}, error) { + s := &Settings{ + Simple: true, + RangeLoops: true, + ForLoops: false, + } + for _, f := range pass.Files { + hints := prealloc.Check([]*ast.File{f}, s.Simple, s.RangeLoops, s.ForLoops) + for _, hint := range hints { + pass.Reportf(hint.Pos, "[%s] Consider preallocating %s", Name, util.FormatCode(hint.DeclaredSliceName)) + } + } + + return nil, nil +} + +func init() { + util.SkipAnalyzer(Analyzer) +} diff --git a/build/linter/staticcheck/BUILD.bazel b/build/linter/staticcheck/BUILD.bazel new file mode 100644 index 0000000000000..6525dc6be346a --- /dev/null +++ b/build/linter/staticcheck/BUILD.bazel @@ -0,0 +1,26 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//build/linter/staticcheck:def.bzl", "ANALYZERS") + +[ + go_library( + name = analyzer, + srcs = [ + "analyzer.go", + "util.go", + ], + importpath = "github.com/pingcap/tidb/build/linter/staticcheck/" + analyzer, + visibility = ["//visibility:public"], + x_defs = {"name": analyzer}, + deps = [ + "//build/linter/util", + "@co_honnef_go_tools//analysis/lint", + "@co_honnef_go_tools//quickfix", + "@co_honnef_go_tools//simple", + "@co_honnef_go_tools//staticcheck", + "@co_honnef_go_tools//stylecheck", + "@co_honnef_go_tools//unused", + "@org_golang_x_tools//go/analysis", + ], + ) + for analyzer in ANALYZERS +] diff --git a/build/linter/staticcheck/analyzer.go b/build/linter/staticcheck/analyzer.go new file mode 100644 index 0000000000000..cb13969093aa9 --- /dev/null +++ b/build/linter/staticcheck/analyzer.go @@ -0,0 +1,33 @@ +// Copyright 2022 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, +// 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 staticcheck + +import ( + "github.com/pingcap/tidb/build/linter/util" + "golang.org/x/tools/go/analysis" +) + +var ( + // Value to be added during stamping + name = "dummy value please replace using x_defs" + + // Analyzer is an analyzer from staticcheck. + Analyzer *analysis.Analyzer +) + +func init() { + Analyzer = FindAnalyzerByName(name) + util.SkipAnalyzer(Analyzer) +} diff --git a/build/linter/staticcheck/def.bzl b/build/linter/staticcheck/def.bzl new file mode 100644 index 0000000000000..2e375d543b710 --- /dev/null +++ b/build/linter/staticcheck/def.bzl @@ -0,0 +1,167 @@ +# All analyzers in staticcheck. +# +# Generate this list by running: +# +# > bazel run //staticcheck/cmd/list_analyzers +# +ANALYZERS = [ + "QF1001", + "QF1002", + "QF1003", + "QF1004", + "QF1005", + "QF1006", + "QF1007", + "QF1008", + "QF1009", + "QF1010", + "QF1011", + "QF1012", + "S1000", + "S1001", + "S1002", + "S1003", + "S1004", + "S1005", + "S1006", + "S1007", + "S1008", + "S1009", + "S1010", + "S1011", + "S1012", + "S1016", + "S1017", + "S1018", + "S1019", + "S1020", + "S1021", + "S1023", + "S1024", + "S1025", + "S1028", + "S1029", + "S1030", + "S1031", + "S1032", + "S1033", + "S1034", + "S1035", + "S1036", + "S1037", + "S1038", + "S1039", + "S1040", + "SA1000", + "SA1001", + "SA1002", + "SA1003", + "SA1004", + "SA1005", + "SA1006", + "SA1007", + "SA1008", + "SA1010", + "SA1011", + "SA1012", + "SA1013", + "SA1014", + "SA1015", + "SA1016", + "SA1017", + "SA1018", + "SA1019", + "SA1020", + "SA1021", + "SA1023", + "SA1024", + "SA1025", + "SA1026", + "SA1027", + "SA1028", + "SA1029", + "SA1030", + "SA2000", + "SA2001", + "SA2002", + "SA2003", + "SA3000", + "SA3001", + "SA4000", + "SA4001", + "SA4003", + "SA4004", + "SA4005", + "SA4006", + "SA4008", + "SA4009", + "SA4010", + "SA4011", + "SA4012", + "SA4013", + "SA4014", + "SA4015", + "SA4016", + "SA4017", + "SA4018", + "SA4019", + "SA4020", + "SA4021", + "SA4022", + "SA4023", + "SA4024", + "SA4025", + "SA4026", + "SA4027", + "SA4028", + "SA4029", + "SA4030", + "SA4031", + "SA5000", + "SA5001", + "SA5002", + "SA5003", + "SA5004", + "SA5005", + "SA5007", + "SA5008", + "SA5009", + "SA5010", + "SA5011", + "SA5012", + "SA6000", + "SA6001", + "SA6002", + "SA6003", + "SA6005", + "SA9001", + "SA9002", + "SA9003", + "SA9004", + "SA9005", + "SA9006", + "SA9007", + "SA9008", + "ST1000", + "ST1001", + "ST1003", + "ST1005", + "ST1006", + "ST1008", + "ST1011", + "ST1012", + "ST1013", + "ST1015", + "ST1016", + "ST1017", + "ST1018", + "ST1019", + "ST1020", + "ST1021", + "ST1022", + "ST1023", + "U1000", +] + +def staticcheck_analyzers(analyzers, prefix_path = "//build/linter/staticcheck"): + return [prefix_path + ":" + a for a in analyzers] diff --git a/build/linter/staticcheck/util.go b/build/linter/staticcheck/util.go new file mode 100644 index 0000000000000..0c1928d564268 --- /dev/null +++ b/build/linter/staticcheck/util.go @@ -0,0 +1,55 @@ +// Copyright 2021 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, +// 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 staticcheck + +import ( + "fmt" + + "golang.org/x/tools/go/analysis" + "honnef.co/go/tools/analysis/lint" + "honnef.co/go/tools/quickfix" + "honnef.co/go/tools/simple" + "honnef.co/go/tools/staticcheck" + "honnef.co/go/tools/stylecheck" + "honnef.co/go/tools/unused" +) + +// Analyzers is the analyzers of staticcheck. +var Analyzers = func() map[string]*analysis.Analyzer { + resMap := make(map[string]*analysis.Analyzer) + + for _, analyzers := range [][]*lint.Analyzer{ + quickfix.Analyzers, + simple.Analyzers, + staticcheck.Analyzers, + stylecheck.Analyzers, + {unused.Analyzer}, + } { + for _, a := range analyzers { + resMap[a.Analyzer.Name] = a.Analyzer + } + } + + return resMap +}() + +// FindAnalyzerByName finds the analyzer with the given name. +func FindAnalyzerByName(name string) *analysis.Analyzer { + if a, ok := Analyzers[name]; ok { + return a + } + + panic(fmt.Sprintf("not a valid staticcheck analyzer: %s", name)) +} diff --git a/build/linter/util/BUILD.bazel b/build/linter/util/BUILD.bazel new file mode 100644 index 0000000000000..4ac3fec064d07 --- /dev/null +++ b/build/linter/util/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "util", + srcs = ["util.go"], + importpath = "github.com/pingcap/tidb/build/linter/util", + visibility = ["//visibility:public"], + deps = [ + "@co_honnef_go_tools//analysis/report", + "@org_golang_x_tools//go/analysis", + ], +) diff --git a/build/linter/util/util.go b/build/linter/util/util.go new file mode 100644 index 0000000000000..d438ea0057a6e --- /dev/null +++ b/build/linter/util/util.go @@ -0,0 +1,149 @@ +// Copyright 2022 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, +// 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 util + +import ( + "fmt" + "go/ast" + "go/token" + "reflect" + "strings" + + "golang.org/x/tools/go/analysis" + "honnef.co/go/tools/analysis/report" +) + +// +type skipType int + +const ( + skipNone skipType = iota + skipLinter + skipFile +) + +// Directive is a comment of the form '//lint: [arguments...]' and `//nolint:`. +// It represents instructions to the static analysis tool. +type Directive struct { + Command skipType + Linters []string + Directive *ast.Comment + Node ast.Node +} + +func parseDirective(s string) (cmd skipType, args []string) { + if strings.HasPrefix(s, "//lint:") { + s = strings.TrimPrefix(s, "//lint:") + fields := strings.Split(s, " ") + switch fields[0] { + case "ignore": + return skipLinter, fields[1:] + case "file-ignore": + return skipFile, fields[1:] + } + return skipNone, nil + } + s = strings.TrimPrefix(s, "//nolint: ") + return skipLinter, []string{s} +} + +// ParseDirectives extracts all directives from a list of Go files. +func ParseDirectives(files []*ast.File, fset *token.FileSet) []Directive { + var dirs []Directive + for _, f := range files { + cm := ast.NewCommentMap(fset, f, f.Comments) + for node, cgs := range cm { + for _, cg := range cgs { + for _, c := range cg.List { + if !strings.HasPrefix(c.Text, "//lint:") && !strings.HasPrefix(c.Text, "//nolint:") { + continue + } + cmd, args := parseDirective(c.Text) + d := Directive{ + Command: cmd, + Linters: args, + Directive: c, + Node: node, + } + dirs = append(dirs, d) + } + } + } + } + return dirs +} + +func doDirectives(pass *analysis.Pass) (interface{}, error) { + return ParseDirectives(pass.Files, pass.Fset), nil +} + +// Directives is a fact that contains a list of directives. +var Directives = &analysis.Analyzer{ + Name: "directives", + Doc: "extracts linter directives", + Run: doDirectives, + RunDespiteErrors: true, + ResultType: reflect.TypeOf([]Directive{}), +} + +// SkipAnalyzer updates an analyzer from `staticcheck` and `golangci-linter` to make it work on nogo. +// They have "lint:ignore" or "nolint" to make the analyzer ignore the code. +func SkipAnalyzer(analyzer *analysis.Analyzer) { + analyzer.Requires = append(analyzer.Requires, Directives) + oldRun := analyzer.Run + analyzer.Run = func(p *analysis.Pass) (interface{}, error) { + pass := *p + oldReport := p.Report + pass.Report = func(diag analysis.Diagnostic) { + dirs := pass.ResultOf[Directives].([]Directive) + for _, dir := range dirs { + cmd := dir.Command + linters := dir.Linters + switch cmd { + case skipLinter: + ignorePos := report.DisplayPosition(pass.Fset, dir.Node.Pos()) + nodePos := report.DisplayPosition(pass.Fset, diag.Pos) + if ignorePos.Filename != nodePos.Filename || ignorePos.Line != nodePos.Line { + continue + } + for _, check := range strings.Split(linters[0], ",") { + if strings.TrimSpace(check) == analyzer.Name { + return + } + } + case skipFile: + ignorePos := report.DisplayPosition(pass.Fset, dir.Node.Pos()) + nodePos := report.DisplayPosition(pass.Fset, diag.Pos) + if ignorePos.Filename == nodePos.Filename { + return + } + default: + continue + } + } + oldReport(diag) + } + return oldRun(&pass) + } +} + +// FormatCode is to format code for nogo. +func FormatCode(code string) string { + if strings.Contains(code, "`") { + return code // TODO: properly escape or remove + } + + return fmt.Sprintf("`%s`", code) +} diff --git a/build/nogo_config.json b/build/nogo_config.json index c462883ab232e..f05718081c8ac 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -201,5 +201,207 @@ ".*_generated\\.go$": "ignore generated code", "parser/digester_test.go": "ignore code" } + }, + "S1002": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1004": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1007": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1009": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1010": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1012": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1019": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code", + "parser/parser.go": "ignore code" + } + }, + "S1020": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1021": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code", + "tools/check/ut.go": "ignore code" + } + }, + "S1024": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "S1030": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA2000": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA2001": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA2003": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA3000": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA3001": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA4009": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5000": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5001": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5002": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5003": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5004": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5005": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5007": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5008": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5009": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5010": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5011": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA5012": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA6000": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA6001": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "SA6005": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, + "prealloc": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code", + "parser/yy_parser.go": "ignore generated code", + "/cgo/": "no need to vet third party code for cgo" + } } } diff --git a/cmd/importer/db.go b/cmd/importer/db.go index 49f3d0ec67ad5..8b0d7353b9adf 100644 --- a/cmd/importer/db.go +++ b/cmd/importer/db.go @@ -117,7 +117,7 @@ func genRowDatas(table *table, count int) ([]string, error) { } func genRowData(table *table) (string, error) { - var values []byte // nolint: prealloc + var values []byte //nolint: prealloc for _, column := range table.columns { data, err := genColumnData(table, column) if err != nil { diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 32c2ec30bbd14..055591b8a9a7c 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -80,7 +80,7 @@ func TestAnalyzeFastSample(t *testing.T) { } handleCols := core.BuildHandleColsForAnalyze(tk.Session(), tblInfo, true, nil) - var colsInfo []*model.ColumnInfo // nolint: prealloc + var colsInfo []*model.ColumnInfo //nolint: prealloc var indicesInfo []*model.IndexInfo for _, col := range tblInfo.Columns { if mysql.HasPriKeyFlag(col.GetFlag()) { diff --git a/executor/builder.go b/executor/builder.go index 630ef3137db58..de77b2b88f252 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4108,6 +4108,7 @@ func (h kvRangeBuilderFromRangeAndPartition) buildKeyRangeSeparately(ranges []*r } func (h kvRangeBuilderFromRangeAndPartition) buildKeyRange(ranges []*ranger.Range) ([]kv.KeyRange, error) { + //nolint: prealloc var ret []kv.KeyRange for _, p := range h.partitions { pid := p.GetPhysicalID() diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index b26687757942a..8eea08a6675b5 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -2844,7 +2844,7 @@ func (e *TiFlashSystemTableRetriever) initialize(sctx sessionctx.Context, tiflas } func (e *TiFlashSystemTableRetriever) dataForTiFlashSystemTables(ctx sessionctx.Context, tidbDatabases string, tidbTables string) ([][]types.Datum, error) { - var columnNames []string // nolint: prealloc + var columnNames []string //nolint: prealloc for _, c := range e.outputCols { if c.Name.O == "TIFLASH_INSTANCE" { continue diff --git a/executor/inspection_result.go b/executor/inspection_result.go index debcf723a3e64..741508c5cf88f 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -727,6 +727,7 @@ func (c thresholdCheckInspection) inspect(ctx context.Context, sctx sessionctx.C c.inspectThreshold3, c.inspectForLeaderDrop, } + //nolint: prealloc var results []inspectionResult for _, inspect := range inspects { re := inspect(ctx, sctx, filter) diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 64729e3705dcb..080a7d5d83a4d 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -178,7 +178,7 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String return nil, err } serversInfo = filterClusterServerInfo(serversInfo, nodeTypes, nodeAddrs) - + //nolint: prealloc var finalRows [][]types.Datum wg := sync.WaitGroup{} ch := make(chan result, len(serversInfo)) @@ -271,7 +271,7 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String close(ch) // Keep the original order to make the result more stable - var results []result // nolint: prealloc + var results []result //nolint: prealloc for result := range ch { if result.err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(result.err) @@ -349,7 +349,7 @@ func (e *clusterServerInfoRetriever) retrieve(ctx context.Context, sctx sessionc wg.Wait() close(ch) // Keep the original order to make the result more stable - var results []result // nolint: prealloc + var results []result //nolint: prealloc for result := range ch { if result.err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(result.err) @@ -565,7 +565,7 @@ func (e *clusterLogRetriever) startRetrieving( // The retrieve progress may be abort ctx, e.cancel = context.WithCancel(ctx) - var results []chan logStreamResult // nolint: prealloc + var results []chan logStreamResult //nolint: prealloc for _, srv := range serversInfo { typ := srv.ServerType address := srv.Address @@ -1070,6 +1070,7 @@ func (e *tikvRegionPeersRetriever) isUnexpectedStoreID(storeID int64, storeMap m func (e *tikvRegionPeersRetriever) packTiKVRegionPeersRows( regionsInfo []helper.RegionInfo, storeMap map[int64]struct{}) ([][]types.Datum, error) { + //nolint: prealloc var rows [][]types.Datum for _, region := range regionsInfo { records := make([][]types.Datum, 0, len(region.Peers)) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index e46490e1cf31d..55ab3d15c7486 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -324,6 +324,7 @@ func (b *builtinConcatSig) Clone() builtinFunc { // evalString evals a builtinConcatSig // See https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_concat func (b *builtinConcatSig) evalString(row chunk.Row) (d string, isNull bool, err error) { + //nolint: prealloc var s []byte for _, a := range b.getArgs() { d, isNull, err = a.EvalString(b.ctx, row) diff --git a/go.mod b/go.mod index bfd038156bf93..cd5327db9c2c2 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,7 @@ require ( cloud.google.com/go/storage v1.21.0 github.com/Azure/azure-sdk-for-go/sdk/azidentity v0.12.0 github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v0.2.0 - github.com/BurntSushi/toml v0.3.1 + github.com/BurntSushi/toml v0.4.1 github.com/DATA-DOG/go-sqlmock v1.5.0 github.com/Jeffail/gabs/v2 v2.5.1 github.com/Shopify/sarama v1.29.0 @@ -87,7 +87,7 @@ require ( golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 golang.org/x/text v0.3.7 golang.org/x/time v0.0.0-20220224211638-0e9765cccd65 - golang.org/x/tools v0.1.8 + golang.org/x/tools v0.1.11-0.20220513221640-090b14e8501f google.golang.org/api v0.69.0 google.golang.org/grpc v1.44.0 gopkg.in/yaml.v2 v2.4.0 @@ -95,7 +95,10 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -require github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581 +require ( + github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581 + honnef.co/go/tools v0.0.1-2020.1.4 +) require ( cloud.google.com/go v0.100.2 // indirect @@ -128,6 +131,7 @@ require ( github.com/go-ole/go-ole v1.2.6 // indirect github.com/golang/glog v1.0.0 // indirect github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect + github.com/golangci/prealloc v0.0.0-20180630174525-215b22d4de21 github.com/google/go-cmp v0.5.7 // indirect github.com/googleapis/gax-go/v2 v2.1.1 // indirect github.com/gorilla/handlers v1.5.1 // indirect @@ -193,6 +197,8 @@ require ( go.opentelemetry.io/otel/trace v0.20.0 // indirect go.opentelemetry.io/proto/otlp v0.7.0 // indirect golang.org/x/crypto v0.0.0-20220214200702-86341886e292 // indirect + golang.org/x/exp/typeparams v0.0.0-20220218215828-6cf2b201936e // indirect + golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4 // indirect golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20220216160803-4663080d8bc8 // indirect @@ -206,3 +212,5 @@ replace github.com/pingcap/tidb/parser => ./parser // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible + +replace honnef.co/go/tools => honnef.co/go/tools v0.3.2 diff --git a/go.sum b/go.sum index 3a6e62f536235..c5f5f515043a7 100644 --- a/go.sum +++ b/go.sum @@ -64,8 +64,9 @@ github.com/Azure/azure-sdk-for-go/sdk/internal v0.8.1 h1:BUYIbDf/mMZ8945v3QkG3Ou github.com/Azure/azure-sdk-for-go/sdk/internal v0.8.1/go.mod h1:KLF4gFr6DcKFZwSuH8w8yEK6DpFl3LP5rhdvAb7Yz5I= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v0.2.0 h1:62Ew5xXg5UCGIXDOM7+y4IL5/6mQJq1nenhBCJAeGX8= github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v0.2.0/go.mod h1:eHWhQKXc1Gv1DvWH//UzgWjWFEo0Pp4pH2vBzjBw8Fc= -github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/toml v0.4.1 h1:GaI7EiDXDRfa8VshkTj7Fym7ha+y8/XxIgD2okUIjLw= +github.com/BurntSushi/toml v0.4.1/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= @@ -340,6 +341,8 @@ github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf/go.mod h1:/XxbfmMg github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golangci/prealloc v0.0.0-20180630174525-215b22d4de21 h1:leSNB7iYzLYSSx3J/s5sVf4Drkc68W2wm4Ixh/mr0us= +github.com/golangci/prealloc v0.0.0-20180630174525-215b22d4de21/go.mod h1:tf5+bzsHdTM0bsB7+8mt0GUMvjCgwLpTapNZHU8AajI= github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= @@ -384,7 +387,6 @@ github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20211122183932-1daafda22083 h1:c8EUapQFi+kjzedr4c6WqbwMdmB95+oDBWZ5XFHFYxY= github.com/google/pprof v0.0.0-20211122183932-1daafda22083/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= -github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= @@ -700,7 +702,6 @@ github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= -github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= @@ -809,6 +810,7 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= @@ -905,6 +907,7 @@ golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220214200702-86341886e292 h1:f+lwQ+GtmgoY+A2YaQxlSOnDjXcQ7ZRLWOHbC6HtRqE= golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -924,6 +927,8 @@ golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMk golang.org/x/exp v0.0.0-20200513190911-00229845015e/go.mod h1:4M0jN8W1tt0AVLNr8HDosyJCDCDuyL9N9+3m7wDWgKw= golang.org/x/exp v0.0.0-20220426173459-3bcf042a4bf5 h1:rxKZ2gOnYxjfmakvUUqh9Gyb6KXfrj7JWTxORTYqb0E= golang.org/x/exp v0.0.0-20220426173459-3bcf042a4bf5/go.mod h1:lgLbSvA5ygNOMpwM/9anMpWVlVJ7Z+cHWq/eFuinpGE= +golang.org/x/exp/typeparams v0.0.0-20220218215828-6cf2b201936e h1:qyrTQ++p1afMkO4DPEeLGq/3oTsdlvdH4vqZUBWzUKM= +golang.org/x/exp/typeparams v0.0.0-20220218215828-6cf2b201936e/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= @@ -942,7 +947,6 @@ golang.org/x/lint v0.0.0-20210508222113-6edffad5e616 h1:VLliZ0d+/avPrXXH+OakdXhp golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= -golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= @@ -951,6 +955,8 @@ golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4 h1:6zppjxzCulZykYSLyVDYbneBfbaBIQPYMevg0bEwv2s= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1004,6 +1010,7 @@ golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd h1:O7DYs+zxREGLKzKoMQrtrEacpb0ZVXA5rIwylE2Xchk= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -1114,6 +1121,7 @@ golang.org/x/sys v0.0.0-20210908233432-aa78b53d3365/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211124211545-fe61309f8881/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211210111614-af8b64212486/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -1157,7 +1165,6 @@ golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBn golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -1171,7 +1178,6 @@ golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= @@ -1207,8 +1213,8 @@ golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.8 h1:P1HhGGuLW4aAclzjtmJdf0mJOjVUZUzOTqkAkWL+l6w= -golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= +golang.org/x/tools v0.1.11-0.20220513221640-090b14e8501f h1:OKYpQQVE3DKSc3r3zHVzq46vq5YH7x8xpR3/k9ixmUg= +golang.org/x/tools v0.1.11-0.20220513221640-090b14e8501f/go.mod h1:SgwaegtQh8clINPpECJMqnxLv9I09HLqnW3RMqW0CA4= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1416,13 +1422,8 @@ gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= -honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.3.2 h1:ytYb4rOqyp1TSa2EPvNVwtPQJctSELKaMyLfqNP4+34= +honnef.co/go/tools v0.3.2/go.mod h1:jzwdWgg7Jdq75wlfblQxO4neNaFFSvgc1tD5Wv8U0Yw= modernc.org/fileutil v1.0.0/go.mod h1:JHsWpkrk/CnVV1H/eGlFf85BEpfkrp56ro8nojIq9Q8= modernc.org/golex v1.0.1/go.mod h1:QCA53QtsT1NdGkaZZkF5ezFwk4IXh4BGNafAARTC254= modernc.org/lex v1.0.0/go.mod h1:G6rxMTy3cH2iA0iXL/HRRv4Znu8MK4higxph/lE7ypk= diff --git a/infoschema/perfschema/tables.go b/infoschema/perfschema/tables.go index 1beea67e64a1b..c0006f9f7413a 100644 --- a/infoschema/perfschema/tables.go +++ b/infoschema/perfschema/tables.go @@ -382,7 +382,7 @@ func dataForRemoteProfile(ctx sessionctx.Context, nodeType, uri string, isGorout close(ch) // Keep the original order to make the result more stable - var results []result // nolint: prealloc + var results []result //nolint: prealloc for result := range ch { if result.err != nil { ctx.GetSessionVars().StmtCtx.AppendWarning(result.err) diff --git a/infoschema/tables.go b/infoschema/tables.go index d1deae5c6739a..57e867808cede 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1580,6 +1580,7 @@ func GetClusterServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { }) type retriever func(ctx sessionctx.Context) ([]ServerInfo, error) + //nolint: prealloc var servers []ServerInfo for _, r := range []retriever{GetTiDBServerInfo, GetPDServerInfo, GetStoreServerInfo} { nodes, err := r(ctx) diff --git a/parser/consistent_test.go b/parser/consistent_test.go index 17163378b5439..e78b7f31ddddd 100644 --- a/parser/consistent_test.go +++ b/parser/consistent_test.go @@ -73,6 +73,7 @@ func extractMiddle(str, startMarker, endMarker string) string { } func extractQuotedWords(strs []string) []string { + //nolint: prealloc var words []string for _, str := range strs { word := extractMiddle(str, "\"", "\"") diff --git a/parser/model/model.go b/parser/model/model.go index eb22b4bee39bd..43e3e4bc5bcfb 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -1426,8 +1426,7 @@ type PolicyInfo struct { } func (p *PolicyInfo) Clone() *PolicyInfo { - var cloned PolicyInfo - cloned = *p + cloned := *p cloned.PlacementSettings = p.PlacementSettings.Clone() return &cloned } @@ -1489,8 +1488,7 @@ func (p *PlacementSettings) String() string { } func (p *PlacementSettings) Clone() *PlacementSettings { - var cloned PlacementSettings - cloned = *p + cloned := *p return &cloned } diff --git a/parser/parser_test.go b/parser/parser_test.go index f908cfa8d7b6e..3ab44e1a232c0 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -5735,6 +5735,7 @@ func TestNotExistsSubquery(t *testing.T) { } func TestWindowFunctionIdentifier(t *testing.T) { + //nolint: prealloc var table []testCase for key := range parser.WindowFuncTokenMapForTest { table = append(table, testCase{fmt.Sprintf("select 1 %s", key), false, fmt.Sprintf("SELECT 1 AS `%s`", key)}) diff --git a/parser/yy_parser.go b/parser/yy_parser.go index 808b4a216ebeb..221dd7a26e790 100644 --- a/parser/yy_parser.go +++ b/parser/yy_parser.go @@ -148,8 +148,7 @@ func (parser *Parser) ParseSQL(sql string, params ...ParseParam) (stmt []ast.Stm parser.src = sql parser.result = parser.result[:0] - var l yyLexer - l = &parser.lexer + var l yyLexer = &parser.lexer yyParse(l, parser) warns, errs := l.Errors() diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 5dc6506f3d2b3..b8df243bd2299 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -115,6 +115,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, } appendColumnPruneTraceStep(la, prunedColumns, opt) appendFunctionPruneTraceStep(la, prunedFunctions, opt) + //nolint: prealloc var selfUsedCols []*expression.Column for _, aggrFunc := range la.AggFuncs { selfUsedCols = expression.ExtractColumnsFromExpressions(selfUsedCols, aggrFunc.Args, nil) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 68d1fa3bf16cb..0294da8bac852 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -563,6 +563,7 @@ func findRoots(t *tracing.PlanTrace) []*tracing.PlanTrace { if t.TP == plancodec.TypeJoin || t.TP == plancodec.TypeDataSource { return []*tracing.PlanTrace{t} } + //nolint: prealloc var r []*tracing.PlanTrace for _, child := range t.Children { r = append(r, findRoots(child)...) diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index c91d74e1b7c28..c7c0f45cf2274 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -212,10 +212,11 @@ func (s *joinReorderDPSolver) dpGraph(visitID2NodeID, nodeID2VisitID []int, join func (s *joinReorderDPSolver) nodesAreConnected(leftMask, rightMask uint, oldPos2NewPos []int, totalEqEdges []joinGroupEqEdge, totalNonEqEdges []joinGroupNonEqEdge) ([]joinGroupEqEdge, []expression.Expression) { - var ( // nolint: prealloc - usedEqEdges []joinGroupEqEdge - otherConds []expression.Expression - ) + //nolint: prealloc + var usedEqEdges []joinGroupEqEdge + //nolint: prealloc + var otherConds []expression.Expression + for _, edge := range totalEqEdges { lIdx := uint(oldPos2NewPos[edge.nodeIDs[0]]) rIdx := uint(oldPos2NewPos[edge.nodeIDs[1]]) diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 9e8198650d25b..d1e85b3d45091 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -1168,7 +1168,7 @@ func (p *MySQLPrivilege) DBIsVisible(user, host, db string) bool { } func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentity) []string { - var gs []string // nolint: prealloc + var gs []string //nolint: prealloc var sortFromIdx int var hasGlobalGrant = false // Some privileges may granted from role inheritance. diff --git a/server/conn.go b/server/conn.go index 85b227b3f3bdf..e5289e4fec5af 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1936,8 +1936,8 @@ func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.Stm } } pointPlans := make([]plannercore.Plan, len(stmts)) - var idxKeys []kv.Key // nolint: prealloc - var rowKeys []kv.Key // nolint: prealloc + var idxKeys []kv.Key //nolint: prealloc + var rowKeys []kv.Key //nolint: prealloc sc := vars.StmtCtx for i, stmt := range stmts { switch stmt.(type) { diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index a3c385f39df6e..e1ba5d121608f 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -326,6 +326,7 @@ func (s *tikvStore) ShowStatus(ctx context.Context, key string) (interface{}, er // GetLockWaits get return lock waits info func (s *tikvStore) GetLockWaits() ([]*deadlockpb.WaitForEntry, error) { stores := s.GetRegionCache().GetStoresByType(tikvrpc.TiKV) + //nolint: prealloc var result []*deadlockpb.WaitForEntry for _, store := range stores { resp, err := s.GetTiKVClient().SendRequest(context.TODO(), store.GetAddr(), tikvrpc.NewRequest(tikvrpc.CmdLockWaitInfo, &kvrpcpb.GetLockWaitInfoRequest{}), time.Second*30) diff --git a/tools/check/ut.go b/tools/check/ut.go index 920afbb9cb257..84b00d34d9e60 100644 --- a/tools/check/ut.go +++ b/tools/check/ut.go @@ -720,6 +720,7 @@ func (n *numa) runTestCase(pkg string, fn string) testResult { start = time.Now() err = cmd.Run() if err != nil { + //lint:ignore S1020 if _, ok := err.(*exec.ExitError); ok { // Retry 3 times to get rid of the weird error: switch err.Error() { @@ -875,6 +876,7 @@ func buildTestBinaryMulti(pkgs []string) error { func testBinaryExist(pkg string) (bool, error) { _, err := os.Stat(testFileFullPath(pkg)) if err != nil { + //lint:ignore S1020 if _, ok := err.(*os.PathError); ok { return false, nil } diff --git a/util/importer/db.go b/util/importer/db.go index e483fa0a86dc7..17956b67f3c07 100644 --- a/util/importer/db.go +++ b/util/importer/db.go @@ -78,7 +78,7 @@ func genRowDatas(table *table, count int) ([]string, error) { } func genRowData(table *table) (string, error) { - var values []byte // nolint: prealloc + var values []byte //nolint: prealloc for _, column := range table.columns { data, err := genColumnData(table, column) if err != nil { diff --git a/util/processinfo.go b/util/processinfo.go index 30b95aec100ae..98d8733b8c111 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -153,7 +153,8 @@ var mapServerStatus2Str = map[uint16]string{ // Param state is a bit-field. (e.g. 0x0003 = "in transaction; autocommit"). func serverStatus2Str(state uint16) string { // l collect server status strings. - var l []string // nolint: prealloc + //nolint: prealloc + var l []string // check each defined server status, if match, append to collector. for _, s := range ascServerStatus { if state&s == 0 { diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 037a5402f048b..cbde3ddef60c6 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -32,7 +32,7 @@ import ( // detachColumnCNFConditions detaches the condition for calculating range from the other conditions. // Please make sure that the top level is CNF form. func detachColumnCNFConditions(sctx sessionctx.Context, conditions []expression.Expression, checker *conditionChecker) ([]expression.Expression, []expression.Expression) { - var accessConditions, filterConditions []expression.Expression // nolint: prealloc + var accessConditions, filterConditions []expression.Expression //nolint: prealloc for _, cond := range conditions { if sf, ok := cond.(*expression.ScalarFunction); ok && sf.FuncName.L == ast.LogicOr { dnfItems := expression.FlattenDNFConditions(sf) From 193f04db7061f386531dc09a5f71bb676beed14d Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 23 Jun 2022 08:32:37 +0800 Subject: [PATCH 14/21] session,sessionctx: enable tidb_enable_paging by default (#35275) close pingcap/tidb#35273 --- executor/analyzetest/analyze_test.go | 3 ++ planner/core/explain.go | 6 ---- planner/core/plan_cost_test.go | 1 + planner/core/plan_test.go | 4 +-- session/bootstrap.go | 2 ++ session/bootstrap_test.go | 28 +++++++++++++++++++ sessionctx/variable/sysvar.go | 5 +++- sessionctx/variable/tidb_vars.go | 1 + .../mockstore/unistore/cophandler/mpp_exec.go | 3 +- 9 files changed, 43 insertions(+), 10 deletions(-) diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index aa2c586bb46ae..b37376132ec5b 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -732,6 +732,9 @@ func testAnalyzeIncremental(tk *testkit.TestKit, t *testing.T, dom *domain.Domai tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) // Test analyze incremental with feedback. + // paging is not compatible with feedback. + tk.MustExec("set @@tidb_enable_paging = off") + tk.MustExec("insert into t values (3,3)") oriProbability := statistics.FeedbackProbability.Load() oriMinLogCount := handle.MinLogScanCount.Load() diff --git a/planner/core/explain.go b/planner/core/explain.go index 218450bd2bc5e..6d8512d5dd94a 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -466,12 +466,6 @@ func (p *PhysicalIndexLookUpReader) ExplainInfo() string { str.WriteString(strconv.FormatUint(p.PushedLimit.Count, 10)) str.WriteString(")") } - if p.Paging { - if p.PushedLimit != nil { - str.WriteString(", ") - } - str.WriteString("paging:true") - } return str.String() } diff --git a/planner/core/plan_cost_test.go b/planner/core/plan_cost_test.go index 6a37e4252977b..d8bf5833412ce 100644 --- a/planner/core/plan_cost_test.go +++ b/planner/core/plan_cost_test.go @@ -460,6 +460,7 @@ func TestNewCostInterfaceRandGen(t *testing.T) { tk.MustExec("use test") tk.MustExec(`create table t (a int primary key, b int, c int, d int, k int, key b(b), key cd(c, d), unique key(k))`) + tk.MustExec(`set @@tidb_enable_paging = off`) queries := []string{ `SELECT a FROM t WHERE a is null AND d in (5307, 15677, 57970)`, diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index f0435241adece..ed05bcce2c429 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -678,7 +678,7 @@ func TestCopPaging(t *testing.T) { for i := 0; i < 10; i++ { tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(testkit.Rows( "Limit 4.00 root offset:0, count:960", - "└─IndexLookUp 4.00 root paging:true", + "└─IndexLookUp 4.00 root ", " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", " └─Selection(Probe) 4.00 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", @@ -689,7 +689,7 @@ func TestCopPaging(t *testing.T) { for i := 0; i < 10; i++ { tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(testkit.Rows( "Limit 3.20 root offset:0, count:960", - "└─IndexLookUp 3.20 root paging:true", + "└─IndexLookUp 3.20 root ", " ├─Selection(Build) 819.20 cop[tikv] gt(mod(test.t.id, 2), 0), le(test.t.id, 1024)", " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", diff --git a/session/bootstrap.go b/session/bootstrap.go index 68e97c84d1dd2..d63181829993d 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -2043,6 +2043,8 @@ func doDMLWorks(s Session) { vVal = variable.AssertionFastStr case variable.TiDBEnableMutationChecker: vVal = variable.On + case variable.TiDBEnablePaging: + vVal = variable.BoolToOnOff(variable.DefTiDBEnablePaging) } value := fmt.Sprintf(`("%s", "%s")`, strings.ToLower(k), vVal) values = append(values, value) diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index d5438998cc506..6b56501d2b2eb 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -1025,3 +1025,31 @@ func TestUpgradeToVer85(t *testing.T) { require.NoError(t, r.Close()) mustExec(t, se, "delete from mysql.bind_info where default_db = 'test'") } + +func TestTiDBEnablePagingVariable(t *testing.T) { + store, dom := createStoreAndBootstrap(t) + se := createSessionAndSetID(t, store) + defer func() { require.NoError(t, store.Close()) }() + defer dom.Close() + + for _, sql := range []string{ + "select @@global.tidb_enable_paging", + "select @@session.tidb_enable_paging", + } { + r := mustExec(t, se, sql) + require.NotNil(t, r) + + req := r.NewChunk(nil) + err := r.Next(context.Background(), req) + require.NoError(t, err) + require.NotEqual(t, 0, req.NumRows()) + + rows := statistics.RowToDatums(req.GetRow(0), r.Fields()) + if variable.DefTiDBEnablePaging { + match(t, rows, "1") + } else { + match(t, rows, "0") + } + r.Close() + } +} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index ce3cdc66bfc7b..aa91b574dc517 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1564,7 +1564,10 @@ var defaultSysVars = []*SysVar{ s.RegardNULLAsPoint = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePaging, Value: Off, Type: TypeBool, Hidden: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePaging, Value: BoolToOnOff(DefTiDBEnablePaging), Type: TypeBool, Hidden: true, SetSession: func(s *SessionVars, val string) error { + s.EnablePaging = TiDBOptOn(val) + return nil + }, SetGlobal: func(s *SessionVars, val string) error { s.EnablePaging = TiDBOptOn(val) return nil }}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 03eca96b0e20c..529c31b6e560d 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -941,6 +941,7 @@ const ( DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1 DefTiDBEnableConcurrentDDL = true DefTiDBSimplifiedMetrics = false + DefTiDBEnablePaging = true ) // Process global variables. diff --git a/store/mockstore/unistore/cophandler/mpp_exec.go b/store/mockstore/unistore/cophandler/mpp_exec.go index 941e9996f5acc..8e079991c7daa 100644 --- a/store/mockstore/unistore/cophandler/mpp_exec.go +++ b/store/mockstore/unistore/cophandler/mpp_exec.go @@ -151,8 +151,9 @@ func (e *tableScanExec) Process(key, value []byte) error { e.rowCnt++ if e.chk.IsFull() { + lastProcessed := kv.Key(append([]byte{}, key...)) // make a copy to avoid data race select { - case e.result <- scanResult{chk: e.chk, lastProcessedKey: kv.Key(key), err: nil}: + case e.result <- scanResult{chk: e.chk, lastProcessedKey: lastProcessed, err: nil}: e.chk = chunk.NewChunkWithCapacity(e.fieldTypes, DefaultBatchSize) case <-e.done: return dbreader.ErrScanBreak From 095e3232013996af79a8053c9700c067d1bceb74 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 23 Jun 2022 09:50:37 +0800 Subject: [PATCH 15/21] docs: add small table caching proposal (#23673) ref pingcap/tidb#25293 --- docs/design/2021-03-30-small-table-caching.md | 174 ++++++++++++++++++ docs/design/imgs/cached-table-read.png | Bin 0 -> 39933 bytes docs/design/imgs/cached-table-write.png | Bin 0 -> 84320 bytes 3 files changed, 174 insertions(+) create mode 100644 docs/design/2021-03-30-small-table-caching.md create mode 100644 docs/design/imgs/cached-table-read.png create mode 100644 docs/design/imgs/cached-table-write.png diff --git a/docs/design/2021-03-30-small-table-caching.md b/docs/design/2021-03-30-small-table-caching.md new file mode 100644 index 0000000000000..2745b8e1026df --- /dev/null +++ b/docs/design/2021-03-30-small-table-caching.md @@ -0,0 +1,174 @@ +# Proposal: Caching a table in memory + +- Author(@tiancaiamao) +- Discussion PR: https://github.com/pingcap/tidb/pull/23673 +- Tracking Issue: https://github.com/pingcap/tidb/issues/25293 + +## Introduction + +For a small, frequently visited, and rarely changed table, caching the whole table in memory in the TiDB server can improve performance. + +## Motivation or Background + +For tables that are too small, they are located in just one region, that region would become a hotspot, and such hotspots would cause a performance bottleneck. By directly caching the small table data in the TiDB layer, such hotspot issues can be solved. + +We are caching tables for some scenarios already, in a way we're not realized. For example, our handling of global variables. To prevent row-at-a-time loading of each system variable from TiKV, we needed to implement the [sysvar cache](https://github.com/pingcap/tidb/pull/24359). This helps reduce `show variables like ..` latency, but the data set is small and the problem is very generic. It also doesn't help in cases where system variables read from `mysql.tidb` instead, which does not have a cache. The data of privilege related tables are also cached. + +It can also be used to improve the performance of join. The cached table could be used as the inner table, cutting down the network cost of loading data to TiDB. An example is the TPC-C test, there is an ITEM table. It stores the information of all the goods sold by the sales company, including the name and price of the goods. During the execution of the "order creation" transaction, the data in this table is used to determine the price of the order. Such a table is also a typical scenario that can be optimized. + +Last but not least, this feature is a voice from our customers. They meet the hotspot region problem, but they think it's too risky to enable follower read or coprocessor cache which may affect the whole cluster (according to their business model, they provide storage service to their customers, and the cluster is shared, they try to avoid a single change affecting the entire users). + +## Detailed Design + +If the update operation is not supported at all, the use case would be too limited. So even if we trade writing ability for reading performance, we should at least support updating. + +### Key algorithm + +There is a conceptual "read-write" lock to implement the caching operation. + +Before caching the data, the table needs to hold a "read lock", so that the data will not be modified afterward. This "read lock" is a record providing some meta-information. All modifications must check the lock meta-information first. If the record is in the read-locked state, modifications are forbidden. + +The "read" lock needs to maintain a lease for a while, and the lease should be renewed continuously. The underlying data will be safe as long as the read lock lease hold. + +When performing a modification, the meta-information should be checked first. If there is a "read" lock, the "read" lock needs to be upgraded to a "write intend" lock. This step prevents the read lock from renewing the lease. After the lease expires, the meta status is change to "write" lock, write operation can be performed afterward. And after the write operation finishes, the "write" lock should be cleared, so that the subsequent "read" lock can be added and the data can be cached again. + +The "write" lock also needs a TTL(time to live) to handle abnormal situations. For example, if the TiDB server crashes after the write lock succeeds, TTL gives us a hint of how to clean the orphan write lock. + +Put all things together, we have this interface: + +``` +type StateRemote interface { + Load() + LockForRead() + LockForWrite() + RenewLease() + WriteAndUnlock() +} +``` + +### Management of lock meta information + +What’s the granularity of the lock? One option is to implement the meta lock at the key-value level. The advantage is that the granularity is finer and the updating performance is better. If we cache the key-value record, modifying one key would not affect the cache of the other keys. The disadvantage is that this will invade the current transaction layer implementation, and make our transaction more complex. Considering the current implementation of the transaction is very complex already, it is not wise to do so. + +The other option is to implement the meta lock at the table level. Each table corresponds to one meta lock record. As long as any key of the table is modified, the table cache becomes invalid. Assuming that we only deal with small tables, and this table is updated infrequently, this option should be acceptable. + +For updating operation, the `StateRemote` interface needs to be implemented. The following part of this section describes the management of lock meta information, where it should be stored, and how it is used. + +Such a table could be used to store the meta-information: + +``` +CREATE TABLE mysql.table_cache_meta ( + tid int primary key, + lock_type enum('NONE','READ','INTEND','WRITE'), + lease uint64, + oldReadLease uint64, +); +``` + +This table contains the `tid`, `lock_type` and `lease` information. + +- `tid` records the ID of the table being cached +- `lock_type` decides whether the current state is read locked or write locked etc +- `lease` is used for the read locks lease renewal, and also for cleanup the write locks +- `oldReadLease` is necessary when the `lock_type` is 'INTEND', it stores the old read lease + +LockForRead() corresponds to this operation: + +``` +update table_cache_meta set lock ='READ', lease = %? where tid = %? and lock !='WRITE' +``` + +RenewLease() corresponds to this operation: + +``` +update table_cache_meta set lease = %? where tid = %? and lock ='READ' +``` + +LockForWrite() corresponds to this operation: + +``` +update table_cache_meta set lock ='INTEND' and oldLease = %? and lease = %?' where tid = %? +sleep() // wait read lease gone +update table_cache_meta set lock ='WRITE' and lease = %?' where tid = %? +``` + +### Caching and writing process + +The way to store the in-memory cached of the table data is the same with the [temporary table](2021-04-20-temporary-table.md). +In short, the KV data of the table is loaded from TiKV, and is stored in a MemBuffer. To read that data, an extra UnionScan executor is used. + + +A `CachedTable` struct will be introduced. It inherits and overloads the `Table` interface. `loadSchema` will periodically load the table information on schema change. For a cached table, the `CachedTable` will be reconstructed. + +`CachedTable` implements the `Table` interface, and overloads the `AddRecord` method. When `AddRecord` is called, the lock meta information needs to be checked first. This operation needs to acquire the "write" lock, that is to say, make the `lock_type` field of the meta table to 'WRITE'. After confirming that the lease time of the previous 'READ' lock has expired, the operation can be executed. + +The read operation flow diagram: + +![](./imgs/cached-table-read.png) + + +The write operation flow diagram: + +![](./imgs/cached-table-write.png) + + +### How to enable this feature + +The syntax of OceanBase's replicated table is to add the `DUPLICATE_SCOPE` option after the CREATE TABLE statement. + +I propose to use "ALTER TABLE t [CACHE|NOCACHE]" for the syntax. This is a switch that can be turned on or off. A user can decide to use it by themself. The write performance will become very bad after caching. + +"ALTER TABLE t CACHE" is a DDL operation. Assuming that all the TiDB instances know that a table is not cached, there is no correctness problem; Assuming that all tidb instances make a consensus that a table is cached, the correctness is guaranteed by the locking algorithm mentioned before. However, during the DDL operation, if some TiDB instances think that the cache is enabled, while some other instances think that the cache is not enabled, there would be a correctness issue. + +To address that problem, an intermediate `Switching` state is introduced, the schema change process is similar to the LOCK TABLE statement's implementation: + +> Disabled => Switching => Enabled + +- In the Disabled state, all TiDB instances know that there is no cache for the table, it can be read and written normally. +- In the Enabled state, all TiDB instances know that the table is cached, and the "lock" meta information should be checked before reading and writing. +- In the switching state, all read operation goes to the original table; and write operation needs to check the "write lock" first. + +The cluster may have Disabled and Switching, or Switching and Enabled on different TiDB instances at the same time, but Disabled and Enabled will not coexist at the same time. + +### Compatibility + +The entry of the feature is the 'ALTER TABLE' statement. If the user does not explicitly call it, there will be no impact. + +Metadata management will introduce persistent data. The old version TiDB does not access or use the meta-data, so there should not be any problem when upgrading or downgrading. + +It is unsafe to mix the old and new versions of TiDB when using cached tables. Rolling updates are safe, but there is a risk that if an old version of a TiDB server re-joins after a table is in cached 'Enabled' state it could modify the contents in an unsafe manner. Currently _downgrade_ is not technically supported by TiDB, but there is no technical mechanism which prevents an older versioned binary joining a newer versioned cluster. We need to solve this issue to perform meta-data upgrades, but it is currently blocked in the requirements phase, because we have not decided from which versions upgrade/downgrade will be supported. + +## Investigation & Alternatives + +### OceanBase's replicated table + +> "In order to cope with the high and low frequency of application access and the low-frequency access to small tables that can always access the latest data, and at the same time to ensure data consistency, currently we can only choose the solution of strong consistent read access to Leader data. However, due to the high access frequency, Leader is easy to become a performance bottleneck. In order to solve the problem of "small table broadcast" demand scenario, OceanBase 2.x version combines its own architecture to provide a copy table function, copying the copy of the related small table to all the OBServers of the tenant of the table. The table says to replicate the table, these copies are called replicas. The update transaction of the replicated table guarantees that the data is synchronized to all full-function replicas and replicas when committing, ensuring that the update transaction can be read on any OBServer of the tenant after the successful commit of the update transaction. Transaction modified data." + +Refer to https://zhuanlan.zhihu.com/p/78402011 + +The biggest challenge of copying data to multiple machines is that the modification of data must take effect on all machines in real time, otherwise some machines read the new data and the other part reads the old data. This must be a function that does not meet the needs of users. OceanBase uses a special broadcast protocol to ensure the copy consistency of the replicated table. When the replicated table is modified, all the copies will be modified at the same time. Moreover, the modification operation will only take effect when the copies on all machines are modified successfully. + +In a distributed system, another thorny problem will arise again. If a machine fails during the process of modifying the replicated table, and the replica of the replicated table on the failed machine cannot be modified, then the replicated table can no longer be modified? If this problem is not solved, the replication table will drag down the user's operation in the event of a machine failure. Ocean Base uses a lease mechanism to solve this problem. The valid copy of the replicated table on each machine will get a lease. When the replicated table is modified, the modification must be synchronized to all leased replicas. In the event of a machine failure, the lease of the replicated table copy on the failed machine becomes invalid, and the invalidated copy will not be synchronized with new modifications, so subsequent modification operations of the replicated table will not be blocked. The invalid copy will also refuse the read operation to ensure that the old data will not be read. When the invalid copy is restored, the missing data can be chased up, and the lease will be re-granted after it catches up to the latest state." +Reference https://developer.aliyun.com/article/750271 + +They are optimized for join's small table broadcast. The agreement was successfully made from the majority to all copies, and write performance was sacrificed in exchange for read performance and consistency. At the implementation level, if you look at it from another perspective, OB is a table-level synchronization, which is equivalent to adding a learner role to the raft layer as a copy table. + +### Reference Tables of SingleStore (formerly MemSQL) + +> "Reference tables are relatively small tables that do not need to be distributed and are present on every node in the cluster. Reference tables are implemented via primary-secondary replication to every node in the cluster from the master aggregator." + +Reference https://docs.singlestore.com/v7.3/key-concepts-and-features/physical-schema-design/other-schema-concepts/reference-tables/ + +### Collocated tables of YugaByteDB + +They put related tables on the same node, which facilitates operations such as joins to reduce network-level overhead. These tables can be cut to multiple tablets, but Colocated tables are deliberately not cut. It is not a concept of caching, but it is also a scenario where the broadcast of join small tables can be optimized. + +> "The assumptions behind tables that are collocated is that their data need not be automatically shared and distributed across nodes" + +Reference https://docs.yugabyte.com/latest/architecture/docdb-sharding/colocated-tables/ +https://github.com/yugabyte/yugabyte-db/blob/master/architecture/design/ysql-colocated-tables.md + +### Oracle also has a similar feature + +https://logicalread.com/oracle-11g-caching-table-in-memory-mc02/#.YFvijK_7QuU + diff --git a/docs/design/imgs/cached-table-read.png b/docs/design/imgs/cached-table-read.png new file mode 100644 index 0000000000000000000000000000000000000000..d78309d71510ddcb1b484103458a51d1bc7ff6c9 GIT binary patch literal 39933 zcmd?Rby$?`*Dg#6gN}qK4ALz~jHEP34oC4N*W*?l7m5mfOIO| z(#^gG<$3(Q``vqg`#8S8zV9zR?&IdVuUOYw=UV5v=DDW2()qKO&*I?VoJT0jYvJJF zLve8M(jkQ4$>9smWAG2IgO<__oV<40IUF2D9E7~=Z5PAE>htAy?j-LVOOCfFsYRIc&5<*OsekP%jk{#S*pQI_!JfB!s`;lzXB_Ywa67Ncea4yNui?mvHqeVGxe zkN+Q|$t)4XA#;+c+5Q;{d@76@4z|<4V@IF+{n0)TWT`y`p;QE6lYPPsv`EhQb9^!q(bYpZrpw@>tgMBc$`x z6!2@*8qzD3*q>Ju84<-tHfr^bELmDEFpVXPI{esOR-CmpS)bu++1{Eam4_X)A)hyz zt0!*xUmYCO6)K~>Mp3>!I<`6Nb&F!$^ky9AwT6B{twP+Us&hF){TD zYR7w@ZSXxalPYaJICC|%m0E~=cI=n{mCpWPG>I0{;M8Xj3}+214E1t9iwVOqPwp?CU)E6h+#^#tJ0 zF80~4{SSvszXaDt=X)#>@w==}6uQ{O78ldrF1+cyeRFQs z#V9>5`YJ=%?z-)0{_^TGgb^J?uItmId|Dq@`nQ!fBMccu>zCsto+UXtsRcq;r`v`y z-da8nrWMtnZWR^ZsU8Y=bWggwmSJNBRa)Bns&+T$;oIy;h@50k)s=2Rq{ZY|+j*Ll zh)h+!h^;NAuI?h$EOi4e8WEeA=M>VMd{wItR&9R`R~SY8`l-RiT7SU4iLWg@2W5gH zD6%CX7dbkMsmg@bsEjz9U?R((Ffy;B;81G*@<(339-Sp4^nAffzIq21mCkcKQLFLh z5}~%uLhsBrMVR6`3RrySz}o1!hOSKK#q_daiP*LFxl2_--QvL7t17sr2s%H>uooAt`F-5?#&*IBfw))BUdf%wPr4`ASCt&JENqvK;b=V69L zFaCw1233lNLlF@Y6=J_?zUHX+PC6GbVS{7Iv6!?7_s#D{;S5KsvGo+;FAO@SD~$>> zRF`Y^>y3MP=c^@43@<_C-haBzfcJ|`*&iNPJGnUEw@METH04~N?jid)2HLRVb!^0M zT&EYRDdYx|+NpPL!T6VLK6@O*1Kpcda?`Y3j!W<$HG^MFK|-Y8l=O>PuR90mw6sC} z#`;{K-`uW3xxP<3XFZ@FJ!g@(NzO88`ZXlhWU-%u*~8IZvO)$npRPy^k=LlxLta5+ zYAkgg^OVF`PE#<%Ap?(?v%c&R3o=6|?k-WIR-MgCWu!$qyhLU$4mH3nduQ;FA5zaV zP!hU1l=aWNVR+83CZPUcAC~g@+%bK^ zu5D(885uh2&~-MXj6jPC8g2N)#5{Yz3zr!>dURKd+NB4R=6uwkfjFBVOG2uG(ct0d zv*{L3o90v$s}tF$!1#IW+qHf+!+(L0%S}ATb*7SfQ2AwZa819;)ET7mltx;L`ZGT0 z*66F%+ElaYy#Cs^n{|(ip_tAqvdwagm+{mga)JiGUQ?p#LnTbvnq5&$SFJ8Dak#-Q zv&~vP<3QExh>;VaPr??XCI^5@>Q6Q)i*f zMAlz8HE_?ia;}}4dF{|dH{%i{iajkc#k@VqMJo6e2i)tZ(B0PdNQQiX#$&0cUmni~ zUR+Z0*hcgL`;(RG+J%baBc&Qf=GwrD-2S&6pGWb|%K%?Z71W5_e^Zt}<(|{D>SW_( zv?xNxoB#ceSiU~Z#9As@cR#v&?!g#UYpSo&y97O~|FNM=6 zm{@AF}S z4{8*VpR_0X!9)O&Vf4haRG56t%&#l&N#W&CjN{Woc3n$8g(7@LVR9*VF4~+9iLNg~ zk=t;AQ_odqN%ode19&*$i5)+|SpqUbB3?pNy$>rH-rrbVpAEo_1nM8Yf5B*PeC*p1 zGLXOV`jcn?Bp#aup8Fd{laR^4XqEApOhRxHQv_n|#h|#zZUHJ>f_av@teL%X1>LjA zz>6HwvT0lZUfqeiPK=iekW~g(vLm}Jc;fp(RmdNHJ&fQ+BQ3x;4xHHyWX5P8OB_6> z3*fpFP?=ssF{vc_^j22}@88%xBN(8uLR1&s-w)*x z0BJ?AJISVr;aR!ng;1EwGO7`{sAwe=B8(zf!L(T)#af(0K4BzAg^7YWPHU+LFx6>C zloOubu3O-Cy~;oskmZEnU^MAC+I?jbAn=B;EUT_qRxl4lml?VqcM$bg3WI~&0p1wb z!9b|~-aiCbWkiOW-rpO&3TFMke$gce>~zHi#xO}R`*y?jIXgju{tepmU(t_cVV5yV zNJx(Y$XL$N_%n_R;iBpnHpty6z?{@jwh4--XN3u?i0Vc{fcwtFh4g?8$)V7X`b>tt z@<@y7Z{yHX#6c?S+?XTz`{65AZ@4@nvhqLDHh!>C%@PcsO1j)xa5s#YLJrEIK9g!IZuXs*u^2Ri<*?3h^Iy8<3;>TrgnPgzOV)Y6-(0>jK zBIlHJ{T$vS<~pd{@3ZoTvmz3y35hHy{==jtSn8mD=EN-eWsu=3_Krv6e&)2DgJMs| ziKJE;#6@f*nC2ndRa=7-L?xv*&wftDu6S;~6rnU=U32fT)@1l+tyjQCjt^p|Qp2}4 z>Zhy6a3EuDZn;lwPV3S+5zwcjz55_v;;@+3Wj1~7E5qUJCWmd!?q?B0*E!U>eCeky zSoDqGX)_~^4;e1XM->m4b@{W^@l<*d%ibN*wiE;N^CeVz&I6`v{Sjv+nz0P({|FcKU?+?6JC@b@>}%$i55pC6%J$K17TtRBrREiNssFbFZD zWwL;Y#H&ju+j`?*I&rFUUa*VxKj*TT_mL?g9%i&R&ht7R(cW5|i#6fiMA_j4-8UuFEnfigQ1K{#t1!{3JyFiLgawaG#NxxOu^T~ex37Pmiv9K{PH-+M$N*x z$BtkVX76iJoUYdftaG@X^qZ&(7*+5Z>ePS9OoA61!?loBr2LZi_IPIA&c7HQS{>%O*+{8gz*yCA@>VXZz&y_p5 z;X6t=$W=abdFUSpDdPdYx>tUS>nHd(*a*jw;kOS8V*%wZwMzU59q zXkcFo!Wnt^$YV1M+_N%_3t-0YjBqz<4Z&2aCz-he;61nkHpu>}XGM-qPEGv< z5B-j_##hx~hL^%6nfYbp=_$jfnAfz)cNrqjp_tCxV>9rUDFmaKJnPnV1-ni%dExgr zO)XSod7}duJnvKT8FE`&Th|abs50BfeQ$}CTU(8!k$*b9w>>6l+L7WJ{Yc)KP&MM! zLxkGtZrlcYoD-s*T119k>wLO0%5#pAJM6t)S(XB9k%VY1j6!GI-}6wt>giDF@X*-N zmRR1ay%&toO}p0@PmN@gLRkeS+{^IFjcukiUSOw@B~;?^?%EW;WiNNdRE(ZUa}+1P z=Yfk(g?(mgs_!e6tJEz3o-hIzCTp@yYOC46s|K4cmS`pQ>yLN4_ zq~mO-F=2gR(3e~R$WWPWz6z}OosKryh2#Rbb#Cx%1WWWKw=c3kRlrB2_F?E3!$M_V z$2^Nj$3sm=v8jt zmu-exnb&528cU9Cyi*OKAKAYCJr#!dupbQX`PGmPAt+P>F7xTpoW~)U!QOzvU2q%m zO+gm__{EaKm8Tq|f{&IP6VG={Xg2d83jNdj7tsSx46I0fOCOp^5t7i}cMO%QSex21 z?Z>3GyxX`wljN{GH$Q(_k7(ZPlIRV2!z+E)F`e&vI(bj883Op?c}7DOSn-?H@v-5} zGEk#=bhxBN%8*eI= z3)%+mP~{!Vj~Nc%Ey#w*Daa?CqCy!J;EkME`BOU>t}^e6y%4T$q|lJ~*7g0n(8^S- zk<`(}< z1H^A~Wx!EL4ES(I?p)$i1Y8D!gw0IAEG>4_1Q=!wT~|6#q*PU5-r=@CwNB(pA?VYq zzzEjv_S&dcx|C0U$I9T6CIpYnLJVy4948|V0~q3)8Bc15myG<8ZE;CfZK*!*w&a-W z?#5@T)TQ%4G!69uphHE(*G3;)D>rokj8sMD_v5Hd6pxAOcz zY#KhcwvBXdjn}>xz&R=6W3WBK!8hU*J1I}eP;K!|u+~j(gRGGf042TMtTJ+Nn9O(; z`3cui$nE73P_M-s_<;1|HmJNkI5hP7d?ljfk1`G6b@}ZIp+Z zfjpdJ>iv{rEm^p<|4&!fy$$UQqgo^Sjb-N+aw`3CnSNB5pw9@6NOQ}f=f*3A zRP{=aK|dUVc@BUYIzl3-0SuE*biag-zIx9lN!qKXHCg=Lmr#bWC-~7Q^O&=+%6;A0Fs4-Q3m z?Is@!b~@(--(%9EDnEYAeEph*-#Fx)-$;kl+LIw_^QzSX=bCkP6jPyU{l^z%gezK# zb!1mSvRe*g)FkpTJKI32tLxgauynXDhIv}rNNt{U&E&_Y=MK$n@L8E>Ac*@*_2b(K zBz}1O{muA(QzS=A6sPtcH=?WKV`Zi9N=cp)gNI-o@`~^|vXpDrfuAw`_2Y9_Z!deI zh%L!HwWxhYacOBY@CUbYloo_%;$4>ZmZbWNs3!(UJ_bLs9V)e!t@Rx$dV8>Ba?&DpG{{aiwMs9HP5ylbLG|}n4@Sb6@m=kaE~*Wu@H zua(B;>zq~*5q=FF(1g`Zf63JBOZ}3W!{SXGO{88D!2&Dsw6#?A>zjO)0r_JxY)YcsVO0HYn65YrSMOpxm{MiufDGxB3R@9O4 z(eRB{QOn+U1Gf03XXUO_>iN>Ts%YJZosD2*zA8V-F-R&bskEQ?t3Vi4|MeZY0Kq8m zr%%{q(nx?U({(mqd}M+~_J2Uu8-E;zHhg+cVLE~F_h{!VVQm_o$QNu#bAw?p#%8j? zIB{{eFSgbMtOomrGmJQ-YXlTx<2%Zni#8G2`I2St{vD6vD5(b}jKoiy$G0#dzb z_)eVA13-+{3E}uuVnO}utGb<(g!=>no>#(Otd z^ZBD7p2V}ZuDswJVz1AotA=3$zC1D$zz9$bItQ4+j&8_ruu*ZHWee5ymyPH0o9oUN z*&sPDa8&?_K`E!+y|l!a=4hR1hG~5r5Y%SlZ{A(}l;zb#_EUfJA2(U;Brd>^%JXm1 z=|DBun;JbEul0}+9LYEKiU{~@f|pwz9X**pk=kce{iuw-RYmB?>n9#tiwXs7THsNh z9MK~d#o#MeC}HdP`oy`!t(I1F9Bv}@?4x*{BD1E4p^}tU!R<`QocRYJu@K~Njg#cN zx$koP zeAjBB_q~6%r@}WO6cb3xaebzN4@qc9zAFjbqI_cA_0bJlYm|C#DQ9d8-GYgUg6|F3 ze9P_ouwqKOg9R;#WYZkQ%0Mo^Q!M~&gVrD-yNG+Q-zfTgQEYY`JWly_3_B{l#?o60 z1JYk!yuX85sRoQUxU{#YZ|m1PV!4X#I~Ju&&UfR^Pfb6+ZH9;tHDBy6P%tNb!Sk(R zYtV8kE79gG;jkHJm}F-!hvkUfS33Lo-0T;lbA1h?8+l&5TX|&3)$mjXr=D91KcC-# z6^HQXm0lcG)3Oq-KWaU&kNTb&YdEL`zr1^g3Hq|L#Zd=LW`pHFGHx4QA4PMRW9M5x z`}GGB4(21!9}SipoBL{s!e?3>HQ<44e%(dfsQP@P6Bc~2zcBsB123u&&f&Z*ZK7Y7jdGRt232P;)57TH4_9 zC83`-h>;e{ww*tF$w5^!;6LgE&S*z5?=nv+2biAaXl4A1f^LiwG`%jbKyVd$(QIF;?*KtLNCzRLf`blsm-I=ihZju)b^vd|8t# z;Vy;g93DIKUapP`|G}SZxj3pR5bKn=<9z;;*EF@Mr)fp}#l()3fR*gSW zBH=}ZTx<`&1kbG#WUm^yN%qx<=VLS%5%<4;de{MvquL%&ea%HISCGy9$bI4~VXSV1 zNsHp^SRm(kc_BlBAOT=63wk%*tvG<+?A9RFoCKFI5qdN@hY4oekW(*9y>M zOUTipTOHaFi)DTbYggzFY=_GWG;D;XODy{cQ!ly2qRxifGz%9AyuCYj?&IY#m;zst_vL7+@TRdnNKJAm&=8sxZQigiNBCr}M`j4X^l+)ON93mlmWg=*Ok z0I@%{h`bnF-#rMvGLI+^8Jo7RAi7{95I#+YmWPXvKrrKut`HJDY~@Cuku1j#l)Q+F zQ9+aIyGB8c2x5^T3kNSiF~0~~%L6c1zYS_9n}#h;@w2ph*Li}8Z1bH()xW3xF4kaN z%c(QgF#aPwG0pbll5o6~N9F9fPE5eEb*EAyrQ5q)(LT?MivS|Xr(#U@ELV~{*Be3~ zJB1@E;>ZZc_@{4HVXK#yPQz8tvV&%2FK8$ogw=(i`qZ(@px{c1vG z$B1uPMHcgS`y8)LwX~3|HE0-|t^HZ=iLxw+tyPFl%$S!2c~f%bQ;^qaz4&u_K)8?= z(b=V;zOcj*$;QVqC2zPe8*}$~R*~WwhV1;;oX3brVAlt;-(D*i^_$gt)wiCVO^n>^ zFEmwcqDHTZ(=C<>lU^27=B?h)iWjjpM1GO=|B)U*|6*gQ<P)wIYZhgN1nAMMamm{A=I_T-?kv%9G z8?c4vZ7;Q;5t(0eAKxs6_uD$9@SDEizW+gn{bn2^D9*@6ms-sx<}~oXTH?_ul*bmc zvc_&raXbg#XT8!*Nbmy)pguK3>uRf~HDNl%=T)nCtmb>5TJMgWn<%y^D-uFpT_3yqMEtxG{sYXH~!R_NE(M zYoeHntd%^BT`l2avDY^Rs!J%@p>H|?$OiNoHMBW}BXKuC1ytPrH(*H>#l!@7@xXPM zFt<7KKgLtLgvRArn{aS53QwZdO*a}+nADqUr~b|pw79}>P*}XZ!a>oc34#TvNU3+p z4t#k@WnxcF^SA)Taw(PxM zf_PT0`8iz*d>o?dU!T`ac~PLj8-80Z*#T77&9K2+Hd(luW852F#Uv`4a~65u5V3ms`0zq?MMMXnO7H-FFb-^1*;o z)0kw`!z-J+Mht61Lru@R!;hE?j|J!y z_^Vmxc4tz(o_qwkqUdd(d;vA#VO3F)Lv8k0oINOCtvHm4>8DB=YR4AM zrnji^R!m;tyd<~dKn@)IzURAWGgzzbpy9Ty8KH_bNvSyvwxEX=AroPSUiK!|#C zgQ5G468PVg@GYgoZT+6jf&iw+ui|qUy5BU% zHp$eDNmGr8;WcoLs93jMR!p4Y$wVOmkgC6I=fE+jpIk|xxFQisCtlj2ZQDAtHDb-9 zZM9YJ*K!B)7<45Qu$=?IUQ zWrkD#g;#ABQk$IeW{6H8NMi+QkUiXie4C!Is1{?$YZ?^t#r&)0W8z2+FE?hWlxa)u zec;QxPW`|4Rg7`uRc6xNTO;;yUXHJy_e-!fjb2Uj+j*N`Bj(c8iat-{E1N@2J735Y zO~3-3m|38?26slcwUo5Zfch-=Zh6ZSsF@kd!R{F<{nb8(-A1M55VW^miD2d2z!MQX zil7=>5Fp#SAh6GgKn|Q+K|XirF4EaD;d^=3$YH{R7x~=jYD4sE zSs0hVkKk8W0+$zhiUsvA5+*)4;-4#9rk<02qc?iuK@-^!Tb0}PZY0Hb=)x3A0@80A zT=r`>o~;w2rNNM?E$4n#1D-4K850-ewj8MJpfAkGFv{0CAsLa(NaTlD`~l|}Cx=3M z*O|fLdY(;_*Q*9LER1GjzSR{Kad1?u0yWYYap9iTDb;cFDyT%fkIytJ0O8}oXBxy* zRXNXv;P5+I3lSNOm$R770ZeMo@RecI6aZnx%;ciCHwa2ccfvYm^6jLX519`Tys>U| z4dqwS!SLJiQF|0*=$`C2k}QCE!hTG4mF$24eidX+JAw~%#rNsD`onu}1Uc|vxv+8j z-%Rx>H!`e`r)SnKQR*di+>fh7&uN<~`LUwD(^VT%wU#*67<9as&s;J5@~gf>TWsP5 zf|q;BSboUzG$j4zhZ0qd*C(5Jh|bd=Bq-W_@~3K+U0P~8AJ|L5@!{te8q@8cp_U(w zB+b6Px=KOnP3Un_Jrg}WmER0c8^`KKF!jCXjcgTuBoV!#9yj+G>`;>=QsuioyPCp{pGfnJyz-#@>iah+~NwrAz%h0%D;$gF1!IsZYo)G=1~M|<|zZm zbI1mpmgh<2rqw-D=Y7msV-xLKh5Cl4)jkoN#f1<6>)<7T^vHr!#lNh;_#w2e8&uw@ z!hf=*dTH>+ibLRY?45}n(vdAwNBu^V455|N3HQ22^jwL2e(&u)*Gu4xn9S;Klm6kD zyam9Z=326hQlQS{-b`6=qmKAry~0m-KqX~0en_(=TUQE;6m0K!C4$gGcoQ>9g= zJB_0dKQB?12dB*bJeHjL`cjJ;;35=jXTIN?nwIr5JX~AIXGk#lSPbY|K`gNfW4dI5-C&s?zahNm zh4S4%7UAHEUtfNX?uvLt-X}`g`xZLslb1K<`D=GZI^q6n!BZ3ams3a8ZOb2Rb(Ym0 ze$n(7?{x&f!ha#R79Y9-82p@|f7FDOr!7DO)NdM@;zDFSwI4$Ly+YU;*+G!}$|d&vw~{ZqrhvWUu5 z;C2}7W{CX#5b*yPGQgkv$=m_inf@hqbzhk`SQ*5fxeNi~3&!0J|DDJPs8B#VLQW(| zayZJb1Tp>A4*e!|e=CCGkjg+6tQr9<_+LTLDR*;nJMRHw&oe`ppK?D~8e|*?t1|e% z(OAV}l~S#lNhEUK*{pWwWzw)*VBTMPoHtG)DQ3*c_CAQS%&dU6i`@pg`Kzcm=ybbzZ<8jzu$wj<7eD1yCT5 z)d?_~0);nlQzC9li>qYC)(MjDJ`a@OVRln-jKr3u8U8UEKNikVJ(#fVD7S%u0sqs* z{_$GQ>wqeZ3_Rfx=Gelh!l@J{t(n)Zl2<%Sm8r6Fq6U9_d~&%_tzeS02vLlt6!{s%(?T!8+}s@Z`(#t(KDUT& zrDLvzha4Ij{-`^vX|vDB$Pp1i9UMw4s->%^Hw6lbkiky5m3iCYonE5=hne>CKfX0Y z4U}3ZCfR+VF0}09#j-dSiQ-QeAwT$k#SP%{m_I0MS(q6a8R`1=jl9Qt``$z8m<&&$>DpsbRPpWOVQYk8V7%0UGg=JFesFOBxP`>DKV?Z|_mbW8BRNN4L_32rtLH zSRFSyey8=y`tiOZ<>FodDZ{7|>!Odl5at7zzKqpvVU_3-X+Cyna&q#qgy+Fdi@>vj z&PPDtNP-t_b8+QDc(2DkjVCMfF*kE7|EKWhXPqk9p=vTRkw9&9mw5I!P5%Z<(|>ec zi8*tQ^5XJvSy>s8^D2Yq&IBgewV}nZ@U;MiuFdy1cl^b-6&ag}{dzMuEyc~YFiE!a zeU$+ubeCHhVRqlHJF!35-&(rU27jkhl*Py@_7FL!H84==WVxCw`O{eSWyvD|lXVUY zI&C4JhGE@@Nem+O(wgC53E^{hpsj>(vg~!zw>NHc-wB)PWFN*Iz^%tv46?~)x z_wHk5t@{Sk+)CL|7&I*wf}a>{cyqP zxE+*Ejz_~cb}0!l<9c5LMTqeN?i6s~B9)mbE&(bVs~iQ+rj1Ec1tysA7db!dBhrF$ zWbpf&oabB0W{qbHA+OP`stdRql%RbP<02Rbok#4_(- z(NY^01Bl}n1E4DSQf7Ds8Cdl2v#|Nsfzg`EfsG^_lY|q$eHr-el3+4Y(1Hf@ahF8-}&XgidZs~PY@2?Zw(#gBTf_(Ky!pXZ@@rzkUv-<87}}}nfOd07sVgafDd8x)@^e6LdCNl81`;xbBz z*W$Wd{L^4(0;Gsxn^zeh9xB23W1du=`-7x`0I=!;x`j8=Gj)JH+i955S#a2atC1Ci z>c~gg-DFREfTYEs5%1cLG8R6Yv^_&M&ajmoUA1?W;sflu*Y~Rt(&>7H*Xu&aBv_46 z$o1(qpn+fm6lWw5D>d)@`k6fU^0_sjDrAGnX}J+EaG3ANq+!*=emI;dI@s!3uCo@* z3K3S7#3lOdj{G1r|JKM4lVOs{NHQ)mP2}v{T!rI&Pi~>z&oASP8UDzTN+&=Y>Iim$ z2Td7>NR#Q-_y?$Z$fw3|7SrGSGc#5jHq{c_!XdqjXqvLO?~vS9c6KfUhkJ2e2rJhz z-<``&^&|n9iLvuGn2h$X?((l#j2to3TK6%6wf`OXe;%!doF-PRww~!aGfMBxF_5f+ zG0v#_bdeH`WjrwH_USWuhSj>$9jSbg92&f>${QOSfhytjKu&glryY>c5v=>l|LB7= z0%W@VC(4XsLh$X^49IsiUD(s1)A65cXh7KvlPW#TY0B^#K^@M0&BbBZS>t&6rL;66 zmO1_>kf4o{qhe_%0`DXMP?E{?f^g9>*JznHd^%zG~`4Wf4O0P z6_ki5puL+NTZ}dRLYDJo=0q6P2rH!Kw|GgnQb3eS#+h2!+ETF)$pw(`w93FZ2l_le zus$F2g+L=7hNXD1&k>&_0fGiF`UGZ{y{X9=X8Yp6MauAoH_9!Z57JB(omxDZe?7^L z@;H((E&OcV!P$2-<(s&2sCf-}zvS_qiLP$kgTWF@KU3)mxmucQ59Ar4PpKts8^f6R zEjk&$>3DxIk!OOwfB)Wof2oYhz%xVOydv`r<_9RBF+4AZ96>gFy4%0DuVo=+L?!w+ABb9Ywt@MN=47V0@l`FaDG9uL@KruIsQJUEo zZnF0#dp!i?iL?j;qzlNPmtu~WzquoeZ7GRL382QC++ z4<5tJJpLzD7DmlW0PAwyOd{~P4REGbw7OU%Bk*&w{h0|jPNEs;gZQJZhhXXyaQ@`Q zLB$t;rzjvn_~UE6Wh#ODo@?f){#P>#;7taYsIr|F^}p}o5OiU&_C2Y&^M5})E&O1o zBrJK)|9K>17#A=FbpXn5`hr9#iE&X&?RTYx9~^C?e)Ak0bU9q;1Ckt&R`w9{Ha~#K zMG5Af!}|x=D^uW_!MdW<_9C#}8!^Z>_t>+S%D`!=5GKN>BrgT2J|OPnt5 zpob?e_pJ1B}q`c0afw8P7pSfXe@ziv62l)~L!5JZ4;1Pu_S|JW4R_HnQRY!A@ifAsH*tCYZ4 z9N1(S-LMu?oz(r)0(%aZL(YHq1?|qE7_yuokP%Y}pOvk%1n1m0$Fjo?_9+1}Ciq)d z|1T&Rm;8U8Yl2-o0a^S$iSi-CEc{yXrJpa(_}IQU_;n-P;M(`pipS)?x9!iHI6jla z?i!PEa)hKczc!P_rA5c-R$Ma2WQnGJ2EN*UDPo^5xjpQSPb#?)FjKYBHF8wGBlCJAD=(EwJ2n=Jy9i4grzuS>M(&ggYR^*Ny9bggjg4y)0^Z_f`1VenrR9UU#Uc zqg{4mp*)Sc&cok*GgW{Wa|vhn#r6Z7xPW+Df}>y7wjY0?TKJ@$L1~%&^4{T0^7i1* zk&&Wyie2?r=d<5|<|tF*%>wabNsqlWeEHELkyy9v&|ee|rg+sA!(PiHAGe#fK(#p6 zUD*8O63YyNYvpNv5xFYvHL7$BG?GS~6~k65;x~xiFoQZfAA{>s$*+~CZ@Dh3pY&q* zWPm?Fgok9)p_u6NdUVhDWSh5^-;!_tu1zX24HGq99p_T9dl5>oRki9{wc9GdzOh1I zWO1v|p-bupiy7A8a)R>CPV!Au{mQ{Jn!|1ZvjhLY;e5L5KQQN}(zc%3?%x>yPVRCT z>J_y#cI^3NZFF>$0&U|+@`+JqKOS9kClXVzn1G~$GQ z46^_~A#k+KQN8mcNXctjAhlwu<7lxXMZMVWm}V%sWr}ybmS$iX7fki&&r~~FP zP3DKbgvQEAZ(Xz9mmM!5@85iD)THV>#v>YjHPo5KmCod@en}CQq@pzFCkxXGLxN3ZWrljrbPUas6WVI`X^unW zqlN`)zZ$`K=WVRQ65a@*{zA2qi|Jb&hox~d4W+j})J z!QYvoSMTLTnYP3C68z+amTb_!DgB7Ze`#eIft^OH)v zJOt%mZ&S=jx~`{3n($;Q`k|tJ+@hQKiu6iVAN|U5ic?CX^M&;AE=ACg%-*|c!vxJU zzml>O3_tkM#No9+CBC9hp>ee*fxt)4wt6#z%FtzkZ;=f#;WJtAm0s_Dgx0e7%f4Z? zUh$spEVamm{Ms5&mZ%oU_fvI<69yd%-7^jjeIxYmQe5YA?E#&CMj^<nJv5!b;{V8wwsLR<^3dE?7`I%*yDPt1fm#%M}a|xp78t=Br|Q^9*Q) z3sQwFV}|Wpcw{{)x!>G}?o~YZXh5Imw;h$xDKtUAAEsXmp%%!3`yokx1R2~@j96R; ziN+!|Ve%tQMZiV#ZFK0#^nxX~CwMG_(or4+a;|10{%fXzWIx=$6k6KN9q_EX5 z#s23=Z@{?1?bqE<^=(xdQd}seO-|lHSr%yRUU%wp;znfZ!Ac-!;`OTKP0$nnD;8Mi zVFo#Rq9I^ghn<4HGg0<+UPT#EvzRxO^l`)3P#REe^7?$c--;w0-ZQsuu(>CF&dWBY zCGO+M)5K|Y7nftd@T4cqFN0CFWw8q=e_7K8L$eGgO3Shcc-(zKLP@KN!rH6syy%uR zSwDGmTq`Ed5~a+$NDFK0=pozM>e)3Gw=bVRXj2asnzmF^u;1Lt?30djB$B=@e^jGI z1^WF_!$+W{PFy5F#`k+hbi*UP zxW%F!%Cfm1hoWPS`o|c6*3<_MrI#p&4q(dQ)7CS!4b}m{vUv4(eU^x9iEThHcCiP*f#I5Xp#vZihP1uVzEtEfF z)#R0fYTkR5AO|{|_*^aVa}3cXn&P2Do>E%ng(9M#TXar(ADT^p7wIsX&&kHVV#$p&Z`Y&2uVrNC%;Q)lBRhq*rq!!?m#&452x5%Ka`T@5T`8g2b8aznoizm>#uxLSs&YEN|R3t|?YHHo$0l+}l zB6CI`#Uxn)1wVly2~<0L!nj50*Q<`}l0YRfA|=}?3Ai--U!SQ~i271wAuC9@bpXSz zjuoj~!PgkdL~=Op>~5vcy;uwX&0h#CV{EnqSq$byGRQ(}qOc2=#j8F`PyChbk@svRO@yzXs z2cdV)OA|NIZww?lb^LBHyEh?vxMkqA|41$q#dP4vY+(N|Lyl;GfmEQH;^3jT#KZO^fX`Z^aP5X65f|Wjp8G!}Uk zXnTB9Qc{rSfUa{E7ZWS}H1?oUw{gn;Ugle+fd#c9)PzyH)c&T@a@Bf*QPqOManjv+ z8eyxQu^Gm%JqQ#N`-j_lDIhy+yGnp;G#i~00_mKO8F9ls>BBk6;~J-h^?ZgWbs>Ug zOsO5U2J$r@LEZ4axjQ*p&`^>dC|Fk+5!XI{kZe70S zCvH}P+`(#qG;JUrU|Yn_?#QGKVA2v79JNO_=DIVht4BmlDsz8zsq#fXTc1AjYi_y$ ztKUuG*5WcQqISffh_&Y1-hJ=AM063wMDnJG7~tqS4*;7aL~p=;OsKra)(V=5a%vvtTEqu~1Yd6o zV8t$7zXfS@Oq!wa{tg}yK7+@A8CfkFYv9x2+Tv%VyT9-ZkPTF(O%ZIbZ0XL|vgT@4 z3pZ=c%saGg!udG6j%~fnwh^@g9m+u9uK6a391Ub^cYwCl=3CjoJW_zTMTzIoYGVzF z(0Wb4xpEC8ML#FWU9tVIU#KbH;vxeBu9k@1NV^Mq>^ue1dS#ch)DkDLw{ghkBah3~ zoMyk2Y|G=eAvvcfRaOIqm{g~xhltJR7(cNk{8mXN zV^nBzuSyzK8t#Oplk$U&oT3{P>3(;GkEpKSPZIRY1BO8XN>P_d61= zIzS-!*1J_SjJCeRUjrP7a>4n!&jd22azu-J>|KD9X^|t9ym;V6hg!I$0mUaGfqwcF zh*ps;H|%Dh>o;;9apLznIw-`DE!P^fp9t3b1EMA0B0Jz0gR%!bdmWxJK{qvD1A-3+ z_ZXxzzb3<|(Jtj$Z&?UHuBNiE=H&g+eRoYtvO7Ky#Lrmw3s}*xzrQTe^2Ecn$tH!= z+qbz0_d|}beJ_yUxmX?H4C{E)TYJYql$`^)boHJ}1SZ1R47RbZyEia9<#+OHVc0bB z0Hn2_B1KN}jd_R<__-7JKK+lc{M_A|LM0v2fa@J8`)K$Q9+f~0Vd19oJL?yZ+j)3F zWX#%-x z$?#x|Kh=#U8gId8FVO12TIn2$`szoijJL?*fP>wJ_X|KlDf*n*h1N>1mlHrD9}%e` z^{jSV80F7I>lcr3nahlw^jL8-LZX=^R{LbDdHwlY0#w@IQCH1RWNZE&6rZ=8;(%iG z2yK5D{G&@2_9{)8?qsQ;F!3o;;B=`#ai}Z@xWP_*pz+$WPdmp~rHO4L?H5R!8Emm} zbSPd{g=-)RjfTz6`MprioM(=}>`@6eOh^1wleW5CJJEX&Au)F({Rkk`PIyq)P>)LAnv?PU&+G z>ht)1-+R7u&RXaEan|~tKUmKie>1-w_ul)C>$lst~4 z2;wFdw>^!@NgU%m=1VWwnTyLJSKwsngnRz<*=LV^W~)L!=HX2Z(;KfBvpJA9qc4I|+Ei(+%Q4y2R-;Fp^5)6dLKwac}rU z{(Etcl3}3;`6k;0py*7k@n|vvft9NoIAa}!MC&KXpTKijOww7|TAP{p++L-dH$Ivk zT|}>jt3fs114TbL+8o89(xZonz_r|68yix7!I_vx5IwH< ztXlF*^HdMZC4jfcLoU^UgcgC)S294#u+G;pzh_7-4UIZ{v&`M4#fPWqvZT{oFY^e{ z6iyIuyiwJOgs2Kqa^tt~8_>LgA{+vcGH6*nNp_GuUTTs3(}w>vZd|v74_Ybwd(gm+ zgQb|L395q7wvj+zq#H)Sq{cwy;s14$hDF0KQv8DWwmeSLg4FK6wZQ%{xG2~8{h9CM zCaLz%Bv}BpW8vyg<)-HjNpPZlbehuv4)po6k!6y%ah$l;)@$4jE0bCxmBM>@85tax z%TrQhlT}{hMDUP%mV*)_XZ`G!=m63Be6cNybtu+MEG zACZ=Y`FT!uJ2m%;Lw5_|Lo&NY+_Aq%x=$yzGeUX?91&^4 zpH3p*YXV6>n0$SA5=wt>0+l8=&YyIB5!q_X)ZqcC0lkJ}mvC@0|5)7U8UJaw%%e(V zwpNDDsZ}=jyF>j;T?>F4r|eI%F*heK?#HnbB_6oBxvBXLjx5a1BKOL7UwdZycy9OK zMmY*zyNPOWNxr3vn$>;?hZ>>c(`40HegB#UB|V2$=??(i*q!cN$8o@}4pGZn^}y;l z_d)6@&WsXpgsx_dds4qG%}ffFd%Q?^;_aK~$L2Hr7Mf-b8ITTKFeDfdQtP_1v3|>i zV*^U$Ix2!cUWlKzWX}#G6j}SG{{DD1go+RO9r!B|PdUyM30x(0Rw;kQ7XNxrZ3X>ME zjk0HQ!p?z!ncRv9ehDZY-Q{YRX}JcKj0A?B0@}Yu@FLBJV&hk|scOFE0*bF7fiTW4 zb)E#k5w8S#9AEV%Qp@!Cqfb(te8nVa_x=O{egb4gq^g5;gmEmB(93`Ql4oSl0JB?= z6|I4eg?+Zmu~xuYm;h+z5i428legqXY6)9SHXb&Dg_u~xZd=j*v3VHc?3!kHpL zr-`#P=RcMP)GivmnQcTv8U9=dNhiQQ&KJ4dEN_2S1*{?CB@hp}1Esy?_SiiQl`C_( zhDUNs2@Th#J1X618yJ;u2y3R_B!AI*=E+B3D5dujULnwRsB*g$z+q&^ly|54ttQHI z<<4eBrQoJsR_&8Fuj=wdSD&GFnX*+;&o_9s0or1la+n}MC(!dt4bcG0!qo4dHo z>r1YLwJ_ki*j5p-?!-cJwS8r?WUqK7b8V)Jbom+j0GUz10$!!0@=N;Vfx_!v@g?0vy=%0`{x;&GYrc2Xkvyh`aVYg ze06Cw1GX$Rip+1(mcAovrCvHY-#cQF> zwayEk>%cb8wPC4zg&)OTr8XW}pET>5cqIl7bYXdoPe!2hv9`6aUbge4d?QPvlpO1K zbZ zkBz#Fp*^Pso~+VW%}+|`8IF9-9w>XTFnFo#{lZZB{!xbI9XQvxp71!~v{t{}g33=& zE8+M$~{Q#NExa{RQkOtb2cu2RXwCT zwY9toDX8DC9w8L_iRrhkCT`=34HJ2H{AHj>q29dM`PlK!Gl0Nb)Z-K|21J6`a~|%4 zw?0GL60g!5E*N#MwhjT1@xLaK_i>BWVv)_Y#e|F{2Q4y_JPg7dUI4}MM*GrDp zAo~Jg4@;MUSLx&tl3BlV-K198^{-qJpU#EttpytNJ8vAngVGm#r<>nB1lWS1jEZP( z2J~B>%7ul??k#>?#iI){dd^1uVl!8Fh3Xf5vfO_(c6e>I@0Yj^V`KwrqwZL1xD+z6 zvxg%juP2(PkV1X&gNwuqSB17B_T{PD%eXYJko<=(KuL(d}@+X%zpc6Ey4HD;GZDB1a7%7nD< z$j-4Hc1AoOA=J*^^XeKV4x}r`kk_C~mdN0MNLulm?{^vObZl%E1Yrh_NRJ&hVq5tx ziwfBZdz=pnd00z*WJigg(ZJ?%@L=ikR5X6z&E0+30Xm8t`6v3U5)~krN;R@(T~5;3 z*VpH>FS@f>ZMW8;MaXrnA0T5Rr5r3OjQHoNg(Qt=9|=1z4vULE^$*`D4L^gl(z?sn zKsC{r?vtvzpYB&yIrz=0mp6xm#i@O~O#8$uCg%I(tui<8L-a#T9&%UNLY}wbE|9yb ztCpcGGvq!NEL-B1ZgFFg@iLK-KOkU!=~8Q-RnTBWnoqKNEc$tKAWh?F^+%0ejpx_J zwkF?rn4SwDUR)bEpG?akX8Z-Y+1pjdf|CUjv>(Uk3gq>1+ifVhKlth-<4GMaXO_wa zr%6TDY99LDK~W3{e-xfnKbNXlWmr8wIeFP+VX(Aj%&xFm7&5RWY`LFa`lA!~Sf@=1 z$-zN9@jwG1+3%ZLu8H@@yav?}?aUq%lpLwTMk?H!WYhJ?1_ieS&0Bk}@nu@iG zj6>DpUbHwM(sbsCo0CtS!T=3-zMzZlB68&IQ*n^`H2ga4lAE-s{=CZZtagjYiFNgmhB6{IdEn+ZgfO|8hxU*?Lw|Jtd`AG7`3wDw|5F`? zjrs_NZD`ff<~qNk|7}F!tMd^D;=f-2j_!ZTX7(i`@;zfR=JtW*QT~4a%BFi#jlU+% z9#Dd8dwobnf2IOC-az2~=qJ`RPSnt2QkVwW?`biS^lBoI7jV_&jCqXy82%FZ$ro)x zS0`QMk2c|1(4KMxO_-OP?w(GfvHRtLFmI9Rt5Kd;;pArMapE}C?%S~?-|vEl^CLgACGHy_U?y!%C_Iv(rPa1 zINY&H5~Mn)@VdkK|1+|%U{nUR(-d3KD#l~{C9$MdjQH~WC4gYwY;UZK zvTyFtXLxOf+10+ih7)D+AN$Cp{(xp|<-^1y%CQc@b1y&%C1s$mwXbh`aNmBw@yhis zEg^hN8=PR7&y^sc-BA&H!vn=c$5p7%T@vA{y4PLWbbGFHVrBC<7UWMhEV$%}5Y8Y~ zv?%0(Kg0&ZeC=T)2Pm3{d5!B1@!#tVA1rk%XDRI8mKwViK^UTcD9XOD|NG|PV0~cW zx(?0dzd+QUtW>zcl={PzIfNe#zT)NBgb$z~>p2lP`A@;xdj8rz$0+rHn+*Yol_jjG2gIy-N^eH*mXY~#sTzWqf&-_x-#|LDZyi%*l{ zkv9Sk)PXu6NbuoFr6WTy;Z!G)e8h!H!vw)xB$r41>)Zz2S44L03f+@#x=C+stgh3( zrkJAX&N6V!!7uEmof_KTC^m2&!Qvi5n8jFN)goiia|9W<*$>XE& zpPvk#kQ#F#cPWmdPZoo^)2?8$-#mEM{q;oT5_Hz^W!N6YGI01n?k5KXG2H(?2k?GCZv; zDtggI|7V!naEU8=kf3i=BL}s^Yky|$FoFq~ll~YU$j58M`o=3ufwHg|MR1CAV!`;} z^?x68hU}U@z4ra%)qvT!Q~B^H#G0DgCC=}2^QKUpGu5m{&Vl~I>k` zu+4;AI31jFRV4ieEgAM}7~-xtofRw%9D;=j2Q>?Dh;SmqqC%k1jeVDp?YcIdi|`p3 zu>f!Bt@~qJ0{k}I`qY(~QoRmeOOyjQ{}GIRM8 zg(`l%-yi)!%?D6~zW{+fWln<`pz|vyfcbFAXKJ_^_`RV`O-;l5%|N1{1-?XxVu%B} z3{uR)CA!vy(#c8@CvxK-tW7k>p8Ixx_I=H|c9Y3hvtlSBy?|=wS71}zX-kxsC~*z{ zp*r3_KW{xXIf*FaiJjU%Ctw{7fL&P~9i7CKloVC3ym=^$ODKu0?L4Mn9p@_hIXWbZ zC$^_Raw2#`v!@}Ap^brIH%&OU{a}y4&62l0zFwqJ%fwR3=w(Ra5%xRvS(U3Bb>~~@ z8hK{|UmTzDcyf-=A}#Q}V=;Lmi=9HVMM{N>aYOFbq=RcbR_dgT$H#?vvb(PAjx4N3 zTRfugk6}+sNO0r0YL^|0^HR*4p-3Y(Cco?#P^+t{wdJfWMKIep6z4_-ig$V2RKk_w z;eQsA$s=#Ys-UyuMNH;^mM(S(%Bm@I+;90H+|echB^Y<)QVE*EpFeREu9d zF_eHOlkf6)3rs}yJb_6%dArq~*7K)P;ns8-uFO&yH`ss*kMOtft@MBXC6k9tC@n4| zu11>j7E_p>-D4z;P%at>5z1d*F_p40Y*m%H@=K(Q_oguZX;UI-q>@RhSb-9hys69c zQl?sKO~Ucd^IxLw$V0V3EBjL|a71tu z(pBwe2nNa*Kb75)&l6SYh%;)pN0wiC=j;6)hZ%Byh-lV^B)JnEA3_o-&&G`WJipp^ z#LS*`Fy^q@eaIci8dc}eSSN`f*OSL?=TOOr2h2&dI5mVMUmuFvdg5VVVIcEQen~$O z?^h~ye#e(}w=5jws6s4BK%?-D7V$4F`3jJvJ7CQUKpvh1*0Vo7(fx8idTh|PhB=gq zsS1p(+fyhDK0kdp{jXC3?ZbvI2RBcYdMM3ym96JpOVi5I_iVUS)9*a-`hh+rMS^Gt zh>0l6=SoujnIbs6^5E36`2O;7PN3M{&r1++mFaOl=qPfW*`ud;oeX52f6iQKILh=6 z5?lW)xPSSGz>XV>XlKBWREDH-W7DCzZ5$z##N?L}?!PF||4uW!0gZT9^+}#g4xqaS z`F|oP|D5o>^^X1jAy4nG_LBW!O8jBn9vs#rI3)FdYN5x3d;-}lA@CClRc!yOYWP1e zQrJ>MoF3=n_ygjI2&fC0aQ<*WK^@ihfD-s0DAGRwM;O4M|E2Ku-%bQhHT{H=nP{xo zRjIOY!%!80m-RnR3)0Y4{&ngis)B~4A>bqosC|wf1_t@Z0@y!(l z6lby0mWOFnbmx7a{*gn)k&@ylGOP#g3aHI4NvLOo5b2`mBZuEu?Sd#rebll`UB-Su zbWC%7Vsx#+TELj<0VWNY89{+w?N+xcqYO#+? zvF5PgW~M*foD!;*9L6K-s2D42C;Ib_`NEJY%L~T#S#CFivWdkj+xv5Rw*pL_<0Jxs zE|rv%qyAf#S8MSlr+P0C{XpmV=st0?Ot^Lcv*N4Tfd@l07j4>#;6%LMVaMvG{h~AD zPF@D+Dt_JBTsgkf_)A#v+)m-mGwm_QS|Bw%21>o8qkL=Tg)uR-py2{iz#lJxu;5NZ zsY!nxnO$09@p6p z4xYJ{b)xD=rwDA7IdW1QZ_d2tplH6pjAU`Tl?+#|FhvO3n@XIXUC>~GB){g(PLnb! zZWDhb=;gpYD;f?JHlj$j9j*}d@l^kE`ec+xp`Gg>@zAX;nfc_< zS6JWp)Xm4+z3->*w%eZ)vs1x{f5oAeaX9VAxMJSzw~7Ma9E{NMBK_==t}Aw~`_W2G zGnSHPT;%#f5+P}Slm}Tu>y~~|zvO0bZ||`OTa7fHJhek*9&Tvw@|nzHJR|f_6vopz zxgv%hov!Tma$PliCHHx}q#wa0Y<+hQ$!{*@rn5lu`^G~XyhML_fupa2W`#9A6 z9tceqgV%DKE7s{$<-+NMe2{nr53Dqt>OHP#Ns?G4Z-l6qG#ekOxCnwnhoBxEe%M{4 ziY$f$mx_r9R`{}iZ^6^Mlx@K+&}+lnU+xAO1^!hZ4ReL3UBPGydtyn8tG6Qz<}%ys?Y zeT>NQ@qvrZ{JWR>m}Kde@v%<#Pz|FqiLVD|n&8xSgZ+It8{&szH915gEDv9 z*HXQrxE#=vz7X}8kX<_Tw_SzgT?C!?Av^>fS;!$w{3nYj%7T~93bCr6Uogd{N%sAE zf%qZJ1AGLMS^wAqK9)*BIH@ZoAi=EGSfkWBz3njK&|7q1>d|@&=w6hvMP#~cTzy5e< z;o%b)T#zjO_bIYPaqo&@339Wmw$L<^%rOTPFu6bRd5cRNJ}pA&YhOr(sZ3(2xk?vx z2tyx5p-bPw1P^>mv}gg>C&>z2z-Tdn#UM25}mh6XZG=KF;?-&_M&%*8^ z{s*G$Nx}cIH8)rFlAWUL$x*^=c*+PB?+P-#x+yde>mT1cgn0;q_|TT~0pqXtn84VZ z*x$Z@AIVe!8YJ)J4ZcDXUZgf6d2^>IVPxd@B){>=1JAAHbLYzE7hdz1R zm=)iGL)`GK~w~Lkk9If#xz1QX9b=j!ud#vd0s#k{g%5gq{fP{=|X=@9iTAANiKQBHK_s0nt ze0tY@mlEvBDZcy1B?R&o-Uqk)n7OW70(p*$!|&uwjF#SH-)+R77fa4snBKT(uxHWO zt87m`sR>h_LAUL9UfggoZW*^q$Sf%GH#YWj{2KzFi)hL;N3%1sDj{Ab^z2&ajSV-w z{;*Iuk?6P>mn*er{&r2KHeyywgaBOyRVSm*q z*l+vG)8LJZx-1^$8*bVU?i5EJqwX!$)i0jmG^e>t=l8~Jep@r$IU)I0e9x~@#Ak-# zf7id~u>NvykC`L5qbAlZGx}rD8Tmbphf!8 zUWN5_HQ0FUv~yCqm!qOd9Z~G&)2F-LHFgWlBx@(Tx4)mOk2$w1#}Islg&66c5mOr1 z@$0+3xVNSa4z70MV<{*v27Pb7(j4W~8Z>xp)??>>&&pyoK6a4X9xkZB?RC+xHlpq9 zveUtE(yur6KCfs=cWsN(gK22`{P_z4c??X>=LrV4+OJyTVdy`-dqRe)O21P29PC36 zfj9{^anjzt2$nNDD`7p%LGAgD^f?5w@f~Yv9kitu_cKsbZ0CdNNj-Zw?@N{gWGi97QiYPTuZZ>AwuSH+|nXu{bk<=W1+zSw(a{m6rL|3Kp>4eU+XjC#^{@_ zcZJ%*R!9CCKDPH&W*U|PRWxdR_gA;hlhdI!s+#19kVg7P@UiF_l`Lb}ne6H5)RaI~ zJaiw3MD4DWxk5;D{W1-^S%WKfYeR9$@z z4gf6deNTnlx)kwz9x$^fN#J9zhYJ$@T2Cr4dMbi~H#K3q)=V|Ek~K8b6qy1sYC3Om z6wC#BdiE9<5pwWl;tK+Dl!DPX)34dRe2NZz`Nx={Hhf<@DKhU?4<^3DqS5SC-1eZ< zwbg{}H+G*+;(Dr$>#20}--i6FiN)MUB-1*CRHOziUd62pme5GIq9$Y;Z7XPCrL8<5 zB#jyDvY-hyTIsd<09lo?&GEh6yBBvV2(5RrWD2BSbV~Apy-Bm*`{gXTBq*!_1$4Y zu^^nh;7cw2p7$9Ln}-4b{?=(uJvLDASDDV&4(vF75>Jv#qgcpv3S+dbRlddiIH6D4 z0&Ki4~|sEKTk@_ZN$eaYz9V0|FP0|`wk-Hz`iEBDyS~|Tg9TC*Z!U0 z+M#>(GBI`mPovQi3C0gmhP4_`ldT=hHmn7+t#tcKU)55m6A+B^=Vh%P$0Wo2N9t%d zZ%Z52YQ?3$!F!4>nNcvTb$C{I4A~q|Yf4_Iio2TRTPkbc_V&J{l;{2@xJu&reyuDw zU(mBidfUakuCX=8nXwis8~BzL40iXfG0Ird+#h|sJQ!vDwNEAA z#=ByWH6}=eR3uTIYCeV+ZR>cXNQ#mS0Tx51l)5`@HdtY3z18?EW^hatR7sXw13fk7 z;VO{J2s7uFIfwc3!lkaL@8Gl(?`hk#40>)`MSzM1HTSBbkKf?iss4RP*?ke%7-!Mx z!ZZ7c_9}zC$n`FQ++V(o!(52#q3cajCjS0 ztuzq7QMsbb;{M};bX|AW7rbqLn^(7}_{|J|jwE6*n|$-flx68I!`ilG-gr~(a>R$g z8^qK}&IroQJT48_wmH_vaY*I(p|kv}*jr*#AeJcm1H=Ppms4$m`JYk=aFVdFm`ygi zE`Llu=eqLIqT_twTz;Pr0oJ5^Zdb`LT*Pgi?1Gy{H3`yuuCBM4K`Vp^*|#yPE#Nj# zgkh+uEo$z$8!NJfA*B5|boNyP>(8PreF~xF+kxZVZ^*{CSrR#Qa@j!`uCmP!Go6{l>#SQ(D7k1)7%G)Fw&f7Z0e12+ehIGBHa&bQult zsJ^$6a!ahZR8!gT=Dy8-{!DVsx&3DAz%KLs?}1{XuJeTxP1!cJjg1fsm(vDTOlsME z4N1j3Cy7Y4H0L*mi~78J?u6k8Wp_XZVd;INlatljwyTKyq~tjZDpXr0yL#AIbHjV( z-MM@321I*PxZFpMw+#&~jEW$)-oC$9-C=KH5G!?!DK=a{)e~TnBn5(EHox3hj;t&A z*yZ%Tn8&}itgFElK|9~{23A>yVwQ~qsm%q>GeXtx+Ma;sjAdnS# zS~*2uN99$D`If3SF7sgk0fV7r=)CzOjIc{NF=GbJh%_bfc;#92QN;s+T>lo6tGbjI zJB~Ft;H*zYjY_)+CjF({oAQub0&t;p}=Ao6~GR~AZmGE?*gHU`~nc07^0+(zF> z=$**qI4uNfR8`zQ+`D}>a3E#uEjFfkFA}g5BL=(ES_=b^S83v$i9#ap%aM`ONVf^z z81e$|?YiPvu<_l7pioM#`DKS$eOcHRXwqb&$S;FuGpz7diq6Im9bc(L{ysLuWQgt- z7k6bY;c@pN1xlid&oxtE|oy1t3nb2pe zb;`(|F56qns;M*rMCAv-Jym{Qn>#J?KnH2%nXDe^AW>n^&_h~GJk%vz7gP=3KG-$$=m!5Kd&WR};R!zW>)HCpuZc8h zP`Qlj2@^}-YtH3iD}5C_j%gKx$~s!Ku<5h=R~zeY)ypIJt@{yk=bVQRx8{+XO0ilo zRnjJ*kH;t_M_Pce_Ldl07BjCyI zp6yr{$N7WLu?GRAI3KjdqCZXaUI(Uc#K1naNDVeNiPC`ngK{>KLXwJ|i(7G{?g4 z%!jV?W~v0XcH}3O-eub4q-yuerArl4EIZUL9ZQ7dJn17$@FYfr?FueB_U*>Ulk~qq znTke#bapUn58#&^YM_>G4d)ipET{-tf3?mX9Wo*3c0O{K;me5rdGqHfyR9jXJ*4_A zRPT>R9y&zjCod(T5gSoqDE6qyz1h?G;gd27lpY&_3lh(V@D3}4`o3gQ5HGx+{y~66 zKiZw+!^GdT!nzgSepM0IH<`=lp%BK|*2Tx$o;;Pt({{v%hI_TV@yUL`+VDUr0y%Ju zpjZrA+C~A<^HqOkI<)UiM?A^!8)9wuC^7VcBCLW|U-T(%cLszP#LhH+aN!CVJQPJa z!+bAtTXTaEApb z``*GQSTW&US+31(_08Sr?_R)OI{YT;UU)^zGumwiwG0R%boLFK=OqoZQ>Z^ia?w~y z@Hmw;u*;u``ee#XxZ*9iYds%SZ$&T;E-8TMkHA6pQV$SvEkU9Anfld8W+vG*22HUq7 z>-ebaYGhNC<+*F_?`|+);u8({ReZ!rA(9jk{}EJu`~?lF4Le7KtnR8?JB?4!m9QqL zFMu(EJawv#Re_D?8VwbYh-jz}qq)bi-JUKnXP74w!;r{2)xNon2quHUw>5bMU99(e zxWd21vJUyRWpdPAJ9Py)DO#;8-{ur<_=YP>$r{#Q^8y)%besZgF)z0c$M*SE7_JoW zF>wsHIn4@T==(3c>8l8G=P6c*q*^PUFS<}hFVOzn@{U`;wPn*Qs)j9Ad{Sm?KqR^urL z##@ZRr%|)2>yBTq15Ey25A1;oz2PlwO^vxV=}U4s_PdE}c#@s(a&BFPAc$7)#BcAy zl^iu&upAj=IDVdoxh^p6X~nQC%{wz`0gMD?f4*+Dq4kUQ@%5R$qB%Zfdx~nHSUixG zJF`{y^rN$kN1stN^@%xBBgsw z9li*c)=~RX7Tj3UtrF}*;No50-|_A$1A0=ddz~2agXI^VR16y^f3>(aSfQp_yd0C# zuE87@`(?sPizffBZ#>D3glCAvptg(8u35O^9v?_CO_OSp6FI zII<|vrG!pZ{@pwtBULdeIz&rnU6xc==y2|Q!P7UJ_tW?XIvwM~=EhQioQcA?i z3>&vJ?BLRf_eA(uvNZj)ibYTI9-W-?yRs*x+ug0~zvESJtuYwveW+y)^H0}(;9%Ht z$adm&_0ZTtKRdTqS#7fyCqOYIkND6^mh;+%&vBOPjVT%|=QYsuNgKYs(__Yq_x=zD zb3Fb8bewtiWo)htBNCsWMoaMxo=;(L7da;Ho*w}L&-uj~FLx>1`d903@Pw|uHw&&c z*nJ5_IEJ$X4G5pj2+_%rBj+yLT_k1~+q>^7eqS$3pBj=($8S7;EG`I#vt6{HA|$)W z#+ifp(86vJ)Lyo^?#t1R%e&_jbuvz4Dle5hsW6OjNLJ=iy>bQgszzU#?ndkqd{3}3 zpiOLLm#wQXES@4Rqd_T!I}dGRv4|CqE9gWT-SU+75L#C`lPdh0Q`5U62xO_iI1;g{ z0Lj7&+Ryv0f!Fo6+P*0d!W8Qlz~<%7TiCkUZQm@S)g(9({hWWRVdF-cW*rEkTx?G+ zy;7Yx(5U>HGfxF0Az8I|X*tD%!Hbhau$Ph9R+!EyF|NP4$a5Ap-A@L?na&0)fbB1PPs-0h9?iy`~sM@;wf2!>vrsGAJ|sVFx!)8D45 zOoUv2R?^wppBNrFt^8~-9d|kDGKgKU?(yaffVD7bNLGA(dEp00bet)vl=oDyu(n>^ z?|WwU)A?3JMfU!1J?UKUgZ=kHA!sJ1?q@9MF%BbRN|C6FY+xDDv!ukhTDXVWc#=9K zR=W4tSmvi(h-k6)Y8ExqnY7XM*C!xIX8KXO*HcyqNo!II#oyVgK1e`(I|rGNaIVe3 zl3h)+?Jt#s&Ha|)sx)iKJ9e)nFEXq*uWp;is3~zZvgq6pXydTl!L#rEGadUDIIf}Nlxuwg_H#p@A+Vx#y0j{8v{fb_xHXR6mr{I ziM+F!ei|BBZnpHE$F#O?>urls+3b){E_l;!cB{(eL-P-+_ipX|;PMi@oK*J|jqFN( z<2hJg(PG_dpCbLO@QT_zNxBhRSFyxhM%YD^eoxH+KEc|D2})KQ)M~PL`D*gle*XxU zuj?tnMMFTOKGp1|g2mgW@OVvi%h52q&nh!aD1Amwzt0CVQo%zA_YQI~UeVv4!i)qM z;ZEa`W{_2vU}B=UOe|QE?+{mvK9^q&Dl5js}G79HN<-5%Ne$)WtATR zw`|&wtvzlXQZiC-Aw|f~g(Ez-ns=H&8isjL)2k*$HI_E8vxi=4=1u;3ZW*v!>iY}r zzmp=*8&n7B(OJ=^y$iI+8|qMd<0Xzj@|{1eEKNrb5n0Gvsko?$#hsOAQteM>qSusE z2)Su;CYV`mgdC!BRoR}D@Sa~lNM8AwYcIforoNoJw)EP@W1%$Izl<(}nL)OE?cFH5 zd=wP5-!ZDJt2;L2bz1ux^?bPV!F|)NH8^Vwr1Fol2%a^SIo7bz=>K{2^Al+s*Y;k+ zb@|+k_i)Cu85Rdi8YfpO#Kt7jEBTZW5bW;eABnIsT}a8}<5T2}w%!gi86?LFtID^9 z_AZ#Uh_Mc}tJMt+epXd5WV5b`L1^IWK}C3&m~yb~WRf0t)9GM>fU)Mr#&D>T3ctFR z!Z?}o*wi)DqhaJUdtcu=H=5T<;X0FD4P>vfc1P_ZI2kDOrS?YOE~?!(>@IcN(A<`3 z*E2B1r?*lF;rv<(A@g@C4<3;^lFTBT-cemQsVd8?u35NxkblgyE0^AdB7LZQ=?m3< zGh9`%dOus=^T(~#**mOe8d|XDXa4LygHeDp{N=Kr+p_f$pWSdN-BwTD1Z}yJc9=N(OKml2Uy!9WP+tz)eZl0G;XU|i z=FNS+phz+c&rE?i@F7Sw|KQ%+SxB#Zoy~v`Id9r6MSy8N`qwwqccgn^WKCsU` zR#|dhtR(zVb)1V@FZ^AzW0&f~wNQiYGYe;h=Sw#RozKy{f9pytX~%(e*$QbY&p&r6 zqWr{<`QWwBRt5bG9$u>@Pj_+_zGCcjoyRQCq8eT>ZSR~t0fL`i--_@CSOoEm)-QYj zKhkq5H!`=Cn?h$cpB(k>>0f@>LKsSZeR8~4R1cpoG%Y%ikCQGil$+mpd#{I7*m$k;Y&ISt^_OZbV9mO@6vq7eRXd%x z_Yp`W9g!jlHh?z_r4zY*V@C5<_7;d*-pVyweH*IksoTdl5^%xfTdsK-U-0_sO~bB& za*plQM-B`IQNn0j*+A8r5pwa0c<^lLS^U0oLVe1*Bawp$sr=4#ugr2`pSs7WBf+o9-SI+s1STNg~fwFe_9mO3nkK#|^l71@Z3S8IbbsQglVIcbm8U4lM zaqs3SQ13#r)=MDgu$w08Hk0r~#xhoDtxRsr{zk#06)lFv0I6&HW2Y|6r;W`@cU?eC2RU{zSJP8V}w;>+^vdzR-E!=~#cn_g+okS4I5{juC#Mmxi*Tj1%wfH}hI z<-9yr!}^?NyC`dJR=Sv^Kqm`IKQrVEUIO<&2Mn!`Qd8%&9zX_4{u=vI(I+w0=_IA| z?<-`;H~syiFEEJBEEQfB##4qeva_1Su4jhf z6D%lPSMWg~lhF1j{#PYND%?L}&|o4)-+GGmPoPRJ$g>kOpkGuq$p0!#+d?hzR|WQW zZ4FN{XIkFI4p+pdq@-d*vsEPACFLaetKCiNAyw%@rJl0h6%|R|;+LRRmjqJW$p5p% zNscw`2pt@QUXP95HpWL8El!=KL;uux=H0vewJk{?S0}^fIbfz#eujz>y`3<6uqHq% z%2w_%&2FBjv#Smw?q~7|N@Wk=B5|}ZbFSY~?`QPyb!6=cJWkk_dF>Rb^z*CM*)B1m z7o6Gljc1|06eX9IXSn?J<^FNjU!y{{2sS+I_3UgD+l_ShUKw*n6^8sfKL3>{N7j`S6J zOtn2p>d@h0sNu_Yb6~$^sOs=@F+n~})8ic7@3+47gW#Oj(Zce={sLc@678kipRSwd zsZa)Q2;{%ftkHLy$=E1aTJKAFDmMSIOXUpDxR~Umy`ssz3jz60PZrvb2As-z)N*&r zUmO=$>z#o5FX(mtTV*7xWrF)63MHhhPX_Ntn3M8fg+*~ z9LsVewCop5qqwmU7c>8FT+RQ+)%<^gt2vIb#6dBBTV4tiUuxuPk`hE!VW|V-dbsWV zuZ09a+|je*adEKB{z+V%#_NIQgUgy=SOAmM@-GC7r)n6(uY3S_`}HTz6sp|c(hYgx zc9vgTFznF3;9B;7W5g_AExgsPG~@gW8gvKhmjJg4O@A%@bE@(+jL1}I~;9* z==t8=aSaeX5VPPtg62Hx?k2+Mz_ws4)_J%;@-EEyKctXEoJ|4!{<_p=yfYA1Eq6fArVxQ4Q}SzQ8u{_wJDd zyT@;M+LM?Z!S|=&41{e=`5lV~yE^O?{}rF69`ccZj17mTLrmBS{CbZ#KO8aqtoc{t z%v~!#?#h+2cVO$~(F(UacpcCDhJU-DmJHUXWYzSjf*c2% zVWuOCfibO46BKb`1ph+d`7N496)mUET{!?9(ZNAd5%l=+GqTH&NE)t`mN@9h0X{Wu z34Z-cSUAbL9ORf7yM_Ij(POQ*fi3Y*qwsLpO-sj_+}N|NC~#QI3Y!yk6IRU)ObB=Xu^@4E43?X^zoAAQ1X< zXK@!G5Ev2yA(w|!g8y-O(a{b5A@jPRtqv*f=KcYJ@I%hwuqJ-i3mNo}SU#ux>T!eT z*m|P{rQsbo_S1F?KQe6wYq_G{6=E=m@xr94oG8>K3LK`ZK8({wKITJ*Sv*whl$On; z9-^6WOy1T!_gsg!feSw+6Eu$IxDm#2s?uuNi)P8(*u)^@b9VJCz2pT&0Z}%RG!NT~_ zED_W~VgL3v2G@A(UmlDMAxwbaBJfgQkN?+0lOB}-Hz)tso2NjA5)2W7>9Bu$ZU~ah z<=?~451yK)IU@OApPL_TL;tVq!h(aK)v=E}nEqwWLLoR<;lHg4s!Tk$3MMXt3A!WUd_T8y6(MRu7X*ds0A>EGhLU!{K>H>aE z*mJbCxysd?On$%OsDQ}^xrY4E936-KbFbv+^7AGpCbrr!zaCyFtf;71kC~&zoxVr# z8^xQ~+$uWBw6Q$dV63nb$3E)r?EE@S={!zcop9%c$xl?nerSdV*qVr9KRda9Y+PRE zq-HXt9)HexxnCK}fwL>@^(&G6DsomEE8^nf?=I2q9}VhKqV2F@GAwe}q*L`Y8RKcu zIv_3%8i_ICe`XcS`OtZ;jk&(*%Pmq>kSUwnJU32o$h&iY{`@(0m{NqV_5{8)qU*7I z|I)Z6{>i#|l71eS=@w7@3k?VwulOod=pQ#1iv;T)ZvV=h9I1D=*woZCHK*y@`{WdM zhMQmSVn!||+b|B-0*$`Fae1 zxvCg!QPOgYLY`B3rICDe`R#1x*C^X^@a*uROH5w^lW$+;(6UkJo@VhbsW3ol2{x36t+{!f9f<2W>9Kp_>UUI12=c! z4g6`GHR3ZF^o#y;(~X8e%eOaAcnWH1Ye%1c&%<-=s&r~l@D4=D87zACMP(xWpZhnI z5mq&-F2**!NHgZvj(p%^+Kg zn@I7G+h~U+IP!VaXmzV*qcoy=)6aN;6!LdHW7(2-;pCJX!M;^{Au=1uPXpy~Pr|U*ktKBQn%N7z{`{%XSy+{yT_UeaF(`Q7?LtIj zobqS2Y_n;}B&w^NKZ~q8g6+o<Lexvac=XNV-I|7D@tS3p(VxWl7N z^}m~1SO@dj<4oy6Qju62+VJX5+x3~QcSW*2O3Sq~Z(N2FjE9W<9wdUc5E!xYj}jhw z6IK;!C&p$^W~-j18uW1?+;e)xwZJl4jDbhN1LJ<`?K!w-;)nX?$}zBtHa`F8z)sry zsk#-OWSXGPFgNx-aca)KgkYye&A80SjGaIOfI#_=eKUeIFL&xoR`Oli*Vexq9hz;m z{j_D0S$!QRsX= z0YCVkY2=}c*(O6;k#l@S+gvB3Mq$Hy!L|9%dLaL}cFuXwhuR|%I+^}IUzqJ~Ee4pH zoQVjUKrn5@Na0{Xb`%DGbT0l2Fmnu{)y^U&|8;$R=5xKrY%g}S-Cl$5Cn(@xDpN4Y ze|~=*QxiRl=yS8pdaAYla;U?ss(_L#Fn3*!!6hcY z)qqN82q*dP_ea|HWU6?#^Z4E; zQcJAjF{)U&I-Y%CoQoNa4-h+b;X2G5w{3s@;grZjYFAW_WVNzj6s=kqAF?myYt}Uz z8E$GMN@^>Z^BVHWdr5m#={gK~hoMPXD$*TZ8rqwyomd1iYaD_2!V(9c%?yyJ62;hj zd~|B#i+$qv&4|c|JP@=D--+9^S;j90+G^Q}ut@-+SLkKwzVq9}!e`TELfvcJX!t2` z+4YB$!_v5j&}|p@K^El2qTS~+{f3^s0oQu1X4xSUf{k8OZEL?53FT?wLN)~NMC^bHKqo6S;4crzhCCy(DV#$cw*o1s)=o?6`|TG#n6E{xawI$7%&JlFfAxQhmjcM;a%&DTYSEe_J6Ea{LI#d$Gv z8tydkcuKVT3hg@zWSG#Y;^)t2p6LoEQwMAHZ35KeZC~=?BMJ@UrY6A2s4yYx2&6Ir zg@Z9>7bqRaML_Mykxw9J)3WS1v_jB!WM8MPbt%wzc9S@+etP8BZY2P*)Q;BG0+4+^ zKfL|*Jb8((m0%Be#2s%tnF31Hh4*j0c}Qhf%a?-y=Y7YE##0%7=3zpslf4Qf7($MB z8JwbI~&U5NTABt-ahxb{Nt`cet{?Xh^kxC{}&wcsAu^m#gSp+ zpNEEwMN-fx+|0qRNj_>6N-adbCzL=G)QdNz0ekbAq1Wm$0$idwe#Vd#W~KiMbH=^1 z#}$O?niel6yp;k?FF35~B)Yt^^09jJ(s<2nJC{#CHhh|Tq!f8EID&qmX~s#AQUc^i z6cI9agB03R5dGYWsyornnt#)oWlQRKZu4&4*UrT0c;8rNSDz(PgkY65&l?zAsErM{A zXgtvjnW95B@pYHpdmPGyZtlzFIqfw=GxEy5BsVWti$2e#ct<-;X4d-W=r5DY6|U8{ zGy7#tO@pqt#&ViiSzXOsAGBER&CyiL7RvlGadpAArJ&1GM05{Ut;&yv#j4S@#jzpf z-29{1BmxODJq-`q5|83t=(d(r&i3a?=EX@UM$dHZvHu)@tb_^jczU7cMSbwA9rCE)(U&#sCtK3sMs#pzb2`S4A za5tEpl%AjLC<**dyYWG^BsHXe;a6!b)$PFfNvwPhIkJg9);a#JuHYzL3t{N_6a1s7Vqw=#5V>kSL?(Qbh$0GQl%l#RhzY$E86d29b4CB?CL z1hTwAGB!1I4c#2e%H`CRCZx!xkQulp{~Y0T#3dM6t-hRP$B)GcM?I*X6=aHew$Jdq zpdfaz$U>Wym9_rN%Q6D7&9~DsCn{@FvyKV*sOLk9==Fs!uhyyPI7&MzHCb&n5J)t4 zpyu=92!hR3lV}%%ODf=I0@$$trU`oP!qhsNo|niBWtl?g7m(9D!tc%e>*E zp@jT;Pe&r#F>8GI#3*Ls*XPd%`rCyyy&aR}feDXOH32NDL!Dn4lR-B9jO(8i6T0$! z@>FV!$F>x^q{Y~Y0HLADb0_7@DXN+L6s)1hCh>bsckGM8OK&Y+2CX2!q4ipkCh==`JF}Lawng zDSTdk-Kpw>C-C<_uk*7ob`1nYRfleN6ka8PHm~F<6|ebnt>;)0o+ry?yt~wEw!V9j zcHgjtN{w#ZZyGjIv3O>Ea;<67-(Zc)Y`*S#NPxiPh=-!^X!7+Z%Dh+FEwUCYGGfQ8 z$&jcEzb<8?@dq~^`zW>GDTeyizN^Vq?tcF;c317!dBbE;GBkeT7tYWJQ8-}2g2eB; zUz*dAkvGdUKsQUEtIxt$D<0R|7Z#JioF+_<-y01jf1{eDo1r-PN>Ke3Bdaw9Xe4G1 z0y4!f(fBtKVJX=xOy9^8-n0>t35}a{pe$ zRCv&Cx-Mwr6%veU7m?TmdscGum9oS1NO-`YVR5CFtJ=`5s0%dbK*RO!Dv+zXhsK-c zd+l0WZD8jO-pMl*S40L@mR}p%IZzqU zMur9NbYAJM`|GU{M0&r!JO=~GBU#S zZG<-0jR3ceFR=EaNg>!AoLIge>^7<7Jn480HwP*yR>edPEWr6L-1hkL;X{4Z9djk^ zOUz%JY1Q<7$KTZJR3w_>h+7>C-nCkZ(9~ygB(RMqfF+<G|iArMavH& z*~~Vj^}&6Q71@3+FjdY78g{J^On%S3dDGp93zWYBbn!&s8+N2Q+(Yr(!=k0j$~j8m z{g@y)RJNhH%4MVP@Ip0fBtNmG3vh%QnsL}UH7ptoRB@cXv5LwfcD2x zEc-AaYq#C6OI{YUM;}G4)h%s)%_B5=rJladv^Y`s_{i&$Vn+S&{Ep`JKmyz-wj#}o zJ_BUZl?OUFLo9}PB1{)1dxP#y?M)EEx`8c8#jSu3heY)9GXd?98gIwaTLwfbze~#) z)sISbDV`z>v&WB{^F1hR7jUrCQQEGVn)Q0`x?@AsMyK` zFePA4J|0npPC4&pVk;iiL4Y*iH{NE|q` zMhc1LJ@;j-22L7L1QeM727>+<%tGfKXNu86JLeY^=G3MIrf!cbY2Jy`rfySEvb=Btno&?Fo3Z7 zqW_7o!i)jWQbB|IS(QBwW`35HIdXQ8M|=s{I9TS}XBI79KfD!rmFeriP>}7r-%L~i zpe+~ve*kSKiN;JZFtdg$IU{nEF%)R^M%s55J_h_fdhIOrPPNAUD0o@C>+j>U8C1{3 zmU(it z0Fy~Ecc<(+0p1jSyh9h(xxfgfPIbZ-F=lR_R9)=Z*?LIh!MXp_ctDm3a4YYL9BNDQ zFc(z5A3(0EBC!NGon0itPHP3X0VWN6{VUqI*$i~^tg0E;gEQCV6F{;#49t#x<(k53%Jwve*PuYIlfS z_evbZ5w)=}>~(Z9Fa1K(ILGd<<>iMzau&=rVS%$^G#M|2)`PeQZc&cnR7E5FMn1pS zbdn-5e-$~#vYIf>ptv*(1Tu%s(|5)t7QRFmX~YdC7x>X@Db0u#gZ?+1>NpSjX947O zYac`tmqsglKH>~FlhF8rL(TV+nz)D~q!%CoB1)%W8(^qx}des9@=#!^o+eb~k?XERI&*0L+{Fz(WTq0L0-@N`+BzkcOt&fe)u}8-7*KqlQB< z`EZYuPW1-2M;Rt7NfKH*wWS75jXx(kX6_1v8dfa;%H55`23<1YMCui;{dT^P^a|Hu z^;~UxGlMf#zKbK8$wCHl(H+;yKBwwlyK-h)bMZ2$c5weeZmj z_4MW`&381gHKh65o6lTzswFe@&1RMOviJZAOD0P82Y-ag=KoE954+xSArWsTE3vV$ z_~Mpx{MjO8-C7EMEBrOSR)7o5Eyxnt0W`-mr$%-w>WCLg;l-yyC;1KGWVcI-`Oy-v zC~_F4F*Wq(I47bKLJjI)9qzQ>*1~W`*WA~LjO z(G9TPj3%Xt&CSjG=bwFPgq8<>?BPVK2Wu&igNFtcJ`tfxMJ5Z9!BBud-D*Ez7TU7U90QvTkoQ2fn!dakW(a}xPW4&L0gXm1 zVO%hxs^3CqH<1B}J>szYlX@g_lDV)!Y2kAnL{4WqZsjlmoOcBxN5#)ekuR1YKg0B&Qtqk@}NWW+T zB{k}hJDt|qB8rfGE0CDM3+~b=QK3l(fr--wf~0Ufy+SJ>Rc^b&NgJKNQX$N~38fi*Hm@EdZFmMGr%c(*j& zv<7ZN1c}AM6{hC8iB}+19aBY>6T6>kmh1NzL84`aJlHISA+e;-e*;p%P)T)#0d^b^ z3+A+>3lZRvp0V#PMcIyN;U;6=Ik09O$H9)E7mk+GM5U8LMkEZ%soM=>+fBZ;sZ75w^`E<&95fc6uq!i-gUK5ufBv$*3L zQM`acZ&$L}s?2}E{>KaOr<7fe0%fEK=N&D_&|-w}xcfH+F2L0kb`l3;XFnG8r_Mf3 z{qFbv8rI(gTCu@JSU38x8o^j z)fIJgX<=*37&*L*00F);#Lx{13j~XHiVqdY0CgrUo9q-sazH4QE%9f2hYVw)BWmG9 z&Up#3hq{6)WszZV0^pt2)tl3<4g!G2vgfg-LwKM`i^a~LO$8QC+MEUgu1c|DB)%tM^e9>f;IT9Zw2799c zn9WDq=C6?nyvqX~r#TD;UeO@28Rm`5LDbo9>GiL|XZz5h0kW4@+gx=>_xr4@fA<3p zeJYZ`i(|v5+2Z2biBqq^;JfsxbV~Hf4$X}_@%v!3T+@L-{*c~7h3}sA5RYiObLY-! z_b~;2GBU*6d7FobDP;@_PY5;r)6;+y;x9uK6cpsYeS2o=v{VqH;~cS5o$$tJ7Vu0z zMIr0|TeBc2hdD^U?URaM?NBG|Ux0)zQSYn6R~e0>VSjzIZ#c-1YAn(|H1roK#RJ+( z5e!hLRgMG&Hrcqmtl-_L3?<(~=e2L|O^l3;2Hq$P^MSYL7-yk{f!6TVb!a#3;zO*PoKUc-kD~`Z~&1Oep6EGQC>PSj(4ER(&&CZQr)@ z>AB?KwDZp%y!d+e$Qp3dS!cweND|6QD~k+`5XhOY@uJthtAa0p6%xZyaolL&l3LZ(3 zv@~xG;`cLNGxu!8nz{0N#gnfIusxP17T2J6E*|&pdl!Le)CTJA0y+n!nig>q(fgng zq_CnzHXuI}9i&rpNA#cL+EM8HaE6B@Gf8Ly%sNeF?=hZ00sA>z!ndhBO~>@=QG#(; zPdjz(;wAT548*s9fes@6mF^?9tz(toU7Vrz80!K0f z303Y*dHD(j?WZ5!_Fv7SrgcHM2?E!C3!b;B_pdIE1utJKquP1bpQrm|q9y8k9i`A5 zHk2YlfKg5N#?e zJhO%gJ-}`7B-#Z_6eL)+`=#SToeFrti2v-{AaJCS5GOakSw`rOyB8 zAh4;Y+u6Cgd_~k#5T1?!!51hMR-i>a?s8x3&tr5Otq_OG42Hxyt>#~(YX^XWijns? z>K!XdTE9GTd!8N#yRl=p;e^2WMmfE^w0DX3k&I)SEyu46=|Bb3;2%08Jygtd=ZSS@ z)=wG&XO;F_j#OU0xGuFFis1x(_z9g~wdG|1KeXVU;thHka&Bo7?%E>;-j0T%`+&lK z_WZe^MCq4LgM)Gx#FuL~Dbe`Kz*GLGL31W(rN7Dg-dQ4O^z_IZsbC)frwBex(el_|Iudfr2k46f4Iv>sV+pm1pvPZ^ z0#W7NFuZ+K<)R~y+?{XFWym9twf1p$b2UI#|gB?6E> z)Af(cFbw}1;BX=WKm&#Q2A)zvc*H?yM5=)T5Q<@k9s$9<$H*3nM2XyOQ?5G>%C7aN z)-S~X6%mh}KS2dP-of14hen*&=Z+;=un-e4+x7R_a45w;}e@fS?$+!rQv z*tr1}55>_TZhq^VKB~}$Zd%fV#36= zCl%AQNOhxf)S|ropJcIJRxaR~BK6aza4H}gYl@_dL z926w*egvYwlJz!=4Swp}J!K}@S+Z?Tc~fCm{I7li>;(n?RSi-b<)O-)R2*Crkb!?*Ra zfldqfg2#2hzfrO}T7PAWAHA)4>{hcb*wZcCf~3JY)p zQrWdTh^dj`GRJ=4EvS==UR6PZ4Z$1c<7w`mG4kb0hYwQvg-JnM>ywmYfNs6;HB=^z z3FAwtN6s!=i9$yxO(BrYJXf+DquCNw$$x#a{A={q{z9on1ooy$9WRbi4N^}-Sw4(% zzi~qdWO`p;lf|(DnbeP}52G$R1I_GwBzdUjst`khu z5-VFaKx2-p|JwTyNQEkPL%qG2uCA_yg+gz|Iupi{xOJeodDeQ!!4;4s=t?UNX94_N zW<5QZTyO2|^?cvui4#4mbA7p|Lkm%6fHwP_%(vP01xV%Q9V^~Em zdUr>jS>-)Df693vpSj9yq&((giS_vR%k^#VDUZ@q@9?SB6R!i_h=W!CqXCHVy3D7U z__VaNB6WhI{;LO?ljWYg3Aa!nl>TCb2M^e$iZpQLV91)^Y|X>G15^bpvY!NCFuu_0 z82Em6cRLVY$u%VTR-zgCCU!7Ac91yH&IjVTA3mHf3uRh1PC2@D^FjNi*ACQmpilsL z*X$OOD&4gst8U|y&Uk{$8vwORuR>e+v&DdCL{)(;Zt*JBP7|djYYIIpH!WuvQb4PB z^hQGvaj|mpP?w|fG$;|+!*c&9@G`8 zpkdILdz#k@2OuRjQL-2kR!VYl{9OzIGeooJeM${s7zL>eSeR@$QbxUN5S_jsmjKWu zQt&SfoI-tH6#b=tY4Q4jvagGCgYJ*n`NC1oZ)e_q+RXU0H~TSFBT;$n?873+zD@TY z1DJI&AQEzAS_>VFEb<<61?6e-*ujN0Y{7u znwK5#ds83T^QOq==7v`1C{S{TuDV&#R7z@j`?IE64 zLc!MklcZ41py%6*5ypD3ml(SqOB?-9%YkV1Q=7*@&7QS3wGjk;7=!rWgLS_;_g8x^ z8vE)|2L;&DD2$(FS^Wb1uEe4An-WkS{+`$}z{I3E(RmXW0g$HR(%!=xUnFZ5q}Q?O ze4oB^-J-Tsg|Ni!;)M^5i!)ZK zYkG}9b>hqazC|&7rujv3#AYBHl1;+W5C1;!@C)C{;xN0ijVdy1_SLnnG-lV)N@Mdl z`47pLJc#)C1toURg(ZxjdOd4oPaLW9!_7sqcJtH@ey|H=cz<$K5TJ`b1UKNs*}@)y z$m;#%1$bXtu9;jCZZZ+$_h?VWyhY=Usqjl%yE>BJTK3-zS-V`5=@6&bnfhj8VmLCr z2(h%sTth0vzz3;3^uMQEqH8UGDdw%DQM^R=*7=yiRa5YmaN(mHDMYVC5+KmFIp7|Y ziYLQu9r^H%6fKif_1L5I><_$P@yRGIji_(=!!H-U&A zEdOO-Q>uy~By4_Y1wa)64{MEM@2~cm|FZSkTd`~Wg1RRt4w@XNs7W@m4ufoNWEfxQ z;wIU`IO7-cV5bM8lsA1GlNfs~wZr%pym;1X!>Jh4+>mb*)Qk0Lq)9r8Q3%Y z^`=dVydjx4bXA4S>Yy&ja}$@1u3VYCdhFv2FjkzFkxZASD_pn1MrzUpw>YTW^eO3R zrA4bB20$D`G56sdQL8LSt7Fg3*cbC>9}Bapxzz{S)NoPI5AW*BvPb7PsYr$f4;~U$ za?;8+2Q<--OEn_qaWaZ-j)r|9l0kR^@q|@8J=7lwOn`hyo`FsmK#F`l-q@PoK zaw3g!`T-@tGZK`Rlu$N6R+Vy>^Jk|KvMEmANyiD|wVEzWniAk(-0y4gQ?}-eYfV^* z21FB>H=q2_B~~zl`o3En2YP^^wzjrY1t|W#zT(YZV5D!&CAoOVf!`Cc^c{@WW2>9D zZ-HGa<5BtX1XIjO+=h*n6_2c}>|0r}Hlqmes3J@RrZK0#aBuxh67Id+j5u!vIJy@e zjhv({vyuQh^zxSiDPU+}&4KV?{_+jx+Hy<-OUZS>u{0@?&jW~)0aK810tDb#j$-op zXU^W*T^})hR4nV6RQI(m0*B{PQYjfv~Ta`F&ylQR0|v~tOK#&yiZThO-_~^dR0fmM=*If zPlk;ESJY;#C%N}9NLaHV+9NM2l9v6RF-ps&uMN<3KBDMS#HpZN@8sig3_v?S>Y@Sy zX1anWn$-K@YlrE85(h>PD71j-tY9)TU5lT0M-r;seO%FWJmLyJ;u7?+{Eg?p?7X}c z!z?uML1q)C5h%RMaHz@c#VTQxnzLfMWsb`N@iE5m_falSSH0NkH%t!o@*sszk^(vdu| z7hdh70}0uRyRzS`oXFR4#vbF2Dpk$qJ9G10UMDYa$^4fvwpKt47R%q_ITx#1=-?qr(8IcS!C`zz6E&{RclL;p4j9jWo zlASUe4*d?6b~NnsQ!!`16(ZJHmw5CrdO!ZC>k{4mcWrksqCU-<GeC2FtP?q{G{$9=m+xRCer86oLqsa^3x+=W@ZHbSih$F7ZS}TZ#^lLh;D((9j2@4i- z+Bh3Bj4E#V%c~adtQyXJA+qtZ z$6ZLZOclI;{t3{$uUc4)cH+Eahbl~t9j&CvZVV>y9QWVM0o+pRjZ)7!J>o&G_K%FZ zfUb}Cc=NLK9fPO2aOl9qmw@;ex$24@p}D6KrQLCtZa&6{H>w7x&ZmQYG>9~|6k7&C;UY>6*f&4XCTpD%zobk zCUbocH-r>@Hk^L0@AO17u{t`fu^If$BDp6=5V9@a5ldo8AQnhDdCVjo@>dS2>um2T zE0z0gaQeNNVT!>CAZ0zCQ~=iRKItP57E>MP-e~hZGgdvgUQqi}WN|;bWt`b>dZ^)f z?$PEI@AGcrrq)3|!k$G|HG?dNpbY1O6B@Dq`9Mh$GPr*huqZLa({Jg}c)>9IV}2qU zP`bm{IJDCFv>g1)<61~KHf?%U`D@{c*>c@Oe^ax5 zPWg#^yiW06smqWAumMUZ){s=Q8u!L>`-~<7gpulTZ;5&U_~78~cDxp*M@C0sVn^ z8UD)rqwBfG$PsYn$i(NfPlKXngy(dA4*^-6m`E0WqR|Iw4bGz&#=&(T#Toi~dwaZW z2lmY}N%Nnrm!b)V+>+_^r0gAG0+`=zj^-wCpZ`8b77>M!PE7|YMDBZsdsSY#_Zdio z^DPrlS1eT2L4;KZ!#gBm>)kg&j~b?H5|_=AhBQUj{m_Z{CF$*TIn}agjFkLjwfTJC-fbGzDx)Qc?LF378uX zN%((w1z*anu!nXMch3UJp~({5Ywwe1bWav#fn$%}3Y(yrThosKAa=HriOG+$0j-SW`cstMab6~D$ z*OMO~nD=|E@K9^e9&i`G`7UskeRH^{>5E!CiL^ot*a8*~LON53L_y{NIZgP@>61~A z5Un!ODnC@Tz6J;-zTfJP#b*;%3J_CtS5ObH1I+oKvukteB<{N{+17O-=y1n-mtKBu zN^V~)DRlqSRm6cm#+cb`sGWq~FM#5ou8EV3^!C4<2Uq|}bddoejqrPys)Zc>1cy69 zB$MAec*A)?>C1ufnUa*20&VzVDvY5!%?B&THj47!dYU*MthAY52unpE{oMKmcSrjT z*5i54ynOLP6qR=D`IH)%nl`d-8?n*1Y)c~#O7r8?y@A~wLD6q3V#JJOleE^IA1tzG zB2nSF41FM+8uy90%dm^g87B$dL0S1Bs?#np^?g_qFJxb(grJCtN}#;XMtceKVu!pc zP-Eq(Yn21G{L#CPYzY0g+jc*PDJJs`)~Dh&u8CZ|+|LadbjQ4o5!vMeoa ziEjFvl+YCtvB&DW+D6wxe28FY3Aw6J49Zx7!c3ax04`!{>(-RMXy8*&*E4(Gcu0>` zQeVqs3QNHq>EW^*vsC2*6*kZ8xf+ZGEZ>oz+f+Neo3J!;qCHFc&K$}h-qsM|!S7+S zwIJyjql(4NwnY%$P2$VX&JgJb?094watTe6^D4%i#U zf4k;#ZU%m-JR9g~q5@wrpofaa6raeE<>OUIs^9*WY;yZ>CsS#wxxjQ|~p1zhz(YNO-4{2aY~iF5@SEflg~s6QB$4#iHMhj+c@#}V zC&#@n3L&wo<~;!0lr~sym3GU2R2%FghaSN@Kl$3Yq=C$vgN=J*| z$P>#P-ZS}sjbxt#cI*Z2P7$>EtjMeySZBOux^OIBRMkoKu^)#Jy`AQ5b%In1j+FD{fF~z2Os$FE;y>` zq}%dS5G>K^5@fkGoH!$#4X=HB!ZSb(+fO;^y^sl9TFJ)G7IfE_Y)6r2joq<=|e z;|RZa0`RD^|9Ip?_-zpZ%|*CM`fpeN_i~S5zqmD{VuFpqVrAZ@&J(TYTb=iV>$|Hn zu+NXyd%leZxma$81ps%@OgM@U%rUw1B@LNN)$_#?7G&GCz-4o})P_vuXn(ytcz&NHRbKoUn&`^PM}w|sJ7hR5s)*36nTZ@d4+N6Fq-!g1!!nV z7!Y~dq(p6llhFV10%Yh*n(pv|UV77;__zeL{q0z;HVi);jOX}lhR2GeUTre~TyAP6 zAHPwJUT=?;*4`q4O(#pP>?;kyGvxIwblYV5gH)xhn8|%+E++G@Y8@Gw7rN_^##@;O&oa);pO?%&$^xfHz@$=M4N(L3m}Aw8v?Q77xIWEA8x(iM%vDHXC&+$ ze&^cil&Ja0!YQbq?$4I?8dbg6^WRVX&jri7_iFqHO7$q$lt+PL+VVfcafpSz=emiuM7FBx#Y4-djs8}#U zXA3xOtnOzQlM3#N^um0B9|yYBED1&fL%;1~Yoqm(W;l;tyG8_HF5Ii^OYS30$i0m`iu|g0*3N(D|CM+d5&&2 z@Lx#)NtH+10S->@Nqi9g`1YYFTET~Z&b_1oCc$@DQnz(^-8Qp~D%}}nPJVyB>GDYO zCSxHFbD;(7TZ;-UJ($E~=TDl;AZe`V^hbr$D`dX??6&Dq1XFh^vulgBP*{)G>+y_9 z1ahuSosdU+#|<$HETecuR}qp)>7H$v@!zEr0>ybB9sH8q{nD{kJe~spo zJ0L<6S_g2YdxKVj_)Ok6gJ!v?!&P~D*TsvE(hbj!Ob>Bh^*3{D*E{O5szK-2GXKmkS{xo|-KHWYzl z$UwK;McCsdkU-LUh^amkZ_5#a%1Nv~^Ss!eRTQ1C&|}F-qc4jz*_hPH$`K>*v9M~1`Ai^o^LonWWP(QV30eBD9y>`n*dAyxE zoRH)a0GKgX$47sgsb3Gf-YFoR9PvI9r#Z4BrK0S84mix@KREV<7Aa(L1Y3?QRL&LQ znTs|{`GV*UH<)E|qrZZWN7C%p;^?mkah5=F93YtJ>FEc?#y)o1l32N8U+xUvu_eGo zv?x?5)IkFAf3aoxPfy}!1ahXkV9p7MPkg8|8ag#45hi@Y(dORYf;{kQJpmi~^qjf+Zw+E49shtz;ofUR5hJo3{;O}uZL6Qc)&3*&;IWP`El@? zTZ=jGlZCuy6LdvV3XD#P-OgG;?bq6@}qZ_8{hPx2Ke z#UOTN5kAN;@^oFnE`^tuEs<#Txf>d~rG{W%(VU1}$6!e8dcNBnv8o45l>p$s{n$Fa z+VslY^#-shI*nGi#rm<4&iTte3zo!~i^;tP1~biP+oh7= zLE_;1)bZd!8hcx$;MG(K85aU;FWit~`NlZ~ZB4PP^#uoGg98>* zLqC$Ei1DbF@*9_Po^VIboqf*4lbSF#Ha79gIA8_HTb*uCR=&}Mr?{;d!=FQs40BqY zY9%p?Z6yw#nglp@jI;CwX%C-|qRH>Fe+SwA9YL=g0 zD!b+r3tua{+WElf>eXkQ7&gxBN2)>oD`anOLy%#FN=aEcxtmrIdcYrocg6z0z-1iK zzy{m}EQ&V@+bBsAJ=m{*>C5;{7Gos}&f)~BE(CzQ`5gzKxdi%!0ATj<{TCUC!!<~V z9Kd7UZivBoz&uvhF3wWq8igUiMZ05!Gur6rnrt2y~b49TGJ*Gw8EV( zwbA;ZntK+?{Ah%yXKSjAWBoemp!Ebv!nkx_5uAEuWl&%y#R>Z<+Kt3a{eCHtT{3y~ z>{%oj#UXPFVK;FwUcB+%-sZexePm_c?z)B=U#%O){RA=EcY@L;D4=KPV2ZsVTSu~P zV64WSm8S5<;je{Z<|+d5zEAee3^4gxxoK=LJxG8F25ub^P# zT*&HNWEfne1Heco627$6m`Z=sCBo-pU?iew*8e2hMk##Ix95&JVflp|IRQ>$#b8>X zUf9gpa=`XUet_V+Q9R=S59UIVIHT4v?B5)s&dt5%cupBbT8Xn*-uu! z8+4&Z zS?7>gHp$wB*PTv4Gb{I;)=HDI#is_@6rEfFmgi4V^jwOSXG=L&0lD~i$m5zuZzSvD z8y<%ts-Kcp>`L*a7Zf%PPApZ%j$4mpUiB92-WkGTYP_x(slc5U!m zhG3d_H;%SW+`tv2?|si)&{ioRJBD>YlJ?Rc>%!2`3`)>~-=_fxi8`;FKXZdI)<%xn@TfIzB-_DxK5^QfyHfy#7f`OF-PGdy{0 z96XM!*!kb=NMSCw;`ge?@5-#fhS zl8WCRdE;^&FivCvxtX_J9StP~-Th(6ujFZePe!YZ3D5g~Bv!qDoBj%@dgpqi=!)ad zyzc1q*rkH03PpS=06w4J{W2`{CCNcP8#VS4P)kGG z2^^>J;N+R`e!~%`AaHZ|Le0)h zfwg-2W7eE215Nv*J;7UdT5i-|T8~T&*&j6vo?>=0nnL++DDOku#@+**>OYDf=el%- zxHcKrB_r8nWnVL-l$F__QuZu+b&-`Kva>2PqHG~jR)jLMi|mZ7|GDY=oB#jw{Chke zPvw5zpZ7TLb6)3lUI&VGax&gvQuU>)8s7|y5$3kU9BtDQ!LSbJkM_Xd-Q7K|U(LBH zJqcBh&5DdM1g)c5{9G%p`PX@`%Q}hNz%8<2E~YzAivC5}a_h4rEfbsP_C+pSb>d?E znR}-PIJPCpRcgtovN-ZmXL4j86v_l{#&G0#ZY(Ly4a|980B36rM ziz_p5Oc8ECp;v5GPh|)YB?TDj$!hO4+grD9PgN8XsNw?W)2U&>HZz6}DK8K}O1l5N z$g1S$6E~XgU)8h=RZ|5emfTsD;h;ej+P?ALZr z%;z6gdnbP{3`q}5id6@pYBpz;dSAXY>e(r-eQ$*;Vt3jcQR1IdB?yg;3j&(hsP|pd zi4kGPJuVYRfKZE%62|wjg<+uB%HL_E$~_*=$M&Y$_z_$PRt@MA*3x`K9*}NRdV~#X z+Pz4FzQ#Ptmn9uOM}sc3)R+9jlU3DoxuUzF=UJpVOHTTtogVTmuj-FulRDA_BGf0I ztUUajI=203t-9t1}%r32#&6($+pdoM1%B*=9g-1#A;BR%TH4PbD`?lx$1 zXn#x;$ph_>ISyoLaq*HKd zY!nvf#fMRbb69+m;R}>b_fL!79Y0c&W7BII$#jnBBu$$Ncb z#RpuZ8;OXZn*Ni>wsh^G{uP10l7GW|qaCw8i>%Wp9so(v+? z`2yv*noFe{h&f+GqEu?)7Ekm1aQoBay*ATApejj}S@&eAum71Ze(HFzt&&RLQe^o89a)Jg2wc>c6WgyfxFh!jEoQWBzj- zor=TN?^0*=c4Gcq8&nSg2H?fzDmU6(9daE{z`E4bqpc!-RgT`B50Tcha`!1d>^wWk#LXSrtumq;sGHmSz$?aBw1c|>zUR=3~4IVvVTe2x#d zPUOi2njq~u(eEK8l7+TovW3D+QhW|2I;QnVx_iIrd~4KkPqB!TGrU`sc4?Ud9KMhA zx7YKYufBMxqY`CL&%6A-*k4ccV7`9t(KoXfHGgh}D*PUKT};8XxHj@4Smld4l7ZQM zCG**{t3#D_zYDuoskaOF;U}Rgko36u<8w~Abt6JzN7>KTzaf_TO#dR~#(kE&$~~Xd zOZ>6Nrdh|5_b^?LO=6*%}#y9F^gTnCH^|)euR&{FuXyX zVT>C)TJ=s`DKC`sXjRFCA#Uss`uzKk>IV=>C=#}LTER2CH3Q`o<8gsgPK*!S@9C8$ zk+r0WgyI2~Gi2Uv(^ULCyD$t=Wi7CHO=`e8|s^VNcq=#`Dd5!=i`lcmx1p39Co z9C>N1{m(UAEQ^O-saF!h3^ikN71IxVcvz;Mth;a_2R&VBobV9#WX;4VfyAc{oY+gxP-Kz(d?T>M%a%^Ft_=V`ySr(;i^Icp&NjUM0| z23CSAfeSf*+~{Ss{^Ss6`yw3S{#v-MDOO!IbYN&~Ry14_lMy>h_4Rr2`0;X& zg8BRNw+HQS!2v%j{I=Qrs!PYpNjx)j6Y8uwvG0I0DD`kt*7fM?-yM8P0hh;&7_nHHoU4HSW`Arl z69Y|*QV;7AMdT_;)Fl+4F!G%JYI{AZAHn91HMUz& z(bc7z469UqsfIw>ETBJ^LYxqQKv?qHXw>O8m7 zW`iqvN*ZiN5+0qyISqaux79m5&kqa(&1ltqCV~eYU^=RhI?cYLSq6GR4Gxx#hsCNxN zzvE5W?_VfzAz6+GZRQZ!h-V6WX3cdP2shFe#k5YY#Bpotr$N*`49ba#q6lGf`9QI& z@4F{sl=LrG+|KNIrg`7D>PdQ31H=F~!EN2`H3gNdo3Q}^2Wcx=4}G|{O4%g(>7qYZ z^WepkzjYdfNpC6M!Ui#%wxfQ)F250ye8eZZ_C)^gUp2M7L2PP_^HcMYh2hpB-8Y!f z1z)N93nh%4xyCB8%t1oPzTr(CI{Rr~bQ+LEkVUId!qX?QF_8urHT6$h;U&qZ6P z!I&$i&4~gSTJFqAS=d_vH-HTVQVZjiQ8$*Dm5C%p$JG=PL33wFNCpgW$ElYvP^x`Ybnt~4%`#c>Kk3tPaJEUKh0#S;b1}B_P047tiGct7)KYHx-QJ# zUJE;3W#+spM6C>9QK9Gv<+GNnMCg~#pPO=rys0QHje8$;oGIXl=yN&Zef`ymNRecz zL9%00sUk92xQ17uUYF%kJ6EQgII%S7T_BLK(%>WT$~Xqr5H)Y6!aF?PLpaa-O&j;& z4YR!Fc(7XELpMF!Uo@%f^%uswLQY|Sv;Lgh-tSIqkcIhR?bA8=gpI{h8YNGY;=l8W#A+ek>>>{D+4HuTyp04-j^$nNwj8!atSWP zUr2cu5qQH&E_HGLV_vHth(Iulh7H_T#^FLJ*2D4N()QuGOA5Y!)=3>fP^SSzWRHu~ zu3u<<)tv}5_=5V~X_04yZ~k`(DZ|r9Z~`Qk7r+TXnDGJCdH-!~pG22i2Yrbx_6--- zDHma7vdJtB`4aQO7~5v?6o5w2)n`z+GZ%77%{hya#lzG$CbElNwOTupC_XNz~~w zP}l-1C7d(@Kbi1f#ElD~4fAMy_tz}SPPm-X1ZMDX-ZBzdfFuyePBBWf+}O7Wg-k05 zBm76@>$t-~DE{Q-*MamUmb$u~P<>>sHEs}2U}a_p5d0qqjB9=nJC0a82XHQ2FGTHc zLtV`$f2Flg5^jy8Ox=)Pl(WYcSk@*VW0!kWsTkxdAKbe^<~PR)DOYEz2=z$D9@j2~ z0Rf~I*A>9GXFO2V1{hM*M(Um*;HLNgq{_#H0|$~o`~_q1Nv7l|6O*}9fnpH7;88d` zG?ioelGjkU5ZWas^BF=m$=8yZpv()>1cC<~@|H3kcGTATM+kWVwMR9-ouT@#927W* zIK9RLA1CR~E9|7yugHURh9Ly^ZGYZ3VEq`{D=^z`0J4COO5k=a}Cz4_tc ziE^j!@92(RZ+{oTWa*XpHK_ghlU!e&w>T2Tm`J15*_4~2_Wh?O3J{FwFsRxQa9Z|0 zCB7`?{0(j2azM2QfjK-VTHF=dhd-Bo#H z0@&pydC$qVFM*9!Q5oIuto5-7J0`8nE06_AP311Q680MPY(bby9ciyg#ZpqUKShV0 znoSXCaNJAfn3Gk@eDp2?+*i9ClSZZL-~B}+EGf6!e^LJaP0C>0>_H~vV8f6CBqmq- z?omjzBqxJp(9Q8-&mBds=7AB;ie+?MxPgpd3X2N?&*d0L=@px!GofJQecO*m`le*! zbb<9#K08BW9z_b)k-_nJObfzvCqM#YBH4BVJ6I_{D8X>_s4rA{9ErQbsTqZp+jE=k zkAcwXGj)^9OvUS#XYu%ecM+taOneLkj(((u(I_cM(Gn=-J)@;(qHXF~q0)`xd(iCW zm?~a`-lyUSA3=}uWLRH3K(-2#WX0!r45ESD${R$_ILPWKFgH+~{&pmQ8nWi*266LP z8Ah@4Hz_Y$xG88)T}<)c-<~ldE2_n)hd_M!+_1T1K)IyfF~6WdvQmu@l+GCK9c<+O z#|5|#bvFq{bdBGV<0MFB&IVjQRcNU#r-TPv`@q%>3B^qgvo%M~pFV7g2{ z$Ra;>vb4Bg?Cj4@j6Jr^t4){KVb27FJw*zokF_tFTE zmpo-v=QAl-T=NQ(Psr4sp%5YKC#ZssV9ESl(WqcNMqap0i0+Ir9yMJILXFcB586YI=jmBj&dk*W7H1aB%ufuy0~5jI692E6PE-ksFY-%Rx^#a3Prdrrd`Bb@Q_@ z%3MMaf59Q=HxTn~DHXD9$EY`i=s^(D1d7EWcYFUTQbPsEjKQDO8NzQo>$0`cmw;?> zhH1kY7tep`I1XPG(4kpx)2(a*s^$Nhg+D=KpoinPIr^rV8)oGU;&`16l!3AtHi_EO z^oXZ-yi2i#QQTk@jTyE80Txq7fIl?@#7?2;2}B36a;Ktp`1gFaRBK;Q@;l`pRq(9k zSxlo26dFJr{O2Yx96ItNB93wjH9c}5P--jj-jj}J&r)=xDLg$gmY|*;z2s{wK&4*i*6=>5x6701Q=o8l`9Z)i5Stn$nwgoAM+xFkPDV0@ zcMyJUMnx44H2`E74>Sq zY59AT-go8b55G>tf5^U2Dm1_Xdxf_7H)dQBa95k<{8I@NgcL}iDynUX50c55p{~je zYJ`OfiOlppK@jx&nrVU=+hyKm8N~5M%=MwnxaKt``8AdV+deV%5b;LgZks- zsmm1)P6`Tkn8?0!=zfC_;qddpJo)psU9xQf&5spd88 z8)jX~9BH_P?zU2o`YS^vMrL*2zN!)d3^nC!FTBH1n7%^dqox+T-JK>9i5Wl5;fd;n?~e#WL$c~h&ThZGzrOSm$O7xl@O`_`DCZ4wyWGpy?Egwlw+*P zszjpnO^@hb39dZ9X6J@S?K>R{gwkV0(l~huLRK8gEd%TwpwNGY*sW)eMREjrmSj1|i?!B=RoI#^(uMZuodlV%$D&$lPS)iI^y;n3JguoV~xG z5=@qAaHBh!7%a8l-XHhpm#vw&ynd#fXb7cY5;h-v>kCNqVuX_1b?g*tmo^(g`$thR z217`v3CxA#vf}WpTZI!qe)f5FRHBMNd9vW1O&X(8#fPRvjdnB~vbl6_uH?bV2%6Gm*>G zt*tV&@or!u7EvLAthTgaM`jV!+#WrPc^jIDRD$B;X% z?yPU-)jSnRk57LtfKuwH<3$MKViYLkY(X97PG4f(dX5A^@9Z(l1CTMI(MJ@Mb49;? zyt)m{dI_f|hKT_uv8s1j_k8;?!ndS;$Oz?G9RpE2E1mpsQQs=3|d34JNI+Yb=u%IYoH5Ti<8;@VDiZa$%+J2!PrSg&nKD&e2)AZ*iW7Sz0)#weGTW6RQLmiR zYyT(>$$=lv?(6hN78kPG&r#GLOz`9;vI7#~);FY~Y}(>ME6m2rKIut;bnq<><;0Oh zY!J;v#Kn`Zd#1lvv%>(=%WclyB&<`g(pjaYdmMWTEbFJq+l_hDw|`NButBi@ThlUx z7jYY9066P4t!a_JKXgrJ#SNHl_R2@`ZLXb5m3vG^$!u+_ID{g_%)2GpIvhP9lwp%mu8vg|zZ|qx?FL|1U!bx}mGkGg+UcVN9M? zyzjr!iPVkwdF#NDDk~%7Hmx!vIlov2r2<`iMmhQDjIx=X~Pq5;+PU0tmhWZXj{=Kr|E5pDY=%IT)N zlnzzKk$_u7tsMJj*0G2H>Jvuyei54vS~)e6_M?L#o-`9Wn0gbu`xeMZMtVpz8d%XD z&HBg_j540s{B(KG*#+$A(b&hrWXD~> z_Nk7tB}A538W}OJeRa9Zql~bmGpYAefcH7lBx(dDy+5uL{~EX+BahzA*0ajwr8$jm zJU0LJWzTj#!u%G3o7jh{A#mbo3cXQA?oE!oXoLn8B$)usUs)X!ZjxOkM&Tp`*v zTBdY^3J$}fy`r{Bx!za$HDpAikse85VnurkU#Z_ayQI*Od7iyvC!6sw3K=+dAKd*@C#-T|RBIHKO6E zhWqcAR!?R=A6uL8rG1&?EY3rtf(V5_q@<)z&nf)ANa$dMP9av~C8kB81IwtIyxP0G z_~G7oTBoWPfK|=!Z5oGzNBF{=Mn*(N?p!oBE{-p3yI>N>L>@~-KBYW<0iwCsCa(cC z0qh%%_r%U8U{pm((=}QdVGTEk6o^hCmJx5^zh<9kV2nxTjLyo+iqz}%F)@?ZiPW^@ zi6tkl@cI;T!E#jMz378bzs?kbrTZb+2b(=aPJuESS%X=&BO)R8;cPf5Liw4Dy!jDq ztsD9U3pknif-(m}TZ`=%qRF<2lAI)ng-)7ao}WqY++b9iF+ci@1pbe=X`n#RqiZs> zj&CamK*_V|X}z^XBEQ98!74mCaRf?CJA1SDO{?gG@1sxk9OM4)v4|2WA0e_Qs`(zO z@(mOCgw{GzlatmydFBF<@%DIx@D;+MUwx=WomUE?7`jQ~*T;Zan?4ABVGBocQ7^qZ zbol8#HwU{CwxjU$fr{f}~T~2L)9HN1l*-1ZXkQ-6-BUMH> zihN$Dy9=XNO~;aHA;9p!S1Ffu@(eFg`2YMpkS@cJnh`IYXfm6*D?TkMXf=y!=Gf1`6;8Ydt+>*s*m)w8%SX`gsoSMXz{$T zLJw?5nO`5k7C$eshQpp$beU?qT>N$>IV{2f7?)`D2m&~@DrdTFQ zF$Y-^FQ^(gZ(%X=66l}pG0>i;k-B{yO%5}LjpHTLM!bdlBbGuTx`Pk?O47Ev1pZ`X zSi1tYwE}%wZm#2cZMY4ju9BS|b%|s!T3Z}(Rvjqfm-%x;$fjNRzGuAlM~?w6HK#>K z|CcS)QDo|KmMwhKqT(SK%L3n2vyx@?0Xrak=XhSM74@a}tg{Z2UF@|^7Kxc%7nG2uV0 zN(vBIc0d==Y;d2?6Z9_axM3FvlFB#mU;<@~2dkpFwVn7R(V% zxTKiYzYm7iQE(TtHH>^txYL;;8<$2A+?CR>QN8p&49)G-&S0Fev^t$c{b{AMFp}~6 zkDg~M9ri_yUpGROK*m{3@<2)$}>E@`k8*;kHpuZoZyRgQS z&`R0FJT+hr#zj5ZPxTyZ;W@)*B5jN@>@1fPZ7Zcba4fxu^!lba{C8I*pryozvtXx6 zu0v5!OtZ^pQ{8q^%5$Bppo(_0(FmK$lC4Hkyo;gfyQ0Wnza96*pktM#NwjX>7)OuZ z#p)?#5^Vjtz?A@=SV4lle2i&reF!Gw!Yf~oQ+%@{*)z;xTvyI^8r#^|eBa$(ThivC zwUfN_P15e3Bn=fpMQA2?^>|c4^T~E&s;1bDnmTjvYZ`U=&ST(nkD6Y205-sshM&ku zAYjZ+EJ1CR;EO$YxR8X$jOz9dlZ`)R@t)Bu{Wri;Ewggo(t2ZPn+QkEa+CXZ#Q(W@ z0zZ-ON(a0oGDl>vD%%kqeDIN3`osjYBi>#<0MZY_6`gGojuE7R8H{Tx+fnMDv6AHO zk|&dN|GcW(Y|#$clIT0b3uB9k?|L4u?1S>5XgEh?PMkp;U|aD#6*WGxrTbW7C4w+Y zt-C^4A97_6@|AYF!#91(cay3^QUDvm%RtF9M%W*f4_xcKwJ=3O9My;|bihmEayfI1 z$d|K-2$c6$x1IQPG#Y>i$$MX4U4QMsDQZmwrAT;3&a%;18r?`#oc8!xEMV*CbdXBvCX@I6YJuH1#>m3iAQ2}F#01bW*m&i zG!R>4MBAG+b&omJL-2}ks+#oqu=p_uoUVeHbCp{DfL{s@*((`C>{K$Uy%uJIM%@zQ zd^BcqLIe>nAS6U!r5PMrn+tLcH){E@H(Jh4p2xzIbiQ6G;E81jl)WCuGoiMN2_-3CFAm%cqwJ`iv;+KBN z-p=M#kX%o&dQ0HBKsO3?;M!qw^;09D2FgSU;(|L-BHLxYiW^;z#-LdA^5nJ6CMXFP z^Q+dOrhHsqqZ!>S3yoFIgEM#hr!xC_9Ex5kbyfll#&(KtC-nGh&TQdU z4>{v)vX^3e60IUs@+D}rZrrH!YYWlrdDQEz1lKALVwEU{YKX}za3rtij5h@WuK*LU zfA$sL|Id#+!w8f~ei~tH(eA#J)0cYim518Y55K zyua3^6a$2@zSZvq;RXYXq+g?~>zi}GZDT6b>7BlnwF*x@-h(2T13w@kCQc_8O-xKE zf++nw?h(yc&FXz+19?Qy>Qkj8s#e{PA3qXqG;L91z%IX$I!K1JO;WOEe7+`0@P^?k zmjb85#3YVdMIZS&9Ca%&b@5L9-FqTd=D*abzr4GQ3z6uQnZ!%)&=#QRA!*t}`lhrd=**Q-Kt zf9t?!;{#vq-M!t}8(&{0I%Zzddm82`?f8D6$}P#`^Ub_V+cPpz+`XcT3!-+NKQksX z{An?P1%wF5XmIsjtObtddj@0*rCR8(pC5q)qgge1_OYNn7vW{YaFoKm4c)IlRB%Mk zqnuh8jK@QnEK7Drebzeqj{(d7wDt0Mqmgg$A85Qn@s1<$RpBk+K~Jm}71`SbY{+2# zvyC9W5f%tGsEB4P!1E|5jkT-y_-kdK+WnyNsiH?C&JmL=M>brZd|xOYxVE>dsg#7y zCnLLnYkR=36u}nwrou}%5r2?yCyGdz*nmpeNbexryq~$m1EG&rsrV*OA=oZPWT!`D zCq(kvwNV&5NJ_MW>#Kc%&ph>$;JTm1#kTB264Y))Se|yO%*~x)N-Lx`KkLEyR?$8& zifXIZbSmt2CH@Hxvmt$cPeg6ry@Nt$PPxRmrpH=5Q&cu@Q>=sq1wS^@-q(myHi+EL z*wqkHIv+E@3!-6aeeoZ*=6N%4A@A){R4#I14EDa-ZLQ}dUdvH|+O2JClo^~v`Cvv- z$>cHQg>aNAPH4FI&hM^^I42KnnjsxriGQwNuE&ucp__$Da59N~&qIj0a6I=paCW$75hY2wc&sY#Bd-`nxckbjJHGL&hZOWU|L6B#!(rmI3XT{| z+!jjeu^y!di0SWD{o|Ut^j02algv=eGB_D3Ns1RBz@;fV5)4WdtBp4fgR%+D3~*tc z7Kh)0V;R|X`&+r~n&vKVI}5uC=I=yqqmZiO-c6!+|9nrv25~<60l$t+18a)bqE$O| zzwl>>dnMC}#y<|dYio+=`S%JM&m%uy+`rDO!K8%9{p+>C(*y0m93me!)3~09bp)E% zbj!;2*>QOy4I(eHhR?e#8Fn+dH;sswr|)E@v2@=${qM7mYp}&1C%)MwXe^%8SNDEk z5hG2{#%_l=(@9gbZ~=!hR%DK11lK=W43t6!ByWK^jPDn7I?$&vy*eTIf)9T zsxp4i&>r5TG`W}!uJbs&Z-lH07hId>UBv1kPSe3XH@F};H7ogigOWT2kwG0`Or#}E zY{@|@s<(EPC`v(6+YtTF_d#oEFQ{hYX{sN&s6L11p$1&xND!WW7?7=S871OLm8WcW7x@m|h3Uz0rND;|X??c7-_I{9a9v&|vLHMoUii)cAs%|eK@)-c z-hSj6M1bJe<2ea=ruKr z?2cW-zq3j8H>gzJh0)=y^d(ZyAKnZCOrx{xu=b;+**P?Hgxxd+!#|w;+bj8hXP@Zu znOj=7oSmK1fByXG?&9z^D@!%_NHV3wKaa;%c!Kq~ddM5wgtZsye@Bcu9C42-z&qvE zIX%qaNRv@%=ioDlZ-wZRy?#R7?ig}`?x}$Qwwd>Fik1b^EfNKC63rlnMNma;<5Qh^ z1(jC1DKb6DAEAs`3|eRBpDeJ{h5jK-^f$Su2%+m46-p6F_~C{YsXCI%fcu*G-~G+~ zD0whM2hDZ*`4o;Hdu@~Qlv#NjYd#XD@vHL`KdloDgy)L1Jh3bUr};tr%W8h?n|BP= zPY)vk5Ap|{Aw@u}tgLFfZlgXDa#Kl&coC;{;3T7t+(`x4e@Y*@ZyEPDM2}T&CQPWw zyEDh6|2|ydy{2mt-J5Ts530rM6{^7$U2Bbcb_O&47+(VN0D!2GU9NOc2VAcNn9`E= z{gj=Plp8xdyF(7uR7@6b=R0(#_5}=cP5di{;*11<_S(=;6&Nb?9~U5yMC@*P-}RS! zZ|Pd>i)+JHS6y_|&ae>NFvW@`9vZZ=5B2fWloU+D1q`0gG@?!gT7w%WWt+40xyYj0 zE2S~mE}Hj0^oAeIPY{;#SpA`WIrh_=rR6t}or~_x(4*bxa`(!SHK1#Kd0V3EOfM}G z$3Iwc=Xj^we1REwFf`fG0afQhj-W}iivmuqs`Hmn(7YfE9E_LHfI8D+SxbyODX8yV z{lXs`TEEwh!_kbnge#q318*TW!FM~Qm1T|^`l7Z*2?6hm_R4? z>Oa3-jCpiUDsUyt=F`fz{9X=qBrM4o6S}dE$oSYVdGvxjabK2INC4oYS=rgm-#%m@ zNm)cx8{l#;NF|r|SSazJP8<{R+nr^mNtjRnYwFo76!AwFg5n0w6(_Ve1)Sh%?Y+SRnqe z7zF*+`q}Q6`1e=bx!iWe5Bl`>3zO~?k*o-4(mK?vyiew)aKIsFO(Iac>RiNDs1o^L zktaWR$dH4nDu%Rry)|SF?8%>cZ{x|4e-8`Z5WU%*3q4<+E^1%Eb?ZIZcpZsq50v|s z;c=p`l1?Sc5F6ra7oc!s7Gb4WwsbM}55{5z9oaqRAa(Kyy$uN{s-2vu7tl2FhUg&(TEyt&WR^hYwRumt=B!$Ki#bQGeEj*H@ zKMJscd*Ay)5JPcqyO>`75i80?$fAngcW2|Z2d#`fTiYH8^)%tp+~DQ*;S55VQ?G=& zKK$`h1h`V8-#Rjm3V!nxbK*nzn`$*M5khhbTe)yzIEj6aaJ`a_2vN0o?_?^mf2PIx zLG)<33O$2%c?kocPcN6j&?3m>Sztt-m7E?<9F9Fh*gS=ExG+wd|qOw6!p46 z2@*SEz@3_Ry%*PV(e%>0U?1L8j_k#tKBKmaxRTcPMatdULt}~w^r8{fbHxUz6=rx% zGaNp&Y)6%Dr%e5L>v zH^&02lX*?-;$L(nu#Ld`Ym#^WDLD--s(y;z=+0jO~SX-fwgO%VKw?nd`m%qroIVy{G$*kL<#}4c6zzGt!s5R=nikp8+L-p zJ%)c{hSz7ohL|IIedN`V9RW8LDG7#|i{b&-QG#J^qy7If%s&YMs(IIu|LdkLX?Yfw z4O^T@N|pD}AgewP*UP6zUy0Yz=ZVG!l?Op9O>U@-$>PqPy~KAP!pzBvA%~_Y=}|mf zA3LiUK-@?rQV%iIDFN8>-pN}{y~qBB&AXi*qGfO#<)K^Y{tt+*4xU~U_YUslhlBTb zyDMMx#2VnngZ2KICZ6#Ga3d?5C*1x2`or%9@5Lo|3;}IG&$IbIE6I<|_Lq4BJT#fZ z`<{nJ1g2%;@wv(}a>8S+Dt)}_?U18Vl#{rwf*dg%T1DFC+vX+Qe-Uw=@6HwIO_i`j zO2KGyz%F4+S>Z5pYtLWyWJWZ1Tadztoy!vNiN&tB)7uXR5R-IgYt#8arNgH=ncxUJ zu<-p`kbhS75Fd=;AM6w{-Qd$tL9y5=b!IGX{^IwqB8 zaGIE{4A6w z`j9?9IsrOsA>!D;H*-T-@kE@EkS*-5Mpl+*hoenuZWig@q-?s8dC%-{Rg@uK?K)Ft zn-_p0J}&K0KDrFbOl=W-78WEsVu-I&7;FHZ8hbilus0b6P(%|%v-U{T>=xuM^U6gw zpUV(x9H1nU6Pk4uF$71qMu1@Ki*gomSXtEsnlL&u_(3u|7o6?ojUh+>srBA-OMT@x z4XHS8h=2O$=6-sM_rAS!Jgi4|lT83S$E&*&h(~R}eJB&8@B}cP%koGZy-CGw(?S`+ zlP9}#&L%!$ryU0Fe`KSuB|ov+zP#I9cR4%rf+o-gt2j>_E(D4Y&!QeS&|p;>5*cH* zE9g|j9g}Cy5>=^B2w2Cu(+(=G-M;9*B4Cyxi{x|T4A8i9?&aG4-+*U&l8!R4y}do= z?AfzN97bzk@1tPdv(FB>({`UXIsXOY6oKkYMV%CtwX_}+vR3*@FyuN0tsK`LymB1- zVV$16C!^*okJi!Ai7qL=4Y(VW{+CS{YKpljUVy3poO9<8Cgvjs_T~JM=iiiX&1Seh zKy8Ei$7@TXwi6;GcQ+Qo(+_rMV@}`vJoT=`5b`b8iLjrSBV+;-bN!ES)^404BB5fH zQV*U?e)i=ncTd)qrcCbdIDx}MjL(yP_5I~Y?}uv_=H;wjuC#r&8?U2ThUiz0Wc5Uo zACEOuq8Y8IKbN*%Cie;j4Ay+GIaqs!8j>yjSw_X&b$ zE)Xeu_wlI+lx18t`=KS)Me&1|!7WOS3o#~g>jd(cFP(c z`l{vXbpic=aLvz(#H3!uUSecJO0&$EE*29Z4tcg1A?gEWrM}+_3t!7-TfyPnO}!O- z$`4ntG*T_CbM~zL@Hcys9QjVsnbz3_-$`Q<%X}94&$Ct9iKAn(KSPS8e!6r-n3>e? zPWdiuy_S5b<@>?NJDQv+IL4EAWxA1C{7>s^l}Wng>BgAl(P$AT^V!iYhZaMYsI>`R zSS5o8(Nqzro(~M%9S;HUJ=J)VRRgU;XOc`s7g)2)9QB_HeJfH;BSt>3qfS z!oN_hc(7j`6)I{7XQR7JX5<;>p|{+f9yWf#qDw^%t>Uqxmn5E(aCt|p>7o=`!|Ed5 z!gH@pk+-}n|F+|jY~Duk6w{H>#)OsEepL1y=~u1UDNury5B4(!Z=lv9yKETC-&Zmo z@IB+Q9LAv=DI$7GAqbRvdnMZx!t~e^GI~v5vJhv>Y_PFFCE*!=Yv0Xhf&C=1@-^w~ za`~ET^apzfDIt#ZYO=HL3i3&8c1|3#XDjJ!-u>oI^K;i3xO}O=#;0hu&Z}K6ym+GM z`i;*yn`jg2ni|DlV~HBU&t!S8QF@H}vmaaBw;MTl&q8)ITJadw(=^oD6IGtsGY>kI zW}L%BeZ}yXPr>DsaK3XE1t#p>Z-f6U;tPOC-tY@97TdBhN%?MIG(q+S0oe)>aE;qu ziBJrzx`mazF-)luOm<`CSEdm~+ob+JBZZ)b(sbMt2%ZUft{FlKPk6nC6S6Aza?1}> zld);aFH%t;l~ehRlZlDxa%hOeNH|R>V+@Sv_s}>!8Yh##KY>YRvJ6V7y+66-xbIm= ze#E^1!(YGB?%`&u_WwGchj*?3c{VM2N&`)OQTl{btUP*)CN!!+DQ~^m?w(asV!B_* zWWBSnS*ov8GSgn$t(7M~kN$NCN~Q&^CNlA{B1xT--ZNM%^qAnR9Pu~~Ub=?c;;?K5 z{q#cKAJv?D5C2{(zK6C=KG01Hy-*m#r~4!Th)&cmTAZtrQN!U4b{`!PjzzovGF9I9 z&dSLAe8c#=iu&(^?glQC2@#wk@X&0)`^ntDZGg*9M=xQBnUHYSo>3^)JVw$J14%r< zE?_;B!L{WCuvYH7zIOH6{U4W}SBeTne;YH^-;ZqL;Y|zVfaKMzTwcQ6`Y8RFay`U- zLKb=8Swrey(4a>Z;|PiIOzmgRyav6g#0`(Dc<-9vXq>wu=Y8QAS!YvO9NtyNOq$<1 z@XFjxqKX}Jou?eL2wV2p$J7AG%=qqH{htx*r=ya+T~qjGbta2zQE}XK7lvRi8h0Me?h6@1v}|WD*1p;t5q;P-vL$oV5s*pyGnM94G$vdUmm>PDP4ky z>3>|b0KYajhtrh;_MUvQAHIh^_%MS97jZcKrWr~B9J$PMXc9_tVk=xo?58i4qL9p$ z?#pze|C=Do)K%R_x)*?$)m)0OIx^MmX0<*78kb{fLDW4aYY!8IT=Z_HOzhT#gI?mu%#sY

4UDP z($pX|_xx*AMi4rXppIws+in&+>=#4Jb*)r5@zfl|xF%(WDkD#CeRtT*#Dxl5Q~`y|Z+K*|01Tu}@h9FL5Y7lnJPc6QbI#3W>WH64|l?)mxoB_6DU{*gNEGi zU+UK3q1uF?X_$H(xydby{rS&<`net9wL2V6vw{@gl|y;Cr*M7IM^2yqGIV$6{E^om zl@}|QbO=%p_R;KvOWIPYy3sUv=4FtTBOtUPeVVb!q(7idee``@9Mb(ZcHn*#bY4Kp zrv0Nl{Nx3{@*dGKhT9X5jsHEe_~G%a5ABg1FopF>o5QXrX)wbf!T;;CQ1RAG-Yfs!+{6x!q0w@Zfcjovj`!f2K4(t)5@6QIjEFx)9Px9h}v zAB*s}Sqp&hZbs#ytok|B*5`?ycrEU0*^q6Whm3=%=cu<`6DqXBKmmIgCUV$3V@X8z zEzUrjX7|+MFK<4|u5Fy*9?!vJ=B_o?LbM!t<;64P1?%%wryo6fWD?4lBa^v6rZo_# z!mbkQ?DEAOhq2&0;@+ncb!n4blnao>fl>8nMiA~dNczp$esY-fB_ItJ(~ z(&b)`IPwyCH5^@Qi7y{eHCSWIO0vlE(c&zBPbXSOuRt{iPxI2vMf^Ex3X5$Hyiqkt zsnP-KVKDV$Kof2vpAbFcILPm+sQm#m8NkZ*FJ8^;Yd(1q;#D0!J~i}rE%={X_f!yd zbO7Ux#r4|2b0KDI2FBR`7BgNV#Eg&i^~10T(41mC34Mg@G+%7&QY(wkyOh%79CmHY437Vp2*d+mwBKkjz9c@3G+s`r-w+4($fHwX$pqx| zt8=duga*Q|%Fq4_8NnUd^*j3i@&Mpz@O86P!EM9EKgxI=&=2P61u+jL!V8dngJ?oI zD)z%+GR)xC6^4`QAtpKT@!Eee)ijF#f0*h^0;YO0HH8wIFe?#~80p*^PYAhES?=t> zGk|ZasAWOU;x5q%Ae$IitxLVh{1E#O36IU?+OQdMZe_O`_MT0x2M$QQn|^=y&6Eo5 z3}FPE!lK!KsfKa_hAL!oQ!`8(|zs9Kk({_D|&5E z@igV6%!q3?KI5tAuRH=D%$)jP^nbWC%`re-5Fw$73o}mpm*YNj*6~U1PMi(1p4*Yb zkQo5&I6}e!N82KLh_ztP0%<9c$SraK$?Vnt?+Du)ussnBBmwi4{xa^*dqZ4)42{x& zhVn1m4ZrEf-+a2AW{`@bZ_w#=>$EQd1KDRjnWk=QjcWfF^8^2^SMq>-yj*v2T4Cki zky>fD%(h*tvtjOV)71SNz@Iy-9FD@#3lqF3$Q4Zr0F1Rx&llD5gn-286x~d|Ozbvr zEzMR<_b^MLmJ;#i{zjS&vyj`lhXig;dAcQzS!usf^t<=$*uj2r7d6N2-kF6m=i5(D z*M(N>eBfL8kfv~>Ker>kHO=Xr!x&qh^LozB(T-G801p?>l{mq zIb>)Y3oNNfm`#o@GY2|O{otnthoY@1+qK;k4M&HeqUwmYXBYsS`UgERe(O>mcHKlW zQ@qpFY0#bVxs{U7fr$T#(6?EKT85{2dTKq{+WaR)HakSvx6fJk=IZea2{E+(Zx9$a zVSi_TZn4(J9ET(Egi0#(%>BsEoIE-hsO9|w3_?iXe zuCV)Xw3nxX;-=7q&wO-S|M9}me5F!FF0viAAxbb8d)Ii%6`SCADI2!H^ReTRNwhA@1D7>Dg zC+E>jZc#l^ZeP@MC0(OQ6Q*m+Q!au*Ulc|K?Y>VR5j8`@S+j=cDDp(7nLwE-Q zV?~9)o}^kAfEys$j(4EIG#HdNNI+>v88kpwZ*QbH_`)ysY2wr41IXYNB$act5B&%D z9tpFkh=PTdBd>wRdnPqr&eK<86g=*UCz?P{Azt&k@2!Die}h z4Zh)*-g7jq=OGa}F0=swN~YcHr~YOgl*Hl5INnQ?kNzz?RvV0u7*Tcu zR)jEwZWZfX8)oTw(c6_E8VCqq3?plc|n~GU6{t0N_WW9+>edB!^bcxA1BS~xPpQToN;fP$AA#rkd zk+uHxdE4TR{mnCz{zHV~fhas^b=t(?|BbGEo~%!EoYD?S+i-DCLzYNiM4XOFO&thy z`~N zmE!mx)M8}cXee= z%1K#|SDnV0-n7b*T;;#myg8p4OVpKeicxhS4I1XApXi-8=rC{9B4HWH58#y(q6 zC*VJdchk(rlrc>-g^!%ZPoBt5&e&#@tljhasxFpyr{huEMws*2yG|vTrPQ2-r9&pNd)hPQzDJec`H0sC1l+-@zpKnfwoS zUdNmN7O?aGYxm5;@@!nV!zy*aEi+n--$ub0a^d{>ar*3My_9<^(1>1Jt(;ltPVZZ) zSyn(EF@C4S^?ZW*>OJ4pGtFmGH{%0g`vmn6{2Mjm0=1E6eF^;yrWmVo1q^#_r&_@? zxMJ5vZ8QBmEIep0vnv11_q#HOtesR-T86UaE?;Gu`O*G-PSTrXPrrmUueIu}iKE$s zn0erXE|p1mj52JgT2I)9Z7vPXckX+3&VOE9M2HBFsuU!y0xfUy81x^N-)6A_oq9?* z&lEtI(SzYZ0H@YAIV;ZLmHW8NO_V)_B6|Qv0Nw&PF)$GqF=+)1P98QJq-#WFvQoBUMI(=B5#Bv`$(36n6j=I1;|`5KChjqZTIQ%8#&oXlvRx;5z*53K&yA$xcN^{t_nwdBewHmzQ6Y>3dqa(%kBlxZqFVXj-6 zi&&pogMf1q134or$~jkc?$=kyzP)nNu?mCxe!%IyrC%M0y zIMK1$Uj9EWz(aWgPF)K4R66{sj2EQ{xzxwwI=woG1D{toO~k;Qqq+}(wx+@zjbP}Y=X079p33bbM(BK; z0Ja$bwL|UX%cqcbAA%>QmEFk16~nAvUNa*f<8L7Kwh$PnI46b6)z7hpUJqp|LbI?o z-XSKZ*X6Jtpi9tK8g2-X{S?K(aa3#3n(b?BV1Td-2Zs?ls|!!PYY&*l-fQK3KV4EGfKXfDWiO?H3I?|~qBqinYU_3nhEl=b^?1~+$36BH&XSINfWU%; z3|erAeC$7(iNiJHHwx+^l4n1vdV5!&KXcV-e3XhEP)5^Bp1%^kuV0UM*ST}&j+QU< znv%_L-pqLLB}x*7z; z_+Mep<3AH`#el!~yIYEHz~ax4x~lCx2p;7Ix?vU{P$W| zrxh37DJEg!ZR7UPEsApJ5**(#ZtmBSwV1K_`Q;g9L#fw9rBkN6N~fbO81@b(ii&sl z)_OX*hjsqy1DG_$f^f%zdMEu%cRJZGTg+vCu!!sk+p7dGs2FDPuiuYc)~e^N$0Q~s zq^1{$PWKiVUb;V09x04@f8-I`F9?wJ+2hkl0SxpT4%*1oAQ8f%Z*thc{MCV}9YKX?C+lozJ;wELT zlu8=a*Ai4kj4_OE&7i5;Q{;NF!T997(!OER%jyGHdarghWWNi4LHlWmui@xzAaB^6 z>-=OMw$g;#&ySXGoz(n1%FW5hD_>?qV7DD*5A^FTu8LW=Lr1i&1!})TFNOpd<1OE% z*%AlNSMATG-dkdN>pik67Ax@jUHprWq&&1d%}75nQ87Y-cyQiL$Otvyx@L^L*Ug3uZTN zRF8i+<;h`kwuwbxoW+JwSdiGNjFJaoCN z%li!z2{Z*CkSE~1;DH;UYRmaaSjvPE#N^}1cNw{GQ%sDy%}sFv(bjRSMm{fh>q|_a z892(3K_eSG&#zY+saIb)>NOKBd@7?XHRvR3WkUa{=OIRWKe<55QY(U(!kRzchd(kHRO14^FyjO7LfjBH-<}efBYzu!BG9 zdVg=wdMfs((zz~dyKm2&sm7rfzZ*%Aoqw~;Y!x6VyrQ&+3Pd;Qy}15m^Go0JUA?2r z5VvNFaB7T@*nNH<%5DdQo}^G;2A@=`Iz<33!|JO12;q2gha*U3W6N?B8&gW7tvBG zRxBH0a`vFtC4IkgwYXC7>1KBddWpLU5~Nse?FECE{kh-1Ou&iPi$L$Vc$^yJ=N~q7 z6U7F1ULVmkJS*QNzZ&Me=HV=Z>*lBW1=GLf+(yFVP|gH_=kd;=Hi{1X@sUcCyFL?g zcPB^To+wb(1NAjd(i|ci!y770n&hQfNqs+X)SWmOTlatB)-B%leLEM04(Cp{{OgOS z#T9s9nFg37-Uc(KY>@%xq7Vm)YAYulQL$I5W?E})G}$JUQ8z}v-1@a+6xz*qKi30a ztoF*=)&@7B4Cu7L&*H|I zrt0#E>VjL%r6)fM>z&%g4q+3RYDhMSelBT;p}v-2t}HBI_AzmkD|GXMPDT_M3XMxB zAm$VAXrGyPuiNYl>wVD0NOoIL+zdzm@>93I#KTikf91QU$w2us_H2-X^chI34qFAu zTPhp$echh2kj7x~PtdcGIJ_4{0R}V6pT|}7Of?7Hv#Ugb*e49l-op6SyO3`Vzy7)& z4)yZg={NSr#B%q#9Ifxk&|&0Xv|kGb1}qHNd1lEL>E%xSb$`8r_m}Mc?AiA~Gf+{A zqy4%gv}h?wH@Fx>gL7uv6uW1TduUi!N?&YozOfBf~9xVY!KZZ#)UQQeJOG#-V<1FWyLiP=q8!OBx z>@YB^BeVay<+XZ3A-w5ZUml@=0ogDM@`0r94FM-pP?Wxtx^J0jMPGZ=4C$G`*QqUl zGJVCZlsMS{^mzJ6WP&q#1!;@7$9y(0+xf=i@$KRxkQ=Dhbs0x4kO|aLZq4^pdYl;b~aQpjVo4n=-kD$i2Hh-n240hWf zjl(mf-Y=07Wqu^<`r+voyS4a&M_tZb3LkJ@iuIK3#K8Y@{p>w>VNyj&_ts@SKX2EL zZ~I}*t9y-W0h`K`f3|{r*6=kn>Pc-D^$|=^)qG4WmzxJ_JRhb3rztUEy2wTplOA! zFB4wcOzljPSf`16T-khmP^tf+wE;Ri ze3&Nx0loc#BEn!d;t)4g#S2)4yo#~Q+ToaO7TfmsibS5++uM-`d7zGt>`aWr1|$4R zgp{Vlel1FfiQqaMUMJzwWT$Q$nPYn3MC^_424XnQB_a}Wn_Jk5W z2D9|-hxI|Z+f`zo`0bfLxgRs@P_oOj@KBb>7cTZUNPM@P2#1QkA+(PZ0j#!+%*{u0 zRAbM%5mB5=qrYkwr=qhdoDQqvPLy`yICAnpcVOkjQ?F7Ig}lPO<#)gD7m@RM=(XAk#Qa>bT7A<>eLJx!H-{k6V&|Dj?}tNURa(=^ zLO+8h|5y3aP_J;>O(y))v9j8j*WYVn_L;NytN<;ll;mbDFL5Ve4a;=$Y6c-p!oI0 z(bx65xh$tIW;{&i95`C6Fucng`&}4}f8@OPjonW_t?JG0c*${Q$xbnEjI7)7j3dG# zeY+k%HP1;YPT-pMvZm%y`h8K^KRbf1ZfJ?gVi4HTr44Wv-^!H`d+P&kx5>Y}Sd~$5 zV+S;R0i|Bj!x#cbM1?&T(=+G4_BY@6Bf@ktdGjpc%aoYn1xXCTMCe~O-7Yig{YLi^ z3I)?zny=gG6PQ}&=x6JGIItIm0YX9(mb!@9zE%^oTgVR^P}V|0+;Un-INrYbgt zdQr9a4jU;Ea|NnVZo|58)GH-uAFhp%rsCobqwJcr10^w|(iE;&H9Q<>si}V#mYrwe0VbHCzUxpe8$G7Z(& z^0>b?p~HR|M4ra_rz8zaRBY?Ir^BrSz}gpl;6rHi)7yAg9pwzWg+~hQjwprY5P0KW0#24t7@W zl~+=oqr^PCQn@F>nkr-dA?<_ZCeJ#CV}m8bmXg)L8l_;>&u#Y1 zsr3u^{|T%P6v4$}n8nYDO4+(M)&hs_ID*3I)@oJnqgkn}l2itbv-{;&q?bCKUs_&q_Xoo?WVfsK3YPv`!r@E56T)_vOHz!TX6`$8A zTOl%_?)HF3+e^ZYMOoxjES~Vs7~+r5V0$ClCO8m{QG0;C1qb9WhJ*1CH!m+Q%#_=@ ziZ@@&(B?ydX*X2Tjod}l%{yt5cwfJLEs1rDnAkax-|5l&_FMf~TaBsY8>+7U)eCXy zWZ1eN${_|7`YdP3JGP+l?|uQAkXbp)*Tu;J8zhuE6HNz=q0KK`wp%ptm;J-5a6tdA z47q@_0?s0#vR^lplVD?)B}=$p1bUUthiy;C?L_4U^HSx}k)ZbDotvMe_h%D~hS+j+ zEd#b1z^wCWV&agQfk_T7&S6MCX<12E$@xgKa!@b|zfkq3ARPQN8)8>)LYlPjImbT5 zX4_=YVVeww&pT{SX1PS5Wtz_<0~?(E8oGrYp9xxz0VI?2A=4Rh(5!8{k1Jv`05zDa3Q) z6n_<-*Uk;gqXST?XI~4-Jf^0k5BQAJ0Jub87T^=iSQa(eVu6}>8k%GZIkNeIm#`P zQs!BZoV}pg2<6KQ3D(GR! ziiZfw;%w86AM);gaoT4}_j-2p2!a_Wa`m~csV%@oP;sHa1=1_<2RMbXWtYz=K(L!1 z5F?9<^7Ee-3jB{L2Bi_k%8h}@D3#udfR13#StmfIM^ZqaI= zoO}>m4s7pxuXYMjcU*S947nS&YP*mBMhCM7kVi3l0|mzMX%-q3-Tor?O_pp(zKrkI z6GIq|uUO6e5i5nt{QijJ1x{B0J06#U5J~(mzm{w%X*=iBc0SSkS31&R-JzA;f~Usf zy;a_ZFTp3aBhtJRP&kNm%Ab#b!_$o9G?Y)w=iFq+;H=@)K_J)psJ$ z864U2R2xxp*m%L#q$Q%seTUjwV^uDP#MBiW18E9tSoUhgs5a=PxU)6yVWM>(UN+*b{*x zgn2fSqPEU!}Q1Y7o#cvlFFqQOXkNSx}N9RK+I&B;yr3;+Yf$i|wNzS{2Kyb!_SbKh)AZIC z6c?@re2QQv&q)rH>T z)!&PPF%o&*Z5`w3^5N*ki`qE{qZwp}6*|xK4|`UE-ukw$g*_4wPy%PXhl_f(9tszK z00`iu;YCOi7w7}-chl~8>1z!q$7wxrwY%q5F~EmPlpRs8(j=w&Q3!n+m^nXE7>W|b zDoEhV-?Z~RIh>$i<&Nceg_{)8P2s?=M8PbyjG9-3j3T3@3vFK2-#i)$D+TWX5tC0?w70`woUR~*T zGmy(zU5yM3e&g&@=(>`x5c?gx9y3@_*_v z^$+-oJ!#LLe>+GCvfMRyzVLLZ`aX2o?$h4 z)`u6cuMe8}9qJv(c$%AZ0v;_0m7YkoH0U(Df{4BNHrCYzdfDuEfO`hqCGiwvBcr*; zZF^NfR9G|r(*1bic&D;F_)z`D1(;bt>`@C-s$0$Hfoo;t);Gfzxw))}4e#q`jy7bWbNRt_`Y{Q} ztc@a?W)iM6yu~@)p_mZZ2C~PS6y^a(+tie25Wj-ieys`47yg(aL!iF}e7pE?ql6_Y zFWgeLTlbdU|7mQQyFUIxR_8qSEemSL;*=iVCg}CrAVYFcLWPgTNYUI7ZE(n?8J}P)tEE=Iz?>cBQ!@x;-6LD$e@?06-T!DGS{Of}c z6{>pw;{p_zc@dtr0^bLPq^A-J@ac@ldu_6VH8U`_>2aO9PB}*t4^}OH#lT(h*Ctw# zw9^)>vV$jdP+&b6-hayuzCmw7N()wj5XrsAZ0JAnJ{a^9=}duv^c3-u5D+v}-YZyv zui@66lsAi6wI)@7!+*GX{425HV+745hg0KKE;jGJRp_e~N1`Pg4lSaXdAPjq-J+&+ z@Olq!3epXk;0ob?Hm9K%;rx6b5G-a!mJMSW@&wQo=U>66)5jQmXF9$NN3hluK%rR`fyn^Mo#pU2PfH+x{gN2Fye1 zAK<2g{E<1+33k=!-Y)0A>AY7(ZGIh%Q=A-o`flXNPF0n4H)HdnuOKZXgWE9RCD-!IDx|0h~I=@6vvPF{S7FWHt;W7D1oy~h}~@X^%~$@G)b3L zMZq^p)I5-(fRl!)wokc+Z~il@qrFyQuP4BI;%l~8p5dkU^*^7t_i#;Ih|^FLTY8(7 zUoqkOev0*q?ers6V`8h19OSA}JZ9uA+~m{y0HUGX56I6$-S<+H>@mXhL{YG*44}Xw z>_YcUF)J!&oH)%0@1bxIL2Q@V9BH6vUXXD|pr`wAY?)U7RjzBEGh#j0+eXKaz7=hZ zEEi>2)q2>CmT>`7=M<`}%&hIVwUgVN^zNk2^3YIiHY*5P zNid0N4U{rcFm2pv<1h#SU{RL?GCt&)v`!Ot0E;$+8DGUqOgRM2)sxjrN0_B$nTOB3 z9{>5ElNYridnYh$Yu=Cou?nmMd76%ZaHI?bqblOK%>+&FHbnAsAYcqkN685Z6M zXsS(@nbS8WDo-}%a5V`tX2;W#hz(R8-K#cHSay2uE}*9>hb+v+#1Cm%4pq4n(aQ}D z0$`qHT6v*R3cCTsI4{qa`HRK-uf^S{ysx@z(==Mkjca$ErSEFk*J7S1zr@Jw7DX#U ze7r4CxAkrRr|ffl*@0>9yDE|YM4+ixudwB8xDoPgo%tEt%vQSfy2k>$tNrUy=cKp< z*EicX=L^jGzh?)=F?NCKakG(-q8)@hQUMq3<51>cvhX)=i8@HMEym(>3w`hVfPkyg z>vd|a-Ii_~8%;K`Q!U%eRw;9etMlDQ*BuANBs5TKr(zCK!-OwB@A~gbKHT4oEq`*N zedqkS<$mH!pxtOrHs1?WEtib0di;fUsO!9=UUpZSCYeOAH;u#qZ{l)LG#?q+=HsqZw~EBv zAX}#z60^HqfBiVQ#jw$#+4ghU4(amTOeog(^>WMv- zPj|}~V{GoC{H0ngz7=2uju-r4FXtq|Sn>WuJy56KmVKQ3g}7$cskx17|0>v=0m_`X zkQ4sw=0~l`!5uKb@3296i*L6!QsOln9Z&?|{COgtE(ZUyRz8)%jksTQ6k4?fMG1nR zVggn{L+*VSE( zY~QjMlR7MSyB?h<-M5Mwr+SMQmeTkkhnyv%es_GiU{fOhL#eO$poMq#Xh|J;@i9eF z4GE0pxris7t(Ia+AICuQ8e0q_D8M6xi5@Hm!5@;+UTqPD5^rW}V_Hc>lXy<}i88_z zhedH8nm^#{+MA=p3PL{oY!<5aP*^VW1($ymUUEs$c1P=rEYVh*;^Ul*?6p2kJ~I@mQK zqM}hze-ob@5#7t0*C}t$znS7uyrdu~4<-+-SC^9v0^Q!!7sc9V4i-Ou{;b+)CcEeV zTw4uzJS2>wj``Bvf|CClV?&tkRJXprkND{w&!${C5B^3$Rmh#5P7beZaq}inj3mTu zUXNhBz+O(!cW;A(NC)^@Ae7jR-Sd`Uw*to!v9+q1QZg)2*6rg`H=egUG1Ezs80!r- z7t(R$K%Q1LWe=h}OMhDKku@K|h5~NpwjF2OKYEzoU2@m2_qD#Um*KTdo9|4XNxdZz zaW06v(hJ}dGLD7Db{niykNq<@69Z;u=X?6JJG9HPTT?F=8~Y4A%K*yy#VZo1SKL_61obFp@n7h6Amp`yjwey*&qIagBLD46U^m2TBPmo< zQfozjLhc}gxc}%hyySp3%tIb4InGJpAbh;nT!V~wPF@evcGTE3k=M?{l-z2aGWP>d zyK}{H{8=pO)!=kVY3b!VCA|z+?jl0ULj5FdwK$t|e5UW69|%knGBSm^2)-4E-4`Ks z6cz(Z(3>5C|BqwRZN$^+`6L1J-IO_fyhp_J1QrQ_wkQ$DQQSSC1fqRLNYP(Z#Kha_L9(TuS+9sFN8WyIviyf+?uu~UiM{Yo z3u|qi!&BXgD~%bYI3D&t!7z@j`C^KlXH*IuFTCmoSo^CbOmOW-6AV$QIgJ+ zc=dbJk*DD+ES$aF2rKhiXW}JdQev=ZDfkaNSsXT09ARBxSjrpkWjJ~Mec)vjh<#y3 z)t=r6^t0&eSt|BtDW6Uu8#D>nml%a+xF-6*p>vdrN{?)JBB0@0tFlrX2%uR<1p0HT zz{L$H^38}-+pnDMgL*wve;v4=I&PPX4M)JG8MtHQ)rs zVU~rQ?wNC5llVA0%Z6FI7RY3K!EC6ZePfM`mh+&cDRI{{rJp8#`wKhj%tSFA$; zb;vZ>kQDvAwS<29?Aa;W1ml>WZxcFCHHQ8D2{(iOe=ms3k>}&p)UvmKRq=XVP8D8{ zkif5vqkJFu!UUl9bY$*pzU-&P-Oq!`R`PHyltyI7Qk;#wvWHGBY(}I8Ra-3!YK?>l z^op;Pf+Iq z@;uBmB7k=ox3a>H>uOTc%A@Wm=c>hc|6J$338(T%q;8#@)78U zqmiBsmzJ=T-9+v5V^M{6^CzZ;hCjVycYv{`!FGRZL4w6w2)^3Q*VXmvBFo?2C0;i) zJU?bm0Cg}bEF-2x1^(mkB=N zC^f#hxj8}9Y5WgoU_xDmmQNoSC2YaI9L8LRq%bS$5O!-QirWcn(1^P_;>GcgefLHn zuq&JOStS10I%TN{JQ94?n=DA46$=XC3*aq(8Lve_)P)Ev)F`h88IyzhSK%6leL*$o zSP{8V?ev=D{tb(ql&^8cYhNy%Klphka64f7Aq_3<4i7iCNzM(WDu%}2VlFf0@V`p2 z7xxNA31ygk>4P>6Grp^rFM`|a@Qa^C#7*sMimcmgFyy7))(%NhuD-T3b$d0}Nvi#FHX^_JTq==Vwl)|07LcQtt{oj_z6F zE{q&G^UgDdmGCRM2t7r^759S68-9Ndp4W{Yay#KKsOIm}LSHnqs18A055pTOWx}lP zqZK-o@r3wKaB+{%pE-?#LlfB8*2kxelD@I6s1(;e;k%_(2HcTH z7v$f?p9kCosMg&<6pUBVP9L^u{|hBQ?+dDeSk@KQKDCK>K+05i4uQWf&UX$Bx@1C4 zx!C3{PY5PY;}I`yDPdm9D2({6!AGib-X!i_>{Deu6&kYgi+bofu%1&!4(1IF3~rA- z)x4ls;!mMWxotDU!ucW-qTo@~qPzgSY>L%lJ<_|L(c{?z@69jpl?lPSox1u4f)WafYZUB@q=R_pu&aN^Yk1;5`hf z!uf=KNebxG0}0&<{F&oUJk~}>^xyu+wFl*u*-7+L_VM+G)rud^eadv+Lg1nTW-3L| zws^ijM5ITeop}Sf&XAE(;P06O$6LPV@nix{kVLMUml2Y@-oKH_K{|a1&7130FaSwok}yRepLXeT?3XVZ|GH4SDI3p8-TCtLsZEFr zz}E$$gc?fDm&?moW&Ne@_GCnZ7lGwJ5`y?U#$)|uwmRluEj91C&}$X_j?LRm1_Pc( zJJk&oFN6YR@l8lAg<*TdhUs8~d8yW(Tfjshs8KlMCQ!47Er|U+C0O98V;Be<;IL;9 z5K+=1F(WqrX7qfJeGgs`od(G@8Tuza1Od>;&gn`L%DM-8`W|gp1)sBUC2IK1eBeP3tbzu1{Casox1*@>THpcgA^<};I58aukEuG+heYeNPMJh=LU7_c> zqFPw+#r=z|T8M5$4e}a%n{I%u1SI;$+qCAiL!~eH`1!-?rV?8VqpGJPE+HdI@AVx* zC(W#lLFwowAa3RKMv31SEd%=bGe{>y!;k|UwN2?q zzl)V9lC%7t$mPwTU~+NgTou?Ur05R#byua=5~h`05^C-?Tyr61_Q1%a7{mMOFDZ4Y zy10HnTGR=Ma9KoJVsqM$?oe6xW2gJDw4YM3U)q^mR!Ad2Mw`J8Ne6LVe{(u!XP%SDyXTi@{} zUDMFW4}A!+2Sm(g>wbT|Hi-2FrzYVa8`=Cpk8BFxq=(~Rm=K+b>V?^)kQya6HA6-A zF{Y!3)9HHH!S|^o?X?dZJ`ishOzpOzOPEH$6}#d&bFe;Fk5fc z686Arqls9b_H*O{N3xUEt%H*hA3DL@yh}%iW1)1fDzKlFbg5sFL@-7=TqO}AP`|57 zpfE7mj%p=XrqqaJk-BuYC>DF}J++CSWNMeks`;CS!uKUAm?o@wN(^lfC?&cs z87ts(ECoL6d!^!Jfn^J&Nli%!`&=RFEhkEGQ2IkI1hrr7%l7PvoBo8{xj)gk$s?79 zy*-6oMiQ0$xL3h*X8Y3=b#wTm_m>k>$W;tB{I5BQ(XzTPtcWS9HVC+>(xrH|5(A_XeHLT)1IfZy*Yya!;8_a?5q^@bb+sAxj^9Itdt)|?=?bC zXex=uLq`#2{O;Z>P!+v;Rqmi=3yY7$fA8I($aDryGHT3Uo&uMrJ>VjoEWo%EV6peX zD=ulo&v`nWi?$v^hcCxJBF?5td;B{`mgbm$!oTBA%oQ(B%)<-Vq=LiN&WKprq$dv< zY$KRb0w|+#y1r@hpJ-3I?^O z1RV5lkKEsxhoQy($|}5?pa;UgsHKts_oCfg^v(4PmZnmVSU0*bc#A}A)=fY;+?-2w zsu<+3TgR`4A(3ST71{NRyKCS|;O~3Sj|d|!@grP!w4kXq9eua*$;!aOTc*b===d7y z41#FT6$h{30x=jFyL~FP2xpNBG_ezn&TE|I{9RILdrwEGvbYaKL#Jm`)|D$b-TdoC z5+z*Yz6N;g?}IKV`+jetE>aZG$kOcw!SAV?81RWD_i7E@iGZwjsAy812h%jbzH#o^ z#-Ng2QK6hqE$#O*j^)Jf*=)h>U_cadsj$kL!jxrINl=;nj;4<=MuVk5F!G^0SFqK+ z>7DO8=hi6?m9gZ?DAt?ALL%{}oGAm9@%&FshQ?TD!((a#@JhK}KA1_g5s|KD2XcNk z1=MAY+`EZ?AcfQuS-AEVDS;z0BM;>oVL;s);rJ7St%47gc-GaDm^zUkl3~3WxJf@} zT6Q@PR&)1nhUaSqld|yM>{)J_i&I&VUeaKGlcmIik&k=s;BsR${Ury!{Cxx1r;(D+ zm%dthgae67WXH|kfW!0K;`YXJ8^O}5wOnoPV|{xt}iOwhO<#z4YyEZf?n0>H<)ri>be;sHm7UmdTVAOYC^y_E{ZR-*+C@hiJgFz1^!8 z`&%9y7z^8G58ZEb5e32>KW4pwGlSaBNX9%#9)ye%0Ll7v&zt)qKZY3EOqpT5OjQ0<<%l&g@Ytg^=L+3&HTr z&Q#GJZQb{b3{b+*CmR^9B0NBsAq&s&^cy4hD_+71F~JI5(M^fuM_y$Yi!{7S#Bd<_ zg&6MvJnisjt6l*0)3tN-a_4|FqgbCj`|LjWw#TLYsWmsaUJ9{Pd>6~H8Uk;89Lq`R z15an2X-#AYTea9t03?BG)XADEqhK`NZ;K|ACPxt7~r0yjpd5YMFc&CX) zpfoVOrUB;~+4=j$VYI5F*H)KkgYzc_Tk|UqzQ1p)oqngc6kixMZ$gCz*&){+7Oa7UA~9s+p3ztpZtX7hV5;jS9bD zTM(}y+P*WEJG-8-f0*6~5daq_496*l1Q2ap`{!Aq)>3%xyXlgm=TIo{oR)jAuEc5Bfeb+y;NnVHpo7Y^ql0gnnw*R{8`)(Zi-UaJoy(Cvxj&2VI6V-Y! zGjW*n*asjMt(7>rHB2@?w^gO4t#R5EceC$}_WBHBXlQ=>kX9k>(kUrQ&?61?RS?qwf?y_Rk`f@4@<*pwT%To-dJ{ z=*lDRi!k6R?Slol!0S0^u-KZ2(i{fVeb|slev7~VdIJ-96lo5N$isE@hdb=_n(nA8 zYK<3QyLl{x@%|lx>T~mpiYm{tA%QSN1#wwT0>adaSPdK$Zw0%UrzOr{60ASbEF<~I z`ua1!}~OFLKb{4jEG;ak=xD}UEU^|>l3_Bf9qQ=P&u`a8I0&C9!*n`uj1@X+@^ig zrXF`yXX{vqI$sysEjEc{^`9VSI$@IaT~_v;ih6QRA@fQ%dPVv0N7}bQP-@sVn%||@i z5{sP7&W-q9J7A7s8T62o5TOb|azl44W%G#~2p2UQ^Kafjb`-|cecp!$9xIuDh z4TIHuIw9FKV@l;pb5^h_0Fvta?`eWoiMR&vOZmO@jSR>Hmw;*V1W;N62KVa!xBv*- zDvrR=HQoPUF-gWxqQ-MQ21cAt)hriY`ZGE_dbm1YDSha~HUGWt`QE|Qa$DO_Z1nCj z#kfLcPC#^#ug4!R^4gi0%O^K-{fFIPtY-XH51rz*rJ!ck>4%{_*w_7w=W-2}__oFw zxC*1Pr6Anvp38X~14=mRiu)tMzB00ci5=xfqf;ijMoJ+oyUtIK(M-P!`G(JXGC%FG z(#@hSaPlcASgyLOVB)5vWar>;5ai1y15s!Lh6d9zCsO1<-5@r!;R<#Sh;gUD)W zFu%t&9Urq2D%^Gn>m?f)!zz0Y=6*e(p{!#Ki#DGx}bxEQ44DB!@Cj6$_3>+;q zt1sfvt%<(QvVcbY@FnB2`F`m9z~6IR{jl!+V&a7N%i)_Q`K=TU>yLLQ8s{^s>7*ai zVrIo!E;X2i_mC^k;6ItUgcoz4CvEt~_RqwJq;zPq4`s+^C{-pO(DJdZBfb7ojy?*S#U7Nam0wd+9+NhOJ+qdcctO=IO3E7eQz{yVp4+|28b;2bw@0fZgG=SBvz{1^lkKRm9Ql6f-WT_u z&y2fQ`?A`u=B&?MOn1u$q-LUt#DS|jv{{+2mHNlK$i@`3;apn(5eJ(v2Jp2#GdJ>= z1Dkm34p=-~GCm7Ytb`JH*fI=Q;Mu(}hf1uBS@6}+?S5<{Wud+~V{EDM!K@7ov ztZ)l%MKpX!@SaN*+iyH<0~aASDzDZjG5$^~4A+}#(}!04gv&zA4Y>xwz4 zWs(DJ16ljjG-WfhCW87KS+JjeTp0nA@RFxb$zPnGH_opV#@j`_#)>?bRlCUG*wkQCcLjYmi8Im@2qYWz9PIEa42Z* zGmW(g_2i@sIh{+BFteiMz1}apk?ivY*NH>d4J}YcTZef3^q7~H;wazNv3jgZ>?B)S z3_MJYwE=i#U)CMEN#m%J%G(<80AJs}ob9cRDXqDz=Ca);|BF9b%7Nt@iQeUp;#60z zt6jxKu!)A!xfy7Qfck9Ju`L+jxcu4jktt@S?pdp3wX!J(<;s6oc}#iFPD8ux^=39OuS=qdDfrFJJww+ElA6?whGdoq$*4GN>N{I z(CVDBH&m|IGH^rc`E4?d!jPwYnJ>y{{>Vm29G0uU zk9nen!Lqy|i_%iCdGQe=rJk0&`-08ZHn4WQ#F!OHGil)~=vbIdd+2eT&OG$#)m0FP zY^c4rHudqWZXVD2rOxM~FwHZ{44hNCE$Iu-g(7(TUKGOx&l09DiD_f>KvdEY&5eEZ zW=>&3eg*M1kPBP_nJVSqy%iq7b)^Aeo%(Llyp7h2r4Pb3XDjFqNZSxnfYk78WGYU* zM{9~&BGETo%@o(z_@c0JchHAUmOFT%<5+QR?rnqWOJzMTwXJvJKQQ;`M?s$NiIO@& z&cd|<3>J*Fbd}3EQ_h)8eb+)ou~4w{Kne^jj&{>_>9+Z0g+=Id)_W1~B&*^VUXHVd z_U8A*7`r%@%Uax@x3kqw5Bw;{P^|qh$9q$^V8uS~jEfyaXQt7WF-0yl9uysE@60u@ zX(l;O6(xu}R^dR!t*{%sF8!-5nv_CCgo_Qe-$b|eS-E6)y~|{r7F=(spgfd;a9wg; z%X$V8C#(+EEZd1O{+Ys8I5Lp~i{`)uX+{u8l-Ck1Hxu=>H!=DAr6K?B%vs=KDDB?( zqcuFL#p{Mx_2Zd(GdV%B00Yp9xw({q+fKhekJPUw_Fv$b?nYZX!8}V4b3OF5W|GQTi}+ zA?&sY4V4Exw{>$QyUnCrFJG=Zv~y=ssu9NJ7Gt5$;>NAuogH@E`8ou2Lo^snH9{jN zxpR`@yZ390hwh+Zx=P}attS2;jKmCEaR0MCdTq$U~&XraA?=*h5r9b1vH z5-IO>_*$PptsS?sJ&{UTREg*y1~{5FfrJXGCaeA8RV`7)I7*{dmXLA)S0c{V9Mwll z(g)DvnF!nD_-huPu`0=U2}p|EZu+)d7D~B8XJc$>$@O!3ddTa>eNzOM>gkHH3;4gM z^ydu2d%2!Qag@vvV`XayN1^`SVI275z#w>qmK}1}Vb5J%U0W^{%)clo(6+a?-y*#j z1^-`BrVUJU0A(r~;~OZO$!sKj7((8=?6|~!1PG9|?H3O3LZF8!(TAk!$d#FXEcBdw z_F36DuiJbyoXKg7ISc_>ia5dRlWCyBBRz<6+xzuBPJ_4)pc8jE@9~pY$QuT8r@c2E zfzVGxK7A`pdGC--KX4Kl5s}A|7jR$#NB(aRX$Aq2$9g(=F&U*%-?WCdj4=V%KW0H5q2l{Y)+Wi4Qh0uz`GijyS>*(fr$jZnJ z@z50cUj^?ymjh4gdk}CV=61|U0yEMj*A<$&hE$gM{C*e1PYIRM5Y$1;B!h6U4(ZZx zzXm~*?`uYT$l7~Xd1H9Kp#Rv>k5s=xmi3|Bk1Jf#RC7ApwxB%tX{XW;&a9;b7~a~5`9!99?O@KUN}`US3Lscf z?34c*<5ov_Bx*RgmW~y8>DH4jq>XEL#Gd5!#_T*!RTT2Jy2=T1d=oPt{jHH7$Sj8cAwpg+_`ZfI@yi(1LN*ohIvNmh`?0(!4=xY*2KXlwr zC_h5|AR)|@&VsSG(L!gVeU)?pZ3H(UZ$Lbg({0~zwJ$sp@Va}-8*>N4qX}uK%QHS3 zKjiHvbgkVhoMyhL?qJ*Mwk8^jMo4Hg$ZJ^NH3AHI3sDqJxwEQuJcOfblAkC@6*}|T z;7N;D3t8j09jQ=kk@Ai_lcyJ#r#WxD_sw%?vuP(3r=GwfVr%uW(y?4ziPL;l7O8_K;4GLJoRX!sOd-wxq;;2FJPZRk$rI7~p7S71a;0R#5(6LX- z<})dsT9X%wr3}cT!edL8ILGBfnZ8Ah#@YZxOQpq%Rj@D&=x6rH`N%BWnkCC$hj;* zc(jSGAeHA=+utc_W6VYW;dUL0)F(s~i!JL}J(<}8RG+Fj$?X@29XItug`?0J7d5R7 zDi63~E1Q|%cJ%}mL`zLM7E7;4f`*MMiq)COM)qs&!wWh58x_oNCANSZrG-$ZF>|W` z9v5OD>U9ut$NRhIeprlOU^$n#L>;BqMyrx_oF)-|SMQ3d@@08*RjpmA8WZb(6j zTQ@OM-o7+%ro3-6EwZgOK;W!hj0S@w++{J`v=hI)-T<83jg$+kGJtYu%wD5GuxlOD4A#k z-6ctfaFxB7^!InQGz_3(7*u_-$Fa%w%=0-i`obg(3-JVCPn6ibr9R{j;W}$ReW-W_ z+9UDDz9K}^|NP>iTF{A1l6H>61Pc%qTBQNHmAL6rC1Ebs*z0+`LHsgEV{*HHRw35Y|#IUK(89(5UxT!D-2cbSQ;7Ok=n-HK25CT_z3+9-r6hOvDw`|@j4_`?9d45mJS zCxtj9pNy$eLpwr@G$a?G!@pK>{T;`BF> zbV|6#FVC|4`X_}FUcdTm^|)fl55%tE`0w8V`l2hZ-vwDW!t}UTtC0O)2=9Y3i$IrZyvtt2s-g(-9=cNiZ9-b|uV0e%U1mHaHl_ROx+oZQbGY-c&yCLT|<;5 zZQyNTs)M)7fF(eFgF{aL)MZG$L$=FiBP8~Ag;o}7pjlfv@|UV8ysXo@PY*biBm3Db zBJo3~!{HQ?!DD}$KG>qE<2lGb8GAQ$W-9myoy#}n+b8wzV(F6NjW3XlocrPMm-PDg z`4`8lDQ3-O>gLF{J^<5+gu8dyAH%Dp#N#RbA(c4+Ib#NFa1!wYQw$(0{v7gb!?ch3_(6e-2Ttz3}0h>=Wv)2wH0xK zee{9r@hNhWPNNurduM{>YL>o&CBn|qsqFIePxXgXcNbf>m-D(mp2RGq2;Y|Y`aDp;&X3bflwM#0!t+rR2@g^IP)ES(; zzrIf|svKp{>2^a64X)g}Po}Fj-EvfVJ2vxjkU*F3Yc{GY`moCtotm&9`2`E;;%WoP z7PB7AkxZ88kzgLPXR$rpXzNeDx-71d#mR+UxFnm&^|P)eEl>yh=DJ?hLWG|ETrL#KU?lw9CG6$z?~ePCxybz!H!JYr0h_JQ2|gLlywP8@6{TEWOd z`P1;Evh)ocI{>~kTBkFEcJ9XNy!9Py2Xz|kvY5E~{I^oC$8RM;zPZ{uxlA$X{n~OQ z|Lp>#R&TP&v7g#Zo`q^U*V>KjY}p zamGLCyjmbJpF8l1X`~I*QjG|Mq%r#&!e=X9W3OCqw(O6CV?z?(Zey z$!ax}mkQ3KlgBJ;kmjLavA$^9@2}aJJUiIYtYIh^H{?Y``Zy(UrJms|_MA5F~hoKEiyN zz{T3F3{hNaXBtQMn<{_zYxMqNpjHA4+ zQ@X1TP16U?_ryUh)Y}%r~T2mZP#Qgk;kC*J9?QyRlNl3uEB?S>l zh@dlAyR$_1wmUSiN+%nj2J?0(~8cADX%7ej)Yg=wphHBgkwTL#K z_x8dyJSFw+vTAkM?uR4QQ4s?kBP+ytW^9cHzL$ls(RFouHT*hbcu`0}!KH6Um;vu1 z%~7a_P`D5;Fx#{@E?OyA2O?Xfp2nJ*nzrV$QWcPfdkU*Ak>$;dk`#+(w*U5RDYZEU z1DX3LdkZwMk0Mf8RJY=(u#})l##&cwZXv=#76($2!nik+DiwN^kaMXYmKS`4BpHvc zJM~&3NrvTdmUgN;r!J!L*1zUQtTzTv9ErsQwzDM730#Hq?;=HKK2-6PKZe*|X zWh8}-f@IO*#YhI|j-qA2syAVfsGP;P24ma1A3uiPxh78LI=AwOdvAncF{>-rm zJ(Do_x;jxPBa7^93w@kF?%llOaBEt%R*bG6E}-6hVJ>3f2g^4v@&{_4paS|+Uz-X+ zMETC+{f=*>>RSv)(EK7<>_oZqeGKlD&3$7Rm#FQcR^K?vv)}aQ#$B|cAXTF_w(Qw# zuS-s)Sm94P#sKo4QnCA&lC8Fq?uh0GfN`813^C=@FG}6KJ$jcC#}3mvPkE`?jNar6 z<1|XbhO?+)tjA&6!bITHnw5*37KcA9ukO5Rt;lEh`z=d}bJhM|m0^TwC?W#OT@%e5 zKP`L)hbtD+@V2hG0chc$Ko|!H3>{{H9Unq<4f+k@=E zn2W37Sk}-bM&eypKVMgAZH9P%DJFlPhp}B3HrB5)NF#^UD*E4t)eAVR%(KaT^CRLe zNCWP3N@LzgH^H|#lOC8(^ypvm;ui{HyFThifz}z*Pnvg&cxY-^z^jTLLVwp$cRHo# z>m*%GG0qk_gO{$R1d`-S#&1AVl#Hc7;2CZutfF~+`+Saw-TW}Sp|2O&qDr+whi0i^{oCL{Zj|CkE9Bm~_Oy@oTDa z{eRtCLM-oM`t$2VW|r~F@R@VcC%AS0ieUoDTrgfNNFkj8+w>j8x&>-2_l@F3@0wcV zf9y7tmf_T8HmCt+y|i0zapSF7mUBmG@4QNUTTZEEVOl{8Q&W}&TGCrZs9f~%@{EG`PJW&0i5(Xz8mz0zgzjNAocor83$9~aq^`8@$9&PLlt9#*Ve*zcb%&K5d z?)m)=p3_Tui0ebmC^})ur*!O5WrE4#-Lu0=;`ye6YWfuj$Iks`&QK*5%BD)kvJU}e zJ=R0u=)BwA-JPls`6dEEZ`3~x`xI1FDT+4je0Wh*r-*|jnypaoTOlOT7CSdBYk>tD zA0AXK7Las0Rk4Hkcvo^0tW+mvy5#;=1tO=B!+O}Ge*3#c;6&Qh1a_IY3|RhVlmTX) z^;w;hn?fZ8PgXBJj3uNW*T#(U;sTV8W=QTsgK?R5g;wTCChT)3oqd0kN0^*BC3FPZ zl;ds8eU9&;OKHOzE^6`=H2%IUwICcgz8@BV;&0RDN{4<$+BIZC`Nmv~A9^)u6 zQ<$(~HGFYTAw9VUQV)NEN3vQM2Ji@36vdq!uj^6_!=Ku)+872wCm3%HD4kh*pXfV~YP@g=4p9w`3lk_1RK^mKjl^o6x~ zrX{H~1u0Fvy)6$LmvY#`Gyt%EtQwsw8CA5Sh&~OLlsR`yv?zD1ZoG}NWX3)=%r|rq z91K|HJdZpkGtH zgRDZJJXoS9`u8Z)VM7)J4|O6{PLM-_kn+bvWvWVem_oIM)7%a>tMo!5ayC4kGg zcD2vZbG~-<6&CB@AK_+Swejex3nl6dz+OwG^VVJ^&!~2m-({&#N^`8~sk>oklE4mlkYabi!l&K^~>-mEErnm{McaLP^t2cso2UEG#6v*;M-c z$%vb%#DG#%^tSC;iKG1PbdAf;)Ye~pQSf(m+*}0NzJ>JqEl0ZW#?4JgBcg7dFpoFZ zD%QU3x7UHt;r&y$+ryk`IFeL4kynVF$B{TuemJjdId?;jRsnd#j~uk#ry(jTK5+0# z(~W$$-AXml7j>IWzZM!eh3Mraj#Bu}-$bns@$4*IYm^V?sceALoyk>stNBy3;lSK!owMW7=oQBgcILj!bAVbAO+=*|(!ylS>uZPAwEWb}} zllVZ^`NcF8Q;()UJkL0EUR0(aZoTBbE|#z7M>(rSIW9uMkGRssBr{m$>-z1R`HqL= zveJ6>j9v22vhNv@SB#}q4<=SV(TII(zfF(*aOlRUuU_AGFY>Fa!V)QJO0dxudU(T^ zAUhRFGI@O5xxgCTC2XXw!~*3LfnkXc&!1*oME=UIi3JMflBn0Cm*Gd|F&ow-)OQy~ z{HCVAYe@xzVQ=m!_}CsV7eqOncq7+7q4sY>B#MhtGq?EVWb7YI#4fJ zEf9f#j_F@#&ne#tL;0289c89QN|2-w=nAbZyrJW%oO#(c`cb)Oz!9`7NLbS7NYqnGy2h=6pfnDfy3Dtwrs(bdayyIH*B!pvH zMb`Co^cCGw(h(7PqP%pw-VTUMYqUl@%*0>oHWB#}uI^U$^&?;G+kJ`5NlU{1SNh+yN)gEh*-;;uNYqu{Ea1W`vB^&R*KaWpB9N}q<;C>_blMnI zXQ6lJ&TB_u9@*wHVFT#?br#G}?y=>>D2TZ+F)=+JZuYDqq`Cqm1~1ksme+$vEF>!= zZt}M7zl+lVQguEeuTsOWhy(lP8dXkY@h1w!Tvk9(fGoZ19JG#y?*MuGBMgg)Th|TJ zJYyIqGe6d^_sWU25lV=I)t3W&)$lm9M^YTDq+CjKJE1={^Ai@4>LbXPk`sq!2R+L= zWqu|U&8b6_0AMr#a}-DP3bt_YCs%sSq<+fpD~c!?br@@+CBn#btsG#0aDS~6$P(T0 zOof!>1fyofJ>Xt9yiTngGjw;KWh+<-k&%&!9->?dvuXg`+2&d)lS%61IF#gvi>2|i z`3zFP>mE((Bs_)$knN|AB($_|IqOAyhH>{O@gv7ok;Z=v%Go)+1t`@z4BW zpUL6L%7H6V7Z5P%!urLsG)SjoTKfdOpZ08=N7bdhBa-gF3-V3T+e$gz^wttW?|=t@ z5geE|w5}VC>&!_t%MSm%1+&jLg%_T@nQX5XTQZWyS0Y{ViW`Ty&0%fz;y6sI&8Bxy z_Y<50I--CP{PoPer>PsYMJ$XCH*gvex7-hQBOq%qs-^YL<1=E8t_PfEFNbc% zdCv$ai*C-`y?a+o!Yc?!GNUCMSMBEpUxO{%9HNlnDo=W61f9X55e_!3)_8PZ)1z2ql#cwx-N@Vlv>7iE&Y7EY5T3uk5Fx>Rux%xVhC}-DU}fX@88I0*7%#^8 zEj~vSdVugDMpZ^1KuI`&I*u|uDF)A`0|A4aOf*|K5gtaOpn&ciFb4aCR7(BlhfM($ zPLf&6IOgY+yZS^(iDjS0F%qW`z!?oO6@f?I=p`ci+&vrhF#j&sh+J2 zaIXSNc+oVn-Wf)ueD4q2?_UpAX;0j5>166cu8j$k|D^kps@UD)<#OlL6HR>4m#Oi% z`!Ra(Mc+*DtbYm2Z_uai8!d6Q$ccSCqk-UbNr$e`SU!6KTpm#P+^eSl{d<)LzJHl;q5I{sNQ3jaVFARV;Vvu)}5RX9%72AP9;b)SX zclP^tZh?u7lz|!l-}cwv52KRxpKZ#@$<@=^aN#@Ysi`nJM9aLkN2x)41R2IsChe%E z?7QJ+NCn06?bA`JHNf<`ei!<+*2G{^~%aO`8-0s_;mDa2+`m`!t_EV!x(=>4%n2U=sceR8@5 z9h1HjBw-}K%hBPU8u}^(bwXwY<}ng8k1lCRy%B~DvBlZ+iL_v4p5z{T8xc(zhtp_D zM!G{FyIJQ4nmycksT)^Oq92qKxdiDPl;}QF zY8doZm}=J3-F>wA!U_9P2u>LSw`~6& z(tGI!ej1ZK`{YbJkKz>(7>k^&s{SQ~#(W2De zUn!m0{xZj&b?%u<4D@SaxF>HBBoQ8iQaJLCh7LTwDI9b_1%Q4^Rf>{TLidT}DoSG> z^`8O@KQkb)o%zn+7(QcnGa~LU^=Dy+3K8ldn)>9dAp(6PgNI)RAn~~<_%QobBtF~g zwQ>Kei>LPtky3xzWhd+pCf-AMFhT3;?iNrrO>e*WIoOsAQC}&pd2MzK*@nlKTXRQG zz&2c)NX7OcU(>JhlM5CpdVn@i03=S}k+~=j=o{uh+CL^K`EsA_()jzLaqrg#(JE^GrvVKg zelPH_d<2daoN*CYdZgvT+cUAe z{s>w-0ZB0Ucbt1ly9{}Q$jl{R07YNM$M4=eha{sf##;K$GAz6b%}y@nxcJHJ7-FRO z;=CS(p&K;yoT(|#))fce)BlG)zJQ@}xE&v|MZa+5M8BhlrX8|T-i}n5`|*zB#)8qD zIkCxjfg@V~6|k`#Jo39>fZ$rCo??>HC44*OYPgSzY~xrbB{IBce%d! z1Oi-VbQ70#Y~)}s5r@>!ACWin@f~CB{s-`8q8^Y28EO#@3Uq(@!rZ?jI2#8lvmpFA z_8wq%&bRE8b0dB5I6JY*R}cY^RUa+7lWOo1p!fwU1vCq|g%hbW)Z&w+kzz?IWGUrA z>gG|S^B=UMHdBqAPO6{wv-RA41@nuciBpfDD=6Es&G}K2fgZx=k_u8VLim7jJ~1s= zU1X^73^Thzwu*qQ9Ah!B$A+mMK$t|U}qwggd1m&a)R~Gj>K|sv_(}51f>3nO;D<8Ed zn~$^xoB-m$@Veub|9TPhu4wQc54D4CR3tb{MnG2cRKN-Jq+7^|suvndc; z7#5OXziqVKx{cE!o+qeypj$}f%}9n6eYQ`6(J^^lKjYf+`S25p98&cLB$*Qcf@F{ZvdylLV+dw^iFNW7c0)!@$?MD~|`WY{BgglA*l!El^AzTpwjNLTZ`2@>R2{ zz^4Na%U#o*Zqa8%{FB6`Ff8g1jV*8|&~2c&q4$9y-QZ=%L+n6jT7ZpC*aQX}3%Y>r zECn4Ko-|2;eCYSkSL(=FsTzE}jLd<7`aFt`l9%)ex6~*wE`J!VtfcZfKlu?IwA!ob z;`MU+ z^W4fQd;U;sYpHm*LW;z)YW4!?cNf3qpAQ$nR+9Gak`kFw6v4A|)n30||8`B^cAquN zXH#V2sa@qh>iOLtKXKBo!*qxjGN8|SXLPxZ4Q_#bKuG9RR{&>ql39Gq;|+?;rS+{0 zm<0D3(|^SJ1z|POD0d{#zymsslH6@fkQ>cDShQcfPzgFO^X`(VM)FU&*Ys@m`SAT& z^x>wAmoWN0gu9xbobL?{a+>BRSqBRCwBO;M4yh0QidDA~pqEDuqbD$Q7GK`;#L+A7 zChthhGWBhOBX|z2g7W^USE|bWI9|7XJ*0&f5idz|2WL5Bcph}`uvKAe?H*n8`K9RP zkhkaS<0O)pT+Y!HQS2P~54|!U=#|YRmCHva??o1k_!!uf+_m&$H{IeQHWFCAYX}%d z;PX;Eznnd706 zUy?&u6E!Nkr$&2{_rZCYCX$87m8B)BWw`p}{e!5nwD zWjvtR_~&(zrJT+Q$R-U{4kq5%Ga#1K9sE*0iLN9wL z+e+h%By07NLEo>Irj?$2Fv(cw3vqJMIU1Xmi4#x-t}^C+TNPqYWxoDUPQd0myc?KZ z=uw7!G}uO^?0pT-WOXXX*yH+%PiSLBdB~@##%z@CR|?>C+M1!jD}QzQjIP80(CMNq zBwI$WcZ{6;q6CzipqMgxQn-j+vQ`Yz)0F=+JD}(KrQ@0phDr;e&Vdx|eF{w*5E0c} zUau*OIHr3pRW=y_713DgLMzF#fK>Z^z}bGYG*Q51At#z6BnR~9c`><-2~|LOc8=t{ zr9?8VhU|}ydkfVMb{erA;n+cmLxU26?Rin>;Jf7xB|1P?CJyMl6T#ArPc?qZ(c8B| zXI6M&JvdawpGRO^Xdwzx;y?wbCPCzEaR*~)s7`^b2%t;q>1Ml3{1s0_T^W%a$>ee^ zH8$W^yp3X6FqSt8U|W1RG1chsBgj@st_)0mz<=s{v=!MNYy36mpL7d<7Os;&nzNYk z7g&+uA<$D9y_%^GuHhpb9o)>33UOYz#uHk}2W4YEh;8}eU429;;M+{`ikeMJq2xOn zYP6dLb5ifg`0V^4x=is>ts#}?_CpB*N}T!JtJ0TgQ$(S=fi zJB*CT0$gL`8@F0mEGqX6E`&psfs4pZKGQE&f1c;8E=@sjkmnIlglxHc@TOWUl0j*$ zSty#qO-D?$e;GZk<{EMoKab7@iqP|$?%T$(ho;&t`o};u(by3Wo%J6Z8ynH6(8sAk zho{(HTOEZ*2-)1SK1@a{cAChDq@*&AI+V*#7aC7#JZ%GZBxH+fJL2iaKWE`ba+BCi7-NBW6^@H& zACNtTwdKO|lN`ek)2GHsvXroH!{lYB(H#g$#>rUygd6M`n@s6h;WRbq2kFx;SwE7= zwt#cECD-T);F8o~uYA`(jv6&ZoZty5zZJJ$@U|(HzbH0O#!ZX>P(LH@mH{sAM-^Ap_YmKR3jzN0%{CNt6SH#+~o(hnH z*aVfIIsXn@#c^?XN+j`=$fPoeZp80#57uUx#Ns-cu??9M6(n0+gjv)ax=Xwsh)gqA znYgCWK;P^nvFMH#<^0KloY57pCm3q4i@~Lg+BvrqaeqJ;So^2+B#NKVD*x&_&mFq< z&B?s#7-pkJuzH`?yJXq4`_o}0#}0#NEFTg>sm!J~MLD$=^?h0<)jVAM8znm#3(AY4 znL$rIysy-04Dqhci8aKjfWvmmRu)PHwLp#6J`#Q;h*Y!lZ=7AaOwV(?L@0ue-uW@& z3iWPeKb+PNH2l$!EbB{7oqS38jy4(;09MjGewX*WWc2fFo9pd|ow3#(w*571R*-HT z^c=-FLz1s;_xBHRP)$s7eKD0*qROruL&hJ`7V8m9L;fS~%J)5CUR)K_n7$5E+0fus z#d3zzydY3ZC5Z`4hn6wB_adS|{qG!Jp(S2!+9^y2ue`8UCNe%)YZ7kv!Yn7(Tpm(< zCpQOHTnD$Tt-!4CN55;)TQE(DBBhl_b*gN-O?Nu%fCeVwK9~=1x$h6?jo5%3jp-}5 z1IPeGW2b@qx2Rj_NCoz9p=YM7gZ2Wj;zwzq_IHxGxsP_)louNH&7z3*O zNqp1XUQO6!iPPXcB*IzV3$yDDs;_TYE*}{yT|D0$5OQ0tnrDkhfpiZu*MOt4`VlbR zT{Bu&R7e}AlYQ^z5Pq&Uyj!kAg+q0HrN}+s=#vn1KMaNK_`o9F0H|&6o^`pPyd_`K z5^mjAROR9Zl?fXeR+tN7-V2!*{bbjHkiWiAJ+gh7nqVbiuX;PH3@m+q4})*Z{BMO{ zU7Cy64T2vlA2DW~fw7Q-u8TO5gP2=lxKy&`vd{W&VE(bc%7<7`F)Vh>(_^Tre1TVT z6h9-Ltfv(L1RIu6;q+r=9JY`6vFgq2Bn|Qt%;*oDH)4t9QgoYox5b^A=|7nAgM<;8 z2LVR;4%eCfN`4?CE6Z}~I?A>XCjLNnn)SS;@7W9aZ@A0o78Y9wR?qGQ;C5Eu6RY~5 zI8$0)G#fF%st>Dzb;TF4jE$3Gk}G!O1H6G!zo_%idOwae6eMIA)T0SCx7j8x4b!6P zQJ!+Bz)P2WfaS41L|DL`R`WZEaO#wo)@zbTlm-9A1^B`EW0qCaI$uIk5DneD?bo0i z!1)S7=uLb06^8D+hv; zLqRHJ-^i!7-P%S1@NS~P*JrulEn~Z6^eA2H+ktJz3VbYmj-M>k*HhB3nw;(H16J`R z!uE*II?}X9W}N$H?k02XgC`WW-e%oDb4GgF0D41&(@}I-rka7GxUm6vY1|{w58cf( z>GC5V6s~@ycxam&A40Mk3+ z+77Qel>8X_CJUpE^Sf_Ux8Br~d6#P1=`#;BbTZ@vsQ5cI7`$! z>koDhsDkhIA=jS##!iRyh}Ab&oxqt|bj7_sxYQ#G&6u+)9~mewGMr`89OSPau)P>c zn4OW2EDIGK=HvT45xER!j=8@>#)FeH*u3k!+RTo68Jx~H-IhpzjBpGDV${(6&24K1ov25ilvSZPV~2NRj^ngi7;Nb${`i9AWR{)1T4 zcbyU!B2ILuTKAY6(=qJ;_NhaVL%W@w;m~D&9s}W$cItX;G01~|S$c>#oUl;eepg6} zWt@XlPa}aOu=lnl>GxbsL0m*-*M*9hL@mtx-M5e5@Ll8ly%H&pH_*|7-C4k}5s;9Y(l=p-0lB@4& z+f4nV-96{ePrpG=T+pHACgszY z`&^+H5N&U=BvJJ`&NVOfd8RSv)4*Nq%_{&tBrA%w>A80-a4BXPoRoT#tu7chEO{yj^?YhwOh<9McCv!!hvW6O4PD}->TaN| zT&n0@F+2PZO8%litm=farexsA;rY{V=lPd{=1yseoE1Me!@}cp>p}TyLS5jR)XL`X zpGC1nKx-T4p3j`+DiUK+jq^9YgSjFgaOJEPxGmS6r_Y#JqHfCKa<1OlnD!cQpIMli zj>k1L*HIERP`0s@{yS~;#O>PEoz7jwpSc{P_adMfqu8-{NEXSLFF{Tcbyy3t?4Gio zEXspvCw;X}G)s)-)eEe@aBz6^XUMzsLYm;nI>#Ux6r9x>g<4ek7S{_p~VYATO~|*pO~Nc9&+B46WoBK!}J7rUhr&_zqKZsX~7;0kpMdkpF8QoIz5E{pFDPPTf=RB4v=yZ|!&HElU` zkUo<5ry+G#Ost`TRmDY?2VZFj)zV2vu87tYYh(V@hkoy!X@$aZXfRmq{odm5@=_eWGQ5C4(AvqO>!q4yJ0VC^r z9V^WzNo8H>`OS%ggQMO8yirXIo(X6{KJ?(*sCV zBiE?%CtfA`V=@loUzxC+0bB|*8gP(EaLeFn?sx1j;@IuWYS0(@{Nq>G8L<^gP{s;& zE;j)Hs7n+_ONq3@ctw<6Hp6g-l(Fu%g=>hHkIxah;h-g;%G}IMy?jO+5fb$HS?jbN zAOGmCJ;g*E!_CL9F4xlS4ExnAEhkbRq&9z3T{#>fN2fgv{eYj$^qcR$qLNu2zMLku z-_~gOWScvl4i(YNARYYXne*i(RA(X_%oP6`u1!ShnVTn>xn}W*5aWtrjS^XnuYO#T zWNQDY)X;%b)qTbVxPcd$c6GjTg+bG}*e;vI3#nrHmlU-ElB5EZl$co0iOUSf=W^C7 zG0;DUYj~yq{KTuF^U;eUjxspfUyI0z8bM3b5$<*v8fi`4Fm$X38LR_x9e)R9Z^b-^ zVcbXTLH9T~;7;vQTD2(bw4$^$WBnFj$TUm!Smj%Z zTCX&mL4*O@JIimlaaQ8oz+o7wKg*`j2fc`htW7AlxC;NsbRiStkJF@rT>oMPxA$oL zOvux*fuC^gAR=$21t?F{rHS}qGek9y!St>mNd;v_k7)&ZEycX>BL~9*GLID&$nN^D z=LtY;U#@%?QbWuB_~hpRWn}EB=Tjn7XEcfGD1KBhDgI35cWrbD@F^PfT|{xc-*jI% zB8~KyMqUpQ!7H~B=A#iqE^=nNOnF_-hAS1Pz~}(dHtB=!nwk(t>W*|A zK=14X@_!*y=(p5lFum+AkjICJDb8er=RJI5Gg*=yrE{`kv}JIZM4YZF<2nHUic1kw z1TjJY2@J{MejU}G+koab^+PXe;N8q1cW$YBz)%oDcQg`w_Ml*kKlgnRIze#X@X?RR zc)Hp8n(b1pD=#sg?ul|~n2 z4~P;*2|6u*pG17i|9le2FkO-lNIAm0yaMw)Z_>`S2>}rA_+(Tq6K=CNTW{bdqGo`M z(F+*Tz?b^cH8!XNOTya2LvAnR7C~toL4v zu@ccQ$oc?}3GN#n(__TAqHIaqLjt)!D)II`%2O1gEXOtT=WhRTfeR2>0C~G|kmnc7 ziog>lLT$hIBv_LE=aYaQJG2Ln$05)92k61V`o4|WKeQfy8!ih`p!j6^D}IFF>0qpV z;^aei_)^awwYNNgd#paQAf2j1mUHU>DZeT^+be+mF_^D_doX~k_=EH!XZk%drRpb7bigi zksvsyGL@hW-*AQi8mW6v0_Lm!*OLHYGX!d8LZ0;}V5HEVHB30KKW_x2hXY!wu#8EK$u#$%SJ<3(h^FI=aVuU737`xv_x8G$ zL#k|?%jJ5=ErXT&rp>0CzAg^p~%naAI}_Q1Hyj!hh>V>_~%7*MiXC9@(;_DSa1K& z8v(xPzeb|7sbk=Wq1d0nN^C-!-n}`gBxf~vb#WS>-^vTs%jqkwKU+I3SmI4fwFc;+ z@&}Jh+CEof-P*Ba)hp4MR@fPj#7|jrgA+j8JNa;sLP4BCnxymg=S^cQSh3u@+_XP_ zW1C$b>XU4fxN&sH4qGOrf|z0$H0fALI49~Ig~flL)phr7WK>i~Tk_H~Bl4eNub}~v zQr@$B2PQFn`I-!&vrkE;#0sy@^9C&M76uLfMn2->*sd;ox7Euc6eE7{0;k2sY>nUPrbqj0T?M z6+stXrxbn=ykrYNoSVvnA`{&evSL3)0g=wkubH-a7<*r(=d<)gq{l=gzdelLC^s0>eL z*b4!1<6UCpbyOY3h5}F8Z+jZccUg{atOUp?5j4-s`W#_LSKYY5II07M_qUACiw){0 z9xLIWTc_kDE#!TWS^$;Go*w3%o%g@JTNF?S&UDxi-8ZOuv|`7U`0e8kBU0sW)2-qL zZLih-YTen?o4pUZp7r~0$1ksM%{t!L7oziT!6$_dZAZ(1X0>=BDNDE&+-=2=TqJ zc2WF)dVS{E+-Y)@Z{lHD^LM?Ha! z*zWs^n{~6;gLOVUn)e1@KrJ@l;=q;X)18d9KLJBM`Xr+cvI(p9)FCm3fkriy!Ajs z-I_X~+9**mXiR_LLkH&imujVk8A)j+3M~hOdhH1!e=Nlrfft`e&d7qnvE7SUo9`SS z)psP*XEdZ$*y}FRl-k<@!HKLyeJ1PIcjet*8+VngB|4&qudnJ~z~qsA3@Rvi6RH?( z(%;50iE^OljGW{qI1g0VoHGSnL2!kzz7!-sk!nKQ_uA(fm?PBZ>~t0uXd^C8MuDH7 zJ+MVsfzm&GX3xPE3pkRDe}apUJCqbgckRk1rM-;dGt@fjmAfH-whglsNI2jDv%>C= zOQK*CZZE_`czD?^Nj9&KR!Fp&$)MOs(O0AzYe{MW*q1L%wuPgnewF*0^m4*G-rw>5 zoEetb7v~E4?OUSO25@39+&1OU#+4hTX8K1!5#H>I>VL$p81Xau$(>IieK>h^QXl~i zWdVt?#AY~}J~nj~*ico1>oPyuK*r|57{`KbacU(yRjN?e%~;+eH6X9YQlmttWs#xe zFL_)1f^lw&p@oS$kX^KrtH`V}b6_w~o3bdDSB8CjsYjc>`UH2NgSwW;@T0ELgCJwQ zc2j`(Jl0TL1jtFYAcU$>u<{aODhEGRrhbZ=Cxl3(6}891relaD=wpPgV=ROWL25p% zR+~9tcj=)D^d*#CSeZcVZ;^$^U}0x?=5q3`UP<$BEzF%UiRU6=t8t7^b02RFnqY3> zBHHA5LR{0;n6xlY5hcMkgr^E8PXm&>%jSN%I8LeMF03(*`uq)g-3D68599aoy%NST z@i<9aayK1cq$Mdj&CjdT9&PN)R-=Rkn9t)Yg^ze6lVx@+{j|9(wgFh;z(x~5`h7?i zsv*+*Iv0^Ns=STeU=&146!axN@0m2lsg=uyWOQaxs;FgOCqnW_d0*kY0FsqtL)Mm@ zdg@3AE&MH!MgvfYOIWN`?qFl#9hj4`s&FlQ{dBjqQ{`$GM5C`XY|_MS7x!}tVQj$j zJgYJOYfrph={!UM-f#irYCsHc)`zQ2F<~i^@xBOUX%py4+PZmCppKK-t}?d#_^C_} z=$nt1x`T`OGsqzj3G^7u7D6fvy_uc$97!vvd4yXaJ&dY!xb^?FckS^|Z(E#cFdi|J zyn0}U6H!s}D2%}f2a#%`NFszhPC`vGGG0?2=c42-b-W2Zyq|+2uaie4c@-j$BaK4t z`c>yv_jB+6_mArzzuCXvXYaMvUVH7mK5MV<@{A&dpvBROs2Wu^gCG}({GR~5KAmu! z^Trp{^63?IK}|h05_zkG7#-gDlT`v5g`7+kyR2kg;oG`4wq6;C7=XuF`)j&2Y_(}Q zOf}*SXkhNi<71#1-1n4)XTz8X2XRM~I_z<9CVA5Qs2Gs4JyevvcEHUfs4my~;Hy{l zcYqH;2_XFj+#0yeh?Rzb864zCPG=;PC*o%v?6cp2`~(L8LbQO$LWeV)j=@mrj;|qF zA?P&s)ZIg00?7#Ai^ijXo==qo2NgaMfji3Kl z^xWrn04Nfry%P6+o29nEs-7L0ZVqvDY;WurQ)1Q#OtJS3BT0ERTEN|kouScDEXIUv zaU#SnZMX1tD^z$O+ZKx#p)n{gq4^`qvEc46^(Po3k=5AE*A5t_qEN2kN7?rt4BHm) zf<}~JR|sR-rCiyh8KJ%_iZrt}A~N4U8ri~Mer<~O0%aL<4JH5sXYaAG{z-#%Lb%Uf z+{9Iun<#Jiezp|$j`cNF!Dd~0bhx_%m%VJE2A?P(GCvCw=-7vB`KXGQB^$p^2EGHc zV!Teof8=mV9c}h97nk^XzOoCmZX6xza2dx$K!bJ+>KhhTr-`feH0$jB>WE|avrqj; z?Cjs0Ovn&R+x)G>{4kOjui2-qW=3#8#=TmySi<>bT-b9?&~#ZGZd{T<5rJjNhijyJ zK63M@_9C8Cz!0~EAg-rpc=M(b1NJq4DoY~!m~|8T%EAtVLggFi)|DWqwd=~)#Pa$i zEbAOZnHzp_BJ^0rcaVA|n&aBcLJk$ha}O%Jqij&M;GB^-6(} zDZBGTK;bs3j;Q`HxWJNN_Wt4QsKR=`>=`VjA2Bvu3%9XCwvi{27RueLR$i>zTwU@Y zv(NSIGi%nBYW|3o?c$mr+L!)X4-vJSgJ!SVkE>7;Kjd8H#b6~#r*5E34-D*)Io)Z& zJnBdZXe(+ccAq6z>=X9%HDMNf8U&faXOi0kszNYS|BW{zlNh~CgoA!vjKEtwR!YBp zz{Ql$#$9Ts_YEvOw~Ji;e9JNUW834$27oy!%}`|?v+=tCVN)VNCRNVIa==CX776Q; z$v+YxM_BM;0y8Ik@+Tx_Sq*C)#+E&T5D@F`Z5e^gt+E@Vm@TByk4#Rt69Dfz^~`0 zsN>#ng{}Babhx~WJg-iXR&iwAw&gVl>W!Uqkmy^Pfh3xW)tPr_T4%y7KSA}eQ%R7h1qLfgNqE`k)SO=I(3#jAdd6rUEah;v$=^TB5zJ^F~q#r zB&!wMj#zhwx$Cd^*E7Z1y}zQRg*XdS1Ez^PZ#B;!?-)wxnv`vETK3gyp#y zs!#OTq81Y>)8K-|q9tBinDePL8393WT1P)E{^+Im3rl&SvULH|WdMv64)%^_Hjq+T zSPksS(0O0FkG0S-4Ss!>Aqpyl;8OY0%;+4JR7m{&V2OWn2Vcj!u{CL7eAz~!Kgq;z z8y(|!WozP=@CL3mtOyUzVWpru2pumwa(VxT#UhU4a@k`B@=6hm5m>!%QiTN++KS+0&(YN-SC-M?mi2d zi}duB+eJQAUy+my}Y@edHHOa^|m;^1Eoc^|&(qJ+_hb z&}c`>c4N`$R9(8ODShXOhZ+@B*39$!XAggVvkU<6*C77Z0LTmrg;h|`>f`3mr$73$ z8>_cU1tN~?#P&XGW8gT{bl`@a>CLtIsB^xJU39QzguVGE!p47-ljLA|@zYiDx!PeKJ{ z$Q_}6#?(W~Zk&6W#WwO|m0!TIj|0&^8N6dEOxRdUVkKi5<1#EB52sO9AUlCsDl^V{G=l+nO-ZXR6xp^QJ%t%~fQaAZI_c>knVerxjLD=0cx`)4>`=XxDFUhZKQe=!Yf z8<*oga|XybbKlA+mB^0Q@$0!1uR{tQdaW%qM@8tewqz=n)QXDm%@woq2^aEU+SwU74($|yhZf#} zm`QG2y!rOg+#{q=v+V_qP%a#7+z~n3Em^#3N%%z`u~+wa)RJs1!%` zl4Q~AhOgpy_jK!f02RU}xA;-&Wj| z+@x_j8sh~5uAK^~d5TVKQ&f^UhRZxrGs$=1+LXIFJN~K&XMhq)3a1*Wx1vGVm5)0^ zC13V2VPH6#{X&ylySe1-;0Po8s<*Imv9YRCtB~g2)&zZVe$%ZSnw7*z>5& zNU>6_Sh&U96(&atW2yHeP9~!~Lva4H1#>vm_o4|?BBOk8oEtllKua(DbRaA32Cv#k z;cE_+%0`|gV$btGAGi$Z(7n5#x&EMG$RwPBd1-c1HvT>@i1q7-J;dplW^3?QQ}|8h zm?PVO!0VKHUFo5cBxK#F97oOxyHaXOXQ3J4g^dT>kSL``+KLvQ=UlprhxSg-`M z12wF|J;(F&VELGd{7dnzr_fs- zDT(@2Za_q)jOb0{(Hyjc{E%Z?A5U4e0tm56(b@hn$8?PPm;&1~8ThOPG@_&EYWzVL zV6!jYT!l)|p(CU|!_IMNhVK{UK=k>&J`5qDpNE}z1cF=MnO%8)1D22|n2>-m#(E4y zh@g?L?i+%f&t?Bn3J>jx=dv9F#|dO7k9oeNp9${T4)QsC!sdq+2#luI0)3n!?H&In zizH~?|6oz??+woe2AS-AjDgP-72#M>GjZW>y)ZBbt^Q`9qJn8y{NMCL_mV;|)pD_l z{6MI*tR&gso@_ArTc4t-u-dpn3tZscon7#*Qm0Y>`*a}GZvd+yZH}q`ahjk#3v|G| zAZi}>Q+ogzCMs-Wo#MZ#@mpY}OWywS&x}jrfHFdmA|H(Y!-Pkb0Rv#bBoNVlZ#N)= n!8O411^z74|C9YcA?+mzO#_V(O)fS)Huy8tJ4nvgId%1)Bq*54 literal 0 HcmV?d00001 From aff49d5b8c12a4d18e862e1d57d326218d448c9d Mon Sep 17 00:00:00 2001 From: JohnHe <12479235+johnhaxx7@users.noreply.github.com> Date: Wed, 22 Jun 2022 19:56:30 -0700 Subject: [PATCH 16/21] executor: Show placement information in SHOW TABLE x REGIONS (#35202) Co-authored-by: Ti Chi Robot --- executor/show.go | 77 +++++++++++++++- executor/split.go | 16 +++- executor/splittest/split_table_test.go | 123 ++++++++++++++++++++++++- planner/core/planbuilder.go | 4 +- 4 files changed, 209 insertions(+), 11 deletions(-) diff --git a/executor/show.go b/executor/show.go index 9075444fd53f4..d1424244bdb47 100644 --- a/executor/show.go +++ b/executor/show.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -99,6 +100,12 @@ type ShowExec struct { Extended bool // Used for `show extended columns from ...` } +type showTableRegionRowItem struct { + regionMeta + schedulingConstraints string + schedulingState string +} + // Next implements the Executor Next interface. func (e *ShowExec) Next(ctx context.Context, req *chunk.Chunk) error { req.GrowAndReset(e.maxChunkSize) @@ -219,7 +226,7 @@ func (e *ShowExec) fetchAll(ctx context.Context) error { case ast.ShowAnalyzeStatus: return e.fetchShowAnalyzeStatus() case ast.ShowRegions: - return e.fetchShowTableRegions() + return e.fetchShowTableRegions(ctx) case ast.ShowBuiltins: return e.fetchShowBuiltins() case ast.ShowBackups: @@ -1818,7 +1825,7 @@ func (e *ShowExec) appendRow(row []interface{}) { } } -func (e *ShowExec) fetchShowTableRegions() error { +func (e *ShowExec) fetchShowTableRegions(ctx context.Context) error { store := e.ctx.GetStore() tikvStore, ok := store.(helper.Storage) if !ok { @@ -1858,22 +1865,82 @@ func (e *ShowExec) fetchShowTableRegions() error { // Get table regions from from pd, not from regionCache, because the region cache maybe outdated. var regions []regionMeta if len(e.IndexName.L) != 0 { + // show table * index * region indexInfo := tb.Meta().FindIndexByName(e.IndexName.L) if indexInfo == nil { return plannercore.ErrKeyDoesNotExist.GenWithStackByArgs(e.IndexName, tb.Meta().Name) } regions, err = getTableIndexRegions(indexInfo, physicalIDs, tikvStore, splitStore) } else { + // show table * region regions, err = getTableRegions(tb, physicalIDs, tikvStore, splitStore) } + if err != nil { + return err + } + regionRowItem, err := e.fetchSchedulingInfo(ctx, regions, tb.Meta()) if err != nil { return err } - e.fillRegionsToChunk(regions) + + e.fillRegionsToChunk(regionRowItem) return nil } +func (e *ShowExec) fetchSchedulingInfo(ctx context.Context, regions []regionMeta, tbInfo *model.TableInfo) ([]showTableRegionRowItem, error) { + scheduleState := make(map[int64]infosync.PlacementScheduleState) + schedulingConstraints := make(map[int64]*model.PlacementSettings) + regionRowItem := make([]showTableRegionRowItem, 0) + tblPlacement, err := e.getTablePlacement(tbInfo) + if err != nil { + return nil, err + } + + if tbInfo.GetPartitionInfo() != nil { + // partitioned table + for _, part := range tbInfo.GetPartitionInfo().Definitions { + _, err = fetchScheduleState(ctx, scheduleState, part.ID) + if err != nil { + return nil, err + } + placement, err := e.getPolicyPlacement(part.PlacementPolicyRef) + if err != nil { + return nil, err + } + if placement == nil { + schedulingConstraints[part.ID] = tblPlacement + } else { + schedulingConstraints[part.ID] = placement + } + } + } else { + // un-partitioned table or index + schedulingConstraints[tbInfo.ID] = tblPlacement + _, err = fetchScheduleState(ctx, scheduleState, tbInfo.ID) + if err != nil { + return nil, err + } + } + var constraintStr string + var scheduleStateStr string + for i := range regions { + if constraint, ok := schedulingConstraints[regions[i].physicalID]; ok && constraint != nil { + constraintStr = constraint.String() + scheduleStateStr = scheduleState[regions[i].physicalID].String() + } else { + constraintStr = "" + scheduleStateStr = "" + } + regionRowItem = append(regionRowItem, showTableRegionRowItem{ + regionMeta: regions[i], + schedulingConstraints: constraintStr, + schedulingState: scheduleStateStr, + }) + } + return regionRowItem, nil +} + func getTableRegions(tb table.Table, physicalIDs []int64, tikvStore helper.Storage, splitStore kv.SplittableStore) ([]regionMeta, error) { regions := make([]regionMeta, 0, len(physicalIDs)) uniqueRegionMap := make(map[uint64]struct{}) @@ -1900,7 +1967,7 @@ func getTableIndexRegions(indexInfo *model.IndexInfo, physicalIDs []int64, tikvS return regions, nil } -func (e *ShowExec) fillRegionsToChunk(regions []regionMeta) { +func (e *ShowExec) fillRegionsToChunk(regions []showTableRegionRowItem) { for i := range regions { e.result.AppendUint64(0, regions[i].region.Id) e.result.AppendString(1, regions[i].start) @@ -1926,6 +1993,8 @@ func (e *ShowExec) fillRegionsToChunk(regions []regionMeta) { e.result.AppendUint64(8, regions[i].readBytes) e.result.AppendInt64(9, regions[i].approximateSize) e.result.AppendInt64(10, regions[i].approximateKeys) + e.result.AppendString(11, regions[i].schedulingConstraints) + e.result.AppendString(12, regions[i].schedulingState) } } diff --git a/executor/split.go b/executor/split.go index e31f894849685..52dba35747c43 100644 --- a/executor/split.go +++ b/executor/split.go @@ -620,6 +620,9 @@ type regionMeta struct { readBytes uint64 approximateSize int64 approximateKeys int64 + + // this is for propagating scheduling info for this region + physicalID int64 } func getPhysicalTableRegions(physicalTableID int64, tableInfo *model.TableInfo, tikvStore helper.Storage, s kv.SplittableStore, uniqueRegionMap map[uint64]struct{}) ([]regionMeta, error) { @@ -784,12 +787,15 @@ func getRegionMeta(tikvStore helper.Storage, regionMetas []*tikv.Region, uniqueR continue } uniqueRegionMap[r.GetID()] = struct{}{} - regions = append(regions, regionMeta{ - region: r.GetMeta(), - leaderID: r.GetLeaderPeerID(), - storeID: r.GetLeaderStoreID(), - }) + regions = append(regions, + regionMeta{ + region: r.GetMeta(), + leaderID: r.GetLeaderPeerID(), + storeID: r.GetLeaderStoreID(), + physicalID: physicalTableID, + }) } + regions, err := getRegionInfo(tikvStore, regions) if err != nil { return regions, err diff --git a/executor/splittest/split_table_test.go b/executor/splittest/split_table_test.go index d578b53638c5c..9f5ceec8a01ea 100644 --- a/executor/splittest/split_table_test.go +++ b/executor/splittest/split_table_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" @@ -266,7 +267,7 @@ func TestShowTableRegion(t *testing.T) { // 4 regions to store record data. // 1 region to store index data. require.Len(t, rows, 5) - require.Len(t, rows[0], 11) + require.Len(t, rows[0], 13) tbl := external.GetTableByName(t, tk, "test", "t_regions") // Check the region start key. require.Equal(t, fmt.Sprintf("t_%d_r", tbl.Meta().ID), rows[0][1]) @@ -274,6 +275,11 @@ func TestShowTableRegion(t *testing.T) { require.Equal(t, fmt.Sprintf("t_%d_r_0", tbl.Meta().ID), rows[2][1]) require.Equal(t, fmt.Sprintf("t_%d_r_5000", tbl.Meta().ID), rows[3][1]) require.Equal(t, fmt.Sprintf("t_%d_r", tbl.Meta().ID), rows[4][2]) + // Check scheduling constraint and scheduling state default value + for i := range rows { + require.Equal(t, "", rows[i][11]) + require.Equal(t, "", rows[i][12]) + } // Test show table index regions. tk.MustQuery(`split table t_regions index idx between (-1000) and (1000) regions 4;`).Check(testkit.Rows("4 1")) @@ -281,11 +287,17 @@ func TestShowTableRegion(t *testing.T) { rows = re.Rows() // The index `idx` of table t_regions should have 4 regions now. require.Len(t, rows, 4) + require.Len(t, rows[0], 13) // Check the region start key. require.Regexp(t, fmt.Sprintf("t_%d.*", tbl.Meta().ID), rows[0][1]) require.Regexp(t, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID), rows[1][1]) require.Regexp(t, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID), rows[2][1]) require.Regexp(t, fmt.Sprintf("t_%d_i_1_.*", tbl.Meta().ID), rows[3][1]) + // Check scheduling constraint and scheduling state default value + for i := range rows { + require.Equal(t, "", rows[i][11]) + require.Equal(t, "", rows[i][12]) + } re = tk.MustQuery("show table t_regions regions") rows = re.Rows() @@ -593,4 +605,113 @@ func TestShowTableRegion(t *testing.T) { // Test show table partition region on non-partition table. err = tk.QueryToErr("show table t partition (p3,p4) index idx regions") require.True(t, terror.ErrorEqual(err, plannercore.ErrPartitionClauseOnNonpartitioned)) + + // Test scheduling info for un-partitioned table with placement policy + tk.MustExec("drop table if exists t1_scheduling") + tk.MustExec("drop placement policy if exists pa1") + tk.MustExec("create placement policy p1 " + + "PRIMARY_REGION=\"cn-east-1\" " + + "REGIONS=\"cn-east-1,cn-east-2\"" + + "SCHEDULE=\"EVEN\"") + tk.MustExec("create table t1_scheduling (id int) placement policy p1") + re = tk.MustQuery("show table t1_scheduling regions") + rows = re.Rows() + require.Len(t, rows, 1) + require.Len(t, rows[0], 13) + tbl = external.GetTableByName(t, tk, "test", "t1_scheduling") + require.Equal(t, fmt.Sprintf("t_%d_", tbl.Meta().ID), rows[0][1]) + require.Equal(t, "PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\"", rows[0][11]) + require.Equal(t, infosync.PlacementScheduleStatePending.String(), rows[0][12]) + + // Test scheduling info for partitioned table with placement policy + tk.MustExec("drop table if exists t2_scheduling") + tk.MustExec("drop placement policy if exists p2") + tk.MustExec("create placement policy p2 " + + "LEADER_CONSTRAINTS=\"[+region=us-east-1]\" " + + "FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\" " + + "FOLLOWERS=3") + tk.MustExec("create table t2_scheduling (id INT) placement policy p1 partition by range (id) (" + + "partition p0 values less than (100) placement policy p2," + + "partition p1 values less than (1000)," + + "partition p2 values less than (10000)" + + ")") + re = tk.MustQuery("show table t2_scheduling regions") + rows = re.Rows() + require.Len(t, rows, 3) + require.Len(t, rows[0], 13) + tbl = external.GetTableByName(t, tk, "test", "t2_scheduling") + require.Equal(t, "LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\"", rows[0][11]) + require.Equal(t, "PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\"", rows[1][11]) + require.Equal(t, "PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\"", rows[2][11]) + require.Equal(t, infosync.PlacementScheduleStatePending.String(), rows[0][12]) + require.Equal(t, infosync.PlacementScheduleStatePending.String(), rows[1][12]) + require.Equal(t, infosync.PlacementScheduleStatePending.String(), rows[2][12]) + + // Test scheduling info for partitioned table after split to regions + tk.MustExec("drop table if exists t3_scheduling") + tk.MustExec("create table t3_scheduling (id INT) placement policy p1 partition by range (id) (" + + "partition p0 values less than (100) placement policy p2," + + "partition p1 values less than (1000)," + + "partition p2 values less than (10000)" + + ")") + tk.MustQuery("split partition table t3_scheduling between (0) and (10000) regions 4") + re = tk.MustQuery("show table t3_scheduling regions") + rows = re.Rows() + require.Len(t, rows, 12) + require.Len(t, rows[0], 13) + for i := range rows { + if i < 4 { + require.Equal(t, "LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\"", rows[i][11]) + } else { + require.Equal(t, "PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\"", rows[i][11]) + } + require.Equal(t, infosync.PlacementScheduleStatePending.String(), rows[i][12]) + } + + // Test scheduling info for un-partitioned table after split index to regions + tk.MustExec("drop table if exists t4_scheduling") + tk.MustExec("create table t4_scheduling (id INT, val INT, index idx1(val)) placement policy p1") + tk.MustQuery("split table t4_scheduling index idx1 between (0) and (12345) regions 3") + re = tk.MustQuery("show table t4_scheduling regions") + rows = re.Rows() + require.Len(t, rows, 4) + require.Len(t, rows[0], 13) + for i := range rows { + require.Equal(t, "PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\"", rows[i][11]) + require.Equal(t, infosync.PlacementScheduleStatePending.String(), rows[i][12]) + } + + // Test scheduling info for partitioned table after split index to regions + tk.MustExec("drop table if exists t5_scheduling") + tk.MustExec("create table t5_scheduling (id INT, val INT, index idx1(val)) placement policy p1 partition by range (id) (" + + "partition p0 values less than (100) placement policy p2," + + "partition p1 values less than (1000)," + + "partition p2 values less than (10000)" + + ")") + tk.MustQuery("split table t5_scheduling index idx1 between (0) and (12345) regions 3") + re = tk.MustQuery("show table t5_scheduling regions") + rows = re.Rows() + require.Len(t, rows, 12) + require.Len(t, rows[0], 13) + for i := range rows { + if i < 4 { + require.Equal(t, "LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\"", rows[i][11]) + } else { + require.Equal(t, "PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\"", rows[i][11]) + } + require.Equal(t, infosync.PlacementScheduleStatePending.String(), rows[i][12]) + } + re = tk.MustQuery("show table t5_scheduling index idx1 regions") + rows = re.Rows() + require.Len(t, rows, 9) + require.Len(t, rows[0], 13) + for i := range rows { + if i < 3 { + require.Equal(t, "LEADER_CONSTRAINTS=\"[+region=us-east-1]\" FOLLOWERS=3 FOLLOWER_CONSTRAINTS=\"[+region=us-east-2]\"", rows[i][11]) + } else { + require.Equal(t, "PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\"", rows[i][11]) + } + require.Equal(t, infosync.PlacementScheduleStatePending.String(), rows[i][12]) + } + } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index e4f60984a75b3..ad8388f15a8f4 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2769,7 +2769,7 @@ func buildShowDDLJobsFields() (*expression.Schema, types.NameSlice) { } func buildTableRegionsSchema() (*expression.Schema, types.NameSlice) { - schema := newColumnsWithNames(11) + schema := newColumnsWithNames(13) schema.Append(buildColumnWithName("", "REGION_ID", mysql.TypeLonglong, 4)) schema.Append(buildColumnWithName("", "START_KEY", mysql.TypeVarchar, 64)) schema.Append(buildColumnWithName("", "END_KEY", mysql.TypeVarchar, 64)) @@ -2781,6 +2781,8 @@ func buildTableRegionsSchema() (*expression.Schema, types.NameSlice) { schema.Append(buildColumnWithName("", "READ_BYTES", mysql.TypeLonglong, 4)) schema.Append(buildColumnWithName("", "APPROXIMATE_SIZE(MB)", mysql.TypeLonglong, 4)) schema.Append(buildColumnWithName("", "APPROXIMATE_KEYS", mysql.TypeLonglong, 4)) + schema.Append(buildColumnWithName("", "SCHEDULING_CONSTRAINTS", mysql.TypeVarchar, 256)) + schema.Append(buildColumnWithName("", "SCHEDULING_STATE", mysql.TypeVarchar, 16)) return schema.col2Schema(), schema.names } From df9b54bcb31f2c356e70b5d901f4b44192855fa3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 23 Jun 2022 11:16:36 +0800 Subject: [PATCH 17/21] txn, session: Add (cluster_)trx_summary table (#28835) ref pingcap/tidb#34457 --- config/config.go | 32 +++++- docs/design/2021-04-26-lock-view.md | 6 +- docs/tidb_http_api.md | 27 +++++ executor/builder.go | 4 +- executor/infoschema_reader.go | 27 +++++ infoschema/cluster.go | 3 + infoschema/infoschema_test.go | 1 + infoschema/tables.go | 10 ++ infoschema/tables_test.go | 31 ++++++ server/http_handler.go | 29 +++++ session/txn.go | 6 ++ session/txninfo/summary.go | 162 ++++++++++++++++++++++++++++ tidb-server/main.go | 3 + 13 files changed, 336 insertions(+), 5 deletions(-) create mode 100644 session/txninfo/summary.go diff --git a/config/config.go b/config/config.go index a701b1501bf15..48e069bcfb50b 100644 --- a/config/config.go +++ b/config/config.go @@ -255,7 +255,8 @@ type Config struct { // BallastObjectSize set the initial size of the ballast object, the unit is byte. BallastObjectSize int `toml:"ballast-object-size" json:"ballast-object-size"` // EnableGlobalKill indicates whether to enable global kill. - EnableGlobalKill bool `toml:"enable-global-kill" json:"enable-global-kill"` + EnableGlobalKill bool `toml:"enable-global-kill" json:"enable-global-kill"` + TrxSummary TrxSummary `toml:"transaction-summary" json:"transaction-summary"` // The following items are deprecated. We need to keep them here temporarily // to support the upgrade process. They can be removed in future. @@ -721,6 +722,22 @@ type PessimisticTxn struct { PessimisticAutoCommit AtomicBool `toml:"pessimistic-auto-commit" json:"pessimistic-auto-commit"` } +// TrxSummary is the config for transaction summary collecting. +type TrxSummary struct { + // how many transaction summary in `transaction_summary` each TiDB node should keep. + TransactionSummaryCapacity uint `toml:"transaction-summary-capacity" json:"transaction-summary-capacity"` + // how long a transaction should be executed to make it be recorded in `transaction_id_digest`. + TransactionIDDigestMinDuration uint `toml:"transaction-id-digest-min-duration" json:"transaction-id-digest-min-duration"` +} + +// Valid Validatse TrxSummary configs +func (config *TrxSummary) Valid() error { + if config.TransactionSummaryCapacity > 5000 { + return errors.New("transaction-summary.transaction-summary-capacity should not be larger than 5000") + } + return nil +} + // DefaultPessimisticTxn returns the default configuration for PessimisticTxn func DefaultPessimisticTxn() PessimisticTxn { return PessimisticTxn{ @@ -731,6 +748,15 @@ func DefaultPessimisticTxn() PessimisticTxn { } } +// DefaultTrxSummary returns the default configuration for TrxSummary collector +func DefaultTrxSummary() TrxSummary { + // TrxSummary is not enabled by default before GA + return TrxSummary{ + TransactionSummaryCapacity: 500, + TransactionIDDigestMinDuration: 2147483647, + } +} + // Plugin is the config for plugin type Plugin struct { Dir string `toml:"dir" json:"dir"` @@ -916,6 +942,7 @@ var defaultConf = Config{ EnableForwarding: defTiKVCfg.EnableForwarding, NewCollationsEnabledOnFirstBootstrap: true, EnableGlobalKill: true, + TrxSummary: DefaultTrxSummary(), } var ( @@ -1183,6 +1210,9 @@ func (c *Config) Valid() error { if err := c.TiKVClient.Valid(); err != nil { return err } + if err := c.TrxSummary.Valid(); err != nil { + return err + } if c.Performance.TxnTotalSizeLimit > 1<<40 { return fmt.Errorf("txn-total-size-limit should be less than %d", 1<<40) diff --git a/docs/design/2021-04-26-lock-view.md b/docs/design/2021-04-26-lock-view.md index 3db4cb6dd8666..09b6e8a114854 100644 --- a/docs/design/2021-04-26-lock-view.md +++ b/docs/design/2021-04-26-lock-view.md @@ -220,11 +220,11 @@ Default: 10000 #### TiDB Config File `transaction-summary.transaction-id-digest-min-duration` -Specifies how long a transaction should be executed to make it be recorded in `transaction_id_digest`. +Specifies how long a transaction should be executed to make it be recorded in `transaction_id_digest` and considered when calculating `trx_summary`. Dynamically changeable via HTTP API. -Value: 0 to 60000 +Value: 0 to 2147483647 Unit: ms @@ -232,7 +232,7 @@ Default: 1000 #### TiDB Config File `transaction-summary.transaction-summary-capacity` -Specifies how many transaction summary in `transaction_summary` each TiDB node should keep. +Specifies how many transaction summary in `trx_summary` each TiDB node should keep. Dynamically changeable via HTTP API. diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index c9d63f5ea4e6b..e820b23324fc2 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -557,3 +557,30 @@ timezone.* # reset the size of the ballast object (2GB in this example) curl -v -X POST -d "2147483648" http://{TiDBIP}:10080/debug/ballast-object-sz ``` + + +1. Set deadlock history table capacity + + ```shell + curl -X POST -d "deadlock_history_capacity={number}" http://{TiDBIP}:10080/settings + ``` + +1. Set whether deadlock history (`DEADLOCKS`) collect retryable deadlocks + + ```shell + curl -X POST -d "deadlock_history_collect_retryable={bool_val}" http://{TiDBIP}:10080/settings + ``` + +1. Set transaction_id to digest mapping minimum duration threshold, only transactions which last longer than this threshold will be collected into `TRX_SUMMARY`. + + ```shell + curl -X POST -d "transaction_id_digest_min_duration={number}" http://{TiDBIP}:10080/settings + ``` + + Unit of duration here is ms. + +1. Set transaction summary table (`TRX_SUMMARY`) capacity + + ```shell + curl -X POST -d "transaction_summary_capacity={number}" http://{TiDBIP}:10080/settings + ``` diff --git a/executor/builder.go b/executor/builder.go index de77b2b88f252..8a44c09aaf033 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1768,7 +1768,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableClientErrorsSummaryByUser), strings.ToLower(infoschema.TableClientErrorsSummaryByHost), strings.ToLower(infoschema.TableAttributes), - strings.ToLower(infoschema.TablePlacementPolicies): + strings.ToLower(infoschema.TablePlacementPolicies), + strings.ToLower(infoschema.TableTrxSummary), + strings.ToLower(infoschema.ClusterTableTrxSummary): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 8eea08a6675b5..7f7766eda2b23 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -168,6 +168,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex err = e.setDataForAttributes(sctx, is) case infoschema.TablePlacementPolicies: err = e.setDataFromPlacementPolicies(sctx) + case infoschema.TableTrxSummary: + err = e.setDataForTrxSummary(sctx) + case infoschema.ClusterTableTrxSummary: + err = e.setDataForClusterTrxSummary(sctx) } if err != nil { return nil, err @@ -2181,6 +2185,29 @@ func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context return nil } +func (e *memtableRetriever) setDataForTrxSummary(ctx sessionctx.Context) error { + hasProcessPriv := hasPriv(ctx, mysql.ProcessPriv) + if !hasProcessPriv { + return nil + } + rows := txninfo.Recorder.DumpTrxSummary() + e.rows = rows + return nil +} + +func (e *memtableRetriever) setDataForClusterTrxSummary(ctx sessionctx.Context) error { + err := e.setDataForTrxSummary(ctx) + if err != nil { + return err + } + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) + if err != nil { + return err + } + e.rows = rows + return nil +} + type stmtSummaryTableRetriever struct { dummyCloser table *model.TableInfo diff --git a/infoschema/cluster.go b/infoschema/cluster.go index 8ba852de0989b..d1d4f3c5a7fac 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -44,6 +44,8 @@ const ( ClusterTableTiDBTrx = "CLUSTER_TIDB_TRX" // ClusterTableDeadlocks is the string constant of cluster dead lock table. ClusterTableDeadlocks = "CLUSTER_DEADLOCKS" + // ClusterTableDeadlocks is the string constant of cluster transaction summary table. + ClusterTableTrxSummary = "CLUSTER_TRX_SUMMARY" ) // memTableToClusterTables means add memory table to cluster table. @@ -55,6 +57,7 @@ var memTableToClusterTables = map[string]string{ TableStatementsSummaryEvicted: ClusterTableStatementsSummaryEvicted, TableTiDBTrx: ClusterTableTiDBTrx, TableDeadlocks: ClusterTableDeadlocks, + TableTrxSummary: ClusterTableTrxSummary, } func init() { diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index 710914ad41159..e414f97d02906 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -294,6 +294,7 @@ func TestInfoTables(t *testing.T) { "TIDB_TRX", "DEADLOCKS", "PLACEMENT_POLICIES", + "TRX_SUMMARY", } for _, tbl := range infoTables { tb, err1 := is.TableByName(util.InformationSchemaName, model.NewCIStr(tbl)) diff --git a/infoschema/tables.go b/infoschema/tables.go index 57e867808cede..622eb1ef9452c 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -180,6 +180,8 @@ const ( TableAttributes = "ATTRIBUTES" // TablePlacementPolicies is the string constant of placement policies table. TablePlacementPolicies = "PLACEMENT_POLICIES" + // TableTrxSummary is the string constant of transaction summary table. + TableTrxSummary = "TRX_SUMMARY" ) const ( @@ -278,6 +280,8 @@ var tableIDMap = map[string]int64{ TableAttributes: autoid.InformationSchemaDBID + 77, TableTiDBHotRegionsHistory: autoid.InformationSchemaDBID + 78, TablePlacementPolicies: autoid.InformationSchemaDBID + 79, + TableTrxSummary: autoid.InformationSchemaDBID + 80, + ClusterTableTrxSummary: autoid.InformationSchemaDBID + 81, } // columnInfo represents the basic column information of all kinds of INFORMATION_SCHEMA tables @@ -1463,6 +1467,11 @@ var tableAttributesCols = []columnInfo{ {name: "RANGES", tp: mysql.TypeBlob, size: types.UnspecifiedLength}, } +var tableTrxSummaryCols = []columnInfo{ + {name: "DIGEST", tp: mysql.TypeVarchar, size: 16, flag: mysql.NotNullFlag, comment: "Digest of a transaction"}, + {name: txninfo.AllSQLDigestsStr, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"}, +} + var tablePlacementPoliciesCols = []columnInfo{ {name: "POLICY_ID", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, {name: "CATALOG_NAME", tp: mysql.TypeVarchar, size: 512, flag: mysql.NotNullFlag}, @@ -1880,6 +1889,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableDataLockWaits: tableDataLockWaitsCols, TableAttributes: tableAttributesCols, TablePlacementPolicies: tablePlacementPoliciesCols, + TableTrxSummary: tableTrxSummaryCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 962a8db8e0d13..cf9075ee21f29 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1479,6 +1479,37 @@ func TestTiDBTrx(t *testing.T) { "[null,null,\"update `test_tidb_trx` set `i` = `i` + ?\"]")) } +func TestTiDBTrxSummary(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := newTestKitWithRoot(t, store) + tk.MustExec("drop table if exists test_tidb_trx") + tk.MustExec("create table test_tidb_trx(i int)") + _, beginDigest := parser.NormalizeDigest("begin") + _, digest := parser.NormalizeDigest("update test_tidb_trx set i = i + 1") + _, commitDigest := parser.NormalizeDigest("commit") + txninfo.Recorder.Clean() + txninfo.Recorder.SetMinDuration(500 * time.Millisecond) + defer txninfo.Recorder.SetMinDuration(2147483647) + txninfo.Recorder.ResizeSummaries(128) + defer txninfo.Recorder.ResizeSummaries(0) + tk.MustExec("begin") + tk.MustExec("update test_tidb_trx set i = i + 1") + time.Sleep(1 * time.Second) + tk.MustExec("update test_tidb_trx set i = i + 1") + tk.MustExec("commit") + // it is possible for TRX_SUMMARY to have other rows (due to parallel execution of tests) + for _, row := range tk.MustQuery("select * from information_schema.TRX_SUMMARY;").Rows() { + // so we just look for the row we are looking for + if row[0] == "1bb679108d0012a8" { + require.Equal(t, strings.TrimSpace(row[1].(string)), "[\""+beginDigest.String()+"\",\""+digest.String()+"\",\""+digest.String()+"\",\""+commitDigest.String()+"\"]") + return + } + } + t.Fatal("cannot find the expected row") +} + func TestInfoSchemaDeadlockPrivilege(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/server/http_handler.go b/server/http_handler.go index 4ef260eed12dd..5db67bcd3be0c 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -47,6 +47,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -755,6 +756,34 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { return } } + if transactionSummaryCapacity := req.Form.Get("transaction_summary_capacity"); transactionSummaryCapacity != "" { + capacity, err := strconv.Atoi(transactionSummaryCapacity) + if err != nil { + writeError(w, errors.New("illegal argument")) + return + } else if capacity < 0 || capacity > 5000 { + writeError(w, errors.New("transaction_summary_capacity out of range, should be in 0 to 5000")) + return + } + cfg := config.GetGlobalConfig() + cfg.TrxSummary.TransactionSummaryCapacity = uint(capacity) + config.StoreGlobalConfig(cfg) + txninfo.Recorder.ResizeSummaries(uint(capacity)) + } + if transactionIDDigestMinDuration := req.Form.Get("transaction_id_digest_min_duration"); transactionIDDigestMinDuration != "" { + duration, err := strconv.Atoi(transactionIDDigestMinDuration) + if err != nil { + writeError(w, errors.New("illegal argument")) + return + } else if duration < 0 || duration > 2147483647 { + writeError(w, errors.New("transaction_id_digest_min_duration out of range, should be in 0 to 2147483647")) + return + } + cfg := config.GetGlobalConfig() + cfg.TrxSummary.TransactionIDDigestMinDuration = uint(duration) + config.StoreGlobalConfig(cfg) + txninfo.Recorder.SetMinDuration(time.Duration(duration) * time.Millisecond) + } } else { writeData(w, config.GetGlobalConfig()) } diff --git a/session/txn.go b/session/txn.go index b867d8a5530c1..5cb87948bd6ce 100644 --- a/session/txn.go +++ b/session/txn.go @@ -136,6 +136,9 @@ func (txn *LazyTxn) resetTxnInfo( currentSQLDigest string, allSQLDigests []string, ) { + if txn.mu.TxnInfo.StartTS != 0 { + txninfo.Recorder.OnTrxEnd(&txn.mu.TxnInfo) + } txn.mu.TxnInfo = txninfo.TxnInfo{} txn.mu.TxnInfo.StartTS = startTS txn.mu.TxnInfo.State = state @@ -270,6 +273,9 @@ func (txn *LazyTxn) changeToInvalid() { txn.mu.Lock() defer txn.mu.Unlock() + if txn.mu.TxnInfo.StartTS != 0 { + txninfo.Recorder.OnTrxEnd(&txn.mu.TxnInfo) + } txn.mu.TxnInfo = txninfo.TxnInfo{} } diff --git a/session/txninfo/summary.go b/session/txninfo/summary.go new file mode 100644 index 0000000000000..e26b2c534cb6a --- /dev/null +++ b/session/txninfo/summary.go @@ -0,0 +1,162 @@ +// Copyright 2021 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, +// 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 txninfo + +import ( + "container/list" + "encoding/json" + "fmt" + "hash/fnv" + "sync" + "time" + + "github.com/pingcap/tidb/types" + "github.com/tikv/client-go/v2/oracle" +) + +func digest(digests []string) uint64 { + // We use FNV-1a hash to generate the 64bit digest + // since 64bit digest use less memory and FNV-1a is faster than most of other hash algorithms + // You can refer to https://softwareengineering.stackexchange.com/questions/49550/which-hashing-algorithm-is-best-for-uniqueness-and-speed + hash := fnv.New64a() + for _, digest := range digests { + hash.Write([]byte(digest)) + } + return hash.Sum64() +} + +type trxSummaryEntry struct { + trxDigest uint64 + digests []string +} + +type trxSummaries struct { + capacity uint + + // lru cache for digest -> trxSummaryEntry + elements map[uint64]*list.Element + cache *list.List +} + +func newTrxSummaries(capacity uint) trxSummaries { + return trxSummaries{ + capacity: capacity, + cache: list.New(), + elements: make(map[uint64]*list.Element), + } +} + +func (s *trxSummaries) onTrxEnd(digests []string) { + key := digest(digests) + element, exists := s.elements[key] + if exists { + s.cache.MoveToFront(element) + return + } + e := trxSummaryEntry{ + trxDigest: key, + digests: digests, + } + s.elements[key] = s.cache.PushFront(e) + if uint(s.cache.Len()) > s.capacity { + last := s.cache.Back() + delete(s.elements, last.Value.(trxSummaryEntry).trxDigest) + s.cache.Remove(last) + } +} + +func (s *trxSummaries) dumpTrxSummary() [][]types.Datum { + var result [][]types.Datum + for element := s.cache.Front(); element != nil; element = element.Next() { + sqls := element.Value.(trxSummaryEntry).digests + // for consistency with other digests in TiDB, we calculate sum256 here to generate varchar(64) digest + digest := fmt.Sprintf("%x", element.Value.(trxSummaryEntry).trxDigest) + + res, err := json.Marshal(sqls) + if err != nil { + panic(err) + } + + result = append(result, []types.Datum{ + types.NewDatum(digest), + types.NewDatum(string(res)), + }) + } + return result +} + +func (s *trxSummaries) resize(capacity uint) { + s.capacity = capacity + for uint(s.cache.Len()) > s.capacity { + last := s.cache.Back() + delete(s.elements, last.Value.(trxSummaryEntry).trxDigest) + s.cache.Remove(last) + } +} + +// TrxHistoryRecorder is a history recorder for transaction. +type TrxHistoryRecorder struct { + mu sync.Mutex + minDuration time.Duration + summaries trxSummaries +} + +// DumpTrxSummary dumps the transaction summary to Datum for displaying in `TRX_SUMMARY` table. +func (recorder *TrxHistoryRecorder) DumpTrxSummary() [][]types.Datum { + recorder.mu.Lock() + defer recorder.mu.Unlock() + return recorder.summaries.dumpTrxSummary() +} + +// OnTrxEnd should be called when a transaction ends, ie. leaves `TIDB_TRX` table. +func (recorder *TrxHistoryRecorder) OnTrxEnd(info *TxnInfo) { + now := time.Now() + startTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6) + if now.Sub(startTime) < recorder.minDuration { + return + } + recorder.mu.Lock() + defer recorder.mu.Unlock() + recorder.summaries.onTrxEnd(info.AllSQLDigests) +} + +func newTrxHistoryRecorder(summariesCap uint) TrxHistoryRecorder { + return TrxHistoryRecorder{ + summaries: newTrxSummaries(summariesCap), + minDuration: 1 * time.Second, + } +} + +// Clean clears the history recorder. For test only. +func (recorder *TrxHistoryRecorder) Clean() { + recorder.summaries.cache = list.New() +} + +// SetMinDuration sets the minimum duration for a transaction to be recorded. +func (recorder *TrxHistoryRecorder) SetMinDuration(d time.Duration) { + recorder.mu.Lock() + defer recorder.mu.Unlock() + recorder.minDuration = d +} + +// ResizeSummaries resizes the summaries capacity. +func (recorder *TrxHistoryRecorder) ResizeSummaries(capacity uint) { + recorder.mu.Lock() + defer recorder.mu.Unlock() + recorder.summaries.resize(capacity) +} + +// Recorder is the recorder instance. +var Recorder TrxHistoryRecorder = newTrxHistoryRecorder(0) diff --git a/tidb-server/main.go b/tidb-server/main.go index 40dd3cacfd5a1..763d4bd5bc6a8 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -45,6 +45,7 @@ import ( "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" @@ -701,6 +702,8 @@ func setGlobalVars() { tikv.SetStoreLivenessTimeout(t) parsertypes.TiDBStrictIntegerDisplayWidth = cfg.DeprecateIntegerDisplayWidth deadlockhistory.GlobalDeadlockHistory.Resize(cfg.PessimisticTxn.DeadlockHistoryCapacity) + txninfo.Recorder.ResizeSummaries(cfg.TrxSummary.TransactionSummaryCapacity) + txninfo.Recorder.SetMinDuration(time.Duration(cfg.TrxSummary.TransactionIDDigestMinDuration) * time.Millisecond) } func setupLog() { From e44277d8fec357ee0b2a2ebdb16fa3a9ffaf5620 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Thu, 23 Jun 2022 12:46:36 +0800 Subject: [PATCH 18/21] sessionctx: support encoding and decoding session contexts (#35648) close pingcap/tidb#35573 --- session/session.go | 25 +- sessionctx/sessionstates/session_states.go | 33 ++- .../sessionstates/session_states_test.go | 231 +++++++++++++++++- sessionctx/variable/sequence_state.go | 20 ++ sessionctx/variable/session.go | 54 ++-- 5 files changed, 332 insertions(+), 31 deletions(-) diff --git a/session/session.go b/session/session.go index 8d03ba0870774..c5c1ead4c65b4 100644 --- a/session/session.go +++ b/session/session.go @@ -2069,17 +2069,20 @@ func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec. sessVars := se.sessionVars // Record diagnostic information for DML statements - if _, ok := s.(*executor.ExecStmt).StmtNode.(ast.DMLNode); ok { - defer func() { - sessVars.LastQueryInfo = variable.QueryInfo{ - TxnScope: sessVars.CheckAndGetTxnScope(), - StartTS: sessVars.TxnCtx.StartTS, - ForUpdateTS: sessVars.TxnCtx.GetForUpdateTS(), - } - if err != nil { - sessVars.LastQueryInfo.ErrMsg = err.Error() - } - }() + if stmt, ok := s.(*executor.ExecStmt).StmtNode.(ast.DMLNode); ok { + // Keep the previous queryInfo for `show session_states` because the statement needs to encode it. + if showStmt, ok := stmt.(*ast.ShowStmt); !ok || showStmt.Tp != ast.ShowSessionStates { + defer func() { + sessVars.LastQueryInfo = sessionstates.QueryInfo{ + TxnScope: sessVars.CheckAndGetTxnScope(), + StartTS: sessVars.TxnCtx.StartTS, + ForUpdateTS: sessVars.TxnCtx.GetForUpdateTS(), + } + if err != nil { + sessVars.LastQueryInfo.ErrMsg = err.Error() + } + }() + } } // Save origTxnCtx here to avoid it reset in the transaction retry. diff --git a/sessionctx/sessionstates/session_states.go b/sessionctx/sessionstates/session_states.go index 312cf891ec80e..baf876ff87b4f 100644 --- a/sessionctx/sessionstates/session_states.go +++ b/sessionctx/sessionstates/session_states.go @@ -15,14 +15,41 @@ package sessionstates import ( + "time" + ptypes "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/types" ) +// QueryInfo represents the information of last executed query. It's used to expose information for test purpose. +type QueryInfo struct { + TxnScope string `json:"txn_scope"` + StartTS uint64 `json:"start_ts"` + ForUpdateTS uint64 `json:"for_update_ts"` + ErrMsg string `json:"error,omitempty"` +} + +// LastDDLInfo represents the information of last DDL. It's used to expose information for test purpose. +type LastDDLInfo struct { + Query string `json:"query"` + SeqNum uint64 `json:"seq_num"` +} + // SessionStates contains all the states in the session that should be migrated when the session // is migrated to another server. It is shown by `show session_states` and recovered by `set session_states`. type SessionStates struct { - UserVars map[string]*types.Datum `json:"user-var-values,omitempty"` - UserVarTypes map[string]*ptypes.FieldType `json:"user-var-types,omitempty"` - SystemVars map[string]string `json:"sys-vars,omitempty"` + UserVars map[string]*types.Datum `json:"user-var-values,omitempty"` + UserVarTypes map[string]*ptypes.FieldType `json:"user-var-types,omitempty"` + SystemVars map[string]string `json:"sys-vars,omitempty"` + PreparedStmtID uint32 `json:"prepared-stmt-id,omitempty"` + Status uint16 `json:"status,omitempty"` + CurrentDB string `json:"current-db,omitempty"` + LastTxnInfo string `json:"txn-info,omitempty"` + LastQueryInfo *QueryInfo `json:"query-info,omitempty"` + LastDDLInfo *LastDDLInfo `json:"ddl-info,omitempty"` + LastFoundRows uint64 `json:"found-rows,omitempty"` + FoundInPlanCache bool `json:"in-plan-cache,omitempty"` + FoundInBinding bool `json:"in-binding,omitempty"` + SequenceLatestValues map[int64]int64 `json:"seq-values,omitempty"` + MPPStoreLastFailTime map[string]time.Time `json:"store-fail-time,omitempty"` } diff --git a/sessionctx/sessionstates/session_states_test.go b/sessionctx/sessionstates/session_states_test.go index 81e4cb6d5285a..847f50f4e9a2b 100644 --- a/sessionctx/sessionstates/session_states_test.go +++ b/sessionctx/sessionstates/session_states_test.go @@ -19,8 +19,10 @@ import ( "strconv" "strings" "testing" + "time" "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/sem" @@ -167,7 +169,10 @@ func TestSystemVars(t *testing.T) { }, } - sem.Enable() + if !sem.IsEnabled() { + sem.Enable() + defer sem.Disable() + } for _, tt := range tests { tk1 := testkit.NewTestKit(t, store) for _, stmt := range tt.stmts { @@ -206,6 +211,230 @@ func TestSystemVars(t *testing.T) { } } +func TestSessionCtx(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("create table test.t1(id int)") + + tests := []struct { + setFunc func(tk *testkit.TestKit) any + checkFunc func(tk *testkit.TestKit, param any) + }{ + { + // check PreparedStmtID + checkFunc: func(tk *testkit.TestKit, param any) { + require.Equal(t, uint32(1), tk.Session().GetSessionVars().GetNextPreparedStmtID()) + }, + }, + { + // check PreparedStmtID + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("prepare stmt from 'select ?'") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + require.Equal(t, uint32(2), tk.Session().GetSessionVars().GetNextPreparedStmtID()) + }, + }, + { + // check Status + checkFunc: func(tk *testkit.TestKit, param any) { + require.Equal(t, mysql.ServerStatusAutocommit, tk.Session().GetSessionVars().Status&mysql.ServerStatusAutocommit) + }, + }, + { + // check Status + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("set autocommit=0") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + require.Equal(t, uint16(0), tk.Session().GetSessionVars().Status&mysql.ServerStatusAutocommit) + }, + }, + { + // check CurrentDB + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select database()").Check(testkit.Rows("")) + }, + }, + { + // check CurrentDB + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("use test") + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select database()").Check(testkit.Rows("test")) + }, + }, + { + // check LastTxnInfo + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@tidb_last_txn_info").Check(testkit.Rows("")) + }, + }, + { + // check LastTxnInfo + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("begin") + tk.MustExec("insert test.t1 value(1)") + tk.MustExec("commit") + rows := tk.MustQuery("select @@tidb_last_txn_info").Rows() + require.NotEqual(t, "", rows[0][0].(string)) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@tidb_last_txn_info").Check(param.([][]interface{})) + }, + }, + { + // check LastQueryInfo + setFunc: func(tk *testkit.TestKit) any { + rows := tk.MustQuery("select @@tidb_last_query_info").Rows() + require.NotEqual(t, "", rows[0][0].(string)) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@tidb_last_query_info").Check(param.([][]interface{})) + }, + }, + { + // check LastQueryInfo + setFunc: func(tk *testkit.TestKit) any { + tk.MustQuery("select * from test.t1") + startTS := tk.Session().GetSessionVars().LastQueryInfo.StartTS + require.NotEqual(t, uint64(0), startTS) + return startTS + }, + checkFunc: func(tk *testkit.TestKit, param any) { + startTS := tk.Session().GetSessionVars().LastQueryInfo.StartTS + require.Equal(t, param.(uint64), startTS) + }, + }, + { + // check LastDDLInfo + setFunc: func(tk *testkit.TestKit) any { + rows := tk.MustQuery("select @@tidb_last_ddl_info").Rows() + require.NotEqual(t, "", rows[0][0].(string)) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@tidb_last_ddl_info").Check(param.([][]interface{})) + }, + }, + { + // check LastDDLInfo + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("truncate table test.t1") + rows := tk.MustQuery("select @@tidb_last_ddl_info").Rows() + require.NotEqual(t, "", rows[0][0].(string)) + return rows + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@tidb_last_ddl_info").Check(param.([][]interface{})) + }, + }, + { + // check LastFoundRows + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("insert test.t1 value(1), (2), (3), (4), (5)") + // SQL_CALC_FOUND_ROWS is not supported now, so we just test normal select. + rows := tk.MustQuery("select * from test.t1 limit 3").Rows() + require.Equal(t, 3, len(rows)) + return "3" + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select found_rows()").Check(testkit.Rows(param.(string))) + }, + }, + { + // check SequenceState + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("create sequence test.s") + tk.MustQuery("select nextval(test.s)").Check(testkit.Rows("1")) + tk.MustQuery("select lastval(test.s)").Check(testkit.Rows("1")) + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select lastval(test.s)").Check(testkit.Rows("1")) + tk.MustQuery("select nextval(test.s)").Check(testkit.Rows("2")) + }, + }, + { + // check MPPStoreLastFailTime + setFunc: func(tk *testkit.TestKit) any { + tk.Session().GetSessionVars().MPPStoreLastFailTime = map[string]time.Time{"store1": time.Now()} + return tk.Session().GetSessionVars().MPPStoreLastFailTime + }, + checkFunc: func(tk *testkit.TestKit, param any) { + failTime := tk.Session().GetSessionVars().MPPStoreLastFailTime + require.Equal(t, 1, len(failTime)) + tm, ok := failTime["store1"] + require.True(t, ok) + require.True(t, param.(map[string]time.Time)["store1"].Equal(tm)) + }, + }, + { + // check FoundInPlanCache + setFunc: func(tk *testkit.TestKit) any { + require.False(t, tk.Session().GetSessionVars().FoundInPlanCache) + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + }, + }, + { + // check FoundInPlanCache + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("prepare stmt from 'select * from test.t1'") + tk.MustQuery("execute stmt") + tk.MustQuery("execute stmt") + require.True(t, tk.Session().GetSessionVars().FoundInPlanCache) + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + }, + }, + { + // check FoundInBinding + setFunc: func(tk *testkit.TestKit) any { + require.False(t, tk.Session().GetSessionVars().FoundInBinding) + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + }, + }, + { + // check FoundInBinding + setFunc: func(tk *testkit.TestKit) any { + tk.MustExec("create session binding for select * from test.t1 using select * from test.t1") + tk.MustQuery("select * from test.t1") + require.True(t, tk.Session().GetSessionVars().FoundInBinding) + return nil + }, + checkFunc: func(tk *testkit.TestKit, param any) { + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + }, + }, + } + + for _, tt := range tests { + tk1 := testkit.NewTestKit(t, store) + var param any + if tt.setFunc != nil { + param = tt.setFunc(tk1) + } + tk2 := testkit.NewTestKit(t, store) + showSessionStatesAndSet(t, tk1, tk2) + tt.checkFunc(tk2, param) + } +} + func showSessionStatesAndSet(t *testing.T, tk1, tk2 *testkit.TestKit) { rows := tk1.MustQuery("show session_states").Rows() require.Len(t, rows, 1) diff --git a/sessionctx/variable/sequence_state.go b/sessionctx/variable/sequence_state.go index bb8b468da2de9..38199b084fa81 100644 --- a/sessionctx/variable/sequence_state.go +++ b/sessionctx/variable/sequence_state.go @@ -50,3 +50,23 @@ func (ss *SequenceState) GetLastValue(sequenceID int64) (int64, bool, error) { } return 0, true, nil } + +// GetAllStates returns a copied latestValueMap. +func (ss *SequenceState) GetAllStates() map[int64]int64 { + ss.mu.Lock() + defer ss.mu.Unlock() + latestValueMap := make(map[int64]int64, len(ss.latestValueMap)) + for seqID, latestValue := range ss.latestValueMap { + latestValueMap[seqID] = latestValue + } + return latestValueMap +} + +// SetAllStates sets latestValueMap as a whole. +func (ss *SequenceState) SetAllStates(latestValueMap map[int64]int64) { + ss.mu.Lock() + defer ss.mu.Unlock() + for seqID, latestValue := range latestValueMap { + ss.latestValueMap[seqID] = latestValue + } +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 735105c57d4db..fe4f469e76134 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1041,10 +1041,10 @@ type SessionVars struct { LastTxnInfo string // LastQueryInfo keeps track the info of last query. - LastQueryInfo QueryInfo + LastQueryInfo sessionstates.QueryInfo // LastDDLInfo keeps track the info of last DDL. - LastDDLInfo LastDDLInfo + LastDDLInfo sessionstates.LastDDLInfo // PartitionPruneMode indicates how and when to prune partitions. PartitionPruneMode atomic2.String @@ -1850,6 +1850,23 @@ func (s *SessionVars) EncodeSessionStates(ctx context.Context, sessionStates *se sessionStates.UserVarTypes[name] = userVarType.Clone() } s.UsersLock.RUnlock() + + // Encode other session contexts. + sessionStates.PreparedStmtID = s.preparedStmtID + sessionStates.Status = s.Status + sessionStates.CurrentDB = s.CurrentDB + sessionStates.LastTxnInfo = s.LastTxnInfo + if s.LastQueryInfo.StartTS != 0 { + sessionStates.LastQueryInfo = &s.LastQueryInfo + } + if s.LastDDLInfo.SeqNum != 0 { + sessionStates.LastDDLInfo = &s.LastDDLInfo + } + sessionStates.LastFoundRows = s.LastFoundRows + sessionStates.SequenceLatestValues = s.SequenceState.GetAllStates() + sessionStates.MPPStoreLastFailTime = s.MPPStoreLastFailTime + sessionStates.FoundInPlanCache = s.PrevFoundInPlanCache + sessionStates.FoundInBinding = s.PrevFoundInBinding return } @@ -1866,6 +1883,25 @@ func (s *SessionVars) DecodeSessionStates(ctx context.Context, sessionStates *se s.UserVarTypes[name] = userVarType.Clone() } s.UsersLock.Unlock() + + // Decode other session contexts. + s.preparedStmtID = sessionStates.PreparedStmtID + s.Status = sessionStates.Status + s.CurrentDB = sessionStates.CurrentDB + s.LastTxnInfo = sessionStates.LastTxnInfo + if sessionStates.LastQueryInfo != nil { + s.LastQueryInfo = *sessionStates.LastQueryInfo + } + if sessionStates.LastDDLInfo != nil { + s.LastDDLInfo = *sessionStates.LastDDLInfo + } + s.LastFoundRows = sessionStates.LastFoundRows + s.SequenceState.SetAllStates(sessionStates.SequenceLatestValues) + if sessionStates.MPPStoreLastFailTime != nil { + s.MPPStoreLastFailTime = sessionStates.MPPStoreLastFailTime + } + s.FoundInPlanCache = sessionStates.FoundInPlanCache + s.FoundInBinding = sessionStates.FoundInBinding return } @@ -2458,20 +2494,6 @@ func writeSlowLogItem(buf *bytes.Buffer, key, value string) { buf.WriteString(SlowLogRowPrefixStr + key + SlowLogSpaceMarkStr + value + "\n") } -// QueryInfo represents the information of last executed query. It's used to expose information for test purpose. -type QueryInfo struct { - TxnScope string `json:"txn_scope"` - StartTS uint64 `json:"start_ts"` - ForUpdateTS uint64 `json:"for_update_ts"` - ErrMsg string `json:"error,omitempty"` -} - -// LastDDLInfo represents the information of last DDL. It's used to expose information for test purpose. -type LastDDLInfo struct { - Query string `json:"query"` - SeqNum uint64 `json:"seq_num"` -} - // TxnReadTS indicates the value and used situation for tx_read_ts type TxnReadTS struct { readTS uint64 From e99d82a9b429b48cfcee24d1207b635a9b977f2e Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 23 Jun 2022 13:50:37 +0800 Subject: [PATCH 19/21] *: fix issue of cast string as time is incompatible with MySQL (#35455) close pingcap/tidb#34539 --- expression/builtin_cast.go | 14 ++--- expression/builtin_cast_vec.go | 10 ++-- expression/builtin_time.go | 41 ++++++------- expression/builtin_time_test.go | 4 +- expression/builtin_time_vec.go | 14 ++--- expression/builtin_time_vec_generated.go | 16 +++--- expression/generator/time_vec.go | 4 +- server/util_test.go | 4 +- sessionctx/variable/varsutil.go | 2 +- statistics/scalar_test.go | 2 +- tablecodec/tablecodec_test.go | 2 +- .../realtikvtest/sessiontest/session_test.go | 11 ++++ types/convert.go | 2 +- types/convert_test.go | 6 +- types/datum.go | 10 ++-- types/datum_test.go | 4 +- types/time.go | 41 +++++++------ types/time_test.go | 57 ++++++++++++------- util/chunk/mutrow_test.go | 2 +- util/codec/codec_test.go | 2 +- util/rowcodec/rowcodec_test.go | 2 +- 21 files changed, 141 insertions(+), 109 deletions(-) diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index c281e2de80302..cc7b9d3683b71 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -917,7 +917,7 @@ func (b *builtinCastRealAsDurationSig) evalDuration(row chunk.Row) (res types.Du if isNull || err != nil { return res, isNull, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), b.tp.GetDecimal()) + res, _, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), b.tp.GetDecimal()) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1095,7 +1095,7 @@ func (b *builtinCastDecimalAsDurationSig) evalDuration(row chunk.Row) (res types if isNull || err != nil { return res, true, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(val.ToString()), b.tp.GetDecimal()) + res, _, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(val.ToString()), b.tp.GetDecimal()) if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) // ErrTruncatedWrongVal needs to be considered NULL. @@ -1318,16 +1318,12 @@ func (b *builtinCastStringAsDurationSig) evalDuration(row chunk.Row) (res types. if isNull || err != nil { return res, isNull, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, val, b.tp.GetDecimal()) + res, isNull, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, val, b.tp.GetDecimal()) if types.ErrTruncatedWrongVal.Equal(err) { sc := b.ctx.GetSessionVars().StmtCtx err = sc.HandleTruncate(err) - // ZeroDuration of error ErrTruncatedWrongVal needs to be considered NULL. - if res == types.ZeroDuration { - return res, true, err - } } - return res, false, err + return res, isNull, err } type builtinCastTimeAsTimeSig struct { @@ -1765,7 +1761,7 @@ func (b *builtinCastJSONAsDurationSig) evalDuration(row chunk.Row) (res types.Du if err != nil { return res, false, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, s, b.tp.GetDecimal()) + res, _, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, s, b.tp.GetDecimal()) if types.ErrTruncatedWrongVal.Equal(err) { sc := b.ctx.GetSessionVars().StmtCtx err = sc.HandleTruncate(err) diff --git a/expression/builtin_cast_vec.go b/expression/builtin_cast_vec.go index ee29a768dd702..141ff49c26f13 100644 --- a/expression/builtin_cast_vec.go +++ b/expression/builtin_cast_vec.go @@ -939,7 +939,7 @@ func (b *builtinCastStringAsDurationSig) vecEvalDuration(input *chunk.Chunk, res if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, buf.GetString(i), b.tp.GetDecimal()) + dur, isNull, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, buf.GetString(i), b.tp.GetDecimal()) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -947,7 +947,7 @@ func (b *builtinCastStringAsDurationSig) vecEvalDuration(input *chunk.Chunk, res if err != nil { return err } - if dur == types.ZeroDuration { + if isNull { result.SetNull(i, true) continue } @@ -1213,7 +1213,7 @@ func (b *builtinCastRealAsDurationSig) vecEvalDuration(input *chunk.Chunk, resul if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(f64s[i], 'f', -1, 64), b.tp.GetDecimal()) + dur, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(f64s[i], 'f', -1, 64), b.tp.GetDecimal()) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1793,7 +1793,7 @@ func (b *builtinCastDecimalAsDurationSig) vecEvalDuration(input *chunk.Chunk, re if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(args[i].ToString()), b.tp.GetDecimal()) + dur, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(args[i].ToString()), b.tp.GetDecimal()) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1880,7 +1880,7 @@ func (b *builtinCastJSONAsDurationSig) vecEvalDuration(input *chunk.Chunk, resul if err != nil { return nil } - dur, err = types.ParseDuration(ctx, s, b.tp.GetDecimal()) + dur, _, err = types.ParseDuration(ctx, s, b.tp.GetDecimal()) if types.ErrTruncatedWrongVal.Equal(err) { err = ctx.HandleTruncate(err) } diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 4a8a5bcca13b4..d5d17bd4a6ecb 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -2158,7 +2158,7 @@ func (b *builtinCurrentTime0ArgSig) evalDuration(row chunk.Row) (types.Duration, return types.Duration{}, true, err } dur := nowTs.In(tz).Format(types.TimeFormat) - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp) + res, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp) if err != nil { return types.Duration{}, true, err } @@ -2186,7 +2186,7 @@ func (b *builtinCurrentTime1ArgSig) evalDuration(row chunk.Row) (types.Duration, return types.Duration{}, true, err } dur := nowTs.In(tz).Format(types.TimeFSPFormat) - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int(fsp)) + res, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int(fsp)) if err != nil { return types.Duration{}, true, err } @@ -2246,7 +2246,7 @@ func (b *builtinTimeSig) evalDuration(row chunk.Row) (res types.Duration, isNull fsp = tmpFsp sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseDuration(sc, expr, fsp) + res, _, err = types.ParseDuration(sc, expr, fsp) if types.ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } @@ -2273,7 +2273,7 @@ func (c *timeLiteralFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if !isDuration(str) { return nil, types.ErrWrongValue.GenWithStackByArgs(types.TimeStr, str) } - duration, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, str, types.GetFsp(str)) + duration, _, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, str, types.GetFsp(str)) if err != nil { return nil, err } @@ -2639,7 +2639,7 @@ func (b *builtinExtractDatetimeFromStringSig) evalInt(row chunk.Row) (int64, boo sc := b.ctx.GetSessionVars().StmtCtx switch strings.ToUpper(unit) { case "DAY_MICROSECOND", "DAY_SECOND", "DAY_MINUTE", "DAY_HOUR": - dur, err := types.ParseDuration(sc, dtStr, types.GetFsp(dtStr)) + dur, _, err := types.ParseDuration(sc, dtStr, types.GetFsp(dtStr)) if err != nil { return 0, true, err } @@ -4373,7 +4373,7 @@ func (b *builtinTimestamp2ArgsSig) evalTime(row chunk.Row) (types.Time, bool, er if !isDuration(arg1) { return types.ZeroTime, true, nil } - duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) } @@ -4535,7 +4535,7 @@ func strDatetimeAddDuration(sc *stmtctx.StatementContext, d string, arg1 types.D // strDurationAddDuration adds duration to duration string, returns a string value. func strDurationAddDuration(sc *stmtctx.StatementContext, d string, arg1 types.Duration) (string, error) { - arg0, err := types.ParseDuration(sc, d, types.MaxFsp) + arg0, _, err := types.ParseDuration(sc, d, types.MaxFsp) if err != nil { return "", err } @@ -4572,7 +4572,7 @@ func strDatetimeSubDuration(sc *stmtctx.StatementContext, d string, arg1 types.D // strDurationSubDuration subtracts duration from duration string, returns a string value. func strDurationSubDuration(sc *stmtctx.StatementContext, d string, arg1 types.Duration) (string, error) { - arg0, err := types.ParseDuration(sc, d, types.MaxFsp) + arg0, _, err := types.ParseDuration(sc, d, types.MaxFsp) if err != nil { return "", err } @@ -4721,7 +4721,7 @@ func (b *builtinAddDatetimeAndStringSig) evalTime(row chunk.Row) (types.Time, bo return types.ZeroDatetime, true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, types.GetFsp(s)) + arg1, _, err := types.ParseDuration(sc, s, types.GetFsp(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -4802,7 +4802,7 @@ func (b *builtinAddDurationAndStringSig) evalDuration(row chunk.Row) (types.Dura return types.ZeroDuration, true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, types.GetFsp(s)) + arg1, _, err := types.ParseDuration(sc, s, types.GetFsp(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -4904,7 +4904,7 @@ func (b *builtinAddStringAndStringSig) evalString(row chunk.Row) (result string, return "", isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err = types.ParseDuration(sc, arg1Str, getFsp4TimeAddSub(arg1Str)) + arg1, _, err = types.ParseDuration(sc, arg1Str, getFsp4TimeAddSub(arg1Str)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -4987,7 +4987,7 @@ func (b *builtinAddDateAndStringSig) evalString(row chunk.Row) (string, bool, er return "", true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) + arg1, _, err := types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -5253,7 +5253,8 @@ func (b *builtinMakeTimeSig) makeTime(hour int64, minute int64, second float64, second = 59 } fsp := b.tp.GetDecimal() - return types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%02d:%02d:%v", hour, minute, second), fsp) + d, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%02d:%02d:%v", hour, minute, second), fsp) + return d, err } // evalDuration evals a builtinMakeTimeIntSig. @@ -5547,7 +5548,7 @@ func (b *builtinSecToTimeSig) evalDuration(row chunk.Row) (types.Duration, bool, secondDemical = float64(second) + demical var dur types.Duration - dur, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.GetDecimal()) + dur, _, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.GetDecimal()) if err != nil { return types.Duration{}, err != nil, err } @@ -5676,7 +5677,7 @@ func (b *builtinSubDatetimeAndStringSig) evalTime(row chunk.Row) (types.Time, bo return types.ZeroDatetime, true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, types.GetFsp(s)) + arg1, _, err := types.ParseDuration(sc, s, types.GetFsp(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -5775,7 +5776,7 @@ func (b *builtinSubStringAndStringSig) evalString(row chunk.Row) (result string, return "", isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err = types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) + arg1, _, err = types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -5867,7 +5868,7 @@ func (b *builtinSubDurationAndStringSig) evalDuration(row chunk.Row) (types.Dura return types.ZeroDuration, true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, types.GetFsp(s)) + arg1, _, err := types.ParseDuration(sc, s, types.GetFsp(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -5945,7 +5946,7 @@ func (b *builtinSubDateAndStringSig) evalString(row chunk.Row) (string, bool, er return "", true, nil } sc := b.ctx.GetSessionVars().StmtCtx - arg1, err := types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) + arg1, _, err := types.ParseDuration(sc, s, getFsp4TimeAddSub(s)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -6310,7 +6311,7 @@ func (b *builtinUTCTimeWithoutArgSig) evalDuration(row chunk.Row) (types.Duratio if err != nil { return types.Duration{}, true, err } - v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), 0) + v, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), 0) return v, false, err } @@ -6341,7 +6342,7 @@ func (b *builtinUTCTimeWithArgSig) evalDuration(row chunk.Row) (types.Duration, if err != nil { return types.Duration{}, true, err } - v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int(fsp)) + v, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int(fsp)) return v, false, err } diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index c081b11f0f0f8..fb8387eb022fb 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -962,7 +962,7 @@ func TestAddTimeSig(t *testing.T) { {"-110:00:00", "1 02:00:00", "-84:00:00"}, } for _, c := range tbl { - dur, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, c.Input, types.GetFsp(c.Input)) + dur, _, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, c.Input, types.GetFsp(c.Input)) require.NoError(t, err) tmpInput := types.NewDurationDatum(dur) tmpInputDuration := types.NewStringDatum(c.InputDuration) @@ -1063,7 +1063,7 @@ func TestSubTimeSig(t *testing.T) { {"235959", "00:00:01", "23:59:58"}, } for _, c := range tbl { - dur, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, c.Input, types.GetFsp(c.Input)) + dur, _, err := types.ParseDuration(ctx.GetSessionVars().StmtCtx, c.Input, types.GetFsp(c.Input)) require.NoError(t, err) tmpInput := types.NewDurationDatum(dur) tmpInputDuration := types.NewStringDatum(c.InputDuration) diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index b292164813dcc..0d6b4321095f5 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -421,7 +421,7 @@ func (b *builtinUTCTimeWithArgSig) vecEvalDuration(input *chunk.Chunk, result *c if fsp < int64(types.MinFsp) { return errors.Errorf("Invalid negative %d specified, must in [0, 6]", fsp) } - res, err := types.ParseDuration(stmtCtx, utc, int(fsp)) + res, _, err := types.ParseDuration(stmtCtx, utc, int(fsp)) if err != nil { return err } @@ -1954,7 +1954,7 @@ func (b *builtinSecToTimeSig) vecEvalDuration(input *chunk.Chunk, result *chunk. second = seconds % 60 } secondDemical := float64(second) + demical - duration, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.GetDecimal()) + duration, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.GetDecimal()) if err != nil { return err } @@ -1975,7 +1975,7 @@ func (b *builtinUTCTimeWithoutArgSig) vecEvalDuration(input *chunk.Chunk, result if err != nil { return err } - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), types.DefaultFsp) + res, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), types.DefaultFsp) if err != nil { return err } @@ -2378,7 +2378,7 @@ func (b *builtinCurrentTime0ArgSig) vecEvalDuration(input *chunk.Chunk, result * } tz := b.ctx.GetSessionVars().Location() dur := nowTs.In(tz).Format(types.TimeFormat) - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp) + res, _, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp) if err != nil { return err } @@ -2426,7 +2426,7 @@ func (b *builtinTimeSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Colum } fsp = tmpFsp - res, err := types.ParseDuration(sc, expr, fsp) + res, _, err := types.ParseDuration(sc, expr, fsp) if types.ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } @@ -2572,7 +2572,7 @@ func (b *builtinCurrentTime1ArgSig) vecEvalDuration(input *chunk.Chunk, result * result.ResizeGoDuration(n, false) durations := result.GoDurations() for i := 0; i < n; i++ { - res, err := types.ParseDuration(stmtCtx, dur, int(i64s[i])) + res, _, err := types.ParseDuration(stmtCtx, dur, int(i64s[i])) if err != nil { return err } @@ -2751,7 +2751,7 @@ func (b *builtinTimestamp2ArgsSig) vecEvalTime(input *chunk.Chunk, result *chunk continue } - duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if err = handleInvalidTimeError(b.ctx, err); err != nil { return err diff --git a/expression/builtin_time_vec_generated.go b/expression/builtin_time_vec_generated.go index 7c18af6e81ddf..4e897affc66d9 100644 --- a/expression/builtin_time_vec_generated.go +++ b/expression/builtin_time_vec_generated.go @@ -122,7 +122,7 @@ func (b *builtinAddDatetimeAndStringSig) vecEvalTime(input *chunk.Chunk, result continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -248,7 +248,7 @@ func (b *builtinAddDurationAndStringSig) vecEvalDuration(input *chunk.Chunk, res continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -406,7 +406,7 @@ func (b *builtinAddStringAndStringSig) vecEvalString(input *chunk.Chunk, result // calculate sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -566,7 +566,7 @@ func (b *builtinAddDateAndStringSig) vecEvalString(input *chunk.Chunk, result *c continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -737,7 +737,7 @@ func (b *builtinSubDatetimeAndStringSig) vecEvalTime(input *chunk.Chunk, result continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -862,7 +862,7 @@ func (b *builtinSubDurationAndStringSig) vecEvalDuration(input *chunk.Chunk, res continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -1020,7 +1020,7 @@ func (b *builtinSubStringAndStringSig) vecEvalString(input *chunk.Chunk, result // calculate sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -1180,7 +1180,7 @@ func (b *builtinSubDateAndStringSig) vecEvalString(input *chunk.Chunk, result *c continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, getFsp4TimeAddSub(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) diff --git a/expression/generator/time_vec.go b/expression/generator/time_vec.go index 3bf16f0076594..73067419f8cfc 100644 --- a/expression/generator/time_vec.go +++ b/expression/generator/time_vec.go @@ -63,7 +63,7 @@ import ( continue }{{ end }} sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, {{if eq .Output.TypeName "String"}}getFsp4TimeAddSub{{else}}types.GetFsp{{end}}(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, {{if eq .Output.TypeName "String"}}getFsp4TimeAddSub{{else}}types.GetFsp{{end}}(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) @@ -191,7 +191,7 @@ func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result continue } sc := b.ctx.GetSessionVars().StmtCtx - arg1Duration, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) + arg1Duration, _, err := types.ParseDuration(sc, arg1, types.GetFsp(arg1)) if err != nil { if terror.ErrorEqual(err, types.ErrTruncatedWrongVal) { sc.AppendWarning(err) diff --git a/server/util_test.go b/server/util_test.go index 889c2512eacad..7a5040b43cd3c 100644 --- a/server/util_test.go +++ b/server/util_test.go @@ -70,7 +70,7 @@ func TestDumpBinaryTime(t *testing.T) { d = dumpBinaryDateTime(nil, parsedTime) require.Equal(t, []byte{0}, d) - myDuration, err := types.ParseDuration(sc, "0000-00-00 00:00:00.000000", 6) + myDuration, _, err := types.ParseDuration(sc, "0000-00-00 00:00:00.000000", 6) require.NoError(t, err) d = dumpBinaryTime(myDuration.Duration) require.Equal(t, []byte{0}, d) @@ -194,7 +194,7 @@ func TestDumpTextValue(t *testing.T) { require.NoError(t, err) require.Equal(t, "2017-01-06 00:00:00", mustDecodeStr(t, bs)) - duration, err := types.ParseDuration(sc, "11:30:45", 0) + duration, _, err := types.ParseDuration(sc, "11:30:45", 0) require.NoError(t, err) d.SetMysqlDuration(duration) columns[0].Type = mysql.TypeDuration diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index ab878d2bb3054..38108e5cba932 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -430,7 +430,7 @@ func parseTimeZone(s string) (*time.Location, error) { // The value can be given as a string indicating an offset from UTC, such as '+10:00' or '-6:00'. // The time zone's value should in [-12:59,+14:00]. if strings.HasPrefix(s, "+") || strings.HasPrefix(s, "-") { - d, err := types.ParseDuration(nil, s[1:], 0) + d, _, err := types.ParseDuration(nil, s[1:], 0) if err == nil { if s[0] == '-' { if d.Duration > 12*time.Hour+59*time.Minute { diff --git a/statistics/scalar_test.go b/statistics/scalar_test.go index 32eca78c5c80b..34a911579137f 100644 --- a/statistics/scalar_test.go +++ b/statistics/scalar_test.go @@ -35,7 +35,7 @@ func getDecimal(value float64) *types.MyDecimal { } func getDuration(value string) types.Duration { - dur, _ := types.ParseDuration(nil, value, 0) + dur, _, _ := types.ParseDuration(nil, value, 0) return dur } diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index 008ee8c5d7e1f..3f813c8747a7a 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -262,7 +262,7 @@ func TestTimeCodec(t *testing.T) { "2016-06-23 11:30:45") require.NoError(t, err) row[2] = types.NewDatum(ts) - du, err := types.ParseDuration(nil, "12:59:59.999999", 6) + du, _, err := types.ParseDuration(nil, "12:59:59.999999", 6) require.NoError(t, err) row[3] = types.NewDatum(du) diff --git a/tests/realtikvtest/sessiontest/session_test.go b/tests/realtikvtest/sessiontest/session_test.go index f726c1118b39f..4bc983d94b40f 100644 --- a/tests/realtikvtest/sessiontest/session_test.go +++ b/tests/realtikvtest/sessiontest/session_test.go @@ -2951,6 +2951,17 @@ func TestCast(t *testing.T) { tk.MustQuery("select cast(0.5 as unsigned)") tk.MustQuery("select cast(-0.5 as signed)") tk.MustQuery("select hex(cast(0x10 as binary(2)))").Check(testkit.Rows("1000")) + + // test for issue: https://github.com/pingcap/tidb/issues/34539 + tk.MustQuery("select cast('0000-00-00' as TIME);").Check(testkit.Rows("00:00:00")) + tk.MustQuery("select cast('1234x' as TIME);").Check(testkit.Rows("00:12:34")) + tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect time value: '1234x'")) + tk.MustQuery("select cast('a' as TIME);").Check(testkit.Rows("")) + tk.MustQuery("select cast('' as TIME);").Check(testkit.Rows("")) + tk.MustQuery("select cast('1234xxxxxxx' as TIME);").Check(testkit.Rows("00:12:34")) + tk.MustQuery("select cast('1234xxxxxxxx' as TIME);").Check(testkit.Rows("")) + tk.MustQuery("select cast('-1234xxxxxxx' as TIME);").Check(testkit.Rows("-00:12:34")) + tk.MustQuery("select cast('-1234xxxxxxxx' as TIME);").Check(testkit.Rows("")) } func TestTableInfoMeta(t *testing.T) { diff --git a/types/convert.go b/types/convert.go index 9433f26dbefc8..a6e33d5b2c51b 100644 --- a/types/convert.go +++ b/types/convert.go @@ -327,7 +327,7 @@ func StrToDuration(sc *stmtctx.StatementContext, str string, fsp int) (d Duratio } } - d, err = ParseDuration(sc, str, fsp) + d, _, err = ParseDuration(sc, str, fsp) if ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } diff --git a/types/convert_test.go b/types/convert_test.go index aed9980635b72..53ec0c93b1e55 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -352,7 +352,7 @@ func TestConvertToString(t *testing.T) { require.NoError(t, err) testToString(t, t1, "2011-11-10 11:11:11.999999") - td, err := ParseDuration(nil, "11:11:11.999999", 6) + td, _, err := ParseDuration(nil, "11:11:11.999999", 6) require.NoError(t, err) testToString(t, td, "11:11:11.999999") @@ -819,7 +819,9 @@ func TestConvert(t *testing.T) { signedDeny(t, mysql.TypeDate, "2012-08-x", "0000-00-00") signedDeny(t, mysql.TypeDatetime, "2012-08-x", "0000-00-00 00:00:00") signedDeny(t, mysql.TypeTimestamp, "2012-08-x", "0000-00-00 00:00:00") - signedDeny(t, mysql.TypeDuration, "2012-08-x", "00:00:00") + signedDeny(t, mysql.TypeDuration, "2012-08-x", "00:20:12") + signedDeny(t, mysql.TypeDuration, "0000-00-00", "00:00:00") + signedDeny(t, mysql.TypeDuration, "1234abc", "00:12:34") // string from string signedAccept(t, mysql.TypeString, "abc", "abc") diff --git a/types/datum.go b/types/datum.go index a450b51a8a25c..1ad86c770a01b 100644 --- a/types/datum.go +++ b/types/datum.go @@ -749,7 +749,7 @@ func (d *Datum) compareString(sc *stmtctx.StatementContext, s string, comparer c dt, err := ParseDatetime(sc, s) return d.GetMysqlTime().Compare(dt), errors.Trace(err) case KindMysqlDuration: - dur, err := ParseDuration(sc, s, MaxFsp) + dur, _, err := ParseDuration(sc, s, MaxFsp) return d.GetMysqlDuration().Compare(dur), errors.Trace(err) case KindMysqlSet: return comparer.Compare(d.GetMysqlSet().String(), s), nil @@ -796,7 +796,7 @@ func (d *Datum) compareMysqlDuration(sc *stmtctx.StatementContext, dur Duration) case KindMysqlDuration: return d.GetMysqlDuration().Compare(dur), nil case KindString, KindBytes: - dDur, err := ParseDuration(sc, d.GetString(), MaxFsp) + dDur, _, err := ParseDuration(sc, d.GetString(), MaxFsp) return dDur.Compare(dur), errors.Trace(err) default: return d.compareFloat64(sc, dur.Seconds()) @@ -1375,13 +1375,13 @@ func (d *Datum) convertToMysqlDuration(sc *stmtctx.StatementContext, target *Fie if timeNum < -MaxDuration { return ret, ErrWrongValue.GenWithStackByArgs(TimeStr, timeStr) } - t, err := ParseDuration(sc, timeStr, fsp) + t, _, err := ParseDuration(sc, timeStr, fsp) ret.SetMysqlDuration(t) if err != nil { return ret, errors.Trace(err) } case KindString, KindBytes: - t, err := ParseDuration(sc, d.GetString(), fsp) + t, _, err := ParseDuration(sc, d.GetString(), fsp) ret.SetMysqlDuration(t) if err != nil { return ret, errors.Trace(err) @@ -1392,7 +1392,7 @@ func (d *Datum) convertToMysqlDuration(sc *stmtctx.StatementContext, target *Fie if err != nil { return ret, errors.Trace(err) } - t, err := ParseDuration(sc, s, fsp) + t, _, err := ParseDuration(sc, s, fsp) ret.SetMysqlDuration(t) if err != nil { return ret, errors.Trace(err) diff --git a/types/datum_test.go b/types/datum_test.go index 75627791c2342..698a69fd6b731 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -98,7 +98,7 @@ func TestToBool(t *testing.T) { require.NoError(t, err) testDatumToBool(t, t1, 1) - td, err := ParseDuration(nil, "11:11:11.999999", 6) + td, _, err := ParseDuration(nil, "11:11:11.999999", 6) require.NoError(t, err) testDatumToBool(t, td, 1) @@ -141,7 +141,7 @@ func TestToInt64(t *testing.T) { require.NoError(t, err) testDatumToInt64(t, t1, int64(20111110111112)) - td, err := ParseDuration(nil, "11:11:11.999999", 6) + td, _, err := ParseDuration(nil, "11:11:11.999999", 6) require.NoError(t, err) testDatumToInt64(t, td, int64(111112)) diff --git a/types/time.go b/types/time.go index d63669d7fe409..14c647f945788 100644 --- a/types/time.go +++ b/types/time.go @@ -1530,7 +1530,7 @@ func (d Duration) Compare(o Duration) int { // but parses str to Duration then compares. func (d Duration) CompareString(sc *stmtctx.StatementContext, str string) (int, error) { // use MaxFsp to parse the string - o, err := ParseDuration(sc, str, MaxFsp) + o, _, err := ParseDuration(sc, str, MaxFsp) if err != nil { return 0, err } @@ -1680,18 +1680,19 @@ func matchFrac(str string, fsp int) (bool, int, string, error) { return overflow, frac, rest, nil } -func matchDuration(str string, fsp int) (Duration, error) { +func matchDuration(str string, fsp int) (Duration, bool, error) { fsp, err := CheckFsp(fsp) if err != nil { - return ZeroDuration, errors.Trace(err) + return ZeroDuration, true, errors.Trace(err) } if len(str) == 0 { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } negative, rest := isNegativeDuration(str) rest = parser.Space0(rest) + charsLen := len(rest) hhmmss := [3]int{} @@ -1703,13 +1704,13 @@ func matchDuration(str string, fsp int) (Duration, error) { } else if hms, remain, err := matchHHMMSSCompact(rest); err == nil { rest, hhmmss = remain, hms } else { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } rest = parser.Space0(rest) overflow, frac, rest, err := matchFrac(rest, fsp) - if err != nil || len(rest) > 0 { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + if err != nil || (len(rest) > 0 && charsLen >= 12) { + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } if overflow { @@ -1718,7 +1719,7 @@ func matchDuration(str string, fsp int) (Duration, error) { } if !checkHHMMSS(hhmmss) { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } if hhmmss[0] > TimeMaxHour { @@ -1728,7 +1729,7 @@ func matchDuration(str string, fsp int) (Duration, error) { } else { t = MaxTime } - return Duration{t, fsp}, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return Duration{t, fsp}, false, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } d := gotime.Duration(hhmmss[0]*3600+hhmmss[1]*60+hhmmss[2])*gotime.Second + gotime.Duration(frac)*gotime.Microsecond //nolint:durationcheck @@ -1736,7 +1737,10 @@ func matchDuration(str string, fsp int) (Duration, error) { d = -d } d, err = TruncateOverflowMySQLTime(d) - return Duration{d, fsp}, errors.Trace(err) + if err == nil && len(rest) > 0 { + return Duration{d, fsp}, false, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + } + return Duration{d, fsp}, false, errors.Trace(err) } // canFallbackToDateTime return true @@ -1776,29 +1780,30 @@ func canFallbackToDateTime(str string) bool { } // ParseDuration parses the time form a formatted string with a fractional seconds part, -// returns the duration type Time value. +// returns the duration type Time value and bool to indicate whether the result is null. // See http://dev.mysql.com/doc/refman/5.7/en/fractional-seconds.html -func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int) (Duration, error) { +func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int) (Duration, bool, error) { rest := strings.TrimSpace(str) - d, err := matchDuration(rest, fsp) + d, isNull, err := matchDuration(rest, fsp) if err == nil { - return d, nil + return d, isNull, nil } if !canFallbackToDateTime(rest) { - return d, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return d, isNull, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } datetime, err := ParseDatetime(sc, rest) if err != nil { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } d, err = datetime.ConvertToDuration() if err != nil { - return ZeroDuration, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) + return ZeroDuration, true, ErrTruncatedWrongVal.GenWithStackByArgs("time", str) } - return d.RoundFrac(fsp, sc.TimeZone) + d, err = d.RoundFrac(fsp, sc.TimeZone) + return d, false, err } // TruncateOverflowMySQLTime truncates d when it overflows, and returns ErrTruncatedWrongVal. diff --git a/types/time_test.go b/types/time_test.go index a1dfa57ea5402..13e9191ec4175 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -334,8 +334,9 @@ func TestTime(t *testing.T) { } for _, test := range table { - duration, err := types.ParseDuration(sc, test.Input, types.MinFsp) + duration, isNull, err := types.ParseDuration(sc, test.Input, types.MinFsp) require.NoError(t, err) + require.False(t, isNull) require.Equal(t, test.Expect, duration.String()) } @@ -349,11 +350,27 @@ func TestTime(t *testing.T) { } for _, test := range table { - duration, err := types.ParseDuration(sc, test.Input, types.MaxFsp) + duration, _, err := types.ParseDuration(sc, test.Input, types.MaxFsp) require.NoError(t, err) require.Equal(t, test.Expect, duration.String()) } + table = []struct { + Input string + Expect string + }{ + {"0x", "00:00:00.000000"}, + {"1x", "00:00:01.000000"}, + {"0000-00-00", "00:00:00.000000"}, + } + + for _, test := range table { + duration, isNull, err := types.ParseDuration(sc, test.Input, types.MaxFsp) + require.False(t, isNull) + require.True(t, types.ErrTruncatedWrongVal.Equal(err)) + require.Equal(t, test.Expect, duration.String()) + } + errTable := []string{ "2011-11-11", "232 10", @@ -361,11 +378,11 @@ func TestTime(t *testing.T) { } for _, test := range errTable { - _, err := types.ParseDuration(sc, test, types.DefaultFsp) + _, _, err := types.ParseDuration(sc, test, types.DefaultFsp) require.Error(t, err) } - duration, err := types.ParseDuration(sc, "4294967295 0:59:59", types.DefaultFsp) + duration, _, err := types.ParseDuration(sc, "4294967295 0:59:59", types.DefaultFsp) require.Error(t, err) require.Equal(t, "838:59:59", duration.String()) @@ -408,15 +425,15 @@ func TestDurationAdd(t *testing.T) { {"00:00:00.099", 3, "00:00:00.001", 3, "00:00:00.100"}, } for _, test := range table { - duration, err := types.ParseDuration(nil, test.Input, test.Fsp) + duration, _, err := types.ParseDuration(nil, test.Input, test.Fsp) require.NoError(t, err) - ta, err := types.ParseDuration(nil, test.InputAdd, test.FspAdd) + ta, _, err := types.ParseDuration(nil, test.InputAdd, test.FspAdd) require.NoError(t, err) result, err := duration.Add(ta) require.NoError(t, err) require.Equal(t, test.Expect, result.String()) } - duration, err := types.ParseDuration(nil, "00:00:00", 0) + duration, _, err := types.ParseDuration(nil, "00:00:00", 0) require.NoError(t, err) ta := new(types.Duration) result, err := duration.Add(*ta) @@ -424,7 +441,7 @@ func TestDurationAdd(t *testing.T) { require.Equal(t, "00:00:00", result.String()) duration = types.Duration{Duration: math.MaxInt64, Fsp: 0} - tatmp, err := types.ParseDuration(nil, "00:01:00", 0) + tatmp, _, err := types.ParseDuration(nil, "00:01:00", 0) require.NoError(t, err) _, err = duration.Add(tatmp) require.Error(t, err) @@ -444,9 +461,9 @@ func TestDurationSub(t *testing.T) { {"00:00:00", 0, "00:00:00.1", 1, "-00:00:00.1"}, } for _, test := range table { - duration, err := types.ParseDuration(sc, test.Input, test.Fsp) + duration, _, err := types.ParseDuration(sc, test.Input, test.Fsp) require.NoError(t, err) - ta, err := types.ParseDuration(sc, test.InputAdd, test.FspAdd) + ta, _, err := types.ParseDuration(sc, test.InputAdd, test.FspAdd) require.NoError(t, err) result, err := duration.Sub(ta) require.NoError(t, err) @@ -475,7 +492,7 @@ func TestTimeFsp(t *testing.T) { } for _, test := range table { - duration, err := types.ParseDuration(sc, test.Input, test.Fsp) + duration, _, err := types.ParseDuration(sc, test.Input, test.Fsp) require.NoError(t, err) require.Equal(t, test.Expect, duration.String()) } @@ -488,7 +505,7 @@ func TestTimeFsp(t *testing.T) { } for _, test := range errTable { - _, err := types.ParseDuration(sc, test.Input, test.Fsp) + _, _, err := types.ParseDuration(sc, test.Input, test.Fsp) require.Error(t, err) } } @@ -753,7 +770,7 @@ func TestToNumber(t *testing.T) { } for _, test := range tblDuration { - v, err := types.ParseDuration(sc, test.Input, test.Fsp) + v, _, err := types.ParseDuration(sc, test.Input, test.Fsp) require.NoError(t, err) // now we can only changetypes.Duration's Fsp to check ToNumber with different Fsp require.Equal(t, test.Expect, v.ToNumber().String()) @@ -897,7 +914,7 @@ func TestRoundFrac(t *testing.T) { } for _, tt := range tbl { - v, err := types.ParseDuration(sc, tt.Input, types.MaxFsp) + v, _, err := types.ParseDuration(sc, tt.Input, types.MaxFsp) require.NoError(t, err) nv, err := v.RoundFrac(tt.Fsp, sc.TimeZone) require.NoError(t, err) @@ -959,7 +976,7 @@ func TestConvert(t *testing.T) { // test different time zone. sc.TimeZone = time.UTC for _, tt := range tblDuration { - v, err := types.ParseDuration(sc, tt.Input, tt.Fsp) + v, _, err := types.ParseDuration(sc, tt.Input, tt.Fsp) require.NoError(t, err) year, month, day := time.Now().In(sc.TimeZone).Date() n := time.Date(year, month, day, 0, 0, 0, 0, sc.TimeZone) @@ -1010,7 +1027,7 @@ func TestCompare(t *testing.T) { } for _, tt := range tbl { - v1, err := types.ParseDuration(nil, tt.Arg1, types.MaxFsp) + v1, _, err := types.ParseDuration(nil, tt.Arg1, types.MaxFsp) require.NoError(t, err) ret, err := v1.CompareString(nil, tt.Arg2) @@ -1034,7 +1051,7 @@ func TestDurationClock(t *testing.T) { } for _, tt := range tbl { - d, err := types.ParseDuration(&stmtctx.StatementContext{TimeZone: time.UTC}, tt.Input, types.MaxFsp) + d, _, err := types.ParseDuration(&stmtctx.StatementContext{TimeZone: time.UTC}, tt.Input, types.MaxFsp) require.NoError(t, err) require.Equal(t, tt.Hour, d.Hour()) require.Equal(t, tt.Minute, d.Minute()) @@ -1151,7 +1168,7 @@ func TestTimeAdd(t *testing.T) { for _, tt := range tbl { v1, err := types.ParseTime(sc, tt.Arg1, mysql.TypeDatetime, types.MaxFsp) require.NoError(t, err) - dur, err := types.ParseDuration(sc, tt.Arg2, types.MaxFsp) + dur, _, err := types.ParseDuration(sc, tt.Arg2, types.MaxFsp) require.NoError(t, err) result, err := types.ParseTime(sc, tt.Ret, mysql.TypeDatetime, types.MaxFsp) require.NoError(t, err) @@ -1976,7 +1993,7 @@ func TestTimeSub(t *testing.T) { require.NoError(t, err) v2, err := types.ParseTime(sc, tt.Arg2, mysql.TypeDatetime, types.MaxFsp) require.NoError(t, err) - dur, err := types.ParseDuration(sc, tt.Ret, types.MaxFsp) + dur, _, err := types.ParseDuration(sc, tt.Ret, types.MaxFsp) require.NoError(t, err) rec := v1.Sub(sc, &v2) require.Equal(t, dur, rec) @@ -2209,7 +2226,7 @@ func BenchmarkTimeAdd(b *testing.B) { TimeZone: time.UTC, } arg1, _ := types.ParseTime(sc, "2017-01-18", mysql.TypeDatetime, types.MaxFsp) - arg2, _ := types.ParseDuration(sc, "12:30:59", types.MaxFsp) + arg2, _, _ := types.ParseDuration(sc, "12:30:59", types.MaxFsp) for i := 0; i < b.N; i++ { _, err := arg1.Add(sc, arg2) if err != nil { diff --git a/util/chunk/mutrow_test.go b/util/chunk/mutrow_test.go index 1293240013a3d..532206cf42782 100644 --- a/util/chunk/mutrow_test.go +++ b/util/chunk/mutrow_test.go @@ -80,7 +80,7 @@ func TestMutRow(t *testing.T) { retTypes := []*types.FieldType{types.NewFieldType(mysql.TypeDuration)} chk := New(retTypes, 1, 1) - dur, err := types.ParseDuration(sc, "01:23:45", 0) + dur, _, err := types.ParseDuration(sc, "01:23:45", 0) require.NoError(t, err) chk.AppendDuration(0, dur) mutRow = MutRowFromTypes(retTypes) diff --git a/util/codec/codec_test.go b/util/codec/codec_test.go index 33af7c59a9840..a1a5461dcd5fa 100644 --- a/util/codec/codec_test.go +++ b/util/codec/codec_test.go @@ -527,7 +527,7 @@ func parseTime(t *testing.T, s string) types.Time { } func parseDuration(t *testing.T, s string) types.Duration { - m, err := types.ParseDuration(nil, s, types.DefaultFsp) + m, _, err := types.ParseDuration(nil, s, types.DefaultFsp) require.NoError(t, err) return m } diff --git a/util/rowcodec/rowcodec_test.go b/util/rowcodec/rowcodec_test.go index 51c965f095cd6..8772a50eb3d5d 100644 --- a/util/rowcodec/rowcodec_test.go +++ b/util/rowcodec/rowcodec_test.go @@ -890,7 +890,7 @@ var ( } } getDuration = func(value string) types.Duration { - dur, _ := types.ParseDuration(nil, value, 0) + dur, _, _ := types.ParseDuration(nil, value, 0) return dur } getOldDatumByte = func(d types.Datum) []byte { From 966844bf5048c2cb0ef4519448b8eb46fb53539a Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 23 Jun 2022 14:28:37 +0800 Subject: [PATCH 20/21] *: enable durationcheck for nogo linter (#35661) ref pingcap/tidb#35345 --- DEPS.bzl | 8 +++++++ br/pkg/lightning/BUILD.bazel | 1 + br/pkg/lightning/errormanager/BUILD.bazel | 1 + build/BUILD.bazel | 1 + build/linter/durationcheck/BUILD.bazel | 12 ++++++++++ build/linter/durationcheck/analyzer.go | 27 +++++++++++++++++++++++ build/linter/util/util.go | 2 +- build/nogo_config.json | 6 +++++ go.mod | 1 + go.sum | 2 ++ kv/BUILD.bazel | 1 + session/BUILD.bazel | 1 + session/session_test/BUILD.bazel | 2 ++ sessionctx/sessionstates/BUILD.bazel | 2 ++ util/dbutil/common.go | 2 +- util/set/BUILD.bazel | 2 ++ 16 files changed, 69 insertions(+), 2 deletions(-) create mode 100644 build/linter/durationcheck/BUILD.bazel create mode 100644 build/linter/durationcheck/analyzer.go diff --git a/DEPS.bzl b/DEPS.bzl index c9625069192e7..b41d46497f96d 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -220,6 +220,14 @@ def go_deps(): sum = "h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE=", version = "v2.1.2", ) + go_repository( + name = "com_github_charithe_durationcheck", + build_file_proto_mode = "disable", + importpath = "github.com/charithe/durationcheck", + sum = "h1:mPP4ucLrf/rKZiIG/a9IPXHGlh8p4CzgpyTy6EEutYk=", + version = "v0.0.9", + ) + go_repository( name = "com_github_cheggaaa_pb_v3", build_file_proto_mode = "disable_global", diff --git a/br/pkg/lightning/BUILD.bazel b/br/pkg/lightning/BUILD.bazel index 933ad8e1162d4..1cafc9ec5fbef 100644 --- a/br/pkg/lightning/BUILD.bazel +++ b/br/pkg/lightning/BUILD.bazel @@ -51,6 +51,7 @@ go_test( "//br/pkg/lightning/checkpoints", "//br/pkg/lightning/config", "//br/pkg/lightning/glue", + "//br/pkg/lightning/log", "//br/pkg/lightning/mydump", "//br/pkg/lightning/web", "@com_github_docker_go_units//:go-units", diff --git a/br/pkg/lightning/errormanager/BUILD.bazel b/br/pkg/lightning/errormanager/BUILD.bazel index e8c61ab00f0b2..7aea8447865e8 100644 --- a/br/pkg/lightning/errormanager/BUILD.bazel +++ b/br/pkg/lightning/errormanager/BUILD.bazel @@ -26,6 +26,7 @@ go_test( embed = [":errormanager"], deps = [ "//br/pkg/lightning/config", + "//br/pkg/lightning/log", "//br/pkg/utils", "@com_github_data_dog_go_sqlmock//:go-sqlmock", "@com_github_stretchr_testify//require", diff --git a/build/BUILD.bazel b/build/BUILD.bazel index 22bc4b679c26a..7bf1a9778f709 100644 --- a/build/BUILD.bazel +++ b/build/BUILD.bazel @@ -81,6 +81,7 @@ nogo( "@org_golang_x_tools//go/analysis/passes/unreachable:go_default_library", "@org_golang_x_tools//go/analysis/passes/unsafeptr:go_default_library", "@org_golang_x_tools//go/analysis/passes/unusedresult:go_default_library", + "//build/linter/durationcheck:durationcheck", "//build/linter/prealloc:prealloc", ] + staticcheck_analyzers(STATICHECK_ANALYZERS), ) diff --git a/build/linter/durationcheck/BUILD.bazel b/build/linter/durationcheck/BUILD.bazel new file mode 100644 index 0000000000000..556720ba0609c --- /dev/null +++ b/build/linter/durationcheck/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "durationcheck", + srcs = ["analyzer.go"], + importpath = "github.com/pingcap/tidb/build/linter/durationcheck", + visibility = ["//visibility:public"], + deps = [ + "//build/linter/util", + "@com_github_charithe_durationcheck//:durationcheck", + ], +) diff --git a/build/linter/durationcheck/analyzer.go b/build/linter/durationcheck/analyzer.go new file mode 100644 index 0000000000000..1ebee429959fd --- /dev/null +++ b/build/linter/durationcheck/analyzer.go @@ -0,0 +1,27 @@ +// Copyright 2022 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, +// 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 durationcheck + +import ( + "github.com/charithe/durationcheck" + "github.com/pingcap/tidb/build/linter/util" +) + +// Analyzer is the analyzer struct of durationcheck. +var Analyzer = durationcheck.Analyzer + +func init() { + util.SkipAnalyzer(Analyzer) +} diff --git a/build/linter/util/util.go b/build/linter/util/util.go index d438ea0057a6e..d476173a973a0 100644 --- a/build/linter/util/util.go +++ b/build/linter/util/util.go @@ -55,7 +55,7 @@ func parseDirective(s string) (cmd skipType, args []string) { } return skipNone, nil } - s = strings.TrimPrefix(s, "//nolint: ") + s = strings.TrimPrefix(s, "//nolint:") return skipLinter, []string{s} } diff --git a/build/nogo_config.json b/build/nogo_config.json index f05718081c8ac..5fe822cce2963 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -80,6 +80,12 @@ ".*_generated\\.go$": "ignore generated code" } }, + "durationcheck": { + "exclude_files": { + "/external/": "no need to vet third party code", + ".*_generated\\.go$": "ignore generated code" + } + }, "errorsas": { "exclude_files": { "/external/": "no need to vet third party code", diff --git a/go.mod b/go.mod index cd5327db9c2c2..8af187bf5e607 100644 --- a/go.mod +++ b/go.mod @@ -97,6 +97,7 @@ require ( require ( github.com/aliyun/alibaba-cloud-sdk-go v1.61.1581 + github.com/charithe/durationcheck v0.0.9 honnef.co/go/tools v0.0.1-2020.1.4 ) diff --git a/go.sum b/go.sum index c5f5f515043a7..77de5a0c0284c 100644 --- a/go.sum +++ b/go.sum @@ -131,6 +131,8 @@ github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghf github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/charithe/durationcheck v0.0.9 h1:mPP4ucLrf/rKZiIG/a9IPXHGlh8p4CzgpyTy6EEutYk= +github.com/charithe/durationcheck v0.0.9/go.mod h1:SSbRIBVfMjCi/kEB6K65XEA83D6prSM8ap1UCpNKtgg= github.com/cheggaaa/pb/v3 v3.0.8 h1:bC8oemdChbke2FHIIGy9mn4DPJ2caZYQnfbRqwmdCoA= github.com/cheggaaa/pb/v3 v3.0.8/go.mod h1:UICbiLec/XO6Hw6k+BHEtHeQFzzBH4i2/qk/ow1EJTA= github.com/cheynewallace/tabby v1.1.1 h1:JvUR8waht4Y0S3JF17G6Vhyt+FRhnqVCkk8l4YrOU54= diff --git a/kv/BUILD.bazel b/kv/BUILD.bazel index 600cec889528f..fed476f803dfc 100644 --- a/kv/BUILD.bazel +++ b/kv/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "//util/dbterror", "//util/logutil", "//util/memory", + "//util/set", "//util/trxevents", "@com_github_coocood_freecache//:freecache", "@com_github_pingcap_errors//:errors", diff --git a/session/BUILD.bazel b/session/BUILD.bazel index e6cfe0f898682..a48f8238cc5ca 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -75,6 +75,7 @@ go_library( "//util/memory", "//util/parser", "//util/rowcodec", + "//util/sem", "//util/sli", "//util/sqlexec", "//util/tableutil", diff --git a/session/session_test/BUILD.bazel b/session/session_test/BUILD.bazel index 47c330e409de7..57900ac9d97cc 100644 --- a/session/session_test/BUILD.bazel +++ b/session/session_test/BUILD.bazel @@ -12,7 +12,9 @@ go_test( "//config", "//domain", "//kv", + "//parser/auth", "//parser/terror", + "//planner/core", "//session", "//store/mockstore", "//testkit", diff --git a/sessionctx/sessionstates/BUILD.bazel b/sessionctx/sessionstates/BUILD.bazel index cb421dfeac1ec..ce2f91c36176a 100644 --- a/sessionctx/sessionstates/BUILD.bazel +++ b/sessionctx/sessionstates/BUILD.bazel @@ -16,7 +16,9 @@ go_test( srcs = ["session_states_test.go"], deps = [ "//errno", + "//sessionctx/variable", "//testkit", + "//util/sem", "@com_github_stretchr_testify//require", ], ) diff --git a/util/dbutil/common.go b/util/dbutil/common.go index ec0ffd785349e..674314af3492f 100644 --- a/util/dbutil/common.go +++ b/util/dbutil/common.go @@ -319,7 +319,7 @@ func GetTimeZoneOffset(ctx context.Context, db QueryExecutor) (time.Duration, er } hour, minute, second := t.Clock() - // nolint:durationcheck + //nolint:durationcheck return time.Duration(hour*3600+minute*60+second) * time.Second * factor, nil } diff --git a/util/set/BUILD.bazel b/util/set/BUILD.bazel index 790799294974f..0f733ad14ee41 100644 --- a/util/set/BUILD.bazel +++ b/util/set/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "float64_set.go", "int_set.go", + "mem_aware_map.go", "set_with_memory_usage.go", "string_set.go", ], @@ -19,6 +20,7 @@ go_test( "float64_set_test.go", "int_set_test.go", "main_test.go", + "mem_aware_map_test.go", "set_with_memory_usage_test.go", "string_set_test.go", ], From e2b55640c179c3fa2d3ce4cca58999c47d7eeabe Mon Sep 17 00:00:00 2001 From: CbcWestwolf <1004626265@qq.com> Date: Thu, 23 Jun 2022 15:12:37 +0800 Subject: [PATCH 21/21] config, sysvar: map `max-server-connections` to `max_connections` (#35453) ref pingcap/tidb#34960 --- config/config.go | 3 +++ config/config.toml.example | 6 +++--- config/config_test.go | 15 +++++++++------ executor/set_test.go | 13 +++++++------ expression/integration_test.go | 4 ++-- server/server.go | 21 ++++++++++++++------- sessionctx/variable/noop.go | 1 - sessionctx/variable/sysvar.go | 6 ++++++ tidb-server/main.go | 2 ++ 9 files changed, 46 insertions(+), 25 deletions(-) diff --git a/config/config.go b/config/config.go index 48e069bcfb50b..0dc6cabc75727 100644 --- a/config/config.go +++ b/config/config.go @@ -117,6 +117,7 @@ var ( map[string]string{ "check-mb4-value-in-utf8": "tidb_check_mb4_value_in_utf8", "enable-collect-execution-info": "tidb_enable_collect_execution_info", + "max-server-connections": "max_connections", }, }, { @@ -474,6 +475,7 @@ type Instance struct { EnableCollectExecutionInfo bool `toml:"tidb_enable_collect_execution_info" json:"tidb_enable_collect_execution_info"` PluginDir string `toml:"plugin_dir" json:"plugin_dir"` PluginLoad string `toml:"plugin_load" json:"plugin_load"` + MaxConnections uint32 `toml:"max_connections" json:"max_connections"` } func (l *Log) getDisableTimestamp() bool { @@ -850,6 +852,7 @@ var defaultConf = Config{ EnableCollectExecutionInfo: true, PluginDir: "/data/deploy/plugin", PluginLoad: "", + MaxConnections: 0, }, Status: Status{ ReportStatus: true, diff --git a/config/config.toml.example b/config/config.toml.example index 1e1fc7aae73f5..afc97e60e74f8 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -89,9 +89,6 @@ repair-mode = false # In repair mode, repairing table which is not in repair list will get wrong database or wrong table error. repair-table-list = [] -# The maximum permitted number of simultaneous client connections. When the value is 0, the number of connections is unlimited. -max-server-connections = 0 - # Whether new collations are enabled, as indicated by its name, this configuration entry take effect ONLY when a TiDB cluster bootstraps for the first time. new_collations_enabled_on_first_bootstrap = true @@ -468,3 +465,6 @@ tidb_slow_log_threshold = 300 # tidb_record_plan_in_slow_log is used to enable record query plan in slow log. # 0 is disable. 1 is enable. tidb_record_plan_in_slow_log = 1 + +# The maximum permitted number of simultaneous client connections. When the value is 0, the number of connections is unlimited. +max_connections = 0 diff --git a/config/config_test.go b/config/config_test.go index 391bd874d3942..2e044062bd4e8 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -282,9 +282,6 @@ repair-mode = false # In repair mode, repairing table which is not in repair list will get wrong database or wrong table error. repair-table-list = [] -# The maximum permitted number of simultaneous client connections. When the value is 0, the number of connections is unlimited. -max-server-connections = 0 - # Whether new collations are enabled, as indicated by its name, this configuration entry take effect ONLY when a TiDB cluster bootstraps for the first time. new_collations_enabled_on_first_bootstrap = true @@ -309,6 +306,11 @@ deprecate-integer-display-length = false # See https://dev.mysql.com/doc/refman/8.0/en/string-type-syntax.html for more details. enable-enum-length-limit = true +[instance] + +# The maximum permitted number of simultaneous client connections. When the value is 0, the number of connections is unlimited. +max_connections = 0 + [log] # Log level: debug, info, warn, error, fatal. level = "info" @@ -707,7 +709,7 @@ unrecognized-option-test = true match, err := regexp.Match("(?:.|\n)*invalid configuration option(?:.|\n)*", []byte(err.Error())) require.NoError(t, err) require.True(t, match) - require.Equal(t, uint32(0), conf.MaxServerConnections) + require.Equal(t, uint32(0), conf.Instance.MaxConnections) err = f.Truncate(0) require.NoError(t, err) @@ -722,7 +724,6 @@ delay-clean-table-lock = 5 split-region-max-num=10000 server-version = "test_version" repair-mode = true -max-server-connections = 200 max-index-length = 3080 index-limit = 70 table-column-count-limit = 4000 @@ -768,6 +769,8 @@ grpc-keepalive-timeout = 10 grpc-concurrent-streams = 2048 grpc-initial-window-size = 10240 grpc-max-send-msg-size = 40960 +[instance] +max_connections = 200 `) require.NoError(t, err) @@ -797,7 +800,7 @@ grpc-max-send-msg-size = 40960 require.Equal(t, uint64(10000), conf.SplitRegionMaxNum) require.True(t, conf.RepairMode) require.Equal(t, uint64(16), conf.TiKVClient.ResolveLockLiteThreshold) - require.Equal(t, uint32(200), conf.MaxServerConnections) + require.Equal(t, uint32(200), conf.Instance.MaxConnections) require.Equal(t, []string{"tiflash"}, conf.IsolationRead.Engines) require.Equal(t, 3080, conf.MaxIndexLength) require.Equal(t, 70, conf.IndexLimit) diff --git a/executor/set_test.go b/executor/set_test.go index eb171e872d8c4..9a7213571fddc 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -1021,16 +1021,17 @@ func TestValidateSetVar(t *testing.T) { result.Check(testkit.Rows("SYSTEM")) // The following cases test value out of range and illegal type when setting system variables. - // See https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html for more details. + // See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html for more details. tk.MustExec("set @@global.max_connections=100001") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '100001'")) result = tk.MustQuery("select @@global.max_connections;") result.Check(testkit.Rows("100000")) + // "max_connections == 0" means there is no limitation on the number of connections. tk.MustExec("set @@global.max_connections=-1") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '-1'")) result = tk.MustQuery("select @@global.max_connections;") - result.Check(testkit.Rows("1")) + result.Check(testkit.Rows("0")) err = tk.ExecToErr("set @@global.max_connections='hello'") require.True(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar)) @@ -1077,7 +1078,7 @@ func TestValidateSetVar(t *testing.T) { tk.MustExec("set @@global.max_connections=-1") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect max_connections value: '-1'")) result = tk.MustQuery("select @@global.max_connections;") - result.Check(testkit.Rows("1")) + result.Check(testkit.Rows("0")) err = tk.ExecToErr("set @@global.max_connections='hello'") require.True(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar)) @@ -1333,15 +1334,15 @@ func TestSelectGlobalVar(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - tk.MustQuery("select @@global.max_connections;").Check(testkit.Rows("151")) - tk.MustQuery("select @@max_connections;").Check(testkit.Rows("151")) + tk.MustQuery("select @@global.max_connections;").Check(testkit.Rows("0")) + tk.MustQuery("select @@max_connections;").Check(testkit.Rows("0")) tk.MustExec("set @@global.max_connections=100;") tk.MustQuery("select @@global.max_connections;").Check(testkit.Rows("100")) tk.MustQuery("select @@max_connections;").Check(testkit.Rows("100")) - tk.MustExec("set @@global.max_connections=151;") + tk.MustExec("set @@global.max_connections=0;") // test for unknown variable. err := tk.ExecToErr("select @@invalid") diff --git a/expression/integration_test.go b/expression/integration_test.go index 8c10fc96b5d30..b3dc43fe084d1 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -6233,11 +6233,11 @@ func TestGlobalCacheCorrectness(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - tk.MustQuery("SHOW VARIABLES LIKE 'max_connections'").Check(testkit.Rows("max_connections 151")) + tk.MustQuery("SHOW VARIABLES LIKE 'max_connections'").Check(testkit.Rows("max_connections 0")) tk.MustExec("SET GLOBAL max_connections=1234") tk.MustQuery("SHOW VARIABLES LIKE 'max_connections'").Check(testkit.Rows("max_connections 1234")) // restore - tk.MustExec("SET GLOBAL max_connections=151") + tk.MustExec("SET GLOBAL max_connections=0") } func TestRedundantColumnResolve(t *testing.T) { diff --git a/server/server.go b/server/server.go index 90b4a7e27101b..13ed052391f59 100644 --- a/server/server.go +++ b/server/server.go @@ -352,7 +352,7 @@ func setTxnScope() { // Export config-related metrics func (s *Server) reportConfig() { metrics.ConfigStatus.WithLabelValues("token-limit").Set(float64(s.cfg.TokenLimit)) - metrics.ConfigStatus.WithLabelValues("max-server-connections").Set(float64(s.cfg.MaxServerConnections)) + metrics.ConfigStatus.WithLabelValues("max_connections").Set(float64(s.cfg.Instance.MaxConnections)) } // Run runs the server. @@ -514,11 +514,18 @@ func (s *Server) onConn(conn *clientConn) { }) terror.Log(err) } - if errors.Cause(err) == io.EOF { + switch errors.Cause(err) { + case io.EOF: // `EOF` means the connection is closed normally, we do not treat it as a noticeable error and log it in 'DEBUG' level. logutil.BgLogger().With(zap.Uint64("conn", conn.connectionID)). Debug("EOF", zap.String("remote addr", conn.bufReadConn.RemoteAddr().String())) - } else { + case errConCount: + if err := conn.writeError(ctx, err); err != nil { + logutil.BgLogger().With(zap.Uint64("conn", conn.connectionID)). + Warn("error in writing errConCount", zap.Error(err), + zap.String("remote addr", conn.bufReadConn.RemoteAddr().String())) + } + default: metrics.HandShakeErrorCounter.Inc() logutil.BgLogger().With(zap.Uint64("conn", conn.connectionID)). Warn("Server.onConn handshake", zap.Error(err), @@ -605,8 +612,8 @@ func (cc *clientConn) connectInfo() *variable.ConnectionInfo { } func (s *Server) checkConnectionCount() error { - // When the value of MaxServerConnections is 0, the number of connections is unlimited. - if int(s.cfg.MaxServerConnections) == 0 { + // When the value of Instance.MaxConnections is 0, the number of connections is unlimited. + if int(s.cfg.Instance.MaxConnections) == 0 { return nil } @@ -614,9 +621,9 @@ func (s *Server) checkConnectionCount() error { conns := len(s.clients) s.rwlock.RUnlock() - if conns >= int(s.cfg.MaxServerConnections) { + if conns >= int(s.cfg.Instance.MaxConnections) { logutil.BgLogger().Error("too many connections", - zap.Uint32("max connections", s.cfg.MaxServerConnections), zap.Error(errConCount)) + zap.Uint32("max connections", s.cfg.Instance.MaxConnections), zap.Error(errConCount)) return errConCount } return nil diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 4f2cdac1aa690..6eb70beabfc99 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -24,7 +24,6 @@ import ( // but changing them has no effect on behavior. var noopSysVars = []*SysVar{ - {Scope: ScopeGlobal, Name: MaxConnections, Value: "151", Type: TypeUnsigned, MinValue: 1, MaxValue: 100000}, // It is unsafe to pretend that any variation of "read only" is enabled when the server // does not support it. It is possible that these features will be supported in future, // but until then... diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index aa91b574dc517..2e7865ac57040 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -431,6 +431,12 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeInstance, Name: PluginDir, Value: "/data/deploy/plugin", ReadOnly: true, GetGlobal: func(s *SessionVars) (string, error) { return config.GetGlobalConfig().Instance.PluginDir, nil }}, + {Scope: ScopeInstance, Name: MaxConnections, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().Instance.MaxConnections), 10), Type: TypeUnsigned, MinValue: 0, MaxValue: 100000, SetGlobal: func(s *SessionVars, val string) error { + config.GetGlobalConfig().Instance.MaxConnections = uint32(TidbOptInt64(val, 0)) + return nil + }, GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatUint(uint64(config.GetGlobalConfig().Instance.MaxConnections), 10), nil + }}, /* The system variables below have GLOBAL scope */ {Scope: ScopeGlobal, Name: MaxPreparedStmtCount, Value: strconv.FormatInt(DefMaxPreparedStmtCount, 10), Type: TypeInt, MinValue: -1, MaxValue: 1048576}, diff --git a/tidb-server/main.go b/tidb-server/main.go index 763d4bd5bc6a8..bc8795875f360 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -559,6 +559,8 @@ func setGlobalVars() { cfg.Instance.CheckMb4ValueInUTF8.Store(cfg.CheckMb4ValueInUTF8.Load()) case "enable-collect-execution-info": cfg.Instance.EnableCollectExecutionInfo = cfg.EnableCollectExecutionInfo + case "max-server-connections": + cfg.Instance.MaxConnections = cfg.MaxServerConnections } case "log": switch oldName {