You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2015/01/28 17:15:46 UTC

[3/4] cassandra git commit: Upgrade metrics library and remove depricated metrics

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java
new file mode 100644
index 0000000..27eec89
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java
@@ -0,0 +1,791 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.metrics;
+
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Method;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.codahale.metrics.*;
+import javax.management.*;
+
+/**
+ * Makes integrating 3.0 metrics API with 2.0.
+ * <p/>
+ * The 3.0 API comes with poor JMX integration
+ */
+public class CassandraMetricsRegistry extends MetricRegistry
+{
+    protected static final Logger logger = LoggerFactory.getLogger(CassandraMetricsRegistry.class);
+
+    public static final CassandraMetricsRegistry Metrics = new CassandraMetricsRegistry();
+    private MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer();
+
+    private CassandraMetricsRegistry()
+    {
+        super();
+    }
+
+    public Counter counter(MetricName name)
+    {
+        Counter counter = counter(name.getMetricName());
+        registerMBean(counter, name.getMBeanName());
+
+        return counter;
+    }
+
+    public Meter meter(MetricName name)
+    {
+        Meter meter = meter(name.getMetricName());
+        registerMBean(meter, name.getMBeanName());
+
+        return meter;
+    }
+
+    public Histogram histogram(MetricName name)
+    {
+        Histogram histogram = register(name, new ClearableHistogram(new EstimatedHistogramReservoir()));
+        registerMBean(histogram, name.getMBeanName());
+
+        return histogram;
+    }
+
+    public Timer timer(MetricName name)
+    {
+        Timer timer = register(name, new Timer(new EstimatedHistogramReservoir()));
+        registerMBean(timer, name.getMBeanName());
+
+        return timer;
+    }
+
+    public <T extends Metric> T register(MetricName name, T metric)
+    {
+        try
+        {
+            register(name.getMetricName(), metric);
+            registerMBean(metric, name.getMBeanName());
+            return metric;
+        }
+        catch (IllegalArgumentException e)
+        {
+            Metric existing = Metrics.getMetrics().get(name.getMetricName());
+            return (T)existing;
+        }
+    }
+
+    public boolean remove(MetricName name)
+    {
+        boolean removed = remove(name.getMetricName());
+
+        try
+        {
+            mBeanServer.unregisterMBean(name.getMBeanName());
+        } catch (InstanceNotFoundException | MBeanRegistrationException e)
+        {
+            logger.debug("Unable to remove mbean");
+        }
+
+        return removed;
+    }
+
+    private void registerMBean(Metric metric, ObjectName name)
+    {
+        AbstractBean mbean;
+
+        if (metric instanceof Gauge)
+        {
+            mbean = new JmxGauge((Gauge<?>) metric, name);
+        } else if (metric instanceof Counter)
+        {
+            mbean = new JmxCounter((Counter) metric, name);
+        } else if (metric instanceof Histogram)
+        {
+            mbean = new JmxHistogram((Histogram) metric, name);
+        } else if (metric instanceof Meter)
+        {
+            mbean = new JmxMeter((Meter) metric, name, TimeUnit.SECONDS);
+        } else if (metric instanceof Timer)
+        {
+            mbean = new JmxTimer((Timer) metric, name, TimeUnit.SECONDS, TimeUnit.MICROSECONDS);
+        } else
+        {
+            throw new IllegalArgumentException("Unknown metric type: " + metric.getClass());
+        }
+
+        try
+        {
+            mBeanServer.registerMBean(mbean, name);
+        } catch (InstanceAlreadyExistsException e)
+        {
+            logger.debug("Metric bean already exists", e);
+        } catch (MBeanRegistrationException e)
+        {
+            logger.debug("Unable to register metric bean", e);
+        } catch (NotCompliantMBeanException e)
+        {
+            logger.warn("Unable to register metric bean", e);
+        }
+    }
+
+    public interface MetricMBean
+    {
+        ObjectName objectName();
+    }
+
+    private abstract static class AbstractBean implements MetricMBean
+    {
+        private final ObjectName objectName;
+
+        AbstractBean(ObjectName objectName)
+        {
+            this.objectName = objectName;
+        }
+
+        @Override
+        public ObjectName objectName()
+        {
+            return objectName;
+        }
+    }
+
+
+    public interface JmxGaugeMBean extends MetricMBean
+    {
+        Object getValue();
+    }
+
+    private static class JmxGauge extends AbstractBean implements JmxGaugeMBean
+    {
+        private final Gauge<?> metric;
+
+        private JmxGauge(Gauge<?> metric, ObjectName objectName)
+        {
+            super(objectName);
+            this.metric = metric;
+        }
+
+        @Override
+        public Object getValue()
+        {
+            return metric.getValue();
+        }
+    }
+
+    public interface JmxHistogramMBean extends MetricMBean
+    {
+        long getCount();
+
+        long getMin();
+
+        long getMax();
+
+        double getMean();
+
+        double getStdDev();
+
+        double get50thPercentile();
+
+        double get75thPercentile();
+
+        double get95thPercentile();
+
+        double get98thPercentile();
+
+        double get99thPercentile();
+
+        double get999thPercentile();
+
+        long[] values();
+    }
+
+    private static class JmxHistogram extends AbstractBean implements JmxHistogramMBean
+    {
+        private final Histogram metric;
+
+        private JmxHistogram(Histogram metric, ObjectName objectName)
+        {
+            super(objectName);
+            this.metric = metric;
+        }
+
+        @Override
+        public double get50thPercentile()
+        {
+            return metric.getSnapshot().getMedian();
+        }
+
+        @Override
+        public long getCount()
+        {
+            return metric.getCount();
+        }
+
+        @Override
+        public long getMin()
+        {
+            return metric.getSnapshot().getMin();
+        }
+
+        @Override
+        public long getMax()
+        {
+            return metric.getSnapshot().getMax();
+        }
+
+        @Override
+        public double getMean()
+        {
+            return metric.getSnapshot().getMean();
+        }
+
+        @Override
+        public double getStdDev()
+        {
+            return metric.getSnapshot().getStdDev();
+        }
+
+        @Override
+        public double get75thPercentile()
+        {
+            return metric.getSnapshot().get75thPercentile();
+        }
+
+        @Override
+        public double get95thPercentile()
+        {
+            return metric.getSnapshot().get95thPercentile();
+        }
+
+        @Override
+        public double get98thPercentile()
+        {
+            return metric.getSnapshot().get98thPercentile();
+        }
+
+        @Override
+        public double get99thPercentile()
+        {
+            return metric.getSnapshot().get99thPercentile();
+        }
+
+        @Override
+        public double get999thPercentile()
+        {
+            return metric.getSnapshot().get999thPercentile();
+        }
+
+        @Override
+        public long[] values()
+        {
+            return metric.getSnapshot().getValues();
+        }
+    }
+
+    public interface JmxCounterMBean extends MetricMBean
+    {
+        long getCount();
+    }
+
+    private static class JmxCounter extends AbstractBean implements JmxCounterMBean
+    {
+        private final Counter metric;
+
+        private JmxCounter(Counter metric, ObjectName objectName)
+        {
+            super(objectName);
+            this.metric = metric;
+        }
+
+        @Override
+        public long getCount()
+        {
+            return metric.getCount();
+        }
+    }
+
+    public interface JmxMeterMBean extends MetricMBean
+    {
+        long getCount();
+
+        double getMeanRate();
+
+        double getOneMinuteRate();
+
+        double getFiveMinuteRate();
+
+        double getFifteenMinuteRate();
+
+        String getRateUnit();
+    }
+
+    private static class JmxMeter extends AbstractBean implements JmxMeterMBean
+    {
+        private final Metered metric;
+        private final double rateFactor;
+        private final String rateUnit;
+
+        private JmxMeter(Metered metric, ObjectName objectName, TimeUnit rateUnit)
+        {
+            super(objectName);
+            this.metric = metric;
+            this.rateFactor = rateUnit.toSeconds(1);
+            this.rateUnit = "events/" + calculateRateUnit(rateUnit);
+        }
+
+        @Override
+        public long getCount()
+        {
+            return metric.getCount();
+        }
+
+        @Override
+        public double getMeanRate()
+        {
+            return metric.getMeanRate() * rateFactor;
+        }
+
+        @Override
+        public double getOneMinuteRate()
+        {
+            return metric.getOneMinuteRate() * rateFactor;
+        }
+
+        @Override
+        public double getFiveMinuteRate()
+        {
+            return metric.getFiveMinuteRate() * rateFactor;
+        }
+
+        @Override
+        public double getFifteenMinuteRate()
+        {
+            return metric.getFifteenMinuteRate() * rateFactor;
+        }
+
+        @Override
+        public String getRateUnit()
+        {
+            return rateUnit;
+        }
+
+        private String calculateRateUnit(TimeUnit unit)
+        {
+            final String s = unit.toString().toLowerCase(Locale.US);
+            return s.substring(0, s.length() - 1);
+        }
+    }
+
+    public interface JmxTimerMBean extends JmxMeterMBean
+    {
+        double getMin();
+
+        double getMax();
+
+        double getMean();
+
+        double getStdDev();
+
+        double get50thPercentile();
+
+        double get75thPercentile();
+
+        double get95thPercentile();
+
+        double get98thPercentile();
+
+        double get99thPercentile();
+
+        double get999thPercentile();
+
+        long[] values();
+
+        String getDurationUnit();
+    }
+
+    static class JmxTimer extends JmxMeter implements JmxTimerMBean
+    {
+        private final Timer metric;
+        private final double durationFactor;
+        private final String durationUnit;
+
+        private JmxTimer(Timer metric,
+                         ObjectName objectName,
+                         TimeUnit rateUnit,
+                         TimeUnit durationUnit)
+        {
+            super(metric, objectName, rateUnit);
+            this.metric = metric;
+            this.durationFactor = 1.0 / durationUnit.toNanos(1);
+            this.durationUnit = durationUnit.toString().toLowerCase(Locale.US);
+        }
+
+        @Override
+        public double get50thPercentile()
+        {
+            return metric.getSnapshot().getMedian() * durationFactor;
+        }
+
+        @Override
+        public double getMin()
+        {
+            return metric.getSnapshot().getMin() * durationFactor;
+        }
+
+        @Override
+        public double getMax()
+        {
+            return metric.getSnapshot().getMax() * durationFactor;
+        }
+
+        @Override
+        public double getMean()
+        {
+            return metric.getSnapshot().getMean() * durationFactor;
+        }
+
+        @Override
+        public double getStdDev()
+        {
+            return metric.getSnapshot().getStdDev() * durationFactor;
+        }
+
+        @Override
+        public double get75thPercentile()
+        {
+            return metric.getSnapshot().get75thPercentile() * durationFactor;
+        }
+
+        @Override
+        public double get95thPercentile()
+        {
+            return metric.getSnapshot().get95thPercentile() * durationFactor;
+        }
+
+        @Override
+        public double get98thPercentile()
+        {
+            return metric.getSnapshot().get98thPercentile() * durationFactor;
+        }
+
+        @Override
+        public double get99thPercentile()
+        {
+            return metric.getSnapshot().get99thPercentile() * durationFactor;
+        }
+
+        @Override
+        public double get999thPercentile()
+        {
+            return metric.getSnapshot().get999thPercentile() * durationFactor;
+        }
+
+        @Override
+        public long[] values()
+        {
+            return metric.getSnapshot().getValues();
+        }
+
+        @Override
+        public String getDurationUnit()
+        {
+            return durationUnit;
+        }
+    }
+
+    /**
+     * A value class encapsulating a metric's owning class and name.
+     */
+    public static class MetricName implements Comparable<MetricName>
+    {
+        private final String group;
+        private final String type;
+        private final String name;
+        private final String scope;
+        private final String mBeanName;
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param klass the {@link Class} to which the {@link Metric} belongs
+         * @param name  the name of the {@link Metric}
+         */
+        public MetricName(Class<?> klass, String name)
+        {
+            this(klass, name, null);
+        }
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param group the group to which the {@link Metric} belongs
+         * @param type  the type to which the {@link Metric} belongs
+         * @param name  the name of the {@link Metric}
+         */
+        public MetricName(String group, String type, String name)
+        {
+            this(group, type, name, null);
+        }
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param klass the {@link Class} to which the {@link Metric} belongs
+         * @param name  the name of the {@link Metric}
+         * @param scope the scope of the {@link Metric}
+         */
+        public MetricName(Class<?> klass, String name, String scope)
+        {
+            this(klass.getPackage() == null ? "" : klass.getPackage().getName(),
+                    klass.getSimpleName().replaceAll("\\$$", ""),
+                    name,
+                    scope);
+        }
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param group the group to which the {@link Metric} belongs
+         * @param type  the type to which the {@link Metric} belongs
+         * @param name  the name of the {@link Metric}
+         * @param scope the scope of the {@link Metric}
+         */
+        public MetricName(String group, String type, String name, String scope)
+        {
+            this(group, type, name, scope, createMBeanName(group, type, name, scope));
+        }
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param group     the group to which the {@link Metric} belongs
+         * @param type      the type to which the {@link Metric} belongs
+         * @param name      the name of the {@link Metric}
+         * @param scope     the scope of the {@link Metric}
+         * @param mBeanName the 'ObjectName', represented as a string, to use when registering the
+         *                  MBean.
+         */
+        public MetricName(String group, String type, String name, String scope, String mBeanName)
+        {
+            if (group == null || type == null)
+            {
+                throw new IllegalArgumentException("Both group and type need to be specified");
+            }
+            if (name == null)
+            {
+                throw new IllegalArgumentException("Name needs to be specified");
+            }
+            this.group = group;
+            this.type = type;
+            this.name = name;
+            this.scope = scope;
+            this.mBeanName = mBeanName;
+        }
+
+        /**
+         * Returns the group to which the {@link Metric} belongs. For class-based metrics, this will be
+         * the package name of the {@link Class} to which the {@link Metric} belongs.
+         *
+         * @return the group to which the {@link Metric} belongs
+         */
+        public String getGroup()
+        {
+            return group;
+        }
+
+        /**
+         * Returns the type to which the {@link Metric} belongs. For class-based metrics, this will be
+         * the simple class name of the {@link Class} to which the {@link Metric} belongs.
+         *
+         * @return the type to which the {@link Metric} belongs
+         */
+        public String getType()
+        {
+            return type;
+        }
+
+        /**
+         * Returns the name of the {@link Metric}.
+         *
+         * @return the name of the {@link Metric}
+         */
+        public String getName()
+        {
+            return name;
+        }
+
+        public String getMetricName()
+        {
+            return MetricRegistry.name(group, type, name, scope);
+        }
+
+        /**
+         * Returns the scope of the {@link Metric}.
+         *
+         * @return the scope of the {@link Metric}
+         */
+        public String getScope()
+        {
+            return scope;
+        }
+
+        /**
+         * Returns {@code true} if the {@link Metric} has a scope, {@code false} otherwise.
+         *
+         * @return {@code true} if the {@link Metric} has a scope
+         */
+        public boolean hasScope()
+        {
+            return scope != null;
+        }
+
+        /**
+         * Returns the MBean name for the {@link Metric} identified by this metric name.
+         *
+         * @return the MBean name
+         */
+        public ObjectName getMBeanName()
+        {
+
+            String mname = mBeanName;
+
+            if (mname == null)
+                mname = getMetricName();
+
+            try
+            {
+
+                return new ObjectName(mname);
+            } catch (MalformedObjectNameException e)
+            {
+                try
+                {
+                    return new ObjectName(ObjectName.quote(mname));
+                } catch (MalformedObjectNameException e1)
+                {
+                    throw new RuntimeException(e1);
+                }
+            }
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o)
+            {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass())
+            {
+                return false;
+            }
+            final MetricName that = (MetricName) o;
+            return mBeanName.equals(that.mBeanName);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return mBeanName.hashCode();
+        }
+
+        @Override
+        public String toString()
+        {
+            return mBeanName;
+        }
+
+        @Override
+        public int compareTo(MetricName o)
+        {
+            return mBeanName.compareTo(o.mBeanName);
+        }
+
+        private static String createMBeanName(String group, String type, String name, String scope)
+        {
+            final StringBuilder nameBuilder = new StringBuilder();
+            nameBuilder.append(ObjectName.quote(group));
+            nameBuilder.append(":type=");
+            nameBuilder.append(ObjectName.quote(type));
+            if (scope != null)
+            {
+                nameBuilder.append(",scope=");
+                nameBuilder.append(ObjectName.quote(scope));
+            }
+            if (name.length() > 0)
+            {
+                nameBuilder.append(",name=");
+                nameBuilder.append(ObjectName.quote(name));
+            }
+            return nameBuilder.toString();
+        }
+
+        /**
+         * If the group is empty, use the package name of the given class. Otherwise use group
+         *
+         * @param group The group to use by default
+         * @param klass The class being tracked
+         * @return a group for the metric
+         */
+        public static String chooseGroup(String group, Class<?> klass)
+        {
+            if (group == null || group.isEmpty())
+            {
+                group = klass.getPackage() == null ? "" : klass.getPackage().getName();
+            }
+            return group;
+        }
+
+        /**
+         * If the type is empty, use the simple name of the given class. Otherwise use type
+         *
+         * @param type  The type to use by default
+         * @param klass The class being tracked
+         * @return a type for the metric
+         */
+        public static String chooseType(String type, Class<?> klass)
+        {
+            if (type == null || type.isEmpty())
+            {
+                type = klass.getSimpleName().replaceAll("\\$$", "");
+            }
+            return type;
+        }
+
+        /**
+         * If name is empty, use the name of the given method. Otherwise use name
+         *
+         * @param name   The name to use by default
+         * @param method The method being tracked
+         * @return a name for the metric
+         */
+        public static String chooseName(String name, Method method)
+        {
+            if (name == null || name.isEmpty())
+            {
+                name = method.getName();
+            }
+            return name;
+        }
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/ClearableHistogram.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ClearableHistogram.java b/src/java/org/apache/cassandra/metrics/ClearableHistogram.java
new file mode 100644
index 0000000..85f2fa9
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/ClearableHistogram.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.metrics;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import com.codahale.metrics.Histogram;
+
+/**
+ * Adds ability to reset a histogram
+ */
+public class ClearableHistogram extends Histogram
+{
+    private final EstimatedHistogramReservoir reservoirRef;
+
+    /**
+     * Creates a new {@link com.codahale.metrics.Histogram} with the given reservoir.
+     *
+     * @param reservoir the reservoir to create a histogram from
+     */
+    public ClearableHistogram(EstimatedHistogramReservoir reservoir)
+    {
+        super(reservoir);
+
+        this.reservoirRef = reservoir;
+    }
+
+    @VisibleForTesting
+    public void clear()
+    {
+        reservoirRef.clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ClientMetrics.java b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
index 59bc94d..4a384eb 100644
--- a/src/java/org/apache/cassandra/metrics/ClientMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
@@ -20,8 +20,10 @@ package org.apache.cassandra.metrics;
 
 import java.util.concurrent.Callable;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
+import com.codahale.metrics.Gauge;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 public class ClientMetrics
 {
@@ -35,15 +37,14 @@ public class ClientMetrics
 
     public void addCounter(String name, final Callable<Integer> provider)
     {
-        Metrics.newGauge(factory.createMetricName(name), new Gauge<Integer>()
+        Metrics.register(factory.createMetricName(name), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 try
                 {
                     return provider.call();
-                }
-                catch (Exception e)
+                } catch (Exception e)
                 {
                     throw new RuntimeException(e);
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java b/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java
index 68a2d21..e3a6970 100644
--- a/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java
@@ -20,20 +20,14 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.concurrent.TimeUnit;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
-import com.yammer.metrics.core.Meter;
+import com.codahale.metrics.Meter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 public class ClientRequestMetrics extends LatencyMetrics
 {
-    @Deprecated public static final Counter readTimeouts = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "ReadTimeouts", null));
-    @Deprecated public static final Counter writeTimeouts = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "WriteTimeouts", null));
-    @Deprecated public static final Counter readUnavailables = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "ReadUnavailables", null));
-    @Deprecated public static final Counter writeUnavailables = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "WriteUnavailables", null));
-    @Deprecated public static final Counter readFailures = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "ReadFailures", null));
-
     public final Meter timeouts;
     public final Meter unavailables;
     public final Meter failures;
@@ -42,16 +36,16 @@ public class ClientRequestMetrics extends LatencyMetrics
     {
         super("ClientRequest", scope);
 
-        timeouts = Metrics.newMeter(factory.createMetricName("Timeouts"), "timeouts", TimeUnit.SECONDS);
-        unavailables = Metrics.newMeter(factory.createMetricName("Unavailables"), "unavailables", TimeUnit.SECONDS);
-        failures = Metrics.newMeter(factory.createMetricName("Failures"), "failures", TimeUnit.SECONDS);
+        timeouts = Metrics.meter(factory.createMetricName("Timeouts"));
+        unavailables = Metrics.meter(factory.createMetricName("Unavailables"));
+        failures = Metrics.meter(factory.createMetricName("Failures"));
     }
 
     public void release()
     {
         super.release();
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("Timeouts"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("Unavailables"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("Failures"));
+        Metrics.remove(factory.createMetricName("Timeouts"));
+        Metrics.remove(factory.createMetricName("Unavailables"));
+        Metrics.remove(factory.createMetricName("Failures"));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
index 07246cf..90b6b80 100644
--- a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
@@ -20,8 +20,9 @@ package org.apache.cassandra.metrics;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
 
+import com.codahale.metrics.*;
+import com.codahale.metrics.Timer;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -31,16 +32,16 @@ import org.apache.cassandra.utils.TopKSampler;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.*;
-import com.yammer.metrics.core.Timer;
-import com.yammer.metrics.util.RatioGauge;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 /**
  * Metrics for {@link ColumnFamilyStore}.
  */
 public class ColumnFamilyMetrics
 {
+
     /** Total amount of data stored in the memtable that resides on-heap, including column related overhead and overwritten rows. */
     public final Gauge<Long> memtableOnHeapSize;
     /** Total amount of data stored in the memtable that resides off-heap, including column related overhead and overwritten rows. */
@@ -130,17 +131,13 @@ public class ColumnFamilyMetrics
     public final Timer coordinatorScanLatency;
 
     /** Time spent waiting for free memtable space, either on- or off-heap */
-    public final Timer waitingOnFreeMemtableSpace;
+    public final Histogram waitingOnFreeMemtableSpace;
 
     private final MetricNameFactory factory;
-    private static final MetricNameFactory globalNameFactory = new AllColumnFamilyMetricNameFactory();;
+    private static final MetricNameFactory globalNameFactory = new AllColumnFamilyMetricNameFactory();
 
     public final Counter speculativeRetries;
 
-    // for backward compatibility
-    @Deprecated public final EstimatedHistogram sstablesPerRead = new EstimatedHistogram(35);
-    @Deprecated public final EstimatedHistogram recentSSTablesPerRead = new EstimatedHistogram(35);
-    
     public final static LatencyMetrics globalReadLatency = new LatencyMetrics(globalNameFactory, "Read");
     public final static LatencyMetrics globalWriteLatency = new LatencyMetrics(globalNameFactory, "Write");
     public final static LatencyMetrics globalRangeLatency = new LatencyMetrics(globalNameFactory, "Range");
@@ -213,35 +210,35 @@ public class ColumnFamilyMetrics
 
         memtableColumnsCount = createColumnFamilyGauge("MemtableColumnsCount", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return cfs.getDataTracker().getView().getCurrentMemtable().getOperations();
             }
         });
         memtableOnHeapSize = createColumnFamilyGauge("MemtableOnHeapSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
             }
         });
         memtableOffHeapSize = createColumnFamilyGauge("MemtableOffHeapSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
             }
         });
         memtableLiveDataSize = createColumnFamilyGauge("MemtableLiveDataSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return cfs.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
             }
         });
         allMemtablesOnHeapSize = createColumnFamilyGauge("AllMemtablesHeapSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
@@ -251,7 +248,7 @@ public class ColumnFamilyMetrics
         });
         allMemtablesOffHeapSize = createColumnFamilyGauge("AllMemtablesOffHeapSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
@@ -261,7 +258,7 @@ public class ColumnFamilyMetrics
         });
         allMemtablesLiveDataSize = createColumnFamilyGauge("AllMemtablesLiveDataSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
@@ -270,9 +267,9 @@ public class ColumnFamilyMetrics
             }
         });
         memtableSwitchCount = createColumnFamilyCounter("MemtableSwitchCount");
-        estimatedRowSizeHistogram = Metrics.newGauge(factory.createMetricName("EstimatedRowSizeHistogram"), new Gauge<long[]>()
+        estimatedRowSizeHistogram = Metrics.register(factory.createMetricName("EstimatedRowSizeHistogram"), new Gauge<long[]>()
         {
-            public long[] value()
+            public long[] getValue()
             {
                 return combineHistograms(cfs.getSSTables(), new GetHistogram()
                 {
@@ -283,9 +280,9 @@ public class ColumnFamilyMetrics
                 });
             }
         });
-        estimatedColumnCountHistogram = Metrics.newGauge(factory.createMetricName("EstimatedColumnCountHistogram"), new Gauge<long[]>()
+        estimatedColumnCountHistogram = Metrics.register(factory.createMetricName("EstimatedColumnCountHistogram"), new Gauge<long[]>()
         {
-            public long[] value()
+            public long[] getValue()
             {
                 return combineHistograms(cfs.getSSTables(), new GetHistogram()
                 {
@@ -299,7 +296,7 @@ public class ColumnFamilyMetrics
         sstablesPerReadHistogram = createColumnFamilyHistogram("SSTablesPerReadHistogram", cfs.keyspace.metric.sstablesPerReadHistogram);
         compressionRatio = createColumnFamilyGauge("CompressionRatio", new Gauge<Double>()
         {
-            public Double value()
+            public Double getValue()
             {
                 double sum = 0;
                 int total = 0;
@@ -315,7 +312,7 @@ public class ColumnFamilyMetrics
             }
         }, new Gauge<Double>() // global gauge
         {
-            public Double value()
+            public Double getValue()
             {
                 double sum = 0;
                 int total = 0;
@@ -339,14 +336,14 @@ public class ColumnFamilyMetrics
         pendingFlushes = createColumnFamilyCounter("PendingFlushes");
         pendingCompactions = createColumnFamilyGauge("PendingCompactions", new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return cfs.getCompactionStrategy().getEstimatedRemainingTasks();
             }
         });
         liveSSTableCount = createColumnFamilyGauge("LiveSSTableCount", new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return cfs.getDataTracker().getSSTables().size();
             }
@@ -355,7 +352,7 @@ public class ColumnFamilyMetrics
         totalDiskSpaceUsed = createColumnFamilyCounter("TotalDiskSpaceUsed");
         minRowSize = createColumnFamilyGauge("MinRowSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long min = 0;
                 for (SSTableReader sstable : cfs.getSSTables())
@@ -367,19 +364,19 @@ public class ColumnFamilyMetrics
             }
         }, new Gauge<Long>() // global gauge
         {
-            public Long value()
+            public Long getValue()
             {
                 long min = Long.MAX_VALUE;
                 for (Metric cfGauge : allColumnFamilyMetrics.get("MinRowSize"))
                 {
-                    min = Math.min(min, ((Gauge<? extends Number>) cfGauge).value().longValue());
+                    min = Math.min(min, ((Gauge<? extends Number>) cfGauge).getValue().longValue());
                 }
                 return min;
             }
         });
         maxRowSize = createColumnFamilyGauge("MaxRowSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long max = 0;
                 for (SSTableReader sstable : cfs.getSSTables())
@@ -391,19 +388,19 @@ public class ColumnFamilyMetrics
             }
         }, new Gauge<Long>() // global gauge
         {
-            public Long value()
+            public Long getValue()
             {
                 long max = 0;
                 for (Metric cfGauge : allColumnFamilyMetrics.get("MaxRowSize"))
                 {
-                    max = Math.max(max, ((Gauge<? extends Number>) cfGauge).value().longValue());
+                    max = Math.max(max, ((Gauge<? extends Number>) cfGauge).getValue().longValue());
                 }
                 return max;
             }
         });
         meanRowSize = createColumnFamilyGauge("MeanRowSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long sum = 0;
                 long count = 0;
@@ -417,7 +414,7 @@ public class ColumnFamilyMetrics
             }
         }, new Gauge<Long>() // global gauge
         {
-            public Long value()
+            public Long getValue()
             {
                 long sum = 0;
                 long count = 0;
@@ -435,7 +432,7 @@ public class ColumnFamilyMetrics
         });
         bloomFilterFalsePositives = createColumnFamilyGauge("BloomFilterFalsePositives", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long count = 0L;
                 for (SSTableReader sstable: cfs.getSSTables())
@@ -445,7 +442,7 @@ public class ColumnFamilyMetrics
         });
         recentBloomFilterFalsePositives = createColumnFamilyGauge("RecentBloomFilterFalsePositives", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long count = 0L;
                 for (SSTableReader sstable : cfs.getSSTables())
@@ -455,7 +452,7 @@ public class ColumnFamilyMetrics
         });
         bloomFilterFalseRatio = createColumnFamilyGauge("BloomFilterFalseRatio", new Gauge<Double>()
         {
-            public Double value()
+            public Double getValue()
             {
                 long falseCount = 0L;
                 long trueCount = 0L;
@@ -470,7 +467,7 @@ public class ColumnFamilyMetrics
             }
         }, new Gauge<Double>() // global gauge
         {
-            public Double value()
+            public Double getValue()
             {
                 long falseCount = 0L;
                 long trueCount = 0L;
@@ -489,7 +486,7 @@ public class ColumnFamilyMetrics
         });
         recentBloomFilterFalseRatio = createColumnFamilyGauge("RecentBloomFilterFalseRatio", new Gauge<Double>()
         {
-            public Double value()
+            public Double getValue()
             {
                 long falseCount = 0L;
                 long trueCount = 0L;
@@ -504,7 +501,7 @@ public class ColumnFamilyMetrics
             }
         }, new Gauge<Double>() // global gauge
         {
-            public Double value()
+            public Double getValue()
             {
                 long falseCount = 0L;
                 long trueCount = 0L;
@@ -523,7 +520,7 @@ public class ColumnFamilyMetrics
         });
         bloomFilterDiskSpaceUsed = createColumnFamilyGauge("BloomFilterDiskSpaceUsed", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (SSTableReader sst : cfs.getSSTables())
@@ -533,7 +530,7 @@ public class ColumnFamilyMetrics
         });
         bloomFilterOffHeapMemoryUsed = createColumnFamilyGauge("BloomFilterOffHeapMemoryUsed", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (SSTableReader sst : cfs.getSSTables())
@@ -543,7 +540,7 @@ public class ColumnFamilyMetrics
         });
         indexSummaryOffHeapMemoryUsed = createColumnFamilyGauge("IndexSummaryOffHeapMemoryUsed", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (SSTableReader sst : cfs.getSSTables())
@@ -553,7 +550,7 @@ public class ColumnFamilyMetrics
         });
         compressionMetadataOffHeapMemoryUsed = createColumnFamilyGauge("CompressionMetadataOffHeapMemoryUsed", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (SSTableReader sst : cfs.getSSTables())
@@ -562,8 +559,14 @@ public class ColumnFamilyMetrics
             }
         });
         speculativeRetries = createColumnFamilyCounter("SpeculativeRetries");
-        keyCacheHitRate = Metrics.newGauge(factory.createMetricName("KeyCacheHitRate"), new RatioGauge()
+        keyCacheHitRate = Metrics.register(factory.createMetricName("KeyCacheHitRate"), new RatioGauge()
         {
+            @Override
+            public Ratio getRatio()
+            {
+                return Ratio.of(getNumerator(), getDenominator());
+            }
+
             protected double getNumerator()
             {
                 long hits = 0L;
@@ -583,13 +586,13 @@ public class ColumnFamilyMetrics
         tombstoneScannedHistogram = createColumnFamilyHistogram("TombstoneScannedHistogram", cfs.keyspace.metric.tombstoneScannedHistogram);
         liveScannedHistogram = createColumnFamilyHistogram("LiveScannedHistogram", cfs.keyspace.metric.liveScannedHistogram);
         colUpdateTimeDeltaHistogram = createColumnFamilyHistogram("ColUpdateTimeDeltaHistogram", cfs.keyspace.metric.colUpdateTimeDeltaHistogram);
-        coordinatorReadLatency = Metrics.newTimer(factory.createMetricName("CoordinatorReadLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
-        coordinatorScanLatency = Metrics.newTimer(factory.createMetricName("CoordinatorScanLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
-        waitingOnFreeMemtableSpace = Metrics.newTimer(factory.createMetricName("WaitingOnFreeMemtableSpace"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
+        coordinatorReadLatency = Metrics.timer(factory.createMetricName("CoordinatorReadLatency"));
+        coordinatorScanLatency = Metrics.timer(factory.createMetricName("CoordinatorScanLatency"));
+        waitingOnFreeMemtableSpace = Metrics.histogram(factory.createMetricName("WaitingOnFreeMemtableSpace"));
 
         trueSnapshotsSize = createColumnFamilyGauge("SnapshotsSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return cfs.trueSnapshotsSize();
             }
@@ -606,8 +609,6 @@ public class ColumnFamilyMetrics
     public void updateSSTableIterated(int count)
     {
         sstablesPerReadHistogram.update(count);
-        recentSSTablesPerRead.add(count);
-        sstablesPerRead.add(count);
     }
 
     /**
@@ -617,18 +618,18 @@ public class ColumnFamilyMetrics
     {
         for(String name : all)
         {
-            allColumnFamilyMetrics.get(name).remove(Metrics.defaultRegistry().allMetrics().get(factory.createMetricName(name)));
-            Metrics.defaultRegistry().removeMetric(factory.createMetricName(name));
+            allColumnFamilyMetrics.get(name).remove(Metrics.getMetrics().get(factory.createMetricName(name)));
+            Metrics.remove(factory.createMetricName(name));
         }
         readLatency.release();
         writeLatency.release();
         rangeLatency.release();
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("EstimatedRowSizeHistogram"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("EstimatedColumnCountHistogram"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("KeyCacheHitRate"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CoordinatorReadLatency"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CoordinatorScanLatency"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("WaitingOnFreeMemtableSpace"));
+        Metrics.remove(factory.createMetricName("EstimatedRowSizeHistogram"));
+        Metrics.remove(factory.createMetricName("EstimatedColumnCountHistogram"));
+        Metrics.remove(factory.createMetricName("KeyCacheHitRate"));
+        Metrics.remove(factory.createMetricName("CoordinatorReadLatency"));
+        Metrics.remove(factory.createMetricName("CoordinatorScanLatency"));
+        Metrics.remove(factory.createMetricName("WaitingOnFreeMemtableSpace"));
     }
 
 
@@ -640,12 +641,12 @@ public class ColumnFamilyMetrics
     {
         return createColumnFamilyGauge(name, gauge, new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (Metric cfGauge : allColumnFamilyMetrics.get(name))
                 {
-                    total = total + ((Gauge<? extends Number>) cfGauge).value().longValue();
+                    total = total + ((Gauge<? extends Number>) cfGauge).getValue().longValue();
                 }
                 return total;
             }
@@ -658,10 +659,10 @@ public class ColumnFamilyMetrics
      */
     protected <G,T> Gauge<T> createColumnFamilyGauge(String name, Gauge<T> gauge, Gauge<G> globalGauge)
     {
-        Gauge<T> cfGauge = Metrics.newGauge(factory.createMetricName(name), gauge);
+        Gauge<T> cfGauge = Metrics.register(factory.createMetricName(name), gauge);
         if (register(name, cfGauge))
         {
-            Metrics.newGauge(globalNameFactory.createMetricName(name), globalGauge);
+            Metrics.register(globalNameFactory.createMetricName(name), globalGauge);
         }
         return cfGauge;
     }
@@ -672,17 +673,17 @@ public class ColumnFamilyMetrics
      */
     protected Counter createColumnFamilyCounter(final String name)
     {
-        Counter cfCounter = Metrics.newCounter(factory.createMetricName(name));
+        Counter cfCounter = Metrics.counter(factory.createMetricName(name));
         if (register(name, cfCounter))
         {
-            Metrics.newGauge(globalNameFactory.createMetricName(name), new Gauge<Long>()
+            Metrics.register(globalNameFactory.createMetricName(name), new Gauge<Long>()
             {
-                public Long value()
+                public Long getValue()
                 {
                     long total = 0;
                     for (Metric cfGauge : allColumnFamilyMetrics.get(name))
                     {
-                        total += ((Counter) cfGauge).count();
+                        total += ((Counter) cfGauge).getCount();
                     }
                     return total;
                 }
@@ -695,13 +696,13 @@ public class ColumnFamilyMetrics
      * Create a histogram-like interface that will register both a CF, keyspace and global level
      * histogram and forward any updates to both
      */
-    protected ColumnFamilyHistogram createColumnFamilyHistogram(String name, Histogram keyspaceHistogram) 
+    protected ColumnFamilyHistogram createColumnFamilyHistogram(String name, Histogram keyspaceHistogram)
     {
-        Histogram cfHistogram = Metrics.newHistogram(factory.createMetricName(name), true);  
+        Histogram cfHistogram = Metrics.histogram(factory.createMetricName(name));
         register(name, cfHistogram);
-        return new ColumnFamilyHistogram(cfHistogram, keyspaceHistogram, Metrics.newHistogram(globalNameFactory.createMetricName(name), true));
+        return new ColumnFamilyHistogram(cfHistogram, keyspaceHistogram, Metrics.histogram(globalNameFactory.createMetricName(name)));
     }
-    
+
     /**
      * Registers a metric to be removed when unloading CF.
      * @return true if first time metric with that name has been registered
@@ -718,13 +719,13 @@ public class ColumnFamilyMetrics
     {
         public final Histogram[] all;
         public final Histogram cf;
-        private ColumnFamilyHistogram(Histogram cf, Histogram keyspace, Histogram global) 
+        private ColumnFamilyHistogram(Histogram cf, Histogram keyspace, Histogram global)
         {
             this.cf = cf;
             this.all = new Histogram[]{cf, keyspace, global};
         }
-        
-        public void update(long i) 
+
+        public void update(long i)
         {
             for(Histogram histo : all)
             {
@@ -746,7 +747,7 @@ public class ColumnFamilyMetrics
             isIndex = cfs.isIndex();
         }
 
-        public MetricName createMetricName(String metricName)
+        public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
         {
             String groupName = ColumnFamilyMetrics.class.getPackage().getName();
             String type = isIndex ? "IndexColumnFamily" : "ColumnFamily";
@@ -758,20 +759,20 @@ public class ColumnFamilyMetrics
             mbeanName.append(",scope=").append(columnFamilyName);
             mbeanName.append(",name=").append(metricName);
 
-            return new MetricName(groupName, type, metricName, keyspaceName + "." + columnFamilyName, mbeanName.toString());
+            return new CassandraMetricsRegistry.MetricName(groupName, type, metricName, keyspaceName + "." + columnFamilyName, mbeanName.toString());
         }
     }
     
     static class AllColumnFamilyMetricNameFactory implements MetricNameFactory
     {
-        public MetricName createMetricName(String metricName)
+        public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
         {
             String groupName = ColumnFamilyMetrics.class.getPackage().getName(); 
             StringBuilder mbeanName = new StringBuilder();
             mbeanName.append(groupName).append(":");
             mbeanName.append("type=ColumnFamily");
             mbeanName.append(",name=").append(metricName);
-            return new MetricName(groupName, "ColumnFamily", metricName, "all", mbeanName.toString());
+            return new CassandraMetricsRegistry.MetricName(groupName, "ColumnFamily", metricName, "all", mbeanName.toString());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
index e9c0719..670fa6b 100644
--- a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
@@ -17,14 +17,13 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
 
-import com.yammer.metrics.core.Timer;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Timer;
 import org.apache.cassandra.db.commitlog.AbstractCommitLogService;
 import org.apache.cassandra.db.commitlog.CommitLogSegmentManager;
 
-import java.util.concurrent.TimeUnit;
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
 /**
  * Metrics for commit log
@@ -46,28 +45,28 @@ public class CommitLogMetrics
 
     public CommitLogMetrics(final AbstractCommitLogService service, final CommitLogSegmentManager allocator)
     {
-        completedTasks = Metrics.newGauge(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
+        completedTasks = Metrics.register(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return service.getCompletedTasks();
             }
         });
-        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Long>()
+        pendingTasks = Metrics.register(factory.createMetricName("PendingTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return service.getPendingTasks();
             }
         });
-        totalCommitLogSize = Metrics.newGauge(factory.createMetricName("TotalCommitLogSize"), new Gauge<Long>()
+        totalCommitLogSize = Metrics.register(factory.createMetricName("TotalCommitLogSize"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return allocator.bytesUsed();
             }
         });
-        waitingOnSegmentAllocation = Metrics.newTimer(factory.createMetricName("WaitingOnSegmentAllocation"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
-        waitingOnCommit = Metrics.newTimer(factory.createMetricName("WaitingOnCommit"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
+        waitingOnSegmentAllocation = Metrics.timer(factory.createMetricName("WaitingOnSegmentAllocation"));
+        waitingOnCommit = Metrics.timer(factory.createMetricName("WaitingOnCommit"));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
index b015130..a62e3c4 100644
--- a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
@@ -19,12 +19,10 @@ package org.apache.cassandra.metrics;
 
 import java.util.*;
 import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.core.Meter;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
 
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -32,6 +30,8 @@ import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 /**
  * Metrics for compaction.
  */
@@ -53,9 +53,9 @@ public class CompactionMetrics implements CompactionManager.CompactionExecutorSt
 
     public CompactionMetrics(final ThreadPoolExecutor... collectors)
     {
-        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
+        pendingTasks = Metrics.register(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 int n = 0;
                 for (String keyspaceName : Schema.instance.getKeyspaces())
@@ -68,9 +68,9 @@ public class CompactionMetrics implements CompactionManager.CompactionExecutorSt
                 return n;
             }
         });
-        completedTasks = Metrics.newGauge(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
+        completedTasks = Metrics.register(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long completedTasks = 0;
                 for (ThreadPoolExecutor collector : collectors)
@@ -78,8 +78,8 @@ public class CompactionMetrics implements CompactionManager.CompactionExecutorSt
                 return completedTasks;
             }
         });
-        totalCompactionsCompleted = Metrics.newMeter(factory.createMetricName("TotalCompactionsCompleted"), "compaction completed", TimeUnit.SECONDS);
-        bytesCompacted = Metrics.newCounter(factory.createMetricName("BytesCompacted"));
+        totalCompactionsCompleted = Metrics.meter(factory.createMetricName("TotalCompactionsCompleted"));
+        bytesCompacted = Metrics.counter(factory.createMetricName("BytesCompacted"));
     }
 
     public void beginCompaction(CompactionInfo.Holder ci)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
index eaf29f8..60020b3 100644
--- a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
@@ -18,11 +18,12 @@
 package org.apache.cassandra.metrics;
 
 import java.net.InetAddress;
-import java.util.concurrent.TimeUnit;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.core.Meter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 import org.apache.cassandra.net.OutboundTcpConnectionPool;
 
@@ -34,8 +35,7 @@ public class ConnectionMetrics
     public static final String TYPE_NAME = "Connection";
 
     /** Total number of timeouts happened on this node */
-    public static final Meter totalTimeouts = Metrics.newMeter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalTimeouts", null), "total timeouts", TimeUnit.SECONDS);
-    private static long recentTimeouts;
+    public static final Meter totalTimeouts = Metrics.meter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalTimeouts", null));
 
     public final String address;
     /** Pending tasks for Command(Mutations, Read etc) TCP Connections */
@@ -53,8 +53,6 @@ public class ConnectionMetrics
 
     private final MetricNameFactory factory;
 
-    private long recentTimeoutCount;
-
     /**
      * Create metrics for given connection pool.
      *
@@ -68,69 +66,51 @@ public class ConnectionMetrics
 
         factory = new DefaultNameFactory("Connection", address);
 
-        commandPendingTasks = Metrics.newGauge(factory.createMetricName("CommandPendingTasks"), new Gauge<Integer>()
+        commandPendingTasks = Metrics.register(factory.createMetricName("CommandPendingTasks"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return connectionPool.cmdCon.getPendingMessages();
             }
         });
-        commandCompletedTasks = Metrics.newGauge(factory.createMetricName("CommandCompletedTasks"), new Gauge<Long>()
+        commandCompletedTasks = Metrics.register(factory.createMetricName("CommandCompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return connectionPool.cmdCon.getCompletedMesssages();
             }
         });
-        commandDroppedTasks = Metrics.newGauge(factory.createMetricName("CommandDroppedTasks"), new Gauge<Long>()
+        commandDroppedTasks = Metrics.register(factory.createMetricName("CommandDroppedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return connectionPool.cmdCon.getDroppedMessages();
             }
         });
-        responsePendingTasks = Metrics.newGauge(factory.createMetricName("ResponsePendingTasks"), new Gauge<Integer>()
+        responsePendingTasks = Metrics.register(factory.createMetricName("ResponsePendingTasks"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return connectionPool.ackCon.getPendingMessages();
             }
         });
-        responseCompletedTasks = Metrics.newGauge(factory.createMetricName("ResponseCompletedTasks"), new Gauge<Long>()
+        responseCompletedTasks = Metrics.register(factory.createMetricName("ResponseCompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return connectionPool.ackCon.getCompletedMesssages();
             }
         });
-        timeouts = Metrics.newMeter(factory.createMetricName("Timeouts"), "timeouts", TimeUnit.SECONDS);
+        timeouts = Metrics.meter(factory.createMetricName("Timeouts"));
     }
 
     public void release()
     {
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CommandPendingTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CommandCompletedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CommandDroppedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("ResponsePendingTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("ResponseCompletedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("Timeouts"));
-    }
-
-    @Deprecated
-    public static long getRecentTotalTimeout()
-    {
-        long total = totalTimeouts.count();
-        long recent = total - recentTimeouts;
-        recentTimeouts = total;
-        return recent;
-    }
-
-    @Deprecated
-    public long getRecentTimeout()
-    {
-        long timeoutCount = timeouts.count();
-        long recent = timeoutCount - recentTimeoutCount;
-        recentTimeoutCount = timeoutCount;
-        return recent;
+        Metrics.remove(factory.createMetricName("CommandPendingTasks"));
+        Metrics.remove(factory.createMetricName("CommandCompletedTasks"));
+        Metrics.remove(factory.createMetricName("CommandDroppedTasks"));
+        Metrics.remove(factory.createMetricName("ResponsePendingTasks"));
+        Metrics.remove(factory.createMetricName("ResponseCompletedTasks"));
+        Metrics.remove(factory.createMetricName("Timeouts"));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java b/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java
index f7f23ee..db51906 100644
--- a/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java
+++ b/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.core.MetricName;
 
 /**
  * MetricNameFactory that generates default MetricName of metrics.
@@ -40,14 +39,14 @@ public class DefaultNameFactory implements MetricNameFactory
         this.scope = scope;
     }
 
-    public MetricName createMetricName(String metricName)
+    public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
     {
         return createMetricName(type, metricName, scope);
     }
 
-    public static MetricName createMetricName(String type, String metricName, String scope)
+    public static CassandraMetricsRegistry.MetricName createMetricName(String type, String metricName, String scope)
     {
-        return new MetricName(GROUP_NAME, type, metricName, scope, createDefaultMBeanName(type, metricName, scope));
+        return new CassandraMetricsRegistry.MetricName(GROUP_NAME, type, metricName, scope, createDefaultMBeanName(type, metricName, scope));
     }
 
     protected static String createDefaultMBeanName(String type, String name, String scope)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
index f94ea04..58c80fb 100644
--- a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
@@ -17,13 +17,11 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.concurrent.TimeUnit;
-
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Meter;
-
+import com.codahale.metrics.Meter;
 import org.apache.cassandra.net.MessagingService;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 /**
  * Metrics for dropped messages by verb.
  */
@@ -32,20 +30,9 @@ public class DroppedMessageMetrics
     /** Number of dropped messages */
     public final Meter dropped;
 
-    private long lastDropped = 0;
-
     public DroppedMessageMetrics(MessagingService.Verb verb)
     {
         MetricNameFactory factory = new DefaultNameFactory("DroppedMessage", verb.toString());
-        dropped = Metrics.newMeter(factory.createMetricName("Dropped"), "dropped", TimeUnit.SECONDS);
-    }
-
-    @Deprecated
-    public int getRecentlyDropped()
-    {
-        long currentDropped = dropped.count();
-        long recentlyDropped = currentDropped - lastDropped;
-        lastDropped = currentDropped;
-        return (int)recentlyDropped;
+        dropped = Metrics.meter(factory.createMetricName("Dropped"));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java b/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java
new file mode 100644
index 0000000..221f2e1
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/EstimatedHistogramReservoir.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.metrics;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import com.codahale.metrics.Reservoir;
+import com.codahale.metrics.Snapshot;
+import com.codahale.metrics.UniformSnapshot;
+import org.apache.cassandra.utils.EstimatedHistogram;
+
+/**
+ * Allows our Histogram implementation to be used by the metrics library.
+ *
+ * Default buckets allows nanosecond timings.
+ */
+public class EstimatedHistogramReservoir implements Reservoir
+{
+    EstimatedHistogram histogram;
+
+    public EstimatedHistogramReservoir()
+    {
+        this(128);
+    }
+
+    public EstimatedHistogramReservoir(int numBuckets)
+    {
+        histogram = new EstimatedHistogram(numBuckets);
+    }
+
+    @Override
+    public int size()
+    {
+        return histogram.getBucketOffsets().length + 1;
+    }
+
+    @Override
+    public void update(long value)
+    {
+        histogram.add(value);
+    }
+
+    @Override
+    public Snapshot getSnapshot()
+    {
+        return new HistogramSnapshot(histogram);
+    }
+
+    @VisibleForTesting
+    public void clear()
+    {
+        histogram.getBuckets(true);
+    }
+
+    class HistogramSnapshot extends UniformSnapshot
+    {
+        EstimatedHistogram histogram;
+
+        public HistogramSnapshot(EstimatedHistogram histogram)
+        {
+            super(histogram.getBuckets(false));
+
+            this.histogram = histogram;
+        }
+
+        @Override
+        public double getValue(double quantile)
+        {
+            return histogram.percentile(quantile);
+        }
+
+        @Override
+        public long getMax()
+        {
+            return histogram.max();
+        }
+
+        @Override
+        public long getMin()
+        {
+            return histogram.min();
+        }
+
+        @Override
+        public double getMean()
+        {
+            return histogram.mean();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java b/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java
index 5f6bd20..c240c03 100644
--- a/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java
@@ -17,14 +17,14 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.concurrent.TimeUnit;
-
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.core.Meter;
-import com.yammer.metrics.util.RatioGauge;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.RatioGauge;
 import org.apache.cassandra.service.FileCacheService;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
+
 public class FileCacheMetrics
 {
     private static final MetricNameFactory factory = new DefaultNameFactory("FileCache");
@@ -40,23 +40,19 @@ public class FileCacheMetrics
 
     public FileCacheMetrics()
     {
-        hits = Metrics.newMeter(factory.createMetricName("Hits"), "hits", TimeUnit.SECONDS);
-        requests = Metrics.newMeter(factory.createMetricName("Requests"), "requests", TimeUnit.SECONDS);
-        hitRate = Metrics.newGauge(factory.createMetricName("HitRate"), new RatioGauge()
+        hits = Metrics.meter(factory.createMetricName("Hits"));
+        requests = Metrics.meter(factory.createMetricName("Requests"));
+        hitRate = Metrics.register(factory.createMetricName("HitRate"), new RatioGauge()
         {
-            protected double getNumerator()
-            {
-                return hits.count();
-            }
-
-            protected double getDenominator()
+            @Override
+            public Ratio getRatio()
             {
-                return requests.count();
+                return Ratio.of(hits.getCount(), requests.getCount());
             }
         });
-        size = Metrics.newGauge(factory.createMetricName("Size"), new Gauge<Long>()
+        size = Metrics.register(factory.createMetricName("Size"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return FileCacheService.instance.sizeInBytes();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
index d002e39..fef8c1f 100644
--- a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.metrics;
 import java.net.InetAddress;
 import java.util.Map.Entry;
 
+import com.codahale.metrics.Counter;
 import org.apache.cassandra.db.HintedHandOffManager;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.utils.UUIDGen;
@@ -29,8 +30,8 @@ import org.slf4j.LoggerFactory;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
 /**
  * Metrics for {@link HintedHandOffManager}.
@@ -39,7 +40,7 @@ public class HintedHandoffMetrics
 {
     private static final Logger logger = LoggerFactory.getLogger(HintedHandoffMetrics.class);
 
-    private final MetricNameFactory factory = new DefaultNameFactory("HintedHandOffManager");
+    private static final MetricNameFactory factory = new DefaultNameFactory("HintedHandOffManager");
 
     /** Total number of hints which are not stored, This is not a cache. */
     private final LoadingCache<InetAddress, DifferencingCounter> notStored = CacheBuilder.newBuilder().build(new CacheLoader<InetAddress, DifferencingCounter>()
@@ -55,7 +56,7 @@ public class HintedHandoffMetrics
     {
         public Counter load(InetAddress address)
         {
-            return Metrics.newCounter(factory.createMetricName("Hints_created-" + address.getHostAddress()));
+            return Metrics.counter(factory.createMetricName("Hints_created-" + address.getHostAddress()));
         }
     });
 
@@ -88,12 +89,12 @@ public class HintedHandoffMetrics
 
         public DifferencingCounter(InetAddress address)
         {
-            this.meter = Metrics.newCounter(factory.createMetricName("Hints_not_stored-" + address.getHostAddress()));
+            this.meter = Metrics.counter(factory.createMetricName("Hints_not_stored-" + address.getHostAddress()));
         }
 
         public long difference()
         {
-            long current = meter.count();
+            long current = meter.getCount();
             long difference = current - reported;
             this.reported = current;
             return difference;
@@ -101,7 +102,7 @@ public class HintedHandoffMetrics
 
         public long count()
         {
-            return meter.count();
+            return meter.getCount();
         }
 
         public void mark()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8896a70b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index fa6f33b..0bac421 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -19,13 +19,17 @@ package org.apache.cassandra.metrics;
 
 import java.util.Set;
 
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.MetricRegistry;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.*;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 /**
  * Metrics for {@link ColumnFamilyStore}.
@@ -104,112 +108,112 @@ public class KeyspaceMetrics
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableColumnsCount.value();
+                return metric.memtableColumnsCount.getValue();
             }
         });
         memtableLiveDataSize = createKeyspaceGauge("MemtableLiveDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableLiveDataSize.value();
+                return metric.memtableLiveDataSize.getValue();
             }
         }); 
         memtableOnHeapDataSize = createKeyspaceGauge("MemtableOnHeapDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableOnHeapSize.value();
+                return metric.memtableOnHeapSize.getValue();
             }
         });
         memtableOffHeapDataSize = createKeyspaceGauge("MemtableOffHeapDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableOffHeapSize.value();
+                return metric.memtableOffHeapSize.getValue();
             }
         });
         allMemtablesLiveDataSize = createKeyspaceGauge("AllMemtablesLiveDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.allMemtablesLiveDataSize.value();
+                return metric.allMemtablesLiveDataSize.getValue();
             }
         });
         allMemtablesOnHeapDataSize = createKeyspaceGauge("AllMemtablesOnHeapDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.allMemtablesOnHeapSize.value();
+                return metric.allMemtablesOnHeapSize.getValue();
             }
         });
         allMemtablesOffHeapDataSize = createKeyspaceGauge("AllMemtablesOffHeapDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.allMemtablesOffHeapSize.value();
+                return metric.allMemtablesOffHeapSize.getValue();
             }
         });
         memtableSwitchCount = createKeyspaceGauge("MemtableSwitchCount", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableSwitchCount.count();
+                return metric.memtableSwitchCount.getCount();
             }
         });
         pendingCompactions = createKeyspaceGauge("PendingCompactions", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return (long) metric.pendingCompactions.value();
+                return (long) metric.pendingCompactions.getValue();
             }
         });
         pendingFlushes = createKeyspaceGauge("PendingFlushes", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return (long) metric.pendingFlushes.count();
+                return (long) metric.pendingFlushes.getCount();
             }
         });
         liveDiskSpaceUsed = createKeyspaceGauge("LiveDiskSpaceUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.liveDiskSpaceUsed.count();
+                return metric.liveDiskSpaceUsed.getCount();
             }
         });
         totalDiskSpaceUsed = createKeyspaceGauge("TotalDiskSpaceUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.totalDiskSpaceUsed.count();
+                return metric.totalDiskSpaceUsed.getCount();
             }
         });
         bloomFilterDiskSpaceUsed = createKeyspaceGauge("BloomFilterDiskSpaceUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.bloomFilterDiskSpaceUsed.value();
+                return metric.bloomFilterDiskSpaceUsed.getValue();
             }
         });
         bloomFilterOffHeapMemoryUsed = createKeyspaceGauge("BloomFilterOffHeapMemoryUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.bloomFilterOffHeapMemoryUsed.value();
+                return metric.bloomFilterOffHeapMemoryUsed.getValue();
             }
         });
         indexSummaryOffHeapMemoryUsed = createKeyspaceGauge("IndexSummaryOffHeapMemoryUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.indexSummaryOffHeapMemoryUsed.value();
+                return metric.indexSummaryOffHeapMemoryUsed.getValue();
             }
         });
         compressionMetadataOffHeapMemoryUsed = createKeyspaceGauge("CompressionMetadataOffHeapMemoryUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.compressionMetadataOffHeapMemoryUsed.value();
+                return metric.compressionMetadataOffHeapMemoryUsed.getValue();
             }
         });
         // latency metrics for ColumnFamilyMetrics to update
@@ -217,10 +221,10 @@ public class KeyspaceMetrics
         writeLatency = new LatencyMetrics(factory, "Write");
         rangeLatency = new LatencyMetrics(factory, "Range");
         // create histograms for ColumnFamilyMetrics to replicate updates to
-        sstablesPerReadHistogram = Metrics.newHistogram(factory.createMetricName("SSTablesPerReadHistogram"), true);
-        tombstoneScannedHistogram = Metrics.newHistogram(factory.createMetricName("TombstoneScannedHistogram"), true);
-        liveScannedHistogram = Metrics.newHistogram(factory.createMetricName("LiveScannedHistogram"), true);
-        colUpdateTimeDeltaHistogram = Metrics.newHistogram(factory.createMetricName("ColUpdateTimeDeltaHistogram"), true);
+        sstablesPerReadHistogram = Metrics.histogram(factory.createMetricName("SSTablesPerReadHistogram"));
+        tombstoneScannedHistogram = Metrics.histogram(factory.createMetricName("TombstoneScannedHistogram"));
+        liveScannedHistogram = Metrics.histogram(factory.createMetricName("LiveScannedHistogram"));
+        colUpdateTimeDeltaHistogram = Metrics.histogram(factory.createMetricName("ColUpdateTimeDeltaHistogram"));
         // add manually since histograms do not use createKeyspaceGauge method
         allMetrics.addAll(Lists.newArrayList("SSTablesPerReadHistogram", "TombstoneScannedHistogram", "LiveScannedHistogram"));
 
@@ -236,7 +240,7 @@ public class KeyspaceMetrics
     {
         for(String name : allMetrics) 
         {
-            Metrics.defaultRegistry().removeMetric(factory.createMetricName(name));
+            Metrics.remove(factory.createMetricName(name));
         }
         // latency metrics contain multiple metrics internally and need to be released manually
         readLatency.release();
@@ -251,7 +255,7 @@ public class KeyspaceMetrics
     {
         /**
          * get value of a metric
-         * @param columnfamilymetrics of a column family in this keyspace
+         * @param metric of a column family in this keyspace
          * @return current value of a metric
          */
         public Long getValue(ColumnFamilyMetrics metric);
@@ -260,15 +264,15 @@ public class KeyspaceMetrics
     /**
      * Creates a gauge that will sum the current value of a metric for all column families in this keyspace
      * @param name
-     * @param MetricValue 
+     * @param extractor
      * @return Gauge&gt;Long> that computes sum of MetricValue.getValue()
      */
     private Gauge<Long> createKeyspaceGauge(String name, final MetricValue extractor)
     {
         allMetrics.add(name);
-        return Metrics.newGauge(factory.createMetricName(name), new Gauge<Long>()
+        return Metrics.register(factory.createMetricName(name), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long sum = 0;
                 for (ColumnFamilyStore cf : keyspace.getColumnFamilyStores())
@@ -289,7 +293,7 @@ public class KeyspaceMetrics
             this.keyspaceName = ks.getName();
         }
 
-        public MetricName createMetricName(String metricName)
+        public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
         {
             String groupName = ColumnFamilyMetrics.class.getPackage().getName();
 
@@ -299,7 +303,7 @@ public class KeyspaceMetrics
             mbeanName.append(",keyspace=").append(keyspaceName);
             mbeanName.append(",name=").append(metricName);
 
-            return new MetricName(groupName, "keyspace", metricName, keyspaceName, mbeanName.toString());
+            return new CassandraMetricsRegistry.MetricName(groupName, "keyspace", metricName, keyspaceName, mbeanName.toString());
         }
     }
 }