Skip to content

Commit

Permalink
Metric for max total commit log size (#51)
Browse files Browse the repository at this point in the history
* metric for max commit log size

* remove extra imports

* doc wording
  • Loading branch information
amandachow authored Sep 9, 2019
1 parent 3609293 commit e1cdd0e
Showing 1 changed file with 10 additions and 0 deletions.
10 changes: 10 additions & 0 deletions src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import com.codahale.metrics.Gauge;
import com.codahale.metrics.Timer;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.commitlog.AbstractCommitLogService;
import org.apache.cassandra.db.commitlog.CommitLogSegmentManager;

Expand All @@ -38,6 +39,8 @@ public class CommitLogMetrics
public Gauge<Long> pendingTasks;
/** Current size used by all the commit log segments */
public Gauge<Long> totalCommitLogSize;
/** Max total size for all commit log segments, as set in cassandra.yaml */
public Gauge<Long> maxTotalCommitLogSizeBytes;
/** Time spent waiting for a CLS to be allocated - under normal conditions this should be zero */
public final Timer waitingOnSegmentAllocation;
/** The time spent waiting on CL sync; for Periodic this is only occurs when the sync is lagging its sync interval */
Expand Down Expand Up @@ -72,5 +75,12 @@ public Long getValue()
return allocator.onDiskSize();
}
});
maxTotalCommitLogSizeBytes = Metrics.register(factory.createMetricName("MaxTotalCommitLogSizeBytes"), new Gauge<Long>()
{
public Long getValue()
{
return DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 1024;
}
});
}
}

0 comments on commit e1cdd0e

Please sign in to comment.