Skip to content

Commit

Permalink
[tbs] address feedback by @petermattis @bdarnell
Browse files Browse the repository at this point in the history
  • Loading branch information
tbg committed Dec 23, 2015
1 parent 4fc2444 commit de175bd
Show file tree
Hide file tree
Showing 10 changed files with 91 additions and 104 deletions.
2 changes: 1 addition & 1 deletion server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ func BootstrapCluster(clusterID string, engines []engine.Engine, stopper *stop.S
}

// NewNode returns a new instance of Node.
func NewNode(ctx storage.StoreContext, metaRegistry metric.Registry, stopper *stop.Stopper) *Node {
func NewNode(ctx storage.StoreContext, metaRegistry *metric.Registry, stopper *stop.Stopper) *Node {
return &Node{
ctx: ctx,
stopper: stopper,
Expand Down
2 changes: 1 addition & 1 deletion server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ type Server struct {
tsDB *ts.DB
tsServer *ts.Server
raftTransport multiraft.Transport
metaRegistry metric.Registry
metaRegistry *metric.Registry
stopper *stop.Stopper
}

Expand Down
10 changes: 5 additions & 5 deletions server/status.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ const (
// statusStorePattern exposes status for a single store.
statusStorePattern = statusPrefix + "stores/:store_id"

statusTransientPattern = statusPrefix + "transient/:store_id"
statusMetricsPattern = statusPrefix + "metrics/:store_id"

// healthEndpoint is a shortcut for local details, intended for use by
// monitoring processes to verify that the server is up.
Expand All @@ -105,14 +105,14 @@ var localRE = regexp.MustCompile(`(?i)local`)
type statusServer struct {
db *client.DB
gossip *gossip.Gossip
metaRegistry metric.Registry
metaRegistry *metric.Registry
router *httprouter.Router
ctx *Context
proxyClient *http.Client
}

// newStatusServer allocates and returns a statusServer.
func newStatusServer(db *client.DB, gossip *gossip.Gossip, metaRegistry metric.Registry, ctx *Context) *statusServer {
func newStatusServer(db *client.DB, gossip *gossip.Gossip, metaRegistry *metric.Registry, ctx *Context) *statusServer {
// Create an http client with a timeout
tlsConfig, err := ctx.GetClientTLSConfig()
if err != nil {
Expand Down Expand Up @@ -143,7 +143,7 @@ func newStatusServer(db *client.DB, gossip *gossip.Gossip, metaRegistry metric.R
server.router.GET(statusNodePattern, server.handleNodeStatus)
server.router.GET(statusStoresPrefix, server.handleStoresStatus)
server.router.GET(statusStorePattern, server.handleStoreStatus)
server.router.GET(statusTransientPattern, server.handleTransient)
server.router.GET(statusMetricsPattern, server.handleMetrics)

server.router.GET(healthEndpoint, server.handleDetailsLocal)
return server
Expand Down Expand Up @@ -598,7 +598,7 @@ func (s *statusServer) handleStoreStatus(w http.ResponseWriter, r *http.Request,
respondAsJSON(w, r, storeStatus)
}

func (s *statusServer) handleTransient(w http.ResponseWriter, r *http.Request, ps httprouter.Params) {
func (s *statusServer) handleMetrics(w http.ResponseWriter, r *http.Request, ps httprouter.Params) {
nodeID, local, err := s.extractNodeID(ps)
if err != nil {
http.Error(w, err.Error(), http.StatusBadRequest)
Expand Down
4 changes: 2 additions & 2 deletions server/status/feed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,6 @@ import (
"github.com/cockroachdb/cockroach/util/stop"
)

const d = time.Second

func wrap(args roachpb.Request) roachpb.BatchRequest {
var ba roachpb.BatchRequest
ba.Add(args)
Expand Down Expand Up @@ -301,6 +299,8 @@ func TestNodeEventFeedTransactionRestart(t *testing.T) {
ner := nodeEventReader{}
ner.readEvents(feed)

d := 5 * time.Second

get := wrap(&roachpb.GetRequest{})
nodefeed.CallComplete(get, d, &roachpb.Error{
TransactionRestart: roachpb.TransactionRestart_BACKOFF})
Expand Down
58 changes: 29 additions & 29 deletions server/status/monitor.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,30 +36,30 @@ import (
// for passing event feed data to these subset structures for accumulation.
type NodeStatusMonitor struct {
latency metric.Histograms
rateSuccess metric.EWMAS
rateError metric.EWMAS
rateSuccess metric.Rates
rateError metric.Rates
numSuccess *metric.Counter
numError *metric.Counter

closer <-chan struct{}

sync.RWMutex // Mutex to guard the following fields
registry metric.Registry
metaRegistry metric.Registry
registry *metric.Registry
metaRegistry *metric.Registry
stores map[roachpb.StoreID]*StoreStatusMonitor
desc roachpb.NodeDescriptor
startedAt int64
}

// NewNodeStatusMonitor initializes a new NodeStatusMonitor instance.
func NewNodeStatusMonitor(metaRegistry metric.Registry, closer <-chan struct{}) *NodeStatusMonitor {
func NewNodeStatusMonitor(metaRegistry *metric.Registry, closer <-chan struct{}) *NodeStatusMonitor {
registry := metric.NewRegistry(closer)
return &NodeStatusMonitor{
latency: metric.RegisterLatency("latency%s", registry),
rateSuccess: metric.RegisterEWMAS("exec.rate.success%s", registry),
rateError: metric.RegisterEWMAS("exec.rate.error%s", registry),
numSuccess: metric.RegisterCounter("exec.num.success", registry),
numError: metric.RegisterCounter("exec.num.error", registry),
latency: registry.Latency("latency%s"),
rateSuccess: registry.Rates("exec.rate.success%s"),
rateError: registry.Rates("exec.rate.error%s"),
numSuccess: registry.Counter("exec.num.success"),
numError: registry.Counter("exec.num.error"),

registry: registry,
metaRegistry: metaRegistry,
Expand Down Expand Up @@ -260,39 +260,39 @@ type StoreStatusMonitor struct {
available *metric.Gauge

sync.Mutex // Mutex to guard the following fields
registry metric.Registry
registry *metric.Registry
stats engine.MVCCStats
ID roachpb.StoreID
desc *roachpb.StoreDescriptor
startedAt int64
}

// NewStoreStatusMonitor constructs a StoreStatusMonitor with the given ID.
func NewStoreStatusMonitor(id roachpb.StoreID, metaRegistry metric.Registry, closer <-chan struct{}) *StoreStatusMonitor {
func NewStoreStatusMonitor(id roachpb.StoreID, metaRegistry *metric.Registry, closer <-chan struct{}) *StoreStatusMonitor {
registry := metric.NewRegistry(closer)
// Format as `cr.store.<metric>.<id>` in output, in analogy to the time
// series data written.
metaRegistry.Add(storeTimeSeriesPrefix+"%s."+id.String(), registry)
return &StoreStatusMonitor{
ID: id,
registry: registry,
rangeCount: metric.RegisterCounter("ranges", registry),
leaderRangeCount: metric.RegisterGauge("ranges.leader", registry),
replicatedRangeCount: metric.RegisterGauge("ranges.replicated", registry),
availableRangeCount: metric.RegisterGauge("ranges.available", registry),
liveBytes: metric.RegisterGauge("livebytes", registry),
keyBytes: metric.RegisterGauge("keybytes", registry),
valBytes: metric.RegisterGauge("valbytes", registry),
intentBytes: metric.RegisterGauge("intentbytes", registry),
liveCount: metric.RegisterGauge("livecount", registry),
keyCount: metric.RegisterGauge("keycount", registry),
valCount: metric.RegisterGauge("valcount", registry),
intentCount: metric.RegisterGauge("intentcount", registry),
intentAge: metric.RegisterGauge("intentage", registry),
gcBytesAge: metric.RegisterGauge("gcbytesage", registry),
lastUpdateNanos: metric.RegisterGauge("lastupdatenanos", registry),
capacity: metric.RegisterGauge("capacity", registry),
available: metric.RegisterGauge("capacity.available", registry),
rangeCount: registry.Counter("ranges"),
leaderRangeCount: registry.Gauge("ranges.leader"),
replicatedRangeCount: registry.Gauge("ranges.replicated"),
availableRangeCount: registry.Gauge("ranges.available"),
liveBytes: registry.Gauge("livebytes"),
keyBytes: registry.Gauge("keybytes"),
valBytes: registry.Gauge("valbytes"),
intentBytes: registry.Gauge("intentbytes"),
liveCount: registry.Gauge("livecount"),
keyCount: registry.Gauge("keycount"),
valCount: registry.Gauge("valcount"),
intentCount: registry.Gauge("intentcount"),
intentAge: registry.Gauge("intentage"),
gcBytesAge: registry.Gauge("gcbytesage"),
lastUpdateNanos: registry.Gauge("lastupdatenanos"),
capacity: registry.Gauge("capacity"),
available: registry.Gauge("capacity.available"),
}
}

Expand Down
2 changes: 1 addition & 1 deletion server/status/recorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ func (nsr *NodeStatusRecorder) GetStatusSummaries() (*NodeStatus, []storage.Stor
// registryRecorder is a helper class for recording time series datapoints
// from a metrics Registry.
type registryRecorder struct {
registry metric.Registry
registry *metric.Registry
prefix string
source string
timestampNanos int64
Expand Down
6 changes: 2 additions & 4 deletions server/status/recorder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -283,12 +283,10 @@ func TestNodeStatusRecorder(t *testing.T) {
generateNodeData(1, "exec.num.error", 100, 1),
generateNodeData(1, "exec.rate.success1h", 100, 0),
generateNodeData(1, "exec.rate.error1h", 100, 0),
generateNodeData(1, "exec.rate.success30m", 100, 0),
generateNodeData(1, "exec.rate.error30m", 100, 0),
generateNodeData(1, "exec.rate.success10m", 100, 0),
generateNodeData(1, "exec.rate.error10m", 100, 0),
generateNodeData(1, "exec.rate.success1m", 100, 0),
generateNodeData(1, "exec.rate.error1m", 100, 0),
generateNodeData(1, "exec.rate.success5s", 100, 0),
generateNodeData(1, "exec.rate.error5s", 100, 0),
}

actual := recorder.GetTimeSeriesData()
Expand Down
4 changes: 2 additions & 2 deletions sql/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,13 +80,13 @@ type Executor struct {

// newExecutor creates an Executor and registers a callback on the
// system config.
func newExecutor(db client.DB, gossip *gossip.Gossip, leaseMgr *LeaseManager, metaRegistry metric.Registry, stopper *stop.Stopper) *Executor {
func newExecutor(db client.DB, gossip *gossip.Gossip, leaseMgr *LeaseManager, metaRegistry *metric.Registry, stopper *stop.Stopper) *Executor {
exec := &Executor{
db: db,
reCache: parser.NewRegexpCache(512),
leaseMgr: leaseMgr,

latency: metric.RegisterLatency("sql.latency%s", metaRegistry),
latency: metaRegistry.Latency("sql.latency%s"),
}
exec.systemConfigCond = sync.NewCond(&exec.systemConfigMu)

Expand Down
2 changes: 1 addition & 1 deletion sql/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ type Server struct {
}

// MakeServer creates a Server.
func MakeServer(ctx *base.Context, db client.DB, gossip *gossip.Gossip, leaseMgr *LeaseManager, metaRegistry metric.Registry, stopper *stop.Stopper) Server {
func MakeServer(ctx *base.Context, db client.DB, gossip *gossip.Gossip, leaseMgr *LeaseManager, metaRegistry *metric.Registry, stopper *stop.Stopper) Server {
return Server{
context: ctx,
Executor: newExecutor(db, gossip, leaseMgr, metaRegistry, stopper),
Expand Down
Loading

0 comments on commit de175bd

Please sign in to comment.