Skip to content

Commit

Permalink
domain: fix stats updating bug when no stats woker (#7864) (#7871)
Browse files Browse the repository at this point in the history
  • Loading branch information
alivxxx authored and ngaut committed Oct 11, 2018
1 parent 173a792 commit 44cb05f
Show file tree
Hide file tree
Showing 7 changed files with 31 additions and 5 deletions.
24 changes: 22 additions & 2 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/grpc-ecosystem/go-grpc-prometheus"
"github.com/juju/errors"
"github.com/ngaut/pools"
"github.com/ngaut/sync2"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
Expand Down Expand Up @@ -52,6 +53,7 @@ type Domain struct {
privHandle *privileges.Handle
statsHandle unsafe.Pointer
statsLease time.Duration
statsUpdating sync2.AtomicInt32
ddl ddl.DDL
m sync.Mutex
SchemaValidator SchemaValidator
Expand Down Expand Up @@ -592,6 +594,20 @@ func (do *Domain) CreateStatsHandle(ctx sessionctx.Context) {
atomic.StorePointer(&do.statsHandle, unsafe.Pointer(statistics.NewHandle(ctx, do.statsLease)))
}

// StatsUpdating checks if the stats worker is updating.
func (do *Domain) StatsUpdating() bool {
return do.statsUpdating.Get() > 0
}

// SetStatsUpdating sets the value of stats updating.
func (do *Domain) SetStatsUpdating(val bool) {
if val {
do.statsUpdating.Set(1)
} else {
do.statsUpdating.Set(0)
}
}

// RunAutoAnalyze indicates if this TiDB server starts auto analyze worker and can run auto analyze job.
var RunAutoAnalyze = true

Expand All @@ -608,6 +624,7 @@ func (do *Domain) UpdateTableStatsLoop(ctx sessionctx.Context) error {
}
owner := do.newStatsOwner()
do.wg.Add(1)
do.SetStatsUpdating(true)
go do.updateStatsWorker(ctx, owner)
if RunAutoAnalyze {
do.wg.Add(1)
Expand Down Expand Up @@ -656,7 +673,11 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager)
} else {
log.Info("[stats] init stats info takes ", time.Now().Sub(t))
}
defer recoverInDomain("updateStatsWorker", false)
defer func() {
do.SetStatsUpdating(false)
recoverInDomain("updateStatsWorker", false)
do.wg.Done()
}()
for {
select {
case <-loadTicker.C:
Expand All @@ -665,7 +686,6 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager)
log.Debug("[stats] update stats info fail: ", errors.ErrorStack(err))
}
case <-do.exit:
do.wg.Done()
return
// This channel is sent only by ddl owner or the drop stats executor.
case t := <-statsHandle.DDLEventCh():
Expand Down
3 changes: 2 additions & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,8 @@ func (e *AnalyzeExec) Next(ctx context.Context, chk *chunk.Chunk) error {
close(taskCh)
dom := domain.GetDomain(e.ctx)
lease := dom.StatsHandle().Lease
if lease > 0 {
// The analyze result will be consumed by background stats worker.
if lease > 0 && dom.StatsUpdating() {
var err1 error
for i := 0; i < len(e.tasks); i++ {
result := <-resultCh
Expand Down
1 change: 1 addition & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,7 @@ func (s *testSuite) SetUpSuite(c *C) {
}
d, err := session.BootstrapSession(s.store)
c.Assert(err, IsNil)
d.SetStatsUpdating(true)
s.domain = d
}

Expand Down
1 change: 1 addition & 0 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ func (s *testSuite) TestDataForTableRowsCountField(c *C) {
defer store.Close()
do, err := session.BootstrapSession(store)
c.Assert(err, IsNil)
do.SetStatsUpdating(true)
defer do.Close()

h := do.StatsHandle()
Expand Down
1 change: 1 addition & 0 deletions plan/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -632,6 +632,7 @@ func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) {
session.SetSchemaLease(0)
session.SetStatsLease(0)
dom, err := session.BootstrapSession(store)
dom.SetStatsUpdating(true)
return store, dom, errors.Trace(err)
}

Expand Down
5 changes: 3 additions & 2 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1104,8 +1104,9 @@ func CreateSession(store kv.Storage) (Session, error) {
}
privilege.BindPrivilegeManager(s, pm)

// Add statsUpdateHandle.
if do.StatsHandle() != nil {
// Add stats collector, and it will be freed by background stats worker
// which periodically updates stats using the collected data.
if do.StatsHandle() != nil && do.StatsUpdating() {
s.statsCollector = do.StatsHandle().NewSessionStatsCollector()
}

Expand Down
1 change: 1 addition & 0 deletions statistics/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -445,5 +445,6 @@ func newStoreWithBootstrap(statsLease time.Duration) (kv.Storage, *domain.Domain
session.SetStatsLease(statsLease)
domain.RunAutoAnalyze = false
do, err := session.BootstrapSession(store)
do.SetStatsUpdating(true)
return store, do, errors.Trace(err)
}

0 comments on commit 44cb05f

Please sign in to comment.