Skip to content

Commit

Permalink
server: add a configuration to enable GC of system.rangelog
Browse files Browse the repository at this point in the history
system.rangelog table currently grows unboundedly. The rate of
growth is slow (as long as there is no replica rebalancing
thrashing), but it can still become a problem in long running
clusters.

This commit adds cluster settings to specify interval and TTL
for rows in system.rangelog.
By default, TTL of system.rangelog is set to 30 days.

Fixes cockroachdb#21260

Release note: Add configuration to enable GC of system.rangelog
  • Loading branch information
Vijay Karthik committed Oct 4, 2018
1 parent 8e130b7 commit 2acfad2
Show file tree
Hide file tree
Showing 5 changed files with 355 additions and 0 deletions.
2 changes: 2 additions & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@
<tr><td><code>server.failed_reservation_timeout</code></td><td>duration</td><td><code>5s</code></td><td>the amount of time to consider the store throttled for up-replication after a failed reservation call</td></tr>
<tr><td><code>server.heap_profile.max_profiles</code></td><td>integer</td><td><code>5</code></td><td>maximum number of profiles to be kept. Profiles with lower score are GC'ed, but latest profile is always kept</td></tr>
<tr><td><code>server.heap_profile.system_memory_threshold_fraction</code></td><td>float</td><td><code>0.85</code></td><td>fraction of system memory beyond which if Rss increases, then heap profile is triggered</td></tr>
<tr><td><code>server.rangelog.gc_interval</code></td><td>duration</td><td><code>30m0s</code></td><td>interval for running gc on rangelog. If storage.rangelog.ttl is non zero, range log entries older than server.rangelog.ttl are deleted</td></tr>
<tr><td><code>server.rangelog.ttl</code></td><td>duration</td><td><code>720h0m0s</code></td><td>if non zero, range log entries older than this duration are deleted periodically based on storage.rangelog.gc_interval</td></tr>
<tr><td><code>server.remote_debugging.mode</code></td><td>string</td><td><code>local</code></td><td>set to enable remote debugging, localhost-only or disable (any, local, off)</td></tr>
<tr><td><code>server.shutdown.drain_wait</code></td><td>duration</td><td><code>0s</code></td><td>the amount of time a server waits in an unready state before proceeding with the rest of the shutdown process</td></tr>
<tr><td><code>server.shutdown.query_wait</code></td><td>duration</td><td><code>10s</code></td><td>the server will wait for at least this amount of time for active queries to finish</td></tr>
Expand Down
150 changes: 150 additions & 0 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"google.golang.org/grpc"

"github.com/cockroachdb/cmux"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/internal/client"
Expand Down Expand Up @@ -83,6 +84,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

const (
// defaultRangeLogGCInterval is the default interval to run gc on rangelog.
defaultRangeLogGCInterval = 30 * time.Minute
)

var (
// Allocation pool for gzipResponseWriters.
gzipResponseWriterPool sync.Pool
Expand Down Expand Up @@ -120,6 +126,25 @@ var (
"feature.",
0,
)

// RangeLogTTL is the TTL for rows in system.rangelog. If non zero, range log
// entries are periodically garbage collectd. The period is given by
// server.rangelog.gc_interval
RangeLogTTL = settings.RegisterDurationSetting(
"server.rangelog.ttl",
"if non zero, range log entries older than this duration are deleted periodically "+
"based on storage.rangelog.gc_interval",
30*24*time.Hour, // 30 days
)

// RangeLogGCInterval is the interval between subsequent runs of gc on
// system.rangelog.
RangeLogGCInterval = settings.RegisterDurationSetting(
"server.rangelog.gc_interval",
"interval for running gc on rangelog. If storage.rangelog.ttl is non zero, "+
"range log entries older than server.rangelog.ttl are deleted",
defaultRangeLogGCInterval,
)
)

// TODO(peter): Until go1.11, ServeMux.ServeHTTP was not safe to call
Expand Down Expand Up @@ -1037,6 +1062,129 @@ func (s *Server) startPersistingHLCUpperBound(
)
}

// gcRangeLog deletes entries in system.rangelog older than the given
// cutoffTimestamp if the server is the lease holder for range 1.
// Leaseholder constraint is present so that only one node in the cluster
// performs rangelog gc.
// It returns the node currently responsible for performing GC, the number of
// rows affected and error (if any).
func (s *Server) gcRangeLog(
ctx context.Context, cutoffTimestamp time.Time,
) (roachpb.NodeID, int, error) {
const selectLeaseHolderStmt = `SELECT lease_holder from crdb_internal.ranges where range_id=1`
const deleteStmt = `DELETE FROM system.rangelog WHERE timestamp <= $1 LIMIT 1000`
var totalRowsAffected int
var gcNode roachpb.NodeID
for {
var rowsAffected int
err := s.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
if storeKnobs, ok := s.cfg.TestingKnobs.Store.(*storage.StoreTestingKnobs); ok && storeKnobs.RangelogGCNode != 0 {
gcNode = storeKnobs.RangelogGCNode

} else {
row, err := s.internalExecutor.QueryRow(
ctx,
"leaseholder-r1",
txn,
selectLeaseHolderStmt,
)
if err != nil {
return err
}

if row == nil {
return errors.New("lease holder not found for range 1")
}

leaseHolder, ok := row[0].(*tree.DInt)
if !ok {
return errors.New("lease holder is not an int")
}
if leaseHolder == nil || *leaseHolder <= 0 {
return errors.Errorf("invalid lease holder %v", leaseHolder)
}

gcNode = roachpb.NodeID(*leaseHolder)
}

if gcNode != s.node.Descriptor.NodeID {
return nil
}

var err error
rowsAffected, err = s.internalExecutor.Exec(
ctx,
"rangelog-gc",
txn,
deleteStmt,
cutoffTimestamp,
)
return err
})
totalRowsAffected += rowsAffected
if err != nil || rowsAffected == 0 {
return gcNode, totalRowsAffected, err
}
}
}

// startRangeLogGC starts a worker which periodically GCs system.rangelog.
// The period is controlled by server.rangelog.gc_interval and the TTL is
// controlled by server.rangelog.ttl
func (s *Server) startRangeLogGC(ctx context.Context) {
s.stopper.RunWorker(ctx, func(ctx context.Context) {
intervalChangeCh := make(chan time.Duration)
interval := RangeLogGCInterval.Get(&s.cfg.Settings.SV)
RangeLogGCInterval.SetOnChange(&s.cfg.Settings.SV, func() {
intervalChangeCh <- RangeLogGCInterval.Get(&s.cfg.Settings.SV)
})
t := time.NewTimer(interval)
defer t.Stop()
for {
select {
case interval = <-intervalChangeCh:
if !t.Stop() {
<-t.C
}
t.Reset(interval)
case <-t.C:
ttl := RangeLogTTL.Get(&s.cfg.Settings.SV)
if ttl > 0 {
cutoffTimestamp := timeutil.Unix(0, s.db.Clock().PhysicalNow()-int64(ttl))
gcNode, rowsAffected, err := s.gcRangeLog(ctx, cutoffTimestamp)
if err != nil {
log.Errorf(
ctx,
"error garbage collecting rangelog %v",
err,
)
}

if rowsAffected > 0 || gcNode == s.node.Descriptor.NodeID {
log.Infof(ctx, "garbage collected %d rows from rangelog", rowsAffected)
}

if gcNode != 0 && gcNode != s.node.Descriptor.NodeID {
log.Infof(ctx, "n%d is currently responsible for rangelog gc", gcNode)
}
}

if storeKnobs, ok := s.cfg.TestingKnobs.Store.(*storage.StoreTestingKnobs); ok && storeKnobs.RangelogGCDone != nil {
select {
case storeKnobs.RangelogGCDone <- struct{}{}:
case <-s.stopper.ShouldStop():
// Test has finished
return
}
}
t.Reset(interval)
case <-s.stopper.ShouldStop():
return
}
}
})
}

// Start starts the server on the specified port, starts gossip and initializes
// the node using the engines from the server's context. This is complex since
// it sets up the listeners and the associated port muxing, but especially since
Expand Down Expand Up @@ -1679,6 +1827,8 @@ func (s *Server) Start(ctx context.Context) error {
})
}

s.startRangeLogGC(ctx)

// Record that this node joined the cluster in the event log. Since this
// executes a SQL query, this must be done after the SQL layer is ready.
s.node.recordJoinEvent()
Expand Down
10 changes: 10 additions & 0 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -827,6 +827,16 @@ func (ts *TestServer) ExecutorConfig() interface{} {
return *ts.execCfg
}

// GCRangeLog deletes rows older than the given cutoffTimestamp from
// system.rangelog
// It returns the node currently responsible for performing GC, the number of
// rows affected and error (if any).
func (ts *TestServer) GCRangeLog(
ctx context.Context, cutoffTimestamp time.Time,
) (roachpb.NodeID, int, error) {
return ts.gcRangeLog(ctx, cutoffTimestamp)
}

type testServerFactoryImpl struct{}

// TestServerFactory can be passed to serverutils.InitTestServerFactory
Expand Down
Loading

0 comments on commit 2acfad2

Please sign in to comment.