Skip to content

Commit

Permalink
*: return error instead of log.Fatal for TableFromMeta
Browse files Browse the repository at this point in the history
  • Loading branch information
siddontang committed Dec 9, 2015
1 parent 512dbae commit 7336360
Show file tree
Hide file tree
Showing 7 changed files with 28 additions and 17 deletions.
8 changes: 6 additions & 2 deletions ddl/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,8 +149,12 @@ func (d *ddl) onDropSchema(t *meta.Meta, job *model.Job) error {
func (d *ddl) dropSchemaData(dbInfo *model.DBInfo, tables []*model.TableInfo) error {
for _, tblInfo := range tables {
alloc := autoid.NewAllocator(d.store, dbInfo.ID)
t := table.TableFromMeta(alloc, tblInfo)
err := d.dropTableData(t)
t, err := table.TableFromMeta(alloc, tblInfo)
if err != nil {
return errors.Trace(err)
}

err = d.dropTableData(t)
if err != nil {
return errors.Trace(err)
}
Expand Down
4 changes: 2 additions & 2 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,8 +152,8 @@ func (d *ddl) onDropTable(t *meta.Meta, job *model.Job) error {

func (d *ddl) getTable(t *meta.Meta, schemaID int64, tblInfo *model.TableInfo) (table.Table, error) {
alloc := autoid.NewAllocator(d.store, schemaID)
tbl := table.TableFromMeta(alloc, tblInfo)
return tbl, nil
tbl, err := table.TableFromMeta(alloc, tblInfo)
return tbl, errors.Trace(err)
}

func (d *ddl) getTableInfo(t *meta.Meta, job *model.Job) (*model.TableInfo, error) {
Expand Down
3 changes: 2 additions & 1 deletion ddl/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,8 @@ func testGetTable(c *C, d *ddl, schemaID int64, tableID int64) table.Table {
return nil
})
alloc := autoid.NewAllocator(d.store, schemaID)
tbl := table.TableFromMeta(alloc, tblInfo)
tbl, err := table.TableFromMeta(alloc, tblInfo)
c.Assert(err, IsNil)
return tbl
}

Expand Down
4 changes: 2 additions & 2 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,8 +86,8 @@ func (do *Domain) loadInfoSchema(txn kv.Transaction) (err error) {
}

log.Infof("loadInfoSchema %d", schemaMetaVersion)
do.infoHandle.Set(schemas, schemaMetaVersion)
return
err = do.infoHandle.Set(schemas, schemaMetaVersion)
return errors.Trace(err)
}

// InfoSchema gets information schema from domain.
Expand Down
10 changes: 8 additions & 2 deletions infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package infoschema
import (
"sync/atomic"

"github.com/juju/errors"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/model"
Expand Down Expand Up @@ -201,7 +202,7 @@ func NewHandle(store kv.Storage) *Handle {
}

// Set sets DBInfo to information schema.
func (h *Handle) Set(newInfo []*model.DBInfo, schemaMetaVersion int64) {
func (h *Handle) Set(newInfo []*model.DBInfo, schemaMetaVersion int64) error {
info := &infoSchema{
schemaNameToID: map[string]int64{},
tableNameToID: map[tableName]int64{},
Expand All @@ -213,12 +214,16 @@ func (h *Handle) Set(newInfo []*model.DBInfo, schemaMetaVersion int64) {
columnIndices: map[int64][]*model.IndexInfo{},
schemaMetaVersion: schemaMetaVersion,
}
var err error
for _, di := range newInfo {
info.schemas[di.ID] = di
info.schemaNameToID[di.Name.L] = di.ID
for _, t := range di.Tables {
alloc := autoid.NewAllocator(h.store, di.ID)
info.tables[t.ID] = table.TableFromMeta(alloc, t)
info.tables[t.ID], err = table.TableFromMeta(alloc, t)
if err != nil {
return errors.Trace(err)
}
tname := tableName{di.Name.L, t.Name.L}
info.tableNameToID[tname] = t.ID
for _, c := range t.Columns {
Expand All @@ -236,6 +241,7 @@ func (h *Handle) Set(newInfo []*model.DBInfo, schemaMetaVersion int64) {
}
}
h.value.Store(info)
return nil
}

// Get gets information schema from Handle.
Expand Down
2 changes: 1 addition & 1 deletion table/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ type Table interface {

// TableFromMeta builds a table.Table from *model.TableInfo.
// Currently, it is assigned to tables.TableFromMeta in tidb package's init function.
var TableFromMeta func(alloc autoid.Allocator, tblInfo *model.TableInfo) Table
var TableFromMeta func(alloc autoid.Allocator, tblInfo *model.TableInfo) (Table, error)

// Ident is the table identifier composed of schema name and table name.
// TODO: Move out
Expand Down
14 changes: 7 additions & 7 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,15 +61,15 @@ type Table struct {
}

// TableFromMeta creates a Table instance from model.TableInfo.
func TableFromMeta(alloc autoid.Allocator, tblInfo *model.TableInfo) table.Table {
func TableFromMeta(alloc autoid.Allocator, tblInfo *model.TableInfo) (table.Table, error) {
if tblInfo.State == model.StateNone {
log.Fatalf("table %s can't be in none state", tblInfo.Name)
return nil, errors.Errorf("table %s can't be in none state", tblInfo.Name)
}

columns := make([]*column.Col, 0, len(tblInfo.Columns))
for _, colInfo := range tblInfo.Columns {
if colInfo.State == model.StateNone {
log.Fatalf("column %s can't be in none state", colInfo.Name)
return nil, errors.Errorf("column %s can't be in none state", colInfo.Name)
}

col := &column.Col{ColumnInfo: *colInfo}
Expand All @@ -78,12 +78,12 @@ func TableFromMeta(alloc autoid.Allocator, tblInfo *model.TableInfo) table.Table

t, err := NewTable(tblInfo.ID, tblInfo.Name.O, columns, alloc)
if err != nil {
log.Fatalf("create new table failed - %s - %v", tblInfo.Name.O, errors.ErrorStack(err))
return nil, errors.Trace(err)
}

for _, idxInfo := range tblInfo.Indices {
if idxInfo.State == model.StateNone {
log.Fatalf("index %s can't be in none state", idxInfo.Name)
return nil, errors.Errorf("index %s can't be in none state", idxInfo.Name)
}

idx := &column.IndexedCol{
Expand All @@ -92,14 +92,14 @@ func TableFromMeta(alloc autoid.Allocator, tblInfo *model.TableInfo) table.Table

idx.X, err = kv.NewKVIndex(t.IndexPrefix(), idxInfo.Name.L, idxInfo.ID, idxInfo.Unique)
if err != nil {
log.Fatalf("create new index failed - %s - %v", idxInfo.Name.O, errors.ErrorStack(err))
return nil, errors.Trace(err)
}

t.AddIndex(idx)
}

t.state = tblInfo.State
return t
return t, nil
}

// NewTable constructs a Table instance.
Expand Down

0 comments on commit 7336360

Please sign in to comment.