diff --git a/.github/workflows/dataflow_engine_chaos.yaml b/.github/workflows/dataflow_engine_chaos.yaml index 999a1968135..3db336064d6 100644 --- a/.github/workflows/dataflow_engine_chaos.yaml +++ b/.github/workflows/dataflow_engine_chaos.yaml @@ -339,15 +339,3 @@ jobs: name: chaos-base-logs.${{ matrix.chaos-obj }} path: | ./logs - - # Send feishu notification if failed. - - name: Feishu notification - continue-on-error: true - uses: foxundermoon/feishu-action@v2 - if: ${{ failure() }} - with: - url: ${{ secrets.ENGINE_FEISHU_NOTIFY_URL }} - msg_type: text - content: | - text: | - dataflow engine chaos job failed, see https://github.com/pingcap/tiflow/actions/runs/${{ github.run_id }} diff --git a/.github/workflows/dm_binlog_999999.yaml b/.github/workflows/dm_binlog_999999.yaml index 51115b2244e..4e4da15f755 100644 --- a/.github/workflows/dm_binlog_999999.yaml +++ b/.github/workflows/dm_binlog_999999.yaml @@ -14,6 +14,11 @@ jobs: test-binlog-999999: name: Test binlog 999999 runs-on: ubuntu-20.04 + services: + docker: + image: docker:19.03.12 + options: >- + --privileged steps: - name: Set up Go env @@ -30,6 +35,14 @@ jobs: with: ref: refs/pull/${{ github.event.inputs.pr }}/head + - name: Set DOCKER_HOST + run: echo "export DOCKER_HOST=unix:///var/run/docker.sock" >> $GITHUB_ENV + + - name: Install docker-compose + run: | + sudo curl -L "https://github.com/docker/compose/releases/download/1.29.2/docker-compose-$(uname -s)-$(uname -m)" -o /usr/local/bin/docker-compose + sudo chmod +x /usr/local/bin/docker-compose + - name: Cache go modules uses: actions/cache@v2 with: @@ -48,7 +61,9 @@ jobs: - name: Setup CI environment run: | - docker-compose -f ./dm/tests/binlog_999999/docker-compose.yml up -d + sudo apt-get update + sudo apt-get install -y curl + sudo docker-compose -f ./dm/tests/binlog_999999/docker-compose.yml up -d curl http://download.pingcap.org/tidb-enterprise-tools-nightly-linux-amd64.tar.gz | tar xz mv tidb-enterprise-tools-nightly-linux-amd64/bin/sync_diff_inspector bin/ curl http://download.pingcap.org/tidb-nightly-linux-amd64.tar.gz | tar xz @@ -57,7 +72,7 @@ jobs: chmod +x minio mv minio bin/ - - name: change binlog sequence number to 999998 + - name: Change binlog sequence number to 999999 run: | while ! mysqladmin -h127.0.0.1 -P3306 -p123456 -uroot ping --connect-timeout=1 > /dev/null 2>&1 ; do echo "wait mysql" @@ -87,11 +102,16 @@ jobs: sleep 1 done + echo "show binary logs;" | mysql -uroot -h127.0.0.1 -P3306 -p123456 + echo "show binary logs;" | mysql -uroot -h127.0.0.1 -P3307 -p123456 + - name: Run test cases run: | RESET_MASTER=false make dm_integration_test CASE=incremental_mode - echo "show binary logs;" | mysql -uroot -h127.0.0.1 -P3306 -p123456 | grep -q "mysql-bin.1000000" - echo "show binary logs;" | mysql -uroot -h127.0.0.1 -P3307 -p123456 | grep -q "mysql-bin.1000000" + echo "show binary logs;" | mysql -uroot -h127.0.0.1 -P3306 -p123456 + echo "show binary logs;" | mysql -uroot -h127.0.0.1 -P3307 -p123456 + echo "show binary logs;" | mysql -uroot -h127.0.0.1 -P3306 -p123456 | grep -q "mysql-bin.1000003" + echo "show binary logs;" | mysql -uroot -h127.0.0.1 -P3307 -p123456 | grep -q "mysql-bin.1000002" - name: Copy logs to hack permission if: ${{ always() }} @@ -112,12 +132,14 @@ jobs: path: | ./logs - # send Slack notify if failed. - # NOTE: With the exception of `GITHUB_TOKEN`, secrets are not passed to the runner when a workflow is triggered from a forked repository. - - name: Slack notification + # Send feishu notification if failed. + - name: Feishu notification + continue-on-error: true + uses: foxundermoon/feishu-action@v2 if: ${{ failure() }} - env: - SLACK_WEBHOOK: ${{ secrets.SLACK_NOTIFY }} - uses: Ilshidur/action-slack@2.1.0 with: - args: "binlog 999999 failed, see https://github.com/pingcap/tiflow/actions/runs/{{ GITHUB_RUN_ID }}" + url: ${{ secrets.ENGINE_FEISHU_NOTIFY_URL }} + msg_type: text + content: | + text: | + binlog 999999 job failed, see https://github.com/pingcap/tiflow/actions/runs/${{ github.run_id }} diff --git a/.github/workflows/dm_upstream_switch.yaml b/.github/workflows/dm_upstream_switch.yaml index 718a525113c..dafc866124c 100644 --- a/.github/workflows/dm_upstream_switch.yaml +++ b/.github/workflows/dm_upstream_switch.yaml @@ -48,6 +48,8 @@ jobs: - name: Setup containers run: | + sudo curl -L "https://github.com/docker/compose/releases/download/1.29.2/docker-compose-$(uname -s)-$(uname -m)" -o /usr/local/bin/docker-compose + sudo chmod +x /usr/local/bin/docker-compose docker-compose -f ./dm/tests/upstream_switch/docker-compose.yml up -d - name: Run test cases @@ -73,12 +75,15 @@ jobs: path: | ./logs - # send Slack notify if failed. - # NOTE: With the exception of `GITHUB_TOKEN`, secrets are not passed to the runner when a workflow is triggered from a forked repository. - - name: Slack notification + # Send feishu notification if failed. + - name: Feishu notification + continue-on-error: true + uses: foxundermoon/feishu-action@v2 if: ${{ failure() }} - env: - SLACK_WEBHOOK: ${{ secrets.SLACK_NOTIFY }} - uses: Ilshidur/action-slack@2.1.0 with: - args: "upstream-switch job failed, see https://github.com/pingcap/tiflow/actions/runs/{{ GITHUB_RUN_ID }}" + url: ${{ secrets.ENGINE_FEISHU_NOTIFY_URL }} + msg_type: text + content: | + text: | + dm upstream switch job failed, see https://github.com/pingcap/tiflow/actions/runs/${{ github.run_id }} + diff --git a/cdc/api/v2/api.go b/cdc/api/v2/api.go index 9d70f3ca37f..033fe177bd5 100644 --- a/cdc/api/v2/api.go +++ b/cdc/api/v2/api.go @@ -86,7 +86,7 @@ func RegisterOpenAPIV2Routes(router *gin.Engine, api OpenAPIV2) { // owner apis ownerGroup := v2.Group("/owner") - unsafeGroup.Use(ownerMiddleware) + ownerGroup.Use(ownerMiddleware) ownerGroup.POST("/resign", api.resignOwner) // common APIs diff --git a/cdc/api/v2/api_helpers.go b/cdc/api/v2/api_helpers.go index 05b93b6e643..d99bb9dc1bb 100644 --- a/cdc/api/v2/api_helpers.go +++ b/cdc/api/v2/api_helpers.go @@ -231,9 +231,6 @@ func (APIV2HelpersImpl) verifyCreateChangefeedConfig( return nil, errors.Cause(err) } if !replicaCfg.ForceReplicate && !cfg.ReplicaConfig.IgnoreIneligibleTable { - if err != nil { - return nil, err - } if len(ineligibleTables) != 0 { return nil, cerror.ErrTableIneligible.GenWithStackByArgs(ineligibleTables) } diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index d7ed71229a5..e255c890cda 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -38,7 +38,6 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" pfilter "github.com/pingcap/tiflow/pkg/filter" "github.com/pingcap/tiflow/pkg/integrity" - "github.com/pingcap/tiflow/pkg/spanz" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -65,19 +64,6 @@ type rowKVEntry struct { PreRowExist bool } -// DDLTableInfo contains the tableInfo about tidb_ddl_job and tidb_ddl_history -// and the column id of `job_meta` in these two tables. -type DDLTableInfo struct { - // ddlJobsTable use to parse all ddl jobs except `create table` - DDLJobTable *model.TableInfo - // It holds the column id of `job_meta` in table `tidb_ddl_jobs`. - JobMetaColumnIDinJobTable int64 - // ddlHistoryTable only use to parse `create table` ddl job - DDLHistoryTable *model.TableInfo - // It holds the column id of `job_meta` in table `tidb_ddl_history`. - JobMetaColumnIDinHistoryTable int64 -} - // Mounter is used to parse SQL events from KV events type Mounter interface { // DecodeEvent accepts `model.PolymorphicEvent` with `RawKVEntry` filled and @@ -306,89 +292,39 @@ func IsLegacyFormatJob(rawKV *model.RawKVEntry) bool { return bytes.HasPrefix(rawKV.Key, metaPrefix) } -// ParseDDLJob parses the job from the raw KV entry. -func ParseDDLJob(rawKV *model.RawKVEntry, ddlTableInfo *DDLTableInfo) (*timodel.Job, error) { +// ParseDDLJob parses the job from the raw KV entry. id is the column id of `job_meta`. +func ParseDDLJob(tblInfo *model.TableInfo, rawKV *model.RawKVEntry, id int64) (*timodel.Job, error) { var v []byte - var datum types.Datum - - // for test case only if bytes.HasPrefix(rawKV.Key, metaPrefix) { + // old queue base job. v = rawKV.Value - job, err := parseJob(v, rawKV.StartTs, rawKV.CRTs, false) - if err != nil || job == nil { - job, err = parseJob(v, rawKV.StartTs, rawKV.CRTs, true) - } - return job, err - } - - recordID, err := tablecodec.DecodeRowKey(rawKV.Key) - if err != nil { - return nil, errors.Trace(err) - } - - tableID := tablecodec.DecodeTableID(rawKV.Key) - - // parse it with tidb_ddl_job - if tableID == spanz.JobTableID { - row, err := decodeRow(rawKV.Value, recordID, ddlTableInfo.DDLJobTable, time.UTC) + } else { + // DDL job comes from `tidb_ddl_job` table after we support concurrent DDL. We should decode the job from the column. + recordID, err := tablecodec.DecodeRowKey(rawKV.Key) if err != nil { return nil, errors.Trace(err) } - datum = row[ddlTableInfo.JobMetaColumnIDinJobTable] - v = datum.GetBytes() - - return parseJob(v, rawKV.StartTs, rawKV.CRTs, false) - } else if tableID == spanz.JobHistoryID { - // parse it with tidb_ddl_history - row, err := decodeRow(rawKV.Value, recordID, ddlTableInfo.DDLHistoryTable, time.UTC) + row, err := decodeRow(rawKV.Value, recordID, tblInfo, time.UTC) if err != nil { return nil, errors.Trace(err) } - datum = row[ddlTableInfo.JobMetaColumnIDinHistoryTable] + datum := row[id] v = datum.GetBytes() - - return parseJob(v, rawKV.StartTs, rawKV.CRTs, true) } - return nil, fmt.Errorf("Unvalid tableID %v in rawKV.Key", tableID) + return parseJob(v, rawKV.StartTs, rawKV.CRTs) } // parseJob unmarshal the job from "v". -// fromHistoryTable is used to distinguish the job is from tidb_dd_job or tidb_ddl_history -// We need to be compatible with the two modes, enable_fast_create_table=on and enable_fast_create_table=off -// When enable_fast_create_table=on, `create table` will only be inserted into tidb_ddl_history after being executed successfully. -// When enable_fast_create_table=off, `create table` just like other ddls will be firstly inserted to tidb_ddl_job, -// and being inserted into tidb_ddl_history after being executed successfully. -// In both two modes, other ddls are all firstly inserted into tidb_ddl_job, and then inserted into tidb_ddl_history after being executed successfully. -// -// To be compatible with these two modes, we will get `create table` ddl from tidb_ddl_history, and all ddls from tidb_ddl_job. -// When enable_fast_create_table=off, for each `create table` ddl we will get twice(once from tidb_ddl_history, once from tidb_ddl_job) -// Because in `handleJob` we will skip the repeated ddls, thus it's ok for us to get `create table` twice. -// Besides, the `create table` from tidb_ddl_job always have a earlier commitTs than from tidb_ddl_history. -// Therefore, we always use the commitTs of ddl from `tidb_ddl_job` as StartTs, which ensures we can get all the dmls. -func parseJob(v []byte, startTs, CRTs uint64, fromHistoryTable bool) (*timodel.Job, error) { +func parseJob(v []byte, startTs, CRTs uint64) (*timodel.Job, error) { var job timodel.Job err := json.Unmarshal(v, &job) if err != nil { return nil, errors.Trace(err) } - - if fromHistoryTable { - // we only want to get `create table` and `create tables` ddl from tidb_ddl_history, so we just throw out others ddls. - // We only want the job with `JobStateSynced`, which is means the ddl job is done successfully. - // Besides, to satisfy the subsequent processing, - // We need to set the job to be Done to make it will replay in schemaStorage - if (job.Type != timodel.ActionCreateTable && job.Type != timodel.ActionCreateTables) || job.State != timodel.JobStateSynced { - return nil, nil - } - job.State = timodel.JobStateDone - } else { - // we need to get all ddl job which is done from tidb_ddl_job - if !job.IsDone() { - return nil, nil - } + if !job.IsDone() { + return nil, nil } - // FinishedTS is only set when the job is synced, // but we can use the entry's ts here job.StartTS = startTs diff --git a/cdc/entry/schema/snapshot.go b/cdc/entry/schema/snapshot.go index b188ffcf5a7..a63608b2770 100644 --- a/cdc/entry/schema/snapshot.go +++ b/cdc/entry/schema/snapshot.go @@ -109,24 +109,6 @@ func (s *Snapshot) FillSchemaName(job *timodel.Job) error { return nil } -// GetSchemaVersion returns the schema version of the meta. -func GetSchemaVersion(meta timeta.Reader) (int64, error) { - // After we get the schema version at startTs, if the diff corresponding to that version does not exist, - // it means that the job is not committed yet, so we should subtract one from the version, i.e., version--. - version, err := meta.GetSchemaVersion() - if err != nil { - return 0, errors.Trace(err) - } - diff, err := meta.GetSchemaDiff(version) - if err != nil { - return 0, errors.Trace(err) - } - if diff == nil { - version-- - } - return version, nil -} - // NewSnapshotFromMeta creates a schema snapshot from meta. func NewSnapshotFromMeta( id model.ChangeFeedID, diff --git a/cdc/entry/schema_storage.go b/cdc/entry/schema_storage.go index 999c8734b5a..784a1affe00 100644 --- a/cdc/entry/schema_storage.go +++ b/cdc/entry/schema_storage.go @@ -16,7 +16,6 @@ package entry import ( "context" "sort" - "strings" "sync" "sync/atomic" "time" @@ -28,6 +27,7 @@ import ( "github.com/pingcap/tiflow/cdc/entry/schema" "github.com/pingcap/tiflow/cdc/kv" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/ddl" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" "github.com/pingcap/tiflow/pkg/retry" @@ -73,9 +73,8 @@ type schemaStorage struct { snaps []*schema.Snapshot snapsMu sync.RWMutex - gcTs uint64 - resolvedTs uint64 - schemaVersion int64 + gcTs uint64 + resolvedTs uint64 filter filter.Filter @@ -92,9 +91,8 @@ func NewSchemaStorage( role util.Role, filter filter.Filter, ) (SchemaStorage, error) { var ( - snap *schema.Snapshot - version int64 - err error + snap *schema.Snapshot + err error ) // storage may be nil in some unit test cases. if storage == nil { @@ -105,7 +103,6 @@ func NewSchemaStorage( if err != nil { return nil, errors.Trace(err) } - version, err = schema.GetSchemaVersion(meta) if err != nil { return nil, errors.Trace(err) } @@ -116,7 +113,6 @@ func NewSchemaStorage( forceReplicate: forceReplicate, filter: filter, id: id, - schemaVersion: version, role: role, }, nil } @@ -194,7 +190,6 @@ func (s *schemaStorage) GetLastSnapshot() *schema.Snapshot { // HandleDDLJob creates a new snapshot in storage and handles the ddl job func (s *schemaStorage) HandleDDLJob(job *timodel.Job) error { if s.skipJob(job) { - s.schemaVersion = job.BinlogInfo.SchemaVersion s.AdvanceResolvedTs(job.BinlogInfo.FinishedTS) return nil } @@ -203,16 +198,15 @@ func (s *schemaStorage) HandleDDLJob(job *timodel.Job) error { var snap *schema.Snapshot if len(s.snaps) > 0 { lastSnap := s.snaps[len(s.snaps)-1] - // We use schemaVersion to check if an already-executed DDL job is processed for a second time. - // Unexecuted DDL jobs should have largest schemaVersions. - if job.BinlogInfo.FinishedTS <= lastSnap.CurrentTs() || job.BinlogInfo.SchemaVersion <= s.schemaVersion { + // already-executed DDL could filted by finishedTs. + if job.BinlogInfo.FinishedTS <= lastSnap.CurrentTs() { log.Info("schemaStorage: ignore foregone DDL", zap.String("namespace", s.id.Namespace), zap.String("changefeed", s.id.ID), zap.String("DDL", job.Query), + zap.String("state", job.State.String()), zap.Int64("jobID", job.ID), zap.Uint64("finishTs", job.BinlogInfo.FinishedTS), - zap.Int64("schemaVersion", s.schemaVersion), zap.Int64("jobSchemaVersion", job.BinlogInfo.SchemaVersion), zap.String("role", s.role.String())) return nil @@ -234,7 +228,6 @@ func (s *schemaStorage) HandleDDLJob(job *timodel.Job) error { return errors.Trace(err) } s.snaps = append(s.snaps, snap) - s.schemaVersion = job.BinlogInfo.SchemaVersion s.AdvanceResolvedTs(job.BinlogInfo.FinishedTS) log.Info("schemaStorage: update snapshot by the DDL job", zap.String("namespace", s.id.Namespace), @@ -243,7 +236,6 @@ func (s *schemaStorage) HandleDDLJob(job *timodel.Job) error { zap.String("table", job.TableName), zap.String("query", job.Query), zap.Uint64("finishedTs", job.BinlogInfo.FinishedTS), - zap.Uint64("schemaVersion", uint64(s.schemaVersion)), zap.String("role", s.role.String())) return nil } @@ -273,8 +265,7 @@ func (s *schemaStorage) AllPhysicalTables(ctx context.Context, ts model.Ts) ([]m log.Debug("get new schema snapshot", zap.Uint64("ts", ts), zap.Uint64("snapTs", snap.CurrentTs()), - zap.Any("tables", res), - zap.String("snapshot", snap.DumpToString())) + zap.Any("tables", res)) return res, nil } @@ -410,7 +401,10 @@ func (s *schemaStorage) BuildDDLEvents( } case timodel.ActionCreateTables: if job.BinlogInfo != nil && job.BinlogInfo.MultipleTableInfos != nil { - querys := strings.Split(job.Query, ";") + querys, err := ddl.SplitQueries(job.Query) + if err != nil { + return nil, errors.Trace(err) + } multiTableInfos := job.BinlogInfo.MultipleTableInfos for index, tableInfo := range multiTableInfos { newTableInfo := model.WrapTableInfo(job.SchemaID, job.SchemaName, job.BinlogInfo.FinishedTS, tableInfo) diff --git a/cdc/entry/schema_test_helper.go b/cdc/entry/schema_test_helper.go index b2bf6465bb5..c4dafe45f06 100644 --- a/cdc/entry/schema_test_helper.go +++ b/cdc/entry/schema_test_helper.go @@ -166,13 +166,23 @@ func (s *SchemaTestHelper) DDL2Jobs(ddl string, jobCnt int) []*timodel.Job { // DML2Event execute the dml and return the corresponding row changed event. // caution: it does not support `delete` since the key value cannot be found // after the query executed. -func (s *SchemaTestHelper) DML2Event(dml string, schema, table string) *model.RowChangedEvent { +func (s *SchemaTestHelper) DML2Event(dml string, schema, table string, partitionID ...string) *model.RowChangedEvent { s.tk.MustExec(dml) tableInfo, ok := s.schemaStorage.GetLastSnapshot().TableByName(schema, table) require.True(s.t, ok) - key, value := s.getLastKeyValue(tableInfo.ID) + tableID := tableInfo.ID + + var partitionTableID int64 = -1 + if len(partitionID) == 1 { + partitionTableID = tableInfo.TableInfo.GetPartitionInfo().GetPartitionIDByName(partitionID[0]) + } + if partitionTableID != -1 { + tableID = partitionTableID + } + + key, value := s.getLastKeyValue(tableID) ts := s.schemaStorage.GetLastSnapshot().CurrentTs() rawKV := &model.RawKVEntry{ OpType: model.OpTypePut, diff --git a/cdc/model/sink.go b/cdc/model/sink.go index 2dd5634c89c..cc33d2b1008 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -714,13 +714,6 @@ type NameBasedColumnIDAllocator struct { nameToIDMap map[string]int64 } -// NewNameBasedColumnIDAllocator creates a new NameBasedColumnIDAllocator -func NewNameBasedColumnIDAllocator(nameToIDMap map[string]int64) *NameBasedColumnIDAllocator { - return &NameBasedColumnIDAllocator{ - nameToIDMap: nameToIDMap, - } -} - // GetColumnID return the column id of the name func (n *NameBasedColumnIDAllocator) GetColumnID(name string) int64 { colID, ok := n.nameToIDMap[name] diff --git a/cdc/owner/ddl_manager.go b/cdc/owner/ddl_manager.go index e709d086213..6ea996e3739 100644 --- a/cdc/owner/ddl_manager.go +++ b/cdc/owner/ddl_manager.go @@ -293,6 +293,8 @@ func (m *ddlManager) tick( continue } + // Note: do not change the key words in the log, it is used to search the + // FinishTS of the DDL job. Some integration tests and users depend on it. log.Info("handle a ddl job", zap.String("namespace", m.changfeedID.Namespace), zap.String("changefeed", m.changfeedID.ID), @@ -307,6 +309,18 @@ func (m *ddlManager) tick( } for _, event := range events { + snap := m.schema.GetLastSnapshot() + if event.Type == timodel.ActionCreateTable || + event.Type == timodel.ActionCreateTables { + if snap.IsIneligibleTableID(event.TableInfo.ID) { + log.Info("table is ineligible, skip the ddl", + zap.String("namespace", m.changfeedID.Namespace), + zap.String("changefeed", m.changfeedID.ID), + zap.String("query", job.Query), + zap.Any("table", event.TableInfo)) + continue + } + } tableName := event.TableInfo.TableName m.pendingDDLs[tableName] = append(m.pendingDDLs[tableName], event) } diff --git a/cdc/processor/tablepb/table.pb.go b/cdc/processor/tablepb/table.pb.go index c4b86e83406..d27572c206b 100644 --- a/cdc/processor/tablepb/table.pb.go +++ b/cdc/processor/tablepb/table.pb.go @@ -179,7 +179,7 @@ type Stats struct { // Number of captured regions. RegionCount uint64 `protobuf:"varint,1,opt,name=region_count,json=regionCount,proto3" json:"region_count,omitempty"` // The current timestamp from the table's point of view. - CurrentTs Ts `protobuf:"varint,2,opt,name=current_ts,json=currentTs,proto3,casttype=Ts" json:"current_ts,omitempty"` + CurrentTs Ts `protobuf:"varint,2,opt,name=current_ts,json=currentTs,proto3,casttype=Ts" json:"current_ts,omitempty"` // Deprecated: Do not use. // Checkponits at each stage. StageCheckpoints map[string]Checkpoint `protobuf:"bytes,3,rep,name=stage_checkpoints,json=stageCheckpoints,proto3" json:"stage_checkpoints" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // The barrier timestamp of the table. @@ -226,6 +226,7 @@ func (m *Stats) GetRegionCount() uint64 { return 0 } +// Deprecated: Do not use. func (m *Stats) GetCurrentTs() Ts { if m != nil { return m.CurrentTs @@ -337,52 +338,52 @@ func init() { func init() { proto.RegisterFile("processor/tablepb/table.proto", fileDescriptor_ae83c9c6cf5ef75c) } var fileDescriptor_ae83c9c6cf5ef75c = []byte{ - // 706 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0xbf, 0x6f, 0xd3, 0x4e, - 0x1c, 0xb5, 0x9d, 0x5f, 0xcd, 0x27, 0xf9, 0x56, 0xee, 0x7d, 0xdb, 0x52, 0x22, 0x91, 0x98, 0xa8, - 0x40, 0xd5, 0x22, 0x07, 0xca, 0x82, 0xba, 0x35, 0x2d, 0xa0, 0xaa, 0x42, 0x42, 0x4e, 0x60, 0x60, - 0x89, 0x1c, 0xfb, 0x70, 0xad, 0x86, 0xb3, 0xe5, 0xbb, 0xb4, 0xca, 0xc6, 0x88, 0xb2, 0xd0, 0x09, - 0xb1, 0x44, 0xea, 0x9f, 0xd3, 0xb1, 0x23, 0x03, 0x8a, 0x20, 0x15, 0x33, 0x7b, 0x27, 0x74, 0x77, - 0x6e, 0xdc, 0x04, 0x86, 0xd0, 0x25, 0x39, 0xdf, 0x7b, 0x9f, 0xa7, 0xf7, 0xde, 0x9d, 0x0e, 0xee, - 0x84, 0x51, 0xe0, 0x60, 0x4a, 0x83, 0xa8, 0xc6, 0xec, 0x76, 0x07, 0x87, 0x6d, 0xf9, 0x6f, 0x86, - 0x51, 0xc0, 0x02, 0xb4, 0x1a, 0xfa, 0xc4, 0x73, 0xec, 0xd0, 0x64, 0xfe, 0xbb, 0x4e, 0x70, 0x6c, - 0x3a, 0xae, 0x63, 0x8e, 0x27, 0xcc, 0x78, 0xa2, 0xb4, 0xe8, 0x05, 0x5e, 0x20, 0x06, 0x6a, 0x7c, - 0x25, 0x67, 0xab, 0x9f, 0x54, 0x48, 0x37, 0x42, 0x9b, 0xa0, 0xc7, 0x30, 0x27, 0x98, 0x2d, 0xdf, - 0x5d, 0x51, 0x0d, 0x75, 0x2d, 0x55, 0x5f, 0x1e, 0x0d, 0x2b, 0xb9, 0x26, 0xdf, 0xdb, 0xdb, 0xbd, - 0x4c, 0x96, 0x56, 0x4e, 0xf0, 0xf6, 0x5c, 0xb4, 0x0a, 0x79, 0xca, 0xec, 0x88, 0xb5, 0x0e, 0x71, - 0x6f, 0x45, 0x33, 0xd4, 0xb5, 0x62, 0x3d, 0x77, 0x39, 0xac, 0xa4, 0xf6, 0x71, 0xcf, 0x9a, 0x13, - 0xc8, 0x3e, 0xee, 0x21, 0x03, 0x72, 0x98, 0xb8, 0x82, 0x93, 0x9a, 0xe4, 0x64, 0x31, 0x71, 0xf7, - 0x71, 0x6f, 0xab, 0xf8, 0xf1, 0xb4, 0xa2, 0x7c, 0x39, 0xad, 0x28, 0x1f, 0xbe, 0x19, 0x4a, 0xf5, - 0x44, 0x05, 0xd8, 0x39, 0xc0, 0xce, 0x61, 0x18, 0xf8, 0x84, 0xa1, 0x0d, 0xf8, 0xcf, 0x19, 0x7f, - 0xb5, 0x18, 0x15, 0xe6, 0xd2, 0xf5, 0xec, 0xe5, 0xb0, 0xa2, 0x35, 0xa9, 0x55, 0x4c, 0xc0, 0x26, - 0x45, 0x0f, 0xa0, 0x10, 0x61, 0x1a, 0x74, 0x8e, 0xb0, 0xcb, 0xa9, 0xda, 0x04, 0x15, 0xae, 0xa0, - 0x26, 0x45, 0x0f, 0x61, 0xbe, 0x63, 0x53, 0xd6, 0xa2, 0x3d, 0xe2, 0x48, 0x6e, 0x6a, 0x52, 0x96, - 0xa3, 0x0d, 0x01, 0x36, 0x69, 0xf5, 0xa7, 0x06, 0x99, 0x06, 0xb3, 0x19, 0x45, 0x77, 0xa1, 0x18, - 0x61, 0xcf, 0x0f, 0x48, 0xcb, 0x09, 0xba, 0x84, 0x49, 0x33, 0x56, 0x41, 0xee, 0xed, 0xf0, 0x2d, - 0x74, 0x0f, 0xc0, 0xe9, 0x46, 0x11, 0x96, 0x6e, 0x27, 0x2d, 0xe4, 0x63, 0xa4, 0x49, 0x11, 0x83, - 0x05, 0xca, 0x6c, 0x0f, 0xb7, 0x92, 0x00, 0xdc, 0x44, 0x6a, 0xad, 0xb0, 0xb9, 0x6d, 0xce, 0x72, - 0xa0, 0xa6, 0x70, 0xc4, 0x7f, 0x3d, 0x9c, 0xf4, 0x45, 0x9f, 0x11, 0x16, 0xf5, 0xea, 0xe9, 0xb3, - 0x61, 0x45, 0xb1, 0x74, 0x3a, 0x05, 0x72, 0x73, 0x6d, 0x3b, 0x8a, 0x7c, 0x1c, 0x71, 0x73, 0xe9, - 0x49, 0x73, 0x31, 0xd2, 0xa4, 0xa5, 0x2e, 0x2c, 0xfd, 0x55, 0x17, 0xe9, 0x90, 0xe2, 0x07, 0xc9, - 0x63, 0xe7, 0x2d, 0xbe, 0x44, 0xcf, 0x21, 0x73, 0x64, 0x77, 0xba, 0x58, 0x24, 0x2d, 0x6c, 0x3e, - 0x9a, 0xcd, 0x7b, 0x22, 0x6c, 0xc9, 0xf1, 0x2d, 0xed, 0xa9, 0x5a, 0xfd, 0xa5, 0x41, 0x41, 0xdc, - 0x32, 0x1e, 0xad, 0x4b, 0x6f, 0x72, 0x27, 0x77, 0x21, 0x4d, 0x43, 0x9b, 0xac, 0x64, 0x84, 0x9b, - 0xf5, 0x19, 0x9b, 0x0c, 0x6d, 0x12, 0x57, 0x26, 0xa6, 0x79, 0x28, 0xca, 0x6c, 0x26, 0x43, 0xcd, - 0xcf, 0x1a, 0x6a, 0x6c, 0x1d, 0x5b, 0x72, 0x1c, 0xbd, 0x01, 0x48, 0x8e, 0x57, 0x5c, 0xb1, 0x1b, - 0x34, 0x14, 0x3b, 0xbb, 0xa6, 0x84, 0x5e, 0x48, 0x7f, 0xf2, 0x04, 0x0b, 0x9b, 0x1b, 0xff, 0x70, - 0x61, 0x62, 0x35, 0x39, 0xbf, 0xfe, 0x59, 0x03, 0x48, 0x6c, 0xa3, 0x2a, 0xe4, 0x5e, 0x93, 0x43, - 0x12, 0x1c, 0x13, 0x5d, 0x29, 0x2d, 0xf5, 0x07, 0xc6, 0x42, 0x02, 0xc6, 0x00, 0x32, 0x20, 0xbb, - 0xdd, 0xa6, 0x98, 0x30, 0x5d, 0x2d, 0x2d, 0xf6, 0x07, 0x86, 0x9e, 0x50, 0xe4, 0x3e, 0xba, 0x0f, - 0xf9, 0x57, 0x11, 0x0e, 0xed, 0xc8, 0x27, 0x9e, 0xae, 0x95, 0x6e, 0xf5, 0x07, 0xc6, 0xff, 0x09, - 0x69, 0x0c, 0xa1, 0x55, 0x98, 0x93, 0x1f, 0xd8, 0xd5, 0x53, 0xa5, 0xe5, 0xfe, 0xc0, 0x40, 0xd3, - 0x34, 0xec, 0xa2, 0x75, 0x28, 0x58, 0x38, 0xec, 0xf8, 0x8e, 0xcd, 0xb8, 0x5e, 0xba, 0x74, 0xbb, - 0x3f, 0x30, 0x96, 0xae, 0x75, 0x9d, 0x80, 0x5c, 0xb1, 0xc1, 0x82, 0x90, 0xb7, 0xa1, 0x67, 0xa6, - 0x15, 0xaf, 0x10, 0x9e, 0x52, 0xac, 0xb1, 0xab, 0x67, 0xa7, 0x53, 0xc6, 0x40, 0xfd, 0xe5, 0xf9, - 0x8f, 0xb2, 0x72, 0x36, 0x2a, 0xab, 0xe7, 0xa3, 0xb2, 0xfa, 0x7d, 0x54, 0x56, 0x4f, 0x2e, 0xca, - 0xca, 0xf9, 0x45, 0x59, 0xf9, 0x7a, 0x51, 0x56, 0xde, 0xd6, 0x3c, 0x9f, 0x1d, 0x74, 0xdb, 0xa6, - 0x13, 0xbc, 0xaf, 0xc5, 0xd5, 0xd7, 0x64, 0xf5, 0x35, 0xc7, 0x75, 0x6a, 0x7f, 0x3c, 0xd7, 0xed, - 0xac, 0x78, 0x6d, 0x9f, 0xfc, 0x0e, 0x00, 0x00, 0xff, 0xff, 0xa3, 0xeb, 0x06, 0x9b, 0xca, 0x05, - 0x00, 0x00, + // 716 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0xbf, 0x6f, 0xdb, 0x38, + 0x18, 0x95, 0xe4, 0xdf, 0x9f, 0x7d, 0x81, 0xc2, 0x4b, 0x72, 0x3e, 0x03, 0x67, 0xeb, 0x8c, 0xdc, + 0x25, 0x48, 0x0e, 0xf2, 0x5d, 0x6e, 0x29, 0xb2, 0xc5, 0x49, 0x5b, 0x04, 0x41, 0x81, 0x42, 0x76, + 0x3b, 0x74, 0x31, 0x64, 0x89, 0x55, 0x84, 0xb8, 0x94, 0x20, 0xd2, 0x09, 0xbc, 0x75, 0x2c, 0xbc, + 0x34, 0x53, 0xd1, 0xc5, 0x40, 0xfe, 0x9c, 0x8c, 0x19, 0x3b, 0x14, 0x46, 0xeb, 0x0c, 0x1d, 0xbb, + 0x67, 0x2a, 0x48, 0x2a, 0x56, 0xec, 0x76, 0x70, 0xb3, 0xd8, 0x14, 0xdf, 0xfb, 0x1e, 0xde, 0x7b, + 0x24, 0x08, 0x7f, 0x84, 0x51, 0xe0, 0x60, 0x4a, 0x83, 0xa8, 0xc1, 0xec, 0x6e, 0x0f, 0x87, 0x5d, + 0xf9, 0x6f, 0x86, 0x51, 0xc0, 0x02, 0xb4, 0x1e, 0xfa, 0xc4, 0x73, 0xec, 0xd0, 0x64, 0xfe, 0xcb, + 0x5e, 0x70, 0x66, 0x3a, 0xae, 0x63, 0x4e, 0x27, 0xcc, 0x78, 0xa2, 0xb2, 0xe2, 0x05, 0x5e, 0x20, + 0x06, 0x1a, 0x7c, 0x25, 0x67, 0xeb, 0x6f, 0x55, 0x48, 0xb7, 0x42, 0x9b, 0xa0, 0xff, 0x20, 0x2f, + 0x98, 0x1d, 0xdf, 0x2d, 0xab, 0x86, 0xba, 0x99, 0x6a, 0xae, 0x4d, 0xc6, 0xb5, 0x5c, 0x9b, 0xef, + 0x1d, 0x1e, 0xdc, 0x24, 0x4b, 0x2b, 0x27, 0x78, 0x87, 0x2e, 0x5a, 0x87, 0x02, 0x65, 0x76, 0xc4, + 0x3a, 0x27, 0x78, 0x50, 0xd6, 0x0c, 0x75, 0xb3, 0xd4, 0xcc, 0xdd, 0x8c, 0x6b, 0xa9, 0x23, 0x3c, + 0xb0, 0xf2, 0x02, 0x39, 0xc2, 0x03, 0x64, 0x40, 0x0e, 0x13, 0x57, 0x70, 0x52, 0xb3, 0x9c, 0x2c, + 0x26, 0xee, 0x11, 0x1e, 0xec, 0x96, 0xde, 0x5c, 0xd4, 0x94, 0xf7, 0x17, 0x35, 0xe5, 0xf5, 0x47, + 0x43, 0xa9, 0x9f, 0xab, 0x00, 0xfb, 0xc7, 0xd8, 0x39, 0x09, 0x03, 0x9f, 0x30, 0xb4, 0x0d, 0xbf, + 0x38, 0xd3, 0xaf, 0x0e, 0xa3, 0xc2, 0x5c, 0xba, 0x99, 0xbd, 0x19, 0xd7, 0xb4, 0x36, 0xb5, 0x4a, + 0x09, 0xd8, 0xa6, 0x68, 0x03, 0x8a, 0x11, 0xa6, 0x41, 0xef, 0x14, 0xbb, 0x9c, 0xaa, 0xcd, 0x50, + 0xe1, 0x16, 0x6a, 0x53, 0xf4, 0x0f, 0x2c, 0xf5, 0x6c, 0xca, 0x3a, 0x74, 0x40, 0x1c, 0xc9, 0x4d, + 0xcd, 0xca, 0x72, 0xb4, 0x25, 0xc0, 0x36, 0xad, 0x7f, 0xd1, 0x20, 0xd3, 0x62, 0x36, 0xa3, 0xe8, + 0x4f, 0x28, 0x45, 0xd8, 0xf3, 0x03, 0xd2, 0x71, 0x82, 0x3e, 0x61, 0xd2, 0x8c, 0x55, 0x94, 0x7b, + 0xfb, 0x7c, 0x0b, 0x6d, 0x00, 0x38, 0xfd, 0x28, 0xc2, 0xd2, 0xad, 0xb4, 0x90, 0x97, 0xb2, 0x65, + 0xd5, 0x2a, 0xc4, 0x58, 0x9b, 0x22, 0x06, 0xcb, 0x94, 0xd9, 0x1e, 0xee, 0x24, 0x11, 0xb8, 0x8d, + 0xd4, 0x66, 0x71, 0x67, 0xcf, 0x5c, 0xe4, 0x48, 0x4d, 0xe1, 0x89, 0xff, 0x7a, 0x38, 0x69, 0x8c, + 0x3e, 0x24, 0x2c, 0x1a, 0x34, 0xd3, 0x97, 0xe3, 0x9a, 0x62, 0xe9, 0x74, 0x0e, 0x44, 0x7f, 0x01, + 0x74, 0xed, 0x28, 0xf2, 0x71, 0xc4, 0xed, 0xa5, 0x67, 0x52, 0x17, 0x62, 0xa4, 0x4d, 0x2b, 0x7d, + 0x58, 0xfd, 0xa1, 0x2e, 0xd2, 0x21, 0xc5, 0x8f, 0x92, 0x07, 0x2f, 0x58, 0x7c, 0x89, 0x1e, 0x41, + 0xe6, 0xd4, 0xee, 0xf5, 0xb1, 0xc8, 0x5a, 0xdc, 0xf9, 0x77, 0x31, 0xef, 0x89, 0xb0, 0x25, 0xc7, + 0x77, 0xb5, 0x07, 0x6a, 0xfd, 0xab, 0x06, 0x45, 0x71, 0xcf, 0x78, 0xb4, 0x3e, 0xbd, 0xcf, 0xad, + 0x3c, 0x80, 0x34, 0x0d, 0x6d, 0x52, 0xce, 0x08, 0x37, 0x5b, 0x0b, 0x36, 0x19, 0xda, 0x24, 0xae, + 0x4c, 0x4c, 0xf3, 0x50, 0x94, 0xd9, 0x4c, 0x86, 0x5a, 0x5a, 0x34, 0xd4, 0xd4, 0x3a, 0xb6, 0xe4, + 0x38, 0x7a, 0x0e, 0x90, 0x1c, 0xaf, 0xb8, 0x64, 0xf7, 0x68, 0x28, 0x76, 0x76, 0x47, 0x09, 0x3d, + 0x96, 0xfe, 0xe4, 0x09, 0x16, 0x77, 0xb6, 0x7f, 0xe2, 0xc2, 0xc4, 0x6a, 0x72, 0x7e, 0xeb, 0x9d, + 0x06, 0x90, 0xd8, 0x46, 0x75, 0xc8, 0x3d, 0x23, 0x27, 0x24, 0x38, 0x23, 0xba, 0x52, 0x59, 0x1d, + 0x8e, 0x8c, 0xe5, 0x04, 0x8c, 0x01, 0x64, 0x40, 0x76, 0xaf, 0x4b, 0x31, 0x61, 0xba, 0x5a, 0x59, + 0x19, 0x8e, 0x0c, 0x3d, 0xa1, 0xc8, 0x7d, 0xf4, 0x37, 0x14, 0x9e, 0x46, 0x38, 0xb4, 0x23, 0x9f, + 0x78, 0xba, 0x56, 0xf9, 0x6d, 0x38, 0x32, 0x7e, 0x4d, 0x48, 0x53, 0x08, 0xad, 0x43, 0x5e, 0x7e, + 0x60, 0x57, 0x4f, 0x55, 0xd6, 0x86, 0x23, 0x03, 0xcd, 0xd3, 0xb0, 0x8b, 0xb6, 0xa0, 0x68, 0xe1, + 0xb0, 0xe7, 0x3b, 0x36, 0xe3, 0x7a, 0xe9, 0xca, 0xef, 0xc3, 0x91, 0xb1, 0x7a, 0xa7, 0xeb, 0x04, + 0xe4, 0x8a, 0x2d, 0x16, 0x84, 0xbc, 0x0d, 0x3d, 0x33, 0xaf, 0x78, 0x8b, 0xf0, 0x94, 0x62, 0x8d, + 0x5d, 0x3d, 0x3b, 0x9f, 0x32, 0x06, 0x9a, 0x4f, 0xae, 0x3e, 0x57, 0x95, 0xcb, 0x49, 0x55, 0xbd, + 0x9a, 0x54, 0xd5, 0x4f, 0x93, 0xaa, 0x7a, 0x7e, 0x5d, 0x55, 0xae, 0xae, 0xab, 0xca, 0x87, 0xeb, + 0xaa, 0xf2, 0xa2, 0xe1, 0xf9, 0xec, 0xb8, 0xdf, 0x35, 0x9d, 0xe0, 0x55, 0x23, 0xae, 0xbe, 0x21, + 0xab, 0x6f, 0x38, 0xae, 0xd3, 0xf8, 0xee, 0xc1, 0xee, 0x66, 0xc5, 0x7b, 0xfb, 0xff, 0xb7, 0x00, + 0x00, 0x00, 0xff, 0xff, 0x53, 0x4d, 0xc5, 0xfa, 0xcc, 0x05, 0x00, 0x00, } func (m *Span) Marshal() (dAtA []byte, err error) { diff --git a/cdc/puller/ddl_puller.go b/cdc/puller/ddl_puller.go index f4b9e456a62..6a5691cfcf8 100644 --- a/cdc/puller/ddl_puller.go +++ b/cdc/puller/ddl_puller.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tiflow/cdc/puller/memorysorter" "github.com/pingcap/tiflow/engine/pkg/clock" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/ddl" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" "github.com/pingcap/tiflow/pkg/spanz" @@ -73,10 +74,12 @@ type ddlJobPullerImpl struct { kvStorage tidbkv.Storage schemaStorage entry.SchemaStorage resolvedTs uint64 - schemaVersion int64 filter filter.Filter - // ddlTableInfo is initialized when receive the first concurrent DDL job. - ddlTableInfo *entry.DDLTableInfo + // ddlJobsTable is initialized when receive the first concurrent DDL job. + // It holds the info of table `tidb_ddl_jobs` of upstream TiDB. + ddlJobsTable *model.TableInfo + // It holds the column id of `job_meta` in table `tidb_ddl_jobs`. + jobMetaColumnID int64 // outputCh sends the DDL job entries to the caller. outputCh chan *model.DDLJobEntry } @@ -195,7 +198,7 @@ func (p *ddlJobPullerImpl) handleRawKVEntry(ctx context.Context, ddlRawKV *model } } - job, err := p.unmarshalDDL(ddlRawKV) + job, err := p.unmarshalDDL(ctx, ddlRawKV) if err != nil { return errors.Trace(err) } @@ -232,18 +235,17 @@ func (p *ddlJobPullerImpl) handleRawKVEntry(ctx context.Context, ddlRawKV *model return nil } -func (p *ddlJobPullerImpl) unmarshalDDL(rawKV *model.RawKVEntry) (*timodel.Job, error) { +func (p *ddlJobPullerImpl) unmarshalDDL(ctx context.Context, rawKV *model.RawKVEntry) (*timodel.Job, error) { if rawKV.OpType != model.OpTypePut { return nil, nil } - if p.ddlTableInfo == nil && !entry.IsLegacyFormatJob(rawKV) { - err := p.initDDLTableInfo() + if p.ddlJobsTable == nil && !entry.IsLegacyFormatJob(rawKV) { + err := p.initJobTableMeta(ctx) if err != nil { return nil, errors.Trace(err) } } - - return entry.ParseDDLJob(rawKV, p.ddlTableInfo) + return entry.ParseDDLJob(p.ddlJobsTable, rawKV, p.jobMetaColumnID) } func (p *ddlJobPullerImpl) getResolvedTs() uint64 { @@ -254,7 +256,7 @@ func (p *ddlJobPullerImpl) setResolvedTs(ts uint64) { atomic.StoreUint64(&p.resolvedTs, ts) } -func (p *ddlJobPullerImpl) initDDLTableInfo() error { +func (p *ddlJobPullerImpl) initJobTableMeta(ctx context.Context) error { version, err := p.kvStorage.CurrentVersion(tidbkv.GlobalTxnScope) if err != nil { return errors.Trace(err) @@ -271,12 +273,10 @@ func (p *ddlJobPullerImpl) initDDLTableInfo() error { return errors.Trace(err) } - tbls, err := snap.ListTables(db.ID) + tbls, err := snap.ListTables(ctx, db.ID) if err != nil { return errors.Trace(err) } - - // for tidb_ddl_job tableInfo, err := findTableByName(tbls, "tidb_ddl_job") if err != nil { return errors.Trace(err) @@ -287,24 +287,8 @@ func (p *ddlJobPullerImpl) initDDLTableInfo() error { return errors.Trace(err) } - p.ddlTableInfo = &entry.DDLTableInfo{} - p.ddlTableInfo.DDLJobTable = model.WrapTableInfo(db.ID, db.Name.L, 0, tableInfo) - p.ddlTableInfo.JobMetaColumnIDinJobTable = col.ID - - // for tidb_ddl_history - historyTableInfo, err := findTableByName(tbls, "tidb_ddl_history") - if err != nil { - return errors.Trace(err) - } - - historyTableCol, err := findColumnByName(historyTableInfo.Columns, "job_meta") - if err != nil { - return errors.Trace(err) - } - - p.ddlTableInfo.DDLHistoryTable = model.WrapTableInfo(db.ID, db.Name.L, 0, historyTableInfo) - p.ddlTableInfo.JobMetaColumnIDinHistoryTable = historyTableCol.ID - + p.ddlJobsTable = model.WrapTableInfo(db.ID, db.Name.L, 0, tableInfo) + p.jobMetaColumnID = col.ID return nil } @@ -318,7 +302,7 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { } if job.BinlogInfo.FinishedTS <= p.getResolvedTs() || - job.BinlogInfo.SchemaVersion <= p.schemaVersion { + job.BinlogInfo.SchemaVersion == 0 /* means the ddl is ignored in upstream */ { log.Info("ddl job finishedTs less than puller resolvedTs,"+ "discard the ddl job", zap.String("namespace", p.changefeedID.Namespace), @@ -391,11 +375,17 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { errors.Trace(err), job.Query, job.StartTS, job.StartTS) } case timodel.ActionCreateTables: + querys, err := ddl.SplitQueries(job.Query) + if err != nil { + return false, errors.Trace(err) + } // we only use multiTableInfos and Querys when we generate job event // So if some table should be discard, we just need to delete the info from multiTableInfos and Querys - if strings.Count(job.Query, ";") != len(job.BinlogInfo.MultipleTableInfos) { + if len(querys) != len(job.BinlogInfo.MultipleTableInfos) { log.Error("the number of queries in `Job.Query` is not equal to "+ "the number of `TableInfo` in `Job.BinlogInfo.MultipleTableInfos`", + zap.Int("numQueries", len(querys)), + zap.Int("numTableInfos", len(job.BinlogInfo.MultipleTableInfos)), zap.String("Job.Query", job.Query), zap.Any("Job.BinlogInfo.MultipleTableInfos", job.BinlogInfo.MultipleTableInfos), zap.Error(cerror.ErrTiDBUnexpectedJobMeta.GenWithStackByArgs())) @@ -406,7 +396,6 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { var newQuerys []string multiTableInfos := job.BinlogInfo.MultipleTableInfos - querys := strings.Split(job.Query, ";") for index, tableInfo := range multiTableInfos { // judge each table whether need to be skip @@ -414,7 +403,7 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { continue } newMultiTableInfos = append(newMultiTableInfos, multiTableInfos[index]) - newQuerys = append(newQuerys, querys[index]+";") + newQuerys = append(newQuerys, querys[index]) } skip = len(newMultiTableInfos) == 0 @@ -480,7 +469,6 @@ func (p *ddlJobPullerImpl) handleJob(job *timodel.Job) (skip bool, err error) { errors.Trace(err), job.Query, job.StartTS, job.StartTS) } p.setResolvedTs(job.BinlogInfo.FinishedTS) - p.schemaVersion = job.BinlogInfo.SchemaVersion return p.checkIneligibleTableDDL(snap, job) } @@ -496,27 +484,50 @@ func (p *ddlJobPullerImpl) checkIneligibleTableDDL(snapBefore *schema.Snapshot, return false, nil } - ineligible := p.schemaStorage.GetLastSnapshot().IsIneligibleTableID(job.TableID) - if !ineligible { + snapAfter := p.schemaStorage.GetLastSnapshot() + + if job.Type == timodel.ActionCreateTable { + // For create table, oldTableID is the new table ID. + isEligibleAfter := !snapAfter.IsIneligibleTableID(job.BinlogInfo.TableInfo.ID) + if isEligibleAfter { + return false, nil + } + } + + // For create tables, we always apply the DDL here. + if job.Type == timodel.ActionCreateTables { + return false, nil + } + + oldTableID := job.TableID + newTableID := job.BinlogInfo.TableInfo.ID + + // If the table is eligible after the DDL, we should apply the DDL. + // No matter its status before the DDL. + isEligibleAfter := !p.schemaStorage.GetLastSnapshot().IsIneligibleTableID(newTableID) + if isEligibleAfter { return false, nil } - // If the table is not in the snapshot before the DDL, + // Steps here means this table is ineligible after the DDL. + // We need to check if its status before the DDL. + + // 1. If the table is not in the snapshot before the DDL, // we should ignore the DDL. - _, exist := snapBefore.PhysicalTableByID(job.TableID) + _, exist := snapBefore.PhysicalTableByID(oldTableID) if !exist { return true, nil } - // If the table after the DDL is ineligible, we should check if it is not ineligible before the DDL. - // If so, we should return an error to inform the user that it is a - // dangerous operation and should be handled manually. - isBeforeineligible := snapBefore.IsIneligibleTableID(job.TableID) - if isBeforeineligible { - log.Warn("ignore the DDL event of ineligible table", + // 2. If the table is ineligible before the DDL, we should ignore the DDL. + isIneligibleBefore := snapBefore.IsIneligibleTableID(oldTableID) + if isIneligibleBefore { + log.Warn("Ignore the DDL event of ineligible table", zap.String("changefeed", p.changefeedID.ID), zap.Any("ddl", job)) return true, nil } + + // 3. If the table is eligible before the DDL, we should return an error. return false, cerror.New(fmt.Sprintf("An eligible table become ineligible after DDL: [%s] "+ "it is a dangerous operation and may cause data loss. If you want to replicate this ddl safely, "+ "pelase pause the changefeed and update the `force-replicate=true` "+ diff --git a/cdc/puller/ddl_puller_test.go b/cdc/puller/ddl_puller_test.go index e743cfe7e1c..728627c5ba0 100644 --- a/cdc/puller/ddl_puller_test.go +++ b/cdc/puller/ddl_puller_test.go @@ -728,7 +728,7 @@ func waitResolvedTsGrowing(t *testing.T, p DDLPuller, targetTs model.Ts) { require.Nil(t, err) } -func TestCcheckIneligibleTableDDL(t *testing.T) { +func TestCheckIneligibleTableDDL(t *testing.T) { ddlJobPuller, helper := newMockDDLJobPuller(t, true) defer helper.Close() @@ -787,4 +787,36 @@ func TestCcheckIneligibleTableDDL(t *testing.T) { require.Error(t, err) require.False(t, skip) require.Contains(t, err.Error(), "An eligible table become ineligible after DDL") + + // case 4: create a ineligible table and truncate it, expect no error. + // It is because the table is ineligible before the DDL. + ddl = helper.DDL2Job("CREATE TABLE test1.t3 (id INT);") + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + require.True(t, skip) + + ddl = helper.DDL2Job("TRUNCATE TABLE test1.t3;") + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + // Skip because the table is ineligible before the DDL. + require.True(t, skip) + + // case 5: create a ineligible table and alter it to eligible, expect no error. + ddl = helper.DDL2Job("CREATE TABLE test1.t4 (id INT not null);") + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + require.True(t, skip) + + // Add a unique key to the table, make it eligible. + ddl = helper.DDL2Job("ALTER TABLE test1.t4 ADD UNIQUE KEY cdc_valid_index (id);") + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + require.False(t, skip) + + // case 6: Batch create tables, including a ineligible table and a eligible table, expect no error. + ddl = helper.DDL2Job(`CREATE TABLE test1.t5 (id INT); + CREATE TABLE test1.t6 (id INT PRIMARY KEY);`) + skip, err = ddlJobPullerImpl.handleJob(ddl) + require.NoError(t, err) + require.False(t, skip) } diff --git a/cdc/redo/writer/memory/encoding_worker.go b/cdc/redo/writer/memory/encoding_worker.go index 9b0cc6a8652..f62f236f888 100644 --- a/cdc/redo/writer/memory/encoding_worker.go +++ b/cdc/redo/writer/memory/encoding_worker.go @@ -131,7 +131,7 @@ func (e *encodingWorkerGroup) Run(ctx context.Context) (err error) { zap.String("namespace", e.changefeed.Namespace), zap.String("changefeed", e.changefeed.ID), zap.Error(err)) - if err != nil && errors.Cause(err) != context.Canceled { + if err != nil { e.closed <- err } close(e.closed) diff --git a/cdc/redo/writer/memory/mem_log_writer_test.go b/cdc/redo/writer/memory/mem_log_writer_test.go index 33e1f426489..5914328b4a1 100644 --- a/cdc/redo/writer/memory/mem_log_writer_test.go +++ b/cdc/redo/writer/memory/mem_log_writer_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/writer" + "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/util" "github.com/stretchr/testify/require" @@ -99,10 +100,45 @@ func testWriteEvents(t *testing.T, events []writer.RedoEvent) { }) require.NoError(t, err) + require.ErrorIs(t, lw.Close(), context.Canceled) + // duplicate close should return the same error require.ErrorIs(t, lw.Close(), context.Canceled) - err = lw.WriteEvents(ctx, events...) - require.NoError(t, err) - err = lw.FlushLog(ctx) - require.NoError(t, err) + functions := map[string]func(error){ + "WriteEvents": func(expected error) { + if expected == nil { + err := lw.WriteEvents(ctx, events...) + require.NoError(t, err) + } else { + require.Eventually( + t, func() bool { + err := lw.WriteEvents(ctx, events...) + return errors.Is(errors.Cause(err), expected) + }, time.Second*2, time.Microsecond*10, + ) + } + }, + "FlushLog": func(expected error) { + if expected == nil { + err := lw.FlushLog(ctx) + require.NoError(t, err) + } else { + require.Eventually( + t, func() bool { + err := lw.WriteEvents(ctx, events...) + return errors.Is(errors.Cause(err), expected) + }, time.Second*2, time.Microsecond*10, + ) + } + }, + } + firstCall := true + for _, f := range functions { + if firstCall { + firstCall = false + f(context.Canceled) + } else { + f(nil) + } + } } diff --git a/cmd/kafka-consumer/consumer.go b/cmd/kafka-consumer/consumer.go index 825f0da9ecf..5693a4b93dc 100644 --- a/cmd/kafka-consumer/consumer.go +++ b/cmd/kafka-consumer/consumer.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) func getPartitionNum(o *option) (int32, error) { @@ -99,6 +100,9 @@ func newConsumer(ctx context.Context, o *option) *consumer { _ = configMap.SetKey("ssl.key.location", o.key) _ = configMap.SetKey("ssl.certificate.location", o.cert) } + if level, err := zapcore.ParseLevel(o.logLevel); err == nil && level.String() == "debug" { + configMap.SetKey("debug", "all") + } client, err := kafka.NewConsumer(configMap) if err != nil { log.Panic("create kafka consumer failed", zap.Error(err)) @@ -121,6 +125,12 @@ func (c *consumer) Consume(ctx context.Context) { } }() for { + select { + case <-ctx.Done(): + log.Info("consumer exist: context cancelled") + return + default: + } msg, err := c.client.ReadMessage(-1) if err != nil { log.Error("read message failed, just continue to retry", zap.Error(err)) @@ -133,10 +143,12 @@ func (c *consumer) Consume(ctx context.Context) { topicPartition, err := c.client.CommitMessage(msg) if err != nil { - log.Error("commit message failed, just continue", zap.Error(err)) + log.Error("commit message failed, just continue", + zap.String("topic", *msg.TopicPartition.Topic), zap.Int32("partition", msg.TopicPartition.Partition), + zap.Any("offset", msg.TopicPartition.Offset), zap.Error(err)) continue } - log.Info("commit message success", + log.Debug("commit message success", zap.String("topic", topicPartition[0].String()), zap.Int32("partition", topicPartition[0].Partition), zap.Any("offset", topicPartition[0].Offset)) } diff --git a/cmd/kafka-consumer/event_group.go b/cmd/kafka-consumer/event_group.go index 23a6c26133b..03e09aa68e2 100644 --- a/cmd/kafka-consumer/event_group.go +++ b/cmd/kafka-consumer/event_group.go @@ -16,37 +16,77 @@ package main import ( "sort" + "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "go.uber.org/zap" ) // EventsGroup could store change event message. type eventsGroup struct { - events []*model.RowChangedEvent + partition int32 + tableID int64 + + events []*model.RowChangedEvent + highWatermark uint64 } // NewEventsGroup will create new event group. -func NewEventsGroup() *eventsGroup { +func NewEventsGroup(partition int32, tableID int64) *eventsGroup { return &eventsGroup{ - events: make([]*model.RowChangedEvent, 0), + partition: partition, + tableID: tableID, + events: make([]*model.RowChangedEvent, 0), } } // Append will append an event to event groups. -func (g *eventsGroup) Append(e *model.RowChangedEvent) { - g.events = append(g.events, e) +func (g *eventsGroup) Append(row *model.RowChangedEvent, offset kafka.Offset) { + g.events = append(g.events, row) + if row.CommitTs > g.highWatermark { + g.highWatermark = row.CommitTs + } + log.Info("DML event received", + zap.Int32("partition", g.partition), + zap.Any("offset", offset), + zap.Uint64("commitTs", row.CommitTs), + zap.Uint64("highWatermark", g.highWatermark), + zap.Int64("physicalTableID", row.GetTableID()), + zap.String("schema", row.TableInfo.GetSchemaName()), + zap.String("table", row.TableInfo.GetTableName()), + zap.Any("columns", row.Columns), zap.Any("preColumns", row.PreColumns)) } // Resolve will get events where CommitTs is less than resolveTs. -func (g *eventsGroup) Resolve(resolveTs uint64) []*model.RowChangedEvent { - sort.Slice(g.events, func(i, j int) bool { - return g.events[i].CommitTs < g.events[j].CommitTs - }) +func (g *eventsGroup) Resolve(resolve uint64, protocol config.Protocol) []*model.RowChangedEvent { + switch protocol { + case config.ProtocolCanalJSON: + sort.Slice(g.events, func(i, j int) bool { + return g.events[i].CommitTs < g.events[j].CommitTs + }) + default: + if !sort.SliceIsSorted(g.events, func(i, j int) bool { + return g.events[i].CommitTs < g.events[j].CommitTs + }) { + log.Warn("events are not sorted", zap.Int32("partition", g.partition), + zap.Int64("tableID", g.tableID), zap.Int("eventCount", len(g.events))) + } + } i := sort.Search(len(g.events), func(i int) bool { - return g.events[i].CommitTs > resolveTs + return g.events[i].CommitTs > resolve }) result := g.events[:i] g.events = g.events[i:] + + if len(result) != 0 && len(g.events) != 0 { + log.Warn("not all events resolved", + zap.Int32("partition", g.partition), zap.Int64("tableID", g.tableID), + zap.Int("resolved", len(result)), zap.Int("remained", len(g.events)), + zap.Uint64("resolveTs", resolve), zap.Uint64("firstCommitTs", g.events[0].CommitTs)) + } + return result } diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index 72993c4857a..ac008e9679e 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -22,7 +22,6 @@ import ( "net/url" "os" "os/signal" - "runtime/debug" "strings" "sync" "syscall" @@ -35,13 +34,10 @@ import ( ) func main() { - debug.SetMemoryLimit(14 * 1024 * 1024 * 1024) - var ( upstreamURIStr string configFile string ) - groupID := fmt.Sprintf("ticdc_kafka_consumer_%s", uuid.New().String()) consumerOption := newOption() flag.StringVar(&configFile, "config", "", "config file for changefeed") @@ -87,6 +83,7 @@ func main() { consumer := newConsumer(ctx, consumerOption) var wg sync.WaitGroup if consumerOption.enableProfiling { + log.Info("profiling is enabled") wg.Add(1) go func() { defer wg.Done() diff --git a/cmd/kafka-consumer/option.go b/cmd/kafka-consumer/option.go index 394e8e13798..e23cc42ae7e 100644 --- a/cmd/kafka-consumer/option.go +++ b/cmd/kafka-consumer/option.go @@ -170,6 +170,7 @@ func (o *option) Adjust(upstreamURI *url.URL, configFile string) error { zap.String("groupID", o.groupID), zap.Int("maxMessageBytes", o.maxMessageBytes), zap.Int("maxBatchSize", o.maxBatchSize), - zap.String("upstreamURI", upstreamURI.String())) + zap.String("upstreamURI", upstreamURI.String()), + zap.String("downstreamURI", o.downstreamURI)) return nil } diff --git a/cmd/kafka-consumer/writer.go b/cmd/kafka-consumer/writer.go index a277348605e..d2407c2cc30 100644 --- a/cmd/kafka-consumer/writer.go +++ b/cmd/kafka-consumer/writer.go @@ -72,6 +72,7 @@ func NewDecoder(ctx context.Context, option *option, upstreamTiDB *sql.DB) (code } type partitionProgress struct { + partition int32 watermark uint64 watermarkOffset kafka.Offset // tableSinkMap -> [tableID]tableSink @@ -81,6 +82,24 @@ type partitionProgress struct { decoder codec.RowEventDecoder } +func newPartitionProgress(partition int32, decoder codec.RowEventDecoder) *partitionProgress { + return &partitionProgress{ + partition: partition, + eventGroups: make(map[int64]*eventsGroup), + decoder: decoder, + } +} + +func (p *partitionProgress) updateWatermark(watermark uint64, offset kafka.Offset) { + atomic.StoreUint64(&p.watermark, watermark) + p.watermarkOffset = offset + log.Info("watermark received", zap.Int32("partition", p.partition), zap.Any("offset", offset), zap.Uint64("watermark", watermark)) +} + +func (p *partitionProgress) loadWatermark() uint64 { + return atomic.LoadUint64(&p.watermark) +} + type writer struct { option *option @@ -104,19 +123,10 @@ func newWriter(ctx context.Context, o *option) *writer { }, progresses: make([]*partitionProgress, o.partitionNum), } - - eventRouter, err := dispatcher.NewEventRouter(o.replicaConfig, o.protocol, o.topic, "kafka") - if err != nil { - log.Panic("initialize the event router failed", - zap.Any("protocol", o.protocol), zap.Any("topic", o.topic), - zap.Any("dispatcherRules", o.replicaConfig.Sink.DispatchRules), zap.Error(err)) - } - w.eventRouter = eventRouter - log.Info("event router created", zap.Any("protocol", o.protocol), - zap.Any("topic", o.topic), zap.Any("dispatcherRules", o.replicaConfig.Sink.DispatchRules)) - - var db *sql.DB - + var ( + db *sql.DB + err error + ) if o.upstreamTiDBDSN != "" { db, err = openDB(ctx, o.upstreamTiDBDSN) if err != nil { @@ -124,17 +134,23 @@ func newWriter(ctx context.Context, o *option) *writer { zap.String("dsn", o.upstreamTiDBDSN)) } } - + decoder, err := NewDecoder(ctx, o, db) for i := 0; i < int(o.partitionNum); i++ { - decoder, err := NewDecoder(ctx, o, db) if err != nil { log.Panic("cannot create the decoder", zap.Error(err)) } - w.progresses[i] = &partitionProgress{ - eventGroups: make(map[int64]*eventsGroup), - decoder: decoder, - } + w.progresses[i] = newPartitionProgress(int32(i), decoder) + } + + eventRouter, err := dispatcher.NewEventRouter(o.replicaConfig, o.protocol, o.topic, "kafka") + if err != nil { + log.Panic("initialize the event router failed", + zap.Any("protocol", o.protocol), zap.Any("topic", o.topic), + zap.Any("dispatcherRules", o.replicaConfig.Sink.DispatchRules), zap.Error(err)) } + w.eventRouter = eventRouter + log.Info("event router created", zap.Any("protocol", o.protocol), + zap.Any("topic", o.topic), zap.Any("dispatcherRules", o.replicaConfig.Sink.DispatchRules)) config.GetGlobalServerConfig().TZ = o.timezone errChan := make(chan error, 1) @@ -164,7 +180,7 @@ func newWriter(ctx context.Context, o *option) *writer { // append DDL wait to be handled, only consider the constraint among DDLs. // for DDL a / b received in the order, a.CommitTs < b.CommitTs should be true. -func (w *writer) appendDDL(ddl *model.DDLEvent) { +func (w *writer) appendDDL(ddl *model.DDLEvent, offset kafka.Offset) { // DDL CommitTs fallback, just crash it to indicate the bug. if w.ddlWithMaxCommitTs != nil && ddl.CommitTs < w.ddlWithMaxCommitTs.CommitTs { log.Warn("DDL CommitTs < maxCommitTsDDL.CommitTs", @@ -185,6 +201,7 @@ func (w *writer) appendDDL(ddl *model.DDLEvent) { w.ddlList = append(w.ddlList, ddl) w.ddlWithMaxCommitTs = ddl + log.Info("DDL message received", zap.Any("offset", offset), zap.Uint64("commitTs", ddl.CommitTs), zap.String("DDL", ddl.Query)) } func (w *writer) getFrontDDL() *model.DDLEvent { @@ -203,7 +220,7 @@ func (w *writer) popDDL() { func (w *writer) getMinWatermark() uint64 { result := uint64(math.MaxUint64) for _, p := range w.progresses { - watermark := atomic.LoadUint64(&p.watermark) + watermark := p.loadWatermark() if watermark < result { result = watermark } @@ -272,6 +289,7 @@ func (w *writer) WriteMessage(ctx context.Context, message *kafka.Message) bool key = message.Key value = message.Value partition = message.TopicPartition.Partition + offset = message.TopicPartition.Offset ) progress := w.progresses[partition] @@ -279,8 +297,7 @@ func (w *writer) WriteMessage(ctx context.Context, message *kafka.Message) bool eventGroup := progress.eventGroups if err := decoder.AddKeyValue(key, value); err != nil { log.Panic("add key value to the decoder failed", - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset), - zap.Error(err)) + zap.Int32("partition", partition), zap.Any("offset", offset), zap.Error(err)) } var ( counter int @@ -291,8 +308,7 @@ func (w *writer) WriteMessage(ctx context.Context, message *kafka.Message) bool ty, hasNext, err := decoder.HasNext() if err != nil { log.Panic("decode message key failed", - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset), - zap.Error(err)) + zap.Int32("partition", partition), zap.Any("offset", offset), zap.Error(err)) } if !hasNext { break @@ -301,7 +317,7 @@ func (w *writer) WriteMessage(ctx context.Context, message *kafka.Message) bool // If the message containing only one event exceeds the length limit, CDC will allow it and issue a warning. if len(key)+len(value) > w.option.maxMessageBytes && counter > 1 { log.Panic("kafka max-messages-bytes exceeded", - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset), + zap.Int32("partition", partition), zap.Any("offset", offset), zap.Int("max-message-bytes", w.option.maxMessageBytes), zap.Int("receivedBytes", len(key)+len(value))) } @@ -317,43 +333,41 @@ func (w *writer) WriteMessage(ctx context.Context, message *kafka.Message) bool ddl, err := decoder.NextDDLEvent() if err != nil { log.Panic("decode message value failed", - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset), - zap.ByteString("value", value), - zap.Error(err)) + zap.Int32("partition", partition), zap.Any("offset", offset), + zap.ByteString("value", value), zap.Error(err)) } if simple, ok := decoder.(*simple.Decoder); ok { cachedEvents := simple.GetCachedEvents() + if len(cachedEvents) != 0 { + log.Info("simple protocol resolved cached events", zap.Int("resolvedCount", len(cachedEvents))) + } for _, row := range cachedEvents { - row.TableInfo.TableName.TableID = row.PhysicalTableID - w.checkPartition(row, partition, message) - if w.checkOldMessage(progress, row.CommitTs, row, partition, message) { - continue - } - group, ok := eventGroup[row.PhysicalTableID] + w.checkPartition(row, partition, message.TopicPartition.Offset) + tableID := row.GetTableID() + group, ok := eventGroup[tableID] if !ok { - group = NewEventsGroup() - eventGroup[row.PhysicalTableID] = group + group = NewEventsGroup(partition, tableID) + eventGroup[tableID] = group } - group.Append(row) + w.appendRow2Group(row, group, progress, offset) } } - // the Query maybe empty if using simple protocol, it's comes from `bootstrap` event. - if partition == 0 && ddl.Query != "" { - w.appendDDL(ddl) - needFlush = true - log.Info("DDL message received", - zap.Int32("partition", partition), - zap.Any("offset", message.TopicPartition.Offset), - zap.Uint64("commitTs", ddl.CommitTs), - zap.String("DDL", ddl.Query)) + // the Query maybe empty if using simple protocol, it's comes from `bootstrap` event, no need to handle it. + if ddl.Query == "" { + continue + } + + if partition == 0 { + w.appendDDL(ddl, offset) } + needFlush = true case model.MessageTypeRow: row, err := decoder.NextRowChangedEvent() if err != nil { log.Panic("decode message value failed", - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset), + zap.Int32("partition", partition), zap.Any("offset", offset), zap.ByteString("value", value), zap.Error(err)) } @@ -362,85 +376,45 @@ func (w *writer) WriteMessage(ctx context.Context, message *kafka.Message) bool if w.option.protocol == config.ProtocolSimple && row == nil { continue } + w.checkPartition(row, partition, message.TopicPartition.Offset) - tableID := row.PhysicalTableID - // simple protocol decoder should have set the table id already. + tableID := row.GetTableID() if w.option.protocol != config.ProtocolSimple { tableID = w.fakeTableIDGenerator. - generateFakeTableID(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName(), row.PhysicalTableID) - row.TableInfo.TableName.TableID = tableID + generateFakeTableID(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName(), tableID) + row.PhysicalTableID = tableID } - - w.checkPartition(row, partition, message) - - if w.checkOldMessage(progress, row.CommitTs, row, partition, message) { - continue - } - - group, ok := eventGroup[tableID] - if !ok { - group = NewEventsGroup() + group := eventGroup[tableID] + if group == nil { + group = NewEventsGroup(partition, tableID) eventGroup[tableID] = group } - group.Append(row) - log.Debug("DML event received", - zap.Int32("partition", partition), - zap.Any("offset", message.TopicPartition.Offset), - zap.Uint64("commitTs", row.CommitTs), - zap.Int64("physicalTableID", row.PhysicalTableID), - zap.Int64("tableID", tableID), - zap.String("schema", row.TableInfo.GetSchemaName()), - zap.String("table", row.TableInfo.GetTableName())) + w.appendRow2Group(row, group, progress, offset) case model.MessageTypeResolved: - ts, err := decoder.NextResolvedEvent() + newWatermark, err := decoder.NextResolvedEvent() if err != nil { log.Panic("decode message value failed", - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset), - zap.ByteString("value", value), - zap.Error(err)) + zap.Int32("partition", partition), zap.Any("offset", offset), + zap.ByteString("value", value), zap.Error(err)) } - log.Debug("watermark event received", - zap.Int32("partition", partition), - zap.Any("offset", message.TopicPartition.Offset), - zap.Uint64("watermark", ts)) - - if w.checkOldMessage(progress, ts, nil, partition, message) { + if w.checkOldMessageForWatermark(newWatermark, partition, offset) { continue } - for tableID, group := range eventGroup { - events := group.Resolve(ts) - if len(events) == 0 { - continue - } - tableSink, ok := progress.tableSinkMap.Load(tableID) - if !ok { - tableSink = w.sinkFactory.CreateTableSinkForConsumer( - model.DefaultChangeFeedID("kafka-consumer"), - spanz.TableIDToComparableSpan(tableID), - events[0].CommitTs, - ) - progress.tableSinkMap.Store(tableID, tableSink) - } - tableSink.(tablesink.TableSink).AppendRowChangedEvents(events...) - log.Debug("append row changed events to table sink", - zap.Uint64("resolvedTs", ts), zap.Int64("tableID", tableID), zap.Int("count", len(events)), - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset)) - } - atomic.StoreUint64(&progress.watermark, ts) - progress.watermarkOffset = message.TopicPartition.Offset + w.resolveRowChangedEvents(eventGroup, newWatermark, progress) + progress.updateWatermark(newWatermark, offset) needFlush = true default: log.Panic("unknown message type", zap.Any("messageType", messageType), - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset)) + zap.Int32("partition", partition), zap.Any("offset", offset)) } } if counter > w.option.maxBatchSize { log.Panic("Open Protocol max-batch-size exceeded", zap.Int("max-batch-size", w.option.maxBatchSize), zap.Int("actual-batch-size", counter), - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset)) + zap.Int32("partition", partition), zap.Any("offset", offset)) } if !needFlush { @@ -450,65 +424,106 @@ func (w *writer) WriteMessage(ctx context.Context, message *kafka.Message) bool return w.Write(ctx, messageType) } -func (w *writer) checkPartition(row *model.RowChangedEvent, partition int32, message *kafka.Message) { +func (w *writer) resolveRowChangedEvents(eventGroup map[int64]*eventsGroup, newWatermark uint64, progress *partitionProgress) { + for tableID, group := range eventGroup { + events := group.Resolve(newWatermark, w.option.protocol) + if len(events) == 0 { + continue + } + tableSink, ok := progress.tableSinkMap.Load(tableID) + if !ok { + tableSink = w.sinkFactory.CreateTableSinkForConsumer( + model.DefaultChangeFeedID("kafka-consumer"), + spanz.TableIDToComparableSpan(tableID), + events[0].CommitTs, + ) + progress.tableSinkMap.Store(tableID, tableSink) + } + tableSink.(tablesink.TableSink).AppendRowChangedEvents(events...) + } +} + +func (w *writer) checkPartition(row *model.RowChangedEvent, partition int32, offset kafka.Offset) { target, _, err := w.eventRouter.GetPartitionForRowChange(row, w.option.partitionNum) if err != nil { log.Panic("cannot calculate partition for the row changed event", - zap.Int32("partition", partition), zap.Any("offset", message.TopicPartition.Offset), - zap.Int32("partitionNum", w.option.partitionNum), zap.Int64("tableID", row.TableInfo.TableName.TableID), + zap.Int32("partition", partition), zap.Any("offset", offset), + zap.Int32("partitionNum", w.option.partitionNum), zap.Int64("tableID", row.GetTableID()), zap.Error(err), zap.Any("event", row)) } if partition != target { log.Panic("RowChangedEvent dispatched to wrong partition", zap.Int32("partition", partition), zap.Int32("expected", target), - zap.Int32("partitionNum", w.option.partitionNum), - zap.Any("offset", message.TopicPartition.Offset), - zap.Int64("tableID", row.TableInfo.TableName.TableID), zap.Any("row", row), + zap.Int32("partitionNum", w.option.partitionNum), zap.Any("offset", offset), + zap.Int64("tableID", row.GetTableID()), zap.Any("row", row), ) } } -func (w *writer) checkOldMessage(progress *partitionProgress, ts uint64, row *model.RowChangedEvent, partition int32, message *kafka.Message) bool { - watermark := atomic.LoadUint64(&progress.watermark) - if row == nil { - watermark := atomic.LoadUint64(&progress.watermark) - if ts < watermark { - if message.TopicPartition.Offset > progress.watermarkOffset { - log.Panic("partition resolved ts fallback, skip it", - zap.Uint64("ts", ts), zap.Any("offset", message.TopicPartition.Offset), - zap.Uint64("watermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset), - zap.Int32("partition", partition)) - } - log.Warn("partition resolved ts fall back, ignore it, since consumer read old offset message", - zap.Uint64("ts", ts), zap.Any("offset", message.TopicPartition.Offset), - zap.Uint64("watermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset), - zap.Int32("partition", partition)) - return true - } +func (w *writer) checkOldMessageForWatermark(newWatermark uint64, partition int32, offset kafka.Offset) bool { + progress := w.progresses[partition] + watermark := progress.loadWatermark() + if newWatermark >= watermark { return false } + if offset > progress.watermarkOffset { + log.Panic("partition resolved ts fallback", + zap.Int32("partition", partition), + zap.Uint64("newWatermark", newWatermark), zap.Any("offset", offset), + zap.Uint64("watermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset)) + } + log.Warn("partition resolved ts fall back, ignore it, since consumer read old offset message", + zap.Int32("partition", partition), + zap.Uint64("newWatermark", newWatermark), zap.Any("offset", offset), + zap.Uint64("watermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset)) + return true +} + +func (w *writer) appendRow2Group(row *model.RowChangedEvent, group *eventsGroup, progress *partitionProgress, offset kafka.Offset) { // if the kafka cluster is normal, this should not hit. // else if the cluster is abnormal, the consumer may consume old message, then cause the watermark fallback. - if ts < watermark { - // if commit message failed, the consumer may read previous message, - // just ignore this message should be fine, otherwise panic. - if message.TopicPartition.Offset > progress.watermarkOffset { - log.Panic("RowChangedEvent fallback row", - zap.Uint64("commitTs", ts), zap.Any("offset", message.TopicPartition.Offset), - zap.Uint64("watermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset), - zap.Int32("partition", partition), zap.Int64("tableID", row.TableInfo.TableName.TableID), - zap.String("schema", row.TableInfo.GetSchemaName()), - zap.String("table", row.TableInfo.GetTableName())) - } - log.Warn("Row changed event fall back, ignore it, since consumer read old offset message", - zap.Uint64("commitTs", row.CommitTs), zap.Any("offset", message.TopicPartition.Offset), + watermark := progress.loadWatermark() + if row.CommitTs < watermark { + log.Warn("RowChanged Event fallback row, since les than the partition watermark, ignore it", + zap.Int64("tableID", row.GetTableID()), zap.Int32("partition", progress.partition), + zap.Uint64("commitTs", row.CommitTs), zap.Any("offset", offset), zap.Uint64("watermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset), - zap.Int32("partition", partition), zap.Int64("tableID", row.TableInfo.TableName.TableID), - zap.String("schema", row.TableInfo.GetSchemaName()), - zap.String("table", row.TableInfo.GetTableName())) - return true + zap.String("schema", row.TableInfo.GetSchemaName()), zap.String("table", row.TableInfo.GetTableName()), + zap.String("protocol", w.option.protocol.String()), zap.Bool("IsPartition", row.TableInfo.TableName.IsPartition)) + return + } + if row.CommitTs >= group.highWatermark { + group.Append(row, offset) + return } - return false + switch w.option.protocol { + case config.ProtocolSimple, config.ProtocolOpen: + // simple protocol set the table id for all row message, it can be known which table the row message belongs to, + // also consider the table partition. + // open protocol set the partition table id if the table is partitioned. + // for normal table, the table id is generated by the fake table id generator by using schema and table name. + // so one event group for one normal table or one table partition, replayed messages can be ignored. + log.Warn("RowChangedEvent fallback row, since less than the group high watermark, ignore it", + zap.Int64("tableID", row.GetTableID()), zap.Int32("partition", progress.partition), + zap.Uint64("commitTs", row.CommitTs), zap.Any("offset", offset), + zap.Uint64("highWatermark", group.highWatermark), + zap.Any("partitionWatermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset), + zap.String("schema", row.TableInfo.GetSchemaName()), zap.String("table", row.TableInfo.GetTableName()), + zap.String("protocol", w.option.protocol.String()), zap.Bool("IsPartition", row.TableInfo.TableName.IsPartition)) + return + default: + // canal-json does not set table id for all messages. + // in the partition table case, all partition tables have the same table id, use the same progress, + // so it's hard to know whether the fallback row comes from the same table partition or not, so do not ignore the row. + } + log.Warn("RowChangedEvent fallback row, since less than the group high watermark, do not ignore it", + zap.Int64("tableID", row.GetTableID()), zap.Int32("partition", progress.partition), + zap.Uint64("commitTs", row.CommitTs), zap.Any("offset", offset), + zap.Uint64("highWatermark", group.highWatermark), + zap.Any("partitionWatermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset), + zap.String("schema", row.TableInfo.GetSchemaName()), zap.String("table", row.TableInfo.GetTableName()), + zap.String("protocol", w.option.protocol.String())) + group.Append(row, offset) } type fakeTableIDGenerator struct { @@ -516,11 +531,8 @@ type fakeTableIDGenerator struct { currentTableID int64 } -func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partition int64) int64 { - key := quotes.QuoteSchema(schema, table) - if partition != 0 { - key = fmt.Sprintf("%s.`%d`", key, partition) - } +func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, tableID int64) int64 { + key := fmt.Sprintf("`%s`.`%s`.`%d`", quotes.EscapeName(schema), quotes.EscapeName(table), tableID) if tableID, ok := g.tableIDs[key]; ok { return tableID } @@ -530,6 +542,7 @@ func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partiti } func syncFlushRowChangedEvents(ctx context.Context, progress *partitionProgress, watermark uint64) { + resolvedTs := model.NewResolvedTs(watermark) for { select { case <-ctx.Done(): @@ -539,10 +552,7 @@ func syncFlushRowChangedEvents(ctx context.Context, progress *partitionProgress, } flushedResolvedTs := true progress.tableSinkMap.Range(func(key, value interface{}) bool { - resolvedTs := model.NewResolvedTs(watermark) tableSink := value.(tablesink.TableSink) - // todo: can we update resolved ts for each table sink concurrently ? - // this maybe helpful to accelerate the consume process, and reduce the memory usage. if err := tableSink.UpdateResolvedTs(resolvedTs); err != nil { log.Panic("Failed to update resolved ts", zap.Error(err)) } diff --git a/deployments/ticdc/docker/kafka-consumer.Dockerfile b/deployments/ticdc/docker/kafka-consumer.Dockerfile index 67c22e1b1ae..4904da4ad72 100644 --- a/deployments/ticdc/docker/kafka-consumer.Dockerfile +++ b/deployments/ticdc/docker/kafka-consumer.Dockerfile @@ -1,4 +1,4 @@ -FROM golang:1.23-alpine as builder +FROM golang:1.23-alpine AS builder RUN apk add --no-cache make bash git build-base WORKDIR /go/src/github.com/pingcap/tiflow COPY . . @@ -8,7 +8,7 @@ RUN --mount=type=cache,target=/root/.cache/go-build make kafka_consumer FROM alpine:3.15 -RUN apk update && apk add tzdata +RUN apk update && apk add tzdata curl ENV TZ=Asia/Shanghai diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index f6ad305846a..6e1af7f9907 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -152,7 +152,7 @@ ErrConfigLoaderCfgConflict,[code=20016:class=config:scope=internal:level=medium] ErrConfigSyncerCfgConflict,[code=20017:class=config:scope=internal:level=medium], "Message: syncer-config-name and syncer should only specify one, Workaround: Please check the `syncer-config-name` and `syncer` config in task configuration file." ErrConfigReadCfgFromFile,[code=20018:class=config:scope=internal:level=medium], "Message: read config file %v" ErrConfigNeedUniqueTaskName,[code=20019:class=config:scope=internal:level=medium], "Message: must specify a unique task name, Workaround: Please check the `name` config in task configuration file." -ErrConfigInvalidTaskMode,[code=20020:class=config:scope=internal:level=medium], "Message: please specify right task-mode, support `full`, `incremental`, `all`, Workaround: Please check the `task-mode` config in task configuration file." +ErrConfigInvalidTaskMode,[code=20020:class=config:scope=internal:level=medium], "Message: please specify right task-mode, support `full`, `incremental`, `all`, `dump`, `load`, Workaround: Please check the `task-mode` config in task configuration file." ErrConfigNeedTargetDB,[code=20021:class=config:scope=internal:level=medium], "Message: must specify target-database, Workaround: Please check the `target-database` config in task configuration file." ErrConfigMetadataNotSet,[code=20022:class=config:scope=internal:level=medium], "Message: mysql-instance(%s) must set meta for task-mode %s, Workaround: Please check the `meta` config in task configuration file." ErrConfigRouteRuleNotFound,[code=20023:class=config:scope=internal:level=medium], "Message: mysql-instance(%d)'s route-rules %s not exist in routes, Workaround: Please check the `route-rules` config in task configuration file." diff --git a/dm/checker/checker.go b/dm/checker/checker.go index 9f4ef5ecdf5..530e1c5fdf3 100644 --- a/dm/checker/checker.go +++ b/dm/checker/checker.go @@ -502,6 +502,7 @@ func (c *Checker) Init(ctx context.Context) (err error) { newLightningPrecheckAdaptor(targetInfoGetter, info), cpdb, pdClient, + targetDB, ) if _, ok := c.checkingItems[config.LightningFreeSpaceChecking]; ok { diff --git a/dm/config/helper.go b/dm/config/helper.go index 49badba0169..36fed03edeb 100644 --- a/dm/config/helper.go +++ b/dm/config/helper.go @@ -26,7 +26,7 @@ func HasDump(taskMode string) bool { // HasLoad returns true if taskMode contains load unit. func HasLoad(taskMode string) bool { switch taskMode { - case ModeAll, ModeFull, ModeLoadSync: + case ModeAll, ModeFull, ModeLoad, ModeLoadSync: return true default: return false diff --git a/dm/config/helper_test.go b/dm/config/helper_test.go index c7cc677c5c7..a644e8c1a7d 100644 --- a/dm/config/helper_test.go +++ b/dm/config/helper_test.go @@ -36,6 +36,10 @@ func TestTaskModeHasFunction(t *testing.T) { require.False(t, HasLoad(ModeDump)) require.False(t, HasSync(ModeDump)) + require.False(t, HasDump(ModeLoad)) + require.True(t, HasLoad(ModeLoad)) + require.False(t, HasSync(ModeLoad)) + require.False(t, HasDump(ModeLoadSync)) require.True(t, HasLoad(ModeLoadSync)) require.True(t, HasSync(ModeLoadSync)) diff --git a/dm/config/subtask.go b/dm/config/subtask.go index 5c28bf88b7e..86ea1da5fab 100644 --- a/dm/config/subtask.go +++ b/dm/config/subtask.go @@ -28,6 +28,7 @@ import ( "time" "github.com/BurntSushi/toml" + "github.com/google/uuid" extstorage "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/pkg/util/dbutil" "github.com/pingcap/tidb/pkg/util/filter" @@ -52,6 +53,7 @@ const ( ModeFull = "full" ModeIncrement = "incremental" ModeDump = "dump" + ModeLoad = "load" ModeLoadSync = "load&sync" DefaultShadowTableRules = "^_(.+)_(?:new|gho)$" @@ -178,17 +180,19 @@ type SubTaskConfig struct { ExtStorage extstorage.ExternalStorage `toml:"-" json:"-"` MetricsFactory promutil.Factory `toml:"-" json:"-"` FrameworkLogger *zap.Logger `toml:"-" json:"-"` - // members below are injected by dataflow engine, UUID should be unique in - // one go runtime. + // members below are injected by dataflow engine + // UUID should be unique in one go runtime. // IOTotalBytes is used build TCPConnWithIOCounter and UUID is used to as a // key to let MySQL driver to find the right TCPConnWithIOCounter. - UUID string `toml:"-" json:"-"` - IOTotalBytes *atomic.Uint64 `toml:"-" json:"-"` + // It will meter TCP io usage to downstream of the subtask + UUID string `toml:"uuid" json:"-"` + IOTotalBytes *atomic.Uint64 `toml:"io-total-bytes" json:"io-total-bytes"` - // meter network usage from upstream + // DumpUUID as same as UUID + // DumpIOTotalBytes meter TCP io usage from upstream of the subtask, other same as IOTotalBytes // e.g., pulling binlog - DumpUUID string `toml:"-" json:"-"` - DumpIOTotalBytes *atomic.Uint64 `toml:"-" json:"-"` + DumpUUID string `toml:"dump-uuid" json:"-"` + DumpIOTotalBytes *atomic.Uint64 `toml:"dump-io-total-bytes" json:"dump-io-total-bytes"` } // SampleSubtaskConfig is the content of subtask.toml in current folder. @@ -212,6 +216,14 @@ func (c *SubTaskConfig) SetFlagSet(flagSet *flag.FlagSet) { c.flagSet = flagSet } +// InitIOCounters init io counter and uuid for syncer. +func (c *SubTaskConfig) InitIOCounters() { + c.IOTotalBytes = atomic.NewUint64(0) + c.DumpIOTotalBytes = atomic.NewUint64(0) + c.UUID = uuid.NewString() + c.DumpUUID = uuid.NewString() +} + // String returns the config's json string. func (c *SubTaskConfig) String() string { cfg, err := json.Marshal(c) @@ -222,6 +234,10 @@ func (c *SubTaskConfig) String() string { } // Toml returns TOML format representation of config. +// Note: The atomic.Uint64 fields (IOTotalBytes and DumpIOTotalBytes) are not +// encoded in the TOML output because they do not implement the necessary +// marshaling interfaces. As a result, these fields will not be included in +// the TOML representation. func (c *SubTaskConfig) Toml() (string, error) { var b bytes.Buffer enc := toml.NewEncoder(&b) @@ -242,6 +258,9 @@ func (c *SubTaskConfig) DecodeFile(fpath string, verifyDecryptPassword bool) err } // Decode loads config from file data. +// Note: The atomic.Uint64 fields (IOTotalBytes and DumpIOTotalBytes) will not +// be populated from the TOML data since they cannot be decoded by toml.Decode(). +// As a result, these fields will remain uninitialized (zero value) after decoding. func (c *SubTaskConfig) Decode(data string, verifyDecryptPassword bool) error { if _, err := toml.Decode(data, c); err != nil { return terror.ErrConfigTomlTransform.Delegate(err, "decode subtask config from data") @@ -329,8 +348,9 @@ func (c *SubTaskConfig) Adjust(verifyDecryptPassword bool) error { c.MetaSchema = defaultMetaSchema } - // adjust dir, no need to do for load&sync mode because it needs its own s3 repository - if HasLoad(c.Mode) && c.Mode != ModeLoadSync { + // adjust dir. Do not do this for both load and load&sync mode, as they are standalone + // mode and should take LoaderConfig.Dir as is + if HasLoad(c.Mode) && c.Mode != ModeLoadSync && c.Mode != ModeLoad { // check isS3 := storage.IsS3Path(c.LoaderConfig.Dir) if isS3 && c.ImportMode == LoadModeLoader { @@ -495,6 +515,12 @@ func (c *SubTaskConfig) Clone() (*SubTaskConfig, error) { if err != nil { return nil, terror.ErrConfigTomlTransform.Delegate(err, "decode subtask config from data") } - + // Manually copy atomic values for atomic.Uint64 doesn't implement TOML marshaling interfaces + if c.IOTotalBytes != nil { + clone.IOTotalBytes = atomic.NewUint64(c.IOTotalBytes.Load()) + } + if c.DumpIOTotalBytes != nil { + clone.DumpIOTotalBytes = atomic.NewUint64(c.DumpIOTotalBytes.Load()) + } return clone, nil } diff --git a/dm/config/subtask_test.go b/dm/config/subtask_test.go index ab0956e403a..c750446ff30 100644 --- a/dm/config/subtask_test.go +++ b/dm/config/subtask_test.go @@ -16,7 +16,9 @@ package config import ( "context" "crypto/rand" + "encoding/json" "reflect" + "sync" "testing" "github.com/DATA-DOG/go-sqlmock" @@ -27,6 +29,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/stretchr/testify/require" + "go.uber.org/atomic" ) func TestSubTask(t *testing.T) { @@ -345,3 +348,74 @@ func TestFetchTZSetting(t *testing.T) { require.NoError(t, err) require.Equal(t, "+01:00", tz) } + +func TestSubTaskConfigMarshalAtomic(t *testing.T) { + var ( + uuid = "test-uuid" + dumpUUID = "test-dump-uuid" + ) + cfg := &SubTaskConfig{ + Name: "test", + SourceID: "source-1", + UUID: uuid, + DumpUUID: dumpUUID, + IOTotalBytes: atomic.NewUint64(100), + DumpIOTotalBytes: atomic.NewUint64(200), + } + require.Equal(t, cfg.IOTotalBytes.Load(), uint64(100)) + require.Equal(t, cfg.DumpIOTotalBytes.Load(), uint64(200)) + + var wg sync.WaitGroup + for i := 0; i < 10; i++ { + wg.Add(1) + go func() { + defer wg.Done() + + data, err := json.Marshal(cfg) + require.NoError(t, err) + jsonMap := make(map[string]interface{}) + err = json.Unmarshal(data, &jsonMap) + require.NoError(t, err) + + // Check atomic values exist and are numbers + ioBytes, ok := jsonMap["io-total-bytes"].(float64) + require.True(t, ok, "io-total-bytes should be a number") + require.GreaterOrEqual(t, ioBytes, float64(100)) + + dumpBytes, ok := jsonMap["dump-io-total-bytes"].(float64) + require.True(t, ok, "dump-io-total-bytes should be a number") + require.GreaterOrEqual(t, dumpBytes, float64(200)) + + // UUID fields should not be present in JSON + _, hasUUID := jsonMap["uuid"] + _, hasDumpUUID := jsonMap["dump-uuid"] + require.False(t, hasUUID, "UUID should not be in JSON") + require.False(t, hasDumpUUID, "DumpUUID should not be in JSON") + }() + + wg.Add(1) + go func() { + defer wg.Done() + + newCfg, err := cfg.Clone() + require.NoError(t, err) + + // Check atomic values exist and are numbers + require.GreaterOrEqual(t, newCfg.IOTotalBytes.Load(), uint64(100)) + require.GreaterOrEqual(t, newCfg.DumpIOTotalBytes.Load(), uint64(200)) + require.Equal(t, newCfg.UUID, uuid) + require.Equal(t, newCfg.DumpUUID, dumpUUID) + }() + + wg.Add(1) + go func() { + defer wg.Done() + cfg.IOTotalBytes.Add(1) + cfg.DumpIOTotalBytes.Add(1) + }() + } + wg.Wait() + + require.Equal(t, cfg.IOTotalBytes.Load(), uint64(110)) + require.Equal(t, cfg.DumpIOTotalBytes.Load(), uint64(210)) +} diff --git a/dm/config/task.go b/dm/config/task.go index 8cedfbd5e65..9f5a77324f0 100644 --- a/dm/config/task.go +++ b/dm/config/task.go @@ -669,7 +669,7 @@ func (c *TaskConfig) adjust() error { return terror.ErrConfigNeedUniqueTaskName.Generate() } switch c.TaskMode { - case ModeFull, ModeIncrement, ModeAll, ModeDump, ModeLoadSync: + case ModeFull, ModeIncrement, ModeAll, ModeDump, ModeLoad, ModeLoadSync: default: return terror.ErrConfigInvalidTaskMode.Generate() } @@ -774,9 +774,9 @@ func (c *TaskConfig) adjust() error { instanceIDs[inst.SourceID] = i switch c.TaskMode { - case ModeFull, ModeAll, ModeDump: + case ModeFull, ModeAll, ModeDump, ModeLoad: if inst.Meta != nil { - log.L().Warn("metadata will not be used. for Full mode, incremental sync will never occur; for All mode, the meta dumped by MyDumper will be used", zap.Int("mysql instance", i), zap.String("task mode", c.TaskMode)) + log.L().Warn("metadata will not be used. for Full/Dump/Load mode, incremental sync will never occur; for All mode, the meta dumped by MyDumper will be used", zap.Int("mysql instance", i), zap.String("task mode", c.TaskMode)) } case ModeIncrement: if inst.Meta == nil { diff --git a/dm/config/task_converters.go b/dm/config/task_converters.go index 2c34e3838bd..98a9373e066 100644 --- a/dm/config/task_converters.go +++ b/dm/config/task_converters.go @@ -101,6 +101,8 @@ func TaskConfigToSubTaskConfigs(c *TaskConfig, sources map[string]dbconfig.DBCon cfg.CleanDumpFile = c.CleanDumpFile + cfg.InitIOCounters() + if err := cfg.Adjust(true); err != nil { return nil, terror.Annotatef(err, "source %s", inst.SourceID) } @@ -308,6 +310,8 @@ func OpenAPITaskToSubTaskConfigs(task *openapi.Task, toDBCfg *dbconfig.DBConfig, if task.IgnoreCheckingItems != nil && len(*task.IgnoreCheckingItems) != 0 { subTaskCfg.IgnoreCheckingItems = *task.IgnoreCheckingItems } + // set syncer IO total bytes counter + subTaskCfg.InitIOCounters() // adjust sub task config if err := subTaskCfg.Adjust(true); err != nil { return nil, terror.Annotatef(err, "source name %s", sourceCfg.SourceName) diff --git a/dm/config/task_converters_test.go b/dm/config/task_converters_test.go index 24eb74a09c0..fc368db2384 100644 --- a/dm/config/task_converters_test.go +++ b/dm/config/task_converters_test.go @@ -16,6 +16,7 @@ import ( "fmt" "testing" + "github.com/google/uuid" "github.com/pingcap/check" "github.com/pingcap/tidb/pkg/util/filter" "github.com/pingcap/tiflow/dm/config/dbconfig" @@ -117,6 +118,13 @@ func testNoShardTaskToSubTaskConfigs(c *check.C) { c.Assert(subTaskConfig.BAList, check.DeepEquals, bAListFromOpenAPITask) // check ignore check items c.Assert(subTaskConfig.IgnoreCheckingItems, check.IsNil) + // check io total bytes counter and uuid + c.Assert(subTaskConfig.IOTotalBytes, check.NotNil) + c.Assert(subTaskConfig.DumpIOTotalBytes, check.NotNil) + c.Assert(subTaskConfig.IOTotalBytes.Load(), check.Equals, uint64(0)) + c.Assert(subTaskConfig.DumpIOTotalBytes.Load(), check.Equals, uint64(0)) + c.Assert(subTaskConfig.UUID, check.HasLen, len(uuid.NewString())) + c.Assert(subTaskConfig.DumpUUID, check.HasLen, len(uuid.NewString())) } func testShardAndFilterTaskToSubTaskConfigs(c *check.C) { diff --git a/dm/config/task_test.go b/dm/config/task_test.go index bb90c5b83e0..fd41681df56 100644 --- a/dm/config/task_test.go +++ b/dm/config/task_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" bf "github.com/pingcap/tiflow/pkg/binlog-filter" "github.com/stretchr/testify/require" + "go.uber.org/atomic" ) var correctTaskConfig = `--- @@ -688,6 +689,8 @@ func TestGenAndFromSubTaskConfigs(t *testing.T) { ValidatorCfg: validatorCfg, CleanDumpFile: true, EnableANSIQuotes: true, + IOTotalBytes: atomic.NewUint64(0), + DumpIOTotalBytes: atomic.NewUint64(0), } ) diff --git a/dm/ctl/master/start_task.go b/dm/ctl/master/start_task.go index 5a0d2fbd51f..2b27f18816a 100644 --- a/dm/ctl/master/start_task.go +++ b/dm/ctl/master/start_task.go @@ -60,13 +60,6 @@ func startTaskFunc(cmd *cobra.Command, _ []string) error { if yamlErr != nil { return yamlErr } - if task.TargetDB != nil && task.TargetDB.Security != nil { - loadErr := task.TargetDB.Security.LoadTLSContent() - if loadErr != nil { - log.L().Warn("load tls content failed", zap.Error(terror.ErrCtlLoadTLSCfg.Generate(loadErr))) - } - content = []byte(task.String()) - } lines := bytes.Split(content, []byte("\n")) // we check if `is-sharding` is explicitly set, to distinguish between `false` from default value @@ -95,6 +88,14 @@ func startTaskFunc(cmd *cobra.Command, _ []string) error { return errors.New("please check output to see error") } + if task.TargetDB != nil && task.TargetDB.Security != nil { + loadErr := task.TargetDB.Security.LoadTLSContent() + if loadErr != nil { + log.L().Warn("load tls content failed", zap.Error(terror.ErrCtlLoadTLSCfg.Generate(loadErr))) + } + content = []byte(task.String()) + } + sources, err := common.GetSourceArgs(cmd) if err != nil { return err diff --git a/dm/errors.toml b/dm/errors.toml index 7f4c65fcf2b..d78d858038e 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -929,7 +929,7 @@ workaround = "Please check the `name` config in task configuration file." tags = ["internal", "medium"] [error.DM-config-20020] -message = "please specify right task-mode, support `full`, `incremental`, `all`" +message = "please specify right task-mode, support `full`, `incremental`, `all`, `dump`, `load`" description = "" workaround = "Please check the `task-mode` config in task configuration file." tags = ["internal", "medium"] diff --git a/dm/master/openapi_controller.go b/dm/master/openapi_controller.go index 9255af64030..bb74026a8e2 100644 --- a/dm/master/openapi_controller.go +++ b/dm/master/openapi_controller.go @@ -563,6 +563,7 @@ func (s *Server) getTaskStatus(ctx context.Context, taskName string) ([]openapi. // add load status if loadS := subTaskStatus.GetLoad(); loadS != nil { openapiSubTaskStatus.LoadStatus = &openapi.LoadStatus{ + Bps: loadS.Bps, FinishedBytes: loadS.FinishedBytes, MetaBinlog: loadS.MetaBinlog, MetaBinlogGtid: loadS.MetaBinlogGTID, @@ -584,6 +585,8 @@ func (s *Server) getTaskStatus(ctx context.Context, taskName string) ([]openapi. SyncerBinlogGtid: syncerS.SyncerBinlogGtid, TotalEvents: syncerS.TotalEvents, TotalTps: syncerS.TotalTps, + IoTotalBytes: syncerS.IoTotalBytes, + DumpIoTotalBytes: syncerS.DumpIOTotalBytes, } if unResolvedGroups := syncerS.GetUnresolvedGroups(); len(unResolvedGroups) > 0 { openapiSubTaskStatus.SyncStatus.UnresolvedGroups = make([]openapi.ShardingGroup, len(unResolvedGroups)) @@ -601,10 +604,12 @@ func (s *Server) getTaskStatus(ctx context.Context, taskName string) ([]openapi. // add dump status if dumpS := subTaskStatus.GetDump(); dumpS != nil { openapiSubTaskStatus.DumpStatus = &openapi.DumpStatus{ + Bps: dumpS.Bps, CompletedTables: dumpS.CompletedTables, EstimateTotalRows: dumpS.EstimateTotalRows, FinishedBytes: dumpS.FinishedBytes, FinishedRows: dumpS.FinishedRows, + Progress: dumpS.Progress, TotalTables: dumpS.TotalTables, } } diff --git a/dm/master/openapi_view_test.go b/dm/master/openapi_view_test.go index aa144edc169..aab897e28cc 100644 --- a/dm/master/openapi_view_test.go +++ b/dm/master/openapi_view_test.go @@ -1056,6 +1056,9 @@ func (s *OpenAPIViewSuite) TestTaskAPI() { s.Equal(float64(0), resultTaskStatus.Data[0].DumpStatus.CompletedTables) s.Equal(int64(1), resultTaskStatus.Data[0].DumpStatus.TotalTables) s.Equal(float64(10), resultTaskStatus.Data[0].DumpStatus.EstimateTotalRows) + s.Equal(int64(0), resultTaskStatus.Data[0].DumpStatus.Bps) + s.Equal(float64(0), resultTaskStatus.Data[0].DumpStatus.FinishedBytes) + s.Equal(float64(5), resultTaskStatus.Data[0].DumpStatus.FinishedRows) // get task status with source name taskStatusURL = fmt.Sprintf("%s/%s/status?source_name_list=%s", taskURL, task.Name, source1Name) diff --git a/dm/openapi/gen.server.go b/dm/openapi/gen.server.go index 6f465412eb0..8e3c3258821 100644 --- a/dm/openapi/gen.server.go +++ b/dm/openapi/gen.server.go @@ -1259,103 +1259,104 @@ var swaggerSpec = []string{ "1wptcdRTDBdHSfo6pC5COeWTkmvEJM9CfvUJfc2R5qL63grIr/pYSk6gGAnyqyCkJMbLIMaJg2j6IZAP", "ASZgDdMExJSlUICVEBk/nk4jGvK9DJNlCLO9kKbTf6+mAkeLKRdwkaCpXGSi58kZlPNO5HSTOE+SPSfZ", "+jDnGSUc/SlRtzlGoeOA1MkbDEGBLhQHeVlDM1gfhfQkltry8fykn+nNin6IH4iVXZRzLXqCudyYTyiB", - "a2vZhh4M5R9SEXFBMwABk8MBM+PHDSgtKpWKvV+fv4cpOpejnQx/kqfZhfJDHCqz9E+iPM1ATnAbJrls", - "ggSKAsWI6jfNu6PjUUTzRYKqvSN5upC+3HiEuMApFCgQVMAkYPRm6JsxJpivUBQs1gJt/NIGC2nIHFhh", - "Io4OR70eau39cZtQLVSaYLqp5GK2U7IZr0EmeplNPQ0WmCR0GSwFjpz8wQQmS/D28uykMOZ5xgVDMAX6", - "1ZqxQy/hPA739yconL2YzOfo5WSxD8PJbP9wH4bz+Ww2OzieT56/OHw5Go9IniQSr4bLWpnIGogeq1+A", - "KPWZsv0DwNSGf4HJ3kz+b384LBE23k4M80Tyyt5UP9BL1GGTYESYoVBQtgY3K8SQAk3vS0KXAHOpGCQ/", - "DYBgG9rhlDHK/o7F6h3i3OnrSJZR9gYgObbFRurXIJRuT+td9QyE2iVqStPYvJrype/N1ADVZxuqicY2", - "PC5JeouE8WjPSEz9DkCoBwUusTDPAJbbVmqN3Kc2pKYZ5vI3w6YmnhZQ3bjpgERuux/DCAo4OHKoR9uO", - "AEcpMGVpByhNKSly9W4kNP8+PBImlNkyEtr3eUDoK2dq+2Brh+FBATc+yJbBlz7cA9K8dPG3DPI7vGTK", - "hWVLJPgDAl+b+DEweVjOyRfVnI8B/aU0wBeC5aHIGfJjoQEMQhV4BPxrUg9q3nw6fXV5Ci5fvT4/BV/E", - "/Av4yxccfQGYiL/M57+A9x8uwfvfz8/Bq98vPwRn7998On13+v5y/PHT2btXn/4B/vv0H/qNX8D018v/", - "90+j91EUYBKhPz6DN+e/X1yefjo9Ab9OfwGn79+evT/96xkh9OQ1ODn97dXv55fgzd9efbo4vfxrLuIX", - "6eIQvPlwfv7q8rT4t3SrXGkJg1o7UosWzkSJcnYdw9Xv8wGRafl6MZdFVedWNZJ3D56ePpjNZvdOT59T", - "GPWHXQmFkTvs6oiC/H5GigQ07rIlFBWq1vPS42/Tg9ElQ5w7H+o4ZThMDaq1AiJ7PmvpOioOwF0kb2Rh", - "78sXvnT5IB56sX8076WG4fo+VvqgPHDUnbAKVyi8ChjiKixpclzG0ESNAGaEHQ1VDzEHGeQcRXvALer3", - "SaKM6zD2YNrUxL1Br45TEFA6xBv0xknOV7UITgdb9Vn/zrBAXMVqGi+dSkZAYZBRTATg8hcowMk7EEKi", - "JRkLAGMZGTBUxqXytSL91jqS4V+TIKREIOLAjX9NwJrm4AYSYWFY2zuHpQFfwnllagprIM3NGHwJ9/2P", - "DtyP7mFf/tNpYNYkbCP7exbBguY0EzjFXOAQ8BVkkSSj1ADSeoMbLFY64262hpJkDXKOIhlhEwBNoApo", - "GOaMA0y8c56cnIO0FpyWW9NMPlr75GJcx1nNNk5N72+WPubMFeRXGYlQ4p9nIKMJDteglnFux/5/ZJgZ", - "v66Qp1lTmNQgnUEQWOdnyuXs+LkwIZ48iGXm5J/sWjt+5boHR7PW0pcrBIrBUoIyxDCNcAiTZA2Myovb", - "KRmNVjQGZnJwDZMcHQO1hGQojkJKIn436BlKISYBz2CIahjMnzXhf4cJTvMUxAwhEGF+BdRbCoa3r++y", - "/K2PJx40j/2Iebu+PF1tzQyFOF4b4Hm+sLJzMWWgBfYeOIsBoQLoN7HkCXXmLlWVAJQgcIOTBCyQUkB7", - "4EJBas52jsE+RM+PDg8OJ/Hzl/FkPkcvJosI7RfpUOlovtCozPsTgA1Jb9PYJe9qW98oIW7TQ1k0fTRV", - "CGVbxFXmOdAPK4/QsmE/88g7lUe+9XFJf7Riq+06l5jqiSr0qE/RoGFxEKrFRBuWiqh/aVB1Pgbzl89f", - "/uIS9tq6HuZz8dw9mK2budwgaMIVVRASoIcHIIQiXAV5FqRlRVQdiJuV9FCYVOJqLMgz7UyVu2OFXz4x", - "d+rVzfizwntvyvOFmtLlJrpLLwoiaq6sTfcpJ0S+3Kc568zqZCIbXdcO+4hegO1SxRfKXS2PY9pypt1Z", - "pXvU8c64SpL1Z2EaibELFOYMi3V7GeVEm1oZzpO6h6fNW4xREpWWbYWjCBHtXC+RKIMae6LaJCBmNFVD", - "lO8VSz+nrZYa4StiIoBJQm9QFISkDfYbmqaUgPdGM19cnAP5Do5xCHXyoCRWL3E4T4IQ+gMva2KtqoqR", - "Nrc5eVZOLDHxTv2bNZ3E4+PpO+MtTP/n2exlURXSQK1/1Su09i/6plpP7krG8LVE7Qqty5IUa/Ge9ZqR", - "UZ2WDhq0AXRKhwnK3jKaZ460cZS0S916NzrGjIsgoaG2Mq5XZDSKos2mFTqb7hqak80nbCVL1OzjCucW", - "IiXY1oJOopZVOq5SOY+vV/NLYpjwVnqktCQqCtcaQIZN6vWaijevt62JcSsrczloPSrdbO1EymgulwpK", - "aWWudY7LvHtBiBN4TR3WTP9e1vWVtGq4fS5JLEJ8Z2GiqYl0Fz46MwCQ8xvKIu+M5YD6lAeHz46GeKJF", - "hsE9t3xozXtwMDtyRbNZkVDoLGVVgypXpYxHul6yQxcpqJZF6zwzKsbJdwbWiw6uCtVex2ZFt73Hn5Bf", - "DS/quIT8qirpGI9y7vL1DG7yYQs/RqkYWG0XODLUZsm6CBf/6tBCHY6PVbjrd3z0qMkw78cmuW+90oN0", - "1bP0F6Voh4irvJ90iW4YdfmeBc/zEphenq9Y5R78y1CW4BB6+LhRjtnOmplqZuNtJ2u7ohq5dOKGdZwF", - "Z9mAOHlHhuWdlZ0MpfQaBSnS58qDLYl+T+WVlSu7gFx5QhG9ISYeKn52p+5hjIKURigQOEVBVORI29ER", - "ThEoHkuzIt8s8s6W3p5xp8apyDVIPzSETessJhSQDtggvzLlgmqADdD+bHY0mc0ns30wf3Y8OzyePRtW", - "Yn0haNa5ZffHSQJLczGY6jcQ67hF40uzOumf8YGY1eoR2k5qnmYDBd2qyt2gEG6wzkkojAZCYh1UW4ee", - "DjYpzug7OLRPSfmD/D6Td6EGGn99IGYXaxJWmKlTdjdm8hFQsNlcoc6pxi4XnyFOk2sUBcpDp+FV4DlK", - "71SzxYURJ2ncJ8V+3VmQ0uDpVKUVOTpyfBJrd0WCyX/oeR3ILiQlMFlKqriWsE/dblY4XJUJMcxB8fJG", - "cXwr6zgwP+gw0SEiIhDZ0EILcwAULNAKk8hKuQ15twwQHUZFPuvEqDbCj5Guq0DXxR3PAXCZcvbBNLDk", - "YCmD9q491wMa2w4ZAjmZFLPYW98p1rVMQW80bRPCRrK26+NhScH69jg3oykHLjpZ4bstVD62cgmzKo+4", - "by7RV6LVlrRLU/nRVp4+NRHjRNKP5TqhAKMIy7dg8rE2uk/vv8bknC5/U5N9knO5zDIiK0hCFOhbt0FR", - "nLeCZIl6az0sl1DHMIDnmYx01JGgKh3Ql3mjKAFZki8xGXLZFi8JZShQh8ySGUryNy70qmEgY8gcR6th", - "zt26Rozr5E+/YkQCGjLUT5midKIc5SYRHE6vQp8LyorqC++BTTWpt4bK707Y3Miv3OEdJUGUq3BGOGZb", - "0Ru5eStIIp1bjRMcChQpTFRkmqf6wDRLdCq6uMigiW/Jl6VmpZJR7r37uOMGrtVBCqVSF0GBpFmzFssQ", - "56beZDQeVcUn7sW0WR+WFlHekHrByo3cJS3RV1sr4QtFUMEeNIkyMNIqxErN1yrEaUVCPonSxaGpLmgu", - "FUuTs+RKZgxQY8bDi6WVUjUV0w1l08j9brBXuvT6BAr4WoaSRcLHzVoF5AVNDDfFeZJIREjIUIqIrmWG", - "6jcZc0imquRI/z7In6wg6VGiDRlsksG5OU2+dpsxh4p3HZoIpPSQnJgDKIqD5ARdo6RlgozuVUbfEVHJ", - "nwt336OWa2NqpAVRmgxRwQYGUxreLuzLoBCIqZIabSr9wPiGV3D97wlTIW3/QYNzB37Lk8SwvdQpvvvL", - "VgpDMmQpZpKL2nk0SGCy/rdLRqk6AGM00SVYPE/llNlqzXEIE4DTIvdcKm7DuFqRSidC/hnHdb63nrXo", - "UCz0RKChacYQ55Or60kGMePdYJnR4OoaqNFu+ByrEI65QCRcd85fmDNMjH+uDnZ1GRxl0pDG6hJfORuA", - "nOdMKou6cOSCuuCQ03nqsQRlcKlqB9ruwN60WD8whrw9M+ZXwdecFpm/mkeD+RVQzxT4jv0sV3oxe+ua", - "XS8fiBVDMKrXOB42rZ2SB/2C3J2QEhPkOCMnDYPPu6h2Ro9TxqAUuhY/JnQpEZPyZ3CsM2L1vIWhgcOF", - "4fzIiaKBqB9F21wEBQh9XFi8IZWMiplynbxbIQJShEQ5AAFGb7jaWDO3S0797p51GFKO6vQ6g5K228DB", - "r2wIJUgFoyRv7qt51D7Vi4LeniyZsyEJk6Y+KDe2UBu+ul4LcfUmsN7sVFF2LoSqojuPetAPS/XQK8h7", - "U/mKu6DIZ/vOSMg2s32WF+YxfZKpggUUYf2mwbxdoGzPxdckXDFK8L/LpdQcAP2BQs1F0hP4mkMisFrK", - "XV2cJQMluolIr1j7aFi/ZOgO9ypnQV1xbNHM+IpV0Npb8mTeEEXNghVJ+m6qKZ91gyXMG0OXcJ90mfUa", - "ADfBaSzmc5b9KZ8yqO5M+PCrwfmeKshsn3Q00o/VCrODOJztHx1M9l+EzyfzOXo+gUfPDiZH4Wzx4jB6", - "9jI+mB3PJ89nh/PD/YPx7Nnh88PoILSGvzh4tj/Znx1Ei/3Doyg6iI7nk/nzmbNHU72g1+q5pB5UldW+", - "NzNaJ9ChW0dt5RC241jUt/m1sN8DyoShBEqnrfvmhrTmZbgWmj3uC2WbccKtDkk3nqepc+spEC+RmxgN", - "justTu5LF9tweLehOLQqrPSFoFmmIoKqBPU3c9PRmfBxJhv8VdM6qyGofTZt5zj4wCRsw6FTD9UEBf86", - "VIZ8PKzogncWmw3kSztp6Uloj8ENTqIQsqjI1NazkYvJr/c8pmwVnfiOL0VVL9fOQg2AVThh7SyYsMyF", - "z04Ijx2uuOchNyOiiOs7MiZtXmDMG9syvyMFBy7gs8gN8gzvKuZI3nWQtMqbd9P0SZUIbqck8C6Velsq", - "Y3MWrpU08e46SjMpH94CFnqN2A3DYrM8ePmW9raFWaX8o/8aarVuP+i+i+IxxInqUcav2gcGHaVwztvg", - "pTrtbz9YKLBqUqfuahqVPAwR5x5wNyusbs81blPDBZS+m/ygHRGHqyG9+CM3N2y0DuuqXekIN/w1ge2N", - "rlb0XkI1t005KKyXoKZOkXd1UuyrvLlDDWNf1WKjz+7Dd6LwdordaiuKW5UVFVIZJyc0dKStT96BDxki", - "rz6egZMPb6TKZcnoeNTX5HQijedEu7SYEtPzVMcXMVUsjoVCvLVAcSp+PDqSBFSJuwwRmOHR8ehA/SQ1", - "vlgpaKcww9Pr+dQ01JkW0xt/qex1dxaptV59PKv3i1NVI1qzqvn2ZzPdI7O8eQOzMv83/RfXlYmVH9XZ", - "lNrdmU5RvWEWtSJTm8jzNIVsPTqWOICyMx2JKeB5uAKQg1q7OgGX3GolN/qsavh92Gvl0ySAEsPXNFo/", - "GO7txnctpM2yYCHXvX3C+5ArmtW2Ys9J+Ntxix91xQ8fypJVm7/HYUxHW8EusoxHhw8IRqtVpWNpbc47", - "BMPqQF4Yrk02ZvpN/6Eiwlut/xKk/UDHTn2I4wQTpMn2Xp+zZ5DBFOld/mfr/N8Cr4jJVf8eKFajwhCM", - "LBhGthrXlROu/Ka/0f/nFuMcOvzwJ7ajVNO10U9+0EYWDsNACat6UD6OhDl6Xu6YhFl98DeSMLMx02/G", - "C9tIwoz3OEDCbPD8EmbB8GNLWP2rBp0bGaV7BXBOyXqLxAkN/+viw3uPKNXBknOVF6/b7BbREKjlKqgi", - "GjYgMj5qBzh/u3x3PggcObAHnJXQpUE+cHSQ1696qs6xfcws5au4gKtaOZR32hRPf80RW1tMjcUqKEc4", - "mNhdeXc7dnzdZg0YEjnTvbV0gd/EtNUp7oa5QKh1k9kEhs/b1b6OZr0OSbE7HiRFS+0GHzSHVPxQxPgq", - "RuO+/be/vrAtZ9vxgYfNHe75g8FT5kSevJ3TnUkBJFFR1AoBQTf2rrs2vK0Dpt+sk4V+K3eiHpZM0akT", - "lgldqP5mOcFf83qbDr/Bqx90DDJ43mvSbYURU33hlmYFJDDhppdY0ShGJXRMOYVLdag57qkzdsDwaj4A", - "sI+nxkNsyC7yyuPYtG3akw59VrZwP3TyoqE8FSBWX41q25cuhuhL4+wMT3zejt1zpfFv64lQCe7t92GN", - "J6aHTBYL3te2TSP9nSSVBPe7PeZrSrvFon0xw5OzLZrID7CpVaegjj3VHy36uaXb3NLSDb3vjqqQbDNh", - "/VQ0DP0xzYnrA3C3xp7sqmaoOjbGOdE9f4tbsA/DYBsojh+cvRyffNtV7jJKauvMVfYi6+Ctqtn1j8ta", - "7Ybfw93gp81pigNqfYo35yXrc+gDQmzd1XVIsnYLrOPvibbdALfeyXZHDqiKtm+6eNWXnB3KHtNv+o8q", - "gzeAWVTN99PjlXFHga9n+Qr3gcs763+3yqX1Fim7xaS6/vnuPFq2eRqiwco+iE/HGnZenHmUs6DGd+x2", - "hH3sj+1XPb0ewsMSDBIe6yLtDvfq0gz70XON7XLWP4uLVTBCqaoogPoDNbpWoIe79BFPn2YqPuPZy0CS", - "5yG/eszTb3NvarEuuknqvnuuNYtnQw1W2eewa1WHfDSXbfbXHG+UnrZs5pZVbetrrQ4mVEROTN/Pp6No", - "S6gqdtfV9EOO9y9147TtHe7b1wW+59G+69OFO3TOX364r77DTXU2DSm5Rqyo3O3afj1wm/tfgNLDAjjW", - "PIw5wCTLhW52b3Sp/vBHgZVu+wz5lenkpD8aQRm4xiEC14hxuFUmaqC0O2x0qQqkFJWJ6Zxtvu9BYwCb", - "H01pEXVvAOcVd8eGmdTidtgj1LPuuGovL+fdS8dfVjf7tiHr5k7X91PvPgCeqD6v7ewmwjU1zWa6lfuZ", - "GvRI+968o7o5G+xvCZ7d0c+mm9Xd2eKb6lm6SQ1fgzs2io7ttqmOsLiEZWBQ7Ou3utN1c/6b1U0FPthY", - "7s42zX44xd62111b7i2Qq+5Y/9z0nSlNG7rvLf19N639VDmiq9hawYCuEQE4Vp8NATxfFGEfK3sV/Sy3", - "9kX6A8zEzvDFI+RKv4d2agSRh77OeB1F1f7d7yupfsoMsNUq6vslGGc/eoKxrK4emGC0TJbnfK7owVf0", - "1xySDqr17eQ7o8gevTjCecai++ObvuwjX9HDr8Nn1C30uydUY359/DPxNrfs3Mm4OquzqysgiUw3WvMD", - "o7kwd9Fw7WLx3aVycC1ZWUX2ei1p/YpEdztB/0GE8md1Wxd/u0vc7s3FG5a8lcVuP1n6ZxHezsqSsxLv", - "gUVJvrdI0IYpiUWCLgTLQ5GznzL11GRq7O9o6yN5wQGDae7+eN/up+9rksctFt80OfNTQn5KyPz7BEt1", - "5tv9YKlTDP1ZsjI981MUN178RxHEh09RWknBphz+uWqxtcRtaDa7vVYBe+tcLuSYHzDzXeK96/dx1Sbf", - "Mfk87GaR9WXZHVT2ZUvzXa+t39FLTOZaheaezbiTZr3Ki2Y/pO7SaO++6qKZX3Opj4+w62JH683n1zTf", - "i2gKMVGt50eS1GYCty4Y9XW7j2g4uMW96Wk//Zrj8GqiNPBEl6VOqq5gNR0zcnlmCu3tQnWDxWoSpRY8", - "atk2NEUX2HJc8cPt59v/CwAA//+dbnwDZLwAAA==", + "a2vZhh4M5R9SEXFBMwABk8MBM+PHDSgtKpWKvV+fv4cpOpejnQx/kqfZhfJDHCqz9E+iPM1ATnAbpkWm", + "/qPZVftrR4ejtvs2Vl5lggSKAsWy9dcimi8SVL1H8nShX0Nc4BQKFAgqYBIwejP0zRgTzFcoChZrgTZ+", + "aYOFMkaX0p4pTmrqcQ22A2UfpZocZr/voGILzyYObhKO1c5ZsLuY+JRsxsOQiV4mVk+DBSYJXQZLgSMn", + "3zGByRK8vTw7KZyEPOOCIZgC/WrNiKKXcB6H+/sTFM5eTOZz9HKy2IfhZLZ/uA/D+Xw2mx0czyfPXxy+", + "HI1HJE8SiVfDFa62rAaix5soQJR6UvkUA8DUDsUCk72Z/N/+cFgibLyoGOaJZJ69qX6gl6jDJsGIMEOh", + "oGwNblaIIQWa3peELgHmUuFIBhsAwTa0ziljlP0di9U7xLnTh5Iso+wYQHJsi43Ur0Eo3anWu+oZCLWr", + "1VZE+tWUL31vpgaoPptTTTS24XFJ0lskjKd8RmLqdyxCPShwiYV5BrDctlKN5H6NOzSUaIZjTTwtoLpx", + "04GO3HY/hhEUcHBEUo/iHYGT0miW3u3UolJS5OrdSGj+fXgkTIi0ZSS0T/WA0FdO2vbB1o7IgwJufJst", + "gy99wwekeRk6bBnkd3jJlGvMlkjwBwS+NvFjYPKwnJMvqjkfA/pLaYAvBMtDkTPkx0IDGIQqoAn416Qe", + "LL35dPrq8hRcvnp9fgq+iPkX8JcvOPoCMBF/mc9/Ae8/XIL3v5+fg1e/X34Izt6/+XT67vT95fjjp7N3", + "rz79A/z36T/0G7+A6a+X/++fRu+jKMAkQn98Bm/Of7+4PP10egJ+nf4CTt+/PXt/+tczQujJa3By+tur", + "388vwZu/vfp0cXr511zEL9LFIXjz4fz81eVp8W/pVrnSHQa1dgQYLZwJGOX9Ooar3+cDIt7y9WIui6rO", + "rWokBR887X0wm83unfY+pzDqD+cSCqP7hnMd0ZX/pRQJaBxrZ7BkPS9jgzbl+sOt4TA16NuKpez5rKXr", + "qDgA1/GVa4sa2eD78pEvbT+I517sH817aWKkpI/1PiiPHXUnzsIVCq8ChrgKY5ocmjE0USOAGWFHT9VD", + "zEEGOUfRHnCrhvskc8Z1GHswbWru3iBZxzUIKJ3jDZLjJOerWsSng7P6rH9nWCCuYjuNl05pI6AwyCgm", + "AnD5CxTg5B0IIdGSjwWAsYwkGCrjWPlakQZsHQ3xr0kQUiIQceDGvyZgTXNwA4mwMKztncMygS/hvDJN", + "hfWQ5mkMvoT7/kcH7kf3sEf/6TRIaxK2kf09i2BBc5oJnGIucAj4CrJIklHqAWntwQ0WK535N1tDSbIG", + "OUeRjMgJgCawBTQMc8YBJt45T07OQVoLZsutaSZBrX1yMa7jzGgbp7f3N2Mfc+ZKClQZjFDin2cgowkO", + "16CW+W7nCv7IMDN+YCFPs6YwqUE64yCwzueUy9nxdmFIPHkTy9jJP9m1dhTLdQ+OZq2lL1cIFIOlBGWI", + "YRrhECbJGhiVF7dTOBqtaAzM5OAaJjk6BmoJyVAchZRE/G7QM5RCTAKewRDVMJg/a8L/DhOc5imIGUIg", + "wvwKqLcUDG9f32X5Wx9PPGg+/RHzfH15vdqaGQpxvDbA83xhZfNiykAL7D1wFgNCBdBvYskT6uxfqioB", + "KEHgBicJWCClgPbAhYLUnDEdg32Inh8dHhxO4ucv48l8jl5MFhHaL9Kn0jF9oVGZ9ycMG5LeprFL3tW2", + "vlFC3KaHsmj6iKwQyraIq0x1oB9WfqFlw37mnXcq73zr45L+6MZW23UuMVUcVQBSn6JBw+JAVouJNiwV", + "Uf/SoOp8DOYvn7/8xSXstXU9zOfiuXswWzdzuUHQhCuqMSRADw9ACEW4CvIsSMvKrDoQNyvpoTCpxNVY", + "kGfamSp3xwrCfGLu1Kub8WeF996U5ws1pctNdJeAFETUXFmb7lNOiHy5T3PWmdXJRDa6rh32Eb0A26WK", + "L5S7Wh7ftOVMu7NK96jjoHGVVOvP2jQSaRcozBkW6/Yyyok2NTucJ3UPT5u3GKMkKi3bCkcRItq5XiJR", + "BjX2RLVJQMxoqoYo3yuWfk5bLTXCV8REAJOE3qAoCEkb7Dc0TSkB741mvrg4B/IdHOMQ6hRCSaxe4nCe", + "BCH0B17WxFpVFSNtbnPyrJxYYuKd+jdrOonHx9N3xluY/s+z2cuiOqWBWv+qV2jtX/RNtZ7clYzha4na", + "FVqXpTHW4j3rNSOjOi0dNGgD6JQOE5S9ZTTPHGnmKGmX3PVudIwZF0FCQ21lXK/IaBRFm00rdPbdNTQn", + "m0/YSpao2ccVzi1ESrCtBZ1ELauFXCV7Hl+v5pfEMOGt9EhpSVQUrjWADJvU6zUVb15vWxPjVlbmctB6", + "VLrZ2omU0VwuFZTSylzrHJd594IQJ/CaOqyZ/r2sLyxp1XD7XJJYhPjOAklTm+kuwHRmACDnN5RF3hnL", + "AfUpDw6fHQ3xRIsMg3tu+dCa9+BgduSKZrMiodBZUqsGVa5KGY90vWSHLlJQLYvWecZUjJPvDKxbHVyd", + "qr2OzYp/e49LIb8aXgRyCflVVQIyHuXc5esZ3OTDFn6MUjGw6i9wZKjNknURLv7VoYU6HB+rgNjv+OhR", + "k2Hej01y33qlB+mqf+kvYtEOEVd5P+kS3TDq8j0LnuclML08X7HKPfiXoSzBIfTwcaMstJ01M1XVxttO", + "1nZlN3LpxA3rSQvOsgFx8o4MyzsrTBlK6TUKUqTPoQdbEv2eyisrV3YBufKEInpDTDxU/OxO3cMYBSmN", + "UCBwioKoyJG2oyOcIlA8lmZFvlnknS29PeNOjVORa5B+aAib1llMKCAdsEF+ZcoL1QAboP3Z7Ggym09m", + "+2D+7Hh2eDx7NqzU+0LQrHPL7o+TBJbmYjDVbyDWcYvGl2Z10j/jAzGr1S+0ndQ8zQYKulUdvEHh3GCd", + "k1AYDYTEOti2Dj0dbFKc6XdwaJ+S8gf5fSbvQg00/vpAzC7WJKwwU6fybszkI6Bgs7lCnVONXS4+Q5wm", + "1ygKlIdOw6vAc6DeqWaLiytO0rhPiv26syClwdOpSitydOT4JNaeCgad/9DzOpBdSEpgspRUcS1hn7rd", + "rHC4KhNimIPi5Y3ieCVvmAa+0oSOCs67vdVKcw5MSDp8ghAREYjBRSHmxClYoBUmkZXjG/JuGZE6rJh8", + "1olRbYQfI01NdF1cbh0AlynAH0wDS/CWjOZZF5PpAQ0+gwyBnEyKWWxe69QjtdREb/huE8JGsrbr42FZ", + "yPr2ODejKXguOln5AluKfWzVEg+3qLmUjCrbuG+O01dq1tYAl6Yipa3UfeorxokkM8t1ogNGEZZvweRj", + "bXSfPXqNyTld/qYm+yTncrkLiKwgCVGgbyUHRZHhCpIl6q1BsVxVHVsBnmcyAlNHlaqkQV92jqIEZEm+", + "xGTIZWS8JJShQB1+S54pyd+48KyGgYwhc0yuhjl36xoxrpNSvdulirk0GeqnX1E6UQ58kwgOZ1yhzwVl", + "RVWI9yCpmtRb2+V3c2xu5FfusJOSIMpVmCUcs63ojdy8FSSRzvnGCQ4FihQmKmLOU32QmyU6RV5cyNDE", + "t+TL0sZSF6mww30McwPX6oCHUqmyoEDS3FqLZYhzUwczGo+qohj3YtrdGJauUV6aesHK2dwlXdJXIyzh", + "C0VQwR40iTIwAizESs3XKhBqRWg+idJFrqkuzC4VS5Oz5EpmDFBjxsOLvpVSNZXfDWXTyElvsFe6hPwE", + "CvhahrhFIsrNWgXkBU0MN8V5kkhESMhQioiuyYbqN2kwRjoQkbxViZN+PMjdrQDq0aUNUWxSw7lHTfZ2", + "WzOHpned6Qik1JGcmAMoinPuBF2jpGWJjApWLoIj4JM/F9GIRzvXxtRIC6I0GaKJDQym0r1dd5hBIRBT", + "FT/aYvqB8Q2v4PrfE6Yi7v5zEOcO/JYnieF+qVp817ytDIvky1LaJBe103yQwGT9b5eoUnU+x2iiK8R4", + "nsops9Wa4xAmAKdFarzU34ZxtT6VvoT8M47rfG89a9GhWOiJQEPTjCHOJ1fXkwxixrvBMqPB1TVQo93w", + "OVYhHHOBSLjunL+wapgYb16dO+sqPcqkPY3VncRyNgA5z5lUFnXhyAV1wSGn85SLCcrgUpU2tL2CvWmx", + "fmDseXtmzK+CrzktEpM1xwbzK6CeKfAd+1mu9GL21jW7Xj4QK4ZgVC/BPGwaPSUP+gW5OyElJiRyx8YK", + "Bp+TUe2MHqdsQil0LX5M6FIiJuXP4FhnxOp5C0MDhwvD+ZETRQNRP4q2uQgKEPq4sHhDKhkVYeU6t7hC", + "BKQIiXIAAozecLWxZm6XnPq9PuusphzV6XwGJW23gYNf2RBKkApdSd7cV/OofegYBb2tazJn3xYmTX1Q", + "bmyhNnxlxxbi6k1gvdmpouzMCVU1gR71oB+W6qFXkPem8hV3vZPP9p2RkG1m+yxnzGP6JFMFCyjC+kWI", + "ebt+2p6Lr0m4YpTgf5dLqTkA+gOFmoukJ/A1h0RgtZS7+DlLBkp0E5FesfbRsH5n0h31Vc6CurHZopnx", + "FavYtbciy7whipIKK6D0XbxTPusGS5g3hi7hPogz6zUAboLTWMznLPszP2Vs3Zn34VeD0z5VrNk+iGkk", + "K6sVZgdxONs/OpjsvwifT+Zz9HwCj54dTI7C2eLFYfTsZXwwO55Pns8O54f7B+PZs8Pnh9FBaA1/cfBs", + "f7I/O4gW+4dHUXQQHc8n8+czZyurer2x1ZpKPagKv31vZrROoEO3jtrKGXHHqa1v82vRvweUCUMJlE5b", + "98USac3LcC00e9wX0TbjhFsdmW48T1Pn1jMhXiI3MRoc3luc3JdctuHwbkNxplZY6QtBs0xFBFWF7G/m", + "OuZoPPoIc16rH6v40Jl88Fd36yyHoPYZup3z4AOTsg3PTj1UExSM7NAd8vGw4hDeWRQ3kEHtJKYnwT0G", + "NziJQsiiInNbz04uJr/e8zi1VRzjO2YVVV1fOys1AFbhhLWzsMOyGz6DITwGueKeh9yMiCKu7/KYNHqB", + "MW9sy/yOFBy4gM80N8gzvAubI5nXQdIqj95N0ydVyrid0sW7VBRuqdzOWWBX0sS76yjNpHx4C23oNWI3", + "DIvN8uLlW9rtFmaV8o/+67LVuv2g+y60xxAnqhkbv2ofIHSU7DlvrZfqtL9dY6HAqkmduqtpVPIwRJx7", + "wN2sALw917hNDRdQ+g71g3aQHK6G9OKP3Ayy0RKtq8amI+7w1y62N7pa0XtZ1tyK5aCwXoKaekre1Xmy", + "r0LoDrWWfdWVjb7ED98xw9tZd6stM25VelRIZZyc0NCRvz55Bz5kiLz6eAZOPryRKpclo+NRX1PYiTSe", + "E+3SYkpMj1gdaMRUsTgWCvHWAsUp+fHoSBJQZfAyRGCGR8ejA/WT1PhipaCdwgxPr+dT0yhoWkxv/KWy", + "h99ZpNZ69fGs3gdPFZtozarm25/N1C2r6oYQzMpE4PRfXFdQVn5UZxNvd8c9RfWGWdSKTG0iz9MUsvXo", + "WOIAyo57JKaA5+EKQA5qbfgEXHKrRd7os7pr4MNeK58mAZQYvqbR+sFwbzf0ayFtlgULue7tE96HXNGs", + "thV7TsLfjlv8qAuF+FCWrNoXPg5jOtoldpFlPDp8QDBaLTgdS2tz3iEYVsf2wnBtsjHTb/oPFRHeav2X", + "IO0HOnbqQxwnmCBNtvf6wD2DDKZI7/I/W/UAFnhFTK76DEGxGhWGYGTBMLLVuK6kcCU6/R9G+NxinEOH", + "H/7EdpRqujb67w/ayMJhGChhVW/Nx5EwRy/PHZMw67sBG0mY2ZjpN+OFbSRhxnscIGE2eH4Js2D4sSWs", + "/hWIzo2M0r0COKdkvUXihIb/dfHhvUeU6mDJucoL4m12i2gI1HIVVBENGxAZH7UDnL9dvjsfBI4c2APO", + "SugaIR84OsjrVz1VR9w+ZpbyVVwUVi0nyrt3iqe/5oitLabGYhWUIxxM7K7Eux07vga0BgyJnOkeYLrg", + "b2La/xR32Fwg1LrebALD5+1qX0cTYoek2J0ZkqJVeIMPmkMqfihifBWjcd/+21+r2Jaz7fggxuYO9/zB", + "4ClzIk/ezumOqwCSqChyhYCgG3vXXRve1gHTb9bJQr+VO1EPS6bo1AnLhC5UH7ac4K95vZ2I3+DVDzoG", + "GTzvde62woipvhhMswISmHDT86xoaKMSOqauwqU61Bz31Bk7YHg1HwDYx1PjITZkF3nlcWzaNu1Jhz4r", + "W9MfOnnRUJ4KEKuvbLXtSxdD9KVxdoYnPm/H7rnS+Lf1RKgE9/b7sMYT00MmiwXva9umkf6ulEqC+90e", + "8/Wp3WLRvpjhydkWTeQH2NSqo1HHnuqPMf3c0m1uaemG3ndHVUi2mbB+Khqb/pjmxPXBvFtjT3ZVM1Sd", + "JeOc6N7Exa3Yh2GwDRTHD85ejk/Z7Sp3GSW1deYqe6Z18FbVlPvHZa12Y/LhbvDT5jTFAbV+ypvzkvX5", + "+AEhtu4+OyRZuwXW8fdu226AW++4uyMHVEV7Ol286kvODmWP6Tf9R5XBG8Asqub76fHKuKPA17N8hfvA", + "5Z31v1vl0nrLlN1iUl3/fHceLdtRDdFgZb/Gp2MNO2/QPMpZUOP7fDvCPurLE7VO7kVz6vt6WIJBwmNd", + "pN3hXl2aYT96rrFdzvpncbEKRihVFQVQf0hH1wr0cJc+4unTTMXnSXsZSPI85FePefpt7k0t1kXXS90f", + "0LVm8WyowSr7MXat6pCP5rLNPqDjjdLTls3csqptfYXWwYSKyInpT/p0FG0JVcXuupp+yPH+pW6ktr3D", + "ffu6wPc82nd9YnGHzvnLDwzWd7ipzqYhJdeIFZW7XduvB25z/wtQelgAx5qHMQeYZLnQTfmNLtUfKCmw", + "0u2pIb8yLZ30xy0oA9c4ROAaMQ63ykQNlHaHjS5VgZSiMjEdvs13SGgMYPPjLi2i7g3gvOLu2DCTWtwO", + "e4R61h1X7eXlvHvp+MvqZt82ZN3c6fp+6t0HwBPV57Wd3US4pqbrTLdyP1ODHmnfm3dUN2eD/S3Bszv6", + "2bS1ujtbfFM9TDep4Wtwx0bRsd1G1REWl7AMDIp9/Vd3um7Of7O6qcAHG8vd2abZD6fY2/a6a8u9BXLV", + "Heufm74zpWlD972lv++mtZ8qR3QVWysY0DUiAMfq8yaA54si7GNl06Kf5da+SH+AmdgZvniEXOn30E6N", + "IPLQ1yKvo6jav/t9JdVPmQG2WkV9vwTj7EdPMJbV1QMTjJbJ8pzPFc34ikabQ9JBtQaefGcU2aMXRzjP", + "WHSjfNOgfeQrevh1+Iy6l373hGrMr49/Jt7mlp07GVdndXZ1BSSRaUtrfmA0F+YuGq5dLL67VA6uJSur", + "yF6vJa1fkehuJ+g/iFD+rG7r4m93idu9uXjDkrey2O0nS/8swttZWXJW4j2wKMn3FgnaMCWxSNCFYHko", + "cvZTpp6aTI39HW19JC84YDDN3R/z2/30fU3yuMXimyZnfkrITwmZf59gqc58ux8sdYqhP0tWpmd+iuLG", + "i/8ogvjwKUorKdiUwz9XLbaWuA3NZrfXKmBvncuFHPMDZr5LvHf9Pq7a5Dsmn4fdLLK+NLuDyr5sab7r", + "tfU7eonJXKvQ3LMZd9KsV3nR7IfUXRrt3VddNPNrLvXxEXZd7Gi9+fya5nsRTSEmqvX8SJLaTODWBaO+", + "bvcRDQe3uDc97adfcxxeTZQGnuiy1EnVFaymY0Yuz0yhvV2obrBYTaLUgkct24am6AJbjit+uP18+38B", + "AAD//3gHYjaUvQAA", } // GetSwagger returns the content of the embedded swagger specification file diff --git a/dm/openapi/gen.types.go b/dm/openapi/gen.types.go index 6ffb8d0e38d..647c25bf2ef 100644 --- a/dm/openapi/gen.types.go +++ b/dm/openapi/gen.types.go @@ -33,6 +33,8 @@ const ( TaskTaskModeFull TaskTaskMode = "full" TaskTaskModeIncremental TaskTaskMode = "incremental" + + TaskTaskModeLoad TaskTaskMode = "load" ) // Defines values for TaskFullMigrateConfAnalyze. @@ -80,6 +82,8 @@ const ( const ( TaskStageFinished TaskStage = "Finished" + TaskStagePaused TaskStage = "Paused" + TaskStageRunning TaskStage = "Running" TaskStageStopped TaskStage = "Stopped" @@ -165,10 +169,12 @@ type DisableRelayRequest struct { // status of dump unit type DumpStatus struct { + Bps int64 `json:"bps"` CompletedTables float64 `json:"completed_tables"` EstimateTotalRows float64 `json:"estimate_total_rows"` FinishedBytes float64 `json:"finished_bytes"` FinishedRows float64 `json:"finished_rows"` + Progress string `json:"progress"` TotalTables int64 `json:"total_tables"` } @@ -260,6 +266,7 @@ type GrafanaTopology struct { // status of load unit type LoadStatus struct { + Bps int64 `json:"bps"` FinishedBytes int64 `json:"finished_bytes"` MetaBinlog string `json:"meta_binlog"` MetaBinlogGtid string `json:"meta_binlog_gtid"` @@ -501,6 +508,8 @@ type SyncStatus struct { // sharding DDL which current is blocking BlockingDdls []string `json:"blocking_ddls"` + DumpIoTotalBytes uint64 `json:"dump_io_total_bytes"` + IoTotalBytes uint64 `json:"io_total_bytes"` MasterBinlog string `json:"master_binlog"` MasterBinlogGtid string `json:"master_binlog_gtid"` RecentTps int64 `json:"recent_tps"` diff --git a/dm/openapi/spec/dm.yaml b/dm/openapi/spec/dm.yaml index c07b6139d25..2fea9d8da86 100644 --- a/dm/openapi/spec/dm.yaml +++ b/dm/openapi/spec/dm.yaml @@ -1456,12 +1456,16 @@ components: type: string meta_binlog_gtid: type: string + bps: + type: integer + format: int64 required: - "finished_bytes" - "total_bytes" - "progress" - "meta_binlog" - "meta_binlog_gtid" + - "bps" SyncStatus: type: object description: "status of sync unit" @@ -1500,6 +1504,12 @@ components: seconds_behind_master: type: integer format: int64 + io_total_bytes: + type: integer + format: uint64 + dump_io_total_bytes: + type: integer + format: uint64 required: - "total_events" - "total_tps" @@ -1513,6 +1523,8 @@ components: - "synced" - "binlog_type" - "seconds_behind_master" + - "io_total_bytes" + - "dump_io_total_bytes" DumpStatus: type: object description: "status of dump unit" @@ -1532,12 +1544,19 @@ components: estimate_total_rows: type: number format: double + bps: + type: integer + format: int64 + progress: + type: string required: - "total_tables" - "completed_tables" - "finished_bytes" - "finished_rows" - "estimate_total_rows" + - "bps" + - "progress" SubTaskStatus: type: object properties: @@ -1809,6 +1828,7 @@ components: - Stopped - Running - Finished + - Paused Task: description: "task" type: object @@ -1826,6 +1846,7 @@ components: - "incremental" - "all" - "dump" + - "load" shard_mode: type: string description: the way to coordinate DDL diff --git a/dm/pb/dmworker.pb.go b/dm/pb/dmworker.pb.go index 098648f4268..38d77a300a7 100644 --- a/dm/pb/dmworker.pb.go +++ b/dm/pb/dmworker.pb.go @@ -961,6 +961,10 @@ type SyncStatus struct { TotalRows int64 `protobuf:"varint,15,opt,name=totalRows,proto3" json:"totalRows,omitempty"` TotalRps int64 `protobuf:"varint,16,opt,name=totalRps,proto3" json:"totalRps,omitempty"` RecentRps int64 `protobuf:"varint,17,opt,name=recentRps,proto3" json:"recentRps,omitempty"` + // meter TCP io to downstream of the subtask + IoTotalBytes uint64 `protobuf:"varint,18,opt,name=ioTotalBytes,proto3" json:"ioTotalBytes,omitempty"` + // meter TCP io from upstream of the subtask + DumpIOTotalBytes uint64 `protobuf:"varint,19,opt,name=dumpIOTotalBytes,proto3" json:"dumpIOTotalBytes,omitempty"` } func (m *SyncStatus) Reset() { *m = SyncStatus{} } @@ -1115,6 +1119,20 @@ func (m *SyncStatus) GetRecentRps() int64 { return 0 } +func (m *SyncStatus) GetIoTotalBytes() uint64 { + if m != nil { + return m.IoTotalBytes + } + return 0 +} + +func (m *SyncStatus) GetDumpIOTotalBytes() uint64 { + if m != nil { + return m.DumpIOTotalBytes + } + return 0 +} + // SourceStatus represents status for source runing on dm-worker type SourceStatus struct { Source string `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"` @@ -3637,192 +3655,193 @@ func init() { func init() { proto.RegisterFile("dmworker.proto", fileDescriptor_51a1b9e17fd67b10) } var fileDescriptor_51a1b9e17fd67b10 = []byte{ - // 2947 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x3a, 0xcd, 0x6f, 0xe5, 0x56, - 0xf5, 0xcf, 0xf6, 0xfb, 0x3c, 0xef, 0x25, 0x71, 0xee, 0x64, 0xe6, 0xf7, 0x9a, 0xce, 0xbc, 0xa6, - 0x9e, 0xaa, 0xbf, 0x34, 0x2a, 0x51, 0x1b, 0x8a, 0x8a, 0x2a, 0x41, 0xdb, 0x49, 0xa6, 0x99, 0x29, - 0x99, 0x66, 0xc6, 0x49, 0x87, 0x15, 0x12, 0x8e, 0xdf, 0xcd, 0x8b, 0x89, 0x9f, 0xed, 0xb1, 0xfd, - 0x12, 0x65, 0x81, 0xd8, 0xb1, 0x85, 0x0d, 0x48, 0x20, 0x36, 0x20, 0xb1, 0x65, 0xc1, 0x1f, 0xc0, - 0x12, 0xba, 0xac, 0x58, 0xb1, 0x42, 0xa8, 0xf3, 0x37, 0xb0, 0x61, 0x81, 0xd0, 0x39, 0xf7, 0x5e, - 0xfb, 0xfa, 0x7d, 0x64, 0x3a, 0x48, 0xec, 0x7c, 0x3e, 0xee, 0xb9, 0xc7, 0xe7, 0xfb, 0xf8, 0x3d, - 0x58, 0x1e, 0x8e, 0x2f, 0xe3, 0xf4, 0x9c, 0xa7, 0xdb, 0x49, 0x1a, 0xe7, 0x31, 0x33, 0x93, 0x13, - 0x67, 0x13, 0xd8, 0x93, 0x09, 0x4f, 0xaf, 0x8e, 0x72, 0x2f, 0x9f, 0x64, 0x2e, 0x7f, 0x36, 0xe1, - 0x59, 0xce, 0x18, 0xd4, 0x23, 0x6f, 0xcc, 0xfb, 0xc6, 0x86, 0xb1, 0xd9, 0x71, 0xe9, 0xd9, 0x49, - 0x60, 0x6d, 0x37, 0x1e, 0x8f, 0xe3, 0xe8, 0xfb, 0x24, 0xc3, 0xe5, 0x59, 0x12, 0x47, 0x19, 0x67, - 0xb7, 0xa0, 0x99, 0xf2, 0x6c, 0x12, 0xe6, 0xc4, 0xdd, 0x76, 0x25, 0xc4, 0x6c, 0xb0, 0xc6, 0xd9, - 0xa8, 0x6f, 0x92, 0x08, 0x7c, 0x44, 0xce, 0x2c, 0x9e, 0xa4, 0x3e, 0xef, 0x5b, 0x84, 0x94, 0x10, - 0xe2, 0x85, 0x5e, 0xfd, 0xba, 0xc0, 0x0b, 0xc8, 0xf9, 0x83, 0x01, 0x37, 0x2a, 0xca, 0xbd, 0xf4, - 0x8d, 0xef, 0x41, 0x4f, 0xdc, 0x21, 0x24, 0xd0, 0xbd, 0xdd, 0x1d, 0x7b, 0x3b, 0x39, 0xd9, 0x3e, - 0xd2, 0xf0, 0x6e, 0x85, 0x8b, 0xbd, 0x0f, 0x4b, 0xd9, 0xe4, 0xe4, 0xd8, 0xcb, 0xce, 0xe5, 0xb1, - 0xfa, 0x86, 0xb5, 0xd9, 0xdd, 0x59, 0xa5, 0x63, 0x3a, 0xc1, 0xad, 0xf2, 0x39, 0xbf, 0x37, 0xa0, - 0xbb, 0x7b, 0xc6, 0x7d, 0x09, 0xa3, 0xa2, 0x89, 0x97, 0x65, 0x7c, 0xa8, 0x14, 0x15, 0x10, 0x5b, - 0x83, 0x46, 0x1e, 0xe7, 0x5e, 0x48, 0xaa, 0x36, 0x5c, 0x01, 0xb0, 0x01, 0x40, 0x36, 0xf1, 0x7d, - 0x9e, 0x65, 0xa7, 0x93, 0x90, 0x54, 0x6d, 0xb8, 0x1a, 0x06, 0xa5, 0x9d, 0x7a, 0x41, 0xc8, 0x87, - 0x64, 0xa6, 0x86, 0x2b, 0x21, 0xd6, 0x87, 0xd6, 0xa5, 0x97, 0x46, 0x41, 0x34, 0xea, 0x37, 0x88, - 0xa0, 0x40, 0x3c, 0x31, 0xe4, 0xb9, 0x17, 0x84, 0xfd, 0xe6, 0x86, 0xb1, 0xd9, 0x73, 0x25, 0xe4, - 0xfc, 0xdb, 0x00, 0xd8, 0x9b, 0x8c, 0x13, 0xa9, 0xe6, 0x06, 0x74, 0x49, 0x83, 0x63, 0xef, 0x24, - 0xe4, 0x19, 0xe9, 0x6a, 0xb9, 0x3a, 0x8a, 0x6d, 0xc2, 0x8a, 0x1f, 0x8f, 0x93, 0x90, 0xe7, 0x7c, - 0x28, 0xb9, 0x50, 0x75, 0xc3, 0x9d, 0x46, 0xb3, 0x37, 0x60, 0xe9, 0x34, 0x88, 0x82, 0xec, 0x8c, - 0x0f, 0xef, 0x5d, 0xe5, 0x5c, 0x98, 0xdc, 0x70, 0xab, 0x48, 0xe6, 0x40, 0x4f, 0x21, 0xdc, 0xf8, - 0x32, 0xa3, 0x17, 0x32, 0xdc, 0x0a, 0x8e, 0xbd, 0x0d, 0xab, 0x3c, 0xcb, 0x83, 0xb1, 0x97, 0xf3, - 0x63, 0x54, 0x85, 0x18, 0x1b, 0xc4, 0x38, 0x4b, 0x40, 0xdf, 0x9f, 0x24, 0x19, 0xbd, 0xa7, 0xe5, - 0xe2, 0x23, 0x5b, 0x87, 0x76, 0x92, 0xc6, 0xa3, 0x94, 0x67, 0x59, 0xbf, 0x45, 0x21, 0x51, 0xc0, - 0xce, 0x17, 0x06, 0xc0, 0x41, 0xec, 0x0d, 0xa5, 0x01, 0x66, 0x94, 0x16, 0x26, 0x98, 0x52, 0x7a, - 0x00, 0x40, 0x36, 0x11, 0x2c, 0x26, 0xb1, 0x68, 0x98, 0xca, 0x85, 0x56, 0xf5, 0x42, 0x3c, 0x3b, - 0xe6, 0xb9, 0x77, 0x2f, 0x88, 0xc2, 0x78, 0x24, 0xc3, 0x5c, 0xc3, 0xb0, 0x37, 0x61, 0xb9, 0x84, - 0xf6, 0x8f, 0x1f, 0xee, 0xd1, 0x9b, 0x76, 0xdc, 0x29, 0xec, 0xec, 0x6b, 0x3a, 0xbf, 0x30, 0x60, - 0xe9, 0xe8, 0xcc, 0x4b, 0x87, 0x41, 0x34, 0xda, 0x4f, 0xe3, 0x49, 0x82, 0x5e, 0xcf, 0xbd, 0x74, - 0xc4, 0x73, 0x99, 0xbe, 0x12, 0xc2, 0xa4, 0xde, 0xdb, 0x3b, 0x40, 0xcd, 0x2d, 0x4c, 0x6a, 0x7c, - 0x16, 0x6f, 0x9e, 0x66, 0xf9, 0x41, 0xec, 0x7b, 0x79, 0x10, 0x47, 0x52, 0xf1, 0x2a, 0x92, 0x12, - 0xf7, 0x2a, 0xf2, 0x29, 0xf2, 0x2c, 0x4a, 0x5c, 0x82, 0xf0, 0x8d, 0x27, 0x91, 0xa4, 0x34, 0x88, - 0x52, 0xc0, 0xce, 0x3f, 0xeb, 0x00, 0x47, 0x57, 0x91, 0x3f, 0x15, 0x63, 0xf7, 0x2f, 0x78, 0x94, - 0x57, 0x63, 0x4c, 0xa0, 0x50, 0x98, 0x08, 0xb9, 0x44, 0x19, 0xb7, 0x80, 0xd9, 0x6d, 0xe8, 0xa4, - 0xdc, 0xe7, 0x51, 0x8e, 0x44, 0x8b, 0x88, 0x25, 0x02, 0xa3, 0x69, 0xec, 0x65, 0x39, 0x4f, 0x2b, - 0xe6, 0xad, 0xe0, 0xd8, 0x16, 0xd8, 0x3a, 0xbc, 0x9f, 0x07, 0x43, 0x69, 0xe2, 0x19, 0x3c, 0xca, - 0xa3, 0x97, 0x50, 0xf2, 0x9a, 0x42, 0x9e, 0x8e, 0x43, 0x79, 0x3a, 0x4c, 0xf2, 0x44, 0x94, 0xcd, - 0xe0, 0x51, 0xde, 0x49, 0x18, 0xfb, 0xe7, 0x41, 0x34, 0x22, 0x07, 0xb4, 0xc9, 0x54, 0x15, 0x1c, - 0xfb, 0x0e, 0xd8, 0x93, 0x28, 0xe5, 0x59, 0x1c, 0x5e, 0xf0, 0x21, 0xf9, 0x31, 0xeb, 0x77, 0xb4, - 0xb2, 0xa3, 0x7b, 0xd8, 0x9d, 0x61, 0xd5, 0x3c, 0x04, 0xa2, 0xd2, 0x48, 0x0f, 0x0d, 0x00, 0x4e, - 0x48, 0x91, 0xe3, 0xab, 0x84, 0xf7, 0xbb, 0x22, 0xee, 0x4a, 0x0c, 0x7b, 0x07, 0x6e, 0x64, 0xdc, - 0x8f, 0xa3, 0x61, 0x76, 0x8f, 0x9f, 0x05, 0xd1, 0xf0, 0x11, 0xd9, 0xa2, 0xdf, 0x23, 0x13, 0xcf, - 0x23, 0x61, 0xc4, 0x90, 0xe2, 0x7b, 0x7b, 0x07, 0x87, 0x97, 0x11, 0x4f, 0xfb, 0x4b, 0x22, 0x62, - 0x2a, 0x48, 0x74, 0xb7, 0x1f, 0x47, 0xa7, 0x61, 0xe0, 0xe7, 0x8f, 0xb2, 0x51, 0x7f, 0x99, 0x78, - 0x74, 0x14, 0xba, 0x34, 0x2f, 0xd2, 0x7a, 0x45, 0xb8, 0xb4, 0x40, 0x14, 0xc1, 0xe0, 0x26, 0x59, - 0xdf, 0xd6, 0x82, 0xc1, 0xd5, 0x83, 0x01, 0x89, 0xab, 0x7a, 0x30, 0xb8, 0x49, 0xe6, 0xfc, 0xc6, - 0x80, 0x9e, 0x5e, 0xdb, 0xb5, 0xae, 0x63, 0x2c, 0xe8, 0x3a, 0xa6, 0xde, 0x75, 0xd8, 0x5b, 0x45, - 0x77, 0x11, 0xdd, 0x82, 0xec, 0xff, 0x38, 0x8d, 0xb1, 0x0c, 0xbb, 0x44, 0x28, 0x1a, 0xce, 0xbb, - 0xd0, 0x4d, 0x79, 0xe8, 0x5d, 0x15, 0x6d, 0x02, 0xf9, 0x57, 0x90, 0xdf, 0x2d, 0xd1, 0xae, 0xce, - 0xe3, 0xfc, 0xc5, 0x84, 0xae, 0x46, 0x9c, 0x89, 0x5d, 0xe3, 0x6b, 0xc6, 0xae, 0xb9, 0x20, 0x76, - 0x37, 0x94, 0x4a, 0x93, 0x93, 0xbd, 0x20, 0x95, 0xe9, 0xac, 0xa3, 0x0a, 0x8e, 0x4a, 0xb2, 0xe8, - 0x28, 0xac, 0xf6, 0x1a, 0xa8, 0xa5, 0xca, 0x34, 0x9a, 0x6d, 0x03, 0x23, 0xd4, 0xae, 0x97, 0xfb, - 0x67, 0x9f, 0x27, 0x32, 0x7a, 0x9a, 0x14, 0x82, 0x73, 0x28, 0xec, 0x35, 0x68, 0x64, 0xb9, 0x37, - 0xe2, 0x94, 0x2a, 0xcb, 0x3b, 0x1d, 0x0a, 0x6d, 0x44, 0xb8, 0x02, 0xaf, 0x19, 0xbf, 0xfd, 0x02, - 0xe3, 0x3b, 0x7f, 0xb4, 0x60, 0xa9, 0xd2, 0x8d, 0xe7, 0x4d, 0x2d, 0xe5, 0x8d, 0xe6, 0x82, 0x1b, - 0x37, 0xa0, 0x3e, 0x89, 0x02, 0xe1, 0xec, 0xe5, 0x9d, 0x1e, 0xd2, 0x3f, 0x8f, 0x82, 0x1c, 0xb3, - 0xc3, 0x25, 0x8a, 0xa6, 0x53, 0xfd, 0x45, 0x01, 0xf1, 0x0e, 0xdc, 0x28, 0x53, 0x73, 0x6f, 0xef, - 0xe0, 0x20, 0xf6, 0xcf, 0x8b, 0x5a, 0x3e, 0x8f, 0xc4, 0x98, 0x98, 0x59, 0xa8, 0xc4, 0x3c, 0xa8, - 0x89, 0xa9, 0xe5, 0xff, 0xa1, 0xe1, 0xe3, 0x14, 0x41, 0x56, 0x92, 0x01, 0xa5, 0x8d, 0x15, 0x0f, - 0x6a, 0xae, 0xa0, 0xb3, 0x37, 0xa0, 0x3e, 0x9c, 0x8c, 0x13, 0x69, 0xab, 0x65, 0xe4, 0x2b, 0xdb, - 0xfa, 0x83, 0x9a, 0x4b, 0x54, 0xe4, 0x0a, 0x63, 0x6f, 0xd8, 0xef, 0x94, 0x5c, 0x65, 0xef, 0x43, - 0x2e, 0xa4, 0x22, 0x17, 0xd6, 0x0c, 0xaa, 0x1f, 0x92, 0xab, 0x2c, 0xdf, 0xc8, 0x85, 0x54, 0xf6, - 0x1e, 0xc0, 0x85, 0x17, 0x06, 0x43, 0xd1, 0x2c, 0xba, 0xc4, 0xbb, 0x86, 0xbc, 0x4f, 0x0b, 0xac, - 0x8c, 0x7a, 0x8d, 0xef, 0x5e, 0x1b, 0x9a, 0x99, 0x08, 0xff, 0xef, 0xc2, 0x6a, 0xc5, 0x67, 0x07, - 0x41, 0x46, 0x06, 0x16, 0xe4, 0xbe, 0xb1, 0x68, 0xd0, 0x52, 0xe7, 0x07, 0x00, 0x64, 0x89, 0xfb, - 0x69, 0x1a, 0xa7, 0x6a, 0xe0, 0x33, 0x8a, 0x81, 0xcf, 0xb9, 0x03, 0x1d, 0xb4, 0xc0, 0x35, 0x64, - 0x7c, 0xf5, 0x45, 0xe4, 0x04, 0x7a, 0xf4, 0xce, 0x4f, 0x0e, 0x16, 0x70, 0xb0, 0x1d, 0x58, 0x13, - 0x53, 0x97, 0x48, 0x82, 0xc7, 0x71, 0x16, 0x90, 0x25, 0x44, 0x3a, 0xce, 0xa5, 0x61, 0x2d, 0xe3, - 0x28, 0xee, 0xe8, 0xc9, 0x81, 0x9a, 0x0b, 0x14, 0xec, 0x7c, 0x0b, 0x3a, 0x78, 0xa3, 0xb8, 0x6e, - 0x13, 0x9a, 0x44, 0x50, 0x76, 0xb0, 0x0b, 0x27, 0x48, 0x85, 0x5c, 0x49, 0x77, 0x7e, 0x66, 0x40, - 0x57, 0x14, 0x39, 0x71, 0xf2, 0x65, 0x6b, 0xdc, 0x46, 0xe5, 0xb8, 0xaa, 0x12, 0xba, 0xc4, 0x6d, - 0x00, 0x2a, 0x53, 0x82, 0xa1, 0x5e, 0x06, 0x45, 0x89, 0x75, 0x35, 0x0e, 0x74, 0x4c, 0x09, 0xcd, - 0x31, 0xed, 0xaf, 0x4c, 0xe8, 0x49, 0x97, 0x0a, 0x96, 0xff, 0x51, 0xb2, 0xca, 0x7c, 0xaa, 0xeb, - 0xf9, 0xf4, 0xa6, 0xca, 0xa7, 0x46, 0xf9, 0x1a, 0x65, 0x14, 0x95, 0xe9, 0x74, 0x57, 0xa6, 0x53, - 0x93, 0xd8, 0x96, 0x54, 0x3a, 0x29, 0x2e, 0x91, 0x4d, 0x77, 0x65, 0x36, 0xb5, 0x4a, 0xa6, 0x22, - 0xa4, 0x8a, 0x64, 0xba, 0x2b, 0x93, 0xa9, 0x5d, 0x32, 0x15, 0x6e, 0x56, 0xb9, 0x74, 0xaf, 0x05, - 0x0d, 0x72, 0xa7, 0xf3, 0x01, 0xd8, 0xba, 0x69, 0x28, 0x27, 0xde, 0x94, 0xc4, 0x4a, 0x28, 0x68, - 0x4c, 0xae, 0x3c, 0xfb, 0x0c, 0x96, 0x2a, 0xa5, 0x08, 0x3b, 0x7e, 0x90, 0xed, 0x7a, 0x91, 0xcf, - 0xc3, 0x62, 0xef, 0xd0, 0x30, 0x5a, 0x90, 0x99, 0xa5, 0x64, 0x29, 0xa2, 0x12, 0x64, 0xda, 0xf6, - 0x60, 0x55, 0xb6, 0x87, 0xbf, 0x1a, 0xd0, 0xd3, 0x0f, 0xe0, 0x02, 0x72, 0x3f, 0x4d, 0x77, 0xe3, - 0xa1, 0xf0, 0x66, 0xc3, 0x55, 0x20, 0x86, 0x3e, 0x3e, 0x86, 0x5e, 0x96, 0xc9, 0x08, 0x2c, 0x60, - 0x49, 0x3b, 0xf2, 0xe3, 0x44, 0xed, 0x83, 0x05, 0x2c, 0x69, 0x07, 0xfc, 0x82, 0x87, 0xb2, 0x41, - 0x15, 0x30, 0xde, 0xf6, 0x88, 0x67, 0x19, 0x86, 0x89, 0xa8, 0xab, 0x0a, 0xc4, 0x53, 0xae, 0x77, - 0xb9, 0xeb, 0x4d, 0x32, 0x2e, 0x67, 0xb6, 0x02, 0x46, 0xb3, 0xe0, 0xde, 0xea, 0xa5, 0xf1, 0x24, - 0x52, 0x93, 0x9a, 0x86, 0x71, 0x2e, 0x61, 0xf5, 0xf1, 0x24, 0x1d, 0x71, 0x0a, 0x62, 0xb5, 0x06, - 0xaf, 0x43, 0x3b, 0x88, 0x3c, 0x3f, 0x0f, 0x2e, 0xb8, 0xb4, 0x64, 0x01, 0x63, 0xfc, 0xe6, 0xc1, - 0x98, 0xcb, 0x51, 0x95, 0x9e, 0x91, 0xff, 0x34, 0x08, 0x39, 0xc5, 0xb5, 0x7c, 0x25, 0x05, 0x53, - 0x8a, 0x8a, 0x9e, 0x2c, 0x97, 0x5c, 0x01, 0x39, 0xbf, 0x36, 0x61, 0xfd, 0x30, 0xe1, 0xa9, 0x97, - 0x73, 0xb1, 0x58, 0x1f, 0xf9, 0x67, 0x7c, 0xec, 0x29, 0x15, 0x6e, 0x83, 0x19, 0x27, 0x74, 0xb9, - 0x8c, 0x77, 0x41, 0x3e, 0x4c, 0x5c, 0x33, 0x4e, 0x48, 0x09, 0x2f, 0x3b, 0x97, 0xb6, 0xa5, 0xe7, - 0x85, 0x5b, 0xf6, 0x3a, 0xb4, 0x87, 0x5e, 0xee, 0x9d, 0x78, 0x19, 0x57, 0x36, 0x55, 0x30, 0x2d, - 0xa4, 0xb8, 0xbf, 0x49, 0x8b, 0x0a, 0x80, 0x24, 0xd1, 0x6d, 0xd2, 0x9a, 0x12, 0x42, 0xee, 0xd3, - 0x70, 0x92, 0x9d, 0x91, 0x19, 0xdb, 0xae, 0x00, 0x50, 0x97, 0x22, 0xe6, 0xdb, 0xb2, 0x5d, 0x0c, - 0x00, 0x4e, 0xd3, 0x78, 0x2c, 0x0a, 0x0b, 0x35, 0xa0, 0xb6, 0xab, 0x61, 0x14, 0xfd, 0x58, 0xac, - 0x2b, 0x50, 0xd2, 0x05, 0xc6, 0xc9, 0x61, 0xe9, 0xe9, 0xbb, 0x32, 0xec, 0x1f, 0xf1, 0xdc, 0x63, - 0xeb, 0x9a, 0x39, 0x00, 0xcd, 0x81, 0x14, 0x69, 0x8c, 0x17, 0x56, 0x0f, 0x55, 0x72, 0x2c, 0xad, - 0xe4, 0x28, 0x0b, 0xd6, 0x29, 0xc4, 0xe9, 0xd9, 0x79, 0x0f, 0xd6, 0xa4, 0x47, 0x9e, 0xbe, 0x8b, - 0xb7, 0x2e, 0xf4, 0x85, 0x20, 0x8b, 0xeb, 0x9d, 0x3f, 0x1b, 0x70, 0x73, 0xea, 0xd8, 0x4b, 0x7f, - 0xaf, 0x78, 0x1f, 0xea, 0xb8, 0xf0, 0xf5, 0x2d, 0x4a, 0xcd, 0xbb, 0x78, 0xc7, 0x5c, 0x91, 0xdb, - 0x08, 0xdc, 0x8f, 0xf2, 0xf4, 0xca, 0xa5, 0x03, 0xeb, 0x9f, 0x42, 0xa7, 0x40, 0xa1, 0xdc, 0x73, - 0x7e, 0xa5, 0xaa, 0xef, 0x39, 0xbf, 0xc2, 0x89, 0xe2, 0xc2, 0x0b, 0x27, 0xc2, 0x34, 0xb2, 0xc1, - 0x56, 0x0c, 0xeb, 0x0a, 0xfa, 0x07, 0xe6, 0xb7, 0x0d, 0xe7, 0xc7, 0xd0, 0x7f, 0xe0, 0x45, 0xc3, - 0x50, 0xc6, 0xa3, 0x28, 0x0a, 0xd2, 0x04, 0xaf, 0x6a, 0x26, 0xe8, 0xa2, 0x14, 0xa2, 0x5e, 0x13, - 0x8d, 0xb7, 0xa1, 0x73, 0xa2, 0xda, 0xa1, 0x34, 0x7c, 0x89, 0xa0, 0x98, 0x79, 0x16, 0x66, 0x72, - 0xad, 0xa4, 0x67, 0xe7, 0x26, 0xdc, 0xd8, 0xe7, 0xb9, 0xb8, 0x7b, 0xf7, 0x74, 0x24, 0x6f, 0x76, - 0x36, 0x61, 0xad, 0x8a, 0x96, 0xc6, 0xb5, 0xc1, 0xf2, 0x4f, 0x8b, 0x56, 0xe3, 0x9f, 0x8e, 0x9c, - 0x23, 0xb8, 0x23, 0xa6, 0xa5, 0xc9, 0x09, 0xaa, 0x80, 0xa5, 0xef, 0xf3, 0x64, 0xe8, 0xe5, 0x5c, - 0xbd, 0xc4, 0x0e, 0xac, 0x65, 0x82, 0xb6, 0x7b, 0x3a, 0x3a, 0x8e, 0xc7, 0xe1, 0x51, 0x9e, 0x06, - 0x91, 0x92, 0x31, 0x97, 0xe6, 0x1c, 0xc0, 0x60, 0x91, 0x50, 0xa9, 0x48, 0x1f, 0x5a, 0xf2, 0x63, - 0x8d, 0x74, 0xb3, 0x02, 0x67, 0xfd, 0xec, 0x8c, 0x60, 0x7d, 0x9f, 0xe7, 0x33, 0x33, 0x53, 0x59, - 0x76, 0xf0, 0x8e, 0xcf, 0xca, 0xf6, 0x58, 0xc0, 0xec, 0x1b, 0xd0, 0x3b, 0x0d, 0xc2, 0x9c, 0xa7, - 0x72, 0xe7, 0x98, 0x89, 0xf5, 0x0a, 0xd9, 0xf9, 0xbb, 0x05, 0xf6, 0xf4, 0x35, 0x85, 0x9f, 0x8c, - 0xb9, 0x55, 0xc3, 0xac, 0x54, 0x0d, 0x06, 0xf5, 0x31, 0x16, 0x76, 0x99, 0x33, 0xf8, 0x5c, 0x26, - 0x5a, 0x7d, 0x41, 0xa2, 0x6d, 0xc2, 0x8a, 0x9c, 0xfe, 0x62, 0xb5, 0xd7, 0xc8, 0x05, 0x62, 0x0a, - 0x8d, 0x03, 0xf3, 0x14, 0x8a, 0xd6, 0x0d, 0x51, 0x6f, 0xe6, 0x91, 0xb4, 0x69, 0xbc, 0xf5, 0x35, - 0xa6, 0xf1, 0x44, 0x10, 0xc4, 0x27, 0x25, 0x69, 0xb2, 0xb6, 0x10, 0x3e, 0x87, 0xc4, 0xde, 0x86, - 0xd5, 0x84, 0x47, 0xb8, 0x68, 0x6b, 0xfc, 0x1d, 0xe2, 0x9f, 0x25, 0xe0, 0x6b, 0x52, 0xab, 0xd4, - 0x78, 0x41, 0xbc, 0xe6, 0x14, 0x1a, 0x37, 0x38, 0x7f, 0x92, 0xc7, 0x17, 0x6a, 0x55, 0xc3, 0x64, - 0x10, 0xcb, 0xf8, 0x0c, 0x1e, 0x75, 0xa8, 0xe0, 0xc8, 0x20, 0x3d, 0xa1, 0xc3, 0x0c, 0xc1, 0xf9, - 0x9d, 0x01, 0x37, 0x4b, 0x07, 0xd3, 0x47, 0xb8, 0x17, 0xec, 0xbd, 0xeb, 0xd0, 0xce, 0x52, 0x9f, - 0x38, 0x55, 0x4f, 0x56, 0x30, 0xf5, 0x88, 0x2c, 0x17, 0x34, 0xd9, 0xc0, 0x14, 0xfc, 0x62, 0xaf, - 0xf7, 0xa1, 0x35, 0xae, 0x36, 0x66, 0x09, 0x3a, 0x7f, 0x32, 0xe0, 0xd5, 0xb9, 0xf1, 0xfe, 0x5f, - 0x7c, 0xd0, 0x85, 0x22, 0x28, 0x32, 0x59, 0x26, 0xaf, 0xdf, 0x3f, 0x70, 0x92, 0xf9, 0x10, 0x96, - 0xf2, 0xd2, 0x32, 0x5c, 0x7d, 0xd0, 0x7d, 0xa5, 0x7a, 0x50, 0x33, 0x9e, 0x5b, 0xe5, 0x77, 0xce, - 0xe1, 0x95, 0x8a, 0xfe, 0x95, 0x9a, 0xb8, 0x43, 0xf3, 0x3d, 0xf2, 0x72, 0x59, 0x19, 0x6f, 0x69, - 0x82, 0xc5, 0x3c, 0x4d, 0x54, 0xb7, 0xe0, 0xab, 0xa4, 0xb8, 0x59, 0x4d, 0x71, 0xe7, 0xb7, 0x26, - 0xac, 0x4c, 0x5d, 0xc5, 0x96, 0xc1, 0x0c, 0x86, 0xd2, 0x91, 0x66, 0x30, 0x5c, 0x98, 0xae, 0xba, - 0x73, 0xad, 0x29, 0xe7, 0x62, 0x81, 0x4a, 0xfd, 0x3d, 0x2f, 0xf7, 0x64, 0xff, 0x57, 0x60, 0xc5, - 0xed, 0x8d, 0x29, 0xb7, 0xf7, 0xa1, 0x35, 0xcc, 0x72, 0x3a, 0x25, 0xb2, 0x52, 0x81, 0x58, 0xda, - 0x29, 0xce, 0xe9, 0xd3, 0x92, 0x98, 0xa8, 0x4a, 0x04, 0xdb, 0x2e, 0x96, 0xba, 0xf6, 0xb5, 0x36, - 0x91, 0x5c, 0xc5, 0x3c, 0xd5, 0x91, 0x45, 0x09, 0xe7, 0x29, 0x2d, 0xa2, 0xa0, 0x1a, 0x51, 0xcf, - 0xa6, 0x0a, 0xa8, 0x74, 0xc8, 0x4b, 0xc7, 0xd3, 0x5b, 0x6a, 0xcc, 0x16, 0xa1, 0x74, 0xa3, 0x1a, - 0x11, 0x95, 0x49, 0xfb, 0x97, 0x06, 0xdc, 0x51, 0xcd, 0x78, 0x7e, 0x20, 0xdc, 0xd5, 0x9a, 0xe3, - 0xac, 0x24, 0xd9, 0x24, 0x69, 0x3e, 0xff, 0x38, 0x0c, 0xc5, 0x62, 0x65, 0xaa, 0xf9, 0x5c, 0x61, - 0x2a, 0x91, 0x61, 0x4d, 0x15, 0xff, 0x35, 0xd2, 0xf6, 0xa1, 0xf8, 0x01, 0xa0, 0xee, 0x0a, 0xc0, - 0xf9, 0x14, 0x06, 0x8b, 0xf4, 0x7a, 0x59, 0x7b, 0x38, 0x57, 0x70, 0x47, 0xb4, 0xb5, 0x52, 0x94, - 0xfa, 0xb9, 0xe7, 0xc5, 0xbd, 0xa9, 0xd2, 0xeb, 0xcd, 0xe9, 0x5e, 0x5f, 0x7c, 0x8a, 0xa4, 0xcf, - 0xdb, 0x96, 0xfe, 0x29, 0x12, 0x31, 0x5b, 0xe7, 0xd0, 0x14, 0xc3, 0x1c, 0x5b, 0x82, 0xce, 0xc3, - 0x88, 0xd2, 0xf7, 0x30, 0xb1, 0x6b, 0xac, 0x0d, 0xf5, 0xa3, 0x3c, 0x4e, 0x6c, 0x83, 0x75, 0xa0, - 0xf1, 0x18, 0xa7, 0x79, 0xdb, 0x64, 0x00, 0x4d, 0xac, 0xf6, 0x63, 0x6e, 0x5b, 0x88, 0x3e, 0xca, - 0xbd, 0x34, 0xb7, 0xeb, 0x88, 0x16, 0xfa, 0xdb, 0x0d, 0xb6, 0x0c, 0xf0, 0xf1, 0x24, 0x8f, 0x25, - 0x5b, 0x13, 0x69, 0x7b, 0x3c, 0xe4, 0x39, 0xb7, 0x5b, 0x5b, 0x3f, 0xa1, 0x23, 0x23, 0x1c, 0x1f, - 0x7a, 0xf2, 0x2e, 0x82, 0xed, 0x1a, 0x6b, 0x81, 0xf5, 0x19, 0xbf, 0xb4, 0x0d, 0xd6, 0x85, 0x96, - 0x3b, 0x89, 0xa2, 0x20, 0x1a, 0x89, 0xfb, 0xe8, 0xea, 0xa1, 0x6d, 0x21, 0x01, 0x15, 0x4a, 0xf8, - 0xd0, 0xae, 0xb3, 0x1e, 0xb4, 0x3f, 0x91, 0x3f, 0x13, 0xd8, 0x0d, 0x24, 0x21, 0x1b, 0x9e, 0x69, - 0x22, 0x89, 0x2e, 0x47, 0xa8, 0x85, 0x10, 0x9d, 0x42, 0xa8, 0xbd, 0x75, 0x08, 0x6d, 0xb5, 0xb9, - 0xb2, 0x15, 0xe8, 0x4a, 0x1d, 0x10, 0x65, 0xd7, 0xf0, 0x85, 0x68, 0xd8, 0xb0, 0x0d, 0x7c, 0x79, - 0xdc, 0x41, 0x6d, 0x13, 0x9f, 0x70, 0xd1, 0xb4, 0x2d, 0x32, 0xc8, 0x55, 0xe4, 0xdb, 0x75, 0x64, - 0xa4, 0x85, 0xc5, 0x1e, 0x6e, 0x3d, 0x82, 0x16, 0x3d, 0x1e, 0xe2, 0x1c, 0xb6, 0x2c, 0xe5, 0x49, - 0x8c, 0x5d, 0x43, 0x9b, 0xe2, 0xed, 0x82, 0xdb, 0x40, 0xdb, 0xd0, 0xeb, 0x08, 0xd8, 0x44, 0x15, - 0x84, 0x9d, 0x04, 0xc2, 0xda, 0xfa, 0xa9, 0x01, 0x6d, 0xb5, 0x6a, 0xb0, 0x1b, 0xb0, 0xa2, 0x8c, - 0x24, 0x51, 0x42, 0xe2, 0x3e, 0xcf, 0x05, 0xc2, 0x36, 0xe8, 0x82, 0x02, 0x34, 0xd1, 0xae, 0x2e, - 0x1f, 0xc7, 0x17, 0x5c, 0x62, 0x2c, 0xbc, 0x12, 0x37, 0x5b, 0x09, 0xd7, 0xf1, 0x00, 0xc2, 0x54, - 0x65, 0xec, 0x06, 0xbb, 0x05, 0x0c, 0xc1, 0x47, 0xc1, 0x08, 0x23, 0x59, 0xcc, 0xff, 0x99, 0xdd, - 0xdc, 0xfa, 0x08, 0xda, 0x6a, 0xcc, 0xd6, 0xf4, 0x50, 0xa8, 0x42, 0x0f, 0x81, 0xb0, 0x8d, 0xf2, - 0x62, 0x89, 0x31, 0xb7, 0x9e, 0xd2, 0x7a, 0x8a, 0x53, 0xaa, 0x66, 0x19, 0x89, 0x91, 0xe1, 0x75, - 0x1e, 0x24, 0xd2, 0xe1, 0x3c, 0x09, 0x3d, 0xbf, 0x08, 0xb0, 0x0b, 0x9e, 0xe6, 0xb6, 0x85, 0xcf, - 0x0f, 0xa3, 0x1f, 0x71, 0x1f, 0x23, 0x0c, 0xdd, 0x10, 0x64, 0xb9, 0xdd, 0xd8, 0x3a, 0x80, 0xee, - 0x53, 0xd5, 0x63, 0x0e, 0x13, 0x7c, 0x01, 0xa5, 0x5c, 0x89, 0xb5, 0x6b, 0x78, 0x27, 0x45, 0x67, - 0x81, 0xb5, 0x0d, 0xb6, 0x0a, 0x4b, 0xe8, 0x8d, 0x12, 0x65, 0x6e, 0x3d, 0x01, 0x36, 0x5b, 0x1d, - 0xd1, 0x68, 0xa5, 0xc2, 0x76, 0x0d, 0x35, 0xf9, 0x8c, 0x5f, 0xe2, 0x33, 0xf9, 0xf0, 0xe1, 0x28, - 0x8a, 0x53, 0x4e, 0x34, 0xe5, 0x43, 0xfa, 0xbe, 0x88, 0x08, 0x6b, 0xeb, 0xe9, 0x54, 0x1f, 0x39, - 0x4c, 0xb4, 0x70, 0x27, 0xd8, 0xae, 0x51, 0xf0, 0x91, 0x14, 0x81, 0x90, 0x06, 0x24, 0x31, 0x02, - 0x63, 0xe2, 0x45, 0xbb, 0x21, 0xf7, 0x52, 0x01, 0x5b, 0x3b, 0xff, 0x6a, 0x42, 0x53, 0x54, 0x05, - 0xf6, 0x11, 0x74, 0xb5, 0x5f, 0x68, 0x19, 0x15, 0xf9, 0xd9, 0xdf, 0x93, 0xd7, 0xff, 0x6f, 0x06, - 0x2f, 0x2a, 0x93, 0x53, 0x63, 0x1f, 0x02, 0x94, 0x8b, 0x37, 0xbb, 0x49, 0xd3, 0xdc, 0xf4, 0x22, - 0xbe, 0xde, 0xa7, 0x4f, 0x36, 0x73, 0x7e, 0x7d, 0x76, 0x6a, 0xec, 0x7b, 0xb0, 0x24, 0xcb, 0x9f, - 0x08, 0x2d, 0x36, 0xd0, 0xd6, 0xa6, 0x39, 0x2b, 0xf5, 0xb5, 0xc2, 0x3e, 0x29, 0x84, 0x89, 0xf0, - 0x61, 0xfd, 0x39, 0x3b, 0x98, 0x10, 0xf3, 0xca, 0xc2, 0xed, 0xcc, 0xa9, 0xb1, 0x7d, 0xe8, 0x8a, - 0x1d, 0x4a, 0x14, 0xf5, 0xdb, 0xc8, 0xbb, 0x68, 0xa9, 0xba, 0x56, 0xa1, 0x5d, 0xe8, 0xe9, 0x6b, - 0x0f, 0x23, 0x4b, 0xce, 0xd9, 0x8f, 0x84, 0x90, 0x79, 0x1b, 0x92, 0x53, 0x63, 0x1e, 0xdc, 0x9a, - 0xbf, 0xbc, 0xb0, 0xd7, 0xcb, 0x6f, 0xcb, 0x0b, 0xb6, 0xa5, 0x75, 0xe7, 0x3a, 0x96, 0xe2, 0x8a, - 0x1f, 0x40, 0xbf, 0xb8, 0xbc, 0x08, 0x6b, 0x19, 0x15, 0x03, 0xa9, 0xda, 0x82, 0x7d, 0x67, 0xfd, - 0xb5, 0x85, 0xf4, 0x42, 0xfc, 0x31, 0xac, 0x96, 0x0c, 0xb1, 0x30, 0x1f, 0xbb, 0x33, 0x73, 0xae, - 0x62, 0xd6, 0xc1, 0x22, 0x72, 0x21, 0xf5, 0x87, 0xe5, 0xc6, 0x5e, 0x95, 0xfc, 0xba, 0xee, 0xdb, - 0xf9, 0xd2, 0x9d, 0xeb, 0x58, 0x8a, 0x1b, 0x1e, 0xc3, 0x4a, 0xa5, 0x9f, 0x2a, 0xd9, 0xd7, 0x36, - 0xd9, 0xeb, 0x02, 0xe2, 0x5e, 0xff, 0x8b, 0xaf, 0x06, 0xc6, 0x97, 0x5f, 0x0d, 0x8c, 0x7f, 0x7c, - 0x35, 0x30, 0x7e, 0xfe, 0x7c, 0x50, 0xfb, 0xf2, 0xf9, 0xa0, 0xf6, 0xb7, 0xe7, 0x83, 0xda, 0x49, - 0x93, 0xfe, 0xd5, 0xf1, 0xcd, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0x31, 0xe0, 0x21, 0xb2, 0xe7, - 0x21, 0x00, 0x00, + // 2972 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x3a, 0x4d, 0x6f, 0xdc, 0xd6, + 0xb5, 0x43, 0x72, 0x3e, 0xcf, 0x8c, 0x24, 0xea, 0x4a, 0xf6, 0x9b, 0x28, 0xf6, 0x44, 0xa1, 0x83, + 0x3c, 0x45, 0xc8, 0x13, 0x12, 0xbd, 0x3c, 0xe4, 0x21, 0x40, 0x9b, 0xc4, 0x92, 0x23, 0x3b, 0x95, + 0x23, 0x9b, 0x52, 0xdc, 0x55, 0x81, 0x52, 0x9c, 0xab, 0x11, 0x2b, 0x0e, 0x49, 0x93, 0x1c, 0x09, + 0x5a, 0x14, 0xdd, 0x75, 0xdb, 0x6e, 0x5a, 0xa0, 0x45, 0x37, 0x2d, 0xd0, 0x6d, 0x17, 0xfd, 0x01, + 0xdd, 0xb5, 0xcd, 0x32, 0xe8, 0xaa, 0xab, 0xa2, 0x48, 0xfe, 0x45, 0x17, 0x45, 0x71, 0xce, 0xbd, + 0x97, 0xbc, 0x9c, 0x0f, 0x39, 0x2e, 0xd0, 0x1d, 0xcf, 0xc7, 0x3d, 0xf7, 0xf0, 0x7c, 0x1f, 0xce, + 0xc0, 0xf2, 0x70, 0x7c, 0x15, 0xa7, 0x17, 0x3c, 0xdd, 0x49, 0xd2, 0x38, 0x8f, 0x99, 0x99, 0x9c, + 0x3a, 0x5b, 0xc0, 0x9e, 0x4e, 0x78, 0x7a, 0x7d, 0x9c, 0x7b, 0xf9, 0x24, 0x73, 0xf9, 0xf3, 0x09, + 0xcf, 0x72, 0xc6, 0xa0, 0x1e, 0x79, 0x63, 0xde, 0x37, 0x36, 0x8d, 0xad, 0x8e, 0x4b, 0xcf, 0x4e, + 0x02, 0xeb, 0x7b, 0xf1, 0x78, 0x1c, 0x47, 0xdf, 0x25, 0x19, 0x2e, 0xcf, 0x92, 0x38, 0xca, 0x38, + 0xbb, 0x0d, 0xcd, 0x94, 0x67, 0x93, 0x30, 0x27, 0xee, 0xb6, 0x2b, 0x21, 0x66, 0x83, 0x35, 0xce, + 0x46, 0x7d, 0x93, 0x44, 0xe0, 0x23, 0x72, 0x66, 0xf1, 0x24, 0xf5, 0x79, 0xdf, 0x22, 0xa4, 0x84, + 0x10, 0x2f, 0xf4, 0xea, 0xd7, 0x05, 0x5e, 0x40, 0xce, 0xef, 0x0c, 0x58, 0xab, 0x28, 0xf7, 0xd2, + 0x37, 0xbe, 0x07, 0x3d, 0x71, 0x87, 0x90, 0x40, 0xf7, 0x76, 0x77, 0xed, 0x9d, 0xe4, 0x74, 0xe7, + 0x58, 0xc3, 0xbb, 0x15, 0x2e, 0xf6, 0x3e, 0x2c, 0x65, 0x93, 0xd3, 0x13, 0x2f, 0xbb, 0x90, 0xc7, + 0xea, 0x9b, 0xd6, 0x56, 0x77, 0x77, 0x95, 0x8e, 0xe9, 0x04, 0xb7, 0xca, 0xe7, 0xfc, 0xd6, 0x80, + 0xee, 0xde, 0x39, 0xf7, 0x25, 0x8c, 0x8a, 0x26, 0x5e, 0x96, 0xf1, 0xa1, 0x52, 0x54, 0x40, 0x6c, + 0x1d, 0x1a, 0x79, 0x9c, 0x7b, 0x21, 0xa9, 0xda, 0x70, 0x05, 0xc0, 0x06, 0x00, 0xd9, 0xc4, 0xf7, + 0x79, 0x96, 0x9d, 0x4d, 0x42, 0x52, 0xb5, 0xe1, 0x6a, 0x18, 0x94, 0x76, 0xe6, 0x05, 0x21, 0x1f, + 0x92, 0x99, 0x1a, 0xae, 0x84, 0x58, 0x1f, 0x5a, 0x57, 0x5e, 0x1a, 0x05, 0xd1, 0xa8, 0xdf, 0x20, + 0x82, 0x02, 0xf1, 0xc4, 0x90, 0xe7, 0x5e, 0x10, 0xf6, 0x9b, 0x9b, 0xc6, 0x56, 0xcf, 0x95, 0x90, + 0xf3, 0x4f, 0x03, 0x60, 0x7f, 0x32, 0x4e, 0xa4, 0x9a, 0x9b, 0xd0, 0x25, 0x0d, 0x4e, 0xbc, 0xd3, + 0x90, 0x67, 0xa4, 0xab, 0xe5, 0xea, 0x28, 0xb6, 0x05, 0x2b, 0x7e, 0x3c, 0x4e, 0x42, 0x9e, 0xf3, + 0xa1, 0xe4, 0x42, 0xd5, 0x0d, 0x77, 0x1a, 0xcd, 0xde, 0x80, 0xa5, 0xb3, 0x20, 0x0a, 0xb2, 0x73, + 0x3e, 0xbc, 0x7f, 0x9d, 0x73, 0x61, 0x72, 0xc3, 0xad, 0x22, 0x99, 0x03, 0x3d, 0x85, 0x70, 0xe3, + 0xab, 0x8c, 0x5e, 0xc8, 0x70, 0x2b, 0x38, 0xf6, 0x36, 0xac, 0xf2, 0x2c, 0x0f, 0xc6, 0x5e, 0xce, + 0x4f, 0x50, 0x15, 0x62, 0x6c, 0x10, 0xe3, 0x2c, 0x01, 0x7d, 0x7f, 0x9a, 0x64, 0xf4, 0x9e, 0x96, + 0x8b, 0x8f, 0x6c, 0x03, 0xda, 0x49, 0x1a, 0x8f, 0x52, 0x9e, 0x65, 0xfd, 0x16, 0x85, 0x44, 0x01, + 0x3b, 0x5f, 0x18, 0x00, 0x87, 0xb1, 0x37, 0x94, 0x06, 0x98, 0x51, 0x5a, 0x98, 0x60, 0x4a, 0xe9, + 0x01, 0x00, 0xd9, 0x44, 0xb0, 0x98, 0xc4, 0xa2, 0x61, 0x2a, 0x17, 0x5a, 0xd5, 0x0b, 0xf1, 0xec, + 0x98, 0xe7, 0xde, 0xfd, 0x20, 0x0a, 0xe3, 0x91, 0x0c, 0x73, 0x0d, 0xc3, 0xde, 0x84, 0xe5, 0x12, + 0x3a, 0x38, 0x79, 0xb4, 0x4f, 0x6f, 0xda, 0x71, 0xa7, 0xb0, 0xb3, 0xaf, 0xe9, 0xfc, 0xcc, 0x80, + 0xa5, 0xe3, 0x73, 0x2f, 0x1d, 0x06, 0xd1, 0xe8, 0x20, 0x8d, 0x27, 0x09, 0x7a, 0x3d, 0xf7, 0xd2, + 0x11, 0xcf, 0x65, 0xfa, 0x4a, 0x08, 0x93, 0x7a, 0x7f, 0xff, 0x10, 0x35, 0xb7, 0x30, 0xa9, 0xf1, + 0x59, 0xbc, 0x79, 0x9a, 0xe5, 0x87, 0xb1, 0xef, 0xe5, 0x41, 0x1c, 0x49, 0xc5, 0xab, 0x48, 0x4a, + 0xdc, 0xeb, 0xc8, 0xa7, 0xc8, 0xb3, 0x28, 0x71, 0x09, 0xc2, 0x37, 0x9e, 0x44, 0x92, 0xd2, 0x20, + 0x4a, 0x01, 0x3b, 0x7f, 0x6c, 0x00, 0x1c, 0x5f, 0x47, 0xfe, 0x54, 0x8c, 0x3d, 0xb8, 0xe4, 0x51, + 0x5e, 0x8d, 0x31, 0x81, 0x42, 0x61, 0x22, 0xe4, 0x12, 0x65, 0xdc, 0x02, 0x66, 0x77, 0xa0, 0x93, + 0x72, 0x9f, 0x47, 0x39, 0x12, 0x2d, 0x22, 0x96, 0x08, 0x8c, 0xa6, 0xb1, 0x97, 0xe5, 0x3c, 0xad, + 0x98, 0xb7, 0x82, 0x63, 0xdb, 0x60, 0xeb, 0xf0, 0x41, 0x1e, 0x0c, 0xa5, 0x89, 0x67, 0xf0, 0x28, + 0x8f, 0x5e, 0x42, 0xc9, 0x6b, 0x0a, 0x79, 0x3a, 0x0e, 0xe5, 0xe9, 0x30, 0xc9, 0x13, 0x51, 0x36, + 0x83, 0x47, 0x79, 0xa7, 0x61, 0xec, 0x5f, 0x04, 0xd1, 0x88, 0x1c, 0xd0, 0x26, 0x53, 0x55, 0x70, + 0xec, 0x5b, 0x60, 0x4f, 0xa2, 0x94, 0x67, 0x71, 0x78, 0xc9, 0x87, 0xe4, 0xc7, 0xac, 0xdf, 0xd1, + 0xca, 0x8e, 0xee, 0x61, 0x77, 0x86, 0x55, 0xf3, 0x10, 0x88, 0x4a, 0x23, 0x3d, 0x34, 0x00, 0x38, + 0x25, 0x45, 0x4e, 0xae, 0x13, 0xde, 0xef, 0x8a, 0xb8, 0x2b, 0x31, 0xec, 0x1d, 0x58, 0xcb, 0xb8, + 0x1f, 0x47, 0xc3, 0xec, 0x3e, 0x3f, 0x0f, 0xa2, 0xe1, 0x63, 0xb2, 0x45, 0xbf, 0x47, 0x26, 0x9e, + 0x47, 0xc2, 0x88, 0x21, 0xc5, 0xf7, 0xf7, 0x0f, 0x8f, 0xae, 0x22, 0x9e, 0xf6, 0x97, 0x44, 0xc4, + 0x54, 0x90, 0xe8, 0x6e, 0x3f, 0x8e, 0xce, 0xc2, 0xc0, 0xcf, 0x1f, 0x67, 0xa3, 0xfe, 0x32, 0xf1, + 0xe8, 0x28, 0x74, 0x69, 0x5e, 0xa4, 0xf5, 0x8a, 0x70, 0x69, 0x81, 0x28, 0x82, 0xc1, 0x4d, 0xb2, + 0xbe, 0xad, 0x05, 0x83, 0xab, 0x07, 0x03, 0x12, 0x57, 0xf5, 0x60, 0x70, 0x45, 0x30, 0x04, 0xf1, + 0x49, 0x99, 0xa7, 0x6c, 0xd3, 0xd8, 0xaa, 0xbb, 0x15, 0x1c, 0x3a, 0x6f, 0x38, 0x19, 0x27, 0x8f, + 0x8e, 0x34, 0xbe, 0x35, 0xe2, 0x9b, 0xc1, 0x3b, 0xbf, 0x32, 0xa0, 0xa7, 0xf7, 0x0a, 0xad, 0x8b, + 0x19, 0x0b, 0xba, 0x98, 0xa9, 0x77, 0x31, 0xf6, 0x56, 0xd1, 0xad, 0x44, 0xf7, 0x21, 0x7f, 0x3e, + 0x49, 0x63, 0x2c, 0xeb, 0x2e, 0x11, 0x8a, 0x06, 0xf6, 0x2e, 0x74, 0x53, 0x1e, 0x7a, 0xd7, 0x45, + 0xdb, 0x41, 0xfe, 0x15, 0xe4, 0x77, 0x4b, 0xb4, 0xab, 0xf3, 0x38, 0x7f, 0x36, 0xa1, 0xab, 0x11, + 0x67, 0x72, 0xc1, 0xf8, 0x86, 0xb9, 0x60, 0x2e, 0xc8, 0x85, 0x4d, 0xa5, 0xd2, 0xe4, 0x74, 0x3f, + 0x48, 0x65, 0x79, 0xd0, 0x51, 0x05, 0x47, 0x25, 0xf9, 0x74, 0x14, 0x76, 0x0f, 0x0d, 0xd4, 0x52, + 0x6f, 0x1a, 0xcd, 0x76, 0x80, 0x11, 0x6a, 0xcf, 0xcb, 0xfd, 0xf3, 0xcf, 0x13, 0x19, 0x8d, 0x4d, + 0x0a, 0xe9, 0x39, 0x14, 0xf6, 0x1a, 0x34, 0xb2, 0xdc, 0x1b, 0x71, 0x4a, 0xbd, 0xe5, 0xdd, 0x0e, + 0xa5, 0x0a, 0x22, 0x5c, 0x81, 0xd7, 0x8c, 0xdf, 0x7e, 0x81, 0xf1, 0x9d, 0xdf, 0x5b, 0xb0, 0x54, + 0xe9, 0xee, 0xf3, 0xa6, 0xa0, 0xf2, 0x46, 0x73, 0xc1, 0x8d, 0x9b, 0x50, 0x9f, 0x44, 0x81, 0x70, + 0xf6, 0xf2, 0x6e, 0x0f, 0xe9, 0x9f, 0x47, 0x41, 0x8e, 0xd9, 0xe6, 0x12, 0x45, 0xd3, 0xa9, 0xfe, + 0xa2, 0x80, 0x78, 0x07, 0xd6, 0xca, 0x54, 0xdf, 0xdf, 0x3f, 0x3c, 0x8c, 0xfd, 0x8b, 0xa2, 0x37, + 0xcc, 0x23, 0x31, 0x26, 0x66, 0x20, 0x2a, 0x59, 0x0f, 0x6b, 0x62, 0x0a, 0xfa, 0x6f, 0x68, 0xf8, + 0x38, 0x95, 0x90, 0x95, 0x64, 0x40, 0x69, 0x63, 0xca, 0xc3, 0x9a, 0x2b, 0xe8, 0xec, 0x0d, 0xa8, + 0x63, 0xfc, 0x4b, 0x5b, 0x2d, 0x23, 0x5f, 0x39, 0x26, 0x3c, 0xac, 0xb9, 0x44, 0x45, 0xae, 0x30, + 0xf6, 0x86, 0xfd, 0x4e, 0xc9, 0x55, 0xf6, 0x52, 0xe4, 0x42, 0x2a, 0x72, 0x61, 0x0d, 0xa2, 0x7a, + 0x24, 0xb9, 0xca, 0x76, 0x80, 0x5c, 0x48, 0x65, 0xef, 0x01, 0x5c, 0x7a, 0x61, 0x30, 0x14, 0xcd, + 0xa7, 0x4b, 0xbc, 0xeb, 0xc8, 0xfb, 0xac, 0xc0, 0xca, 0xa8, 0xd7, 0xf8, 0xee, 0xb7, 0xa1, 0x99, + 0x89, 0xf0, 0xff, 0x36, 0xac, 0x56, 0x7c, 0x76, 0x18, 0x64, 0x64, 0x60, 0x41, 0xee, 0x1b, 0x8b, + 0x06, 0x37, 0x75, 0x7e, 0x00, 0x40, 0x96, 0x78, 0x90, 0xa6, 0x71, 0xaa, 0x06, 0x48, 0xa3, 0x18, + 0x20, 0x9d, 0xbb, 0xd0, 0x41, 0x0b, 0xdc, 0x40, 0xc6, 0x57, 0x5f, 0x44, 0x4e, 0xa0, 0x47, 0xef, + 0xfc, 0xf4, 0x70, 0x01, 0x07, 0xdb, 0x85, 0x75, 0x31, 0xc5, 0x89, 0x24, 0x78, 0x12, 0x67, 0x01, + 0x59, 0x42, 0xa4, 0xe3, 0x5c, 0x1a, 0xd6, 0x46, 0x8e, 0xe2, 0x8e, 0x9f, 0x1e, 0xaa, 0x39, 0x43, + 0xc1, 0xce, 0xff, 0x41, 0x07, 0x6f, 0x14, 0xd7, 0x6d, 0x41, 0x93, 0x08, 0xca, 0x0e, 0x76, 0xe1, + 0x04, 0xa9, 0x90, 0x2b, 0xe9, 0xce, 0x4f, 0x0c, 0xe8, 0x8a, 0x22, 0x27, 0x4e, 0xbe, 0x6c, 0x8d, + 0xdb, 0xac, 0x1c, 0x57, 0x55, 0x42, 0x97, 0xb8, 0x03, 0x40, 0x65, 0x4a, 0x30, 0xd4, 0xcb, 0xa0, + 0x28, 0xb1, 0xae, 0xc6, 0x81, 0x8e, 0x29, 0xa1, 0x39, 0xa6, 0xfd, 0x85, 0x09, 0x3d, 0xe9, 0x52, + 0xc1, 0xf2, 0x1f, 0x4a, 0x56, 0x99, 0x4f, 0x75, 0x3d, 0x9f, 0xde, 0x54, 0xf9, 0xd4, 0x28, 0x5f, + 0xa3, 0x8c, 0xa2, 0x32, 0x9d, 0xee, 0xc9, 0x74, 0x6a, 0x12, 0xdb, 0x92, 0x4a, 0x27, 0xc5, 0x25, + 0xb2, 0xe9, 0x9e, 0xcc, 0xa6, 0x56, 0xc9, 0x54, 0x84, 0x54, 0x91, 0x4c, 0xf7, 0x64, 0x32, 0xb5, + 0x4b, 0xa6, 0xc2, 0xcd, 0x2a, 0x97, 0xee, 0xb7, 0xa0, 0x41, 0xee, 0x74, 0x3e, 0x00, 0x5b, 0x37, + 0x0d, 0xe5, 0xc4, 0x9b, 0x92, 0x58, 0x09, 0x05, 0x8d, 0xc9, 0x95, 0x67, 0x9f, 0xc3, 0x52, 0xa5, + 0x14, 0xe1, 0x04, 0x11, 0x64, 0x7b, 0x5e, 0xe4, 0xf3, 0xb0, 0xd8, 0x63, 0x34, 0x8c, 0x16, 0x64, + 0x66, 0x29, 0x59, 0x8a, 0xa8, 0x04, 0x99, 0xb6, 0x8d, 0x58, 0x95, 0x6d, 0xe4, 0x2f, 0x06, 0xf4, + 0xf4, 0x03, 0xb8, 0xd0, 0x3c, 0x48, 0xd3, 0xbd, 0x78, 0x28, 0xbc, 0xd9, 0x70, 0x15, 0x88, 0xa1, + 0x8f, 0x8f, 0xa1, 0x97, 0x65, 0x32, 0x02, 0x0b, 0x58, 0xd2, 0x8e, 0xfd, 0x38, 0x51, 0xfb, 0x65, + 0x01, 0x4b, 0xda, 0x21, 0xbf, 0xe4, 0xa1, 0x6c, 0x50, 0x05, 0x8c, 0xb7, 0x3d, 0xe6, 0x59, 0x86, + 0x61, 0x22, 0xea, 0xaa, 0x02, 0xf1, 0x94, 0xeb, 0x5d, 0xed, 0x79, 0x93, 0x8c, 0xcb, 0x19, 0xb0, + 0x80, 0xd1, 0x2c, 0xb8, 0x07, 0x7b, 0x69, 0x3c, 0x89, 0xd4, 0xe4, 0xa7, 0x61, 0x9c, 0x2b, 0x58, + 0x7d, 0x32, 0x49, 0x47, 0x9c, 0x82, 0x58, 0xad, 0xd5, 0x1b, 0xd0, 0x0e, 0x22, 0xcf, 0xcf, 0x83, + 0x4b, 0x2e, 0x2d, 0x59, 0xc0, 0x18, 0xbf, 0x79, 0x30, 0xe6, 0x72, 0xf4, 0xa5, 0x67, 0xe4, 0x3f, + 0x0b, 0x42, 0x4e, 0x71, 0x2d, 0x5f, 0x49, 0xc1, 0x94, 0xa2, 0xa2, 0x27, 0xcb, 0xa5, 0x59, 0x40, + 0xce, 0x2f, 0x4d, 0xd8, 0x38, 0x4a, 0x78, 0xea, 0xe5, 0x5c, 0x2c, 0xea, 0xc7, 0xfe, 0x39, 0x1f, + 0x7b, 0x4a, 0x85, 0x3b, 0x60, 0xc6, 0x09, 0x5d, 0x2e, 0xe3, 0x5d, 0x90, 0x8f, 0x12, 0xd7, 0x8c, + 0x13, 0x52, 0xc2, 0xcb, 0x2e, 0xa4, 0x6d, 0xe9, 0x79, 0xe1, 0xd6, 0xbe, 0x01, 0xed, 0xa1, 0x97, + 0x7b, 0xa7, 0x5e, 0xc6, 0x95, 0x4d, 0x15, 0x4c, 0x0b, 0x2e, 0xee, 0x83, 0xd2, 0xa2, 0x02, 0x20, + 0x49, 0x74, 0x9b, 0xb4, 0xa6, 0x84, 0x90, 0xfb, 0x2c, 0x9c, 0x64, 0xe7, 0x64, 0xc6, 0xb6, 0x2b, + 0x00, 0xd4, 0xa5, 0x88, 0xf9, 0xb6, 0x6c, 0x17, 0x03, 0x80, 0xb3, 0x34, 0x1e, 0x8b, 0xc2, 0x42, + 0x0d, 0xa8, 0xed, 0x6a, 0x18, 0x45, 0x3f, 0x11, 0xeb, 0x0f, 0x94, 0x74, 0x81, 0x71, 0x72, 0x58, + 0x7a, 0xf6, 0xae, 0x0c, 0xfb, 0xc7, 0x3c, 0xf7, 0xd8, 0x86, 0x66, 0x0e, 0x40, 0x73, 0x20, 0x45, + 0x1a, 0xe3, 0x85, 0xd5, 0x43, 0x95, 0x1c, 0x4b, 0x2b, 0x39, 0xca, 0x82, 0x75, 0x0a, 0x71, 0x7a, + 0x76, 0xde, 0x83, 0x75, 0xe9, 0x91, 0x67, 0xef, 0xe2, 0xad, 0x0b, 0x7d, 0x21, 0xc8, 0xe2, 0x7a, + 0xe7, 0x4f, 0x06, 0xdc, 0x9a, 0x3a, 0xf6, 0xd2, 0xdf, 0x3f, 0xde, 0x87, 0x3a, 0x2e, 0x90, 0x7d, + 0x8b, 0x52, 0xf3, 0x1e, 0xde, 0x31, 0x57, 0xe4, 0x0e, 0x02, 0x0f, 0xa2, 0x3c, 0xbd, 0x76, 0xe9, + 0xc0, 0xc6, 0xa7, 0xd0, 0x29, 0x50, 0x28, 0xf7, 0x82, 0x5f, 0xab, 0xea, 0x7b, 0xc1, 0xaf, 0x71, + 0xa2, 0xb8, 0xf4, 0xc2, 0x89, 0x30, 0x8d, 0x6c, 0xb0, 0x15, 0xc3, 0xba, 0x82, 0xfe, 0x81, 0xf9, + 0xff, 0x86, 0xf3, 0x43, 0xe8, 0x3f, 0xf4, 0xa2, 0x61, 0x28, 0xe3, 0x51, 0x14, 0x05, 0x69, 0x82, + 0x57, 0x35, 0x13, 0x74, 0x51, 0x0a, 0x51, 0x6f, 0x88, 0xc6, 0x3b, 0xd0, 0x39, 0x55, 0xed, 0x50, + 0x1a, 0xbe, 0x44, 0x50, 0xcc, 0x3c, 0x0f, 0x33, 0xb9, 0xa6, 0xd2, 0xb3, 0x73, 0x0b, 0xd6, 0x0e, + 0x78, 0x2e, 0xee, 0xde, 0x3b, 0x1b, 0xc9, 0x9b, 0x9d, 0x2d, 0x58, 0xaf, 0xa2, 0xa5, 0x71, 0x6d, + 0xb0, 0xfc, 0xb3, 0xa2, 0xd5, 0xf8, 0x67, 0x23, 0xe7, 0x18, 0xee, 0x8a, 0x69, 0x69, 0x72, 0x8a, + 0x2a, 0x60, 0xe9, 0xfb, 0x3c, 0x19, 0x7a, 0x39, 0x57, 0x2f, 0xb1, 0x0b, 0xeb, 0x99, 0xa0, 0xed, + 0x9d, 0x8d, 0x4e, 0xe2, 0x71, 0x78, 0x9c, 0xa7, 0x41, 0xa4, 0x64, 0xcc, 0xa5, 0x39, 0x87, 0x30, + 0x58, 0x24, 0x54, 0x2a, 0xd2, 0x87, 0x96, 0xfc, 0xf8, 0x23, 0xdd, 0xac, 0xc0, 0x59, 0x3f, 0x3b, + 0x23, 0xd8, 0x38, 0xe0, 0xf9, 0xcc, 0xcc, 0x54, 0x96, 0x1d, 0xbc, 0xe3, 0xb3, 0xb2, 0x3d, 0x16, + 0x30, 0xfb, 0x1f, 0xe8, 0x9d, 0x05, 0x61, 0xce, 0x53, 0xb9, 0x73, 0xcc, 0xc4, 0x7a, 0x85, 0xec, + 0xfc, 0xcd, 0x02, 0x7b, 0xfa, 0x9a, 0xc2, 0x4f, 0xc6, 0xdc, 0xaa, 0x61, 0x56, 0xaa, 0x06, 0x83, + 0xfa, 0x18, 0x0b, 0xbb, 0xcc, 0x19, 0x7c, 0x2e, 0x13, 0xad, 0xbe, 0x20, 0xd1, 0xb6, 0x60, 0x45, + 0x4e, 0x7f, 0xb1, 0xda, 0x6b, 0xe4, 0x02, 0x31, 0x85, 0xc6, 0x81, 0x79, 0x0a, 0x45, 0xeb, 0x86, + 0xa8, 0x37, 0xf3, 0x48, 0xda, 0x34, 0xde, 0xfa, 0x06, 0xd3, 0x78, 0x22, 0x08, 0xe2, 0x13, 0x95, + 0x34, 0x59, 0x5b, 0x08, 0x9f, 0x43, 0x62, 0x6f, 0xc3, 0x6a, 0xc2, 0x23, 0x5c, 0xdc, 0x35, 0xfe, + 0x0e, 0xf1, 0xcf, 0x12, 0xf0, 0x35, 0xa9, 0x55, 0x6a, 0xbc, 0x20, 0x5e, 0x73, 0x0a, 0x8d, 0x1b, + 0x9c, 0x3f, 0xc9, 0xe3, 0x4b, 0xb5, 0xaa, 0x61, 0x32, 0x88, 0xe5, 0x7e, 0x06, 0x8f, 0x3a, 0x54, + 0x70, 0x64, 0x90, 0x9e, 0xd0, 0x61, 0x86, 0xe0, 0xfc, 0xc6, 0x80, 0x5b, 0xa5, 0x83, 0xe9, 0xa3, + 0xde, 0x0b, 0xf6, 0xde, 0x0d, 0x68, 0x67, 0xa9, 0x4f, 0x9c, 0xaa, 0x27, 0x2b, 0x98, 0x7a, 0x44, + 0x96, 0x0b, 0x9a, 0x6c, 0x60, 0x0a, 0x7e, 0xb1, 0xd7, 0xfb, 0xd0, 0x1a, 0x57, 0x1b, 0xb3, 0x04, + 0x9d, 0x3f, 0x18, 0xf0, 0xea, 0xdc, 0x78, 0xff, 0x37, 0x3e, 0x10, 0x43, 0x11, 0x14, 0x99, 0x2c, + 0x93, 0x37, 0xef, 0x1f, 0x38, 0xc9, 0x7c, 0x08, 0x4b, 0x79, 0x69, 0x19, 0xae, 0x3e, 0x10, 0xbf, + 0x52, 0x3d, 0xa8, 0x19, 0xcf, 0xad, 0xf2, 0x3b, 0x17, 0xf0, 0x4a, 0x45, 0xff, 0x4a, 0x4d, 0xdc, + 0xa5, 0xf9, 0x1e, 0x79, 0xb9, 0xac, 0x8c, 0xb7, 0x35, 0xc1, 0x62, 0x9e, 0x26, 0xaa, 0x5b, 0xf0, + 0x55, 0x52, 0xdc, 0xac, 0xa6, 0xb8, 0xf3, 0x6b, 0x13, 0x56, 0xa6, 0xae, 0x62, 0xcb, 0x60, 0x06, + 0x43, 0xe9, 0x48, 0x33, 0x18, 0x2e, 0x4c, 0x57, 0xdd, 0xb9, 0xd6, 0x94, 0x73, 0xb1, 0x40, 0xa5, + 0xfe, 0xbe, 0x97, 0x7b, 0xb2, 0xff, 0x2b, 0xb0, 0xe2, 0xf6, 0xc6, 0x94, 0xdb, 0xfb, 0xd0, 0x1a, + 0x66, 0x39, 0x9d, 0x12, 0x59, 0xa9, 0x40, 0x2c, 0xed, 0x14, 0xe7, 0xf4, 0xa9, 0x4a, 0x4c, 0x54, + 0x25, 0x82, 0xed, 0x14, 0x4b, 0x5d, 0xfb, 0x46, 0x9b, 0x48, 0xae, 0x62, 0x9e, 0xea, 0xc8, 0xa2, + 0x84, 0xf3, 0x94, 0x16, 0x51, 0x50, 0x8d, 0xa8, 0xe7, 0x53, 0x05, 0x54, 0x3a, 0xe4, 0xa5, 0xe3, + 0xe9, 0x2d, 0x35, 0x66, 0x8b, 0x50, 0x5a, 0xab, 0x46, 0x44, 0x65, 0xd2, 0xfe, 0xb9, 0x01, 0x77, + 0x55, 0x33, 0x9e, 0x1f, 0x08, 0xf7, 0xb4, 0xe6, 0x38, 0x2b, 0x49, 0x36, 0x49, 0x9a, 0xcf, 0x3f, + 0x0e, 0x43, 0xb1, 0x58, 0x99, 0x6a, 0x3e, 0x57, 0x98, 0x4a, 0x64, 0x58, 0x53, 0xc5, 0x7f, 0x9d, + 0xb4, 0x7d, 0x24, 0x7e, 0x50, 0xa8, 0xbb, 0x02, 0x70, 0x3e, 0x85, 0xc1, 0x22, 0xbd, 0x5e, 0xd6, + 0x1e, 0xce, 0x35, 0xdc, 0x15, 0x6d, 0xad, 0x14, 0xa5, 0x7e, 0x3e, 0x7a, 0x71, 0x6f, 0xaa, 0xf4, + 0x7a, 0x73, 0xba, 0xd7, 0x17, 0x9f, 0x36, 0xe9, 0x73, 0xb9, 0xa5, 0x7f, 0xda, 0x44, 0xcc, 0xf6, + 0x05, 0x34, 0xc5, 0x30, 0xc7, 0x96, 0xa0, 0xf3, 0x28, 0xa2, 0xf4, 0x3d, 0x4a, 0xec, 0x1a, 0x6b, + 0x43, 0xfd, 0x38, 0x8f, 0x13, 0xdb, 0x60, 0x1d, 0x68, 0x3c, 0xc1, 0x69, 0xde, 0x36, 0x19, 0x40, + 0x13, 0xab, 0xfd, 0x98, 0xdb, 0x16, 0xa2, 0x8f, 0x73, 0x2f, 0xcd, 0xed, 0x3a, 0xa2, 0x85, 0xfe, + 0x76, 0x83, 0x2d, 0x03, 0x7c, 0x3c, 0xc9, 0x63, 0xc9, 0xd6, 0x44, 0xda, 0x3e, 0x0f, 0x79, 0xce, + 0xed, 0xd6, 0xf6, 0x8f, 0xe8, 0xc8, 0x08, 0xc7, 0x87, 0x9e, 0xbc, 0x8b, 0x60, 0xbb, 0xc6, 0x5a, + 0x60, 0x7d, 0xc6, 0xaf, 0x6c, 0x83, 0x75, 0xa1, 0xe5, 0x4e, 0xa2, 0x28, 0x88, 0x46, 0xe2, 0x3e, + 0xba, 0x7a, 0x68, 0x5b, 0x48, 0x40, 0x85, 0x12, 0x3e, 0xb4, 0xeb, 0xac, 0x07, 0xed, 0x4f, 0xe4, + 0xcf, 0x0e, 0x76, 0x03, 0x49, 0xc8, 0x86, 0x67, 0x9a, 0x48, 0xa2, 0xcb, 0x11, 0x6a, 0x21, 0x44, + 0xa7, 0x10, 0x6a, 0x6f, 0x1f, 0x41, 0x5b, 0x6d, 0xae, 0x6c, 0x05, 0xba, 0x52, 0x07, 0x44, 0xd9, + 0x35, 0x7c, 0x21, 0x1a, 0x36, 0x6c, 0x03, 0x5f, 0x1e, 0x77, 0x50, 0xdb, 0xc4, 0x27, 0x5c, 0x34, + 0x6d, 0x8b, 0x0c, 0x72, 0x1d, 0xf9, 0x76, 0x1d, 0x19, 0x69, 0x61, 0xb1, 0x87, 0xdb, 0x8f, 0xa1, + 0x45, 0x8f, 0x47, 0x38, 0x87, 0x2d, 0x4b, 0x79, 0x12, 0x63, 0xd7, 0xd0, 0xa6, 0x78, 0xbb, 0xe0, + 0x36, 0xd0, 0x36, 0xf4, 0x3a, 0x02, 0x36, 0x51, 0x05, 0x61, 0x27, 0x81, 0xb0, 0xb6, 0x7f, 0x6c, + 0x40, 0x5b, 0xad, 0x1a, 0x6c, 0x0d, 0x56, 0x94, 0x91, 0x24, 0x4a, 0x48, 0x3c, 0xe0, 0xb9, 0x40, + 0xd8, 0x06, 0x5d, 0x50, 0x80, 0x26, 0xda, 0xd5, 0xe5, 0xe3, 0xf8, 0x92, 0x4b, 0x8c, 0x85, 0x57, + 0xe2, 0x66, 0x2b, 0xe1, 0x3a, 0x1e, 0x40, 0x98, 0xaa, 0x8c, 0xdd, 0x60, 0xb7, 0x81, 0x21, 0xf8, + 0x38, 0x18, 0x61, 0x24, 0x8b, 0xf9, 0x3f, 0xb3, 0x9b, 0xdb, 0x1f, 0x41, 0x5b, 0x8d, 0xd9, 0x9a, + 0x1e, 0x0a, 0x55, 0xe8, 0x21, 0x10, 0xb6, 0x51, 0x5e, 0x2c, 0x31, 0xe6, 0xf6, 0x33, 0x5a, 0x4f, + 0x71, 0x4a, 0xd5, 0x2c, 0x23, 0x31, 0x32, 0xbc, 0x2e, 0x82, 0x44, 0x3a, 0x9c, 0x27, 0xa1, 0xe7, + 0x17, 0x01, 0x76, 0xc9, 0xd3, 0xdc, 0xb6, 0xf0, 0xf9, 0x51, 0xf4, 0x03, 0xee, 0x63, 0x84, 0xa1, + 0x1b, 0x82, 0x2c, 0xb7, 0x1b, 0xdb, 0x87, 0xd0, 0x7d, 0xa6, 0x7a, 0xcc, 0x51, 0x82, 0x2f, 0xa0, + 0x94, 0x2b, 0xb1, 0x76, 0x0d, 0xef, 0xa4, 0xe8, 0x2c, 0xb0, 0xb6, 0xc1, 0x56, 0x61, 0x09, 0xbd, + 0x51, 0xa2, 0xcc, 0xed, 0xa7, 0xc0, 0x66, 0xab, 0x23, 0x1a, 0xad, 0x54, 0xd8, 0xae, 0xa1, 0x26, + 0x9f, 0xf1, 0x2b, 0x7c, 0x26, 0x1f, 0x3e, 0x1a, 0x45, 0x71, 0xca, 0x89, 0xa6, 0x7c, 0x48, 0xdf, + 0x17, 0x11, 0x61, 0x6d, 0x3f, 0x9b, 0xea, 0x23, 0x47, 0x89, 0x16, 0xee, 0x04, 0xdb, 0x35, 0x0a, + 0x3e, 0x92, 0x22, 0x10, 0xd2, 0x80, 0x24, 0x46, 0x60, 0x4c, 0xbc, 0x68, 0x2f, 0xe4, 0x5e, 0x2a, + 0x60, 0x6b, 0xf7, 0x1f, 0x4d, 0x68, 0x8a, 0xaa, 0xc0, 0x3e, 0x82, 0xae, 0xf6, 0x8b, 0x2f, 0xa3, + 0x22, 0x3f, 0xfb, 0xfb, 0xf4, 0xc6, 0x7f, 0xcd, 0xe0, 0x45, 0x65, 0x72, 0x6a, 0xec, 0x43, 0x80, + 0x72, 0xf1, 0x66, 0xb7, 0x68, 0x9a, 0x9b, 0x5e, 0xc4, 0x37, 0xfa, 0xf4, 0xc9, 0x66, 0xce, 0xaf, + 0xd9, 0x4e, 0x8d, 0x7d, 0x07, 0x96, 0x64, 0xf9, 0x13, 0xa1, 0xc5, 0x06, 0xda, 0xda, 0x34, 0x67, + 0xa5, 0xbe, 0x51, 0xd8, 0x27, 0x85, 0x30, 0x11, 0x3e, 0xac, 0x3f, 0x67, 0x07, 0x13, 0x62, 0x5e, + 0x59, 0xb8, 0x9d, 0x39, 0x35, 0x76, 0x00, 0x5d, 0xb1, 0x43, 0x89, 0xa2, 0x7e, 0x07, 0x79, 0x17, + 0x2d, 0x55, 0x37, 0x2a, 0xb4, 0x07, 0x3d, 0x7d, 0xed, 0x61, 0x64, 0xc9, 0x39, 0xfb, 0x91, 0x10, + 0x32, 0x6f, 0x43, 0x72, 0x6a, 0xcc, 0x83, 0xdb, 0xf3, 0x97, 0x17, 0xf6, 0x7a, 0xf9, 0x6d, 0x79, + 0xc1, 0xb6, 0xb4, 0xe1, 0xdc, 0xc4, 0x52, 0x5c, 0xf1, 0x3d, 0xe8, 0x17, 0x97, 0x17, 0x61, 0x2d, + 0xa3, 0x62, 0x20, 0x55, 0x5b, 0xb0, 0xef, 0x6c, 0xbc, 0xb6, 0x90, 0x5e, 0x88, 0x3f, 0x81, 0xd5, + 0x92, 0x21, 0x16, 0xe6, 0x63, 0x77, 0x67, 0xce, 0x55, 0xcc, 0x3a, 0x58, 0x44, 0x2e, 0xa4, 0x7e, + 0xbf, 0xdc, 0xd8, 0xab, 0x92, 0x5f, 0xd7, 0x7d, 0x3b, 0x5f, 0xba, 0x73, 0x13, 0x4b, 0x71, 0xc3, + 0x13, 0x58, 0xa9, 0xf4, 0x53, 0x25, 0xfb, 0xc6, 0x26, 0x7b, 0x53, 0x40, 0xdc, 0xef, 0x7f, 0xf1, + 0xd5, 0xc0, 0xf8, 0xf2, 0xab, 0x81, 0xf1, 0xf7, 0xaf, 0x06, 0xc6, 0x4f, 0xbf, 0x1e, 0xd4, 0xbe, + 0xfc, 0x7a, 0x50, 0xfb, 0xeb, 0xd7, 0x83, 0xda, 0x69, 0x93, 0xfe, 0x25, 0xf2, 0xbf, 0xff, 0x0a, + 0x00, 0x00, 0xff, 0xff, 0x1d, 0x07, 0x2d, 0x0f, 0x37, 0x22, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -4692,6 +4711,20 @@ func (m *SyncStatus) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.DumpIOTotalBytes != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.DumpIOTotalBytes)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x98 + } + if m.IoTotalBytes != 0 { + i = encodeVarintDmworker(dAtA, i, uint64(m.IoTotalBytes)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x90 + } if m.RecentRps != 0 { i = encodeVarintDmworker(dAtA, i, uint64(m.RecentRps)) i-- @@ -7063,6 +7096,12 @@ func (m *SyncStatus) Size() (n int) { if m.RecentRps != 0 { n += 2 + sovDmworker(uint64(m.RecentRps)) } + if m.IoTotalBytes != 0 { + n += 2 + sovDmworker(uint64(m.IoTotalBytes)) + } + if m.DumpIOTotalBytes != 0 { + n += 2 + sovDmworker(uint64(m.DumpIOTotalBytes)) + } return n } @@ -9639,6 +9678,44 @@ func (m *SyncStatus) Unmarshal(dAtA []byte) error { break } } + case 18: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IoTotalBytes", wireType) + } + m.IoTotalBytes = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.IoTotalBytes |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 19: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DumpIOTotalBytes", wireType) + } + m.DumpIOTotalBytes = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowDmworker + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DumpIOTotalBytes |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipDmworker(dAtA[iNdEx:]) diff --git a/dm/pkg/dumpling/utils.go b/dm/pkg/dumpling/utils.go index 0b6cf273917..01efb4b4f8d 100644 --- a/dm/pkg/dumpling/utils.go +++ b/dm/pkg/dumpling/utils.go @@ -123,7 +123,8 @@ func ParseMetaDataByReader(filename string, rd io.Reader) (*binlog.Location, *bi } switch line { - case "SHOW MASTER STATUS:": + case "SHOW BINARY LOG STATUS:", + "SHOW MASTER STATUS:": if err3 := parsePosAndGTID(&pos, >idStr); err3 != nil { return nil, nil, err3 } diff --git a/dm/pkg/dumpling/utils_test.go b/dm/pkg/dumpling/utils_test.go index 55bf0610a1c..7aa373e6b5a 100644 --- a/dm/pkg/dumpling/utils_test.go +++ b/dm/pkg/dumpling/utils_test.go @@ -84,6 +84,29 @@ Finished dump at: 2018-12-27 19:51:22`, mysql.Position{}, "", }, + { + `Started dump at: 2018-12-27 19:51:22 +SHOW BINARY LOG STATUS: + Log: mysql-bin.000004 + Pos: 3295818 + GTID: + +SHOW SLAVE STATUS: + Host: 10.128.27.98 + Log: mysql-bin.000003 + Pos: 329635 + GTID: + +Finished dump at: 2018-12-27 19:51:22`, + mysql.Position{ + Name: "mysql-bin.000004", + Pos: 3295818, + }, + "", + false, + mysql.Position{}, + "", + }, { // with empty line after multiple GTID sets `Started dump at: 2020-05-21 18:14:49 SHOW MASTER STATUS: diff --git a/dm/pkg/parser/common_test.go b/dm/pkg/parser/common_test.go index 4f0a3b66925..f81e7506593 100644 --- a/dm/pkg/parser/common_test.go +++ b/dm/pkg/parser/common_test.go @@ -278,11 +278,11 @@ var testCases = []testCase{ []string{"ALTER TABLE `xtest`.`xt1` DROP INDEX IF EXISTS `i1`"}, }, { - "alter table `t1` drop foreign key if exists fk_t2_id", - []string{"ALTER TABLE `test`.`t1` DROP FOREIGN KEY IF EXISTS `fk_t2_id`"}, + "alter table `t1` drop foreign key fk_t2_id", + []string{"ALTER TABLE `test`.`t1` DROP FOREIGN KEY `fk_t2_id`"}, [][]*filter.Table{{genTableName("test", "t1")}}, [][]*filter.Table{{genTableName("xtest", "xt1")}}, - []string{"ALTER TABLE `xtest`.`xt1` DROP FOREIGN KEY IF EXISTS `fk_t2_id`"}, + []string{"ALTER TABLE `xtest`.`xt1` DROP FOREIGN KEY `fk_t2_id`"}, }, { "alter table `t1` drop partition if exists p2", diff --git a/dm/pkg/storage/utils.go b/dm/pkg/storage/utils.go index 065c2f0c6ee..3a77d93a72f 100644 --- a/dm/pkg/storage/utils.go +++ b/dm/pkg/storage/utils.go @@ -21,9 +21,11 @@ import ( "strings" gstorage "cloud.google.com/go/storage" + "github.com/Azure/azure-sdk-for-go/sdk/storage/azblob/bloberror" "github.com/aws/aws-sdk-go/aws/awserr" "github.com/aws/aws-sdk-go/service/s3" "github.com/pingcap/errors" + backuppb "github.com/pingcap/kvproto/pkg/brpb" bstorage "github.com/pingcap/tidb/br/pkg/storage" ) @@ -133,25 +135,36 @@ func CollectDirFiles(ctx context.Context, dir string, storage bstorage.ExternalS // RemoveAll remove files in dir. func RemoveAll(ctx context.Context, dir string, storage bstorage.ExternalStorage) error { var err error + var backend *backuppb.StorageBackend if storage == nil { storage, err = CreateStorage(ctx, dir) if err != nil { return err } + backend, err = bstorage.ParseBackend(dir, nil) + if err != nil { + return err + } } err = storage.WalkDir(ctx, &bstorage.WalkOption{}, func(filePath string, size int64) error { err2 := storage.DeleteFile(ctx, filePath) - if errors.Cause(err2) == gstorage.ErrObjectNotExist { - // ignore not exist error when we delete files + // ignore not exist error when we delete files and backend is gcs/azure blob storage + if backend != nil && backend.GetGcs() != nil && errors.Cause(err2) == gstorage.ErrObjectNotExist { + return nil + } + if backend != nil && backend.GetAzureBlobStorage() != nil && bloberror.HasCode(err2, bloberror.BlobNotFound) { return nil } return err2 }) if err == nil { err = storage.DeleteFile(ctx, "") - if errors.Cause(err) == gstorage.ErrObjectNotExist { - // ignore not exist error when we delete files + // ignore not exist error when we delete files and backend is gcs/azure blob storage + if backend != nil && backend.GetGcs() != nil && errors.Cause(err) == gstorage.ErrObjectNotExist { + return nil + } + if backend != nil && backend.GetAzureBlobStorage() != nil && bloberror.HasCode(err, bloberror.BlobNotFound) { return nil } } diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index dfa040e989f..602a4b58313 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -935,7 +935,7 @@ var ( ErrConfigSyncerCfgConflict = New(codeConfigSyncerCfgConflict, ClassConfig, ScopeInternal, LevelMedium, "syncer-config-name and syncer should only specify one", "Please check the `syncer-config-name` and `syncer` config in task configuration file.") ErrConfigReadCfgFromFile = New(codeConfigReadCfgFromFile, ClassConfig, ScopeInternal, LevelMedium, "read config file %v", "") ErrConfigNeedUniqueTaskName = New(codeConfigNeedUniqueTaskName, ClassConfig, ScopeInternal, LevelMedium, "must specify a unique task name", "Please check the `name` config in task configuration file.") - ErrConfigInvalidTaskMode = New(codeConfigInvalidTaskMode, ClassConfig, ScopeInternal, LevelMedium, "please specify right task-mode, support `full`, `incremental`, `all`", "Please check the `task-mode` config in task configuration file.") + ErrConfigInvalidTaskMode = New(codeConfigInvalidTaskMode, ClassConfig, ScopeInternal, LevelMedium, "please specify right task-mode, support `full`, `incremental`, `all`, `dump`, `load`", "Please check the `task-mode` config in task configuration file.") ErrConfigNeedTargetDB = New(codeConfigNeedTargetDB, ClassConfig, ScopeInternal, LevelMedium, "must specify target-database", "Please check the `target-database` config in task configuration file.") ErrConfigMetadataNotSet = New(codeConfigMetadataNotSet, ClassConfig, ScopeInternal, LevelMedium, "mysql-instance(%s) must set meta for task-mode %s", "Please check the `meta` config in task configuration file.") ErrConfigRouteRuleNotFound = New(codeConfigRouteRuleNotFound, ClassConfig, ScopeInternal, LevelMedium, "mysql-instance(%d)'s route-rules %s not exist in routes", "Please check the `route-rules` config in task configuration file.") diff --git a/dm/proto/dmworker.proto b/dm/proto/dmworker.proto index bc067104cf3..4d92b336f96 100644 --- a/dm/proto/dmworker.proto +++ b/dm/proto/dmworker.proto @@ -166,6 +166,10 @@ message SyncStatus { int64 totalRows = 15; int64 totalRps = 16; int64 recentRps = 17; + // meter TCP io to downstream of the subtask + uint64 ioTotalBytes = 18; + // meter TCP io from upstream of the subtask + uint64 dumpIOTotalBytes = 19; } // SourceStatus represents status for source runing on dm-worker diff --git a/dm/syncer/dbconn/db.go b/dm/syncer/dbconn/db.go index 64090843d78..ef8f5276d76 100644 --- a/dm/syncer/dbconn/db.go +++ b/dm/syncer/dbconn/db.go @@ -246,7 +246,7 @@ func (conn *DBConn) retryableFn(tctx *tcontext.Context, queries, args any) func( if err != nil { tctx.L().Error("reset connection failed", zap.Int("retry", retryTime), zap.String("queries", utils.TruncateInterface(queries, -1)), - zap.String("arguments", utils.TruncateInterface(args, -1)), + log.ZapRedactString("arguments", utils.TruncateInterface(args, -1)), log.ShortError(err)) return false } @@ -257,7 +257,7 @@ func (conn *DBConn) retryableFn(tctx *tcontext.Context, queries, args any) func( if dbutil.IsRetryableError(err) { tctx.L().Warn("execute statements", zap.Int("retry", retryTime), zap.String("queries", utils.TruncateInterface(queries, -1)), - zap.String("arguments", utils.TruncateInterface(args, -1)), + log.ZapRedactString("arguments", utils.TruncateInterface(args, -1)), log.ShortError(err)) return true } diff --git a/dm/syncer/status.go b/dm/syncer/status.go index 4544646f145..e2687ab7e42 100644 --- a/dm/syncer/status.go +++ b/dm/syncer/status.go @@ -39,6 +39,12 @@ func (s *Syncer) Status(sourceStatus *binlog.SourceStatus) interface{} { SyncerBinlog: syncerLocation.Position.String(), SecondsBehindMaster: s.secondsBehindMaster.Load(), } + if s.cfg.IOTotalBytes != nil { + st.IoTotalBytes = s.cfg.IOTotalBytes.Load() + } + if s.cfg.DumpIOTotalBytes != nil { + st.DumpIOTotalBytes = s.cfg.DumpIOTotalBytes.Load() + } if syncerLocation.GetGTID() != nil { st.SyncerBinlogGtid = syncerLocation.GetGTID().String() diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 75491e0084c..ed08dbc1632 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -3376,6 +3376,12 @@ func (s *Syncer) CheckCanUpdateCfg(newCfg *config.SubTaskConfig) error { oldCfg.EnableGTID = newCfg.EnableGTID oldCfg.CaseSensitive = newCfg.CaseSensitive + // Skip comparing atomic values since they cannot be marshaled/unmarshaled + // by the TOML encoder/decoder. These fields will remain uninitialized (zero value) + // after decoding. + newCfg.IOTotalBytes = oldCfg.IOTotalBytes + newCfg.DumpIOTotalBytes = oldCfg.DumpIOTotalBytes + if oldCfg.String() != newCfg.String() { s.tctx.L().Warn("can not update cfg", zap.Stringer("old cfg", oldCfg), zap.Stringer("new cfg", newCfg)) return terror.ErrWorkerUpdateSubTaskConfig.Generatef("can't update subtask config for syncer because new config contains some fields that should not be changed, task: %s", s.cfg.Name) diff --git a/dm/tests/_utils/run_downstream_cluster b/dm/tests/_utils/run_downstream_cluster index 36038c74e07..0d755cbd8d2 100755 --- a/dm/tests/_utils/run_downstream_cluster +++ b/dm/tests/_utils/run_downstream_cluster @@ -1,7 +1,7 @@ -#!/bin/bash +#!/usr/bin/env bash # tools to run a TiDB cluster # parameter 1: work directory -set -eu +set -eux WORK_DIR=$1 export PD_PEER_ADDR="127.0.0.1:2380" @@ -24,7 +24,7 @@ start_pd() { max-replicas = 1 EOF - pd-server --version + bin/pd-server --version mkdir -p "$WORK_DIR/pd" bin/pd-server \ --client-urls "http://$PD_ADDR" \ @@ -32,14 +32,21 @@ EOF --log-file "$WORK_DIR/pd.log" \ --config "$WORK_DIR/pd.toml" \ --data-dir "$WORK_DIR/pd" & - # wait until PD is online... + sleep 5 i=0 - while ! curl "http://$PD_ADDR/pd/api/v1/version"; do + while true; do + response=$(curl -s -o /dev/null -w "%{http_code}" "http://$PD_ADDR/pd/api/v1/version" || echo "") + echo "curl response: $response" + if [ "$response" -eq 200 ]; then + echo 'Start PD success' + break + fi i=$((i + 1)) if [ "$i" -gt 20 ]; then echo 'Failed to start PD' return 1 fi + echo 'Waiting for PD ready...' sleep 3 done } @@ -47,6 +54,7 @@ EOF start_tikv() { echo "Starting TiKV..." mkdir -p "$WORK_DIR/tikv" + bin/tikv-server --version bin/tikv-server \ --pd "$PD_ADDR" \ -A "$TIKV_ADDR" \ @@ -54,18 +62,35 @@ start_tikv() { --log-file "$WORK_DIR/tikv.log" \ --log-level info \ -s "$WORK_DIR/tikv" & - while ! curl "http://$PD_ADDR/pd/api/v1/cluster/status" | tee /dev/stderr | grep '"is_initialized": true'; do + sleep 5 + i=0 + while true; do + response=$(curl -s "http://$PD_ADDR/pd/api/v1/cluster/status" || echo "") + + if [ -z "$response" ]; then + echo "Failed to connect to PD server" + else + echo "PD response: $response" + if echo "$response" | grep -q '"is_initialized": true'; then + echo "TiKV cluster initialized successfully" + break + fi + fi + i=$((i + 1)) if [ "$i" -gt 20 ]; then - echo 'Failed to initialize TiKV cluster' + echo 'Failed to initialize TiKV cluster after 20 attempts' return 1 fi + + echo 'Waiting for TiKV ready...' sleep 5 done } start_tidb() { echo "Starting TiDB..." + bin/tidb-server -V bin/tidb-server \ -P 4000 \ --status 10080 \ @@ -73,14 +98,22 @@ start_tidb() { --store tikv \ --path "$PD_ADDR" \ --log-file "$WORK_DIR/tidb.log" & - + sleep 5 + # wait until TiDB is online... i=0 - while ! curl "http://$TIDB_IP:10080/status"; do + while true; do + response=$(curl -s -o /dev/null -w "%{http_code}" "http://$TIDB_IP:10080/status" || echo "") + echo "curl response: $response" + if [ "$response" -eq 200 ]; then + echo 'Start TiDB success' + break + fi i=$((i + 1)) if [ "$i" -gt 50 ]; then echo 'Failed to start TiDB' return 1 fi + echo 'Waiting for TiDB ready...' sleep 3 done } diff --git a/dm/tests/_utils/run_sql b/dm/tests/_utils/run_sql index 555fc4a2337..66d91a40a28 100755 --- a/dm/tests/_utils/run_sql +++ b/dm/tests/_utils/run_sql @@ -18,5 +18,5 @@ fi if [[ $# -ge 4 ]]; then echo "$1" | iconv -f utf8 -t $4 | mysql -u$user -h127.0.0.1 -P$2 -p$3 --default-character-set $4 -E >>$OUTFILE else - mysql -u$user -h127.0.0.1 -P$2 -p$3 --default-character-set utf8 -E -e "$1" >>$OUTFILE + MYSQL_PWD=$3 mysql -u$user -h127.0.0.1 -P$2 --default-character-set utf8 -E -e "$1" >>$OUTFILE fi diff --git a/dm/tests/many_tables/run.sh b/dm/tests/many_tables/run.sh index 3fc99f2c4bb..0ffb9a6a680 100644 --- a/dm/tests/many_tables/run.sh +++ b/dm/tests/many_tables/run.sh @@ -151,7 +151,9 @@ function run() { pkill -hup tidb-server 2>/dev/null || true wait_process_exit tidb-server # now worker will process some binlog events, save table checkpoint and meet downstream error + echo "start incremental_data_2" incremental_data_2 + echo "finish incremental_data_2" sleep 30 resume_num=$(grep 'unit process error' $WORK_DIR/worker1/log/dm-worker.log | wc -l) diff --git a/dm/tests/openapi/client/openapi_task_check b/dm/tests/openapi/client/openapi_task_check index 3fbf9a67249..bdd9574c852 100755 --- a/dm/tests/openapi/client/openapi_task_check +++ b/dm/tests/openapi/client/openapi_task_check @@ -4,6 +4,7 @@ import sys import requests SHARD_TASK_NAME = "test-shard" +LOAD_TASK_NAME = "test-load" ILLEGAL_CHAR_TASK_NAME = "t-Ë!s`t" SOURCE1_NAME = "mysql-01" SOURCE2_NAME = "mysql-02" @@ -105,10 +106,10 @@ def create_task_with_precheck(task_name, ignore_check, is_success, check_result) assert resp.status_code == 400 assert check_result in resp.json()["error_msg"] -def create_noshard_task_success(task_name, tartget_table_name=""): +def create_noshard_task_success(task_name, tartget_table_name="", task_mode="all"): task = { "name": task_name, - "task_mode": "all", + "task_mode": task_mode, "meta_schema": "dm-meta", "enhance_online_schema_change": True, "on_duplicate": "error", @@ -308,6 +309,45 @@ def create_dump_task_success(): print("create_dump_task_success resp=", resp.json()) assert resp.status_code == 201 +def create_load_task_success(): + task = { + "name": LOAD_TASK_NAME, + "task_mode": "load", + "meta_schema": "dm-meta", + "enhance_online_schema_change": True, + "on_duplicate": "error", + "target_config": { + "host": "127.0.0.1", + "port": 4000, + "user": "root", + "password": "", + }, + "table_migrate_rule": [ + { + "source": { + "source_name": SOURCE1_NAME, + "schema": "openapi", + "table": "*", + }, + "target": {"schema": "openapi", "table": ""}, + } + ], + "source_config": { + "full_migrate_conf": { + "export_threads": 4, + "import_threads": 16, + "data_dir": "./exported_data", + "consistency": "auto", + }, + "source_conf": [ + {"source_name": SOURCE1_NAME} + ], + }, + } + resp = requests.post(url=API_ENDPOINT, json={"task": task}) + print("create_load_task_success resp=", resp.json()) + assert resp.status_code == 201 + def start_task_success(task_name, source_name): url = API_ENDPOINT + "/" + task_name + "/start" req = {} @@ -413,6 +453,73 @@ def get_task_status_success(task_name, total): print("get_task_status_success resp=", data) assert data["total"] == int(total) +def check_sync_task_status_success( + task_name, + min_dump_io_total_bytes=2000, + max_dump_io_total_bytes=51200, + min_io_total_bytes=2000, + max_io_total_bytes=51200 +): + url = API_ENDPOINT + "/" + task_name + "/status" + resp = requests.get(url=url) + data = resp.json() + assert resp.status_code == 200 + print("check_sync_task_status_success resp=", data) + + for item in data.get("data", []): + # Only validate sync task status + assert item.get("unit") == "Sync" + sync_status = item.get("sync_status", {}) + dump_io_total_bytes = sync_status.get("dump_io_total_bytes") + io_total_bytes = sync_status.get("io_total_bytes") + assert dump_io_total_bytes is not None and io_total_bytes is not None + assert dump_io_total_bytes > int(min_dump_io_total_bytes) and io_total_bytes > int(min_io_total_bytes) + assert dump_io_total_bytes < int(max_dump_io_total_bytes) and io_total_bytes < int(max_io_total_bytes) + +def check_load_task_finished_status_success(task_name, finished_bytes, total_bytes): + url = API_ENDPOINT + "/" + task_name + "/status" + resp = requests.get(url=url) + data = resp.json() + assert resp.status_code == 200 + print("check_load_task_finished_status_success resp=", data) + + for item in data.get("data", []): + # Only validate load task status + assert item.get("unit") == "Load" + assert item.get("stage") == "Finished" + load_status = item.get("load_status", {}) + assert load_status.get("progress") == "100.00 %" + assert load_status.get("finished_bytes") == int(finished_bytes) + assert load_status.get("total_bytes") == int(total_bytes) + +def check_dump_task_finished_status_success(task_name, completed_tables, total_tables, estimate_total_rows,finished_rows,finished_bytes): + url = API_ENDPOINT + "/" + task_name + "/status" + resp = requests.get(url=url) + data = resp.json() + assert resp.status_code == 200 + print("check_dump_task_finished_status_success resp=", data) + + for item in data.get("data", []): + # Only validate dump task status + assert item.get("unit") == "Dump" + assert item.get("stage") == "Finished" + dump_status = item.get("dump_status", {}) + assert dump_status.get("progress") == "100.00 %" + assert dump_status.get("completed_tables") == int(completed_tables) + assert dump_status.get("total_tables") == int(total_tables) + assert dump_status.get("estimate_total_rows") == int(estimate_total_rows) + assert dump_status.get("finished_rows") == int(finished_rows) + assert dump_status.get("finished_bytes") == int(finished_bytes) + +def check_task_stage_success(task_name, total, stage): + url = API_ENDPOINT + "/" + task_name + "/status" + resp = requests.get(url=url) + data = resp.json() + assert resp.status_code == 200 + print("check_task_stage_success resp=", data) + assert data["total"] == int(total) + for status in data["data"]: + assert status["stage"] == stage def get_task_status_success_but_worker_meet_error(task_name, total): url = API_ENDPOINT + "/" + task_name + "/status" @@ -633,7 +740,7 @@ def check_noshard_task_dump_status_success(task_name, total): resp = requests.get(url=url) data = resp.json() assert resp.status_code == 200 - print("check_dump_status_success resp=", data) + print("check_noshard_task_dump_status_success resp=", data) assert data["data"][0]["dump_status"]["finished_bytes"] == int(total) @@ -743,6 +850,7 @@ if __name__ == "__main__": "create_noshard_task_success": create_noshard_task_success, "create_shard_task_success": create_shard_task_success, "create_dump_task_success": create_dump_task_success, + "create_load_task_success": create_load_task_success, "create_incremental_task_with_gtid_success": create_incremental_task_with_gtid_success, "delete_task_failed": delete_task_failed, "delete_task_success": delete_task_success, @@ -769,6 +877,10 @@ if __name__ == "__main__": "check_noshard_task_dump_status_success": check_noshard_task_dump_status_success, "do_complex_operations": do_complex_operations, "create_task_with_precheck": create_task_with_precheck, + "check_task_stage_success": check_task_stage_success, + "check_sync_task_status_success": check_sync_task_status_success, + "check_load_task_finished_status_success": check_load_task_finished_status_success, + "check_dump_task_finished_status_success": check_dump_task_finished_status_success, } func = FUNC_MAP[sys.argv[1]] diff --git a/dm/tests/openapi/conf/diff_config_no_shard_one_source.toml b/dm/tests/openapi/conf/diff_config_no_shard_one_source.toml new file mode 100644 index 00000000000..fa7a8305957 --- /dev/null +++ b/dm/tests/openapi/conf/diff_config_no_shard_one_source.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/ticdc_dm_test/output" + + source-instances = ["mysql1"] + + target-instance = "tidb0" + + target-check-tables = ["openapi.t1", "openapi.t2"] + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +password = "123456" +port = 3306 +user = "root" + +[data-sources.tidb0] +host = "127.0.0.1" +password = "" +port = 4000 +user = "root" diff --git a/dm/tests/openapi/run.sh b/dm/tests/openapi/run.sh index aceae7c16b2..44ccee9cc39 100644 --- a/dm/tests/openapi/run.sh +++ b/dm/tests/openapi/run.sh @@ -28,6 +28,9 @@ function init_dump_data() { run_sql_source1 "CREATE TABLE openapi.t1(i TINYINT, j INT UNIQUE KEY);" run_sql_source1 "INSERT INTO openapi.t1(i,j) VALUES (1, 2),(3,4);" + + run_sql_source1 "CREATE TABLE openapi.t2(i TINYINT, j INT UNIQUE KEY);" + run_sql_source1 "INSERT INTO openapi.t2(i,j) VALUES (1, 2),(3,4);" } function init_shard_data() { @@ -182,11 +185,12 @@ function test_relay() { } -function test_dump_task() { - echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>START TEST OPENAPI: dump TASK" +function test_dump_and_load_task() { + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>START TEST OPENAPI: dump & load TASK" prepare_database - task_name="test-dump" + task_name_dump="test-dump" + task_name_load="test-load" # create source successfully openapi_source_check "create_source1_success" @@ -207,25 +211,93 @@ function test_dump_task() { # create dump task success openapi_task_check "create_dump_task_success" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status $task_name" \ + "query-status $task_name_dump" \ "\"stage\": \"Stopped\"" 1 init_dump_data # start dump task success - openapi_task_check "start_task_success" $task_name "" - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status $task_name" \ - "\"stage\": \"Running\"" 1 + openapi_task_check "start_task_success" $task_name_dump "" # wait dump task finish run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status $task_name" 100 \ + "query-status $task_name_dump" 100 \ + "\"stage\": \"Finished\"" 1 + openapi_task_check "check_dump_task_finished_status_success" $task_name_dump 2 2 4 4 228 + + # create load task success + openapi_task_check "create_load_task_success" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status $task_name_load" \ + "\"stage\": \"Stopped\"" 1 + + # use the data from the same dir of dump task + + # start load task success + openapi_task_check "start_task_success" $task_name_load "" + + # wait load task finish + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status $task_name_load" 100 \ "\"stage\": \"Finished\"" 1 + check_sync_diff $WORK_DIR $cur/conf/diff_config_no_shard_one_source.toml + clean_cluster_sources_and_tasks - echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>TEST OPENAPI: dump TASK" + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>TEST OPENAPI: dump & load TASK" + +} + +function test_full_mode_task() { + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>START TEST OPENAPI: FULL MODE TASK" + prepare_database + + task_name="test-no-shard" + target_table_name="" + # create source successfully + openapi_source_check "create_source1_success" + openapi_source_check "list_source_success" 1 + + # get source status success + openapi_source_check "get_source_status_success" "mysql-01" + + # create source successfully + openapi_source_check "create_source2_success" + # get source list success + openapi_source_check "list_source_success" 2 + + # get source status success + openapi_source_check "get_source_status_success" "mysql-02" + + # create no shard task in full mode success + openapi_task_check "create_noshard_task_success" $task_name "" "full" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status $task_name" \ + "\"stage\": \"Stopped\"" 2 + openapi_task_check "check_task_stage_success" $task_name 2 "Stopped" + + init_noshard_data + + # start task success + openapi_task_check "start_task_success" $task_name "" + + # get task status and load task status + openapi_task_check "get_task_status_success" "$task_name" 2 + + # wait full task finish + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status $task_name" 100 \ + "\"stage\": \"Finished\"" 2 + openapi_task_check "check_task_stage_success" $task_name 2 "Finished" + check_sync_diff $WORK_DIR $cur/conf/diff_config_no_shard.toml + + # check load task status + openapi_task_check "check_load_task_finished_status_success" "$task_name" 107 107 + + # delete source success and clean data for other test + clean_cluster_sources_and_tasks + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>TEST OPENAPI: FULL MODE TASK" } function test_shard_task() { @@ -281,8 +353,9 @@ function test_shard_task() { # get illegal char task_status failed openapi_task_check get_illegal_char_task_status_failed - # get task status success + # get task status and check sync task status success openapi_task_check "get_task_status_success" "$task_name" 2 + openapi_task_check "check_sync_task_status_success" "$task_name" 3000 7000 19000 30000 # get task list openapi_task_check "get_task_list" 1 @@ -342,8 +415,9 @@ function test_noshard_task() { # get task status failed openapi_task_check "get_task_status_failed" "not a task name" - # get task status success + # get task status and check sync task status success openapi_task_check "get_task_status_success" "$task_name" 2 + openapi_task_check "check_sync_task_status_success" "$task_name" 2500 5000 18000 42000 # delete source with force openapi_source_check "delete_source_with_force_success" "mysql-01" @@ -405,6 +479,7 @@ function test_complex_operations_of_source_and_task() { init_noshard_data check_sync_diff $WORK_DIR $cur/conf/diff_config_no_shard.toml openapi_task_check "get_task_status_success" "$task_name" 2 + openapi_task_check "check_sync_task_status_success" "$task_name" 2500 5000 18000 42000 # do some complex operations openapi_task_check "do_complex_operations" "$task_name" @@ -1092,7 +1167,7 @@ function run() { test_shard_task test_multi_tasks test_noshard_task - test_dump_task + test_dump_and_load_task test_task_templates test_noshard_task_dump_status test_complex_operations_of_source_and_task @@ -1101,6 +1176,7 @@ function run() { test_start_task_with_condition test_stop_task_with_condition test_reverse_https + test_full_mode_task # NOTE: this test case MUST running at last, because it will offline some members of cluster test_cluster diff --git a/dm/tests/requirements.txt b/dm/tests/requirements.txt index f2293605cf1..8ca17f61940 100644 --- a/dm/tests/requirements.txt +++ b/dm/tests/requirements.txt @@ -1 +1,2 @@ requests +ipaddress diff --git a/dm/tests/tls/conf/dm-task-3.yaml b/dm/tests/tls/conf/dm-task-3.yaml new file mode 100644 index 00000000000..e172a098040 --- /dev/null +++ b/dm/tests/tls/conf/dm-task-3.yaml @@ -0,0 +1,49 @@ +--- +name: test3 +task-mode: all +shard-mode: "pessimistic" +meta-schema: "dm_meta" + +target-database: + host: "127.0.0.1" + port: 4400 + user: "root" + password: "" + security: + ssl-ca: "dir-placeholer/task-ca.pem" + ssl-cert: "dir-placeholer/dm.pem" + ssl-key: "dir-placeholer/dm.key" + +mysql-instances: + - source-id: "mysql-replica-01" + black-white-list: "instance" + route-rules: ["route-rule-1"] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +black-white-list: + instance: + do-dbs: ["tls"] + +routes: + route-rule-1: + schema-pattern: "tls" + target-schema: "tls3" + +mydumpers: + global: + threads: 4 + chunk-filesize: 0 + skip-tz-utc: true + extra-args: "--statement-size=100" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/tls/run.sh b/dm/tests/tls/run.sh index 3fc76387f38..199bc478c5c 100644 --- a/dm/tests/tls/run.sh +++ b/dm/tests/tls/run.sh @@ -42,8 +42,13 @@ EOF --log-file "$WORK_DIR/tidb.log" 2>&1 & sleep 5 + + echo "show databases without TLS" + mysql -uroot -h127.0.0.1 -P4400 --default-character-set utf8 -E -e "SHOW DATABASES;" + echo "drop database tls with TLS" # if execute failed, print tidb's log for debug mysql -uroot -h127.0.0.1 -P4400 --default-character-set utf8 --ssl-ca $cur/conf/ca.pem --ssl-cert $cur/conf/dm.pem --ssl-key $cur/conf/dm.key -E -e "drop database if exists tls" || (cat $WORK_DIR/tidb.log && exit 1) + echo "drop database dm_meta with TLS" mysql -uroot -h127.0.0.1 -P4400 --default-character-set utf8 --ssl-ca $cur/conf/ca.pem --ssl-cert $cur/conf/dm.pem --ssl-key $cur/conf/dm.key -E -e "drop database if exists dm_meta" } @@ -103,6 +108,7 @@ function test_worker_handle_multi_tls_tasks() { cp $cur/conf/dm-worker2.toml $WORK_DIR/ cp $cur/conf/dm-task.yaml $WORK_DIR/ cp $cur/conf/dm-task-2.yaml $WORK_DIR/ + cp $cur/conf/dm-task-3.yaml $WORK_DIR/ sed -i "s%dir-placeholer%$cur\/conf%g" $WORK_DIR/dm-master1.toml sed -i "s%dir-placeholer%$cur\/conf%g" $WORK_DIR/dm-master2.toml @@ -111,6 +117,7 @@ function test_worker_handle_multi_tls_tasks() { sed -i "s%dir-placeholer%$cur\/conf%g" $WORK_DIR/dm-worker2.toml sed -i "s%dir-placeholer%$cur\/conf%g" $WORK_DIR/dm-task.yaml sed -i "s%dir-placeholer%$cur\/conf%g" $WORK_DIR/dm-task-2.yaml + sed -i "s%dir-placeholer%$cur\/conf%g" $WORK_DIR/dm-task-3.yaml run_dm_master $WORK_DIR/master1 $MASTER_PORT1 $WORK_DIR/dm-master1.toml run_dm_master $WORK_DIR/master2 $MASTER_PORT2 $WORK_DIR/dm-master2.toml @@ -133,6 +140,8 @@ function test_worker_handle_multi_tls_tasks() { "start-task $WORK_DIR/dm-task.yaml --remove-meta=true" run_dm_ctl_with_tls_and_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" $cur/conf/ca.pem $cur/conf/dm.pem $cur/conf/dm.key \ "start-task $WORK_DIR/dm-task-2.yaml --remove-meta=true" + run_dm_ctl_with_tls_and_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" $cur/conf/ca.pem $cur/conf/dm.pem $cur/conf/dm.key \ + "start-task $WORK_DIR/dm-task-3.yaml --remove-meta=true" run_dm_ctl_with_tls_and_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" $cur/conf/ca.pem $cur/conf/dm.pem $cur/conf/dm.key \ "query-status test" \ @@ -142,6 +151,10 @@ function test_worker_handle_multi_tls_tasks() { "query-status test2" \ "\"result\": true" 2 \ "\"unit\": \"Sync\"" 1 + run_dm_ctl_with_tls_and_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" $cur/conf/ca.pem $cur/conf/dm.pem $cur/conf/dm.key \ + "query-status test3" \ + "\"result\": true" 2 \ + "\"unit\": \"Sync\"" 1 echo "check data" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml diff --git a/dm/tests/upstream_switch/case.sh b/dm/tests/upstream_switch/case.sh index 012b4df8ff3..82f04026b22 100644 --- a/dm/tests/upstream_switch/case.sh +++ b/dm/tests/upstream_switch/case.sh @@ -2,6 +2,7 @@ set -eu +export DM_MASTER_EXTRA_ARG="" CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) PATH=$CUR/../_utils:$PATH # for sync_diff_inspector diff --git a/dm/worker/subtask.go b/dm/worker/subtask.go index 8db5b050a63..e5c2e5980a0 100644 --- a/dm/worker/subtask.go +++ b/dm/worker/subtask.go @@ -69,6 +69,8 @@ func createRealUnits(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, wor us = append(us, syncer.NewSyncer(cfg, etcdClient, relay)) case config.ModeDump: us = append(us, dumpling.NewDumpling(cfg)) + case config.ModeLoad: + us = append(us, loader.NewLightning(cfg, etcdClient, workerName)) case config.ModeLoadSync: us = append(us, loader.NewLightning(cfg, etcdClient, workerName)) us = append(us, syncer.NewSyncer(cfg, etcdClient, relay)) diff --git a/engine/enginepb/datarw.pb.go b/engine/enginepb/datarw.pb.go index bbb6a4c9f6b..d2bd6de6824 100644 --- a/engine/enginepb/datarw.pb.go +++ b/engine/enginepb/datarw.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.0 +// protoc-gen-go v1.34.2 // protoc v3.20.1 // source: engine/proto/datarw.proto @@ -733,7 +733,7 @@ func file_engine_proto_datarw_proto_rawDescGZIP() []byte { } var file_engine_proto_datarw_proto_msgTypes = make([]protoimpl.MessageInfo, 12) -var file_engine_proto_datarw_proto_goTypes = []interface{}{ +var file_engine_proto_datarw_proto_goTypes = []any{ (*GenerateDataRequest)(nil), // 0: enginepb.GenerateDataRequest (*GenerateDataResponse)(nil), // 1: enginepb.GenerateDataResponse (*CheckDirRequest)(nil), // 2: enginepb.CheckDirRequest @@ -773,7 +773,7 @@ func file_engine_proto_datarw_proto_init() { return } if !protoimpl.UnsafeEnabled { - file_engine_proto_datarw_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[0].Exporter = func(v any, i int) any { switch v := v.(*GenerateDataRequest); i { case 0: return &v.state @@ -785,7 +785,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[1].Exporter = func(v any, i int) any { switch v := v.(*GenerateDataResponse); i { case 0: return &v.state @@ -797,7 +797,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[2].Exporter = func(v any, i int) any { switch v := v.(*CheckDirRequest); i { case 0: return &v.state @@ -809,7 +809,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[3].Exporter = func(v any, i int) any { switch v := v.(*CheckDirResponse); i { case 0: return &v.state @@ -821,7 +821,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[4].Exporter = func(v any, i int) any { switch v := v.(*IsReadyRequest); i { case 0: return &v.state @@ -833,7 +833,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[5].Exporter = func(v any, i int) any { switch v := v.(*IsReadyResponse); i { case 0: return &v.state @@ -845,7 +845,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[6].Exporter = func(v any, i int) any { switch v := v.(*ListFilesReq); i { case 0: return &v.state @@ -857,7 +857,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[7].Exporter = func(v any, i int) any { switch v := v.(*ListFilesResponse); i { case 0: return &v.state @@ -869,7 +869,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[8].Exporter = func(v any, i int) any { switch v := v.(*ReadLinesRequest); i { case 0: return &v.state @@ -881,7 +881,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[9].Exporter = func(v any, i int) any { switch v := v.(*ReadLinesResponse); i { case 0: return &v.state @@ -893,7 +893,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[10].Exporter = func(v any, i int) any { switch v := v.(*WriteLinesRequest); i { case 0: return &v.state @@ -905,7 +905,7 @@ func file_engine_proto_datarw_proto_init() { return nil } } - file_engine_proto_datarw_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_datarw_proto_msgTypes[11].Exporter = func(v any, i int) any { switch v := v.(*WriteLinesResponse); i { case 0: return &v.state diff --git a/engine/enginepb/executor.pb.go b/engine/enginepb/executor.pb.go index 1c316b94193..c04627ea745 100644 --- a/engine/enginepb/executor.pb.go +++ b/engine/enginepb/executor.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.0 +// protoc-gen-go v1.34.2 // protoc v3.20.1 // source: engine/proto/executor.proto @@ -422,7 +422,7 @@ func file_engine_proto_executor_proto_rawDescGZIP() []byte { } var file_engine_proto_executor_proto_msgTypes = make([]protoimpl.MessageInfo, 6) -var file_engine_proto_executor_proto_goTypes = []interface{}{ +var file_engine_proto_executor_proto_goTypes = []any{ (*PreDispatchTaskRequest)(nil), // 0: enginepb.PreDispatchTaskRequest (*PreDispatchTaskResponse)(nil), // 1: enginepb.PreDispatchTaskResponse (*ConfirmDispatchTaskRequest)(nil), // 2: enginepb.ConfirmDispatchTaskRequest @@ -453,7 +453,7 @@ func file_engine_proto_executor_proto_init() { } file_engine_proto_projects_proto_init() if !protoimpl.UnsafeEnabled { - file_engine_proto_executor_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_executor_proto_msgTypes[0].Exporter = func(v any, i int) any { switch v := v.(*PreDispatchTaskRequest); i { case 0: return &v.state @@ -465,7 +465,7 @@ func file_engine_proto_executor_proto_init() { return nil } } - file_engine_proto_executor_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_executor_proto_msgTypes[1].Exporter = func(v any, i int) any { switch v := v.(*PreDispatchTaskResponse); i { case 0: return &v.state @@ -477,7 +477,7 @@ func file_engine_proto_executor_proto_init() { return nil } } - file_engine_proto_executor_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_executor_proto_msgTypes[2].Exporter = func(v any, i int) any { switch v := v.(*ConfirmDispatchTaskRequest); i { case 0: return &v.state @@ -489,7 +489,7 @@ func file_engine_proto_executor_proto_init() { return nil } } - file_engine_proto_executor_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_executor_proto_msgTypes[3].Exporter = func(v any, i int) any { switch v := v.(*ConfirmDispatchTaskResponse); i { case 0: return &v.state @@ -501,7 +501,7 @@ func file_engine_proto_executor_proto_init() { return nil } } - file_engine_proto_executor_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_executor_proto_msgTypes[4].Exporter = func(v any, i int) any { switch v := v.(*RemoveLocalResourceRequest); i { case 0: return &v.state @@ -513,7 +513,7 @@ func file_engine_proto_executor_proto_init() { return nil } } - file_engine_proto_executor_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_executor_proto_msgTypes[5].Exporter = func(v any, i int) any { switch v := v.(*RemoveLocalResourceResponse); i { case 0: return &v.state diff --git a/engine/enginepb/master.pb.go b/engine/enginepb/master.pb.go index 5168ded9d3a..c81d3e28678 100644 --- a/engine/enginepb/master.pb.go +++ b/engine/enginepb/master.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.0 +// protoc-gen-go v1.34.2 // protoc v3.20.1 // source: engine/proto/master.proto @@ -1782,248 +1782,248 @@ var file_engine_proto_master_proto_rawDesc = []byte{ 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x74, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x03, 0x74, 0x74, 0x6c, 0x22, 0x13, 0x0a, 0x11, 0x48, 0x65, 0x61, 0x72, 0x74, - 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xc0, 0x01, 0x0a, - 0x08, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x12, 0x13, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x03, 0xe0, 0x41, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, - 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, - 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x36, 0x0a, 0x06, - 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x65, - 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, - 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, - 0x62, 0x65, 0x6c, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, - 0x49, 0x0a, 0x17, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x45, 0x78, 0x65, 0x63, 0x75, - 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2e, 0x0a, 0x08, 0x65, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x65, - 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, - 0x52, 0x08, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x22, 0x16, 0x0a, 0x14, 0x4c, 0x69, - 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x22, 0x49, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x30, 0x0a, 0x09, 0x65, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, - 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, - 0x6f, 0x72, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x22, 0x63, 0x0a, - 0x06, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x61, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, - 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x22, 0x14, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x41, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, - 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x2a, 0x0a, 0x07, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x10, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4d, 0x61, 0x73, 0x74, - 0x65, 0x72, 0x52, 0x07, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x22, 0x95, 0x01, 0x0a, 0x13, - 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x09, - 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x73, 0x12, 0x30, 0x0a, 0x09, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x03, + 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xc1, 0x01, 0x0a, + 0x08, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x12, 0x14, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x42, 0x04, 0xe2, 0x41, 0x01, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, + 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x36, 0x0a, + 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, + 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, + 0x72, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, + 0x61, 0x62, 0x65, 0x6c, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x49, 0x0a, 0x17, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2e, 0x0a, 0x08, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, + 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, + 0x72, 0x52, 0x08, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x22, 0x16, 0x0a, 0x14, 0x4c, + 0x69, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x49, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x30, 0x0a, 0x09, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x6f, 0x72, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x22, 0x63, + 0x0a, 0x06, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, + 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x22, 0x14, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, + 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x41, 0x0a, 0x13, 0x4c, 0x69, 0x73, + 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x2a, 0x0a, 0x07, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x10, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4d, 0x61, 0x73, + 0x74, 0x65, 0x72, 0x52, 0x07, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x22, 0x95, 0x01, 0x0a, + 0x13, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x64, 0x12, 0x33, 0x0a, + 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x73, 0x12, 0x30, 0x0a, 0x09, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, + 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6c, 0x65, 0x63, + 0x74, 0x6f, 0x72, 0x73, 0x22, 0x5c, 0x0a, 0x14, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, + 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x49, 0x64, 0x12, 0x23, 0x0a, + 0x0d, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x41, 0x64, + 0x64, 0x72, 0x22, 0x12, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x3a, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x61, + 0x64, 0x76, 0x65, 0x72, 0x74, 0x69, 0x73, 0x65, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0d, 0x61, 0x64, 0x76, 0x65, 0x72, 0x74, 0x69, 0x73, 0x65, 0x41, 0x64, + 0x64, 0x72, 0x22, 0x15, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x69, 0x67, 0x6e, 0x4c, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xee, 0x03, 0x0a, 0x03, 0x4a, 0x6f, + 0x62, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, + 0x64, 0x12, 0x26, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x12, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x2f, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, + 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x04, 0xe2, + 0x41, 0x01, 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x12, 0x1c, 0x0a, 0x06, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0c, 0x42, 0x04, 0xe2, 0x41, 0x01, 0x03, 0x52, 0x06, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, + 0x12, 0x2f, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x13, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x45, + 0x72, 0x72, 0x6f, 0x72, 0x42, 0x04, 0xe2, 0x41, 0x01, 0x03, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x12, 0x30, 0x0a, 0x09, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, - 0x6f, 0x72, 0x73, 0x22, 0x5c, 0x0a, 0x14, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, - 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x65, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0a, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, - 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x41, 0x64, 0x64, - 0x72, 0x22, 0x12, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x3a, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x64, - 0x76, 0x65, 0x72, 0x74, 0x69, 0x73, 0x65, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0d, 0x61, 0x64, 0x76, 0x65, 0x72, 0x74, 0x69, 0x73, 0x65, 0x41, 0x64, 0x64, - 0x72, 0x22, 0x15, 0x0a, 0x13, 0x52, 0x65, 0x73, 0x69, 0x67, 0x6e, 0x4c, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xeb, 0x03, 0x0a, 0x03, 0x4a, 0x6f, 0x62, - 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, - 0x12, 0x26, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x12, - 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x54, 0x79, - 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x2e, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, - 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, - 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x42, 0x03, 0xe0, 0x41, - 0x03, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x12, 0x1b, 0x0a, 0x06, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, - 0x42, 0x03, 0xe0, 0x41, 0x03, 0x52, 0x06, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x12, 0x2e, 0x0a, - 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x65, - 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x42, 0x03, 0xe0, 0x41, 0x03, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x30, 0x0a, - 0x09, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x12, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x53, 0x65, 0x6c, 0x65, - 0x63, 0x74, 0x6f, 0x72, 0x52, 0x09, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x1a, - 0x35, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x42, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, - 0x0a, 0x0b, 0x54, 0x79, 0x70, 0x65, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, - 0x0b, 0x0a, 0x07, 0x43, 0x56, 0x53, 0x44, 0x65, 0x6d, 0x6f, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, - 0x44, 0x4d, 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x43, 0x44, 0x43, 0x10, 0x03, 0x12, 0x0b, 0x0a, - 0x07, 0x46, 0x61, 0x6b, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x04, 0x22, 0x6a, 0x0a, 0x05, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x55, 0x6e, 0x6b, 0x6e, - 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, - 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, - 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x46, - 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x61, 0x6e, - 0x63, 0x65, 0x6c, 0x69, 0x6e, 0x67, 0x10, 0x05, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x61, 0x6e, 0x63, - 0x65, 0x6c, 0x65, 0x64, 0x10, 0x06, 0x22, 0x6f, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x03, 0x6a, 0x6f, - 0x62, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, - 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x52, 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x1b, 0x0a, 0x09, 0x74, - 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x6a, - 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, - 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x22, 0x82, 0x01, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x4a, + 0x6f, 0x72, 0x73, 0x1a, 0x35, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04, + 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, + 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x42, 0x0a, 0x04, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x79, 0x70, 0x65, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, + 0x6e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x56, 0x53, 0x44, 0x65, 0x6d, 0x6f, 0x10, 0x01, + 0x12, 0x06, 0x0a, 0x02, 0x44, 0x4d, 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x43, 0x44, 0x43, 0x10, + 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x61, 0x6b, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x04, 0x22, 0x6a, + 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x64, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x75, 0x6e, 0x6e, 0x69, 0x6e, + 0x67, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x03, 0x12, + 0x0c, 0x0a, 0x08, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, + 0x09, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x69, 0x6e, 0x67, 0x10, 0x05, 0x12, 0x0c, 0x0a, 0x08, + 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x65, 0x64, 0x10, 0x06, 0x22, 0x6f, 0x0a, 0x10, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, + 0x0a, 0x03, 0x6a, 0x6f, 0x62, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x65, 0x6e, + 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x52, 0x03, 0x6a, 0x6f, 0x62, 0x12, + 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, + 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x22, 0x82, 0x01, 0x0a, 0x0d, + 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, + 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1b, 0x0a, + 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, + 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, + 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x22, 0x83, 0x02, 0x0a, 0x0f, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, + 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, + 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, + 0x0a, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, + 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x12, 0x26, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x12, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, + 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x65, 0x6e, 0x67, + 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x5d, 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, + 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x04, 0x6a, 0x6f, + 0x62, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, + 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x26, 0x0a, + 0x0f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x5e, 0x0a, 0x10, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, - 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, - 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, - 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x83, 0x02, 0x0a, - 0x0f, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x08, 0x70, 0x61, 0x67, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x1d, 0x0a, - 0x0a, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x70, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x1b, 0x0a, 0x09, - 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, - 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, - 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x63, 0x6c, - 0x75, 0x64, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x26, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x12, 0x2e, - 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x54, 0x79, 0x70, - 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, - 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, - 0x74, 0x65, 0x22, 0x5d, 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, - 0x4a, 0x6f, 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x6e, 0x65, 0x78, - 0x74, 0x5f, 0x70, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0d, 0x6e, 0x65, 0x78, 0x74, 0x50, 0x61, 0x67, 0x65, 0x54, 0x6f, 0x6b, 0x65, - 0x6e, 0x22, 0x5e, 0x0a, 0x10, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, - 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, - 0x64, 0x22, 0x5e, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, - 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x49, - 0x64, 0x22, 0x3c, 0x0a, 0x15, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, - 0x6f, 0x72, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x02, 0x74, 0x70, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, - 0x62, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x02, 0x74, 0x70, 0x22, - 0x30, 0x0a, 0x16, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, - 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x22, 0x1b, 0x0a, 0x19, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, - 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x34, - 0x0a, 0x1a, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x2a, 0x32, 0x0a, 0x09, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x4d, 0x65, 0x74, 0x61, 0x53, - 0x74, 0x6f, 0x72, 0x65, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x70, 0x70, 0x4d, 0x65, 0x74, - 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x10, 0x01, 0x32, 0x9c, 0x06, 0x0a, 0x09, 0x44, 0x69, 0x73, - 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x12, 0x77, 0x0a, 0x10, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, - 0x65, 0x72, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x12, 0x21, 0x2e, 0x65, 0x6e, 0x67, - 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x45, 0x78, - 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x12, 0x2e, - 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, - 0x72, 0x22, 0x2c, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x26, 0x22, 0x1a, 0x2f, 0x61, 0x70, 0x69, 0x2f, - 0x76, 0x31, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x72, 0x65, 0x67, - 0x69, 0x73, 0x74, 0x65, 0x72, 0x3a, 0x08, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x12, - 0x6b, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, - 0x12, 0x1e, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1f, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x19, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x13, 0x12, 0x11, 0x2f, 0x61, 0x70, 0x69, 0x2f, - 0x76, 0x31, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x63, 0x0a, 0x0b, - 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x1c, 0x2e, 0x65, 0x6e, - 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, - 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x65, 0x6e, 0x67, 0x69, - 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x17, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x11, - 0x12, 0x0f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x61, 0x73, 0x74, 0x65, 0x72, - 0x73, 0x12, 0x46, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x1a, + 0x65, 0x63, 0x74, 0x49, 0x64, 0x22, 0x5e, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, + 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, + 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, + 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x6a, 0x65, 0x63, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x70, 0x72, 0x6f, 0x6a, + 0x65, 0x63, 0x74, 0x49, 0x64, 0x22, 0x3c, 0x0a, 0x15, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x65, + 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, + 0x0a, 0x02, 0x74, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x13, 0x2e, 0x65, 0x6e, 0x67, + 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, + 0x02, 0x74, 0x70, 0x22, 0x30, 0x0a, 0x16, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x65, 0x74, 0x61, + 0x53, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x0a, + 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x1b, 0x0a, 0x19, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, + 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x22, 0x34, 0x0a, 0x1a, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2a, 0x32, 0x0a, 0x09, 0x53, 0x74, 0x6f, 0x72, + 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x13, 0x0a, 0x0f, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x4d, + 0x65, 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x41, 0x70, + 0x70, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x10, 0x01, 0x32, 0x9c, 0x06, 0x0a, + 0x09, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x12, 0x77, 0x0a, 0x10, 0x52, 0x65, + 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x12, 0x21, + 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, + 0x65, 0x72, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x12, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x45, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x6f, 0x72, 0x22, 0x2c, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x26, 0x3a, 0x08, 0x65, + 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x22, 0x1a, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x2f, 0x72, 0x65, 0x67, 0x69, 0x73, + 0x74, 0x65, 0x72, 0x12, 0x6b, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, + 0x74, 0x6f, 0x72, 0x73, 0x12, 0x1e, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x19, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x13, 0x12, 0x11, 0x2f, + 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x6f, 0x72, 0x73, + 0x12, 0x63, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, + 0x1c, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, + 0x61, 0x73, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, + 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4d, 0x61, 0x73, + 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x17, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x11, 0x12, 0x0f, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6d, 0x61, + 0x73, 0x74, 0x65, 0x72, 0x73, 0x12, 0x46, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, + 0x61, 0x74, 0x12, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x48, 0x65, + 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, - 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x65, 0x6e, 0x67, - 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0e, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x12, 0x1f, 0x2e, 0x65, 0x6e, - 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x65, 0x74, 0x61, - 0x53, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x65, - 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x65, 0x74, - 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x61, 0x0a, 0x12, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x23, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, - 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x65, 0x6e, - 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x6f, 0x72, - 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, - 0x12, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4c, - 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x65, - 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x82, 0xd3, 0xe4, 0x93, 0x02, - 0x10, 0x12, 0x0e, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6c, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x12, 0x64, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x69, 0x67, 0x6e, 0x4c, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x12, 0x1d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x73, - 0x69, 0x67, 0x6e, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x1d, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x17, - 0x22, 0x15, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, - 0x2f, 0x72, 0x65, 0x73, 0x69, 0x67, 0x6e, 0x32, 0x60, 0x0a, 0x0d, 0x54, 0x61, 0x73, 0x6b, 0x53, - 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, - 0x64, 0x75, 0x6c, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, - 0x65, 0x70, 0x62, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x61, 0x73, 0x6b, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, - 0x70, 0x62, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xc1, 0x03, 0x0a, 0x0a, 0x4a, 0x6f, - 0x62, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x12, 0x51, 0x0a, 0x09, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x12, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, - 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x0d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, - 0x22, 0x19, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x13, 0x22, 0x0c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, - 0x31, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x3a, 0x03, 0x6a, 0x6f, 0x62, 0x12, 0x4d, 0x0a, 0x06, 0x47, - 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x12, 0x17, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, - 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, - 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x22, 0x1b, 0x82, - 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x12, 0x13, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6a, - 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x3d, 0x2a, 0x7d, 0x12, 0x57, 0x0a, 0x08, 0x4c, 0x69, - 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x19, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, - 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, - 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x14, 0x82, - 0xd3, 0xe4, 0x93, 0x02, 0x0e, 0x12, 0x0c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6a, - 0x6f, 0x62, 0x73, 0x12, 0x5a, 0x0a, 0x09, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, - 0x12, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x43, 0x61, 0x6e, 0x63, - 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x65, - 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x22, 0x22, 0x82, 0xd3, 0xe4, - 0x93, 0x02, 0x1c, 0x22, 0x1a, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6a, 0x6f, 0x62, - 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x3d, 0x2a, 0x7d, 0x2f, 0x63, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x12, - 0x5c, 0x0a, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x12, 0x1a, 0x2e, 0x65, - 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, - 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x22, 0x1b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x2a, 0x13, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, - 0x31, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x3d, 0x2a, 0x7d, 0x42, 0x2b, 0x5a, - 0x29, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x69, 0x6e, 0x67, - 0x63, 0x61, 0x70, 0x2f, 0x74, 0x69, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x65, 0x6e, 0x67, 0x69, 0x6e, - 0x65, 0x2f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, + 0x0e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x12, + 0x1f, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x20, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, + 0x79, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x12, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x6f, + 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x23, 0x2e, 0x65, 0x6e, 0x67, + 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x6f, 0x72, 0x61, + 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x24, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x09, 0x47, 0x65, 0x74, 0x4c, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x12, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, + 0x47, 0x65, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1b, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4c, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x82, + 0xd3, 0xe4, 0x93, 0x02, 0x10, 0x12, 0x0e, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6c, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x64, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x69, 0x67, 0x6e, 0x4c, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x1d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, + 0x2e, 0x52, 0x65, 0x73, 0x69, 0x67, 0x6e, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x1d, 0x82, 0xd3, + 0xe4, 0x93, 0x02, 0x17, 0x22, 0x15, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6c, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x2f, 0x72, 0x65, 0x73, 0x69, 0x67, 0x6e, 0x32, 0x60, 0x0a, 0x0d, 0x54, + 0x61, 0x73, 0x6b, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x72, 0x12, 0x4f, 0x0a, 0x0c, + 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1d, 0x2e, 0x65, + 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x54, 0x61, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x65, 0x6e, + 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x54, + 0x61, 0x73, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xc1, 0x03, + 0x0a, 0x0a, 0x4a, 0x6f, 0x62, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x12, 0x51, 0x0a, 0x09, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x12, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, + 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, + 0x2e, 0x4a, 0x6f, 0x62, 0x22, 0x19, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x13, 0x3a, 0x03, 0x6a, 0x6f, + 0x62, 0x22, 0x0c, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x12, + 0x4d, 0x0a, 0x06, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x12, 0x17, 0x2e, 0x65, 0x6e, 0x67, 0x69, + 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x0d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, + 0x62, 0x22, 0x1b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x12, 0x13, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x76, 0x31, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x3d, 0x2a, 0x7d, 0x12, 0x57, + 0x0a, 0x08, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x19, 0x2e, 0x65, 0x6e, 0x67, + 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x14, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x0e, 0x12, 0x0c, 0x2f, 0x61, 0x70, 0x69, 0x2f, + 0x76, 0x31, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x5a, 0x0a, 0x09, 0x43, 0x61, 0x6e, 0x63, 0x65, + 0x6c, 0x4a, 0x6f, 0x62, 0x12, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, + 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x0d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x4a, 0x6f, 0x62, 0x22, + 0x22, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x1c, 0x22, 0x1a, 0x2f, 0x61, 0x70, 0x69, 0x2f, 0x76, 0x31, + 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x3d, 0x2a, 0x7d, 0x2f, 0x63, 0x61, 0x6e, + 0x63, 0x65, 0x6c, 0x12, 0x5c, 0x0a, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4a, 0x6f, 0x62, + 0x12, 0x1a, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, + 0x6d, 0x70, 0x74, 0x79, 0x22, 0x1b, 0x82, 0xd3, 0xe4, 0x93, 0x02, 0x15, 0x2a, 0x13, 0x2f, 0x61, + 0x70, 0x69, 0x2f, 0x76, 0x31, 0x2f, 0x6a, 0x6f, 0x62, 0x73, 0x2f, 0x7b, 0x69, 0x64, 0x3d, 0x2a, + 0x7d, 0x42, 0x2b, 0x5a, 0x29, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, + 0x70, 0x69, 0x6e, 0x67, 0x63, 0x61, 0x70, 0x2f, 0x74, 0x69, 0x66, 0x6c, 0x6f, 0x77, 0x2f, 0x65, + 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x70, 0x62, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2040,7 +2040,7 @@ func file_engine_proto_master_proto_rawDescGZIP() []byte { var file_engine_proto_master_proto_enumTypes = make([]protoimpl.EnumInfo, 4) var file_engine_proto_master_proto_msgTypes = make([]protoimpl.MessageInfo, 28) -var file_engine_proto_master_proto_goTypes = []interface{}{ +var file_engine_proto_master_proto_goTypes = []any{ (StoreType)(0), // 0: enginepb.StoreType (Selector_Op)(0), // 1: enginepb.Selector.Op (Job_Type)(0), // 2: enginepb.Job.Type @@ -2135,7 +2135,7 @@ func file_engine_proto_master_proto_init() { } file_engine_proto_resources_proto_init() if !protoimpl.UnsafeEnabled { - file_engine_proto_master_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[0].Exporter = func(v any, i int) any { switch v := v.(*Selector); i { case 0: return &v.state @@ -2147,7 +2147,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[1].Exporter = func(v any, i int) any { switch v := v.(*HeartbeatRequest); i { case 0: return &v.state @@ -2159,7 +2159,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[2].Exporter = func(v any, i int) any { switch v := v.(*HeartbeatResponse); i { case 0: return &v.state @@ -2171,7 +2171,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[3].Exporter = func(v any, i int) any { switch v := v.(*Executor); i { case 0: return &v.state @@ -2183,7 +2183,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[4].Exporter = func(v any, i int) any { switch v := v.(*RegisterExecutorRequest); i { case 0: return &v.state @@ -2195,7 +2195,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[5].Exporter = func(v any, i int) any { switch v := v.(*ListExecutorsRequest); i { case 0: return &v.state @@ -2207,7 +2207,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[6].Exporter = func(v any, i int) any { switch v := v.(*ListExecutorsResponse); i { case 0: return &v.state @@ -2219,7 +2219,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[7].Exporter = func(v any, i int) any { switch v := v.(*Master); i { case 0: return &v.state @@ -2231,7 +2231,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[8].Exporter = func(v any, i int) any { switch v := v.(*ListMastersRequest); i { case 0: return &v.state @@ -2243,7 +2243,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[9].Exporter = func(v any, i int) any { switch v := v.(*ListMastersResponse); i { case 0: return &v.state @@ -2255,7 +2255,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[10].Exporter = func(v any, i int) any { switch v := v.(*ScheduleTaskRequest); i { case 0: return &v.state @@ -2267,7 +2267,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[11].Exporter = func(v any, i int) any { switch v := v.(*ScheduleTaskResponse); i { case 0: return &v.state @@ -2279,7 +2279,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[12].Exporter = func(v any, i int) any { switch v := v.(*GetLeaderRequest); i { case 0: return &v.state @@ -2291,7 +2291,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[13].Exporter = func(v any, i int) any { switch v := v.(*GetLeaderResponse); i { case 0: return &v.state @@ -2303,7 +2303,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[14].Exporter = func(v any, i int) any { switch v := v.(*ResignLeaderRequest); i { case 0: return &v.state @@ -2315,7 +2315,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[15].Exporter = func(v any, i int) any { switch v := v.(*Job); i { case 0: return &v.state @@ -2327,7 +2327,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[16].Exporter = func(v any, i int) any { switch v := v.(*CreateJobRequest); i { case 0: return &v.state @@ -2339,7 +2339,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[17].Exporter = func(v any, i int) any { switch v := v.(*GetJobRequest); i { case 0: return &v.state @@ -2351,7 +2351,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[18].Exporter = func(v any, i int) any { switch v := v.(*ListJobsRequest); i { case 0: return &v.state @@ -2363,7 +2363,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[19].Exporter = func(v any, i int) any { switch v := v.(*ListJobsResponse); i { case 0: return &v.state @@ -2375,7 +2375,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[20].Exporter = func(v any, i int) any { switch v := v.(*CancelJobRequest); i { case 0: return &v.state @@ -2387,7 +2387,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[21].Exporter = func(v any, i int) any { switch v := v.(*DeleteJobRequest); i { case 0: return &v.state @@ -2399,7 +2399,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[22].Exporter = func(v any, i int) any { switch v := v.(*QueryMetaStoreRequest); i { case 0: return &v.state @@ -2411,7 +2411,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[23].Exporter = func(v any, i int) any { switch v := v.(*QueryMetaStoreResponse); i { case 0: return &v.state @@ -2423,7 +2423,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[24].Exporter = func(v any, i int) any { switch v := v.(*QueryStorageConfigRequest); i { case 0: return &v.state @@ -2435,7 +2435,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[25].Exporter = func(v any, i int) any { switch v := v.(*QueryStorageConfigResponse); i { case 0: return &v.state @@ -2447,7 +2447,7 @@ func file_engine_proto_master_proto_init() { return nil } } - file_engine_proto_master_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_master_proto_msgTypes[27].Exporter = func(v any, i int) any { switch v := v.(*Job_Error); i { case 0: return &v.state diff --git a/engine/enginepb/projects.pb.go b/engine/enginepb/projects.pb.go index e7a72aae763..c8bccfd6007 100644 --- a/engine/enginepb/projects.pb.go +++ b/engine/enginepb/projects.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.0 +// protoc-gen-go v1.34.2 // protoc v3.20.1 // source: engine/proto/projects.proto @@ -104,7 +104,7 @@ func file_engine_proto_projects_proto_rawDescGZIP() []byte { } var file_engine_proto_projects_proto_msgTypes = make([]protoimpl.MessageInfo, 1) -var file_engine_proto_projects_proto_goTypes = []interface{}{ +var file_engine_proto_projects_proto_goTypes = []any{ (*ProjectInfo)(nil), // 0: enginepb.ProjectInfo } var file_engine_proto_projects_proto_depIdxs = []int32{ @@ -121,7 +121,7 @@ func file_engine_proto_projects_proto_init() { return } if !protoimpl.UnsafeEnabled { - file_engine_proto_projects_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_projects_proto_msgTypes[0].Exporter = func(v any, i int) any { switch v := v.(*ProjectInfo); i { case 0: return &v.state diff --git a/engine/enginepb/resources.pb.go b/engine/enginepb/resources.pb.go index fdd1822a8b4..4915c07aaa2 100644 --- a/engine/enginepb/resources.pb.go +++ b/engine/enginepb/resources.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.0 +// protoc-gen-go v1.34.2 // protoc v3.20.1 // source: engine/proto/resources.proto @@ -471,7 +471,7 @@ func file_engine_proto_resources_proto_rawDescGZIP() []byte { } var file_engine_proto_resources_proto_msgTypes = make([]protoimpl.MessageInfo, 7) -var file_engine_proto_resources_proto_goTypes = []interface{}{ +var file_engine_proto_resources_proto_goTypes = []any{ (*CreateResourceRequest)(nil), // 0: enginepb.CreateResourceRequest (*CreateResourceResponse)(nil), // 1: enginepb.CreateResourceResponse (*ResourceKey)(nil), // 2: enginepb.ResourceKey @@ -505,7 +505,7 @@ func file_engine_proto_resources_proto_init() { } file_engine_proto_projects_proto_init() if !protoimpl.UnsafeEnabled { - file_engine_proto_resources_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_resources_proto_msgTypes[0].Exporter = func(v any, i int) any { switch v := v.(*CreateResourceRequest); i { case 0: return &v.state @@ -517,7 +517,7 @@ func file_engine_proto_resources_proto_init() { return nil } } - file_engine_proto_resources_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_resources_proto_msgTypes[1].Exporter = func(v any, i int) any { switch v := v.(*CreateResourceResponse); i { case 0: return &v.state @@ -529,7 +529,7 @@ func file_engine_proto_resources_proto_init() { return nil } } - file_engine_proto_resources_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_resources_proto_msgTypes[2].Exporter = func(v any, i int) any { switch v := v.(*ResourceKey); i { case 0: return &v.state @@ -541,7 +541,7 @@ func file_engine_proto_resources_proto_init() { return nil } } - file_engine_proto_resources_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_resources_proto_msgTypes[3].Exporter = func(v any, i int) any { switch v := v.(*QueryResourceRequest); i { case 0: return &v.state @@ -553,7 +553,7 @@ func file_engine_proto_resources_proto_init() { return nil } } - file_engine_proto_resources_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_resources_proto_msgTypes[4].Exporter = func(v any, i int) any { switch v := v.(*QueryResourceResponse); i { case 0: return &v.state @@ -565,7 +565,7 @@ func file_engine_proto_resources_proto_init() { return nil } } - file_engine_proto_resources_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_resources_proto_msgTypes[5].Exporter = func(v any, i int) any { switch v := v.(*RemoveResourceRequest); i { case 0: return &v.state @@ -577,7 +577,7 @@ func file_engine_proto_resources_proto_init() { return nil } } - file_engine_proto_resources_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_resources_proto_msgTypes[6].Exporter = func(v any, i int) any { switch v := v.(*RemoveResourceResponse); i { case 0: return &v.state diff --git a/engine/enginepb/test.pb.go b/engine/enginepb/test.pb.go index 787ff75592f..22435f85ab9 100644 --- a/engine/enginepb/test.pb.go +++ b/engine/enginepb/test.pb.go @@ -1,6 +1,6 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.0 +// protoc-gen-go v1.34.2 // protoc v3.20.1 // source: engine/proto/test.proto @@ -246,7 +246,7 @@ func file_engine_proto_test_proto_rawDescGZIP() []byte { var file_engine_proto_test_proto_enumTypes = make([]protoimpl.EnumInfo, 1) var file_engine_proto_test_proto_msgTypes = make([]protoimpl.MessageInfo, 2) -var file_engine_proto_test_proto_goTypes = []interface{}{ +var file_engine_proto_test_proto_goTypes = []any{ (Record_RecordType)(0), // 0: enginepb.Record.RecordType (*Record)(nil), // 1: enginepb.Record (*TestBinlogRequest)(nil), // 2: enginepb.TestBinlogRequest @@ -268,7 +268,7 @@ func file_engine_proto_test_proto_init() { return } if !protoimpl.UnsafeEnabled { - file_engine_proto_test_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_test_proto_msgTypes[0].Exporter = func(v any, i int) any { switch v := v.(*Record); i { case 0: return &v.state @@ -280,7 +280,7 @@ func file_engine_proto_test_proto_init() { return nil } } - file_engine_proto_test_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + file_engine_proto_test_proto_msgTypes[1].Exporter = func(v any, i int) any { switch v := v.(*TestBinlogRequest); i { case 0: return &v.state diff --git a/engine/executor/dm/worker.go b/engine/executor/dm/worker.go index 21445c8f3b8..502b280e6db 100644 --- a/engine/executor/dm/worker.go +++ b/engine/executor/dm/worker.go @@ -152,12 +152,12 @@ func (w *dmWorker) InitImpl(ctx context.Context) error { if err := w.messageAgent.UpdateClient(w.masterID, w); err != nil { return err } - // for dump/load&sync mode task, we needn't to setup external storage - // these two tasks will directly read/write data from/to user specified external storage without executor's management - // for all/full mode task, the dump/load units run on a same executor, so they can access the s3 data under a same executor - // but for dump/load&sync mode task, import API needs a clear S3 URI without exector's prefix, - // what's more, dump/load units may not be executed on a same executor, - // so we choose to use user's own external storage and don't set up here. + // For dump/load/load&sync mode tasks, we don’t need to set up external storage. + // These tasks directly read/write data to/from user-specified external storage without the executor's management. + // In all/full mode tasks, dump/load units run on the same executor, allowing access to S3 data under the same executor's namespace. + // However, for dump/load & sync mode tasks, the import API requires a plain S3 URI without the executor's prefix. + // Additionally, dump/load units may not run on the same executor, + // so we opt to use the user’s external storage directly instead of configuring it here. if (w.cfg.Mode == dmconfig.ModeAll || w.cfg.Mode == dmconfig.ModeFull) && w.needExtStorage { if err := w.setupStorage(ctx); err != nil { return err @@ -258,8 +258,10 @@ func (w *dmWorker) updateStatusWhenStageChange(ctx context.Context) error { return w.UpdateStatus(ctx, status) } - // now we are in StageFinished - // for all and full mode, resource is managed by engine, we need to discard them + // Now we are in StageFinished + // For all and full mode, resource is managed by engine, we need to discard them + // In standalone modes (e.g., dump and load), we use user-specified storage. + // No additional operations on storage are needed, leaving management to the user. if w.cfg.Mode == dmconfig.ModeAll || w.cfg.Mode == dmconfig.ModeFull { switch w.workerType { case frameModel.WorkerDMDump: diff --git a/go.mod b/go.mod index 84f0c3f609e..929e90c020e 100644 --- a/go.mod +++ b/go.mod @@ -4,6 +4,7 @@ go 1.23 require ( cloud.google.com/go/storage v1.39.1 + github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c github.com/DATA-DOG/go-sqlmock v1.5.0 github.com/IBM/sarama v1.41.2 @@ -26,7 +27,7 @@ require ( github.com/docker/go-units v0.5.0 github.com/dustin/go-humanize v1.0.1 github.com/edwingeng/deque v0.0.0-20191220032131-8596380dee17 - github.com/fatih/color v1.17.0 + github.com/fatih/color v1.18.0 github.com/gavv/monotime v0.0.0-20190418164738-30dba4353424 github.com/getkin/kin-openapi v0.80.0 github.com/gin-gonic/gin v1.9.1 @@ -67,11 +68,11 @@ require ( github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f github.com/pingcap/failpoint v0.0.0-20240528011301-b51a646c7c86 - github.com/pingcap/kvproto v0.0.0-20240924080114-4a3e17f5e62d + github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8 github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d - github.com/pingcap/tidb v1.1.0-beta.0.20241107131230-e2505e95a03c + github.com/pingcap/tidb v1.1.0-beta.0.20241128113414-7518a29fb1d0 github.com/pingcap/tidb-dashboard v0.0.0-20240326110213-9768844ff5d7 - github.com/pingcap/tidb/pkg/parser v0.0.0-20241014034929-94b2ac04a0c4 + github.com/pingcap/tidb/pkg/parser v0.0.0-20241119124618-50b5cd27d413 github.com/prometheus/client_golang v1.20.5 github.com/prometheus/client_model v0.6.1 github.com/r3labs/diff v1.1.0 @@ -89,9 +90,9 @@ require ( github.com/swaggo/swag v1.16.3 github.com/syndtr/goleveldb v1.0.1-0.20210305035536-64b5b1c73954 github.com/thanhpk/randstr v1.0.6 - github.com/tikv/client-go/v2 v2.0.8-0.20241023023120-691e80ae0ea9 + github.com/tikv/client-go/v2 v2.0.8-0.20241120024459-05d115b3e88b github.com/tikv/pd v1.1.0-beta.0.20240407022249-7179657d129b - github.com/tikv/pd/client v0.0.0-20241016064947-b70107ec31e6 + github.com/tikv/pd/client v0.0.0-20241111073742-238d4d79ea31 github.com/tinylib/msgp v1.1.6 github.com/uber-go/atomic v1.4.0 github.com/vmihailenco/msgpack/v5 v5.3.5 @@ -111,11 +112,11 @@ require ( go.uber.org/ratelimit v0.2.0 go.uber.org/zap v1.27.0 golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 - golang.org/x/net v0.30.0 + golang.org/x/net v0.31.0 golang.org/x/oauth2 v0.23.0 - golang.org/x/sync v0.8.0 - golang.org/x/sys v0.26.0 - golang.org/x/text v0.19.0 + golang.org/x/sync v0.9.0 + golang.org/x/sys v0.27.0 + golang.org/x/text v0.20.0 golang.org/x/time v0.7.0 google.golang.org/genproto/googleapis/api v0.0.0-20240401170217-c3f982113cda google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 @@ -129,7 +130,6 @@ require ( require ( cloud.google.com/go/kms v1.15.8 // indirect - github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.0.0 // indirect github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect github.com/andybalholm/brotli v1.0.5 // indirect @@ -173,7 +173,6 @@ require ( github.com/klauspost/cpuid/v2 v2.2.4 // indirect github.com/ks3sdklib/aws-sdk-go v1.2.9 // indirect github.com/kylelemons/godebug v1.1.0 // indirect - github.com/lance6716/pebble v0.0.0-20241104073946-6f55c09bd183 // indirect github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect @@ -200,8 +199,8 @@ require ( golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect google.golang.org/genproto v0.0.0-20240401170217-c3f982113cda // indirect gopkg.in/inf.v0 v0.9.1 // indirect - k8s.io/api v0.28.6 // indirect - k8s.io/apimachinery v0.28.6 // indirect + k8s.io/api v0.29.11 // indirect + k8s.io/apimachinery v0.29.11 // indirect k8s.io/klog/v2 v2.120.1 // indirect k8s.io/utils v0.0.0-20230726121419-3b25d923346b // indirect sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd // indirect @@ -330,7 +329,7 @@ require ( github.com/pingcap/fn v1.0.0 // indirect github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 // indirect github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 - github.com/pingcap/tipb v0.0.0-20241008083645-0bcddae67837 // indirect + github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect @@ -339,7 +338,7 @@ require ( github.com/prometheus/procfs v0.15.1 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect github.com/rivo/uniseg v0.4.7 // indirect - github.com/rogpeppe/go-internal v1.12.0 // indirect + github.com/rogpeppe/go-internal v1.13.1 // indirect github.com/rs/cors v1.7.0 // indirect github.com/sasha-s/go-deadlock v0.3.5 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect @@ -382,10 +381,10 @@ require ( go.opentelemetry.io/otel/sdk v1.24.0 // indirect go.opentelemetry.io/otel/trace v1.24.0 // indirect go.opentelemetry.io/proto/otlp v1.1.0 // indirect - golang.org/x/crypto v0.28.0 // indirect - golang.org/x/mod v0.21.0 // indirect - golang.org/x/term v0.25.0 - golang.org/x/tools v0.26.0 // indirect + golang.org/x/crypto v0.29.0 // indirect + golang.org/x/mod v0.22.0 // indirect + golang.org/x/term v0.26.0 + golang.org/x/tools v0.27.0 // indirect google.golang.org/api v0.170.0 // indirect gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect diff --git a/go.sum b/go.sum index 4c5951e860b..ced4f1bc2a2 100644 --- a/go.sum +++ b/go.sum @@ -308,8 +308,8 @@ github.com/envoyproxy/go-control-plane v0.10.2-0.20220325020618-49ff273808a1/go. github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= -github.com/fatih/color v1.17.0 h1:GlRw1BRJxkpqUCBKzKOw098ed57fEsKeNjpTe3cSjK4= -github.com/fatih/color v1.17.0/go.mod h1:YZ7TlrGPkiz6ku9fK3TLD/pl3CpsiFyu8N92HLgmosI= +github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= +github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= github.com/fatih/structs v1.1.0 h1:Q7juDM0QtcnhCpeyLGQKyg4TOIghuNXrkL32pHAUMxo= github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= github.com/felixge/fgprof v0.9.3 h1:VvyZxILNuCiUCSXtPtYmmtGvb65nqXh2QFWc0Wpf2/g= @@ -445,7 +445,6 @@ github.com/godbus/dbus/v5 v5.0.4 h1:9349emZab16e7zQvpmsbtjc18ykshndd8y2PG3sgJbA= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/gateway v1.1.0 h1:u0SuhL9+Il+UbjM9VIE3ntfRujKbvVpFvNB4HbjeVQ0= github.com/gogo/gateway v1.1.0/go.mod h1:S7rR8FRQyG3QFESeSv4l2WnsyzlCLG0CzBbUUo/mbic= -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.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= @@ -482,7 +481,6 @@ github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5y 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.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= @@ -493,6 +491,7 @@ github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw 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/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -719,8 +718,6 @@ github.com/labstack/echo/v4 v4.10.0/go.mod h1:S/T/5fy/GigaXnHTkh0ZGe4LpkkQysvRjF github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/labstack/gommon v0.4.0 h1:y7cvthEAEbU0yHOf4axH8ZG2NH8knB9iNSoTO8dyIk8= github.com/labstack/gommon v0.4.0/go.mod h1:uW6kP17uPlLJsD3ijUYn3/M5bAxtlZhMI6m3MFxTMTM= -github.com/lance6716/pebble v0.0.0-20241104073946-6f55c09bd183 h1:CrFmpCAT5PGMgmQadTa2lXZrjuvpknONB1/pyxiyDsM= -github.com/lance6716/pebble v0.0.0-20241104073946-6f55c09bd183/go.mod h1:ZxnWA3Ab0ufDIyppyzL16j6HFNpdXeiU/1cE4Wlv/lQ= github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= github.com/leodido/go-urn v1.2.4 h1:XlAE/cm/ms7TE/VMVoduSpNBoyc2dOxHs5MZSwAN63Q= @@ -840,8 +837,8 @@ github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= 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/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= -github.com/onsi/gomega v1.20.1/go.mod h1:DtrZpjmvpn2mPm4YWQa0/ALMDj9v4YxLgojwPeREyVo= +github.com/onsi/gomega v1.29.0 h1:KIA/t2t5UBzoirT4H9tsML45GEbo3ouUnBHsCfD2tVg= +github.com/onsi/gomega v1.29.0/go.mod h1:9sxs+SwGrKI0+PWe4Fxa9tFQQBG5xSsSbMXOI8PPpoQ= github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U= github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= github.com/opencontainers/image-spec v1.0.3-0.20211202183452-c5a74bcca799 h1:rc3tiVYb5z54aKaDfakKn0dDjIyPpTtszkjuMzyt7ec= @@ -897,8 +894,8 @@ github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= 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-20240924080114-4a3e17f5e62d h1:vSdKTrF6kpcd56G5BLP0Bz88Nho2tDo7IR1+oSsBAfc= -github.com/pingcap/kvproto v0.0.0-20240924080114-4a3e17f5e62d/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= +github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8 h1:aNNifhc6xCjXKejjiNYtJJLFNMXnoDiXxkJIg1JErQE= +github.com/pingcap/kvproto v0.0.0-20241120022153-92b0414aeed8/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= @@ -906,14 +903,14 @@ github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d h1:y3EueKVfVykdpTyfU github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d/go.mod h1:ORfBOFp1eteu2odzsyaxI+b8TzJwgjwyQcGhI+9SfEA= github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 h1:T4pXRhBflzDeAhmOQHNPRRogMYxP13V7BkYw3ZsoSfE= github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5/go.mod h1:rlimy0GcTvjiJqvD5mXTRr8O2eNZPBrcUgiWVYp9530= -github.com/pingcap/tidb v1.1.0-beta.0.20241107131230-e2505e95a03c h1:PLop/4ZOM5Uxd9co41gIKztalJfe+jSxYpuu9DdndBE= -github.com/pingcap/tidb v1.1.0-beta.0.20241107131230-e2505e95a03c/go.mod h1:tAyfBFGzdPPLJrSU76pAtddlNtWMLynRplC/ESWvvuM= +github.com/pingcap/tidb v1.1.0-beta.0.20241128113414-7518a29fb1d0 h1:VLYgUJUkje1htz0uy5177KWjQiVNSTLunaPfxeeQWXk= +github.com/pingcap/tidb v1.1.0-beta.0.20241128113414-7518a29fb1d0/go.mod h1:VjIgD6ji/BPFhVeItPzQzN7XUbLjAshYAJdq4HDgA38= github.com/pingcap/tidb-dashboard v0.0.0-20240326110213-9768844ff5d7 h1:eFu98FbfJB7PKWOtkaV6YNXXJWqDhczQX56j/iucgU4= github.com/pingcap/tidb-dashboard v0.0.0-20240326110213-9768844ff5d7/go.mod h1:ucZBRz52icb23T/5Z4CsuUHmarYiin7p2MeiVBe+o8c= -github.com/pingcap/tidb/pkg/parser v0.0.0-20241014034929-94b2ac04a0c4 h1:+HMT6yCrQ0GGkhOPtP22u0/PnPmfXir9WC3NezF3OHo= -github.com/pingcap/tidb/pkg/parser v0.0.0-20241014034929-94b2ac04a0c4/go.mod h1:Hju1TEWZvrctQKbztTRwXH7rd41Yq0Pgmq4PrEKcq7o= -github.com/pingcap/tipb v0.0.0-20241008083645-0bcddae67837 h1:tyIymn821fB8gUmqafdvLlcFkVOpgyJXImoYJ8n9oJE= -github.com/pingcap/tipb v0.0.0-20241008083645-0bcddae67837/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tidb/pkg/parser v0.0.0-20241119124618-50b5cd27d413 h1:Yr9Hc6y70JvAcRMCqUu5fdpzHc+tnHxxnQLmeVLnxxo= +github.com/pingcap/tidb/pkg/parser v0.0.0-20241119124618-50b5cd27d413/go.mod h1:Hju1TEWZvrctQKbztTRwXH7rd41Yq0Pgmq4PrEKcq7o= +github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e h1:7DdrYVwWpYr4o1AyKl8T376B4h2RsMEjkmom8MxQuuM= +github.com/pingcap/tipb v0.0.0-20241105053214-f91fdb81a69e/go.mod h1:zrnYy8vReNODg8G0OiYaX9OK+kpq+rK1jHmvd1DnIWw= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= @@ -969,8 +966,8 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= github.com/rogpeppe/go-internal v1.8.0/go.mod h1:WmiCO8CzOY8rg0OYDC4/i/2WRWAB6poM+XZ2dLUbcbE= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= -github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= -github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= +github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= +github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= github.com/rs/cors v1.7.0 h1:+88SsELBHx5r+hZ8TCkggzSstaWNbDvThkVK8H6f9ik= 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= @@ -1108,12 +1105,12 @@ github.com/tidwall/rtree v0.0.0-20180113144539-6cd427091e0e/go.mod h1:/h+UnNGt0I github.com/tidwall/tinyqueue v0.0.0-20180302190814-1e39f5511563/go.mod h1:mLqSmt7Dv/CNneF2wfcChfN1rvapyQr01LGKnKex0DQ= github.com/tidwall/tinyqueue v0.1.1 h1:SpNEvEggbpyN5DIReaJ2/1ndroY8iyEGxPYxoSaymYE= github.com/tidwall/tinyqueue v0.1.1/go.mod h1:O/QNHwrnjqr6IHItYrzoHAKYhBkLI67Q096fQP5zMYw= -github.com/tikv/client-go/v2 v2.0.8-0.20241023023120-691e80ae0ea9 h1:1Fgp6FqjgXEj/CKcegdXu3wLo77sx7JM9NPC7sF0io0= -github.com/tikv/client-go/v2 v2.0.8-0.20241023023120-691e80ae0ea9/go.mod h1:WAp0oZxDL+3GX+QhJdG0quubJUzEH8LrFofmIxleJhs= +github.com/tikv/client-go/v2 v2.0.8-0.20241120024459-05d115b3e88b h1:/hmt2FCt34rCVBX9dswiSdHOkppP67VWaESryTxDKc8= +github.com/tikv/client-go/v2 v2.0.8-0.20241120024459-05d115b3e88b/go.mod h1:NI2GfVlB9n7DsIGCxrKcD4psrcuFNEV8m1BgyzK1Amc= github.com/tikv/pd v1.1.0-beta.0.20240407022249-7179657d129b h1:t2XoZp4UHrkPpYPsxbRTRVExJnriWlh+ZsDIfpYyd98= github.com/tikv/pd v1.1.0-beta.0.20240407022249-7179657d129b/go.mod h1:7HJMdb0O5umNpZIFt8e/wKAcEmH99n2HsYgXX+vZj3k= -github.com/tikv/pd/client v0.0.0-20241016064947-b70107ec31e6 h1:u0z6yR68sg0pextuabJv/bD4mvwBe8iFeQOdymBUy0E= -github.com/tikv/pd/client v0.0.0-20241016064947-b70107ec31e6/go.mod h1:W5a0sDadwUpI9k8p7M77d3jo253ZHdmua+u4Ho4Xw8U= +github.com/tikv/pd/client v0.0.0-20241111073742-238d4d79ea31 h1:oAYc4m5Eu1OY9ogJ103VO47AYPHvhtzbUPD8L8B67Qk= +github.com/tikv/pd/client v0.0.0-20241111073742-238d4d79ea31/go.mod h1:W5a0sDadwUpI9k8p7M77d3jo253ZHdmua+u4Ho4Xw8U= github.com/tinylib/msgp v1.1.6 h1:i+SbKraHhnrf9M5MYmvQhFnbLhAXSDWF8WWsuyRdocw= github.com/tinylib/msgp v1.1.6/go.mod h1:75BAfg2hauQhs3qedfdDZmWAPcFMAvJE5b9rGOMufyw= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= @@ -1301,8 +1298,8 @@ golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= -golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= -golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= +golang.org/x/crypto v0.29.0 h1:L5SG1JTTXupVV3n6sUqMTeWbjAyfPwoda2DLX8J8FrQ= +golang.org/x/crypto v0.29.0/go.mod h1:+F4F4N5hv6v38hfeYwTdx20oUvLLc+QfrE9Ax9HtgRg= 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-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1358,8 +1355,8 @@ 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/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.21.0 h1:vvrHzRwRfVKSiLrG+d4FMl/Qi4ukBCE6kZlTUkDYRT0= -golang.org/x/mod v0.21.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= +golang.org/x/mod v0.22.0 h1:D4nJWe9zXqHOmWqj4VMOJhvzj7bEZg4wEYa759z1pH4= +golang.org/x/mod v0.22.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= golang.org/x/net v0.0.0-20180530234432-1e491301e022/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= 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= @@ -1403,8 +1400,8 @@ golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= +golang.org/x/net v0.31.0 h1:68CPQngjLL0r2AlUKiSxtQFKvzRVbnzLwMUn5SzcLHo= +golang.org/x/net v0.31.0/go.mod h1:P4fl1q7dY2hnZFxEk4pPSkDHF+QqjitcnDjUQyMM+pM= 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= @@ -1423,8 +1420,8 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.9.0 h1:fEo0HyrW1GIgZdpbhCRO0PkJajUS5H9IFUztCgEo2jQ= +golang.org/x/sync v0.9.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= 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-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1498,8 +1495,8 @@ golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.27.0 h1:wBqf8DvsY9Y/2P8gAfPDEYNuS30J4lPHJxXSb/nJZ+s= +golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1508,8 +1505,8 @@ golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= -golang.org/x/term v0.25.0 h1:WtHI/ltw4NvSUig5KARz9h521QvRC8RmF/cuYqifU24= -golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= +golang.org/x/term v0.26.0 h1:WEQa6V3Gja/BhNxg540hBip/kkaYtRg3cxg4oXSw4AU= +golang.org/x/term v0.26.0/go.mod h1:Si5m1o57C5nBNQo5z1iq+XDijt21BDBDp2bK0QI8e3E= 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= @@ -1524,8 +1521,8 @@ golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.20.0 h1:gK/Kv2otX8gz+wn7Rmb3vT96ZwuoxnQlY+HlJVj7Qug= +golang.org/x/text v0.20.0/go.mod h1:D4IsuqiFMhST5bX19pQ9ikHC2GsaKyk/oF+pn3ducp4= 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= @@ -1580,7 +1577,6 @@ golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200918232735-d647fc253266/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= golang.org/x/tools v0.0.0-20201022035929-9cf592e881e9/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210114065538-d78b04bdf963/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= @@ -1589,8 +1585,8 @@ golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= -golang.org/x/tools v0.26.0 h1:v/60pFQmzmT9ExmjDv2gGIfi3OqfKoEP6I5+umXlbnQ= -golang.org/x/tools v0.26.0/go.mod h1:TPVVj70c7JJ3WCazhD8OdXcZg/og+b9+tH/KxylGwH0= +golang.org/x/tools v0.27.0 h1:qEKojBykQkQ4EynWy4S8Weg69NumxKdn40Fce3uc/8o= +golang.org/x/tools v0.27.0/go.mod h1:sUi0ZgbwW9ZPAq26Ekut+weQPR5eIM6GQLQ1Yjm1H0Q= 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= @@ -1745,10 +1741,10 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh 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.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= -k8s.io/api v0.28.6 h1:yy6u9CuIhmg55YvF/BavPBBXB+5QicB64njJXxVnzLo= -k8s.io/api v0.28.6/go.mod h1:AM6Ys6g9MY3dl/XNaNfg/GePI0FT7WBGu8efU/lirAo= -k8s.io/apimachinery v0.28.6 h1:RsTeR4z6S07srPg6XYrwXpTJVMXsjPXn0ODakMytSW0= -k8s.io/apimachinery v0.28.6/go.mod h1:QFNX/kCl/EMT2WTSz8k4WLCv2XnkOLMaL8GAVRMdpsA= +k8s.io/api v0.29.11 h1:6FwDo33f1WX5Yu0RQTX9YAd3wth8Ik0B4SXQKsoQfbk= +k8s.io/api v0.29.11/go.mod h1:3TDAW1OpFbz/Yx5r0W06b6eiAfHEwtH61VYDzpTU4Ng= +k8s.io/apimachinery v0.29.11 h1:55+6ue9advpA7T0sX2ZJDHCLKuiFfrAAR/39VQN9KEQ= +k8s.io/apimachinery v0.29.11/go.mod h1:i3FJVwhvSp/6n8Fl4K97PJEP8C+MM+aoDq4+ZJBf70Y= k8s.io/klog/v2 v2.120.1 h1:QXU6cPEOIslTGvZaXvFWiP9VKyeet3sawzTOvdXb4Vw= k8s.io/klog/v2 v2.120.1/go.mod h1:3Jpz1GvMt720eyJH1ckRHK1EDfpxISzJ7I9OYgaDtPE= k8s.io/utils v0.0.0-20230726121419-3b25d923346b h1:sgn3ZU783SCgtaSJjpcVVlRqd6GSnlTLKgpAAttJvpI= diff --git a/metrics/alertmanager/ticdc.rules.yml b/metrics/alertmanager/ticdc.rules.yml index 03002763237..09b444602b6 100644 --- a/metrics/alertmanager/ticdc.rules.yml +++ b/metrics/alertmanager/ticdc.rules.yml @@ -88,17 +88,17 @@ groups: summary: cdc processor exits with error - alert: ticdc_changefeed_meet_error - expr: (max_over_time(ticdc_owner_status[1m]) == 1) > 0 + expr: (max_over_time(ticdc_owner_status[1m]) == 1 or max_over_time(ticdc_owner_status[1m]) == 6) > 0 for: 1m labels: env: ENV_LABELS_ENV level: warning - expr: (max_over_time(ticdc_owner_status[1m]) == 1) > 0 + expr: (max_over_time(ticdc_owner_status[1m]) == 1 or max_over_time(ticdc_owner_status[1m]) == 6) > 0 annotations: description: 'cluster: ENV_LABELS_ENV, instance: {{ $labels.instance }}, values: {{ $value }}' value: '{{ $value }}' summary: cdc changefeed meet error - + # tikv related alter rules - alert: tikv_cdc_min_resolved_ts_no_change_for_1m expr: changes(tikv_cdc_min_resolved_ts[1m]) < 1 and ON (instance) tikv_cdc_region_resolve_status{status="resolved"} > 0 and ON (instance) tikv_cdc_captured_region_total > 0 @@ -134,4 +134,4 @@ groups: annotations: description: 'cluster: ENV_LABELS_ENV, instance: {{ $labels.instance }}, values:{{ $value }}' value: '{{ $value }}' - summary: TiCDC heap memory usage is over 10 GB \ No newline at end of file + summary: TiCDC heap memory usage is over 10 GB diff --git a/pkg/ddl/util.go b/pkg/ddl/util.go new file mode 100644 index 00000000000..70e789a33ac --- /dev/null +++ b/pkg/ddl/util.go @@ -0,0 +1,62 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "strings" + + "github.com/pingcap/log" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/format" + // NOTE: Do not remove the `test_driver` import. + // For details, refer to: https://github.com/pingcap/parser/issues/43 + _ "github.com/pingcap/tidb/pkg/parser/test_driver" + "github.com/pingcap/tiflow/pkg/errors" + "go.uber.org/zap" +) + +// SplitQueries takes a string containing multiple SQL statements and splits them into individual SQL statements. +// This function is designed for scenarios like batch creation of tables, where multiple `CREATE TABLE` statements +// might be combined into a single query string. +func SplitQueries(queries string) ([]string, error) { + // Note: The parser is not thread-safe, so we create a new instance of the parser for each use. + // However, the overhead of creating a new parser is minimal, so there is no need to worry about performance. + p := parser.New() + stmts, warns, err := p.ParseSQL(queries) + for _, w := range warns { + log.Warn("parse sql warnning", zap.Error(w)) + } + if err != nil { + return nil, errors.WrapError(errors.ErrTiDBUnexpectedJobMeta, err) + } + + var res []string + for _, stmt := range stmts { + var sb strings.Builder + err := stmt.Restore(&format.RestoreCtx{ + Flags: format.DefaultRestoreFlags, + In: &sb, + }) + if err != nil { + return nil, errors.WrapError(errors.ErrTiDBUnexpectedJobMeta, err) + } + // The (ast.Node).Restore function generates a SQL string representation of the AST (Abstract Syntax Tree) node. + // By default, the resulting SQL string does not include a trailing semicolon ";". + // Therefore, we explicitly append a semicolon here to ensure the SQL statement is complete. + sb.WriteByte(';') + res = append(res, sb.String()) + } + + return res, nil +} diff --git a/pkg/ddl/util_test.go b/pkg/ddl/util_test.go new file mode 100644 index 00000000000..0ba4727eea9 --- /dev/null +++ b/pkg/ddl/util_test.go @@ -0,0 +1,159 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +// TestSplitQueries tests the SplitQueries function +func TestSplitQueries(t *testing.T) { + tests := []struct { + name string + input string + expected []string + expectedError bool + }{ + { + name: "Empty input", + input: "", + expected: []string(nil), + expectedError: false, + }, + { + name: "Single query without trailing semicolon", + input: "CREATE TABLE test (id INT)", + expected: []string{"CREATE TABLE `test` (`id` INT);"}, + expectedError: false, + }, + { + name: "Single query with trailing semicolon", + input: "CREATE TABLE test (id INT);", + expected: []string{"CREATE TABLE `test` (`id` INT);"}, + expectedError: false, + }, + { + name: "Multiple queries with trailing semicolons", + input: ` +CREATE TABLE test1 (id INT); +CREATE TABLE test2 (name VARCHAR(20)); +INSERT INTO test1 VALUES (1); +`, + expected: []string{ + "CREATE TABLE `test1` (`id` INT);", + "CREATE TABLE `test2` (`name` VARCHAR(20));", + "INSERT INTO `test1` VALUES (1);", + }, + expectedError: false, + }, + { + name: "Query with semicolons inside column values", + input: ` +CREATE TABLE test (name VARCHAR(50)); +INSERT INTO test VALUES ('This; is; a test'); +`, + expected: []string{ + "CREATE TABLE `test` (`name` VARCHAR(50));", + "INSERT INTO `test` VALUES (_UTF8MB4'This; is; a test');", + }, + expectedError: false, + }, + { + name: "Query with escaped quotes inside strings", + input: ` +CREATE TABLE test (name VARCHAR(50)); +INSERT INTO test VALUES ('This ''is'' a test'); +`, + expected: []string{ + "CREATE TABLE `test` (`name` VARCHAR(50));", + "INSERT INTO `test` VALUES (_UTF8MB4'This ''is'' a test');", + }, + expectedError: false, + }, + { + name: "Nested queries or functions with semicolons", + input: ` +CREATE TABLE test (id INT, name VARCHAR(50)); +INSERT INTO test VALUES (1, CONCAT('Name;', 'Test')); +`, + expected: []string{ + "CREATE TABLE `test` (`id` INT,`name` VARCHAR(50));", + "INSERT INTO `test` VALUES (1,CONCAT(_UTF8MB4'Name;', _UTF8MB4'Test'));", + }, + expectedError: false, + }, + { + name: "Malformed SQL query", + input: "CREATE TABLE test (id INT;", + expected: nil, + expectedError: true, + }, + { + name: "SQL injection edge case", + input: ` +CREATE TABLE users (id INT, name VARCHAR(50)); +INSERT INTO users VALUES (1, 'test; DROP TABLE users; --'); +`, + expected: []string{ + "CREATE TABLE `users` (`id` INT,`name` VARCHAR(50));", + "INSERT INTO `users` VALUES (1,_UTF8MB4'test; DROP TABLE users; --');", + }, + expectedError: false, + }, + { + name: "Complex queries with comments", + input: ` +-- This is a comment +CREATE TABLE test (id INT); -- Inline comment +/* Multi-line +comment */ +INSERT INTO test VALUES (1); +`, + expected: []string{ + "CREATE TABLE `test` (`id` INT);", + "INSERT INTO `test` VALUES (1);", + }, + expectedError: false, + }, + { + name: "Queries with whitespace and newlines", + input: ` + + CREATE TABLE test (id INT); + + INSERT INTO test VALUES (1); + +`, + expected: []string{ + "CREATE TABLE `test` (`id` INT);", + "INSERT INTO `test` VALUES (1);", + }, + expectedError: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + result, err := SplitQueries(tt.input) + if tt.expectedError { + assert.Error(t, err) + } else { + assert.NoError(t, err) + assert.Equal(t, tt.expected, result) + } + }) + } +} diff --git a/pkg/redo/config.go b/pkg/redo/config.go index 8865e9f4f61..4b7821b1048 100644 --- a/pkg/redo/config.go +++ b/pkg/redo/config.go @@ -190,7 +190,7 @@ func IsBlackholeStorage(scheme string) bool { // InitExternalStorage init an external storage. var InitExternalStorage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { - s, err := util.GetExternalStorageWithTimeout(ctx, uri.String(), DefaultTimeout) + s, err := util.GetExternalStorageWithDefaultTimeout(ctx, uri.String()) if err != nil { return nil, errors.WrapError(errors.ErrStorageInitialize, err, fmt.Sprintf("can't init external storage for %s", uri.String())) diff --git a/pkg/sink/codec/avro/avro_test.go b/pkg/sink/codec/avro/avro_test.go index b1baae8a7ed..69723e5e58f 100644 --- a/pkg/sink/codec/avro/avro_test.go +++ b/pkg/sink/codec/avro/avro_test.go @@ -76,6 +76,7 @@ func TestDMLEventE2E(t *testing.T) { decodedEvent, err := decoder.NextRowChangedEvent() require.NoError(t, err) require.NotNil(t, decodedEvent) + require.Equal(t, decodedEvent.GetTableID(), int64(0)) TeardownEncoderAndSchemaRegistry4Testing() } diff --git a/pkg/sink/codec/canal/canal_json_decoder.go b/pkg/sink/codec/canal/canal_json_decoder.go index cd1a0a92c98..bc381e16986 100644 --- a/pkg/sink/codec/canal/canal_json_decoder.go +++ b/pkg/sink/codec/canal/canal_json_decoder.go @@ -26,6 +26,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/storage" + timodel "github.com/pingcap/tidb/pkg/meta/model" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec" @@ -37,6 +40,11 @@ import ( "golang.org/x/text/encoding/charmap" ) +type tableKey struct { + schema string + table string +} + // batchDecoder decodes the byte into the original message. type batchDecoder struct { data []byte @@ -48,6 +56,8 @@ type batchDecoder struct { upstreamTiDB *sql.DB bytesDecoder *encoding.Decoder + + tableInfoCache map[tableKey]*model.TableInfo } // NewBatchDecoder return a decoder for canal-json @@ -72,10 +82,11 @@ func NewBatchDecoder( } return &batchDecoder{ - config: codecConfig, - storage: externalStorage, - upstreamTiDB: db, - bytesDecoder: charmap.ISO8859_1.NewDecoder(), + config: codecConfig, + storage: externalStorage, + upstreamTiDB: db, + bytesDecoder: charmap.ISO8859_1.NewDecoder(), + tableInfoCache: make(map[tableKey]*model.TableInfo), }, nil } @@ -263,6 +274,52 @@ func (b *batchDecoder) assembleHandleKeyOnlyRowChangedEvent( return b.NextRowChangedEvent() } +func setColumnInfos( + tableInfo *timodel.TableInfo, + rawColumns map[string]interface{}, + mysqlType map[string]string, + pkNames map[string]struct{}, +) { + mockColumnID := int64(100) + for name := range rawColumns { + columnInfo := new(timodel.ColumnInfo) + columnInfo.ID = mockColumnID + columnInfo.Name = pmodel.NewCIStr(name) + if utils.IsBinaryMySQLType(mysqlType[name]) { + columnInfo.AddFlag(mysql.BinaryFlag) + } + if _, isPK := pkNames[name]; isPK { + columnInfo.AddFlag(mysql.PriKeyFlag) + } + tableInfo.Columns = append(tableInfo.Columns, columnInfo) + mockColumnID++ + } +} + +func setIndexes( + tableInfo *timodel.TableInfo, + pkNames map[string]struct{}, +) { + indexColumns := make([]*timodel.IndexColumn, 0, len(pkNames)) + for idx, col := range tableInfo.Columns { + name := col.Name.O + if _, ok := pkNames[name]; ok { + indexColumns = append(indexColumns, &timodel.IndexColumn{ + Name: pmodel.NewCIStr(name), + Offset: idx, + }) + } + } + indexInfo := &timodel.IndexInfo{ + ID: 1, + Name: pmodel.NewCIStr("primary"), + Columns: indexColumns, + Unique: true, + Primary: true, + } + tableInfo.Indices = append(tableInfo.Indices, indexInfo) +} + // NextRowChangedEvent implements the RowEventDecoder interface // `HasNext` should be called before this. func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { @@ -282,7 +339,7 @@ func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { } } - result, err := canalJSONMessage2RowChange(b.msg) + result, err := b.canalJSONMessage2RowChange() if err != nil { return nil, err } @@ -299,6 +356,17 @@ func (b *batchDecoder) NextDDLEvent() (*model.DDLEvent, error) { } result := canalJSONMessage2DDLEvent(b.msg) + + schema := *b.msg.getSchema() + table := *b.msg.getTable() + // if receive a table level DDL, just remove the table info to trigger create a new one. + if schema != "" && table != "" { + cacheKey := tableKey{ + schema: schema, + table: table, + } + delete(b.tableInfoCache, cacheKey) + } b.msg = nil return result, nil } diff --git a/pkg/sink/codec/canal/canal_json_message.go b/pkg/sink/codec/canal/canal_json_message.go index c4ce63f6ddb..74234875ed8 100644 --- a/pkg/sink/codec/canal/canal_json_message.go +++ b/pkg/sink/codec/canal/canal_json_message.go @@ -14,12 +14,12 @@ package canal import ( - "sort" "strconv" "strings" "github.com/pingcap/log" timodel "github.com/pingcap/tidb/pkg/meta/model" + pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tiflow/cdc/model" cerrors "github.com/pingcap/tiflow/pkg/errors" @@ -154,36 +154,68 @@ func (c *canalJSONMessageWithTiDBExtension) getCommitTs() uint64 { return c.Extensions.CommitTs } -func canalJSONMessage2RowChange(msg canalJSONMessageInterface) (*model.RowChangedEvent, error) { +func (b *batchDecoder) queryTableInfo(msg canalJSONMessageInterface) *model.TableInfo { + cacheKey := tableKey{ + schema: *msg.getSchema(), + table: *msg.getTable(), + } + tableInfo, ok := b.tableInfoCache[cacheKey] + if !ok { + tableInfo = newTableInfo(msg) + b.tableInfoCache[cacheKey] = tableInfo + } + return tableInfo +} + +func newTableInfo(msg canalJSONMessageInterface) *model.TableInfo { + schema := *msg.getSchema() + table := *msg.getTable() + tidbTableInfo := &timodel.TableInfo{} + tidbTableInfo.Name = pmodel.NewCIStr(table) + + rawColumns := msg.getData() + pkNames := msg.pkNameSet() + mysqlType := msg.getMySQLType() + setColumnInfos(tidbTableInfo, rawColumns, mysqlType, pkNames) + setIndexes(tidbTableInfo, pkNames) + return model.WrapTableInfo(100, schema, 1000, tidbTableInfo) +} + +func (b *batchDecoder) canalJSONMessage2RowChange() (*model.RowChangedEvent, error) { + msg := b.msg result := new(model.RowChangedEvent) + result.TableInfo = b.queryTableInfo(msg) result.CommitTs = msg.getCommitTs() + mysqlType := msg.getMySQLType() var err error if msg.eventType() == canal.EventType_DELETE { // for `DELETE` event, `data` contain the old data, set it as the `PreColumns` - preCols, err := canalJSONColumnMap2RowChangeColumns(msg.getData(), mysqlType) - result.TableInfo = model.BuildTableInfoWithPKNames4Test(*msg.getSchema(), *msg.getTable(), preCols, msg.pkNameSet()) - result.PreColumns = model.Columns2ColumnDatas(preCols, result.TableInfo) - return result, err + result.PreColumns, err = canalJSONColumnMap2RowChangeColumns(msg.getData(), mysqlType, result.TableInfo) + if err != nil { + return nil, err + } + return result, nil } // for `INSERT` and `UPDATE`, `data` contain fresh data, set it as the `Columns` - cols, err := canalJSONColumnMap2RowChangeColumns(msg.getData(), mysqlType) - result.TableInfo = model.BuildTableInfoWithPKNames4Test(*msg.getSchema(), *msg.getTable(), cols, msg.pkNameSet()) - result.Columns = model.Columns2ColumnDatas(cols, result.TableInfo) + result.Columns, err = canalJSONColumnMap2RowChangeColumns(msg.getData(), mysqlType, result.TableInfo) if err != nil { return nil, err } // for `UPDATE`, `old` contain old data, set it as the `PreColumns` if msg.eventType() == canal.EventType_UPDATE { - preCols, err := canalJSONColumnMap2RowChangeColumns(msg.getOld(), mysqlType) - if len(preCols) < len(cols) { - newPreCols := make([]*model.Column, 0, len(preCols)) + preCols, err := canalJSONColumnMap2RowChangeColumns(msg.getOld(), mysqlType, result.TableInfo) + if err != nil { + return nil, err + } + if len(preCols) < len(result.Columns) { + newPreCols := make([]*model.ColumnData, 0, len(preCols)) j := 0 // Columns are ordered by name - for _, col := range cols { - if j < len(preCols) && col.Name == preCols[j].Name { + for _, col := range result.Columns { + if j < len(preCols) && col.ColumnID == preCols[j].ColumnID { newPreCols = append(newPreCols, preCols[j]) j += 1 } else { @@ -192,45 +224,45 @@ func canalJSONMessage2RowChange(msg canalJSONMessageInterface) (*model.RowChange } preCols = newPreCols } - if len(preCols) != len(cols) { - log.Panic("column count mismatch", zap.Any("preCols", preCols), zap.Any("cols", cols)) - } - result.PreColumns = model.Columns2ColumnDatas(preCols, result.TableInfo) - if err != nil { - return nil, err + result.PreColumns = preCols + if len(preCols) != len(result.Columns) { + log.Panic("column count mismatch", zap.Any("preCols", preCols), zap.Any("cols", result.Columns)) } } return result, nil } -func canalJSONColumnMap2RowChangeColumns(cols map[string]interface{}, mysqlType map[string]string) ([]*model.Column, error) { - result := make([]*model.Column, 0, len(cols)) - for name, value := range cols { +func canalJSONColumnMap2RowChangeColumns( + cols map[string]interface{}, + mysqlType map[string]string, + tableInfo *model.TableInfo, +) ([]*model.ColumnData, error) { + result := make([]*model.ColumnData, 0, len(cols)) + for _, columnInfo := range tableInfo.Columns { + name := columnInfo.Name.O + value, ok := cols[name] + if !ok { + continue + } mysqlTypeStr, ok := mysqlType[name] if !ok { // this should not happen, else we have to check encoding for mysqlType. return nil, cerrors.ErrCanalDecodeFailed.GenWithStack( "mysql type does not found, column: %+v, mysqlType: %+v", name, mysqlType) } - col := canalJSONFormatColumn(value, name, mysqlTypeStr) + col := canalJSONFormatColumn(columnInfo.ID, value, mysqlTypeStr) result = append(result, col) } - if len(result) == 0 { - return nil, nil - } - sort.Slice(result, func(i, j int) bool { - return strings.Compare(result[i].Name, result[j].Name) > 0 - }) + return result, nil } -func canalJSONFormatColumn(value interface{}, name string, mysqlTypeStr string) *model.Column { +func canalJSONFormatColumn(columnID int64, value interface{}, mysqlTypeStr string) *model.ColumnData { mysqlType := utils.ExtractBasicMySQLType(mysqlTypeStr) - result := &model.Column{ - Type: mysqlType, - Name: name, - Value: value, + result := &model.ColumnData{ + ColumnID: columnID, + Value: value, } if result.Value == nil { return result diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go index d8682314733..c75700d2d79 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder_test.go @@ -640,50 +640,128 @@ func TestCanalJSONContentCompatibleE2E(t *testing.T) { codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = true codecConfig.ContentCompatible = true + codecConfig.OnlyOutputUpdatedColumns = true builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) require.NoError(t, err) encoder := builder.Build() - _, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) - err = encoder.AppendRowChangedEvent(ctx, "", insertEvent, func() {}) + decoder, err := NewBatchDecoder(ctx, codecConfig, nil) require.NoError(t, err) - message := encoder.Build()[0] + _, insertEvent, updateEvent, deleteEvent := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) + events := []*model.RowChangedEvent{ + insertEvent, + updateEvent, + deleteEvent, + } - decoder, err := NewBatchDecoder(ctx, codecConfig, nil) - require.NoError(t, err) + for _, event := range events { + err = encoder.AppendRowChangedEvent(ctx, "", event, func() {}) + require.NoError(t, err) - err = decoder.AddKeyValue(message.Key, message.Value) - require.NoError(t, err) + message := encoder.Build()[0] - messageType, hasNext, err := decoder.HasNext() + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + + messageType, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, messageType, model.MessageTypeRow) + + decodedEvent, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + require.Equal(t, decodedEvent.CommitTs, event.CommitTs) + require.Equal(t, decodedEvent.TableInfo.GetSchemaName(), event.TableInfo.GetSchemaName()) + require.Equal(t, decodedEvent.TableInfo.GetTableName(), event.TableInfo.GetTableName()) + + obtainedColumns := make(map[string]*model.ColumnData, len(decodedEvent.Columns)) + for _, column := range decodedEvent.Columns { + colName := decodedEvent.TableInfo.ForceGetColumnName(column.ColumnID) + obtainedColumns[colName] = column + } + for _, col := range event.Columns { + colName := event.TableInfo.ForceGetColumnName(col.ColumnID) + decoded, ok := obtainedColumns[colName] + require.True(t, ok) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value) + default: + require.EqualValues(t, v, decoded.Value) + } + } + + obtainedPreColumns := make(map[string]*model.ColumnData, len(decodedEvent.PreColumns)) + for _, column := range decodedEvent.PreColumns { + colName := decodedEvent.TableInfo.ForceGetColumnName(column.ColumnID) + obtainedPreColumns[colName] = column + } + for _, col := range event.PreColumns { + colName := event.TableInfo.ForceGetColumnName(col.ColumnID) + decoded, ok := obtainedPreColumns[colName] + require.True(t, ok) + switch v := col.Value.(type) { + case types.VectorFloat32: + require.EqualValues(t, v.String(), decoded.Value) + default: + require.EqualValues(t, v, decoded.Value) + } + } + } +} + +func TestE2EPartitionTable(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + ctx := context.Background() + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) require.NoError(t, err) - require.True(t, hasNext) - require.Equal(t, messageType, model.MessageTypeRow) + encoder := builder.Build() - decodedEvent, err := decoder.NextRowChangedEvent() + decoder, err := NewBatchDecoder(ctx, codecConfig, nil) require.NoError(t, err) - require.Equal(t, decodedEvent.CommitTs, insertEvent.CommitTs) - require.Equal(t, decodedEvent.TableInfo.GetSchemaName(), insertEvent.TableInfo.GetSchemaName()) - require.Equal(t, decodedEvent.TableInfo.GetTableName(), insertEvent.TableInfo.GetTableName()) - obtainedColumns := make(map[string]*model.ColumnData, len(decodedEvent.Columns)) - for _, column := range decodedEvent.Columns { - colName := decodedEvent.TableInfo.ForceGetColumnName(column.ColumnID) - obtainedColumns[colName] = column - } - for _, col := range insertEvent.Columns { - colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID) - decoded, ok := obtainedColumns[colName] - require.True(t, ok) - switch v := col.Value.(type) { - case types.VectorFloat32: - require.EqualValues(t, v.String(), decoded.Value) - default: - require.EqualValues(t, v, decoded.Value) - } + helper.Tk().MustExec("use test") + + createPartitionTableDDL := helper.DDL2Event(`create table test.t(a int primary key, b int) partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than MAXVALUE)`) + require.NotNil(t, createPartitionTableDDL) + + insertEvent := helper.DML2Event(`insert into test.t values (1, 1)`, "test", "t", "p0") + require.NotNil(t, insertEvent) + + insertEvent1 := helper.DML2Event(`insert into test.t values (11, 11)`, "test", "t", "p1") + require.NotNil(t, insertEvent1) + + insertEvent2 := helper.DML2Event(`insert into test.t values (21, 21)`, "test", "t", "p2") + require.NotNil(t, insertEvent2) + + events := []*model.RowChangedEvent{insertEvent, insertEvent1, insertEvent2} + + for _, event := range events { + err = encoder.AppendRowChangedEvent(ctx, "", event, nil) + require.NoError(t, err) + message := encoder.Build()[0] + + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + tp, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, model.MessageTypeRow, tp) + + decodedEvent, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + // canal-json does not support encode the table id, so it's 0 + require.Equal(t, decodedEvent.GetTableID(), int64(0)) } } diff --git a/pkg/sink/codec/common/config.go b/pkg/sink/codec/common/config.go index be97c6a20c5..ea5ba1cbbcd 100644 --- a/pkg/sink/codec/common/config.go +++ b/pkg/sink/codec/common/config.go @@ -74,7 +74,7 @@ type Config struct { OutputOldValue bool OutputHandleKey bool - // for open protocol + // for open protocol, and canal-json OnlyOutputUpdatedColumns bool // Whether old value should be excluded in the output. OpenOutputOldValue bool diff --git a/pkg/sink/codec/encoder.go b/pkg/sink/codec/encoder.go index 748644e9015..e3b232c9f0c 100644 --- a/pkg/sink/codec/encoder.go +++ b/pkg/sink/codec/encoder.go @@ -17,6 +17,7 @@ import ( "bytes" "context" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/sink/codec/common" ) @@ -82,6 +83,13 @@ func IsColumnValueEqual(preValue, updatedValue interface{}) bool { if ok1 && ok2 { return bytes.Equal(preValueBytes, updatedValueBytes) } + + preValueVector, ok1 := preValue.(types.VectorFloat32) + updatedValueVector, ok2 := updatedValue.(types.VectorFloat32) + if ok1 && ok2 { + return preValueVector.Compare(updatedValueVector) == 0 + } + // mounter use the same table info to parse the value, // the value type should be the same return preValue == updatedValue diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index ad828f030a5..160ffc4253c 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/tidb/pkg/util/cpu" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/sink/dmlsink" "github.com/pingcap/tiflow/pkg/config" @@ -69,6 +70,11 @@ func NewEncoderGroup( if concurrency <= 0 { concurrency = config.DefaultEncoderGroupConcurrency } + limitConcurrency := cpu.GetCPUCount() * 10 + if concurrency > limitConcurrency { + concurrency = limitConcurrency + log.Warn("limit concurrency to avoid crash", zap.Int("concurrency", concurrency), zap.Any("limitConcurrency", limitConcurrency)) + } inputCh := make([]chan *future, concurrency) for i := 0; i < concurrency; i++ { inputCh[i] = make(chan *future, defaultInputChanSize) diff --git a/pkg/sink/codec/open/open_protocol_encoder_test.go b/pkg/sink/codec/open/open_protocol_encoder_test.go index 5ad82506bd2..e781819134e 100644 --- a/pkg/sink/codec/open/open_protocol_encoder_test.go +++ b/pkg/sink/codec/open/open_protocol_encoder_test.go @@ -263,6 +263,58 @@ func TestEncodeDecodeE2E(t *testing.T) { } } +func TestE2EPartitionTable(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + ctx := context.Background() + codecConfig := common.NewConfig(config.ProtocolOpen) + + builder, err := NewBatchEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() + + decoder, err := NewBatchDecoder(ctx, codecConfig, nil) + require.NoError(t, err) + + helper.Tk().MustExec("use test") + + createPartitionTableDDL := helper.DDL2Event(`create table test.t(a int primary key, b int) partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than MAXVALUE)`) + require.NotNil(t, createPartitionTableDDL) + + insertEvent := helper.DML2Event(`insert into test.t values (1, 1)`, "test", "t", "p0") + require.NotNil(t, insertEvent) + + insertEvent1 := helper.DML2Event(`insert into test.t values (11, 11)`, "test", "t", "p1") + require.NotNil(t, insertEvent1) + + insertEvent2 := helper.DML2Event(`insert into test.t values (21, 21)`, "test", "t", "p2") + require.NotNil(t, insertEvent2) + + events := []*model.RowChangedEvent{insertEvent, insertEvent1, insertEvent2} + + for _, event := range events { + err = encoder.AppendRowChangedEvent(ctx, "", event, nil) + require.NoError(t, err) + message := encoder.Build()[0] + + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + tp, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, model.MessageTypeRow, tp) + + decodedEvent, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + // table id should be set to the partition table id, the PhysicalTableID + require.Equal(t, decodedEvent.GetTableID(), event.GetTableID()) + } +} + func TestE2EDDLCompression(t *testing.T) { helper := entry.NewSchemaTestHelper(t) defer helper.Close() diff --git a/pkg/sink/codec/simple/avro.go b/pkg/sink/codec/simple/avro.go index 8fc0a54e1ee..96f83ac695f 100644 --- a/pkg/sink/codec/simple/avro.go +++ b/pkg/sink/codec/simple/avro.go @@ -252,7 +252,7 @@ func (a *avroMarshaller) newDMLMessageMap( dmlMessagePayload["version"] = defaultVersion dmlMessagePayload["database"] = event.TableInfo.GetSchemaName() dmlMessagePayload["table"] = event.TableInfo.GetTableName() - dmlMessagePayload["tableID"] = event.TableInfo.ID + dmlMessagePayload["tableID"] = event.GetTableID() dmlMessagePayload["commitTs"] = int64(event.CommitTs) dmlMessagePayload["buildTs"] = time.Now().UnixMilli() dmlMessagePayload["schemaVersion"] = int64(event.TableInfo.UpdateTS) diff --git a/pkg/sink/codec/simple/encoder_test.go b/pkg/sink/codec/simple/encoder_test.go index b26dece1aff..9fad6dae03b 100644 --- a/pkg/sink/codec/simple/encoder_test.go +++ b/pkg/sink/codec/simple/encoder_test.go @@ -203,6 +203,78 @@ func TestEncodeDMLEnableChecksum(t *testing.T) { require.Nil(t, decodedRow) } +func TestE2EPartitionTable(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + helper.Tk().MustExec("use test") + + createPartitionTableDDL := helper.DDL2Event(`create table test.t(a int primary key, b int) partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than MAXVALUE)`) + require.NotNil(t, createPartitionTableDDL) + + insertEvent := helper.DML2Event(`insert into test.t values (1, 1)`, "test", "t", "p0") + require.NotNil(t, insertEvent) + + insertEvent1 := helper.DML2Event(`insert into test.t values (11, 11)`, "test", "t", "p1") + require.NotNil(t, insertEvent1) + + insertEvent2 := helper.DML2Event(`insert into test.t values (21, 21)`, "test", "t", "p2") + require.NotNil(t, insertEvent2) + + events := []*model.RowChangedEvent{insertEvent, insertEvent1, insertEvent2} + + ctx := context.Background() + codecConfig := common.NewConfig(config.ProtocolSimple) + + for _, format := range []common.EncodingFormatType{ + common.EncodingFormatAvro, + common.EncodingFormatJSON, + } { + codecConfig.EncodingFormat = format + builder, err := NewBuilder(ctx, codecConfig) + require.NoError(t, err) + enc := builder.Build() + + decoder, err := NewDecoder(ctx, codecConfig, nil) + require.NoError(t, err) + + message, err := enc.EncodeDDLEvent(createPartitionTableDDL) + require.NoError(t, err) + + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + tp, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, model.MessageTypeDDL, tp) + + decodedDDL, err := decoder.NextDDLEvent() + require.NoError(t, err) + require.NotNil(t, decodedDDL) + + for _, event := range events { + err = enc.AppendRowChangedEvent(ctx, "", event, nil) + require.NoError(t, err) + message := enc.Build()[0] + + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + tp, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, model.MessageTypeRow, tp) + + decodedEvent, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + // table id should be set to the partition table id, the PhysicalTableID + require.Equal(t, decodedEvent.GetTableID(), event.GetTableID()) + } + } +} + func TestEncodeDDLSequence(t *testing.T) { helper := entry.NewSchemaTestHelper(t) defer helper.Close() @@ -1145,6 +1217,86 @@ func TestEncoderOtherTypes(t *testing.T) { } } +func TestE2EPartitionTableDMLBeforeDDL(t *testing.T) { + helper := entry.NewSchemaTestHelper(t) + defer helper.Close() + + helper.Tk().MustExec("use test") + + createPartitionTableDDL := helper.DDL2Event(`create table test.t(a int primary key, b int) partition by range (a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than MAXVALUE)`) + require.NotNil(t, createPartitionTableDDL) + + insertEvent := helper.DML2Event(`insert into test.t values (1, 1)`, "test", "t", "p0") + require.NotNil(t, insertEvent) + + insertEvent1 := helper.DML2Event(`insert into test.t values (11, 11)`, "test", "t", "p1") + require.NotNil(t, insertEvent1) + + insertEvent2 := helper.DML2Event(`insert into test.t values (21, 21)`, "test", "t", "p2") + require.NotNil(t, insertEvent2) + + events := []*model.RowChangedEvent{insertEvent, insertEvent1, insertEvent2} + + ctx := context.Background() + codecConfig := common.NewConfig(config.ProtocolSimple) + + for _, format := range []common.EncodingFormatType{ + common.EncodingFormatAvro, + common.EncodingFormatJSON, + } { + codecConfig.EncodingFormat = format + builder, err := NewBuilder(ctx, codecConfig) + require.NoError(t, err) + + enc := builder.Build() + + decoder, err := NewDecoder(ctx, codecConfig, nil) + require.NoError(t, err) + + codecConfig.EncodingFormat = format + for _, event := range events { + err = enc.AppendRowChangedEvent(ctx, "", event, nil) + require.NoError(t, err) + message := enc.Build()[0] + + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + tp, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, model.MessageTypeRow, tp) + + decodedEvent, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + require.Nil(t, decodedEvent) + } + + message, err := enc.EncodeDDLEvent(createPartitionTableDDL) + require.NoError(t, err) + + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) + tp, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, model.MessageTypeDDL, tp) + + decodedDDL, err := decoder.NextDDLEvent() + require.NoError(t, err) + require.NotNil(t, decodedDDL) + + cachedEvents := decoder.GetCachedEvents() + for idx, decodedRow := range cachedEvents { + require.NotNil(t, decodedRow) + require.NotNil(t, decodedRow.TableInfo) + require.Equal(t, decodedRow.GetTableID(), events[idx].GetTableID()) + } + } +} + func TestEncodeDMLBeforeDDL(t *testing.T) { helper := entry.NewSchemaTestHelper(t) defer helper.Close() diff --git a/pkg/sink/codec/simple/message.go b/pkg/sink/codec/simple/message.go index 3a275cba45d..ab915ad1029 100644 --- a/pkg/sink/codec/simple/message.go +++ b/pkg/sink/codec/simple/message.go @@ -441,7 +441,7 @@ func decodeColumns( if rawData == nil { return nil } - var result []*model.ColumnData + result := make([]*model.ColumnData, 0, len(tableInfo.Columns)) for _, info := range tableInfo.Columns { value, ok := rawData[info.Name.O] if !ok { @@ -554,7 +554,7 @@ func (a *jsonMarshaller) newDMLMessage( Version: defaultVersion, Schema: event.TableInfo.GetSchemaName(), Table: event.TableInfo.GetTableName(), - TableID: event.TableInfo.ID, + TableID: event.GetTableID(), CommitTs: event.CommitTs, BuildTs: time.Now().UnixMilli(), SchemaVersion: event.TableInfo.UpdateTS, diff --git a/pkg/sink/kafka/claimcheck/claim_check.go b/pkg/sink/kafka/claimcheck/claim_check.go index 488186f2d81..f4bca23dfe7 100644 --- a/pkg/sink/kafka/claimcheck/claim_check.go +++ b/pkg/sink/kafka/claimcheck/claim_check.go @@ -31,10 +31,6 @@ import ( "go.uber.org/zap" ) -const ( - defaultTimeout = 5 * time.Minute -) - // ClaimCheck manage send message to the claim-check external storage. type ClaimCheck struct { storage storage.ExternalStorage @@ -59,7 +55,7 @@ func New(ctx context.Context, config *config.LargeMessageHandleConfig, changefee zap.String("storageURI", util.MaskSensitiveDataInURI(config.ClaimCheckStorageURI))) start := time.Now() - externalStorage, err := util.GetExternalStorageWithTimeout(ctx, config.ClaimCheckStorageURI, defaultTimeout) + externalStorage, err := util.GetExternalStorageWithDefaultTimeout(ctx, config.ClaimCheckStorageURI) if err != nil { log.Error("create external storage failed", zap.String("namespace", changefeedID.Namespace), diff --git a/pkg/spanz/span.go b/pkg/spanz/span.go index 43b715c1a93..1eed64e1a4c 100644 --- a/pkg/spanz/span.go +++ b/pkg/spanz/span.go @@ -29,8 +29,6 @@ import ( const ( // JobTableID is the id of `tidb_ddl_job`. JobTableID = ddl.JobTableID - // JobHistoryID is the id of `tidb_ddl_history` - JobHistoryID = ddl.HistoryTableID ) // UpperBoundKey represents the maximum value. @@ -64,17 +62,12 @@ func GetTableRange(tableID int64) (startKey, endKey []byte) { // GetAllDDLSpan return all cdc interested spans for DDL. func GetAllDDLSpan() []tablepb.Span { - spans := make([]tablepb.Span, 0, 2) + spans := make([]tablepb.Span, 0, 1) start, end := GetTableRange(JobTableID) spans = append(spans, tablepb.Span{ StartKey: ToComparableKey(start), EndKey: ToComparableKey(end), }) - start, end = GetTableRange(JobHistoryID) - spans = append(spans, tablepb.Span{ - StartKey: ToComparableKey(start), - EndKey: ToComparableKey(end), - }) return spans } diff --git a/pkg/util/external_storage.go b/pkg/util/external_storage.go index 1d7a23c661b..c557cde3172 100644 --- a/pkg/util/external_storage.go +++ b/pkg/util/external_storage.go @@ -35,6 +35,8 @@ import ( "golang.org/x/sync/errgroup" ) +const defaultTimeout = 5 * time.Minute + // GetExternalStorageFromURI creates a new storage.ExternalStorage from a uri. func GetExternalStorageFromURI( ctx context.Context, uri string, @@ -42,18 +44,18 @@ func GetExternalStorageFromURI( return GetExternalStorage(ctx, uri, nil, DefaultS3Retryer()) } -// GetExternalStorageWithTimeout creates a new storage.ExternalStorage from a uri +// GetExternalStorageWithDefaultTimeout creates a new storage.ExternalStorage from a uri // without retry. It is the caller's responsibility to set timeout to the context. -func GetExternalStorageWithTimeout( - ctx context.Context, uri string, timeout time.Duration, -) (storage.ExternalStorage, error) { - ctx, cancel := context.WithTimeout(ctx, timeout) +func GetExternalStorageWithDefaultTimeout(ctx context.Context, uri string) (storage.ExternalStorage, error) { + ctx, cancel := context.WithTimeout(ctx, defaultTimeout) defer cancel() - s, err := GetExternalStorage(ctx, uri, nil, nil) + // total retry time is [1<<7, 1<<8] = [128, 256] + 30*6 = [308, 436] seconds + r := NewS3Retryer(7, 1*time.Second, 2*time.Second) + s, err := GetExternalStorage(ctx, uri, nil, r) return &extStorageWithTimeout{ ExternalStorage: s, - timeout: timeout, + timeout: defaultTimeout, }, err } diff --git a/tests/integration_tests/availability/owner.sh b/tests/integration_tests/availability/owner.sh index b78c0ecd78e..82eb2ad69c2 100755 --- a/tests/integration_tests/availability/owner.sh +++ b/tests/integration_tests/availability/owner.sh @@ -13,6 +13,7 @@ function test_owner_ha() { test_owner_retryable_error test_gap_between_watch_capture test_delete_owner_key + test_resign_owner } # test_kill_owner starts two captures and kill the owner # we expect the live capture will be elected as the new @@ -219,7 +220,7 @@ function test_delete_owner_key() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8301" --logsuffix test_gap_between_watch_capture.server2 ensure $MAX_RETRIES "$CDC_BINARY cli capture list 2>&1 | grep -v \"$owner_id\" | grep id" capture_pid=$(ps -C $CDC_BINARY -o pid= | awk '{print $1}' | grep -v "$owner_pid") - capture_id=$($CDC_BINARY cli capture list 2>&1 | awk -F '"' '/id/{print $4}' | grep -v "$owner_id") + capture_id=$($CDC_BINARY cli capture list 2>&1 | awk -F '"' '/\"id/{print $4}' | grep -v "$owner_id") echo "capture_id:" $capture_id etcdctl del $owner_key @@ -242,3 +243,30 @@ function test_delete_owner_key() { echo "delete_owner_key pass" cleanup_process $CDC_BINARY } + +# test_resign_owner resign the owner by sending +# the resign owner v2 API +# We expect when the owner is resigned, the new owner will be elected +function test_resign_owner() { + echo "run test case test_resign_owner" + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --logsuffix test_resign_owner.server1 + # ensure the server become the owner + ensure $MAX_RETRIES "$CDC_BINARY cli capture list 2>&1 | grep '\"is-owner\": true'" + owner_pid=$(ps -C $CDC_BINARY -o pid= | awk '{print $1}') + owner_id=$($CDC_BINARY cli capture list 2>&1 | awk -F '"' '/\"id/{print $4}') + echo "owner pid:" $owner_pid + echo "owner id" $owner_id + + # run another server + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8301" --logsuffix test_resign_owner.server2 + ensure $MAX_RETRIES "$CDC_BINARY cli capture list 2>&1 | grep -v \"$owner_id\" | grep id" + capture_id=$($CDC_BINARY cli capture list 2>&1 | awk -F '"' '/\"id/{print $4}' | grep -v "$owner_id") + echo "capture_id:" $capture_id + + # resign the owner + curl -X POST http://127.0.0.1:8301/api/v2/owner/resign + # check that the new owner is elected + ensure $MAX_RETRIES "$CDC_BINARY cli capture list --server 'http://127.0.0.1:8301' 2>&1 |grep $capture_id -A1 | grep '\"is-owner\": true'" + echo "test_resign_owner: pass" + cleanup_process $CDC_BINARY +} diff --git a/tests/integration_tests/bank/case.go b/tests/integration_tests/bank/case.go index 6b00903dfd5..b13e3915102 100644 --- a/tests/integration_tests/bank/case.go +++ b/tests/integration_tests/bank/case.go @@ -14,6 +14,7 @@ package main import ( + "bufio" "context" "database/sql" "encoding/json" @@ -21,6 +22,10 @@ import ( "io" "math/rand" "net/http" + "os" + "path/filepath" + "regexp" + "strconv" "strings" "sync/atomic" "time" @@ -623,7 +628,8 @@ func getDownStreamSyncedEndTs(ctx context.Context, db *sql.DB, tidbAPIEndpoint, log.Error("get downstream sync end ts failed due to timeout", zap.String("table", tableName), zap.Error(ctx.Err())) return 0, ctx.Err() case <-time.After(2 * time.Second): - result, ok := tryGetEndTs(db, tidbAPIEndpoint, tableName) + // result, ok := tryGetEndTs(db, tidbAPIEndpoint, tableName) + result, ok := tryGetEndTsFromLog(db, tableName) if ok { return result, nil } @@ -675,3 +681,73 @@ func tryGetEndTs(db *sql.DB, tidbAPIEndpoint, tableName string) (result uint64, zap.Uint64("ts", ddlJob[0].Binlog.FinishedTS)) return ddlJob[0].Binlog.FinishedTS, true } + +func tryGetEndTsFromLog(db *sql.DB, tableName string) (result uint64, ok bool) { + query := "SELECT JOB_ID FROM information_schema.ddl_jobs WHERE table_name = ?" + log.Info("try get end ts", zap.String("query", query), zap.String("tableName", tableName)) + var jobID uint64 + row := db.QueryRow(query, tableName) + if err := row.Scan(&jobID); err != nil { + if err != sql.ErrNoRows { + log.Info("rows scan failed", zap.Error(err)) + } + return 0, false + } + + log.Info("try parse finishedTs from ticdc log", zap.String("tableName", tableName)) + + logFilePath := "/tmp/tidb_cdc_test/bank" + cdcLogFiles := make([]string, 0) + // walk all file with cdc prefix + err := filepath.WalkDir(logFilePath, func(path string, d os.DirEntry, err error) error { + if err != nil { + return err + } + if !d.IsDir() { + if strings.Contains(d.Name(), "down") && strings.Contains(d.Name(), "cdc") && strings.Contains(d.Name(), "log") { + cdcLogFiles = append(cdcLogFiles, path) + fmt.Println(path) + } + } + return nil + }) + if err != nil { + log.Error("Failed to walk dir: %v", zap.Error(err)) + } + log.Info("total files", zap.Any("file", cdcLogFiles)) + + logRegex := regexp.MustCompile(`handle a ddl job`) + tableNameRegex := regexp.MustCompile(tableName + "`") + timeStampRegex := regexp.MustCompile(`finishedTs=([0-9]+)`) + for _, f := range cdcLogFiles { + file, err := os.Open(f) + if err != nil { + log.Error("Failed to open file: %v", zap.Error(err)) + } + defer file.Close() + + scanner := bufio.NewScanner(file) + for scanner.Scan() { + line := scanner.Text() + if !logRegex.MatchString(line) || !tableNameRegex.MatchString(line) { + continue + } + + matches := timeStampRegex.FindStringSubmatch(line) + if len(matches) > 1 { + fmt.Println("found first match line: ", matches[1], ": ", line) + // convert to uint64 + result, err := strconv.ParseUint(matches[1], 10, 64) + if err != nil { + log.Error("Failed to parse uint64: %v", zap.Error(err)) + } + return result, true + } + } + + if err := scanner.Err(); err != nil { + log.Error("Error scanning file: %v", zap.Error(err)) + } + } + return 0, false +} diff --git a/tests/integration_tests/bank/run.sh b/tests/integration_tests/bank/run.sh index e0c1a5cd6de..36cee26803f 100644 --- a/tests/integration_tests/bank/run.sh +++ b/tests/integration_tests/bank/run.sh @@ -20,8 +20,10 @@ function prepare() { run_sql "CREATE DATABASE bank" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - run_cdc_cli changefeed create --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT}/" + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8400" --pd "http://${DOWN_PD_HOST}:${DOWN_PD_PORT}" --logsuffix "down" + run_cdc_cli changefeed create --sink-uri="blackhole://" -c "changefeed-for-find-finished-ts" --server "http://127.0.0.1:8400" } trap stop_tidb_cluster EXIT diff --git a/tests/integration_tests/batch_add_table/run.sh b/tests/integration_tests/batch_add_table/run.sh index 0989096627b..467f6f253b4 100644 --- a/tests/integration_tests/batch_add_table/run.sh +++ b/tests/integration_tests/batch_add_table/run.sh @@ -38,7 +38,7 @@ function run_with_fast_create_table() { ## to generate batch create ddl. In changefeed.toml, we filter test.t_1 for ((i = 1; i <= 100; i++)); do - mysql -h ${UP_TIDB_HOST} -P ${UP_TIDB_PORT} -u root -D "test" -e "create table t_$i (a int primary key , b int)" & + mysql -h ${UP_TIDB_HOST} -P ${UP_TIDB_PORT} -u root -D "test" -e 'create table t_'$i' (a int primary key , `test_;semicolon;` int)' & done for ((i = 1; i <= 100; i++)); do diff --git a/tests/integration_tests/ddl_attributes/run.sh b/tests/integration_tests/ddl_attributes/run.sh index d9570e43a90..1efde254008 100644 --- a/tests/integration_tests/ddl_attributes/run.sh +++ b/tests/integration_tests/ddl_attributes/run.sh @@ -56,7 +56,7 @@ function run() { check_contains "CREATE TABLE \`placement_t2\` " TTL_MARK='![ttl]' - CREATE_TTL_SQL_CONTAINS1="/*T${TTL_MARK} TTL=\`t\` + INTERVAL 1 DAY */ /*T${TTL_MARK} TTL_ENABLE='OFF' */ /*T${TTL_MARK} TTL_JOB_INTERVAL='1h' */" + CREATE_TTL_SQL_CONTAINS1="/*T${TTL_MARK} TTL=\`t\` + INTERVAL 1 DAY */ /*T${TTL_MARK} TTL_ENABLE='OFF' */ /*T${TTL_MARK} TTL_JOB_INTERVAL='24h' */" CREATE_TTL_SQL_CONTAINS2="/*T${TTL_MARK} TTL=\`t\` + INTERVAL 1 DAY */ /*T${TTL_MARK} TTL_ENABLE='OFF' */ /*T${TTL_MARK} TTL_JOB_INTERVAL='7h' */" run_sql "show create table ddl_attributes.ttl_t1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && diff --git a/tests/integration_tests/region_merge/run.sh b/tests/integration_tests/region_merge/run.sh index ed297be9aca..d6d2353ae24 100644 --- a/tests/integration_tests/region_merge/run.sh +++ b/tests/integration_tests/region_merge/run.sh @@ -12,12 +12,16 @@ function split_and_random_merge() { pd_addr=$1 scale=$2 echo "split_and_random_merge scale: $scale" - run_sql "SPLIT TABLE region_merge.t1 BETWEEN (-9223372036854775808) AND (9223372036854775807) REGIONS $scale;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} || true + run_sql "ALTER TABLE region_merge.t1 ATTRIBUTES 'merge_option=deny';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "SELECT count(distinct region_id) from information_schema.tikv_region_status where db_name = 'region_merge' and table_name = 't1';" && + cat $OUT_DIR/sql_res.region_merge.txt + run_sql "SPLIT TABLE region_merge.t1 BETWEEN (-9223372036854775808) AND (9223372036854775807) REGIONS $scale;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "SELECT count(distinct region_id) from information_schema.tikv_region_status where db_name = 'region_merge' and table_name = 't1';" && cat $OUT_DIR/sql_res.region_merge.txt run_sql "insert into region_merge.t1 values (-9223372036854775808),(0),(1),(9223372036854775807);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "delete from region_merge.t1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - # sleep 5s to wait some region merge + run_sql "delete from region_merge.t1 where id=-9223372036854775808 or id=0 or id=1 or id=9223372036854775807;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "ALTER TABLE region_merge.t1 ATTRIBUTES 'merge_option=allow';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + # sleep 5s to wait some regions merge sleep 5 } diff --git a/tools/check/go.mod b/tools/check/go.mod index 9f869541869..ec6abb43de9 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -85,7 +85,7 @@ require ( github.com/gobwas/glob v0.2.3 // indirect github.com/gofrs/flock v0.12.1 // indirect github.com/golang/glog v1.0.0 // indirect - github.com/golang/protobuf v1.5.3 // indirect + github.com/golang/protobuf v1.5.4 // indirect github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2 // indirect github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a // indirect github.com/golangci/go-misc v0.0.0-20220329215616-d24fe342adfe // indirect diff --git a/tools/check/go.sum b/tools/check/go.sum index b710465dfc0..f797a06f316 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -377,6 +377,8 @@ 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/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2 h1:23T5iq8rbUYlhpt5DB4XJkc6BU31uODLD1o1gKvZmD0= github.com/golangci/check v0.0.0-20180506172741-cfe4005ccda2/go.mod h1:k9Qvh+8juN+UKMCS/3jFtGICgW8O96FVaZsaxdzDkR4= github.com/golangci/dupl v0.0.0-20180902072040-3e9179ac440a h1:w8hkcTqaFpzKqonE9uMCefW1WDie15eSP/4MssdenaM= @@ -1445,7 +1447,12 @@ google.golang.org/genproto v0.0.0-20210108203827-ffc7fda8c3d7/go.mod h1:FWY/as6D google.golang.org/genproto v0.0.0-20210226172003-ab064af71705/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f h1:P8EiVSxZwC6xH2niv2N66aqwMtYFg+D54gbjpcqKJtM= google.golang.org/genproto v0.0.0-20220719170305-83ca9fad585f/go.mod h1:GkXuJDJ6aQ7lnJcRF+SJVgFdQhypqgl3LB1C9vabdRE= +google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 h1:wpZ8pe2x1Q3f2KyT5f8oP/fa9rHAKgFPr/HZdNuS+PQ= google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17/go.mod h1:J7XzRzVy1+IPwWHZUzoD0IccYZIrXILAQpc+Qy9CMhY= +google.golang.org/genproto/googleapis/api v0.0.0-20240725223205-93522f1f2a9f h1:b1Ln/PG8orm0SsBbHZWke8dDp2lrCD4jSmfglFpTZbk= +google.golang.org/genproto/googleapis/api v0.0.0-20240725223205-93522f1f2a9f/go.mod h1:AHT0dDg3SoMOgZGnZk29b5xTbPHMoEC8qthmBLJCpys= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 h1:pPJltXNxVzT4pK9yD8vR9X75DaWYYmLGMsEvBfFQZzQ= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= 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.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=