diff --git a/.github/workflows/dataflow_engine_chaos.yaml b/.github/workflows/dataflow_engine_chaos.yaml index e2140226ee5..3db336064d6 100644 --- a/.github/workflows/dataflow_engine_chaos.yaml +++ b/.github/workflows/dataflow_engine_chaos.yaml @@ -333,21 +333,9 @@ jobs: # Upload logs as artifact seems not stable, so we set `continue-on-error: true` here. - name: Upload logs continue-on-error: true - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v4 if: ${{ always() }} with: 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 fa0aaf93899..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() }} @@ -105,19 +125,21 @@ jobs: # Update logs as artifact seems not stable, so we set `continue-on-error: true` here. - name: Upload logs continue-on-error: true - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v4 if: ${{ always() }} with: name: test-binlog-999999 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_chaos.yaml b/.github/workflows/dm_chaos.yaml index 1a121a6a75b..5652604cb75 100644 --- a/.github/workflows/dm_chaos.yaml +++ b/.github/workflows/dm_chaos.yaml @@ -67,8 +67,19 @@ jobs: path: tools/bin key: ${{ runner.os }}-ticdc-tools-${{ hashFiles('tools/check/go.sum') }} - - name: Create k8s Kind Cluster - uses: helm/kind-action@v1.4.0 + - name: install k3s + run: | + curl -fsSL https://get.k3s.io | sh -s - --write-kubeconfig-mode 644 \ + "${k3s_disable_command:---disable}" metrics-server \ + "${k3s_disable_command:---disable}" traefik \ + --flannel-backend=none \ + --docker + shell: bash + + - name: Export KUBECONFIG environment variable + run: | + echo 'KUBECONFIG=/etc/rancher/k3s/k3s.yaml' >> $GITHUB_ENV + shell: bash - name: Print cluster information run: | @@ -96,12 +107,7 @@ jobs: cp -r $GITHUB_WORKSPACE/dm/chaos/cases/conf/ $GITHUB_WORKSPACE/bin/ docker build -f $GITHUB_WORKSPACE/dm/chaos/manifests/Dockerfile -t dm:chaos $GITHUB_WORKSPACE/bin docker image list - - # Load DM docker image into KIND, see https://kind.sigs.k8s.io/docs/user/quick-start/#loading-an-image-into-your-cluster - - name: Load DM docker image into KIND - run: | - kind load docker-image dm:chaos --name chart-testing - + # Set up upstream instances - name: Set up sources run: | @@ -252,18 +258,21 @@ jobs: - name: Wait for chaos test case complete run: | $GITHUB_WORKSPACE/dm/chaos/scripts/check-case.sh + + - name: Setup tmate session + if: ${{ failure() }} + uses: mxschmitt/action-tmate@v3 - name: Copy logs to hack permission if: ${{ always() }} run: | mkdir ./logs - kubectl get pods --no-headers -o custom-columns=":metadata.name"|grep -E "dm-"|xargs -I{} sudo kubectl cp {}:/log/{}.log ./logs/{}.log || true - kind export logs ./logs/kind --name chart-testing + kubectl get pods --no-headers -o custom-columns=":metadata.name"|grep -E "dm-"|xargs -I{} kubectl cp {}:/log/{}.log ./logs/{}.log || true sudo chown -R runner ./logs # Update logs as artifact seems not stable, so we set `continue-on-error: true` here. - name: Upload logs continue-on-error: true - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v4 if: ${{ always() }} with: name: chaos-base-logs.${{ matrix.chaos-obj }} diff --git a/.github/workflows/dm_mariadb_master_down_and_up.yaml b/.github/workflows/dm_mariadb_master_down_and_up.yaml index 74ccbc2191c..a2b8e91aabc 100644 --- a/.github/workflows/dm_mariadb_master_down_and_up.yaml +++ b/.github/workflows/dm_mariadb_master_down_and_up.yaml @@ -57,7 +57,7 @@ jobs: # Update logs as artifact seems not stable, so we set `continue-on-error: true` here. - name: Upload logs continue-on-error: true - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v4 if: ${{ always() }} with: name: upstream-switch-logs diff --git a/.github/workflows/dm_upstream_switch.yaml b/.github/workflows/dm_upstream_switch.yaml index 21d367ee014..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 @@ -66,19 +68,22 @@ jobs: # Update logs as artifact seems not stable, so we set `continue-on-error: true` here. - name: Upload logs continue-on-error: true - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v4 if: ${{ always() }} with: name: upstream-switch-logs 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/.github/workflows/upgrade_dm_via_tiup.yaml b/.github/workflows/upgrade_dm_via_tiup.yaml index 385662b9962..bc820c9c263 100644 --- a/.github/workflows/upgrade_dm_via_tiup.yaml +++ b/.github/workflows/upgrade_dm_via_tiup.yaml @@ -167,7 +167,7 @@ jobs: # Update logs as artifact seems not stable, so we set `continue-on-error: true` here. - name: Upload logs continue-on-error: true - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v4 if: ${{ failure() }} with: name: upgrade-via-tiup-${{ matrix.previous_v2 }} 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 b351291ca5f..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 @@ -196,14 +182,20 @@ func (m *mounter) unmarshalAndMountRowChanged(ctx context.Context, raw *model.Ra return nil, cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(physicalTableID) } if bytes.HasPrefix(key, recordPrefix) { - rowKV, err := m.unmarshalRowKVEntry(tableInfo, raw.Key, raw.Value, raw.OldValue, baseInfo) + recordID, err := tablecodec.DecodeRowKey(raw.Key) + if err != nil { + return nil, errors.Trace(err) + } + baseInfo.RecordID = recordID + + rowKV, err := m.unmarshalRowKVEntry(tableInfo, raw.Value, raw.OldValue, baseInfo) if err != nil { return nil, errors.Trace(err) } if rowKV == nil { return nil, nil } - row, rawRow, err := m.mountRowKVEntry(tableInfo, rowKV, checksumKey, raw.ApproximateDataSize()) + row, rawRow, err := m.mountRowKVEntry(tableInfo, rowKV, recordID, checksumKey, raw.ApproximateDataSize()) if err != nil { return nil, err } @@ -231,28 +223,21 @@ func (m *mounter) unmarshalAndMountRowChanged(ctx context.Context, raw *model.Ra func (m *mounter) unmarshalRowKVEntry( tableInfo *model.TableInfo, - rawKey []byte, rawValue []byte, rawOldValue []byte, base baseKVEntry, ) (*rowKVEntry, error) { - recordID, err := tablecodec.DecodeRowKey(rawKey) - if err != nil { - return nil, errors.Trace(err) - } - base.RecordID = recordID - var ( row, preRow map[int64]types.Datum rowExist, preRowExist bool ) - row, rowExist, err = m.decodeRow(rawValue, recordID, tableInfo, false) + row, rowExist, err := m.decodeRow(rawValue, base.RecordID, tableInfo, false) if err != nil { return nil, errors.Trace(err) } - preRow, preRowExist, err = m.decodeRow(rawOldValue, recordID, tableInfo, true) + preRow, preRowExist, err = m.decodeRow(rawOldValue, base.RecordID, tableInfo, true) if err != nil { return nil, errors.Trace(err) } @@ -307,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 @@ -490,33 +425,34 @@ func (m *mounter) verifyColumnChecksum( checksum, err := calculateColumnChecksum(columnInfos, rawColumns, m.tz) if err != nil { - log.Error("failed to calculate the checksum", zap.Uint32("first", first), zap.Error(err)) + log.Error("failed to calculate the checksum", + zap.Uint32("first", first), zap.Any("columnInfos", columnInfos), + zap.Any("rawColumns", rawColumns), zap.Error(err)) return 0, false, err } // the first checksum matched, it hits in the most case. if checksum == first { - log.Debug("checksum matched", zap.Uint32("checksum", checksum), zap.Uint32("first", first)) return checksum, true, nil } extra, ok := decoder.GetExtraChecksum() if ok && checksum == extra { - log.Debug("extra checksum matched, this may happen the upstream TiDB is during the DDL execution phase", - zap.Uint32("checksum", checksum), zap.Uint32("extra", extra)) return checksum, true, nil } if !skipFail { log.Error("cannot found the extra checksum, the first checksum mismatched", - zap.Uint32("checksum", checksum), zap.Uint32("first", first), zap.Uint32("extra", extra)) + zap.Uint32("checksum", checksum), zap.Uint32("first", first), zap.Uint32("extra", extra), + zap.Any("columnInfos", columnInfos), zap.Any("rawColumns", rawColumns), zap.Any("tz", m.tz)) return checksum, false, nil } if time.Since(m.lastSkipOldValueTime) > time.Minute { log.Warn("checksum mismatch on the old value, "+ "this may caused by Add Column / Drop Column executed, skip verification", - zap.Uint32("checksum", checksum), zap.Uint32("first", first), zap.Uint32("extra", extra)) + zap.Uint32("checksum", checksum), zap.Uint32("first", first), zap.Uint32("extra", extra), + zap.Any("columnInfos", columnInfos), zap.Any("rawColumns", rawColumns), zap.Any("tz", m.tz)) m.lastSkipOldValueTime = time.Now() } return checksum, true, nil @@ -602,7 +538,7 @@ func newDatum(value interface{}, ft types.FieldType) (types.Datum, error) { func verifyRawBytesChecksum( tableInfo *model.TableInfo, columns []*model.ColumnData, decoder *rowcodec.DatumMapDecoder, - key kv.Key, tz *time.Location, + handle kv.Handle, key kv.Key, tz *time.Location, ) (uint32, bool, error) { expected, ok := decoder.GetChecksum() if !ok { @@ -621,12 +557,14 @@ func verifyRawBytesChecksum( columnInfo := tableInfo.ForceGetColumnInfo(columnID) datum, err := newDatum(col.Value, columnInfo.FieldType) if err != nil { + log.Error("build datum for raw checksum calculation failed", + zap.Any("col", col), zap.Any("columnInfo", columnInfo), zap.Error(err)) return 0, false, errors.Trace(err) } datums = append(datums, &datum) columnIDs = append(columnIDs, columnID) } - obtained, err := decoder.CalculateRawChecksum(tz, columnIDs, datums, key, nil) + obtained, err := decoder.CalculateRawChecksum(tz, columnIDs, datums, key, handle, nil) if err != nil { return 0, false, errors.Trace(err) } @@ -635,7 +573,10 @@ func verifyRawBytesChecksum( } log.Error("raw bytes checksum mismatch", - zap.Uint32("expected", expected), zap.Uint32("obtained", obtained)) + zap.Int("version", decoder.ChecksumVersion()), + zap.Uint32("expected", expected), zap.Uint32("obtained", obtained), + zap.Any("tableInfo", tableInfo), zap.Any("columns", columns), + zap.Any("handle", handle.String()), zap.Any("tz", tz)) return expected, false, nil } @@ -645,7 +586,7 @@ func verifyRawBytesChecksum( func (m *mounter) verifyChecksum( tableInfo *model.TableInfo, columnInfos []*timodel.ColumnInfo, columns []*model.ColumnData, rawColumns []types.Datum, - key kv.Key, isPreRow bool, + handle kv.Handle, key kv.Key, isPreRow bool, ) (uint32, bool, error) { if !m.integrity.Enabled() { return 0, true, nil @@ -665,9 +606,12 @@ func (m *mounter) verifyChecksum( // Update / Delete event correctly, after Add Column / Drop column DDL, // since the table schema does not contain complete column information. return m.verifyColumnChecksum(columnInfos, rawColumns, decoder, isPreRow) - case 1: - expected, matched, err := verifyRawBytesChecksum(tableInfo, columns, decoder, key, m.tz) + case 1, 2: + expected, matched, err := verifyRawBytesChecksum(tableInfo, columns, decoder, handle, key, m.tz) if err != nil { + log.Error("calculate raw checksum failed", + zap.Int("version", version), zap.Any("tz", m.tz), zap.Any("handle", handle.String()), + zap.Any("key", key), zap.Any("columns", columns), zap.Error(err)) return 0, false, errors.Trace(err) } if !matched { @@ -675,7 +619,9 @@ func (m *mounter) verifyChecksum( } columnChecksum, err := calculateColumnChecksum(columnInfos, rawColumns, m.tz) if err != nil { - log.Error("failed to calculate column-level checksum, after raw checksum verification passed", zap.Error(err)) + log.Error("failed to calculate column-level checksum, after raw checksum verification passed", + zap.Any("columnsInfo", columnInfos), zap.Any("rawColumns", rawColumns), + zap.Any("tz", m.tz), zap.Error(err)) return 0, false, errors.Trace(err) } return columnChecksum, true, nil @@ -685,7 +631,7 @@ func (m *mounter) verifyChecksum( } func (m *mounter) mountRowKVEntry( - tableInfo *model.TableInfo, row *rowKVEntry, key kv.Key, dataSize int64, + tableInfo *model.TableInfo, row *rowKVEntry, handle kv.Handle, key kv.Key, dataSize int64, ) (*model.RowChangedEvent, model.RowChangedDatums, error) { var ( rawRow model.RowChangedDatums @@ -719,19 +665,15 @@ func (m *mounter) mountRowKVEntry( return nil, rawRow, errors.Trace(err) } - preChecksum, matched, err = m.verifyChecksum(tableInfo, columnInfos, preCols, preRawCols, key, true) + preChecksum, matched, err = m.verifyChecksum(tableInfo, columnInfos, preCols, preRawCols, handle, key, true) if err != nil { - log.Error("calculate the previous columns checksum failed", - zap.Any("tableInfo", tableInfo), - zap.Any("rawCols", preRawCols)) return nil, rawRow, errors.Trace(err) } if !matched { log.Error("previous columns checksum mismatch", - zap.Uint32("checksum", preChecksum), - zap.Any("tableInfo", tableInfo), - zap.Any("rawCols", preRawCols)) + zap.Uint32("checksum", preChecksum), zap.Any("tableInfo", tableInfo), + zap.Any("preCols", preCols), zap.Any("rawCols", preRawCols)) if m.integrity.ErrorHandle() { return nil, rawRow, cerror.ErrCorruptedDataMutation. GenWithStackByArgs(m.changefeedID.Namespace, m.changefeedID.ID) @@ -751,18 +693,14 @@ func (m *mounter) mountRowKVEntry( return nil, rawRow, errors.Trace(err) } - currentChecksum, matched, err = m.verifyChecksum(tableInfo, columnInfos, cols, rawCols, key, false) + currentChecksum, matched, err = m.verifyChecksum(tableInfo, columnInfos, cols, rawCols, handle, key, false) if err != nil { - log.Error("calculate the current columns checksum failed", - zap.Any("tableInfo", tableInfo), - zap.Any("rawCols", rawCols)) return nil, rawRow, errors.Trace(err) } if !matched { log.Error("current columns checksum mismatch", - zap.Uint32("checksum", currentChecksum), - zap.Any("tableInfo", tableInfo), - zap.Any("rawCols", rawCols)) + zap.Uint32("checksum", currentChecksum), zap.Any("tableInfo", tableInfo), + zap.Any("cols", cols), zap.Any("rawCols", rawCols)) if m.integrity.ErrorHandle() { return nil, rawRow, cerror.ErrCorruptedDataMutation. GenWithStackByArgs(m.changefeedID.Namespace, m.changefeedID.ID) 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/kv/metrics.go b/cdc/kv/metrics.go index f2f2c022553..74267b5a186 100644 --- a/cdc/kv/metrics.go +++ b/cdc/kv/metrics.go @@ -149,7 +149,7 @@ var ( Namespace: "ticdc", Subsystem: "kvclient", Name: "region_worker_busy_ratio", - Help: "Busy ratio (X ms in 1s) for region worker.", + Help: "Busy ratio for region worker.", }, []string{"namespace", "changefeed", "table", "store", "type"}) workerChannelSize = prometheus.NewGaugeVec( prometheus.GaugeOpts{ diff --git a/cdc/kv/shared_client.go b/cdc/kv/shared_client.go index 789fbfa6f30..3f116c47f01 100644 --- a/cdc/kv/shared_client.go +++ b/cdc/kv/shared_client.go @@ -85,6 +85,7 @@ var ( metricFeedDuplicateRequestCounter = eventFeedErrorCounter.WithLabelValues("DuplicateRequest") metricFeedUnknownErrorCounter = eventFeedErrorCounter.WithLabelValues("Unknown") metricFeedRPCCtxUnavailable = eventFeedErrorCounter.WithLabelValues("RPCCtxUnavailable") + metricGetStoreErr = eventFeedErrorCounter.WithLabelValues("GetStoreErr") metricStoreSendRequestErr = eventFeedErrorCounter.WithLabelValues("SendRequestToStore") metricKvIsBusyCounter = eventFeedErrorCounter.WithLabelValues("KvIsBusy") metricKvCongestedCounter = eventFeedErrorCounter.WithLabelValues("KvCongested") @@ -108,6 +109,10 @@ func (e *rpcCtxUnavailableErr) Error() string { e.verID.GetID(), e.verID.GetVer(), e.verID.GetConfVer()) } +type getStoreErr struct{} + +func (e *getStoreErr) Error() string { return "get store error" } + type sendRequestToStoreErr struct{} func (e *sendRequestToStoreErr) Error() string { return "send request to store error" } @@ -739,6 +744,13 @@ func (s *SharedClient) doHandleError(ctx context.Context, errInfo regionErrorInf metricFeedRPCCtxUnavailable.Inc() s.scheduleRangeRequest(ctx, errInfo.span, errInfo.subscribedTable) return nil + case *getStoreErr: + metricGetStoreErr.Inc() + bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff) + // cannot get the store the region belongs to, so we need to reload the region. + s.regionCache.OnSendFail(bo, errInfo.rpcCtx, true, err) + s.scheduleRangeRequest(ctx, errInfo.span, errInfo.subscribedTable) + return nil case *sendRequestToStoreErr: metricStoreSendRequestErr.Inc() bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff) diff --git a/cdc/kv/shared_client_test.go b/cdc/kv/shared_client_test.go index 765a38f0726..b6db06e5c82 100644 --- a/cdc/kv/shared_client_test.go +++ b/cdc/kv/shared_client_test.go @@ -20,6 +20,7 @@ import ( "testing" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/cdcpb" "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/store/mockstore/mockcopr" @@ -261,6 +262,105 @@ func TestConnectToOfflineOrFailedTiKV(t *testing.T) { } } +func TestGetStoreFailed(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + wg := &sync.WaitGroup{} + + events1 := make(chan *cdcpb.ChangeDataEvent, 10) + srv1 := newMockChangeDataServer(events1) + server1, addr1 := newMockService(ctx, t, srv1, wg) + + rpcClient, cluster, pdClient, _ := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler()) + + pdClient = &mockPDClient{Client: pdClient, versionGen: defaultVersionGen} + + grpcPool := sharedconn.NewConnAndClientPool(&security.Credential{}, nil) + + regionCache := tikv.NewRegionCache(pdClient) + + pdClock := pdutil.NewClock4Test() + + kvStorage, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) + require.Nil(t, err) + lockResolver := txnutil.NewLockerResolver(kvStorage, model.ChangeFeedID{}) + + invalidStore1 := "localhost:1" + invalidStore2 := "localhost:2" + cluster.AddStore(1, addr1) + cluster.AddStore(2, invalidStore1) + cluster.AddStore(3, invalidStore2) + cluster.Bootstrap(11, []uint64{1, 2, 3}, []uint64{4, 5, 6}, 4) + + client := NewSharedClient( + model.ChangeFeedID{ID: "test"}, + &config.ServerConfig{ + KVClient: &config.KVClientConfig{ + WorkerConcurrent: 1, + GrpcStreamConcurrent: 1, + AdvanceIntervalInMs: 10, + }, + Debug: &config.DebugConfig{Puller: &config.PullerConfig{LogRegionDetails: false}}, + }, + false, pdClient, grpcPool, regionCache, pdClock, lockResolver, + ) + + defer func() { + cancel() + client.Close() + _ = kvStorage.Close() + regionCache.Close() + pdClient.Close() + srv1.wg.Wait() + server1.Stop() + wg.Wait() + }() + + wg.Add(1) + go func() { + defer wg.Done() + err := client.Run(ctx) + require.Equal(t, context.Canceled, errors.Cause(err)) + }() + + failpoint.Enable("github.com/pingcap/tiflow/pkg/version/GetStoreFailed", `return(true)`) + subID := client.AllocSubscriptionID() + span := tablepb.Span{TableID: 1, StartKey: []byte("a"), EndKey: []byte("b")} + eventCh := make(chan MultiplexingEvent, 50) + client.Subscribe(subID, span, 1, eventCh) + + makeTsEvent := func(regionID, ts, requestID uint64) *cdcpb.ChangeDataEvent { + return &cdcpb.ChangeDataEvent{ + Events: []*cdcpb.Event{ + { + RegionId: regionID, + RequestId: requestID, + Event: &cdcpb.Event_ResolvedTs{ResolvedTs: ts}, + }, + }, + } + } + + checkTsEvent := func(event model.RegionFeedEvent, ts uint64) { + require.Equal(t, ts, event.Resolved.ResolvedTs) + } + + events1 <- mockInitializedEvent(11, uint64(subID)) + ts := oracle.GoTimeToTS(pdClock.CurrentTime()) + events1 <- makeTsEvent(11, ts, uint64(subID)) + select { + case <-eventCh: + require.True(t, false, "should not get event when get store failed") + case <-time.After(5 * time.Second): + } + failpoint.Disable("github.com/pingcap/tiflow/pkg/version/GetStoreFailed") + select { + case event := <-eventCh: + checkTsEvent(event.RegionFeedEvent, ts) + case <-time.After(5 * time.Second): + require.True(t, false, "reconnection not succeed in 5 second") + } +} + type mockChangeDataServer struct { ch chan *cdcpb.ChangeDataEvent wg sync.WaitGroup diff --git a/cdc/kv/shared_stream.go b/cdc/kv/shared_stream.go index 7a9b1204f89..419cf4b0d12 100644 --- a/cdc/kv/shared_stream.go +++ b/cdc/kv/shared_stream.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/kv/sharedconn" "github.com/pingcap/tiflow/pkg/chann" + cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/util" "github.com/pingcap/tiflow/pkg/version" "go.uber.org/zap" @@ -90,12 +91,31 @@ func newStream(ctx context.Context, c *SharedClient, g *errgroup.Group, r *reque if err := waitForPreFetching(); err != nil { return err } - if canceled := stream.run(ctx, c, r); canceled { - return nil + var regionErr error + if err := version.CheckStoreVersion(ctx, c.pd, r.storeID); err != nil { + log.Info("event feed check store version fails", + zap.String("namespace", c.changefeed.Namespace), + zap.String("changefeed", c.changefeed.ID), + zap.Uint64("streamID", stream.streamID), + zap.Uint64("storeID", r.storeID), + zap.String("addr", r.storeAddr), + zap.Error(err)) + if errors.Cause(err) == context.Canceled { + return nil + } else if cerrors.Is(err, cerrors.ErrGetAllStoresFailed) { + regionErr = &getStoreErr{} + } else { + regionErr = &sendRequestToStoreErr{} + } + } else { + if canceled := stream.run(ctx, c, r); canceled { + return nil + } + regionErr = &sendRequestToStoreErr{} } for _, m := range stream.clearStates() { for _, state := range m { - state.markStopped(&sendRequestToStoreErr{}) + state.markStopped(regionErr) sfEvent := newEventItem(nil, state, stream) slot := hashRegionID(state.region.verID.GetID(), len(c.workers)) _ = c.workers[slot].sendEvent(ctx, sfEvent) @@ -108,7 +128,7 @@ func newStream(ctx context.Context, c *SharedClient, g *errgroup.Group, r *reque // It means it's a special task for stopping the table. continue } - c.onRegionFail(newRegionErrorInfo(region, &sendRequestToStoreErr{})) + c.onRegionFail(newRegionErrorInfo(region, regionErr)) } if err := util.Hang(ctx, time.Second); err != nil { return err @@ -135,17 +155,6 @@ func (s *requestedStream) run(ctx context.Context, c *SharedClient, rs *requeste } } - if err := version.CheckStoreVersion(ctx, c.pd, rs.storeID); err != nil { - log.Info("event feed check store version fails", - zap.String("namespace", c.changefeed.Namespace), - zap.String("changefeed", c.changefeed.ID), - zap.Uint64("streamID", s.streamID), - zap.Uint64("storeID", rs.storeID), - zap.String("addr", rs.storeAddr), - zap.Error(err)) - return isCanceled() - } - log.Info("event feed going to create grpc stream", zap.String("namespace", c.changefeed.Namespace), zap.String("changefeed", c.changefeed.ID), @@ -310,7 +319,9 @@ func (s *requestedStream) send(ctx context.Context, c *SharedClient, rs *request if s.multiplexing != nil { req := &cdcpb.ChangeDataRequest{ RequestId: uint64(subscriptionID), - Request: &cdcpb.ChangeDataRequest_Deregister_{}, + Request: &cdcpb.ChangeDataRequest_Deregister_{ + Deregister: &cdcpb.ChangeDataRequest_Deregister{}, + }, } if err = s.multiplexing.Client().Send(req); err != nil { log.Warn("event feed send deregister request to grpc stream failed", diff --git a/cdc/model/sink.go b/cdc/model/sink.go index dd6396dace1..cc33d2b1008 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -556,8 +556,8 @@ func (r *RowChangedEvent) GetHandleKeyColumnValues() []string { } // HandleKeyColInfos returns the column(s) and colInfo(s) corresponding to the handle key(s) -func (r *RowChangedEvent) HandleKeyColInfos() ([]*Column, []rowcodec.ColInfo) { - pkeyCols := make([]*Column, 0) +func (r *RowChangedEvent) HandleKeyColInfos() ([]*ColumnData, []rowcodec.ColInfo) { + pkeyCols := make([]*ColumnData, 0) pkeyColInfos := make([]rowcodec.ColInfo, 0) var cols []*ColumnData @@ -571,7 +571,7 @@ func (r *RowChangedEvent) HandleKeyColInfos() ([]*Column, []rowcodec.ColInfo) { colInfos := tableInfo.GetColInfosForRowChangedEvent() for i, col := range cols { if col != nil && tableInfo.ForceGetColumnFlagType(col.ColumnID).IsHandleKey() { - pkeyCols = append(pkeyCols, columnData2Column(col, tableInfo)) + pkeyCols = append(pkeyCols, col) pkeyColInfos = append(pkeyColInfos, colInfos[i]) } } @@ -580,6 +580,31 @@ func (r *RowChangedEvent) HandleKeyColInfos() ([]*Column, []rowcodec.ColInfo) { return pkeyCols, pkeyColInfos } +// HandleKeyColDataXInfos returns the columnDataX(s) and colInfo(s) corresponding to the handle key(s) +func (r *RowChangedEvent) HandleKeyColDataXInfos() ([]ColumnDataX, []rowcodec.ColInfo) { + pkeyColDataXs := make([]ColumnDataX, 0) + pkeyColInfos := make([]rowcodec.ColInfo, 0) + + var cols []*ColumnData + if r.IsDelete() { + cols = r.PreColumns + } else { + cols = r.Columns + } + + tableInfo := r.TableInfo + colInfos := tableInfo.GetColInfosForRowChangedEvent() + for i, col := range cols { + if col != nil && tableInfo.ForceGetColumnFlagType(col.ColumnID).IsHandleKey() { + pkeyColDataXs = append(pkeyColDataXs, GetColumnDataX(col, tableInfo)) + pkeyColInfos = append(pkeyColInfos, colInfos[i]) + } + } + + // It is okay not to have handle keys, so the empty array is an acceptable result + return pkeyColDataXs, pkeyColInfos +} + // ApproximateBytes returns approximate bytes in memory consumed by the event. func (r *RowChangedEvent) ApproximateBytes() int { const sizeOfRowEvent = int(unsafe.Sizeof(*r)) @@ -689,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 b88a0e8e0cc..728627c5ba0 100644 --- a/cdc/puller/ddl_puller_test.go +++ b/cdc/puller/ddl_puller_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util/codec" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" @@ -531,16 +530,6 @@ func TestHandleJob(t *testing.T) { job := &timodel.Job{ Type: timodel.ActionFlashbackCluster, BinlogInfo: &timodel.HistoryInfo{}, - Args: []interface{}{ - 998, - map[string]interface{}{}, - true, /* tidb_gc_enable */ - variable.On, /* tidb_enable_auto_analyze */ - variable.Off, /* tidb_super_read_only */ - 0, /* totalRegions */ - 0, /* startTS */ - 0, /* commitTS */ - }, } skip, err := ddlJobPullerImpl.handleJob(job) require.NoError(t, err) @@ -739,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() @@ -798,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/common/metric.go b/cdc/redo/common/metric.go index 74011bab63b..b9c2389161a 100644 --- a/cdc/redo/common/metric.go +++ b/cdc/redo/common/metric.go @@ -29,7 +29,7 @@ var ( Subsystem: subsystem, Name: "write_bytes_total", Help: "Total number of bytes redo log written", - }, []string{"namespace", "changefeed"}) + }, []string{"namespace", "changefeed", "type"}) // RedoFsyncDurationHistogram records the latency distributions of fsync called by redo writer. RedoFsyncDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ @@ -37,18 +37,18 @@ var ( Subsystem: subsystem, Name: "fsync_duration_seconds", Help: "The latency distributions of fsync called by redo writer", - Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), - }, []string{"namespace", "changefeed"}) + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 16), + }, []string{"namespace", "changefeed", "type"}) // RedoFlushAllDurationHistogram records the latency distributions of flushAll // called by redo writer. RedoFlushAllDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "flushall_duration_seconds", + Name: "flush_all_duration_seconds", Help: "The latency distributions of flushall called by redo writer", - Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), - }, []string{"namespace", "changefeed"}) + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 16), + }, []string{"namespace", "changefeed", "type"}) // RedoTotalRowsCountGauge records the total number of rows written to redo log. RedoTotalRowsCountGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ @@ -56,7 +56,7 @@ var ( Subsystem: subsystem, Name: "total_rows_count", Help: "The total count of rows that are processed by redo writer", - }, []string{"namespace", "changefeed"}) + }, []string{"namespace", "changefeed", "type"}) // RedoWriteLogDurationHistogram records the latency distributions of writeLog. RedoWriteLogDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ @@ -64,8 +64,8 @@ var ( Subsystem: subsystem, Name: "write_log_duration_seconds", Help: "The latency distributions of writeLog called by redoManager", - Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), - }, []string{"namespace", "changefeed"}) + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 16), + }, []string{"namespace", "changefeed", "type"}) // RedoFlushLogDurationHistogram records the latency distributions of flushLog. RedoFlushLogDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ @@ -73,8 +73,8 @@ var ( Subsystem: subsystem, Name: "flush_log_duration_seconds", Help: "The latency distributions of flushLog called by redoManager", - Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), - }, []string{"namespace", "changefeed"}) + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 16), + }, []string{"namespace", "changefeed", "type"}) // RedoWorkerBusyRatio records the busy ratio of redo bgUpdateLog worker. RedoWorkerBusyRatio = prometheus.NewCounterVec( @@ -82,8 +82,8 @@ var ( Namespace: namespace, Subsystem: subsystem, Name: "worker_busy_ratio", - Help: "Busy ratio (X ms in 1s) for redo bgUpdateLog worker.", - }, []string{"namespace", "changefeed"}) + Help: "Busy ratio for redo bgUpdateLog worker.", + }, []string{"namespace", "changefeed", "type"}) ) // InitMetrics registers all metrics in this file diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index a2dd002d6a6..c490751b695 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -231,13 +231,13 @@ func newLogManager( logBuffer: chann.NewAutoDrainChann[cacheEvents](), rtsMap: spanz.SyncMap{}, metricWriteLogDuration: common.RedoWriteLogDurationHistogram. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), + WithLabelValues(changefeedID.Namespace, changefeedID.ID, logType), metricFlushLogDuration: common.RedoFlushLogDurationHistogram. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), + WithLabelValues(changefeedID.Namespace, changefeedID.ID, logType), metricTotalRowsCount: common.RedoTotalRowsCountGauge. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), + WithLabelValues(changefeedID.Namespace, changefeedID.ID, logType), metricRedoWorkerBusyRatio: common.RedoWorkerBusyRatio. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), + WithLabelValues(changefeedID.Namespace, changefeedID.ID, logType), } } @@ -521,10 +521,10 @@ func (m *logManager) bgUpdateLog(ctx context.Context, flushDuration time.Duratio logErrCh := make(chan error, 1) handleErr := func(err error) { logErrCh <- err } - overseerTicker := time.NewTicker(time.Second * 5) + overseerDuration := time.Second * 5 + overseerTicker := time.NewTicker(overseerDuration) defer overseerTicker.Stop() var workTimeSlice time.Duration - startToWork := time.Now() for { select { case <-ctx.Done(): @@ -536,10 +536,8 @@ func (m *logManager) bgUpdateLog(ctx context.Context, flushDuration time.Duratio return nil // channel closed } err = m.handleEvent(ctx, event, &workTimeSlice) - case now := <-overseerTicker.C: - busyRatio := int(workTimeSlice.Seconds() / now.Sub(startToWork).Seconds() * 1000) - m.metricRedoWorkerBusyRatio.Add(float64(busyRatio)) - startToWork = now + case <-overseerTicker.C: + m.metricRedoWorkerBusyRatio.Add(workTimeSlice.Seconds()) workTimeSlice = 0 case err = <-logErrCh: } diff --git a/cdc/redo/meta_manager.go b/cdc/redo/meta_manager.go index cb79af6ff72..078939b8411 100644 --- a/cdc/redo/meta_manager.go +++ b/cdc/redo/meta_manager.go @@ -144,7 +144,7 @@ func (m *metaManager) preStart(ctx context.Context) error { m.extStorage = extStorage m.metricFlushLogDuration = common.RedoFlushLogDurationHistogram. - WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, redo.RedoMetaFileType) err = m.preCleanupExtStorage(ctx) if err != nil { @@ -469,17 +469,23 @@ func (m *metaManager) flush(ctx context.Context, meta common.LogMeta) error { return nil } -// Cleanup removes all redo logs of this manager, it is called when changefeed is removed -// only owner should call this method. -func (m *metaManager) Cleanup(ctx context.Context) error { - common.RedoWriteLogDurationHistogram. - DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) +func (m *metaManager) cleanup(logType string) { common.RedoFlushLogDurationHistogram. - DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, logType) + common.RedoWriteLogDurationHistogram. + DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, logType) common.RedoTotalRowsCountGauge. - DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, logType) common.RedoWorkerBusyRatio. - DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, logType) +} + +// Cleanup removes all redo logs of this manager, it is called when changefeed is removed +// only owner should call this method. +func (m *metaManager) Cleanup(ctx context.Context) error { + m.cleanup(redo.RedoMetaFileType) + m.cleanup(redo.RedoRowLogFileType) + m.cleanup(redo.RedoDDLLogFileType) return m.deleteAllLogs(ctx) } diff --git a/cdc/redo/writer/file/file.go b/cdc/redo/writer/file/file.go index e12a5df59de..39d3b4c4ce8 100644 --- a/cdc/redo/writer/file/file.go +++ b/cdc/redo/writer/file/file.go @@ -104,11 +104,11 @@ func NewFileWriter( storage: extStorage, metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), } if w.op.GetUUIDGenerator != nil { w.uuidGenerator = w.op.GetUUIDGenerator() @@ -213,11 +213,11 @@ func (w *Writer) Close() error { } common.RedoFlushAllDurationHistogram. - DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) + DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, w.cfg.LogType) common.RedoFsyncDurationHistogram. - DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) + DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, w.cfg.LogType) common.RedoWriteBytesGauge. - DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) + DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, w.cfg.LogType) ctx, cancel := context.WithTimeout(context.Background(), redo.CloseTimeout) defer cancel() @@ -399,7 +399,6 @@ func (w *Writer) flushAndRotateFile() error { if err != nil { return nil } - w.metricFlushAllDuration.Observe(time.Since(start).Seconds()) return err diff --git a/cdc/redo/writer/file/file_test.go b/cdc/redo/writer/file/file_test.go index ca13a775e30..68965ec24ca 100644 --- a/cdc/redo/writer/file/file_test.go +++ b/cdc/redo/writer/file/file_test.go @@ -67,11 +67,11 @@ func TestWriterWrite(t *testing.T) { uint64buf: make([]byte, 8), running: *atomic.NewBool(true), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test-cf"), + WithLabelValues("default", "test-cf", redo.RedoRowLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test-cf"), + WithLabelValues("default", "test-cf", redo.RedoRowLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test-cf"), + WithLabelValues("default", "test-cf", redo.RedoRowLogFileType), uuidGenerator: uuidGen, } @@ -158,11 +158,11 @@ func TestWriterWrite(t *testing.T) { uint64buf: make([]byte, 8), running: *atomic.NewBool(true), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test-cf11"), + WithLabelValues("default", "test-cf11", redo.RedoRowLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test-cf11"), + WithLabelValues("default", "test-cf11", redo.RedoRowLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test-cf11"), + WithLabelValues("default", "test-cf11", redo.RedoRowLogFileType), uuidGenerator: uuidGen, } @@ -253,11 +253,11 @@ func TestNewFileWriter(t *testing.T) { uint64buf: make([]byte, 8), storage: mockStorage, metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), uuidGenerator: uuidGen, } w.running.Store(true) @@ -310,11 +310,11 @@ func TestRotateFileWithFileAllocator(t *testing.T) { }, uint64buf: make([]byte, 8), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), storage: mockStorage, uuidGenerator: uuidGen, } @@ -377,11 +377,11 @@ func TestRotateFileWithoutFileAllocator(t *testing.T) { }, uint64buf: make([]byte, 8), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoDDLLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoDDLLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoDDLLogFileType), storage: mockStorage, uuidGenerator: uuidGen, } 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/file_worker.go b/cdc/redo/writer/memory/file_worker.go index b92e37e59ee..3bb398954a1 100644 --- a/cdc/redo/writer/memory/file_worker.go +++ b/cdc/redo/writer/memory/file_worker.go @@ -144,9 +144,9 @@ func newFileWorkerGroup( }, flushCh: make(chan *fileCache), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), } } @@ -179,9 +179,9 @@ func (f *fileWorkerGroup) Run( func (f *fileWorkerGroup) close() { common.RedoFlushAllDurationHistogram. - DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID) + DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID, f.cfg.LogType) common.RedoWriteBytesGauge. - DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID) + DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID, f.cfg.LogType) } func (f *fileWorkerGroup) bgFlushFileCache(egCtx context.Context) error { @@ -191,6 +191,7 @@ func (f *fileWorkerGroup) bgFlushFileCache(egCtx context.Context) error { return errors.Trace(egCtx.Err()) case file := <-f.flushCh: start := time.Now() + if err := file.writer.Close(); err != nil { return errors.Trace(err) } 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/cdc/scheduler/internal/v3/keyspan/splitter_write.go b/cdc/scheduler/internal/v3/keyspan/splitter_write.go index 9773597b582..653328ba618 100644 --- a/cdc/scheduler/internal/v3/keyspan/splitter_write.go +++ b/cdc/scheduler/internal/v3/keyspan/splitter_write.go @@ -170,11 +170,11 @@ func (m *writeSplitter) splitRegionsByWrittenKeysV1( restRegions := len(regions) - i regionCount++ spanWriteWeight += regions[i].WrittenKeys - // If the restSpans count is one, and the restWeight is less than writeLimitPerSpan, + // If the restSpans count is one, and the restWeight is less than equal to writeLimitPerSpan, // we will use the rest regions as the last span. If the restWeight is larger than writeLimitPerSpan, - // then we need to add more restSpans (restWeight / writeLimitPerSpan) to split the rest regions. + // then we need to add more restSpans (restWeight / writeLimitPerSpan) + 1 to split the rest regions. if restSpans == 1 { - if restWeight < int64(writeLimitPerSpan) { + if restWeight <= int64(writeLimitPerSpan) { spans = append(spans, tablepb.Span{ TableID: tableID, StartKey: tablepb.Key(decodeKey(regions[spanStartIndex].StartKey)), @@ -191,11 +191,12 @@ func (m *writeSplitter) splitRegionsByWrittenKeysV1( regionCounts = append(regionCounts, lastSpanRegionCount) weights = append(weights, lastSpanWriteWeight) writeKeys = append(writeKeys, lastSpanWriteKey) + spanStartIndex = len(regions) break } // If the restWeight is larger than writeLimitPerSpan, // then we need to update the restSpans. - restSpans = int(restWeight) / int(writeLimitPerSpan) + restSpans = int(restWeight)/int(writeLimitPerSpan) + 1 } // If the restRegions is less than equal to restSpans, @@ -237,6 +238,32 @@ func (m *writeSplitter) splitRegionsByWrittenKeysV1( spanStartIndex = i + 1 } } + // All regions should be processed and append to spans + if spanStartIndex != len(regions) { + spans = append(spans, tablepb.Span{ + TableID: tableID, + StartKey: tablepb.Key(decodeKey(regions[spanStartIndex].StartKey)), + EndKey: tablepb.Key(decodeKey(regions[len(regions)-1].EndKey)), + }) + lastSpanRegionCount := len(regions) - spanStartIndex + lastSpanWriteWeight := uint64(0) + lastSpanWriteKey := uint64(0) + for j := spanStartIndex; j < len(regions); j++ { + lastSpanWriteKey += regions[j].WrittenKeys + lastSpanWriteWeight += regions[j].WrittenKeys + } + regionCounts = append(regionCounts, lastSpanRegionCount) + weights = append(weights, lastSpanWriteWeight) + writeKeys = append(writeKeys, lastSpanWriteKey) + log.Warn("some regions are added to the last span, it should not appear", + zap.Int("spanStartIndex", spanStartIndex), + zap.Int("regionsLength", len(regions)), + zap.Int("restSpans", restSpans), + zap.Int64("restWeight", restWeight), + zap.Any("prevSpan", spans[len(spans)-2]), + zap.Any("lastSpan", spans[len(spans)-1]), + ) + } return &splitRegionsInfo{ RegionCounts: regionCounts, Weights: weights, diff --git a/cdc/scheduler/internal/v3/keyspan/splitter_write_test.go b/cdc/scheduler/internal/v3/keyspan/splitter_write_test.go index 2987857c776..6c69b7f20b1 100644 --- a/cdc/scheduler/internal/v3/keyspan/splitter_write_test.go +++ b/cdc/scheduler/internal/v3/keyspan/splitter_write_test.go @@ -227,13 +227,44 @@ func TestSplitRegionEven(t *testing.T) { info := splitter.splitRegionsByWrittenKeysV1(tblID, regions, 5) require.Len(t, info.RegionCounts, 5) require.Len(t, info.Weights, 5) + count := 0 for i, w := range info.Weights { if i == 4 { require.Equal(t, uint64(9576), w, i) } else { require.Equal(t, uint64(9591), w, i) } + count += info.RegionCounts[i] } + require.Equal(t, count, regionCount) +} + +func TestSplitLargeRegion(t *testing.T) { + tblID := model.TableID(1) + regionCount := spanRegionLimit*5 + 1000 + regions := make([]pdutil.RegionInfo, regionCount) + for i := 0; i < regionCount; i++ { + regions[i] = pdutil.RegionInfo{ + ID: uint64(i), + StartKey: "" + strconv.Itoa(i), + EndKey: "" + strconv.Itoa(i), + WrittenKeys: 2, + } + } + splitter := newWriteSplitter(model.ChangeFeedID4Test("test", "test"), nil, 4) + info := splitter.splitRegionsByWrittenKeysV1(tblID, regions, 5) + require.Len(t, info.RegionCounts, 6) + require.Len(t, info.Weights, 6) + count := 0 + for i, c := range info.RegionCounts { + if i == 5 { + require.Equal(t, 1000, c, i) + } else { + require.Equal(t, spanRegionLimit, c, i) + } + count += c + } + require.Equal(t, count, regionCount) } func TestSpanRegionLimitBase(t *testing.T) { @@ -247,9 +278,12 @@ func TestSpanRegionLimitBase(t *testing.T) { spanNum := getSpansNumber(len(regions), captureNum) info := splitter.splitRegionsByWrittenKeysV1(0, cloneRegions(regions), spanNum) require.Len(t, info.RegionCounts, spanNum) + count := 0 for _, c := range info.RegionCounts { require.LessOrEqual(t, c, int(spanRegionLimit*1.1)) + count += c } + require.Equal(t, count, len(regions)) } func TestSpanRegionLimit(t *testing.T) { @@ -273,7 +307,7 @@ func TestSpanRegionLimit(t *testing.T) { } // total region number - totalRegionNumbers := spanRegionLimit * 10 + totalRegionNumbers := spanRegionLimit*10 + 100 // writtenKeys over 20000 percentage percentOver20000 := 1 @@ -318,7 +352,12 @@ func TestSpanRegionLimit(t *testing.T) { spanNum := getSpansNumber(len(regions), captureNum) info := splitter.splitRegionsByWrittenKeysV1(0, cloneRegions(regions), spanNum) require.LessOrEqual(t, spanNum, len(info.RegionCounts)) - for _, c := range info.RegionCounts { - require.LessOrEqual(t, c, int(spanRegionLimit*1.1)) + count := 0 + for i, c := range info.RegionCounts { + if i != len(info.RegionCounts)-1 { + require.LessOrEqual(t, c, int(spanRegionLimit*1.1)) + } + count += c } + require.Equal(t, count, totalRegionNumbers) } diff --git a/cdc/sink/ddlsink/mq/mq_ddl_sink_test.go b/cdc/sink/ddlsink/mq/mq_ddl_sink_test.go index fb341c638b5..c3dfc873c19 100644 --- a/cdc/sink/ddlsink/mq/mq_ddl_sink_test.go +++ b/cdc/sink/ddlsink/mq/mq_ddl_sink_test.go @@ -290,4 +290,5 @@ func TestGetDLLDispatchRuleByProtocol(t *testing.T) { require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolMaxwell)) require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolCraft)) require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolSimple)) + require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolDebezium)) } diff --git a/cdc/sink/dmlsink/cloudstorage/dml_worker.go b/cdc/sink/dmlsink/cloudstorage/dml_worker.go index 731e6085803..ddb95675cec 100644 --- a/cdc/sink/dmlsink/cloudstorage/dml_worker.go +++ b/cdc/sink/dmlsink/cloudstorage/dml_worker.go @@ -130,7 +130,7 @@ func newDMLWorker( WithLabelValues(changefeedID.Namespace, changefeedID.ID), metricFlushDuration: mcloudstorage.CloudStorageFlushDurationHistogram. WithLabelValues(changefeedID.Namespace, changefeedID.ID), - metricsWorkerBusyRatio: mcloudstorage.CloudStorageWorkerBusyRatioCounter. + metricsWorkerBusyRatio: mcloudstorage.CloudStorageWorkerBusyRatio. WithLabelValues(changefeedID.Namespace, changefeedID.ID, strconv.Itoa(id)), } @@ -158,19 +158,16 @@ func (d *dmlWorker) run(ctx context.Context) error { // flushMessages flushed messages of active tables to cloud storage. // active tables are those tables that have received events after the last flush. func (d *dmlWorker) flushMessages(ctx context.Context) error { - var flushTimeSlice, totalTimeSlice time.Duration - overseerTicker := time.NewTicker(d.config.FlushInterval * 2) + var flushTimeSlice time.Duration + overseerDuration := d.config.FlushInterval * 2 + overseerTicker := time.NewTicker(overseerDuration) defer overseerTicker.Stop() - startToWork := time.Now() for { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) - case now := <-overseerTicker.C: - totalTimeSlice = now.Sub(startToWork) - busyRatio := flushTimeSlice.Seconds() / totalTimeSlice.Seconds() * 1000 - d.metricsWorkerBusyRatio.Add(busyRatio) - startToWork = now + case <-overseerTicker.C: + d.metricsWorkerBusyRatio.Add(flushTimeSlice.Seconds()) flushTimeSlice = 0 case batchedTask := <-d.toBeFlushedCh: if atomic.LoadUint64(&d.isClosed) == 1 { diff --git a/cdc/sink/metrics/cloudstorage/metrics.go b/cdc/sink/metrics/cloudstorage/metrics.go index 52905b71346..74506ea7c4a 100644 --- a/cdc/sink/metrics/cloudstorage/metrics.go +++ b/cdc/sink/metrics/cloudstorage/metrics.go @@ -56,12 +56,12 @@ var ( }, []string{"namespace", "changefeed"}) // CloudStorageWorkerBusyRatio records the busy ratio of CloudStorage bgUpdateLog worker. - CloudStorageWorkerBusyRatioCounter = prometheus.NewCounterVec( + CloudStorageWorkerBusyRatio = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: namespace, Subsystem: subsystem, Name: "cloud_storage_worker_busy_ratio", - Help: "Busy ratio (X ms in 1s) for cloud storage sink dml worker.", + Help: "Busy ratio for cloud storage sink dml worker.", }, []string{"namespace", "changefeed", "id"}) ) @@ -71,5 +71,5 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(CloudStorageFileCountGauge) registry.MustRegister(CloudStorageWriteDurationHistogram) registry.MustRegister(CloudStorageFlushDurationHistogram) - registry.MustRegister(CloudStorageWorkerBusyRatioCounter) + registry.MustRegister(CloudStorageWorkerBusyRatio) } diff --git a/cmd/dm-worker/main.go b/cmd/dm-worker/main.go index d5d91d71803..8d8f79eaf9e 100644 --- a/cmd/dm-worker/main.go +++ b/cmd/dm-worker/main.go @@ -52,6 +52,7 @@ func main() { Format: cfg.LogFormat, Level: strings.ToLower(cfg.LogLevel), }) + log.SetRedactLog(cfg.RedactInfoLog) if err != nil { common.PrintLinesf("init logger error %s", terror.Message(err)) os.Exit(2) 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 87ee37e7cd1..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,39 +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 - 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)) } @@ -358,128 +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 } - - 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", tableID), - 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", tableID), zap.Any("row", row), - ) - } - - watermark := atomic.LoadUint64(&progress.watermark) - // 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 row.CommitTs < 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", row.CommitTs), zap.Any("offset", message.TopicPartition.Offset), - zap.Uint64("watermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset), - zap.Int32("partition", partition), zap.Int64("tableID", 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), - zap.Uint64("watermark", watermark), zap.Any("watermarkOffset", progress.watermarkOffset), - zap.Int32("partition", partition), zap.Int64("tableID", tableID), - zap.String("schema", row.TableInfo.GetSchemaName()), - zap.String("table", row.TableInfo.GetTableName())) - 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)) - - 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)) + 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 { @@ -489,16 +424,115 @@ func (w *writer) WriteMessage(ctx context.Context, message *kafka.Message) bool return w.Write(ctx, messageType) } +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", 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", offset), + zap.Int64("tableID", row.GetTableID()), zap.Any("row", row), + ) + } +} + +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. + 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.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 + } + 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 { tableIDs map[string]int64 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 } @@ -508,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(): @@ -517,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/cmd/storage-consumer/main.go b/cmd/storage-consumer/main.go index 19af984164c..09561f67c17 100644 --- a/cmd/storage-consumer/main.go +++ b/cmd/storage-consumer/main.go @@ -314,11 +314,8 @@ func (c *consumer) emitDMLEvents( // Always enable tidb extension for canal-json protocol // because we need to get the commit ts from the extension field. c.codecCfg.EnableTiDBExtension = true - decoder, err = canal.NewBatchDecoder(ctx, c.codecCfg, nil) - if err != nil { - return errors.Trace(err) - } - err := decoder.AddKeyValue(nil, content) + decoder = canal.NewCanalJSONTxnEventDecoder(c.codecCfg) + err = decoder.AddKeyValue(nil, content) if err != nil { return errors.Trace(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/chaos/cases/cases.go b/dm/chaos/cases/cases.go index 275e3da5f73..c2baa13a7a2 100644 --- a/dm/chaos/cases/cases.go +++ b/dm/chaos/cases/cases.go @@ -27,7 +27,7 @@ var ( // NOTE: items in `doSchemas` should be specified in the corresponding task files (`filenames`). // TODO: can not support optimistic shard DDL now because go-sqlsmith will generated some statements like // `ALTER TABLE `db_optimistic`.`dxeyydwb` ADD COLUMN (`zuhxfgsce` INT(18) NOT NULL)` which has NOT NULL column without DEFAULT value. - filenames = []string{"task-single.yaml", "task-pessimistic.yaml", "task-optimistic.yaml"} + filenames = []string{"task-single.yaml"} doSchemas = []string{"db_single", "db_pessimistic", "db_optimistic"} ) diff --git a/dm/chaos/cases/task.go b/dm/chaos/cases/task.go index f8712095d4d..24cece74e3e 100644 --- a/dm/chaos/cases/task.go +++ b/dm/chaos/cases/task.go @@ -282,7 +282,7 @@ func (t *task) createTask() error { if err != nil { return err } else if !resp.Result && !strings.Contains(resp.Msg, "already exist") { // imprecise match - return fmt.Errorf("fail to start task: %s", resp.Msg) + return fmt.Errorf("fail to start task: %v", resp) } return nil } diff --git a/dm/chaos/manifests/dm-master.yaml b/dm/chaos/manifests/dm-master.yaml index cbd7567114a..1e397b1a05f 100644 --- a/dm/chaos/manifests/dm-master.yaml +++ b/dm/chaos/manifests/dm-master.yaml @@ -74,7 +74,7 @@ spec: - ReadWriteOnce resources: requests: - storage: 1Gi + storage: 100Mi - metadata: name: dm-master-log spec: diff --git a/dm/chaos/manifests/dm-worker.yaml b/dm/chaos/manifests/dm-worker.yaml index b9987a17694..2fa5288e34d 100644 --- a/dm/chaos/manifests/dm-worker.yaml +++ b/dm/chaos/manifests/dm-worker.yaml @@ -74,7 +74,7 @@ spec: - ReadWriteOnce resources: requests: - storage: 1Gi + storage: 100Mi - metadata: name: dm-worker-log spec: diff --git a/dm/chaos/manifests/sources.yaml b/dm/chaos/manifests/sources.yaml index feeac10cc3b..21337a66b56 100644 --- a/dm/chaos/manifests/sources.yaml +++ b/dm/chaos/manifests/sources.yaml @@ -64,7 +64,7 @@ spec: - ReadWriteOnce resources: requests: - storage: 5Gi + storage: 500Mi --- apiVersion: apps/v1 @@ -87,7 +87,7 @@ spec: spec: containers: - name: mysql8 - image: mysql:latest + image: mysql:8.0 imagePullPolicy: IfNotPresent volumeMounts: - mountPath: "/var/lib/mysql" diff --git a/dm/chaos/manifests/tidb.yaml b/dm/chaos/manifests/tidb.yaml index 3914aece28a..b1b230a4fde 100644 --- a/dm/chaos/manifests/tidb.yaml +++ b/dm/chaos/manifests/tidb.yaml @@ -49,4 +49,4 @@ spec: - ReadWriteOnce resources: requests: - storage: 5Gi + storage: 500Mi 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 12964ec809d..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", - "iaqPrQQIJslQR7ICoUd7NoSvib9zV5oM7bZfDt3uOi0RSCkgOTEHUBQnyAm6RknL9hilq6y9I5SSPxd+", - "vkcf18bUSAuiNBmiew0Mpia8XdGXQSEQU7U02kb6gfENr+D63xOmYtn+EwbnDvyWJ4nhd6lMfBeXrdyF", - "5MRSviQXtRNokMBk/W+XcFJ18sVoomuveJ7KKbPVmuMQJgCnRdK51NiGcbUGld6D/DOO63xvPWvRoVjo", - "iUBD04whzidX15MMYsa7wTKjwdU1UKPd8DlWIRxzgUi47py/sGOYGMdcnejq+jfKpAWN1e29cjYAOc+Z", - "VBZ14cgFdcEhp/MUYgnK4FIVDbT9gL1psX5gLHh7Zsyvgq85LVJ+NVcG8yugninwHftZrvRi9tY1u14+", - "ECuGYFQvbjxsmjklD/oFuTshJSa6cYZMGgafW1HtjB6nrEApdC1+TOhSIiblz+BYZ8TqeQtDA4cLw/mR", - "E0UDUT+KtrkIChD6uLB4QyoZFSzlOmu3QgSkCIlyAAKM3nC1sWZul5z6/TzrFKQc1eluBiVtt4GDX9kQ", - "SpCKQkne3FfzqH2cFwW9zVgyZycSJk19UG5soTZ8Bb0W4upNYL3ZqaLsJAhV1XYe9aAfluqhV5D3pvIV", - "dyWRz/adkZBtZvss98tj+iRTBQsowvoVg3m7Mtmei69JuGKU4H+XS6k5APoDhZqLpCfwNYdEYLWUu6w4", - "SwZKdBORXrH20bB+u9Ad51XOgrrb2KKZ8RWraLW31sm8IYpiBSuE9F1RUz7rBkuYN4Yu4T7iMus1AG6C", - "01jM5yz7cz1lNN2Z6eFXgxM9VXTZPuJo5B2rFWYHcTjbPzqY7L8In0/mc/R8Ao+eHUyOwtnixWH07GV8", - "MDueT57PDueH+wfj2bPD54fRQWgNf3HwbH+yPzuIFvuHR1F0EB3PJ/PnM2dzpnolr9VsST2oSqp9b2a0", - "TqBDt47ayulrx3mob/Nr8b4HlAlDCZROW/eVDWnNy3AtNHvcF8M244RbHYtuPE9T59ZzH14iNzEaHNBb", - "nNyXJ7bh8G5DcVpVWOkLQbNMRQRV7elv5oqjM9PjzDL4y6V1OkNQ+1DaTm7wgdnXhkOnHqoJCv51qAz5", - "eFi1Be+sMhvIl3a20pPJHoMbnEQhZFGRoq2nIReTX+95PtmqNvGdW4qqUK6dfhoAq3DC2lkpYZkLn50Q", - "Hjtccc9DbkZEEdeXY0y+vMCYN7ZlfkcKDlzAZ5Eb5BneTsyRtesgaZUw76bpk6oN3E4t4F1K9LZUv+as", - "WCtp4t11lGZSPryVK/QasRuGxWYJ8PIt7W0Ls0r5R//902rdftB9N8RjiBPVnIxftU8KOmrgnNfAS3Xa", - "33ewUGDVpE7d1TQqeRgizj3gblZR3Z5r3KaGCyh9KflBWyEOV0N68UfuatjoGdZVtNIRbviLAdsbXa3o", - "vX1qrplyUFgvQU2BIu9qodhXcnOH4sW+csVGg92Hb0HhbRG71R4UtyorKqQyTk5o6Ehbn7wDHzJEXn08", - "Aycf3kiVy5LR8aivu+lEGs+JdmkxJabZqY4vYqpYHAuFeGuB4jj8eHQkCagSdxkiMMOj49GB+klqfLFS", - "0E5hhqfX86nppDMtpjf+Utnk7ixSa736eFZvFKfKRbRmVfPtz2a6OWZ55QZmZf5v+i+uSxIrP6qzG7W7", - "JZ2iesMsakWmNpHnaQrZenQscQBlSzoSU8DzcAUgB7U+dQIuudVDbvRZFe/7sNfKp0kAJYavabR+MNzb", - "He9aSJtlwUKue/uE9yFXNKttxZ6T8LfjFj/qUh8+lCWr/n6Pw5iOfoJdZBmPDh8QjFaPSsfS2px3CIbV", - "erwwXJtszPSb/kNFhLda/yVI+4GOnfoQxwkmSJPtvT5nzyCDKdK7/M/Wwb8FXhGTq8Y9UKxGhSEYWTCM", - "bDWuSyZc+U1/h//PLcY5dPjhT2xHqaZro5H8oI0sHIaBElY1n3wcCXM0u9wxCbMa4G8kYWZjpt+MF7aR", - "hBnvcYCE2eD5JcyC4ceWsPrnDDo3Mkr3CuCckvUWiRMa/tfFh/ceUaqDJecqb1y32S2iIVDLVVBFNGxA", - "ZHzUDnD+dvnufBA4cmAPOCuhS4N84Oggr1/1VC1j+5hZyldx81b1cCgvsyme/pojtraYGotVUI5wMLG7", - "5O527PiszRowJHKmm2rpyr6J6adTXApzgVBrI7MJDJ+3q30dXXodkmK3OkiKXtoNPmgOqfihiPFVjMZ9", - "+29/dmFbzrbjyw6bO9zzB4OnzIk8eTunW5ICSKKimhUCgm7sXXdteFsHTL9ZJwv9Vu5EPSyZolMnLBO6", - "UI3NcoK/5vX+HH6DVz/oGGTwvPej2wojpvqmLc0KSGDCTROxokOMSuiYcgqX6lBz3FNn7IDh1XwAYB9P", - "jYfYkF3klcexadu0Jx36rOzdfujkRUN5KkCsPhfVti9dDNGXxtkZnvi8HbvnSuPf1hOhEtzb78MaT0wP", - "mSwWvK9tm0b6A0kqCe53e8xnlHaLRftihidnWzSRH2BTqxZBHXuqv1b0c0u3uaWlG3rfHVUh2WbC+qno", - "FPpjmhPXl99ujT3ZVc1QtWqMc6Kb/RbXXx+GwTZQHD84ezm+9bar3GWU1NaZq2xC1sFbVZfrH5e12p2+", - "h7vBT5vTFAfUGhRvzkvWd9AHhNi6neuQZO0WWMffDG27AW69he2OHFAV/d508aovOTuUPabf9B9VBm8A", - "s6ia76fHK+OOAl/P8hXuA5d31v9ulUvrvVF2i0l1/fPdebTs7zREg5UNEJ+ONey8OPMoZ0GND9jtCPvY", - "X9mvmnk9hIclGCQ8Np/+97tXl2bYj55rbJez/llcrIIRSlVFAdRfptG1Aj3cpY94+jRT8f3OXgaSPA/5", - "1WOefpt7U4t10UZSN9xzrVk8G2qwygaHXas65KO5bLOx5nij9LRlM7esalufaXUwoSJyYhp+Ph1FW0JV", - "sbuuph9yvH+pO6Zt73Dfvi7wPY/2Xd8s3KFz/vKLffUdbqqzaUjJNWJF5W7X9uuB29z/ApQeFsCx5mHM", - "ASZZLnSXe6NL9Rc/Cqx0v2fIr0wnJ/21CMrANQ4RuEaMw60yUQOl3WGjS1UgpahMTMts82EPGgPY/FpK", - "i6h7AzivuDs2zKQWt8MeoZ51x1V7eTnvXjr+srrZtw1ZN3e6vp969wHwRPV5bWc3Ea6paTbTrdzP1KBH", - "2vfmHdXN2WB/S/Dsjn423azuzhbfVLPSTWr4GtyxUXRs90t1hMUlLAODYl+j1Z2um/PfrG4q8MHGcne2", - "afbDKfa2ve7acm+BXHXH+uem70xp2tB9b+nvu2ntp8oRXcXWCgZ0jQjAsfpeCOD5ogj7WNmr6Ge5tS/S", - "H2AmdoYvHiFX+j20UyOIPPR1xusoqvbvfl9J9VNmgK1WUd8vwTj70ROMZXX1wASjZbI853NFD76iv+aQ", - "dFCtbyffGUX26MURzjMW3R/f9GUf+Yoefh0+o26h3z2hGvPr45+Jt7ll507G1VmdXV0BSWS60ZofGM2F", - "uYuGaxeL7y6Vg2vJyiqy12tJ61ckutsJ+g8ilD+r27r4213idm8u3rDkrSx2+8nSP4vwdlaWnJV4DyxK", - "8r1FgjZMSSwSdCFYHoqc/ZSppyZTY39HWx/JCw4YTHP3V/t2P31fkzxusfimyZmfEvJTQubfJ1iqM9/u", - "B0udYujPkpXpmZ+iuPHiP4ogPnyK0koKNuXwz1WLrSVuQ7PZ7bUK2FvnciHH/ICZ7xLvXb+Pqzb5jsnn", - "YTeLrE/K7qCyL1ua73pt/Y5eYjLXKjT3bMadNOtVXjT7IXWXRnv3VRfN/JpLfXyEXRc7Wm8+v6b5XkRT", - "iIlqPT+SpDYTuHXBqK/bfUTDwS3uTU/76dcch1cTpYEnuix1UnUFq+mYkcszU2hvF6obLFaTKLXgUcu2", - "oSm6wJbjih9uP9/+XwAAAP//rLmSJF28AAA=", + "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 a1ac719d35d..647c25bf2ef 100644 --- a/dm/openapi/gen.types.go +++ b/dm/openapi/gen.types.go @@ -28,9 +28,13 @@ const ( const ( TaskTaskModeAll TaskTaskMode = "all" + TaskTaskModeDump TaskTaskMode = "dump" + TaskTaskModeFull TaskTaskMode = "full" TaskTaskModeIncremental TaskTaskMode = "incremental" + + TaskTaskModeLoad TaskTaskMode = "load" ) // Defines values for TaskFullMigrateConfAnalyze. @@ -78,6 +82,8 @@ const ( const ( TaskStageFinished TaskStage = "Finished" + TaskStagePaused TaskStage = "Paused" + TaskStageRunning TaskStage = "Running" TaskStageStopped TaskStage = "Stopped" @@ -163,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"` } @@ -258,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"` @@ -499,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 9881b1538a2..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 @@ -1825,6 +1845,8 @@ components: - "full" - "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/conn/baseconn.go b/dm/pkg/conn/baseconn.go index 3a58445e640..157b248281a 100644 --- a/dm/pkg/conn/baseconn.go +++ b/dm/pkg/conn/baseconn.go @@ -117,13 +117,13 @@ func (conn *BaseConn) QuerySQL(tctx *tcontext.Context, query string, args ...int } tctx.L().Debug("query statement", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(args, -1))) + log.ZapRedactString("argument", utils.TruncateInterface(args, -1))) rows, err := conn.DBConn.QueryContext(tctx.Context(), query, args...) if err != nil { tctx.L().ErrorFilterContextCanceled("query statement failed", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(args, -1)), + log.ZapRedactString("argument", utils.TruncateInterface(args, -1)), log.ShortError(err)) return nil, terror.ErrDBQueryFailed.Delegate(err, utils.TruncateString(query, -1)) } @@ -183,7 +183,7 @@ func (conn *BaseConn) ExecuteSQLWithIgnoreError(tctx *tcontext.Context, hVec *pr if tctx.L().Core().Enabled(zap.DebugLevel) { tctx.L().Debug("execute statement", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(arg, -1))) + log.ZapRedactString("argument", utils.TruncateInterface(arg, -1))) } startTime = time.Now() @@ -198,21 +198,21 @@ func (conn *BaseConn) ExecuteSQLWithIgnoreError(tctx *tcontext.Context, hVec *pr if ignoreErr != nil && ignoreErr(err2) { tctx.L().Warn("execute statement failed and will ignore this error", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(arg, -1)), + log.ZapRedactString("argument", utils.TruncateInterface(arg, -1)), log.ShortError(err2)) continue } tctx.L().ErrorFilterContextCanceled("execute statement failed", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(arg, -1)), log.ShortError(err2)) + log.ZapRedactString("argument", utils.TruncateInterface(arg, -1)), log.ShortError(err2)) startTime = time.Now() rerr := txn.Rollback() if rerr != nil { tctx.L().Error("rollback failed", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(arg, -1)), + log.ZapRedactString("argument", utils.TruncateInterface(arg, -1)), log.ShortError(rerr)) } else if hVec != nil { hVec.WithLabelValues("rollback", task).Observe(time.Since(startTime).Seconds()) diff --git a/dm/pkg/conn/basedb.go b/dm/pkg/conn/basedb.go index f7493c0260d..4ec7d591d4e 100644 --- a/dm/pkg/conn/basedb.go +++ b/dm/pkg/conn/basedb.go @@ -247,7 +247,7 @@ func (d *BaseDB) ExecContext(tctx *tcontext.Context, query string, args ...inter if tctx.L().Core().Enabled(zap.DebugLevel) { tctx.L().Debug("exec context", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(args, -1))) + log.ZapRedactString("argument", utils.TruncateInterface(args, -1))) } return d.DB.ExecContext(tctx.Ctx, query, args...) } @@ -257,7 +257,7 @@ func (d *BaseDB) QueryContext(tctx *tcontext.Context, query string, args ...inte if tctx.L().Core().Enabled(zap.DebugLevel) { tctx.L().Debug("query context", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(args, -1))) + log.ZapRedactString("argument", utils.TruncateInterface(args, -1))) } return d.DB.QueryContext(tctx.Ctx, query, args...) } @@ -286,7 +286,7 @@ func (d *BaseDB) DoTxWithRetry(tctx *tcontext.Context, queries []string, args [] if tctx.L().Core().Enabled(zap.DebugLevel) { tctx.L().Debug("exec in tx", zap.String("query", utils.TruncateString(q, -1)), - zap.String("argument", utils.TruncateInterface(args[i], -1))) + log.ZapRedactString("argument", utils.TruncateInterface(args[i], -1))) } if _, err = tx.ExecContext(tctx.Ctx, q, args[i]...); err != nil { return nil, errors.Trace(err) 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/log/log.go b/dm/pkg/log/log.go index 93c2cf7bb9e..76c1af6e127 100644 --- a/dm/pkg/log/log.go +++ b/dm/pkg/log/log.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "strings" + "sync/atomic" "github.com/pingcap/errors" pclog "github.com/pingcap/log" @@ -194,3 +195,32 @@ func WrapStringerField(message string, object fmt.Stringer) zap.Field { func WithCtx(ctx context.Context) Logger { return Logger{appLogger.With(getZapFieldsFromCtx(ctx)...)} } + +var enabledRedactLog atomic.Bool + +func init() { + SetRedactLog(false) +} + +// IsRedactLogEnabled indicates whether the log desensitization is enabled. +func IsRedactLogEnabled() bool { + return enabledRedactLog.Load() +} + +// SetRedactLog sets enabledRedactLog. +func SetRedactLog(enabled bool) { + enabledRedactLog.Store(enabled) +} + +// RedactString receives string argument and return omitted information if redact log enabled. +func RedactString(arg string) string { + if IsRedactLogEnabled() { + return "?" + } + return arg +} + +// ZapRedactString receives string argument and return omitted information in zap.Field if redact log enabled. +func ZapRedactString(key, arg string) zap.Field { + return zap.String(key, RedactString(arg)) +} 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/causality.go b/dm/syncer/causality.go index 028830ce111..6320a1a8e41 100644 --- a/dm/syncer/causality.go +++ b/dm/syncer/causality.go @@ -23,11 +23,27 @@ import ( "go.uber.org/zap" ) -// causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. -// causality groups sqls that maybe contain causal relationships, and syncer executes them linearly. -// if some conflicts exist in more than one groups, causality generate a conflict job and reset. +// causality provides a simple mechanism to ensure correctness when we are running +// DMLs concurrently. +// As a table might have one or multiple keys (including PK and UKs), row changes +// might depend on other row changes, together they form a dependency graph, only +// row changes without dependency can run concurrently. +// currently, row changes for a table from upstream are dispatched to DML workers +// by their keys, to make sure row changes with same keys are dispatched to the +// same worker, but this cannot handle dependencies cross row changes with +// different keys. +// suppose we have a table `t(a int unique, b int unique)`, and following row changes: +// - insert t(a=1, b=1), we put to worker 1 +// - insert t(a=2, b=2), we put to worker 2 +// - delete t(a=2, b=2), we put to worker 2 +// - update t set b=2 where a=1, this row change depends on all above row changes, +// we must at least wait all row changes related to (a=2, b=2) finish before +// dispatch it to worker 1, else data inconsistency might happen. +// +// causality is used to detect this kind of dependencies, and it will generate a +// conflict job to wait all DMLs in DML workers are executed before we can continue +// dispatching. // this mechanism meets quiescent consistency to ensure correctness. -// causality relation is consisted of groups of keys separated by flush job, and such design helps removed flushed dml job keys. type causality struct { relation *causalityRelation outCh chan *job diff --git a/dm/syncer/dbconn/db.go b/dm/syncer/dbconn/db.go index 5dbfc2b678a..ef8f5276d76 100644 --- a/dm/syncer/dbconn/db.go +++ b/dm/syncer/dbconn/db.go @@ -121,7 +121,7 @@ func (conn *DBConn) QuerySQL( ctx.L().Warn("query statement too slow", zap.Duration("cost time", cost), zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(args, -1))) + log.ZapRedactString("argument", utils.TruncateInterface(args, -1))) } } return ret, err @@ -130,7 +130,7 @@ func (conn *DBConn) QuerySQL( if err != nil { tctx.L().ErrorFilterContextCanceled("query statement failed after retry", zap.String("query", utils.TruncateString(query, -1)), - zap.String("argument", utils.TruncateInterface(args, -1)), + log.ZapRedactString("argument", utils.TruncateInterface(args, -1)), log.ShortError(err)) return nil, err } @@ -195,7 +195,7 @@ func (conn *DBConn) ExecuteSQLWithIgnore( ctx.L().Warn("execute transaction too slow", zap.Duration("cost time", cost), zap.String("query", utils.TruncateInterface(queries, -1)), - zap.String("argument", utils.TruncateInterface(args, -1))) + log.ZapRedactString("argument", utils.TruncateInterface(args, -1))) } } return ret, err @@ -203,7 +203,7 @@ func (conn *DBConn) ExecuteSQLWithIgnore( if err != nil { tctx.L().ErrorFilterContextCanceled("execute statements failed after retry", 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 ret.(int), err } @@ -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_dm_ctl_with_retry b/dm/tests/_utils/run_dm_ctl_with_retry index b27149c057d..ea53e806e03 100755 --- a/dm/tests/_utils/run_dm_ctl_with_retry +++ b/dm/tests/_utils/run_dm_ctl_with_retry @@ -8,8 +8,15 @@ workdir=$1 master_addr=$2 cmd=$3 +retry_times=10 # Default retry times -shift 3 +# Validate retry_times is a number +if [[ $4 =~ ^[0-9]+$ ]] && [[ $# -gt 4 ]]; then + retry_times=$4 + shift 4 +else + shift 3 +fi PWD=$(pwd) binary=$PWD/bin/dmctl.test @@ -18,8 +25,8 @@ dmctl_log=$workdir/dmctl.$ts.log pid=$$ all_matched=true -echo "dmctl test cmd: \"$cmd\"" -for ((k = 0; k < 10; k++)); do +echo "dmctl test cmd: \"$cmd\" with max $retry_times retries" +for ((k = 0; k < retry_times; k++)); do echo "$cmd" | $binary -test.coverprofile="$TEST_DIR/cov.$TEST_NAME.dmctl.$ts.$pid.out" DEVEL --master-addr=$master_addr >$dmctl_log 2>&1 all_matched=true for ((i = 1; i < $#; i += 2)); do 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/dmctl_basic/conf/get_worker1.toml b/dm/tests/dmctl_basic/conf/get_worker1.toml index c69ee084370..b2b3981b989 100644 --- a/dm/tests/dmctl_basic/conf/get_worker1.toml +++ b/dm/tests/dmctl_basic/conf/get_worker1.toml @@ -3,6 +3,7 @@ log-level = "debug" log-file = "/tmp/dm_test/dmctl_basic/worker1/log/dm-worker.log" log-format = "text" log-rotate = "" +redact-info-log = false join = "http://127.0.0.1:8261" worker-addr = "0.0.0.0:8262" advertise-addr = "127.0.0.1:8262" diff --git a/dm/tests/dmctl_basic/conf/get_worker2.toml b/dm/tests/dmctl_basic/conf/get_worker2.toml index a8c9458f54d..7f4b9903b85 100644 --- a/dm/tests/dmctl_basic/conf/get_worker2.toml +++ b/dm/tests/dmctl_basic/conf/get_worker2.toml @@ -3,6 +3,7 @@ log-level = "debug" log-file = "/tmp/dm_test/dmctl_basic/worker2/log/dm-worker.log" log-format = "text" log-rotate = "" +redact-info-log = false join = "http://127.0.0.1:8261" worker-addr = "0.0.0.0:8263" advertise-addr = "127.0.0.1:8263" diff --git a/dm/tests/import_v10x/run.sh b/dm/tests/import_v10x/run.sh index cb453613b3f..d560f6ad122 100644 --- a/dm/tests/import_v10x/run.sh +++ b/dm/tests/import_v10x/run.sh @@ -88,7 +88,7 @@ function run() { run_sql "show create table \`dm_meta\`.\`test_syncer_checkpoint\`" $TIDB_PORT $TIDB_PASSWORD check_contains "\`exit_safe_binlog_name\` varchar(128) DEFAULT ''" # different TiDB will output DEFAULT 0 or DEFAULT '0' - check_contains "\`exit_safe_binlog_pos\` int(10) unsigned DEFAULT " + check_contains "\`exit_safe_binlog_pos\` int" check_contains "\`exit_safe_binlog_gtid\` text DEFAULT NULL" run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 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/mariadb_master_down_and_up/docker-compose.yml b/dm/tests/mariadb_master_down_and_up/docker-compose.yml index e60a02156e4..9a555558853 100644 --- a/dm/tests/mariadb_master_down_and_up/docker-compose.yml +++ b/dm/tests/mariadb_master_down_and_up/docker-compose.yml @@ -38,5 +38,5 @@ services: hostname: tidb ports: - "4000:4000" - image: pingcap/tidb:latest + image: pingcap/tidb:nightly restart: always diff --git a/dm/tests/openapi/client/openapi_task_check b/dm/tests/openapi/client/openapi_task_check index 4389131d572..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", @@ -268,6 +269,84 @@ def create_shard_task_success(): print("create_shard_task_success resp=", resp.json()) assert resp.status_code == 201 +def create_dump_task_success(): + task = { + "name": "test-dump", + "task_mode": "dump", + "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": "t"}, + } + ], + "source_config": { + "full_migrate_conf": { + "export_threads": 4, + "import_threads": 16, + "data_dir": "./exported_data", + "consistency": "auto", + }, + "incr_migrate_conf": {"repl_threads": 16, "repl_batch": 100}, + "source_conf": [ + {"source_name": SOURCE1_NAME}, + ], + } + } + resp = requests.post(url=API_ENDPOINT, json={"task": task}) + 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" @@ -374,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" @@ -594,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) @@ -703,6 +849,8 @@ if __name__ == "__main__": "create_task_failed": create_task_failed, "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, @@ -729,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 5577da3d0d1..44ccee9cc39 100644 --- a/dm/tests/openapi/run.sh +++ b/dm/tests/openapi/run.sh @@ -24,6 +24,15 @@ function init_noshard_data() { run_sql_source2 "INSERT INTO openapi.t2(i,j) VALUES (3, 4);" } +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() { run_sql_source1 "CREATE TABLE openapi.t(i TINYINT, j INT UNIQUE KEY);" run_sql_source2 "CREATE TABLE openapi.t(i TINYINT, j INT UNIQUE KEY);" @@ -176,6 +185,121 @@ function test_relay() { } +function test_dump_and_load_task() { + echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>START TEST OPENAPI: dump & load TASK" + prepare_database + + task_name_dump="test-dump" + task_name_load="test-load" + + # create source successfully + openapi_source_check "create_source1_success" + # get source list success + openapi_source_check "list_source_success" 1 + + # 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-01" + + # create task success: not valid task create request + openapi_task_check "create_task_failed" + + # 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_dump" \ + "\"stage\": \"Stopped\"" 1 + + init_dump_data + + # start dump task success + 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_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 & 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() { echo ">>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>START TEST OPENAPI: SHARD TASK" prepare_database @@ -229,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 @@ -290,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" @@ -353,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" @@ -1040,6 +1167,7 @@ function run() { test_shard_task test_multi_tasks test_noshard_task + test_dump_and_load_task test_task_templates test_noshard_task_dump_status test_complex_operations_of_source_and_task @@ -1048,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/tests/upstream_switch/docker-compose.yml b/dm/tests/upstream_switch/docker-compose.yml index 2beb3787da0..55d8cfabac5 100644 --- a/dm/tests/upstream_switch/docker-compose.yml +++ b/dm/tests/upstream_switch/docker-compose.yml @@ -76,5 +76,5 @@ services: networks: db-networks: ipv4_address: 172.28.128.8 - image: pingcap/tidb:latest + image: pingcap/tidb:nightly restart: always diff --git a/dm/worker/config.go b/dm/worker/config.go index e169d9eb6fb..1104b9a0b93 100644 --- a/dm/worker/config.go +++ b/dm/worker/config.go @@ -66,6 +66,7 @@ func NewConfig() *Config { fs.StringVar(&cfg.LogLevel, "L", "info", "log level: debug, info, warn, error, fatal") fs.StringVar(&cfg.LogFile, "log-file", "", "log file path") fs.StringVar(&cfg.LogFormat, "log-format", "text", `the format of the log, "text" or "json"`) + fs.BoolVar(&cfg.RedactInfoLog, "redact-info-log", false, "whether to enable the log redaction") // fs.StringVar(&cfg.LogRotate, "log-rotate", "day", "log file rotate type, hour/day") // NOTE: add `advertise-addr` for dm-master if needed. fs.StringVar(&cfg.Join, "join", "", `join to an existing cluster (usage: dm-master cluster's "${master-addr}")`) @@ -91,6 +92,11 @@ type Config struct { LogFormat string `toml:"log-format" json:"log-format"` LogRotate string `toml:"log-rotate" json:"log-rotate"` + // RedactInfoLog indicates that whether to enable the log redaction. It can be the following values: + // - false: disable redact log. + // - true: enable redact log, which will replace the sensitive information with "?". + RedactInfoLog bool `toml:"redact-info-log" json:"redact-info-log"` + Join string `toml:"join" json:"join" ` WorkerAddr string `toml:"worker-addr" json:"worker-addr"` AdvertiseAddr string `toml:"advertise-addr" json:"advertise-addr"` diff --git a/dm/worker/join.go b/dm/worker/join.go index 7b39bb79463..476e0077233 100644 --- a/dm/worker/join.go +++ b/dm/worker/join.go @@ -31,6 +31,8 @@ import ( "google.golang.org/grpc/credentials" ) +const retryTimes = 5 + // GetJoinURLs gets the endpoints from the join address. func GetJoinURLs(addrs string) []string { // TODO: handle pm1=xxxx:1234,pm2=xxxx:1234,pm3=xxxx:1234 @@ -63,46 +65,51 @@ func (s *Server) JoinMaster(endpoints []string) error { } var errorStr string - for _, endpoint := range endpoints { - ctx1, cancel1 := context.WithTimeout(ctx, 3*time.Second) - //nolint:staticcheck - conn, err := grpc.DialContext( - ctx1, - utils.UnwrapScheme(endpoint), - grpc.WithBlock(), - grpcTLS, - grpc.WithBackoffMaxDelay(3*time.Second), - ) - cancel1() - if err != nil { - if conn != nil { - conn.Close() + // retry to connect master + for i := 0; i < retryTimes; i++ { + for _, endpoint := range endpoints { + ctx1, cancel1 := context.WithTimeout(ctx, 3*time.Second) + //nolint:staticcheck + conn, err := grpc.DialContext( + ctx1, + utils.UnwrapScheme(endpoint), + grpc.WithBlock(), + grpcTLS, + grpc.WithBackoffMaxDelay(3*time.Second), + ) + cancel1() + if err != nil { + if conn != nil { + conn.Close() + } + log.L().Error("fail to dial dm-master", zap.String("endpoint", endpoint), zap.Error(err)) + errorStr = err.Error() + continue + } + client := pb.NewMasterClient(conn) + ctx1, cancel1 = context.WithTimeout(ctx, 3*time.Second) + resp, err := client.RegisterWorker(ctx1, req) + cancel1() + conn.Close() + if err != nil { + log.L().Error("fail to register worker", zap.String("endpoint", endpoint), zap.Error(err)) + errorStr = err.Error() + continue + } + if !resp.GetResult() { + log.L().Error("fail to register worker", zap.String("endpoint", endpoint), zap.String("error", resp.Msg)) + errorStr = resp.Msg + continue } - log.L().Error("fail to dial dm-master", zap.String("endpoint", endpoint), zap.Error(err)) - errorStr = err.Error() - continue - } - client := pb.NewMasterClient(conn) - ctx1, cancel1 = context.WithTimeout(ctx, 3*time.Second) - resp, err := client.RegisterWorker(ctx1, req) - cancel1() - conn.Close() - if err != nil { - log.L().Error("fail to register worker", zap.String("endpoint", endpoint), zap.Error(err)) - errorStr = err.Error() - continue - } - if !resp.GetResult() { - log.L().Error("fail to register worker", zap.String("endpoint", endpoint), zap.String("error", resp.Msg)) - errorStr = resp.Msg - continue - } - // worker do calls decrypt, but the password is decrypted already, - // but in case we need it later, init it. - encrypt.InitCipher(resp.GetSecretKey()) + // worker do calls decrypt, but the password is decrypted already, + // but in case we need it later, init it. + encrypt.InitCipher(resp.GetSecretKey()) - return nil + return nil + } + log.L().Warn("retry to connect master", zap.Int("retry", i+1), zap.Int("total", retryTimes)) + time.Sleep(retryConnectSleepTime) } return terror.ErrWorkerFailConnectMaster.Generate(endpoints, errorStr) } 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/errors.toml b/errors.toml index 6dee6a90c34..8bac0b36a0b 100755 --- a/errors.toml +++ b/errors.toml @@ -226,6 +226,11 @@ error = ''' cannot find mysql.tidb_ddl_job schema ''' +["CDC:ErrDDLUnsupportType"] +error = ''' +unsupport ddl type %s, query %s +''' + ["CDC:ErrDatumUnflatten"] error = ''' unflatten datume data diff --git a/go.mod b/go.mod index 8ce5bf41de4..e91ed9dc34d 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 @@ -68,12 +69,12 @@ 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.20241014034929-94b2ac04a0c4 + 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/prometheus/client_golang v1.20.4 + 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 github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 @@ -90,9 +91,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.20241008085809-c3e10ae7c8fc + 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-20240926021936-642f0e919b0d + 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 @@ -112,12 +113,12 @@ 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/time v0.5.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 google.golang.org/grpc v1.64.0 @@ -130,7 +131,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 @@ -160,7 +160,7 @@ require ( github.com/go-logr/stdr v1.2.2 // indirect github.com/go-resty/resty/v2 v2.11.0 // indirect github.com/goccy/go-reflect v1.2.0 // indirect - github.com/golang-jwt/jwt/v4 v4.5.0 // indirect + github.com/golang-jwt/jwt/v4 v4.5.1 // indirect github.com/golang-jwt/jwt/v5 v5.2.1 // indirect github.com/google/flatbuffers v2.0.8+incompatible // indirect github.com/google/gofuzz v1.2.0 // indirect @@ -200,8 +200,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 @@ -252,7 +252,7 @@ require ( github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect github.com/desertbit/timer v0.0.0-20180107155436-c41aec40b27f // indirect github.com/dgraph-io/ristretto v0.1.1 // indirect - github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 // indirect + github.com/dgryski/go-farm v0.0.0-20240924180020-3414d57e47da // indirect github.com/dvsekhvalnov/jose2go v1.5.0 // indirect github.com/eapache/go-resiliency v1.4.0 // indirect github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 // indirect @@ -326,11 +326,11 @@ require ( github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7 // indirect github.com/philhofer/fwd v1.1.1 // indirect github.com/pierrec/lz4 v2.6.1+incompatible // indirect - github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d // indirect + github.com/pingcap/badger v1.5.1-0.20241015064302-38533b6cbf8d // indirect 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 +339,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 +382,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 f08cf324d1b..b27c43c1f58 100644 --- a/go.sum +++ b/go.sum @@ -268,8 +268,8 @@ github.com/dgraph-io/ristretto v0.1.1/go.mod h1:S1GPSBCYCIhmVNfcth17y2zZtQT6wzkz github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= -github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= -github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dgryski/go-farm v0.0.0-20240924180020-3414d57e47da h1:aIftn67I1fkbMa512G+w+Pxci9hJPB8oMnkcP3iZF38= +github.com/dgryski/go-farm v0.0.0-20240924180020-3414d57e47da/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dimfeld/httptreemux v5.0.1+incompatible h1:Qj3gVcDNoOthBAqftuD596rm4wg/adLLz5xh5CmpiCA= github.com/dimfeld/httptreemux v5.0.1+incompatible/go.mod h1:rbUlSV+CCpv/SuqUTP/8Bk2O3LyUV436/yaRGkhP6Z0= github.com/docker/distribution v2.8.1+incompatible h1:Q50tZOPR6T/hjNsyc9g8/syEs6bk8XXApsHjKukMl68= @@ -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= @@ -456,8 +455,8 @@ github.com/goji/httpauth v0.0.0-20160601135302-2da839ab0f4d/go.mod h1:nnjvkQ9ptG github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= github.com/golang-jwt/jwt v3.2.2+incompatible h1:IfV12K8xAKAnZqdXVzCZ+TOjboZ2keLg81eXfW3O+oY= github.com/golang-jwt/jwt v3.2.2+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= -github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= -github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= +github.com/golang-jwt/jwt/v4 v4.5.1 h1:JdqV9zKUdtaa9gdPlywC3aeoEsR681PlKC+4F5gQgeo= +github.com/golang-jwt/jwt/v4 v4.5.1/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= @@ -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= @@ -840,8 +839,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= @@ -880,8 +879,8 @@ github.com/pierrec/lz4 v2.6.1+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= -github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d h1:AEcvKyVM8CUII3bYzgz8haFXtGiqcrtXW1csu/5UELY= -github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d/go.mod h1:p8QnkZnmyV8L/M/jzYb8rT7kv3bz9m7bn1Ju94wDifs= +github.com/pingcap/badger v1.5.1-0.20241015064302-38533b6cbf8d h1:eHcokyHxm7HVM+7+Qy1zZwC7NhX9wVNX8oQDcSZw1qI= +github.com/pingcap/badger v1.5.1-0.20241015064302-38533b6cbf8d/go.mod h1:KiO2zumBCWx7yoVYoFRpb+DNrwEPk1pR1LF7NvOACMQ= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 h1:HVl5539r48eA+uDuX/ziBmQCxzT1pGrzWbKuXT46Bq0= github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= @@ -897,8 +896,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 +905,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.20241014034929-94b2ac04a0c4 h1:p6/3EiEs5RtSgVmr99sJ3sSEe6MuBg3VLt2P3EG/N4s= -github.com/pingcap/tidb v1.1.0-beta.0.20241014034929-94b2ac04a0c4/go.mod h1:pKErppbl4OhrGZVbAuS5ToknZ5Tw9KeNvU4FEgXuoZc= +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= @@ -930,8 +929,8 @@ github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:Om github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3gMACTjAbMZBjXAqTOzOwFaj2Ld6cjeQ7Rig= github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= -github.com/prometheus/client_golang v1.20.4 h1:Tgh3Yr67PaOv/uTqloMsCEdeuFTatm5zIq5+qNN23vI= -github.com/prometheus/client_golang v1.20.4/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= @@ -969,8 +968,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 +1107,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.20241008085809-c3e10ae7c8fc h1:07m9V+2BT06XC/P72EuM3tnlB6xJ6hTHNCju/x1Tfzg= -github.com/tikv/client-go/v2 v2.0.8-0.20241008085809-c3e10ae7c8fc/go.mod h1:JZq2+O34RjrronQ9+sSrnPuUT0BHFMjayg11Sosi1JY= +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-20240926021936-642f0e919b0d h1:iBIi3+grJWLNI5c7BLVezgOonYtL/EQqSUDsPVK2kUw= -github.com/tikv/pd/client v0.0.0-20240926021936-642f0e919b0d/go.mod h1:uBHhxAM/SPCMabt483gI/pN/+JXIMKYXohK96s+PwT8= +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 +1300,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 +1357,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 +1402,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 +1422,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 +1497,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 +1507,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,16 +1523,16 @@ 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= golang.org/x/time v0.0.0-20201208040808-7e3f01d25324/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= -golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/time v0.7.0 h1:ntUhktv3OPE6TgYxXWv9vKvUSJyIFJlyohwbkEwPrKQ= +golang.org/x/time v0.7.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -1580,7 +1579,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 +1587,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 +1743,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/metrics/grafana/TiCDC-Monitor-Summary.json b/metrics/grafana/TiCDC-Monitor-Summary.json index d9d59b613a3..c2ffd96e7b5 100644 --- a/metrics/grafana/TiCDC-Monitor-Summary.json +++ b/metrics/grafana/TiCDC-Monitor-Summary.json @@ -2544,11 +2544,11 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_total_rows_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_redo_total_rows_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance,type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" }, @@ -2652,9 +2652,9 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_write_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_redo_write_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance,type)", "interval": "", - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" } @@ -2812,7 +2812,7 @@ "targets": [ { "exemplar": true, - "expr": "max(rate(ticdc_redo_flushall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (le)", + "expr": "max(rate(ticdc_redo_flush_all_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (le)", "format": "heatmap", "interval": "", "intervalFactor": 2, @@ -2823,7 +2823,7 @@ ], "timeFrom": null, "timeShift": null, - "title": "Redo flushall duration", + "title": "Redo flush all duration", "tooltip": { "show": true, "showHistogram": true @@ -3164,5 +3164,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC-Summary", "uid": "McUpY954z", - "version": 2 -} + "version": 3 +} \ No newline at end of file diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index aa159db19e3..e2b37d14ab0 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -9404,7 +9404,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_sink_cloud_storage_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",namespace=~\"$namespace\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])/10) by (namespace,changefeed,id,instance)", + "expr": "sum(rate(ticdc_sink_cloud_storage_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",namespace=~\"$namespace\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])*100) by (namespace,changefeed,id,instance)", "hide": false, "interval": "", "legendFormat": "{{namespace}}-{{changefeed}}-{{id}}-{{instance}}", @@ -18963,7 +18963,7 @@ "targets": [ { "exemplar": true, - "expr": "max(rate(ticdc_redo_flushall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (le)", + "expr": "max(rate(ticdc_redo_flush_all_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (le)", "format": "heatmap", "interval": "", "intervalFactor": 2, @@ -18974,7 +18974,7 @@ ], "timeFrom": null, "timeShift": null, - "title": "Redo flushall duration", + "title": "Redo flush all duration", "tooltip": { "show": true, "showHistogram": true @@ -19197,11 +19197,11 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_total_rows_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_redo_total_rows_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance,type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" }, @@ -19305,9 +19305,9 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_write_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_redo_write_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance,type)", "interval": "", - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" } @@ -19403,9 +19403,9 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])/10) by (changefeed,instance)", + "expr": "sum(rate(ticdc_redo_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])*100) by (changefeed,instance,type)", "interval": "", - "legendFormat": "{{changefeed}}-{{instance}}", + "legendFormat": "{{changefeed}}-{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" } @@ -21651,5 +21651,5 @@ "timezone": "browser", "title": "${DS_TEST-CLUSTER}-TiCDC", "uid": "YiGL8hBZ1", - "version": 61 -} + "version": 62 +} \ No newline at end of file diff --git a/pkg/cmd/cmd.go b/pkg/cmd/cmd.go index 704c5488ede..eecac0da4c1 100644 --- a/pkg/cmd/cmd.go +++ b/pkg/cmd/cmd.go @@ -35,6 +35,15 @@ func NewCmd() *cobra.Command { } } +// AddTiCDCCommandTo add all cdc subcommands to `cmd`. +// Exported for ticdc new arch. +func AddTiCDCCommandTo(cmd *cobra.Command) { + cmd.AddCommand(server.NewCmdServer()) + cmd.AddCommand(cli.NewCmdCli()) + cmd.AddCommand(version.NewCmdVersion()) + cmd.AddCommand(redo.NewCmdRedo()) +} + // Run runs the root command. func Run() { cmd := NewCmd() @@ -42,10 +51,7 @@ func Run() { cmd.SetOut(os.Stdout) cmd.SetErr(os.Stderr) - cmd.AddCommand(server.NewCmdServer()) - cmd.AddCommand(cli.NewCmdCli()) - cmd.AddCommand(version.NewCmdVersion()) - cmd.AddCommand(redo.NewCmdRedo()) + AddTiCDCCommandTo(cmd) if err := cmd.Execute(); err != nil { cmd.PrintErrln(err) 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/errors/cdc_errors.go b/pkg/errors/cdc_errors.go index 637887bc0f9..b83791b0169 100644 --- a/pkg/errors/cdc_errors.go +++ b/pkg/errors/cdc_errors.go @@ -54,6 +54,10 @@ var ( "cannot find mysql.tidb_ddl_job schema", errors.RFCCodeText("CDC:ErrDDLSchemaNotFound"), ) + ErrDDLUnsupportType = errors.Normalize( + "unsupport ddl type %s, query %s", + errors.RFCCodeText("CDC:ErrDDLUnsupportType"), + ) ErrGRPCDialFailed = errors.Normalize( "grpc dial failed", errors.RFCCodeText("CDC:ErrGRPCDialFailed"), 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.go b/pkg/sink/codec/avro/avro.go index d7e646b1f5d..6ac329a22ea 100644 --- a/pkg/sink/codec/avro/avro.go +++ b/pkg/sink/codec/avro/avro.go @@ -49,7 +49,8 @@ type BatchEncoder struct { } type avroEncodeInput struct { - columns []*model.Column + *model.TableInfo + columns []*model.ColumnData colInfos []rowcodec.ColInfo } @@ -82,11 +83,12 @@ func (a *BatchEncoder) encodeKey(ctx context.Context, topic string, e *model.Row return nil, nil } - keyColumns := &avroEncodeInput{ - columns: cols, - colInfos: colInfos, + keyColumns := avroEncodeInput{ + TableInfo: e.TableInfo, + columns: cols, + colInfos: colInfos, } - avroCodec, header, err := a.getKeySchemaCodec(ctx, topic, &e.TableInfo.TableName, e.TableInfo.Version, keyColumns) + avroCodec, header, err := a.getKeySchemaCodec(ctx, topic, e.TableInfo.TableName, e.TableInfo.Version, keyColumns) if err != nil { return nil, errors.Trace(err) } @@ -119,7 +121,7 @@ func topicName2SchemaSubjects(topicName, subjectSuffix string) string { } func (a *BatchEncoder) getValueSchemaCodec( - ctx context.Context, topic string, tableName *model.TableName, tableVersion uint64, input *avroEncodeInput, + ctx context.Context, topic string, tableName model.TableName, tableVersion uint64, input avroEncodeInput, ) (*goavro.Codec, []byte, error) { schemaGen := func() (string, error) { schema, err := a.value2AvroSchema(tableName, input) @@ -139,7 +141,7 @@ func (a *BatchEncoder) getValueSchemaCodec( } func (a *BatchEncoder) getKeySchemaCodec( - ctx context.Context, topic string, tableName *model.TableName, tableVersion uint64, keyColumns *avroEncodeInput, + ctx context.Context, topic string, tableName model.TableName, tableVersion uint64, keyColumns avroEncodeInput, ) (*goavro.Codec, []byte, error) { schemaGen := func() (string, error) { schema, err := a.key2AvroSchema(tableName, keyColumns) @@ -163,15 +165,16 @@ func (a *BatchEncoder) encodeValue(ctx context.Context, topic string, e *model.R return nil, nil } - input := &avroEncodeInput{ - columns: e.GetColumns(), - colInfos: e.TableInfo.GetColInfosForRowChangedEvent(), + input := avroEncodeInput{ + TableInfo: e.TableInfo, + columns: e.Columns, + colInfos: e.TableInfo.GetColInfosForRowChangedEvent(), } if len(input.columns) == 0 { return nil, nil } - avroCodec, header, err := a.getValueSchemaCodec(ctx, topic, &e.TableInfo.TableName, e.TableInfo.Version, input) + avroCodec, header, err := a.getValueSchemaCodec(ctx, topic, e.TableInfo.TableName, e.TableInfo.Version, input) if err != nil { return nil, errors.Trace(err) } @@ -386,12 +389,12 @@ var type2TiDBType = map[byte]string{ mysql.TypeTiDBVectorFloat32: "TiDBVECTORFloat32", } -func getTiDBTypeFromColumn(col *model.Column) string { - tt := type2TiDBType[col.Type] - if col.Flag.IsUnsigned() && (tt == "INT" || tt == "BIGINT") { +func getTiDBTypeFromColumn(col model.ColumnDataX) string { + tt := type2TiDBType[col.GetType()] + if col.GetFlag().IsUnsigned() && (tt == "INT" || tt == "BIGINT") { return tt + " UNSIGNED" } - if col.Flag.IsBinary() && tt == "TEXT" { + if col.GetFlag().IsBinary() && tt == "TEXT" { return "BLOB" } return tt @@ -448,61 +451,19 @@ func mysqlTypeFromTiDBType(tidbType string) byte { return result } -const ( - replacementChar = "_" - numberPrefix = "_" -) - -// sanitizeName escapes not permitted chars for avro -// debezium-core/src/main/java/io/debezium/schema/FieldNameSelector.java -// https://avro.apache.org/docs/current/spec.html#names -func sanitizeName(name string) string { - changed := false - var sb strings.Builder - for i, c := range name { - if i == 0 && (c >= '0' && c <= '9') { - sb.WriteString(numberPrefix) - sb.WriteRune(c) - changed = true - } else if !(c == '_' || - ('a' <= c && c <= 'z') || - ('A' <= c && c <= 'Z') || - ('0' <= c && c <= '9')) { - sb.WriteString(replacementChar) - changed = true - } else { - sb.WriteRune(c) - } - } - - sanitizedName := sb.String() - if changed { - log.Warn( - "Name is potentially not safe for serialization, replace it", - zap.String("name", name), - zap.String("replacedName", sanitizedName), - ) - } - return sanitizedName -} - // sanitizeTopic escapes ".", it may have special meanings for sink connectors func sanitizeTopic(name string) string { - return strings.ReplaceAll(name, ".", replacementChar) -} - -// https://github.com/debezium/debezium/blob/9f7ede0e0695f012c6c4e715e96aed85eecf6b5f \ -// /debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/antlr/ \ -// MySqlAntlrDdlParser.java#L374 -func escapeEnumAndSetOptions(option string) string { - option = strings.ReplaceAll(option, ",", "\\,") - option = strings.ReplaceAll(option, "\\'", "'") - option = strings.ReplaceAll(option, "''", "'") - return option + return strings.ReplaceAll(name, ".", "_") } +// | [()*] func getAvroNamespace(namespace string, schema string) string { - return sanitizeName(namespace) + "." + sanitizeName(schema) + ns := common.SanitizeName(namespace) + s := common.SanitizeName(schema) + if s != "" { + return ns + "." + s + } + return ns } type avroSchema struct { @@ -561,30 +522,29 @@ func (a *BatchEncoder) schemaWithExtension( return top } -func (a *BatchEncoder) columns2AvroSchema( - tableName *model.TableName, - input *avroEncodeInput, -) (*avroSchemaTop, error) { +func (a *BatchEncoder) columns2AvroSchema(tableName model.TableName, input avroEncodeInput) (*avroSchemaTop, error) { top := &avroSchemaTop{ Tp: "record", - Name: sanitizeName(tableName.Table), + Name: common.SanitizeName(tableName.Table), Namespace: getAvroNamespace(a.namespace, tableName.Schema), Fields: nil, } - for i, col := range input.columns { - if col == nil { + for _, col := range input.columns { + colx := model.GetColumnDataX(col, input.TableInfo) + if colx.ColumnData == nil { continue } - avroType, err := a.columnToAvroSchema(col, input.colInfos[i].Ft) + + avroType, err := a.columnToAvroSchema(colx) if err != nil { return nil, err } field := make(map[string]interface{}) - field["name"] = sanitizeName(col.Name) + field["name"] = common.SanitizeName(colx.GetName()) - copied := *col - copied.Value = copied.Default - defaultValue, _, err := a.columnToAvroData(&copied, input.colInfos[i].Ft) + copied := colx + copied.ColumnData = &model.ColumnData{ColumnID: colx.ColumnID, Value: colx.GetDefaultValue()} + defaultValue, _, err := a.columnToAvroData(copied) if err != nil { log.Error("fail to get default value for avro schema") return nil, errors.Trace(err) @@ -592,14 +552,14 @@ func (a *BatchEncoder) columns2AvroSchema( // goavro doesn't support set default value for logical type // https://github.com/linkedin/goavro/issues/202 if _, ok := avroType.(avroLogicalTypeSchema); ok { - if col.Flag.IsNullable() { + if colx.GetFlag().IsNullable() { field["type"] = []interface{}{"null", avroType} field["default"] = nil } else { field["type"] = avroType } } else { - if col.Flag.IsNullable() { + if colx.GetFlag().IsNullable() { // https://stackoverflow.com/questions/22938124/avro-field-default-values if defaultValue == nil { field["type"] = []interface{}{"null", avroType} @@ -619,12 +579,9 @@ func (a *BatchEncoder) columns2AvroSchema( return top, nil } -func (a *BatchEncoder) value2AvroSchema( - tableName *model.TableName, - input *avroEncodeInput, -) (string, error) { +func (a *BatchEncoder) value2AvroSchema(tableName model.TableName, input avroEncodeInput) (string, error) { if a.config.EnableRowChecksum { - sort.Sort(input) + sort.Sort(&input) } top, err := a.columns2AvroSchema(tableName, input) @@ -647,10 +604,7 @@ func (a *BatchEncoder) value2AvroSchema( return string(str), nil } -func (a *BatchEncoder) key2AvroSchema( - tableName *model.TableName, - keyColumns *avroEncodeInput, -) (string, error) { +func (a *BatchEncoder) key2AvroSchema(tableName model.TableName, keyColumns avroEncodeInput) (string, error) { top, err := a.columns2AvroSchema(tableName, keyColumns) if err != nil { return "", err @@ -664,24 +618,24 @@ func (a *BatchEncoder) key2AvroSchema( return string(str), nil } -func (a *BatchEncoder) columns2AvroData( - input *avroEncodeInput, -) (map[string]interface{}, error) { +func (a *BatchEncoder) columns2AvroData(input avroEncodeInput) (map[string]interface{}, error) { ret := make(map[string]interface{}, len(input.columns)) - for i, col := range input.columns { - if col == nil { + for _, col := range input.columns { + colx := model.GetColumnDataX(col, input.TableInfo) + if colx.ColumnData == nil { continue } - data, str, err := a.columnToAvroData(col, input.colInfos[i].Ft) + + data, str, err := a.columnToAvroData(colx) if err != nil { return nil, err } // https: //pkg.go.dev/github.com/linkedin/goavro/v2#Union - if col.Flag.IsNullable() { - ret[sanitizeName(col.Name)] = goavro.Union(str, data) + if colx.GetFlag().IsNullable() { + ret[common.SanitizeName(colx.GetName())] = goavro.Union(str, data) } else { - ret[sanitizeName(col.Name)] = data + ret[common.SanitizeName(colx.GetName())] = data } } @@ -689,12 +643,10 @@ func (a *BatchEncoder) columns2AvroData( return ret, nil } -func (a *BatchEncoder) columnToAvroSchema( - col *model.Column, - ft *types.FieldType, -) (interface{}, error) { +func (a *BatchEncoder) columnToAvroSchema(col model.ColumnDataX) (interface{}, error) { tt := getTiDBTypeFromColumn(col) - switch col.Type { + + switch col.GetType() { case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: // BOOL/TINYINT/SMALLINT/MEDIUMINT return avroSchema{ @@ -702,7 +654,7 @@ func (a *BatchEncoder) columnToAvroSchema( Parameters: map[string]string{tidbType: tt}, }, nil case mysql.TypeLong: // INT - if col.Flag.IsUnsigned() { + if col.GetFlag().IsUnsigned() { return avroSchema{ Type: "long", Parameters: map[string]string{tidbType: tt}, @@ -714,7 +666,7 @@ func (a *BatchEncoder) columnToAvroSchema( }, nil case mysql.TypeLonglong: // BIGINT t := "long" - if col.Flag.IsUnsigned() && + if col.GetFlag().IsUnsigned() && a.config.AvroBigintUnsignedHandlingMode == common.BigintUnsignedHandlingModeString { t = "string" } @@ -733,9 +685,9 @@ func (a *BatchEncoder) columnToAvroSchema( Parameters: map[string]string{tidbType: tt}, }, nil case mysql.TypeBit: - displayFlen := ft.GetFlen() + displayFlen := col.GetColumnInfo().FieldType.GetFlen() if displayFlen == -1 { - displayFlen, _ = mysql.GetDefaultFieldLengthAndDecimal(col.Type) + displayFlen, _ = mysql.GetDefaultFieldLengthAndDecimal(col.GetType()) } return avroSchema{ Type: "bytes", @@ -746,6 +698,7 @@ func (a *BatchEncoder) columnToAvroSchema( }, nil case mysql.TypeNewDecimal: if a.config.AvroDecimalHandlingMode == common.DecimalHandlingModePrecise { + ft := col.GetColumnInfo().FieldType defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(ft.GetType()) displayFlen, displayDecimal := ft.GetFlen(), ft.GetDecimal() // length not specified, set it to system type default @@ -780,7 +733,7 @@ func (a *BatchEncoder) columnToAvroSchema( mysql.TypeLongBlob, mysql.TypeBlob: t := "string" - if col.Flag.IsBinary() { + if col.GetFlag().IsBinary() { t = "bytes" } return avroSchema{ @@ -788,9 +741,10 @@ func (a *BatchEncoder) columnToAvroSchema( Parameters: map[string]string{tidbType: tt}, }, nil case mysql.TypeEnum, mysql.TypeSet: - es := make([]string, 0, len(ft.GetElems())) - for _, e := range ft.GetElems() { - e = escapeEnumAndSetOptions(e) + elems := col.GetColumnInfo().FieldType.GetElems() + es := make([]string, 0, len(elems)) + for _, e := range elems { + e = common.EscapeEnumAndSetOptions(e) es = append(es, e) } return avroSchema{ @@ -821,20 +775,17 @@ func (a *BatchEncoder) columnToAvroSchema( Parameters: map[string]string{tidbType: tt}, }, nil default: - log.Error("unknown mysql type", zap.Any("mysqlType", col.Type)) + log.Error("unknown mysql type", zap.Any("mysqlType", col.GetType())) return nil, cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") } } -func (a *BatchEncoder) columnToAvroData( - col *model.Column, - ft *types.FieldType, -) (interface{}, string, error) { +func (a *BatchEncoder) columnToAvroData(col model.ColumnDataX) (interface{}, string, error) { if col.Value == nil { return nil, "null", nil } - switch col.Type { + switch col.GetType() { case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24: if v, ok := col.Value.(string); ok { n, err := strconv.ParseInt(v, 10, 32) @@ -843,7 +794,7 @@ func (a *BatchEncoder) columnToAvroData( } return int32(n), "int", nil } - if col.Flag.IsUnsigned() { + if col.GetFlag().IsUnsigned() { return int32(col.Value.(uint64)), "int", nil } return int32(col.Value.(int64)), "int", nil @@ -853,18 +804,18 @@ func (a *BatchEncoder) columnToAvroData( if err != nil { return nil, "", cerror.WrapError(cerror.ErrAvroEncodeFailed, err) } - if col.Flag.IsUnsigned() { + if col.GetFlag().IsUnsigned() { return n, "long", nil } return int32(n), "int", nil } - if col.Flag.IsUnsigned() { + if col.GetFlag().IsUnsigned() { return int64(col.Value.(uint64)), "long", nil } return int32(col.Value.(int64)), "int", nil case mysql.TypeLonglong: if v, ok := col.Value.(string); ok { - if col.Flag.IsUnsigned() { + if col.GetFlag().IsUnsigned() { if a.config.AvroBigintUnsignedHandlingMode == common.BigintUnsignedHandlingModeString { return v, "string", nil } @@ -880,7 +831,7 @@ func (a *BatchEncoder) columnToAvroData( } return n, "long", nil } - if col.Flag.IsUnsigned() { + if col.GetFlag().IsUnsigned() { if a.config.AvroBigintUnsignedHandlingMode == common.BigintUnsignedHandlingModeLong { return int64(col.Value.(uint64)), "long", nil } @@ -930,7 +881,7 @@ func (a *BatchEncoder) columnToAvroData( mysql.TypeBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: - if col.Flag.IsBinary() { + if col.GetFlag().IsBinary() { if v, ok := col.Value.(string); ok { return []byte(v), "bytes", nil } @@ -944,7 +895,7 @@ func (a *BatchEncoder) columnToAvroData( if v, ok := col.Value.(string); ok { return v, "string", nil } - elements := ft.GetElems() + elements := col.GetColumnInfo().FieldType.GetElems() number := col.Value.(uint64) enumVar, err := types.ParseEnumValue(elements, number) if err != nil { @@ -956,7 +907,7 @@ func (a *BatchEncoder) columnToAvroData( if v, ok := col.Value.(string); ok { return v, "string", nil } - elements := ft.GetElems() + elements := col.GetColumnInfo().FieldType.GetElems() number := col.Value.(uint64) setVar, err := types.ParseSetValue(elements, number) if err != nil { @@ -985,7 +936,7 @@ func (a *BatchEncoder) columnToAvroData( } return nil, "", cerror.ErrAvroEncodeFailed default: - log.Error("unknown mysql type", zap.Any("value", col.Value), zap.Any("mysqlType", col.Type)) + log.Error("unknown mysql type", zap.Any("value", col.Value), zap.Any("mysqlType", col.GetType())) return nil, "", cerror.ErrAvroEncodeFailed.GenWithStack("unknown mysql type") } } diff --git a/pkg/sink/codec/avro/avro_test.go b/pkg/sink/codec/avro/avro_test.go index a2efab87ab0..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() } @@ -307,14 +308,14 @@ func TestAvroEnvelope(t *testing.T) { func TestSanitizeName(t *testing.T) { t.Parallel() - require.Equal(t, "normalColumnName123", sanitizeName("normalColumnName123")) + require.Equal(t, "normalColumnName123", common.SanitizeName("normalColumnName123")) require.Equal( t, "_1ColumnNameStartWithNumber", - sanitizeName("1ColumnNameStartWithNumber"), + common.SanitizeName("1ColumnNameStartWithNumber"), ) - require.Equal(t, "A_B", sanitizeName("A.B")) - require.Equal(t, "columnNameWith__", sanitizeName("columnNameWith中文")) + require.Equal(t, "A_B", common.SanitizeName("A.B")) + require.Equal(t, "columnNameWith__", common.SanitizeName("columnNameWith中文")) } func TestGetAvroNamespace(t *testing.T) { @@ -335,6 +336,12 @@ func TestGetAvroNamespace(t *testing.T) { "N_amespace.S_chema", getAvroNamespace("N-amespace", "S.chema"), ) + + require.Equal( + t, + "normalNamespace", + getAvroNamespace("normalNamespace", ""), + ) } func TestArvoAppendRowChangedEventWithCallback(t *testing.T) { diff --git a/pkg/sink/codec/canal/canal_json_decoder.go b/pkg/sink/codec/canal/canal_json_decoder.go index cd1a0a92c98..bc3ebd9544d 100644 --- a/pkg/sink/codec/canal/canal_json_decoder.go +++ b/pkg/sink/codec/canal/canal_json_decoder.go @@ -17,15 +17,18 @@ import ( "bytes" "context" "database/sql" + "encoding/json" "path/filepath" "strconv" "strings" "time" - "github.com/goccy/go-json" "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,10 +40,49 @@ import ( "golang.org/x/text/encoding/charmap" ) +type tableKey struct { + schema string + table string +} + +type bufferedJSONDecoder struct { + buf *bytes.Buffer + decoder *json.Decoder +} + +func newBufferedJSONDecoder() *bufferedJSONDecoder { + buf := new(bytes.Buffer) + decoder := json.NewDecoder(buf) + return &bufferedJSONDecoder{ + buf: buf, + decoder: decoder, + } +} + +// Write writes data to the buffer. +func (b *bufferedJSONDecoder) Write(data []byte) (n int, err error) { + return b.buf.Write(data) +} + +// Decode decodes the buffer into the original message. +func (b *bufferedJSONDecoder) Decode(v interface{}) error { + return b.decoder.Decode(v) +} + +// Len returns the length of the buffer. +func (b *bufferedJSONDecoder) Len() int { + return b.buf.Len() +} + +// Bytes returns the buffer content. +func (b *bufferedJSONDecoder) Bytes() []byte { + return b.buf.Bytes() +} + // batchDecoder decodes the byte into the original message. type batchDecoder struct { - data []byte - msg canalJSONMessageInterface + msg canalJSONMessageInterface + decoder *bufferedJSONDecoder config *common.Config @@ -48,6 +90,8 @@ type batchDecoder struct { upstreamTiDB *sql.DB bytesDecoder *encoding.Decoder + + tableInfoCache map[tableKey]*model.TableInfo } // NewBatchDecoder return a decoder for canal-json @@ -71,11 +115,22 @@ func NewBatchDecoder( GenWithStack("handle-key-only is enabled, but upstream TiDB is not provided") } + var msg canalJSONMessageInterface = &JSONMessage{} + if codecConfig.EnableTiDBExtension { + msg = &canalJSONMessageWithTiDBExtension{ + JSONMessage: &JSONMessage{}, + Extensions: &tidbExtension{}, + } + } + return &batchDecoder{ - config: codecConfig, - storage: externalStorage, - upstreamTiDB: db, - bytesDecoder: charmap.ISO8859_1.NewDecoder(), + config: codecConfig, + msg: msg, + decoder: newBufferedJSONDecoder(), + storage: externalStorage, + upstreamTiDB: db, + bytesDecoder: charmap.ISO8859_1.NewDecoder(), + tableInfoCache: make(map[tableKey]*model.TableInfo), }, nil } @@ -89,51 +144,23 @@ func (b *batchDecoder) AddKeyValue(_, value []byte) error { return errors.Trace(err) } - b.data = value + if _, err = b.decoder.Write(value); err != nil { + return errors.Trace(err) + } return nil } // HasNext implements the RowEventDecoder interface func (b *batchDecoder) HasNext() (model.MessageType, bool, error) { - if b.data == nil { + if b.decoder.Len() == 0 { return model.MessageTypeUnknown, false, nil } - var ( - msg canalJSONMessageInterface = &JSONMessage{} - encodedData []byte - ) - if b.config.EnableTiDBExtension { - msg = &canalJSONMessageWithTiDBExtension{ - JSONMessage: &JSONMessage{}, - Extensions: &tidbExtension{}, - } - } - - if len(b.config.Terminator) > 0 { - idx := bytes.IndexAny(b.data, b.config.Terminator) - if idx >= 0 { - encodedData = b.data[:idx] - b.data = b.data[idx+len(b.config.Terminator):] - } else { - encodedData = b.data - b.data = nil - } - } else { - encodedData = b.data - b.data = nil - } - - if len(encodedData) == 0 { - return model.MessageTypeUnknown, false, nil - } - - if err := json.Unmarshal(encodedData, msg); err != nil { - log.Error("canal-json decoder unmarshal data failed", - zap.Error(err), zap.ByteString("data", encodedData)) + if err := b.decoder.Decode(b.msg); err != nil { + log.Error("canal-json decoder decode failed", + zap.Error(err), zap.ByteString("data", b.decoder.Bytes())) return model.MessageTypeUnknown, false, err } - b.msg = msg return b.msg.messageType(), true, nil } @@ -206,18 +233,15 @@ func (b *batchDecoder) assembleHandleKeyOnlyRowChangedEvent( table = message.Table eventType = message.EventType ) - - handleKeyData := message.getData() - pkNames := make([]string, 0, len(handleKeyData)) - for name := range handleKeyData { - pkNames = append(pkNames, name) + conditions := make(map[string]interface{}, len(message.pkNameSet())) + for name := range message.pkNameSet() { + conditions[name] = message.getData()[name] } - result := &canalJSONMessageWithTiDBExtension{ JSONMessage: &JSONMessage{ Schema: schema, Table: table, - PKNames: pkNames, + PKNames: message.PKNames, EventType: eventType, }, @@ -227,7 +251,7 @@ func (b *batchDecoder) assembleHandleKeyOnlyRowChangedEvent( } switch eventType { case "INSERT": - holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs, schema, table, handleKeyData) + holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs, schema, table, conditions) data, mysqlType, err := b.buildData(holder) if err != nil { return nil, err @@ -235,7 +259,7 @@ func (b *batchDecoder) assembleHandleKeyOnlyRowChangedEvent( result.MySQLType = mysqlType result.Data = []map[string]interface{}{data} case "UPDATE": - holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs, schema, table, handleKeyData) + holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs, schema, table, conditions) data, mysqlType, err := b.buildData(holder) if err != nil { return nil, err @@ -243,14 +267,14 @@ func (b *batchDecoder) assembleHandleKeyOnlyRowChangedEvent( result.MySQLType = mysqlType result.Data = []map[string]interface{}{data} - holder = common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs-1, schema, table, message.getOld()) + holder = common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs-1, schema, table, conditions) old, _, err := b.buildData(holder) if err != nil { return nil, err } result.Old = []map[string]interface{}{old} case "DELETE": - holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs-1, schema, table, handleKeyData) + holder := common.MustSnapshotQuery(ctx, b.upstreamTiDB, commitTs-1, schema, table, conditions) data, mysqlType, err := b.buildData(holder) if err != nil { return nil, err @@ -263,6 +287,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,11 +352,10 @@ func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { } } - result, err := canalJSONMessage2RowChange(b.msg) + result, err := b.canalJSONMessage2RowChange() if err != nil { return nil, err } - b.msg = nil return result, nil } @@ -299,7 +368,16 @@ func (b *batchDecoder) NextDDLEvent() (*model.DDLEvent, error) { } result := canalJSONMessage2DDLEvent(b.msg) - b.msg = nil + 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) + } return result, nil } @@ -318,6 +396,5 @@ func (b *batchDecoder) NextResolvedEvent() (uint64, error) { return 0, cerror.ErrCanalDecodeFailed. GenWithStack("MessageTypeResolved tidb extension not found") } - b.msg = nil return withExtensionEvent.Extensions.WatermarkTs, nil } diff --git a/pkg/sink/codec/canal/canal_json_message.go b/pkg/sink/codec/canal/canal_json_message.go index c4ce63f6ddb..bd1c2619c40 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" @@ -37,6 +37,9 @@ type canalJSONMessageInterface interface { getSchema() *string getTable() *string getCommitTs() uint64 + getPhysicalTableID() int64 + getTableID() int64 + isPartition() bool getQuery() string getOld() map[string]interface{} getData() map[string]interface{} @@ -84,6 +87,18 @@ func (c *JSONMessage) getCommitTs() uint64 { return 0 } +func (c *JSONMessage) getTableID() int64 { + return 0 +} + +func (c *JSONMessage) getPhysicalTableID() int64 { + return 0 +} + +func (c *JSONMessage) isPartition() bool { + return false +} + func (c *JSONMessage) getQuery() string { return c.Query } @@ -136,6 +151,8 @@ func (c *JSONMessage) pkNameSet() map[string]struct{} { type tidbExtension struct { CommitTs uint64 `json:"commitTs,omitempty"` + TableID int64 `json:"tableId,omitempty"` + PhysicalTableID int64 `json:"partitionId,omitempty"` WatermarkTs uint64 `json:"watermarkTs,omitempty"` OnlyHandleKey bool `json:"onlyHandleKey,omitempty"` ClaimCheckLocation string `json:"claimCheckLocation,omitempty"` @@ -154,36 +171,86 @@ func (c *canalJSONMessageWithTiDBExtension) getCommitTs() uint64 { return c.Extensions.CommitTs } -func canalJSONMessage2RowChange(msg canalJSONMessageInterface) (*model.RowChangedEvent, error) { +func (c *canalJSONMessageWithTiDBExtension) getTableID() int64 { + return c.Extensions.TableID +} + +func (c *canalJSONMessageWithTiDBExtension) getPhysicalTableID() int64 { + if c.Extensions.PhysicalTableID != 0 { + return c.Extensions.PhysicalTableID + } + return c.Extensions.TableID +} + +func (c *canalJSONMessageWithTiDBExtension) isPartition() bool { + return c.Extensions.PhysicalTableID != 0 +} + +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() + result.PhysicalTableID = msg.getPhysicalTableID() 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 } + result.TableInfo.TableName.IsPartition = msg.isPartition() + result.TableInfo.TableName.TableID = msg.getTableID() // 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 +259,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.go b/pkg/sink/codec/canal/canal_json_row_event_encoder.go index e1efefb3f32..b5ad2d5e0db 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder.go @@ -255,6 +255,18 @@ func newJSONMessageForDML( out.RawByte('{') out.RawString("\"commitTs\":") out.Uint64(e.CommitTs) + out.RawByte(',') + + // the logical table id + out.RawString("\"tableId\":") + out.Int64(e.TableInfo.ID) + + // the physical table id + if e.TableInfo.IsPartitionTable() { + out.RawByte(',') + out.RawString("\"partitionId\":") + out.Int64(e.GetTableID()) + } // only send handle key may happen in 2 cases: // 1. delete event, and set only handle key config. no need to encode `onlyHandleKey` field 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..99233fd658f 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 @@ -45,9 +45,8 @@ func TestDMLE2E(t *testing.T) { _, insertEvent, updateEvent, deleteEvent := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) ctx := context.Background() - + codecConfig := common.NewConfig(config.ProtocolCanalJSON) for _, enableTiDBExtension := range []bool{true, false} { - codecConfig := common.NewConfig(config.ProtocolCanalJSON) codecConfig.EnableTiDBExtension = enableTiDBExtension builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) require.NoError(t, err) @@ -76,6 +75,8 @@ func TestDMLE2E(t *testing.T) { require.True(t, decodedEvent.IsInsert()) if enableTiDBExtension { require.Equal(t, insertEvent.CommitTs, decodedEvent.CommitTs) + require.Equal(t, insertEvent.GetTableID(), decodedEvent.GetTableID()) + require.Equal(t, insertEvent.TableInfo.IsPartitionTable(), decodedEvent.TableInfo.IsPartitionTable()) } require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedEvent.TableInfo.GetSchemaName()) require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedEvent.TableInfo.GetTableName()) @@ -113,6 +114,11 @@ func TestDMLE2E(t *testing.T) { decodedEvent, err = decoder.NextRowChangedEvent() require.NoError(t, err) require.True(t, decodedEvent.IsUpdate()) + if enableTiDBExtension { + require.Equal(t, updateEvent.CommitTs, decodedEvent.CommitTs) + require.Equal(t, updateEvent.GetTableID(), decodedEvent.GetTableID()) + require.Equal(t, updateEvent.TableInfo.IsPartitionTable(), decodedEvent.TableInfo.IsPartitionTable()) + } err = encoder.AppendRowChangedEvent(ctx, "", deleteEvent, func() {}) require.NoError(t, err) @@ -129,6 +135,11 @@ func TestDMLE2E(t *testing.T) { decodedEvent, err = decoder.NextRowChangedEvent() require.NoError(t, err) require.True(t, decodedEvent.IsDelete()) + if enableTiDBExtension { + require.Equal(t, deleteEvent.CommitTs, decodedEvent.CommitTs) + require.Equal(t, deleteEvent.GetTableID(), decodedEvent.GetTableID()) + require.Equal(t, deleteEvent.TableInfo.IsPartitionTable(), decodedEvent.TableInfo.IsPartitionTable()) + } } } @@ -249,6 +260,8 @@ func TestCanalJSONClaimCheckE2E(t *testing.T) { require.NoError(t, err, rawValue) require.Equal(t, insertEvent.CommitTs, decodedLargeEvent.CommitTs) + require.Equal(t, insertEvent.GetTableID(), decodedLargeEvent.GetTableID()) + require.Equal(t, insertEvent.TableInfo.IsPartitionTable(), decodedLargeEvent.TableInfo.IsPartitionTable()) require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedLargeEvent.TableInfo.GetSchemaName()) require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedLargeEvent.TableInfo.GetTableName()) require.Nil(t, nil, decodedLargeEvent.PreColumns) @@ -516,6 +529,7 @@ func TestDDLEventWithExtension(t *testing.T) { require.NoError(t, err) require.Equal(t, ddlEvent.Query, decodedDDL.Query) require.Equal(t, ddlEvent.CommitTs, decodedDDL.CommitTs) + require.Equal(t, ddlEvent.TableInfo.IsPartitionTable(), decodedDDL.TableInfo.IsPartitionTable()) require.Equal(t, ddlEvent.TableInfo.TableName.Schema, decodedDDL.TableInfo.TableName.Schema) require.Equal(t, ddlEvent.TableInfo.TableName.Table, decodedDDL.TableInfo.TableName.Table) } @@ -613,26 +627,29 @@ func TestMaxMessageBytes(t *testing.T) { ctx := context.Background() topic := "" - // the test message length is smaller than max-message-bytes - maxMessageBytes := 300 - codecConfig := common.NewConfig(config.ProtocolCanalJSON).WithMaxMessageBytes(maxMessageBytes) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + for _, enableTiDBExtension := range []bool{true, false} { + codecConfig.EnableTiDBExtension = enableTiDBExtension + // the test message length is smaller than max-message-bytes + codecConfig.WithMaxMessageBytes(300) - builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) - require.NoError(t, err) - encoder := builder.Build() + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() - err = encoder.AppendRowChangedEvent(ctx, topic, row, nil) - require.NoError(t, err) + err = encoder.AppendRowChangedEvent(ctx, topic, row, nil) + require.NoError(t, err) - // the test message length is larger than max-message-bytes - codecConfig = codecConfig.WithMaxMessageBytes(100) + // the test message length is larger than max-message-bytes + codecConfig = codecConfig.WithMaxMessageBytes(100) - builder, err = NewJSONRowEventEncoderBuilder(ctx, codecConfig) - require.NoError(t, err) + builder, err = NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) - encoder = builder.Build() - err = encoder.AppendRowChangedEvent(ctx, topic, row, nil) - require.Error(t, err, cerror.ErrMessageTooLarge) + encoder = builder.Build() + err = encoder.AppendRowChangedEvent(ctx, topic, row, nil) + require.Error(t, err, cerror.ErrMessageTooLarge) + } } func TestCanalJSONContentCompatibleE2E(t *testing.T) { @@ -640,205 +657,129 @@ 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() {}) - require.NoError(t, err) - - message := encoder.Build()[0] - decoder, err := NewBatchDecoder(ctx, codecConfig, nil) require.NoError(t, err) - 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, 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) - } + _, insertEvent, updateEvent, deleteEvent := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) + events := []*model.RowChangedEvent{ + insertEvent, + updateEvent, + deleteEvent, } -} -func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { - _, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) - ctx := context.Background() + for _, event := range events { + err = encoder.AppendRowChangedEvent(ctx, "", event, func() {}) + require.NoError(t, err) - for _, encodeEnable := range []bool{false, true} { - encodeConfig := common.NewConfig(config.ProtocolCanalJSON) - encodeConfig.EnableTiDBExtension = encodeEnable - encodeConfig.Terminator = config.CRLF + message := encoder.Build()[0] - builder, err := NewJSONRowEventEncoderBuilder(ctx, encodeConfig) + err = decoder.AddKeyValue(message.Key, message.Value) require.NoError(t, err) - encoder := builder.Build() - err = encoder.AppendRowChangedEvent(ctx, "", insertEvent, nil) + messageType, hasNext, err := decoder.HasNext() require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, messageType, model.MessageTypeRow) - messages := encoder.Build() - require.Equal(t, 1, len(messages)) - msg := messages[0] - - for _, decodeEnable := range []bool{false, true} { - decodeConfig := common.NewConfig(config.ProtocolCanalJSON) - decodeConfig.EnableTiDBExtension = decodeEnable - decoder, err := NewBatchDecoder(ctx, decodeConfig, nil) - require.NoError(t, err) - err = decoder.AddKeyValue(msg.Key, msg.Value) - require.NoError(t, err) - - ty, hasNext, err := decoder.HasNext() - require.NoError(t, err) - require.True(t, hasNext) - require.Equal(t, model.MessageTypeRow, ty) - - decodedEvent, err := decoder.NextRowChangedEvent() - require.NoError(t, err) + 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()) - if encodeEnable && decodeEnable { - require.Equal(t, insertEvent.CommitTs, decodedEvent.CommitTs) + 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) } - require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedEvent.TableInfo.GetSchemaName()) - require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedEvent.TableInfo.GetTableName()) + } - decodedColumns := make(map[string]*model.ColumnData, len(decodedEvent.Columns)) - for _, column := range decodedEvent.Columns { - colName := decodedEvent.TableInfo.ForceGetColumnName(column.ColumnID) - decodedColumns[colName] = column - } - for _, col := range insertEvent.Columns { - colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID) - decoded, ok := decodedColumns[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) } - - _, hasNext, _ = decoder.HasNext() - require.False(t, hasNext) - - decodedEvent, err = decoder.NextRowChangedEvent() - require.Error(t, err) - require.Nil(t, decodedEvent) } } } -func TestNewCanalJSONBatchDecoder4DDLMessage(t *testing.T) { +func TestE2EPartitionTable(t *testing.T) { helper := entry.NewSchemaTestHelper(t) defer helper.Close() - sql := `create table test.person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))` - ddlEvent := helper.DDL2Event(sql) - ctx := context.Background() - for _, encodeEnable := range []bool{false, true} { - codecConfig := common.NewConfig(config.ProtocolCanalJSON) - codecConfig.EnableTiDBExtension = encodeEnable - - builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) - require.NoError(t, err) - encoder := builder.Build() - - result, err := encoder.EncodeDDLEvent(ddlEvent) - require.NoError(t, err) - require.NotNil(t, result) + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.EnableTiDBExtension = true - for _, decodeEnable := range []bool{false, true} { - codecConfig := common.NewConfig(config.ProtocolCanalJSON) - codecConfig.EnableTiDBExtension = decodeEnable - decoder, err := NewBatchDecoder(ctx, codecConfig, nil) - require.NoError(t, err) - err = decoder.AddKeyValue(nil, result.Value) - require.NoError(t, err) + builder, err := NewJSONRowEventEncoderBuilder(ctx, codecConfig) + require.NoError(t, err) + encoder := builder.Build() - ty, hasNext, err := decoder.HasNext() - require.Nil(t, err) - require.True(t, hasNext) - require.Equal(t, model.MessageTypeDDL, ty) + decoder, err := NewBatchDecoder(ctx, codecConfig, nil) + require.NoError(t, err) - consumed, err := decoder.NextDDLEvent() - require.Nil(t, err) + helper.Tk().MustExec("use test") - if encodeEnable && decodeEnable { - require.Equal(t, ddlEvent.CommitTs, consumed.CommitTs) - } else { - require.Equal(t, uint64(0), consumed.CommitTs) - } + 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) - require.Equal(t, ddlEvent.TableInfo.TableName.Schema, consumed.TableInfo.TableName.Schema) - require.Equal(t, ddlEvent.TableInfo.TableName.Table, consumed.TableInfo.TableName.Table) - require.Equal(t, ddlEvent.Query, consumed.Query) + insertEvent := helper.DML2Event(`insert into test.t values (1, 1)`, "test", "t", "p0") + require.NotNil(t, insertEvent) - ty, hasNext, err = decoder.HasNext() - require.Nil(t, err) - require.False(t, hasNext) - require.Equal(t, model.MessageTypeUnknown, ty) + insertEvent1 := helper.DML2Event(`insert into test.t values (11, 11)`, "test", "t", "p1") + require.NotNil(t, insertEvent1) - consumed, err = decoder.NextDDLEvent() - require.NotNil(t, err) - require.Nil(t, consumed) - } - } -} + insertEvent2 := helper.DML2Event(`insert into test.t values (21, 21)`, "test", "t", "p2") + require.NotNil(t, insertEvent2) -func TestCanalJSONBatchDecoderWithTerminator(t *testing.T) { - encodedValue := `{"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"INSERT","es":1668067205238,"ts":1668067206650,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2014-06-04","LastName":"Smith","OfficeLocation":"New York","id":"101"}],"old":null} -{"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"UPDATE","es":1668067229137,"ts":1668067230720,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2015-10-08","LastName":"Smith","OfficeLocation":"Los Angeles","id":"101"}],"old":[{"FirstName":"Bob","HireDate":"2014-06-04","LastName":"Smith","OfficeLocation":"New York","id":"101"}]} -{"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"DELETE","es":1668067230388,"ts":1668067231725,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2015-10-08","LastName":"Smith","OfficeLocation":"Los Angeles","id":"101"}],"old":null}` - ctx := context.Background() - codecConfig := common.NewConfig(config.ProtocolCanalJSON) - codecConfig.Terminator = "\n" - decoder, err := NewBatchDecoder(ctx, codecConfig, nil) - require.NoError(t, err) + events := []*model.RowChangedEvent{insertEvent, insertEvent1, insertEvent2} - err = decoder.AddKeyValue(nil, []byte(encodedValue)) - require.NoError(t, err) + for _, event := range events { + err = encoder.AppendRowChangedEvent(ctx, "", event, nil) + require.NoError(t, err) + message := encoder.Build()[0] - cnt := 0 - for { + err = decoder.AddKeyValue(message.Key, message.Value) + require.NoError(t, err) tp, hasNext, err := decoder.HasNext() - if !hasNext { - break - } require.NoError(t, err) + require.True(t, hasNext) require.Equal(t, model.MessageTypeRow, tp) - cnt++ - event, err := decoder.NextRowChangedEvent() + + decodedEvent, err := decoder.NextRowChangedEvent() require.NoError(t, err) - require.NotNil(t, event) + require.Equal(t, decodedEvent.GetTableID(), event.GetTableID()) + require.Equal(t, decodedEvent.TableInfo.TableName.TableID, event.TableInfo.TableName.TableID) + require.Equal(t, decodedEvent.TableInfo.IsPartitionTable(), event.TableInfo.IsPartitionTable()) } - require.Equal(t, 3, cnt) } diff --git a/pkg/sink/codec/canal/canal_json_txn_event_decoder.go b/pkg/sink/codec/canal/canal_json_txn_event_decoder.go new file mode 100644 index 00000000000..de537720107 --- /dev/null +++ b/pkg/sink/codec/canal/canal_json_txn_event_decoder.go @@ -0,0 +1,173 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package canal + +import ( + "bytes" + "encoding/json" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + canal "github.com/pingcap/tiflow/proto/canal" + "go.uber.org/zap" +) + +type canalJSONTxnEventDecoder struct { + data []byte + + config *common.Config + msg canalJSONMessageInterface +} + +// NewCanalJSONTxnEventDecoder return a new CanalJSONTxnEventDecoder. +func NewCanalJSONTxnEventDecoder( + codecConfig *common.Config, +) *canalJSONTxnEventDecoder { + return &canalJSONTxnEventDecoder{ + config: codecConfig, + } +} + +// AddKeyValue set the key value to the decoder +func (d *canalJSONTxnEventDecoder) AddKeyValue(_, value []byte) error { + value, err := common.Decompress(d.config.LargeMessageHandle.LargeMessageHandleCompression, value) + if err != nil { + log.Error("decompress data failed", + zap.String("compression", d.config.LargeMessageHandle.LargeMessageHandleCompression), + zap.Error(err)) + + return errors.Trace(err) + } + d.data = value + return nil +} + +// HasNext return true if there is any event can be returned. +func (d *canalJSONTxnEventDecoder) HasNext() (model.MessageType, bool, error) { + if d.data == nil { + return model.MessageTypeUnknown, false, nil + } + var ( + msg canalJSONMessageInterface = &JSONMessage{} + encodedData []byte + ) + + if d.config.EnableTiDBExtension { + msg = &canalJSONMessageWithTiDBExtension{ + JSONMessage: &JSONMessage{}, + Extensions: &tidbExtension{}, + } + } + + idx := bytes.IndexAny(d.data, d.config.Terminator) + if idx >= 0 { + encodedData = d.data[:idx] + d.data = d.data[idx+len(d.config.Terminator):] + } else { + encodedData = d.data + d.data = nil + } + + if len(encodedData) == 0 { + return model.MessageTypeUnknown, false, nil + } + + if err := json.Unmarshal(encodedData, msg); err != nil { + log.Error("canal-json decoder unmarshal data failed", + zap.Error(err), zap.ByteString("data", encodedData)) + return model.MessageTypeUnknown, false, err + } + d.msg = msg + return d.msg.messageType(), true, nil +} + +// NextRowChangedEvent implements the RowEventDecoder interface +// `HasNext` should be called before this. +func (d *canalJSONTxnEventDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { + if d.msg == nil || d.msg.messageType() != model.MessageTypeRow { + return nil, cerror.ErrCanalDecodeFailed. + GenWithStack("not found row changed event message") + } + result, err := d.canalJSONMessage2RowChange() + if err != nil { + return nil, err + } + d.msg = nil + return result, nil +} + +func (d *canalJSONTxnEventDecoder) canalJSONMessage2RowChange() (*model.RowChangedEvent, error) { + msg := d.msg + result := new(model.RowChangedEvent) + result.TableInfo = newTableInfo(msg) + result.CommitTs = msg.getCommitTs() + result.PhysicalTableID = msg.getPhysicalTableID() + + 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` + 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` + 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, 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 result.Columns { + if j < len(preCols) && col.ColumnID == preCols[j].ColumnID { + newPreCols = append(newPreCols, preCols[j]) + j += 1 + } else { + newPreCols = append(newPreCols, col) + } + } + preCols = newPreCols + } + 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 +} + +// NextResolvedEvent implements the RowEventDecoder interface +func (d *canalJSONTxnEventDecoder) NextResolvedEvent() (uint64, error) { + return 0, nil +} + +// NextDDLEvent implements the RowEventDecoder interface +func (d *canalJSONTxnEventDecoder) NextDDLEvent() (*model.DDLEvent, error) { + return nil, nil +} diff --git a/pkg/sink/codec/canal/canal_json_txn_event_decoder_test.go b/pkg/sink/codec/canal/canal_json_txn_event_decoder_test.go new file mode 100644 index 00000000000..3f222082cfd --- /dev/null +++ b/pkg/sink/codec/canal/canal_json_txn_event_decoder_test.go @@ -0,0 +1,124 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package canal + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/sink/codec/utils" + "github.com/stretchr/testify/require" +) + +func TestNewCanalJSONBatchDecoder4RowMessage(t *testing.T) { + _, insertEvent, _, _ := utils.NewLargeEvent4Test(t, config.GetDefaultReplicaConfig()) + ctx := context.Background() + + for _, encodeEnable := range []bool{true, false} { + encodeConfig := common.NewConfig(config.ProtocolCanalJSON) + encodeConfig.EnableTiDBExtension = encodeEnable + encodeConfig.Terminator = config.CRLF + + builder, err := NewJSONRowEventEncoderBuilder(ctx, encodeConfig) + require.NoError(t, err) + encoder := builder.Build() + + err = encoder.AppendRowChangedEvent(ctx, "", insertEvent, nil) + require.NoError(t, err) + + messages := encoder.Build() + require.Equal(t, 1, len(messages)) + msg := messages[0] + + for _, decodeEnable := range []bool{true, false} { + decodeConfig := common.NewConfig(config.ProtocolCanalJSON) + decodeConfig.EnableTiDBExtension = decodeEnable + + decoder := NewCanalJSONTxnEventDecoder(decodeConfig) + err = decoder.AddKeyValue(msg.Key, msg.Value) + require.NoError(t, err) + + ty, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, model.MessageTypeRow, ty) + + decodedEvent, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + + if encodeEnable && decodeEnable { + require.Equal(t, insertEvent.CommitTs, decodedEvent.CommitTs) + require.Equal(t, insertEvent.GetTableID(), decodedEvent.GetTableID()) + require.Equal(t, insertEvent.TableInfo.IsPartitionTable(), decodedEvent.TableInfo.IsPartitionTable()) + } + require.Equal(t, insertEvent.TableInfo.GetSchemaName(), decodedEvent.TableInfo.GetSchemaName()) + require.Equal(t, insertEvent.TableInfo.GetTableName(), decodedEvent.TableInfo.GetTableName()) + + decodedColumns := make(map[string]*model.ColumnData, len(decodedEvent.Columns)) + for _, column := range decodedEvent.Columns { + colName := decodedEvent.TableInfo.ForceGetColumnName(column.ColumnID) + decodedColumns[colName] = column + } + for _, col := range insertEvent.Columns { + colName := insertEvent.TableInfo.ForceGetColumnName(col.ColumnID) + decoded, ok := decodedColumns[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) + } + } + + _, hasNext, _ = decoder.HasNext() + require.False(t, hasNext) + + decodedEvent, err = decoder.NextRowChangedEvent() + require.Error(t, err) + require.Nil(t, decodedEvent) + } + } +} + +func TestCanalJSONBatchDecoderWithTerminator(t *testing.T) { + encodedValue := `{"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"INSERT","es":1668067205238,"ts":1668067206650,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2014-06-04","LastName":"Smith","OfficeLocation":"New York","id":"101"}],"old":null} +{"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"UPDATE","es":1668067229137,"ts":1668067230720,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2015-10-08","LastName":"Smith","OfficeLocation":"Los Angeles","id":"101"}],"old":[{"FirstName":"Bob","HireDate":"2014-06-04","LastName":"Smith","OfficeLocation":"New York","id":"101"}]} +{"id":0,"database":"test","table":"employee","pkNames":["id"],"isDdl":false,"type":"DELETE","es":1668067230388,"ts":1668067231725,"sql":"","sqlType":{"FirstName":12,"HireDate":91,"LastName":12,"OfficeLocation":12,"id":4},"mysqlType":{"FirstName":"varchar","HireDate":"date","LastName":"varchar","OfficeLocation":"varchar","id":"int"},"data":[{"FirstName":"Bob","HireDate":"2015-10-08","LastName":"Smith","OfficeLocation":"Los Angeles","id":"101"}],"old":null}` + codecConfig := common.NewConfig(config.ProtocolCanalJSON) + codecConfig.Terminator = "\n" + decoder := NewCanalJSONTxnEventDecoder(codecConfig) + + err := decoder.AddKeyValue(nil, []byte(encodedValue)) + require.NoError(t, err) + + cnt := 0 + for { + tp, hasNext, err := decoder.HasNext() + if !hasNext { + break + } + require.NoError(t, err) + require.Equal(t, model.MessageTypeRow, tp) + cnt++ + event, err := decoder.NextRowChangedEvent() + require.NoError(t, err) + require.NotNil(t, event) + } + require.Equal(t, 3, cnt) +} 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/common/helper.go b/pkg/sink/codec/common/helper.go index 7da4dbcabbb..3257fee5a5c 100644 --- a/pkg/sink/codec/common/helper.go +++ b/pkg/sink/codec/common/helper.go @@ -18,6 +18,8 @@ import ( "database/sql" "fmt" "math" + "strings" + "unsafe" "github.com/go-sql-driver/mysql" "github.com/pingcap/log" @@ -292,3 +294,96 @@ func trimLeadingZeroBytes(bytes []byte) []byte { } return bytes[pos:] } + +const ( + replacementChar = "_" + numberPrefix = 'x' +) + +// EscapeEnumAndSetOptions escapes ",", "\" and "”" +// https://github.com/debezium/debezium/blob/9f7ede0e0695f012c6c4e715e96aed85eecf6b5f/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/antlr/MySqlAntlrDdlParser.java#L374 +func EscapeEnumAndSetOptions(option string) string { + option = strings.ReplaceAll(option, ",", "\\,") + option = strings.ReplaceAll(option, "\\'", "'") + option = strings.ReplaceAll(option, "''", "'") + return option +} + +func isValidFirstCharacter(c rune) bool { + return (c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || c == '_' +} + +func isValidNonFirstCharacter(c rune) bool { + return isValidFirstCharacter(c) || (c >= '0' && c <= '9') +} + +func isValidNonFirstCharacterForTopicName(c rune) bool { + return isValidNonFirstCharacter(c) || c == '.' +} + +// SanitizeName escapes not permitted chars +// https://avro.apache.org/docs/1.12.0/specification/#names +// see https://github.com/debezium/debezium/blob/main/debezium-core/src/main/java/io/debezium/schema/SchemaNameAdjuster.java +func SanitizeName(name string) string { + changed := false + var sb strings.Builder + for i, c := range name { + if i == 0 && !isValidFirstCharacter(c) { + sb.WriteString(replacementChar) + if c >= '0' && c <= '9' { + sb.WriteRune(c) + } + changed = true + } else if !isValidNonFirstCharacter(c) { + sb.WriteString(replacementChar) + changed = true + } else { + sb.WriteRune(c) + } + } + + sanitizedName := sb.String() + if changed { + log.Warn( + "Name is potentially not safe for serialization, replace it", + zap.String("name", name), + zap.String("replacedName", sanitizedName), + ) + } + return sanitizedName +} + +// SanitizeTopicName escapes not permitted chars for topic name +// https://github.com/debezium/debezium/blob/main/debezium-api/src/main/java/io/debezium/spi/topic/TopicNamingStrategy.java +func SanitizeTopicName(name string) string { + changed := false + var sb strings.Builder + for _, c := range name { + if !isValidNonFirstCharacterForTopicName(c) { + sb.WriteString(replacementChar) + changed = true + } else { + sb.WriteRune(c) + } + } + + sanitizedName := sb.String() + if changed { + log.Warn( + "Table name sanitize", + zap.String("name", name), + zap.String("replacedName", sanitizedName), + ) + } + return sanitizedName +} + +// UnsafeBytesToString create string from byte slice without copying +func UnsafeBytesToString(b []byte) string { + return *(*string)(unsafe.Pointer(&b)) +} + +// UnsafeStringToBytes create byte slice from string without copying +func UnsafeStringToBytes(s string) []byte { + return *(*[]byte)(unsafe.Pointer(&s)) +} diff --git a/pkg/sink/codec/craft/message_decoder.go b/pkg/sink/codec/craft/message_decoder.go index 2ea6632c301..c104277e827 100644 --- a/pkg/sink/codec/craft/message_decoder.go +++ b/pkg/sink/codec/craft/message_decoder.go @@ -16,7 +16,6 @@ package craft import ( "encoding/binary" "math" - "unsafe" "github.com/pingcap/errors" pmodel "github.com/pingcap/tidb/pkg/meta/model" @@ -24,13 +23,9 @@ import ( "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/sink/codec/common" ) -// / create string from byte slice without copying -func unsafeBytesToString(b []byte) string { - return *(*string)(unsafe.Pointer(&b)) -} - // / Primitive type decoders func decodeUint8(bits []byte) ([]byte, byte, error) { if len(bits) < 1 { @@ -140,7 +135,7 @@ func decodeBytes(bits []byte) ([]byte, []byte, error) { func decodeString(bits []byte) ([]byte, string, error) { bits, bytes, err := decodeBytes(bits) if err == nil { - return bits, unsafeBytesToString(bytes), nil + return bits, common.UnsafeBytesToString(bytes), nil } return bits, "", errors.Trace(err) } @@ -161,7 +156,7 @@ func decodeStringChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte data := allocator.stringSlice(size) for i := 0; i < size; i++ { - data[i] = unsafeBytesToString(newBits[:larray[i]]) + data[i] = common.UnsafeBytesToString(newBits[:larray[i]]) newBits = newBits[larray[i]:] } return newBits, data, nil @@ -185,7 +180,7 @@ func decodeNullableStringChunk(bits []byte, size int, allocator *SliceAllocator) if larray[i] == -1 { continue } - s := unsafeBytesToString(newBits[:larray[i]]) + s := common.UnsafeBytesToString(newBits[:larray[i]]) data[i] = &s newBits = newBits[larray[i]:] } @@ -324,7 +319,7 @@ func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ switch ty { case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeJSON, mysql.TypeNewDecimal: // value type for these mysql types are string - return unsafeBytesToString(bits), nil + return common.UnsafeBytesToString(bits), nil case mysql.TypeEnum, mysql.TypeSet, mysql.TypeBit: // value type for thest mysql types are uint64 _, u64, err := decodeUvarint(bits) diff --git a/pkg/sink/codec/craft/message_encoder.go b/pkg/sink/codec/craft/message_encoder.go index 8f887226bdc..f09c9e6016c 100644 --- a/pkg/sink/codec/craft/message_encoder.go +++ b/pkg/sink/codec/craft/message_encoder.go @@ -16,23 +16,13 @@ package craft import ( "encoding/binary" "math" - "unsafe" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/sink/codec/common" ) -// create byte slice from string without copying -func unsafeStringToBytes(s string) []byte { - return *(*[]byte)(unsafe.Pointer( - &struct { - string - Cap int - }{s, len(s)}, - )) -} - // Primitive type encoders func encodeFloat64(bits []byte, data float64) []byte { v := math.Float64bits(data) @@ -193,7 +183,7 @@ func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagTyp switch ty { case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeJSON, mysql.TypeNewDecimal: // value type for these mysql types are string - return unsafeStringToBytes(value.(string)) + return common.UnsafeStringToBytes(value.(string)) case mysql.TypeEnum, mysql.TypeSet, mysql.TypeBit: // value type for these mysql types are uint64 return encodeUvarint(allocator.byteSlice(binary.MaxVarintLen64)[:0], value.(uint64)) diff --git a/pkg/sink/codec/debezium/codec.go b/pkg/sink/codec/debezium/codec.go index 7c62c66388e..4fd2f6fe699 100644 --- a/pkg/sink/codec/debezium/codec.go +++ b/pkg/sink/codec/debezium/codec.go @@ -15,7 +15,6 @@ package debezium import ( "bytes" - "encoding/binary" "fmt" "io" "strconv" @@ -23,12 +22,13 @@ import ( "time" "github.com/pingcap/log" + timodel "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec/common" + "github.com/pingcap/tiflow/pkg/sink/codec/internal" "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" @@ -53,6 +53,7 @@ func (c *dbzCodec) writeDebeziumFieldValues( colx := model.GetColumnDataX(col, tableInfo) err = c.writeDebeziumFieldValue(writer, colx, colInfos[i].Ft) if err != nil { + log.Error("write Debezium field value meet error", zap.Error(err)) break } } @@ -68,11 +69,26 @@ func (c *dbzCodec) writeDebeziumFieldSchema( switch col.GetType() { case mysql.TypeBit: n := ft.GetFlen() + var v uint64 + var err error + if col.GetDefaultValue() != nil { + val, ok := col.GetDefaultValue().(string) + if !ok { + return + } + v, err = strconv.ParseUint(parseBit(val, n), 2, 64) + if err != nil { + return + } + } if n == 1 { writer.WriteObjectElement(func() { writer.WriteStringField("type", "boolean") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + writer.WriteBoolField("default", v != 0) // bool + } }) } else { writer.WriteObjectElement(func() { @@ -84,6 +100,9 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("length", fmt.Sprintf("%d", n)) }) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + c.writeBinaryField(writer, "default", getBitFromUint64(n, v)) // binary + } }) } @@ -93,6 +112,9 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("type", "string") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } }) case mysql.TypeEnum: @@ -102,9 +124,17 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("name", "io.debezium.data.Enum") writer.WriteIntField("version", 1) writer.WriteObjectField("parameters", func() { - writer.WriteStringField("allowed", strings.Join(ft.GetElems(), ",")) + elems := ft.GetElems() + parameters := make([]string, 0, len(elems)) + for _, ele := range elems { + parameters = append(parameters, common.EscapeEnumAndSetOptions(ele)) + } + writer.WriteStringField("allowed", strings.Join(parameters, ",")) }) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } }) case mysql.TypeSet: @@ -117,13 +147,9 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("allowed", strings.Join(ft.GetElems(), ",")) }) writer.WriteStringField("field", col.GetName()) - }) - - case mysql.TypeNewDecimal: - writer.WriteObjectElement(func() { - writer.WriteStringField("type", "double") - writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) - writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } }) case mysql.TypeDate, mysql.TypeNewDate: @@ -133,6 +159,30 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("name", "io.debezium.time.Date") writer.WriteIntField("version", 1) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + t, err := time.Parse("2006-01-02", v) + if err != nil { + // For example, time may be invalid like 1000-00-00 + // return nil, nil + if mysql.HasNotNullFlag(ft.GetFlag()) { + writer.WriteInt64Field("default", 0) + } + return + } + year := t.Year() + if year < 70 { + // treats "0018" as 2018 + t = t.AddDate(2000, 0, 0) + } else if year < 100 { + // treats "0099" as 1999 + t = t.AddDate(1900, 0, 0) + } + writer.WriteInt64Field("default", t.UTC().Unix()/60/60/24) + } }) case mysql.TypeDatetime: @@ -146,6 +196,41 @@ func (c *dbzCodec) writeDebeziumFieldSchema( } writer.WriteIntField("version", 1) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + if v == "CURRENT_TIMESTAMP" { + writer.WriteInt64Field("default", 0) + return + } + t, err := types.StrToDateTime(types.DefaultStmtNoWarningContext, v, ft.GetDecimal()) + if err != nil { + writer.WriteInt64Field("default", 0) + return + } + gt, err := t.GoTime(time.UTC) + if err != nil { + if mysql.HasNotNullFlag(ft.GetFlag()) { + writer.WriteInt64Field("default", 0) + } + return + } + year := gt.Year() + if year < 70 { + // treats "0018" as 2018 + gt = gt.AddDate(2000, 0, 0) + } else if year < 100 { + // treats "0099" as 1999 + gt = gt.AddDate(1900, 0, 0) + } + if ft.GetDecimal() <= 3 { + writer.WriteInt64Field("default", gt.UnixMilli()) + } else { + writer.WriteInt64Field("default", gt.UnixMicro()) + } + } }) case mysql.TypeTimestamp: @@ -155,6 +240,39 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("name", "io.debezium.time.ZonedTimestamp") writer.WriteIntField("version", 1) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + if v == "CURRENT_TIMESTAMP" { + return + } + t, err := types.StrToDateTime(types.DefaultStmtNoWarningContext, v, ft.GetDecimal()) + if err != nil { + writer.WriteInt64Field("default", 0) + return + } + if t.Compare(types.MinTimestamp) < 0 { + if mysql.HasNotNullFlag(ft.GetFlag()) { + writer.WriteStringField("default", "1970-01-01T00:00:00Z") + } + return + } + gt, err := t.GoTime(time.UTC) + if err != nil { + writer.WriteInt64Field("default", 0) + return + } + str := gt.Format("2006-01-02T15:04:05") + fsp := ft.GetDecimal() + if fsp > 0 { + tmp := fmt.Sprintf(".%06d", gt.Nanosecond()/1000) + str = str + tmp[:1+fsp] + } + str += "Z" + writer.WriteStringField("default", str) + } }) case mysql.TypeDuration: @@ -164,6 +282,17 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("name", "io.debezium.time.MicroTime") writer.WriteIntField("version", 1) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + d, _, _, err := types.StrToDuration(types.DefaultStmtNoWarningContext.WithLocation(c.config.TimeZone), v, ft.GetDecimal()) + if err != nil { + return + } + writer.WriteInt64Field("default", d.Microseconds()) + } }) case mysql.TypeJSON: @@ -173,6 +302,9 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("name", "io.debezium.data.Json") writer.WriteIntField("version", 1) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } }) case mysql.TypeTiny: // TINYINT @@ -180,6 +312,17 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("type", "int16") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } }) case mysql.TypeShort: // SMALLINT @@ -191,6 +334,17 @@ func (c *dbzCodec) writeDebeziumFieldSchema( } writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } }) case mysql.TypeInt24: // MEDIUMINT @@ -198,17 +352,39 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("type", "int32") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } }) case mysql.TypeLong: // INT writer.WriteObjectElement(func() { - if mysql.HasUnsignedFlag(ft.GetFlag()) { + if col.GetFlag().IsUnsigned() { writer.WriteStringField("type", "int64") } else { writer.WriteStringField("type", "int32") } writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } }) case mysql.TypeLonglong: // BIGINT @@ -216,20 +392,59 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("type", "int64") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } }) case mysql.TypeFloat: writer.WriteObjectElement(func() { - writer.WriteStringField("type", "float") + if ft.GetDecimal() != -1 { + writer.WriteStringField("type", "double") + } else { + writer.WriteStringField("type", "float") + } writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } }) - case mysql.TypeDouble: + case mysql.TypeDouble, mysql.TypeNewDecimal: + // https://dev.mysql.com/doc/refman/8.4/en/numeric-types.html + // MySQL also treats REAL as a synonym for DOUBLE PRECISION (a nonstandard variation), unless the REAL_AS_FLOAT SQL mode is enabled. writer.WriteObjectElement(func() { writer.WriteStringField("type", "double") writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + writer.WriteFloat64Field("default", floatV) + } }) case mysql.TypeYear: @@ -239,6 +454,24 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteStringField("name", "io.debezium.time.Year") writer.WriteIntField("version", 1) writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + v, ok := col.GetDefaultValue().(string) + if !ok { + return + } + floatV, err := strconv.ParseFloat(v, 64) + if err != nil { + return + } + if floatV < 70 { + // treats "DEFAULT 1" as 2001 + floatV += 2000 + } else if floatV < 100 { + // treats "DEFAULT 99" as 1999 + floatV += 1900 + } + writer.WriteFloat64Field("default", floatV) + } }) case mysql.TypeTiDBVectorFloat32: @@ -247,7 +480,11 @@ func (c *dbzCodec) writeDebeziumFieldSchema( writer.WriteBoolField("optional", !mysql.HasNotNullFlag(ft.GetFlag())) writer.WriteStringField("name", "io.debezium.data.TiDBVectorFloat32") writer.WriteStringField("field", col.GetName()) + if col.GetDefaultValue() != nil { + writer.WriteAnyField("default", col.GetDefaultValue()) + } }) + default: log.Warn( "meet unsupported field type", @@ -265,172 +502,188 @@ func (c *dbzCodec) writeDebeziumFieldValue( col model.ColumnDataX, ft *types.FieldType, ) error { - if col.Value == nil { + value := getValue(col) + if value == nil { writer.WriteNullField(col.GetName()) return nil } switch col.GetType() { case mysql.TypeBit: - v, ok := col.Value.(uint64) - if !ok { + n := ft.GetFlen() + var v uint64 + switch val := value.(type) { + case uint64: + v = val + case string: + hexValue, err := strconv.ParseUint(parseBit(val, n), 2, 64) + if err != nil { + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type string for bit column %s, error:%s", + col.GetName(), err.Error()) + } + v = hexValue + default: return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for bit column %s", col.Value, col.GetName()) } - // Debezium behavior: // BIT(1) → BOOLEAN // BIT(>1) → BYTES The byte[] contains the bits in little-endian form and is sized to // contain the specified number of bits. - n := ft.GetFlen() if n == 1 { writer.WriteBoolField(col.GetName(), v != 0) return nil } else { - var buf [8]byte - binary.LittleEndian.PutUint64(buf[:], v) - numBytes := n / 8 - if n%8 != 0 { - numBytes += 1 - } - c.writeBinaryField(writer, col.GetName(), buf[:numBytes]) + c.writeBinaryField(writer, col.GetName(), getBitFromUint64(n, v)) return nil } case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: - v, ok := col.Value.([]byte) - if !ok { - return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for string column %s", - col.Value, - col.GetName()) - } - - if col.GetFlag().IsBinary() { - c.writeBinaryField(writer, col.GetName(), v) - return nil - } else { - writer.WriteStringField(col.GetName(), string(hack.String(v))) - return nil + isBinary := col.GetFlag().IsBinary() + switch v := value.(type) { + case []byte: + if !isBinary { + writer.WriteStringField(col.GetName(), common.UnsafeBytesToString(v)) + } else { + c.writeBinaryField(writer, col.GetName(), v) + } + case string: + if isBinary { + c.writeBinaryField(writer, col.GetName(), common.UnsafeStringToBytes(v)) + } + writer.WriteStringField(col.GetName(), v) } + return nil case mysql.TypeEnum: - v, ok := col.Value.(uint64) + v, ok := value.(uint64) if !ok { return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for enum column %s", - col.Value, + value, col.GetName()) } - enumVar, err := types.ParseEnumValue(ft.GetElems(), v) if err != nil { // Invalid enum value inserted in non-strict mode. writer.WriteStringField(col.GetName(), "") return nil } - writer.WriteStringField(col.GetName(), enumVar.Name) return nil case mysql.TypeSet: - v, ok := col.Value.(uint64) + v, ok := value.(uint64) if !ok { return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for set column %s", - col.Value, + value, col.GetName()) } - setVar, err := types.ParseSetValue(ft.GetElems(), v) if err != nil { // Invalid enum value inserted in non-strict mode. writer.WriteStringField(col.GetName(), "") return nil } - writer.WriteStringField(col.GetName(), setVar.Name) return nil case mysql.TypeNewDecimal: - v, ok := col.Value.(string) + v, ok := value.(string) if !ok { return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for decimal column %s", - col.Value, + value, col.GetName()) } - floatV, err := strconv.ParseFloat(v, 64) if err != nil { return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, err) } - writer.WriteFloat64Field(col.GetName(), floatV) return nil case mysql.TypeDate, mysql.TypeNewDate: - v, ok := col.Value.(string) + v, ok := value.(string) if !ok { return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for date column %s", - col.Value, + value, col.GetName()) } - t, err := time.Parse("2006-01-02", v) if err != nil { // For example, time may be invalid like 1000-00-00 // return nil, nil if mysql.HasNotNullFlag(ft.GetFlag()) { writer.WriteInt64Field(col.GetName(), 0) - return nil } else { writer.WriteNullField(col.GetName()) - return nil } + return nil + } + year := t.Year() + if year < 70 { + // treats "0018" as 2018 + t = t.AddDate(2000, 0, 0) + } else if year < 100 { + // treats "0099" as 1999 + t = t.AddDate(1900, 0, 0) } - writer.WriteInt64Field(col.GetName(), t.Unix()/60/60/24) + writer.WriteInt64Field(col.GetName(), t.UTC().Unix()/60/60/24) return nil case mysql.TypeDatetime: // Debezium behavior from doc: // > Such columns are converted into epoch milliseconds or microseconds based on the // > column's precision by using UTC. - - // TODO: For Default Value = CURRENT_TIMESTAMP, the result is incorrect. - v, ok := col.Value.(string) + v, ok := value.(string) if !ok { return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for datetime column %s", - col.Value, + value, col.GetName()) } - - t, err := time.Parse("2006-01-02 15:04:05.999999", v) + if v == "CURRENT_TIMESTAMP" { + writer.WriteInt64Field(col.GetName(), 0) + return nil + } + t, err := types.StrToDateTime(types.DefaultStmtNoWarningContext, v, ft.GetDecimal()) + if err != nil { + return cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + err) + } + gt, err := t.GoTime(time.UTC) if err != nil { - // For example, time may be 1000-00-00 if mysql.HasNotNullFlag(ft.GetFlag()) { writer.WriteInt64Field(col.GetName(), 0) - return nil } else { writer.WriteNullField(col.GetName()) - return nil } + return nil + } + year := gt.Year() + if year < 70 { + // treats "0018" as 2018 + gt = gt.AddDate(2000, 0, 0) + } else if year < 100 { + // treats "0099" as 1999 + gt = gt.AddDate(1900, 0, 0) } - if ft.GetDecimal() <= 3 { - writer.WriteInt64Field(col.GetName(), t.UnixMilli()) - return nil + writer.WriteInt64Field(col.GetName(), gt.UnixMilli()) } else { - writer.WriteInt64Field(col.GetName(), t.UnixMicro()) - return nil + writer.WriteInt64Field(col.GetName(), gt.UnixMicro()) } + return nil case mysql.TypeTimestamp: // Debezium behavior from doc: @@ -442,33 +695,44 @@ func (c *dbzCodec) writeDebeziumFieldValue( // > based on the server (or session's) current time zone. The time zone will be queried from // > the server by default. If this fails, it must be specified explicitly by the database // > connectionTimeZone MySQL configuration option. - v, ok := col.Value.(string) + v, ok := value.(string) if !ok { return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for timestamp column %s", - col.Value, + value, col.GetName()) } - - t, err := time.ParseInLocation("2006-01-02 15:04:05.999999", v, c.config.TimeZone) + if v == "CURRENT_TIMESTAMP" { + writer.WriteNullField(col.GetName()) + return nil + } + t, err := types.StrToDateTime(types.DefaultStmtNoWarningContext.WithLocation(c.config.TimeZone), v, ft.GetDecimal()) if err != nil { - // For example, time may be invalid like 1000-00-00 - if mysql.HasNotNullFlag(ft.GetFlag()) { - t = time.Unix(0, 0) - } else { + return cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + err) + } + if t.Compare(types.MinTimestamp) < 0 { + if col.Value == nil { writer.WriteNullField(col.GetName()) - return nil + } else { + writer.WriteStringField(col.GetName(), "1970-01-01T00:00:00Z") } + return nil } - - str := t.UTC().Format("2006-01-02T15:04:05") + gt, err := t.GoTime(c.config.TimeZone) + if err != nil { + return cerror.WrapError( + cerror.ErrDebeziumEncodeFailed, + err) + } + str := gt.UTC().Format("2006-01-02T15:04:05") fsp := ft.GetDecimal() if fsp > 0 { - tmp := fmt.Sprintf(".%06d", t.Nanosecond()/1000) + tmp := fmt.Sprintf(".%06d", gt.Nanosecond()/1000) str = str + tmp[:1+fsp] } str += "Z" - writer.WriteStringField(col.GetName(), str) return nil @@ -476,51 +740,109 @@ func (c *dbzCodec) writeDebeziumFieldValue( // Debezium behavior from doc: // > Represents the time value in microseconds and does not include // > time zone information. MySQL allows M to be in the range of 0-6. - v, ok := col.Value.(string) + v, ok := value.(string) if !ok { return cerror.ErrDebeziumEncodeFailed.GenWithStack( "unexpected column value type %T for time column %s", - col.Value, + value, col.GetName()) } - - d, _, _, err := types.StrToDuration(types.DefaultStmtNoWarningContext, v, ft.GetDecimal()) + d, _, _, err := types.StrToDuration(types.DefaultStmtNoWarningContext.WithLocation(c.config.TimeZone), v, ft.GetDecimal()) if err != nil { return cerror.WrapError( cerror.ErrDebeziumEncodeFailed, err) } - writer.WriteInt64Field(col.GetName(), d.Microseconds()) return nil - case mysql.TypeLonglong: - if col.GetFlag().IsUnsigned() { - // Handle with BIGINT UNSIGNED. - // Debezium always produce INT64 instead of UINT64 for BIGINT. - v, ok := col.Value.(uint64) - if !ok { + case mysql.TypeLonglong, mysql.TypeLong, mysql.TypeInt24, mysql.TypeShort, mysql.TypeTiny: + // Note: Although Debezium's doc claims to use INT32 for INT, but it + // actually uses INT64. Debezium also uses INT32 for SMALLINT. + isUnsigned := col.GetFlag().IsUnsigned() + maxValue := types.GetMaxValue(ft) + minValue := types.GetMinValue(ft) + switch v := value.(type) { + case uint64: + if !isUnsigned { return cerror.ErrDebeziumEncodeFailed.GenWithStack( - "unexpected column value type %T for unsigned bigint column %s", - col.Value, + "unexpected column value type %T for unsigned int column %s", + value, col.GetName()) } + if ft.GetType() == mysql.TypeLonglong && v == maxValue.GetUint64() || v > maxValue.GetUint64() { + writer.WriteAnyField(col.GetName(), -1) + } else { + writer.WriteInt64Field(col.GetName(), int64(v)) + } + case int64: + if isUnsigned { + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for int column %s", + value, + col.GetName()) + } + if v < minValue.GetInt64() || v > maxValue.GetInt64() { + writer.WriteAnyField(col.GetName(), -1) + } else { + writer.WriteInt64Field(col.GetName(), v) + } + case string: + if isUnsigned { + t, err := strconv.ParseUint(v, 10, 64) + if err != nil { + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type string for unsigned int column %s", + col.GetName()) + } + if ft.GetType() == mysql.TypeLonglong && t == maxValue.GetUint64() || t > maxValue.GetUint64() { + writer.WriteAnyField(col.GetName(), -1) + } else { + writer.WriteInt64Field(col.GetName(), int64(t)) + } + } else { + t, err := strconv.ParseInt(v, 10, 64) + if err != nil { + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type string for int column %s", + col.GetName()) + } + if t < minValue.GetInt64() || t > maxValue.GetInt64() { + writer.WriteAnyField(col.GetName(), -1) + } else { + writer.WriteInt64Field(col.GetName(), t) + } + } + } + return nil - writer.WriteInt64Field(col.GetName(), int64(v)) - return nil + case mysql.TypeDouble, mysql.TypeFloat: + if v, ok := value.(string); ok { + val, err := strconv.ParseFloat(v, 64) + if err != nil { + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type string for int column %s", + col.GetName()) + } + writer.WriteFloat64Field(col.GetName(), val) + } else { + writer.WriteAnyField(col.GetName(), value) } + return nil case mysql.TypeTiDBVectorFloat32: - v := col.Value.(types.VectorFloat32).String() - writer.WriteStringField(col.GetName(), v) + v, ok := value.(types.VectorFloat32) + if !ok { + return cerror.ErrDebeziumEncodeFailed.GenWithStack( + "unexpected column value type %T for unsigned vector column %s", + value, + col.GetName()) + } + writer.WriteStringField(col.GetName(), v.String()) return nil - - // Note: Although Debezium's doc claims to use INT32 for INT, but it - // actually uses INT64. Debezium also uses INT32 for SMALLINT. - // So we only handle with TypeLonglong here. } - writer.WriteAnyField(col.GetName(), col.Value) + writer.WriteAnyField(col.GetName(), value) return nil } @@ -529,7 +851,135 @@ func (c *dbzCodec) writeBinaryField(writer *util.JSONWriter, fieldName string, v writer.WriteBase64StringField(fieldName, value) } -func (c *dbzCodec) EncodeRowChangedEvent( +func (c *dbzCodec) writeSourceSchema(writer *util.JSONWriter) { + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "struct") + writer.WriteArrayField("fields", func() { + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "version") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "connector") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "name") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int64") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "ts_ms") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("name", "io.debezium.data.Enum") + writer.WriteIntField("version", 1) + writer.WriteObjectField("parameters", func() { + writer.WriteStringField("allowed", "true,last,false,incremental") + }) + writer.WriteStringField("default", "false") + writer.WriteStringField("field", "snapshot") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "db") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "sequence") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "table") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int64") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "server_id") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "gtid") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "file") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int64") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "pos") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int32") + writer.WriteBoolField("optional", false) + writer.WriteStringField("field", "row") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "int64") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "thread") + }) + writer.WriteObjectElement(func() { + writer.WriteStringField("type", "string") + writer.WriteBoolField("optional", true) + writer.WriteStringField("field", "query") + }) + }) + writer.WriteBoolField("optional", false) + writer.WriteStringField("name", "io.debezium.connector.mysql.Source") + writer.WriteStringField("field", "source") + }) +} + +// EncodeKey encode RowChangedEvent into key message +func (c *dbzCodec) EncodeKey( + e *model.RowChangedEvent, + dest io.Writer, +) error { + // schema field describes the structure of the primary key, or the unique key if the table does not have a primary key, for the table that was changed. + // see https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-events + colDataXs, colInfos := e.HandleKeyColDataXInfos() + jWriter := util.BorrowJSONWriter(dest) + defer util.ReturnJSONWriter(jWriter) + + var err error + jWriter.WriteObject(func() { + jWriter.WriteObjectField("payload", func() { + for i, col := range colDataXs { + err = c.writeDebeziumFieldValue(jWriter, col, colInfos[i].Ft) + } + }) + if !c.config.DebeziumDisableSchema { + jWriter.WriteObjectField("schema", func() { + jWriter.WriteStringField("type", "struct") + jWriter.WriteStringField("name", + fmt.Sprintf("%s.Key", getSchemaTopicName(c.clusterID, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName()))) + jWriter.WriteBoolField("optional", false) + jWriter.WriteArrayField("fields", func() { + for i, col := range colDataXs { + c.writeDebeziumFieldSchema(jWriter, col, colInfos[i].Ft) + } + }) + }) + } + }) + return err +} + +// EncodeValue encode RowChangedEvent into value message +func (c *dbzCodec) EncodeValue( e *model.RowChangedEvent, dest io.Writer, ) error { @@ -570,7 +1020,6 @@ func (c *dbzCodec) EncodeRowChangedEvent( // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events jWriter.WriteInt64Field("ts_ms", c.nowFunc().UnixMilli()) jWriter.WriteNullField("transaction") - if e.IsInsert() { // op: Mandatory string that describes the type of operation that caused the connector to generate the event. // Valid values are: @@ -610,10 +1059,8 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteObjectField("schema", func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Envelope", - c.clusterID, - e.TableInfo.GetSchemaName(), - e.TableInfo.GetTableName())) + jWriter.WriteStringField("name", + fmt.Sprintf("%s.Envelope", getSchemaTopicName(c.clusterID, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName()))) jWriter.WriteIntField("version", 1) jWriter.WriteArrayField("fields", func() { // schema is the same for `before` and `after`. So we build a new buffer to @@ -635,16 +1082,24 @@ func (c *dbzCodec) EncodeRowChangedEvent( colx := model.GetColumnDataX(col, e.TableInfo) c.writeDebeziumFieldSchema(fieldsWriter, colx, colInfos[i].Ft) } + if e.TableInfo.HasVirtualColumns() { + for _, colInfo := range e.TableInfo.Columns { + if model.IsColCDCVisible(colInfo) { + continue + } + data := &model.ColumnData{ColumnID: colInfo.ID} + colx := model.GetColumnDataX(data, e.TableInfo) + c.writeDebeziumFieldSchema(fieldsWriter, colx, &colInfo.FieldType) + } + } util.ReturnJSONWriter(fieldsWriter) fieldsJSON = fieldsBuf.String() } jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Value", - c.clusterID, - e.TableInfo.GetSchemaName(), - e.TableInfo.GetTableName())) + jWriter.WriteStringField("name", + fmt.Sprintf("%s.Value", getSchemaTopicName(c.clusterID, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName()))) jWriter.WriteStringField("field", "before") jWriter.WriteArrayField("fields", func() { jWriter.WriteRaw(fieldsJSON) @@ -653,115 +1108,543 @@ func (c *dbzCodec) EncodeRowChangedEvent( jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "struct") jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("name", fmt.Sprintf("%s.%s.%s.Value", - c.clusterID, - e.TableInfo.GetSchemaName(), - e.TableInfo.GetTableName())) + jWriter.WriteStringField("name", + fmt.Sprintf("%s.Value", getSchemaTopicName(c.clusterID, e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName()))) jWriter.WriteStringField("field", "after") jWriter.WriteArrayField("fields", func() { jWriter.WriteRaw(fieldsJSON) }) }) + c.writeSourceSchema(jWriter) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("type", "string") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("field", "op") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("type", "int64") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("field", "ts_ms") + }) jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "struct") jWriter.WriteArrayField("fields", func() { jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "string") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "version") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "connector") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "name") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "int64") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "ts_ms") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("name", "io.debezium.data.Enum") - jWriter.WriteIntField("version", 1) - jWriter.WriteObjectField("parameters", func() { - jWriter.WriteStringField("allowed", "true,last,false,incremental") - }) - jWriter.WriteStringField("default", "false") - jWriter.WriteStringField("field", "snapshot") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "db") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "sequence") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "table") + jWriter.WriteStringField("field", "id") }) jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "int64") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "server_id") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "gtid") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "file") + jWriter.WriteStringField("field", "total_order") }) jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "int64") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "pos") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "int32") - jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("field", "row") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "int64") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "thread") - }) - jWriter.WriteObjectElement(func() { - jWriter.WriteStringField("type", "string") - jWriter.WriteBoolField("optional", true) - jWriter.WriteStringField("field", "query") + jWriter.WriteStringField("field", "data_collection_order") }) - // Below are extra TiDB fields - // jWriter.WriteObjectElement(func() { - // jWriter.WriteStringField("type", "int64") - // jWriter.WriteBoolField("optional", false) - // jWriter.WriteStringField("field", "commit_ts") - // }) - // jWriter.WriteObjectElement(func() { - // jWriter.WriteStringField("type", "string") - // jWriter.WriteBoolField("optional", false) - // jWriter.WriteStringField("field", "cluster_id") - // }) }) + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("name", "event.block") + jWriter.WriteIntField("version", 1) + jWriter.WriteStringField("field", "transaction") + }) + }) + }) + } + }) + return err +} + +// EncodeDDLEvent encode DDLEvent into debezium change event +func (c *dbzCodec) EncodeDDLEvent( + e *model.DDLEvent, + keyDest io.Writer, + dest io.Writer, +) error { + keyJWriter := util.BorrowJSONWriter(keyDest) + jWriter := util.BorrowJSONWriter(dest) + defer util.ReturnJSONWriter(keyJWriter) + defer util.ReturnJSONWriter(jWriter) + + commitTime := oracle.GetTimeFromTS(e.CommitTs) + var changeType string + // refer to: https://docs.pingcap.com/tidb/dev/mysql-compatibility#ddl-operations + switch e.Type { + case timodel.ActionCreateSchema, + timodel.ActionCreateTable, + timodel.ActionCreateView: + changeType = "CREATE" + case timodel.ActionAddColumn, + timodel.ActionModifyColumn, + timodel.ActionDropColumn, + timodel.ActionMultiSchemaChange, + timodel.ActionAddTablePartition, + timodel.ActionRemovePartitioning, + timodel.ActionReorganizePartition, + timodel.ActionExchangeTablePartition, + timodel.ActionAlterTablePartitioning, + timodel.ActionTruncateTablePartition, + timodel.ActionDropTablePartition, + timodel.ActionRebaseAutoID, + timodel.ActionSetDefaultValue, + timodel.ActionModifyTableComment, + timodel.ActionModifyTableCharsetAndCollate, + timodel.ActionModifySchemaCharsetAndCollate, + timodel.ActionAddIndex, + timodel.ActionAlterIndexVisibility, + timodel.ActionRenameIndex, + timodel.ActionRenameTable, + timodel.ActionRecoverTable, + timodel.ActionAddPrimaryKey, + timodel.ActionDropPrimaryKey, + timodel.ActionAlterTTLInfo, + timodel.ActionAlterTTLRemove: + changeType = "ALTER" + case timodel.ActionDropSchema, + timodel.ActionDropTable, + timodel.ActionDropIndex, + timodel.ActionDropView: + changeType = "DROP" + default: + return cerror.ErrDDLUnsupportType.GenWithStackByArgs(e.Type, e.Query) + } + + var err error + dbName, tableName := getDBTableName(e) + // message key + keyJWriter.WriteObject(func() { + keyJWriter.WriteObjectField("payload", func() { + if e.Type == timodel.ActionDropTable { + keyJWriter.WriteStringField("databaseName", e.PreTableInfo.GetSchemaName()) + } else { + keyJWriter.WriteStringField("databaseName", dbName) + } + }) + if !c.config.DebeziumDisableSchema { + keyJWriter.WriteObjectField("schema", func() { + keyJWriter.WriteStringField("type", "struct") + keyJWriter.WriteStringField("name", "io.debezium.connector.mysql.SchemaChangeKey") + keyJWriter.WriteBoolField("optional", false) + keyJWriter.WriteIntField("version", 1) + keyJWriter.WriteArrayField("fields", func() { + keyJWriter.WriteObjectElement(func() { + keyJWriter.WriteStringField("field", "databaseName") + keyJWriter.WriteBoolField("optional", false) + keyJWriter.WriteStringField("type", "string") + }) + }) + }) + } + }) + + // message value + jWriter.WriteObject(func() { + jWriter.WriteObjectField("payload", func() { + jWriter.WriteObjectField("source", func() { + jWriter.WriteStringField("version", "2.4.0.Final") + jWriter.WriteStringField("connector", "TiCDC") + jWriter.WriteStringField("name", c.clusterID) + jWriter.WriteInt64Field("ts_ms", commitTime.UnixMilli()) + jWriter.WriteStringField("snapshot", "false") + if e.TableInfo == nil { + jWriter.WriteStringField("db", "") + jWriter.WriteStringField("table", "") + } else { + jWriter.WriteStringField("db", dbName) + jWriter.WriteStringField("table", tableName) + } + jWriter.WriteInt64Field("server_id", 0) + jWriter.WriteNullField("gtid") + jWriter.WriteStringField("file", "") + jWriter.WriteInt64Field("pos", 0) + jWriter.WriteInt64Field("row", 0) + jWriter.WriteInt64Field("thread", 0) + jWriter.WriteNullField("query") + + // The followings are TiDB extended fields + jWriter.WriteUint64Field("commit_ts", e.CommitTs) + jWriter.WriteStringField("cluster_id", c.clusterID) + }) + jWriter.WriteInt64Field("ts_ms", c.nowFunc().UnixMilli()) + + if e.Type == timodel.ActionDropTable { + jWriter.WriteStringField("databaseName", e.PreTableInfo.GetSchemaName()) + } else { + jWriter.WriteStringField("databaseName", dbName) + } + jWriter.WriteNullField("schemaName") + jWriter.WriteStringField("ddl", e.Query) + jWriter.WriteArrayField("tableChanges", func() { + // return early if there is no table changes + if tableName == "" { + return + } + jWriter.WriteObjectElement(func() { + // Describes the kind of change. The value is one of the following: + // CREATE: Table created. + // ALTER: Table modified. + // DROP: Table deleted. + jWriter.WriteStringField("type", changeType) + // In the case of a table rename, this identifier is a concatenation of , table names. + if e.Type == timodel.ActionRenameTable { + jWriter.WriteStringField("id", fmt.Sprintf("\"%s\".\"%s\",\"%s\".\"%s\"", + e.PreTableInfo.GetSchemaName(), + e.PreTableInfo.GetTableName(), + dbName, + tableName)) + } else { + jWriter.WriteStringField("id", fmt.Sprintf("\"%s\".\"%s\"", + dbName, + tableName)) + } + // return early if there is no table info + if e.Type == timodel.ActionDropTable { + jWriter.WriteNullField("table") + return + } + jWriter.WriteObjectField("table", func() { + jWriter.WriteStringField("defaultCharsetName", e.TableInfo.Charset) + jWriter.WriteArrayField("primaryKeyColumnNames", func() { + for _, pk := range e.TableInfo.GetPrimaryKeyColumnNames() { + jWriter.WriteStringElement(pk) + } + }) + jWriter.WriteArrayField("columns", func() { + parseColumns(e.Query, e.TableInfo.Columns) + for pos, col := range e.TableInfo.Columns { + if col.Hidden { + continue + } + jWriter.WriteObjectElement(func() { + flag := col.GetFlag() + jdbcType := internal.MySQLType2JdbcType(col.GetType(), mysql.HasBinaryFlag(flag)) + expression, name := getExpressionAndName(col.FieldType) + jWriter.WriteStringField("name", col.Name.O) + jWriter.WriteIntField("jdbcType", int(jdbcType)) + jWriter.WriteNullField("nativeType") + if col.Comment != "" { + jWriter.WriteStringField("comment", col.Comment) + } else { + jWriter.WriteNullField("comment") + } + if col.DefaultValue == nil { + jWriter.WriteNullField("defaultValueExpression") + } else { + v, ok := col.DefaultValue.(string) + if ok { + if strings.ToUpper(v) == "CURRENT_TIMESTAMP" { + // https://debezium.io/documentation/reference/3.0/connectors/mysql.html#mysql-temporal-types + jWriter.WriteAnyField("defaultValueExpression", "1970-01-01 00:00:00") + } else if v == "" { + jWriter.WriteNullField("defaultValueExpression") + } else if col.DefaultValueBit != nil { + jWriter.WriteStringField("defaultValueExpression", parseBit(v, col.GetFlen())) + } else { + jWriter.WriteStringField("defaultValueExpression", v) + } + } else { + jWriter.WriteAnyField("defaultValueExpression", col.DefaultValue) + } + } + elems := col.GetElems() + if len(elems) != 0 { + // Format is ENUM ('e1', 'e2') or SET ('e1', 'e2') + jWriter.WriteArrayField("enumValues", func() { + for _, ele := range elems { + jWriter.WriteStringElement(fmt.Sprintf("'%s'", ele)) + } + }) + } else { + jWriter.WriteNullField("enumValues") + } + + jWriter.WriteStringField("typeName", name) + jWriter.WriteStringField("typeExpression", expression) + + charsetName := getCharset(col.FieldType) + if charsetName != "" { + jWriter.WriteStringField("charsetName", charsetName) + } else { + jWriter.WriteNullField("charsetName") + } + + length := getLen(col.FieldType) + if length != -1 { + jWriter.WriteIntField("length", length) + } else { + jWriter.WriteNullField("length") + } + + scale := getScale(col.FieldType) + if scale != -1 { + jWriter.WriteFloat64Field("scale", scale) + } else { + jWriter.WriteNullField("scale") + } + jWriter.WriteIntField("position", pos+1) + jWriter.WriteBoolField("optional", !mysql.HasNotNullFlag(flag)) + + autoIncrementFlag := mysql.HasAutoIncrementFlag(flag) + jWriter.WriteBoolField("autoIncremented", autoIncrementFlag) + jWriter.WriteBoolField("generated", autoIncrementFlag) + }) + } + }) + jWriter.WriteNullField("comment") + }) + }) + }) + }) + + if !c.config.DebeziumDisableSchema { + jWriter.WriteObjectField("schema", func() { + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "struct") + jWriter.WriteIntField("version", 1) + jWriter.WriteStringField("name", "io.debezium.connector.mysql.SchemaChangeValue") + jWriter.WriteArrayField("fields", func() { + c.writeSourceSchema(jWriter) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "ts_ms") jWriter.WriteBoolField("optional", false) - jWriter.WriteStringField("name", "io.debezium.connector.mysql.Source") - jWriter.WriteStringField("field", "source") + jWriter.WriteStringField("type", "int64") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "databaseName") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "schemaName") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "ddl") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "tableChanges") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "array") + jWriter.WriteObjectField("items", func() { + jWriter.WriteStringField("name", "io.debezium.connector.schema.Change") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "struct") + jWriter.WriteIntField("version", 1) + jWriter.WriteArrayField("fields", func() { + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "type") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "id") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "table") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "struct") + jWriter.WriteStringField("name", "io.debezium.connector.schema.Table") + jWriter.WriteIntField("version", 1) + jWriter.WriteArrayField("fields", func() { + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "defaultCharsetName") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "primaryKeyColumnNames") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "array") + jWriter.WriteObjectField("items", func() { + jWriter.WriteStringField("type", "string") + jWriter.WriteBoolField("optional", false) + }) + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "columns") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "array") + jWriter.WriteObjectField("items", func() { + jWriter.WriteStringField("name", "io.debezium.connector.schema.Column") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "struct") + jWriter.WriteIntField("version", 1) + jWriter.WriteArrayField("fields", func() { + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "name") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "jdbcType") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "nativeType") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "typeName") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "typeExpression") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "charsetName") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "length") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "scale") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "position") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("type", "int32") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "optional") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "boolean") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "autoIncremented") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "boolean") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "generated") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "boolean") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "comment") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "defaultValueExpression") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "enumValues") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "array") + jWriter.WriteObjectField("items", func() { + jWriter.WriteStringField("type", "string") + jWriter.WriteBoolField("optional", false) + }) + }) + }) + }) + }) + jWriter.WriteObjectElement(func() { + jWriter.WriteStringField("field", "comment") + jWriter.WriteBoolField("optional", true) + jWriter.WriteStringField("type", "string") + }) + }) + }) + }) + }) + }) + }) + }) + } + }) + return err +} + +// EncodeCheckpointEvent encode checkpointTs into debezium change event +func (c *dbzCodec) EncodeCheckpointEvent( + ts uint64, + keyDest io.Writer, + dest io.Writer, +) error { + keyJWriter := util.BorrowJSONWriter(keyDest) + jWriter := util.BorrowJSONWriter(dest) + defer util.ReturnJSONWriter(keyJWriter) + defer util.ReturnJSONWriter(jWriter) + commitTime := oracle.GetTimeFromTS(ts) + var err error + // message key + keyJWriter.WriteObject(func() { + keyJWriter.WriteObjectField("payload", func() {}) + if !c.config.DebeziumDisableSchema { + keyJWriter.WriteObjectField("schema", func() { + keyJWriter.WriteStringField("type", "struct") + keyJWriter.WriteStringField("name", + fmt.Sprintf("%s.%s.Key", common.SanitizeName(c.clusterID), "watermark")) + keyJWriter.WriteBoolField("optional", false) + keyJWriter.WriteArrayField("fields", func() { + }) + }) + } + }) + // message value + jWriter.WriteObject(func() { + jWriter.WriteObjectField("payload", func() { + jWriter.WriteObjectField("source", func() { + jWriter.WriteStringField("version", "2.4.0.Final") + jWriter.WriteStringField("connector", "TiCDC") + jWriter.WriteStringField("name", c.clusterID) + // ts_ms: In the source object, ts_ms indicates the time that the change was made in the database. + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events + jWriter.WriteInt64Field("ts_ms", commitTime.UnixMilli()) + // snapshot field is a string of true,last,false,incremental + jWriter.WriteStringField("snapshot", "false") + jWriter.WriteStringField("db", "") + jWriter.WriteStringField("table", "") + jWriter.WriteInt64Field("server_id", 0) + jWriter.WriteNullField("gtid") + jWriter.WriteStringField("file", "") + jWriter.WriteInt64Field("pos", 0) + jWriter.WriteInt64Field("row", 0) + jWriter.WriteInt64Field("thread", 0) + jWriter.WriteNullField("query") + + // The followings are TiDB extended fields + jWriter.WriteUint64Field("commit_ts", ts) + jWriter.WriteStringField("cluster_id", c.clusterID) + }) + + // ts_ms: displays the time at which the connector processed the event + // https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-create-events + jWriter.WriteInt64Field("ts_ms", c.nowFunc().UnixMilli()) + jWriter.WriteNullField("transaction") + jWriter.WriteStringField("op", "m") + }) + + if !c.config.DebeziumDisableSchema { + jWriter.WriteObjectField("schema", func() { + jWriter.WriteStringField("type", "struct") + jWriter.WriteBoolField("optional", false) + jWriter.WriteStringField("name", + fmt.Sprintf("%s.%s.Envelope", common.SanitizeName(c.clusterID), "watermark")) + jWriter.WriteIntField("version", 1) + jWriter.WriteArrayField("fields", func() { + c.writeSourceSchema(jWriter) jWriter.WriteObjectElement(func() { jWriter.WriteStringField("type", "string") jWriter.WriteBoolField("optional", false) @@ -800,6 +1683,5 @@ func (c *dbzCodec) EncodeRowChangedEvent( }) } }) - return err } diff --git a/pkg/sink/codec/debezium/codec_test.go b/pkg/sink/codec/debezium/codec_test.go index 6e8529164ce..b637163c336 100644 --- a/pkg/sink/codec/debezium/codec_test.go +++ b/pkg/sink/codec/debezium/codec_test.go @@ -18,18 +18,704 @@ import ( "testing" "time" + timodel "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/stretchr/testify/require" "github.com/thanhpk/randstr" ) +func TestDDLEvent(t *testing.T) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + + query := "RENAME TABLE test.table1 to test.table2" + tableInfo := model.BuildTableInfo("test", "table1", []*model.Column{{ + Name: "id", + Type: mysql.TypeLong, + Flag: model.PrimaryKeyFlag | model.HandleKeyFlag, + }}, [][]int{{0}}) + preTableInfo := model.BuildTableInfo("test", "table2", []*model.Column{{ + Name: "id", + Type: mysql.TypeLong, + Flag: model.PrimaryKeyFlag | model.HandleKeyFlag, + }}, [][]int{{0}}) + e := &model.DDLEvent{ + CommitTs: 1, + TableInfo: tableInfo, + PreTableInfo: preTableInfo, + Type: timodel.ActionNone, + } + keyBuf := bytes.NewBuffer(nil) + buf := bytes.NewBuffer(nil) + err := codec.EncodeDDLEvent(e, keyBuf, buf) + require.ErrorIs(t, err, cerror.ErrDDLUnsupportType) + + e = &model.DDLEvent{ + CommitTs: 1, + TableInfo: tableInfo, + PreTableInfo: preTableInfo, + Query: query, + Type: timodel.ActionRenameTable, + } + keyBuf.Reset() + buf.Reset() + codec.config.DebeziumDisableSchema = false + err = codec.EncodeDDLEvent(e, keyBuf, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "databaseName": "test" + }, + "schema": { + "type": "struct", + "name": "io.debezium.connector.mysql.SchemaChangeKey", + "optional": false, + "version": 1, + "fields": [ + { + "field": "databaseName", + "optional": false, + "type": "string" + } + ] + } + }`, keyBuf.String()) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "test", + "table": "table1", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "databaseName": "test", + "schemaName": null, + "ddl": "RENAME TABLE test.table1 to test.table2", + "tableChanges": [ + { + "type": "ALTER", + "id": "\"test\".\"table2\",\"test\".\"table1\"", + "table": { + "defaultCharsetName": "", + "primaryKeyColumnNames": ["id"], + "columns": [ + { + "name": "id", + "jdbcType": 4, + "nativeType": null, + "comment": null, + "defaultValueExpression": null, + "enumValues": null, + "typeName": "INT", + "typeExpression": "INT", + "charsetName": null, + "length": 0, + "scale": null, + "position": 1, + "optional": false, + "autoIncremented": false, + "generated": false + } + ], + "comment": null + } + } + ] + }, + "schema": { + "optional": false, + "type": "struct", + "version": 1, + "name": "io.debezium.connector.mysql.SchemaChangeValue", + "fields": [ + { + "field": "source", + "name": "io.debezium.connector.mysql.Source", + "optional": false, + "type": "struct", + "fields": [ + { + "field": "version", + "optional": false, + "type": "string" + }, + { + "field": "connector", + "optional": false, + "type": "string" + }, + { + "field": "name", + "optional": false, + "type": "string" + }, + { + "field": "ts_ms", + "optional": false, + "type": "int64" + }, + { + "field": "snapshot", + "optional": true, + "type": "string", + "parameters": { + "allowed": "true,last,false,incremental" + }, + "default": "false", + "name": "io.debezium.data.Enum", + "version": 1 + }, + { + "field": "db", + "optional": false, + "type": "string" + }, + { + "field": "sequence", + "optional": true, + "type": "string" + }, + { + "field": "table", + "optional": true, + "type": "string" + }, + { + "field": "server_id", + "optional": false, + "type": "int64" + }, + { + "field": "gtid", + "optional": true, + "type": "string" + }, + { + "field": "file", + "optional": false, + "type": "string" + }, + { + "field": "pos", + "optional": false, + "type": "int64" + }, + { + "field": "row", + "optional": false, + "type": "int32" + }, + { + "field": "thread", + "optional": true, + "type": "int64" + }, + { + "field": "query", + "optional": true, + "type": "string" + } + ] + }, + { + "field": "ts_ms", + "optional": false, + "type": "int64" + }, + { + "field": "databaseName", + "optional": true, + "type": "string" + }, + { + "field": "schemaName", + "optional": true, + "type": "string" + }, + { + "field": "ddl", + "optional": true, + "type": "string" + }, + { + "field": "tableChanges", + "optional": false, + "type": "array", + "items": { + "name": "io.debezium.connector.schema.Change", + "optional": false, + "type": "struct", + "version": 1, + "fields": [ + { + "field": "type", + "optional": false, + "type": "string" + }, + { + "field": "id", + "optional": false, + "type": "string" + }, + { + "field": "table", + "optional": true, + "type": "struct", + "name": "io.debezium.connector.schema.Table", + "version": 1, + "fields": [ + { + "field": "defaultCharsetName", + "optional": true, + "type": "string" + }, + { + "field": "primaryKeyColumnNames", + "optional": true, + "type": "array", + "items": { + "type": "string", + "optional": false + } + }, + { + "field": "columns", + "optional": false, + "type": "array", + "items": { + "name": "io.debezium.connector.schema.Column", + "optional": false, + "type": "struct", + "version": 1, + "fields": [ + { + "field": "name", + "optional": false, + "type": "string" + }, + { + "field": "jdbcType", + "optional": false, + "type": "int32" + }, + { + "field": "nativeType", + "optional": true, + "type": "int32" + }, + { + "field": "typeName", + "optional": false, + "type": "string" + }, + { + "field": "typeExpression", + "optional": true, + "type": "string" + }, + { + "field": "charsetName", + "optional": true, + "type": "string" + }, + { + "field": "length", + "optional": true, + "type": "int32" + }, + { + "field": "scale", + "optional": true, + "type": "int32" + }, + { + "field": "position", + "optional": false, + "type": "int32" + }, + { + "field": "optional", + "optional": true, + "type": "boolean" + }, + { + "field": "autoIncremented", + "optional": true, + "type": "boolean" + }, + { + "field": "generated", + "optional": true, + "type": "boolean" + }, + { + "field": "comment", + "optional": true, + "type": "string" + }, + { + "field": "defaultValueExpression", + "optional": true, + "type": "string" + }, + { + "field": "enumValues", + "optional": true, + "type": "array", + "items": { + "type": "string", + "optional": false + } + } + ] + } + }, + { + "field": "comment", + "optional": true, + "type": "string" + } + ] + } + ] + } + } + ] + } + }`, buf.String()) + + codec.config.DebeziumDisableSchema = true + + query = "CREATE TABLE test.table1" + e = &model.DDLEvent{ + CommitTs: 1, + TableInfo: tableInfo, + Query: query, + Type: timodel.ActionCreateTable, + } + keyBuf.Reset() + buf.Reset() + err = codec.EncodeDDLEvent(e, keyBuf, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "databaseName": "test" + } + }`, keyBuf.String()) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "test", + "table": "table1", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "databaseName": "test", + "schemaName": null, + "ddl": "CREATE TABLE test.table1", + "tableChanges": [ + { + "type": "CREATE", + "id": "\"test\".\"table1\"", + "table": { + "defaultCharsetName": "", + "primaryKeyColumnNames": ["id"], + "columns": [ + { + "name": "id", + "jdbcType": 4, + "nativeType": null, + "comment": null, + "defaultValueExpression": null, + "enumValues": null, + "typeName": "INT", + "typeExpression": "INT", + "charsetName": null, + "length": 0, + "scale": null, + "position": 1, + "optional": false, + "autoIncremented": false, + "generated": false + } + ], + "comment": null + } + } + ] + } + }`, buf.String()) + + query = "DROP TABLE test.table2" + e = &model.DDLEvent{ + CommitTs: 1, + PreTableInfo: preTableInfo, + Query: query, + Type: timodel.ActionDropTable, + } + keyBuf.Reset() + buf.Reset() + err = codec.EncodeDDLEvent(e, keyBuf, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "databaseName": "test" + } + }`, keyBuf.String()) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "", + "table": "", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 1, + "cluster_id": "test_cluster" + }, + "ts_ms": 1701326309000, + "databaseName": "test", + "schemaName": null, + "ddl": "DROP TABLE test.table2", + "tableChanges": [] + } + }`, buf.String()) +} + +func TestCheckPointEvent(t *testing.T) { + codec := &dbzCodec{ + config: common.NewConfig(config.ProtocolDebezium), + clusterID: "test_cluster", + nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, + } + codec.config.DebeziumDisableSchema = false + + var ts uint64 = 3 + keyBuf := bytes.NewBuffer(nil) + buf := bytes.NewBuffer(nil) + err := codec.EncodeCheckpointEvent(ts, keyBuf, buf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": {}, + "schema": { + "fields": [], + "optional": false, + "name": "test_cluster.watermark.Key", + "type": "struct" + } + }`, keyBuf.String()) + require.JSONEq(t, ` + { + "payload": { + "source": { + "version": "2.4.0.Final", + "connector": "TiCDC", + "name": "test_cluster", + "ts_ms": 0, + "snapshot": "false", + "db": "", + "table": "", + "server_id": 0, + "gtid": null, + "file": "", + "pos": 0, + "row": 0, + "thread": 0, + "query": null, + "commit_ts": 3, + "cluster_id": "test_cluster" + }, + "op":"m", + "ts_ms": 1701326309000, + "transaction": null + }, + "schema": { + "type": "struct", + "optional": false, + "name": "test_cluster.watermark.Envelope", + "version": 1, + "fields": [ + { + "type": "struct", + "fields": [ + { + "type": "string", + "optional": false, + "field": "version" + }, + { + "type": "string", + "optional": false, + "field": "connector" + }, + { + "type": "string", + "optional": false, + "field": "name" + }, + { + "type": "int64", + "optional": false, + "field": "ts_ms" + }, + { + "type": "string", + "optional": true, + "name": "io.debezium.data.Enum", + "version": 1, + "parameters": { + "allowed": "true,last,false,incremental" + }, + "default": "false", + "field": "snapshot" + }, + { + "type": "string", + "optional": false, + "field": "db" + }, + { + "type": "string", + "optional": true, + "field": "sequence" + }, + { + "type": "string", + "optional": true, + "field": "table" + }, + { + "type": "int64", + "optional": false, + "field": "server_id" + }, + { + "type": "string", + "optional": true, + "field": "gtid" + }, + { + "type": "string", + "optional": false, + "field": "file" + }, + { + "type": "int64", + "optional": false, + "field": "pos" + }, + { + "type": "int32", + "optional": false, + "field": "row" + }, + { + "type": "int64", + "optional": true, + "field": "thread" + }, + { + "type": "string", + "optional": true, + "field": "query" + } + ], + "optional": false, + "name": "io.debezium.connector.mysql.Source", + "field": "source" + }, + { + "type": "string", + "optional": false, + "field": "op" + }, + { + "type": "int64", + "optional": true, + "field": "ts_ms" + }, + { + "type": "struct", + "fields": [ + { + "type": "string", + "optional": false, + "field": "id" + }, + { + "type": "int64", + "optional": false, + "field": "total_order" + }, + { + "type": "int64", + "optional": false, + "field": "data_collection_order" + } + ], + "optional": true, + "name": "event.block", + "version": 1, + "field": "transaction" + } + ] + } + }`, buf.String()) +} + func TestEncodeInsert(t *testing.T) { codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), - clusterID: "test-cluster", + clusterID: "test_cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, } codec.config.DebeziumDisableSchema = true @@ -38,8 +724,8 @@ func TestEncodeInsert(t *testing.T) { tableInfo := model.BuildTableInfo("test", "table1", []*model.Column{{ Name: "tiny", Type: mysql.TypeTiny, - Flag: model.NullableFlag, - }}, nil) + Flag: model.NullableFlag | model.HandleKeyFlag | model.PrimaryKeyFlag, + }}, [][]int{{0}}) e := &model.RowChangedEvent{ CommitTs: 1, TableInfo: tableInfo, @@ -50,7 +736,17 @@ func TestEncodeInsert(t *testing.T) { } buf := bytes.NewBuffer(nil) - err := codec.EncodeRowChangedEvent(e, buf) + keyBuf := bytes.NewBuffer(nil) + err := codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + } + } + `, keyBuf.String()) + err = codec.EncodeValue(e, buf) require.Nil(t, err) require.JSONEq(t, ` { @@ -61,8 +757,8 @@ func TestEncodeInsert(t *testing.T) { }, "op": "c", "source": { - "cluster_id": "test-cluster", - "name": "test-cluster", + "cluster_id": "test_cluster", + "name": "test_cluster", "commit_ts": 1, "connector": "TiCDC", "db": "test", @@ -85,8 +781,30 @@ func TestEncodeInsert(t *testing.T) { `, buf.String()) codec.config.DebeziumDisableSchema = false + keyBuf.Reset() + err = codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + }, + "schema": { + "fields": [ + { + "field":"tiny", + "optional":true, + "type":"int16" + } + ], + "name": "test_cluster.test.table1.Key", + "optional": false, + "type":"struct" + } + } + `, keyBuf.String()) buf.Reset() - err = codec.EncodeRowChangedEvent(e, buf) + err = codec.EncodeValue(e, buf) require.Nil(t, err) require.JSONEq(t, ` { @@ -94,7 +812,7 @@ func TestEncodeInsert(t *testing.T) { "source": { "version": "2.4.0.Final", "connector": "TiCDC", - "name": "test-cluster", + "name": "test_cluster", "ts_ms": 0, "snapshot": "false", "db": "test", @@ -107,7 +825,7 @@ func TestEncodeInsert(t *testing.T) { "thread": 0, "query": null, "commit_ts": 1, - "cluster_id": "test-cluster" + "cluster_id": "test_cluster" }, "ts_ms": 1701326309000, "transaction": null, @@ -118,20 +836,20 @@ func TestEncodeInsert(t *testing.T) { "schema": { "type": "struct", "optional": false, - "name": "test-cluster.test.table1.Envelope", + "name": "test_cluster.test.table1.Envelope", "version": 1, "fields": [ { "type": "struct", "optional": true, - "name": "test-cluster.test.table1.Value", + "name": "test_cluster.test.table1.Value", "field": "before", "fields": [{ "type": "int16", "optional": true, "field": "tiny" }] }, { "type": "struct", "optional": true, - "name": "test-cluster.test.table1.Value", + "name": "test_cluster.test.table1.Value", "field": "after", "fields": [{ "type": "int16", "optional": true, "field": "tiny" }] }, @@ -193,7 +911,7 @@ func TestEncodeInsert(t *testing.T) { func TestEncodeUpdate(t *testing.T) { codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), - clusterID: "test-cluster", + clusterID: "test_cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, } codec.config.DebeziumDisableSchema = true @@ -201,8 +919,8 @@ func TestEncodeUpdate(t *testing.T) { tableInfo := model.BuildTableInfo("test", "table1", []*model.Column{{ Name: "tiny", Type: mysql.TypeTiny, - Flag: model.NullableFlag, - }}, nil) + Flag: model.NullableFlag | model.HandleKeyFlag | model.PrimaryKeyFlag, + }}, [][]int{{0}}) e := &model.RowChangedEvent{ CommitTs: 1, TableInfo: tableInfo, @@ -217,7 +935,18 @@ func TestEncodeUpdate(t *testing.T) { } buf := bytes.NewBuffer(nil) - err := codec.EncodeRowChangedEvent(e, buf) + keyBuf := bytes.NewBuffer(nil) + err := codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + } + } + `, keyBuf.String()) + + err = codec.EncodeValue(e, buf) require.Nil(t, err) require.JSONEq(t, ` { @@ -230,8 +959,8 @@ func TestEncodeUpdate(t *testing.T) { }, "op": "u", "source": { - "cluster_id": "test-cluster", - "name": "test-cluster", + "cluster_id": "test_cluster", + "name": "test_cluster", "commit_ts": 1, "connector": "TiCDC", "db": "test", @@ -254,8 +983,31 @@ func TestEncodeUpdate(t *testing.T) { `, buf.String()) codec.config.DebeziumDisableSchema = false + keyBuf.Reset() + err = codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + }, + "schema": { + "fields": [ + { + "field":"tiny", + "optional":true, + "type":"int16" + } + ], + "name": "test_cluster.test.table1.Key", + "optional": false, + "type":"struct" + } + } + `, keyBuf.String()) + buf.Reset() - err = codec.EncodeRowChangedEvent(e, buf) + err = codec.EncodeValue(e, buf) require.Nil(t, err) require.JSONEq(t, ` { @@ -263,7 +1015,7 @@ func TestEncodeUpdate(t *testing.T) { "source": { "version": "2.4.0.Final", "connector": "TiCDC", - "name": "test-cluster", + "name": "test_cluster", "ts_ms": 0, "snapshot": "false", "db": "test", @@ -276,7 +1028,7 @@ func TestEncodeUpdate(t *testing.T) { "thread": 0, "query": null, "commit_ts": 1, - "cluster_id": "test-cluster" + "cluster_id": "test_cluster" }, "ts_ms": 1701326309000, "transaction": null, @@ -287,20 +1039,20 @@ func TestEncodeUpdate(t *testing.T) { "schema": { "type": "struct", "optional": false, - "name": "test-cluster.test.table1.Envelope", + "name": "test_cluster.test.table1.Envelope", "version": 1, "fields": [ { "type": "struct", "optional": true, - "name": "test-cluster.test.table1.Value", + "name": "test_cluster.test.table1.Value", "field": "before", "fields": [{ "type": "int16", "optional": true, "field": "tiny" }] }, { "type": "struct", "optional": true, - "name": "test-cluster.test.table1.Value", + "name": "test_cluster.test.table1.Value", "field": "after", "fields": [{ "type": "int16", "optional": true, "field": "tiny" }] }, @@ -360,8 +1112,20 @@ func TestEncodeUpdate(t *testing.T) { codec.config.DebeziumOutputOldValue = false codec.config.DebeziumDisableSchema = true + + keyBuf.Reset() + err = codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 1 + } + } + `, keyBuf.String()) + buf.Reset() - err = codec.EncodeRowChangedEvent(e, buf) + err = codec.EncodeValue(e, buf) require.Nil(t, err) require.JSONEq(t, ` { @@ -369,7 +1133,7 @@ func TestEncodeUpdate(t *testing.T) { "source": { "version": "2.4.0.Final", "connector": "TiCDC", - "name": "test-cluster", + "name": "test_cluster", "ts_ms": 0, "snapshot": "false", "db": "test", @@ -382,7 +1146,7 @@ func TestEncodeUpdate(t *testing.T) { "thread": 0, "query": null, "commit_ts": 1, - "cluster_id": "test-cluster" + "cluster_id": "test_cluster" }, "ts_ms": 1701326309000, "transaction": null, @@ -396,7 +1160,7 @@ func TestEncodeUpdate(t *testing.T) { func TestEncodeDelete(t *testing.T) { codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), - clusterID: "test-cluster", + clusterID: "test_cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, } codec.config.DebeziumOutputOldValue = false @@ -405,8 +1169,8 @@ func TestEncodeDelete(t *testing.T) { tableInfo := model.BuildTableInfo("test", "table1", []*model.Column{{ Name: "tiny", Type: mysql.TypeTiny, - Flag: model.NullableFlag, - }}, nil) + Flag: model.NullableFlag | model.HandleKeyFlag | model.PrimaryKeyFlag, + }}, [][]int{{0}}) e := &model.RowChangedEvent{ CommitTs: 1, TableInfo: tableInfo, @@ -417,7 +1181,18 @@ func TestEncodeDelete(t *testing.T) { } buf := bytes.NewBuffer(nil) - err := codec.EncodeRowChangedEvent(e, buf) + keyBuf := bytes.NewBuffer(nil) + err := codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 2 + } + } + `, keyBuf.String()) + + err = codec.EncodeValue(e, buf) require.Nil(t, err) require.JSONEq(t, ` { @@ -428,8 +1203,8 @@ func TestEncodeDelete(t *testing.T) { "after": null, "op": "d", "source": { - "cluster_id": "test-cluster", - "name": "test-cluster", + "cluster_id": "test_cluster", + "name": "test_cluster", "commit_ts": 1, "connector": "TiCDC", "db": "test", @@ -452,8 +1227,32 @@ func TestEncodeDelete(t *testing.T) { `, buf.String()) codec.config.DebeziumDisableSchema = false + + keyBuf.Reset() + err = codec.EncodeKey(e, keyBuf) + require.Nil(t, err) + require.JSONEq(t, ` + { + "payload": { + "tiny": 2 + }, + "schema": { + "fields": [ + { + "field":"tiny", + "optional":true, + "type":"int16" + } + ], + "name": "test_cluster.test.table1.Key", + "optional": false, + "type":"struct" + } + } + `, keyBuf.String()) + buf.Reset() - err = codec.EncodeRowChangedEvent(e, buf) + err = codec.EncodeValue(e, buf) require.Nil(t, err) require.JSONEq(t, ` { @@ -461,7 +1260,7 @@ func TestEncodeDelete(t *testing.T) { "source": { "version": "2.4.0.Final", "connector": "TiCDC", - "name": "test-cluster", + "name": "test_cluster", "ts_ms": 0, "snapshot": "false", "db": "test", @@ -474,7 +1273,7 @@ func TestEncodeDelete(t *testing.T) { "thread": 0, "query": null, "commit_ts": 1, - "cluster_id": "test-cluster" + "cluster_id": "test_cluster" }, "ts_ms": 1701326309000, "transaction": null, @@ -485,20 +1284,20 @@ func TestEncodeDelete(t *testing.T) { "schema": { "type": "struct", "optional": false, - "name": "test-cluster.test.table1.Envelope", + "name": "test_cluster.test.table1.Envelope", "version": 1, "fields": [ { "type": "struct", "optional": true, - "name": "test-cluster.test.table1.Value", + "name": "test_cluster.test.table1.Value", "field": "before", "fields": [{ "type": "int16", "optional": true, "field": "tiny" }] }, { "type": "struct", "optional": true, - "name": "test-cluster.test.table1.Value", + "name": "test_cluster.test.table1.Value", "field": "after", "fields": [{ "type": "int16", "optional": true, "field": "tiny" }] }, @@ -560,7 +1359,7 @@ func TestEncodeDelete(t *testing.T) { func BenchmarkEncodeOneTinyColumn(b *testing.B) { codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), - clusterID: "test-cluster", + clusterID: "test_cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, } codec.config.DebeziumDisableSchema = true @@ -578,19 +1377,22 @@ func BenchmarkEncodeOneTinyColumn(b *testing.B) { }}, tableInfo), } + keyBuf := bytes.NewBuffer(nil) buf := bytes.NewBuffer(nil) b.ResetTimer() for n := 0; n < b.N; n++ { + keyBuf.Reset() buf.Reset() - codec.EncodeRowChangedEvent(e, buf) + codec.EncodeKey(e, keyBuf) + codec.EncodeValue(e, buf) } } func BenchmarkEncodeLargeText(b *testing.B) { codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), - clusterID: "test-cluster", + clusterID: "test_cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, } codec.config.DebeziumDisableSchema = true @@ -608,19 +1410,22 @@ func BenchmarkEncodeLargeText(b *testing.B) { }}, tableInfo), } + keyBuf := bytes.NewBuffer(nil) buf := bytes.NewBuffer(nil) b.ResetTimer() for n := 0; n < b.N; n++ { + keyBuf.Reset() buf.Reset() - codec.EncodeRowChangedEvent(e, buf) + codec.EncodeKey(e, keyBuf) + codec.EncodeValue(e, buf) } } func BenchmarkEncodeLargeBinary(b *testing.B) { codec := &dbzCodec{ config: common.NewConfig(config.ProtocolDebezium), - clusterID: "test-cluster", + clusterID: "test_cluster", nowFunc: func() time.Time { return time.Unix(1701326309, 0) }, } codec.config.DebeziumDisableSchema = true @@ -639,11 +1444,14 @@ func BenchmarkEncodeLargeBinary(b *testing.B) { }}, tableInfo), } + keyBuf := bytes.NewBuffer(nil) buf := bytes.NewBuffer(nil) b.ResetTimer() for n := 0; n < b.N; n++ { + keyBuf.Reset() buf.Reset() - codec.EncodeRowChangedEvent(e, buf) + codec.EncodeKey(e, keyBuf) + codec.EncodeValue(e, buf) } } diff --git a/pkg/sink/codec/debezium/debezium_test.go b/pkg/sink/codec/debezium/debezium_test.go index d18292c2bef..aa39bfe211f 100644 --- a/pkg/sink/codec/debezium/debezium_test.go +++ b/pkg/sink/codec/debezium/debezium_test.go @@ -182,6 +182,8 @@ func (s *debeziumSuite) TestDataTypes() { dataDbzOutput, err := os.ReadFile("testdata/datatype.dbz.json") s.Require().Nil(err) + keyDbzOutput, err := os.ReadFile("testdata/datatype.dbz.key.json") + s.Require().Nil(err) helper := NewSQLTestHelper(s.T(), "foo", string(dataDDL)) @@ -202,4 +204,5 @@ func (s *debeziumSuite) TestDataTypes() { messages := encoder.Build() s.Require().Len(messages, 1) s.requireDebeziumJSONEq(dataDbzOutput, messages[0].Value) + s.requireDebeziumJSONEq(keyDbzOutput, messages[0].Key) } diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go index 4dcb1960e8f..085a292ffd0 100644 --- a/pkg/sink/codec/debezium/encoder.go +++ b/pkg/sink/codec/debezium/encoder.go @@ -18,11 +18,13 @@ import ( "context" "time" + "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink/codec" "github.com/pingcap/tiflow/pkg/sink/codec/common" + "go.uber.org/zap" ) // BatchEncoder encodes message into Debezium format. @@ -35,21 +37,52 @@ type BatchEncoder struct { // EncodeCheckpointEvent implements the RowEventEncoder interface func (d *BatchEncoder) EncodeCheckpointEvent(ts uint64) (*common.Message, error) { - // Currently ignored. Debezium MySQL Connector does not emit such event. - return nil, nil + keyMap := bytes.Buffer{} + valueBuf := bytes.Buffer{} + err := d.codec.EncodeCheckpointEvent(ts, &keyMap, &valueBuf) + if err != nil { + return nil, errors.Trace(err) + } + key, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + keyMap.Bytes(), + ) + if err != nil { + return nil, err + } + value, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + valueBuf.Bytes(), + ) + if err != nil { + return nil, err + } + result := common.NewResolvedMsg(config.ProtocolDebezium, key, value, ts) + return result, nil } -// AppendRowChangedEvent implements the RowEventEncoder interface -func (d *BatchEncoder) AppendRowChangedEvent( - _ context.Context, - _ string, - e *model.RowChangedEvent, - callback func(), -) error { +func (d *BatchEncoder) encodeKey(e *model.RowChangedEvent) ([]byte, error) { + keyBuf := bytes.Buffer{} + err := d.codec.EncodeKey(e, &keyBuf) + if err != nil { + return nil, errors.Trace(err) + } + // TODO: Use a streaming compression is better. + key, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + keyBuf.Bytes(), + ) + return key, err +} + +func (d *BatchEncoder) encodeValue(e *model.RowChangedEvent) ([]byte, error) { valueBuf := bytes.Buffer{} - err := d.codec.EncodeRowChangedEvent(e, &valueBuf) + err := d.codec.EncodeValue(e, &valueBuf) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) } // TODO: Use a streaming compression is better. value, err := common.Compress( @@ -57,11 +90,27 @@ func (d *BatchEncoder) AppendRowChangedEvent( d.config.LargeMessageHandle.LargeMessageHandleCompression, valueBuf.Bytes(), ) - if err != nil { + return value, err +} + +// AppendRowChangedEvent implements the RowEventEncoder interface +func (d *BatchEncoder) AppendRowChangedEvent( + _ context.Context, + _ string, + e *model.RowChangedEvent, + callback func(), +) error { + var key []byte + var value []byte + var err error + if key, err = d.encodeKey(e); err != nil { + return errors.Trace(err) + } + if value, err = d.encodeValue(e); err != nil { return errors.Trace(err) } m := &common.Message{ - Key: nil, + Key: key, Value: value, Ts: e.CommitTs, Schema: e.TableInfo.GetSchemaNamePtr(), @@ -77,10 +126,36 @@ func (d *BatchEncoder) AppendRowChangedEvent( } // EncodeDDLEvent implements the RowEventEncoder interface -// DDL message unresolved tso func (d *BatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*common.Message, error) { - // Schema Change Events are currently not supported. - return nil, nil + valueBuf := bytes.Buffer{} + keyMap := bytes.Buffer{} + err := d.codec.EncodeDDLEvent(e, &keyMap, &valueBuf) + if err != nil { + if errors.ErrDDLUnsupportType.Equal(err) { + log.Warn("encode ddl event failed, just ignored", zap.Error(err)) + return nil, nil + } + return nil, errors.Trace(err) + } + key, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + keyMap.Bytes(), + ) + if err != nil { + return nil, err + } + value, err := common.Compress( + d.config.ChangefeedID, + d.config.LargeMessageHandle.LargeMessageHandleCompression, + valueBuf.Bytes(), + ) + if err != nil { + return nil, err + } + result := common.NewDDLMsg(config.ProtocolDebezium, key, value, e) + + return result, nil } // Build implements the RowEventEncoder interface diff --git a/pkg/sink/codec/debezium/helper.go b/pkg/sink/codec/debezium/helper.go new file mode 100644 index 00000000000..a2c957b39ab --- /dev/null +++ b/pkg/sink/codec/debezium/helper.go @@ -0,0 +1,265 @@ +// 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 debezium + +import ( + "encoding/binary" + "fmt" + "strings" + + "github.com/pingcap/log" + timodel "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/sink/codec/common" + "go.uber.org/zap" +) + +type visiter struct { + columnsMap map[pmodel.CIStr]*timodel.ColumnInfo +} + +func (v *visiter) Enter(n ast.Node) (node ast.Node, skipChildren bool) { + return n, false +} + +func (v *visiter) Leave(n ast.Node) (node ast.Node, ok bool) { + switch col := n.(type) { + case *ast.ColumnDef: + c := v.columnsMap[col.Name.Name] + if col.Options != nil { + parseOptions(col.Options, c) + } + if col.Tp != nil { + parseType(c, col) + } + c.Comment = "" // disable comment + } + return n, true +} + +func extractValue(expr ast.ExprNode) any { + switch v := expr.(type) { + case *driver.ValueExpr: + return fmt.Sprintf("%v", v.GetValue()) + case *ast.FuncCallExpr: + return v.FnName.String() + } + return nil +} + +func parseType(c *timodel.ColumnInfo, col *ast.ColumnDef) { + ft := col.Tp + switch ft.GetType() { + case mysql.TypeDatetime, mysql.TypeDuration, mysql.TypeTimestamp: + c.SetDecimal(ft.GetDecimal()) + if c.OriginDefaultValue != nil { + c.SetDefaultValue(c.OriginDefaultValue) + } + case mysql.TypeYear: + c.SetFlen(ft.GetFlen()) + if c.OriginDefaultValue != nil { + c.SetDefaultValue(c.OriginDefaultValue) + } + default: + } +} + +func parseOptions(options []*ast.ColumnOption, c *timodel.ColumnInfo) { + for _, option := range options { + switch option.Tp { + case ast.ColumnOptionDefaultValue: + defaultValue := extractValue(option.Expr) + if defaultValue == nil { + continue + } + if err := c.SetOriginDefaultValue(defaultValue); err != nil { + log.Error("failed to set default value") + } + } + } +} + +func parseColumns(sql string, columns []*timodel.ColumnInfo) { + p := parser.New() + stmt, err := p.ParseOneStmt(sql, mysql.DefaultCharset, mysql.DefaultCollationName) + if err != nil { + log.Error("format query parse one stmt failed", zap.Error(err)) + } + + columnsMap := make(map[pmodel.CIStr]*timodel.ColumnInfo, len(columns)) + for _, col := range columns { + columnsMap[col.Name] = col + } + stmt.Accept(&visiter{columnsMap: columnsMap}) +} + +func parseBit(s string, n int) string { + var result string + if len(s) > 0 { + // Leading zeros may be omitted + result = fmt.Sprintf("%0*b", n%8, s[0]) + } + for i := 1; i < len(s); i++ { + result += fmt.Sprintf("%08b", s[i]) + } + return result +} + +func getCharset(ft types.FieldType) string { + if ft.GetCharset() == "binary" { + return "" + } + switch ft.GetType() { + case mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeNewDecimal, mysql.TypeString, mysql.TypeVarchar, + mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeEnum, mysql.TypeSet: + return ft.GetCharset() + } + return "" +} + +func getLen(ft types.FieldType) int { + defaultFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(ft.GetType()) + decimal := ft.GetDecimal() + flen := ft.GetFlen() + switch ft.GetType() { + case mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeDatetime: + return decimal + case mysql.TypeBit, mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTiDBVectorFloat32, + mysql.TypeLonglong, mysql.TypeFloat, mysql.TypeDouble: + if flen != defaultFlen { + return flen + } + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong: + if mysql.HasUnsignedFlag(ft.GetFlag()) { + defaultFlen -= 1 + } + if ft.GetType() == mysql.TypeTiny && mysql.HasZerofillFlag(ft.GetFlag()) { + defaultFlen += 1 + } + if flen != defaultFlen { + return flen + } + case mysql.TypeYear, mysql.TypeNewDecimal: + return flen + case mysql.TypeSet: + return 2*len(ft.GetElems()) - 1 + case mysql.TypeEnum: + return 1 + } + return -1 +} + +func getScale(ft types.FieldType) float64 { + switch ft.GetType() { + case mysql.TypeNewDecimal, mysql.TypeFloat, mysql.TypeDouble: + return float64(ft.GetDecimal()) + } + return -1 +} + +func getSuffix(ft types.FieldType) string { + suffix := "" + decimal := ft.GetDecimal() + flen := ft.GetFlen() + defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(ft.GetType()) + isDecimalNotDefault := decimal != defaultDecimal && decimal != 0 && decimal != -1 + + // displayFlen and displayDecimal are flen and decimal values with `-1` substituted with default value. + displayFlen, displayDecimal := flen, decimal + if displayFlen == -1 { + displayFlen = defaultFlen + } + if displayDecimal == -1 { + displayDecimal = defaultDecimal + } + + switch ft.GetType() { + case mysql.TypeDouble: + // 1. flen Not Default, decimal Not Default -> Valid + // 2. flen Not Default, decimal Default (-1) -> Invalid + // 3. flen Default, decimal Not Default -> Valid + // 4. flen Default, decimal Default -> Valid (hide)W + if isDecimalNotDefault { + suffix = fmt.Sprintf("(%d,%d)", displayFlen, displayDecimal) + } + case mysql.TypeNewDecimal: + suffix = fmt.Sprintf("(%d,%d)", displayFlen, displayDecimal) + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString: + if !mysql.HasBinaryFlag(ft.GetFlag()) && displayFlen != 1 { + suffix = fmt.Sprintf("(%d)", displayFlen) + } + case mysql.TypeYear: + suffix = fmt.Sprintf("(%d)", flen) + case mysql.TypeTiDBVectorFloat32: + if flen != -1 { + suffix = fmt.Sprintf("(%d)", flen) + } + case mysql.TypeNull: + suffix = "(0)" + } + return suffix +} + +func getExpressionAndName(ft types.FieldType) (string, string) { + prefix := strings.ToUpper(types.TypeToStr(ft.GetType(), ft.GetCharset())) + switch ft.GetType() { + case mysql.TypeYear, mysql.TypeBit, mysql.TypeVarchar, mysql.TypeString, mysql.TypeNewDecimal: + return prefix, prefix + } + cs := prefix + getSuffix(ft) + var suf string + if mysql.HasZerofillFlag(ft.GetFlag()) { + suf = " UNSIGNED ZEROFILL" + } else if mysql.HasUnsignedFlag(ft.GetFlag()) { + suf = " UNSIGNED" + } + return cs + suf, prefix + suf +} + +func getBitFromUint64(n int, v uint64) []byte { + var buf [8]byte + binary.LittleEndian.PutUint64(buf[:], v) + numBytes := n / 8 + if n%8 != 0 { + numBytes += 1 + } + return buf[:numBytes] +} + +func getValue(col model.ColumnDataX) any { + if col.Value == nil { + return col.GetDefaultValue() + } + return col.Value +} + +func getDBTableName(e *model.DDLEvent) (string, string) { + if e.TableInfo == nil { + return "", "" + } + return e.TableInfo.GetSchemaName(), e.TableInfo.GetTableName() +} + +func getSchemaTopicName(namespace string, schema string, table string) string { + return fmt.Sprintf("%s.%s.%s", + common.SanitizeName(namespace), + common.SanitizeName(schema), + common.SanitizeTopicName(table)) +} diff --git a/pkg/sink/codec/debezium/helper_test.go b/pkg/sink/codec/debezium/helper_test.go new file mode 100644 index 00000000000..8713da21e22 --- /dev/null +++ b/pkg/sink/codec/debezium/helper_test.go @@ -0,0 +1,80 @@ +// 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 debezium + +import ( + "testing" + + 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/tidb/pkg/types" + "github.com/pingcap/tiflow/cdc/model" + "github.com/stretchr/testify/require" +) + +func TestGetColumns(t *testing.T) { + sql := "CREATE TABLE test (id INT PRIMARY KEY, val1 datetime default current_timestamp, val2 time(2) default 0, val3 timestamp(3) default now(), val4 YEAR(4) default 1970 comment 'first');" + columnInfos := []*timodel.ColumnInfo{ + { + Name: pmodel.NewCIStr("id"), + FieldType: *types.NewFieldType(mysql.TypeLong), + }, + { + Name: pmodel.NewCIStr("val1"), + FieldType: *types.NewFieldType(mysql.TypeDatetime), + }, + { + Name: pmodel.NewCIStr("val2"), + FieldType: *types.NewFieldType(mysql.TypeDuration), + }, + { + Name: pmodel.NewCIStr("val3"), + FieldType: *types.NewFieldType(mysql.TypeTimestamp), + }, + { + Name: pmodel.NewCIStr("val4"), + FieldType: *types.NewFieldType(mysql.TypeYear), + }, + } + parseColumns(sql, columnInfos) + require.Equal(t, columnInfos[1].GetDefaultValue(), "CURRENT_TIMESTAMP") + require.Equal(t, columnInfos[2].GetDecimal(), 2) + require.Equal(t, columnInfos[2].GetDefaultValue(), "0") + require.Equal(t, columnInfos[3].GetDecimal(), 3) + require.Equal(t, columnInfos[3].GetDefaultValue(), "CURRENT_TIMESTAMP") + require.Equal(t, columnInfos[4].GetFlen(), 4) + require.Equal(t, columnInfos[4].GetDefaultValue(), "1970") + require.Equal(t, columnInfos[4].Comment, "") +} + +func TestGetValue(t *testing.T) { + column := &model.Column{ + Default: 1, + } + data := model.Column2ColumnDataXForTest(column) + v := getValue(data) + require.Equal(t, v, int64(1)) + data.Value = 2 + v = getValue(data) + require.Equal(t, v, 2) +} + +func TestGetSchemaTopicName(t *testing.T) { + namespace := "default" + schema := "1A.B" + table := "columnNameWith中文" + name := getSchemaTopicName(namespace, schema, table) + require.Equal(t, name, "default._1A_B.columnNameWith__") +} diff --git a/pkg/sink/codec/debezium/testdata/datatype.dbz.key.json b/pkg/sink/codec/debezium/testdata/datatype.dbz.key.json new file mode 100644 index 00000000000..ce133dbda22 --- /dev/null +++ b/pkg/sink/codec/debezium/testdata/datatype.dbz.key.json @@ -0,0 +1,17 @@ +{ + "payload": { + "pk": 1 + }, + "schema": { + "fields": [ + { + "field": "pk", + "optional": false, + "type": "int32" + } + ], + "name": "dbserver1.test.foo.Key", + "optional": false, + "type": "struct" + } +} \ No newline at end of file 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/internal/java.go b/pkg/sink/codec/internal/java.go index b25421fb6ba..8c3d29763f0 100644 --- a/pkg/sink/codec/internal/java.go +++ b/pkg/sink/codec/internal/java.go @@ -20,31 +20,33 @@ type JavaSQLType int32 // jdk 1.8 const ( - JavaSQLTypeBIT JavaSQLType = -7 - JavaSQLTypeTINYINT JavaSQLType = -6 - JavaSQLTypeSMALLINT JavaSQLType = 5 - JavaSQLTypeINTEGER JavaSQLType = 4 - JavaSQLTypeBIGINT JavaSQLType = -5 - JavaSQLTypeREAL JavaSQLType = 7 - JavaSQLTypeDOUBLE JavaSQLType = 8 - JavaSQLTypeDECIMAL JavaSQLType = 3 - JavaSQLTypeCHAR JavaSQLType = 1 - JavaSQLTypeVARCHAR JavaSQLType = 12 - JavaSQLTypeDATE JavaSQLType = 91 - JavaSQLTypeTIME JavaSQLType = 92 - JavaSQLTypeTIMESTAMP JavaSQLType = 93 - JavaSQLTypeBINARY JavaSQLType = -2 - JavaSQLTypeVARBINARY JavaSQLType = -3 - JavaSQLTypeLONGVARBINARY JavaSQLType = -4 - JavaSQLTypeNULL JavaSQLType = 0 - JavaSQLTypeBLOB JavaSQLType = 2004 - JavaSQLTypeCLOB JavaSQLType = 2005 + JavaSQLTypeBIT JavaSQLType = -7 + JavaSQLTypeTINYINT JavaSQLType = -6 + JavaSQLTypeSMALLINT JavaSQLType = 5 + JavaSQLTypeINTEGER JavaSQLType = 4 + JavaSQLTypeBIGINT JavaSQLType = -5 + JavaSQLTypeREAL JavaSQLType = 7 + JavaSQLTypeDOUBLE JavaSQLType = 8 + JavaSQLTypeDECIMAL JavaSQLType = 3 + JavaSQLTypeCHAR JavaSQLType = 1 + JavaSQLTypeVARCHAR JavaSQLType = 12 + JavaSQLTypeDATE JavaSQLType = 91 + JavaSQLTypeTIME JavaSQLType = 92 + JavaSQLTypeTIMESTAMP JavaSQLType = 93 + JavaSQLTypeBINARY JavaSQLType = -2 + JavaSQLTypeVARBINARY JavaSQLType = -3 + JavaSQLTypeLONGVARBINARY JavaSQLType = -4 + JavaSQLTypeNULL JavaSQLType = 0 + JavaSQLTypeBLOB JavaSQLType = 2004 + JavaSQLTypeCLOB JavaSQLType = 2005 + JavaSQLTypeFLOAT JavaSQLType = 6 + JavaSQLTypeNUMERIC JavaSQLType = 2 + JavaSQLTypeOTHER JavaSQLType = 1111 + JavaSQLTypeNCHAR JavaSQLType = -15 + JavaSQLTypeTIMESTAMP_WITH_TIMEZONE JavaSQLType = 2014 //nolint // unused // JavaSQLTypeLONGVARCHAR JavaSQLType = -1 - // JavaSQLTypeFLOAT JavaSQLType = 6 - // JavaSQLTypeNUMERIC JavaSQLType = 2 - // JavaSQLTypeOTHER JavaSQLType = 1111 // JavaSQLTypeJAVA_OBJECT JavaSQLType = 2000 // JavaSQLTypeDISTINCT JavaSQLType = 2001 // JavaSQLTypeSTRUCT JavaSQLType = 2002 @@ -53,14 +55,12 @@ const ( // JavaSQLTypeDATALINK JavaSQLType = 70 // JavaSQLTypeBOOLEAN JavaSQLType = 16 // JavaSQLTypeROWID JavaSQLType = -8 - // JavaSQLTypeNCHAR JavaSQLType = -15 // JavaSQLTypeNVARCHAR JavaSQLType = -9 // JavaSQLTypeLONGNVARCHAR JavaSQLType = -16 // JavaSQLTypeNCLOB JavaSQLType = 2011 // JavaSQLTypeSQLXML JavaSQLType = 2009 // JavaSQLTypeREF_CURSOR JavaSQLType = 2012 // JavaSQLTypeTIME_WITH_TIMEZONE JavaSQLType = 2013 - // JavaSQLTypeTIMESTAMP_WITH_TIMEZONE JavaSQLType = 2014 ) // MySQLType2JavaType converts the mysql protocol types to java sql types @@ -150,3 +150,76 @@ func MySQLType2JavaType(mysqlType byte, isBinary bool) JavaSQLType { return JavaSQLTypeVARCHAR } } + +// MySQLType2JdbcType converts the mysql protocol types to jdbc type +func MySQLType2JdbcType(mysqlType byte, isBinary bool) JavaSQLType { + switch mysqlType { + case mysql.TypeTiny, mysql.TypeShort: + return JavaSQLTypeSMALLINT + + case mysql.TypeLong, mysql.TypeInt24, mysql.TypeYear: + return JavaSQLTypeINTEGER + + case mysql.TypeFloat: + return JavaSQLTypeFLOAT + + case mysql.TypeDouble: + return JavaSQLTypeDOUBLE + + case mysql.TypeNull: + return JavaSQLTypeNULL + + case mysql.TypeNewDecimal: + return JavaSQLTypeDECIMAL // equal to JavaSQLTypeNUMERIC + + case mysql.TypeTimestamp: + return JavaSQLTypeTIMESTAMP_WITH_TIMEZONE //nolint + + case mysql.TypeDatetime: + return JavaSQLTypeTIMESTAMP + + case mysql.TypeLonglong: + return JavaSQLTypeBIGINT + + case mysql.TypeDate, mysql.TypeNewDate: + return JavaSQLTypeDATE + + case mysql.TypeDuration: + return JavaSQLTypeTIME + + case mysql.TypeEnum, mysql.TypeSet: + return JavaSQLTypeCHAR + + case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + if isBinary { + return JavaSQLTypeBLOB + } + return JavaSQLTypeVARCHAR + + case mysql.TypeVarString, mysql.TypeVarchar: + if isBinary { + return JavaSQLTypeVARBINARY + } + return JavaSQLTypeVARCHAR + + case mysql.TypeString: + if isBinary { + return JavaSQLTypeBINARY + } + return JavaSQLTypeCHAR + // return JavaSQLTypeNCHAR + + // Geometry is not supported, this should not hit. + case mysql.TypeGeometry: + return JavaSQLTypeOTHER + + case mysql.TypeBit: + return JavaSQLTypeBIT + + case mysql.TypeJSON: + return JavaSQLTypeOTHER + + default: + return JavaSQLTypeVARCHAR + } +} 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/pkg/version/check.go b/pkg/version/check.go index 766623b0a47..835289bf90b 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -24,6 +24,7 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/util/engine" @@ -199,6 +200,9 @@ func checkPDVersion(ctx context.Context, pdAddr string, credential *security.Cre // CheckStoreVersion checks whether the given TiKV is compatible with this CDC. // If storeID is 0, it checks all TiKV. func CheckStoreVersion(ctx context.Context, client pd.Client, storeID uint64) error { + failpoint.Inject("GetStoreFailed", func() { + failpoint.Return(cerror.WrapError(cerror.ErrGetAllStoresFailed, fmt.Errorf("unknown store %d", storeID))) + }) var stores []*metapb.Store var err error if storeID == 0 { 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/ddl_with_exists/conf/diff_config.toml b/tests/integration_tests/ddl_with_exists/conf/diff_config.toml new file mode 100644 index 00000000000..b13ade0d46e --- /dev/null +++ b/tests/integration_tests/ddl_with_exists/conf/diff_config.toml @@ -0,0 +1,28 @@ +# diff Configuration. +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] +output-dir = "/tmp/tidb_cdc_test/ddl_with_exists/sync_diff/output" + +source-instances = ["mysql1"] + +target-instance = "tidb0" + +target-check-tables = ["ddl_with_exists.*"] + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 4000 +user = "root" +password = "" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "" diff --git a/tests/integration_tests/ddl_with_exists/run.sh b/tests/integration_tests/ddl_with_exists/run.sh new file mode 100755 index 00000000000..b7810ac2e8f --- /dev/null +++ b/tests/integration_tests/ddl_with_exists/run.sh @@ -0,0 +1,57 @@ +#!/bin/bash + +set -eu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +function run() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + owner_pid=$(ps -C $CDC_BINARY -o pid= | awk '{print $1}') + + # this test contains `recover table`, which requires super privilege, so we + # can't use the normal user + TOPIC_NAME="ticdc-ddl-mamager-test-$RANDOM" + case $SINK_TYPE in + kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; + storage) SINK_URI="file://$WORK_DIR/storage_test/$TOPIC_NAME?protocol=canal-json&enable-tidb-extension=true" ;; + pulsar) + run_pulsar_cluster $WORK_DIR normal + SINK_URI="pulsar://127.0.0.1:6650/$TOPIC_NAME?protocol=canal-json&enable-tidb-extension=true" + ;; + *) SINK_URI="mysql://root@127.0.0.1:3306/" ;; + esac + changefeed_id="ddl-with-exists" + run_cdc_cli changefeed create --sink-uri="$SINK_URI" -c=${changefeed_id} + + case $SINK_TYPE in + kafka) run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; + storage) run_storage_consumer $WORK_DIR $SINK_URI "" "" ;; + pulsar) run_pulsar_consumer --upstream-uri $SINK_URI ;; + esac + + run_sql "CREATE DATABASE ddl_with_exists" + + cd $CUR + GO111MODULE=on go run test.go + + run_sql "CREATE TABLE ddl_with_exists.finish_mark (a int primary key);" + check_table_exists ddl_with_exists.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + # make sure all tables are equal in upstream and downstream + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 180 + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +# run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/ddl_with_exists/test.go b/tests/integration_tests/ddl_with_exists/test.go new file mode 100644 index 00000000000..e7c018e49de --- /dev/null +++ b/tests/integration_tests/ddl_with_exists/test.go @@ -0,0 +1,102 @@ +// 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 main + +import ( + "database/sql" + "fmt" + "log" + "math/rand" + "os" + "sync" + "time" + + _ "github.com/go-sql-driver/mysql" +) + +func main() { + upHost := GetEnvDefault("UP_TIDB_HOST", "127.0.0.1") + upPort := GetEnvDefault("UP_TIDB_PORT", "4000") + dsn := fmt.Sprintf("root@tcp(%s:%s)/", upHost, upPort) + db, err := sql.Open("mysql", dsn) + if err != nil { + log.Fatal("open db failed:", dsn, ", err: ", err) + } + defer db.Close() + + if err = db.Ping(); err != nil { + log.Fatal("ping db failed:", dsn, ", err: ", err) + } + log.Println("connect to tidb success, dsn: ", dsn) + + createTable := `create table if not exists ddl_with_exists.t%d ( + id int primary key auto_increment, + name varchar(255) + );` + addColumn := "alter table ddl_with_exists.t%d add column if not exists age int;" + dropColumn := "alter table ddl_with_exists.t%d drop column if exists age;" + addIndex := "alter table ddl_with_exists.t%d add index if not exists idx1(id);" + dropIndex := "alter table ddl_with_exists.t%d drop index if exists idx1;" + + concurrency := 16 + maxTableCnt := 20 + db.SetMaxOpenConns(concurrency) + + start := time.Now() + for i := 0; i < maxTableCnt; i++ { + _, err := db.Exec(fmt.Sprintf(createTable, i)) + if err != nil { + log.Fatal("create table failed:", i, ", err: ", err) + } + } + log.Println("create table cost:", time.Since(start).Seconds(), "s") + + var wg sync.WaitGroup + for i := 0; i < concurrency; i++ { + wg.Add(1) + go func() { + defer wg.Done() + log.Println("worker start:", i) + for j := 0; j < 20; j++ { + idx := rand.Intn(maxTableCnt) + ddl := fmt.Sprintf(createTable, idx) + switch rand.Intn(5) { + case 0: + ddl = fmt.Sprintf(addColumn, idx) + case 1: + ddl = fmt.Sprintf(dropColumn, idx) + case 2: + ddl = fmt.Sprintf(addIndex, idx) + case 3: + ddl = fmt.Sprintf(dropIndex, idx) + default: + } + _, err := db.Exec(ddl) + if err != nil { + log.Println(err) + } + } + log.Println("worker exit:", i) + }() + } + wg.Wait() +} + +func GetEnvDefault(key, defaultV string) string { + val, ok := os.LookupEnv(key) + if !ok { + return defaultV + } + return val +} diff --git a/tests/integration_tests/debezium/README.md b/tests/integration_tests/debezium/README.md index f2c546ffdc0..87ef70a3074 100644 --- a/tests/integration_tests/debezium/README.md +++ b/tests/integration_tests/debezium/README.md @@ -22,7 +22,7 @@ curl -i -X POST \ "database.user": "debezium", "database.password": "dbz", "database.server.id": "184054", - "topic.prefix": "dbserver1", + "topic.prefix": "default", "schema.history.internal.kafka.bootstrap.servers": "kafka:9092", "schema.history.internal.kafka.topic": "schemahistory.test", "transforms": "x", @@ -41,9 +41,9 @@ tiup playground nightly --tiflash 0 --ticdc 1 ``` ``` -tiup cdc cli changefeed create \ - --server=http://127.0.0.1:8300 \ - --sink-uri="kafka://127.0.0.1:9094/output_ticdc?protocol=debezium&kafka-version=2.4.0" +tiup cdc cli changefeed create -c test \ + --server=http://127.0.0.1:8300 --config changefeed.toml \ + --sink-uri="kafka://127.0.0.1:9094/output_ticdc?protocol=debezium" ``` ``` diff --git a/tests/integration_tests/debezium/changefeed.toml b/tests/integration_tests/debezium/changefeed.toml new file mode 100644 index 00000000000..995c2d4ed03 --- /dev/null +++ b/tests/integration_tests/debezium/changefeed.toml @@ -0,0 +1 @@ +force-replicate = true \ No newline at end of file diff --git a/tests/integration_tests/debezium/docker-compose.yml b/tests/integration_tests/debezium/docker-compose.yml index 2041a980070..aa7f09fe336 100644 --- a/tests/integration_tests/debezium/docker-compose.yml +++ b/tests/integration_tests/debezium/docker-compose.yml @@ -45,6 +45,7 @@ services: - CONFIG_STORAGE_TOPIC=my_connect_configs - OFFSET_STORAGE_TOPIC=my_connect_offsets - STATUS_STORAGE_TOPIC=my_connect_statuses + - LANG=C.UTF-8 # watcher_dbz: # For Debug Purpose # restart: always # image: quay.io/debezium/kafka:2.4 diff --git a/tests/integration_tests/debezium/go.mod b/tests/integration_tests/debezium/go.mod index 454c383b03a..5bcf5da15c7 100644 --- a/tests/integration_tests/debezium/go.mod +++ b/tests/integration_tests/debezium/go.mod @@ -3,82 +3,85 @@ module github.com/breezewish/checker go 1.23.0 require ( + github.com/alecthomas/chroma v0.10.0 + github.com/fatih/color v1.17.0 github.com/go-sql-driver/mysql v1.7.1 - github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873 - github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4 + github.com/google/go-cmp v0.6.0 + github.com/google/uuid v1.6.0 + github.com/pingcap/tidb v1.1.0-beta.0.20240920050217-adbc0034ffb4 + github.com/pingcap/tidb/pkg/parser v0.0.0-20240920050217-adbc0034ffb4 + github.com/segmentio/kafka-go v0.4.45 github.com/thessem/zap-prettyconsole v0.3.0 - go.uber.org/zap v1.26.0 + go.uber.org/zap v1.27.0 ) require ( github.com/Code-Hex/dd v1.1.0 // indirect - github.com/alecthomas/chroma v0.10.0 // indirect github.com/beorn7/perks v1.0.1 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect github.com/cloudfoundry/gosigar v1.3.6 // indirect - github.com/cockroachdb/errors v1.8.1 // indirect - github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f // indirect - github.com/cockroachdb/redact v1.0.8 // indirect - github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 // indirect + github.com/cockroachdb/errors v1.11.1 // indirect + github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect + github.com/cockroachdb/redact v1.1.5 // indirect github.com/coreos/go-semver v0.3.1 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 // indirect github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 // indirect github.com/dlclark/regexp2 v1.4.0 // indirect - github.com/fatih/color v1.16.0 // indirect - github.com/go-ole/go-ole v1.2.6 // indirect + github.com/docker/go-units v0.5.0 // indirect + github.com/getsentry/sentry-go v0.27.0 // indirect + github.com/go-ole/go-ole v1.3.0 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang/protobuf v1.5.3 // indirect + github.com/golang/protobuf v1.5.4 // indirect github.com/google/btree v1.1.2 // indirect - github.com/google/go-cmp v0.6.0 // indirect - github.com/google/uuid v1.3.1 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect - github.com/klauspost/compress v1.17.1 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.17.9 // indirect github.com/kr/pretty v0.3.1 // indirect github.com/kr/text v0.2.0 // indirect github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a // indirect github.com/mattn/go-colorable v0.1.13 // indirect github.com/mattn/go-isatty v0.0.20 // indirect - github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect github.com/opentracing/opentracing-go v1.2.0 // indirect github.com/pierrec/lz4/v4 v4.1.15 // indirect - github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 // indirect - github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c // indirect - github.com/pingcap/kvproto v0.0.0-20230925123611-87bebcc0d071 // indirect - github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 // indirect - github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect + github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f // indirect + github.com/pingcap/failpoint v0.0.0-20240527053858-9b3b6e34194a // indirect + github.com/pingcap/kvproto v0.0.0-20240910154453-b242104f8d31 // indirect + github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d // indirect + github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect - github.com/prometheus/client_golang v1.17.0 // indirect - github.com/prometheus/client_model v0.5.0 // indirect - github.com/prometheus/common v0.45.0 // indirect - github.com/prometheus/procfs v0.12.0 // indirect + github.com/prometheus/client_golang v1.20.2 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.57.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect - github.com/rogpeppe/go-internal v1.11.0 // indirect - github.com/segmentio/kafka-go v0.4.45 // indirect - github.com/shirou/gopsutil/v3 v3.23.10 // indirect + github.com/rogpeppe/go-internal v1.12.0 // indirect + github.com/shirou/gopsutil/v3 v3.24.4 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a // indirect - github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e // indirect - github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865 // indirect + github.com/tikv/client-go/v2 v2.0.8-0.20240911041506-e7894a7b27ba // indirect + github.com/tikv/pd/client v0.0.0-20240914083230-71f6f96816e9 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/twmb/murmur3 v1.1.6 // indirect - github.com/yusufpapurcu/wmi v1.2.3 // indirect - go.etcd.io/etcd/api/v3 v3.5.10 // indirect - go.etcd.io/etcd/client/pkg/v3 v3.5.10 // indirect - go.etcd.io/etcd/client/v3 v3.5.10 // indirect + github.com/yusufpapurcu/wmi v1.2.4 // indirect + go.etcd.io/etcd/api/v3 v3.5.12 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.12 // indirect + go.etcd.io/etcd/client/v3 v3.5.12 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/exp v0.0.0-20231006140011-7918f672742d // indirect - golang.org/x/net v0.18.0 // indirect - golang.org/x/sync v0.4.0 // indirect - golang.org/x/sys v0.14.0 // indirect - golang.org/x/text v0.14.0 // indirect - google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/grpc v1.59.0 // indirect - google.golang.org/protobuf v1.31.0 // indirect + golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.25.0 // indirect + golang.org/x/text v0.18.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 // indirect + google.golang.org/grpc v1.63.2 // indirect + google.golang.org/protobuf v1.34.2 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect ) diff --git a/tests/integration_tests/debezium/go.sum b/tests/integration_tests/debezium/go.sum index 3fe1773d286..f34982e8f18 100644 --- a/tests/integration_tests/debezium/go.sum +++ b/tests/integration_tests/debezium/go.sum @@ -1,197 +1,133 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 h1:mFRzDkZVAjdal+s7s0MwaRv9igoPqLRdzOLzw/8Xvq8= github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358/go.mod h1:chxPXzSsl7ZWRAuOIE23GDNzjWuZquvFlgA8xmpunjU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/BurntSushi/toml v1.3.2 h1:o7IhLm0Msx3BaB+n3Ag7L8EVlByGnpq14C4YWiu/gL8= -github.com/BurntSushi/toml v1.3.2/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= -github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= -github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= +github.com/BurntSushi/toml v1.4.0 h1:kuoIxZQy2WRRk1pttg9asf+WVv6tWQuBNVmK8+nqPr0= +github.com/BurntSushi/toml v1.4.0/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= github.com/Code-Hex/dd v1.1.0 h1:VEtTThnS9l7WhpKUIpdcWaf0B8Vp0LeeSEsxA1DZseI= github.com/Code-Hex/dd v1.1.0/go.mod h1:VaMyo/YjTJ3d4qm/bgtrUkT2w+aYwJ07Y7eCWyrJr1w= -github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= -github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= -github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= -github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= github.com/alecthomas/chroma v0.10.0 h1:7XDcGkCQopCNKjZHfYrNLraA+M7e0fMiJ/Mfikbfjek= github.com/alecthomas/chroma v0.10.0/go.mod h1:jtJATyUxlIORhUOFNA9NZDWGAQ8wpxQQqNSB4rjA/1s= -github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= +github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3dyBCFEj5IhUbnKptjxatkF07cF2ak3yi77so= +github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= +github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= -github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= -github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= -github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= -github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= -github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= -github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= -github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= -github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= -github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= -github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= +github.com/cockroachdb/errors v1.11.1 h1:xSEW75zKaKCWzR3OfxXUxgrk/NtT4G1MiOv5lWZazG8= +github.com/cockroachdb/errors v1.11.1/go.mod h1:8MUxA3Gi6b25tYlFEBGLf+D8aISL+M4MIpiWMSNRfxw= +github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b h1:r6VH0faHjZeQy818SGhaone5OnYfxFR/+AzdY3sf5aE= +github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b/go.mod h1:Vz9DsVWQQhf3vs21MhPMZpMGSht7O/2vFW2xusFUVOs= +github.com/cockroachdb/redact v1.1.5 h1:u1PMllDkdFfPWaNGMyLD1+so+aq3uUItthCFqzwPJ30= +github.com/cockroachdb/redact v1.1.5/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64 h1:W1SHiII3e0jVwvaQFglwu3kS9NLxOeTpvik7MbKCyuQ= +github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64/go.mod h1:F86k/6c7aDUdwSUevnLpHS/3Q9hzYCE99jGk2xsHnt0= github.com/coocood/freecache v1.2.1 h1:/v1CqMq45NFH9mp/Pt142reundeBM0dVUD3osQBeu/U= github.com/coocood/freecache v1.2.1/go.mod h1:RBUWa/Cy+OHdfTGFEhEuE1pMCMX51Ncizj7rthiQ3vk= -github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= -github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= -github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 h1:NnLfQ77q0G4k2Of2c1ceQ0ec6MkLQyDp+IGdVM0D8XM= +github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= github.com/coreos/go-semver v0.3.1 h1:yi21YpKnrx1gt5R+la8n5WgS0kCrsPp33dmEyHReZr4= github.com/coreos/go-semver v0.3.1/go.mod h1:irMmmIw/7yzSRPWryHsK7EYSg09caPQL03VsM8rvUec= github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 h1:X6mKGhCFOxrKeeHAjv/3UvT6e5RRxW6wRdlqlV6/H4w= github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37/go.mod h1:DC3JtzuG7kxMvJ6dZmf2ymjNyoXwgtklr7FN+Um2B0U= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dgraph-io/ristretto v0.1.1 h1:6CWw5tJNgpegArSHpNHJKldNeq03FQCwYvfMVWajOK8= github.com/dgraph-io/ristretto v0.1.1/go.mod h1:S1GPSBCYCIhmVNfcth17y2zZtQT6wzkzgwUve0VDWWA= -github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13 h1:fAjc9m62+UWV/WAFKLNi6ZS0675eEUC9y3AlwSbQu1Y= github.com/dgryski/go-farm v0.0.0-20200201041132-a6ae2369ad13/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dlclark/regexp2 v1.4.0 h1:F1rxgk7p4uKjwIQxBs9oAXe5CqrXlCduYEJvrF4u93E= github.com/dlclark/regexp2 v1.4.0/go.mod h1:2pZnwuY/m+8K6iRw6wQdMtk+rH5tNGR1i55kozfMjCc= +github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= +github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/dolthub/maphash v0.1.0 h1:bsQ7JsF4FkkWyrP3oCnFJgrCUAFbFf3kOl4L/QxPDyQ= github.com/dolthub/maphash v0.1.0/go.mod h1:gkg4Ch4CdCDu5h6PMriVLawB7koZ+5ijb9puGMV50a4= github.com/dolthub/swiss v0.2.1 h1:gs2osYs5SJkAaH5/ggVJqXQxRXtWshF6uE0lgR/Y3Gw= github.com/dolthub/swiss v0.2.1/go.mod h1:8AhKZZ1HK7g18j7v7k6c5cYIGEZJcPn0ARsai8cUrh0= -github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= -github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= +github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= +github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= -github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= -github.com/fatih/color v1.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM= -github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE= -github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= -github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= -github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= -github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/fatih/color v1.17.0 h1:GlRw1BRJxkpqUCBKzKOw098ed57fEsKeNjpTe3cSjK4= +github.com/fatih/color v1.17.0/go.mod h1:YZ7TlrGPkiz6ku9fK3TLD/pl3CpsiFyu8N92HLgmosI= github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= -github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= -github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= -github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= +github.com/getsentry/sentry-go v0.27.0 h1:Pv98CIbtB3LkMWmXi4Joa5OOcwbmnX88sF5qbK3r3Ps= +github.com/getsentry/sentry-go v0.27.0/go.mod h1:lc76E2QywIyW8WuBnwl8Lc4bkmQH4+w1gwTf25trprY= github.com/go-asn1-ber/asn1-ber v1.5.4 h1:vXT6d/FNDiELJnLb6hGNa309LMsrCoYFvpwHDF0+Y1A= github.com/go-asn1-ber/asn1-ber v1.5.4/go.mod h1:hEBeB/ic+5LoWskz+yKT7vGhhPYkProFKoKdwZRWMe0= -github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= -github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= -github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= -github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-errors/errors v1.4.2 h1:J6MZopCL4uSllY1OfXM374weqZFFItUbrImctkmUxIA= +github.com/go-errors/errors v1.4.2/go.mod h1:sIVyrIiJhuEF+Pj9Ebtd6P/rEYROXFi3BopGUQ5a5Og= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-ldap/ldap/v3 v3.4.4 h1:qPjipEpt+qDa6SI/h1fzuGWoRUY+qqQ9sOZq67/PYUs= github.com/go-ldap/ldap/v3 v3.4.4/go.mod h1:fe1MsuN5eJJ1FeLT/LEBVdWfNWKh459R7aXgXtJC+aI= -github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= -github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= +github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrtU8EI= github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= -github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= -github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= 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.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/glog v1.1.2 h1:DVjP2PbBOzHyzA+dn3WhHIq4NdVu3Q+pvivFICf/7fo= -github.com/golang/glog v1.1.2/go.mod h1:zR+okUeTbrL6EL3xHUDxZuEtGv04p5shwip1+mL/rLQ= +github.com/golang/glog v1.2.0 h1:uCdmnmatrKCgMBlM4rMuJZWOkPDqdbZPnrMXDY4gI68= +github.com/golang/glog v1.2.0/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= -github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= -github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= -github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= -github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= -github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -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/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v1.1.2 h1:xf4v41cLI2Z6FxbKm+8Bu+m8ifhj15JuZ9sa0jZCMUU= github.com/google/btree v1.1.2/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= -github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= -github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= -github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= -github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= -github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= -github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= -github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= -github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= -github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= +github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= github.com/influxdata/tdigest v0.0.1 h1:XpFptwYmnEKUqmkcDjrzffswZ3nvNeevbUSLPP/ZzIY= github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= -github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= -github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= -github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= -github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= -github.com/jellydator/ttlcache/v3 v3.0.1 h1:cHgCSMS7TdQcoprXnWUptJZzyFsqs18Lt8VVhRuZYVU= -github.com/jellydator/ttlcache/v3 v3.0.1/go.mod h1:WwTaEmcXQ3MTjOm4bsZoDFiCu/hMvNWLO1w67RXz6h4= -github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= -github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= -github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= -github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= -github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= -github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= -github.com/kataras/iris/v12 v12.0.1/go.mod h1:udK4vLQKkdDqMGJJVd/msuMtN6hpYJhg/lSzuxjhO+U= -github.com/kataras/neffos v0.0.10/go.mod h1:ZYmJC07hQPW67eKuzlfY7SO3bC0mw83A3j6im82hfqw= -github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiDuX9AhMbDPkGYSPugBOV6yTZB1l2K9Z0= -github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= +github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= -github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.17.1 h1:NE3C767s2ak2bweCZo3+rdP4U/HoyVXLv/X9f2gPS5g= -github.com/klauspost/compress v1.17.1/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= -github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/klauspost/cpuid v1.3.1 h1:5JNjFYYQrZeKRJ0734q51WCEEn2huer72Dc7K+R/b6s= +github.com/klauspost/cpuid v1.3.1/go.mod h1:bYW4mA6ZgKPob1/Dlai2LviZJO7KGI3uoWLd42rAQw4= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= @@ -199,49 +135,31 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= -github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a h1:N9zuLhTvBSRt0gWSiJswwQ2HqDmtX/ZCDJURnKUt1Ik= github.com/lufia/plan9stats v0.0.0-20230326075908-cb1d2100619a/go.mod h1:JKx41uQRwqlTZabZc+kILPrO/3jlKnQ2Z8b7YiVw5cE= -github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= -github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= -github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= -github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= -github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= -github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= -github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= -github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= -github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= -github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= -github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= -github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= -github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= -github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= -github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= +github.com/ncw/directio v1.0.5 h1:JSUBhdjEvVaJvOoyPAbcW0fnd0tvRXD76wEfZ1KcQz4= +github.com/ncw/directio v1.0.5/go.mod h1:rX/pKEYkOXBGOggmcyJeJGloCkleSvphPx2eV3t6ROk= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 h1:7KAv7KMGTTqSmYZtNdcNTgsos+vFzULLwyElndwn+5c= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7/go.mod h1:iWMfgwqYW+e8n5lC/jjNEhwcjbRDpl5NT7n2h+4UNcI= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3bPQ/0cuYh2H4rkg0tytX/07k= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8= -github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= -github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= github.com/onsi/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/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= @@ -249,106 +167,104 @@ github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKw github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36 h1:64bxqeTEN0/xoEqhKGowgihNuzISS9rEG6YUMU4bzJo= -github.com/petermattis/goid v0.0.0-20211229010228-4d14c490ee36/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= -github.com/pierrec/lz4 v2.6.1+incompatible h1:9UY3+iC23yxF0UfGaYrGplQ+79Rg+h/q9FV9ix19jjM= +github.com/otiai10/copy v1.2.0 h1:HvG945u96iNadPoG2/Ja2+AUJeW5YuFQMixq9yirC+k= +github.com/otiai10/copy v1.2.0/go.mod h1:rrF5dJ5F0t/EWSYODDu4j9/vEeYHMkc8jt0zJChqQWw= +github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7 h1:Dx7Ovyv/SFnMFw3fD4oEoeorXc6saIiQ23LrGLth0Gw= +github.com/petermattis/goid v0.0.0-20240813172612-4fcff4a6cae7/go.mod h1:pxMtw7cyUw6B2bRH0ZBANSPg+AoSud1I1iyJHI69jH4= github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d h1:AEcvKyVM8CUII3bYzgz8haFXtGiqcrtXW1csu/5UELY= +github.com/pingcap/badger v1.5.1-0.20230103063557-828f39b09b6d/go.mod h1:p8QnkZnmyV8L/M/jzYb8rT7kv3bz9m7bn1Ju94wDifs= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 h1:m5ZsBa5o/0CkzZXfXLaThzKuR85SnHHetqBCpzQ30h8= -github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= -github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= +github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f h1:FxA+NgsdHNOv+/hZGxUh8Gb3WuZqgqmxDwztEOiA1v4= +github.com/pingcap/errors v0.11.5-0.20240318064555-6bd07397691f/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= +github.com/pingcap/failpoint v0.0.0-20240527053858-9b3b6e34194a h1:UgrcL8INjEbPRKE2h8yVgZvjOn2OGkxK9CFvoBWzgbk= +github.com/pingcap/failpoint v0.0.0-20240527053858-9b3b6e34194a/go.mod h1:gPdo4h708R0CrwKM/DO0/6xJ64fz9vxzp2yKE2QON+s= 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-20230925123611-87bebcc0d071 h1:giqmIJSWHs+jhHfd+rth8CXWR18KAtqJu4imY1YdA6o= -github.com/pingcap/kvproto v0.0.0-20230925123611-87bebcc0d071/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20240910154453-b242104f8d31 h1:6BY+3T6Hqpw9UZ/D7Om/xB+Xik3NkkYxBV6qCzUdUvU= +github.com/pingcap/kvproto v0.0.0-20240910154453-b242104f8d31/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= -github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= -github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= -github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873 h1:IOQPa3ItEg9RrCBgkLGO6VsKRh72LXAbdKiSfPuPQmw= -github.com/pingcap/tidb v1.1.0-beta.0.20231117065153-a4f85c356873/go.mod h1:yXjpyctIACAIDBLGwwRis4X8MDUO8sFS4QerAsOouAc= -github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4 h1:xrrIEHBBSeBuCL6mbyoMG4R4/ro+WpEql9XKFWtDXRw= -github.com/pingcap/tidb/pkg/parser v0.0.0-20231116213047-1f7c1e02bcd4/go.mod h1:yRkiqLFwIqibYg2P7h4bclHjHcJiIFRLKhGRyBcKYus= -github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f h1:NCiI4Wyu4GkViLGTu6cYcxt79LZ1SenBBQX1OwEV6Jg= -github.com/pingcap/tipb v0.0.0-20230919054518-dfd7d194838f/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/log v1.1.1-0.20240314023424-862ccc32f18d h1:y3EueKVfVykdpTyfUnQGqft0ud+xVFuCdp1XkVL0X1E= +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.20240920050217-adbc0034ffb4 h1:BnoMi+RibczSp8TjfPcvb2RVviQQvIjfqDM8fn4/xwU= +github.com/pingcap/tidb v1.1.0-beta.0.20240920050217-adbc0034ffb4/go.mod h1:XqXA8+jTqBZ4bu/xNjWSzBK33nIB1z0adUv0vYXQgVE= +github.com/pingcap/tidb/pkg/parser v0.0.0-20240920050217-adbc0034ffb4 h1:GBf6ho/7J4HVf5N+DaCfUfP7B0Yt1k48GgMBv/SvBjs= +github.com/pingcap/tidb/pkg/parser v0.0.0-20240920050217-adbc0034ffb4/go.mod h1:bGciNq+1X3xXPyRvgHFPKh2naQVH6HPiBbfqK3qSqpw= +github.com/pingcap/tipb v0.0.0-20240823074000-a40c2347786e h1:chR6iRwU9MUivOYNH+26UqrO8Y7t3ZltX+Jukv+f+iM= +github.com/pingcap/tipb v0.0.0-20240823074000-a40c2347786e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b h1:0LFwY6Q3gMACTjAbMZBjXAqTOzOwFaj2Ld6cjeQ7Rig= github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= -github.com/prometheus/client_golang v1.17.0 h1:rl2sfwZMtSthVU752MqfjQozy7blglC+1SOtjMAMh+Q= -github.com/prometheus/client_golang v1.17.0/go.mod h1:VeL+gMmOAxkS2IqfCq0ZmHSL+LjWfWDUmp1mBz9JgUY= +github.com/prometheus/client_golang v1.20.2 h1:5ctymQzZlyOON1666svgwn3s6IKWgfbjsejTMiXIyjg= +github.com/prometheus/client_golang v1.20.2/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= -github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= -github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= -github.com/prometheus/common v0.45.0/go.mod h1:YJmSTw9BoKxJplESWWxlbyttQR4uaEcGyv9MZjVOJsY= -github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= -github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.57.0 h1:Ro/rKjwdq9mZn1K5QPctzh+MA4Lp0BuYk5ZZEVhoNcY= +github.com/prometheus/common v0.57.0/go.mod h1:7uRPFSUTbfZWsJ7MHY56sqt7hLQu3bxXHDnNhl8E9qI= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/qri-io/jsonpointer v0.1.1 h1:prVZBZLL6TW5vsSB9fFHFAMBLI4b0ri5vribQlTJiBA= +github.com/qri-io/jsonpointer v0.1.1/go.mod h1:DnJPaYgiKu56EuDp8TU5wFLdZIcAnb/uH9v37ZaMV64= +github.com/qri-io/jsonschema v0.2.1 h1:NNFoKms+kut6ABPf6xiKNM5214jzxAhDBrPHCJ97Wg0= +github.com/qri-io/jsonschema v0.2.1/go.mod h1:g7DPkiOsK1xv6T/Ao5scXRkd+yTFygcANPBaaqW+VrI= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec h1:W09IVJc94icq4NjY3clb7Lk8O1qJ8BdBEF8z0ibU0rE= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= -github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= -github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= -github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/sasha-s/go-deadlock v0.2.0 h1:lMqc+fUb7RrFS3gQLtoQsJ7/6TV/pAIFvBsqX73DK8Y= -github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= -github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= +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/sasha-s/go-deadlock v0.3.5 h1:tNCOEEDG6tBqrNDOX35j/7hL5FcFViG6awUGROb2NsU= +github.com/sasha-s/go-deadlock v0.3.5/go.mod h1:bugP6EGbdGYObIlx7pUZtWqlvo8k9H6vCBBsiChJQ5U= github.com/segmentio/kafka-go v0.4.45 h1:prqrZp1mMId4kI6pyPolkLsH6sWOUmDxmmucbL4WS6E= github.com/segmentio/kafka-go v0.4.45/go.mod h1:HjF6XbOKh0Pjlkr5GVZxt6CsjjwnmhVOfURM5KMd8qg= -github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= -github.com/shirou/gopsutil/v3 v3.23.10 h1:/N42opWlYzegYaVkWejXWJpbzKv2JDy3mrgGzKsh9hM= -github.com/shirou/gopsutil/v3 v3.23.10/go.mod h1:JIE26kpucQi+innVlAUnIEOSBhBUkirr5b44yr55+WE= +github.com/shirou/gopsutil/v3 v3.24.4 h1:dEHgzZXt4LMNm+oYELpzl9YCqV65Yr/6SfrvgRBtXeU= +github.com/shirou/gopsutil/v3 v3.24.4/go.mod h1:lTd2mdiOspcqLgAnr9/nGi71NkeMpWKdmhuxm9GusH8= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= -github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= -github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= -github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/thessem/zap-prettyconsole v0.3.0 h1:jreGIwOwkfqpA1NWTsbXXr0ZSL68b39KpH54q9Hz58U= github.com/thessem/zap-prettyconsole v0.3.0/go.mod h1:93z1PhlPAYOWIOhPJvNkxE9cVM8jE5xB6/6HCtnvmcA= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= -github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e h1:kl8+gDOfPfRqkc1VDhhjhezMvsbfRENYsm/FqSIDnwg= -github.com/tikv/client-go/v2 v2.0.8-0.20231114060955-8fc8a528217e/go.mod h1:fEAE7GS/lta+OasPOacdgy6RlJIRaq9/Cyr2WbSYcBE= -github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865 h1:Gkvo77EevOpBGIdV1c8gwRqPhVbgLPRy82tXNEFpGTc= -github.com/tikv/pd/client v0.0.0-20231114041114-86831ce71865/go.mod h1:cd6zBqRM9aogxf26K8NnFRPVtq9BnRE59tKEpX8IaWQ= +github.com/tidwall/btree v1.7.0 h1:L1fkJH/AuEh5zBnnBbmTwQ5Lt+bRJ5A8EWecslvo9iI= +github.com/tidwall/btree v1.7.0/go.mod h1:twD9XRA5jj9VUQGELzDO4HPQTNJsoWWfYEL+EUQ2cKY= +github.com/tikv/client-go/v2 v2.0.8-0.20240911041506-e7894a7b27ba h1:dwuuYqPYxU0xcv0bnDgT7I4btQ5c3joBG1HmNOhCTdo= +github.com/tikv/client-go/v2 v2.0.8-0.20240911041506-e7894a7b27ba/go.mod h1:4HDOAx8OXAJPtqhCZ03IhChXgaFs4B3+vSrPWmiPxjg= +github.com/tikv/pd/client v0.0.0-20240914083230-71f6f96816e9 h1:J9LChGMzo95eBrjE03NHITDWgxfPgskH+QrCnlW61/Y= +github.com/tikv/pd/client v0.0.0-20240914083230-71f6f96816e9/go.mod h1:uBHhxAM/SPCMabt483gI/pN/+JXIMKYXohK96s+PwT8= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= @@ -361,38 +277,25 @@ github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR8 github.com/uber/jaeger-client-go v2.22.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= -github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= -github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= -github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= -github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= -github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= -github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= +github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY= github.com/xdg-go/scram v1.1.2/go.mod h1:RT/sEzTbU5y00aCK8UOx6R7YryM0iF1N2MOmC3kKLN4= +github.com/xdg-go/stringprep v1.0.4 h1:XLI/Ng3O1Atzq0oBs3TWm+5ZVgkq2aqdlvP9JtoZ6c8= github.com/xdg-go/stringprep v1.0.4/go.mod h1:mPGuuIYwz7CmR2bT9j4GbQqutWS1zV24gijq1dTyGkM= -github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= -github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= -github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= -github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= -github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= -github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= -github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= -github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= -github.com/yusufpapurcu/wmi v1.2.3 h1:E1ctvB7uKFMOJw3fdOW32DwGE9I7t++CRUEMKvFoFiw= -github.com/yusufpapurcu/wmi v1.2.3/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= -go.etcd.io/etcd/api/v3 v3.5.10 h1:szRajuUUbLyppkhs9K6BRtjY37l66XQQmw7oZRANE4k= -go.etcd.io/etcd/api/v3 v3.5.10/go.mod h1:TidfmT4Uycad3NM/o25fG3J07odo4GBB9hoxaodFCtI= -go.etcd.io/etcd/client/pkg/v3 v3.5.10 h1:kfYIdQftBnbAq8pUWFXfpuuxFSKzlmM5cSn76JByiT0= -go.etcd.io/etcd/client/pkg/v3 v3.5.10/go.mod h1:DYivfIviIuQ8+/lCq4vcxuseg2P2XbHygkKwFo9fc8U= -go.etcd.io/etcd/client/v3 v3.5.10 h1:W9TXNZ+oB3MCd/8UjxHTWK5J9Nquw9fQBLJd5ne5/Ao= -go.etcd.io/etcd/client/v3 v3.5.10/go.mod h1:RVeBnDz2PUEZqTpgqwAtUd8nAPf5kjyFyND7P1VkOKc= +github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo0= +github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= +go.etcd.io/etcd/api/v3 v3.5.12 h1:W4sw5ZoU2Juc9gBWuLk5U6fHfNVyY1WC5g9uiXZio/c= +go.etcd.io/etcd/api/v3 v3.5.12/go.mod h1:Ot+o0SWSyT6uHhA56al1oCED0JImsRiU9Dc26+C2a+4= +go.etcd.io/etcd/client/pkg/v3 v3.5.12 h1:EYDL6pWwyOsylrQyLp2w+HkQ46ATiOvoEdMarindU2A= +go.etcd.io/etcd/client/pkg/v3 v3.5.12/go.mod h1:seTzl2d9APP8R5Y2hFL3NVlD6qC/dOT+3kvrqPyTas4= +go.etcd.io/etcd/client/v3 v3.5.12 h1:v5lCPXn1pf1Uu3M4laUE2hp/geOTc5uPcYYsNe1lDxg= +go.etcd.io/etcd/client/v3 v3.5.12/go.mod h1:tSbBCakoWmmddL+BKVAJHa9km+O/E+bumDe9mSbPiqw= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= -go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -402,8 +305,8 @@ go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0 go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= -go.uber.org/mock v0.3.0 h1:3mUxI1No2/60yUYax92Pt8eNOEecx2D3lcXZh2NEZJo= -go.uber.org/mock v0.3.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc= +go.uber.org/mock v0.4.0 h1:VcM4ZOtdbR4f6VXfiOpwpVJDL6lCReaZ6mw31wqh7KU= +go.uber.org/mock v0.4.0/go.mod h1:a6FSlNadKUHUa9IP5Vyt1zh4fC7uAwxMutEAscFbkZc= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= @@ -413,24 +316,22 @@ go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= +go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= -go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= -golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= +go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= -golang.org/x/crypto v0.15.0 h1:frVn1TEaCEaZcn3Tmd7Y2b5KKPaZ+I32Q2OA3kYp5TA= -golang.org/x/crypto v0.15.0/go.mod h1:4ChreQoLWfG3xLDer1WdlH5NdlQ3+mwnQq1YTKY+72g= +golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= +golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231006140011-7918f672742d h1:jtJma62tbqLibJ5sFQz8bKtEM8rJBtfilJ2qTU199MI= -golang.org/x/exp v0.0.0-20231006140011-7918f672742d/go.mod h1:ldy0pHrwJyGW56pPQzzkH36rKxoZW1tw7ZJpeKx+hdo= +golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0 h1:985EYyeCOxTpcgOTJpflJUwOeEz0CQOdPt73OzpE9F8= +golang.org/x/exp v0.0.0-20240404231335-c0f41cb1a7a0/go.mod h1:/lliqkxwWAhPjf5oSOIJup2XcqJaw8RGS6k3TGEc7GI= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -442,92 +343,75 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= 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.18.0 h1:mIYleuAkSbHh0tCv7RvjL3F6ZVbLjq4+R7zbOn3Kokg= -golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/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.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= -golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= 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= -golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 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.14.0 h1:Vz7Qs629MkJkGyHxUlRHizWJRG2j8fbQKjELVSNhy7Q= -golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.19.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= +golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= 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= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= 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/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= -golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= 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.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= -golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= +golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -538,52 +422,37 @@ golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= 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.14.0 h1:jvNa2pY0M4r62jkRQ6RwEZZyPcymeL9XZMLBbV7U2nc= -golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= +golang.org/x/tools v0.24.0 h1:J1shsA93PJUEVaUSaay7UXAyE8aimq3GW0pjlolpa24= +golang.org/x/tools v0.24.0/go.mod h1:YhNqVBIfWHdzvTLs0d8LCuMhkKUgSUKldakyV7W/WDQ= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b h1:+YaDE2r2OG8t/z5qmsh7Y+XXwCbvadxxZ0YY6mTdrVA= -google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:CgAqfJo+Xmu0GwA0411Ht3OU3OntXwsGmrmjI8ioGXI= -google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b h1:CIC2YMXmIhYw6evmhPxBKJ4fmLbOFtXQN/GV3XOZR8k= -google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:IBQ646DjkDkvUIsVq/cc03FUFQ9wbZu7yE396YcL870= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b h1:ZlWIi1wSK56/8hn4QcBp/j9M7Gt3U/3hZw3mC7vDICo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc= +google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237 h1:RFiFrvy37/mpSpdySBDrUdipW/dHwsRwh3J3+A9VgT4= +google.golang.org/genproto/googleapis/api v0.0.0-20240318140521-94a12d6c2237/go.mod h1:Z5Iiy3jtmioajWHDGFk7CeugTyHtPvMHA4UTmUkyalE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 h1:AgADTJarZTBqgjiUzRgfaBchgYB3/WFTC80GPwsMcRI= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= -google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= -google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= -google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= -google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= -google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= -google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= -google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= -google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= -google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= -google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/grpc v1.63.2 h1:MUeiw1B2maTVZthpU5xvASfTh3LDbxHd6IJ6QQVU+xM= +google.golang.org/grpc v1.63.2/go.mod h1:WAX/8DgncnokcFUldAxq7GeB5DXHDbMF+lLvDomNkRA= +google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9 h1:ATnmU8nL2NfIyTSiBvJVDIDIr3qBmeW+c7z7XU21eWs= +google.golang.org/grpc/examples v0.0.0-20231221225426-4f03f3ff32c9/go.mod h1:j5uROIAAgi3YmtiETMt1LW0d/lHqQ7wwrIY4uGRXLQ4= +google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= +google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= -gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= -gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= -gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYskCTPBJVb9jqSc= @@ -592,10 +461,10 @@ gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWD gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20200605160147-a5ece683394c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/tests/integration_tests/debezium/run.sh b/tests/integration_tests/debezium/run.sh index 5640312ccde..19e8d766577 100644 --- a/tests/integration_tests/debezium/run.sh +++ b/tests/integration_tests/debezium/run.sh @@ -35,7 +35,7 @@ curl -i -X POST \ "database.user": "debezium", "database.password": "dbz", "database.server.id": "184054", - "topic.prefix": "dbserver1", + "topic.prefix": "default", "schema.history.internal.kafka.bootstrap.servers": "127.0.0.1:9092", "schema.history.internal.kafka.topic": "schemahistory.test", "transforms": "x", @@ -50,7 +50,7 @@ EOF start_tidb_cluster --workdir $WORK_DIR run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY -run_cdc_cli changefeed create --sink-uri="kafka://127.0.0.1:9092/output_ticdc?protocol=debezium&kafka-version=2.4.0" +run_cdc_cli changefeed create -c test --sink-uri="kafka://127.0.0.1:9092/output_ticdc?protocol=debezium&kafka-version=2.4.0" --config "$CUR/changefeed.toml" cd $CUR go run ./src diff --git a/tests/integration_tests/debezium/sql/data_types.sql b/tests/integration_tests/debezium/sql/data_types.sql index 48452db04fe..53489702764 100644 --- a/tests/integration_tests/debezium/sql/data_types.sql +++ b/tests/integration_tests/debezium/sql/data_types.sql @@ -37,8 +37,7 @@ CREATE TABLE t_datetime( col_5 DATETIME(5), col_6 DATETIME(6), col_z DATETIME DEFAULT 0, - -- TODO: - -- col_default_current_timestamp DATETIME DEFAULT CURRENT_TIMESTAMP, + col_default_current_timestamp DATETIME DEFAULT CURRENT_TIMESTAMP, pk INT PRIMARY KEY ); @@ -52,7 +51,7 @@ INSERT INTO t_datetime VALUES ( '2023-11-16 12:34:56.123456', '2023-11-16 12:34:56.123456', NULL, - -- '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', 1 ); @@ -66,7 +65,7 @@ INSERT INTO t_datetime VALUES ( '2023-11-16 12:34:56', '2023-11-16 12:34:56', NULL, - -- '2023-11-16 12:34:56', + '2023-11-16 12:34:56', 2 ); @@ -82,7 +81,7 @@ INSERT INTO t_datetime VALUES ( '2023-11-16 12:34:56.123456', '2023-11-16 12:34:56.123456', NULL, - -- '2023-11-16 12:34:56.123456', + '2023-11-16 12:34:56.123456', 3 ); @@ -98,7 +97,7 @@ INSERT INTO t_datetime VALUES ( NULL, NULL, NULL, - -- NULL, + NULL, 4 ); @@ -144,27 +143,6 @@ INSERT INTO t_time VALUES ( 3 ); -/* - -Commented out because Debezium produce wrong result: - -"col":-3020399000000, -"col_0":-3020399000000, -"col_1":-3020400147483, -"col_5":-3020399048576, -"col_6":-3020399048576, - -INSERT INTO t_time VALUES ( - '-838:59:59.000000', - '-838:59:59.000000', - '-838:59:59.000000', - '-838:59:59.000000', - '-838:59:59.000000', - 3 -); - -*/ - INSERT INTO t_time VALUES ( '838:59:59.000000', '838:59:59.000000', @@ -183,6 +161,25 @@ INSERT INTO t_time VALUES ( 5 ); +/* + +Commented out because Debezium produce wrong result: + +"col":-3020399000000, +"col_0":-3020399000000, +"col_1":-3020400147483, +"col_5":-3020399048576, +"col_6":-3020399048576, +INSERT INTO t_time VALUES ( + '-838:59:59.000000', + '-838:59:59.000000', + '-838:59:59.000000', + '-838:59:59.000000', + '-838:59:59.000000', + 6 +); +*/ + /* ---------------------------------------------------------------------- -- TIMESTAMP type diff --git a/tests/integration_tests/debezium/sql/ddl.sql b/tests/integration_tests/debezium/sql/ddl.sql new file mode 100644 index 00000000000..3a28efba4a1 --- /dev/null +++ b/tests/integration_tests/debezium/sql/ddl.sql @@ -0,0 +1,135 @@ +CREATE TABLE t1 ( + PK INT PRIMARY KEY, + COL INT +); + +CREATE DATABASE foo; +USE foo; + +CREATE TABLE bar ( + PK INT PRIMARY KEY AUTO_INCREMENT, + COL INT +); +INSERT INTO bar VALUES (1, 1); + +/* VIEW */ +CREATE VIEW V1 AS +SELECT * +FROM bar +WHERE COL > 2; +DROP VIEW IF EXISTS V1; + +/* ALTER COLUMN */ +ALTER TABLE bar +ADD COLUMN COL2 INT; +ALTER TABLE bar +MODIFY COLUMN COL2 FLOAT; +ALTER TABLE bar +DROP COLUMN COL2; + +/* Rebase AutoID */ +ALTER TABLE bar AUTO_INCREMENT=310; + +/* Set DEFAULT value */ +ALTER TABLE bar +ALTER COL SET DEFAULT 3; + +/* Modify TABLE comment */ +ALTER TABLE bar COMMENT = 'New table comment'; + +/* Modify TABLE charset */ +ALTER TABLE bar CHARACTER SET = utf8mb4 COLLATE utf8mb4_unicode_ci; + +/* Modify DATABASE charset */ +ALTER DATABASE foo CHARACTER SET utf8mb4 COLLATE utf8mb4_unicode_ci; + +CREATE TABLE t1 ( + PK INT PRIMARY KEY, + COL INT +); + +/* MultiSchemaChange */ +ALTER TABLE test.t1 +CHANGE COL COL2 VARCHAR(255); +ALTER TABLE foo.t1 +CHANGE COL COL2 VARCHAR(255); + +/* PARTITION */ +CREATE TABLE t2 ( + id INT NOT NULL, + year_col INT NOT NULL +) +PARTITION BY RANGE (year_col) ( + PARTITION p0 VALUES LESS THAN (1991), + PARTITION p1 VALUES LESS THAN (1995), + PARTITION p2 VALUES LESS THAN (1999) +); +ALTER TABLE t2 ADD PARTITION (PARTITION p3 VALUES LESS THAN (2002)); +ALTER TABLE t2 REORGANIZE PARTITION p3 INTO ( + PARTITION p31 VALUES LESS THAN (2002), + PARTITION p32 VALUES LESS THAN (2005), + PARTITION p33 VALUES LESS THAN (2008) +); +ALTER TABLE t2 REORGANIZE PARTITION p31,p32,p33,p2 INTO (PARTITION p21 VALUES LESS THAN (2008)); +ALTER TABLE t2 TRUNCATE PARTITION p0; +ALTER TABLE t2 DROP PARTITION p0; + +/* ALTER INDEX visibility */ +CREATE TABLE t3 ( + i INT, + j INT, + k INT, + INDEX i_idx (i) INVISIBLE +) ENGINE = InnoDB; +CREATE INDEX j_idx ON t3 (j) INVISIBLE; +ALTER TABLE t3 ADD INDEX k_idx (k) INVISIBLE; +ALTER TABLE t3 ALTER INDEX i_idx VISIBLE; +ALTER TABLE t3 ALTER INDEX i_idx INVISIBLE; + +/* RENAME TABLE */ +RENAME TABLE t3 TO renam_t3; + +/* INDEX */ +CREATE TABLE t4 (col1 INT PRIMARY KEY, col2 INT); +CREATE INDEX idx1 ON t4 ((col1 + col2)); +CREATE INDEX idx2 ON t4 ((col1 + col2), (col1 - col2), col1); +DROP INDEX idx1 ON t4; +ALTER TABLE t4 ADD INDEX ((col1 * 40) DESC); +ALTER TABLE t4 RENAME INDEX idx2 TO new_idx2; + +/* + Adding a new column and setting it to the PRIMARY KEY is not supported. + https://docs.pingcap.com/tidb/stable/sql-statement-add-column#mysql-compatibility + ALTER TABLE t4 ADD COLUMN `id` INT(10) primary KEY; +*/ +/* + Dropping primary key columns or columns covered by the composite index is not supported. + https://docs.pingcap.com/tidb/stable/sql-statement-drop-column#mysql-compatibility + ALTER TABLE t4 DROP PRIMARY KEY; +*/ + +/* EXCHANGE PARTITION */ +CREATE TABLE t5 ( + id INT NOT NULL PRIMARY KEY, + fname VARCHAR(30), + lname VARCHAR(30) +) + PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (50), + PARTITION p1 VALUES LESS THAN (100), + PARTITION p2 VALUES LESS THAN (150), + PARTITION p3 VALUES LESS THAN (MAXVALUE) +); +INSERT INTO t5 VALUES (1669, "Jim", "Smith"); +CREATE TABLE t6 LIKE t5; +ALTER TABLE t6 REMOVE PARTITIONING; +ALTER TABLE foo.t5 EXCHANGE PARTITION p0 WITH TABLE foo.t6; + +/* + Debezium does not support recover table. + DROP TABLE t1; + RECOVER TABLE t1; +*/ + +DROP TABLE foo.bar; +DROP DATABASE IF EXISTS foo; diff --git a/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql b/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql index 245452e0e27..0925bcfb23e 100644 --- a/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/binary_column_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: binary_column_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_254_binary_column_test ( id INT AUTO_INCREMENT NOT NULL, file_uuid BINARY(16), diff --git a/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql b/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql index 8ad20ccad17..448e55282e3 100644 --- a/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/binary_mode_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: binary_column_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_1814_binary_mode_test ( id INT AUTO_INCREMENT NOT NULL, blob_col BLOB NOT NULL, diff --git a/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql b/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql new file mode 100644 index 00000000000..3b013498585 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/connector_read_binary_field_test.sql @@ -0,0 +1,15 @@ +SET time_zone = 'UTC'; +-- Create a table, mainly MySQL time type fields +CREATE TABLE binary_field +( + id INT AUTO_INCREMENT PRIMARY KEY, + now_time TIME, + now_date DATE, + now_date_time DATETIME, + now_time_stamp TIMESTAMP +) ENGINE = innodb + AUTO_INCREMENT = 1 + DEFAULT CHARSET = utf8; + +INSERT INTO binary_field +VALUES (default, now(), now(), now(), now()); diff --git a/tests/integration_tests/debezium/sql/debezium/connector_test.sql b/tests/integration_tests/debezium/sql/debezium/connector_test.sql index 8ae03647b01..82b9564fb80 100644 --- a/tests/integration_tests/debezium/sql/debezium/connector_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/connector_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: connector_test --- ---------------------------------------------------------------------------------------------------------------- - -- Create and populate our products using a single insert with many rows CREATE TABLE products ( id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, @@ -12,15 +8,23 @@ CREATE TABLE products ( ALTER TABLE products AUTO_INCREMENT = 101; INSERT INTO products -VALUES (default,"scooter","Small 2-wheel scooter",3.14), - (default,"car battery","12V car battery",8.1), - (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), - (default,"hammer","12oz carpenter's hammer",0.75), - (default,"hammer","14oz carpenter's hammer",0.875), - (default,"hammer","16oz carpenter's hammer",1.0), - (default,"rocks","box of assorted rocks",5.3), - (default,"jacket","water resistent black wind breaker",0.1), - (default,"spare tire","24 inch spare tire",22.2); +VALUES (default,"scooter","Small 2-wheel scooter",3.14); +INSERT INTO products +VALUES (default,"car battery","12V car battery",8.1); +INSERT INTO products +VALUES (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8); +INSERT INTO products +VALUES (default,"hammer","12oz carpenter's hammer",0.75); +INSERT INTO products +VALUES (default,"hammer","14oz carpenter's hammer",0.875); +INSERT INTO products +VALUES (default,"hammer","16oz carpenter's hammer",1.0); +INSERT INTO products +VALUES (default,"rocks","box of assorted rocks",5.3); +INSERT INTO products +VALUES (default,"jacket","water resistent black wind breaker",0.1); +INSERT INTO products +VALUES (default,"spare tire","24 inch spare tire",22.2); -- Create and populate the products on hand using multiple inserts CREATE TABLE products_on_hand ( @@ -49,10 +53,13 @@ CREATE TABLE customers ( INSERT INTO customers -VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), - (default,"George","Bailey","gbailey@foobar.com"), - (default,"Edward","Walker","ed@walker.com"), - (default,"Anne","Kretchmar","annek@noanswer.org"); +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"); +INSERT INTO customers +VALUES (default,"George","Bailey","gbailey@foobar.com"); +INSERT INTO customers +VALUES (default,"Edward","Walker","ed@walker.com"); +INSERT INTO customers +VALUES (default,"Anne","Kretchmar","annek@noanswer.org"); -- Create some very simple orders CREATE TABLE orders ( @@ -66,8 +73,12 @@ CREATE TABLE orders ( ) AUTO_INCREMENT = 10001; INSERT INTO orders -VALUES (default, '2016-01-16', 1001, 1, 102), - (default, '2016-01-17', 1002, 2, 105), - (default, '2016-02-18', 1004, 3, 109), - (default, '2016-02-19', 1002, 2, 106), - (default, '16-02-21', 1003, 1, 107); +VALUES (default, '2016-01-16', 1001, 1, 102); +INSERT INTO orders +VALUES (default, '2016-01-17', 1002, 2, 105); +INSERT INTO orders +VALUES (default, '2016-02-18', 1004, 3, 109); +INSERT INTO orders +VALUES (default, '2016-02-19', 1002, 2, 106); +INSERT INTO orders +VALUES (default, '16-02-21', 1003, 1, 107); diff --git a/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql b/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql index 22ab5d1dda6..bf01c629293 100644 --- a/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql +++ b/tests/integration_tests/debezium/sql/debezium/connector_test_ro.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: connector_test_ro --- ---------------------------------------------------------------------------------------------------------------- - -- Create and populate our products using a single insert with many rows CREATE TABLE Products ( PRIMARY KEY (id), @@ -13,15 +9,23 @@ CREATE TABLE Products ( ALTER TABLE Products AUTO_INCREMENT = 101; INSERT INTO Products -VALUES (default,"scooter","Small 2-wheel scooter",3.14), - (default,"car battery","12V car battery",8.1), - (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8), - (default,"hammer","12oz carpenter's hammer",0.75), - (default,"hammer2","14oz carpenter's hammer",8.75E-1), - (default,"hammer3","16oz carpenter's hammer",1.0), - (default,"rocks","box of assorted rocks",5.3), - (default,"jacket","water resistent black wind breaker",0.1), - (default,"spare tire","24 inch spare tire",22.2); +VALUES (default,"scooter","Small 2-wheel scooter",3.14); +INSERT INTO Products +VALUES (default,"car battery","12V car battery",8.1); +INSERT INTO Products +VALUES (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",0.8); +INSERT INTO Products +VALUES (default,"hammer","12oz carpenter's hammer",0.75); +INSERT INTO Products +VALUES (default,"hammer2","14oz carpenter's hammer",8.75E-1); +INSERT INTO Products +VALUES (default,"hammer3","16oz carpenter's hammer",1.0); +INSERT INTO Products +VALUES (default,"rocks","box of assorted rocks",5.3); +INSERT INTO Products +VALUES (default,"jacket","water resistent black wind breaker",0.1); +INSERT INTO Products +VALUES (default,"spare tire","24 inch spare tire",22.2); -- Create and populate the products on hand using multiple inserts CREATE TABLE products_on_hand ( @@ -50,10 +54,13 @@ CREATE TABLE customers ( INSERT INTO customers -VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), - (default,"George","Bailey","gbailey@foobar.com"), - (default,"Edward","Walker","ed@walker.com"), - (default,"Anne","Kretchmar","annek@noanswer.org"); +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"); +INSERT INTO customers +VALUES (default,"George","Bailey","gbailey@foobar.com"); +INSERT INTO customers +VALUES (default,"Edward","Walker","ed@walker.com"); +INSERT INTO customers +VALUES (default,"Anne","Kretchmar","annek@noanswer.org"); -- Create some very simple orders CREATE TABLE orders ( @@ -67,11 +74,15 @@ CREATE TABLE orders ( ) AUTO_INCREMENT = 10001; INSERT INTO orders -VALUES (default, '2016-01-16', 1001, 1, 102), - (default, '2016-01-17', 1002, 2, 105), - (default, '2016-02-18', 1004, 3, 109), - (default, '2016-02-19', 1002, 2, 106), - (default, '2016-02-21', 1003, 1, 107); +VALUES (default, '2016-01-16', 1001, 1, 102); +INSERT INTO orders +VALUES (default, '2016-01-17', 1002, 2, 105); +INSERT INTO orders +VALUES (default, '2016-02-18', 1004, 3, 109); +INSERT INTO orders +VALUES (default, '2016-02-19', 1002, 2, 106); +INSERT INTO orders +VALUES (default, '2016-02-21', 1003, 1, 107); -- DBZ-342 handle TIME values that exceed the value range of java.sql.Time @@ -83,5 +94,3 @@ CREATE TABLE dbz_342_timetest ( c5 TIME(6) ); INSERT INTO dbz_342_timetest VALUES ('517:51:04.777', '-13:14:50', '-733:00:00.0011', '-1:59:59.0011', '-838:59:58.999999'); - -CREATE DATABASE IF NOT EXISTS emptydb; diff --git a/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql b/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql index 08740721ba3..3ac6788d81f 100644 --- a/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/datetime_key_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: datetime_key_test --- ---------------------------------------------------------------------------------------------------------------- - SET sql_mode=''; CREATE TABLE dbz_1194_datetime_key_test ( id INT AUTO_INCREMENT NOT NULL, diff --git a/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql b/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql index 64cd7742ad5..4782dc80359 100644 --- a/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/decimal_column_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: decimal_column_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_751_decimal_column_test ( id INT AUTO_INCREMENT NOT NULL, rating1 DECIMAL, diff --git a/tests/integration_tests/debezium/sql/debezium/decimal_test.sql b/tests/integration_tests/debezium/sql/debezium/decimal_test.sql new file mode 100644 index 00000000000..a458083d79e --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/decimal_test.sql @@ -0,0 +1,10 @@ +CREATE TABLE `DBZ730` ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + A NUMERIC(3, 2) NOT NULL DEFAULT 1.23, + B DECIMAL(4, 3) NOT NULL DEFAULT 2.321, + C NUMERIC(7, 5) NULL DEFAULT '12.678', + D NUMERIC(7, 5) NULL DEFAULT '15.28', + E DECIMAL(65, 18) NULL DEFAULT '0.000000000000000000' +) ENGINE=InnoDB AUTO_INCREMENT=15851 DEFAULT CHARSET=utf8; +INSERT INTO `DBZ730`(A, B, C, D) +VALUES (1.33, -2.111 , 3.444, NULL); diff --git a/tests/integration_tests/debezium/sql/debezium/default_value.sql b/tests/integration_tests/debezium/sql/debezium/default_value.sql new file mode 100644 index 00000000000..60e58df660c --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/default_value.sql @@ -0,0 +1,154 @@ +CREATE TABLE UNSIGNED_TINYINT_TABLE ( + id int PRIMARY KEY, + A TINYINT UNSIGNED NULL DEFAULT 0, + B TINYINT UNSIGNED NULL DEFAULT '10', + C TINYINT UNSIGNED NULL, + D TINYINT UNSIGNED NOT NULL, + E TINYINT UNSIGNED NOT NULL DEFAULT 0, + F TINYINT UNSIGNED NOT NULL DEFAULT '0', + G TINYINT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_TINYINT_TABLE VALUES (1, DEFAULT, DEFAULT, 0, 1, DEFAULT, DEFAULT, NULL); + +CREATE TABLE UNSIGNED_SMALLINT_TABLE ( + id int PRIMARY KEY, + A SMALLINT UNSIGNED NULL DEFAULT 0, + B SMALLINT UNSIGNED NULL DEFAULT '10', + C SMALLINT UNSIGNED NULL, + D SMALLINT UNSIGNED NOT NULL, + E SMALLINT UNSIGNED NOT NULL DEFAULT 0, + F SMALLINT UNSIGNED NOT NULL DEFAULT '0', + G SMALLINT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_SMALLINT_TABLE VALUES (1, 1, 1, 1, 0, 1, 1, NULL); + +CREATE TABLE UNSIGNED_MEDIUMINT_TABLE ( + id int PRIMARY KEY, + A MEDIUMINT UNSIGNED NULL DEFAULT 0, + B MEDIUMINT UNSIGNED NULL DEFAULT '10', + C MEDIUMINT UNSIGNED NULL, + D MEDIUMINT UNSIGNED NOT NULL, + E MEDIUMINT UNSIGNED NOT NULL DEFAULT 0, + F MEDIUMINT UNSIGNED NOT NULL DEFAULT '0', + G MEDIUMINT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_MEDIUMINT_TABLE VALUES (1, 1, 1, 1, 0, 1, 1, NULL); + +CREATE TABLE UNSIGNED_INT_TABLE ( + id int PRIMARY KEY, + A INT UNSIGNED NULL DEFAULT 0, + B INT UNSIGNED NULL DEFAULT '10', + C INT UNSIGNED NULL, + D INT UNSIGNED NOT NULL, + E INT UNSIGNED NOT NULL DEFAULT 0, + F INT UNSIGNED NOT NULL DEFAULT '0', + G INT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_INT_TABLE VALUES (1, 1, 1, 1, 0, 1, 1, NULL); + +CREATE TABLE UNSIGNED_BIGINT_TABLE ( + id int PRIMARY KEY, + A BIGINT UNSIGNED NULL DEFAULT 0, + B BIGINT UNSIGNED NULL DEFAULT '10', + C BIGINT UNSIGNED NULL, + D BIGINT UNSIGNED NOT NULL, + E BIGINT UNSIGNED NOT NULL DEFAULT 0, + F BIGINT UNSIGNED NOT NULL DEFAULT '0', + G BIGINT UNSIGNED NULL DEFAULT '100' +); +INSERT INTO UNSIGNED_BIGINT_TABLE VALUES (1, 1, 1, 1, 0, 1, 1, NULL); + +CREATE TABLE STRING_TABLE ( + id int PRIMARY KEY, + A CHAR(1) NULL DEFAULT 'A', + B CHAR(1) NULL DEFAULT 'b', + C VARCHAR(10) NULL DEFAULT 'CC', + F CHAR(1) DEFAULT NULL, + G VARCHAR(10) DEFAULT NULL, + I VARCHAR(10) NULL DEFAULT '100' +); +INSERT INTO STRING_TABLE +VALUES (1, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, NULL); + +CREATE TABLE BIT_TABLE ( + id int PRIMARY KEY, + A BIT(1) NULL DEFAULT NULL, + B BIT(1) DEFAULT 0, + C BIT(1) DEFAULT 1, + D BIT(1) DEFAULT b'0', + E BIT(1) DEFAULT b'1', + H BIT(10) DEFAULT b'0101000010', + I BIT(10) DEFAULT NULL, + J BIT(25) DEFAULT b'0000000000000000100001111', + K BIT(25) DEFAULT b'0000000010110000100001111' +); +INSERT INTO BIT_TABLE +VALUES (1, false, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, NULL, DEFAULT, NULL); + +CREATE TABLE NUMBER_TABLE ( + id int PRIMARY KEY, + A TINYINT NULL DEFAULT 10, + B SMALLINT NOT NULL DEFAULT '5', + C INTEGER NOT NULL DEFAULT 0, + D BIGINT NOT NULL DEFAULT 20, + E INT NULL DEFAULT NULL, + F INT NULL DEFAULT 30 +); +INSERT INTO NUMBER_TABLE +VALUES (1, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, NULL); + +CREATE TABLE FlOAT_DOUBLE_TABLE ( + id int PRIMARY KEY, + F FLOAT NULL DEFAULT 0, + G DOUBLE NOT NULL DEFAULT 1.1, + H DOUBLE NULL DEFAULT 3.3 +); +INSERT INTO FlOAT_DOUBLE_TABLE +VALUES (1, DEFAULT, DEFAULT, NULL); + +-- set sql_mode REAL_AS_FLOAT is necessary +set @@session.sql_mode=concat(@@session.sql_mode, ',REAL_AS_FLOAT'); +CREATE TABLE REAL_TABLE ( + id int PRIMARY KEY, + A REAL NOT NULL DEFAULT 1, + B REAL NULL DEFAULT NULL, + C REAL NULL DEFAULT 3 +); +INSERT INTO REAL_TABLE +VALUES (1, DEFAULT ,DEFAULT, NULL); + +CREATE TABLE NUMERIC_DECIMAL_TABLE ( + id int PRIMARY KEY, + A NUMERIC(3, 2) NOT NULL DEFAULT 1.23, + B DECIMAL(4, 3) NOT NULL DEFAULT 2.321, + C NUMERIC(7, 5) NULL DEFAULT '12.678', + D NUMERIC(7, 5) NULL DEFAULT '15.28' +); +INSERT INTO NUMERIC_DECIMAL_TABLE +VALUES (1, 1.33 ,2.111 , 3.444, NULL); + +CREATE TABLE DATE_TIME_TABLE ( + id int PRIMARY KEY, + A DATE NOT NULL DEFAULT '1976-08-23', + C DATETIME DEFAULT '2018-01-03 00:00:10', + D DATETIME(1) DEFAULT '2018-01-03 00:00:10.7', + E DATETIME(6) DEFAULT '2018-01-03 00:00:10.123456', + F YEAR NOT NULL DEFAULT 1, + G TIME DEFAULT '00:00:00', + H TIME(1) DEFAULT '23:00:00.7', + I TIME(6) DEFAULT '23:00:00.123456', + J TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + K TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP, + L TIME(1) DEFAULT '-23:45:56.7', + M TIME(6) DEFAULT '123:00:00.123456' +); +INSERT INTO DATE_TIME_TABLE +VALUES (1, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, NULL, DEFAULT, DEFAULT); + +CREATE TABLE DBZ_771_CUSTOMERS ( + id INTEGER NOT NULL PRIMARY KEY, + CUSTOMER_TYPE ENUM ('b2c','b2b') NOT NULL default 'b2c' +); + +INSERT INTO DBZ_771_CUSTOMERS +VALUES (1, 'b2b'); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql b/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql new file mode 100644 index 00000000000..d6eba9d5778 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/default_value_all_zero_time.sql @@ -0,0 +1,33 @@ +SET sql_mode=''; +CREATE TABLE all_zero_date_and_time_table ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + A TIMESTAMP NOT NULL DEFAULT '0000-00-00 00:00:00', + B TIMESTAMP NULL DEFAULT '0000-00-00 00:00:00', + C TIMESTAMP DEFAULT '0000-00-00 00:00:00' NOT NULL, + D TIMESTAMP DEFAULT '0000-00-00 00:00:00' NULL, + E DATE NOT NULL DEFAULT '0000-00-00', + F DATE NULL DEFAULT '0000-00-00', + G DATE DEFAULT '0000-00-00' NOT NULL, + H DATE DEFAULT '0000-00-00' NULL, + I DATETIME NOT NULL DEFAULT '0000-00-00 00:00:00', + J DATETIME NULL DEFAULT '0000-00-00 00:00:00', + K DATETIME DEFAULT '0000-00-00 00:00:00' NOT NULL, + L DATETIME DEFAULT '0000-00-00 00:00:00' NULL +); +INSERT INTO all_zero_date_and_time_table +VALUES (DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT); + +CREATE TABLE part_zero_date_and_time_table ( + id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, + A TIMESTAMP NOT NULL DEFAULT '0000-00-00 00:00:00', + B TIMESTAMP NULL DEFAULT '0000-00-00 00:00:00', + C DATETIME NOT NULL DEFAULT '0000-00-00 01:00:00.000', + D DATETIME NULL DEFAULT '0000-00-00 01:00:00.000', + E DATE NOT NULL DEFAULT '1000-00-00', + /* TiCDC convert '1000-00-00 01:00:00.000' to "1000-00-00" + F DATE NULL DEFAULT '1000-00-00 01:00:00.000',*/ + G TIME DEFAULT '0000-00-00 01:00:00.000' NOT NULL, + H TIME DEFAULT '0000-00-00 01:00:00.000' NULL +); +INSERT INTO part_zero_date_and_time_table +VALUES (DEFAULT, DEFAULT, DEFAULT, DEFAULT, DEFAULT, /*DEFAULT,*/ DEFAULT, DEFAULT, DEFAULT); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/default_value_generated.sql b/tests/integration_tests/debezium/sql/debezium/default_value_generated.sql new file mode 100644 index 00000000000..ed4841620ef --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/default_value_generated.sql @@ -0,0 +1,8 @@ +/* see https://github.com/pingcap/tiflow/issues/11704 */ +CREATE TABLE GENERATED_TABLE ( + id int PRIMARY KEY, + A SMALLINT UNSIGNED, + B SMALLINT UNSIGNED AS (2 * A) STORED, + C SMALLINT UNSIGNED AS (3 * A) STORED NOT NULL +); +INSERT INTO GENERATED_TABLE VALUES (1, 15, DEFAULT, DEFAULT); diff --git a/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql b/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql index a8db5dfe308..9c261cbf46f 100644 --- a/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/enum_column_test.sql @@ -1,7 +1,9 @@ +SET time_zone='UTC'; +/* Debezium convert charsetName to "utf8mb4" when column collate is utf8_unicode_ci */ CREATE TABLE `test_stations_10` ( `id` int(10) unsigned NOT NULL AUTO_INCREMENT, - `name` varchar(500) COLLATE utf8_unicode_ci NOT NULL, - `type` enum('station', 'post_office') COLLATE utf8_unicode_ci NOT NULL DEFAULT 'station', + `name` varchar(500) /* COLLATE utf8_unicode_ci */ NOT NULL, + `type` enum('station', 'post_office') /* COLLATE utf8_unicode_ci */ NOT NULL DEFAULT 'station', `created` datetime DEFAULT CURRENT_TIMESTAMP, `modified` datetime DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, PRIMARY KEY (`id`) @@ -9,8 +11,9 @@ CREATE TABLE `test_stations_10` ( INSERT INTO test_stations_10 (`name`, `type`) values ( 'ha Tinh 7', 'station' ); -ALTER TABLE `test_stations_10` - MODIFY COLUMN `type` ENUM('station', 'post_office', 'plane', 'ahihi_dongok', 'now', 'test', 'a,b', 'c,\'d', 'g,''h') - CHARACTER SET 'utf8' COLLATE 'utf8_unicode_ci' NOT NULL DEFAULT 'station'; +/* Debezium datetime length is not correct */ +-- ALTER TABLE `test_stations_10` +-- MODIFY COLUMN `type` ENUM('station', 'post_office', 'plane', 'ahihi_dongok', 'now', 'test', 'a,b' /*'c,\'d', 'g,''h'*/) +-- /* CHARACTER SET 'utf8' COLLATE 'utf8_unicode_ci' */ NOT NULL DEFAULT 'station'; -INSERT INTO test_stations_10 ( `name`, `type` ) values ( 'Ha Tinh 1', 'now' ); +-- INSERT INTO test_stations_10 ( `name`, `type` ) values ( 'Ha Tinh 1', 'now' ); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/float_test.sql b/tests/integration_tests/debezium/sql/debezium/float_test.sql new file mode 100644 index 00000000000..da2d3bbf019 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/float_test.sql @@ -0,0 +1,22 @@ +/* + TiDB uses this value only to determine whether to use FLOAT or DOUBLE for the resulting data type. + If p is from 0 to 24, the data type becomes FLOAT with no M or D values. + If p is from 25 to 53, the data type becomes DOUBLE with no M or D values. +*/ +SET sql_mode=''; +CREATE TABLE `DBZ3865` ( + `id` INT NOT NULL AUTO_INCREMENT, + `f1` FLOAT DEFAULT 5.6, + `f2` FLOAT(10, 2) DEFAULT NULL, + `f3` FLOAT(35, 5) DEFAULT NULL, + /* TiDB incorrect length output. issue:https://github.com/pingcap/tidb/issues/57060 + `f4_23` FLOAT(23) DEFAULT NULL + `f4_24` FLOAT(24) DEFAULT NULL, + `f4_25` FLOAT(25) DEFAULT NULL, */ + `weight` FLOAT UNSIGNED DEFAULT '0', + PRIMARY KEY (`ID`) +) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8mb4; +/* Debezium incorrect output: + f2: 5.610000133514404 + f3: 30.12346076965332 */ +INSERT INTO DBZ3865(f1,/* f2, f3, f4_23, f4_24,*/ weight) VALUE (5.6,/* 5.61, 30.123456, 64.1, 64.1,*/ 64.1234); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/history-dbz.sql b/tests/integration_tests/debezium/sql/debezium/history-dbz.sql new file mode 100644 index 00000000000..5a328e45126 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/history-dbz.sql @@ -0,0 +1,2 @@ +CREATE TABLE `t-1` (ID1 INT PRIMARY KEY); +CREATE TABLE `t.2` (ID2 INT PRIMARY KEY); diff --git a/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql b/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql index 679bb0169dc..4d805f52c19 100644 --- a/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql +++ b/tests/integration_tests/debezium/sql/debezium/multitable_dbz_871.sql @@ -1,10 +1,9 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: multitable_statement --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE t1 (ID INT PRIMARY KEY); CREATE TABLE t2 (ID INT PRIMARY KEY); CREATE TABLE t3 (ID INT PRIMARY KEY); CREATE TABLE t4 (ID INT PRIMARY KEY); -DROP TABLE t1,t2,t3,t4; \ No newline at end of file +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE t3; +DROP TABLE t4; \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-123.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-123.ddl new file mode 100644 index 00000000000..b09b625a012 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-123.ddl @@ -0,0 +1,12 @@ +CREATE TABLE `DBZ123` ( + `Id` bigint(20) NOT NULL AUTO_INCREMENT, + `Provider_ID` bigint(20) NOT NULL, + `External_ID` varchar(255) NOT NULL, + `Name` varchar(255) NOT NULL, + `Is_Enabled` bit(1) NOT NULL DEFAULT b'1', + /* BLOB, TEXT, GEOMETRY or JSON column 'binaryRepresentation' can't have a default value + `binaryRepresentation` BLOB NOT NULL DEFAULT x'cafe', */ + `BonusFactor` decimal(19,8) NOT NULL, + PRIMARY KEY (`Id`), + UNIQUE KEY `game_unq` (`Provider_ID`,`External_ID`) +) ENGINE=InnoDB AUTO_INCREMENT=2374 DEFAULT CHARSET=utf8 \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-162.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-162.ddl new file mode 100644 index 00000000000..caa4d4ae445 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-162.ddl @@ -0,0 +1,7 @@ +CREATE TABLE `test` (id INT(11) UNSIGNED NOT NULL PRIMARY KEY AUTO_INCREMENT); + +ALTER TABLE `test` CHANGE `id` `collection_id` INT(11) +UNSIGNED +NOT NULL +AUTO_INCREMENT; + diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-193.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-193.ddl new file mode 100644 index 00000000000..0d3026a4c3e --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-193.ddl @@ -0,0 +1,10 @@ +CREATE TABLE `roles` ( +`id` varchar(32) NOT NULL, +`name` varchar(100) NOT NULL, +`context` varchar(20) NOT NULL, +`organization_id` int(11) DEFAULT NULL, +`client_id` varchar(32) NOT NULL, +`scope_action_ids` text NOT NULL, +PRIMARY KEY (`id`), +FULLTEXT KEY `scope_action_ids_idx` (`scope_action_ids`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-198i.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-198i.ddl new file mode 100644 index 00000000000..36eb69a9d3f --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-198i.ddl @@ -0,0 +1,23 @@ +create database `NextTimeTable`; +use `NextTimeTable`; +create table NextTimeTable.REFERENCED ( + SUBJECT_ID int not null, + PRIMARY KEY (SUBJECT_ID) + ); + +create table `NextTimeTable`.`TIMETABLE_SUBJECT_GROUP_MAPPING` ( + pk1 int not null, + `SUBJECT_ID` int not null, + `other` int, + CONSTRAINT `FK69atxmt7wrwpb4oekyravsx9l` FOREIGN KEY (`SUBJECT_ID`) REFERENCES `NextTimeTable`.`REFERENCED`(`SUBJECT_ID`) + ); + +/* TiCDC discards it due to unsupported DDL type. +Alter table `NextTimeTable`.`TIMETABLE_SUBJECT_GROUP_MAPPING` +drop foreign key `FK69atxmt7wrwpb4oekyravsx9l`; +Alter table `NextTimeTable`.`TIMETABLE_SUBJECT_GROUP_MAPPING` +drop index `FK69atxmt7wrwpb4oekyravsx9l`; +Alter table `NextTimeTable`.`TIMETABLE_SUBJECT_GROUP_MAPPING` +drop column `SUBJECT_ID`; +*/ +create table `NextTimeTable`.`table1` ( pk1 int not null PRIMARY KEY, `id` int not null, `other` int ); diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-dbz-200.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-200.ddl new file mode 100644 index 00000000000..797ed730488 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-dbz-200.ddl @@ -0,0 +1,25 @@ +CREATE TABLE `customfield` ( + `ENCODEDKEY` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + `ID` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `CREATIONDATE` datetime DEFAULT NULL, + `LASTMODIFIEDDATE` datetime DEFAULT NULL, + `DATATYPE` varchar(256) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `ISDEFAULT` bit(1) DEFAULT NULL, + `ISREQUIRED` bit(1) DEFAULT NULL, + `NAME` varchar(256) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `VALUES` mediumblob, + `AMOUNTS` mediumblob, + `DESCRIPTION` varchar(256) DEFAULT NULL, + `TYPE` varchar(256) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `VALUELENGTH` varchar(256) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL DEFAULT 'SHORT', + `INDEXINLIST` int(11) DEFAULT '-1', + `CUSTOMFIELDSET_ENCODEDKEY_OID` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + `STATE` varchar(256) NOT NULL DEFAULT 'NORMAL', + `VALIDATIONPATTERN` varchar(256) DEFAULT NULL, + `VIEWUSAGERIGHTSKEY` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `EDITUSAGERIGHTSKEY` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `BUILTINCUSTOMFIELDID` varchar(255) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `UNIQUE` varchar(32) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + STORAGE varchar(32) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + KEY `index1` (`ID`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8; diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-quoted.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-quoted.ddl new file mode 100644 index 00000000000..cad5ecae495 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-quoted.ddl @@ -0,0 +1,36 @@ +-- +-- Contains statements with quoted identifiers +-- +CREATE TABLE `customers` ( + `id` int(11) NOT NULL AUTO_INCREMENT, + `first_name` varchar(255) NOT NULL, + `last_name` varchar(255) NOT NULL, + `email` varchar(255) NOT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `email` (`email`) +) ENGINE=InnoDB AUTO_INCREMENT=1005 DEFAULT CHARSET=latin1; +CREATE TABLE `products` ( + `id` int(11) NOT NULL AUTO_INCREMENT, + `name` varchar(255) NOT NULL, + `description` varchar(512) DEFAULT NULL, + `weight` float DEFAULT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=110 DEFAULT CHARSET=latin1; +CREATE TABLE `orders` ( + `order_number` int(11) NOT NULL AUTO_INCREMENT, + `order_date` date NOT NULL, + `purchaser` int(11) NOT NULL, + `quantity` int(11) NOT NULL, + `product_id` int(11) NOT NULL, + PRIMARY KEY (`order_number`), + KEY `order_customer` (`purchaser`), + KEY `ordered_product` (`product_id`), + CONSTRAINT `orders_ibfk_1` FOREIGN KEY (`purchaser`) REFERENCES `customers` (`id`), + CONSTRAINT `orders_ibfk_2` FOREIGN KEY (`product_id`) REFERENCES `products` (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=10006 DEFAULT CHARSET=latin1; +CREATE TABLE `products_on_hand` ( + `product_id` int(11) NOT NULL, + `quantity` int(11) NOT NULL, + PRIMARY KEY (`product_id`), + CONSTRAINT `products_on_hand_ibfk_1` FOREIGN KEY (`product_id`) REFERENCES `products` (`id`) +) ENGINE=InnoDB DEFAULT CHARSET=latin1 \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-test-create.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-test-create.ddl new file mode 100644 index 00000000000..7052708998c --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-test-create.ddl @@ -0,0 +1,1134 @@ +-- +-- BUILD SCRIPT +-- RDBMS: MYSQL 5.0 +-- + +CREATE TABLE AUDITENTRIES +( + TIMESTAMP VARCHAR(50) NOT NULL, + CONTEXT VARCHAR(64) NOT NULL, + ACTIVITY VARCHAR(64) NOT NULL, + RESOURCES VARCHAR(4000) NOT NULL, + PRINCIPAL VARCHAR(255) NOT NULL, + HOSTNAME VARCHAR(64) NOT NULL, + VMID VARCHAR(64) NOT NULL +); + +CREATE TABLE AUTHPERMTYPES +( + PERMTYPEUID NUMERIC(10) NOT NULL PRIMARY KEY, + DISPLAYNAME VARCHAR(250) NOT NULL, + FACTORYCLASSNAME VARCHAR(80) NOT NULL +); + +CREATE TABLE AUTHPOLICIES +( + POLICYUID NUMERIC(10) NOT NULL PRIMARY KEY, + DESCRIPTION VARCHAR(250), + POLICYNAME VARCHAR(250) NOT NULL +); + +CREATE TABLE AUTHPRINCIPALS +( + PRINCIPALTYPE NUMERIC(10) NOT NULL, + PRINCIPALNAME VARCHAR(255) NOT NULL, + POLICYUID NUMERIC(10) NOT NULL REFERENCES AUTHPOLICIES (POLICYUID) , + GRANTOR VARCHAR(255) NOT NULL, + CONSTRAINT PK_AUTHPOLICYPRINCIPALS UNIQUE (PRINCIPALNAME, POLICYUID) +); + +CREATE TABLE AUTHREALMS +( + REALMUID NUMERIC(10) NOT NULL PRIMARY KEY, + REALMNAME VARCHAR(250) NOT NULL UNIQUE, + DESCRIPTION VARCHAR(550) +); + +CREATE TABLE CFG_STARTUP_STATE +(STATE INTEGER DEFAULT 0 , +LASTCHANGED VARCHAR(50) ); + +CREATE TABLE IDTABLE +( + IDCONTEXT VARCHAR(20) NOT NULL PRIMARY KEY, + NEXTID NUMERIC +); + +CREATE TABLE LOGMESSAGETYPES +( + MESSAGELEVEL NUMERIC(10) NOT NULL PRIMARY KEY, + NAME VARCHAR(64) NOT NULL, + DISPLAYNAME VARCHAR(64) +); + +CREATE TABLE MM_PRODUCTS +( + PRODUCT_UID NUMERIC NOT NULL PRIMARY KEY, + PRODUCT_NAME VARCHAR(50) NOT NULL, + PRODUCT_DISPLAY_NM VARCHAR(100) +); + +CREATE TABLE PRINCIPALTYPES +( + PRINCIPALTYPEUID NUMERIC(10) NOT NULL PRIMARY KEY, + PRINCIPALTYPE VARCHAR(60) NOT NULL, + DISPLAYNAME VARCHAR(80) NOT NULL, + LASTCHANGEDBY VARCHAR(255) NOT NULL, + LASTCHANGED VARCHAR(50) +); +-- ========= STATEMENT 10 ============ + +CREATE TABLE RT_MDLS +( + MDL_UID NUMERIC(10) NOT NULL PRIMARY KEY, + MDL_UUID VARCHAR(64) NOT NULL, + MDL_NM VARCHAR(255) NOT NULL, + MDL_VERSION VARCHAR(50), + DESCRIPTION VARCHAR(255), + MDL_URI VARCHAR(255), + MDL_TYPE NUMERIC(3), + IS_PHYSICAL CHAR(1) NOT NULL, + MULTI_SOURCED CHAR(1) DEFAULT '0', + VISIBILITY NUMERIC(10) + ); + +CREATE TABLE RT_MDL_PRP_NMS +( + PRP_UID NUMERIC(10) NOT NULL PRIMARY KEY, + MDL_UID NUMERIC(10) NOT NULL , + PRP_NM VARCHAR(255) NOT NULL +); + +CREATE TABLE RT_MDL_PRP_VLS +( + PRP_UID NUMERIC(10) NOT NULL , + PART_ID NUMERIC(10) NOT NULL, + PRP_VL VARCHAR(255) NOT NULL, + CONSTRAINT PK_MDL_PRP_VLS UNIQUE (PRP_UID, PART_ID) +); + + +CREATE TABLE RT_VIRTUAL_DBS +( + VDB_UID NUMERIC(10) NOT NULL PRIMARY KEY, + VDB_VERSION VARCHAR(50) NOT NULL, + VDB_NM VARCHAR(255) NOT NULL, + DESCRIPTION VARCHAR(255), + PROJECT_GUID VARCHAR(64), + VDB_STATUS NUMERIC NOT NULL, + WSDL_DEFINED CHAR(1) DEFAULT '0', + VERSION_BY VARCHAR(100), + VERSION_DATE VARCHAR(50) NOT NULL, + CREATED_BY VARCHAR(100), + CREATION_DATE VARCHAR(50), + UPDATED_BY VARCHAR(100), + UPDATED_DATE VARCHAR(50), + VDB_FILE_NM VARCHAR(2048) +); + +CREATE TABLE SERVICESESSIONS +( + SESSIONUID NUMERIC(10) NOT NULL PRIMARY KEY, + PRINCIPAL VARCHAR(255) NOT NULL, + APPLICATION VARCHAR(128) NOT NULL, + CREATIONTIME VARCHAR(50), + CLIENTCOUNT NUMERIC(10) NOT NULL, + STATE NUMERIC(10) NOT NULL, + STATETIME VARCHAR(50), + USESSUBSCRIBER CHAR(1) NOT NULL, + PRODUCTINFO1 VARCHAR(255), + PRODUCTINFO2 VARCHAR(255), + PRODUCTINFO3 VARCHAR(255), + PRODUCTINFO4 VARCHAR(255) +); +-- ========= STATEMENT 15 ============ +CREATE INDEX RTMDLS_NM_IX ON RT_MDLS (MDL_NM); + +CREATE INDEX RTVIRTUALDBS_NM_IX ON RT_VIRTUAL_DBS (VDB_NM); + +CREATE INDEX RTVIRTUALDBS_VRSN_IX ON RT_VIRTUAL_DBS (VDB_VERSION); + +CREATE UNIQUE INDEX MDL_PRP_NMS_UIX ON RT_MDL_PRP_NMS (MDL_UID, PRP_NM); + +CREATE UNIQUE INDEX PRNCIPALTYP_UIX ON PRINCIPALTYPES (PRINCIPALTYPE); +-- ========= STATEMENT 20 ============ +CREATE UNIQUE INDEX AUTHPOLICIES_NAM_UIX ON AUTHPOLICIES (POLICYNAME); + +CREATE TABLE AUTHPERMISSIONS +( + PERMISSIONUID NUMERIC(10) NOT NULL PRIMARY KEY, + RESOURCENAME VARCHAR(250) NOT NULL, + ACTIONS NUMERIC(10) NOT NULL, + CONTENTMODIFIER VARCHAR(250), + PERMTYPEUID NUMERIC(10) NOT NULL REFERENCES AUTHPERMTYPES (PERMTYPEUID) , + REALMUID NUMERIC(10) NOT NULL REFERENCES AUTHREALMS (REALMUID) , + POLICYUID NUMERIC(10) NOT NULL REFERENCES AUTHPOLICIES (POLICYUID) +); + + +CREATE TABLE LOGENTRIES +( + TIMESTAMP VARCHAR(50) NOT NULL, + CONTEXT VARCHAR(64) NOT NULL, + MSGLEVEL NUMERIC(10) NOT NULL REFERENCES LOGMESSAGETYPES (MESSAGELEVEL) , + EXCEPTION VARCHAR(4000), + MESSAGE VARCHAR(2000) NOT NULL, + HOSTNAME VARCHAR(64) NOT NULL, + VMID VARCHAR(64) NOT NULL, + THREAModeShapeME VARCHAR(64) NOT NULL, + VMSEQNUM NUMERIC(7) NOT NULL +); + +CREATE TABLE PRODUCTSSESSIONS +( + PRODUCT_UID NUMERIC NOT NULL, + SESSION_UID NUMERIC NOT NULL, + PRIMARY KEY (PRODUCT_UID, SESSION_UID) +); + +-- ALTER TABLE PRODUCTSSESSIONS +-- ADD CONSTRAINT FK_PRODSESS_PRODS +-- FOREIGN KEY (PRODUCT_UID) +-- REFERENCES MM_PRODUCTS (PRODUCT_UID); + +-- ALTER TABLE PRODUCTSSESSIONS +-- ADD CONSTRAINT FK_PRODSESS_SVCSES +-- FOREIGN KEY (SESSION_UID) +-- REFERENCES SERVICESESSIONS (SESSIONUID); + + +CREATE TABLE RT_VDB_MDLS +( + VDB_UID NUMERIC(10) NOT NULL , + MDL_UID NUMERIC(10) NOT NULL , + CNCTR_BNDNG_NM VARCHAR(255) +); + +CREATE INDEX AWA_SYS_MSGLEVEL_1E6F845E ON LOGENTRIES (MSGLEVEL); + +CREATE UNIQUE INDEX AUTHPERM_UIX ON AUTHPERMISSIONS ( POLICYUID, RESOURCENAME); + +CREATE TABLE CS_EXT_FILES ( + FILE_UID INTEGER NOT NULL, + CHKSUM NUMERIC(20), + FILE_NAME VARCHAR(255) NOT NULL, + FILE_CONTENTS LONGBLOB, + CONFIG_CONTENTS LONGTEXT, + SEARCH_POS INTEGER, + IS_ENABLED CHAR(1), + FILE_DESC VARCHAR(4000), + CREATED_BY VARCHAR(100), + CREATION_DATE VARCHAR(50), + UPDATED_BY VARCHAR(100), + UPDATE_DATE VARCHAR(50), + FILE_TYPE VARCHAR(30), + CONSTRAINT PK_CS_EXT_FILES PRIMARY KEY (FILE_UID) +) +; +-- ========= STATEMENT 30 ============ +-- ALTER TABLE CS_EXT_FILES ADD CONSTRAINT CSEXFILS_FIL_NA_UK UNIQUE (FILE_NAME); + +CREATE TABLE MMSCHEMAINFO_CA +( + SCRIPTNAME VARCHAR(50), + SCRIPTEXECUTEDBY VARCHAR(50), + SCRIPTREV VARCHAR(50), + RELEASEDATE VARCHAR(50), + DATECREATED DATE, + DATEUPDATED DATE, + UPDATEID VARCHAR(50), + METAMATRIXSERVERURL VARCHAR(100) +) +; + +CREATE TABLE CS_SYSTEM_PROPS ( + PROPERTY_NAME VARCHAR(255), + PROPERTY_VALUE VARCHAR(255) +); + +CREATE UNIQUE INDEX SYSPROPS_KEY ON CS_SYSTEM_PROPS (PROPERTY_NAME); + +CREATE TABLE CFG_LOCK ( + USER_NAME VARCHAR(50) NOT NULL, + DATETIME_ACQUIRED VARCHAR(50) NOT NULL, + DATETIME_EXPIRE VARCHAR(50) NOT NULL, + HOST VARCHAR(100), + LOCK_TYPE NUMERIC (1) ); + + +CREATE TABLE TX_MMXCMDLOG +(REQUESTID VARCHAR(255) NOT NULL, +TXNUID VARCHAR(50) NULL, +CMDPOINT NUMERIC(10) NOT NULL, +SESSIONUID VARCHAR(255) NOT NULL, +APP_NAME VARCHAR(255) NULL, +PRINCIPAL_NA VARCHAR(255) NOT NULL, +VDBNAME VARCHAR(255) NOT NULL, +VDBVERSION VARCHAR(50) NOT NULL, +CREATED_TS VARCHAR(50) NULL, +ENDED_TS VARCHAR(50) NULL, +CMD_STATUS NUMERIC(10) NOT NULL, +SQL_ID NUMERIC(10), +FINL_ROWCNT NUMERIC(10) +) +; + +CREATE TABLE TX_SRCCMDLOG +(REQUESTID VARCHAR(255) NOT NULL, +NODEID NUMERIC(10) NOT NULL, +SUBTXNUID VARCHAR(50) NULL, +CMD_STATUS NUMERIC(10) NOT NULL, +MDL_NM VARCHAR(255) NOT NULL, +CNCTRNAME VARCHAR(255) NOT NULL, +CMDPOINT NUMERIC(10) NOT NULL, +SESSIONUID VARCHAR(255) NOT NULL, +PRINCIPAL_NA VARCHAR(255) NOT NULL, +CREATED_TS VARCHAR(50) NULL, +ENDED_TS VARCHAR(50) NULL, +SQL_ID NUMERIC(10) NULL, +FINL_ROWCNT NUMERIC(10) NULL +) +; + + +CREATE TABLE TX_SQL ( SQL_ID NUMERIC(10) NOT NULL, + SQL_VL TEXT ) +; +ALTER TABLE TX_SQL + ADD CONSTRAINT TX_SQL_PK +PRIMARY KEY (SQL_ID) +; +-- ========= STATEMENT 39 ============ + +-- +-- The ITEMS table stores the raw, structure-independent information about the items contained by the Repository. This table is capable of persisting multiple versions of an item. +-- +CREATE TABLE MBR_ITEMS +( + ITEM_ID_P1 NUMERIC(20) NOT NULL, + ITEM_ID_P2 NUMERIC(20) NOT NULL, + ITEM_VERSION VARCHAR(80) NOT NULL, + ITEM_NAME VARCHAR(255) NOT NULL, + UPPER_ITEM_NAME VARCHAR(255) NOT NULL, + COMMENT_FLD VARCHAR(2000), + LOCK_HOLDER VARCHAR(100), + LOCK_DATE VARCHAR(50), + CREATED_BY VARCHAR(100) NOT NULL, + CREATION_DATE VARCHAR(50) NOT NULL, + ITEM_TYPE NUMERIC(10) NOT NULL +); + +-- +-- The ITEM_CONTENTS table stores the contents for items (files) stored in the repository. This table is capable of persisting multiple versions of the contents for an item. +-- +CREATE TABLE MBR_ITEM_CONTENTS +( + ITEM_ID_P1 NUMERIC(20) NOT NULL, + ITEM_ID_P2 NUMERIC(20) NOT NULL, + ITEM_VERSION VARCHAR(80) NOT NULL, + ITEM_CONTENT LONGBLOB NOT NULL +); + +-- +-- The ENTRIES table stores the structure information for all the objects stored in the Repository. This includes both folders and items. +-- +CREATE TABLE MBR_ENTRIES +( + ENTRY_ID_P1 NUMERIC(20) NOT NULL, + ENTRY_ID_P2 NUMERIC(20) NOT NULL, + ENTRY_NAME VARCHAR(255) NOT NULL, + UPPER_ENTRY_NAME VARCHAR(255) NOT NULL, + ITEM_ID_P1 NUMERIC(20), + ITEM_ID_P2 NUMERIC(20), + ITEM_VERSION VARCHAR(80), + PARENT_ENTRY_ID_P1 NUMERIC(20), + PARENT_ENTRY_ID_P2 NUMERIC(20), + DELETED NUMERIC(1) NOT NULL +); + +-- +-- The LABELS table stores the various labels that have been defined. +-- +CREATE TABLE MBR_LABELS +( + LABEL_ID_P1 NUMERIC(20) NOT NULL, + LABEL_ID_P2 NUMERIC(20) NOT NULL, + LABEL_FLD VARCHAR(255) NOT NULL, + COMMENT_FLD VARCHAR(2000), + CREATED_BY VARCHAR(100) NOT NULL, + CREATION_DATE VARCHAR(50) NOT NULL +); + +-- +-- The ITEM_LABELS table maintains the relationships between the ITEMS and the LABELs; that is, the labels that have been applied to each of the item versions. (This is a simple intersect table.) +-- +CREATE TABLE MBR_ITEM_LABELS +( + ITEM_ID_P1 NUMERIC(20) NOT NULL, + ITEM_ID_P2 NUMERIC(20) NOT NULL, + ITEM_VERSION VARCHAR(80) NOT NULL, + LABEL_ID_P1 NUMERIC(20) NOT NULL, + LABEL_ID_P2 NUMERIC(20) NOT NULL +); + +-- +-- The ITEM_LABELS table maintains the relationships between the ITEMS and the LABELs; that is, the labels that have been applied to each of the item versions. (This is a simple intersect table.) +-- +CREATE TABLE MBR_FOLDER_LABELS +( + ENTRY_ID_P1 NUMERIC(20) NOT NULL, + ENTRY_ID_P2 NUMERIC(20) NOT NULL, + LABEL_ID_P1 NUMERIC(20) NOT NULL, + LABEL_ID_P2 NUMERIC(20) NOT NULL +); + +CREATE TABLE MBR_ITEM_TYPES +( + ITEM_TYPE_CODE NUMERIC(10) NOT NULL, + ITEM_TYPE_NM VARCHAR(20) NOT NULL +); + +CREATE TABLE MBR_POLICIES +( + POLICY_NAME VARCHAR(250) NOT NULL, + CREATION_DATE VARCHAR(50), + CHANGE_DATE VARCHAR(50), + GRANTOR VARCHAR(32) +); + +CREATE TABLE MBR_POL_PERMS +( + ENTRY_ID_P1 NUMERIC(20) NOT NULL, + ENTRY_ID_P2 NUMERIC(20) NOT NULL, + POLICY_NAME VARCHAR(250) NOT NULL, + CREATE_BIT CHAR(1) NOT NULL, + READ_BIT CHAR(1) NOT NULL, + UPDATE_BIT CHAR(1) NOT NULL, + DELETE_BIT CHAR(1) NOT NULL +); + +CREATE TABLE MBR_POL_USERS +( + POLICY_NAME VARCHAR(250) NOT NULL, + USER_NAME VARCHAR(80) NOT NULL +); + +CREATE UNIQUE INDEX MBR_ENT_NM_PNT_IX ON MBR_ENTRIES (UPPER_ENTRY_NAME,PARENT_ENTRY_ID_P1,PARENT_ENTRY_ID_P2); +-- ========= STATEMENT 50 ============ +CREATE INDEX MBR_ITEMS_ID_IX ON MBR_ITEMS (ITEM_ID_P1,ITEM_ID_P2); + +CREATE INDEX MBR_ENT_PARNT_IX ON MBR_ENTRIES (PARENT_ENTRY_ID_P1); + +CREATE INDEX MBR_ENT_NM_IX ON MBR_ENTRIES (UPPER_ENTRY_NAME); + +ALTER TABLE MBR_ITEMS + ADD CONSTRAINT PK_ITEMS + PRIMARY KEY (ITEM_ID_P1,ITEM_ID_P2,ITEM_VERSION); + +ALTER TABLE MBR_ITEM_CONTENTS + ADD CONSTRAINT PK_ITEM_CONTENTS + PRIMARY KEY (ITEM_ID_P1,ITEM_ID_P2,ITEM_VERSION); + +ALTER TABLE MBR_ENTRIES + ADD CONSTRAINT PK_ENTRIES + PRIMARY KEY (ENTRY_ID_P1,ENTRY_ID_P2); + +ALTER TABLE MBR_LABELS + ADD CONSTRAINT PK_LABELS + PRIMARY KEY (LABEL_ID_P1,LABEL_ID_P2); + +ALTER TABLE MBR_ITEM_LABELS + ADD CONSTRAINT PK_ITEM_LABELS + PRIMARY KEY (ITEM_ID_P1,ITEM_ID_P2,ITEM_VERSION,LABEL_ID_P1,LABEL_ID_P2); + +ALTER TABLE MBR_FOLDER_LABELS + ADD CONSTRAINT PK_FOLDER_LABELS + PRIMARY KEY (ENTRY_ID_P1,ENTRY_ID_P2,LABEL_ID_P1,LABEL_ID_P2); + +ALTER TABLE MBR_POLICIES + ADD CONSTRAINT PK_POLICIES + PRIMARY KEY (POLICY_NAME); +-- ========= STATEMENT 60 ============ +ALTER TABLE MBR_POL_PERMS + ADD CONSTRAINT PK_POL_PERMS + PRIMARY KEY (ENTRY_ID_P1,ENTRY_ID_P2,POLICY_NAME); + +ALTER TABLE MBR_POL_USERS + ADD CONSTRAINT PK_POL_USERS + PRIMARY KEY (POLICY_NAME,USER_NAME); +-- (generated from DtcBase/ObjectIndex) + + + +CREATE OR REPLACE VIEW MBR_READ_ENTRIES (ENTRY_ID_P1,ENTRY_ID_P2,USER_NAME) AS +SELECT MBR_POL_PERMS.ENTRY_ID_P1, MBR_POL_PERMS.ENTRY_ID_P2, + MBR_POL_USERS.USER_NAME +FROM MBR_POL_PERMS, MBR_POL_USERS , CS_SYSTEM_PROPS +where MBR_POL_PERMS.POLICY_NAME=MBR_POL_USERS.POLICY_NAME + AND (CS_SYSTEM_PROPS.PROPERTY_NAME='metamatrix.authorization.metabase.CheckingEnabled' + AND CS_SYSTEM_PROPS.PROPERTY_VALUE ='true' + AND MBR_POL_PERMS.READ_BIT='1') +UNION ALL +SELECT ENTRY_ID_P1, ENTRY_ID_P2, NULL +FROM MBR_ENTRIES ,CS_SYSTEM_PROPS +WHERE CS_SYSTEM_PROPS.PROPERTY_NAME='metamatrix.authorization.metabase.CheckingEnabled' + AND CS_SYSTEM_PROPS.PROPERTY_VALUE ='false' +; + + +CREATE INDEX MBR_POL_PERMS_IX1 ON MBR_POL_PERMS (POLICY_NAME, READ_BIT); + +CREATE INDEX LOGENTRIES_TMSTMP_IX ON LOGENTRIES (TIMESTAMP); + +CREATE TABLE DD_TXN_STATES +( + ID INTEGER primary key NOT NULL, + STATE VARCHAR(128) NOT NULL +); + +CREATE TABLE DD_TXN_LOG +( + ID BIGINT NOT NULL, + USER_NME VARCHAR(128), + BEGIN_TXN VARCHAR(50), + END_TXN VARCHAR(50), + ACTION VARCHAR(128), + TXN_STATE INTEGER +); + + +CREATE TABLE DD_SHREDQUEUE +( + QUEUE_ID NUMERIC(19) NOT NULL, + UUID1 NUMERIC(20) NOT NULL, + UUID2 NUMERIC(20) NOT NULL, + OBJECT_ID VARCHAR(44) NOT NULL, + NAME VARCHAR(128) NOT NULL, + VERSION VARCHAR(20), + MDL_PATH VARCHAR(2000), + CMD_ACTION NUMERIC(1) NOT NULL, + TXN_ID NUMERIC(19) , + SUB_BY_NME VARCHAR(100), + SUB_BY_DATE VARCHAR(50) +); + + +CREATE UNIQUE INDEX DDSQ_QUE_IX ON DD_SHREDQUEUE (QUEUE_ID) +; +CREATE UNIQUE INDEX DDSQ_TXN_IX ON DD_SHREDQUEUE (TXN_ID) +; +-- ========= STATEMENT 70 ============ +CREATE INDEX DDSQ_UUID_IX ON DD_SHREDQUEUE (OBJECT_ID) +; + +-- == new DTC start == +-- (generated from Models) + +CREATE TABLE MMR_MODELS +( + ID BIGINT NOT NULL, + NAME VARCHAR(256), + PATH VARCHAR(1024), + NAMESPACE VARCHAR(1024), + IS_METAMODEL SMALLINT, + VERSION VARCHAR(64), + IS_INCOMPLETE SMALLINT + -- SHRED_TIME DATETIME +); + +-- (generated from Resources) + +CREATE TABLE MMR_RESOURCES +( + MODEL_ID BIGINT NOT NULL, + CONTENT LONGTEXT NOT NULL +); + +-- (generated from Objects) + +CREATE TABLE MMR_OBJECTS +( + ID BIGINT NOT NULL, + MODEL_ID BIGINT NOT NULL, + NAME VARCHAR(256), + PATH VARCHAR(1024), + CLASS_NAME VARCHAR(256), + UUID VARCHAR(64), + NDX_PATH VARCHAR(256), + IS_UNRESOLVED SMALLINT +); + +-- (generated from ResolvedObjects) + +CREATE TABLE MMR_RESOLVED_OBJECTS +( + OBJ_ID BIGINT NOT NULL, + MODEL_ID BIGINT NOT NULL, + CLASS_ID BIGINT NOT NULL, + CONTAINER_ID BIGINT +); + +-- (generated from ReferenceFeatures) + +CREATE TABLE MMR_REF_FEATURES +( + MODEL_ID BIGINT NOT NULL, + OBJ_ID BIGINT NOT NULL, + NDX INT, + DATATYPE_ID BIGINT, + LOWER_BOUND INT, + UPPER_BOUND INT, + IS_CHANGEABLE SMALLINT, + IS_UNSETTABLE SMALLINT, + IS_CONTAINMENT SMALLINT, + OPPOSITE_ID BIGINT +); + +-- (generated from AttributeFeatures) + +CREATE TABLE MMR_ATTR_FEATURES +( + MODEL_ID BIGINT NOT NULL, + OBJ_ID BIGINT NOT NULL, + NDX INT, + DATATYPE_ID BIGINT, + LOWER_BOUND INT, + UPPER_BOUND INT, + IS_CHANGEABLE SMALLINT, + IS_UNSETTABLE SMALLINT +); + +-- (generated from References) + +CREATE TABLE MMR_REFS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + TO_ID BIGINT NOT NULL +); + +-- (generated from BooleanAttributes) + +CREATE TABLE MMR_BOOLEAN_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE SMALLINT NOT NULL +); + +-- (generated from ByteAttributes) + +CREATE TABLE MMR_BYTE_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE CHAR(1) NOT NULL +); +-- ========= STATEMENT 80 ============ +-- (generated from CharAttributes) + +CREATE TABLE MMR_CHAR_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE CHAR(1) +); + +-- (generated from ClobAttributes) + +CREATE TABLE MMR_CLOB_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE LONGTEXT +); + +-- (generated from DoubleAttributes) + +CREATE TABLE MMR_DOUBLE_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE DOUBLE NOT NULL +); + +-- (generated from EnumeratedAttributes) + +CREATE TABLE MMR_ENUM_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE INT NOT NULL +); + +-- (generated from FloatAttributes) + +CREATE TABLE MMR_FLOAT_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE FLOAT NOT NULL +); + +-- (generated from IntAttributes) + +CREATE TABLE MMR_INT_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE INT NOT NULL +); + +-- (generated from LongAttributes) + +CREATE TABLE MMR_LONG_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE BIGINT NOT NULL +); + + +-- (generated from ShortAttributes) + +CREATE TABLE MMR_SHORT_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE SMALLINT NOT NULL +); + +-- (generated from StringAttributes) + +CREATE TABLE MMR_STRING_ATTRS +( + MODEL_ID BIGINT NOT NULL, + OBJECT_ID BIGINT NOT NULL, + FEATURE_ID BIGINT NOT NULL, + NDX INT NOT NULL, + VALUE VARCHAR(4000) +); + +-- Index length too long for MMR_MODELS(NAME,PATH) +CREATE INDEX MOD_PATH_NDX ON MMR_MODELS (NAME); +-- ========= STATEMENT 90 ============ +-- Specified key was too long; max key length is 3072 bytes +-- CREATE INDEX MOD_PATH2_NDX ON MMR_MODELS (PATH); + +-- CREATE INDEX MOD_NAMESPACE_NDX ON MMR_MODELS (NAMESPACE); + +CREATE INDEX OBJ_UUID_NDX ON MMR_OBJECTS (UUID); + +CREATE INDEX RES_OBJ_MODEL_NDX ON MMR_RESOLVED_OBJECTS (MODEL_ID); + +CREATE INDEX RES_OBJ_CLASS_NDX ON MMR_RESOLVED_OBJECTS (CLASS_ID); + +CREATE INDEX RF_DATATYPE_NDX ON MMR_REF_FEATURES (DATATYPE_ID); + +CREATE INDEX RF_MODEL_NDX ON MMR_REF_FEATURES (MODEL_ID); + +CREATE INDEX AF_DATATYPE_NDX ON MMR_ATTR_FEATURES (DATATYPE_ID); + +CREATE INDEX AF_MODEL_NDX ON MMR_ATTR_FEATURES (MODEL_ID); + +CREATE INDEX BOL_FEATURE_NDX ON MMR_BOOLEAN_ATTRS (FEATURE_ID); +-- ========= STATEMENT 100 ============ +CREATE INDEX BOL_MODEL_NDX ON MMR_BOOLEAN_ATTRS (MODEL_ID); + +CREATE INDEX BYT_FEATURE_NDX ON MMR_BYTE_ATTRS (FEATURE_ID); + +CREATE INDEX BYT_MODEL_NDX ON MMR_BYTE_ATTRS (MODEL_ID); + +CREATE INDEX CHR_FEATURE_NDX ON MMR_CHAR_ATTRS (FEATURE_ID); + +CREATE INDEX CHR_MODEL_NDX ON MMR_CHAR_ATTRS (MODEL_ID); + +CREATE INDEX CLOB_FEATURE_NDX ON MMR_CLOB_ATTRS (FEATURE_ID); + +CREATE INDEX CLOB_MODEL_NDX ON MMR_CLOB_ATTRS (MODEL_ID); + +CREATE INDEX DBL_FEATURE_NDX ON MMR_DOUBLE_ATTRS (FEATURE_ID); + +CREATE INDEX DBL_MODEL_NDX ON MMR_DOUBLE_ATTRS (MODEL_ID); + +CREATE INDEX ENUM_FEATURE_NDX ON MMR_ENUM_ATTRS (FEATURE_ID); +-- ========= STATEMENT 110 ============ +CREATE INDEX ENUM_MODEL_NDX ON MMR_ENUM_ATTRS (MODEL_ID); + +CREATE INDEX FLT_FEATURE_NDX ON MMR_FLOAT_ATTRS (FEATURE_ID); + +CREATE INDEX FLT_MODEL_NDX ON MMR_FLOAT_ATTRS (MODEL_ID); + +CREATE INDEX INT_FEATURE_NDX ON MMR_INT_ATTRS (FEATURE_ID); + +CREATE INDEX INT_MODEL_NDX ON MMR_INT_ATTRS (MODEL_ID); + +CREATE INDEX LNG_FEATURE_NDX ON MMR_LONG_ATTRS (FEATURE_ID); + +CREATE INDEX LNG_MODEL_NDX ON MMR_LONG_ATTRS (MODEL_ID); + +CREATE INDEX REF_FEATURE_NDX ON MMR_REFS (FEATURE_ID); + +CREATE INDEX REF_TO_NDX ON MMR_REFS (TO_ID); + +CREATE INDEX REF_MODEL_NDX ON MMR_REFS (MODEL_ID); +-- ========= STATEMENT 120 ============ +CREATE INDEX SHR_FEATURE_NDX ON MMR_SHORT_ATTRS (FEATURE_ID); + +CREATE INDEX SHR_MODEL_NDX ON MMR_SHORT_ATTRS (MODEL_ID); + +CREATE INDEX STR_FEATURE_NDX ON MMR_STRING_ATTRS (FEATURE_ID); + +CREATE INDEX STR_MODEL_NDX ON MMR_STRING_ATTRS (MODEL_ID); + +-- DATETIME length is incorrect +ALTER TABLE MMR_MODELS + ADD CONSTRAINT MOD_PK + PRIMARY KEY (ID); + +ALTER TABLE MMR_RESOURCES + ADD CONSTRAINT RSRC_PK + PRIMARY KEY (MODEL_ID); + +ALTER TABLE MMR_OBJECTS + ADD CONSTRAINT OBJ_PK + PRIMARY KEY (ID); + +ALTER TABLE MMR_RESOLVED_OBJECTS + ADD CONSTRAINT RES_OBJ_PK + PRIMARY KEY (OBJ_ID); + +ALTER TABLE MMR_REF_FEATURES + ADD CONSTRAINT RF_PK + PRIMARY KEY (OBJ_ID); + +ALTER TABLE MMR_ATTR_FEATURES + ADD CONSTRAINT AF_PK + PRIMARY KEY (OBJ_ID); +-- ========= STATEMENT 130 ============ +ALTER TABLE MMR_REFS + ADD CONSTRAINT REF_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_BOOLEAN_ATTRS + ADD CONSTRAINT BOL_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_BYTE_ATTRS + ADD CONSTRAINT BYT_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_CHAR_ATTRS + ADD CONSTRAINT CHR_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_CLOB_ATTRS + ADD CONSTRAINT CLOB_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_DOUBLE_ATTRS + ADD CONSTRAINT DBL_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_ENUM_ATTRS + ADD CONSTRAINT ENUM_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_FLOAT_ATTRS + ADD CONSTRAINT FLT_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_INT_ATTRS + ADD CONSTRAINT INT_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_LONG_ATTRS + ADD CONSTRAINT LNG_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); +-- ========= STATEMENT 140 ============ +ALTER TABLE MMR_SHORT_ATTRS + ADD CONSTRAINT SHR_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + +ALTER TABLE MMR_STRING_ATTRS + ADD CONSTRAINT STR_PK + PRIMARY KEY (OBJECT_ID,FEATURE_ID,NDX); + + + -- View for obtaining the features by metaclass + -- (don't use parenthesis) + +CREATE OR REPLACE VIEW MMR_FEATURES AS + SELECT MMR_MODELS.NAMESPACE AS NAMESPACE, + PARENTS.NAME AS CLASS_NAME, + MMR_OBJECTS.NAME AS FEATURE_NAME, + MMR_ATTR_FEATURES.OBJ_ID AS FEATURE_ID, + 'Attribute' AS FEATURE_TYPE + FROM MMR_MODELS JOIN MMR_OBJECTS ON MMR_MODELS.ID=MMR_OBJECTS.MODEL_ID + JOIN MMR_ATTR_FEATURES ON MMR_OBJECTS.ID = MMR_ATTR_FEATURES.OBJ_ID + JOIN MMR_RESOLVED_OBJECTS ON MMR_OBJECTS.ID = MMR_RESOLVED_OBJECTS.OBJ_ID + JOIN MMR_OBJECTS PARENTS ON MMR_RESOLVED_OBJECTS.CONTAINER_ID = PARENTS.ID + UNION ALL + SELECT MMR_MODELS.NAMESPACE AS NAMESPACE, + PARENTS.NAME AS CLASS_NAME, + MMR_OBJECTS.NAME AS FEATURE_NAME, + MMR_REF_FEATURES.OBJ_ID AS FEATURE_ID, + 'Reference' AS FEATURE_TYPE + FROM MMR_MODELS JOIN MMR_OBJECTS ON MMR_MODELS.ID=MMR_OBJECTS.MODEL_ID + JOIN MMR_REF_FEATURES ON MMR_OBJECTS.ID = MMR_REF_FEATURES.OBJ_ID + JOIN MMR_RESOLVED_OBJECTS ON MMR_OBJECTS.ID = MMR_RESOLVED_OBJECTS.OBJ_ID + JOIN MMR_OBJECTS PARENTS ON MMR_RESOLVED_OBJECTS.CONTAINER_ID = PARENTS.ID + ; + + -- View for obtaining the feature values + -- (don't use parenthesis) + +CREATE OR REPLACE VIEW MMR_FEATURE_VALUES AS + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + VALUE AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_BOOLEAN_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + VALUE AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_BYTE_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + VALUE AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_CHAR_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + VALUE AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_DOUBLE_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + VALUE AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_FLOAT_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + VALUE AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_INT_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + VALUE AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_LONG_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + VALUE AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_SHORT_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + VALUE AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_STRING_ATTRS + UNION ALL + SELECT OBJECT_ID, MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + VALUE AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_CLOB_ATTRS + UNION ALL + SELECT MMR_ENUM_ATTRS.OBJECT_ID, MMR_ENUM_ATTRS.MODEL_ID, MMR_ENUM_ATTRS.FEATURE_ID, MMR_ENUM_ATTRS.NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + MMR_OBJECTS.ID AS ENUM_ID, + MMR_REFS.NDX AS ENUM_VALUE, + MMR_OBJECTS.NAME AS ENUM_NAME, + NULL AS REF_OBJ_ID, + NULL AS REF_OBJ_NAME + FROM MMR_ENUM_ATTRS JOIN MMR_OBJECTS ON MMR_ENUM_ATTRS.VALUE = MMR_OBJECTS.ID + JOIN MMR_RESOLVED_OBJECTS ON MMR_OBJECTS.ID = MMR_RESOLVED_OBJECTS.OBJ_ID + JOIN MMR_REFS ON MMR_RESOLVED_OBJECTS.CONTAINER_ID = MMR_REFS.OBJECT_ID + AND MMR_RESOLVED_OBJECTS.OBJ_ID = MMR_REFS.TO_ID + UNION ALL + SELECT OBJECT_ID, MMR_REFS.MODEL_ID AS MODEL_ID, FEATURE_ID, NDX, + NULL AS BOOLEAN_VALUE, + NULL AS BYTE_VALUE, + NULL AS CHAR_VALUE, + NULL AS DOUBLE_VALUE, + NULL AS FLOAT_VALUE, + NULL AS INT_VALUE, + NULL AS LONG_VALUE, + NULL AS SHORT_VALUE, + NULL AS STRING_VALUE, + NULL AS CLOB_VALUE, + NULL AS ENUM_ID, + NULL AS ENUM_VALUE, + NULL AS ENUM_NAME, + MMR_OBJECTS.ID AS REF_OBJ_ID, + MMR_OBJECTS.NAME AS REF_OBJ_NAME + FROM MMR_REFS JOIN MMR_OBJECTS ON MMR_REFS.TO_ID = MMR_OBJECTS.ID; + +-- == new DTC end == + +-- Debezium output is null +-- INSERT INTO MMSCHEMAINFO_CA (SCRIPTNAME,SCRIPTEXECUTEDBY,SCRIPTREV, +-- RELEASEDATE, DATECREATED,DATEUPDATED, UPDATEID,METAMATRIXSERVERURL) +-- SELECT 'MM_CREATE.SQL',USER(),'Seneca.3117', '10/03/2008 12:01 AM',SYSDATE(),SYSDATE(),'',''; +-- ========= STATEMENT 145 ============ diff --git a/tests/integration_tests/debezium/sql/debezium/mysql-ticketmonster-liquibase.ddl b/tests/integration_tests/debezium/sql/debezium/mysql-ticketmonster-liquibase.ddl new file mode 100644 index 00000000000..154075f6722 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/mysql-ticketmonster-liquibase.ddl @@ -0,0 +1,84 @@ +CREATE TABLE `Appearance` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `event_id` bigint(20) DEFAULT NULL, + `event_name` varchar(255) DEFAULT NULL, + `venue_id` bigint(20) DEFAULT NULL, + `venue_name` varchar(255) DEFAULT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UKb2ol0eoqtadvfoxhsnqcajgqa` (`event_id`,`venue_id`) +) ENGINE=InnoDB AUTO_INCREMENT=7 DEFAULT CHARSET=latin1; + +CREATE TABLE `Booking` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `cancellationCode` varchar(255) NOT NULL, + `contactEmail` varchar(255) NOT NULL, + `createdOn` datetime(6) NOT NULL, + `performance_id` bigint(20) DEFAULT NULL, + `performance_name` varchar(255) DEFAULT NULL, + PRIMARY KEY (`id`) +) ENGINE=InnoDB AUTO_INCREMENT=8 DEFAULT CHARSET=latin1; + +CREATE TABLE `Section` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `description` varchar(255) NOT NULL, + `name` varchar(255) NOT NULL, + `numberOfRows` int(11) NOT NULL, + `rowCapacity` int(11) NOT NULL, + `venue_id` bigint(20) DEFAULT NULL, + `venue_name` varchar(255) DEFAULT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UKruosqireipse41rdsuvhqj050` (`name`,`venue_id`) +) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=latin1; + +CREATE TABLE `SectionAllocation` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `allocated` longblob, + `occupiedCount` int(11) NOT NULL, + `performance_id` bigint(20) DEFAULT NULL, + `performance_name` varchar(255) DEFAULT NULL, + `version` bigint(20) NOT NULL, + `section_id` bigint(20) NOT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UK25wlm457x8dmc00we5uw7an3s` (`performance_id`,`section_id`), + KEY `FK60388cvbhb1xyrdhhe546t6dl` (`section_id`), + CONSTRAINT `FK60388cvbhb1xyrdhhe546t6dl` FOREIGN KEY (`section_id`) REFERENCES `Section` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION +) ENGINE=InnoDB AUTO_INCREMENT=57 DEFAULT CHARSET=latin1; + +CREATE TABLE `TicketCategory` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `description` varchar(255) NOT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UK_43455ipnchbn6r4bg8pviai3g` (`description`) +) ENGINE=InnoDB AUTO_INCREMENT=3 DEFAULT CHARSET=latin1; + +CREATE TABLE `Ticket` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `price` float NOT NULL, + `number` int(11) NOT NULL, + `rowNumber` int(11) NOT NULL, + `section_id` bigint(20) DEFAULT NULL, + `ticketCategory_id` bigint(20) NOT NULL, + `tickets_id` bigint(20) DEFAULT NULL, + PRIMARY KEY (`id`), + KEY `FK7xoel6i5b4nrphore8ns2jtld` (`section_id`), + KEY `FK88jejylfnpfqcslai19n4naqf` (`ticketCategory_id`), + KEY `FKolbt9u28gyshci6ek9ep0rl5d` (`tickets_id`), + CONSTRAINT `FK7xoel6i5b4nrphore8ns2jtld` FOREIGN KEY (`section_id`) REFERENCES `Section` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION, + CONSTRAINT `FK88jejylfnpfqcslai19n4naqf` FOREIGN KEY (`ticketCategory_id`) REFERENCES `TicketCategory` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION, + CONSTRAINT `FKolbt9u28gyshci6ek9ep0rl5d` FOREIGN KEY (`tickets_id`) REFERENCES `Booking` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION +) ENGINE=InnoDB AUTO_INCREMENT=13 DEFAULT CHARSET=latin1; + +CREATE TABLE `TicketPriceGuide` ( + `id` bigint(20) NOT NULL AUTO_INCREMENT, + `price` float NOT NULL, + `section_id` bigint(20) NOT NULL, + `show_id` bigint(20) NOT NULL, + `ticketCategory_id` bigint(20) NOT NULL, + PRIMARY KEY (`id`), + UNIQUE KEY `UKro227lwq9ma9gy3ik6gl27xgm` (`section_id`,`show_id`,`ticketCategory_id`), + KEY `FK2nddwnrovke2wgpb8ffahqw` (`show_id`), + KEY `FK3d06sbv9l20tk2wa6yjsw9xdd` (`ticketCategory_id`), + CONSTRAINT `FK2nddwnrovke2wgpb8ffahqw` FOREIGN KEY (`show_id`) REFERENCES `Appearance` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION, + CONSTRAINT `FK3d06sbv9l20tk2wa6yjsw9xdd` FOREIGN KEY (`ticketCategory_id`) REFERENCES `TicketCategory` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION, + CONSTRAINT `FKaqmyqif55ipri4x65o8syt85k` FOREIGN KEY (`section_id`) REFERENCES `Section` (`id`) ON DELETE NO ACTION ON UPDATE NO ACTION +) ENGINE=InnoDB AUTO_INCREMENT=38 DEFAULT CHARSET=latin1; \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql b/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql deleted file mode 100644 index 48cf6071c43..00000000000 --- a/tests/integration_tests/debezium/sql/debezium/nationalized_character_test.sql +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE `NC_TEST` ( - `id` INT NOT NULL AUTO_INCREMENT, - `nc1` nchar default null, - `nc2` nchar(5) default null, - `nc3` nvarchar(25) default null, - PRIMARY KEY (`ID`) -) ENGINE=InnoDB AUTO_INCREMENT=1 DEFAULT CHARSET=utf8mb4; -INSERT INTO NC_TEST(nc1,nc2,nc3) VALUES ('a', '123', 'hello'); \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql b/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql index 6b60b39b928..d26e7d0455d 100644 --- a/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/numeric_column_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: numeric_column_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_751_numeric_column_test ( id INT AUTO_INCREMENT NOT NULL, rating1 NUMERIC, diff --git a/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql b/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql index 7d6da0e272f..aae184d3f04 100644 --- a/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/readbinlog_test.sql @@ -1,8 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: readbinlog_test --- Database needs to be populated to break dependency between MetadataIT and MySqlConnectorIT.shouldValidateAcceptableConfiguration run order --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE person ( name VARCHAR(255) primary key, birthdate DATE NULL, diff --git a/tests/integration_tests/debezium/sql/debezium/real_test.sql b/tests/integration_tests/debezium/sql/debezium/real_test.sql index 7fcf64f78ff..507d5efb708 100644 --- a/tests/integration_tests/debezium/sql/debezium/real_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/real_test.sql @@ -1,3 +1,5 @@ +-- set sql_mode REAL_AS_FLOAT is necessary +set @@session.sql_mode=concat(@@session.sql_mode, ',REAL_AS_FLOAT'); CREATE TABLE `REAL_TEST` ( `id` INT NOT NULL AUTO_INCREMENT, `r1` real default 1.25, diff --git a/tests/integration_tests/debezium/sql/debezium/regression_test.sql b/tests/integration_tests/debezium/sql/debezium/regression_test.sql index 5d396f4d44d..2073c6fbb12 100644 --- a/tests/integration_tests/debezium/sql/debezium/regression_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/regression_test.sql @@ -1,6 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: regression_test --- ---------------------------------------------------------------------------------------------------------------- -- The integration test for this database expects to scan all of the binlog events associated with this database -- without error or problems. The integration test does not modify any records in this database, so this script -- must contain all operations to these tables. @@ -42,6 +39,7 @@ CREATE TABLE dbz84_integer_types_table ( INSERT INTO dbz84_integer_types_table VALUES(127,-128,128,255, default,201,202,203, default,301,302,303, default,401,402,403, default,501,502,503, 1); +SET time_zone='UTC'; -- DBZ-85 handle fractional part of seconds CREATE TABLE dbz_85_fractest ( c1 DATE, @@ -86,7 +84,6 @@ INSERT INTO dbz_123_bitvaluetest VALUES (1,2,64,23989979, 1); INSERT INTO dbz_123_bitvaluetest VALUES (b'1',b'10',b'01000000',b'1011011100000111011011011', 2); -- DBZ-104 handle create table like ... -DROP DATABASE IF EXISTS connector_test; CREATE DATABASE connector_test; CREATE TABLE connector_test.customers ( id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, @@ -95,10 +92,13 @@ CREATE TABLE connector_test.customers ( email VARCHAR(255) NOT NULL UNIQUE KEY ) AUTO_INCREMENT=1001; INSERT INTO connector_test.customers -VALUES (default,"Sally","Thomas","sally.thomas@acme.com"), - (default,"George","Bailey","gbailey@foobar.com"), - (default,"Edward","Walker","ed@walker.com"), - (default,"Anne","Kretchmar","annek@noanswer.org"); +VALUES (default,"Sally","Thomas","sally.thomas@acme.com"); +INSERT INTO connector_test.customers +VALUES (default,"George","Bailey","gbailey@foobar.com"); +INSERT INTO connector_test.customers +VALUES (default,"Edward","Walker","ed@walker.com"); +INSERT INTO connector_test.customers +VALUES (default,"Anne","Kretchmar","annek@noanswer.org"); -- DBZ-147 handle decimal value CREATE TABLE dbz_147_decimalvalues ( diff --git a/tests/integration_tests/debezium/sql/debezium/strategy_test.sql b/tests/integration_tests/debezium/sql/debezium/strategy_test.sql index a7ba9e50f51..4575cfdb546 100644 --- a/tests/integration_tests/debezium/sql/debezium/strategy_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/strategy_test.sql @@ -1,3 +1,4 @@ +SET time_zone='UTC'; CREATE TABLE `dbz4180` ( id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, a NUMERIC(10, 2) NOT NULL DEFAULT 1.23, diff --git a/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql b/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql index fc7b80aac2a..23a79a32a72 100644 --- a/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/table_column_comment_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: table_column_comment_test --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE dbz_4000_comment_test ( id INT AUTO_INCREMENT NOT NULL COMMENT 'pk', name VARCHAR(255) NOT NULL COMMENT 'this is name column', diff --git a/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql b/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql index c80182cb59f..84f55bd7542 100644 --- a/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/timestamp_column_test.sql @@ -1,3 +1,4 @@ +SET time_zone='UTC'; CREATE TABLE t_user_black_list ( `id` int(10) unsigned NOT NULL, `data` varchar(20), @@ -6,10 +7,11 @@ CREATE TABLE t_user_black_list ( PRIMARY KEY (`id`) ); -ALTER TABLE t_user_black_list - MODIFY COLUMN `update_time` datetime(0) NOT NULL - DEFAULT CURRENT_TIMESTAMP(0) COMMENT 'update_time' AFTER create_time; +-- Debezium `create_time` length is nil +-- ALTER TABLE t_user_black_list +-- MODIFY COLUMN `update_time` datetime(0) NOT NULL +-- DEFAULT CURRENT_TIMESTAMP(0) COMMENT 'update_time' AFTER create_time; -INSERT INTO t_user_black_list (`id`,`create_time`,`data`) VALUES (1, CURRENT_TIMESTAMP(), 'test'); +-- INSERT INTO t_user_black_list (`id`,`create_time`,`data`) VALUES (1, CURRENT_TIMESTAMP(), 'test'); UPDATE t_user_black_list SET `data` = 'test2' WHERE `id` = 1; \ No newline at end of file diff --git a/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql b/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql index 878cb7cb2a2..2159c51b7b4 100644 --- a/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/topic_name_sanitization_test.sql @@ -1,7 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: topic-name.sanitization-it --- ---------------------------------------------------------------------------------------------------------------- - CREATE TABLE `dbz_878_some|test@data` ( id INT, some_col VARCHAR(255), diff --git a/tests/integration_tests/debezium/sql/debezium/transaction_metadata_test.sql b/tests/integration_tests/debezium/sql/debezium/transaction_metadata_test.sql new file mode 100644 index 00000000000..9ac761ad2ea --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/transaction_metadata_test.sql @@ -0,0 +1,26 @@ +-- Create and populate our products using a single insert with many rows +CREATE TABLE products ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512), + weight FLOAT +); + +-- Create some customers ... +CREATE TABLE customers ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + first_name VARCHAR(255) NOT NULL, + last_name VARCHAR(255) NOT NULL, + email VARCHAR(255) NOT NULL UNIQUE KEY +) AUTO_INCREMENT=1001; + +-- Create some very simple orders +CREATE TABLE orders ( + order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + order_date DATE NOT NULL, + purchaser INTEGER NOT NULL, + quantity INTEGER NOT NULL, + product_id INTEGER NOT NULL, + FOREIGN KEY order_customer (purchaser) REFERENCES customers(id), + FOREIGN KEY ordered_product (product_id) REFERENCES products(id) +) AUTO_INCREMENT = 10001; diff --git a/tests/integration_tests/debezium/sql/debezium/transactionpayload_test.sql b/tests/integration_tests/debezium/sql/debezium/transactionpayload_test.sql new file mode 100644 index 00000000000..a1b4d302288 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/transactionpayload_test.sql @@ -0,0 +1,29 @@ +-- Create and populate our products using a single insert with many rows +CREATE TABLE products ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512), + weight FLOAT, + code BINARY(16) +); + +-- Create some customers ... +CREATE TABLE customers ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + first_name VARCHAR(255) NOT NULL, + last_name VARCHAR(255) NOT NULL, + email VARCHAR(255) NOT NULL UNIQUE KEY +) AUTO_INCREMENT=1001; + +-- Create some very simple orders +CREATE TABLE orders ( + order_number INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + order_date DATE NOT NULL, + purchaser INTEGER NOT NULL, + quantity INTEGER NOT NULL, + product_id INTEGER NOT NULL, + FOREIGN KEY order_customer (purchaser) REFERENCES customers(id), + FOREIGN KEY ordered_product (product_id) REFERENCES products(id) +) AUTO_INCREMENT = 10001; + +CREATE DATABASE IF NOT EXISTS transactionpayload_test; diff --git a/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql b/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql index 1ee023a636d..638b9a5a0d0 100644 --- a/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql +++ b/tests/integration_tests/debezium/sql/debezium/unsigned_integer_test.sql @@ -1,6 +1,3 @@ --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: unsigned_integer_test --- ---------------------------------------------------------------------------------------------------------------- -- The integration test for this database expects to scan all of the binlog events associated with this database -- without error or problems. The integration test does not modify any records in this database, so this script -- must contain all operations to these tables. diff --git a/tests/integration_tests/debezium/sql/debezium/year_test.sql b/tests/integration_tests/debezium/sql/debezium/year_test.sql new file mode 100644 index 00000000000..4766d56d947 --- /dev/null +++ b/tests/integration_tests/debezium/sql/debezium/year_test.sql @@ -0,0 +1,45 @@ +SET time_zone='UTC'; +CREATE TABLE dbz_1143_year_test ( + id INT AUTO_INCREMENT NOT NULL, + y18 YEAR, + y0018 YEAR, + y2018 YEAR, + d18 DATE, + d0018 DATE, + d2018 DATE, + dt18 DATETIME, + dt0018 DATETIME, + dt2018 DATETIME, + y78 YEAR, + y0078 YEAR, + y1978 YEAR, + d78 DATE, + d0078 DATE, + d1978 DATE, + dt78 DATETIME, + dt0078 DATETIME, + dt1978 DATETIME, + PRIMARY KEY (id) +) DEFAULT CHARSET=utf8; + +INSERT INTO dbz_1143_year_test VALUES ( + default, + '18', + '0018', + '2018', + '18-04-01', + '0018-04-01', + '2018-04-01', + '18-04-01 12:34:56', + '0018-04-01 12:34:56', + '2018-04-01 12:34:56', + '78', + '0078', + '1978', + '78-04-01', + '0078-04-01', + '1978-04-01', + '78-04-01 12:34:56', + '0078-04-01 12:34:56', + '1978-04-01 12:34:56' +); diff --git a/tests/integration_tests/debezium/sql/dml.sql b/tests/integration_tests/debezium/sql/dml.sql index f8fc3f5644c..ce5cf746d2d 100644 --- a/tests/integration_tests/debezium/sql/dml.sql +++ b/tests/integration_tests/debezium/sql/dml.sql @@ -11,9 +11,13 @@ INSERT INTO foo VALUES (3, 3); /* Update PK */ UPDATE foo SET PK = 5, COL = 5 WHERE COL = 3; +SELECT * FROM foo WHERE COL = 3; /* Update Multiple Rows */ UPDATE foo SET COL = 4; +SELECT * FROM foo WHERE PK = 1; +SELECT * FROM foo WHERE PK = 2; +SELECT * FROM foo WHERE PK = 3; /* Update Single Row */ UPDATE foo SET COL = 1 WHERE PK = 5; diff --git a/tests/integration_tests/debezium/src/main.go b/tests/integration_tests/debezium/src/main.go index d9abea1ca57..452b3258484 100644 --- a/tests/integration_tests/debezium/src/main.go +++ b/tests/integration_tests/debezium/src/main.go @@ -94,12 +94,7 @@ func main() { readerTiCDC = prepareKafkaConn(*topicTiCDC) defer readerTiCDC.Close() - dbMySQL = prepareDBConn(KindMySQL, *dbConnMySQL) - defer dbMySQL.MustClose() - dbTiDB = prepareDBConn(KindTiDB, *dbConnTiDB) - defer dbTiDB.MustClose() - - if !runAllTestCases(*testCaseDir) { + if !runAllTestCases(*testCaseDir, *dbConnMySQL, *dbConnTiDB) { os.Exit(1) } } diff --git a/tests/integration_tests/debezium/src/test_cases.go b/tests/integration_tests/debezium/src/test_cases.go index 6d2fb9b416c..f076662c922 100644 --- a/tests/integration_tests/debezium/src/test_cases.go +++ b/tests/integration_tests/debezium/src/test_cases.go @@ -14,6 +14,7 @@ package main import ( + "bytes" "context" "encoding/json" "errors" @@ -29,15 +30,36 @@ import ( "github.com/google/go-cmp/cmp" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/format" + "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/segmentio/kafka-go" "go.uber.org/zap" ) +var timeOut = time.Second * 10 + var ( nFailed = 0 nPassed = 0 ) +var ( + msgKey = "key" + msgValue = "value" +) + +var defaultLength = map[string]float64{ + "INTEGER": 11, + "INTEGER UNSIGNED": 10, + "INT": 11, + "INT UNSIGNED": 10, + "TINYINT": 4, + "TINYINT UNSIGNED": 3, + "BIGINT": 20, + "BIT": 1, + "CHAR": 1, +} + func parseSQLText(data string) (res []ast.StmtNode, warns []error, err error) { p := parser.New() statements, warns, err := p.Parse(data, "utf8mb4", "utf8mb4_bin") @@ -65,25 +87,23 @@ func readAndParseSQLText(sqlFilePath string) []ast.StmtNode { return statements } -func runAllTestCases(dir string) bool { +func runAllTestCases(dir, dbConnMySQL, dbConnTiDB string) bool { var files []string err := filepath.Walk(dir, func(path string, info os.FileInfo, err error) error { if info.IsDir() { return nil } - if !strings.HasSuffix(info.Name(), ".sql") { - return nil + if strings.HasSuffix(info.Name(), ".sql") || strings.HasSuffix(info.Name(), ".ddl") { + files = append(files, path) } - files = append(files, path) return nil }) if err != nil { logger.Panic("Failed to read test case directory", zap.String("dir", dir), zap.Error(err)) } - for _, path := range files { logger.Info("Run", zap.String("case", path)) - runTestCase(path) + runTestCase(path, dbConnMySQL, dbConnTiDB) } if nFailed > 0 { @@ -101,24 +121,32 @@ func runAllTestCases(dir string) bool { return nFailed == 0 } -func resetDB(db *DBHelper) { - db.MustExec("drop database if exists `" + *dbName + "`;") - db.MustExec("create database `" + *dbName + "`;") - db.MustExec("use `" + *dbName + "`;") +func resetDB() { + runSingleQuery("drop database if exists `"+*dbName+"`;", false) + runSingleQuery("create database `"+*dbName+"`;", false) + runSingleQuery("use `"+*dbName+"`;", false) } -func runTestCase(testCasePath string) bool { - resetDB(dbMySQL) - resetDB(dbTiDB) +func runTestCase(testCasePath, dbConnMySQL, dbConnTiDB string) { + dbMySQL = prepareDBConn(KindMySQL, dbConnMySQL) + defer dbMySQL.MustClose() + dbTiDB = prepareDBConn(KindTiDB, dbConnTiDB) + defer dbTiDB.MustClose() + + resetDB() statementKindsToWaitCDCRecord := map[string]bool{ - "Delete": true, - "Insert": true, - "Replace": true, - "Update": true, + "Delete": true, + "Insert": true, + "Replace": true, + "Update": true, + "CreateDatabase": true, + "DropDatabase": true, + "CreateTable": true, + "AlterTable": true, + "DropTable": true, } - hasError := false stmtAsts := readAndParseSQLText(testCasePath) for _, stmt := range stmtAsts { query := strings.TrimSpace(stmt.Text()) @@ -128,102 +156,136 @@ func runTestCase(testCasePath string) bool { if v, ok := statementKindsToWaitCDCRecord[statementKind]; v && ok { waitCDCRows = true } - if runSingleQuery(query, waitCDCRows) { nPassed++ } else { nFailed++ - hasError = true } } - - return hasError } -func fetchNextCDCRecord(reader *kafka.Reader, kind Kind, timeout time.Duration) (map[string]any, error) { +func fetchNextCDCRecord(reader *kafka.Reader, kind Kind, timeout time.Duration) (map[string]any, map[string]any, error) { + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() for { - ctx, cancel := context.WithTimeout(context.Background(), timeout) m, err := reader.FetchMessage(ctx) - cancel() - if err != nil { if errors.Is(err, context.DeadlineExceeded) { - return nil, nil + return nil, nil, nil } - return nil, fmt.Errorf("Failed to read CDC record of %s: %w", kind, err) + return nil, nil, fmt.Errorf("Failed to read CDC record of %s: %w", kind, err) + } + + if err = reader.CommitMessages(ctx, m); err != nil { + return nil, nil, fmt.Errorf("Failed to commit CDC record of %s: %w", kind, err) } if len(m.Value) == 0 { continue } + var keyMap map[string]any var obj map[string]any + err = json.Unmarshal(m.Key, &keyMap) + if err != nil { + return nil, nil, fmt.Errorf("Failed to parse CDC record of %s (msg key=%s): %w", kind, m.Key, err) + } err = json.Unmarshal(m.Value, &obj) if err != nil { - return nil, fmt.Errorf("Failed to parse CDC record of %s (msg=%s): %w", kind, m.Value, err) + return nil, nil, fmt.Errorf("Failed to parse CDC record of %s (msg value=%s): %w", kind, m.Value, err) } - // Ignore DDL events in the Debezium's output - if kind == KindMySQL { - schema, ok := obj["schema"] - if !ok { - return nil, fmt.Errorf("Unexpected CDC record of %s: schema field not exist in %s", kind, m.Value) + payload, ok := obj["payload"].(map[string]any) + if !ok { + return nil, nil, fmt.Errorf("Unexpected CDC record of %s: payload field not exist in %s", kind, m.Value) + } + if kind == KindTiDB { + op, ok := payload["op"] + // Ignore Checkpoint events in the TiCDC's output + if ok && op == "m" { + continue } - if schema.(map[string]any)["name"] == "io.debezium.connector.mysql.SchemaChangeValue" { + // Only handle DDL received from partition-0 should be enough. + if !ok && m.Partition != 0 { continue } } - - return obj, nil - } -} - -func fetchAllCDCRecords(reader *kafka.Reader, kind Kind) []map[string]any { - var records []map[string]any - for { - waitTimeout := time.Millisecond * 1000 - if len(records) == 0 { - // Wait a bit longer for the first record - if kind == KindMySQL { - waitTimeout = 10 * time.Second - } else if kind == KindTiDB { - waitTimeout = 20 * time.Second - } + if ddl, ok := payload["ddl"]; ok { + payload["ddl"] = normalizeSQL(ddl.(string)) } - obj, err := fetchNextCDCRecord(reader, kind, waitTimeout) - if err != nil { - logger.Error( - "Received error when fetching CDC record", - zap.Error(err), - zap.String("kind", string(kind))) - break - } - if obj == nil { - // No more records - break + // HACK + // In DDL events, some data types can't decode correctly, but the query is exact. + if kind == KindMySQL { + if tableChanges, ok := payload["tableChanges"]; ok { + if tables, ok := tableChanges.([]any); ok && len(tables) > 0 { + if table0, ok := tables[0].(map[string]any); ok { + if table, ok := table0["table"]; ok && table != nil { + if columns, ok := table.(map[string]any); ok { + for _, col := range columns["columns"].([]any) { + col := col.(map[string]any) + v := col["typeName"].(string) + switch v { + case "INT", "INT UNSIGNED", "INTEGER", "INTEGER UNSIGNED", "TINYINT", "TINYINT UNSIGNED", "BIGINT", + "BIT", "CHAR": + if col["length"] == defaultLength[v] { + col["length"] = nil + } + } + switch v { + case "INTEGER", "INTEGER UNSIGNED": + col["typeName"] = replaceString(col["typeName"], "INTEGER", "INT") + col["typeExpression"] = replaceString(col["typeExpression"], "INTEGER", "INT") + case "NUMERIC": + col["typeName"] = replaceString(col["typeName"], "NUMERIC", "DECIMAL") + col["typeExpression"] = replaceString(col["typeExpression"], "NUMERIC", "DECIMAL") + col["jdbcType"] = float64(3) + case "REAL": + col["typeName"] = replaceString(col["typeName"], "REAL", "FLOAT") + col["typeExpression"] = replaceString(col["typeExpression"], "REAL", "FLOAT") + col["jdbcType"] = float64(6) + } + } + } + } + } + } + } } - - records = append(records, obj) + return keyMap, obj, nil } +} - return records +func replaceString(s any, old any, new any) string { + return strings.Replace(s.(string), old.(string), new.(string), 1) } var ignoredRecordPaths = map[string]bool{ - `{map[string]any}["schema"]`: true, `{map[string]any}["payload"].(map[string]any)["source"]`: true, `{map[string]any}["payload"].(map[string]any)["ts_ms"]`: true, } var headingColor = color.New(color.FgHiWhite, color.Bold) -func printObj(obj any) { +func printRecord(obj any) { v, _ := json.MarshalIndent(obj, "", " ") quick.Highlight(os.Stdout, string(v), "json", "terminal16m", "vs") fmt.Println() } +func normalizeSQL(sql string) string { + p := parser.New() + p.SetSQLMode(mysql.ModeRealAsFloat) // necessary + stmt, err := p.ParseOneStmt(sql, "", "") + buf := new(bytes.Buffer) + if err != nil { + panic(fmt.Sprintf("parse sql failed %s", err)) + } + restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, buf) + stmt.Restore(restoreCtx) + return buf.String() +} + func runSingleQuery(query string, waitCDCRows bool) bool { { wg := &sync.WaitGroup{} @@ -238,8 +300,22 @@ func runSingleQuery(query string, waitCDCRows bool) bool { }() wg.Wait() } - if !waitCDCRows { + wg := &sync.WaitGroup{} + wg.Add(2) + go func() { + if _, _, err := fetchNextCDCRecord(readerDebezium, KindMySQL, timeOut); err != nil { + logger.Error("fetch record failed", zap.Error(err)) + } + wg.Done() + }() + go func() { + if _, _, err := fetchNextCDCRecord(readerTiCDC, KindTiDB, timeOut); err != nil { + logger.Error("fetch record failed", zap.Error(err)) + } + wg.Done() + }() + wg.Wait() return true } @@ -253,38 +329,31 @@ func runSingleQuery(query string, waitCDCRows bool) bool { testCasePassed = false } - var objsDebezium []map[string]any - var objsTiCDC []map[string]any + var keyMapsDebezium map[string]any + var objsDebezium map[string]any + var keyMapsTiCDC map[string]any + var objsTiCDC map[string]any { wg := &sync.WaitGroup{} wg.Add(2) go func() { - objsDebezium = fetchAllCDCRecords(readerDebezium, KindMySQL) + var err error + keyMapsDebezium, objsDebezium, err = fetchNextCDCRecord(readerDebezium, KindMySQL, timeOut) + if err != nil { + logger.Error("fetch record failed", zap.Error(err)) + } wg.Done() }() go func() { - objsTiCDC = fetchAllCDCRecords(readerTiCDC, KindTiDB) + var err error + keyMapsTiCDC, objsTiCDC, err = fetchNextCDCRecord(readerTiCDC, KindTiDB, timeOut) + if err != nil { + logger.Error("fetch record failed", zap.Error(err)) + } wg.Done() }() wg.Wait() } - - if len(objsDebezium) != len(objsTiCDC) { - onError(fmt.Errorf( - "Mismatch CDC rows: Got %d rows from Debezium and %d rows from TiCDC", - len(objsDebezium), - len(objsTiCDC))) - headingColor.Print("\nDebezium output:\n\n") - for _, obj := range objsDebezium { - printObj(obj) - } - headingColor.Print("\nTiCDC output:\n\n") - for _, obj := range objsTiCDC { - printObj(obj) - } - return testCasePassed - } - cmpOption := cmp.FilterPath( func(p cmp.Path) bool { path := p.GoString() @@ -294,17 +363,17 @@ func runSingleQuery(query string, waitCDCRows bool) bool { cmp.Ignore(), ) - for i := 0; i < len(objsDebezium); i++ { - objDebezium := objsDebezium[i] - objTiCDC := objsTiCDC[i] - if diff := cmp.Diff(objDebezium, objTiCDC, cmpOption); diff != "" { - onError(fmt.Errorf("Found mismatch CDC record (output row #%d)", i+1)) - headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") - quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") - fmt.Println() - continue - } + if diff := cmp.Diff(keyMapsDebezium, keyMapsTiCDC, cmpOption); diff != "" { + onError(fmt.Errorf("Found mismatch CDC record (msg type %s)", msgKey)) + headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") + quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") + fmt.Println() + } + if diff := cmp.Diff(objsDebezium, objsTiCDC, cmpOption); diff != "" { + onError(fmt.Errorf("Found mismatch CDC record (msg type %s)", msgValue)) + headingColor.Print("\nCDC Result Diff (-debezium +ticdc):\n\n") + quick.Highlight(os.Stdout, diff, "diff", "terminal16m", "murphy") + fmt.Println() } - return testCasePassed } diff --git a/tests/integration_tests/foreign_key/conf/cf.toml b/tests/integration_tests/foreign_key/conf/cf.toml new file mode 100644 index 00000000000..34e89975aeb --- /dev/null +++ b/tests/integration_tests/foreign_key/conf/cf.toml @@ -0,0 +1,2 @@ +[filter] +rules = ['*.*', '!foreign_key.t12'] \ No newline at end of file diff --git a/tests/integration_tests/foreign_key/conf/diff_config.toml b/tests/integration_tests/foreign_key/conf/diff_config.toml index e2b61765800..4451d961132 100644 --- a/tests/integration_tests/foreign_key/conf/diff_config.toml +++ b/tests/integration_tests/foreign_key/conf/diff_config.toml @@ -13,7 +13,7 @@ check-struct-only = false target-instance = "tidb0" - target-check-tables = ["foreign_key.?*"] + target-check-tables = ["foreign_key.?*", "!foreign_key.t12"] [data-sources] [data-sources.mysql1] diff --git a/tests/integration_tests/foreign_key/data/prepare.sql b/tests/integration_tests/foreign_key/data/prepare.sql index 77d296283fe..7fc79521034 100644 --- a/tests/integration_tests/foreign_key/data/prepare.sql +++ b/tests/integration_tests/foreign_key/data/prepare.sql @@ -62,5 +62,10 @@ alter table t11 add constraint fk_6 foreign key (b) references t10(id) on delete delete from t10 where id=1; update t10 set id=id+10 where id=2; -create table finish_mark (id int PRIMARY KEY); +-- Test foreign key only sync child. The `t12` table is excluded in the log filter. +create table t12 (id int key); +create table t13 (id int key, constraint fk_3 foreign key (id) references t12 (id)); +insert into t12 values (1),(2),(3),(4),(5); +insert into t13 values (1),(2),(3),(4),(5); +create table finish_mark (id int PRIMARY KEY); diff --git a/tests/integration_tests/foreign_key/run.sh b/tests/integration_tests/foreign_key/run.sh index 43c522437a8..656b058c417 100644 --- a/tests/integration_tests/foreign_key/run.sh +++ b/tests/integration_tests/foreign_key/run.sh @@ -30,7 +30,7 @@ function run() { ;; *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/" ;; esac - run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config=$CUR/conf/cf.toml case $SINK_TYPE in kafka) run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&partition-num=4&version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; storage) run_storage_consumer $WORK_DIR $SINK_URI "" "" ;; @@ -41,6 +41,10 @@ function run() { check_table_exists foreign_key.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + check_table_not_exists foreign_key.t12 ${DOWN_TIDB_HOST} ${DOWN_TIDB_HOST} + run_sql "select count(1) from foreign_key.t13;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_contains "count(1): 5" + cleanup_process $CDC_BINARY } 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/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index df2ab20ed4d..78f86f40935 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -11,7 +11,7 @@ group_num=${group#G} # Other tests that only support mysql: batch_update_to_no_batch ddl_reentrant # changefeed_fast_fail changefeed_resume_with_checkpoint_ts sequence # multi_cdc_cluster capture_suicide_while_balance_table -mysql_only="bdr_mode capture_suicide_while_balance_table syncpoint syncpoint_check_ts server_config_compatibility changefeed_dup_error_restart" +mysql_only="bdr_mode capture_suicide_while_balance_table syncpoint syncpoint_check_ts server_config_compatibility changefeed_dup_error_restart safe_mode" mysql_only_http="http_api http_api_tls api_v2 http_api_tls_with_user_auth cli_tls_with_auth" mysql_only_consistent_replicate="consistent_replicate_ddl consistent_replicate_gbk consistent_replicate_nfs consistent_replicate_storage_file consistent_replicate_storage_file_large_value consistent_replicate_storage_s3 consistent_partition_table" @@ -38,7 +38,7 @@ groups=( # G02 "$mysql_only_consistent_replicate $kafka_only_v2 $storage_only_csv" # G03 - 'row_format drop_many_tables processor_stop_delay partition_table' + 'row_format drop_many_tables processor_stop_delay partition_table ddl_with_exists' # G04 'foreign_key ddl_puller_lag ddl_only_block_related_table changefeed_auto_stop' # G05 diff --git a/tests/integration_tests/safe_mode/conf/diff_config.toml b/tests/integration_tests/safe_mode/conf/diff_config.toml new file mode 100644 index 00000000000..7bf5d3ffe98 --- /dev/null +++ b/tests/integration_tests/safe_mode/conf/diff_config.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/safe_mode/sync_diff/output" + + source-instances = ["mysql1"] + + target-instance = "tidb0" + + target-check-tables = ["safe_mode.t"] + +[data-sources] +[data-sources.mysql1] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + +[data-sources.tidb0] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" diff --git a/tests/integration_tests/safe_mode/data/create_table.sql b/tests/integration_tests/safe_mode/data/create_table.sql new file mode 100644 index 00000000000..9ed09d75ae4 --- /dev/null +++ b/tests/integration_tests/safe_mode/data/create_table.sql @@ -0,0 +1,5 @@ +drop database if exists `safe_mode`; +create database `safe_mode`; +use `safe_mode`; + +create table t(id int key, a varchar(200)); diff --git a/tests/integration_tests/safe_mode/data/insert.sql b/tests/integration_tests/safe_mode/data/insert.sql new file mode 100644 index 00000000000..2cd44003173 --- /dev/null +++ b/tests/integration_tests/safe_mode/data/insert.sql @@ -0,0 +1,3 @@ +use `safe_mode`; +insert into t values(1, "hello"); +insert into t values(2, "world"); diff --git a/tests/integration_tests/safe_mode/data/update.sql b/tests/integration_tests/safe_mode/data/update.sql new file mode 100644 index 00000000000..97963570c7d --- /dev/null +++ b/tests/integration_tests/safe_mode/data/update.sql @@ -0,0 +1,4 @@ +use `safe_mode`; +-- update non key column +update t set a = "hello2" where id = 1; +update t set a = "world2" where id = 2; diff --git a/tests/integration_tests/safe_mode/run.sh b/tests/integration_tests/safe_mode/run.sh new file mode 100755 index 00000000000..eac8e3c8352 --- /dev/null +++ b/tests/integration_tests/safe_mode/run.sh @@ -0,0 +1,51 @@ +#!/bin/bash + +set -eu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +function run() { + if [ "$SINK_TYPE" != "mysql" ]; then + return + fi + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + run_sql_file $CUR/data/create_table.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql_file $CUR/data/create_table.sql ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + # insert data into upstream but not downstream + run_sql_file $CUR/data/insert.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + case $SINK_TYPE in + *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?safe-mode=true" ;; + esac + run_cdc_cli changefeed create --sink-uri="$SINK_URI" + + # test update sql can be split into delete + replace at all times in safe mode + # otherwise the update sql will have no effect on the downstream and the downstream will have no data. + sleep 10 + run_sql_file $CUR/data/update.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + run_sql "CREATE TABLE safe_mode.finish_mark (a int primary key);" + sleep 30 + check_table_exists "safe_mode.finish_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" 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=