Skip to content

Commit

Permalink
add-index part
Browse files Browse the repository at this point in the history
  • Loading branch information
D3Hunter committed Sep 26, 2023
1 parent 250fb07 commit 2719303
Show file tree
Hide file tree
Showing 2 changed files with 38 additions and 3 deletions.
10 changes: 7 additions & 3 deletions ddl/backfilling_dist_scheduler.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,8 @@ type backfillDistScheduler struct {
taskTable scheduler.TaskTable
backendCtx ingest.BackendCtx
jobID int64

uniqueIndex bool
}

func newBackfillDistScheduler(ctx context.Context, id string, task *proto.Task, taskTable scheduler.TaskTable, d *ddl) scheduler.Scheduler {
Expand Down Expand Up @@ -146,6 +148,7 @@ func (s *backfillDistScheduler) Init(ctx context.Context) error {
if err != nil {
return errors.Trace(err)
}
s.uniqueIndex = idx.Unique
s.backendCtx = bc
s.jobID = job.ID
return nil
Expand All @@ -160,9 +163,10 @@ func (s *backfillDistScheduler) GetSubtaskExecutor(ctx context.Context, task *pr
}
}

func (*backfillDistScheduler) IsIdempotent(*proto.Subtask) bool {
// TODO: implement this later.
return false
func (s *backfillDistScheduler) IsIdempotent(*proto.Subtask) bool {
// unique index depends on MVCC to do deduplicate, so it's not idempotent.
// we might save engine TS later to make it idempotent.
return !s.uniqueIndex
}

func (s *backfillDistScheduler) Close() {
Expand Down
31 changes: 31 additions & 0 deletions ddl/backfilling_dist_scheduler_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2023 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package ddl

import (
"testing"

"github.com/pingcap/tidb/disttask/framework/proto"
"github.com/stretchr/testify/require"
)

func TestIsIdempotent(t *testing.T) {
s := &backfillDistScheduler{
uniqueIndex: false,
}
require.True(t, s.IsIdempotent(&proto.Subtask{}))
s.uniqueIndex = true
require.False(t, s.IsIdempotent(&proto.Subtask{}))
}

0 comments on commit 2719303

Please sign in to comment.