Skip to content

Commit

Permalink
ddl: tiny clean up (pingcap#2468)
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala authored and hanfei1991 committed Jan 15, 2017
1 parent 8d426be commit 5940259
Show file tree
Hide file tree
Showing 5 changed files with 39 additions and 38 deletions.
6 changes: 1 addition & 5 deletions ddl/bg_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/terror"
)

// handleBgJobQueue handles the background job queue.
Expand All @@ -34,11 +33,8 @@ func (d *ddl) handleBgJobQueue() error {
err := kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error {
t := meta.NewMeta(txn)
owner, err := d.checkOwner(t, bgJobFlag)
if terror.ErrorEqual(err, errNotOwner) {
return nil
}
if err != nil {
return errors.Trace(err)
return errors.Trace(filterError(err, errNotOwner))
}

// Get the first background job and run it.
Expand Down
15 changes: 6 additions & 9 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -169,13 +169,10 @@ func (d *ddl) onAddColumn(t *meta.Meta, job *model.Job) error {
err = d.runReorgJob(func() error {
return d.addTableColumn(tbl, columnInfo, reorgInfo, job)
})
if terror.ErrorEqual(err, errWaitReorgTimeout) {
if err != nil {
// If the timeout happens, we should return.
// Then check for the owner and re-wait job to finish.
return nil
}
if err != nil {
return errors.Trace(err)
return errors.Trace(filterError(err, errWaitReorgTimeout))
}
}

Expand Down Expand Up @@ -357,11 +354,11 @@ func (d *ddl) backfillColumnInTxn(t table.Table, colMeta *columnMeta, handles []
log.Debug("[ddl] backfill column...", handle)
rowKey := t.RecordKey(handle)
rowVal, err := txn.Get(rowKey)
if terror.ErrorEqual(err, kv.ErrNotExist) {
// If row doesn't exist, skip it.
continue
}
if err != nil {
if terror.ErrorEqual(err, kv.ErrNotExist) {
// If row doesn't exist, skip it.
continue
}
return 0, errors.Trace(err)
}

Expand Down
7 changes: 7 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -372,6 +372,13 @@ func (d *ddl) setHook(h Callback) {
d.hook = h
}

func filterError(err, exceptErr error) error {
if terror.ErrorEqual(err, exceptErr) {
return nil
}
return errors.Trace(err)
}

// DDL error codes.
const (
codeInvalidWorker terror.ErrCode = 1
Expand Down
22 changes: 10 additions & 12 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,11 +255,11 @@ func (d *ddl) onCreateIndex(t *meta.Meta, job *model.Job) error {
err = d.runReorgJob(func() error {
return d.addTableIndex(tbl, indexInfo, reorgInfo, job)
})
if terror.ErrorEqual(err, errWaitReorgTimeout) {
// if timeout, we should return, check for the owner and re-wait job done.
return nil
}
if err != nil {
if terror.ErrorEqual(err, errWaitReorgTimeout) {
// if timeout, we should return, check for the owner and re-wait job done.
return nil
}
if terror.ErrorEqual(err, kv.ErrKeyExists) {
log.Warnf("[ddl] run DDL job %v err %v, convert job to rollback job", job, err)
err = d.convert2RollbackJob(t, job, tblInfo, indexInfo)
Expand Down Expand Up @@ -346,13 +346,10 @@ func (d *ddl) onDropIndex(t *meta.Meta, job *model.Job) error {
err = d.runReorgJob(func() error {
return d.dropTableIndex(indexInfo, job)
})
if terror.ErrorEqual(err, errWaitReorgTimeout) {
if err != nil {
// If the timeout happens, we should return.
// Then check for the owner and re-wait job to finish.
return nil
}
if err != nil {
return errors.Trace(err)
return errors.Trace(filterError(err, errWaitReorgTimeout))
}

// All reorganization jobs are done, drop this index.
Expand Down Expand Up @@ -711,9 +708,10 @@ func (d *ddl) iterateSnapshotRows(t table.Table, version uint64, seekHandle int6
}

err = kv.NextUntil(it, util.RowKeyPrefixFilter(rk))
if terror.ErrorEqual(err, kv.ErrNotExist) {
break
} else if err != nil {
if err != nil {
if terror.ErrorEqual(err, kv.ErrNotExist) {
break
}
return errors.Trace(err)
}
}
Expand Down
27 changes: 15 additions & 12 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,10 +69,11 @@ func (d *ddl) onDropTable(t *meta.Meta, job *model.Job) error {

// Check this table's database.
tblInfo, err := t.GetTable(schemaID, tableID)
if terror.ErrorEqual(err, meta.ErrDBNotExists) {
job.State = model.JobCancelled
return errors.Trace(infoschema.ErrDatabaseNotExists)
} else if err != nil {
if err != nil {
if terror.ErrorEqual(err, meta.ErrDBNotExists) {
job.State = model.JobCancelled
return errors.Trace(infoschema.ErrDatabaseNotExists)
}
return errors.Trace(err)
}

Expand Down Expand Up @@ -149,10 +150,11 @@ func (d *ddl) getTable(schemaID int64, tblInfo *model.TableInfo) (table.Table, e
func getTableInfo(t *meta.Meta, job *model.Job, schemaID int64) (*model.TableInfo, error) {
tableID := job.TableID
tblInfo, err := t.GetTable(schemaID, tableID)
if terror.ErrorEqual(err, meta.ErrDBNotExists) {
job.State = model.JobCancelled
return nil, errors.Trace(infoschema.ErrDatabaseNotExists)
} else if err != nil {
if err != nil {
if terror.ErrorEqual(err, meta.ErrDBNotExists) {
job.State = model.JobCancelled
return nil, errors.Trace(infoschema.ErrDatabaseNotExists)
}
return nil, errors.Trace(err)
} else if tblInfo == nil {
job.State = model.JobCancelled
Expand Down Expand Up @@ -260,10 +262,11 @@ func (d *ddl) onRenameTable(t *meta.Meta, job *model.Job) error {
func checkTableNotExists(t *meta.Meta, job *model.Job, schemaID int64, tableName string) error {
// Check this table's database.
tables, err := t.ListTables(schemaID)
if terror.ErrorEqual(err, meta.ErrDBNotExists) {
job.State = model.JobCancelled
return errors.Trace(infoschema.ErrDatabaseNotExists)
} else if err != nil {
if err != nil {
if terror.ErrorEqual(err, meta.ErrDBNotExists) {
job.State = model.JobCancelled
return errors.Trace(infoschema.ErrDatabaseNotExists)
}
return errors.Trace(err)
}

Expand Down

0 comments on commit 5940259

Please sign in to comment.