Skip to content

Commit eae342b

Browse files
committed
Merge branch 'master' into paging-unistore
2 parents 68f94d1 + e1f9e0a commit eae342b

File tree

96 files changed

+6404
-5000
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

96 files changed

+6404
-5000
lines changed

README.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ For more details and latest updates, see [TiDB docs](https://docs.pingcap.com/ti
2626

2727
TiDB Cloud is the fully-managed service of TiDB, currently available on AWS and GCP.
2828

29-
Quickly check out TiDB Cloud with [a free trial](https://tidbcloud.com/signup).
29+
Quickly check out TiDB Cloud with [a free trial](https://tidbcloud.com/free-trial).
3030

3131
See [TiDB Cloud Quick Start Guide](https://docs.pingcap.com/tidbcloud/tidb-cloud-quickstart).
3232

br/pkg/lightning/BUILD.bazel

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ go_library(
1717
"//br/pkg/lightning/config",
1818
"//br/pkg/lightning/glue",
1919
"//br/pkg/lightning/log",
20+
"//br/pkg/lightning/metric",
2021
"//br/pkg/lightning/mydump",
2122
"//br/pkg/lightning/restore",
2223
"//br/pkg/lightning/tikv",
@@ -25,9 +26,12 @@ go_library(
2526
"//br/pkg/storage",
2627
"//br/pkg/utils",
2728
"//br/pkg/version/build",
29+
"//util/promutil",
2830
"@com_github_pingcap_errors//:errors",
2931
"@com_github_pingcap_failpoint//:failpoint",
3032
"@com_github_pingcap_kvproto//pkg/import_sstpb",
33+
"@com_github_prometheus_client_golang//prometheus",
34+
"@com_github_prometheus_client_golang//prometheus/collectors",
3135
"@com_github_prometheus_client_golang//prometheus/promhttp",
3236
"@com_github_shurcool_httpgzip//:httpgzip",
3337
"@org_uber_go_zap//:zap",

br/pkg/lightning/backend/backend.go

Lines changed: 16 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -340,17 +340,23 @@ func (be Backend) OpenEngine(ctx context.Context, config *EngineConfig, tableNam
340340
return nil, err
341341
}
342342

343-
openCounter := metric.ImporterEngineCounter.WithLabelValues("open")
344-
openCounter.Inc()
343+
if m, ok := metric.FromContext(ctx); ok {
344+
openCounter := m.ImporterEngineCounter.WithLabelValues("open")
345+
openCounter.Inc()
346+
}
345347

346348
logger.Info("open engine")
347349

348350
failpoint.Inject("FailIfEngineCountExceeds", func(val failpoint.Value) {
349-
closedCounter := metric.ImporterEngineCounter.WithLabelValues("closed")
350-
openCount := metric.ReadCounter(openCounter)
351-
closedCount := metric.ReadCounter(closedCounter)
352-
if injectValue := val.(int); openCount-closedCount > float64(injectValue) {
353-
panic(fmt.Sprintf("forcing failure due to FailIfEngineCountExceeds: %v - %v >= %d", openCount, closedCount, injectValue))
351+
if m, ok := metric.FromContext(ctx); ok {
352+
closedCounter := m.ImporterEngineCounter.WithLabelValues("closed")
353+
openCounter := m.ImporterEngineCounter.WithLabelValues("open")
354+
openCount := metric.ReadCounter(openCounter)
355+
356+
closedCount := metric.ReadCounter(closedCounter)
357+
if injectValue := val.(int); openCount-closedCount > float64(injectValue) {
358+
panic(fmt.Sprintf("forcing failure due to FailIfEngineCountExceeds: %v - %v >= %d", openCount, closedCount, injectValue))
359+
}
354360
}
355361
})
356362

@@ -380,7 +386,9 @@ func (be Backend) ResolveDuplicateRows(ctx context.Context, tbl table.Table, tab
380386
func (engine *OpenedEngine) Close(ctx context.Context, cfg *EngineConfig) (*ClosedEngine, error) {
381387
closedEngine, err := engine.unsafeClose(ctx, cfg)
382388
if err == nil {
383-
metric.ImporterEngineCounter.WithLabelValues("closed").Inc()
389+
if m, ok := metric.FromContext(ctx); ok {
390+
m.ImporterEngineCounter.WithLabelValues("closed").Inc()
391+
}
384392
}
385393
return closedEngine, err
386394
}

br/pkg/lightning/backend/kv/sql2kv.go

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -63,10 +63,13 @@ type tableKVEncoder struct {
6363
genCols []genCol
6464
// convert auto id for shard rowid or auto random id base on row id generated by lightning
6565
autoIDFn autoIDConverter
66+
metrics *metric.Metrics
6667
}
6768

68-
func NewTableKVEncoder(tbl table.Table, options *SessionOptions) (Encoder, error) {
69-
metric.KvEncoderCounter.WithLabelValues("open").Inc()
69+
func NewTableKVEncoder(tbl table.Table, options *SessionOptions, metrics *metric.Metrics) (Encoder, error) {
70+
if metrics != nil {
71+
metrics.KvEncoderCounter.WithLabelValues("open").Inc()
72+
}
7073
meta := tbl.Meta()
7174
cols := tbl.Cols()
7275
se := newSession(options)
@@ -108,6 +111,7 @@ func NewTableKVEncoder(tbl table.Table, options *SessionOptions) (Encoder, error
108111
se: se,
109112
genCols: genCols,
110113
autoIDFn: autoIDFn,
114+
metrics: metrics,
111115
}, nil
112116
}
113117

@@ -188,7 +192,9 @@ func collectGeneratedColumns(se *session, meta *model.TableInfo, cols []*table.C
188192

189193
func (kvcodec *tableKVEncoder) Close() {
190194
kvcodec.se.Close()
191-
metric.KvEncoderCounter.WithLabelValues("closed").Inc()
195+
if kvcodec.metrics != nil {
196+
kvcodec.metrics.KvEncoderCounter.WithLabelValues("close").Inc()
197+
}
192198
}
193199

194200
// RowArrayMarshaler wraps a slice of types.Datum for logging the content into zap.

br/pkg/lightning/backend/kv/sql2kv_test.go

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -89,7 +89,7 @@ func TestEncode(t *testing.T) {
8989
strictMode, err := NewTableKVEncoder(tbl, &SessionOptions{
9090
SQLMode: mysql.ModeStrictAllTables,
9191
Timestamp: 1234567890,
92-
})
92+
}, nil)
9393
require.NoError(t, err)
9494
pairs, err := strictMode.Encode(logger, rows, 1, []int{0, 1}, "1.csv", 1234)
9595
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) {
121121
mockMode, err := NewTableKVEncoder(mockTbl, &SessionOptions{
122122
SQLMode: mysql.ModeStrictAllTables,
123123
Timestamp: 1234567891,
124-
})
124+
}, nil)
125125
require.NoError(t, err)
126126
_, err = mockMode.Encode(logger, rowsWithPk2, 2, []int{0, 1}, "1.csv", 1234)
127127
require.EqualError(t, err, "mock error")
@@ -131,7 +131,7 @@ func TestEncode(t *testing.T) {
131131
SQLMode: mysql.ModeNone,
132132
Timestamp: 1234567892,
133133
SysVars: map[string]string{"tidb_row_format_version": "1"},
134-
})
134+
}, nil)
135135
require.NoError(t, err)
136136
pairs, err = noneMode.Encode(logger, rows, 1, []int{0, 1}, "1.csv", 1234)
137137
require.NoError(t, err)
@@ -208,7 +208,7 @@ func TestDecodeIndex(t *testing.T) {
208208
strictMode, err := NewTableKVEncoder(tbl, &SessionOptions{
209209
SQLMode: mysql.ModeStrictAllTables,
210210
Timestamp: 1234567890,
211-
})
211+
}, nil)
212212
require.NoError(t, err)
213213
pairs, err := strictMode.Encode(logger, rows, 1, []int{0, 1, -1}, "1.csv", 123)
214214
data := pairs.(*KvPairs)
@@ -247,7 +247,7 @@ func TestEncodeRowFormatV2(t *testing.T) {
247247
SQLMode: mysql.ModeNone,
248248
Timestamp: 1234567892,
249249
SysVars: map[string]string{"tidb_row_format_version": "2"},
250-
})
250+
}, nil)
251251
require.NoError(t, err)
252252
pairs, err := noneMode.Encode(logger, rows, 1, []int{0, 1}, "1.csv", 1234)
253253
require.NoError(t, err)
@@ -295,7 +295,7 @@ func TestEncodeTimestamp(t *testing.T) {
295295
"tidb_row_format_version": "1",
296296
"time_zone": "+08:00",
297297
},
298-
})
298+
}, nil)
299299
require.NoError(t, err)
300300
pairs, err := encoder.Encode(logger, nil, 70, []int{-1, 1}, "1.csv", 1234)
301301
require.NoError(t, err)
@@ -320,7 +320,7 @@ func TestEncodeDoubleAutoIncrement(t *testing.T) {
320320
SysVars: map[string]string{
321321
"tidb_row_format_version": "2",
322322
},
323-
})
323+
}, nil)
324324
require.NoError(t, err)
325325

326326
strDatumForID := types.NewStringDatum("1")
@@ -386,7 +386,7 @@ func TestEncodeMissingAutoValue(t *testing.T) {
386386
SysVars: map[string]string{
387387
"tidb_row_format_version": "2",
388388
},
389-
})
389+
}, nil)
390390
require.NoError(t, err)
391391

392392
realRowID := encoder.(*tableKVEncoder).autoIDFn(rowID)
@@ -447,7 +447,7 @@ func TestDefaultAutoRandoms(t *testing.T) {
447447
Timestamp: 1234567893,
448448
SysVars: map[string]string{"tidb_row_format_version": "2"},
449449
AutoRandomSeed: 456,
450-
})
450+
}, nil)
451451
require.NoError(t, err)
452452
logger := log.Logger{Logger: zap.NewNop()}
453453
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) {
482482
Timestamp: 1234567893,
483483
SysVars: map[string]string{"tidb_row_format_version": "2"},
484484
AutoRandomSeed: 456,
485-
})
485+
}, nil)
486486
require.NoError(t, err)
487487
logger := log.Logger{Logger: zap.NewNop()}
488488
keyMap := make(map[int64]struct{}, 16)
@@ -636,7 +636,7 @@ func SetUpTest(b *testing.B) *benchSQL2KVSuite {
636636
// Construct the corresponding KV encoder.
637637
tbl, err := tables.TableFromMeta(NewPanickingAllocators(0), tableInfo)
638638
require.NoError(b, err)
639-
encoder, err := NewTableKVEncoder(tbl, &SessionOptions{SysVars: map[string]string{"tidb_row_format_version": "2"}})
639+
encoder, err := NewTableKVEncoder(tbl, &SessionOptions{SysVars: map[string]string{"tidb_row_format_version": "2"}}, nil)
640640
require.NoError(b, err)
641641
logger := log.Logger{Logger: zap.NewNop()}
642642

br/pkg/lightning/backend/local/local.go

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -233,6 +233,7 @@ type local struct {
233233
importClientFactory ImportClientFactory
234234

235235
bufferPool *membuf.Pool
236+
metrics *metric.Metrics
236237
}
237238

238239
func openDuplicateDB(storeDir string) (*pebble.DB, error) {
@@ -334,6 +335,9 @@ func NewLocalBackend(
334335
importClientFactory: importClientFactory,
335336
bufferPool: membuf.NewPool(membuf.WithAllocator(manual.Allocator{})),
336337
}
338+
if m, ok := metric.FromContext(ctx); ok {
339+
local.metrics = m
340+
}
337341
if err = local.checkMultiIngestSupport(ctx); err != nil {
338342
return backend.MakeBackend(nil), common.ErrCheckMultiIngest.Wrap(err).GenWithStackByArgs()
339343
}
@@ -1247,7 +1251,9 @@ loopWrite:
12471251
engine.importedKVSize.Add(rangeStats.totalBytes)
12481252
engine.importedKVCount.Add(rangeStats.count)
12491253
engine.finishedRanges.add(finishedRange)
1250-
metric.BytesCounter.WithLabelValues(metric.BytesStateImported).Add(float64(rangeStats.totalBytes))
1254+
if local.metrics != nil {
1255+
local.metrics.BytesCounter.WithLabelValues(metric.BytesStateImported).Add(float64(rangeStats.totalBytes))
1256+
}
12511257
}
12521258
return errors.Trace(err)
12531259
}
@@ -1685,7 +1691,7 @@ func (local *local) MakeEmptyRows() kv.Rows {
16851691
}
16861692

16871693
func (local *local) NewEncoder(tbl table.Table, options *kv.SessionOptions) (kv.Encoder, error) {
1688-
return kv.NewTableKVEncoder(tbl, options)
1694+
return kv.NewTableKVEncoder(tbl, options, local.metrics)
16891695
}
16901696

16911697
func engineSSTDir(storeDir string, engineUUID uuid.UUID) string {

br/pkg/lightning/lightning.go

Lines changed: 47 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ import (
4343
"github.com/pingcap/tidb/br/pkg/lightning/config"
4444
"github.com/pingcap/tidb/br/pkg/lightning/glue"
4545
"github.com/pingcap/tidb/br/pkg/lightning/log"
46+
"github.com/pingcap/tidb/br/pkg/lightning/metric"
4647
"github.com/pingcap/tidb/br/pkg/lightning/mydump"
4748
"github.com/pingcap/tidb/br/pkg/lightning/restore"
4849
"github.com/pingcap/tidb/br/pkg/lightning/tikv"
@@ -51,6 +52,9 @@ import (
5152
"github.com/pingcap/tidb/br/pkg/storage"
5253
"github.com/pingcap/tidb/br/pkg/utils"
5354
"github.com/pingcap/tidb/br/pkg/version/build"
55+
"github.com/pingcap/tidb/util/promutil"
56+
"github.com/prometheus/client_golang/prometheus"
57+
"github.com/prometheus/client_golang/prometheus/collectors"
5458
"github.com/prometheus/client_golang/prometheus/promhttp"
5559
"github.com/shurcooL/httpgzip"
5660
"go.uber.org/zap"
@@ -69,6 +73,9 @@ type Lightning struct {
6973
serverLock sync.Mutex
7074
status restore.LightningStatus
7175

76+
promFactory promutil.Factory
77+
promRegistry promutil.Registry
78+
7279
cancelLock sync.Mutex
7380
curTask *config.Config
7481
cancel context.CancelFunc // for per task context, which maybe different from lightning context
@@ -94,12 +101,16 @@ func New(globalCfg *config.GlobalConfig) *Lightning {
94101

95102
redact.InitRedact(globalCfg.Security.RedactInfoLog)
96103

104+
promFactory := promutil.NewDefaultFactory()
105+
promRegistry := promutil.NewDefaultRegistry()
97106
ctx, shutdown := context.WithCancel(context.Background())
98107
return &Lightning{
99-
globalCfg: globalCfg,
100-
globalTLS: tls,
101-
ctx: ctx,
102-
shutdown: shutdown,
108+
globalCfg: globalCfg,
109+
globalTLS: tls,
110+
ctx: ctx,
111+
shutdown: shutdown,
112+
promFactory: promFactory,
113+
promRegistry: promRegistry,
103114
}
104115
}
105116

@@ -181,7 +192,16 @@ func httpHandleWrapper(h http.HandlerFunc) http.HandlerFunc {
181192
func (l *Lightning) goServe(statusAddr string, realAddrWriter io.Writer) error {
182193
mux := http.NewServeMux()
183194
mux.Handle("/", http.RedirectHandler("/web/", http.StatusFound))
184-
mux.Handle("/metrics", promhttp.Handler())
195+
196+
registry := l.promRegistry
197+
registry.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{}))
198+
registry.MustRegister(collectors.NewGoCollector())
199+
if gatherer, ok := registry.(prometheus.Gatherer); ok {
200+
handler := promhttp.InstrumentMetricHandler(
201+
registry, promhttp.HandlerFor(gatherer, promhttp.HandlerOpts{}),
202+
)
203+
mux.Handle("/metrics", handler)
204+
}
185205

186206
mux.HandleFunc("/debug/pprof/", pprof.Index)
187207
mux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline)
@@ -242,8 +262,12 @@ func (l *Lightning) RunOnce(taskCtx context.Context, taskCfg *config.Config, glu
242262
failpoint.Inject("SetTaskID", func(val failpoint.Value) {
243263
taskCfg.TaskID = int64(val.(int))
244264
})
245-
246-
return l.run(taskCtx, taskCfg, &options{glue: glue})
265+
o := &options{
266+
glue: glue,
267+
promFactory: l.promFactory,
268+
promRegistry: l.promRegistry,
269+
}
270+
return l.run(taskCtx, taskCfg, o)
247271
}
248272

249273
func (l *Lightning) RunServer() error {
@@ -260,7 +284,10 @@ func (l *Lightning) RunServer() error {
260284
if err != nil {
261285
return err
262286
}
263-
o := &options{}
287+
o := &options{
288+
promFactory: l.promFactory,
289+
promRegistry: l.promRegistry,
290+
}
264291
err = l.run(context.Background(), task, o)
265292
if err != nil && !common.IsContextCanceledError(err) {
266293
restore.DeliverPauser.Pause() // force pause the progress on error
@@ -280,7 +307,10 @@ func (l *Lightning) RunServer() error {
280307
// - WithCheckpointStorage: caller has opened an external storage for lightning and want to save checkpoint
281308
// in it. Otherwise, lightning will save checkpoint by the Checkpoint.DSN in config
282309
func (l *Lightning) RunOnceWithOptions(taskCtx context.Context, taskCfg *config.Config, opts ...Option) error {
283-
o := &options{}
310+
o := &options{
311+
promFactory: l.promFactory,
312+
promRegistry: l.promRegistry,
313+
}
284314
for _, opt := range opts {
285315
opt(o)
286316
}
@@ -331,7 +361,14 @@ func (l *Lightning) run(taskCtx context.Context, taskCfg *config.Config, o *opti
331361

332362
utils.LogEnvVariables()
333363

334-
ctx, cancel := context.WithCancel(taskCtx)
364+
metrics := metric.NewMetrics(o.promFactory)
365+
metrics.RegisterTo(o.promRegistry)
366+
defer func() {
367+
metrics.UnregisterFrom(o.promRegistry)
368+
}()
369+
370+
ctx := metric.NewContext(taskCtx, metrics)
371+
ctx, cancel := context.WithCancel(ctx)
335372
l.cancelLock.Lock()
336373
l.cancel = cancel
337374
l.curTask = taskCfg

br/pkg/lightning/lightning_serial_test.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,11 @@ func TestRun(t *testing.T) {
6161
path, _ := filepath.Abs(".")
6262
ctx := context.Background()
6363
invalidGlue := glue.NewExternalTiDBGlue(nil, 0)
64-
o := &options{glue: invalidGlue}
64+
o := &options{
65+
glue: invalidGlue,
66+
promRegistry: lightning.promRegistry,
67+
promFactory: lightning.promFactory,
68+
}
6569
err = lightning.run(ctx, &config.Config{
6670
Mydumper: config.MydumperRuntime{
6771
SourceDir: "file://" + filepath.ToSlash(path),

br/pkg/lightning/metric/BUILD.bazel

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ go_library(
66
importpath = "github.com/pingcap/tidb/br/pkg/lightning/metric",
77
visibility = ["//visibility:public"],
88
deps = [
9+
"//util/promutil",
910
"@com_github_prometheus_client_golang//prometheus",
1011
"@com_github_prometheus_client_model//go",
1112
],
@@ -16,7 +17,9 @@ go_test(
1617
srcs = ["metric_test.go"],
1718
deps = [
1819
":metric",
20+
"//util/promutil",
1921
"@com_github_prometheus_client_golang//prometheus",
22+
"@com_github_stretchr_testify//assert",
2023
"@com_github_stretchr_testify//require",
2124
],
2225
)

0 commit comments

Comments
 (0)