You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2013/08/07 19:55:27 UTC

[2/3] git commit: Log Merkle tree precision stats

Log Merkle tree precision stats

patch by Benedict; reviewed by yukim for CASSANDRA-2698


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/18f0234b
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/18f0234b
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/18f0234b

Branch: refs/heads/trunk
Commit: 18f0234bf908d1f970bb694d0bb58539ab427387
Parents: 3205c10
Author: Benedict <su...@laerad.com>
Authored: Wed Aug 7 12:19:09 2013 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Wed Aug 7 12:19:58 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/repair/Validator.java  |  61 ++++-
 .../cassandra/utils/EstimatedHistogram.java     |  65 ++++++
 .../cassandra/utils/HistogramBuilder.java       | 158 +++++++++++++
 .../org/apache/cassandra/utils/MerkleTree.java  | 233 ++++++++++++++-----
 .../cassandra/repair/DifferencerTest.java       |   2 +-
 .../cassandra/utils/HistogramBuilderTest.java   | 109 +++++++++
 .../apache/cassandra/utils/MerkleTreeTest.java  |   9 +-
 8 files changed, 572 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/18f0234b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9509113..4dca1b8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 2.0.1
  * Notify indexer of columns shadowed by range tombstones (CASSANDRA-5614)
+ * Log Merkle tree stats (CASSANDRA-2698)
 
 
 2.0.0

http://git-wip-us.apache.org/repos/asf/cassandra/blob/18f0234b/src/java/org/apache/cassandra/repair/Validator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/Validator.java b/src/java/org/apache/cassandra/repair/Validator.java
index 4e8e865..e7efcaa 100644
--- a/src/java/org/apache/cassandra/repair/Validator.java
+++ b/src/java/org/apache/cassandra/repair/Validator.java
@@ -61,8 +61,6 @@ public class Validator implements Runnable
     private transient MerkleTree.TreeRangeIterator ranges;
     private transient DecoratedKey lastKey;
 
-    public final static MerkleTree.RowHash EMPTY_ROW = new MerkleTree.RowHash(null, new byte[0]);
-
     /**
      * Create Validator with default size of initial Merkle Tree.
      */
@@ -145,7 +143,7 @@ public class Validator implements Runnable
         while (!range.contains(row.key.token))
         {
             // add the empty hash, and move to the next range
-            range.addHash(EMPTY_ROW);
+            range.ensureHashInitialised();
             range = ranges.next();
         }
 
@@ -153,13 +151,52 @@ public class Validator implements Runnable
         range.addHash(rowHash(row));
     }
 
+    static class CountingDigest extends MessageDigest
+    {
+        private long count;
+        private MessageDigest underlying;
+
+        public CountingDigest(MessageDigest underlying)
+        {
+            super(underlying.getAlgorithm());
+            this.underlying = underlying;
+        }
+
+        @Override
+        protected void engineUpdate(byte input)
+        {
+            underlying.update(input);
+            count += 1;
+        }
+
+        @Override
+        protected void engineUpdate(byte[] input, int offset, int len)
+        {
+            underlying.update(input, offset, len);
+            count += len;
+        }
+
+        @Override
+        protected byte[] engineDigest()
+        {
+            return underlying.digest();
+        }
+
+        @Override
+        protected void engineReset()
+        {
+            underlying.reset();
+        }
+
+    }
+
     private MerkleTree.RowHash rowHash(AbstractCompactedRow row)
     {
         validated++;
         // MerkleTree uses XOR internally, so we want lots of output bits here
-        MessageDigest digest = FBUtilities.newMessageDigest("SHA-256");
+        CountingDigest digest = new CountingDigest(FBUtilities.newMessageDigest("SHA-256"));
         row.update(digest);
-        return new MerkleTree.RowHash(row.key.token, digest.digest());
+        return new MerkleTree.RowHash(row.key.token, digest.digest(), digest.count);
     }
 
     /**
@@ -170,7 +207,15 @@ public class Validator implements Runnable
         completeTree();
 
         StageManager.getStage(Stage.ANTI_ENTROPY).execute(this);
-        logger.debug("Validated " + validated + " rows into AEService tree for " + desc);
+
+        if (logger.isDebugEnabled())
+        {
+            // log distribution of rows in tree
+            logger.debug("Validated " + validated + " rows into AEService tree for " + desc + " with row count distribution:");
+            tree.histogramOfRowCountPerLeaf().log(logger);
+            logger.debug("Validated " + validated + " rows into AEService tree for " + desc + " with row size distribution:");
+            tree.histogramOfRowSizePerLeaf().log(logger);
+        }
     }
 
     @VisibleForTesting
@@ -179,11 +224,11 @@ public class Validator implements Runnable
         assert ranges != null : "Validator was not prepared()";
 
         if (range != null)
-            range.addHash(EMPTY_ROW);
+            range.ensureHashInitialised();
         while (ranges.hasNext())
         {
             range = ranges.next();
-            range.addHash(EMPTY_ROW);
+            range.ensureHashInitialised();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/18f0234b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/EstimatedHistogram.java b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
index 54407a4..3a30290 100644
--- a/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
+++ b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
@@ -27,6 +27,7 @@ import com.google.common.base.Objects;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.ISerializer;
+import org.slf4j.Logger;
 
 public class EstimatedHistogram
 {
@@ -226,6 +227,70 @@ public class EstimatedHistogram
         return buckets.get(buckets.length() - 1) > 0;
     }
 
+    /**
+     * log.debug() every record in the histogram
+     *
+     * @param log
+     */
+    public void log(Logger log)
+    {
+
+        // only print overflow if there is any
+        int nameCount;
+        if (buckets.get(buckets.length() - 1) == 0)
+            nameCount = buckets.length() - 1;
+        else
+            nameCount = buckets.length();
+        String[] names = new String[nameCount];
+
+        int maxNameLength = 0;
+        for (int i = 0; i < nameCount; i++)
+        {
+            names[i] = nameOfRange(bucketOffsets, i);
+            maxNameLength = Math.max(maxNameLength, names[i].length());
+        }
+
+        // emit log records
+        String formatstr = "%" + maxNameLength + "s: %d";
+        for (int i = 0; i < nameCount; i++)
+        {
+            long count = buckets.get(i);
+            // sort-of-hack to not print empty ranges at the start that are only used to demarcate the
+            // first populated range. for code clarity we don't omit this record from the maxNameLength
+            // calculation, and accept the unnecessary whitespace prefixes that will occasionally occur
+            if (i == 0 && count == 0)
+                continue;
+            log.debug(String.format(formatstr, names[i], count));
+        }
+    }
+
+    private static String nameOfRange(long[] bucketOffsets, int index)
+    {
+        StringBuilder sb = new StringBuilder();
+        appendRange(sb, bucketOffsets, index);
+        return sb.toString();
+    }
+
+    private static void appendRange(StringBuilder sb, long[] bucketOffsets, int index)
+    {
+        sb.append("[");
+        if (index == 0)
+            if (bucketOffsets[0] > 0)
+                // by original definition, this histogram is for values greater than zero only;
+                // if values of 0 or less are required, an entry of lb-1 must be inserted at the start
+                sb.append("1");
+            else
+                sb.append("-Inf");
+        else
+            sb.append(bucketOffsets[index - 1] + 1);
+        sb.append("..");
+        if (index == bucketOffsets.length)
+            sb.append("Inf");
+        else
+            sb.append(bucketOffsets[index]);
+        sb.append("]");
+    }
+
     @Override
     public boolean equals(Object o)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/18f0234b/src/java/org/apache/cassandra/utils/HistogramBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/HistogramBuilder.java b/src/java/org/apache/cassandra/utils/HistogramBuilder.java
new file mode 100644
index 0000000..c2accb2
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/HistogramBuilder.java
@@ -0,0 +1,158 @@
+/*
+ * 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.utils;
+
+import java.util.Arrays;
+
+/**
+ * Simple class for constructing an EsimtatedHistogram from a set of predetermined values
+ */
+public class HistogramBuilder
+{
+
+    public HistogramBuilder() {}
+    public HistogramBuilder(long[] values)
+    {
+        for (long value : values)
+        {
+            add(value);
+        }
+    }
+
+    private long[] values = new long[10];
+    int count = 0;
+
+    public void add(long value)
+    {
+        if (count == values.length)
+            values = Arrays.copyOf(values, values.length << 1);
+        values[count++] = value;
+    }
+
+    /**
+     * See {@link #buildWithStdevRangesAroundMean(int)}
+     * @return buildWithStdevRangesAroundMean(3)
+     */
+    public EstimatedHistogram buildWithStdevRangesAroundMean()
+    {
+        return buildWithStdevRangesAroundMean(3);
+    }
+
+    /**
+     * Calculate the min, mean, max and standard deviation of the items in the builder, and
+     * generate an EstimatedHistogram with upto <code>maxdev</code> stdev size ranges  either
+     * side of the mean, until min/max are hit; if either min/max are not reached a further range is
+     * inserted at the relevant ends. e.g., with a <code>maxdevs</code> of 3, there may be <i>up to</i> 8 ranges
+     * (between 9 boundaries, the middle being the mean); the middle 6 will have the same size (stdev)
+     * with the outermost two stretching out to min and max.
+     *
+     * @param maxdevs
+     * @return
+     */
+    public EstimatedHistogram buildWithStdevRangesAroundMean(int maxdevs)
+    {
+        if (maxdevs < 0)
+            throw new IllegalArgumentException("maxdevs must be greater than or equal to zero");
+
+        final int count = this.count;
+        final long[] values = this.values;
+
+        if (count == 0)
+            return new EstimatedHistogram(new long[] { }, new long[] { 0 });
+
+        long min = Long.MAX_VALUE, max = Long.MIN_VALUE;
+        double sum = 0, sumsq = 0;
+        for (int i = 0 ; i < count ; i++)
+        {
+            final long value = values[i];
+            sum += value;
+            sumsq += value * value;
+            if (value < min)
+                min = value;
+            if (value > max)
+                max = value;
+        }
+
+        final long mean = (long)Math.round(sum / count);
+        final double stdev =
+                Math.sqrt((sumsq / count) - (mean * (double) mean));
+
+        // build the ranges either side of the mean
+        final long[] lowhalf = buildRange(mean, min, true, stdev, maxdevs);
+        final long[] highhalf = buildRange(mean, max, false, stdev, maxdevs);
+
+        // combine the ranges
+        final long[] ranges = new long[lowhalf.length + highhalf.length + 1];
+        System.arraycopy(lowhalf, 0, ranges, 0, lowhalf.length);
+        ranges[lowhalf.length] = mean;
+        System.arraycopy(highhalf, 0, ranges, lowhalf.length + 1, highhalf.length);
+
+        final EstimatedHistogram hist = new EstimatedHistogram(ranges, new long[ranges.length + 1]);
+        for (int i = 0 ; i < count ; i++)
+            hist.add(values[i]);
+        return hist;
+    }
+
+    private static long[] buildRange(long mean, long minormax, boolean ismin, double stdev, int maxdevs)
+    {
+        if (minormax == mean)
+            // minormax == mean we have no range to produce, but given the exclusive starts
+            // that begin at zero by default (or -Inf) in EstimatedHistogram we have to generate a min range
+            // to indicate where we start from
+            return ismin ? new long[] { mean - 1 } : new long[0];
+
+        if (stdev < 1)
+        {
+            // deal with stdevs too small to generate sensible ranges
+            return ismin ? new long[] { minormax - 1, mean - 1 } :
+                           new long[] { minormax };
+        }
+
+        long larger, smaller;
+        if (ismin) { larger = mean;     smaller = minormax; }
+        else       { larger = minormax; smaller = mean;     }
+
+        double stdevsTo = (larger - smaller) / stdev;
+        if (stdevsTo > 0 && stdevsTo < 1)
+            // always round up if there's just one non-empty range
+            stdevsTo = 1;
+        else
+            // otherwise round to the nearest half stdev, to avoid tiny ranges at the start/end
+            stdevsTo = Math.round(stdevsTo);
+
+        // limit to 4 stdev ranges - last range will contain everything to boundary
+        final int len = Math.min(maxdevs + 1, (int) stdevsTo);
+        final long[] range = new long[len];
+        long next = ismin ? minormax - 1 : minormax;
+        for (int i = 0 ; i < range.length ; i++)
+        {
+            long delta = (range.length - (i + 1)) * (long) stdev;
+            if (ismin)
+            {
+                range[i] = next;
+                next = mean - delta;
+            }
+            else
+            {
+                range[len - 1 - i] = next;
+                next = mean + delta;
+            }
+        }
+        return range;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/18f0234b/src/java/org/apache/cassandra/utils/MerkleTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/MerkleTree.java b/src/java/org/apache/cassandra/utils/MerkleTree.java
index a16c2b9..ce71ec4 100644
--- a/src/java/org/apache/cassandra/utils/MerkleTree.java
+++ b/src/java/org/apache/cassandra/utils/MerkleTree.java
@@ -64,6 +64,7 @@ public class MerkleTree implements Serializable
     public static final int CONSISTENT = 0;
     public static final int FULLY_INCONSISTENT = 1;
     public static final int PARTIALLY_INCONSISTENT = 2;
+    private static final byte[] EMPTY_HASH = new byte[0];
 
     public final byte hashdepth;
 
@@ -227,11 +228,14 @@ public class MerkleTree implements Serializable
         if (!ltree.fullRange.equals(rtree.fullRange))
             throw new IllegalArgumentException("Difference only make sense on tree covering the same range (but " + ltree.fullRange + " != " + rtree.fullRange + ")");
 
-        List<TreeRange> diff = new ArrayList<TreeRange>();
-        TreeRange active = new TreeRange(null, ltree.fullRange.left, ltree.fullRange.right, (byte)0, null);
+        List<TreeRange> diff = new ArrayList<>();
+        TreeDifference active = new TreeDifference(ltree.fullRange.left, ltree.fullRange.right, (byte)0);
 
-        byte[] lhash = ltree.hash(active);
-        byte[] rhash = rtree.hash(active);
+        Hashable lnode = ltree.find(active);
+        Hashable rnode = rtree.find(active);
+        byte[] lhash = lnode.hash();
+        byte[] rhash = rnode.hash();
+        active.setSize(lnode.sizeOfRange(), rnode.sizeOfRange());
 
         if (lhash != null && rhash != null && !Arrays.equals(lhash, rhash))
         {
@@ -256,14 +260,19 @@ public class MerkleTree implements Serializable
             return CONSISTENT;
 
         Token midpoint = ltree.partitioner().midpoint(active.left, active.right);
-        TreeRange left = new TreeRange(null, active.left, midpoint, inc(active.depth), null);
-        TreeRange right = new TreeRange(null, midpoint, active.right, inc(active.depth), null);
-        byte[] lhash;
-        byte[] rhash;
+        TreeDifference left = new TreeDifference(active.left, midpoint, inc(active.depth));
+        TreeDifference right = new TreeDifference(midpoint, active.right, inc(active.depth));
+        byte[] lhash, rhash;
+        Hashable lnode, rnode;
 
         // see if we should recurse left
-        lhash = ltree.hash(left);
-        rhash = rtree.hash(left);
+        lnode = ltree.find(left);
+        rnode = rtree.find(left);
+        lhash = lnode.hash();
+        rhash = rnode.hash();
+        left.setSize(lnode.sizeOfRange(), rnode.sizeOfRange());
+        left.setRows(lnode.rowsInRange(), rnode.rowsInRange());
+
         int ldiff = CONSISTENT;
         boolean lreso = lhash != null && rhash != null;
         if (lreso && !Arrays.equals(lhash, rhash))
@@ -271,10 +280,14 @@ public class MerkleTree implements Serializable
         else if (!lreso)
             ldiff = FULLY_INCONSISTENT;
 
-
         // see if we should recurse right
-        lhash = ltree.hash(right);
-        rhash = rtree.hash(right);
+        lnode = ltree.find(right);
+        rnode = rtree.find(right);
+        lhash = lnode.hash();
+        rhash = rnode.hash();
+        right.setSize(lnode.sizeOfRange(), rnode.sizeOfRange());
+        right.setRows(lnode.rowsInRange(), rnode.rowsInRange());
+
         int rdiff = CONSISTENT;
         boolean rreso = lhash != null && rhash != null;
         if (rreso && !Arrays.equals(lhash, rhash))
@@ -363,54 +376,57 @@ public class MerkleTree implements Serializable
      */
     public byte[] hash(Range<Token> range)
     {
+        return find(range).hash();
+    }
+
+    /**
+     * Find the {@link Hashable} node that matches the given {@code range}.
+     *
+     * @param range Range to find
+     * @return {@link Hashable} found. If nothing found, return {@link Leaf} with null hash.
+     */
+    private Hashable find(Range<Token> range)
+    {
         try
         {
-            return hashHelper(root, new Range<Token>(fullRange.left, fullRange.right), range);
+            return findHelper(root, new Range<Token>(fullRange.left, fullRange.right), range);
         }
         catch (StopRecursion e)
         {
-            return null;
+            return new Leaf();
         }
     }
 
     /**
      * @throws StopRecursion If no match could be found for the range.
      */
-    private byte[] hashHelper(Hashable hashable, Range<Token> active, Range<Token> range) throws StopRecursion
+    private Hashable findHelper(Hashable current, Range<Token> activeRange, Range<Token> find) throws StopRecursion
     {
-        if (hashable instanceof Leaf)
+        if (current instanceof Leaf)
         {
-            if (!range.contains(active))
+            if (!find.contains(activeRange))
                 // we are not fully contained in this range!
                 throw new StopRecursion.BadRange();
-            return hashable.hash();
+            return current;
         }
         // else: node.
 
-        Inner node = (Inner)hashable;
-        Range<Token> leftactive = new Range<Token>(active.left, node.token);
-        Range<Token> rightactive = new Range<Token>(node.token, active.right);
+        Inner node = (Inner)current;
+        Range<Token> leftRange = new Range<Token>(activeRange.left, node.token);
+        Range<Token> rightRange = new Range<Token>(node.token, activeRange.right);
 
-        if (range.contains(active))
-        {
+        if (find.contains(activeRange))
             // this node is fully contained in the range
-            if (node.hash() != null)
-                // we had a cached value
-                return node.hash();
-            // continue recursing to hash our children
-            byte[] lhash = hashHelper(node.lchild(), leftactive, range);
-            byte[] rhash = hashHelper(node.rchild(), rightactive, range);
-            // cache the computed value (even if it is null)
-            node.hash(lhash, rhash);
-            return node.hash();
-        } // else: one of our children contains the range
-
-        if (leftactive.contains(range))
+            return node.calc();
+
+        // else: one of our children contains the range
+
+        if (leftRange.contains(find))
             // left child contains/matches the range
-            return hashHelper(node.lchild, leftactive, range);
-        else if (rightactive.contains(range))
+            return findHelper(node.lchild, leftRange, find);
+        else if (rightRange.contains(find))
             // right child contains/matches the range
-            return hashHelper(node.rchild, rightactive, range);
+            return findHelper(node.rchild, rightRange, find);
         else
             throw new StopRecursion.BadRange();
     }
@@ -479,6 +495,26 @@ public class MerkleTree implements Serializable
         return new TreeRangeIterator(this);
     }
 
+    public EstimatedHistogram histogramOfRowSizePerLeaf()
+    {
+        HistogramBuilder histbuild = new HistogramBuilder();
+        for (TreeRange range : new TreeRangeIterator(this))
+        {
+            histbuild.add(range.hashable.sizeOfRange);
+        }
+        return histbuild.buildWithStdevRangesAroundMean();
+    }
+
+    public EstimatedHistogram histogramOfRowCountPerLeaf()
+    {
+        HistogramBuilder histbuild = new HistogramBuilder();
+        for (TreeRange range : new TreeRangeIterator(this))
+        {
+            histbuild.add(range.hashable.rowsInRange);
+        }
+        return histbuild.buildWithStdevRangesAroundMean();
+    }
+
     @Override
     public String toString()
     {
@@ -489,6 +525,59 @@ public class MerkleTree implements Serializable
         return buff.toString();
     }
 
+    public static class TreeDifference extends TreeRange
+    {
+        private static final long serialVersionUID = 6363654174549968183L;
+
+        private long sizeOnLeft;
+        private long sizeOnRight;
+        private long rowsOnLeft;
+        private long rowsOnRight;
+
+        void setSize(long sizeOnLeft, long sizeOnRight)
+        {
+            this.sizeOnLeft = sizeOnLeft;
+            this.sizeOnRight = sizeOnRight;
+        }
+
+        void setRows(long rowsOnLeft, long rowsOnRight)
+        {
+            this.rowsOnLeft = rowsOnLeft;
+            this.rowsOnRight = rowsOnRight;
+        }
+
+        public long sizeOnLeft()
+        {
+            return sizeOnLeft;
+        }
+
+        public long sizeOnRight()
+        {
+            return sizeOnRight;
+        }
+
+        public long rowsOnLeft()
+        {
+            return rowsOnLeft;
+        }
+
+        public long rowsOnRight()
+        {
+            return rowsOnRight;
+        }
+
+        public TreeDifference(Token left, Token right, byte depth)
+        {
+            super(null, left, right, depth, null);
+        }
+
+        public long totalRows()
+        {
+            return rowsOnLeft + rowsOnRight;
+        }
+
+    }
+
     /**
      * The public interface to a range in the tree.
      *
@@ -531,7 +620,16 @@ public class MerkleTree implements Serializable
             assert tree != null : "Not intended for modification!";
             assert hashable instanceof Leaf;
 
-            hashable.addHash(entry.hash);
+            hashable.addHash(entry.hash, entry.size);
+        }
+
+        public void ensureHashInitialised()
+        {
+            assert tree != null : "Not intended for modification!";
+            assert hashable instanceof Leaf;
+
+            if (hashable.hash == null)
+                hashable.hash = EMPTY_HASH;
         }
 
         public void addAll(Iterator<RowHash> entries)
@@ -662,6 +760,21 @@ public class MerkleTree implements Serializable
             rchild = child;
         }
 
+        Hashable calc()
+        {
+            if (hash == null)
+            {
+                // hash and size haven't been calculated; calc children then compute
+                Hashable lnode = lchild.calc();
+                Hashable rnode = rchild.calc();
+                // cache the computed value
+                hash(lnode.hash, rnode.hash);
+                sizeOfRange = lnode.sizeOfRange + rnode.sizeOfRange;
+                rowsInRange = lnode.rowsInRange + rnode.rowsInRange;
+            }
+            return this;
+        }
+
         /**
          * Recursive toString.
          */
@@ -729,12 +842,12 @@ public class MerkleTree implements Serializable
             public long serializedSize(Inner inner, int version)
             {
                 int size = inner.hash == null
-                         ? TypeSizes.NATIVE.sizeof(-1)
-                         : TypeSizes.NATIVE.sizeof(inner.hash().length) + inner.hash().length;
+                ? TypeSizes.NATIVE.sizeof(-1)
+                        : TypeSizes.NATIVE.sizeof(inner.hash().length) + inner.hash().length;
 
                 size += Token.serializer.serializedSize(inner.token, TypeSizes.NATIVE)
-                        + Hashable.serializer.serializedSize(inner.lchild, version)
-                        + Hashable.serializer.serializedSize(inner.rchild, version);
+                + Hashable.serializer.serializedSize(inner.lchild, version)
+                + Hashable.serializer.serializedSize(inner.rchild, version);
                 return size;
             }
         }
@@ -768,11 +881,6 @@ public class MerkleTree implements Serializable
             super(hash);
         }
 
-        public Leaf(byte[] lefthash, byte[] righthash)
-        {
-            super(Hashable.binaryHash(lefthash, righthash));
-        }
-
         public void toString(StringBuilder buff, int maxdepth)
         {
             buff.append(toString());
@@ -826,16 +934,18 @@ public class MerkleTree implements Serializable
     {
         public final Token token;
         public final byte[] hash;
-        public RowHash(Token token, byte[] hash)
+        public final long size;
+        public RowHash(Token token, byte[] hash, long size)
         {
             this.token = token;
             this.hash  = hash;
+            this.size = size;
         }
 
         @Override
         public String toString()
         {
-            return "#<RowHash " + token + " " + Hashable.toString(hash) + ">";
+            return "#<RowHash " + token + " " + Hashable.toString(hash) + " @ " + size + " bytes>";
         }
     }
 
@@ -848,6 +958,8 @@ public class MerkleTree implements Serializable
         private static final IVersionedSerializer<Hashable> serializer = new HashableSerializer();
 
         protected byte[] hash;
+        protected long sizeOfRange;
+        protected long rowsInRange;
 
         protected Hashable(byte[] hash)
         {
@@ -859,11 +971,26 @@ public class MerkleTree implements Serializable
             return hash;
         }
 
+        public long sizeOfRange()
+        {
+            return sizeOfRange;
+        }
+
+        public long rowsInRange()
+        {
+            return rowsInRange;
+        }
+
         void hash(byte[] hash)
         {
             this.hash = hash;
         }
 
+        Hashable calc()
+        {
+            return this;
+        }
+
         /**
          * Sets the value of this hash to binaryHash of its children.
          * @param lefthash Hash of left child.
@@ -878,12 +1005,14 @@ public class MerkleTree implements Serializable
          * Mixes the given value into our hash. If our hash is null,
          * our hash will become the given value.
          */
-        void addHash(byte[] righthash)
+        void addHash(byte[] righthash, long sizeOfRow)
         {
             if (hash == null)
                 hash = righthash;
             else
                 hash = binaryHash(hash, righthash);
+            this.sizeOfRange += sizeOfRow;
+            this.rowsInRange += 1;
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/18f0234b/test/unit/org/apache/cassandra/repair/DifferencerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/DifferencerTest.java b/test/unit/org/apache/cassandra/repair/DifferencerTest.java
index 2502620..3f259f2 100644
--- a/test/unit/org/apache/cassandra/repair/DifferencerTest.java
+++ b/test/unit/org/apache/cassandra/repair/DifferencerTest.java
@@ -132,7 +132,7 @@ public class DifferencerTest extends SchemaLoader
         tree.init();
         for (MerkleTree.TreeRange r : tree.invalids())
         {
-            r.addHash(Validator.EMPTY_ROW);
+            r.ensureHashInitialised();
         }
         return tree;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/18f0234b/test/unit/org/apache/cassandra/utils/HistogramBuilderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/HistogramBuilderTest.java b/test/unit/org/apache/cassandra/utils/HistogramBuilderTest.java
new file mode 100644
index 0000000..dfceaf3
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/HistogramBuilderTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.utils;
+
+import java.util.concurrent.atomic.AtomicLongArray;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class HistogramBuilderTest
+{
+
+    @Test
+    public void testStdevEmpty()
+    {
+        EstimatedHistogram hist = new HistogramBuilder().buildWithStdevRangesAroundMean();
+        assertArrayEquals(new long[] { }, hist.getBucketOffsets());
+        assertArrayEquals(new long[] { 0 }, toArray(hist.buckets));
+    }
+
+    @Test
+    public void testStdevSingletonRanges()
+    {
+        EstimatedHistogram hist;
+        hist = new HistogramBuilder(new long[] { 5, 5, 5, 5, 5 }).buildWithStdevRangesAroundMean();
+        assertArrayEquals(new long[] { 4, 5 }, hist.getBucketOffsets());
+        assertArrayEquals(new long[] { 0, 5, 0 }, toArray(hist.buckets));
+        // should behave exactly the same for negative numbers
+        hist = new HistogramBuilder(new long[] { -1 }).buildWithStdevRangesAroundMean();
+        assertArrayEquals(new long[] { -2, -1 }, hist.getBucketOffsets());
+        assertArrayEquals(new long[] { 0, 1, 0 }, toArray(hist.buckets));
+    }
+
+    @Test
+    public void testStdevNearZeroStdev()
+    {
+        EstimatedHistogram hist;
+        long[] vals = new long[100000];
+        vals[0] = 99;
+        vals[1] = 101;
+        for (int i = 2 ; i < vals.length ; i++)
+            vals[i] = 100;
+        hist = new HistogramBuilder(vals).buildWithStdevRangesAroundMean();
+        assertArrayEquals(new long[] { 98, 99, 100, 101 }, hist.getBucketOffsets());
+        assertArrayEquals(new long[] { 0, 1, vals.length - 2, 1, 0 }, toArray(hist.buckets));
+    }
+
+    @Test
+    public void testStdev()
+    {
+        long[] vals;
+        EstimatedHistogram hist;
+        vals = new long[] { -10, -3, -2, -2, -1, -1, -1, -1, -1, -1, -1, 0, 1, 1, 1, 1, 1, 1, 1, 2, 2, 3, 10 };
+        hist = new HistogramBuilder(vals).buildWithStdevRangesAroundMean(2);
+        assertArrayEquals(new long[] { -11, -6, -3, 0, 3, 6, 10 }, hist.getBucketOffsets());
+        assertArrayEquals(new long[] { 0, 1, 1, 10, 10, 0, 1, 0 }, toArray(hist.buckets));
+    }
+
+    private static long[] toArray(AtomicLongArray a)
+    {
+        final long[] r = new long[a.length()];
+        for (int i = 0 ; i < r.length ; i++)
+            r[i] = a.get(i);
+        return r;
+    }
+
+    @Test
+    public void testStdevLargeNumbers()
+    {
+        long[] vals;
+        EstimatedHistogram hist;
+        vals = new long[100000];
+        for (int i = 0 ; i < vals.length ; i++)
+        {
+            if (i < vals.length * 0.6f)
+                vals[i] = 60;
+            else if (i < vals.length * 0.8f)
+                vals[i] = 120;
+            else if (i < vals.length * 0.9f)
+                vals[i] = 180;
+            else if (i < vals.length * 0.95f)
+                vals[i] = 240;
+            else if (i < vals.length * 0.98f)
+                vals[i] = 320;
+            else
+                vals[i] = 1000;
+        }
+        hist = new HistogramBuilder(vals).buildWithStdevRangesAroundMean(2);
+        assertArrayEquals(new long[] { 59, 120, 260, 400, 1000 }, hist.getBucketOffsets());
+        assertArrayEquals(new long[] { 0, 80000, 15000, 3000, 2000, 0 }, toArray(hist.buckets));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/18f0234b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
index 70fe3a0..2b459c6 100644
--- a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
+++ b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
@@ -25,16 +25,15 @@ import com.google.common.collect.AbstractIterator;
 import com.google.common.io.ByteArrayDataInput;
 import com.google.common.io.ByteArrayDataOutput;
 import com.google.common.io.ByteStreams;
-import org.apache.hadoop.hdfs.server.common.Storage;
+
 import org.junit.Before;
 import org.junit.Test;
-
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.MerkleTree.Hashable;
 import org.apache.cassandra.utils.MerkleTree.RowHash;
+import org.apache.cassandra.utils.MerkleTree.TreeDifference;
 import org.apache.cassandra.utils.MerkleTree.TreeRange;
 import org.apache.cassandra.utils.MerkleTree.TreeRangeIterator;
 
@@ -322,7 +321,7 @@ public class MerkleTreeTest
         // validate the tree
         TreeRangeIterator ranges = mt.invalids();
         for (TreeRange range : ranges)
-            range.addHash(new RowHash(range.right, new byte[0]));
+            range.addHash(new RowHash(range.right, new byte[0], 0));
 
         assert mt.hash(new Range<>(tok(-1), tok(-1))) != null :
             "Could not hash tree " + mt;
@@ -491,7 +490,7 @@ public class MerkleTreeTest
         public RowHash computeNext()
         {
             if (tokens.hasNext())
-                return new RowHash(tokens.next(), DUMMY);
+                return new RowHash(tokens.next(), DUMMY, DUMMY.length);
             return endOfData();
         }
     }