You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2014/11/25 21:28:54 UTC

[01/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Repository: hbase
Updated Branches:
  refs/heads/HBASE-12259 [created] eca32aa4a


http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRMapReader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRMapReader.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRMapReader.java
new file mode 100644
index 0000000..4e7a798
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRMapReader.java
@@ -0,0 +1,102 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.security.NoSuchAlgorithmException;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestRMapReader {
+  static String base = "file:/some/rmap.json";
+
+  RMapReader reader;
+  URI current, next, v2, v3;
+
+  @Test
+  public void shouldReturnVersionedURI() throws URISyntaxException {
+    assertEquals(v2, RMapReader.getVersionedURI(base, 2));
+  }
+
+  @Test
+  public void shouldReturnCurrentVersionedURI() throws URISyntaxException {
+    assertEquals(current, RMapReader.getVersionedURI(base, RMapReader.CURRENT));
+  }
+
+  @Test
+  public void shouldReturnNextVersionedURI() throws URISyntaxException {
+    assertEquals(next, RMapReader.getVersionedURI(base, RMapReader.NEXT));
+  }
+
+  @Test
+  public void shouldReturnAbsoluteVersion() throws URISyntaxException {
+    assertEquals(2, RMapReader.getVersion(v2));
+  }
+
+  @Test
+  public void shouldReturnCurrentSymbolicVersion() throws URISyntaxException {
+    assertEquals(RMapReader.CURRENT, RMapReader.getVersion(current));
+  }
+
+  @Test
+  public void shouldReturnNextSymbolicVersion() throws URISyntaxException {
+    assertEquals(RMapReader.NEXT, RMapReader.getVersion(next));
+  }
+
+  @Test
+  public void shouldReturnUnknownSymbolicVersion() throws URISyntaxException {
+    assertEquals(RMapReader.UNKNOWN,
+            RMapReader.getVersion(new URI(base + "?version=FOO")));
+  }
+
+  @Test
+  public void shouldResolveSymbolicVersionAndReturnRMap()
+          throws URISyntaxException, IOException, RMapException {
+    // Stub the abstract methods and forward call to RMapReader.readRMap().
+    // This is a bit frowned upon.
+    when(reader.resolveSymbolicVersion(current)).thenReturn(v2);
+    when(reader.readRMapAsString(v2)).thenReturn("{}");
+    when(reader.readRMap(current)).thenCallRealMethod();
+
+    RMapJSON rmap = reader.readRMap(current);
+    assertEquals(v2, rmap.uri);
+    assertEquals("{}", rmap.rmap.toString());
+  }
+
+  @Test
+  public void shouldReturnMD5HashAsHex() throws NoSuchAlgorithmException {
+    assertEquals("99914b932bd37a50b983c5e7c90ae93b",
+            RMapReader.getSignature("{}"));
+  }
+
+  @Test
+  public void shouldReturnCurrentVersion() throws URISyntaxException {
+    when(reader.resolveSymbolicVersion(current)).thenReturn(v2);
+    when(reader.getCurrentVersion(base)).thenCallRealMethod();
+
+    assertEquals(2, reader.getCurrentVersion(base));
+  }
+
+  @Test
+  public void shoudlReturnNextVersion() throws URISyntaxException {
+    when(reader.resolveSymbolicVersion(next)).thenReturn(v3);
+    when(reader.getNextVersion(base)).thenCallRealMethod();
+
+    assertEquals(3, reader.getNextVersion(base));
+  }
+
+  @Before
+  public void setUp() throws URISyntaxException, IOException, RMapException {
+    reader = mock(RMapReader.class);
+    // URIs can not be created outside of the method.
+    current = RMapReader.getVersionedURI(base, RMapReader.CURRENT);
+    next = RMapReader.getVersionedURI(base, RMapReader.NEXT);
+    v2 = RMapReader.getVersionedURI(base, 2);
+    v3 = RMapReader.getVersionedURI(base, 3);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRegionLocator.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRegionLocator.java
new file mode 100644
index 0000000..3a93aed
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRegionLocator.java
@@ -0,0 +1,180 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.json.JSONException;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestRegionLocator {
+  Configuration conf;
+  RegionLocator locator;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = HBaseConfiguration.create();
+    conf.set(HConstants.RMAP_SUBSCRIPTION,
+            getClass().getResource("rmap.json").toURI().toString());
+    conf.set(HConstants.HYDRABASE_DCNAME, "DUMMYCLUSTER1");
+
+    locator = new RegionLocator(conf);
+    locator.refresh();
+  }
+
+  @Test
+  public void testGetRegionsForServer() throws IOException, JSONException {
+    List<HRegionInfo> regions = locator.getRegionsForServer(
+            new HServerAddress("10.159.9.45:60020"));
+    assertNotNull(regions);
+    assertEquals(28, regions.size());
+    for (HRegionInfo region : regions) {
+      HTableDescriptor table = region.getTableDesc();
+      if ("VerificationTest_DummyTable".equals(table.getNameAsString())) {
+        assertEquals(3, table.getColumnFamilies().length);
+      } else if ("wutang".equals(table.getNameAsString())) {
+        assertEquals(5, table.getColumnFamilies().length);
+      }
+    }
+  }
+
+  @Test
+  public void testGetRegionsForTableWithPeersInMultipleCells() throws Exception {
+    List<HRegionInfo> regions = locator.getRegionsForTable(
+            Bytes.toBytes("RPCBenchmarkingTable"));
+    assertNotNull(regions);
+    assertEquals(3, regions.size());
+    for (HRegionInfo region : regions) {
+      if ("2aaaaaaa".equals(Bytes.toString(region.getStartKey()))) {
+        Map<String, Map<HServerAddress, Integer>> expectedPeers = new HashMap<>();
+        Map<HServerAddress, Integer> dummyCluster1 = new HashMap<>();
+        dummyCluster1.put(new HServerAddress("10.159.9.45:60020"), 1);
+        dummyCluster1.put(new HServerAddress("10.159.9.47:60020"), 2);
+        expectedPeers.put("DUMMYCLUSTER1", dummyCluster1);
+        Map<HServerAddress, Integer> dummyCluster2 = new HashMap<>();
+        dummyCluster2.put(new HServerAddress("10.159.9.42:60020"), 3);
+        dummyCluster2.put(new HServerAddress("10.159.9.43:60020"), 4);
+        expectedPeers.put("DUMMYCLUSTER2", dummyCluster2);
+        Map<HServerAddress, Integer> dummyCluster3 = new HashMap<>();
+        dummyCluster3.put(new HServerAddress("10.159.9.49:60020"), 5);
+        expectedPeers.put("DUMMYCLUSTER3", dummyCluster3);
+        assertEquals(expectedPeers, region.getPeers());
+        Map<HServerAddress, String> peersWithCluster = region
+                .getPeersWithCluster();
+        assertEquals("DUMMYCLUSTER1",
+                peersWithCluster.get(new HServerAddress("10.159.9.45:60020")));
+        assertEquals("DUMMYCLUSTER1",
+                peersWithCluster.get(new HServerAddress("10.159.9.47:60020")));
+        assertEquals("DUMMYCLUSTER2",
+                peersWithCluster.get(new HServerAddress("10.159.9.42:60020")));
+        assertEquals("DUMMYCLUSTER2",
+                peersWithCluster.get(new HServerAddress("10.159.9.43:60020")));
+        assertEquals("DUMMYCLUSTER3",
+                peersWithCluster.get(new HServerAddress("10.159.9.49:60020")));
+
+      }
+    }
+  }
+
+  @Test
+  public void testLocateRegionSingleRegion() throws IOException, JSONException {
+    byte[] row = new byte[1];
+    for (int i = 0; i < 256; ++i) {
+      row[0] = (byte) i;
+      HRegionInfo region = locator.findRegion(Bytes.toBytes("wutang"), row);
+      assertNotNull(region);
+      assertEquals(5, region.getTableDesc().getColumnFamilies().length);
+      assertEquals("b2696f3faa4bd5767f2800bbcc2687c0", region.getEncodedName());
+      assertEquals("", Bytes.toString(region.getStartKey()));
+      assertEquals("", Bytes.toString(region.getEndKey()));
+      assertEquals(1370994021138L, region.getRegionId());
+      assertTrue(region.getFavoredNodes() == null ||
+        region.getFavoredNodes().length == 0);
+      assertEquals(1, region.getPeersWithRank().size());
+      assertEquals(new Integer(1),
+              region.getPeersWithRank().get(new HServerAddress("10.159.9.45:60020")));
+      assertEquals(1, region.getPeers().size());
+      assertEquals(1, region.getPeers().get("DUMMYCLUSTER1").size());
+      assertEquals(new Integer(1), region.getPeers().get("DUMMYCLUSTER1")
+              .get(new HServerAddress("10.159.9.45:60020")));
+
+    }
+  }
+
+  @Test
+  public void testLocateRegion() throws IOException, JSONException {
+    HRegionInfo region;
+
+    // Test first region
+    region = locator.findRegion(Bytes.toBytes("loadtest"),
+            Bytes.toBytes("0"));
+    assertNotNull(region);
+    assertEquals(1, region.getTableDesc().getColumnFamilies().length);
+    assertEquals("", Bytes.toString(region.getStartKey()));
+    assertEquals("11111111", Bytes.toString(region.getEndKey()));
+
+    // Test last region
+    region = locator.findRegion(Bytes.toBytes("loadtest"),
+            Bytes.toBytes("f"));
+    assertNotNull(region);
+    assertEquals(1, region.getTableDesc().getColumnFamilies().length);
+    assertEquals("eeeeeeee", Bytes.toString(region.getStartKey()));
+    assertEquals("", Bytes.toString(region.getEndKey()));
+
+    // Test regions in the middle
+    region = locator.findRegion(Bytes.toBytes("loadtest"),
+            Bytes.toBytes("9"));
+    assertNotNull(region);
+    assertEquals(1, region.getTableDesc().getColumnFamilies().length);
+    assertEquals("88888888", Bytes.toString(region.getStartKey()));
+    assertEquals("99999999", Bytes.toString(region.getEndKey()));
+
+    region = locator.findRegion(Bytes.toBytes("loadtest"),
+            Bytes.toBytes("9abcdefg"));
+    assertNotNull(region);
+    assertEquals(1, region.getTableDesc().getColumnFamilies().length);
+    assertEquals("99999999", Bytes.toString(region.getStartKey()));
+    assertEquals("aaaaaaaa", Bytes.toString(region.getEndKey()));
+
+    // Test boundaries
+    region = locator.findRegion(Bytes.toBytes("loadtest"),
+            Bytes.toBytes("66666666"));
+    assertNotNull(region);
+    assertEquals(1, region.getTableDesc().getColumnFamilies().length);
+    assertEquals("66666666", Bytes.toString(region.getStartKey()));
+    assertEquals("77777777", Bytes.toString(region.getEndKey()));
+
+    region = locator.findRegion(Bytes.toBytes("loadtest"),
+            Bytes.toBytes("cccccccc0"));
+    assertNotNull(region);
+    assertEquals(1, region.getTableDesc().getColumnFamilies().length);
+    assertEquals("cccccccc", Bytes.toString(region.getStartKey()));
+    assertEquals("dddddddd", Bytes.toString(region.getEndKey()));
+
+    region = locator.findRegion(Bytes.toBytes("loadtest"),
+            Bytes.toBytes("2222222"));
+    assertNotNull(region);
+    assertEquals(1, region.getTableDesc().getColumnFamilies().length);
+    assertEquals("11111111", Bytes.toString(region.getStartKey()));
+    assertEquals("22222222", Bytes.toString(region.getEndKey()));
+  }
+
+  @Test
+  public void shouldReturnAllRegionsGroupedByTable() {
+    List<List<HRegionInfo>> regionsByTable =
+            locator.getAllRegionsGroupByTable();
+    assertTrue(regionsByTable.size() > 0);
+    for (List<HRegionInfo> regions : regionsByTable) {
+      assertTrue(regions.size() > 0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/resources/org/apache/hadoop/hbase/consensus/rmap/rmap.json
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/resources/org/apache/hadoop/hbase/consensus/rmap/rmap.json b/hbase-consensus/src/test/resources/org/apache/hadoop/hbase/consensus/rmap/rmap.json
new file mode 100644
index 0000000..0d88cd9
--- /dev/null
+++ b/hbase-consensus/src/test/resources/org/apache/hadoop/hbase/consensus/rmap/rmap.json
@@ -0,0 +1,1228 @@
+{"tables": {
+    "-ROOT-": {
+        "IS_META": "true",
+        "IS_ROOT": "true",
+        "families": {"info": {
+            "BLOCKCACHE": "true",
+            "BLOCKSIZE": "8192",
+            "BLOOMFILTER": "NONE",
+            "BLOOMFILTER_ERRORRATE": "0.01",
+            "COMPRESSION": "NONE",
+            "DATA_BLOCK_ENCODING": "NONE",
+            "ENCODE_ON_DISK": "true",
+            "IN_MEMORY": "true",
+            "REPLICATION_SCOPE": "0",
+            "TTL": "2147483647",
+            "VERSIONS": "10"
+        }},
+        "regions": [{
+            "encoded": "70236052",
+            "end_key": "",
+            "favored_nodes": {"DUMMYCLUSTER1": []},
+            "id": 0,
+            "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+            "start_key": ""
+        }]
+    },
+    ".META.": {
+        "IS_META": "true",
+        "IS_ROOT": "false",
+        "families": {
+            "historian": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "8192",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "604800",
+                "VERSIONS": "2147483647"
+            },
+            "info": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "8192",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "true",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "10"
+            }
+        },
+        "regions": [{
+            "encoded": "1028785192",
+            "end_key": "",
+            "favored_nodes": {"DUMMYCLUSTER1": []},
+            "id": 1,
+            "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+            "start_key": ""
+        }]
+    },
+    "RPCBenchmarkingTable": {
+        "IS_META": "false",
+        "IS_ROOT": "false",
+        "families": {"cf": {
+            "BLOCKCACHE": "true",
+            "BLOCKSIZE": "65536",
+            "BLOOMFILTER": "NONE",
+            "BLOOMFILTER_ERRORRATE": "0.01",
+            "COMPRESSION": "NONE",
+            "DATA_BLOCK_ENCODING": "NONE",
+            "ENCODE_ON_DISK": "true",
+            "IN_MEMORY": "false",
+            "REPLICATION_SCOPE": "0",
+            "TTL": "2147483647",
+            "VERSIONS": "2147483647"
+        }},
+        "regions": [
+            {
+                "encoded": "aeeb54dc6fbca609443bd35796b59da5",
+                "end_key": "2aaaaaaa",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1373324048180,
+                "peers": {"DUMMYCLUSTER1": [
+                    "10.159.9.41:60020:1",
+                    "10.159.9.45:60020:2",
+                    "10.159.9.47:60020:3"
+                ]},
+                "start_key": ""
+            },
+            {
+                "encoded": "15a13ba77b809e76a4e27d30e67c1969",
+                "end_key": "55555554",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1373324048180,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1", "10.159.9.47:60020:2"],
+                    "DUMMYCLUSTER2": ["10.159.9.42:60020:3", "10.159.9.43:60020:4"],
+                    "DUMMYCLUSTER3": ["10.159.9.49:60020:5"]},
+                "start_key": "2aaaaaaa"
+            },
+            {
+                "encoded": "ad811cadfcbb0d756d71350aac27b83d",
+                "end_key": "",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1373324048180,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "55555554"
+            }
+        ]
+    },
+    "VerificationTest_DummyTable": {
+        "IS_META": "false",
+        "IS_ROOT": "false",
+        "families": {
+            "ActionLog": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "ROW",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "2147483647"
+            },
+            "IndexSnapshot": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "ROWCOL",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "GZ",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "1"
+            },
+            "Snapshot": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "LZO",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "1"
+            }
+        },
+        "regions": [
+            {
+                "encoded": "bc878af16710f710372057e8432c0a10",
+                "end_key": "06666666",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509227,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": ""
+            },
+            {
+                "encoded": "a5cf8ffbd2ba3841694568e5f193c89d",
+                "end_key": "0ccccccc",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509232,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "06666666"
+            },
+            {
+                "encoded": "4334848cde16d1dc4e31f7bef82f134d",
+                "end_key": "11111110",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60021",
+                    "10.159.9.47:60021",
+                    "10.159.9.45:60021"
+                ]},
+                "id": 1378513686287,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "08888888"
+            },
+            {
+                "encoded": "30fa9e953780cce299a0aa8c28862188",
+                "end_key": "13333332",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509232,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "0ccccccc"
+            },
+            {
+                "encoded": "34dc2ca9659fe7b004c64ed67a8d508e",
+                "end_key": "19999998",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60021",
+                    "10.159.9.45:60021",
+                    "10.159.9.49:60021"
+                ]},
+                "id": 1378513686287,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "11111110"
+            },
+            {
+                "encoded": "27e06426ad2c17ee1d30e24e0d99dbbd",
+                "end_key": "19999998",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509232,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "13333332"
+            },
+            {
+                "encoded": "a56cd0af1570fa7e4342066bc53e3c16",
+                "end_key": "1ffffffe",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509232,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "19999998"
+            },
+            {
+                "encoded": "e89172200b7c13ee0ddf5dd3d77e0b10",
+                "end_key": "26666664",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509232,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "1ffffffe"
+            },
+            {
+                "encoded": "8bc393423bd0ade41a21e302733071a4",
+                "end_key": "2aaaaaa8",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60021",
+                    "10.159.9.47:60021",
+                    "10.159.9.45:60021"
+                ]},
+                "id": 1378513686287,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "22222220"
+            },
+            {
+                "encoded": "a417333fff7b3b8eb4f21ef342366987",
+                "end_key": "2cccccca",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "26666664"
+            },
+            {
+                "encoded": "71f12b479a151a9125d2b65be6e05583",
+                "end_key": "33333330",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "2cccccca"
+            },
+            {
+                "encoded": "af28c78d0e444281b1bdb66346bca4d2",
+                "end_key": "39999996",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "33333330"
+            },
+            {
+                "encoded": "900cceec5f403669c5454104d71dc349",
+                "end_key": "3ffffffc",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "39999996"
+            },
+            {
+                "encoded": "c233cd158d7d1155accd4dc3c5ca8569",
+                "end_key": "44444440",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60021",
+                    "10.159.9.45:60021",
+                    "10.159.9.47:60021"
+                ]},
+                "id": 1378513686287,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "3bbbbbb8"
+            },
+            {
+                "encoded": "74814b1ab76d66fd498ab05a67834f2b",
+                "end_key": "46666662",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "3ffffffc"
+            },
+            {
+                "encoded": "04320c73cf436388f587eaf80cbb5a0f",
+                "end_key": "4cccccc8",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "46666662"
+            },
+            {
+                "encoded": "2b1c4b896cd6bcb73305f1cd7070b691",
+                "end_key": "5333332e",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "4cccccc8"
+            },
+            {
+                "encoded": "060adf717d5f2e484307243292144b62",
+                "end_key": "55555550",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60021",
+                    "10.159.9.47:60021",
+                    "10.159.9.49:60021"
+                ]},
+                "id": 1378513686287,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "4cccccc8"
+            },
+            {
+                "encoded": "8f7bfa2957fd885e13a10e1653604cf7",
+                "end_key": "59999994",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "5333332e"
+            },
+            {
+                "encoded": "ee4ae3dfea198e19607cef988593aaa6",
+                "end_key": "5dddddd8",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60021",
+                    "10.159.9.47:60021",
+                    "10.159.9.45:60021"
+                ]},
+                "id": 1378513686287,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "55555550"
+            },
+            {
+                "encoded": "149c3b21f88c7c966387f70129958e2a",
+                "end_key": "5ffffffa",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "59999994"
+            },
+            {
+                "encoded": "6da67b4a75ab96c3daedf06eb8b2d0f8",
+                "end_key": "66666660",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "5ffffffa"
+            },
+            {
+                "encoded": "1d8abb6d0e17ae817855f37baae97301",
+                "end_key": "6cccccc6",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "66666660"
+            },
+            {
+                "encoded": "0d370153e98f42edb877df4fd9b30a13",
+                "end_key": "6eeeeee8",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60021",
+                    "10.159.9.49:60021",
+                    "10.159.9.47:60021"
+                ]},
+                "id": 1378513686287,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "66666660"
+            },
+            {
+                "encoded": "7fc3f2e96163150764391524d78faabe",
+                "end_key": "7333332c",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509233,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "6cccccc6"
+            },
+            {
+                "encoded": "556fd79f7050054341b0334dd6737283",
+                "end_key": "77777770",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1378513686287,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "6eeeeee8"
+            },
+            {
+                "encoded": "2ff4557b7612ef3458e6ab66fa4ba863",
+                "end_key": "79999992",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509234,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "7333332c"
+            },
+            {
+                "encoded": "6f5d500b9c145dec4c6d10fc6a4dfac2",
+                "end_key": "",
+                "favored_nodes": {"DUMMYCLUSTER1": []},
+                "id": 1377570509234,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "79999992"
+            }
+        ]
+    },
+    "bench.GetFromMemory": {
+        "IS_META": "false",
+        "IS_ROOT": "false",
+        "families": {"cf1": {
+            "BLOCKCACHE": "true",
+            "BLOCKSIZE": "65536",
+            "BLOOMFILTER": "NONE",
+            "BLOOMFILTER_ERRORRATE": "0.01",
+            "COMPRESSION": "NONE",
+            "DATA_BLOCK_ENCODING": "NONE",
+            "ENCODE_ON_DISK": "true",
+            "IN_MEMORY": "false",
+            "REPLICATION_SCOPE": "0",
+            "TTL": "2147483647",
+            "VERSIONS": "1"
+        }},
+        "regions": [
+            {
+                "encoded": "752dc66912cb39f5eb7b92e451da18c2",
+                "end_key": "               33333",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985980,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": ""
+            },
+            {
+                "encoded": "bf8d5882d8d973664c9591270dbe00d2",
+                "end_key": "               66666",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "               33333"
+            },
+            {
+                "encoded": "de98e2f7e2113cc77dcd0a89bfa60386",
+                "end_key": "               99999",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "               66666"
+            },
+            {
+                "encoded": "2fc3cc5b061cfcf6bff6055c1952f262",
+                "end_key": "              133332",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "               99999"
+            },
+            {
+                "encoded": "025bc86eb8e79864051b1902b1c23e29",
+                "end_key": "              166665",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              133332"
+            },
+            {
+                "encoded": "8deaf56223f5e7fda030aa1519fc3fe8",
+                "end_key": "              199998",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              166665"
+            },
+            {
+                "encoded": "61fa640797f6544e6bb7ba3055df6188",
+                "end_key": "              233331",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              199998"
+            },
+            {
+                "encoded": "2e59595ca667b499f6ddca7a92413d68",
+                "end_key": "              266664",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              233331"
+            },
+            {
+                "encoded": "ed80a537f8e07b7cf11f7601d8ba4bc0",
+                "end_key": "              299997",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              266664"
+            },
+            {
+                "encoded": "99c56d734d7705a0f84a846cd0460023",
+                "end_key": "              333330",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              299997"
+            },
+            {
+                "encoded": "025af7c469e493d0f4d2e201445a7fa6",
+                "end_key": "              366663",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              333330"
+            },
+            {
+                "encoded": "fb88e2beda1b4190170b6010ec0f9269",
+                "end_key": "              399996",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              366663"
+            },
+            {
+                "encoded": "714b8f200f89bf1b4cd09a249d5b5cc5",
+                "end_key": "              433329",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              399996"
+            },
+            {
+                "encoded": "935a1096f947c835b23dfe7e61d23f02",
+                "end_key": "              466662",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              433329"
+            },
+            {
+                "encoded": "f1a67a99e5ea9e1a20c67b0b89b65e25",
+                "end_key": "              499995",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              466662"
+            },
+            {
+                "encoded": "263a7b72b53cf31b61b13b26d3f5b53f",
+                "end_key": "              533328",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              499995"
+            },
+            {
+                "encoded": "bfdea63d2057dacf67f317be21a12a2e",
+                "end_key": "              566661",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              533328"
+            },
+            {
+                "encoded": "8b19b5178def407317fa730dbf9c71ad",
+                "end_key": "              599994",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              566661"
+            },
+            {
+                "encoded": "11b8638a9d8bcc2e63a2d46d87e6cbce",
+                "end_key": "              633327",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              599994"
+            },
+            {
+                "encoded": "9f05e9792db879d4dad20b9801c77b15",
+                "end_key": "              666660",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              633327"
+            },
+            {
+                "encoded": "f8c8ad4bda7ea52e186163c9e7304002",
+                "end_key": "              699993",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985981,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              666660"
+            },
+            {
+                "encoded": "903380d234c5b63e41ac304bcd98a282",
+                "end_key": "              733326",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              699993"
+            },
+            {
+                "encoded": "1a9a6228a63c1dc7b906c8007a02030a",
+                "end_key": "              766659",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              733326"
+            },
+            {
+                "encoded": "8ba67ecc37405b87fa25f4df994db676",
+                "end_key": "              799992",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              766659"
+            },
+            {
+                "encoded": "3d5f4c998f4575408b7c886cfb140726",
+                "end_key": "              833325",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              799992"
+            },
+            {
+                "encoded": "418567521b700188c29da48ec2f16595",
+                "end_key": "              866658",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              833325"
+            },
+            {
+                "encoded": "47613d1d1b61b3213274a252560caf66",
+                "end_key": "              899991",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              866658"
+            },
+            {
+                "encoded": "c722a4fb626a2c97ed12cf79d0c28db5",
+                "end_key": "              933324",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              899991"
+            },
+            {
+                "encoded": "83d117cff870bcb726555d2a74da2883",
+                "end_key": "              966657",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              933324"
+            },
+            {
+                "encoded": "01341957c6cab305fb9849939b80f1cc",
+                "end_key": "              999990",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              966657"
+            },
+            {
+                "encoded": "aeb74d8be75abed243fc44316773b4b8",
+                "end_key": "",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359146985982,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              999990"
+            }
+        ]
+    },
+    "bench.ScanFromMemoryPerf": {
+        "IS_META": "false",
+        "IS_ROOT": "false",
+        "families": {"cf1": {
+            "BLOCKCACHE": "true",
+            "BLOCKSIZE": "65536",
+            "BLOOMFILTER": "NONE",
+            "BLOOMFILTER_ERRORRATE": "0.01",
+            "COMPRESSION": "NONE",
+            "DATA_BLOCK_ENCODING": "NONE",
+            "ENCODE_ON_DISK": "true",
+            "IN_MEMORY": "false",
+            "REPLICATION_SCOPE": "0",
+            "TTL": "2147483647",
+            "VERSIONS": "1"
+        }},
+        "regions": [
+            {
+                "encoded": "b3f1bebd2e3ecc09c7bdb814df11ddec",
+                "end_key": "              333333",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1359106312755,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": ""
+            },
+            {
+                "encoded": "9a9a1365e0275602b5f6d31fd6868666",
+                "end_key": "              666666",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359106312755,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "              333333"
+            },
+            {
+                "encoded": "6cfd11340ab440f01d06fd7a12542d18",
+                "end_key": "              999999",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1359106312755,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "              666666"
+            },
+            {
+                "encoded": "c0608b0c4a8203fce212cf682d66260e",
+                "end_key": "",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1359106312755,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "              999999"
+            }
+        ]
+    },
+    "gt": {
+        "IS_META": "false",
+        "IS_ROOT": "false",
+        "families": {"f1": {
+            "BLOCKCACHE": "true",
+            "BLOCKSIZE": "65536",
+            "BLOOMFILTER": "NONE",
+            "BLOOMFILTER_ERRORRATE": "0.01",
+            "COMPRESSION": "NONE",
+            "DATA_BLOCK_ENCODING": "NONE",
+            "ENCODE_ON_DISK": "true",
+            "IN_MEMORY": "false",
+            "REPLICATION_SCOPE": "0",
+            "TTL": "2147483647",
+            "VERSIONS": "3"
+        }},
+        "regions": [{
+            "encoded": "1fbfc42f2ba0f38050dc621ea0fb91f9",
+            "end_key": "",
+            "favored_nodes": {"DUMMYCLUSTER1": []},
+            "id": 1370992737275,
+            "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+            "start_key": ""
+        }]
+    },
+    "loadtest": {
+        "IS_META": "false",
+        "IS_ROOT": "false",
+        "families": {"test_cf": {
+            "BLOCKCACHE": "true",
+            "BLOCKSIZE": "65536",
+            "BLOOMFILTER": "NONE",
+            "BLOOMFILTER_ERRORRATE": "0.01",
+            "COMPRESSION": "NONE",
+            "DATA_BLOCK_ENCODING": "NONE",
+            "ENCODE_ON_DISK": "true",
+            "IN_MEMORY": "false",
+            "REPLICATION_SCOPE": "0",
+            "TTL": "2147483647",
+            "VERSIONS": "3"
+        }},
+        "regions": [
+            {
+                "encoded": "381ace1cb168fa561386392433633209",
+                "end_key": "11111111",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": ""
+            },
+            {
+                "encoded": "a19df6afa13b268f5248b9f2fc959144",
+                "end_key": "22222222",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "11111111"
+            },
+            {
+                "encoded": "5dd62a32fb74d57048e244cba2dcf543",
+                "end_key": "33333333",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "22222222"
+            },
+            {
+                "encoded": "e5bf1a0ff241c51eeb624b2579c10663",
+                "end_key": "44444444",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "33333333"
+            },
+            {
+                "encoded": "3d315a9174c31c69f695e87374985f70",
+                "end_key": "55555555",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "44444444"
+            },
+            {
+                "encoded": "4bf2f50d674aa9453a5f50e1671aec08",
+                "end_key": "66666666",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "55555555"
+            },
+            {
+                "encoded": "1be6d637db0ff13542ace772aee370d9",
+                "end_key": "77777777",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "66666666"
+            },
+            {
+                "encoded": "982b616d487cb61189bb200c8804a43c",
+                "end_key": "88888888",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "77777777"
+            },
+            {
+                "encoded": "e8bd9c7f504c1ecc75b95712fd35e72f",
+                "end_key": "99999999",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "88888888"
+            },
+            {
+                "encoded": "f803f103c5077f2599872974d5e12f9c",
+                "end_key": "aaaaaaaa",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "99999999"
+            },
+            {
+                "encoded": "76ecef40ce061a3c89ea75f4a0fcc473",
+                "end_key": "bbbbbbbb",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "aaaaaaaa"
+            },
+            {
+                "encoded": "d392640f9d0d211f24c46e72ced124b1",
+                "end_key": "cccccccc",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "bbbbbbbb"
+            },
+            {
+                "encoded": "251e2f558469317bf341e09377534e20",
+                "end_key": "dddddddd",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+                "start_key": "cccccccc"
+            },
+            {
+                "encoded": "74eeb4a9b5a3a6e9a7ce998bec08244f",
+                "end_key": "eeeeeeee",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.49:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.45:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.49:60020:1"]},
+                "start_key": "dddddddd"
+            },
+            {
+                "encoded": "1a8ba16929bdb03b5ddacc7a2d433b15",
+                "end_key": "",
+                "favored_nodes": {"DUMMYCLUSTER1": [
+                    "10.159.9.45:60020",
+                    "10.159.9.47:60020",
+                    "10.159.9.49:60020"
+                ]},
+                "id": 1373318348662,
+                "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+                "start_key": "eeeeeeee"
+            }
+        ]
+    },
+    "t1": {
+        "IS_META": "false",
+        "IS_ROOT": "false",
+        "families": {
+            "f1": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "2"
+            },
+            "f2": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "5",
+                "VERSIONS": "3"
+            }
+        },
+        "regions": [{
+            "encoded": "c08c6360d57fd0d3aacac33955e2be7e",
+            "end_key": "",
+            "favored_nodes": {"DUMMYCLUSTER1": []},
+            "id": 1370366235604,
+            "peers": {"DUMMYCLUSTER1": ["10.159.9.47:60020:1"]},
+            "start_key": ""
+        }]
+    },
+    "wutang": {
+        "IS_META": "false",
+        "IS_ROOT": "false",
+        "families": {
+            "wtang1": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "3"
+            },
+            "wtang2": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "3"
+            },
+            "wtang3": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "3"
+            },
+            "wtang4": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "3"
+            },
+            "wtang5": {
+                "BLOCKCACHE": "true",
+                "BLOCKSIZE": "65536",
+                "BLOOMFILTER": "NONE",
+                "BLOOMFILTER_ERRORRATE": "0.01",
+                "COMPRESSION": "NONE",
+                "DATA_BLOCK_ENCODING": "NONE",
+                "ENCODE_ON_DISK": "true",
+                "IN_MEMORY": "false",
+                "REPLICATION_SCOPE": "0",
+                "TTL": "2147483647",
+                "VERSIONS": "3"
+            }
+        },
+        "regions": [{
+            "encoded": "b2696f3faa4bd5767f2800bbcc2687c0",
+            "end_key": "",
+            "favored_nodes": {"DUMMYCLUSTER1": []},
+            "id": 1370994021138,
+            "peers": {"DUMMYCLUSTER1": ["10.159.9.45:60020:1"]},
+            "start_key": ""
+        }]
+    }
+}}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f533bdd..f6948e4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -64,15 +64,16 @@
     <module>hbase-annotations</module>
     <module>hbase-rest</module>
     <module>hbase-checkstyle</module>
+    <module>hbase-consensus</module>
   </modules>
   <!--Add apache snapshots in case we want to use unreleased versions of plugins:
-      e.g. surefire 2.18-SNAPSHOT-->
   <pluginRepositories>
     <pluginRepository>
       <id>apache.snapshots</id>
       <url>http://repository.apache.org/snapshots/</url>
     </pluginRepository>
   </pluginRepositories>
+      e.g. surefire 2.18-SNAPSHOT-->
   <scm>
     <connection>scm:git:git://git.apache.org/hbase.git</connection>
     <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/hbase.git</developerConnection>
@@ -418,6 +419,76 @@
       <organizationUrl>http://www.taobao.com</organizationUrl>
     </developer>
   </developers>
+  <repositories>
+    <repository>
+      <id>apache release</id>
+      <url>https://repository.apache.org/content/repositories/releases/</url>
+    </repository>
+    <repository>
+      <id>mirror.facebook.net</id>
+      <name>facebook mirror</name>
+      <url>http://mirror.facebook.net/maven/repository/</url>
+      <snapshots>
+        <enabled>true</enabled>
+      </snapshots>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+    </repository>
+    <repository>
+      <id>java.net</id>
+      <name>Java.Net</name>
+      <url>http://download.java.net/maven/2/</url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+    </repository>
+    <repository>
+      <id>codehaus</id>
+      <name>Codehaus Public</name>
+      <url>http://repository.codehaus.org/</url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+    </repository>
+    <repository>
+      <id>repository.jboss.org</id>
+      <url>http://repository.jboss.org/nexus/content/groups/public-jboss/</url>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+    <repository>
+      <id>ghelmling.testing</id>
+      <name>Gary Helmling test repo</name>
+      <url>http://people.apache.org/~garyh/mvn/</url>
+      <snapshots>
+        <enabled>true</enabled>
+      </snapshots>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+    </repository>
+  </repositories>
+  <pluginRepositories>
+    <pluginRepository>
+      <id>ghelmling.testing</id>
+      <name>Gary Helmling test repo</name>
+      <url>http://people.apache.org/~garyh/mvn/</url>
+      <snapshots>
+        <enabled>true</enabled>
+      </snapshots>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+    </pluginRepository>
+  </pluginRepositories>
   <build>
     <pluginManagement>
       <plugins>
@@ -1025,6 +1096,10 @@
     <joni.version>2.1.2</joni.version>
     <!-- Plugin Dependencies -->
     <maven.assembly.version>2.4</maven.assembly.version>
+    <airlift.version>0.91</airlift.version>
+    <jmxutils.version>1.16</jmxutils.version>
+    <swift.version>0.13.1</swift.version>
+    <disruptor.version>3.3.0</disruptor.version>
     <maven.antrun.version>1.6</maven.antrun.version>
     <jamon.plugin.version>2.3.4</jamon.plugin.version>
     <findbugs-annotations>1.3.9-1</findbugs-annotations>
@@ -1203,6 +1278,16 @@
         <artifactId>jettison</artifactId>
         <version>${jettison.version}</version>
       </dependency>
+    <dependency>
+      <groupId>com.facebook.swift</groupId>
+      <artifactId>swift-root</artifactId>
+      <version>${swift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.facebook.swift</groupId>
+      <artifactId>swift-codec</artifactId>
+      <version>${swift.version}</version>
+    </dependency>
       <dependency>
         <groupId>log4j</groupId>
         <artifactId>log4j</artifactId>


[18/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HConstants.java
new file mode 100644
index 0000000..8e6acb7
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -0,0 +1,1296 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.io.hfile.Compression;
+//Amit: For Raft this is not needed: import org.apache.hadoop.hbase.ipc.HRegionInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.nativeio.NativeIO;
+
+/**
+ * HConstants holds a bunch of HBase-related constants
+ */
+public final class HConstants {
+  /**
+   * Status codes used for return values of bulk operations.
+   */
+  public enum OperationStatusCode {
+    NOT_RUN,
+    SUCCESS,
+    FAILURE;
+  }
+
+  /** long constant for zero */
+  public static final Long ZERO_L = Long.valueOf(0L);
+  public static final String NINES = "99999999999999";
+  public static final String ZEROES = "00000000000000";
+
+  // For migration
+
+  /** name of version file */
+  public static final String VERSION_FILE_NAME = "hbase.version";
+
+  /**
+   * Current version of file system.
+   * Version 4 supports only one kind of bloom filter.
+   * Version 5 changes versions in catalog table regions.
+   * Version 6 enables blockcaching on catalog tables.
+   * Version 7 introduces hfile -- hbase 0.19 to 0.20..
+   */
+  // public static final String FILE_SYSTEM_VERSION = "6";
+  public static final String FILE_SYSTEM_VERSION = "7";
+
+  // Configuration parameters
+
+  //TODO: Is having HBase homed on port 60k OK?
+
+  /** Cluster is in distributed mode or not */
+  public static final String CLUSTER_DISTRIBUTED = "hbase.cluster.distributed";
+
+  /** Cluster is standalone or pseudo-distributed */
+  public static final String CLUSTER_IS_LOCAL = "false";
+
+  /** Cluster is fully-distributed */
+  public static final String CLUSTER_IS_DISTRIBUTED = "true";
+
+  /** default host address */
+  public static final String DEFAULT_HOST = "0.0.0.0";
+
+  /** Parameter name for port master listens on. */
+  public static final String MASTER_PORT = "hbase.master.port";
+
+  /** default port that the master listens on */
+  public static final int DEFAULT_MASTER_PORT = 60000;
+
+  /** default port for master web api */
+  public static final int DEFAULT_MASTER_INFOPORT = 60010;
+
+  /** Configuration key for master web API port */
+  public static final String MASTER_INFO_PORT = "hbase.master.info.port";
+
+  /** Parameter name for the master type being backup (waits for primary to go inactive). */
+  public static final String MASTER_TYPE_BACKUP = "hbase.master.backup";
+
+  /** by default every master is a possible primary master unless the conf explicitly overrides it */
+  public static final boolean DEFAULT_MASTER_TYPE_BACKUP = false;
+
+  /** Configuration key for enabling table-level locks for schema changes */
+  public static final String MASTER_SCHEMA_CHANGES_LOCK_ENABLE =
+    "hbase.master.schemaChanges.lock.enable";
+
+  /** by default we should enable table-level locks for schema changes */
+  public static final boolean DEFAULT_MASTER_SCHEMA_CHANGES_LOCK_ENABLE = true;
+
+  /** Configuration key for time out for schema modification locks */
+  public static final String MASTER_SCHEMA_CHANGES_LOCK_TIMEOUT_MS =
+    "hbase.master.schemaChanges.lock.timeout.ms";
+
+  public static final int DEFAULT_MASTER_SCHEMA_CHANGES_LOCK_TIMEOUT_MS =
+    60 * 1000;
+
+  /** Configuration key for time out for schema modification try lock */
+  public static final String MASTER_SCHEMA_CHANGES_TRY_LOCK_TIMEOUT_MS =
+      "hbase.master.schemaChanges.trylock.timeout.ms";
+
+  public static final int DEFAULT_MASTER_SCHEMA_CHANGES_TRY_LOCK_TIMEOUT_MS =
+      5 * 1000;
+
+  /** Configuration key for for schema modification wait interval. */
+  public static final String MASTER_SCHEMA_CHANGES_WAIT_INTERVAL_MS =
+      "hbase.regionserver.alterTable.waitInterval.ms";
+
+  public static final int DEFAULT_MASTER_SCHEMA_CHANGES_WAIT_INTERVAL_MS =
+      1000;
+
+  /** Configuration key for for schema modification max concurrent regions closed. */
+  public static final String MASTER_SCHEMA_CHANGES_MAX_CONCURRENT_REGION_CLOSE =
+      "hbase.regionserver.alterTable.maxConcurrentClose";
+
+  public static final int DEFAULT_MASTER_SCHEMA_CHANGES_MAX_CONCURRENT_REGION_CLOSE =
+      5;
+
+  /** Name of ZooKeeper quorum configuration parameter. */
+  public static final String ZOOKEEPER_QUORUM = "hbase.zookeeper.quorum";
+
+  /** Name of ZooKeeper config file in conf/ directory. */
+  public static final String ZOOKEEPER_CONFIG_NAME = "zoo.cfg";
+
+  /** Common prefix of ZooKeeper configuration properties */
+  public static final String ZK_CFG_PROPERTY_PREFIX =
+      "hbase.zookeeper.property.";
+
+  public static final int ZK_CFG_PROPERTY_PREFIX_LEN =
+    ZK_CFG_PROPERTY_PREFIX.length();
+
+  /** Parameter name for number of times to retry writes to ZooKeeper. */
+  public static final String ZOOKEEPER_RETRIES = "zookeeper.retries";
+
+  /** Parameter name for the strategy whether aborting the process
+   *  when zookeeper session expired.
+   */
+  public static final String ZOOKEEPER_SESSION_EXPIRED_ABORT_PROCESS =
+    "hbase.zookeeper.sessionExpired.abortProcess";
+
+  /** Parameter name for number of times to retry to connection to ZooKeeper. */
+  public static final String ZOOKEEPER_CONNECTION_RETRY_NUM =
+    "zookeeper.connection.retry.num";
+
+  /**
+   * The ZK client port key in the ZK properties map. The name reflects the
+   * fact that this is not an HBase configuration key.
+   */
+  public static final String CLIENT_PORT_STR = "clientPort";
+
+  /** Parameter name for the client port that the zookeeper listens on */
+  public static final String ZOOKEEPER_CLIENT_PORT =
+      ZK_CFG_PROPERTY_PREFIX + CLIENT_PORT_STR;
+
+  /** Default number of times to retry writes to ZooKeeper. */
+  public static final int DEFAULT_ZOOKEEPER_RETRIES = 5;
+
+  /** Parameter name for ZooKeeper session time out.*/
+  public static final String ZOOKEEPER_SESSION_TIMEOUT =
+    "zookeeper.session.timeout";
+
+  /** Default value for ZooKeeper session time out. */
+  public static final int DEFAULT_ZOOKEEPER_SESSION_TIMEOUT = 60 * 1000;
+
+  /** Parameter name for ZooKeeper pause between retries. In milliseconds. */
+  public static final String ZOOKEEPER_PAUSE = "zookeeper.pause";
+  /** Default ZooKeeper pause value. In milliseconds. */
+  public static final int DEFAULT_ZOOKEEPER_PAUSE = 2 * 1000;
+
+  /** default client port that the zookeeper listens on */
+  public static final int DEFAULT_ZOOKEPER_CLIENT_PORT = 2181;
+
+  /** Parameter name for the root dir in ZK for this cluster */
+  public static final String ZOOKEEPER_ZNODE_PARENT = "zookeeper.znode.parent";
+
+  public static final String DEFAULT_ZOOKEEPER_ZNODE_PARENT = "/hbase";
+
+  /** Parameter name for port region server listens on. */
+  public static final String REGIONSERVER_PORT = "hbase.regionserver.port";
+
+  /** Default port region server listens on. */
+  public static final int DEFAULT_REGIONSERVER_PORT = 60020;
+
+  /** default port for region server web api */
+  public static final int DEFAULT_REGIONSERVER_INFOPORT = 60030;
+
+  /** A configuration key for regionserver info port */
+  public static final String REGIONSERVER_INFO_PORT =
+    "hbase.regionserver.info.port";
+
+  /** A flag that enables automatic selection of regionserver info port */
+  public static final String REGIONSERVER_INFO_PORT_AUTO =
+      REGIONSERVER_INFO_PORT + ".auto";
+
+  /** Parameter name for what region server interface to use. */
+  public static final String REGION_SERVER_CLASS = "hbase.regionserver.class";
+
+  /** Parameter name for what region server implementation to use. */
+  public static final String REGION_SERVER_IMPL= "hbase.regionserver.impl";
+
+  /** Parameter name for whether region server is running in the hydrabase mode. */
+  public static final String HYDRABASE = "hbase.hydrabase";
+
+  /**
+   * Parameter name which can be used to uniqely identify a HydraBase cluster.
+   * This is used for example when exporting metrics MBeans.
+   */
+  public static final String HYDRABASE_CLUSTER_NAME =
+          "hbase.hydrabase.cluster.name";
+
+  public static final boolean DEFAULT_HYDRABASE = false;
+
+//Amit: For Raft this is not needed:   /** Default region server interface class name. */
+//Amit: For Raft this is not needed:   public static final String DEFAULT_REGION_SERVER_CLASS = HRegionInterface.class.getName();
+
+  /** Parameter name for enabling regionChecker */
+  public static final String REGION_CHECKER_ENABLED = "hbase.master.regionchecker.enabled";
+  /** Default value for enabling regionChecker */
+  public static final Boolean DEFAULT_REGION_CHECKER_ENABLED = false;
+
+  /** Parameter name for what compaction manager to use. */
+  public static final String COMPACTION_MANAGER_CLASS = "hbase.compactionmanager.class";
+
+  /** Parameter name for the number of large compaction threads */
+  public static final String LARGE_COMPACTION_THREADS =
+          "hbase.regionserver.thread.compaction.large";
+
+  /** Default number of large compaction threads */
+  public static final int DEFAULT_LARGE_COMPACTION_THREADS = 1;
+
+  /** Parameter name for the number of large compaction threads */
+  public static final String SMALL_COMPACTION_THREADS =
+          "hbase.regionserver.thread.compaction.small";
+
+  /** Default number of small compaction threads */
+  public static final int DEFAULT_SMALL_COMPACTION_THREADS = 1;
+
+  /** Prefix for Compaction related configurations in Store */
+  public static final String HSTORE_COMPACTION_PREFIX =
+          "hbase.hstore.compaction.";
+
+  /** Parameter name for the number of split threads */
+  public static final String SPLIT_THREADS = "hbase.regionserver.thread.split";
+
+  /** Default number of split threads */
+  public static final int DEFAULT_SPLIT_THREADS = 1;
+
+  /** Parameter name for what master implementation to use. */
+  public static final String MASTER_IMPL = "hbase.master.impl";
+
+  /** Parameter name for how often threads should wake up */
+  public static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency";
+
+  /** Parameter name for how often a region should should perform a major compaction */
+  public static final String MAJOR_COMPACTION_PERIOD = "hbase.hregion.majorcompaction";
+
+  /** Parameter name for HBase instance root directory */
+  public static final String HBASE_DIR = "hbase.rootdir";
+
+  /** Parameter name for explicit region placement */
+  public static final String LOAD_BALANCER_IMPL = "hbase.loadbalancer.impl";
+
+  /** Used to construct the name of the log directory for a region server
+   * Use '.' as a special character to seperate the log files from table data */
+  public static final String HREGION_LOGDIR_NAME = ".logs";
+
+  /** Like the previous, but for old logs that are about to be deleted */
+  public static final String HREGION_OLDLOGDIR_NAME = ".oldlogs";
+
+  /** Boolean config to determine if we should use a subdir structure
+   * in the .oldlogs directory */
+  public static final String HREGION_OLDLOGDIR_USE_SUBDIR_STRUCTURE =
+    "hbase.regionserver.oldlogs.use.subdir.structure";
+
+  /** Boolean config to determine if we should use a subdir structure in
+   * the .oldlogs directory by default */
+  public static final boolean HREGION_OLDLOGDIR_USE_SUBDIR_STRUCTURE_DEFAULT =
+    true;
+
+  /** Used to construct the name of the compaction directory during compaction */
+  public static final String HREGION_COMPACTIONDIR_NAME = "compaction.dir";
+
+  /** Conf key for the max file size after which we split the region */
+  public static final String HREGION_MAX_FILESIZE =
+      "hbase.hregion.max.filesize";
+
+  /** File Extension used while splitting an HLog into regions (HBASE-2312) */
+  public static final String HLOG_SPLITTING_EXT = "-splitting";
+
+  /**
+   * The max number of threads used for opening and closing stores or store
+   * files in parallel
+   */
+  public static final String HSTORE_OPEN_AND_CLOSE_THREADS_MAX =
+    "hbase.hstore.open.and.close.threads.max";
+
+  /**
+   * The default number for the max number of threads used for opening and
+   * closing stores or store files in parallel
+   */
+  public static final int DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX = 8;
+
+  /**
+   * The max number of threads used for opening and closing regions
+   * in parallel
+   */
+  public static final String HREGION_OPEN_AND_CLOSE_THREADS_MAX =
+    "hbase.region.open.and.close.threads.max";
+
+  /**
+   * The default number for the max number of threads used for opening and
+   * closing regions in parallel
+   */
+  public static final int DEFAULT_HREGION_OPEN_AND_CLOSE_THREADS_MAX = 20;
+
+  /**
+   * The max number of threads used for splitting logs
+   * in parallel
+   */
+  public static final String HREGIONSERVER_SPLITLOG_WORKERS_NUM =
+    "hbase.hregionserver.hlog.split.workers.num";
+
+  /**
+   * If using quorum reads from HDFS, the maximum size of the thread pool.
+   * value <= 0 disables quorum reads.
+   */
+  public static final String HDFS_QUORUM_READ_THREADS_MAX =
+    "hbase.dfsclient.quorum.reads.threads.max";
+
+  /**
+   * The default number for the size of thread pool used in quorum reads.
+   * value <= 0 disables quorum reads.
+   */
+  public static final int DEFAULT_HDFS_QUORUM_READ_THREADS_MAX = 50;
+
+  /**
+   * If using quorum reads from HDFS, the timeout of using another region server.
+   */
+  public static final String HDFS_QUORUM_READ_TIMEOUT_MILLIS =
+    "hbase.dfsclient.quorum.reads.timeout";
+  public static final long DEFAULT_HDFS_QUORUM_READ_TIMEOUT_MILLIS = 0;
+
+  /** Default maximum file size */
+  public static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024;
+
+  /** Default minimum number of files to be compacted */
+  public static final int DEFAULT_MIN_FILES_TO_COMPACT = 3;
+
+  /** Default value for files without minFlushTime in metadata */
+  public static final long NO_MIN_FLUSH_TIME = -1;
+
+  /** Conf key for the memstore size at which we flush the memstore */
+  public static final String HREGION_MEMSTORE_FLUSH_SIZE =
+      "hbase.hregion.memstore.flush.size";
+
+  /** Conf key for enabling Per Column Family flushing of memstores */
+  public static final String HREGION_MEMSTORE_PER_COLUMN_FAMILY_FLUSH =
+      "hbase.hregion.memstore.percolumnfamilyflush.enabled";
+
+  /** Default value for the Per Column Family flush knob */
+  public static final Boolean DEFAULT_HREGION_MEMSTORE_PER_COLUMN_FAMILY_FLUSH =
+    false;
+
+  /**
+   * If Per Column Family flushing is enabled, this is the minimum size
+   * at which a column family's memstore is flushed.
+   */
+  public static final String HREGION_MEMSTORE_COLUMNFAMILY_FLUSH_SIZE =
+      "hbase.hregion.memstore.percolumnfamilyflush.flush.size";
+
+  public static final String HREGION_MEMSTORE_BLOCK_MULTIPLIER =
+      "hbase.hregion.memstore.block.multiplier";
+  public static final String HREGION_MEMSTORE_WAIT_ON_BLOCK =
+      "hbase.hregion.memstore.block.waitonblock";
+
+  /** Default size of a reservation block   */
+  public static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5;
+
+  /** Maximum value length, enforced on KeyValue construction */
+  public static final int MAXIMUM_VALUE_LENGTH = Integer.MAX_VALUE;
+
+  /** Conf key for enabling/disabling server profiling */
+  public static final String HREGIONSERVER_ENABLE_SERVERSIDE_PROFILING =
+      "hbase.regionserver.enable.serverside.profiling";
+
+  /** Conf key for the preload blocks count if preloading is enabled for some scanner */
+  public static final String SCAN_PRELOAD_BLOCK_COUNT =
+      "hbase.regionserver.preload.block.count";
+  /** Default number of blocks to preload during sequential scan of hfile (if enabled)*/
+  public static final int DEFAULT_PRELOAD_BLOCK_COUNT = 64;
+  /** Conf key for the core preload threads */
+  public static final String CORE_PRELOAD_THREAD_COUNT = "hbase.regionserver.core.preload.thread.count";
+  /** Default number of core preload threads per region server */
+  public static final int DEFAULT_CORE_PRELOAD_THREAD_COUNT = 1;
+  /** Conf key for the max preload threads */
+  public static final String MAX_PRELOAD_THREAD_COUNT = "hbase.regionserver.max.preload.thread.count";
+  /** Defualt number of core preload threads per region server */
+  public static final int DEFAULT_MAX_PRELOAD_THREAD_COUNT = 64;
+  /** Conf key for max preload blocks kept in cache per hfilescanner */
+  public static final String MAX_PRELOAD_BLOCKS_KEPT_IN_CACHE =
+      "hbase.regionserver.preload.blocks.kept.in.cache";
+  /** Default maximum number of preload blocks to keep in block cache per hfilescanner */
+  public static final int DEFAULT_MAX_PRELOAD_BLOCKS_KEPT_IN_CACHE = 128;
+
+  // Always store the location of the root table's HRegion.
+  // This HRegion is never split.
+
+
+  // region name = table + startkey + regionid. This is the row key.
+  // each row in the root and meta tables describes exactly 1 region
+  // Do we ever need to know all the information that we are storing?
+
+  // Note that the name of the root table starts with "-" and the name of the
+  // meta table starts with "." Why? it's a trick. It turns out that when we
+  // store region names in memory, we use a SortedMap. Since "-" sorts before
+  // "." (and since no other table name can start with either of these
+  // characters, the root region will always be the first entry in such a Map,
+  // followed by all the meta regions (which will be ordered by their starting
+  // row key as well), followed by all user tables. So when the Master is
+  // choosing regions to assign, it will always choose the root region first,
+  // followed by the meta regions, followed by user regions. Since the root
+  // and meta regions always need to be on-line, this ensures that they will
+  // be the first to be reassigned if the server(s) they are being served by
+  // should go down.
+
+
+  //
+  // New stuff.  Making a slow transition.
+  //
+
+  /** The root table's name.*/
+  public static final byte [] ROOT_TABLE_NAME = Bytes.toBytes("-ROOT-");
+
+  /** The META table's name. */
+  public static final byte [] META_TABLE_NAME = Bytes.toBytes(".META.");
+
+  /** delimiter used between portions of a region name */
+  public static final int META_ROW_DELIMITER = ',';
+
+  /** The catalog family as a string*/
+  public static final String CATALOG_FAMILY_STR = "info";
+
+  /** The catalog family */
+  public static final byte [] CATALOG_FAMILY = Bytes.toBytes(CATALOG_FAMILY_STR);
+
+  /** The catalog historian family */
+  public static final byte [] CATALOG_HISTORIAN_FAMILY = Bytes.toBytes("historian");
+
+  /** The regioninfo column qualifier */
+  public static final byte [] REGIONINFO_QUALIFIER = Bytes.toBytes("regioninfo");
+
+  /** The server column qualifier */
+  public static final byte [] SERVER_QUALIFIER = Bytes.toBytes("server");
+
+  /** The startcode column qualifier */
+  public static final byte [] STARTCODE_QUALIFIER = Bytes.toBytes("serverstartcode");
+
+  /** The lower-half split region column qualifier */
+  public static final byte [] SPLITA_QUALIFIER = Bytes.toBytes("splitA");
+
+  /** The upper-half split region column qualifier */
+  public static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB");
+
+  /** The favored nodes column qualifier*/
+  public static final byte [] FAVOREDNODES_QUALIFIER = Bytes.toBytes("favorednodes");
+
+  // Other constants
+
+  /**
+   * An empty instance.
+   */
+  public static final byte [] EMPTY_BYTE_ARRAY = new byte [0];
+
+  public static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.wrap(EMPTY_BYTE_ARRAY);
+
+  /**
+   * Used by scanners, etc when they want to start at the beginning of a region
+   */
+  public static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY;
+
+  public static final ByteBuffer EMPTY_START_ROW_BUF = ByteBuffer.wrap(EMPTY_START_ROW);
+
+  /**
+   * Last row in a table.
+   */
+  public static final byte [] EMPTY_END_ROW = EMPTY_START_ROW;
+
+  public static final ByteBuffer EMPTY_END_ROW_BUF = ByteBuffer.wrap(EMPTY_END_ROW);
+
+  /**
+    * Used by scanners and others when they're trying to detect the end of a
+    * table
+    */
+  public static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY;
+
+  /**
+   * Max length a row can have because of the limitation in TFile.
+   */
+  public static final int MAX_ROW_LENGTH = Short.MAX_VALUE;
+
+  /** When we encode strings, we always specify UTF8 encoding */
+  public static final String UTF8_ENCODING = "UTF-8";
+
+  /**
+   * Timestamp to use when we want to refer to the latest cell.
+   * This is the timestamp sent by clients when no timestamp is specified on
+   * commit.
+   */
+  public static final long LATEST_TIMESTAMP = Long.MAX_VALUE;
+
+  /**
+   * Timestamp to use when we want to refer to the oldest cell.
+   */
+  public static final long OLDEST_TIMESTAMP = Long.MIN_VALUE;
+
+  /**
+   * LATEST_TIMESTAMP in bytes form
+   */
+  public static final byte [] LATEST_TIMESTAMP_BYTES = Bytes.toBytes(LATEST_TIMESTAMP);
+
+  /**
+   * Define for 'return-all-versions'.
+   */
+  public static final int ALL_VERSIONS = Integer.MAX_VALUE;
+
+  /**
+   * Unlimited time-to-live.
+   */
+//  public static final int FOREVER = -1;
+  public static final int FOREVER = Integer.MAX_VALUE;
+
+  /**
+   * Seconds in a week
+   */
+  public static final int WEEK_IN_SECONDS = 7 * 24 * 3600;
+
+  //TODO: although the following are referenced widely to format strings for
+  //      the shell. They really aren't a part of the public API. It would be
+  //      nice if we could put them somewhere where they did not need to be
+  //      public. They could have package visibility
+  public static final String NAME = "NAME";
+  public static final String VERSIONS = "VERSIONS";
+  public static final String IN_MEMORY = "IN_MEMORY";
+  public static final String CONFIG = "CONFIG";
+
+  /**
+   * This is a retry backoff multiplier table similar to the BSD TCP syn
+   * backoff table, a bit more aggressive than simple exponential backoff.
+   */
+  public static int RETRY_BACKOFF[] = { 1, 1, 1, 2, 2, 4, 4, 8, 16, 32 };
+
+  public static final String REGION_IMPL = "hbase.hregion.impl";
+
+  /** modifyTable op for replacing the table descriptor */
+  public static enum Modify {
+    CLOSE_REGION,
+    MOVE_REGION,
+    TABLE_COMPACT,
+    TABLE_FLUSH,
+    TABLE_MAJOR_COMPACT,
+    TABLE_SET_HTD,
+    TABLE_SPLIT,
+    TABLE_EXPLICIT_SPLIT
+  }
+
+  /**
+   * Scope tag for locally scoped data.
+   * This data will not be replicated.
+   */
+  public static final int REPLICATION_SCOPE_LOCAL = 0;
+
+  /**
+   * Scope tag for globally scoped data.
+   * This data will be replicated to all peers.
+   */
+  public static final int REPLICATION_SCOPE_GLOBAL = 1;
+
+  /**
+   * Default cluster ID, cannot be used to identify a cluster so a key with
+   * this value means it wasn't meant for replication.
+   */
+  public static final byte DEFAULT_CLUSTER_ID = 0;
+
+    /**
+     * Parameter name for maximum number of bytes returned when calling a
+     * scanner's next method.
+     */
+  public static final String HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY = "hbase.client.scanner.max.result.size";
+
+  /**
+   * Parameter name for the number of threads for the ParallelScanner
+   */
+  public static final String HBASE_CLIENT_PARALLEL_SCANNER_THREAD =
+    "hbase.client.parallel.scanner.thread";
+
+  /**
+   * The default number of threads for the ParallelScanner
+   */
+  public static final int HBASE_CLIENT_PARALLEL_SCANNER_THREAD_DEFAULT = 100;
+
+  /**
+   * Maximum number of bytes returned when calling a scanner's next method.
+   * Note that when a single row is larger than this limit the row is still
+   * returned completely.
+   *
+   * The default value is unlimited.
+   */
+  public static final long DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE = Long.MAX_VALUE;
+
+
+  /**
+   * Maximum number of bytes returned when calling a scanner's next method.
+   * Used with partialRow parameter on the client side.  Note that when a
+   * single row is larger than this limit, the row is still returned completely
+   * if partialRow is true, otherwise, the row will be truncated in order to
+   * fit the memory.
+   */
+  public static final int DEFAULT_HBASE_SCANNER_MAX_RESULT_SIZE = Integer.MAX_VALUE;
+
+  /**
+   * HRegion server lease period in milliseconds. Clients must report in within this period
+   * else they are considered dead. Unit measured in ms (milliseconds).
+   */
+  public static final String HBASE_REGIONSERVER_LEASE_PERIOD_KEY   = "hbase.regionserver.lease.period";
+
+
+  /**
+   * Default value of {@link #HBASE_REGIONSERVER_LEASE_PERIOD_KEY}.
+   */
+  public static final long DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD = 60000;
+
+  /**
+   * timeout for each RPC
+   */
+  public static final String HBASE_RPC_TIMEOUT_KEY = "hbase.rpc.timeout";
+  public static final String HBASE_RS_REPORT_TIMEOUT_KEY = "hbase.regionserverReport.timeout";
+
+  /**
+   * Default value of {@link #HBASE_RPC_TIMEOUT_KEY}
+   */
+  public static final int DEFAULT_HBASE_RPC_TIMEOUT = 60000;
+  public static final int DEFAULT_RS_REPORT_TIMEOUT = 3000;
+
+  /**
+   * pause between rpc or connect retries
+   */
+  public static final String HBASE_CLIENT_PAUSE = "hbase.client.pause";
+  public static final int DEFAULT_HBASE_CLIENT_PAUSE = 1000;
+
+  /**
+   * compression for each RPC and its default value
+   */
+  public static final String HBASE_RPC_COMPRESSION_KEY = "hbase.rpc.compression";
+  public static final Compression.Algorithm DEFAULT_HBASE_RPC_COMPRESSION =
+    Compression.Algorithm.NONE;
+
+  public static final String
+      REPLICATION_ENABLE_KEY = "hbase.replication";
+
+  /**
+   * Configuration key for the size of the block cache
+   */
+  public static final String HFILE_BLOCK_CACHE_SIZE_KEY =
+      "hfile.block.cache.size";
+
+  public static final float HFILE_BLOCK_CACHE_SIZE_DEFAULT = 0.25f;
+
+  /** The delay when re-trying a socket operation in a loop (HBASE-4712) */
+  public static final int SOCKET_RETRY_WAIT_MS = 200;
+
+  /** Host name of the local machine */
+  public static final String LOCALHOST = "localhost";
+
+  public static final String LOCALHOST_IP = "127.0.0.1";
+
+  /** Conf key that enables distributed log splitting */
+  public static final String DISTRIBUTED_LOG_SPLITTING_KEY =
+      "hbase.master.distributed.log.splitting";
+
+  public static final int REGION_SERVER_MSG_INTERVAL = 1 * 1000;
+
+  /** The number of favored nodes for each region */
+  public static final int FAVORED_NODES_NUM = 3;
+
+  public static final String UNKNOWN_RACK = "Unknown Rack";
+
+  /** Delay when waiting for a variable (HBASE-4712) */
+  public static final int VARIABLE_WAIT_TIME_MS = 40;
+
+  public static final String LOAD_BALANCER_SLOP_KEY = "hbase.regions.slop";
+
+  // Thrift server configuration options
+
+  /** Configuration key prefix for the stand-alone thrift proxy */
+  public static final String THRIFT_PROXY_PREFIX = "hbase.thrift.";
+
+  /** Configuration key prefix for thrift server embedded into the region server */
+  public static final String RS_THRIFT_PREFIX = "hbase.regionserver.thrift.";
+
+  /** Default port for the stand-alone thrift proxy */
+  public static final int DEFAULT_THRIFT_PROXY_PORT = 9090;
+
+  /** Default port for the thrift server embedded into regionserver */
+  public static final int DEFAULT_RS_THRIFT_SERVER_PORT = 9091;
+
+  /** Configuration key suffix for thrift server type (e.g. thread pool, nonblocking, etc.) */
+  public static final String THRIFT_SERVER_TYPE_SUFFIX = "server.type";
+
+  /** Configuration key suffix for the IP address for thrift server to bind to */
+  public static final String THRIFT_BIND_SUFFIX = "ipaddress";
+
+  /** Configuration key suffix for whether to use compact Thrift transport */
+  public static final String THRIFT_COMPACT_SUFFIX = "compact";
+
+  /** Configuration key suffix for whether to use framed Thrift transport */
+  public static final String THRIFT_FRAMED_SUFFIX = "framed";
+
+  /** Configuration key suffix for Thrift server port */
+  public static final String THRIFT_PORT_SUFFIX = "port";
+
+  /** The number of HLogs for each region server */
+  public static final String HLOG_CNT_PER_SERVER = "hbase.regionserver.hlog.cnt.perserver";
+
+  public static final String HLOG_FORMAT_BACKWARD_COMPATIBILITY =
+      "hbase.regionserver.hlog.format.backward.compatibility";
+
+  /**
+   * The byte array represents for NO_NEXT_INDEXED_KEY;
+   * The actual value is irrelevant because this is always compared by reference.
+   */
+  public static final byte [] NO_NEXT_INDEXED_KEY = Bytes.toBytes("NO_NEXT_INDEXED_KEY");
+
+  public static final int MULTIPUT_SUCCESS = -1;
+
+  public static final boolean[] BOOLEAN_VALUES = { false, true };
+
+  public static final int IPC_CALL_PARAMETER_LENGTH_MAX = 1000;
+
+  /**
+   * Used in Configuration to get/set the KV aggregator
+   */
+  public static final String KV_AGGREGATOR = "kvaggregator";
+
+  /**
+   * Used in Configuration to get/set the compaction hook
+   */
+  public static final String COMPACTION_HOOK = "compaction_hook";
+
+  /**
+   * Absolute path of the external jar which will contain the custom compaction hook
+   */
+  public static final String COMPACTION_HOOK_JAR = "compaction_hook_jar";
+
+  public static final String GENERAL_BLOOM_FILTER = "general_bloom_filter";
+
+  public static final String DELETE_FAMILY_BLOOM_FILTER = "delete_family_bloom_filter";
+
+  public static final String DELETE_COLUMN_BLOOM_FILTER = "delete_column_bloom_filter";
+
+  public static final String ROWKEY_PREFIX_BLOOM_FILTER = "rowkey_prefix_bloom_filter";
+
+  /**
+   * This will enable/disable the usage of delete col bloom filter. Note that
+   * this won't enable/disable the delete bloom filter for being written/read.
+   * In fact, we could read and write it but we will not use it when we scan
+   * data, thus we won't do the optimized reads. In order to disable/enable the
+   * filter for write&read both, use
+   * BloomFilterFactory.IO_STOREFILE_DELETEFAMILY_BLOOM_ENABLED
+   */
+  public static final boolean USE_DELETE_COLUMN_BLOOM_FILTER = true;
+  public static final String USE_DELETE_COLUMN_BLOOM_FILTER_STRING = "use_delete_column_bloom_filter";
+
+  // Delaying the region server load balancing by the following amount for a
+  // load balancing where source is a favored region server.
+  public static final String HBASE_REGION_ASSIGNMENT_LOADBALANCER_WAITTIME_MS
+                                = "hbase.master.assignment.load.balancer.waittime.ms";
+  public static final int DEFAULT_HBASE_REGION_ASSIGNMENT_LOADBALANCER_WAITTIME_MS = 60000;
+
+  /*
+   * This defines the number of buckets used for computing the histogram of
+   * pread latency.
+   */
+  public static final String PREAD_LATENCY_HISTOGRAM_NUM_BUCKETS =
+      "hbase.histogrambasedmetric.numbuckets.preadlatency";
+
+  /*
+   * This defines the number of buckets used for computing the histogram of
+   * pread latency during compaction.
+   */
+  public static final String PREAD_COMPACTION_LATENCY_HISTOGRAM_NUM_BUCKETS =
+      "hbase.histogrambasedmetric.numbuckets.preadcompactionlatency";
+  public static final String HISTOGRAM_BASED_METRICS_WINDOW =
+      "hbase.histogrambasedmetric.window";
+  /*
+   * This is the folder address for the hard links folder where the
+   * hard links are created during creating a read only store.
+   */
+  public static final String READ_ONLY_HARDLINKS_FOLDER =
+      "hbase.store.readonly.hardlinks.folder";
+  public static final String READ_ONLY_HARDLINKS_FOLDER_DEFAULT =
+      "/tmp/hardlinks/";
+
+  public static final String CLIENT_SOCKED_CLOSED_EXC_MSG = "Interrupting the read request";
+  public static final String SERVER_INTERRUPTED_CALLS_KEY = "serverInterruptedCalls";
+
+  public static final String RMAP_SUBSCRIPTION = "hbase.rmap.subscriptions";
+
+  public static final String DEFAULT_RMAP_NAME = "rmap.json";
+
+  /**
+   * How much time to wait in HydraBaseAdmin.applyRMap() for the RMap to be
+   * successfully applied.
+   */
+  public static final int APPLY_RMAP_TIMEOUT_MILLIS = 5000;
+
+  public static final String APPLY_RMAP_RETRIES_KEY = "hbase.consensus.rmap.applyrmapretries";
+  public static final int DEFAULT_APPLY_RMAP_RETRIES = 10;
+
+  public static final String STATE_MACHINE_POLL_INTERVAL =
+    "hbase.consensus.statemachine.poll.interval.ms";
+
+  public static final int DEFAULT_STATE_MACHINE_POLL_INTERVAL = 10;
+
+  /** Progress timeout interval key */
+  public static final String PROGRESS_TIMEOUT_INTERVAL_KEY =
+    "hbase.consensus.progress.timeout.interval";
+
+  /** Default progress timeout interval in millisecs */
+  public static final int PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS = 5000;
+
+  /** AppendEntries timeout */
+  public static final String APPEND_ENTRIES_TIMEOUT_KEY = "hbase.consensus.append.entries.timeout";
+  public static final int DEFAULT_APPEND_ENTRIES_TIMEOUT_IN_MILLISECONDS =
+         6 * PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS;  // 30 seconds
+
+  public static final String DEFAULT_TRANSACTION_LOG_DIRECTORY = "/tmp/wal/";
+
+  public static final String DEFAULT_RAFT_METADATA_DIRECTORY = "/tmp/metadata";
+
+  public static final int CONSENSUS_SERVER_PORT_JUMP = 100;
+
+  public static final int RAFT_PENDING_EDITS_QUEUE_LENGTH = 50;
+
+  /** Raft log version */
+  public static final int RAFT_LOG_VERSION = 1;
+
+  /** Undefined the term and index */
+  public static final long UNDEFINED_TERM_INDEX = -1;
+
+  /** Seed term */
+  public static final long SEED_TERM = UNDEFINED_TERM_INDEX - 1;
+
+  /** Undefined the raft log file version */
+  public static final int UNDEFINED_VERSION = -1;
+
+  /** Undefined the payload size */
+  public static final int UNDEFINED_PAYLOAD_SIZE = -1;
+
+  /** The raft log file header */
+  public final static int RAFT_FILE_HEADER_SIZE = Bytes.SIZEOF_INT + Bytes.SIZEOF_LONG * 2;
+
+  /** The raft log file transaction header */
+  public final static int RAFT_TXN_HEADER_SIZE = Bytes.SIZEOF_INT + 2 * Bytes.SIZEOF_LONG;
+
+  /** The minimum number of new logs available for immediate use to a quorum */
+  public static final String RAFT_MAX_NUM_NEW_LOGS_KEY = "hbase.hydrabase.max.new.logs";
+  public static final int RAFT_MAX_NUM_NEW_LOGS = 2;
+
+  /** The number of threads to perform the log roll */
+  public static final int RAFT_LOG_ROLL_POOL_SIZE = 1;
+
+  /** The number of threads to perform the log roll */
+  public static final int RAFT_LOG_DELETE_POOL_SIZE = 1;
+
+  /** Max size of the Consensus log */
+  public static final String RAFT_LOG_ROLL_SIZE_KEY = "hbase.hydrabase.log.roll.size";
+  public static final long DEFAULT_RAFT_LOG_ROLL_SIZE = 256 * 1024 * 1024;
+
+  /** Directory name for the current logs */
+  public static final String RAFT_CURRENT_LOG_DIRECTORY_NAME = "current";
+
+  /** Directory name for the finalized logs */
+  public static final String RAFT_FINALIZED_LOG_DIRECTORY_NAME = "finalized";
+
+  /** Directory name for the candidate logs */
+  public static final String RAFT_CANDIDATE_LOG_DIRECTORY_NAME_KEY = "hbase.consensus.logs.candidates.subdir";
+
+  /** Directory name for the candidate logs */
+  public static final String RAFT_CANDIDATE_LOG_DIRECTORY_NAME_DEFAULT = "candidates";
+
+  /** minimum number of entries for a trimmed candidate log */
+  public static final String RAFT_CANDIDATE_LOG_MIN_SIZE_KEY = "hbase.consensus.logs.candidate.min.size";
+  public static final long RAFT_CANDIDATE_LOG_MIN_SIZE_DEFAULT = 100L;
+
+  public static final String RAFT_CANDIDATE_FAST_COPY_LOG_KEY = "hbase.consensus.logs.use.fast.copy";
+  public static final boolean RAFT_CANDIDATE_FAST_COPY_LOG_DEFAULT = false;
+
+  public static final String RAFT_CANDIDATE_LOG_PROMOTION_ENABLED_KEY =
+    "hbase.consensus.logs.candidate.promotion.enabled";
+
+  public static final boolean RAFT_CANDIDATE_LOG_PROMOTION_ENABLED_DEFAULT = false;
+
+  /** Path separator */
+  public static final String PATH_SEPARATOR = "/";
+
+  public static final String RAFT_PEERSERVER_CONNECTION_RETRY_INTERVAL =
+    "hbase.quorum.peer.server.retry.interval";
+
+  public static final String RAFT_PEERSERVER_CONNECTION_RETRY_CNT =
+    "hbase.quorum.peer.server.retry.cnt";
+
+  public static final String RAFT_PEERSERVER_CONNECTION_TIMEOUT_MS =
+    "hbase.quorum.peer.server.connection.timeout.ms";
+
+  public static final String RAFT_PEERSERVER_READ_TIMEOUT_MS =
+    "hbase.quorum.peer.server.read.timeout.ms";
+
+  public static final String RAFT_PEERSERVER_WRITE_TIMEOUT_MS =
+    "hbase.quorum.peer.server.write.timeout.ms";
+
+  public static final String RAFT_PEERSERVER_HANDLE_RPC_TIMEOUT_MS =
+    "hbase.quorum.peer.server.handle.rpc.error.timeout.ms";
+
+  public static final int RAFT_BATCH_APPEND_MAX_EDITS_DEFAULT = 512;
+
+  public static final long RAFT_BATCH_APPEND_MAX_BYTES_DEFAULT = 1024 * 1024L;
+
+  public static final long RAFT_PEERSERVER_HANDLE_RPC_TIMEOUT_MS_DEFAULT = 120000L;
+
+  public static final String RAFT_BATCH_APPEND_TRY_CANDIDATE_LOGS_PROMOTION_THRESHOLD_KEY =
+    "hbase.quorum.raft.batch.append.candidate.logs.promotion.threshold";
+
+  public static final long RAFT_BATCH_APPEND_TRY_CANDIDATE_LOGS_PROMOTION_THRESHOLD_DEFAULT = 1024L;
+
+  /**
+   * Controls whether we use one or two throttles to control the insert in the queue
+   */
+  public static final String USE_MULTIPLE_THROTTLES = "hbase.server.multithrottler";
+
+  /**
+   * How much memory do we want for the blocking callqueue, used in HBaseServer
+   */
+   public static final long MAX_CALL_QUEUE_MEMORY_SIZE = 1024*1024*1024;
+   public static final String MAX_CALL_QUEUE_MEMORY_SIZE_STRING = "max.callqueue.memory.size";
+
+   /**
+    * Used in HBase Server, when we use the multithrottler for the callQueue
+    */
+   public static final long MAX_SMALLER_CALL_QUEUE_MEMORY_SIZE = 256*1024*1024;
+   public static final String MAX_SMALLER_CALL_QUEUE_MEMORY_SIZE_STRING = "max.smaller.callqueue.memory.size";
+   public static final long MAX_LARGER_CALL_QUEUE_MEMORY_SIZE = 768*1024*1024;
+   public static final String MAX_LARGER_CALL_QUEUE_MEMORY_SIZE_STRING = "max.larger.callqueue.memory.size";
+   public static final int SMALL_QUEUE_REQUEST_LIMIT = 25*1024*1024;
+   public static final String SMALL_QUEUE_REQUEST_LIMIT_STRING = "small.queue.request.limit";
+
+  // These are the IO priority values for various regionserver operations. Note
+  // that these are priorities relative to each other. See the man page for
+  // ioprio_set for more details. The default priority for a process with nice
+  // value 0 is 4. The priorities range from 0 (highest) to 7 (lowest).
+  //
+  // The thinking behind the various priorities are as follows :
+  // 1. PREAD priority is the highest since client reads are extremely critical.
+  // 2. Although HLOG sync is as important as a pread (since the client
+  // blocks on it.). But the HLOG sync never hits disk in the critical path
+  // and these priorities are when the kernel scheduler writes data to the
+  // persistent store. This priority will only be considered when we close the
+  // HLOG and help in reducing any stalls while closing the hlog.
+  // 3. The priority for flush is more than compaction since if we don't flush
+  // quickly enough, the memstore might grow too much and block client updates.
+  public static final int PREAD_PRIORITY = 0;
+  public static final int HLOG_PRIORITY = 1;
+  public static final int FLUSH_PRIORITY = 2;
+  public static final int COMPACT_PRIORITY = 3;
+
+  // We use the Best Effort class always since RealTime and Idle are too
+  // extreme. Again check man pages for ioprio_set for more details.
+//  public static final int IOPRIO_CLASSOF_SERVICE = NativeIO.IOPRIO_CLASS_BE;
+
+  public static final String HBASE_ENABLE_QOS_KEY = "hbase.enable.qos";
+  public static final String HBASE_ENABLE_SYNCFILERANGE_THROTTLING_KEY = "hbase.enable.syncfilerange.throttling";
+
+  /*
+   * MSLAB Constants
+   */
+  public final static String MSLAB_CHUNK_POOL_MAX_SIZE_KEY = "hbase.hregion.memstore.chunkpool.maxsize";
+  public final static String MSLAB_CHUNK_POOL_INITIAL_SIZE_KEY = "hbase.hregion.memstore.chunkpool.initialsize";
+  public final static float MSLAB_POOL_MAX_SIZE_DEFAULT = 0.0f;
+  public final static float MSLAB_POOL_INITIAL_SIZE_DEFAULT = 0.0f;
+
+  public final static String MSLAB_CHUNK_SIZE_KEY = "hbase.hregion.memstore.mslab.chunksize";
+  public final static int MSLAB_CHUNK_SIZE_DEFAULT = 2 * 1024 * 1024;
+
+  public final static String MSLAB_MAX_ALLOC_KEY = "hbase.hregion.memstore.mslab.max.allocation";
+  public final static int MSLAB_MAX_ALLOC_DEFAULT = 256  * 1024; // allocs bigger than this don't go through allocator
+
+  public final static String MSLAB_MAX_SIZE_KEY = "hbase.hregion.memstore.mslab.max.size";
+  public final static float MSLAB_MAX_SIZE_DEFAULT = 1.25f; // Stop using SLAB if larger than this percentage of memstore size
+
+  public final static float MSLAB_PCT_LOWER_LIMIT = 0.0f;
+  public final static float MSLAB_PCT_UPPER_LIMIT = 2.0f;
+
+
+  /*
+   * Memstore Linear search limit
+   */
+  public final static String MEMSTORE_RESEEK_LINEAR_SEARCH_LIMIT_KEY = "hbase.hregion.memstore.linear.search.limit";
+  public final static int MEMSTORE_RESEEK_LINEAR_SEARCH_LIMIT_DEFAULT = 20;
+
+  public static final String USE_MSLAB_KEY = "hbase.hregion.memstore.mslab.enabled";
+  public static final boolean USE_MSLAB_DEFAULT = false;
+
+  /**
+   * This wait time is used to periodically probe until
+   * we exhaust the timeout in the window
+   */
+  public static final String WAIT_TIME_FOR_FLUSH_MS =
+      "hbase.hregion.flush.waittime";
+  public static final long DEFAULT_WAIT_TIME_FOR_FLUSH_MS = 100; //ms
+
+  /**
+   * The knob to turn on the ClientLocalScanner to flush and wait for the
+   * region flush to finish before it opens the store files.
+   * Set the socket timeout for the RPC appropriately for this.
+   */
+  public static final String CLIENT_LOCAL_SCANNER_FLUSH_AND_WAIT =
+      "hbase.clientlocalscanner.flush.and.wait";
+  public static final boolean DEFAULT_CLIENT_LOCAL_SCANNER_FLUSH_AND_WAIT =
+      false;
+
+  /**
+   * The acceptable staleness of a flush. Say if this value is set to 10s,
+   * if there was a flush in the last 10s, we would not flush again.
+   */
+  public static final String CLIENT_LOCAL_SCANNER_FLUSH_ACCEPTABLE_STALENESS_MS =
+      "hbase.clientlocalscanner.flush.acceptable.staleness";
+  public static final long DEFAULT_CLIENT_LOCAL_SCANNER_FLUSH_ACCEPTABLE_STALENESS_MS =
+      30000; // ms
+
+  /**
+   * The extra wait time that we wait for the flush to take place.
+   */
+  public static final String CLIENT_LOCAL_SCANNER_MAX_WAITTIME_FOR_FLUSH_MS =
+      "hbase.clientlocal.scanner.flush.maxwaittime";
+  public static final int DEFAULT_CLIENT_LOCAL_SCANNER_MAX_WAITTIME_FOR_FLUSH_MS
+    = 10000; // ms
+
+  public static final String RAFT_TRANSACTION_LOG_DIRECTORY_KEY =
+    "hbase.consensus.log.path";
+
+  public static final String RAFT_METADATA_DIRECTORY_KEY =
+    "hbase.consensus.metadata.path";
+
+  public static final String RAFT_TRANSACTION_LOG_IS_SYNC_KEY =
+    "hbase.consensus.log.issync";
+
+  public static final boolean RAFT_TRANSACTION_LOG_IS_SYNC_DEFAULT = true;
+
+  public static final String RAFT_LOG_DELETION_INTERVAL_KEY =
+    "hbase.quorum.log.deletion.interval";
+
+  public static final String RAFT_LOG_ROLL_INTERVAL_KEY =
+    "hbase.quorum.log.roll.interval";
+
+  public static final int DEFAULT_RAFT_LOG_ROLL_INTERVAL = 60000;
+
+  public static final int DEFAULT_RAFT_LOG_DELETION_INTERVAL = 60000;
+
+  public static final String QUORUM_MONITORING_PAGE_KEY = "paxos";
+
+  public static final String QUORUM_MONITORING_PAGE_PATH = "/" + QUORUM_MONITORING_PAGE_KEY;
+
+  public static final String ROLE_ASSIGNMENT_POLICY_CLASS = "hbase.role.assignment.policy.class";
+
+  public static final String CONSENSUS_TRANSPORT_PROTOCOL_KEY =
+    "hbase.consensus.transport.protocol";
+
+  public static final String CONSENSUS_TRANSPORT_PROTOCOL_DEFAULT =
+    "compact";
+
+  public static final String CONSENSUS_TRANSPORT_CODEC_KEY =
+    "hbase.consensus.transport.codec";
+
+  public static final String CONSENSUS_TRANSPORT_CODEC_DEFAULT =
+    "framed";
+
+  public static final String ENABLE_STEPDOWN_ON_HIGHER_RANK_CAUGHT_UP = "hbase.quorum.enable" +
+    ".stepdown.on.higher.rank.caught.up";
+
+  public static final String ROLE_ASSIGNMENT_POLICY_LOG_ONLY_DCS = "hbase.role.assignment.policy.log.only.dcs";
+
+  public static final String CONSENSUS_PUSH_APPEND_BATCH_SAME_TERM_KEY = "hbase.hydrabase.push.append.batch.same.term";
+
+  public static final String CONSENSUS_PUSH_APPEND_MAX_BATCH_LOGS_KEY = "hbase.hydrabase.push.append.max.batch.logs";
+
+  public static final String CONSENSUS_PUSH_APPEND_MAX_BATCH_BYTES_KEY = "hbase.hydrabase.push.append.max.batch.bytes";
+
+  public static final byte CONSENSUS_PAYLOAD_MAGIC_VALUE = 84;
+
+  public static final short CONSENSUS_PAYLOAD_HEADER_LENGTH = 4;
+
+  public static final byte BATCHED_WALEDIT_TYPE = 1;
+
+  public static final byte BATCHED_WALEDIT_VERSION = 3;
+
+  public static final byte QUORUM_MEMBERSHIP_CHANGE_TYPE = 2;
+
+  public static final byte QUORUM_MEMBERSHIP_CHANGE_VERSION = 1;
+
+  public static final String CONSENSUS_TRANCTION_LOG_RETENTION_TIME_KEY =
+    "hbase.consensus.log.retention.time";
+
+  public static final long CONSENSUS_TRANCTION_LOG_RETENTION_TIME_DEFAULT_VALUE = 60 * 60 * 1000;
+
+  public static final String CONSENSUS_TRANSACTION_LOG_COMPRESSION_CODEC_KEY =
+          "hbase.consensus.log.compression";
+  public static final String
+          CONSENSUS_TRANSACTION_LOG_COMPRESSION_CODEC_DEFAULT =
+          Compression.Algorithm.NONE.getName();
+
+  public static final String ARENA_CAPACITY_KEY = "hbase.arena.capacity";
+
+  public static final int ARENA_CAPACITY_DEFAULT = 2 * 1024 * 1024;
+
+  public static final String ARENA_BUCKETS_KEY = "hbase.arena.buckets";
+  public static final String ALL_HOSTS_LIST_PREFIX = "hbase.hydrabase.regionplacement.all.hosts.fileprefix";
+  public static final long RANDOM_SEED_FOR_REGION_PLACEMENT = 5555;// some random deterministic number. doesn't matter what it is.
+
+  public static final String USE_ARENA_KEY = "hbase.hydrabase.arena.enabled";
+
+  public static final boolean USE_ARENA_DEFAULT = false;
+
+  // Retry creating new transaction log file after several seconds
+  public static final long RETRY_TRANSACTION_LOG_CREATION_DELAY_IN_SECS = 10;
+
+  public static final String LOG_CREATOR_THREADPOOL_MAXSIZE_KEY = "hbase.hydrabase.logcreator.maxthreadnum";
+
+  /** Sleep interval for Quorum Client */
+  public static String QUORUM_CLIENT_SLEEP_INTERVAL_KEY = "hbase.consensus.quorum.client.sleep.interval";
+
+  /** Default interval for Quorum Client */
+  public static int QUORUM_CLIENT_SLEEP_INTERVAL_DEFAULT = 100;
+
+  /** Wait for leader timeout for QuorumClient */
+  public static final String QUORUM_CLIENT_COMMIT_DEADLINE_KEY =
+          "hbase.consensus.quorum.client.commit.deadline";
+  public static final long QUORUM_CLIENT_COMMIT_DEADLINE_DEFAULT =
+          6 * PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS;
+
+  /** Commit deadline for Quorum Agent */
+  public static final String QUORUM_AGENT_COMMIT_DEADLINE_KEY =
+          "hbase.consensus.quorum.client.commit.timeout";
+  public static final long QUORUM_AGENT_COMMIT_DEADLINE_DEFAULT = 1000;
+
+  /** Time after which a commit may be declared to have timed out. */
+  public static final String QUORUM_AGENT_COMMIT_TIMEOUT_KEY =
+          "hbase.consensus.quorum.client.commit.wait";
+  public static final long QUORUM_AGENT_COMMIT_TIMEOUT_DEFAULT = 100;
+
+  /** Commit queue entries limit */
+  public static final String QUORUM_AGENT_COMMIT_QUEUE_ENTRIES_LIMIT_KEY =
+          "hbase.consensus.quorum.agent.commit.entries.limit";
+  public static final long QUORUM_AGENT_COMMIT_QUEUE_ENTRIES_LIMIT_DEFAULT = 150;
+
+  /** Commit queue size limit */
+  public static final String QUORUM_AGENT_COMMIT_QUEUE_SIZE_LIMIT_KEY =
+          "hbase.consensus.quorum.agent.commit.queue.size.limit";
+  public static final long QUORUM_AGENT_COMMIT_QUEUE_SIZE_LIMIT_DEFAULT =
+          64 * 1024 * 1024; // 64 MB
+
+  /**
+   * Whether or not to use the {@link org.apache.hadoop.hbase.consensus.quorum.AggregateTimer}
+   * for timers.
+   */
+  public static String QUORUM_USE_AGGREGATE_TIMER_KEY = "hbase.consensus.quorum.useaggregatetimer";
+
+  /** Default value for the choice of timer */
+  public static boolean QUORUM_USE_AGGREGATE_TIMER_DEFAULT = true;
+
+  /**
+   * Knob to use the {@link org.apache.hadoop.hbase.consensus.fsm.FSMMultiplexer}
+   * for Raft State Machines.
+   */
+  public static String USE_FSMMUX_FOR_RSM = "hbase.consensus.quorum.usefsmmux.rsm";
+
+  public static boolean USE_FSMMUX_FOR_RSM_DEFAULT = true;
+
+  /**
+   * Knob to use the {@link org.apache.hadoop.hbase.consensus.fsm.FSMMultiplexer}
+   * for Peer State Machines.
+   */
+  public static String USE_FSMMUX_FOR_PSM = "hbase.consensus.quorum.usefsmmux.psm";
+
+  public static boolean USE_FSMMUX_FOR_PSM_DEFAULT = true;
+
+  public static final String CLIENT_SIDE_SCAN = "hbase.client.side.scan";
+  public static final boolean DEFAULT_CLIENT_SIDE_SCAN = false;
+  public static final String USE_HFILEHISTOGRAM = "hbase.client.hfilehistogram.enabled";
+  public static final boolean DEFAULT_USE_HFILEHISTOGRAM = true;
+
+  public static final String IN_MEMORY_BLOOM_ENABLED =
+      "hbase.hregion.memstore.bloom.filter.enabled";
+  public static final boolean DEFAULT_IN_MEMORY_BLOOM_ENABLED = false;
+
+  public static final String CONSENSUS_SERVER_WORKER_THREAD =
+    "hbase.consensus.thrift.worker.thread";
+
+  public static final int DEFAULT_CONSENSUS_SERVER_WORKER_THREAD = 10;
+
+  public static final int DEFAULT_CONSENSUS_SERVER_QUEUE_LENGTH = 1000;
+
+  public static final String CONSENSUS_SERVER_IO_THREAD =
+    "hbase.consensus.thrift.io.thread";
+
+  public static final int DEFAULT_CONSENSUS_SERVER_IO_THREAD = 10;
+
+  public static final String RAFT_LOG_READER_PREFETCH_KEY = "hbase.consensus.log.prefetch.size.bytes";
+  public static final int DEFAULT_RAFT_LOG_READER_PREFETCH_SIZE = 1048576;
+
+  public static String FSM_WRITEOPS_THREADPOOL_SIZE_KEY = "hbase.consensus.quorum.fsmwriteopsnumthreads";
+
+  /** Number of threads in the FSM Write Ops threadpool */
+  public static int FSM_WRITEOPS_THREADPOOL_SIZE_DEFAULT = 5;
+
+  public static String FSM_READOPS_THREADPOOL_SIZE_KEY = "hbase.consensus.quorum.fsmreadopsnumthreads";
+
+  /** Number of threads in the FSM Read Ops threadpool */
+  public static int FSM_READOPS_THREADPOOL_SIZE_DEFAULT = 10;
+
+  /** Number of threads in the FSM Multiplexer */
+  public static String FSM_MUX_THREADPOOL_SIZE_KEY = "hbase.consensus.quorum.fsm.muxthreadpool";
+
+  public static int DEFAULT_FSM_MUX_THREADPOOL_SIZE = 10;
+
+  public static String DEFER_REPLAY_IN_WITNESS_MODE = "hbase.hydrabase.log.witness.defer.replay";
+
+  public static String INVALID = "INVALID";
+
+  public static final int DEFAULT_QUORUM_CLIENT_NUM_WORKERS = 5;
+
+  public static final String QUORUM_CLIENT_RETRY_SLEEP_INTERVAL =
+    "hbase.hydrabase.quorum.client.retry.sleep.interval";
+
+  public static final long QUORUM_CLIENT_RETRY_SLEEP_INTERVAL_DEFAULT = 100;
+
+  public static final String MAXWAIT_TO_READ_FROM_DISK_NS =
+      "hbase.hydrabase.maxwait.toread.from.disk";
+
+  public static final long MAXWAIT_TO_READ_FROM_DISK_NS_DEFAULT = 500 * 1000 * 1000;
+
+  public static final String ALLOW_MAJORITY_MOVE =
+      "hbase.hydrabase.allow.majority.move";
+
+  public static final boolean ALLOW_MAJORITY_MOVE_DEFAULT = false;
+
+  public static final String HYDRABASE_REGIONPLACEMENT_UNBALANCE_TOLERANCE =
+      "hbase.hydrabase.regionplacement.unbalance.tolerance";
+  public static final int DEFAULT_HYDRABASE_REGIONPLACEMENT_UNBALANCE_TOLERANCE = 0;
+
+  public static final String HYDRABASE_DCNAMES = "hbase.hydrabase.dcnames";
+  public static final String HYDRABASE_DCNAME = "titan.cell.name";
+
+  // Whether the master can auto-generate and apply RMaps on host addition and
+  // failure.
+  public static final String HYDRABASE_AUTO_APPLY_RMAP =
+    "hbase.hydrabase.master.autoapplyrmap.enabled";
+  public static final boolean DEFAULT_HYDRABASE_AUTO_APPLY_RMAP = true;
+
+  // How long to wait between auto applying rmap.
+  public static final String HYDRABASE_AUTO_APPLY_RMAP_DURATION_MS_KEY =
+    "hbase.hydrabase.master.autoapplyrmap.durationms";
+  public static final int DEFAULT_HYDRABASE_AUTO_APPLY_RMAP_DURATION_MS =
+    5 * 60 * 1000;
+
+  // The default threshold for Region Placement validation to pass
+  public static final String RP_VALIDATION_THRESHOLD_KEY =
+    "hbase.hydrabase.master.regionplacement.validation.threshold";
+  public static final int DEFAULT_RP_VALIDATION_THRESHOLD = 10;
+
+  public static final int DEFAULT_HBASE_QUORUM_SIZE = 3;
+
+  private HConstants() {
+    // Can't be instantiated with this constructor.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
new file mode 100644
index 0000000..a46e187
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -0,0 +1,838 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.*;
+import java.util.Map.Entry;
+
+import com.google.common.base.Joiner;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JenkinsHash;
+import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.hadoop.io.VersionedWritable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * HRegion information.
+ * Contains HRegion id, start and end keys, a reference to this
+ * HRegions' table descriptor, etc.
+ */
+public class HRegionInfo extends VersionedWritable implements WritableComparable<HRegionInfo>{
+  private static final byte VERSION = 0;
+  private static final Log LOG = LogFactory.getLog(HRegionInfo.class);
+  protected Map<String, InetSocketAddress[]> favoredNodesMap = new HashMap<>();
+
+  /**
+   * The new format for a region name contains its encodedName at the end.
+   * The encoded name also serves as the directory name for the region
+   * in the filesystem.
+   *
+   * New region name format:
+   *    &lt;tablename>,,&lt;startkey>,&lt;regionIdTimestamp>.&lt;encodedName>.
+   * where,
+   *    &lt;encodedName> is a hex version of the MD5 hash of
+   *    &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
+   *
+   * The old region name format:
+   *    &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
+   * For region names in the old format, the encoded name is a 32-bit
+   * JenkinsHash integer value (in its decimal notation, string form).
+   *<p>
+   * **NOTE**
+   *
+   * ROOT, the first META region, and regions created by an older
+   * version of HBase (0.20 or prior) will continue to use the
+   * old region name format.
+   */
+
+  /** Separator used to demarcate the encodedName in a region name
+   * in the new format. See description on new format above.
+   */
+  private static final int ENC_SEPARATOR = '.';
+  public  static final int MD5_HEX_LENGTH   = 32;
+
+  /**
+   * Does region name contain its encoded name?
+   * @param regionName region name
+   * @return boolean indicating if this a new format region
+   *         name which contains its encoded name.
+   */
+  private static boolean hasEncodedName(final byte[] regionName) {
+    // check if region name ends in ENC_SEPARATOR
+    if ((regionName.length >= 1)
+        && (regionName[regionName.length - 1] == ENC_SEPARATOR)) {
+      // region name is new format. it contains the encoded name.
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * @param regionName
+   * @return the encodedName
+   */
+  public static String encodeRegionName(final byte [] regionName) {
+    String encodedName;
+    if (hasEncodedName(regionName)) {
+      // region is in new format:
+      // <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
+      encodedName = Bytes.toString(regionName,
+          regionName.length - MD5_HEX_LENGTH - 1,
+          MD5_HEX_LENGTH);
+    } else {
+      // old format region name. ROOT and first META region also
+      // use this format.EncodedName is the JenkinsHash value.
+      int hashVal = Math.abs(JenkinsHash.getInstance().hash(regionName,
+                                                            regionName.length,
+                                                            0));
+      encodedName = String.valueOf(hashVal);
+    }
+    return encodedName;
+  }
+
+  /** delimiter used between portions of a region name */
+  public static final int DELIMITER = ',';
+
+  /** HRegionInfo for root region */
+  public static final HRegionInfo ROOT_REGIONINFO =
+    new HRegionInfo(0L, HTableDescriptor.ROOT_TABLEDESC);
+
+  /** Encoded name for the root region. This is always the same. */
+  public static final String ROOT_REGION_ENCODED_NAME_STR =
+      HRegionInfo.ROOT_REGIONINFO.getEncodedName();
+
+  /** HRegionInfo for first meta region */
+  public static final HRegionInfo FIRST_META_REGIONINFO =
+    new HRegionInfo(1L, HTableDescriptor.META_TABLEDESC);
+
+  private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
+  private boolean offLine = false;
+  private long regionId = -1;
+  private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
+  private String regionNameStr = "";
+  private boolean split = false;
+  private byte [] splitPoint = null;
+  private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
+  protected HTableDescriptor tableDesc = null;
+  private int hashCode = -1;
+  //TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
+  public static final String NO_HASH = null;
+  private volatile String encodedName = NO_HASH;
+
+  // Peers of the Consensus Quorum
+  private QuorumInfo quorumInfo;
+  // For compatability with non-hydrabase mode
+  public static String LOCAL_DC_KEY = "LOCAL_DC_KEY_FOR_NON_HYDRABASE_MODE";
+
+  private void setHashCode() {
+    int result = Arrays.hashCode(this.regionName);
+    result ^= this.regionId;
+    result ^= Arrays.hashCode(this.startKey);
+    result ^= Arrays.hashCode(this.endKey);
+    result ^= Boolean.valueOf(this.offLine).hashCode();
+    result ^= this.tableDesc.hashCode();
+    this.hashCode = result;
+  }
+
+  /**
+   * Private constructor used constructing HRegionInfo for the catalog root and
+   * first meta regions
+   */
+  private HRegionInfo(long regionId, HTableDescriptor tableDesc) {
+    super();
+    this.regionId = regionId;
+    this.tableDesc = tableDesc;
+
+    // Note: Root & First Meta regions names are still in old format
+    this.regionName = createRegionName(tableDesc.getName(), null,
+                                       regionId, false);
+    this.regionNameStr = Bytes.toStringBinary(this.regionName);
+    setHashCode();
+  }
+
+  /** Default constructor - creates empty object */
+  public HRegionInfo() {
+    super();
+    this.tableDesc = new HTableDescriptor();
+  }
+
+  /**
+   * Construct HRegionInfo with explicit parameters
+   *
+   * @param tableDesc the table descriptor
+   * @param startKey first key in region
+   * @param endKey end of key range
+   * @throws IllegalArgumentException
+   */
+  public HRegionInfo(final HTableDescriptor tableDesc, final byte [] startKey,
+      final byte [] endKey)
+  throws IllegalArgumentException {
+    this(tableDesc, startKey, endKey, false);
+  }
+
+  /**
+   * Construct HRegionInfo with explicit parameters
+   *
+   * @param tableDesc the table descriptor
+   * @param startKey first key in region
+   * @param endKey end of key range
+   * @param split true if this region has split and we have daughter regions
+   * regions that may or may not hold references to this region.
+   * @throws IllegalArgumentException
+   */
+  public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey,
+      final byte [] endKey, final boolean split)
+  throws IllegalArgumentException {
+    this(tableDesc, startKey, endKey, split, System.currentTimeMillis());
+  }
+
+  /**
+   * Construct HRegionInfo with explicit parameters
+   *
+   * @param tableDesc the table descriptor
+   * @param startKey first key in region
+   * @param endKey end of key range
+   * @param split true if this region has split and we have daughter regions
+   * regions that may or may not hold references to this region.
+   * @param regionid Region id to use.
+   * @throws IllegalArgumentException
+   */
+  public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey,
+    final byte [] endKey, final boolean split, final long regionid)
+  throws IllegalArgumentException {
+    this(tableDesc, startKey, endKey, split, regionid, null, null);
+  }
+
+  /**
+   * Construct HRegionInfo with explicit parameters
+   *
+   * @param tableDesc the table descriptor
+   * @param startKey first key in region
+   * @param endKey end of key range
+   * @param split true if this region has split and we have daughter regions
+   * regions that may or may not hold references to this region.
+   * @param regionid Region id to use.
+   * @throws IllegalArgumentException
+   */
+  public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey,
+    final byte [] endKey, final boolean split, final long regionid,
+      final Map<String, Map<HServerAddress, Integer>> peers,
+    final Map<String, InetSocketAddress[]> favoredNodesMap)
+  throws IllegalArgumentException {
+    super();
+    if (tableDesc == null) {
+      throw new IllegalArgumentException("tableDesc cannot be null");
+    }
+    this.offLine = false;
+    this.regionId = regionid;
+    this.regionName = createRegionName(tableDesc.getName(), startKey, regionId,
+        !tableDesc.isMetaRegion());
+    this.regionNameStr = Bytes.toStringBinary(this.regionName);
+    this.split = split;
+    this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
+    this.startKey = startKey == null?
+      HConstants.EMPTY_START_ROW: startKey.clone();
+    this.tableDesc = tableDesc;
+    this.quorumInfo = new QuorumInfo(peers, getEncodedName());
+    this.favoredNodesMap = favoredNodesMap == null ?
+            new HashMap<String, InetSocketAddress[]>() : favoredNodesMap;
+    setHashCode();
+  }
+
+  /**
+   * Costruct a copy of another HRegionInfo
+   *
+   * @param other
+   */
+  public HRegionInfo(HRegionInfo other) {
+    super();
+    this.endKey = other.getEndKey();
+    this.offLine = other.isOffline();
+    this.regionId = other.getRegionId();
+    this.regionName = other.getRegionName();
+    this.regionNameStr = Bytes.toStringBinary(this.regionName);
+    this.split = other.isSplit();
+    this.startKey = other.getStartKey();
+    this.tableDesc = other.getTableDesc();
+    this.hashCode = other.hashCode();
+    this.encodedName = other.getEncodedName();
+    this.quorumInfo = other.quorumInfo;
+    this.favoredNodesMap = other.favoredNodesMap;
+  }
+
+  private static byte [] createRegionName(final byte [] tableName,
+      final byte [] startKey, final long regionid, boolean newFormat) {
+    return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param id Region id.
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey and id
+   */
+  public static byte [] createRegionName(final byte [] tableName,
+      final byte [] startKey, final String id, boolean newFormat) {
+    return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
+  }
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param id Region id
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey and id
+   */
+  public static byte [] createRegionName(final byte [] tableName,
+      final byte [] startKey, final byte [] id, boolean newFormat) {
+    byte [] b = new byte [tableName.length + 2 + id.length +
+       (startKey == null? 0: startKey.length) +
+       (newFormat ? (MD5_HEX_LENGTH + 2) : 0)];
+
+    int offset = tableName.length;
+    System.arraycopy(tableName, 0, b, 0, offset);
+    b[offset++] = DELIMITER;
+    if (startKey != null && startKey.length > 0) {
+      System.arraycopy(startKey, 0, b, offset, startKey.length);
+      offset += startKey.length;
+    }
+    b[offset++] = DELIMITER;
+    System.arraycopy(id, 0, b, offset, id.length);
+    offset += id.length;
+
+    if (newFormat) {
+      //
+      // Encoded name should be built into the region name.
+      //
+      // Use the region name thus far (namely, <tablename>,<startKey>,<id>)
+      // to compute a MD5 hash to be used as the encoded name, and append
+      // it to the byte buffer.
+      //
+      String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
+      byte [] md5HashBytes = Bytes.toBytes(md5Hash);
+
+      if (md5HashBytes.length != MD5_HEX_LENGTH) {
+        LOG.error("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
+                  "; Got=" + md5HashBytes.length);
+      }
+
+      // now append the bytes '.<encodedName>.' to the end
+      b[offset++] = ENC_SEPARATOR;
+      System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
+      offset += MD5_HEX_LENGTH;
+      b[offset++] = ENC_SEPARATOR;
+    }
+
+    return b;
+  }
+
+  /**
+   * Separate elements of a regionName.
+   * @param regionName
+   * @return Array of byte[] containing tableName, startKey and id
+   * @throws IOException
+   */
+  public static byte [][] parseRegionName(final byte [] regionName)
+  throws IOException {
+    int offset = -1;
+    for (int i = 0; i < regionName.length; i++) {
+      if (regionName[i] == DELIMITER) {
+        offset = i;
+        break;
+      }
+    }
+    if(offset == -1) {
+      throw new IOException("Invalid regionName format: " +
+                            Bytes.toStringBinary(regionName));
+    }
+    byte [] tableName = new byte[offset];
+    System.arraycopy(regionName, 0, tableName, 0, offset);
+    offset = -1;
+    for (int i = regionName.length - 1; i > 0; i--) {
+      if(regionName[i] == DELIMITER) {
+        offset = i;
+        break;
+      }
+    }
+    if(offset == -1) {
+      throw new IOException("Invalid regionName format: " +
+                            Bytes.toStringBinary(regionName));
+    }
+    byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
+    if(offset != tableName.length + 1) {
+      startKey = new byte[offset - tableName.length - 1];
+      System.arraycopy(regionName, tableName.length + 1, startKey, 0,
+          offset - tableName.length - 1);
+    }
+    byte [] id = new byte[regionName.length - offset - 1];
+    System.arraycopy(regionName, offset + 1, id, 0,
+        regionName.length - offset - 1);
+    byte [][] elements = new byte[3][];
+    elements[0] = tableName;
+    elements[1] = startKey;
+    elements[2] = id;
+    return elements;
+  }
+
+  /** @return the regionId */
+  public long getRegionId(){
+    return regionId;
+  }
+
+  /**
+   * @return the regionName as an array of bytes.
+   * @see #getRegionNameAsString()
+   */
+  public byte [] getRegionName(){
+    return regionName;
+  }
+
+  /**
+   * @return Region name as a String for use in logging, etc.
+   */
+  public String getRegionNameAsString() {
+    if (hasEncodedName(this.regionName)) {
+      // new format region names already have their encoded name.
+      return this.regionNameStr;
+    }
+
+    // old format. regionNameStr doesn't have the region name.
+    //
+    //
+    return this.regionNameStr + "." + this.getEncodedName();
+  }
+
+  /** @return the encoded region name */
+  public synchronized String getEncodedName() {
+    if (this.encodedName == NO_HASH) {
+      this.encodedName = encodeRegionName(this.regionName);
+    }
+    return this.encodedName;
+  }
+
+  /** @return the startKey */
+  public byte [] getStartKey(){
+    return startKey;
+  }
+
+  /** @return the endKey */
+  public byte [] getEndKey(){
+    return endKey;
+  }
+
+  /**
+   * Returns true if the given inclusive range of rows is fully contained
+   * by this region. For example, if the region is foo,a,g and this is
+   * passed ["b","c"] or ["a","c"] it will return true, but if this is passed
+   * ["b","z"] it will return false.
+   * @throws IllegalArgumentException if the range passed is invalid (ie end < start)
+   */
+  public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
+    if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
+      throw new IllegalArgumentException(
+      "Invalid range: " + Bytes.toStringBinary(rangeStartKey) +
+      " > " + Bytes.toStringBinary(rangeEndKey));
+    }
+
+    boolean firstKeyInRange = Bytes.compareTo(rangeStartKey, startKey) >= 0;
+    boolean lastKeyInRange =
+      Bytes.compareTo(rangeEndKey, endKey) < 0 ||
+      Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY);
+    return firstKeyInRange && lastKeyInRange;
+  }
+
+  /**
+   * Return true if the given row falls in this region.
+   */
+  public boolean containsRow(byte[] row) {
+    return Bytes.compareTo(row, startKey) >= 0 &&
+      (Bytes.compareTo(row, endKey) < 0 ||
+       Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
+  }
+
+  /** @return the tableDesc */
+  public HTableDescriptor getTableDesc(){
+    return tableDesc;
+  }
+
+  /**
+   * @param newDesc new table descriptor to use
+   */
+  public void setTableDesc(HTableDescriptor newDesc) {
+    this.tableDesc = newDesc;
+  }
+
+  /** @return true if this is the root region */
+  public boolean isRootRegion() {
+    return this.tableDesc.isRootRegion();
+  }
+
+  /** @return true if this is the meta table */
+  public boolean isMetaTable() {
+    return this.tableDesc.isMetaTable();
+  }
+
+  /** @return true if this region is a meta region */
+  public boolean isMetaRegion() {
+    return this.tableDesc.isMetaRegion();
+  }
+
+  /**
+   * @return True if has been split and has daughters.
+   */
+  public boolean isSplit() {
+    return this.split;
+  }
+
+  /**
+   * @param split set split status
+   */
+  public void setSplit(boolean split) {
+    this.split = split;
+  }
+
+  /**
+   * @return point to explicitly split the region on
+   */
+  public byte[] getSplitPoint() {
+    return (this.splitPoint != null && this.splitPoint.length > 0)
+      ? this.splitPoint : null;
+  }
+
+  /**
+   * @param splitPoint set split status & position to split on
+   */
+  public void setSplitPoint(byte[] splitPoint) {
+    this.split = true;
+    this.splitPoint = splitPoint;
+  }
+
+  /**
+   * @return True if this region is offline.
+   */
+  public boolean isOffline() {
+    return this.offLine;
+  }
+
+  /**
+   * @param offLine set online - offline status
+   */
+  public void setOffline(boolean offLine) {
+    this.offLine = offLine;
+  }
+
+  /**
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    return String.format("REGION => {%s => '%s', STARTKEY => '%s', " +
+            "ENDKEY => '%s', ENCODED => %s, OFFLINE => %s, SPLIT => %s, " +
+            "TABLE => {%s}, FAVORED_NODES_MAP => {%s}}",
+            HConstants.NAME, regionNameStr, Bytes.toStringBinary(startKey),
+            Bytes.toStringBinary(endKey), getEncodedName(), isOffline(),
+            isSplit(), tableDesc.toString(),
+            favoredNodesMap != null ? prettyPrintFavoredNodesMap() : "");
+  }
+  
+  /**
+   * @see java.lang.Object#equals(java.lang.Object)
+   *
+   * TODO (arjen): this does not consider split and split point!
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null) {
+      return false;
+    }
+    if (!(o instanceof HRegionInfo)) {
+      return false;
+    }
+
+    HRegionInfo that = (HRegionInfo)o;
+    if (this.compareTo(that) != 0) {
+      return false;
+    }
+
+    if (this.quorumInfo == null && that.quorumInfo != null) {
+      return false;
+    }
+    if (this.quorumInfo != null && !this.quorumInfo.equals(that.quorumInfo)) {
+      return false;
+    }
+
+    return hasSameFavoredNodesMap(that);
+  }
+
+  /**
+   * @see java.lang.Object#hashCode()
+   */
+  @Override
+  public int hashCode() {
+    return this.hashCode;
+  }
+
+  /** @return the object version number */
+  @Override
+  public byte getVersion() {
+    return VERSION;
+  }
+
+  //
+  // Writable
+  //
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    Bytes.writeByteArray(out, endKey);
+    out.writeBoolean(offLine);
+    out.writeLong(regionId);
+    Bytes.writeByteArray(out, regionName);
+    out.writeBoolean(split);
+    if (split) {
+      Bytes.writeByteArray(out, splitPoint);
+    }
+    Bytes.writeByteArray(out, startKey);
+    tableDesc.write(out);
+    out.writeInt(hashCode);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    this.endKey = Bytes.readByteArray(in);
+    this.offLine = in.readBoolean();
+    this.regionId = in.readLong();
+    this.regionName = Bytes.readByteArray(in);
+    this.regionNameStr = Bytes.toStringBinary(this.regionName);
+    this.split = in.readBoolean();
+    if (this.split) {
+      this.splitPoint = Bytes.readByteArray(in);
+    }
+    this.startKey = Bytes.readByteArray(in);
+    this.tableDesc.readFields(in);
+    this.hashCode = in.readInt();
+    if (quorumInfo == null) {
+      quorumInfo = new QuorumInfo(
+        new HashMap<String, Map<HServerAddress, Integer>>(),
+        HRegionInfo.encodeRegionName(regionName));
+    }
+  }
+
+  //
+  // Comparable
+  //
+
+  public int compareTo(HRegionInfo o) {
+    if (o == null) {
+      return 1;
+    }
+
+    // Are regions of same table?
+    int result = this.tableDesc.compareTo(o.tableDesc);
+    if (result != 0) {
+      return result;
+    }
+
+    // Compare start keys.
+    result = Bytes.compareTo(this.startKey, o.startKey);
+    if (result != 0) {
+      return result;
+    }
+
+    // Compare end keys.
+    return Bytes.compareTo(this.endKey, o.endKey);
+  }
+
+  /**
+   * @return Comparator to use comparing {@link KeyValue}s.
+   */
+  public KVComparator getComparator() {
+    return isRootRegion()? KeyValue.ROOT_COMPARATOR: isMetaRegion()?
+      KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
+  }
+
+  public Map<HServerAddress,Integer> getPeersWithRank() {
+    return getQuorumInfo().getPeersWithRank();
+  }
+
+  public Map<HServerAddress, String> getPeersWithCluster() {
+    return getQuorumInfo().getPeersWithCluster();
+  }
+
+  @Deprecated
+  public InetSocketAddress[] getFavoredNodes() {
+    return getFavoredNodes(LOCAL_DC_KEY);
+  }
+
+  public InetSocketAddress[] getFavoredNodes(String dcKey) {
+    return this.favoredNodesMap != null?
+              this.favoredNodesMap.get(dcKey):
+                null;
+  }
+
+  @Deprecated
+  public void setFavoredNodes(InetSocketAddress[] favoredNodes) {
+    setFavoredNodes(LOCAL_DC_KEY, favoredNodes);
+  }
+
+  public void setFavoredNodes(String dcName, InetSocketAddress[] favoredNodes) {
+    if (this.favoredNodesMap == null) {
+      this.favoredNodesMap = new HashMap<>();
+    }
+    this.favoredNodesMap.put(dcName, favoredNodes);
+    setHashCode();
+  }
+
+  public void setPeers(Map<String, Map<HServerAddress, Integer>> peers) {
+    this.quorumInfo.setPeers(peers);
+  }
+
+  public Map<String, Map<HServerAddress, Integer>> getPeers() {
+    QuorumInfo quorumInfo = getQuorumInfo();
+    if (quorumInfo != null) {
+      return quorumInfo.getPeers();
+    }
+    return null;
+  }
+
+  public Map<String, InetSocketAddress[]> getFavoredNodesMap() {
+    return favoredNodesMap;
+  }
+
+  public void setFavoredNodesMap(
+          final Map<String, InetSocketAddress[]> favoredNodesMap) {
+    this.favoredNodesMap = favoredNodesMap;
+  }
+
+  public boolean hasSameFavoredNodesMap(final HRegionInfo that) {
+    if (that == null) {
+      return false;
+    }
+
+    if (!this.favoredNodesMap.keySet().equals(that.favoredNodesMap.keySet())) {
+      return false;
+    }
+
+    for (String domain : this.favoredNodesMap.keySet()) {
+      if (!Arrays.equals(this.favoredNodesMap.get(domain),
+              that.favoredNodesMap.get(domain))) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public QuorumInfo getQuorumInfo() {
+    return quorumInfo;
+  }
+
+  public void setQuorumInfo(final QuorumInfo quorumInfo) {
+    this.quorumInfo = quorumInfo;
+  }
+
+  public String prettyPrintFavoredNodesMap() {
+    if (favoredNodesMap == null) {
+      return "";
+    }
+    StringBuilder sb = new StringBuilder(128);
+    Iterator<Entry<String, InetSocketAddress[]>> it
+            = favoredNodesMap.entrySet().iterator();
+    while (it.hasNext()) {
+      Map.Entry<String, InetSocketAddress[]> domain = it.next();
+      InetSocketAddress[] favoredNodes = domain.getValue();
+      sb.append(domain.getKey());
+      sb.append(" => [");
+      if (favoredNodes != null) {
+        sb.append(Joiner.on(", ").join(favoredNodes));
+      }
+      sb.append(it.hasNext() ? "], " : "]");
+    }
+    return sb.toString();
+  }
+
+  public static class MultiDCHRegionInfo extends HRegionInfo {
+    private Map<String, Map<HServerAddress, Integer>> combinedPeersMap;
+
+    public MultiDCHRegionInfo(String dcsite, HRegionInfo regionInfo) {
+      super(regionInfo);
+      this.favoredNodesMap = new HashMap<>();
+      this.favoredNodesMap.put(dcsite, regionInfo.getFavoredNodes());
+      this.combinedPeersMap = regionInfo.getPeers();
+    }
+
+    public void merge(String otherDC, HRegionInfo other) {
+      this.favoredNodesMap.put(otherDC, other.getFavoredNodes());
+    }
+
+    public void validate(int quorumSize, Map<String, Integer> maxPeersPerDC)
+      throws IllegalArgument {
+      if (favoredNodesMap.size() == 0) {
+        return;
+      }
+
+      int rankNum = quorumSize;
+      for (String cluster : maxPeersPerDC.keySet()) {
+        int numPeerAssignedPerDC = maxPeersPerDC.get(cluster).intValue();
+        if (combinedPeersMap.get(cluster) == null) {
+          combinedPeersMap.put(cluster, new HashMap
+            <HServerAddress, Integer>());
+        }
+        InetSocketAddress[] peerAddr = favoredNodesMap.get(cluster);
+        for (InetSocketAddress addr : peerAddr) {
+          this.combinedPeersMap.get(cluster).put(new HServerAddress(addr), rankNum--);
+          if (--numPeerAssignedPerDC == 0) {
+            break;
+          }
+        }
+        if (rankNum <= 0) {
+          break;
+        }
+      }
+
+      if (rankNum > 0) {
+        throw new IllegalArgument("Not enough nodes to complete the peer" +
+          " the peer assignment.");
+      }
+    }
+
+    @Override
+    public Map<String, Map<HServerAddress, Integer>> getPeers() {
+      return combinedPeersMap;
+    }
+  }
+}


[19/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
HBASE-12476 HydraBase consensus protocol

Signed-off-by: Elliott Clark <el...@fb.com>
Signed-off-by: Elliott Clark <ec...@apache.org>


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

Branch: refs/heads/HBASE-12259
Commit: eca32aa4ac4538b0334ea83d3b3bc57ec29e7f65
Parents: e6b4300
Author: Gaurav Menghani <ga...@fb.com>
Authored: Fri Nov 21 11:26:55 2014 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Tue Nov 25 12:17:42 2014 -0800

----------------------------------------------------------------------
 hbase-consensus/pom.xml                         |  349 +++
 .../apache/hadoop/hbase/HBaseConfiguration.java |  145 ++
 .../apache/hadoop/hbase/HColumnDescriptor.java  | 1001 ++++++++
 .../org/apache/hadoop/hbase/HConstants.java     | 1296 ++++++++++
 .../org/apache/hadoop/hbase/HRegionInfo.java    |  838 +++++++
 .../org/apache/hadoop/hbase/HServerAddress.java |  243 ++
 .../apache/hadoop/hbase/HTableDescriptor.java   |  792 ++++++
 .../java/org/apache/hadoop/hbase/KeyValue.java  | 2300 ++++++++++++++++++
 .../apache/hadoop/hbase/RegionException.java    |   56 +
 .../client/NoLeaderForRegionException.java      |   22 +
 .../hbase/conf/ConfigurationObserver.java       |   36 +
 .../hbase/consensus/client/FetchTask.java       |   65 +
 .../hbase/consensus/client/QuorumClient.java    |  285 +++
 .../consensus/client/QuorumLoadTestClient.java  |  161 ++
 .../client/QuorumThriftClientAgent.java         |  342 +++
 .../consensus/client/QuorumThriftClientCLI.java |   72 +
 .../CommitQueueOverloadedException.java         |    9 +
 .../exceptions/LeaderNotReadyException.java     |    9 +
 .../exceptions/NewLeaderException.java          |   20 +
 .../exceptions/NotEnoughMemoryException.java    |   16 +
 .../hadoop/hbase/consensus/fsm/Conditional.java |    9 +
 .../consensus/fsm/ConstitutentFSMService.java   |  178 ++
 .../hadoop/hbase/consensus/fsm/Event.java       |   52 +
 .../hadoop/hbase/consensus/fsm/EventType.java   |    3 +
 .../fsm/FSMLargeOpsExecutorService.java         |   91 +
 .../hadoop/hbase/consensus/fsm/FSMMetrics.java  |   24 +
 .../hbase/consensus/fsm/FiniteStateMachine.java |  200 ++
 .../consensus/fsm/FiniteStateMachineIf.java     |   57 +
 .../fsm/FiniteStateMachineService.java          |   67 +
 .../fsm/FiniteStateMachineServiceImpl.java      |  132 +
 .../hbase/consensus/fsm/MutableContext.java     |    3 +
 .../hadoop/hbase/consensus/fsm/OnEvent.java     |   13 +
 .../hadoop/hbase/consensus/fsm/State.java       |   67 +
 .../hadoop/hbase/consensus/fsm/StateType.java   |    3 +
 .../hadoop/hbase/consensus/fsm/Transition.java  |   33 +
 .../hbase/consensus/fsm/TransitionType.java     |    3 +
 .../hbase/consensus/fsm/Unconditional.java      |    9 +
 .../apache/hadoop/hbase/consensus/fsm/Util.java |   27 +
 .../hbase/consensus/log/CachedFileChannel.java  |  232 ++
 .../consensus/log/CandidateLogsManager.java     | 1249 ++++++++++
 .../log/CommitLogManagerInterface.java          |   69 +
 .../hbase/consensus/log/InMemoryLogManager.java |  242 ++
 .../consensus/log/LogFetchPlanCreator.java      |   13 +
 .../hadoop/hbase/consensus/log/LogFileInfo.java |  138 ++
 .../consensus/log/LogFileInfoIterator.java      |  109 +
 .../hbase/consensus/log/LogFileInterface.java   |   25 +
 .../hbase/consensus/log/LogFileViewer.java      |  108 +
 .../hadoop/hbase/consensus/log/LogReader.java   |  392 +++
 .../hadoop/hbase/consensus/log/LogWriter.java   |  201 ++
 .../hbase/consensus/log/RandomAccessLog.java    |  451 ++++
 .../hadoop/hbase/consensus/log/ReadOnlyLog.java |  185 ++
 .../hbase/consensus/log/RemoteLogFetcher.java   |   96 +
 .../hadoop/hbase/consensus/log/SeedLogFile.java |   52 +
 .../consensus/log/TransactionLogCreator.java    |  151 ++
 .../consensus/log/TransactionLogManager.java    | 1827 ++++++++++++++
 .../consensus/metrics/ConsensusMetrics.java     |  379 +++
 .../hbase/consensus/metrics/PeerMetrics.java    |  113 +
 .../hbase/consensus/protocol/ConsensusHost.java |   55 +
 .../protocol/DataStoreCommitEvent.java          |   37 +
 .../hadoop/hbase/consensus/protocol/EditId.java |  101 +
 .../hbase/consensus/protocol/Payload.java       |   25 +
 .../consensus/quorum/AbstractPeerManager.java   |  134 +
 .../hbase/consensus/quorum/AggregateTimer.java  |   93 +
 .../quorum/AppendConsensusSession.java          |  247 ++
 .../quorum/AppendConsensusSessionInterface.java |   17 +
 .../quorum/AppendResponseCallBack.java          |   26 +
 .../consensus/quorum/ConsensusSession.java      |    9 +
 .../consensus/quorum/ConstituentTimer.java      |  109 +
 .../quorum/HeartbeatTimeoutCallback.java        |   31 +
 .../consensus/quorum/ImmutableRaftContext.java  |  147 ++
 .../quorum/JointAppendConsensusSession.java     |   97 +
 .../quorum/JointConsensusPeerManager.java       |  179 ++
 .../quorum/JointVoteConsensusSession.java       |   56 +
 .../consensus/quorum/MutableRaftContext.java    |  125 +
 .../consensus/quorum/PeerManagerInterface.java  |   39 +
 .../quorum/ProgressTimeoutCallback.java         |   32 +
 .../hbase/consensus/quorum/QuorumAgent.java     |  463 ++++
 .../hbase/consensus/quorum/QuorumInfo.java      |  357 +++
 .../quorum/QuorumMembershipChangeRequest.java   |   38 +
 .../consensus/quorum/RaftQuorumContext.java     | 1415 +++++++++++
 .../hbase/consensus/quorum/RepeatingTimer.java  |   93 +
 .../hbase/consensus/quorum/ReseedRequest.java   |   27 +
 .../hbase/consensus/quorum/SessionResult.java   |    9 +
 .../quorum/SingleConsensusPeerManager.java      |   83 +
 .../consensus/quorum/TimeoutEventHandler.java   |    5 +
 .../hadoop/hbase/consensus/quorum/Timer.java    |   12 +
 .../consensus/quorum/VoteConsensusSession.java  |  101 +
 .../quorum/VoteConsensusSessionInterface.java   |   11 +
 .../hbase/consensus/raft/RaftStateMachine.java  |  231 ++
 .../raft/events/AppendNotCompleted.java         |   32 +
 .../raft/events/AppendRequestEvent.java         |   23 +
 .../raft/events/AppendRequestTimeout.java       |   24 +
 .../raft/events/AppendResponseEvent.java        |   17 +
 .../consensus/raft/events/AppendRetry.java      |   28 +
 .../consensus/raft/events/AppendSucceeded.java  |   28 +
 .../consensus/raft/events/IsCandidate.java      |   23 +
 .../hbase/consensus/raft/events/IsFollower.java |   22 +
 .../hbase/consensus/raft/events/IsLeader.java   |   23 +
 .../events/IsTransactionLogNotAccessible.java   |   19 +
 .../consensus/raft/events/NeedStepDown.java     |   29 +
 .../NoQuorumMembershipChangeInProgress.java     |   23 +
 .../raft/events/ProgressTimeoutEvent.java       |    9 +
 .../events/QuorumMembershipChangeEvent.java     |   18 +
 .../QuorumMembershipChangeInProgress.java       |   23 +
 .../consensus/raft/events/RaftEventType.java    |   19 +
 .../raft/events/ReplicateEntriesEvent.java      |   90 +
 .../raft/events/ReseedRequestEvent.java         |   23 +
 .../hbase/consensus/raft/events/StartEvent.java |    9 +
 .../hbase/consensus/raft/events/VoteFailed.java |   28 +
 .../consensus/raft/events/VoteNotCompleted.java |   31 +
 .../consensus/raft/events/VoteRequestEvent.java |   24 +
 .../raft/events/VoteResponseEvent.java          |   17 +
 .../consensus/raft/events/VoteSucceeded.java    |   29 +
 .../hbase/consensus/raft/states/AckClient.java  |   28 +
 .../consensus/raft/states/BecomeFollower.java   |   27 +
 .../consensus/raft/states/BecomeLeader.java     |   73 +
 .../hbase/consensus/raft/states/Candidate.java  |   19 +
 .../raft/states/ChangeQuorumMembership.java     |  167 ++
 .../hbase/consensus/raft/states/Follower.java   |   19 +
 .../hbase/consensus/raft/states/Halt.java       |   26 +
 .../raft/states/HandleAppendRequest.java        |  402 +++
 .../raft/states/HandleAppendResponse.java       |   64 +
 .../HandleQuorumMembershipChangeRequest.java    |  108 +
 .../raft/states/HandleReseedRequest.java        |   40 +
 .../raft/states/HandleVoteRequest.java          |  105 +
 .../raft/states/HandleVoteResponse.java         |   59 +
 .../hbase/consensus/raft/states/Leader.java     |   19 +
 .../consensus/raft/states/ProgressTimeout.java  |   18 +
 .../consensus/raft/states/RaftAsyncState.java   |   17 +
 .../hbase/consensus/raft/states/RaftState.java  |   29 +
 .../consensus/raft/states/RaftStateType.java    |   35 +
 .../raft/states/ReSendAppendRequest.java        |   31 +
 .../raft/states/SendAppendRequest.java          |   82 +
 .../consensus/raft/states/SendVoteRequest.java  |  101 +
 .../hbase/consensus/raft/states/Start.java      |   18 +
 .../raft/transitions/RaftTransitionType.java    |   44 +
 .../rmap/GetHydraBaseRegionInfoUtil.java        |   67 +
 .../hadoop/hbase/consensus/rmap/HDFSReader.java |  140 ++
 .../hbase/consensus/rmap/LocalReader.java       |   96 +
 .../consensus/rmap/NoSuchRMapException.java     |   10 +
 .../hadoop/hbase/consensus/rmap/Parser.java     |  146 ++
 .../hbase/consensus/rmap/RMapConfiguration.java |  330 +++
 .../hbase/consensus/rmap/RMapException.java     |   11 +
 .../hadoop/hbase/consensus/rmap/RMapJSON.java   |   34 +
 .../hadoop/hbase/consensus/rmap/RMapReader.java |  205 ++
 .../hbase/consensus/rmap/RegionLocator.java     |  142 ++
 .../hbase/consensus/rpc/AppendRequest.java      |  217 ++
 .../hbase/consensus/rpc/AppendResponse.java     |   94 +
 .../hadoop/hbase/consensus/rpc/LogState.java    |  151 ++
 .../hadoop/hbase/consensus/rpc/PeerStatus.java  |  107 +
 .../hadoop/hbase/consensus/rpc/Request.java     |    8 +
 .../hadoop/hbase/consensus/rpc/VoteRequest.java |   94 +
 .../hbase/consensus/rpc/VoteResponse.java       |   61 +
 .../consensus/server/ConsensusService.java      |   70 +
 .../consensus/server/ConsensusServiceImpl.java  |  248 ++
 .../InstrumentedConsensusServiceImpl.java       |  241 ++
 .../consensus/server/LocalConsensusServer.java  |  354 +++
 .../consensus/server/peer/AbstractPeer.java     |  447 ++++
 .../server/peer/PeerConsensusServer.java        |  152 ++
 .../hbase/consensus/server/peer/PeerServer.java |   18 +
 .../server/peer/PeerServerImmutableContext.java |   10 +
 .../server/peer/PeerServerMutableContext.java   |   23 +
 .../consensus/server/peer/PeerStateMachine.java |   84 +
 .../server/peer/ReconnectTimeoutCallback.java   |   21 +
 .../peer/events/PeerAppendRequestEvent.java     |   18 +
 .../peer/events/PeerAppendResponseEvent.java    |   18 +
 .../server/peer/events/PeerServerEventType.java |   16 +
 .../peer/events/PeerVoteRequestEvent.java       |   19 +
 .../peer/events/PeerVoteResponseEvent.java      |   18 +
 .../server/peer/states/PeerFollower.java        |   19 +
 .../peer/states/PeerHandleAppendResponse.java   |  282 +++
 .../server/peer/states/PeerHandleRPCError.java  |   27 +
 .../peer/states/PeerHandleVoteResponse.java     |   19 +
 .../server/peer/states/PeerRecovery.java        |   19 +
 .../peer/states/PeerSendAppendRequest.java      |   50 +
 .../server/peer/states/PeerSendVoteRequest.java |   22 +
 .../peer/states/PeerServerAsyncState.java       |   18 +
 .../server/peer/states/PeerServerState.java     |   23 +
 .../server/peer/states/PeerServerStateType.java |   16 +
 .../consensus/server/peer/states/Start.java     |   18 +
 .../transition/PeerServerTransitionType.java    |   22 +
 .../hadoop/hbase/consensus/util/RaftUtil.java   |  107 +
 .../hadoop/hbase/io/hfile/Compression.java      |  345 +++
 .../hbase/io/hfile/ReusableStreamGzipCodec.java |  133 +
 .../io/hfile/bucket/CacheFullException.java     |   54 +
 .../hbase/ipc/ByteBufferOutputStream.java       |  104 +
 .../thrift/exceptions/ThriftHBaseException.java |  151 ++
 .../hadoop/hbase/metrics/MetricsBase.java       |  160 ++
 .../apache/hadoop/hbase/metrics/TimeStat.java   |   88 +
 .../hbase/regionserver/DataStoreState.java      |  101 +
 .../hbase/regionserver/RaftEventListener.java   |   17 +
 .../regionserver/RegionOverloadedException.java |   47 +
 .../PercentileMetric.java/PercentileMetric.java |  115 +
 .../hbase/regionserver/wal/AbstractWAL.java     |   69 +
 .../hadoop/hbase/regionserver/wal/WALEdit.java  |  521 ++++
 .../hbase/thrift/generated/IllegalArgument.java |  400 +++
 .../org/apache/hadoop/hbase/util/Arena.java     |   12 +
 .../hadoop/hbase/util/BucketAllocator.java      |  455 ++++
 .../hbase/util/BucketAllocatorException.java    |   34 +
 .../org/apache/hadoop/hbase/util/Bytes.java     | 1612 ++++++++++++
 .../apache/hadoop/hbase/util/ConditionUtil.java |   62 +
 .../hadoop/hbase/util/DaemonThreadFactory.java  |   46 +
 .../org/apache/hadoop/hbase/util/HasThread.java |  101 +
 .../org/apache/hadoop/hbase/util/Histogram.java |  390 +++
 .../apache/hadoop/hbase/util/InHeapArena.java   |   79 +
 .../apache/hadoop/hbase/util/InfoServer.java    |  148 ++
 .../hadoop/hbase/util/InjectionEvent.java       |   45 +
 .../hadoop/hbase/util/InjectionHandler.java     |  171 ++
 .../apache/hadoop/hbase/util/MemoryBuffer.java  |   48 +
 .../java/org/apache/hadoop/hbase/util/Pair.java |  131 +
 .../org/apache/hadoop/hbase/util/Threads.java   |  264 ++
 .../serial/AsyncSerialExecutorServiceImpl.java  |  101 +
 .../util/serial/SerialExecutorService.java      |   34 +
 .../hadoop/hbase/consensus/LocalTestBed.java    |  944 +++++++
 .../hbase/consensus/RaftTestDataProvider.java   |  130 +
 .../hadoop/hbase/consensus/RaftTestUtil.java    |  751 ++++++
 .../consensus/ReplicationLoadForUnitTest.java   |   90 +
 .../hbase/consensus/SimulatedNetworkTester.java |   48 +
 .../hadoop/hbase/consensus/TestBasicCommit.java |  121 +
 .../consensus/TestBasicLeaderElection.java      |   74 +
 .../hbase/consensus/TestBasicPeerFailure.java   |  170 ++
 .../hbase/consensus/TestBasicPeerSeeding.java   |  100 +
 .../hbase/consensus/TestBasicPeerSlow.java      |  230 ++
 .../hbase/consensus/TestBasicQuorumCommit.java  |  101 +
 .../TestBasicQuorumMembershipChange.java        |  219 ++
 .../consensus/TestBasicSeedCommitIndex.java     |  115 +
 .../hbase/consensus/TestCommitDeadline.java     |   85 +
 .../hbase/consensus/TestLogFileViewer.java      |   79 +
 .../hbase/consensus/TestLogWriterAndReader.java |  105 +
 .../consensus/TestLowerRankBecomingLeader.java  |  124 +
 .../consensus/TestPersistLastVotedFor.java      |   83 +
 .../hbase/consensus/TestRaftEventListener.java  |  130 +
 .../hbase/consensus/TestRandomAccessLog.java    |   93 +
 .../hadoop/hbase/consensus/TestReadOnlyLog.java |  147 ++
 .../fsm/TestAsyncStatesInRaftStateMachine.java  |  242 ++
 .../consensus/fsm/TestFiniteStateMachine.java   |  194 ++
 .../fsm/TestFiniteStateMachineService.java      |  144 ++
 .../consensus/fsm/TestIncompleteStates.java     |  291 +++
 .../consensus/log/TestCachedFileChannel.java    |   76 +
 .../consensus/log/TestRemoteLogFetcher.java     |  167 ++
 .../log/TestTransactionLogCreator.java          |   40 +
 .../consensus/metrics/TestPeerMetrics.java      |   23 +
 .../consensus/quorum/TestAggregateTimer.java    |  221 ++
 .../consensus/quorum/TestConstituentTimer.java  |  147 ++
 .../consensus/quorum/TestRepeatingTimer.java    |  143 ++
 .../hadoop/hbase/consensus/rmap/TestParser.java |   93 +
 .../consensus/rmap/TestRMapConfiguration.java   |   55 +
 .../hbase/consensus/rmap/TestRMapReader.java    |  102 +
 .../hbase/consensus/rmap/TestRegionLocator.java |  180 ++
 .../hadoop/hbase/consensus/rmap/rmap.json       | 1228 ++++++++++
 pom.xml                                         |   87 +-
 251 files changed, 39327 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-consensus/pom.xml b/hbase-consensus/pom.xml
new file mode 100644
index 0000000..6a048eb
--- /dev/null
+++ b/hbase-consensus/pom.xml
@@ -0,0 +1,349 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+  /**
+   * 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.
+   */
+  -->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+
+  <artifactId>hbase-consensus</artifactId>
+  <name>HBase - Consensus</name>
+  <description>Implementation of the consensus module using Raft</description>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <version>${maven.assembly.version}</version>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-surefire-plugin</artifactId>
+          <version>2.15</version>
+          <dependencies>
+            <dependency>
+              <groupId>org.apache.maven.surefire</groupId>
+              <artifactId>surefire-junit47</artifactId>
+              <version>2.15</version>
+            </dependency>
+          </dependencies>
+          <configuration>
+            <forkedProcessTimeoutInSeconds>2000</forkedProcessTimeoutInSeconds>
+            <argLine>-enableassertions -Xmx1024m -Djava.library.path=${basedir}/lib/native/Linux-amd64-64/</argLine>
+            <redirectTestOutputToFile>true</redirectTestOutputToFile>
+            <systemPropertyVariables>
+              <jacoco-agent.destfile>target/coverage-reports/jacoco-unit.exec</jacoco-agent.destfile>
+            </systemPropertyVariables>
+          </configuration>
+        <!-- Always skip the second part executions, since we only run
+          simple unit tests in this module -->
+        <executions>
+          <execution>
+            <id>secondPartTestsExecution</id>
+            <phase>test</phase>
+            <goals>
+              <goal>test</goal>
+            </goals>
+            <configuration>
+              <skip>true</skip>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+    </plugins>
+
+
+    <resources>
+      <resource>
+        <directory>hbase-consensus/src/main/resources/</directory>
+        <includes>
+          <include>hbase-default.xml</include>
+        </includes>
+      </resource>
+    </resources>
+  </build>
+
+  <properties>
+    <guava.version>12.0.1</guava.version>
+    <compileSource>1.7</compileSource>
+    <airlift.version>0.91</airlift.version>
+    <jmxutils.version>1.16</jmxutils.version>
+    <swift.version>0.13.1</swift.version>
+    <disruptor.version>3.3.0</disruptor.version>
+  </properties>
+
+  <dependencyManagement>
+    <dependencies>
+      <!--
+      <dependency>
+              <groupId>com.facebook.swift</groupId>
+              <artifactId>swift-root</artifactId>
+              <version>${swift.version}</version>
+              <type>pom</type>
+      </dependency>
+      !-->
+
+    </dependencies>
+  </dependencyManagement>
+
+  <dependencies>
+    <!-- Intra-project dependencies -->
+      <dependency>
+          <groupId>org.ow2.asm</groupId>
+          <artifactId>asm-all</artifactId>
+          <version>4.1</version>
+      </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol</artifactId>
+    </dependency>
+    <!-- General dependencies -->
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+      <version>2.2.4</version>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-all</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.htrace</groupId>
+      <artifactId>htrace-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.airlift</groupId>
+      <artifactId>configuration</artifactId>
+      <version>${airlift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.airlift</groupId>
+      <artifactId>units</artifactId>
+      <version>${airlift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.airlift</groupId>
+      <artifactId>stats</artifactId>
+      <version>${airlift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.weakref</groupId>
+      <artifactId>jmxutils</artifactId>
+      <version>${jmxutils.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.facebook.swift</groupId>
+      <artifactId>swift-codec</artifactId>
+      <version>${swift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.facebook.swift</groupId>
+      <artifactId>swift-service</artifactId>
+      <version>${swift.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.json</groupId>
+      <artifactId>json</artifactId>
+      <version>20090211</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-httpclient</groupId>
+      <artifactId>commons-httpclient</artifactId>
+      <version>3.1</version>
+    </dependency>
+    <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+      <version>${disruptor.version}</version>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <!-- Skip the tests in this module -->
+    <profile>
+      <id>skipClientTests</id>
+      <activation>
+        <property>
+          <name>skipClientTests</name>
+        </property>
+      </activation>
+      <properties>
+        <surefire.skipFirstPart>true</surefire.skipFirstPart>
+      </properties>
+    </profile>
+    <!-- profile against Hadoop 1.0.x: This is the default. It has to have the same
+ activation property as the parent Hadoop 1.0.x profile to make sure it gets run at
+ the same time. -->
+    <profile>
+      <id>hadoop-1.1</id>
+      <activation>
+        <property>
+            <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+            <!--h1--><name>hadoop.profile</name><value>1.1</value>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!--
+      profile for building against Hadoop 2.0.0-alpha. Activate using:
+       mvn -Dhadoop.profile=2.0
+    -->
+    <profile>
+      <id>hadoop-2.0</id>
+      <activation>
+        <property>
+            <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+            <!--h2--><name>!hadoop.profile</name>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+          <exclusions>
+            <exclusion>
+              <groupId>javax.servlet.jsp</groupId>
+              <artifactId>jsp-api</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-server</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>javax.servlet</groupId>
+              <artifactId>servlet-api</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>tomcat</groupId>
+              <artifactId>jasper-compiler</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>tomcat</groupId>
+              <artifactId>jasper-runtime</artifactId>
+            </exclusion>
+          </exclusions>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+
+        <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-annotations</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+
+    <!--
+      profile for building against Hadoop 3.0.x. Activate using:
+       mvn -Dhadoop.profile=3.0
+    -->
+    <profile>
+      <id>hadoop-3.0</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>3.0</value>
+        </property>
+      </activation>
+      <properties>
+        <hadoop.version>3.0-SNAPSHOT</hadoop.version>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
new file mode 100644
index 0000000..00145f1
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HBaseConfiguration.java
@@ -0,0 +1,145 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase;
+
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Adds HBase configuration files to a Configuration
+ */
+public class HBaseConfiguration extends Configuration {
+
+  private static final Log LOG = LogFactory.getLog(HBaseConfiguration.class);
+
+  /**
+   * Instantinating HBaseConfiguration() is deprecated. Please use
+   * HBaseConfiguration#create() to construct a plain Configuration
+   */
+  @Deprecated
+  public HBaseConfiguration() {
+    //TODO:replace with private constructor, HBaseConfiguration should not extend Configuration
+    super();
+    addHbaseResources(this);
+    LOG.warn("instantiating HBaseConfiguration() is deprecated. Please use" +
+    		" HBaseConfiguration#create() to construct a plain Configuration");
+  }
+
+  /**
+   * Instantiating HBaseConfiguration() is deprecated. Please use
+   * HBaseConfiguration#create(conf) to construct a plain Configuration
+   */
+  @Deprecated
+  public HBaseConfiguration(final Configuration c) {
+    //TODO:replace with private constructor
+    this();
+    for (Entry<String, String>e: c) {
+      set(e.getKey(), e.getValue());
+    }
+  }
+
+  public static Configuration addHbaseResources(Configuration conf) {
+    // the sequence matters
+    conf.addResource("hbase-default.xml");
+    conf.addResource("hbase-site.xml");
+    conf.addResource("hbase-site-custom.xml");
+    conf.addResource("hbase-compactions.xml");
+    return conf;
+  }
+
+  /**
+   * Creates a Configuration with HBase resources
+   * @return a Configuration with HBase resources
+   */
+  public static Configuration create() {
+    Configuration conf = new Configuration();
+    return addHbaseResources(conf);
+  }
+  /**
+   * Creates a clone of passed configuration.
+   * @param that Configuration to clone.
+   * @return a Configuration created with the hbase-*.xml files plus
+   * the given configuration.
+   */
+  public static Configuration create(final Configuration that) {
+    Configuration conf = create();
+    for (Entry<String, String>e: that) {
+      conf.set(e.getKey(), e.getValue());
+    }
+    return conf;
+  }
+
+  /**
+   * Returns the hash code value for this HBaseConfiguration. The hash code of a
+   * HBaseConfiguration is defined by the xor of the hash codes of its entries.
+   *
+   * @see Configuration#iterator() How the entries are obtained.
+   */
+  @Override
+  @Deprecated
+  public int hashCode() {
+    return hashCode(this);
+  }
+
+  /**
+   * Returns the hash code value for this HBaseConfiguration. The hash code of a
+   * Configuration is defined by the xor of the hash codes of its entries.
+   *
+   * @see Configuration#iterator() How the entries are obtained.
+   */
+  public static int hashCode(Configuration conf) {
+    int hash = 0;
+
+    Iterator<Entry<String, String>> propertyIterator = conf.iterator();
+    while (propertyIterator.hasNext()) {
+      hash ^= propertyIterator.next().hashCode();
+    }
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (!(obj instanceof HBaseConfiguration))
+      return false;
+    HBaseConfiguration otherConf = (HBaseConfiguration) obj;
+    if (size() != otherConf.size()) {
+      return false;
+    }
+    Iterator<Entry<String, String>> propertyIterator = this.iterator();
+    while (propertyIterator.hasNext()) {
+      Entry<String, String> entry = propertyIterator.next();
+      String key = entry.getKey();
+      String value = entry.getValue();
+      if (!value.equals(otherConf.getRaw(key))) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
new file mode 100644
index 0000000..0dbe7fd
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@ -0,0 +1,1001 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+//import org.apache.hadoop.hbase.regionserver.StoreFile;
+//import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * An HColumnDescriptor contains information about a column family such as the
+ * number of versions, compression settings, etc.
+ *
+ * It is used as input when creating a table or adding a column. Once set, the
+ * parameters that specify a column cannot be changed without deleting the
+ * column and recreating it. If there is data stored in the column, it will be
+ * deleted when the column is deleted.
+ */
+public class HColumnDescriptor implements WritableComparable<HColumnDescriptor> {
+  static final Log LOG = LogFactory.getLog(HColumnDescriptor.class);
+  // For future backward compatibility
+
+  // Version 3 was when column names become byte arrays and when we picked up
+  // Time-to-live feature.  Version 4 was when we moved to byte arrays, HBASE-82.
+  // Version 5 was when bloom filter descriptors were removed.
+  // Version 6 adds metadata as a map where keys and values are byte[].
+  // Version 7 -- add new compression and hfile blocksize to HColumnDescriptor (HBASE-1217)
+  // Version 8 -- reintroduction of bloom filters, changed from boolean to enum
+  // Version 9 -- add data block encoding
+  // Version 10 -- add flash back time
+  private static final byte COLUMN_DESCRIPTOR_VERSION = (byte) 10;
+
+  // These constants are used as FileInfo keys
+  public static final String COMPRESSION = "COMPRESSION";
+  public static final String COMPRESSION_COMPACT = "COMPRESSION_COMPACT";
+  public static final String ENCODE_ON_DISK =
+      "ENCODE_ON_DISK";
+  public static final String DATA_BLOCK_ENCODING =
+      "DATA_BLOCK_ENCODING";
+  public static final String BLOCKCACHE = "BLOCKCACHE";
+
+  /**
+   * The type of compression.
+   * @see org.apache.hadoop.io.SequenceFile.Writer
+   * @deprecated Compression now means which compression library
+   * rather than 'what' to compress.
+   */
+  @Deprecated
+  public static enum CompressionType {
+    /** Do not compress records. */
+    NONE,
+    /** Compress values only, each separately. */
+    RECORD,
+    /** Compress sequences of records together in blocks. */
+    BLOCK
+  }
+
+  public static final String BLOCKSIZE = "BLOCKSIZE";
+  public static final String LENGTH = "LENGTH";
+  public static final String TTL = "TTL";
+  // The amount of time in seconds in the past upto which we support FlashBack
+  // queries. Ex. 60 * 60 * 24 indicates we support FlashBack queries upto 1 day
+  // ago.
+  public static final String FLASHBACK_QUERY_LIMIT = "FLASHBACK_QUERY_LIMIT";
+  public static final String BLOOMFILTER = "BLOOMFILTER";
+  public static final String BLOOMFILTER_ERRORRATE = "BLOOMFILTER_ERRORRATE";
+  public static final String FOREVER = "FOREVER";
+  public static final String REPLICATION_SCOPE = "REPLICATION_SCOPE";
+  public static final String ROWKEY_PREFIX_LENGTH_FOR_BLOOMFILTER = "ROWKEY_PREFIX_LENGTH";
+  public static final String HFILEHISTOGRAM_BUCKET_COUNT =
+      "HFILEHISTOGRAM_BUCKET_COUNT";
+
+  /**
+   * Default compression type.
+   */
+  public static final String DEFAULT_COMPRESSION =
+    Compression.Algorithm.NONE.getName();
+
+  /**
+   * Default value of the flag that enables data block encoding on disk, as
+   * opposed to encoding in cache only. We encode blocks everywhere by default,
+   * as long as {@link #DATA_BLOCK_ENCODING} is not NONE.
+   */
+  public static final boolean DEFAULT_ENCODE_ON_DISK = true;
+
+  /** Default data block encoding algorithm. */
+  public static final String DEFAULT_DATA_BLOCK_ENCODING =
+      DataBlockEncoding.NONE.toString();
+
+  /**
+   * Default number of versions of a record to keep.
+   */
+  public static final int DEFAULT_VERSIONS = 3;
+
+  /**
+   * Default setting for whether to serve from memory or not.
+   */
+  public static final boolean DEFAULT_IN_MEMORY = false;
+
+  /**
+   * Default setting for whether to use a block cache or not.
+   */
+  public static final boolean DEFAULT_BLOCKCACHE = true;
+
+  /**
+   * Default size of blocks in files store to the filesytem.  Use smaller for
+   * faster random-access at expense of larger indices (more memory consumption).
+   */
+  public static final int DEFAULT_BLOCKSIZE = 1024; //HFile.DEFAULT_BLOCKSIZE;
+
+//  /**
+//   * Default setting for whether or not to use bloomfilters.
+//   */
+  public static final String DEFAULT_BLOOMFILTER = " "; // StoreFile.BloomType.NONE.toString();
+
+  /**
+   * Default setting for the RowKey Prefix Length for the Bloomfilter.
+   */
+  public static final int DEFAULT_ROWKEY_PREFIX_LENGTH_FOR_BLOOM = -1;
+
+  /**
+   * Default value for bloom filter error rate.
+   */
+  public static final float DEFAULT_BLOOMFILTER_ERROR_RATE = 0.01f;
+
+
+  /**
+   * Default time to live of cell contents.
+   */
+  public static final int DEFAULT_TTL = HConstants.FOREVER;
+
+  /**
+   * Default flash back time. Flash back time is the number of seconds in the
+   * past upto which we support flash back queries.
+   */
+  public static final int DEFAULT_FLASHBACK_QUERY_LIMIT = 0;
+
+  /**
+   * Default scope.
+   */
+  public static final int DEFAULT_REPLICATION_SCOPE = HConstants.REPLICATION_SCOPE_LOCAL;
+
+  private final static Map<String, String> DEFAULT_VALUES
+    = new HashMap<String, String>();
+  private final static Set<ImmutableBytesWritable> RESERVED_KEYWORDS
+    = new HashSet<ImmutableBytesWritable>();
+  static {
+      DEFAULT_VALUES.put(BLOOMFILTER, DEFAULT_BLOOMFILTER);
+      DEFAULT_VALUES.put(BLOOMFILTER_ERRORRATE,
+          String.valueOf(DEFAULT_BLOOMFILTER_ERROR_RATE));
+      DEFAULT_VALUES.put(REPLICATION_SCOPE, String.valueOf(DEFAULT_REPLICATION_SCOPE));
+      DEFAULT_VALUES.put(HConstants.VERSIONS, String.valueOf(DEFAULT_VERSIONS));
+      DEFAULT_VALUES.put(COMPRESSION, DEFAULT_COMPRESSION);
+      DEFAULT_VALUES.put(TTL, String.valueOf(DEFAULT_TTL));
+      DEFAULT_VALUES.put(FLASHBACK_QUERY_LIMIT,
+        String.valueOf(DEFAULT_FLASHBACK_QUERY_LIMIT));
+      DEFAULT_VALUES.put(BLOCKSIZE, String.valueOf(DEFAULT_BLOCKSIZE));
+      DEFAULT_VALUES.put(HConstants.IN_MEMORY, String.valueOf(DEFAULT_IN_MEMORY));
+      DEFAULT_VALUES.put(BLOCKCACHE, String.valueOf(DEFAULT_BLOCKCACHE));
+      DEFAULT_VALUES.put(ENCODE_ON_DISK,
+          String.valueOf(DEFAULT_ENCODE_ON_DISK));
+      DEFAULT_VALUES.put(DATA_BLOCK_ENCODING,
+          String.valueOf(DEFAULT_DATA_BLOCK_ENCODING));
+      DEFAULT_VALUES.put(ROWKEY_PREFIX_LENGTH_FOR_BLOOMFILTER,
+          String.valueOf(DEFAULT_ROWKEY_PREFIX_LENGTH_FOR_BLOOM));
+//      DEFAULT_VALUES.put(HFILEHISTOGRAM_BUCKET_COUNT,
+//          String.valueOf(HFileHistogram.DEFAULT_HFILEHISTOGRAM_BINCOUNT));
+      for (String s : DEFAULT_VALUES.keySet()) {
+        RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(s)));
+      }
+  }
+
+  /*
+   * Cache here the HCD value.
+   * Question: its OK to cache since when we're reenable, we create a new HCD?
+   */
+  private volatile Integer blocksize = null;
+
+  // Column family name
+  private byte [] name;
+
+  // Column metadata
+  protected final Map<ImmutableBytesWritable, ImmutableBytesWritable> values =
+    new HashMap<ImmutableBytesWritable,ImmutableBytesWritable>();
+
+  /*
+   * Cache the max versions rather than calculate it every time.
+   */
+  private int cachedMaxVersions = -1;
+
+  /**
+   * Default constructor. Must be present for Writable.
+   */
+  public HColumnDescriptor() {
+    this.name = null;
+  }
+
+  /**
+   * Construct a column descriptor specifying only the family name
+   * The other attributes are defaulted.
+   *
+   * @param familyName Column family name. Must be 'printable' -- digit or
+   * letter -- and may not contain a <code>:<code>
+   */
+  public HColumnDescriptor(final String familyName) {
+    this(Bytes.toBytes(familyName));
+  }
+
+  /**
+   * Construct a column descriptor specifying only the family name
+   * The other attributes are defaulted.
+   *
+   * @param familyName Column family name. Must be 'printable' -- digit or
+   * letter -- and may not contain a <code>:<code>
+   */
+  public HColumnDescriptor(final byte [] familyName) {
+    this (familyName == null || familyName.length <= 0?
+      HConstants.EMPTY_BYTE_ARRAY: familyName, DEFAULT_VERSIONS,
+      DEFAULT_COMPRESSION, DEFAULT_IN_MEMORY, DEFAULT_BLOCKCACHE,
+      DEFAULT_TTL, DEFAULT_BLOOMFILTER);
+  }
+
+  /**
+   * Constructor.
+   * Makes a deep copy of the supplied descriptor.
+   * Can make a modifiable descriptor from an UnmodifyableHColumnDescriptor.
+   * @param desc The descriptor.
+   */
+  public HColumnDescriptor(HColumnDescriptor desc) {
+    super();
+    this.name = desc.name.clone();
+    for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
+        desc.values.entrySet()) {
+      this.values.put(e.getKey(), e.getValue());
+    }
+  }
+
+  /**
+   * Constructor
+   * @param familyName Column family name. Must be 'printable' -- digit or
+   * letter -- and may not contain a <code>:<code>
+   * @param maxVersions Maximum number of versions to keep
+   * @param compression Compression type
+   * @param inMemory If true, column data should be kept in an HRegionServer's
+   * cache
+   * @param blockCacheEnabled If true, MapFile blocks should be cached
+   * @param timeToLive Time-to-live of cell contents, in seconds
+   * (use HConstants.FOREVER for unlimited TTL)
+   * @param bloomFilter Bloom filter type for this column
+   *
+   * @throws IllegalArgumentException if passed a family name that is made of
+   * other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> or contains
+   * a <code>:</code>
+   * @throws IllegalArgumentException if the number of versions is &lt;= 0
+   * @deprecated use {@link #HColumnDescriptor(String)} and setters
+   */
+  @Deprecated
+  public HColumnDescriptor(final byte [] familyName, final int maxVersions,
+      final String compression, final boolean inMemory,
+      final boolean blockCacheEnabled,
+      final int timeToLive, final String bloomFilter) {
+    this(familyName, maxVersions, compression, inMemory, blockCacheEnabled,
+      DEFAULT_BLOCKSIZE, timeToLive, bloomFilter, DEFAULT_REPLICATION_SCOPE);
+  }
+
+  /**
+   * Constructor
+   * @param familyName Column family name. Must be 'printable' -- digit or
+   * letter -- and may not contain a <code>:<code>
+   * @param maxVersions Maximum number of versions to keep
+   * @param compression Compression type
+   * @param inMemory If true, column data should be kept in an HRegionServer's
+   * cache
+   * @param blockCacheEnabled If true, MapFile blocks should be cached
+   * @param blocksize Block size to use when writing out storefiles.  Use
+   * smaller block sizes for faster random-access at expense of larger indices
+   * (more memory consumption).  Default is usually 64k.
+   * @param timeToLive Time-to-live of cell contents, in seconds
+   * (use HConstants.FOREVER for unlimited TTL)
+   * @param bloomFilter Bloom filter type for this column
+   * @param scope The scope tag for this column
+   *
+   * @throws IllegalArgumentException if passed a family name that is made of
+   * other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> or contains
+   * a <code>:</code>
+   * @throws IllegalArgumentException if the number of versions is &lt;= 0
+   * @deprecated use {@link #HColumnDescriptor(String)} and setters
+   */
+  @Deprecated
+  public HColumnDescriptor(final byte [] familyName, final int maxVersions,
+      final String compression, final boolean inMemory,
+      final boolean blockCacheEnabled, final int blocksize,
+      final int timeToLive, final String bloomFilter, final int scope) {
+    this(familyName, maxVersions,
+        compression, DEFAULT_ENCODE_ON_DISK, DEFAULT_DATA_BLOCK_ENCODING,
+        inMemory, blockCacheEnabled, blocksize, timeToLive, bloomFilter,
+        scope, DEFAULT_BLOOMFILTER_ERROR_RATE);
+  }
+
+  /**
+   * Constructor
+   * @param familyName Column family name. Must be 'printable' -- digit or
+   * letter -- and may not contain a <code>:<code>
+   * @param maxVersions Maximum number of versions to keep
+   * @param compression Compression type
+   * @param encodeOnDisk whether to use the specified data block encoding
+   *        on disk. If false, the encoding will be used in cache only.
+   * @param dataBlockEncoding data block encoding
+   * @param inMemory If true, column data should be kept in an HRegionServer's
+   * cache
+   * @param blockCacheEnabled If true, MapFile blocks should be cached
+   * @param blocksize
+   * @param timeToLive Time-to-live of cell contents, in seconds
+   * (use HConstants.FOREVER for unlimited TTL)
+   * @param bloomFilter Bloom filter type for this column
+   * @param scope The scope tag for this column
+   * @param bloomErrorRate Bloom filter error rate for this column
+   * @throws IllegalArgumentException if passed a family name that is made of
+   * other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> or contains
+   * a <code>:</code>
+   * @throws IllegalArgumentException if the number of versions is &lt;= 0
+   * @deprecated use {@link #HColumnDescriptor(String)} and setters
+   */
+  @Deprecated
+  public HColumnDescriptor(final byte[] familyName,
+      final int maxVersions,
+      final String compression, final boolean encodeOnDisk,
+      final String dataBlockEncoding, final boolean inMemory,
+      final boolean blockCacheEnabled, final int blocksize,
+      final int timeToLive, final String bloomFilter, final int scope,
+      float bloomErrorRate) {
+    isLegalFamilyName(familyName);
+    this.name = familyName;
+
+    if (maxVersions <= 0) {
+      // TODO: Allow maxVersion of 0 to be the way you say "Keep all versions".
+      // Until there is support, consider 0 or < 0 -- a configuration error.
+      throw new IllegalArgumentException("Maximum versions must be positive");
+    }
+    setMaxVersions(maxVersions);
+    setInMemory(inMemory);
+    setBlockCacheEnabled(blockCacheEnabled);
+    setTimeToLive(timeToLive);
+    setCompressionType(Compression.Algorithm.
+      valueOf(compression.toUpperCase()));
+    setEncodeOnDisk(encodeOnDisk);
+    setDataBlockEncoding(DataBlockEncoding.
+        valueOf(dataBlockEncoding.toUpperCase()));
+//    setBloomFilterType(StoreFile.BloomType.
+//      valueOf(bloomFilter.toUpperCase()));
+    setBloomFilterErrorRate(bloomErrorRate);
+    setBlocksize(blocksize);
+    setScope(scope);
+  }
+
+  /**
+   * @param b Family name.
+   * @return <code>b</code>
+   * @throws IllegalArgumentException If not null and not a legitimate family
+   * name: i.e. 'printable' and ends in a ':' (Null passes are allowed because
+   * <code>b</code> can be null when deserializing).  Cannot start with a '.'
+   * either.
+   */
+  public static byte [] isLegalFamilyName(final byte [] b) {
+    if (b == null) {
+      return b;
+    }
+    if (b[0] == '.') {
+      throw new IllegalArgumentException("Family names cannot start with a " +
+        "period: " + Bytes.toString(b));
+    }
+    for (int i = 0; i < b.length; i++) {
+      if (Character.isISOControl(b[i]) || b[i] == ':') {
+        throw new IllegalArgumentException("Illegal character <" + b[i] +
+          ">. Family names cannot contain control characters or colons: " +
+          Bytes.toString(b));
+      }
+    }
+    return b;
+  }
+
+  public void setName(byte[] name) {
+    this.name = name;
+  }
+
+  /**
+   * @return Name of this column family
+   */
+  public byte [] getName() {
+    return name;
+  }
+
+  /**
+   * @return Name of this column family
+   */
+  public String getNameAsString() {
+    return Bytes.toString(this.name);
+  }
+
+  /**
+   * @param key The key.
+   * @return The value.
+   */
+  public byte[] getValue(byte[] key) {
+    ImmutableBytesWritable ibw = values.get(new ImmutableBytesWritable(key));
+    if (ibw == null)
+      return null;
+    return ibw.get();
+  }
+
+  /**
+   * @param key The key.
+   * @return The value as a string.
+   */
+  public String getValue(String key) {
+    byte[] value = getValue(Bytes.toBytes(key));
+    if (value == null)
+      return null;
+    return Bytes.toString(value);
+  }
+
+  /**
+   * @return All values.
+   */
+  public Map<ImmutableBytesWritable,ImmutableBytesWritable> getValues() {
+    // shallow pointer copy
+    return Collections.unmodifiableMap(values);
+  }
+
+  /**
+   * @param key The key.
+   * @param value The value.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setValue(byte[] key, byte[] value) {
+    values.put(new ImmutableBytesWritable(key),
+      new ImmutableBytesWritable(value));
+    return this;
+  }
+
+  /**
+   * @param key Key whose key and value we're to remove from HCD parameters.
+   */
+  public void remove(final byte [] key) {
+    values.remove(new ImmutableBytesWritable(key));
+  }
+
+  /**
+   * @param key The key.
+   * @param value The value.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setValue(String key, String value) {
+    if (value == null) {
+      remove(Bytes.toBytes(key));
+    } else {
+      setValue(Bytes.toBytes(key), Bytes.toBytes(value));
+    }
+    return this;
+  }
+
+  /** @return compression type being used for the column family */
+  public Compression.Algorithm getCompression() {
+    String n = getValue(COMPRESSION);
+    if (n != null) {
+      return Compression.Algorithm.valueOf(n.toUpperCase());
+    } else {
+      return Compression.Algorithm.valueOf(DEFAULT_COMPRESSION);
+    }
+  }
+
+  /** @return maximum number of versions */
+  public synchronized int getMaxVersions() {
+    if (this.cachedMaxVersions == -1) {
+      String value = getValue(HConstants.VERSIONS);
+      this.cachedMaxVersions = (value != null)?
+        Integer.valueOf(value).intValue(): DEFAULT_VERSIONS;
+    }
+    return this.cachedMaxVersions;
+  }
+
+  /**
+   * @param maxVersions maximum number of versions
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setMaxVersions(int maxVersions) {
+    setValue(HConstants.VERSIONS, Integer.toString(maxVersions));
+    cachedMaxVersions = maxVersions;
+    return this;
+  }
+
+  /**
+   * @return Blocksize.
+   */
+  public synchronized int getBlocksize() {
+    if (this.blocksize == null) {
+      String value = getValue(BLOCKSIZE);
+      this.blocksize = (value != null)?
+        Integer.decode(value): Integer.valueOf(DEFAULT_BLOCKSIZE);
+    }
+    return this.blocksize.intValue();
+  }
+
+  /**
+   * @param s Blocksize to use when writing out storefiles/hfiles on this
+   * column family.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setBlocksize(int s) {
+    setValue(BLOCKSIZE, Integer.toString(s));
+    this.blocksize = null;
+    return this;
+  }
+
+  /**
+   * @return Compression type setting.
+   */
+  public Compression.Algorithm getCompressionType() {
+    return getCompression();
+  }
+
+  /**
+   * Compression types supported in hbase.
+   * LZO is not bundled as part of the hbase distribution.
+   * See <a href="http://wiki.apache.org/hadoop/UsingLzoCompression">LZO Compression</a>
+   * for how to enable it.
+   * @param type Compression type setting.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setCompressionType(Compression.Algorithm type) {
+    String compressionType = type.getName().toUpperCase();
+    return setValue(COMPRESSION, compressionType);
+  }
+
+  /**
+   * @param compressionTypeStr compression type as a string
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setCompressionType(String compressionTypeStr) {
+    return setCompressionType(
+        Compression.Algorithm.valueOf(compressionTypeStr.toUpperCase()));
+  }
+
+  /** @return data block encoding algorithm used on disk */
+  public DataBlockEncoding getDataBlockEncodingOnDisk() {
+    String encodeOnDiskStr = getValue(ENCODE_ON_DISK);
+    boolean encodeOnDisk;
+    if (encodeOnDiskStr == null) {
+      encodeOnDisk = DEFAULT_ENCODE_ON_DISK;
+    } else {
+      encodeOnDisk = Boolean.valueOf(encodeOnDiskStr);
+    }
+
+    if (!encodeOnDisk) {
+      // No encoding on disk.
+      return DataBlockEncoding.NONE;
+    }
+    return getDataBlockEncoding();
+  }
+
+  /**
+   * Set the flag indicating that we only want to encode data block in cache
+   * but not on disk.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setEncodeOnDisk(boolean encodeOnDisk) {
+    return setValue(ENCODE_ON_DISK, String.valueOf(encodeOnDisk));
+  }
+
+  /**
+   * @return the data block encoding algorithm used in block cache and
+   *         optionally on disk
+   */
+  public DataBlockEncoding getDataBlockEncoding() {
+    String type = getValue(DATA_BLOCK_ENCODING);
+    if (type == null) {
+      type = DEFAULT_DATA_BLOCK_ENCODING;
+    }
+    return DataBlockEncoding.valueOf(type);
+  }
+
+  /**
+   * Set data block encoding algorithm used in block cache.
+   * @param type What kind of data block encoding will be used.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setDataBlockEncoding(DataBlockEncoding type) {
+    String name;
+    if (type != null) {
+      name = type.toString();
+    } else {
+      name = DataBlockEncoding.NONE.toString();
+    }
+    return setValue(DATA_BLOCK_ENCODING, name);
+  }
+
+  /**
+   * @return True if we are to keep all in use HRegionServer cache.
+   */
+  public boolean isInMemory() {
+    String value = getValue(HConstants.IN_MEMORY);
+    if (value != null)
+      return Boolean.valueOf(value).booleanValue();
+    return DEFAULT_IN_MEMORY;
+  }
+
+  /**
+   * @param inMemory True if we are to keep all values in the HRegionServer
+   * cache
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setInMemory(boolean inMemory) {
+    return setValue(HConstants.IN_MEMORY, Boolean.toString(inMemory));
+  }
+
+  /**
+   * @return Time-to-live of cell contents, in seconds.
+   */
+  public int getTimeToLive() {
+    String value = getValue(TTL);
+    return (value != null)? Integer.valueOf(value).intValue(): DEFAULT_TTL;
+  }
+
+  /**
+   * @param timeToLive Time-to-live of cell contents, in seconds.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setTimeToLive(int timeToLive) {
+    return setValue(TTL, Integer.toString(timeToLive));
+  }
+
+  /**
+   * @return the time in seconds for how far back in the past we support flash
+   *         back queries.
+   */
+  public int getFlashBackQueryLimit() {
+    String value = getValue(FLASHBACK_QUERY_LIMIT);
+    return (value != null) ? Integer.valueOf(value).intValue()
+        : DEFAULT_FLASHBACK_QUERY_LIMIT;
+  }
+
+  /**
+   * @param flashBackQueryLimit
+   *          the time in seconds for how far back in the past we support flash
+   *          back queries.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setFlashBackQueryLimit(int flashBackQueryLimit) {
+    if (flashBackQueryLimit < 0) {
+      throw new IllegalArgumentException(
+          "FlashBackQueryLimit cannot be negative");
+    }
+    return setValue(FLASHBACK_QUERY_LIMIT,
+        Integer.toString(flashBackQueryLimit));
+  }
+
+  /**
+   * @return True if MapFile blocks should be cached.
+   */
+  public boolean isBlockCacheEnabled() {
+    String value = getValue(BLOCKCACHE);
+    if (value != null)
+      return Boolean.valueOf(value).booleanValue();
+    return DEFAULT_BLOCKCACHE;
+  }
+
+  /**
+   * @param blockCacheEnabled True if MapFile blocks should be cached.
+   * @return this (for chained invocation)
+   */
+  public HColumnDescriptor setBlockCacheEnabled(boolean blockCacheEnabled) {
+    return setValue(BLOCKCACHE, Boolean.toString(blockCacheEnabled));
+  }
+
+//  /**
+//   * @return bloom filter type used for new StoreFiles in ColumnFamily
+//   */
+//  public StoreFile.BloomType getBloomFilterType() {
+//    String n = getValue(BLOOMFILTER);
+//    if (n == null) {
+//      n = DEFAULT_BLOOMFILTER;
+//    }
+//    return StoreFile.BloomType.valueOf(n.toUpperCase());
+//  }
+
+  public int getIntValueFromString(String key, int defaultValue,
+      String parseErrorMessage) {
+    String n = getValue(key);
+    int returnValue = defaultValue;
+    if (n != null) {
+      try {
+        returnValue = Integer.parseInt(n);
+      } catch (Throwable e) {
+        LOG.error("Invalid Input " + n + ". " + parseErrorMessage, e);
+      }
+    }
+    return returnValue;
+  }
+
+  public int getHFileHistogramBucketCount() {
+    return 0;
+//      getIntValueFromString(
+//        HFILEHISTOGRAM_BUCKET_COUNT,
+//        HFileHistogram.DEFAULT_HFILEHISTOGRAM_BINCOUNT,
+//        "Cannot parse the histogram bin count");
+  }
+
+  /**
+   * @return the number of bytes as row key prefix for the bloom filter
+   */
+  public int getRowPrefixLengthForBloom() {
+    return getIntValueFromString(
+        ROWKEY_PREFIX_LENGTH_FOR_BLOOMFILTER,
+        DEFAULT_ROWKEY_PREFIX_LENGTH_FOR_BLOOM,
+        "Cannot parse row key prefix length");
+  }
+
+  public void setRowKeyPrefixLengthForBloom(int prefixLength) {
+    if (prefixLength > 0) {
+      setValue(ROWKEY_PREFIX_LENGTH_FOR_BLOOMFILTER, String.valueOf(prefixLength));
+    }
+  }
+
+  public void setHFileHistogramBucketCount(int histogramBucketCount) {
+    if (histogramBucketCount > 0) {
+      setValue(
+          HFILEHISTOGRAM_BUCKET_COUNT,
+          String.valueOf(histogramBucketCount));
+    }
+  }
+
+//  /**
+//   * @param bt bloom filter type
+//   * @return this (for chained invocation)
+//   */
+//  public HColumnDescriptor setBloomFilterType(final StoreFile.BloomType bt) {
+//    return setValue(BLOOMFILTER, bt.toString());
+//  }
+
+//  /**
+//   * @param bloomTypeStr bloom filter type as a string
+//   * @return this (for chained invocation)
+//   */
+//  public HColumnDescriptor setBloomFilterType(String bloomTypeStr) {
+//    return setBloomFilterType(BloomType.valueOf(bloomTypeStr.toUpperCase()));
+//  }
+
+  public void setBloomFilterErrorRate(float bloomErrorRate) {
+    setValue(BLOOMFILTER_ERRORRATE, Float.toString(bloomErrorRate));
+  }
+  public float getBloomFilterErrorRate() {
+    String value = getValue(BLOOMFILTER_ERRORRATE);
+    return (value != null)? Float.valueOf(value).floatValue() : DEFAULT_BLOOMFILTER_ERROR_RATE;
+  }
+
+  /**
+   * @return the scope tag
+   */
+  public int getScope() {
+    String value = getValue(REPLICATION_SCOPE);
+    if (value != null) {
+      return Integer.valueOf(value).intValue();
+    }
+    return DEFAULT_REPLICATION_SCOPE;
+  }
+
+ /**
+  * @param scope the scope tag
+  * @return this (for chained invocation)
+  */
+  public HColumnDescriptor setScope(int scope) {
+    return setValue(REPLICATION_SCOPE, Integer.toString(scope));
+  }
+
+  /**
+   * @see java.lang.Object#toString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder s = new StringBuilder();
+    s.append('{');
+    s.append(HConstants.NAME);
+    s.append(" => '");
+    s.append(Bytes.toString(name));
+    s.append("'");
+    s.append(getValues(true));
+    s.append('}');
+    return s.toString();
+  }
+
+  public String toStringCustomizedValues() {
+    StringBuilder s = new StringBuilder();
+    s.append('{');
+    s.append(HConstants.NAME);
+    s.append(" => '");
+    s.append(Bytes.toString(name));
+    s.append("'");
+    s.append(getValues(false));
+    s.append('}');
+    return s.toString();
+  }
+
+  private StringBuilder getValues(boolean printDefaults) {
+    StringBuilder s = new StringBuilder();
+
+    boolean hasAdvancedKeys = false;
+
+    // print all reserved keys first
+    for (ImmutableBytesWritable k : values.keySet()) {
+      if (!RESERVED_KEYWORDS.contains(k)) {
+        hasAdvancedKeys = true;
+        continue;
+      }
+      String key = Bytes.toString(k.get());
+      String value = Bytes.toString(values.get(k).get());
+      if (printDefaults
+          || !DEFAULT_VALUES.containsKey(key)
+          || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
+        s.append(", ");
+        s.append(key);
+        s.append(" => ");
+        s.append('\'').append(value).append('\'');
+      }
+    }
+
+    // print all other keys as advanced options
+    if (hasAdvancedKeys) {
+      s.append(", ");
+      s.append(HConstants.CONFIG).append(" => ");
+      s.append('{');
+      boolean printComma = false;
+      for (ImmutableBytesWritable k : values.keySet()) {
+        if (RESERVED_KEYWORDS.contains(k)) {
+          continue;
+        }
+        String key = Bytes.toString(k.get());
+        String value = Bytes.toString(values.get(k).get());
+        if (printComma) {
+          s.append(", ");
+        }
+        printComma = true;
+        s.append('\'').append(key).append('\'');
+        s.append(" => ");
+        s.append('\'').append(value).append('\'');
+      }
+      s.append('}');
+    }
+    return s;
+  }
+
+  public static Map<String, String> getDefaultValues() {
+    return Collections.unmodifiableMap(DEFAULT_VALUES);
+  }
+
+  /**
+   * @see java.lang.Object#equals(java.lang.Object)
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof HColumnDescriptor)) {
+      return false;
+    }
+    return compareTo((HColumnDescriptor)obj) == 0;
+  }
+
+  /**
+   * @see java.lang.Object#hashCode()
+   */
+  @Override
+  public int hashCode() {
+    int result = Bytes.hashCode(this.name);
+    result ^= Byte.valueOf(COLUMN_DESCRIPTOR_VERSION).hashCode();
+    result ^= values.hashCode();
+    return result;
+  }
+
+  // Writable
+
+  public void readFields(DataInput in) throws IOException {
+    int version = in.readByte();
+    if (version < 6) {
+      if (version <= 2) {
+        Text t = new Text();
+        t.readFields(in);
+        this.name = t.getBytes();
+//        if(KeyValue.getFamilyDelimiterIndex(this.name, 0, this.name.length)
+//            > 0) {
+//          this.name = stripColon(this.name);
+//        }
+      } else {
+        this.name = Bytes.readByteArray(in);
+      }
+      this.values.clear();
+      setMaxVersions(in.readInt());
+      int ordinal = in.readInt();
+      setCompressionType(Compression.Algorithm.values()[ordinal]);
+      setInMemory(in.readBoolean());
+//      setBloomFilterType(in.readBoolean() ? BloomType.ROW : BloomType.NONE);
+//      if (getBloomFilterType() != BloomType.NONE && version < 5) {
+//        // If a bloomFilter is enabled and the column descriptor is less than
+//        // version 5, we need to skip over it to read the rest of the column
+//        // descriptor. There are no BloomFilterDescriptors written to disk for
+//        // column descriptors with a version number >= 5
+//        throw new UnsupportedClassVersionError(this.getClass().getName() +
+//            " does not support backward compatibility with versions older " +
+//            "than version 5");
+//      }
+      if (version > 1) {
+        setBlockCacheEnabled(in.readBoolean());
+      }
+      if (version > 2) {
+       setTimeToLive(in.readInt());
+      }
+    } else {
+      // version 6+
+      this.name = Bytes.readByteArray(in);
+      this.values.clear();
+      int numValues = in.readInt();
+      for (int i = 0; i < numValues; i++) {
+        ImmutableBytesWritable key = new ImmutableBytesWritable();
+        ImmutableBytesWritable value = new ImmutableBytesWritable();
+        key.readFields(in);
+        value.readFields(in);
+
+//        // in version 8, the BloomFilter setting changed from bool to enum
+//        if (version < 8 && Bytes.toString(key.get()).equals(BLOOMFILTER)) {
+//          value.set(Bytes.toBytes(
+//              Boolean.getBoolean(Bytes.toString(value.get()))
+//                ? BloomType.ROW.toString()
+//                : BloomType.NONE.toString()));
+//        }
+
+        values.put(key, value);
+      }
+      if (version == 6) {
+        // Convert old values.
+        setValue(COMPRESSION, Compression.Algorithm.NONE.getName());
+      }
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeByte(COLUMN_DESCRIPTOR_VERSION);
+    Bytes.writeByteArray(out, this.name);
+    out.writeInt(values.size());
+    for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
+        values.entrySet()) {
+      e.getKey().write(out);
+      e.getValue().write(out);
+    }
+  }
+
+  // Comparable
+
+  public int compareTo(HColumnDescriptor o) {
+    int result = Bytes.compareTo(this.name, o.getName());
+    if (result == 0) {
+      // punt on comparison for ordering, just calculate difference
+      result = this.values.hashCode() - o.values.hashCode();
+      if (result < 0)
+        result = -1;
+      else if (result > 0)
+        result = 1;
+    }
+    return result;
+  }
+}


[14/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CandidateLogsManager.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CandidateLogsManager.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CandidateLogsManager.java
new file mode 100644
index 0000000..08d2472
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CandidateLogsManager.java
@@ -0,0 +1,1249 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.util.Pair;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.NoSuchElementException;
+
+/**
+ *  Manages the candidate log files for a given quorum on a member node. It holds
+ *  a collection of committed transaction logs fetched from other members. These
+ *  "candidate" logs can overlap in their ranges. The transaction log manager can
+ *  inform the candidate log manager of its current indices so that the latter
+ *  can merge some of its holdings into the committed transaction logs category.
+ *
+ *  <p/>
+ *  The candidate log directory is of a flat structure. Each candidate log file is
+ *  uniquely identified by its name.
+ *
+ *  <p/>
+ *  Data Structures
+ *
+ *  We maintain two maps of LogFileInfo, one indexed by initial index and the other
+ *  by filename.
+ *
+ *  candidateLogsByIndex --
+ *
+ *  candidateLogsByName --
+ *
+ *  Invariants
+ *
+ *  1. The two maps, candidateLogsByIndex and candidateLogsByName agree.
+ *  2. No two LogFileInfo overlap each other.
+ */
+public class CandidateLogsManager {
+  private static final Logger LOG = LoggerFactory.getLogger(CandidateLogsManager.class);
+  private static final Logger CheckInvariantsLogger = LoggerFactory.getLogger(
+      CandidateLogsManager.class.getName()+"#checkInvariants");
+
+  /****************************************************************************
+   *                             member variables
+   ****************************************************************************/
+  private final String contextName;
+  private final String logsDirectory;
+  private final String logPrefix;
+
+  private Configuration conf;
+  private volatile long minCandidateLogSize =
+    HConstants.RAFT_CANDIDATE_LOG_MIN_SIZE_DEFAULT;
+
+  /** RW lock to manage the log roll and read access to the log */
+  private final ReentrantReadWriteLock indexLock;
+
+  /**
+   *  indexed by starting index.
+   */
+  private TreeMap<Long, LogFileInfo> candidateLogsByIndex;
+
+  /**
+   *  indexed by base filename
+   */
+  private Map<String, LogFileInfo> candidateLogsByName;
+
+
+  /****************************************************************************
+   *                             constructor
+   ****************************************************************************/
+  public CandidateLogsManager(Configuration conf, String contextName, String dir) {
+    this.indexLock = new ReentrantReadWriteLock();
+    this.candidateLogsByIndex = new TreeMap<Long, LogFileInfo>();
+    this.candidateLogsByName = new HashMap<String, LogFileInfo>();
+    this.contextName = contextName;
+    if (!dir.endsWith(HConstants.PATH_SEPARATOR)) {
+      this.logsDirectory = dir + HConstants.PATH_SEPARATOR;
+    } else {
+      this.logsDirectory = dir;
+    }
+
+    updateConf(conf);
+
+    logPrefix = this + " : ";
+  }
+
+  /**
+   *  initialize
+   *
+   *  Just make the directories.
+   */
+  public void initialize() throws IOException {
+    createLogDirectory(getCandidateLogsDirectory().toString());
+    createLogDirectory(getCandidateLogsWorkingDirectory().toString());
+  }
+
+  public void dumpState() {
+    if (!getConf().getBoolean(
+          "hbase.raft.candidate.logs.manager.dump.state.enabled",
+          false)
+    ) {
+      return;
+    }
+    if (isTraceEnabled()) {
+      indexLock.readLock().lock();
+      try {
+        LOG.trace(logPrefix() + "CandidateLogsByName:");
+        for (String filename : candidateLogsByName.keySet()) {
+          LOG.trace(logPrefix() + filename + " => " +
+            candidateLogsByName.get(filename));
+        }
+        LOG.trace(logPrefix() + "CandidateLogsByIndex:");
+        for (Long index : candidateLogsByIndex.keySet()) {
+          LOG.trace(logPrefix() + index + " => " +
+            candidateLogsByIndex.get(index));
+        }
+      } finally {
+        indexLock.readLock().unlock();
+      }
+    }
+  }
+  /****************************************************************************
+   *                          invariants checker
+   ****************************************************************************/
+  public boolean checkInvariants() {
+    long t0 = System.nanoTime();
+    indexLock.readLock().lock();
+    try {
+      boolean passed = true;
+      // every file in candidateLogsByName is in candidateLogsByIndex and vice
+      // versa. We don't use Set.equals because it may be an N^2 algorithm on
+      // generic types.
+      dumpState();
+      for (String filename : candidateLogsByName.keySet()) {
+        LogFileInfo info = candidateLogsByName.get(filename);
+        if (info == null) {
+          LOG.error(logPrefix() + "Null value found in candidateLogsByName for filename " + filename);
+          passed = false;
+        } else if (!info.validate()) {
+          LOG.error(logPrefix() + "Failed to validate candidate log from candidateLogsByName[" + filename + "]: " + info);
+          passed = false;
+        } else if (!filename.equals(info.getFilename())) {
+          LOG.error(logPrefix() + "Candidate log info [" + info + "] contains a different file name than " + filename
+              + ": " + info.getFilename());
+          passed = false;
+        } else if (candidateLogsByIndex.get(info.getInitialIndex()) == null) {
+          LOG.error(logPrefix() + "Candidate log info [" + info + "] is missing from candidateLogsByIndex but present in candidateLogsByName");
+          passed = false;
+        } else if (!info.equals(candidateLogsByIndex.get(info.getInitialIndex()))) {
+          LOG.error(logPrefix() + "Mismatched LogFileInfo found between "
+              + "candidateLogsByName[" + filename + "]"
+              + " and "
+              + " candidateLogsByIndex[" + info.getInitialIndex() + "]"
+              + " : [" + info + "] vs. [" + candidateLogsByIndex.get(info.getInitialIndex()) + "]");
+          passed = false;
+        }
+      }
+
+      for (Long index : candidateLogsByIndex.keySet()) {
+        LogFileInfo info = candidateLogsByIndex.get(index);
+        if (info == null) {
+          LOG.error(logPrefix() + "Null value found in candidateLogsByIndex for index " + index);
+          passed = false;
+        } else if (!info.validate()) {
+          LOG.error(logPrefix() + "Failed to validate candidate log from candidateLogsByIndex[" + index + "]: " + info);
+          passed = false;
+        } else if (index != info.getInitialIndex()) {
+          LOG.error(logPrefix() + "Candidate log info [" + info + "] contains a different index than " + index);
+          passed = false;
+        } else if (candidateLogsByName.get(info.getFilename()) == null) {
+          LOG.error(logPrefix() + "Candidate log info [" + info + "] is present in candidateLogsByIndex but missing from candidateLogsByName");
+          passed = false;
+        } else if (!info.equals(candidateLogsByName.get(info.getFilename()))) {
+          LOG.error(logPrefix() + "Mismatched LogFileInfo found between "
+              + "candidateLogsByIndex[" + index + "]"
+              + " and "
+              + "candidateLogsByName[" + info.getFilename() + "]"
+              + " : [" + info + "] vs. [" + candidateLogsByName.get(info.getFilename()) + "]");
+          passed = false;
+        }
+      }
+
+      // no two files in candidateLogsByIndex overlap
+      long prevLastIndex = Long.MIN_VALUE;
+      for (LogFileInfo info : candidateLogsByIndex.values()) {
+        if (prevLastIndex >= info.getInitialIndex()) {
+          LOG.error(logPrefix() + "Previous last index " + prevLastIndex + " >= the first index of " + info);
+          passed = false;
+        }
+        prevLastIndex = info.getLastIndex();
+      }
+      return passed;
+    } finally {
+      indexLock.readLock().unlock();
+      if (CheckInvariantsLogger.isDebugEnabled()) {
+        CheckInvariantsLogger.debug(logPrefix() + "checkInvariants took " + (System.nanoTime()-t0)/1000L + " us");
+      }
+    }
+  }
+
+
+  /****************************************************************************
+   *                   auxiliary: conf, dirs, etc
+   ****************************************************************************/
+  public final synchronized void updateConf(Configuration conf) {
+    this.conf = new Configuration(conf);
+    minCandidateLogSize = conf.getLong(
+        HConstants.RAFT_CANDIDATE_LOG_MIN_SIZE_KEY,
+        HConstants.RAFT_CANDIDATE_LOG_MIN_SIZE_DEFAULT
+    );
+  }
+
+  public final long getMinimumCandidateLogSize() {
+    return minCandidateLogSize;
+  }
+
+  public final synchronized Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   *  getCandidateLogsDirectory
+   *
+   *  Returns the absolute path of the directory holding the candidate logs.
+   *
+   *  @return         Path
+   */
+  public final Path getCandidateLogsDirectory() {
+    return Paths.get(
+        logsDirectory
+       + getConf().get(HConstants.RAFT_CANDIDATE_LOG_DIRECTORY_NAME_KEY,
+          HConstants.RAFT_CANDIDATE_LOG_DIRECTORY_NAME_DEFAULT));
+  }
+
+  /**
+   *  getCandidateLogsWorkingDirectory
+   *
+   *  Returns the absolute path of the working directory where the fetcher(s)
+   *  can store temporary candidate log files while they are being written
+   *  to.
+   *
+   *  @return         Path
+   */
+  public final Path getCandidateLogsWorkingDirectory() {
+    return Paths.get(
+        logsDirectory
+        + getConf().get(HConstants.RAFT_CANDIDATE_LOG_DIRECTORY_NAME_KEY,
+          HConstants.RAFT_CANDIDATE_LOG_DIRECTORY_NAME_DEFAULT)
+        + HConstants.PATH_SEPARATOR + "tmp");
+  }
+
+
+  /**
+   *  isInCandidateLogsDirectory
+   *
+   *  Checks if a given file is in one of the subdirectory of the main candidate logs
+   *  directory. This does not assume a flat directory.
+   */
+  public boolean isInCandidateLogsDirectory(File file) {
+    if (file == null) {
+      return false;
+    }
+    if (isDebugEnabled()) LOG.debug(logPrefix() + "checking if " + file.getAbsolutePath() + " is in " + getCandidateLogsDirectory());
+    Path parentDir = Paths.get(file.getParentFile().getAbsolutePath());
+    boolean isInCandidateLogsDirectory = false;
+    while (parentDir != null) {
+      if (parentDir.equals(getCandidateLogsDirectory())) {
+        return true;
+      }
+      parentDir = parentDir.getParent();
+    }
+    return false;
+  }
+
+  /****************************************************************************
+   *                   core routines: scan, add and I/O
+   ****************************************************************************/
+
+  /**
+   *  scan
+   *
+   *  Scan the candidate logs directory and add files to the collection.
+   */
+  public void scan() throws IOException {
+    List<File> candidateLogs = null;
+    try {
+      candidateLogs = scanCandidateLogsDirectory();
+    } catch (IOException ex) {
+      LOG.error(logPrefix() + "Failed to read the candidate logs directory " + getCandidateLogsDirectory(), ex);
+      throw ex;
+    }
+    if (candidateLogs == null) {
+      throw new IOException(
+          "Failed to read the candidate logs directory " + getCandidateLogsDirectory());
+    }
+    for (File f : candidateLogs) {
+      try {
+        if (addLogFile(f) == null) {
+          LOG.error(logPrefix() + "Failed to add file " + f.getAbsolutePath() + "; deleting it ......");
+          safeDeleteFile(f);
+        }
+      } catch (IOException ex) {
+        LOG.error(logPrefix() + "Failed to add file " + f.getAbsolutePath() + "; deleting it ......", ex);
+        safeDeleteFile(f);
+      }
+
+    }
+  }
+
+  /**
+   *  addLogFile
+   *
+   *  Add a candidate file to the collection.
+   *
+   *  @param  path    String
+   *  @return         LogFileInfo
+   */
+  public LogFileInfo addLogFile(String path) throws IOException {
+    if (path == null) {
+      LOG.error(logPrefix() + "a null path is passed to addLogFile");
+      return null;
+    }
+    return addLogFile(new File(path));
+  }
+
+  /**
+   *  addLogFile
+   *
+   *  Add a file already in the candidate logs directory. No renaming will be done.
+   *
+   *  The possible I/O actions are truncating and deleting files.
+   */
+  public LogFileInfo addLogFile(File file) throws IOException {
+    if (!checkFile(file)) {
+      LOG.error(logPrefix() + "Invalid file passed to addLogFile: " + file);
+      return null;
+    }
+    // we only process a file if it's in the candiate logs directory
+    if (file.getParentFile().getAbsolutePath() == null) {
+      LOG.error(logPrefix() + "The new candidate file " + file.getAbsolutePath()
+          + " is not in any directory; weird!");
+      return null;
+    } else if (!isInCandidateLogsDirectory(file)) {
+      LOG.error(logPrefix() + "The new candidate file " + file.getAbsolutePath() + " is in "
+          + file.getParentFile().getAbsolutePath()
+          + " that is different from the candidate logs directory "
+          + getCandidateLogsDirectory());
+      return null;
+    }
+    LOG.info(logPrefix() +"Adding log file " + file);
+    // check if the file is already in the collection
+    String filename = file.getName();
+    RandomAccessLog newLogFile = null;
+    indexLock.writeLock().lock();
+    try {
+      LogFileInfo info = candidateLogsByName.get(filename);
+      if (info != null) {
+        if (isDebugEnabled()) LOG.debug(logPrefix() + "File " + file.getAbsolutePath() + " is already indexed");
+        return info;
+      }
+
+      LogFileInfo newFileInfo = null;
+      RandomAccessLog logFile = createRandomAccessLog(file, true);
+      FileLock lock = lockRandomAccessLog(logFile);
+      try {
+        // Rebuild the index by scanning the log file
+        logFile.rebuild(toString());
+
+        if (logFile.getInitialIndex() == HConstants.UNDEFINED_TERM_INDEX) {
+          throw new IOException(
+              "found invalid initial index in " + file.getAbsolutePath());
+        } else if (logFile.getLastIndex() == HConstants.UNDEFINED_TERM_INDEX) {
+          throw new IOException("found invalid last index in " + file.getAbsolutePath());
+        }
+
+        List<LogFileInfo> deleteTargets = new ArrayList<LogFileInfo>();
+        Pair<LogFileInfo, LogFileInfo> result = computeMergeAndDelete(
+            logFile.getInitialIndex(),
+            logFile.getLastIndex(),
+            candidateLogsByIndex,
+            deleteTargets
+        );
+        if (result == null) {
+          LOG.warn(logPrefix() + "failed to find an appropriate hole for " + logFile);
+          return null;
+        }
+        LogFileInfo headOverlapLog = result.getFirst();
+        LogFileInfo tailOverlapLog = result.getSecond();
+
+        // Prepare the new file first by truncating if necessary.
+        if (headOverlapLog == null) {
+          if (tailOverlapLog != null) {
+            LOG.info(logPrefix() +"truncating new log file to the initial index of tail overlap at " + tailOverlapLog.getInitialIndex());
+            truncate(logFile, tailOverlapLog.getInitialIndex());
+            File newFile = new File(logFile.getFile().getAbsolutePath() + ".subcopy_to_"
+                + logFile.getCurrentTerm() + "_" + logFile.getInitialIndex() + "_" + (tailOverlapLog.getInitialIndex()-1));
+
+            logFile.finalizeForWrite();
+
+            LOG.info(logPrefix() +"Renaming the truncated file " + file.getAbsolutePath() + " to " + newFile.getAbsolutePath());
+            if (!renameFile(file, newFile)) {
+              LOG.error(logPrefix() + "failed to rename the truncated file " + file.getAbsolutePath() + " to " + newFile.getAbsolutePath());
+              return null;
+            } else {
+              LOG.info(logPrefix() +"Renamed the truncated file " + file.getAbsolutePath() + " to " + newFile.getAbsolutePath());
+              file = newFile;
+            }
+          } else {
+            // We are done preparing the new file.
+            logFile.finalizeForWrite();
+          }
+          newFileInfo = createLogFileInfo(file, logFile);
+          // we still have the file locked; therefore the mod time is legit.
+          newFileInfo.setLastVerifiedModificationTime(getModificationTime(file));
+        } else {
+          long newInitialIndex = headOverlapLog.getLastIndex() + 1;
+          long newLastIndex = tailOverlapLog != null ? tailOverlapLog.getInitialIndex() - 1 : logFile.getLastIndex();
+
+          assert newInitialIndex <= logFile.getLastIndex();
+          assert newInitialIndex >= logFile.getInitialIndex();
+          assert newLastIndex >= logFile.getInitialIndex();
+          assert newLastIndex <= logFile.getLastIndex();
+
+          LOG.info(logPrefix() +"About to truncate new log file " + logFile + " to the range + ["
+              + newInitialIndex + ", " + newLastIndex + "]");
+          if (newLastIndex + 1 - newInitialIndex < getMinimumCandidateLogSize()) {
+            LOG.warn(logPrefix() + "the resultant log file would be smaller than the threshold " + getMinimumCandidateLogSize()
+                + "; skip it.");
+            return null;
+          }
+          File newFile = new File(file.getAbsolutePath() + ".subcopy_to_"
+              + logFile.getCurrentTerm() + "_" + newInitialIndex + "_" + newLastIndex);
+          newLogFile = subCopyRandomAccessLog(logFile, newInitialIndex, newLastIndex, newFile);
+          if (newLogFile == null) {
+            throw new IOException("failed to copy a subsection [" + newInitialIndex + ", " + newLastIndex
+                + "] of the target random access log " + logFile);
+          }
+          newLogFile.finalizeForWrite();
+          newFileInfo = createLogFileInfo(newFile, newLogFile);
+          newFileInfo.setLastVerifiedModificationTime(getModificationTime(newFile));
+          safeDeleteFile(file);
+        }
+        if (newFileInfo == null) {
+          throw new IOException("WEIRD: we have a null newLogFileInfo!");
+        }
+        for (LogFileInfo deleteTarget : deleteTargets) {
+          LOG.info(logPrefix() +"safe-delete the log file " + deleteTarget);
+          safeDeleteCandidateLog(deleteTarget);
+        }
+        insertIntoCollection(newFileInfo);
+        return newFileInfo;
+      } catch (Exception ex) {
+        LOG.error(logPrefix() + "is unable to rebuild the current log file " + logFile.getFileName(), ex);
+        if (newLogFile != null) {
+          LOG.warn(logPrefix() + "also deleting the new log file " + newLogFile.getFileName());
+          safeDeleteFile(newLogFile.getFile());
+        }
+        throw new IOException("is unable to rebuild the current log file " +
+          logFile.getFileName(), ex);
+      } finally {
+        unlockRandomAccessLog(lock);
+      }
+    } catch (IOException ex) {
+      LOG.error(logPrefix() + "caught an exception while adding file " + file.getAbsolutePath(), ex);
+    } finally {
+      indexLock.writeLock().unlock();
+    }
+    return null;
+  }
+
+  /**
+   *  safeDeleteCandidateLog
+   *
+   *  @param  info      LogFileInfo
+   *
+   *  @return boolean
+   */
+  public boolean safeDeleteCandidateLog(LogFileInfo info) {
+    indexLock.writeLock().lock();
+    try {
+      removeFromCollection(info);
+      return safeDeleteFile(new File(info.getAbsolutePath()));
+    } finally {
+      indexLock.writeLock().unlock();
+    }
+  }
+
+  /**
+   *  truncate
+   *
+   *  Truncates a RandomAccessLog file. This method exists mainly for unit tests.
+   *
+   *  @param  logFile   RandomAccessLog
+   *  @param  index     long
+   *  @return EditId    the last edit id
+   */
+  private EditId truncate(RandomAccessLog logFile, long index) throws IOException {
+    logFile.truncate(index);
+    logFile.finalizeForWrite();
+    return new EditId(logFile.getCurrentTerm(), logFile.getLastIndex());
+  }
+
+  /****************************************************************************
+   *                   basic I/O routines (mockable)
+   ****************************************************************************/
+
+  /**
+   *  createLogDirectory
+   *
+   */
+  protected void createLogDirectory(String path) throws IOException {
+    File currentDirectory = new File (path);
+
+    if (!currentDirectory.exists()) {
+      if (!currentDirectory.mkdirs()) {
+        throw new IOException("Cannot create dir at " +
+          currentDirectory.getAbsolutePath());
+      }
+    }
+  }
+
+  /**
+   *  fastCopy
+   *
+   *  Copy a contiguous range of transactions from one RandomAccessLog to another
+   *  File by copying bytes.
+   *
+   *  @param  oldLog
+   *  @param  newInitialIndex
+   *  @param  newLastIndex
+   *  @param  tmpFile
+   *
+   *
+   */
+  protected void fastCopy(
+      RandomAccessLog   oldLog,
+      final long        newInitialIndex,
+      final long        newLastIndex,
+      File              tmpFile
+  ) throws IOException {
+    LOG.info(logPrefix() +"fast-copying range  [" + newInitialIndex + ", " + newLastIndex
+                  + "] of " + oldLog + " to tmp file " + tmpFile.getAbsolutePath());
+    String sessionKey = tmpFile.getAbsolutePath();
+    long term = oldLog.getCurrentTerm();
+    LogReader srcReader = new LogReader(oldLog.getFile());
+    try {
+      RandomAccessFile srcFile = srcReader.getRandomAccessFile();
+      long copyStartOffset = oldLog.getTransactionFileOffset(term, newInitialIndex);
+      long copyEndOffset = newLastIndex < oldLog.getLastIndex() ?
+        oldLog.getTransactionFileOffset(term, newLastIndex + 1) : srcFile.length();
+
+
+      FileChannel inChannel = null;
+      try {
+        inChannel = new FileInputStream(oldLog.getFile()).getChannel();
+        FileChannel outChannel = null;
+        try {
+          outChannel = new FileOutputStream(tmpFile).getChannel();
+
+          ByteBuffer bbuf = LogWriter.generateFileHeader(term, newInitialIndex);
+          do {
+            outChannel.write(bbuf);
+          } while (bbuf.remaining() > 0);
+
+          LOG.info(logPrefix() +"copying the offset window ["
+              + copyStartOffset + ", " + copyEndOffset + ") from "
+              + oldLog + " to " + tmpFile);
+          long copyOffset = copyStartOffset;
+          while (copyOffset < copyEndOffset) {
+            if (isDebugEnabled()) {
+              LOG.debug(logPrefix() + "---- copying [" + copyOffset + ", " + copyEndOffset
+                  + ") using FileChannel.transferTo ......");
+            }
+            long n = inChannel.transferTo(copyOffset, copyEndOffset - copyOffset, outChannel);
+            if (isDebugEnabled()) {
+              LOG.debug(logPrefix() + "---- copied " + n + " bytes from [" + copyOffset + ", " + copyEndOffset
+                  + ") using FileChannel.transferTo.");
+            }
+            if (n <= 0) {
+              LOG.error(logPrefix() + "FileChannel.transferTo return 0!");
+              throw new IOException("FileChannel.transferTo return 0 while copying ["
+                  + copyOffset + ", " + copyEndOffset + ") of " + oldLog + " to "
+                  + tmpFile);
+            }
+            copyOffset += n;
+          }
+        } finally {
+          if (outChannel != null) {
+            outChannel.close();
+          }
+        }
+      } finally {
+        if (inChannel != null) {
+          inChannel.close();
+        }
+      }
+
+      // create a new log and perform some sanity test
+      if (isDebugEnabled()) {
+        LOG.debug(logPrefix() + "Checking the result of copying " + oldLog + " to " + tmpFile);
+      }
+      RandomAccessLog newLog = createRandomAccessLog(tmpFile, true);
+      newLog.rebuild(sessionKey);
+      if (newLog.getInitialIndex() != newInitialIndex) {
+        throw new IOException("The copied file has the wrong initial index: "
+            + newLog.getInitialIndex() + " vs. " + newInitialIndex);
+      } else if (newLog.getLastIndex() != newLastIndex) {
+        throw new IOException("The copied file has the wrong last index: "
+            + newLog.getLastIndex() + " vs. " + newLastIndex);
+      }
+      // TODO: shall we check the file size??
+    } catch (NoSuchElementException ex) {
+      safeDeleteFile(tmpFile);
+      throw new IOException("Invalid indices specified", ex);
+    } catch (IOException ex) {
+      LOG.warn(logPrefix() + "Failed to fast-copy range  [" + newInitialIndex + ", " + newLastIndex
+          + "] of " + oldLog + " to tmp file " + tmpFile.getAbsolutePath(), ex);
+      safeDeleteFile(tmpFile);
+      throw ex;
+    } finally {
+      srcReader.close();
+    }
+  }
+
+  /**
+   *  slowCopy
+   *
+   *  Copy a contiguous range of transactions from one RandomAccessLog to anther
+   *  File by reading and appending one transaction at a time.
+   *
+   *  @param  oldLog
+   *  @param  newInitialIndex
+   *  @param  newLastIndex
+   *  @param  tmpFile
+   */
+  protected void slowCopy(
+      RandomAccessLog   oldLog,
+      final long        newInitialIndex,
+      final long        newLastIndex,
+      File              tmpFile
+  ) throws IOException {
+    LOG.info(logPrefix() +"slow-copying range  [" + newInitialIndex + ", " + newLastIndex
+                  + "] of " + oldLog + " to tmp file " + tmpFile.getAbsolutePath());
+
+    String sessionKey = tmpFile.getAbsolutePath();
+    long term = oldLog.getCurrentTerm();
+    LOG.debug(logPrefix() + "slowCopy term = " + term);
+    try {
+
+      RandomAccessLog newLog = new RandomAccessLog(tmpFile, false);
+      for (long index=newInitialIndex; index<=newLastIndex; index++) {
+        MemoryBuffer
+          buffer = oldLog.getTransaction(term, index, sessionKey, null);
+        if (isDebugEnabled()) {
+          LOG.debug(logPrefix() + "Writing " + buffer.getBuffer().remaining()
+            + " bytes at index " + index + " to " + tmpFile.getAbsolutePath());
+        }
+        newLog.append(term, index, buffer.getBuffer());
+      }
+      newLog.finalizeForWrite();
+    } catch (IOException ex) {
+      safeDeleteFile(tmpFile);
+      throw ex;
+    }
+  }
+
+  /**
+   *  subCopyRandomAccessLog
+   *
+   *  Copy a contiguous range of transactions from one RandomAccessLog to anther
+   *  File by using a temporary file as the intermediary.
+   *
+   *  @param  oldLog
+   *  @param  newInitialIndex
+   *  @param  newLastIndex
+   *  @param  newFile
+   */
+  public RandomAccessLog subCopyRandomAccessLog(
+      RandomAccessLog   oldLog,
+      final long        newInitialIndex,
+      final long        newLastIndex,
+      File              newFile
+  ) throws IOException {
+    if (newFile == null || oldLog == null) {
+      return null;
+    }
+    if (newInitialIndex > newLastIndex ||
+        newInitialIndex < oldLog.getInitialIndex() ||
+        newInitialIndex > oldLog.getLastIndex() ||
+        newLastIndex > oldLog.getLastIndex() ||
+        newLastIndex < oldLog.getInitialIndex()
+    ) {
+      LOG.error(logPrefix() + "Invalid range [" + newInitialIndex + ", " + newLastIndex
+          + "] specified fo subcopying " + oldLog);
+      return null;
+    }
+
+    File tmpFile = new File(getCandidateLogsWorkingDirectory()
+        + HConstants.PATH_SEPARATOR
+        + ("tmp_subcopy_at_" + System.nanoTime() + newFile.getName()));
+
+    if (getConf().getBoolean(
+          HConstants.RAFT_CANDIDATE_FAST_COPY_LOG_KEY,
+          HConstants.RAFT_CANDIDATE_FAST_COPY_LOG_DEFAULT)
+    ) {
+      fastCopy(oldLog, newInitialIndex, newLastIndex, tmpFile);
+    } else {
+      slowCopy(oldLog, newInitialIndex, newLastIndex, tmpFile);
+    }
+
+    LOG.info(logPrefix() +"Renaming " + tmpFile.getAbsolutePath() + " to " + newFile.getAbsolutePath());
+    if (!renameFile(tmpFile, newFile)) {
+      LOG.error(logPrefix() + "Failed to rename " + tmpFile.getAbsolutePath() + " to " + newFile.getAbsolutePath());
+      safeDeleteFile(tmpFile);
+      safeDeleteFile(newFile);
+      return null;
+    }
+
+    LOG.info(logPrefix() +"Copied range  [" + newInitialIndex + ", " + newLastIndex
+                  + "] of " + oldLog + " to tmp file " + newFile.getAbsolutePath());
+    return createRandomAccessLog(newFile, true);
+  }
+
+  public boolean renameFile(File oldFile, File newFile) {
+    return oldFile.renameTo(newFile);
+  }
+
+  protected long getModificationTime(File f) {
+    return f.lastModified();
+  }
+
+  protected LogFileInfo createLogFileInfo(File file, RandomAccessLog logFile) {
+    return new LogFileInfo(
+        file.getAbsolutePath(),
+        file.length(),
+        new EditId(logFile.getCurrentTerm(), logFile.getInitialIndex()),
+        new EditId(logFile.getCurrentTerm(), logFile.getLastIndex()),
+        file.lastModified(), RandomAccessLog.populateCreationTime(file)
+    );
+  }
+
+  /**
+   *  scanCandidateLogsDirectory
+   *
+   *  Scan the candidate logs directory for all log files.
+   */
+  protected List<File> scanCandidateLogsDirectory() throws IOException {
+    List<File> files = new ArrayList<File>();
+    Path candidateLogsDirectory = getCandidateLogsDirectory();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(candidateLogsDirectory)) {
+      for (Path entry: stream) {
+        File f = entry.toFile();
+        if (checkFile(f)) {
+          files.add(f);
+        }
+      }
+    } catch (IOException ex) {
+      LOG.error(logPrefix() + "Failed to read the candidate logs directory " + candidateLogsDirectory, ex);
+      throw ex;
+    }
+    return files;
+  }
+
+  /**
+   *  checkFile
+   *
+   *  @param  f File
+   *  @return boolean
+   *
+   */
+  protected boolean checkFile(File f) {
+    return f != null && f.isFile();
+  }
+
+  /**
+   *  createRandomAccessLog
+   *
+   *  @param  file    File
+   *  @param  sync    boolean
+   */
+  public RandomAccessLog createRandomAccessLog(File file, boolean sync) throws IOException {
+    return new RandomAccessLog(file, sync);
+  }
+
+  public RandomAccessFile createRandomAccessFile(File file, boolean isSync) throws IOException {
+    return new RandomAccessFile(file, isSync ? "rws" : "rw");
+  }
+
+  public LogWriter createLogWriter(File file, boolean isSync) throws IOException {
+    return new LogWriter(createRandomAccessFile(file, isSync), isSync);
+  }
+
+  /**
+   *  lockRandomAccessLog
+   *
+   *  @param  logFile RandomAccessLog
+   *  @return         FileLock
+   */
+  protected FileLock lockRandomAccessLog(RandomAccessLog logFile) throws IOException {
+    RandomAccessFile raf = logFile.getRandomAccessFile();
+    return raf.getChannel().lock();
+  }
+
+  /**
+   *  unlockRandomAccessLog
+   *
+   *  @param  lock    FileLock
+   */
+  protected void unlockRandomAccessLog(FileLock lock) throws IOException {
+    lock.release();
+  }
+
+  /**
+   *  safeDeleteFile
+   *
+   *  Deletes a file represented by the file object. Only a file in the candidate logs
+   *  directory can be deleted.
+   *
+   *  @param  file      File
+   *
+   *  @return boolean
+   */
+  public boolean safeDeleteFile(File file) {
+    if (!isInCandidateLogsDirectory(file)) {
+      LOG.error(logPrefix() + "The to-be-deleted file " + file.getAbsolutePath() + " is not in "
+          + getCandidateLogsDirectory());
+      return false;
+    }
+    LOG.info(logPrefix() +"deleting file " + file.getAbsolutePath());
+    if (!file.delete()) {
+      LOG.error(logPrefix() + "failed to delete file " + file.getAbsolutePath());
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  /****************************************************************************
+   *                   core routines: collection management
+   ****************************************************************************/
+  public Collection<LogFileInfo> getAllScannedFiles() {
+    indexLock.readLock().lock();
+    try {
+      return candidateLogsByIndex.values();
+    } finally {
+      indexLock.readLock().unlock();
+    }
+  }
+
+  /**
+   *  insertIntoCollection
+   *
+   *  Insert a log file into the collection. This doesn't cause the file to be deleted.
+   */
+  public void insertIntoCollection(LogFileInfo info) throws IOException {
+    if (info == null) {
+      return;
+    }
+    indexLock.writeLock().lock();
+    try {
+      if (candidateLogsByIndex.get(info.getInitialIndex()) != null) {
+        throw new IOException("there is already a log file at index " + info.getInitialIndex());
+      }
+      if (candidateLogsByName.get(info.getFilename()) != null) {
+        throw new IOException("there is already a log file with name " + info.getFilename());
+      }
+      candidateLogsByIndex.put(info.getInitialIndex(), info);
+      candidateLogsByName.put(info.getFilename(), info);
+    } finally {
+      indexLock.writeLock().unlock();
+      checkInvariants();
+    }
+  }
+
+  /**
+   *  removeFromCollection
+   *
+   *  Remove a log file from the collection. This doesn't cause the file to be deleted.
+   */
+  public void removeFromCollection(LogFileInfo info) {
+    if (info == null) {
+      return;
+    }
+    indexLock.writeLock().lock();
+    try {
+      candidateLogsByIndex.remove(info.getInitialIndex());
+      candidateLogsByName.remove(info.getFilename());
+    } finally {
+      indexLock.writeLock().unlock();
+      checkInvariants();
+    }
+  }
+
+  /**
+   *  getByIndex
+   *
+   *  Retrieves the LogFileInfo containing the index
+   */
+  public LogFileInfo getByIndex(long index) {
+    indexLock.readLock().lock();
+    try {
+      // FACT: headMap contains keys <= index
+      SortedMap<Long, LogFileInfo> headMap = candidateLogsByIndex.headMap(index+1);
+      if (headMap != null && !headMap.isEmpty()) {
+        LogFileInfo info = headMap.get(headMap.lastKey());
+        if (info.getInitialIndex() <= index && info.getLastIndex() >= index) {
+          return info;
+        }
+      }
+      return null;
+    } finally {
+      indexLock.readLock().unlock();
+    }
+  }
+
+  /**
+   *  coverage
+   *
+   *  Computes the number of transactions current set of candidate logs
+   *  contain.
+   */
+  public long coverage() {
+    indexLock.readLock().lock();
+    try {
+      long coverage = 0L;
+      for (Long index : candidateLogsByIndex.keySet()) {
+        coverage += candidateLogsByIndex.get(index).getTxnCount();
+      }
+      return coverage;
+    } finally {
+      indexLock.readLock().unlock();
+    }
+  }
+
+  /**
+   *  size
+   *
+   *  Returns the number of indexed candidate logs.
+   */
+  public int size() {
+    indexLock.readLock().lock();
+    try {
+      return candidateLogsByIndex.size();
+    } finally {
+      indexLock.readLock().unlock();
+    }
+  }
+
+  public boolean isEmpty() {
+    indexLock.readLock().lock();
+    try {
+      return candidateLogsByIndex.isEmpty();
+    } finally {
+      indexLock.readLock().unlock();
+    }
+  }
+
+  public boolean exists(EditId id) {
+    if (id == null) {
+      return false;
+    }
+    LogFileInfo info = getByIndex(id.getIndex());
+    return info != null && info.getFirstEditId().getTerm() <= id.getTerm()
+      && id.getTerm() <= info.getLastEditId().getTerm();
+  }
+
+  /**
+   *  prune
+   *
+   *  Delete all files with edits below index.
+   */
+  public void prune(long index) {
+    List<LogFileInfo> targets = new ArrayList<LogFileInfo>();
+    indexLock.readLock().lock();
+    try {
+      for (LogFileInfo info : candidateLogsByIndex.values()) {
+        if (info.getLastIndex() <= index) {
+          targets.add(info);
+        }
+      }
+    } finally {
+      indexLock.readLock().unlock();
+    }
+    for (LogFileInfo info : targets) {
+      LOG.info(logPrefix() +"pruning file " + info);
+      safeDeleteCandidateLog(info);
+    }
+  }
+
+  /**
+   *  getContiguousLogsContaining
+   *
+   *  @param  id                  EditId
+   *  @return List<LogFileInfo>
+   */
+  public List<LogFileInfo> getContiguousLogsContaining(EditId id) {
+    return getContiguousLogsContaining(id, Long.MAX_VALUE);
+  }
+
+  public List<LogFileInfo> getContiguousLogsContaining(EditId id, long lastLogIndex) {
+    if (id == null || id.getIndex() == HConstants.UNDEFINED_TERM_INDEX) {
+      return null;
+    }
+    List<LogFileInfo> candidates = new ArrayList<LogFileInfo>();
+    indexLock.readLock().lock();
+    try {
+      // FACT: headMap contains keys <= id.getIndex()
+      SortedMap<Long, LogFileInfo> headMap = candidateLogsByIndex.headMap(id.getIndex() + 1);
+      if (headMap == null || headMap.isEmpty()) {
+        return candidates;
+      }
+      LogFileInfo firstLog = headMap.get(headMap.lastKey());
+      if (firstLog.getLastIndex() < id.getIndex()) {
+        return candidates;
+      }
+      if (firstLog.getFirstEditId().getTerm() > id.getTerm() ||
+          firstLog.getLastEditId().getTerm() < id.getTerm())
+      {
+        return candidates;
+      }
+
+      candidates.add(firstLog);
+
+      // FACT: tailMap contains keys > firstLog.getLastIndex()
+      SortedMap<Long, LogFileInfo> tailMap = candidateLogsByIndex.tailMap(firstLog.getLastIndex() + 1);
+      if (tailMap == null || tailMap.isEmpty()) {
+        return candidates;
+      }
+
+      for (LogFileInfo nextLog : tailMap.values()) {
+        if (nextLog.getLastIndex() > lastLogIndex) {
+          break;
+        }
+        if (nextLog.getInitialIndex() == candidates.get(candidates.size()-1).getLastIndex() + 1) {
+          candidates.add(nextLog);
+        } else {
+          break;
+        }
+      }
+      return candidates;
+    } finally {
+      indexLock.readLock().unlock();
+    }
+  }
+
+  public String toString() {
+    return "CandidatesLogManager[" + contextName + "]";
+  }
+
+  /**
+   *  Invariants
+   *
+   *  Merge Invariants.
+   *
+   *  1.  initialIndex <= lastIndex
+   *  2.  headOverlapLog == null || initialIndex <= headOverlapLog.getLastIndex() < lastIndex
+   *  3.  tailOverlapLog == null || tailOverlapLog.getInitialIndex() <= lastIndex
+   *  4.  (tailOverlapLog != null && headOverlapLog != null) ==>
+   *      headOverlapLog.getLastIndex() + 1 < tailOverlapLog.getInitialIndex()
+   *  5.  foreach log in deleteTargets
+   *        log.getInitialIndex() > initialIndex && log.getLastIndex() <= lastIndex
+   */
+  public void checkMergeInvariants(
+      long                          initialIndex,
+      long                          lastIndex,
+      TreeMap<Long, LogFileInfo>    logs,
+      LogFileInfo                   headOverlapLog,
+      LogFileInfo                   tailOverlapLog,
+      List<LogFileInfo>             deleteTargets
+  ) {
+    if (isDebugEnabled()) LOG.debug(logPrefix() + "checkMergeInvariants: "
+        + "initialIndex=" + initialIndex
+        + ", lastIndex=" + lastIndex
+        + ", headOverlapLog=" + headOverlapLog
+        + ", tailOverlapLog=" + tailOverlapLog
+        + "\ndeleteTargets=" + deleteTargets
+    );
+    dumpState();
+
+    assert initialIndex <= lastIndex;
+
+    // note the head overlap may be extended in the walk
+    assert headOverlapLog == null ||
+      (initialIndex <= headOverlapLog.getLastIndex() && lastIndex > headOverlapLog.getLastIndex());
+
+    assert tailOverlapLog == null || tailOverlapLog.getInitialIndex() <= lastIndex;
+
+    if (tailOverlapLog != null && headOverlapLog != null) {
+      assert headOverlapLog.getLastIndex() + 1 < tailOverlapLog.getInitialIndex();
+    }
+    for (LogFileInfo log : deleteTargets) {
+      assert log.getInitialIndex() > initialIndex && log.getLastIndex() <= lastIndex;
+    }
+  }
+
+  /**
+   *  computeMergeAndDelete
+   *
+   *  Given a set of existing candidate logs (non-overlapping) and the initial and the last
+   *  indices of a new candidate log, computes one of the following.
+   *
+   *  1. If the new log partially or completely fills a hole in the existing set's coverage,
+   *     returns the existing logs that partially overlap the beginning and the end of the
+   *     new log, respectively, as well as any existing candidate logs that are fully subsumed
+   *     by this new log.
+   *  2. Otherwise, the new log file's coverage is already contained in the existing set;
+   *     returns null.
+   *
+   *  Isolated for unit testing.
+   */
+  public Pair<LogFileInfo, LogFileInfo> computeMergeAndDelete(
+      long                          initialIndex,
+      long                          lastIndex,
+      TreeMap<Long, LogFileInfo>    logs,
+      List<LogFileInfo>             deleteTargets
+  ) throws IOException {
+    LogFileInfo headOverlapLog = null;
+
+    // FACT: headMap contains keys <= initialIndex
+    SortedMap<Long, LogFileInfo> headMap = logs.headMap(initialIndex+1);
+
+    if (headMap != null && !headMap.isEmpty()) {
+      LogFileInfo log = headMap.get(headMap.lastKey());
+      if (lastIndex <= log.getLastIndex()) {
+        LOG.info(logPrefix() +" found an existing candidate log file " + log
+             + " that already covers the range of a new candidate file [" + initialIndex + ", " + lastIndex + "]");
+        return null;
+      } else if (log.getLastIndex() >= initialIndex) {
+        headOverlapLog = log;
+        if (isDebugEnabled()) LOG.debug(logPrefix() + " found an existing candidate log file " + headOverlapLog
+            + " that overlaps the range of a new candidate file [" + initialIndex + ", " + lastIndex + "]");
+      }
+    }
+    checkMergeInvariants(initialIndex, lastIndex, logs, headOverlapLog, null, deleteTargets);
+
+    // FACT: tailMap contains keys > initialIndex
+    SortedMap<Long, LogFileInfo> tailMap = logs.tailMap(initialIndex+1);
+
+    // FACT: headMap and tailMap are disjoint
+
+    if (tailMap != null && !tailMap.isEmpty()) {
+      tailMap = tailMap.headMap(lastIndex + 1);
+    }
+
+    // FACT: tailMap contains keys > initialIndex and <= lastIndex
+
+    // FACT: headMap and tailMap are disjoint
+
+    if (tailMap == null || tailMap.isEmpty()) {
+      checkMergeInvariants(initialIndex, lastIndex, logs, headOverlapLog, null, deleteTargets);
+      return new Pair<LogFileInfo, LogFileInfo>(headOverlapLog, null);
+    }
+
+    LogFileInfo tailOverlapLog = null;
+    for (LogFileInfo thisLogFile : tailMap.values()) {
+      // FACT: thisLogFile.getInitialIndex() > initialIndex
+      if (headOverlapLog != null && (headOverlapLog.getLastIndex() + 1) == thisLogFile.getInitialIndex()) {
+        // existing log files abut, update headOverlapLogLastIndex
+        LOG.info(logPrefix() +"Existing head overlap " + headOverlapLog
+            + " and next log file " + thisLogFile + " abut!");
+        if (thisLogFile.getLastIndex() >= lastIndex) {
+          LOG.info(logPrefix() +" found an existing set of candidate log files until " + headOverlapLog
+               + " that already cover the range of a new candidate file [" + initialIndex + ", " + lastIndex + "]");
+          return null;
+        }
+        headOverlapLog = thisLogFile;
+      } else if (headOverlapLog != null && headOverlapLog.getLastIndex() >= thisLogFile.getInitialIndex()) {
+        // ERROR!!
+        LOG.error(logPrefix() + "detected overlapping ranges at " + thisLogFile);
+        return null;
+      } else if (thisLogFile.getLastIndex() <= lastIndex) {
+        //  FACT: (headOverlapLogLastIndex == null || headOverlapLogLastIndex  + 1 < thisLogFile.getInitialIndex())
+        //        && thisLogFile.getLastIndex() <= lastIndex
+        //
+        //  Action: We can delete this file
+        if (isDebugEnabled()) LOG.debug(logPrefix() + " found an existing candidate log " + thisLogFile
+            + " that is fully contained by our new candidate file [" + initialIndex + ", " + lastIndex + "]");
+        deleteTargets.add(thisLogFile);
+      } else {
+        //        headOverlapLogLastIndex == null
+        //    &&  thisLogFile.getInitialIndex() > initialIndex
+        //    &&  thisLogFile.getLastIndex() > lastIndex
+        // OR
+        //        initialIndex  <= headOverlapLogLastIndex
+        //    &&  headOverlapLogLastIndex + 1 < thisLogFile.getInitialIndex()
+        //    &&  lastIndex < thisLogFile.getLastIndex()
+        if (isDebugEnabled()) LOG.debug(logPrefix() + " found a log file " + thisLogFile
+            + " that overlaps the tail of our new candidate file [" + initialIndex + ", " + lastIndex + "]");
+        tailOverlapLog = thisLogFile;
+        break;
+      }
+      checkMergeInvariants(initialIndex, lastIndex, logs, headOverlapLog, tailOverlapLog, deleteTargets);
+    }
+    checkMergeInvariants(initialIndex, lastIndex, logs, headOverlapLog, tailOverlapLog, deleteTargets);
+    return new Pair<LogFileInfo, LogFileInfo>(headOverlapLog, tailOverlapLog);
+  }
+
+  private boolean isTraceEnabled() {
+    return LOG.isTraceEnabled();
+  }
+
+  private boolean isDebugEnabled() {
+    return LOG.isDebugEnabled();
+  }
+
+  private String logPrefix() {
+    return logPrefix;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CommitLogManagerInterface.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CommitLogManagerInterface.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CommitLogManagerInterface.java
new file mode 100644
index 0000000..75f2b6b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CommitLogManagerInterface.java
@@ -0,0 +1,69 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.apache.hadoop.hbase.consensus.rpc.LogState;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.apache.hadoop.hbase.util.Pair;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public interface CommitLogManagerInterface {
+
+  public final static EditId UNDEFINED_EDIT_ID = new EditId(HConstants.UNDEFINED_TERM_INDEX,
+    HConstants.UNDEFINED_TERM_INDEX);
+
+  /** Append the txns for this editId; Return false if the append fails */
+  public boolean append(EditId editId, long commitIndex, final ByteBuffer data);
+
+  /** Return whether this editId exists in the transaction log */
+  public boolean isExist(EditId editId);
+
+  /** Truncate the log up to this editId. Return false if the editId does not
+   * exist. */
+  public boolean truncate(EditId editId) throws IOException;
+
+  /** Return the previous editId before the editId; Return null if this editId
+   * does not exist */
+  public EditId getPreviousEditID(EditId editId);
+
+  /** Return the last editId in the log */
+  public EditId getLastEditID();
+
+  /** Return the first editId in the log */
+  public long getFirstIndex();
+
+  public void initialize(final ImmutableRaftContext context);
+
+  public boolean isAccessible();
+
+  public Pair<EditId, MemoryBuffer> getNextEditIdTransaction(
+      final String sessionKey,
+      final long currentIndex,
+      final Arena arena) throws IOException;
+
+  public EditId getLastValidTransactionId();
+
+  public LogState getLogState();
+
+  public String dumpLogs(int n);
+
+  public String getPath();
+
+  public EditId getEditId(final long index);
+
+  public Pair<EditId, EditId> greedyIncorporateCandidateLogs(
+      String    sessionKey,
+      long      lastLogIndex
+  );
+
+  void fillLogGap(long seedIndex) throws IOException;
+
+  public List<LogFileInfo> getCommittedLogStatus(long minIndex);
+
+  public void stop();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/InMemoryLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/InMemoryLogManager.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/InMemoryLogManager.java
new file mode 100644
index 0000000..968950b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/InMemoryLogManager.java
@@ -0,0 +1,242 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.apache.hadoop.hbase.consensus.rpc.LogState;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.io.hfile.bucket.CacheFullException;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.BucketAllocatorException;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class InMemoryLogManager implements CommitLogManagerInterface {
+  private static final Logger LOG = LoggerFactory.getLogger(InMemoryLogManager.class);
+
+  ArrayList<EditId> transactionList;
+  Map<EditId, ByteBuffer> transactionMap;
+
+  @Override
+  public synchronized boolean append(EditId editId, long commitIndex,
+                                     final ByteBuffer txns) {
+    EditId cloneId = editId.clone();
+    if (transactionList.isEmpty() ||
+      (transactionList.get(transactionList.size() - 1).compareTo(editId) < 0)) {
+      transactionList.add(cloneId);
+      transactionMap.put(cloneId, txns);
+      return true;
+    }
+
+    return false;
+  }
+
+  @Override
+  public synchronized boolean isExist(EditId editId) {
+    if (editId.getTerm() == 0) {
+      return true;
+    }
+    return transactionMap.containsKey(editId);
+  }
+
+  @Override
+  public synchronized boolean truncate(EditId editId) {
+    if (editId.equals(TransactionLogManager.UNDEFINED_EDIT_ID)) {
+      transactionMap.clear();
+      transactionList.clear();
+      return true;
+    }
+    if (!isExist(editId)) {
+      return false;
+    }
+
+    if (editId.getTerm() == 0) {
+      transactionList.clear();
+      transactionMap.clear();
+      return true;
+    }
+
+    // delete all the elements up to this editId
+    int index = transactionList.indexOf(editId);
+    while (transactionList.size() > (index + 1)) {
+      EditId last = transactionList.remove(transactionList.size() - 1);
+      transactionMap.remove(last);
+    }
+
+    return true;
+  }
+
+  private synchronized MemoryBuffer getTransaction(String sessionKey,
+                                          EditId editId,
+                                          final Arena arena) {
+    final ByteBuffer entry = transactionMap.get(editId);
+    MemoryBuffer buffer = null;
+
+    if (entry != null) {
+      try {
+        buffer = arena.allocateByteBuffer(entry.remaining());
+      } catch (CacheFullException | BucketAllocatorException e) {
+        if (LOG.isTraceEnabled()) {
+          LOG.error("Unable to allocate buffer from arena", e);
+        }
+      }
+      buffer.getBuffer().put(entry.array(),
+        entry.arrayOffset() + entry.position(), entry.remaining());
+      buffer.flip();
+    }
+    return buffer;
+  }
+
+  @Override
+  public synchronized EditId getPreviousEditID(EditId editId) {
+    int index = transactionList.indexOf(editId);
+    if (index > 0) {
+      return transactionList.get(index - 1).clone();
+    } else {
+      return TransactionLogManager.UNDEFINED_EDIT_ID;
+    }
+  }
+
+  /**
+   * Used for hinting the leader, from where to catch up. This method tries to
+   * locate a latest term for which there are at least two edits and then returns
+   * the second last editid for that term.
+   * @return EditId
+   */
+  public synchronized EditId getLastValidTransactionId() {
+    int index = transactionList.size() - 1;
+
+    for (int ii = index ; ii > 0; ii--) {
+      if (transactionList.get(ii).getTerm() == transactionList.get(ii - 1).getTerm()) {
+        return transactionList.get(ii - 1);
+      }
+    }
+    return TransactionLogManager.UNDEFINED_EDIT_ID;
+  }
+
+  @Override
+  public synchronized EditId getLastEditID() {
+    if (transactionList.isEmpty()) {
+      return TransactionLogManager.UNDEFINED_EDIT_ID;
+    }
+    return transactionList.get(transactionList.size()-1).clone();
+  }
+
+  @Override
+  public long getFirstIndex() {
+    if (transactionList != null) {
+      return transactionList.get(0).getIndex();
+    }
+    return UNDEFINED_EDIT_ID.getIndex();
+  }
+
+  @Override
+  public void initialize(ImmutableRaftContext context) {
+    transactionList = new ArrayList<>();
+    transactionMap = new HashMap<>();
+  }
+
+  @Override
+  public boolean isAccessible() {
+    return true;
+  }
+
+  private synchronized EditId getNextEditId(EditId currentId) {
+    if (currentId.getTerm() == 0 && !transactionList.isEmpty()) {
+      return transactionList.get(0).clone();
+    }
+
+    int index = transactionList.indexOf(currentId);
+
+    if (++index < transactionList.size()) {
+      return transactionList.get(index).clone();
+    }
+    return TransactionLogManager.UNDEFINED_EDIT_ID;
+  }
+
+  @Override
+  public synchronized Pair<EditId, MemoryBuffer> getNextEditIdTransaction(
+      final String  sessionKey,
+      final long  currentIndex,
+      final Arena arena
+  ) throws IOException {
+    EditId nextId = getNextEditId(getEditId(currentIndex));
+    if (nextId == null || nextId.equals(TransactionLogManager.UNDEFINED_EDIT_ID)) {
+      return null;
+    } else {
+      MemoryBuffer buffer = getTransaction(sessionKey, nextId, arena);
+      return new Pair<>(nextId, buffer);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return dumpLogs(Integer.MAX_VALUE);
+  }
+
+  @Override
+  public synchronized String dumpLogs(int n) {
+    if (n <= 0) {
+      n = transactionList.size();
+    }
+
+    return RaftUtil.listToString(transactionList
+      .subList(Math.max(transactionList.size() - n, 0),
+        transactionList.size() - 1));
+  }
+
+  @Override
+  public synchronized LogState getLogState() {
+    LogState logState = new LogState(null);
+    logState.addUncommittedLogFile(new LogFileInfo("Dummy", 0,
+        transactionList.get(0), transactionList.get(transactionList.size() - 1), 0, 0));
+
+    return logState;
+  }
+
+  @Override
+  public List<LogFileInfo> getCommittedLogStatus(long minIndex) {
+    throw new NotImplementedException("InMemoryLogManager does not track committed log");
+  }
+
+  @Override
+  public String getPath() {
+    return "@RAM";
+  }
+
+  @Override public EditId getEditId(long index) {
+    for (EditId edit : transactionList) {
+      if (edit.getIndex() == index) {
+        return edit;
+      }
+    }
+    return UNDEFINED_EDIT_ID;
+  }
+
+  @Override
+  public Pair<EditId, EditId> greedyIncorporateCandidateLogs(
+      String    sessionKey,
+      long      lastLogIndex
+  ) {
+    return null;
+  }
+
+  @Override
+  public void fillLogGap(long seedIndex) throws IOException {
+
+  }
+
+  @Override
+  public void stop() {
+    // do nothing
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFetchPlanCreator.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFetchPlanCreator.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFetchPlanCreator.java
new file mode 100644
index 0000000..46c52f9
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFetchPlanCreator.java
@@ -0,0 +1,13 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.consensus.client.FetchTask;
+
+import java.util.Collection;
+
+/**
+ * Interface for creating a list of FetchTasks. Implementation classes
+ * contain algorithms to make decision based on given information.
+ */
+public interface LogFetchPlanCreator {
+  Collection<FetchTask> createFetchTasks();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInfo.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInfo.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInfo.java
new file mode 100644
index 0000000..b64be68
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInfo.java
@@ -0,0 +1,138 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.consensus.log;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+
+import java.io.File;
+
+@ThriftStruct
+public final class LogFileInfo implements Comparable<LogFileInfo> {
+  private final EditId firstEditId;
+  private final EditId lastEditId;
+  private final long nbytes;
+  private final String path;
+  private byte[] md5sum = null;
+  private long lastVerifiedModificationTime = 0L;
+  private final long creationTime;
+
+  @ThriftConstructor
+  public LogFileInfo(
+      @ThriftField(1) String path,
+      @ThriftField(2) long nbytes,
+      @ThriftField(3) EditId start,
+      @ThriftField(4) EditId last,
+      @ThriftField(5) long lastVerifiedModificationTime,
+      @ThriftField(6) long creationTime) {
+    this.path = path;
+    this.nbytes = nbytes;
+    this.firstEditId = start;
+    this.lastEditId = last;
+    this.lastVerifiedModificationTime = lastVerifiedModificationTime;
+    this.creationTime = creationTime;
+  }
+
+  @ThriftField(1)
+  public String getAbsolutePath() {
+    return path;
+  }
+
+  @ThriftField(2)
+  public long length() {
+    return nbytes;
+  }
+
+  @ThriftField(3)
+  public EditId getFirstEditId() {
+    return firstEditId;
+  }
+
+  @ThriftField(4)
+  public EditId getLastEditId() {
+    return lastEditId;
+  }
+
+  @ThriftField(5)
+  public long getLastVerifiedModificationTime() {
+    return lastVerifiedModificationTime;
+  }
+
+  @ThriftField(6)
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  public void setMD5(byte[] md5) {
+    this.md5sum = md5;
+  }
+
+  public byte[] getMD5() {
+    return md5sum;
+  }
+
+  public void setLastVerifiedModificationTime(long modtime) {
+    this.lastVerifiedModificationTime = modtime;
+  }
+
+  public long getTxnCount() {
+    return getLastIndex() + 1L - getInitialIndex();
+  }
+
+  public String getFilename() {
+    return (new File(getAbsolutePath())).getName();
+  }
+
+  public long getInitialIndex() {
+    return getFirstEditId().getIndex();
+  }
+
+  public long getLastIndex() {
+    return getLastEditId().getIndex();
+  }
+
+  public boolean validate() {
+    return getFirstEditId() != null && getLastEditId() != null
+      && getFirstEditId().getTerm() <= getLastEditId().getTerm()
+      && getInitialIndex() <= getLastIndex()
+      && getAbsolutePath() != null
+      && getFilename() != null
+      && length() > 0;
+  }
+
+  public String toString() {
+    return "LogFileInfo[" + getFirstEditId() + " .. " + getLastEditId()
+      + ", " + length() + " bytes, " + getAbsolutePath() + ", "
+      + ", @ " + getLastVerifiedModificationTime() + ", createdAt "
+      + getCreationTime() + "]";
+  }
+
+  @Override
+  public int compareTo(LogFileInfo that) {
+    int ret = Long.valueOf(getInitialIndex()).compareTo(that.getInitialIndex());
+    if (ret != 0) {
+      return ret;
+    } else {
+      return Long.valueOf(getLastIndex()).compareTo(that.getLastIndex());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInfoIterator.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInfoIterator.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInfoIterator.java
new file mode 100644
index 0000000..744395e
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInfoIterator.java
@@ -0,0 +1,109 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.consensus.client.FetchTask;
+import org.apache.hadoop.hbase.util.Pair;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Iterate through a list of peers' committed logs and create fetch plan
+ * in round robin fashion. It guarantees coverage of maximum index range on all
+ * peers, but log file selection is not optimal. Seeking less overlaps or local
+ * peer preference etc. are not taken into account.
+ */
+public class LogFileInfoIterator implements LogFetchPlanCreator {
+
+  private Map<String, FetchTask> tasks;
+  private long minIndex;
+
+  /**
+   * Hold lists of log files for each peer. In the pair, string is the peer's address.
+   * Iterator is used to keep position on each list.
+   */
+  private LinkedList<Pair<String, Iterator<LogFileInfo>>> iterators = new LinkedList<>();
+
+  public LogFileInfoIterator(List<Pair<String, List<LogFileInfo>>> statuses, long minIndex) {
+    this.minIndex = minIndex;
+
+    tasks = new HashMap<>(statuses.size());
+    for (Pair<String, List<LogFileInfo>> status : statuses) {
+      Collections.sort(status.getSecond());
+      String peerAddress = status.getFirst();
+
+      // Create a FetchTask for each peer. Don't create too many download requests.
+      FetchTask task = new FetchTask(peerAddress);
+      tasks.put(peerAddress, task);
+
+      iterators.add(new Pair<>(peerAddress, status.getSecond().iterator()));
+    }
+  }
+
+  @Override
+  public Collection<FetchTask> createFetchTasks() {
+    Pair<String, LogFileInfo> pair;
+
+    // In each loop, take out a log file that contains the target index
+    // and add it to the corresponding fetch task. Then we increase current
+    // target index to < largest log index + 1 > as the next
+    // target index.
+    while ((pair = next(minIndex)) != null) {
+      LogFileInfo info = pair.getSecond();
+      minIndex = info.getLastIndex() + 1;
+      tasks.get(pair.getFirst()).addTask(info);
+    }
+
+    return tasks.values();
+  }
+
+  /**
+   * Iterate through sorted log file lists. Try to take a file from a
+   * peer at the same time and do round robin within peers. The file's
+   * index range has to cover targetIndex.
+   *
+   * Basic logic:
+   * 1. Take an iterator of log file list of one peer
+   * 2. Try to find a log file that covers the target index
+   * 3. If the peer does not have any file that covers target index, discard its iterator
+   *    If a log file qualifies, return it with the peer's address
+   * 4. If there is possibility to find another targetIndex, put the iterator
+   *    to the end of the list. So peers are queried in a rotation.
+   *
+   * @param targetIndex the index that has to be in the result file
+   * @return log file that contained target index and the peer it belongs to.
+   *         null if no matching file is found.
+   */
+  private Pair<String, LogFileInfo> next(long targetIndex) {
+    while (true) {
+      // Select the peer at list head
+      Pair<String, Iterator<LogFileInfo>> pair = iterators.pollFirst();
+      if (pair == null) {
+        return null;
+      }
+      Iterator<LogFileInfo> it = pair.getSecond();
+      while (it.hasNext()) {
+        LogFileInfo info = it.next();
+        if (targetIndex < info.getInitialIndex()) {
+          // Current peer has missing logs? Discard it!
+          break;
+        }
+        if (targetIndex > info.getLastIndex()) {
+          // Target index is larger than the file's last index, so will try
+          // next file on the same iterator.
+          continue;
+        }
+        // Current log file contains the target index. Insert the iterator back
+        // to list end and return current file.
+        if (it.hasNext()) {
+          iterators.addLast(pair);
+        }
+        return new Pair<>(pair.getFirst(), info);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInterface.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInterface.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInterface.java
new file mode 100644
index 0000000..2499ec1
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileInterface.java
@@ -0,0 +1,25 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+public interface LogFileInterface {
+  public long getInitialIndex();
+  public long getLastIndex();
+  public long getCurrentTerm();
+  public MemoryBuffer getTransaction(
+    long term, long index, String sessionKey, final Arena arena)
+    throws IOException, NoSuchElementException;
+  public long getLastModificationTime();
+  public void closeAndDelete() throws IOException;
+  public long getTxnCount();
+  public String getFileName();
+  public File getFile();
+  public long getFileSize();
+  public String getFileAbsolutePath();
+  public long getCreationTime();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileViewer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileViewer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileViewer.java
new file mode 100644
index 0000000..b89351b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogFileViewer.java
@@ -0,0 +1,108 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.commons.cli.*;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+public class LogFileViewer {
+  private static final Logger
+    LOG = LoggerFactory.getLogger(LogFileViewer.class);
+
+  /**
+   * @param args
+   * @throws ParseException
+   */
+  public static void main(String[] args) throws ParseException, IOException {
+
+    Options options = new Options();
+
+    options.addOption("f", "filepath", true,
+      "location of the file.");
+    options.addOption("d", "detail", true,
+      "Dump a detailed information about the transactions in the file");
+
+    if (args.length == 0) {
+      printHelp(options);
+      return;
+    }
+
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd = parser.parse(options, args);
+
+    boolean detailed = false;
+    String filePath = null;
+
+    if (!cmd.hasOption("f")) {
+      printHelp(options);
+      return;
+    }
+
+    filePath = cmd.getOptionValue("f");
+
+    if (cmd.hasOption("d")) {
+      detailed = true;
+    }
+
+    final File logFile = new File(filePath);
+
+    if (!logFile.exists()) {
+      LOG.error("The specified file " + filePath + " does not exists.");
+      return ;
+    }
+
+    dumpFileInfo(logFile, detailed);
+  }
+
+  public static void printHelp(final Options options) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("LogFileViewer", options, true);
+  }
+
+  public static void dumpFileInfo(final File logFile, boolean detailed)
+    throws IOException {
+    long index;
+    LogReader reader = new LogReader(logFile);
+    reader.initialize();
+
+    index = reader.getInitialIndex();
+
+    LOG.info("File Name: " + reader.getFile().getName());
+    LOG.info("Term: " + reader.getCurrentTerm());
+    LOG.info("Initial Index: " + index);
+
+    MemoryBuffer buffer;
+
+    try {
+      while (true) {
+        buffer = reader.seekAndRead(index, null);
+        if (detailed) {
+          LOG.info("(index:" + index + ", offset:" +
+            reader.getCurrentIndexFileOffset() + ", ");
+          try {
+            List<WALEdit> edits = WALEdit.deserializeFromByteBuffer(
+              buffer.getBuffer());
+            LOG.info("Size: " + buffer.getBuffer().limit() +
+              ", Number of edits : " + edits.size());
+          } catch (Exception e) {
+            LOG.info("(" + index + ":" + buffer.getBuffer().position() +
+              ")");
+          }
+          LOG.info("),");
+        }
+        buffer.getBuffer().clear();
+        ++index;
+      }
+    } catch (IOException e) {
+      LOG.error("Last Readable Index: " + (--index), e);
+    }
+
+    reader.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogReader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogReader.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogReader.java
new file mode 100644
index 0000000..659b466
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogReader.java
@@ -0,0 +1,392 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.apache.hadoop.util.PureJavaCrc32;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+
+/**
+ * LogReader provides interfaces to perform read operations against a log file.
+ * This class is not thread safe.
+ *
+ * If multiple threads need to read from the same file, each thread should have
+ * its own LogReader object.
+ *
+ * Each LogReader shall be initialized first before reading. It maintains the
+ * reader position internally. The client is able to seek to a particular
+ * offset or index, and then start to read the transaction since there.
+ *
+ * Here is the log file format:
+ *
+ * ------------------
+ * | File Header:   |
+ * |  Version    4B |
+ * |  Term       8B |
+ * |  Index      8B |
+ * ------------------
+ * | Entry # N      |
+ * |  Entry Header  |
+ * |   Size     4B  |
+ * |   CRC      8B  |
+ * |   Index    8B  |
+ * |  Entry PayLoad |
+ * ------------------
+ * | Entry # N      |
+ * |  Entry Header  |
+ * |   Size     4B  |
+ * |   CRC      8B  |
+ * |   Index    8B  |
+ * |  Entry PayLoad |
+ * ----------------
+ *     .....
+ * ------------------
+ * | Entry # N      |
+ * |  Entry Header  |
+ * |   Size     4B  |
+ * |   CRC      8B  |
+ * |   Index    8B  |
+ * |  Entry PayLoad |
+ * ------------------
+ */
+@NotThreadSafe
+public class LogReader {
+
+  public static final Log LOG = LogFactory.getLog(LogReader.class);
+  private final File file;
+
+  // File header meta data
+  private int version = HConstants.UNDEFINED_VERSION;
+  private long currentTerm = HConstants.UNDEFINED_TERM_INDEX;
+  private long initialIndex = HConstants.UNDEFINED_TERM_INDEX;
+
+  // The invariant for the current position of the file
+  private int curPayLoadSize = HConstants.UNDEFINED_PAYLOAD_SIZE;
+  private long curIndex = HConstants.UNDEFINED_TERM_INDEX;
+  private long curPayLoadOffset = HConstants.UNDEFINED_PAYLOAD_SIZE;
+  private long curCRC = 0;
+
+  //TODO: check if we can share the headBuffer across multiple readers
+  private ByteBuffer headBuffer =
+    ByteBuffer.allocateDirect(HConstants.RAFT_TXN_HEADER_SIZE);
+
+  /** The CRC instance to compute CRC-32 of a log entry payload */
+  private PureJavaCrc32 crc32 = new PureJavaCrc32();
+  CachedFileChannel data;
+
+  public LogReader (File file) throws IOException {
+    this.file = file;
+    data = new CachedFileChannel(new RandomAccessFile(this.file, "r"),
+        TransactionLogManager.getLogPrefetchSize());
+  }
+
+  public void initialize()throws IOException {
+    readFileHeader();
+  }
+
+  public File getFile() {
+    return file;
+  }
+
+  public RandomAccessFile getRandomAccessFile() {
+    return data.getRandomAccessFile();
+  }
+
+  public void initialize(int version, long currentTerm, long initialIndex) {
+    this.version = version;
+    this.currentTerm = currentTerm;
+    this.initialIndex = initialIndex;
+  }
+
+  /**
+   * Seek to a particular index and read the transactions from the entry payload
+   * @param index
+   * @param arena Arena
+   * @return transactions of this index
+   * @throws IOException if there is no such index in the log
+   */
+  public MemoryBuffer seekAndRead(long index, final Arena arena) throws IOException {
+    long onDiskIndex = HConstants.UNDEFINED_TERM_INDEX;
+    if (this.curIndex > index) {
+      resetPosition();
+    }
+
+    // Read the transaction header and try to match the index
+    while (hasMore()) {
+      if (curIndex == index) {
+        break;
+      }
+
+      // Read the transaction header
+      readEntryHeader();
+
+      // Compare the index
+      if (curIndex != index) {
+        // Skip the payload for the non-match
+        skipEntryPayload();
+      }
+    }
+
+    // Throw the exception if there is no matched index.
+    if (curIndex != index) {
+      resetPosition();
+      throw new IOException("There is no expected index: " + index + " in " +
+        "this log. On disk index " + onDiskIndex + " " + this.getInitialIndex() +
+      this.getCurrentTerm() + " current Index" + curIndex);
+    }
+
+    // Return the transactions
+    return readEntryPayLoad(arena);
+  }
+
+  /**
+   * Seek to a particular index, and sanity check the index,
+   * and then read the transactions from the entry payload
+   * @param offset
+   * @param expectedIndex
+   * @return transactions of this index
+   * @throws IOException if the on disk index does not match with the expected index
+   */
+  public MemoryBuffer seekAndRead(long offset, long expectedIndex,
+                                  final Arena arena) throws IOException {
+    // Set the file position
+    data.position(offset);
+
+    // Read the transaction header and verify it
+    long onDiskIndex = this.readEntryHeader();
+    if (onDiskIndex != expectedIndex) {
+      // Reset the position
+      resetPosition();
+
+      // Throw out the exceptions
+      throw new IOException("The expected index " + expectedIndex + " does not match with "
+      + " on disk index " + onDiskIndex + " at the offset " + offset);
+    }
+
+    // Return the payload
+    return readEntryPayLoad(arena);
+  }
+
+  /**
+   * Whether there are more transactions in the file
+   * @return true if there are more transactions in the file
+   * @throws IOException
+   */
+  public boolean hasMore() throws IOException {
+    if (data.size() > data.position()) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Returns the next index from the current position.
+   *
+   * This assumes that the current position is at the start of the payload
+   * of the current index.
+   *
+   * @return next index
+   * @throws IOException
+   */
+  long next() throws IOException {
+    long onDiskIndex = HConstants.UNDEFINED_TERM_INDEX;
+    long lastIndex = curIndex;
+
+    // Skip to the next entry
+    if (curIndex != HConstants.UNDEFINED_TERM_INDEX) {
+      try {
+        skipEntryPayload();
+      } catch (IOException e) {
+        LOG.error("Unable to move to the next entry in the file. The data might" +
+          "be corrupted. Current Index :" + curIndex + ", Current Payload " +
+          "offset :" + curPayLoadOffset, e);
+
+        // We were unable to read the current entry, which means that it might
+        // be corrupted. Reset the position to the start of the entry.
+        data.position(
+          curPayLoadOffset - HConstants.RAFT_TXN_HEADER_SIZE);
+        return HConstants.UNDEFINED_TERM_INDEX;
+      }
+    }
+
+    // Are there more entries ??
+    if (hasMore()) {
+      // Read the transaction header
+      onDiskIndex = this.readEntryHeader();
+    }
+
+    // Verify the next entry. All the entries must be continuous in the log
+    if ((lastIndex != HConstants.UNDEFINED_TERM_INDEX) &&
+      (lastIndex + 1 != onDiskIndex)) {
+        LOG.warn("Next index: " + onDiskIndex + " is not continuous. Previous " +
+          "index: " + lastIndex + ", term " + currentTerm
+          + ". File: " + (getFile() != null ? getFile().getAbsolutePath() : "?")
+          );
+
+      // If we had more data but not contiguous, reset the position to the
+      // start of the unknown index
+      if (onDiskIndex != HConstants.UNDEFINED_TERM_INDEX) {
+        data.position(getCurrentIndexFileOffset());
+      }
+      return HConstants.UNDEFINED_TERM_INDEX;
+    }
+
+    // Return the transactions
+    return onDiskIndex;
+  }
+
+  /**
+   * Get the start offset of the current index in the file.
+   *
+   * This assumes that the current position is at the start of the payload of
+   * the current index.
+   *
+   * @return offset of the current index
+   * @throws IOException
+   */
+  long getCurrentIndexFileOffset() throws IOException {
+    return data.position() - HConstants.RAFT_FILE_HEADER_SIZE;
+  }
+
+  /**
+   * Close the reader
+   * @throws IOException
+   */
+  public void close() throws IOException {
+    this.data.close();
+  }
+
+  /**
+   * @return the version of the log file
+   */
+  public int getVersion() {
+    return version;
+  }
+
+  /**
+   * @return the term of the log file
+   */
+  public long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  /**
+   * @return the initial index of the log file
+   */
+  public long getInitialIndex() {
+    return initialIndex;
+  }
+
+  public long getCurrentPosition() throws IOException {
+    return data.position();
+  }
+
+  public void resetPosition() throws IOException {
+    this.data.position(HConstants.RAFT_FILE_HEADER_SIZE);
+    this.curIndex = HConstants.UNDEFINED_TERM_INDEX;
+    this.curPayLoadSize = HConstants.UNDEFINED_PAYLOAD_SIZE;
+  }
+
+  private void skipEntryPayload() throws IOException {
+    // Get the current position and the target position
+    long targetPosition = this.data.position() + this.curPayLoadSize;
+
+    // Set the target position for the channel
+    data.position(targetPosition) ;
+  }
+
+  /**
+   *
+   *  readEntryPayLoad
+   *
+   *  @param  arena  Arena to be used for memory allocation
+   */
+  private MemoryBuffer readEntryPayLoad(final Arena arena) throws IOException {
+
+    MemoryBuffer buffer = TransactionLogManager.allocateBuffer(arena,
+        curPayLoadSize);
+
+    if (curPayLoadOffset != data.position()) {
+      data.position(curPayLoadOffset);
+    }
+
+    buffer.getBuffer().limit(buffer.getBuffer().position() + curPayLoadSize);
+    data.read(buffer.getBuffer());
+    buffer.flip();
+
+    if (!verifyCRC(buffer.getBuffer())) {
+      long faultyIndex = this.curIndex;
+      resetPosition();
+      throw new IOException("The CRC for the entry payload of index " +
+        faultyIndex + " does not match with the expected CRC read from the " +
+        "entry header.");
+    }
+    return buffer;
+  }
+
+  private long readEntryHeader() throws IOException {
+
+    // Read the header of the transaction
+
+    headBuffer.clear();
+    data.read(headBuffer);
+    headBuffer.flip();
+
+    // Read the payload size
+    this.curPayLoadSize = headBuffer.getInt();
+
+    // Read the CRC
+    this.curCRC = headBuffer.getLong();
+
+    // Read the current index
+    this.curIndex = headBuffer.getLong();
+
+    // Cache the offset
+    this.curPayLoadOffset = data.position();
+
+    // Sanity check the payload size
+    assert this.curPayLoadSize <= this.data.size() - this.data.position();
+
+    // Return the OnDisk Index
+    return curIndex;
+  }
+
+  private void readFileHeader() throws IOException {
+    // Set the position to 0
+    this.data.position(0);
+
+    // Read the FileHeader to buffer
+    headBuffer.clear();
+    this.data.read(headBuffer);
+    headBuffer.flip();
+
+    if (headBuffer.remaining() != HConstants.RAFT_FILE_HEADER_SIZE) {
+      throw new IOException("Cannot read the header of the file " + file);
+    }
+    this.version = headBuffer.getInt();
+    this.currentTerm = headBuffer.getLong();
+    this.initialIndex = headBuffer.getLong();
+  }
+
+  /**
+   * Verify the CRC for the payload buffer and return the result
+   * @param payloadBuffer
+   * @return true if the CRC of the payload buffer is equal to the curCRC from the header
+   */
+  private boolean verifyCRC(ByteBuffer payloadBuffer) {
+    crc32.reset();
+    crc32.update(payloadBuffer.array(),
+      payloadBuffer.position() + payloadBuffer.arrayOffset(),
+      payloadBuffer.remaining());
+
+    return crc32.getValue() == this.curCRC;
+  }
+}


[07/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/LocalConsensusServer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/LocalConsensusServer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/LocalConsensusServer.java
new file mode 100644
index 0000000..3de7ab8
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/LocalConsensusServer.java
@@ -0,0 +1,354 @@
+package org.apache.hadoop.hbase.consensus.server;
+
+import io.airlift.units.Duration;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.AggregateTimer;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+import org.apache.hadoop.hbase.util.InfoServer;
+import org.apache.hadoop.hbase.util.serial.AsyncSerialExecutorServiceImpl;
+import org.apache.hadoop.hbase.util.serial.SerialExecutorService;
+import org.apache.log4j.Level;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.jboss.netty.logging.InternalLoggerFactory;
+import org.jboss.netty.logging.Slf4JLoggerFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.weakref.jmx.MBeanExporter;
+
+import com.facebook.nifty.codec.DefaultThriftFrameCodecFactory;
+import com.facebook.nifty.codec.ThriftFrameCodecFactory;
+import com.facebook.nifty.core.NiftyTimer;
+import com.facebook.nifty.duplex.TDuplexProtocolFactory;
+import com.facebook.swift.codec.ThriftCodecManager;
+import com.facebook.swift.service.ThriftEventHandler;
+import com.facebook.swift.service.ThriftServer;
+import com.facebook.swift.service.ThriftServerConfig;
+import com.facebook.swift.service.ThriftServiceProcessor;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+
+public class LocalConsensusServer {
+  public static final ImmutableMap<String,TDuplexProtocolFactory>
+    SWIFT_PROTOCOL_FACTORIES = ImmutableMap.of(
+    "binary", TDuplexProtocolFactory.fromSingleFactory(
+      new TBinaryProtocol.Factory()),
+    "compact", TDuplexProtocolFactory.fromSingleFactory(
+      new TCompactProtocol.Factory())
+  );
+
+  public static final ImmutableMap<String,ThriftFrameCodecFactory>
+    SWIFT_FRAME_CODEC_FACTORIES = ImmutableMap.<String, ThriftFrameCodecFactory>of(
+    "buffered", new DefaultThriftFrameCodecFactory(),
+    "framed", new DefaultThriftFrameCodecFactory()
+  );
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+    LocalConsensusServer.class);
+  private final Configuration conf;
+  private ThriftServer server;
+  private final ConsensusService handler;
+  private final List<ThriftEventHandler> eventHandlers;
+
+  private InfoServer infoServer;
+
+  private ThriftServerConfig config;
+
+  private ExecutorService execServiceForThriftClients;
+
+  public final AggregateTimer aggregateTimer = new AggregateTimer();
+  public SerialExecutorService serialExecutorService;
+
+  @Inject
+  public LocalConsensusServer(final ConsensusService handler,
+                              final List<ThriftEventHandler> eventHandlers,
+                              final Configuration conf) {
+    this.eventHandlers = eventHandlers;
+    this.handler = handler;
+    this.conf = conf;
+
+    this.execServiceForThriftClients =
+        Executors.newFixedThreadPool(
+          HConstants.DEFAULT_QUORUM_CLIENT_NUM_WORKERS,
+          new DaemonThreadFactory("QuorumClient-"));
+  }
+
+  public void initialize(final ThriftServerConfig config) {
+    if (!(InternalLoggerFactory.getDefaultFactory()
+            instanceof Slf4JLoggerFactory)) {
+      InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory());
+    }
+
+    // Set the transport protocol
+    config.setProtocolName(conf.get(HConstants.CONSENSUS_TRANSPORT_PROTOCOL_KEY,
+      HConstants.CONSENSUS_TRANSPORT_PROTOCOL_DEFAULT));
+
+    // Set the codec factory
+    config.setTransportName(conf.get(HConstants.CONSENSUS_TRANSPORT_CODEC_KEY,
+      HConstants.CONSENSUS_TRANSPORT_CODEC_DEFAULT));
+
+    // "ConsensusAsyncSerialExecutor"
+    int numThreads = conf.getInt(HConstants.FSM_MUX_THREADPOOL_SIZE_KEY,
+      HConstants.DEFAULT_FSM_MUX_THREADPOOL_SIZE);
+    serialExecutorService =
+        new AsyncSerialExecutorServiceImpl(numThreads, "AsyncSerialExecutor");
+
+    ThriftServer newServer = new ThriftServer(
+      new ThriftServiceProcessor(
+        new ThriftCodecManager(), eventHandlers, handler),
+      config,
+      new NiftyTimer("thrift"),
+      SWIFT_FRAME_CODEC_FACTORIES,
+      SWIFT_PROTOCOL_FACTORIES,
+      ThriftServer.DEFAULT_WORKER_EXECUTORS,
+      ThriftServer.DEFAULT_SECURITY_FACTORY);
+
+    LOG.info("Start the thrift server with a unbounded queue !");
+
+    synchronized (this) {
+      server = newServer;
+      this.config = config;
+    }
+  }
+
+  public void startService() {
+    LOG.info("Starting the consensus service at port : " + config.getPort());
+    server.start();
+    LOG.info("Started the consensus service at port : " + config.getPort());
+  }
+
+  public void restartService() {
+    LOG.info("Restarting the consensus service at port : " + config.getPort());
+    getThriftServer().close();
+    initialize(this.config);
+    getThriftServer().start();
+    LOG.info("Restarted the consensus service at port: " + config.getPort());
+  }
+
+  public void stopService() {
+    LOG.info("Stopping the consensus service at port : " + config.getPort());
+    LOG.info(
+      com.google.common.base.Throwables.getStackTraceAsString(new Throwable()));
+    handler.stopService();
+    server.close();
+    execServiceForThriftClients.shutdownNow();
+  }
+
+  public ConsensusService getHandler() {
+    return handler;
+  }
+
+  public synchronized ThriftServer getThriftServer() {
+    return server;
+  }
+
+  public void startInfoServer(HServerAddress local, Configuration conf) throws IOException {
+    this.infoServer = new InfoServer(HConstants.QUORUM_MONITORING_PAGE_KEY,
+      local.getBindAddress(), server.getPort() + 1, false,
+                                     conf);
+    this.infoServer.setAttribute(HConstants.QUORUM_MONITORING_PAGE_KEY, this);
+    this.infoServer.start();
+  }
+
+  public void shutdownInfoServer() throws Exception {
+    this.infoServer.stop();
+  }
+
+  public QuorumClient getQuorumClient(QuorumInfo quorumInfo) throws IOException {
+    return new QuorumClient(quorumInfo, this.conf, execServiceForThriftClients);
+  }
+
+  @Override
+  public String toString() {
+    return "LocalConsensusServer@" + config.getPort()
+      + "[" + (server.isRunning() ? "RUNNING" : "OFFLINE") + "@" + server.getPort() + "]";
+  }
+
+  public static void main(String[] args) {
+    LOG.info("Starting Consensus service server");
+    final Configuration configuration = HBaseConfiguration.create();
+
+    Options opt = new Options();
+    opt.addOption("region", true, "the region id for the load test");
+    opt.addOption("servers", true, "A list of quorum server delimited by ,");
+    opt.addOption("localIndex", true, "The index of the local quorum server from the server list");
+    opt.addOption("debug", false, "Set the log4j as debug level." );
+    opt.addOption("jetty", false, "Enable the jetty server" );
+
+    String regionId = null;
+    String serverList;
+    String servers[] = null;
+    HServerAddress localHost = null;
+    int localIndex = -1;
+    HashMap<HServerAddress, Integer> peers = new HashMap<>();
+    int rank = 5;
+    boolean enableJetty = false;
+
+    try {
+      CommandLine cmd = new GnuParser().parse(opt, args);
+
+      if (cmd.hasOption("region")) {
+        regionId = cmd.getOptionValue("region");
+      }
+
+      if (cmd.hasOption("servers")) {
+        serverList = cmd.getOptionValue("servers");
+        servers = serverList.split(",");
+      }
+
+      if (cmd.hasOption("localIndex")) {
+        localIndex = Integer.parseInt(cmd.getOptionValue("localIndex"));
+      }
+
+      if (!cmd.hasOption("debug")) {
+        org.apache.log4j.Logger.getLogger(
+                "org.apache.hadoop.hbase.consensus").setLevel(Level.INFO);
+      }
+
+
+      if (cmd.hasOption("jetty")) {
+        enableJetty = true;
+      }
+
+      if (regionId == null || regionId.isEmpty() || servers == null || servers.length == 0) {
+        LOG.error("Wrong args !");
+        printHelp(opt);
+        System.exit(-1);
+      }
+
+      System.out.println("*******************");
+      System.out.println("region: " + regionId);
+      String hostname = null;
+
+      try {
+        hostname = InetAddress.getLocalHost().getHostName();
+        System.out.println("Current host name is " + hostname);
+      } catch (UnknownHostException e) {
+        System.out.println("Not able to retrieve the local host name");
+        System.exit(-1);
+      }
+
+      for (int i = 0; i < servers.length; i++) {
+        String localConsensusServerAddress = servers[i];
+
+        HServerAddress cur = new HServerAddress(localConsensusServerAddress);
+        peers.put(RaftUtil.getHRegionServerAddress(cur), rank--);
+        System.out.println("Quorum server " + localConsensusServerAddress);
+
+        if (localIndex != -1) { // local mode
+          if (localIndex == i) {
+            localHost = cur;
+          }
+        } else if (localConsensusServerAddress.contains(hostname)) { // remote mode
+          localHost = cur;
+        }
+      }
+      if (localHost == null) {
+        System.out.println("Error: no local server");
+        printHelp(opt);
+        System.exit(-1);
+      }
+      System.out.println("Local Quorum server " + localHost);
+      System.out.println("*******************");
+    } catch (Exception e) {
+      e.printStackTrace();
+      printHelp(opt);
+      System.exit(-1);
+    }
+
+    if (localIndex != -1) {
+      // overwrite the log directory for the local mode
+      configuration.set("hbase.consensus.log.path",  "/tmp/wal/" + localHost.getHostAddressWithPort());
+    }
+
+    // Start the local consensus server
+    ThriftServerConfig config =  new ThriftServerConfig().
+      setWorkerThreads(configuration.getInt(
+        HConstants.CONSENSUS_SERVER_WORKER_THREAD,
+        HConstants.DEFAULT_CONSENSUS_SERVER_WORKER_THREAD)).
+      setIoThreadCount(configuration.getInt(
+        HConstants.CONSENSUS_SERVER_IO_THREAD,
+        HConstants.DEFAULT_CONSENSUS_SERVER_IO_THREAD)).
+      setPort(localHost.getPort()).
+      setIdleConnectionTimeout(new Duration(1, TimeUnit.DAYS));
+
+    // Set the max progress timeout as 30 sec;
+    // The actual progress timeout will be the max progress timeout/rank
+    // And the heartbeat timeout will be the the max timeout/20;
+    configuration.setInt(HConstants.PROGRESS_TIMEOUT_INTERVAL_KEY, 30 * 1000);
+
+    MBeanExporter mbeanExporter = MBeanExporter.withPlatformMBeanServer();
+
+    LocalConsensusServer consensusServer = new LocalConsensusServer(
+      ConsensusServiceImpl.createConsensusServiceImpl(),
+      new ArrayList<ThriftEventHandler>(),
+      configuration);
+    consensusServer.initialize(config);
+    consensusServer.startService();
+    LOG.info("Started Consensus Service with port:" + config.getPort());
+
+    if (enableJetty) {
+      try {
+        consensusServer.startInfoServer(localHost, configuration);
+        System.out.println("Start the jetty server at " +
+          localHost.getBindAddress() + ":" + localHost.getPort() + 1);
+      } catch (Exception e) {
+        LOG.error("Unable to start the jetty server ", e);
+      }
+    }
+
+    // Set the region with the peers
+    HRegionInfo regionInfo = RaftUtil.createDummyRegionInfo(regionId, peers);
+
+    // Create the RaftQuorumContext
+    RaftQuorumContext context = new RaftQuorumContext(regionInfo.getQuorumInfo(),
+      configuration, localHost,
+      (regionInfo.getTableDesc().getNameAsString() + "."),
+      consensusServer.aggregateTimer,
+      consensusServer.serialExecutorService,
+      consensusServer.execServiceForThriftClients
+    );
+    context.getConsensusMetrics().export(mbeanExporter);
+
+    // Initialize the raft context
+    context.initializeAll(HConstants.UNDEFINED_TERM_INDEX);
+
+    // Register the raft context into the consensus server
+    consensusServer.getHandler().addRaftQuorumContext(context);
+    LOG.info("Registered the region " + regionId + " with the consensus server");
+  }
+
+  private static void printHelp(Options opt) {
+    new HelpFormatter().printHelp(
+      "QuorumLoadTestClient -region regionID -servers h1:port,h2:port," +
+        "h3:port...] [-localIndex index]", opt);
+  }
+
+  public ExecutorService getExecServiceForThriftClients() {
+    return execServiceForThriftClients;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/AbstractPeer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/AbstractPeer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/AbstractPeer.java
new file mode 100644
index 0000000..23bc9d6
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/AbstractPeer.java
@@ -0,0 +1,447 @@
+package org.apache.hadoop.hbase.consensus.server.peer;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.fsm.FiniteStateMachine;
+import org.apache.hadoop.hbase.consensus.fsm.FiniteStateMachineService;
+import org.apache.hadoop.hbase.consensus.fsm.FiniteStateMachineServiceImpl;
+import org.apache.hadoop.hbase.consensus.fsm.Util;
+import org.apache.hadoop.hbase.consensus.log.TransactionLogManager;
+import org.apache.hadoop.hbase.consensus.metrics.PeerMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.raft.events.AppendResponseEvent;
+import org.apache.hadoop.hbase.consensus.raft.events.VoteResponseEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteResponse;
+import org.apache.hadoop.hbase.consensus.server.ConsensusService;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerAppendRequestEvent;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerAppendResponseEvent;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerServerEventType;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerVoteRequestEvent;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
+import org.apache.hadoop.hbase.regionserver.RaftEventListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.weakref.jmx.JmxException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+@NotThreadSafe
+public abstract class AbstractPeer implements PeerServer {
+  public int rank;
+
+  protected MutableRaftContext raftContext;
+  protected FiniteStateMachineService stateMachine;
+  protected int connectionRetry = 1;
+  protected int connectionRetryInterval;
+
+  private static Logger LOG = LoggerFactory.getLogger(AbstractPeer.class);
+  private final HServerAddress address;
+  private String sourceString = null;
+  /** Last successfully acked Edit Id by the peer */
+  private EditId lastEditID = TransactionLogManager.UNDEFINED_EDIT_ID;
+  private AppendRequest latestRequest;
+  private volatile boolean isStopped = false;
+  protected RaftEventListener dataStoreEventListener;
+  private final static long DEFAULT_ASYNC_SEND_OUTLIER = 50 * 1000 * 1000; // 50ms
+  private boolean useAggregateTimer;
+  private long timeLastRequestWasSent = 0;
+  private boolean useMuxForPSM;
+
+  private final PeerMetrics metrics;
+
+  public AbstractPeer(final HServerAddress address, final int rank,
+                      final MutableRaftContext replica,
+                      final Configuration conf) {
+    this.rank = rank;
+    this.address = address;
+    this.raftContext = replica;
+    connectionRetry = conf.getInt(
+      HConstants.RAFT_PEERSERVER_CONNECTION_RETRY_CNT, 1);
+    connectionRetryInterval = conf.getInt(
+      HConstants.RAFT_PEERSERVER_CONNECTION_RETRY_INTERVAL, 1000);
+
+    // MBean name properties can not contain colons, so replace with a period.
+    // TODO (arjen): Make sure this works for IPv6.
+    String peerId = address.getHostNameWithPort().replace(':', '.');
+    metrics = replica.getConsensusMetrics().createPeerMetrics(peerId);
+    useAggregateTimer = conf.getBoolean(
+      HConstants.QUORUM_USE_AGGREGATE_TIMER_KEY,
+      HConstants.QUORUM_USE_AGGREGATE_TIMER_DEFAULT);
+    useMuxForPSM = conf.getBoolean(HConstants.USE_FSMMUX_FOR_PSM,
+      HConstants.USE_FSMMUX_FOR_PSM_DEFAULT);
+  }
+
+  public HServerAddress getAddress() {
+    return address;
+  }
+
+  @Override
+  public int getRank() {
+    return rank;
+  }
+
+  @Override
+  public void setRank(int rank) {
+    this.rank = rank;
+  }
+
+  @Override
+  public AppendRequest getLatestRequest() {
+    return latestRequest;
+  }
+
+  @Override
+  public void setLatestRequest(final AppendRequest latestRequest) {
+    this.latestRequest = latestRequest;
+  }
+
+  protected MutableRaftContext getRaftContext() {
+    return raftContext;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return raftContext.getConf();
+  }
+
+  @Override
+  public void registerDataStoreEventListener(RaftEventListener listener) {
+    this.dataStoreEventListener = listener;
+  }
+
+  @Override
+  public void updatePeerAvailabilityStatus(boolean isAvailable) {
+    if (this.dataStoreEventListener != null) {
+      this.dataStoreEventListener.updatePeerAvailabilityStatus(
+        this.address.getHostAddressWithPort(), isAvailable);
+    } else {
+      LOG.warn("dataStoreEventListener has not been registered for this peer: " + toString());
+    }
+  }
+
+  public FiniteStateMachineService getStateMachineService() {
+    return stateMachine;
+  }
+
+  abstract protected ConsensusService getConsensusServiceAgent();
+  abstract protected void resetConnection()
+    throws ExecutionException, InterruptedException, TimeoutException;
+  abstract protected boolean clearConsensusServiceAgent(ConsensusService localAgent);
+
+  public void setStateMachineService(final FiniteStateMachineService fsm) {
+    stateMachine = fsm;
+  }
+
+  public void initializeStateMachine() {
+    if (stateMachine == null) {
+      FiniteStateMachine fsm = new PeerStateMachine(
+        "PeerFSM-" + raftContext.getQuorumName() + ":" +
+          raftContext.getRanking() + "-->" +
+          this.address.getHostAddressWithPort() +  ":" + this.rank,
+        this);
+
+      FiniteStateMachineService fsmService;
+      if (useMuxForPSM) {
+        fsmService = ((RaftQuorumContext) raftContext)
+          .createFiniteStateMachineService(fsm);
+      } else {
+        fsmService = new FiniteStateMachineServiceImpl(fsm);
+      }
+      setStateMachineService(fsmService);
+    }
+    stateMachine.offer(new Event(PeerServerEventType.START));
+  }
+
+  @Override
+  public void initialize() {
+    isStopped = false;
+    lastEditID = raftContext.getLastLogIndex();
+    try {
+      raftContext.getConsensusMetrics().exportPeerMetrics(metrics);
+    } catch (JmxException e) {
+      LOG.warn(String.format("Could not export metrics MBean: %s, reason: %s",
+              metrics, e.getReason()));
+    }
+    initializeStateMachine();
+  }
+
+  @Override
+  public void resetPeerContext() {
+    lastEditID = raftContext.getLastLogIndex();
+    latestRequest = null;
+  }
+
+  @Override
+  public void sendAppendEntries(AppendRequest request) {
+    // Append the event to the state machine
+    stateMachine.offer(new PeerAppendRequestEvent(request));
+  }
+
+  @Override
+  public void sendRequestVote(VoteRequest request) {
+    // Append the event to the state machine
+    stateMachine.offer(new PeerVoteRequestEvent(request));
+  }
+
+  @Override
+  public String getPeerServerName() {
+    return address.getHostAddressWithPort();
+  }
+
+  @Override
+  public String toString() {
+    return "[PeerServer: " + this.address + ", rank:" + this.rank + "]";
+  }
+
+  @Override
+  public MutableRaftContext getQuorumContext() {
+    return raftContext;
+  }
+
+  @Override
+  public EditId getLastEditID() {
+    return lastEditID;
+  }
+
+  @Override
+  public void enqueueEvent(final Event e) {
+    stateMachine.offer(e);
+  }
+
+  @Override
+  public void setLastEditID(final EditId lastEditID) {
+    this.lastEditID = lastEditID;
+  }
+
+  @Override
+  public void stop() {
+    isStopped = true;
+    stateMachine.offer(new Event(PeerServerEventType.HALT));
+    stateMachine.shutdown();
+    if (!Util.awaitTermination(
+        stateMachine, 3, 3, TimeUnit.SECONDS)) {
+      LOG.error("State Machine Service " + stateMachine.getName() +
+              " did not shutdown");
+    }
+    try {
+      raftContext.getConsensusMetrics().unexportPeerMetrics(metrics);
+    } catch (JmxException e) {
+      LOG.warn(String.format(
+                      "Could not un-export metrics MBean: %s, reason: %s",
+                      metrics, e.getReason()));
+    }
+  }
+
+  private String getTargetString() {
+    return this.toString();
+  }
+
+  private String getSourceString() {
+    // we don't have to synchronize it
+    if (sourceString == null) {
+      if (getRaftContext() != null) {
+        sourceString = getRaftContext().getMyAddress();
+        return sourceString;
+      } else {
+        return "?:?";
+      }
+    } else {
+      return sourceString;
+    }
+  }
+
+  @Override
+  public void sendAppendRequestWithCallBack(final AppendRequest request) {
+    if (isStopped) {
+      if (LOG.isTraceEnabled() || request.isTraceable()) {
+        LOG.debug("[AppendRequest Trace] " + getSourceString() + " not sending " + request + " to "
+          + getTargetString() + "; because we are stopped!");
+      }
+      return;
+    }
+
+    final ConsensusService localAgent = getConsensusServiceAgent();
+    if (localAgent == null) {
+      LOG.warn("Failed to get localAgent for " + this + " for the request " + request);
+      triggerRPCErrorEvent();
+      return;
+    }
+
+    assert request.validateFields();
+    assert request.getLeaderId() != null;
+
+    if (LOG.isTraceEnabled() || request.isTraceable()) {
+      LOG.debug("[AppendRequest Trace] " + getSourceString() + ": sending " + request + " to " +
+        getTargetString());
+    }
+    final long start = System.nanoTime();
+
+    long time = System.currentTimeMillis();
+
+    if (time - timeLastRequestWasSent > 1000) {
+      LOG.warn(getSourceString() + "-->" +
+        getTargetString() + " did not send append request in last " +
+        (time - timeLastRequestWasSent) + " ms . Current req " + request);
+    }
+    timeLastRequestWasSent = time;
+
+    RaftUtil.getThriftClientManager().getNiftyChannel(localAgent).executeInIoThread(
+      new Runnable() {
+        @Override public void run() {
+          try {
+
+            ListenableFuture<AppendResponse> futureResponse =
+              localAgent.appendEntries(request);
+            Futures.addCallback(futureResponse, new FutureCallback<AppendResponse>() {
+              @Override
+              public void onSuccess(AppendResponse response) {
+                getQuorumContext().offerEvent(new AppendResponseEvent(response));
+                metrics.getAppendEntriesLatency().add(System.nanoTime() - start,
+                  TimeUnit.NANOSECONDS);
+                if (LOG.isTraceEnabled() || request.isTraceable()) {
+                  LOG.debug("[AppendRequest Trace] Received the response: " + response);
+                }
+                getStateMachineService().offer(new PeerAppendResponseEvent(response));
+              }
+
+              @Override
+              public void onFailure(Throwable e) {
+                handleFailure(localAgent, request, e);
+              }
+            });
+          } catch (Throwable t) {
+            LOG.error("Could not send async request", t);
+            handleFailure(localAgent, request, t);
+          }
+        }
+      });
+
+  }
+
+  private void handleFailure(ConsensusService localAgent, AppendRequest request, Throwable e) {
+    if (LOG.isDebugEnabled()) {
+      if (e instanceof ThriftHBaseException) {
+        e = ((ThriftHBaseException)e).getServerJavaException();
+      }
+
+      LOG.debug(request + " (" + getSourceString() + "-->" +
+          getTargetString() + ") FAILED due to ", e);
+    }
+    if (shouldRetry(e)) {
+      appendRequestFailed(localAgent);
+    }
+  }
+
+  private boolean shouldRetry(Throwable e) {
+    return RaftUtil.isNetworkError(e);
+  }
+
+  protected void appendRequestFailed(ConsensusService localAgent) {
+    metrics.incAppendEntriesFailures();
+
+    // If we were the first one to fail and continue to retry
+    if (clearConsensusServiceAgent(localAgent)) {
+      // Sending no response to process, will automatically let it to recover from
+      // the last acked Edit id.
+      stateMachine.offer(new PeerAppendResponseEvent(null));
+    }
+  }
+
+  @Override
+  public void sendVoteRequestWithCallBack(final VoteRequest request) {
+    if (isStopped) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(getSourceString() + " not sending " + request + " to "
+          + getTargetString() + "; because we are stopped!");
+      }
+      return;
+    }
+
+    final ConsensusService localAgent = getConsensusServiceAgent();
+    if (localAgent == null) {
+      voteRequestFailed(localAgent);
+      LOG.warn("Failed to get localAgent for " + this);
+      this.triggerRPCErrorEvent();
+      return;
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(getSourceString() + ": sending " + request + " to " +
+        getTargetString());
+    }
+
+    RaftUtil.getThriftClientManager().getNiftyChannel(localAgent).executeInIoThread(
+      new Runnable() {
+        @Override public void run() {
+          try {
+            ListenableFuture<VoteResponse> futureResponse =
+              localAgent.requestVote(request);
+            if (futureResponse == null) {
+              LOG.error(getSourceString() + ": got a null response when sending " +
+                request + " to " + getTargetString());
+              return;
+            }
+
+            Futures.addCallback(futureResponse, new FutureCallback<VoteResponse>() {
+              @Override
+              public void onSuccess(VoteResponse response) {
+                // Add the event to the state machine
+                getRaftContext().offerEvent(new VoteResponseEvent(response));
+              }
+
+              @Override
+              public void onFailure(Throwable e) {
+                LOG.error(request + " (" + getSourceString() + "-->" +
+                  getTargetString() + ") FAILED due to ", e);
+                voteRequestFailed(localAgent);
+              }
+            });
+          } catch (Throwable e) {
+            voteRequestFailed(localAgent);
+          }
+
+        }
+      });
+ }
+
+  protected void voteRequestFailed(ConsensusService localAgent) {
+    metrics.incVoteRequestFailures();
+    getRaftContext().offerEvent(new VoteResponseEvent(
+        new VoteResponse(this.getPeerServerName(),
+          HConstants.UNDEFINED_TERM_INDEX, VoteResponse.VoteResult.FAILURE)
+    ));
+    if (localAgent != null) {
+      clearConsensusServiceAgent(localAgent);
+    }
+  }
+
+  protected void triggerRPCErrorEvent() {
+    metrics.incRPCErrorEvents();
+    // Create an RPCError Event and add it to the SM
+    stateMachine.offer(new Event(PeerServerEventType.PEER_RPC_ERROR));
+  }
+
+  @Override
+  public PeerMetrics getMetrics() {
+    return metrics;
+  }
+
+  public void calculateAndSetAppendLag() {
+    getMetrics().setAppendEntriesLag(
+            latestRequest.getLogId(0).getIndex() - getLastEditID().getIndex());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerConsensusServer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerConsensusServer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerConsensusServer.java
new file mode 100644
index 0000000..eb8003f
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerConsensusServer.java
@@ -0,0 +1,152 @@
+package org.apache.hadoop.hbase.consensus.server.peer;
+
+import com.facebook.nifty.client.FramedClientConnector;
+import com.facebook.nifty.duplex.TDuplexProtocolFactory;
+import com.facebook.swift.service.ThriftClient;
+import com.facebook.swift.service.ThriftClientConfig;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.airlift.units.Duration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.server.ConsensusService;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+@NotThreadSafe
+public class PeerConsensusServer extends AbstractPeer {
+  private static Logger LOG = LoggerFactory.getLogger(PeerConsensusServer.class);
+
+  private ThriftClientConfig thriftClientConf;
+  private ThriftClient<ConsensusService> thriftClient;
+  private ListenableFuture<ConsensusService> futureConnection;
+  private ConsensusService agent;
+
+  public PeerConsensusServer(final HServerAddress address,
+                             final int rank,
+                             final MutableRaftContext replica,
+                             final Configuration conf) {
+    super(address, rank, replica, conf);
+    int connectionTimeout = conf.getInt(
+            HConstants.RAFT_PEERSERVER_CONNECTION_TIMEOUT_MS, 1000);
+    int readTimeout = conf.getInt(
+            HConstants.RAFT_PEERSERVER_READ_TIMEOUT_MS, 10000);
+    int writeTimeout = conf.getInt(
+            HConstants.RAFT_PEERSERVER_WRITE_TIMEOUT_MS, 10000);
+
+    thriftClientConf = new ThriftClientConfig()
+            .setConnectTimeout(new Duration(connectionTimeout,
+                    TimeUnit.MILLISECONDS))
+            .setReadTimeout(new Duration(readTimeout, TimeUnit.MILLISECONDS))
+            .setWriteTimeout(new Duration(writeTimeout, TimeUnit.MILLISECONDS));
+  }
+
+  @Override
+  public void stop() {
+    super.stop();
+    synchronized (this) {
+      if (futureConnection != null) {
+        futureConnection.cancel(true);
+      }
+      if (agent != null) {
+        try {
+          agent.close();
+        } catch (Exception e) {
+          LOG.error("Cannot close the agent. Error: ", e);
+        }
+      }
+    }
+  }
+
+  @Override
+  protected void resetConnection()
+    throws ExecutionException, InterruptedException, TimeoutException {
+    LOG.debug("Resetting the peer connection to " + this + "(" + getAddress() + ")");
+    thriftClient =
+            new ThriftClient<>(RaftUtil.getThriftClientManager(), ConsensusService.class,
+                    thriftClientConf, this.toString());
+    ListenableFuture<ConsensusService> futureConnection = thriftClient.open(
+      new FramedClientConnector(getAddress().getInetSocketAddress(),
+        TDuplexProtocolFactory.fromSingleFactory(
+          new TCompactProtocol.Factory())));
+    setAgent(futureConnection, futureConnection.get(
+      1000,
+      TimeUnit.MILLISECONDS));
+  }
+
+  @Override
+  protected boolean clearConsensusServiceAgent(ConsensusService localAgent) {
+    // TODO: double check with Nifty community to better handler exceptions
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(getRaftContext().getMyAddress() + " attempting to close "
+        + localAgent + "(" + getAddress() + ")");
+    }
+    try {
+      localAgent.close();
+    } catch (Exception e) {
+      LOG.error("Unable to close the connection: " + e);
+    } finally {
+      return resetAgentIfEqual(localAgent);
+    }
+  }
+
+  private synchronized ConsensusService getAgent() {
+    return this.agent;
+  }
+
+  private synchronized void setAgent(ListenableFuture<ConsensusService> futureConnection, ConsensusService agent) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(getRaftContext().getMyAddress() + " setting agent to " +
+        agent + "(" + getAddress() + ")" +  " using " + futureConnection);
+    }
+    this.futureConnection = futureConnection;
+    this.agent = agent;
+  }
+
+  private synchronized boolean resetAgentIfEqual(ConsensusService localAgent) {
+    boolean success = false;
+    if (agent == localAgent) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(getRaftContext().getMyAddress() + " resetting agent "
+          + agent + " to null" + "(" + getAddress() + ")");
+      }
+      agent = null;
+      success = true;
+    }
+    return success;
+  }
+
+  @Override
+  protected ConsensusService getConsensusServiceAgent() {
+    if (getAgent() == null) {
+      synchronized (this) {
+        if (getAgent() == null) {
+          for (int i = 0; i < connectionRetry; i++) {
+            try {
+              resetConnection();
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("New connection established from " + getRaftContext().getMyAddress()
+                  + " to " + this + ": agent = " + agent + "(" + getAddress() + ")");
+              }
+            } catch (Throwable t) {
+              LOG.error("Failed to reset the connection to " + this +  " to " +
+                getAddress() + " due to " + t);
+              try {
+                Thread.sleep(connectionRetryInterval * i);
+              } catch (InterruptedException e) {}
+            }
+          }
+        }
+      }
+    }
+    return getAgent();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServer.java
new file mode 100644
index 0000000..81dbd57
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServer.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.server.peer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.regionserver.RaftEventListener;
+
+public interface PeerServer extends PeerServerMutableContext {
+  public void sendAppendEntries(AppendRequest request);
+  public void sendRequestVote(VoteRequest request);
+  public int getRank();
+  public void setRank(int rank);
+  public String getPeerServerName();
+  public Configuration getConf();
+  public void initialize();
+  public void stop();
+  public void registerDataStoreEventListener(RaftEventListener listener);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServerImmutableContext.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServerImmutableContext.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServerImmutableContext.java
new file mode 100644
index 0000000..1f90a0c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServerImmutableContext.java
@@ -0,0 +1,10 @@
+package org.apache.hadoop.hbase.consensus.server.peer;
+
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+
+public interface PeerServerImmutableContext {
+  EditId getLastEditID();
+  AppendRequest getLatestRequest();
+  int getRank();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServerMutableContext.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServerMutableContext.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServerMutableContext.java
new file mode 100644
index 0000000..e98a1e2
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerServerMutableContext.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.server.peer;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.metrics.PeerMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.Timer;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+
+public interface PeerServerMutableContext extends PeerServerImmutableContext {
+  void sendVoteRequestWithCallBack(VoteRequest request);
+  void sendAppendRequestWithCallBack(AppendRequest request);
+  MutableRaftContext getQuorumContext();
+  void enqueueEvent(final Event e);
+  void setLastEditID(final EditId id);
+  void stop();
+  void resetPeerContext();
+  void updatePeerAvailabilityStatus(boolean isAvailable);
+  PeerMetrics getMetrics();
+  void setLatestRequest(AppendRequest latestRequest);
+  void calculateAndSetAppendLag();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerStateMachine.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerStateMachine.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerStateMachine.java
new file mode 100644
index 0000000..90d3493
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/PeerStateMachine.java
@@ -0,0 +1,84 @@
+package org.apache.hadoop.hbase.consensus.server.peer;
+
+import org.apache.hadoop.hbase.consensus.fsm.*;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerServerEventType;
+import org.apache.hadoop.hbase.consensus.server.peer.states.*;
+import org.apache.hadoop.hbase.consensus.server.peer.transition.PeerServerTransitionType;
+
+public class PeerStateMachine extends FiniteStateMachine {
+
+  public PeerStateMachine(final String name, final PeerServer peerServer) {
+    super(name);
+
+    // Define all the states:
+    State start = new Start(peerServer);
+    State peerFollower = new PeerFollower(peerServer);
+    State handleVoteResponse = new PeerHandleVoteResponse(peerServer);
+    State sendVoteRequest = new PeerSendVoteRequest(peerServer);
+    State handleAppendResponse = new PeerHandleAppendResponse(peerServer);
+    State sendAppendRequest = new PeerSendAppendRequest(peerServer);
+    State handleRPCError = new PeerHandleRPCError(peerServer);
+    State halt = new PeerServerState(PeerServerStateType.HALT, peerServer);
+
+    // Define all the transitions:
+    Transition onStart =
+            new Transition(PeerServerTransitionType.ON_START,
+                    new OnEvent(PeerServerEventType.START));
+
+    Transition onAppendRequestReceived =
+            new Transition(PeerServerTransitionType.ON_APPEND_REQUEST,
+                    new OnEvent(PeerServerEventType.PEER_APPEND_REQUEST_RECEIVED));
+
+    Transition onAppendResponseReceived =
+            new Transition(PeerServerTransitionType.ON_APPEND_RESPONSE,
+                    new OnEvent(PeerServerEventType.PEER_APPEND_RESPONSE_RECEIVED));
+
+    Transition onVoteRequestReceived =
+            new Transition(PeerServerTransitionType.ON_VOTE_REQUEST,
+                    new OnEvent(PeerServerEventType.PEER_VOTE_REQUEST_RECEIVED));
+
+    Transition onVoteResponseReceived =
+            new Transition(PeerServerTransitionType.ON_VOTE_RESPONSE,
+                    new OnEvent(PeerServerEventType.PEER_VOTE_RESPONSE_RECEIVED));
+
+    Transition onRPCErrorReceived =
+            new Transition(PeerServerTransitionType.ON_RPC_ERROR,
+                    new OnEvent(PeerServerEventType.PEER_RPC_ERROR));
+
+    Transition peerReachable =
+            new Transition(PeerServerTransitionType.ON_PEER_IS_REACHABLE,
+                    new OnEvent(PeerServerEventType.PEER_REACHABLE));
+
+    Transition unConditional =
+            new Transition(PeerServerTransitionType.UNCONDITIONAL,
+                    new Unconditional());
+
+    Transition onHalt = new Transition(PeerServerTransitionType.ON_HALT,
+                    new OnEvent(PeerServerEventType.HALT));
+
+    // Add the transitions and states into the state machine
+    addTransition(start, peerFollower, onStart);
+
+    addTransition(peerFollower, handleVoteResponse, onVoteResponseReceived);
+    addTransition(peerFollower, handleAppendResponse, onAppendResponseReceived);
+    addTransition(peerFollower, handleAppendResponse, peerReachable);
+    addTransition(peerFollower, sendVoteRequest, onVoteRequestReceived);
+    addTransition(peerFollower, sendAppendRequest, onAppendRequestReceived);
+    addTransition(peerFollower, handleRPCError, onRPCErrorReceived);
+    addTransition(peerFollower, halt, onHalt);
+
+    addTransition(handleAppendResponse, peerFollower, unConditional);
+
+    addTransition(sendVoteRequest, peerFollower, unConditional);
+
+    addTransition(sendAppendRequest, peerFollower, unConditional);
+
+    addTransition(handleVoteResponse, peerFollower, unConditional);
+
+    addTransition(handleRPCError, peerFollower, peerReachable);
+    addTransition(handleRPCError, sendVoteRequest, onVoteRequestReceived);
+
+    // Set the initial state
+    setStartState(start);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/ReconnectTimeoutCallback.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/ReconnectTimeoutCallback.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/ReconnectTimeoutCallback.java
new file mode 100644
index 0000000..e6809a4
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/ReconnectTimeoutCallback.java
@@ -0,0 +1,21 @@
+package org.apache.hadoop.hbase.consensus.server.peer;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.TimeoutEventHandler;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerServerEventType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReconnectTimeoutCallback implements TimeoutEventHandler {
+  private static final Logger
+    LOG = LoggerFactory.getLogger(ReconnectTimeoutCallback.class);
+  private final PeerServer peer;
+
+  public ReconnectTimeoutCallback(final PeerServer peer) {
+    this.peer = peer;
+  }
+  public void onTimeout() {
+    LOG.debug(peer + " Reconnect timeout Triggered");
+    peer.enqueueEvent(new Event(PeerServerEventType.PEER_REACHABLE));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerAppendRequestEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerAppendRequestEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerAppendRequestEvent.java
new file mode 100644
index 0000000..0417feb
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerAppendRequestEvent.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.server.peer.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+
+public class PeerAppendRequestEvent extends Event {
+
+  private final AppendRequest request;
+
+  public PeerAppendRequestEvent(AppendRequest request) {
+    super(PeerServerEventType.PEER_APPEND_REQUEST_RECEIVED);
+    this.request = request;
+  }
+
+  public AppendRequest getAppendRequest() {
+    return this.request;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerAppendResponseEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerAppendResponseEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerAppendResponseEvent.java
new file mode 100644
index 0000000..410cd97
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerAppendResponseEvent.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.server.peer.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+
+public class PeerAppendResponseEvent extends Event {
+
+  private final AppendResponse response;
+
+  public PeerAppendResponseEvent(AppendResponse response) {
+    super(PeerServerEventType.PEER_APPEND_RESPONSE_RECEIVED);
+    this.response = response;
+  }
+
+  public AppendResponse getAppendResponse() {
+    return this.response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerServerEventType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerServerEventType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerServerEventType.java
new file mode 100644
index 0000000..23f68c4
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerServerEventType.java
@@ -0,0 +1,16 @@
+package org.apache.hadoop.hbase.consensus.server.peer.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.EventType;
+
+public enum PeerServerEventType implements EventType {
+  NONE,
+  START,
+  PEER_VOTE_REQUEST_RECEIVED,
+  PEER_VOTE_RESPONSE_RECEIVED,
+  PEER_APPEND_REQUEST_RECEIVED,
+  PEER_APPEND_RESPONSE_RECEIVED,
+  PEER_RPC_ERROR,
+  PEER_REACHABLE,
+  HALT,
+  MAX
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerVoteRequestEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerVoteRequestEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerVoteRequestEvent.java
new file mode 100644
index 0000000..43a90a0
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerVoteRequestEvent.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.server.peer.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+
+public class PeerVoteRequestEvent extends Event {
+
+  private final VoteRequest request;
+
+  public PeerVoteRequestEvent(VoteRequest request) {
+    super(PeerServerEventType.PEER_VOTE_REQUEST_RECEIVED);
+    this.request = request;
+  }
+
+  public VoteRequest getVoteRequest() {
+    return this.request;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerVoteResponseEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerVoteResponseEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerVoteResponseEvent.java
new file mode 100644
index 0000000..f5cf85c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/events/PeerVoteResponseEvent.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.server.peer.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.VoteResponse;
+
+public class PeerVoteResponseEvent extends Event {
+
+  private final VoteResponse response;
+
+  public PeerVoteResponseEvent(VoteResponse response) {
+    super(PeerServerEventType.PEER_VOTE_RESPONSE_RECEIVED);
+    this.response = response;
+  }
+
+  public VoteResponse getVoteResponse() {
+    return this.response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerFollower.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerFollower.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerFollower.java
new file mode 100644
index 0000000..d32c471
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerFollower.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+
+public class PeerFollower extends PeerServerState {
+  private static Logger LOG = LoggerFactory.getLogger(PeerFollower.class);
+
+  public PeerFollower(PeerServerMutableContext context) {
+    super(PeerServerStateType.PEER_FOLLOWER, context);
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    // TODO
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleAppendResponse.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleAppendResponse.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleAppendResponse.java
new file mode 100644
index 0000000..40205a5
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleAppendResponse.java
@@ -0,0 +1,282 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.fsm.FSMLargeOpsExecutorService;
+import org.apache.hadoop.hbase.consensus.log.TransactionLogManager;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerAppendRequestEvent;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerAppendResponseEvent;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+public class PeerHandleAppendResponse extends PeerServerAsyncState {
+  private static Logger LOG = LoggerFactory.getLogger(PeerHandleAppendResponse.class);
+  private static int globalMaxBatchLogs = HConstants.RAFT_BATCH_APPEND_MAX_EDITS_DEFAULT;
+  private static long globalMaxBatchBytes = HConstants.RAFT_BATCH_APPEND_MAX_BYTES_DEFAULT;
+  private static boolean batchSameTerm = false;
+  private final List<MemoryBuffer> buffers = new ArrayList<>();
+  private final Arena arena;
+  private ListenableFuture<?> createAppendRequestFromLogFuture = null;
+  private static long maxwaitToReadFromDiskNs;
+
+  public PeerHandleAppendResponse(PeerServerMutableContext context) {
+    super(PeerServerStateType.HANDLE_APPEND_RESPONSE, context);
+    arena = context.getQuorumContext().getArena();
+    globalMaxBatchLogs = getConf().getInt(
+      HConstants.CONSENSUS_PUSH_APPEND_MAX_BATCH_LOGS_KEY,
+      globalMaxBatchLogs);
+    globalMaxBatchBytes = getConf().getLong(
+      HConstants.CONSENSUS_PUSH_APPEND_MAX_BATCH_BYTES_KEY,
+      globalMaxBatchBytes);
+    batchSameTerm = getConf().getBoolean(
+      HConstants.CONSENSUS_PUSH_APPEND_BATCH_SAME_TERM_KEY,
+      false);
+    maxwaitToReadFromDiskNs =
+        getConf().getLong(HConstants.MAXWAIT_TO_READ_FROM_DISK_NS,
+          HConstants.MAXWAIT_TO_READ_FROM_DISK_NS_DEFAULT);
+  }
+
+  public static void setGlobalMaxBatchLogs(int n) {
+    globalMaxBatchLogs = Math.max(1, n);
+  }
+
+  private Configuration getConf() {
+    return c.getQuorumContext().getConf();
+  }
+
+  /**
+   *  batchSameTerm
+   *
+   *  Whether to batch edits that have the same term.
+   */
+  public boolean batchSameTerm() {
+    return batchSameTerm;
+  }
+
+  public int getMaxBatchLogs() {
+    return globalMaxBatchLogs;
+  }
+
+  public long getMaxBatchBytes() {
+    return globalMaxBatchBytes;
+  }
+
+  @Override
+  public boolean isComplete() {
+    return (createAppendRequestFromLogFuture == null ||
+      createAppendRequestFromLogFuture.isDone());
+  }
+
+  @Override
+  public ListenableFuture<?> getAsyncCompletion() {
+    return createAppendRequestFromLogFuture;
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    createAppendRequestFromLogFuture = null;
+
+    // Nothing to do if you are not the leader
+    if (!c.getQuorumContext().isLeader()) {
+      return;
+    }
+
+    // Add the event to the raft state machine
+    // update the last successfully committed edit by the peer
+    if (e instanceof PeerAppendResponseEvent) {
+      final AppendResponse response = ((PeerAppendResponseEvent)e).getAppendResponse();
+      if (response != null) {
+        c.setLastEditID(response.getPrevEditID());
+        // Update the peer's availability
+        c.updatePeerAvailabilityStatus(true);
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(c + " got null as AppendResponse");
+        }
+      }
+    }
+
+    final EditId startEditId = c.getLastEditID();
+    final EditId logManagerLastEditID =
+      c.getQuorumContext().getLogManager().getLastEditID();
+
+    // If the log manager has new entries
+    if (logManagerLastEditID.compareTo(startEditId) > 0) {
+      Runnable createAppendRequestRunnable = new Runnable() {
+        @Override
+        public void run() {
+          AppendRequest nextRequest = createAppendRequestFromLog(startEditId,
+            logManagerLastEditID);
+
+          if (nextRequest != null) {
+            if (LOG.isTraceEnabled()) {
+              LOG.trace(c + " is sending " + nextRequest);
+            }
+            c.enqueueEvent(new PeerAppendRequestEvent(nextRequest));
+          }
+        }
+      };
+
+      createAppendRequestFromLogFuture =
+        FSMLargeOpsExecutorService.submitToReadOpsThreadPool(
+          createAppendRequestRunnable);
+    } else if (c.getLatestRequest() != null &&
+      c.getLatestRequest().getLogId(0).compareTo(startEditId) > 0) {
+      // We have seen new entries which might have not been made in the log manager.
+      // Lets send it.
+      AppendRequest nextRequest = c.getLatestRequest();
+      if (nextRequest != null) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(c + " is sending " + nextRequest);
+        }
+        c.enqueueEvent(new PeerAppendRequestEvent(nextRequest));
+      }
+    }
+  }
+
+  /**
+   * Depending upon the batching parameters, it fetches the next set of transactions
+   * for the log manager.
+   *
+   * @param startEditId EditId after which, transactions will be fetched
+   * @param logManagerLastEditID latest EditId reported by the log manager
+   * @return AppendRequest, or null in case of error
+   */
+  private AppendRequest createAppendRequestFromLog(final EditId startEditId,
+                                                   final EditId logManagerLastEditID) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(c + " adding lagging entries starting from " + startEditId +
+        "; log manager has " + logManagerLastEditID);
+    }
+
+    if (arena != null) {
+      // Reset the buffers as we are going to read new transactions
+      for (MemoryBuffer buffer : buffers) {
+        arena.freeByteBuffer(buffer);
+      }
+    }
+
+    buffers.clear();
+
+    long start = System.nanoTime();
+
+    // If the peer is at a lesser or equal term as yours and if there are more
+    // entries present in the log, then send the missing transactions.
+    List<ByteBuffer> edits = new ArrayList<>();
+    List<EditId> logIds = new ArrayList<EditId>();
+    long nbytes = 0;
+
+    EditId lastEditID = startEditId;
+    int maxEdits = getMaxBatchLogs();
+    long maxBytes = getMaxBatchBytes();
+
+    ConsensusHost leader = c.getQuorumContext().getLeader();
+    // we could check if leader corresponds to c.getQuorumContext().
+    // But it's unnecessary.
+    collect_edits_loop:
+    while (c.getQuorumContext().isLeader() &&
+      logManagerLastEditID.compareTo(lastEditID) > 0 &&
+      nbytes < maxBytes && edits.size() < maxEdits &&
+      // time taken so far is smaller than defined threshold
+        (System.nanoTime() - start) < maxwaitToReadFromDiskNs) {
+      try {
+        Pair<EditId, MemoryBuffer> idTxn =
+          c.getQuorumContext().getLogManager().getNextEditIdTransaction(
+          c.toString(), lastEditID.getIndex(), arena);
+
+        if (idTxn == null ||
+          idTxn.getFirst() == null ||
+          idTxn.getFirst().equals(TransactionLogManager.UNDEFINED_EDIT_ID)) {
+          LOG.error(c.getQuorumContext() + " could not locate next id after "
+            + lastEditID + ". Returning.." + idTxn);
+          break collect_edits_loop;
+        }
+
+        buffers.add(idTxn.getSecond());
+        // We successfully found the entry, lets update the sharedPoolBuffer
+        EditId nextEditId = idTxn.getFirst();
+
+        // we currently cannot handle logs with differing terms on the receiver side
+        if (batchSameTerm() && logIds.size() > 0 &&
+          nextEditId.getTerm() != logIds.get(0).getTerm()) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("---- " + c + " stop at lagging entry " + nextEditId +
+              "[prev=" + lastEditID + "] due to changing terms from "
+              + logIds.get(0));
+          }
+          break collect_edits_loop;
+        } else {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("---- " + c + " trying to add lagging entry: " +
+              nextEditId + "[prev=" + lastEditID + "]");
+          }
+        }
+
+        ByteBuffer edit = idTxn.getSecond().getBuffer();
+        edits.add(edit);
+        logIds.add(nextEditId);
+        nbytes += edit.remaining();
+        lastEditID = nextEditId;
+      } catch (IOException ex) {
+        LOG.error(
+          c.getQuorumContext() + " is unable to read next transaction " +
+            "for previous id " + lastEditID + " for " + c, ex);
+        break collect_edits_loop;
+      }
+    }
+
+    if (edits.size() == 0) {
+      LOG.warn("NOTHING to APPEND!");
+      return null;
+    }
+
+    if (!leader.equals(c.getQuorumContext().getLeader())) {
+      // Note that this covers the following cases
+      // 1. I'm a leader both before and after the loop; but there was at least
+      // one re-election in between.
+      // 2. I was a leader before the loop and no longer the leader afterwards.
+      LOG.warn(c.getQuorumContext() + " aborts sending AppendRequest because " +
+        "the leader has changed from " + leader + " to " +
+        c.getQuorumContext().getLeader());
+      return null;
+    }
+
+    long batchCommitIndex = Math.max(logIds.get(logIds.size() - 1).getIndex(),
+      c.getQuorumContext().getCommittedEdit().getIndex());
+
+    // Update the metrics
+    long elapsed = System.nanoTime() - start;
+    c.getMetrics().getBatchRecoverySize().add(logIds.size());
+    c.getMetrics().getBatchRecoveryLatency().add(elapsed, TimeUnit.NANOSECONDS);
+
+    // Create request from nextEditId;
+    return (new AppendRequest(
+      c.getQuorumContext().getQuorumName(),
+      leader,
+      false,
+      batchCommitIndex,
+      c.getQuorumContext().getMinUnPersistedIndexAcrossQuorum(),
+      startEditId,
+      logIds,
+      edits
+    ));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleRPCError.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleRPCError.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleRPCError.java
new file mode 100644
index 0000000..33f0575
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleRPCError.java
@@ -0,0 +1,27 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerAppendResponseEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PeerHandleRPCError extends PeerServerState {
+  private static Logger LOG = LoggerFactory.getLogger(PeerHandleRPCError.class);
+
+  public PeerHandleRPCError(PeerServerMutableContext context) {
+    super(PeerServerStateType.HANDLE_RPC_ERROR, context);
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    c.updatePeerAvailabilityStatus(false);
+  }
+
+  @Override
+  public void onExit(final Event e) {
+    super.onExit(e);
+    c.enqueueEvent(new PeerAppendResponseEvent(null));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleVoteResponse.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleVoteResponse.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleVoteResponse.java
new file mode 100644
index 0000000..c977829
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerHandleVoteResponse.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+
+public class PeerHandleVoteResponse extends PeerServerState {
+  private static Logger LOG = LoggerFactory.getLogger(PeerHandleVoteResponse.class);
+
+  public PeerHandleVoteResponse(PeerServerMutableContext context) {
+    super(PeerServerStateType.HANDLE_VOTE_RESPONSE, context);
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    // TODO
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerRecovery.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerRecovery.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerRecovery.java
new file mode 100644
index 0000000..14dbabd
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerRecovery.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+
+public class PeerRecovery extends PeerServerState {
+  private static Logger LOG = LoggerFactory.getLogger(PeerRecovery.class);
+
+  public PeerRecovery(PeerServerMutableContext context) {
+    super(PeerServerStateType.RECOVERY, context);
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    // TODO
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerSendAppendRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerSendAppendRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerSendAppendRequest.java
new file mode 100644
index 0000000..7b9e095
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerSendAppendRequest.java
@@ -0,0 +1,50 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerAppendRequestEvent;
+
+public class PeerSendAppendRequest extends PeerServerState {
+  private static Logger LOG = LoggerFactory.getLogger(PeerSendAppendRequest.class);
+
+  public PeerSendAppendRequest(PeerServerMutableContext context) {
+    super(PeerServerStateType.SEND_APPEND_REQUEST, context);
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+
+    final AppendRequest request = (((PeerAppendRequestEvent)e)).getAppendRequest();
+
+    // Seen a new request, update the latest request
+    AppendRequest lastSeenAppendRequest = c.getLatestRequest();
+    if (lastSeenAppendRequest == null ||
+      lastSeenAppendRequest.getLogId(0).compareTo(request.getLogId(0)) < 0) {
+      c.setLatestRequest(request);
+    }
+    if (request.isTraceable()) {
+      LOG.debug("[AppendRequest Trace] " + c.toString() + " is processing " + request + " " +
+        "lastRequest " + c.getLatestRequest() + " lastEditID " + c.getLastEditID());
+    }
+    c.calculateAndSetAppendLag();
+
+    // Send the request only if a) the latest EditId ack'ed by the peer is the
+    // as the EditId of the previous request or b) if the latest ack'ed Id is
+    // the same as the current Id.
+    //
+    // Case b is important if no majority has been reached yet on the request,
+    // and it needs to be retried. It needs to be resent to the peer to act as a
+    // heartbeat and keep its progress timer from firing. Failing to do so would
+    // lead to unnecessary leader election.
+    if (request.getPrevLogId().compareTo(c.getLastEditID()) == 0 ||
+            request.getLogId(0).compareTo(c.getLastEditID()) == 0) {
+      c.sendAppendRequestWithCallBack(request);
+    } else if (request.isTraceable()) {
+        LOG.debug("[AppendRequest Trace] Not able to send the new append request out: " +
+          "request: " + request + " and the lastEditID is " + c.getLastEditID());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerSendVoteRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerSendVoteRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerSendVoteRequest.java
new file mode 100644
index 0000000..28f6b35
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerSendVoteRequest.java
@@ -0,0 +1,22 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerVoteRequestEvent;
+
+public class PeerSendVoteRequest extends PeerServerState {
+  private static Logger LOG = LoggerFactory.getLogger(PeerSendVoteRequest.class);
+
+  public PeerSendVoteRequest(PeerServerMutableContext context) {
+    super(PeerServerStateType.SEND_VOTE_REQUEST, context);
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    PeerVoteRequestEvent voteRequestEvent = (PeerVoteRequestEvent)e;
+    this.c.sendVoteRequestWithCallBack(voteRequestEvent.getVoteRequest());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerAsyncState.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerAsyncState.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerAsyncState.java
new file mode 100644
index 0000000..6b13fa9
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerAsyncState.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+
+/**
+ * A {@link PeerServerState} which has async tasks.
+ */
+public class PeerServerAsyncState extends PeerServerState {
+  public PeerServerAsyncState(final PeerServerStateType t,
+                              PeerServerMutableContext c) {
+    super(t, c);
+  }
+
+  @Override
+  public boolean isAsyncState() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerState.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerState.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerState.java
new file mode 100644
index 0000000..a21201b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerState.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.fsm.State;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+
+public class PeerServerState extends State {
+  private static Logger LOG = LoggerFactory.getLogger(PeerServerState.class);
+
+  protected PeerServerMutableContext c;
+  public PeerServerState(final PeerServerStateType t, PeerServerMutableContext c) {
+    super(t);
+    this.c = c;
+  }
+
+  @Override
+  public void onEntry(final Event e) {}
+
+  @Override
+  public void onExit(final Event e) {}
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerStateType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerStateType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerStateType.java
new file mode 100644
index 0000000..6f3a781
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/PeerServerStateType.java
@@ -0,0 +1,16 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+
+import org.apache.hadoop.hbase.consensus.fsm.StateType;
+
+public enum PeerServerStateType implements StateType {
+  START,
+  PEER_FOLLOWER,
+  SEND_VOTE_REQUEST,
+  HANDLE_VOTE_RESPONSE,
+  SEND_APPEND_REQUEST,
+  HANDLE_APPEND_RESPONSE,
+  HANDLE_RPC_ERROR,
+  RECOVERY,
+  HALT,
+  MAX
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/Start.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/Start.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/Start.java
new file mode 100644
index 0000000..0d76636
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/states/Start.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.server.peer.states;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServerMutableContext;
+
+public class Start extends PeerServerState {
+  private static Logger LOG = LoggerFactory.getLogger(Start.class);
+
+  public Start(PeerServerMutableContext context) {
+    super(PeerServerStateType.START, context);
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/transition/PeerServerTransitionType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/transition/PeerServerTransitionType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/transition/PeerServerTransitionType.java
new file mode 100644
index 0000000..1958bed
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/peer/transition/PeerServerTransitionType.java
@@ -0,0 +1,22 @@
+package org.apache.hadoop.hbase.consensus.server.peer.transition;
+
+import org.apache.hadoop.hbase.consensus.fsm.TransitionType;
+
+public enum PeerServerTransitionType implements TransitionType {
+  NONE,
+  UNCONDITIONAL,
+  ON_START,
+  ON_VOTE_REQUEST,
+  ON_VOTE_RESPONSE,
+
+  ON_APPEND_REQUEST,
+  ON_APPEND_RESPONSE,
+
+  ON_APPEND_NACK_RESPONSE,
+  ON_APPEND_ACK_RESPONSE,
+  ON_PEER_IS_REACHABLE,
+  ON_RPC_ERROR,
+
+  ON_HALT,
+  MAX
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/util/RaftUtil.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/util/RaftUtil.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/util/RaftUtil.java
new file mode 100644
index 0000000..d73cb8e
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/util/RaftUtil.java
@@ -0,0 +1,107 @@
+package org.apache.hadoop.hbase.consensus.util;
+
+import com.facebook.nifty.client.NettyClientConfig;
+import com.facebook.nifty.client.NettyClientConfigBuilder;
+import com.facebook.nifty.client.NiftyClient;
+import com.facebook.swift.codec.ThriftCodecManager;
+import com.facebook.swift.service.ThriftClientEventHandler;
+import com.facebook.swift.service.ThriftClientManager;
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.consensus.quorum.AggregateTimer;
+import org.apache.hadoop.hbase.consensus.quorum.RepeatingTimer;
+import org.apache.hadoop.hbase.consensus.quorum.TimeoutEventHandler;
+import org.apache.hadoop.hbase.consensus.quorum.Timer;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.jboss.netty.channel.socket.nio.NioSocketChannelConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class RaftUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RaftUtil.class);
+  private static ThriftClientManager clientManager;
+
+  static {
+    try {
+      NettyClientConfigBuilder clientConfigBuilder =
+        NettyClientConfig.newBuilder();
+      final NioSocketChannelConfig socketConfig =
+        clientConfigBuilder.getSocketChannelConfig();
+
+      socketConfig.setKeepAlive(true);
+      socketConfig.setTcpNoDelay(true);
+      clientConfigBuilder.setBossThreadCount(2);
+
+      clientManager = new ThriftClientManager(
+        new ThriftCodecManager(),
+        new NiftyClient(clientConfigBuilder.build()),
+        ImmutableSet.<ThriftClientEventHandler>of());
+    } catch (Throwable t) {
+      LOG.error("Unable to initialize ThriftClientManager.", t);
+      throw t;
+    }
+  }
+
+  public static HRegionInfo createDummyRegionInfo(String region) {
+    return createDummyRegionInfo(region, null);
+  }
+
+  public static HRegionInfo createDummyRegionInfo(String region, Map<HServerAddress,
+    Integer> peers) {
+    HRegionInfo regionInfo = new HRegionInfo(new HTableDescriptor(region),
+        Bytes.toBytes("00000000"), Bytes.toBytes("ffffffff"), false, 1000,
+        null, null);
+    Map<String, Map<HServerAddress, Integer>> peerMap = new HashMap<>();
+    peerMap.put(HRegionInfo.LOCAL_DC_KEY, peers);
+    regionInfo.setPeers(peerMap);
+    return regionInfo;
+  }
+
+  public static <T> String listToString(List<T> list) {
+    if (list == null) {
+      return null;
+    }
+    return Joiner.on(", ").useForNull("null").join(list);
+  }
+
+  public static HServerAddress getHRegionServerAddress(HServerAddress
+                                                            localConsensusServerAddress) {
+    return new HServerAddress(localConsensusServerAddress.getBindAddress(),
+      localConsensusServerAddress.getPort() - HConstants.CONSENSUS_SERVER_PORT_JUMP);
+  }
+
+  public static HServerAddress getLocalConsensusAddress(HServerAddress
+                                                         regionServerAddress) {
+    return new HServerAddress(regionServerAddress.getBindAddress(),
+      regionServerAddress.getPort() + HConstants.CONSENSUS_SERVER_PORT_JUMP);
+  }
+
+  public static ThriftClientManager getThriftClientManager() {
+    return clientManager;
+  }
+
+  public static Timer createTimer(boolean useAggregateTimer, final String name,
+                                  final long delay, TimeUnit unit,
+                                  final TimeoutEventHandler callback,
+                                  final AggregateTimer aggregateTimer) {
+    if (useAggregateTimer) {
+      return aggregateTimer.createTimer(name, delay, unit, callback);
+    }
+    return new RepeatingTimer(name, delay, unit, callback);
+  }
+
+  public static boolean isNetworkError(Throwable e) {
+    return 
+      e instanceof org.apache.thrift.transport.TTransportException
+      || e instanceof com.facebook.swift.service.RuntimeTTransportException;
+  }
+}


[11/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AggregateTimer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AggregateTimer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AggregateTimer.java
new file mode 100644
index 0000000..d10ef7a
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AggregateTimer.java
@@ -0,0 +1,93 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import java.util.Set;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+
+/**
+ * This is a Timer that can work with multiple ConstituentTimer objects, and
+ * invoke their appropriate callbacks when required.
+ *
+ * The main benefit over RepeatingTimer is to not have to use a separate thread
+ * for each timer.
+ */
+public class AggregateTimer {
+  public static final Log LOG = LogFactory.getLog(AggregateTimer.class);
+  private ScheduledExecutorService executor;
+  Set<ConstituentTimer> timers;
+
+  public class TimerEvent implements Runnable {
+    final ConstituentTimer timer;
+    private boolean cancelled = false;
+
+    public TimerEvent(ConstituentTimer timer) {
+      this.timer = timer;
+    }
+
+    public synchronized void cancel() {
+      cancelled = true;
+    }
+
+    @Override
+    public synchronized void run() {
+      try {
+        if (cancelled || timer.isStopped()) {
+          return;
+        }
+
+        timer.onTimeOut();
+        if (!timer.isStopped()) {
+          schedule(this);
+        }
+      } catch (Exception e) {
+        LOG.error("Timer caught an unknown exception ", e);
+        throw e;
+      }
+    }
+  }
+
+  public AggregateTimer() {
+    this.timers = new ConcurrentSkipListSet<>();
+    this.executor = Executors.newSingleThreadScheduledExecutor(
+      new DaemonThreadFactory("aggregate-timer"));
+  }
+
+  public ConstituentTimer createTimer(
+    String timerName, final long delay, final TimeUnit unit,
+    final TimeoutEventHandler callback) {
+    ConstituentTimer timer =
+      new ConstituentTimer(this, timerName, delay, unit, callback);
+    submitNewTimerEvent(timer);
+
+    return timer;
+  }
+
+  public TimerEvent submitNewTimerEvent(final ConstituentTimer timer) {
+    if (!timer.isStopped()) {
+      TimerEvent event = new TimerEvent(timer);
+      schedule(event);
+      return event;
+    }
+    return null;
+  }
+
+  void schedule(TimerEvent event) {
+    executor.schedule(event,
+      event.timer.getDelayMillis() + event.timer.getBackOffInterval(),
+      TimeUnit.MILLISECONDS);
+  }
+
+  public void shutdown() {
+    executor.shutdown();
+  }
+
+  public void shutdownNow() {
+    executor.shutdownNow();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendConsensusSession.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendConsensusSession.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendConsensusSession.java
new file mode 100644
index 0000000..ec46247
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendConsensusSession.java
@@ -0,0 +1,247 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class represents a session of an in-progress AppendRequest, and holds
+ * information such as if the session is complete, majority acked, or it failed
+ * for some reason.
+ */
+public class AppendConsensusSession implements AppendConsensusSessionInterface {
+  private static Logger LOG = LoggerFactory.getLogger(AppendConsensusSession.class);
+  private final int majorityCount;
+  private final EditId id;
+  private final AppendRequest request;
+  private final ReplicateEntriesEvent replicateEntriesEvent;
+
+  /** The key of ackMap is the peer host name and value is the time elapsed for the ack */
+  private final Map<String, Long> ackMap = new HashMap<>();
+  private final Set<String> lagSet = new HashSet<>();
+  private final Set<String> highTermSet = new HashSet<>();
+  private final Set<String> peers;
+  private final ConsensusMetrics metrics;
+  private long sessionStartTime;
+
+  private final int currentRank;
+
+  /** This boolean flag indicates whether there is any higher rank peer caught up the transacations
+   * and potentially ready for taking over the leadership. */
+  private boolean isHigherRankPeerCaughtup = false;
+
+  private final boolean enableStepDownOnHigherRankCaughtUp;
+
+  private static final long WRITE_OUTLIERS_DEFAULT_MS = 5; // 5 ms
+
+  private SessionResult currentResult = SessionResult.NOT_COMPLETED;
+
+  private static final int APPEND_RETRY_MAX = 5;
+
+  private int tries = 0;
+  private final int maxTries;
+
+  ImmutableRaftContext c;
+
+  /**
+   * To construct a AppendConsensusSession.
+   * @param majorityCount The majority number of the quorum size.
+   * @param request The append request.
+   * @param event   The ReplicatedEntriesEvent for this corresponding appendRequest.
+   * @param rank The rank of the current peer
+   * @param enableStepDownOnHigherRankCaughtUp Whether to step down voluntarily if the higher rank peer
+   *                                       has caught up.
+   */
+  public AppendConsensusSession(ImmutableRaftContext c,
+                                int majorityCount,
+                                final AppendRequest request,
+                                final ReplicateEntriesEvent event,
+                                final ConsensusMetrics metrics,
+                                final int rank,
+                                final boolean enableStepDownOnHigherRankCaughtUp,
+                                final int maxTries,
+                                final Set<String> peers) {
+    this.c = c;
+    this.majorityCount = majorityCount;
+    this.request = request;
+    assert request.validateFields();
+    assert request.logCount() == 1;
+    this.id = request.getLogId(0).clone(); //??
+    this.replicateEntriesEvent = event;
+    this.metrics = metrics;
+    this.currentRank = rank;
+    this.enableStepDownOnHigherRankCaughtUp = enableStepDownOnHigherRankCaughtUp;
+    this.peers = peers;
+    this.maxTries = maxTries;
+    this.sessionStartTime = System.nanoTime();
+    resetInvariants();
+  }
+
+  @Override
+  public void reset() {
+    resetInvariants();
+
+    if (++tries >= APPEND_RETRY_MAX) {
+      this.request.enableTraceable();
+    }
+  }
+
+  private void resetInvariants() {
+    ackMap.clear();
+    lagSet.clear();
+    highTermSet.clear();
+    isHigherRankPeerCaughtup = false;
+    currentResult = SessionResult.NOT_COMPLETED;
+  }
+
+  @Override
+  public boolean isComplete() {
+    return getResult().equals(SessionResult.NOT_COMPLETED) ? false : true;
+  }
+
+  private void traceLogAppendOutliers() {
+    long elapsed = System.nanoTime() - sessionStartTime;
+    if (elapsed > (WRITE_OUTLIERS_DEFAULT_MS * 1000)) {
+      StringBuffer sb = new StringBuffer();
+      sb.append("AppendConsensusSession outlier: " + request.toString() +
+        " took " + elapsed + " ns; [");
+      for (Map.Entry<String, Long> entry : ackMap.entrySet()) {
+        sb.append(entry.getKey() + " -> " + entry.getValue() + " ; ");
+      }
+      sb.append("]");
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(sb.toString());
+      }
+    }
+  }
+
+  private void generateResult() {
+    if (!currentResult.equals(SessionResult.NOT_COMPLETED)) {
+      return;
+    }
+
+    // Check if the majority has been reached and the I (leader) have ACK-ed too.
+    // We do this because we cannot serve reads until the leader has ACK-ed.
+    // If I am not part of the peers, my vote doesn't matter.
+    if (ackMap.size() >= majorityCount
+      && (!peers.contains(c.getMyAddress()) ||
+      ackMap.containsKey(c.getMyAddress()))) {
+      long elapsed = System.nanoTime() - sessionStartTime;
+      metrics.getAppendEntriesLatency().add(elapsed, TimeUnit.NANOSECONDS);
+
+      if (isStepDownOnHigherRankCaughtUpEnabled() && isHigherRankPeerCaughtup) {
+        // When there is at least one higher rank peer caught up all the transactions,
+        // the current leader needs to step down voluntarily.
+        metrics.incHigherRankCaughtUpStepDown();
+        currentResult = SessionResult.STEP_DOWN;
+        LOG.debug("Going to step down voluntarily from leadership");
+      } else {
+        // Otherwise, return majority acked.
+        if (LOG.isTraceEnabled()) {
+          traceLogAppendOutliers();
+        }
+        currentResult = SessionResult.MAJORITY_ACKED;
+      }
+
+    } else if (highTermSet.size() >= majorityCount) {
+      metrics.incAppendEntriesStepDown();
+      currentResult =  SessionResult.STEP_DOWN;
+    } else if (lagSet.size() + highTermSet.size() >= majorityCount) {
+      metrics.incAppendEntriesRetries();
+      currentResult = SessionResult.RETRY;
+    }
+  }
+
+  private boolean isStepDownOnHigherRankCaughtUpEnabled() {
+    return enableStepDownOnHigherRankCaughtUp;
+  }
+
+  @Override
+  public SessionResult getResult() {
+    generateResult();
+    if (request.isTraceable()) {
+      LOG.debug(String.format("[AppendRequest Trace] %s and current result is %s",
+        request.toString(), currentResult.toString()));
+    }
+    return currentResult;
+  }
+
+  @Override
+  public EditId getSessionId() {
+    return id;
+  }
+
+  @Override
+  public void cancel() {
+    this.metrics.incHeartBeatCanceled();
+    this.currentResult = SessionResult.CANCELED;
+  }
+
+  @Override
+  public boolean isTimeout() {
+    boolean timedout = tries >= maxTries;
+    if (timedout) {
+      LOG.info(String.format("Append Request (%s) timed out", this.request.toString()));
+    }
+    return timedout;
+  }
+
+  @Override
+  public AppendRequest getAppendRequest() {
+    return request;
+  }
+
+  @Override
+  public ReplicateEntriesEvent getReplicateEntriesEvent() {
+    return replicateEntriesEvent;
+  }
+
+  @Override
+  public void incrementAck(final EditId id, final String address, final int rank,
+                           boolean canTakeover) {
+    assert this.id.equals(id);
+    if (peers.contains(address)) {
+      ackMap.put(address, System.nanoTime() - sessionStartTime);
+      if (rank > currentRank && canTakeover) {
+        isHigherRankPeerCaughtup = true;
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("Append Request (%s) received a higher" +
+            "rank ack from %s", request.toString(), address));
+        }
+      }
+    }
+  }
+
+  @Override
+  public void incrementHighTermCnt(final EditId id, final String address) {
+    assert this.id.equals(id);
+    if (peers.contains(address)) {
+      highTermSet.add(address);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("Append Request (%s) received a higher term" +
+            "response from %s", request.toString(), address));
+      }
+    }
+  }
+
+  @Override
+  public void incrementLagCnt(final EditId id, final String address) {
+    assert this.id.equals(id);
+    if (peers.contains(address)) {
+      lagSet.add(address);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("Append Request (%s) received a nak response " +
+          "from %s", request.toString(), address));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendConsensusSessionInterface.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendConsensusSessionInterface.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendConsensusSessionInterface.java
new file mode 100644
index 0000000..44c483b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendConsensusSessionInterface.java
@@ -0,0 +1,17 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+
+public interface AppendConsensusSessionInterface extends ConsensusSession {
+  AppendRequest getAppendRequest();
+  boolean isTimeout();
+  ReplicateEntriesEvent getReplicateEntriesEvent();
+  void incrementAck(final EditId id, final String address, final int rank,
+                    boolean canTakeover);
+  void incrementHighTermCnt(final EditId id, final String address);
+  void incrementLagCnt(final EditId id, final String address);
+  void reset();
+  void cancel();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendResponseCallBack.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendResponseCallBack.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendResponseCallBack.java
new file mode 100644
index 0000000..12b9e79
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AppendResponseCallBack.java
@@ -0,0 +1,26 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.FutureCallback;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+
+public class AppendResponseCallBack implements FutureCallback<AppendResponse>{
+  private static Logger LOG = LoggerFactory.getLogger(AppendResponseCallBack.class);
+  private ConsensusSession session;
+
+  public AppendResponseCallBack(ConsensusSession session) {
+    this.session = session;
+  }
+
+  public void onSuccess(AppendResponse response) {
+
+  }
+
+  @Override
+  public void onFailure(Throwable arg0) {
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ConsensusSession.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ConsensusSession.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ConsensusSession.java
new file mode 100644
index 0000000..448dcf8
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ConsensusSession.java
@@ -0,0 +1,9 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+
+public interface ConsensusSession {
+  boolean isComplete();
+  SessionResult getResult();
+  EditId getSessionId();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ConstituentTimer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ConstituentTimer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ConstituentTimer.java
new file mode 100644
index 0000000..8b11a32
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ConstituentTimer.java
@@ -0,0 +1,109 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This is a timer which is a part of an AggregateTimer, which holds several
+ * such timers, and invokes the callbacks on the child timers as required.
+ *
+ * The timer retains the behavior of the regular timer, and the user need not
+ * know that this timer doesn't own an exclusive thread.
+ */
+public class ConstituentTimer implements Timer {
+  public static final Log LOG = LogFactory.getLog(ConstituentTimer.class);
+  private volatile long delayMillis;
+  private volatile long backOffInterval;
+
+  private volatile boolean isStopped = true;
+
+  AggregateTimer aggregateTimer;
+  TimeoutEventHandler callback;
+  AggregateTimer.TimerEvent timerEvent;
+
+  String timerName;
+
+  /**
+   * @param aggregateTimer The AggregateTimer object to use.
+   * @param delay Delay between the timeouts
+   * @param timeUnit The time unit of the delay
+   * @param callback The callback to register
+   */
+  public ConstituentTimer(AggregateTimer aggregateTimer,
+                          String timerName,
+                          long delay,
+                          TimeUnit timeUnit,
+                          TimeoutEventHandler callback) {
+    this.aggregateTimer = aggregateTimer;
+    this.callback = callback;
+    this.delayMillis = TimeUnit.MILLISECONDS.convert(delay, timeUnit);
+    this.backOffInterval = 0;
+    this.timerName = timerName;
+  }
+
+  @Override
+  public synchronized void start() {
+    if (isStopped) {
+      isStopped = false;
+      timerEvent = aggregateTimer.submitNewTimerEvent(this);
+      backOffInterval = 0;
+    }
+  }
+
+  @Override
+  public synchronized void stop() {
+    if (!isStopped) {
+      isStopped = true;
+      timerEvent.cancel();
+      timerEvent = null;
+    }
+  }
+
+  @Override
+  public synchronized void reset() {
+    if (!isStopped) {
+      // Reset happens by proactively removing and inserting the timer event
+      // again.
+      timerEvent.cancel();
+      timerEvent = aggregateTimer.submitNewTimerEvent(this);
+      backOffInterval = 0;
+    }
+  }
+
+  @Override
+  public synchronized void shutdown() {
+    stop();
+  }
+
+  @Override
+  public synchronized void backoff(long backOffTime, TimeUnit units) {
+    backOffInterval = TimeUnit.MILLISECONDS.convert(backOffTime, units);
+  }
+
+  @Override
+  public synchronized void setDelay(long delay, TimeUnit unit) {
+    delayMillis = TimeUnit.MILLISECONDS.convert(delay, unit);
+  }
+
+  public void onTimeOut() {
+    callback.onTimeout();
+  }
+
+  public long getDelayMillis() {
+    return delayMillis;
+  }
+
+  public long getBackOffInterval() {
+    return backOffInterval;
+  }
+
+  public boolean isStopped() {
+    return isStopped;
+  }
+
+  public String getTimerName() {
+    return timerName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/HeartbeatTimeoutCallback.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/HeartbeatTimeoutCallback.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/HeartbeatTimeoutCallback.java
new file mode 100644
index 0000000..77f8718
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/HeartbeatTimeoutCallback.java
@@ -0,0 +1,31 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import java.nio.ByteBuffer;
+
+public class
+  HeartbeatTimeoutCallback implements TimeoutEventHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(
+          HeartbeatTimeoutCallback.class);
+  private final RaftQuorumContext replica;
+  private static final ReplicateEntriesEvent HEARTBEAT_EVENT = new ReplicateEntriesEvent(true,
+      ByteBuffer.allocate(1));
+
+  public HeartbeatTimeoutCallback(final RaftQuorumContext replica) {
+    this.replica = replica;
+  }
+
+  @Override
+  public void onTimeout() {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("HeartBeat Triggered on " + replica);
+    }
+
+    // When there is no append request for a long time, in order to avoid
+    // progress timeouts, we offer heartbeats which are no-ops.
+    replica.getConsensusMetrics().incHeartBeatTimeouts();
+    replica.offerEvent(HEARTBEAT_EVENT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ImmutableRaftContext.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ImmutableRaftContext.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ImmutableRaftContext.java
new file mode 100644
index 0000000..fdfa4bd
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ImmutableRaftContext.java
@@ -0,0 +1,147 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.log.CommitLogManagerInterface;
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.regionserver.RaftEventListener;
+
+import java.util.Map;
+
+/**
+ * Declares a set of immutable methods which can be used to make decisions
+ * during various events in the state machine.
+ */
+public interface ImmutableRaftContext {
+  /**
+   * Tells whether it is currently leader or not.
+   * @return
+   */
+  boolean isLeader();
+
+  /**
+   * Tells whether it is currently a candidate or not.
+   * @return
+   */
+  boolean isCandidate();
+
+  /**
+   * Tells whether it is currently a follower or not.
+   * @return
+   */
+  boolean isFollower();
+
+  /**
+   * Returns the current {term, index} for the state.
+   * @return
+   */
+  EditId getCurrentEdit();
+
+  /**
+   * Returns the last committed {term, index} for the state.
+   * @return
+   */
+  EditId getCommittedEdit();
+
+  /**
+   * Returns the last round's {term, index}.
+   * @return
+   */
+  EditId getPreviousEdit();
+
+  /**
+   * Get the current leader's information.
+   * @return
+   */
+  ConsensusHost getLeader();
+
+  /**
+   * Returns the ID for the current server
+   * @return
+   */
+  String getMyAddress();
+
+  /**
+   * Returns the majority cnt for the current quorum, including the current server
+   * @return
+   */
+  int getMajorityCnt();
+
+  /**
+   * Get the id of the last peer we voted for.
+   * @return
+   */
+  ConsensusHost getLastVotedFor();
+
+  /**
+   * Return the outstanding append session.
+   * @return
+   */
+  AppendConsensusSessionInterface getOutstandingAppendSession();
+
+  /**
+   * Return the outstanding append session if it matches the given edit.
+   * @param id
+   * @return
+   */
+  AppendConsensusSessionInterface getAppendSession(final EditId id);
+
+  /**
+   * Return the outstanding election session.
+   * @return
+   */
+  VoteConsensusSessionInterface getOutstandingElectionSession();
+
+  /**
+   * Return the outstanding election session if it matches the given edit.
+   * @param id
+   * @return
+   */
+  VoteConsensusSessionInterface getElectionSession(final EditId id);
+
+  /**
+   * Is transaction log accessible
+   */
+  boolean isLogAccessible();
+
+  int getRanking();
+
+  boolean validateLogEntry(final EditId id);
+
+  String getQuorumName();
+
+  Configuration getConf();
+
+  EditId getLastLogIndex();
+
+  void stop(boolean wait);
+
+  CommitLogManagerInterface getLogManager();
+
+  QuorumInfo getQuorumInfo();
+
+  RaftEventListener getDataStoreEventListener();
+
+  long getMinUnPersistedIndexAcrossQuorum();
+
+  ConsensusMetrics getConsensusMetrics();
+
+  Map<HServerAddress,Integer> getNewConfiguration();
+
+  QuorumMembershipChangeRequest getUpdateMembershipRequest();
+
+  long getPurgeIndex();
+
+  PeerStatus getStatus();
+
+  int getAppendEntriesMaxTries();
+
+  long getLastAppendRequestReceivedTime();
+
+  int getNumPendingEvents();
+
+  boolean isPartOfNewQuorum();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointAppendConsensusSession.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointAppendConsensusSession.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointAppendConsensusSession.java
new file mode 100644
index 0000000..23c77e0
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointAppendConsensusSession.java
@@ -0,0 +1,97 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+
+import java.util.Set;
+
+public class JointAppendConsensusSession implements AppendConsensusSessionInterface {
+
+  private AppendConsensusSession oldConfigSession;
+  private AppendConsensusSession newConfigSession;
+
+  public JointAppendConsensusSession(ImmutableRaftContext c,
+                                int majorityCount,
+                                final AppendRequest request,
+                                final ReplicateEntriesEvent event,
+                                final ConsensusMetrics metrics,
+                                final int rank,
+                                final boolean enableStepDownOnHigherRankCaughtUp,
+                                final int maxTries,
+                                final Set<String> oldPeers,
+                                final Set<String> newPeers) {
+    oldConfigSession = new AppendConsensusSession(c, majorityCount, request,
+      event, metrics, rank, enableStepDownOnHigherRankCaughtUp,
+            maxTries, oldPeers);
+    newConfigSession = new AppendConsensusSession(c, majorityCount, request,
+      event, metrics, rank, enableStepDownOnHigherRankCaughtUp,
+            maxTries, newPeers);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return oldConfigSession.isComplete() && newConfigSession.isComplete();
+  }
+
+  @Override
+  public SessionResult getResult() {
+    if (oldConfigSession.getResult() != newConfigSession.getResult()) {
+      return SessionResult.NOT_COMPLETED;
+    }
+
+    return newConfigSession.getResult();
+  }
+
+  @Override
+  public EditId getSessionId() {
+    return oldConfigSession.getSessionId();
+  }
+
+  @Override
+  public void incrementAck(final EditId id, final String address, final int rank,
+                           boolean canTakeover) {
+    oldConfigSession.incrementAck(id, address, rank, canTakeover);
+    newConfigSession.incrementAck(id, address, rank, canTakeover);
+  }
+
+  @Override
+  public void incrementHighTermCnt(final EditId id, final String address) {
+    oldConfigSession.incrementHighTermCnt(id, address);
+    newConfigSession.incrementHighTermCnt(id, address);
+  }
+
+  @Override
+  public void incrementLagCnt(final EditId id, final String address) {
+    oldConfigSession.incrementLagCnt(id, address);
+    newConfigSession.incrementLagCnt(id, address);
+  }
+
+  @Override
+  public ReplicateEntriesEvent getReplicateEntriesEvent() {
+    return oldConfigSession.getReplicateEntriesEvent();
+  }
+
+  @Override
+  public void reset() {
+    oldConfigSession.reset();
+    newConfigSession.reset();
+  }
+
+  @Override
+  public AppendRequest getAppendRequest() {
+    return oldConfigSession.getAppendRequest();
+  }
+
+  @Override
+  public void cancel() {
+    oldConfigSession.cancel();
+    newConfigSession.cancel();
+  }
+
+  @Override
+  public boolean isTimeout() {
+    return oldConfigSession.isTimeout() || newConfigSession.isTimeout();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointConsensusPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointConsensusPeerManager.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointConsensusPeerManager.java
new file mode 100644
index 0000000..a8830fa
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointConsensusPeerManager.java
@@ -0,0 +1,179 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServer;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerServerEventType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+/**
+ * Class to handle peers, leader elections, transactions while a quorum change is
+ * in progress. Particularly, it has the knowledge about both the old and the new
+ * config and makes sure, that all the decisions are made in agreement with both
+ * the configs.
+ *
+ * TODO: Get rid of oldPeerServers/newPeerServers notion.
+ */
+public class JointConsensusPeerManager extends AbstractPeerManager {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    JointConsensusPeerManager.class);
+
+  // Peers in the old config
+  private Map<String, PeerServer> oldPeerServers;
+
+  // Peers in the new config
+  private Map<String, PeerServer> newPeerServers;
+
+  // Contains all the peers involved in both the configs
+  private Map<String, PeerServer> allPeers;
+
+  private final QuorumInfo newConfig;
+
+  public JointConsensusPeerManager(final MutableRaftContext c,
+                                   final QuorumInfo newConfig) {
+    super(c);
+    this.newConfig = newConfig;
+    allPeers = new HashMap<>();
+  }
+
+  @Override
+  public void initializePeers() {
+
+    // Initialize the old peers
+    if (oldPeerServers == null) {
+      oldPeerServers = super.initializePeers(c.getQuorumInfo().getPeersWithRank());
+    }
+    allPeers.putAll(oldPeerServers);
+
+    // Initialize the new peers
+    if (newPeerServers == null) {
+      newPeerServers = new HashMap<>();
+      Map<HServerAddress, Integer> newPeers = new HashMap<>();
+
+      // There can be an overlap between the new and old configuration. Hence,
+      // we should initialize a peer only once. So, lets remove the peers which
+      // are already initialized.
+      newPeers.putAll(newConfig.getPeersWithRank());
+      Iterator<Map.Entry<HServerAddress, Integer>> newPeerIterator =
+        newPeers.entrySet().iterator();
+      HServerAddress peerAddress;
+      while (newPeerIterator.hasNext()) {
+        Map.Entry<HServerAddress, Integer> e = newPeerIterator.next();
+        peerAddress = e.getKey();
+        int newPeerRank = e.getValue();
+        String consensusServerAddress =
+          RaftUtil.getLocalConsensusAddress(peerAddress).getHostAddressWithPort();
+        if (oldPeerServers.get(consensusServerAddress) != null) {
+          PeerServer oldPeerServer = oldPeerServers.get(consensusServerAddress);
+          oldPeerServer.setRank(newPeerRank);
+          newPeerServers.put(consensusServerAddress, oldPeerServer);
+          oldPeerServers.remove(consensusServerAddress);
+          newPeerIterator.remove();
+        }
+      }
+
+      // Initialize the remaining peers
+      final Map<String, PeerServer> newServers = super.initializePeers(newPeers);
+      newPeerServers.putAll(newServers);
+      allPeers.putAll(newServers);
+    }
+  }
+
+  @Override
+  public void setPeerServers(Map<String, PeerServer> peers) {
+    oldPeerServers = peers;
+    newPeerServers = peers;
+  }
+
+  public void setOldPeerServers(Map<String, PeerServer> peers) {
+    oldPeerServers = peers;
+  }
+
+  public Map<String, PeerServer> getNewPeerServers() {
+    return newPeerServers;
+  }
+
+  @Override
+  public Map<String, PeerServer> getPeerServers() {
+    return allPeers;
+  }
+
+  @Override
+  public void resetPeers() {
+    super.resetPeers(allPeers);
+  }
+
+  @Override
+  public void setPeerReachable(String address) {
+
+    PeerServer server = null;
+    if ((server = allPeers.get(address)) != null) {
+      server.enqueueEvent(new Event(PeerServerEventType.PEER_REACHABLE));
+    }
+  }
+
+  @Override
+  public void sendVoteRequestToQuorum(VoteRequest request) {
+    super.broadcastVoteRequest(allPeers, request);
+  }
+
+  @Override
+  public void sendAppendRequestToQuorum(AppendRequest request) {
+    LOG.info("Sending an appendRequest to quorum " + c.getQuorumName() +
+      " via the JointConsensusPeerManager ");
+    super.broadcastAppendRequest(allPeers, request);
+  }
+
+  @Override
+  public void stop() {
+    super.stop(allPeers);
+  }
+
+  public void stopOldPeers() {
+    for (String peer : oldPeerServers.keySet()) {
+      if (newPeerServers.get(peer) == null) {
+        oldPeerServers.get(peer).stop();
+      }
+    }
+  }
+
+  @Override
+  public String getState() {
+   return super.getState(allPeers);
+  }
+
+  @Override
+  public AppendConsensusSessionInterface createAppendConsensusSession(
+          int majorityCount, AppendRequest request, ReplicateEntriesEvent event,
+          ConsensusMetrics metrics, int rank,
+          boolean enableStepDownOnHigherRankCaughtUp) {
+    return new JointAppendConsensusSession(c, majorityCount, request, event,
+      metrics, rank, enableStepDownOnHigherRankCaughtUp,
+      c.getAppendEntriesMaxTries(), c.getQuorumInfo().getPeersAsString(),
+      newConfig.getPeersAsString());
+  }
+
+  @Override
+  public VoteConsensusSessionInterface createVoteConsensusSession(
+    int majorityCount, VoteRequest request, ConsensusMetrics metrics) {
+    return new JointVoteConsensusSession(majorityCount, request, metrics, c.getQuorumInfo().getPeersAsString(),
+      newConfig.getPeersAsString());
+  }
+
+  public List<QuorumInfo> getConfigs() {
+    return Arrays.asList(c.getQuorumInfo(), newConfig);
+  }
+
+  @Override
+  public boolean isInJointQuorumMode() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointVoteConsensusSession.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointVoteConsensusSession.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointVoteConsensusSession.java
new file mode 100644
index 0000000..fb3abe3
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/JointVoteConsensusSession.java
@@ -0,0 +1,56 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+
+import java.util.Set;
+
+public class JointVoteConsensusSession implements VoteConsensusSessionInterface {
+  VoteConsensusSession oldConfigSession;
+  VoteConsensusSession newConfigSession;
+
+  public JointVoteConsensusSession(int majorityCount, final VoteRequest request,
+                                   final ConsensusMetrics metrics,
+                                   final Set<String> oldPeers,
+                                   final Set<String> newPeers) {
+    oldConfigSession = new VoteConsensusSession(majorityCount, request, metrics, oldPeers);
+    newConfigSession = new VoteConsensusSession(majorityCount, request, metrics, newPeers);
+  }
+
+
+  @Override public void incrementAck(EditId id, String address) {
+    oldConfigSession.incrementAck(id, address);
+    newConfigSession.incrementAck(id, address);
+  }
+
+  @Override public void incrementNack(EditId id, String address) {
+    oldConfigSession.incrementNack(id, address);
+    newConfigSession.incrementNack(id, address);
+  }
+
+  @Override public void setVoteSessionFailed(EditId id) {
+    oldConfigSession.setVoteSessionFailed(id);
+    newConfigSession.setVoteSessionFailed(id);
+  }
+
+  @Override public VoteRequest getRequest() {
+    return oldConfigSession.getRequest();
+  }
+
+  @Override public boolean isComplete() {
+    return oldConfigSession.isComplete() && newConfigSession.isComplete();
+  }
+
+  @Override public SessionResult getResult() {
+    if (oldConfigSession.getResult() != newConfigSession.getResult()) {
+      return SessionResult.NOT_COMPLETED;
+    }
+
+    return oldConfigSession.getResult();
+  }
+
+  @Override public EditId getSessionId() {
+    return oldConfigSession.getSessionId();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/MutableRaftContext.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/MutableRaftContext.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/MutableRaftContext.java
new file mode 100644
index 0000000..9802d21
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/MutableRaftContext.java
@@ -0,0 +1,125 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.util.Arena;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * Set of methods used by the states in the raft protocol to update the state
+ * machine on various events;
+ */
+public interface MutableRaftContext extends ImmutableRaftContext {
+  /**
+   * Updates the current edit
+   * @param id
+   */
+  void setCurrentEditId(final EditId id);
+
+  /**
+   * Updates the last committed edit id.
+   * @param id
+   */
+  void advanceCommittedIndex(final EditId id);
+
+  /**
+   * Set the leader id.
+   * @param hostId
+   */
+  void setLeader(final ConsensusHost hostId);
+
+  /**
+   * Updates the last committed edit id.
+   * @param id
+   */
+  void setPreviousEditId(final EditId id);
+
+  /**
+   * Set the last voted for id.
+   * @param hostId
+   */
+  ListenableFuture<Void> setVotedFor(final ConsensusHost hostId);
+
+  /**
+   * clear the leader id.
+   */
+  void clearLeader();
+
+  /**
+   * Clear the last voted for id.
+   */
+  void clearVotedFor();
+
+  void appendToLog(final EditId currLogId, final long commitIndex,
+                   final ByteBuffer data);
+
+  void setElectionSession(VoteConsensusSessionInterface session);
+
+  void setAppendSession(AppendConsensusSessionInterface session);
+
+  void sendVoteRequestToQuorum(VoteRequest request);
+
+  void truncateLogEntries(final EditId lastValidEntryId) throws IOException;
+  boolean offerEvent(final Event e);
+
+  Timer getHeartbeatTimer();
+
+  Timer getProgressTimer();
+
+  ListenableFuture<?> sendAppendRequest(ReplicateEntriesEvent event);
+
+  void setLastAppendRequestReceivedTime(long timeMillis);
+
+  ListenableFuture<?> sendEmptyAppendRequest();
+
+  void leaderStepDown();
+
+  void candidateStepDown();
+
+  void resendOutstandingAppendRequest();
+
+  void resetPeers();
+
+  void setPeerReachable(String address);
+
+  String getLeaderNotReadyMsg();
+
+  void updatePeerAckedId(String address, EditId remoteEdit);
+
+  void setMinAckedIndexAcrossAllPeers(long index);
+
+  void setUpdateMembershipRequest(
+    QuorumMembershipChangeRequest request);
+
+  PeerManagerInterface getPeerManager();
+
+  HServerAddress getServerAddress();
+
+  void updateToJointQuorumMembership(final QuorumInfo config) throws IOException;
+
+  void updateToNewQuorumMembership(final QuorumInfo config)
+    throws IOException;
+
+  void handleQuorumChangeRequest(final ByteBuffer buffer) throws IOException;
+
+  Arena getArena();
+
+  void reseedStartIndex(long index) throws IOException;
+
+  void setQuorumInfo(final QuorumInfo update);
+
+  void cleanUpJointStates();
+
+  boolean canTakeOver();
+
+  ExecutorService getExecServiceForThriftClients();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/PeerManagerInterface.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/PeerManagerInterface.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/PeerManagerInterface.java
new file mode 100644
index 0000000..536a4ae
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/PeerManagerInterface.java
@@ -0,0 +1,39 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServer;
+
+import java.util.List;
+import java.util.Map;
+
+public interface PeerManagerInterface {
+  void initializePeers();
+  void setPeerServers(final Map<String, PeerServer> peers);
+  Map<String, PeerServer> getPeerServers();
+  void resetPeers();
+  void setPeerReachable(String address);
+  void sendVoteRequestToQuorum(VoteRequest request);
+  void sendAppendRequestToQuorum(AppendRequest request);
+  void updatePeerAckedId(String address, EditId remoteEdit);
+  long getMinUnPersistedIndexAcrossQuorum();
+  void setMinAckedIndexAcrossAllPeers(long index);
+  void stop();
+  String getState();
+  AppendConsensusSessionInterface createAppendConsensusSession(int majorityCount,
+                                                     final AppendRequest request,
+                                                     final ReplicateEntriesEvent event,
+                                                     final ConsensusMetrics metrics,
+                                                     final int rank,
+                                                     final boolean enableStepDownOnHigherRankCaughtUp);
+  VoteConsensusSessionInterface createVoteConsensusSession(
+    int majorityCount,
+    final VoteRequest request,
+    final ConsensusMetrics metrics);
+
+  List<QuorumInfo> getConfigs();
+  boolean isInJointQuorumMode();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ProgressTimeoutCallback.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ProgressTimeoutCallback.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ProgressTimeoutCallback.java
new file mode 100644
index 0000000..9cabd76
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ProgressTimeoutCallback.java
@@ -0,0 +1,32 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.raft.events.ProgressTimeoutEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Date;
+
+public class ProgressTimeoutCallback implements TimeoutEventHandler {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    ProgressTimeoutCallback.class);
+  private final RaftQuorumContext replica;
+
+  public ProgressTimeoutCallback(final RaftQuorumContext replica) {
+    this.replica = replica;
+  }
+
+  public void onTimeout() {
+    LOG.info(replica + " has a progress timeout! " +
+      " current edit: " +
+      replica.getCurrentEdit() + ", Last AppendRequest was received at : " +
+      new Date(replica.getLastAppendRequestReceivedTime()));
+
+    if (System.currentTimeMillis() - replica.getLastAppendRequestReceivedTime() >=
+      replica.getProgressTimeoutForMeMillis()) {
+      replica.getConsensusMetrics().incProgressTimeouts();
+      replica.offerEvent(new ProgressTimeoutEvent());
+    } else {
+      LOG.info(replica + " Ignoring the progress timer.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java
new file mode 100644
index 0000000..8120dd2
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumAgent.java
@@ -0,0 +1,463 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.NoLeaderForRegionException;
+import org.apache.hadoop.hbase.conf.ConfigurationObserver;
+import org.apache.hadoop.hbase.consensus.exceptions.CommitQueueOverloadedException;
+import org.apache.hadoop.hbase.consensus.exceptions.NewLeaderException;
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.metrics.TimeStat;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractWAL;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+/**
+ * This is an agent that runs on the RaftQuorumContext side, and it is
+ * responsible for getting entries committed within a particular time window.
+ *
+ * There is a continuously running 'WAL Syncer' task, which will take a list
+ * of WALEdits to sync on the quorum.
+ */
+public class QuorumAgent implements ConfigurationObserver {
+  private static final Logger LOG = LoggerFactory.getLogger(QuorumAgent.class);
+
+  private volatile RaftQuorumContext context;
+
+  /** The period of time the agent will wait for a commit to replicate. */
+  private volatile long commitTimeoutInMillis;
+
+  /** The period of time after which the syncCommit method is to return. */
+  private volatile long commitDeadlineInMillis;
+
+  /**
+   * The maximum number of entries allowed in the commit queue. Appends beyond
+   * this number will be fast failed until the queue is drained.
+   */
+  private volatile long commitQueueEntriesLimit;
+
+  /**
+   * The maximum size of the commit queue in KB. Appends will be fast failed
+   * once the queue reaches this limit until it is drained.
+   */
+  private volatile long commitQueueSizeLimit;
+
+  /** The interval between retries */
+  private volatile long sleepIntervalInMillis;
+
+  private final Compression.Algorithm compressionCodec;
+
+  // Lock to guarantee the ordering of log entries in WAL
+  private final ReentrantLock appendLock = new ReentrantLock(true);
+  private final Condition groupCommitBuffer = appendLock.newCondition();
+
+  private LinkedList<WALEdit> currentList = new LinkedList<>();
+  private LinkedList<WALEdit> syncList = new LinkedList<>();
+
+  private SettableFuture<Long> currentResult;
+  private SettableFuture<Long> futureResult = SettableFuture.create();
+
+  private final ExecutorService executor;
+  private volatile boolean isSyncStopped = false;
+
+  private volatile long lastSequenceID = -1;
+
+  private Random random = new Random();
+
+  public QuorumAgent(RaftQuorumContext context) {
+    this.context = context;
+    Configuration conf = context.getConf();
+
+    commitQueueEntriesLimit = conf.getLong(
+      HConstants.QUORUM_AGENT_COMMIT_QUEUE_ENTRIES_LIMIT_KEY,
+      HConstants.QUORUM_AGENT_COMMIT_QUEUE_ENTRIES_LIMIT_DEFAULT);
+    commitQueueSizeLimit = conf.getLong(
+      HConstants.QUORUM_AGENT_COMMIT_QUEUE_SIZE_LIMIT_KEY,
+      HConstants.QUORUM_AGENT_COMMIT_QUEUE_SIZE_LIMIT_DEFAULT);
+    commitTimeoutInMillis = conf.getLong(
+      HConstants.QUORUM_AGENT_COMMIT_TIMEOUT_KEY,
+      HConstants.QUORUM_AGENT_COMMIT_TIMEOUT_DEFAULT);
+    commitDeadlineInMillis = conf.getLong(
+      HConstants.QUORUM_AGENT_COMMIT_DEADLINE_KEY,
+      HConstants.QUORUM_AGENT_COMMIT_DEADLINE_DEFAULT);
+    sleepIntervalInMillis = conf.getLong(
+      HConstants.QUORUM_CLIENT_SLEEP_INTERVAL_KEY,
+      HConstants.QUORUM_CLIENT_SLEEP_INTERVAL_DEFAULT);
+
+    compressionCodec = Compression.getCompressionAlgorithmByName(conf.get(
+      HConstants.CONSENSUS_TRANSACTION_LOG_COMPRESSION_CODEC_KEY,
+      HConstants.CONSENSUS_TRANSACTION_LOG_COMPRESSION_CODEC_DEFAULT));
+
+    executor = Executors.newSingleThreadExecutor(
+      new DaemonThreadFactory("Quorum-Syncer-"+ context.getQuorumName() + "-"));
+    submitWALSyncerTask();
+  }
+
+  private void setCommitQueueLimits(final Configuration conf) {
+    long newCommitQueueEntriesLimit = conf.getLong(
+      HConstants.QUORUM_AGENT_COMMIT_QUEUE_ENTRIES_LIMIT_KEY,
+      HConstants.QUORUM_AGENT_COMMIT_QUEUE_ENTRIES_LIMIT_DEFAULT);
+    long newCommitQueueSizeLimit = conf.getLong(
+      HConstants.QUORUM_AGENT_COMMIT_QUEUE_SIZE_LIMIT_KEY,
+      HConstants.QUORUM_AGENT_COMMIT_QUEUE_SIZE_LIMIT_DEFAULT);
+
+    if (commitQueueEntriesLimit != newCommitQueueEntriesLimit) {
+      commitQueueEntriesLimit = newCommitQueueEntriesLimit;
+      LOG.debug("Set commit queue entries limit for region: %s, entries: %d",
+        getRaftQuorumContext().getQuorumName(), commitQueueEntriesLimit);
+    }
+    if (commitQueueSizeLimit != newCommitQueueSizeLimit) {
+      commitQueueSizeLimit = newCommitQueueSizeLimit;
+      LOG.debug("Set commit queue size limit for region: %s, size: %d",
+        getRaftQuorumContext().getQuorumName(), commitQueueSizeLimit);
+    }
+  }
+
+  private void setCommitQueueTimings(final Configuration conf) {
+    long newCommitTimeoutInMillis = conf.getLong(
+      HConstants.QUORUM_AGENT_COMMIT_TIMEOUT_KEY,
+      HConstants.QUORUM_AGENT_COMMIT_TIMEOUT_DEFAULT);
+    long newCommitDeadlineInMillis = conf.getLong(
+      HConstants.QUORUM_AGENT_COMMIT_DEADLINE_KEY,
+      HConstants.QUORUM_AGENT_COMMIT_DEADLINE_DEFAULT);
+    long newSleepIntervalInMillis = conf.getLong(
+      HConstants.QUORUM_CLIENT_SLEEP_INTERVAL_KEY,
+      HConstants.QUORUM_CLIENT_SLEEP_INTERVAL_DEFAULT);
+
+    if (commitTimeoutInMillis != newCommitTimeoutInMillis) {
+      commitTimeoutInMillis = newCommitTimeoutInMillis;
+      LOG.debug("Set commit timeout for region: %s, %d ms",
+        getRaftQuorumContext().getQuorumName(), commitTimeoutInMillis);
+    }
+    if (commitDeadlineInMillis != newCommitDeadlineInMillis) {
+      commitDeadlineInMillis = newCommitDeadlineInMillis;
+      LOG.debug("Set commit deadline for region: %s, %d ms",
+        getRaftQuorumContext().getQuorumName(), commitDeadlineInMillis);
+    }
+    if (sleepIntervalInMillis != newSleepIntervalInMillis) {
+      sleepIntervalInMillis = newSleepIntervalInMillis;
+      LOG.debug("Set commit sleep interval for region: %s, %d ms",
+        getRaftQuorumContext().getQuorumName(), sleepIntervalInMillis);
+    }
+  }
+
+  public RaftQuorumContext getRaftQuorumContext() {
+    return this.context;
+  }
+
+  @Override
+  public void notifyOnChange(Configuration conf) {
+    setCommitQueueLimits(conf);
+    setCommitQueueTimings(conf);
+  }
+
+  private long getBackoffTimeMillis() {
+    double p50 = getRaftQuorumContext().getConsensusMetrics().
+      getAppendEntriesLatency().getP50();
+    // Get a random number of microseconds, up to half the p50.
+    int randomMicros = random.nextInt() % (int) (p50 / 2.0);
+    return (long) Math.max(1.0, (p50 + randomMicros) / 1000.0);
+  }
+
+  public boolean isLeader() {
+    return context.isLeader();
+  }
+
+  public String getPath() {
+    return context.getLogManager().getPath();
+  }
+
+  private void checkBeforeCommit() throws IOException {
+    // check whether the current peer is the leader
+    if (!isLeader()) {
+      throw new NoLeaderForRegionException("Current region server " +
+        context.getMyAddress() +
+        " is not the leader for the region " + context.getQuorumName());
+    }
+
+    if (this.isSyncStopped) {
+      throw new IOException("QuorumWAL syncer thread for " + context.getQuorumName() +
+        " has been stopped !");
+    }
+  }
+
+  private ListenableFuture<Long> internalCommit(List<WALEdit> edits)
+    throws IOException {
+    SettableFuture<Long> future = null;
+
+    // Add the transaction into the group commit queue
+    this.appendLock.lock();
+    try {
+      if (isSyncStopped) {
+        throw new IOException("QuorumWAL syncer thread for " +
+          context.getQuorumName() + " has been stopped!");
+      }
+      if (currentList.size() > commitQueueEntriesLimit) {
+        getRaftQuorumContext().getConsensusMetrics()
+          .incCommitQueueEntriesLimitExceeded();
+        throw new CommitQueueOverloadedException(String.format(
+          "Exceeded entries limit for region: %s, limit: %d, entries: %d",
+          context.getQuorumName(), commitQueueEntriesLimit,
+          currentList.size()), getBackoffTimeMillis());
+      }
+
+      currentList.addAll(edits);
+
+      this.groupCommitBuffer.signal();
+      future = futureResult;
+    } finally {
+      this.appendLock.unlock();
+    }
+    return future;
+  }
+
+  private ListenableFuture<Long> internalCommit(WALEdit edits)
+          throws IOException {
+    return internalCommit(Arrays.asList(edits));
+  }
+
+  /**
+   * Append to the log synchronously.
+   * @param edits WALEdit to append.
+   * @return The commit index of the committed edit.
+   * @throws IOException
+   */
+  public long syncAppend(WALEdit edits) throws IOException {
+    checkBeforeCommit();
+
+    // increase the write size
+    AbstractWAL.getWriteSizeHistogram().addValue(edits.getTotalKeyValueLength());
+
+    long start = System.nanoTime();
+    ListenableFuture<Long> future = internalCommit(edits);
+    // Wait for the group commit finish;
+    try {
+      // Wait for the transaction to complete
+      long seq = future.get();
+      // increase the sync time
+      double syncMicros = (System.nanoTime() - start) / 1000.0;
+      getRaftQuorumContext().getConsensusMetrics().getFsSyncLatency()
+        .add((long)syncMicros, TimeUnit.MICROSECONDS);
+      AbstractWAL.getSyncTimeHistogram().addValue(syncMicros);
+      return seq;
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Append to the log asynchronously.
+   * @param edits
+   * @return Future containing the commit index.
+   * @throws IOException
+   */
+  public ListenableFuture<Long> asyncAppend(WALEdit edits) throws IOException {
+    checkBeforeCommit();
+    return internalCommit(edits);
+  }
+
+  /**
+   * Same as asyncAppend(WALEdit), but for a list of WALEdit.
+   * @param edits
+   * @return The commit index for the list of WALEdits
+   * @throws IOException
+   */
+  public ListenableFuture<Long> asyncAppend(List<WALEdit> edits) throws IOException {
+    checkBeforeCommit();
+    return internalCommit(edits);
+  }
+
+  public long getLastSequenceID() {
+    return lastSequenceID;
+  }
+
+  /**
+   * Stop the RaftQuorumContext. Called by an external RegionClosing thread pool
+   */
+  public void close() {
+    this.isSyncStopped = true;
+    this.executor.shutdown();
+    context.stop(true);
+
+    appendLock.lock();
+    try {
+      if (futureResult != null) {
+        futureResult.setException(new IOException("WAL already closed"));
+      }
+      if (currentResult != null) {
+        currentResult.setException(new IOException("WAL already closed"));
+      }
+    } finally {
+      appendLock.unlock();
+    }
+  }
+
+  /**
+   * A blocking API to replicate the entries to the quorum. It waits until the
+   * commit is succeeded or failed. This method is guaranteed to allow a
+   * 'commitTimeoutInMillis' period for the commit to complete while maintaining
+   * a strict deadline of 'commitDeadlineInMillis' on when this method
+   * completes.
+   *
+   * @param entries The list of the @WALEdit to replicate
+   * @return the commit index of the replicated entries.
+   * @throws IOException if the quorum threw an exception during the replication
+   */
+  private long syncCommit(List<WALEdit> entries,
+                          final SettableFuture<Long> result) throws Exception {
+    ByteBuffer serializedEntries;
+    ConsensusMetrics metrics = getRaftQuorumContext().getConsensusMetrics();
+
+    try (TimeStat.BlockTimer latency =
+                 metrics.getLogSerializationLatency().time()) {
+      serializedEntries = WALEdit.serializeToByteBuffer(entries,
+              System.currentTimeMillis(), compressionCodec);
+    }
+    int appendEntriesSize = WALEdit.getWALEditsSize(entries);
+    metrics.getAppendEntriesSize().add(appendEntriesSize);
+    metrics.getAppendEntriesBatchSize().add(entries.size());
+    if (!compressionCodec.equals(Compression.Algorithm.NONE)) {
+      int compressedSize = serializedEntries.remaining() -
+              WALEdit.PAYLOAD_HEADER_SIZE;
+      metrics.getAppendEntriesCompressedSize().add(compressedSize);
+    } else {
+      // We don't use any compression, so the compressed size would be the
+      // same as the original size.
+      metrics.getAppendEntriesCompressedSize().add(appendEntriesSize);
+    }
+
+      if (!context.isLeader()) {
+        ConsensusHost leader = context.getLeader();
+        throw new NewLeaderException(
+                leader == null ? "No leader" : leader.getHostId());
+      }
+
+      ReplicateEntriesEvent event = new ReplicateEntriesEvent(false,
+              serializedEntries, result);
+      if (!context.offerEvent(event)) {
+        ConsensusHost leader = context.getLeader();
+        throw new NewLeaderException(
+                leader == null ? "No leader" : leader.getHostId());
+      }
+      try {
+        return result.get(commitDeadlineInMillis, TimeUnit.MILLISECONDS);
+      } catch (Throwable e) {
+        if (e instanceof TimeoutException) {
+          metrics.incAppendEntriesMissedDeadline();
+          LOG.warn(String.format(
+                  "%s Failed to commit within the deadline of %dms", context,
+                  commitDeadlineInMillis));
+          throw e;
+        } else {
+          LOG.error(context + " Quorum commit failed", e);
+          throw new Exception("Quorum commit failed because " + e);
+        }
+      }
+  }
+
+  private void submitWALSyncerTask() {
+    executor.submit(new Runnable() {
+      @Override
+      public void run() {
+        while (!isSyncStopped) {
+          try {
+            SettableFuture<Long> nextResult = SettableFuture.create();
+            // Switch the current list under the appendLock
+            appendLock.lock();
+            try {
+              if (isSyncStopped) {
+                throw new IOException("QuorumWAL syncer thread for " +
+                  context.getQuorumName() + " has been stopped !");
+              }
+
+              if (currentList.isEmpty()) {
+                // wake up every 100ms to check if sync thread has to shut down
+                groupCommitBuffer.await(100, TimeUnit.MILLISECONDS);
+              }
+
+              if (!currentList.isEmpty()) {
+                // switch the buffer
+                assert syncList.isEmpty();
+                LinkedList<WALEdit> tmp = syncList;
+                syncList = currentList;
+                currentList = tmp;
+
+                // Create a new futureResult for the next queue
+                currentResult = futureResult;
+                futureResult = nextResult;
+              } else {
+                continue;
+              }
+
+            } catch (Exception e) {
+              // unexpected exception
+            } finally {
+              appendLock.unlock();
+            }
+
+            // Group commit to the quorum
+            long groupCommitID;
+            long start = System.nanoTime();
+            try {
+              groupCommitID = syncCommit(syncList, currentResult);
+
+              // Set the last commitID
+              assert groupCommitID > lastSequenceID;
+              lastSequenceID = groupCommitID;
+
+            } catch (Throwable e) {
+              // Signal all the rpc threads with the exception
+              currentResult.setException(e);
+            } finally {
+              // Clear the sync buffer
+              syncList.clear();
+            }
+            // Add the group sync time
+            double gsyncMicros = (System.nanoTime() - start) / 1000.0;
+            getRaftQuorumContext().getConsensusMetrics().getFsGSyncLatency()
+              .add((long) gsyncMicros, TimeUnit.MICROSECONDS);
+            AbstractWAL.getGSyncTimeHistogram().addValue(gsyncMicros);
+          } catch (Throwable e) {
+            LOG.error("Unexpected exception: ", e);
+          }
+        }
+      }
+    });
+  }
+
+  public void setLastSequenceID(long seqid)
+    throws IOException, ExecutionException, InterruptedException {
+    lastSequenceID = seqid;
+    context.reseedIndex(seqid);
+  }
+
+  public long getLastCommittedIndex() {
+    return context.getLogManager().getLastValidTransactionId().getIndex();
+  }
+
+  public Compression.Algorithm getCompressionCodec() {
+    return compressionCodec;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumInfo.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumInfo.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumInfo.java
new file mode 100644
index 0000000..66e5406
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumInfo.java
@@ -0,0 +1,357 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.util.*;
+import java.nio.ByteBuffer;
+
+public class QuorumInfo {
+  public static final int PAYLOAD_HEADER_SIZE =
+          + Bytes.SIZEOF_BYTE   // Magic value
+          + Bytes.SIZEOF_BYTE   // Payload type
+          + Bytes.SIZEOF_BYTE;  // Payload version
+
+  private Map<String, Map<HServerAddress, Integer>> peers = null;
+  private Map<HServerAddress, Integer> peersWithRank = null;
+  private Set<String> peersAsString = null;
+  private final String quorumName;
+
+  public QuorumInfo(final Map<String, Map<HServerAddress, Integer>> peers,
+                    final String quorumName) {
+    this.peers = peers;
+    this.quorumName = quorumName;
+    populateInternalMaps();
+  }
+
+  public QuorumInfo(final QuorumInfo info) {
+    this.quorumName = info.quorumName;
+    peers = new HashMap<>();
+    for (String domain : info.getPeers().keySet()) {
+      final Map<HServerAddress, Integer> peersInDomain = new HashMap<>();
+      for (HServerAddress peer : info.getPeers().get(domain).keySet()) {
+        peersInDomain.put(new HServerAddress(peer.getHostname(), peer.getPort()),
+          info.getPeersWithRank().get(peer));
+      }
+      peers.put(domain, peersInDomain);
+    }
+    populateInternalMaps();
+  }
+
+  public int getQuorumSize() {
+    return (peers == null  ? 0 : peers.values().iterator().next().size());
+  }
+
+  public Map<HServerAddress,Integer> getPeersWithRank() {
+    return peersWithRank;
+  }
+
+  public Set<String> getPeersAsString() {
+    return peersAsString;
+  }
+
+  public Map<HServerAddress, String> getPeersWithCluster() {
+    if (peers != null) {
+      // TODO: Consider cache this map instead of computing it every time
+      Map<HServerAddress, String> peersWithCluster = new TreeMap<HServerAddress, String>();
+      for (Map.Entry<String, Map<HServerAddress, Integer>> entry : peers
+        .entrySet()) {
+        String cluster = entry.getKey();
+        for (HServerAddress serverAddress : entry.getValue().keySet()) {
+          peersWithCluster.put(serverAddress, cluster);
+        }
+      }
+      return peersWithCluster;
+    }
+    return null;
+  }
+
+  public Map<String, Map<HServerAddress, Integer>> getPeers() {
+    return peers;
+  }
+
+  public void setPeers(Map<String, Map<HServerAddress, Integer>> peers) {
+    this.peers = peers;
+    populateInternalMaps();
+  }
+
+  public String getQuorumName() {
+    return quorumName;
+  }
+
+  public static ByteBuffer serializeToBuffer(final List<QuorumInfo> configs) {
+    final ByteBuffer payload = ByteBuffer.allocate(
+      getPayloadSize(configs) + PAYLOAD_HEADER_SIZE);
+
+    // Write the MAGIC VALUE
+    payload.put(HConstants.CONSENSUS_PAYLOAD_MAGIC_VALUE);
+
+    // Write that the payload is Quorum Membership Change
+    payload.put(HConstants.QUORUM_MEMBERSHIP_CHANGE_TYPE);
+
+    // Write the version of Quorum Membership Change
+    payload.put(HConstants.QUORUM_MEMBERSHIP_CHANGE_VERSION);
+
+    // Write the total number of WALEdits
+    payload.putInt(configs.size());
+
+    byte[] quorumName, dcName, currPeerInfo = null;
+    for (QuorumInfo s : configs) {
+      // Quorum Name
+      quorumName = s.getQuorumName().getBytes();
+      payload.putInt(quorumName.length);
+      payload.put(quorumName);
+
+      // Num of DC's
+      payload.putInt(s.getPeers().size());
+      for (String dc : s.getPeers().keySet()) {
+        // DC Name
+        dcName = dc.getBytes();
+        payload.putInt(dcName.length);
+        payload.put(dcName);
+
+        Set<HServerAddress> numPeers = s.getPeers().get(dc).keySet();
+
+        // Number of peers
+        payload.putInt(numPeers.size());
+
+        for (HServerAddress peer : numPeers) {
+          // Peer Info
+          currPeerInfo = peer.getHostAddressWithPort().getBytes();
+          payload.putInt(currPeerInfo.length);
+
+          payload.put(currPeerInfo);
+
+          // Peer Rank
+          payload.putInt(s.getPeers().get(dc).get(peer));
+        }
+      }
+    }
+
+    payload.flip();
+    return payload;
+  }
+
+  /**
+   * This method reads the ByteBuffer and returns valid List of QuorumInfo objects.
+   * This method assumes that the contents of the ByteBuffer are immutable.
+   *
+   * This method does not modify the members of the ByteBuffer like position,
+   * limit, mark and capacity. It should be thread safe.
+   * @param data
+   * @return
+   */
+  public static List<QuorumInfo> deserializeFromByteBuffer(final ByteBuffer data) {
+    if (!isQuorumChangeRequest(data)) {
+      return null;
+    }
+
+    // The check above already read the magic value and type fields, so move on
+    // to the version field.
+    int currOffset = data.position() + Bytes.SIZEOF_BYTE + Bytes.SIZEOF_BYTE;
+
+    // Read the version
+    if (data.get(currOffset) != HConstants.QUORUM_MEMBERSHIP_CHANGE_VERSION) {
+      return null;
+    }
+    currOffset += Bytes.SIZEOF_BYTE;
+
+    int numConfigs = data.getInt(currOffset);
+
+    currOffset += Bytes.SIZEOF_INT;
+    List<QuorumInfo> configs = new ArrayList<>();
+
+    int numDCs, numPeers, quorumNameLength, dcNameLength, peerNameLength = 0;
+    String quorumName, dcName, peerName;
+    Map<String, Map<HServerAddress, Integer>> dcLevelInfo;
+    Map<HServerAddress, Integer> perDCPeersMap;
+
+    for (int confIndex = 0; confIndex < numConfigs; ++confIndex) {
+
+      // Quorum Name
+      quorumNameLength = data.getInt(currOffset);
+      currOffset += Bytes.SIZEOF_INT;
+
+      quorumName = new String(data.array(), data.arrayOffset() + currOffset,
+        quorumNameLength);
+      currOffset += quorumNameLength;
+
+      numDCs = data.getInt(currOffset);
+      currOffset += Bytes.SIZEOF_INT;
+
+      // Initialize the dc map
+      dcLevelInfo = new HashMap<>(numDCs);
+
+      for (int dcIndex = 0; dcIndex < numDCs; ++dcIndex) {
+
+        // DC Name
+        dcNameLength = data.getInt(currOffset);
+        currOffset += Bytes.SIZEOF_INT;
+
+        dcName = new String(data.array(), data.arrayOffset() + currOffset,
+          dcNameLength);
+        currOffset += dcNameLength;
+
+        // Num of peers in this DC
+        numPeers = data.getInt(currOffset);
+        currOffset += Bytes.SIZEOF_INT;
+
+        // Initialize the peerMap
+        perDCPeersMap = new HashMap<>(numPeers);
+        for (int peerIndex = 0; peerIndex < numPeers; ++peerIndex) {
+          // Peer Name
+          peerNameLength = data.getInt(currOffset);
+          currOffset += Bytes.SIZEOF_INT;
+
+          peerName = new String(data.array(), data.arrayOffset() + currOffset, peerNameLength);
+          currOffset += peerNameLength;
+
+          // Put the peer name and rank in the peer Map
+          perDCPeersMap.put(new HServerAddress(peerName), data.getInt(currOffset));
+          currOffset += Bytes.SIZEOF_INT;
+        }
+
+        // add the dc info to map
+        dcLevelInfo.put(dcName, perDCPeersMap);
+      }
+
+      // add the config to the list of configs to return
+      configs.add(new QuorumInfo(dcLevelInfo, quorumName));
+    }
+    return configs;
+  }
+
+  public static int getPayloadSize(final List<QuorumInfo> configs) {
+    // Number of Lists
+    int size = Bytes.SIZEOF_INT;
+
+    for (QuorumInfo s : configs) {
+      // Quorum Name length
+      size += Bytes.SIZEOF_INT;
+      size += s.getQuorumName().length();
+
+      // Num of DC's
+      size += Bytes.SIZEOF_INT;
+
+      for (String dc : s.getPeers().keySet()) {
+        // DC Name length
+        size += Bytes.SIZEOF_INT;
+        size += dc.getBytes().length;
+
+        Set<HServerAddress> numPeers = s.getPeers().get(dc).keySet();
+
+        // Number of peers
+        size += Bytes.SIZEOF_INT;
+
+        for (HServerAddress peer : numPeers) {
+          // Peer Address in String format
+          size += Bytes.SIZEOF_INT;
+          size += peer.getHostAddressWithPort().length();
+          // Peer Rank
+          size += Bytes.SIZEOF_INT;
+        }
+      }
+    }
+    return size;
+  }
+
+  /**
+   * Test whether the given buffer contains a quorum change request. This method
+   * does not change the position pointer while reading the buffer data.
+   * @param data buffer containing a possible quorum change request
+   * @return true if the buffer contains a change request, false otherwise
+   */
+  public static boolean isQuorumChangeRequest(final ByteBuffer data) {
+    int currOffset = data.position();
+
+    // Read the Magic Value
+    if (data.remaining() < PAYLOAD_HEADER_SIZE ||
+      data.get(currOffset) != HConstants.CONSENSUS_PAYLOAD_MAGIC_VALUE) {
+      return false;
+    }
+    currOffset += Bytes.SIZEOF_BYTE;
+
+    // Read the type
+    if (data.get(currOffset) != HConstants.QUORUM_MEMBERSHIP_CHANGE_TYPE) {
+      return false;
+    }
+    return true;
+  }
+
+  public void refresh() {
+    populateInternalMaps();
+  }
+
+  private void populateInternalMaps() {
+    if (peers != null) {
+      peersAsString = new HashSet<>();
+      peersWithRank = new TreeMap<>();
+      for (Map<HServerAddress, Integer> map : peers.values()) {
+        peersWithRank.putAll(map);
+        for (HServerAddress peer : map.keySet()) {
+          peersAsString.add(RaftUtil.getLocalConsensusAddress(peer).getHostAddressWithPort());
+        }
+      }
+    }
+  }
+
+  public String getDomain(final String serverAddr) {
+    String domain = "";
+    for (String c : peers.keySet()) {
+      for (HServerAddress peer : peers.get(c).keySet()) {
+        if (serverAddr.equals(peer.getHostAddressWithPort())) {
+          domain = c;
+          break;
+        }
+      }
+    }
+    return domain;
+  }
+
+  public int getRank(final HServerAddress address) {
+    int rank = 0;
+    if (peersWithRank.containsKey(address)) {
+      rank = peersWithRank.get(address);
+    }
+    return rank;
+  }
+
+  @Override
+  public boolean equals(final Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null) {
+      return false;
+    }
+    if (!(o instanceof QuorumInfo)) {
+      return false;
+    }
+    QuorumInfo that = (QuorumInfo)o;
+    if (!this.quorumName.equals(that.quorumName)) {
+      return false;
+    }
+    if (!this.peers.equals(that.peers)) {
+      return false;
+    }
+    if (!this.peersAsString.equals(that.peersAsString)) {
+      return false;
+    }
+    if (!this.peersWithRank.equals(that.peersWithRank)) {
+      return false;
+    }
+    return true;
+  }
+
+  public boolean hasEqualReplicaSet(final QuorumInfo that) {
+    return this.peersWithRank.keySet().equals(that.peersWithRank.keySet());
+  }
+
+  @Override
+  public String toString() {
+    return String.format("{ Quorum Name = %s, peersWithRank = %s }",
+      getQuorumName(), peersWithRank);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumMembershipChangeRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumMembershipChangeRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumMembershipChangeRequest.java
new file mode 100644
index 0000000..9d1d525
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/QuorumMembershipChangeRequest.java
@@ -0,0 +1,38 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import com.google.common.util.concurrent.AbstractFuture;
+
+public class QuorumMembershipChangeRequest extends AbstractFuture<Boolean> {
+
+  public enum QuorumMembershipChangeState {
+    PENDING,
+    JOINT_CONFIG_COMMIT_IN_PROGRESS,
+    NEW_CONFIG_COMMIT_IN_PROGRESS,
+    COMPLETE,
+    FAILED
+  };
+
+  final QuorumInfo config;
+  QuorumMembershipChangeState currentState;
+
+  public QuorumMembershipChangeRequest(QuorumInfo config) {
+    this.config = config;
+    currentState = QuorumMembershipChangeState.PENDING;
+  }
+
+  public QuorumInfo getConfig() {
+    return config;
+  }
+
+  public QuorumMembershipChangeState getCurrentState() {
+    return currentState;
+  }
+
+  public void setCurrentState(QuorumMembershipChangeState currentState) {
+    this.currentState = currentState;
+  }
+
+  public void setResponse(boolean b) {
+    set(b);
+  }
+}


[13/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogWriter.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogWriter.java
new file mode 100644
index 0000000..4a1947a
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/LogWriter.java
@@ -0,0 +1,201 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import java.nio.ByteBuffer;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.hadoop.util.PureJavaCrc32;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+/**
+ * LogWriter provides interfaces to perform writer operations against a log file.
+ * This class is not thread safe. And there shall be ONLY one logWriter for each log file.
+ *
+ * It assumes the client will write the header of the file first, and then it can append
+ * commits or truncate the commits, finally the caller will close the writer.
+ *
+ *  Here is the log file format:
+ * ------------------
+ * | File Header:   |
+ * |  Version    4B |
+ * |  Term       8B |
+ * |  Index      8B |
+ * ------------------
+ * | Entry # N      |
+ * |  Entry Header  |
+ * |   Size     4B  |
+ * |   CRC      8B  |
+ * |   Index    8B  |
+ * |  Entry PayLoad |
+ * ------------------
+ * | Entry # N      |
+ * |  Entry Header  |
+ * |   Size     4B  |
+ * |   CRC      8B  |
+ * |   Index    8B  |
+ * |  Entry PayLoad |
+ * ----------------
+ *     .....
+ * ------------------
+ * | Entry # N      |
+ * |  Entry Header  |
+ * |   Size     4B  |
+ * |   CRC      8B  |
+ * |   Index    8B  |
+ * |  Entry PayLoad |
+ * ------------------
+ */
+@NotThreadSafe
+public class LogWriter {
+
+  private final Logger
+    LOG = LoggerFactory.getLogger(LogWriter.class);
+
+  private static final int CONSENSUS_LOG_DEFAULT_PAYLOAD_SIZE = 16 * 1024;
+  // The outside application should not have direct access to this object as
+  // the offset is maintained by the LogWriter
+  private final RandomAccessFile raf;
+  private boolean isSync = false;
+  private long currOffset;
+  private ByteBuffer buffer = ByteBuffer.allocateDirect(
+    CONSENSUS_LOG_DEFAULT_PAYLOAD_SIZE);
+
+  /** The CRC instance to compute CRC-32 of a log entry payload */
+  private PureJavaCrc32 crc32 = new PureJavaCrc32();
+
+  public LogWriter(RandomAccessFile raf, boolean isSync) {
+    this.raf = raf;
+    this.isSync = isSync;
+    this.currOffset = 0;
+  }
+
+
+  public RandomAccessFile getRandomAccessFile() {
+    return raf;
+  }
+
+  /**
+   * Write the header data to the log file
+   *
+   * @param term The term of the log file. Each log file only contains transactions
+   *             for  this term
+   * @param index The initial index for this log file.
+   * @throws IOException
+   */
+  public void writeFileHeader(long term, long index) throws IOException {
+    buffer.clear();
+
+    buffer.putInt(HConstants.RAFT_LOG_VERSION);
+
+    // Write the index to the buffer
+    buffer.putLong(term);
+    buffer.putLong(index);
+    buffer.flip();
+    currOffset += raf.getChannel().write(buffer);
+  }
+
+  public static ByteBuffer generateFileHeader(long term, long index) {
+    ByteBuffer bbuf = ByteBuffer.allocate(HConstants.RAFT_FILE_HEADER_SIZE);
+    bbuf.putInt(HConstants.RAFT_LOG_VERSION);
+    bbuf.putLong(term);
+    bbuf.putLong(index);
+    bbuf.flip();
+    return bbuf;
+  }
+
+  /**
+   * Append an specific commit (index with its transactions) to the log file
+   *
+   * @param index The index of the transactions
+   * @param data  The transaction list
+   * @return offset The file offset where this commit starts.
+   * @throws IOException
+   */
+  public long append(long index, ByteBuffer data) throws IOException {
+
+    // Get the current file offset right before this entry
+    long offset = currOffset;
+
+    ByteBuffer buffer = getBuffer(data.remaining() +
+      HConstants.RAFT_TXN_HEADER_SIZE);
+
+    // Clear the buffer
+    buffer.clear();
+
+    buffer.putInt(data.remaining());
+
+    // Update the CRC for the entry payload
+    this.crc32.reset();
+    this.crc32.update(data.array(), data.arrayOffset() + data.position(),
+      data.remaining());
+
+    // Write CRC value
+    buffer.putLong(crc32.getValue());
+
+    // Write the index to the buffer
+    buffer.putLong(index);
+
+    // Write the payload
+    buffer.put(data.array(), data.position() + data.arrayOffset(),
+      data.remaining());
+
+    // Reset the position
+    buffer.flip();
+
+    // Write the header
+    currOffset += raf.getChannel().write(buffer, currOffset);
+
+    buffer.clear();
+    // Sync the file if enabled
+    if (this.isSync) {
+      raf.getChannel().force(true);
+      raf.getFD().sync();
+    }
+
+    // Return the starting file offset before this entry
+    return offset;
+  }
+
+  /**
+   * Truncate the log from a specific offset
+   * @param offset
+   * @throws IOException
+   */
+  public void truncate(long offset) throws IOException {
+    this.raf.getChannel().truncate(offset);
+
+    // Need to always sync the data to the log file
+    this.raf.getChannel().force(true);
+    currOffset = offset;
+  }
+
+  /**
+   * Close the writer; No more writer operation is allowed after the log is closed.
+   * @throws IOException
+   */
+  public void close() throws IOException {
+    this.raf.close();
+  }
+
+  public long getCurrentPosition() {
+    return currOffset;
+  }
+
+  private ByteBuffer getBuffer(int payloadSize) {
+    if (buffer.capacity() >= payloadSize) {
+      return buffer;
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Allocating a new byte buffer of size " + payloadSize);
+    }
+    return ByteBuffer.allocate(payloadSize);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/RandomAccessLog.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/RandomAccessLog.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/RandomAccessLog.java
new file mode 100644
index 0000000..840dbae
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/RandomAccessLog.java
@@ -0,0 +1,451 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The RandomAccessLog provides the random access interface for the transaction
+ * log. This class is not thread-safe in general.
+ *
+ * This class holds one LogWriter instance, which delegates the write operation.
+ * And assume there is one thread accessing these write APIs.
+ * In addition, if the log file has been finalized for write, the file will be
+ * immutable.
+ *
+ * Also this class maintains a map of LogReader instance, which delegates the
+ * read operation. The map is indexed by the session key from the client, and
+ * the client will use its own session key to access these read APIs.
+ * Concurrent access from different session key is thread safe. But concurrent
+ * access from the same session key is NOT thread safe.
+ */
+@NotThreadSafe
+public class RandomAccessLog implements LogFileInterface {
+
+  public static long UNKNOWN_CREATION_TIME = -1;
+
+  private final Logger LOG = LoggerFactory.getLogger(RandomAccessLog.class);
+
+  /** It maps from the index to its starting file offset. */
+  private final ConcurrentHashMap<Long, Long> indexToOffsetMap;
+
+  private volatile long firstIndex = Long.MAX_VALUE;
+
+  private volatile long lastIndex = Long.MIN_VALUE;
+
+  /** The actual file */
+  private File file;
+
+  /** The LogWriter instance, which delegates the write operation. */
+  private final LogWriter writer;
+
+  /** Indicates whether the log has been finalized for write operation. */
+  private boolean isFinalized = false;
+
+  /** It maps from the reader session key to the LogReader instance. */
+  private final Map<String, LogReader> readerMap = new ConcurrentHashMap<>();
+
+  /** The current term of the log file. Each log file only has one term. */
+  private volatile long currentTerm = HConstants.UNDEFINED_TERM_INDEX;
+
+  /** Creation Time */
+  private final long creationTime;
+
+  /** only here to support mocking */
+  protected RandomAccessLog() {
+    file = null;
+    writer = null;
+    indexToOffsetMap = new ConcurrentHashMap<Long, Long>();
+    creationTime = UNKNOWN_CREATION_TIME;
+  }
+
+  public RandomAccessLog(File file, boolean isSync) throws IOException {
+    this(file, new RandomAccessFile(file, "rw"), isSync);
+  }
+
+  public RandomAccessLog(File file, RandomAccessFile raf, boolean isSync)
+    throws IOException {
+    this.creationTime = populateCreationTime(file);
+    this.file = file;
+    RandomAccessFile writeRAF = raf;
+    this.writer = new LogWriter(writeRAF, isSync);
+    this.indexToOffsetMap = new ConcurrentHashMap<Long, Long>();
+  }
+
+  public RandomAccessFile getRandomAccessFile() {
+    return writer.getRandomAccessFile();
+  }
+
+  /**
+   * Append the term, index and its transactions into the commit log.
+   * If this is the first entry of the log, it will write a file header as well.
+   * And return the starting offset for this commit entry
+   *
+   * @param term
+   * @param index
+   * @param data
+   * @return offset the start offset for this commit entry
+   * @throws IOException
+   */
+  public long append(long term, long index, final ByteBuffer data) throws IOException {
+    try {
+      // initialize the file
+      if (!isInitialized(term)) {
+        initialize(term, index);
+      }
+
+      // Append transactions and get the offset.
+      long offset = this.writer.append(index, data);
+      updateIndexMap(index, offset);
+
+      // Return the starting offset for this entry
+      return offset;
+    } catch (Exception e) {
+      // TODO:
+      LOG.error("Cannot append to the transaction log ", e);
+      throw e;
+    }
+  }
+
+  private void updateIndexMap(long index, long offset) {
+    if (index < this.firstIndex) {
+      this.firstIndex = index;
+    }
+
+    if (index > this.lastIndex) {
+      this.lastIndex = index;
+    }
+
+    // Update the index to offset map
+    indexToOffsetMap.put(index, offset);
+  }
+
+  /**
+   * Truncate the log by removing all the entry with larger or the same index.
+   *
+   * @param index
+   * @throws IOException
+   */
+  public void truncate(long index) throws IOException {
+    // Verify the term and index
+    Long offset = indexToOffsetMap.get(index);
+    if (offset == null) {
+      throw new IOException("No such index " + index + "in the current log");
+    }
+
+    // Truncate the file
+    try {
+      this.writer.truncate(offset);
+    } catch (IOException e) {
+      LOG.error("Cannot truncate to the transaction log ", e);
+      throw e;
+    }
+
+    // Update the meta data
+    removeIndexUpTo(index);
+  }
+
+  /**
+   * Remove all the indexes which is equal or larger than this index
+   * @param index
+   */
+  private void removeIndexUpTo(long index) {
+    if (index == this.firstIndex) {
+      // Reset all the meta data
+      this.indexToOffsetMap.clear();
+      firstIndex = Long.MAX_VALUE;
+      lastIndex = Long.MIN_VALUE;
+      return;
+    }
+
+    // Iterate over the indexToOffsetMap
+    for (Long key : this.indexToOffsetMap.keySet()) {
+      if (key >= index) {
+        this.indexToOffsetMap.remove(key);
+      }
+    }
+
+    // Update the lastIndex correctly
+    this.lastIndex = index - 1;
+  }
+
+  /**
+   *  getTransactionFileOffset
+   *
+   *  Get the file offset of a transaction.
+   *
+   *  @param  term
+   *  @param  index
+   *  @return long  offset
+   */
+  public long getTransactionFileOffset(long term, long index) throws NoSuchElementException {
+    // Sanity check the term and index
+    if (term != this.currentTerm || !this.indexToOffsetMap.containsKey(index)) {
+      throw new NoSuchElementException("No such index " + index +
+        " and term " + term + " in the current log " + toString());
+    }
+    return this.indexToOffsetMap.get(index);
+  }
+
+  /**
+   * Get the transactions for the given term, index and session key.
+   *
+   * @param term The term of the queried transaction.
+   * @param index The index of the quereid transaction.
+   * @param sessionKey The session key of the reader.
+   * @return transactions
+   * @throws IOException if the term does not match with the term of the log,
+   * or no such index in the log.
+   */
+  public MemoryBuffer getTransaction(long term, long index, String sessionKey,
+                             final Arena arena)
+    throws IOException, NoSuchElementException {
+    // Sanity check the term and index
+    if (term != this.currentTerm || !this.indexToOffsetMap.containsKey(index)) {
+      throw new NoSuchElementException("No such index " + index +
+        " and term " + term + " in the current log " + toString());
+    }
+
+    // Get the file offset from the map
+    long offset = this.indexToOffsetMap.get(index);
+
+    // Get the LogReader for this sessionKey
+    LogReader reader = this.getReader(sessionKey);
+
+    // Seek to the offset and read the transaction
+    return reader.seekAndRead(offset, index, arena);
+  }
+
+  @Override public long getLastModificationTime() {
+    return file.lastModified();
+  }
+
+  /**
+   * @return the absolute path of this log
+   */
+  public String getFileName() {
+    return file.getAbsolutePath();
+  }
+
+  /**
+   * Finalize the log file. The log file is immutable since then.
+   * @throws IOException
+   */
+  public void finalizeForWrite() throws IOException {
+    // Close the writer
+    this.writer.close();
+
+    // Mark the file as finalized
+    isFinalized = true;
+  }
+
+  /**
+   * Delete the current log file
+   * @throws IOException
+   */
+  @Override
+  public void closeAndDelete() throws IOException {
+    try{
+      finalizeForWrite();
+      removeAllReaders();
+    } catch (IOException e) {
+      LOG.error("Cannot close to the transaction log ", e);
+    } finally {
+      file.delete();
+    }
+  }
+
+  /**
+   * @return true if the log file has been finalized
+   */
+  public boolean isFinalized() {
+    return isFinalized;
+  }
+
+  /**   *
+   * @return the number of entries in the log;
+   */
+  public long getTxnCount() {
+    return this.indexToOffsetMap.isEmpty() ? 0 : (lastIndex - firstIndex + 1);
+  }
+
+  /**
+   * @return the initial index in the log
+   */
+  @Override
+  public long getInitialIndex() {
+    if (indexToOffsetMap.isEmpty()) {
+      return HConstants.UNDEFINED_TERM_INDEX;
+    }
+    return this.firstIndex;
+  }
+
+  /**
+   * @return the last index in the log
+   */
+  @Override
+  public long getLastIndex() {
+    if (indexToOffsetMap.isEmpty()) {
+      return HConstants.UNDEFINED_TERM_INDEX;
+    }
+    return this.lastIndex;
+  }
+
+  /**
+   * @return the current term of the log file
+   */
+  public long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  public File getFile() {
+    return file;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("RandomAccessLog{");
+    sb.append("term=").append(getCurrentTerm())
+      .append(", index=[").append(getInitialIndex())
+        .append(", ").append(getLastIndex()).append("]")
+      .append(", path=").append(getFileName())
+      .append(", ntxns=").append(getTxnCount())
+      .append(", finalized=").append(isFinalized())
+    ;
+    return sb.toString();
+  }
+
+
+  /**
+   * Remove and close the registered reader from the reader map by the session key
+   * @param sessionKey
+   */
+  public void removeReader(String sessionKey) throws IOException {
+    LogReader reader = this.readerMap.remove(sessionKey);
+    if (reader != null) {
+      reader.close();
+    }
+  }
+
+  public void removeAllReaders() throws IOException {
+    synchronized (this) {
+      for (LogReader reader : readerMap.values()) {
+        reader.close();
+      }
+      readerMap.clear();
+    }
+  }
+
+  /**
+   * Rebuilds the in-memory index=>offset map by scanning the file.
+   * @param sessionKey
+   * @throws IOException
+   */
+  public void rebuild(final String sessionKey) throws IOException {
+    LogReader reader = getReader(sessionKey);
+    long index, lastKnownGoodOffset;
+
+    currentTerm = reader.getCurrentTerm();
+    if (reader.getVersion() != HConstants.RAFT_LOG_VERSION) {
+      throw new IOException("Unable to verify the version.");
+    }
+    while (reader.hasMore()) {
+      index = reader.next();
+      if (index == HConstants.UNDEFINED_TERM_INDEX) {
+        break;
+      }
+
+      updateIndexMap(index, reader.getCurrentIndexFileOffset());
+    }
+
+    lastKnownGoodOffset = reader.getCurrentPosition();
+    LOG.debug("Resetting the write offset for " + reader.getFile().getAbsoluteFile() +
+      " to " + lastKnownGoodOffset);
+
+    // truncate the entries from the last known index
+    writer.truncate(lastKnownGoodOffset);
+
+    // Reset the reader position as we are done with the rebuild
+    reader.resetPosition();
+  }
+
+  /**
+   * A utility function to retrieve the log reader by the sessionKey
+   * @param sessionKey
+   * @return
+   * @throws IOException
+   */
+  private LogReader getReader(String sessionKey) throws IOException {
+    LogReader reader = this.readerMap.get(sessionKey);
+    if (reader == null) {
+      synchronized (this) {
+        reader = this.readerMap.get(sessionKey);
+        if (reader == null) {
+          // Initialize the reader
+          reader = new LogReader(this.file);
+          reader.initialize();
+
+          // Add the reader to the map
+          this.readerMap.put(sessionKey, reader);
+        }
+      }
+    }
+    return reader;
+  }
+
+  private boolean isInitialized(long term) {
+    if (currentTerm == HConstants.UNDEFINED_TERM_INDEX) {
+      return false;
+    } else if (currentTerm == term) {
+      return true;
+    } else {
+      throw new IllegalArgumentException("Expect the same currentTerm (" + currentTerm
+        + " ) for each commit log file. Requested term : " + term);
+    }
+  }
+
+  private void initialize(long term, long index) throws IOException {
+    this.writer.writeFileHeader(term, index);
+    // Update the meta data;
+    this.currentTerm = term;
+  }
+
+  @Override
+  public long getFileSize() {
+    return file.length();
+  }
+
+  @Override
+  public String getFileAbsolutePath() {
+    return file.getAbsolutePath();
+  }
+
+  @Override public long getCreationTime() {
+    return creationTime;
+  }
+
+  public static long populateCreationTime(final File file) {
+    try {
+      BasicFileAttributes attributes =
+        Files.readAttributes(file.toPath(), BasicFileAttributes.class);
+      return attributes.creationTime().toMillis();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    return UNKNOWN_CREATION_TIME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/ReadOnlyLog.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/ReadOnlyLog.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/ReadOnlyLog.java
new file mode 100644
index 0000000..4edc875
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/ReadOnlyLog.java
@@ -0,0 +1,185 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The ReadOnly log provides read interface for the transaction log.
+ *
+ * Also this class maintains a map of LogReader instance, which delegates the read operation.
+ * The map is indexed by the session key from the client, and the client will use
+ * its own session key to access these read APIs. Concurrent access from different session key is
+ * thread safe. But concurrent access from the same session key is NOT thread safe.
+ *
+ * The naming convention will be "log_<term>_<initial-index>_<last-index>"
+ */
+public class ReadOnlyLog implements LogFileInterface {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyLog.class);
+
+  protected final long initialIndex;
+  protected final long lastIndex;
+  protected final long currentTerm;
+  protected final File file;
+  protected final long creationTime;
+
+  private final Map<String, LogReader> readerMap = new ConcurrentHashMap<>();
+
+  public ReadOnlyLog(File file, long term, long initIndex) {
+    this.creationTime = RandomAccessLog.populateCreationTime(file);
+    this.file = file;
+    this.currentTerm = term;
+    this.initialIndex = initIndex;
+    this.lastIndex = Long.parseLong(file.getName().split("_")[2]);
+  }
+
+  public ReadOnlyLog(File file, String sessionKey) throws IOException {
+    this.creationTime = RandomAccessLog.populateCreationTime(file);
+
+    this.file = file;
+
+    // Get the LogReader
+    LogReader reader = getReader(sessionKey);
+
+    // Set up the invariants
+    this.currentTerm = reader.getCurrentTerm();
+    this.initialIndex = reader.getInitialIndex();
+    this.lastIndex = Long.parseLong(file.getName().split("_")[2]);
+  }
+
+  /**
+   * Read the transactions for the given index
+   *
+   * @param index The query index
+   * @param sessionKey The session key of the log reader
+   * @return transactions of the given index
+   * @throws IOException if the term does not match up or the index is not found.
+   */
+  public MemoryBuffer getTransaction(long term, long index, String sessionKey,
+                                     final Arena arena)
+    throws IOException, NoSuchElementException {
+    if (term != currentTerm) {
+      throw new NoSuchElementException ("The term " + term + " does not exist in this log file");
+    }
+    // Get the LogReader for this sessionKey
+    LogReader reader = this.getReader(sessionKey);
+    return reader.seekAndRead(index, arena);
+  }
+
+  @Override public long getLastModificationTime() {
+    return file.lastModified();
+  }
+
+  /**
+   * Remove and close the registered reader from the reader map by the session key
+   * @param sessionKey
+   */
+  public void removeReader(String sessionKey) throws IOException {
+    LogReader reader = this.readerMap.remove(sessionKey);
+    if (reader != null) {
+      reader.close();
+    }
+  }
+
+  /**
+   * @return the initial index of the log
+   */
+  @Override
+  public long getInitialIndex() {
+    return initialIndex;
+  }
+
+  /**
+   * @return the term of the log
+   */
+  public long getCurrentTerm() {
+    return currentTerm;
+  }
+
+  /**
+   * Close all the LogReader instances and delete the file
+   * @throws IOException
+   */
+  @Override
+  public void closeAndDelete() throws IOException {
+    for (String key : this.readerMap.keySet()) {
+      removeReader(key);
+    }
+    this.file.delete();
+  }
+
+  @Override public long getTxnCount() {
+    return this.getLastIndex() - this.getInitialIndex() + 1;
+  }
+
+  @Override public String getFileName() {
+    return file.getName();
+  }
+
+  @Override public File getFile() {
+    return file;
+  }
+
+  /** 
+   * Returns the last index in this log.
+   *
+   * @return
+   * @throws IOException
+   */
+  @Override
+  public long getLastIndex() {
+    return lastIndex;
+  }
+
+  private LogReader getReader(String sessionKey) throws IOException {
+    LogReader reader = this.readerMap.get(sessionKey);
+    if (reader == null) {
+      synchronized (this) {
+        reader = this.readerMap.get(sessionKey);
+        if (reader == null) {
+          // Initialize the reader
+          reader = new LogReader(this.file);
+          reader.initialize();
+
+          // Add the reader to the map
+          this.readerMap.put(sessionKey, reader);
+        }
+      }
+    }
+    return reader;
+  }
+
+  @Override
+  public long getFileSize() {
+    return file.length();
+  }
+
+  @Override
+  public String getFileAbsolutePath() {
+    return file.getAbsolutePath();
+  }
+
+  @Override public long getCreationTime() {
+    return creationTime;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("ReadOnlyLog{");
+    sb.append("term=").append(getCurrentTerm())
+      .append(", index=[").append(getInitialIndex())
+      .append(", ").append(getLastIndex()).append("]")
+      .append(", path=").append(getFileName())
+      .append(", ntxns=").append(getTxnCount());
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/RemoteLogFetcher.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/RemoteLogFetcher.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/RemoteLogFetcher.java
new file mode 100644
index 0000000..dc3c30f
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/RemoteLogFetcher.java
@@ -0,0 +1,96 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.hbase.consensus.client.FetchTask;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * Generate fetch plan from local quorum context.
+ * Also control the actual fetch work.
+ */
+public class RemoteLogFetcher {
+  private static final Logger LOG = LoggerFactory.getLogger(RemoteLogFetcher.class);
+
+  private RaftQuorumContext context;
+  QuorumClient quorumClient;
+
+  public RemoteLogFetcher() {}
+
+  public RemoteLogFetcher(RaftQuorumContext context) {
+    this.context = context;
+  }
+
+  /**
+   * Create a fetch plan from local context and download log files.
+   */
+  public void doSpontaneousFetch() throws Exception {
+    initializeQuorumClients();
+    List<Pair<String, List<LogFileInfo>>> statuses =
+      getCommittedLogInfoFromAllPeers();
+    Collection<FetchTask> tasks = createFetchTasks(
+        statuses, context.getCommittedEdit().getIndex());
+    executeTasks(tasks, context.getQuorumName());
+  }
+
+  /**
+   * Instantiate connections to peers
+   */
+  public void initializeQuorumClients() throws IOException {
+    this.quorumClient =
+        new QuorumClient(context.getQuorumInfo(), context.getConf(),
+            context.getExecServiceForThriftClients());
+  }
+
+  /**
+   * Ask peers for information of committed log files which have greater index than
+   * the latest local committed index
+   *
+   * @return each list item contains committed log info of a peer
+   */
+  public List<Pair<String, List<LogFileInfo>>> getCommittedLogInfoFromAllPeers()
+    throws Exception {
+    return getPeerCommittedLogStatus(context.getCommittedEdit().getIndex());
+  }
+
+  /**
+   * Ask peers for information of committed log files which have greater index than
+   * a given index. It's only used by tests.
+   *
+   * @param minIndex
+   * @return each list item contains committed log info of a peer
+   */
+  protected List<Pair<String, List<LogFileInfo>>> getPeerCommittedLogStatus(long minIndex)
+
+    throws Exception {
+    List<Pair<String, List<LogFileInfo>>> statuses = quorumClient.getPeerCommittedLogStatus(
+        context.getServerAddress(), context.getQuorumName(), minIndex);
+
+    return statuses;
+  }
+
+  protected Collection<FetchTask> createFetchTasks(
+      List<Pair<String, List<LogFileInfo>>> statuses, long minIndex) {
+    LogFetchPlanCreator planCreator = new LogFileInfoIterator(statuses, minIndex);
+    return planCreator.createFetchTasks();
+  }
+
+  /**
+   * Execute fetch tasks either generated locally or pushed from a remote server
+   *
+   * @param tasks each task item contains the address of one peer and a list of files to
+   *              be downloaded from it
+   */
+  public ListenableFuture<Void> executeTasks(Collection<FetchTask> tasks, String regionId) {
+    //TODO to be implemented in part 2
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/SeedLogFile.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/SeedLogFile.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/SeedLogFile.java
new file mode 100644
index 0000000..05f3c02
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/SeedLogFile.java
@@ -0,0 +1,52 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.HConstants;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * This is a special log file used to denote the seed Index for the transaction
+ * log manager.
+ */
+public class SeedLogFile extends ReadOnlyLog {
+
+  public SeedLogFile(final File file) {
+    super(file, HConstants.SEED_TERM, getInitialSeedIndex(file));
+  }
+
+  /**
+   * Remove and close the registered reader from the reader map by the session key
+   * @param sessionKey
+   */
+  public void removeReader(String sessionKey) throws IOException {
+    // Do-nothing
+  }
+
+  /**
+   * Close all the LogReader instances and delete the file
+   * @throws IOException
+   */
+  @Override
+  public void closeAndDelete() throws IOException {
+    super.closeAndDelete();
+  }
+
+  @Override public long getTxnCount() {
+    return lastIndex - initialIndex + 1;
+  }
+
+  public static boolean isSeedFile(final File f) {
+    return isSeedFile(f.getName());
+  }
+
+  public static boolean isSeedFile(String fileName) {
+    String[] split = fileName.split("_");
+    return split[0].equals("-2");
+  }
+
+  public static long getInitialSeedIndex(final File f) {
+    final String[] split = f.getName().split("_");
+    return Long.parseLong(split[1]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/TransactionLogCreator.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/TransactionLogCreator.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/TransactionLogCreator.java
new file mode 100644
index 0000000..a20bf70
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/TransactionLogCreator.java
@@ -0,0 +1,151 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Log Creator to maintain a pool of logs readily available to use.
+ *
+ * TODO: change this to be a ConsensusServer level to handle all quorums.
+ */
+public class TransactionLogCreator {
+  private static final Logger LOG = LoggerFactory.getLogger(TransactionLogCreator.class);
+
+  /** We use a single thread by default to create new log files for all quorums */
+  private static final ScheduledExecutorService threadPool = Executors.newScheduledThreadPool(1,
+      new DaemonThreadFactory("TransactionLogCreator-"));
+
+  /** List of opened random access files to use from. */
+  private final LinkedBlockingQueue<RandomAccessLog> futureLogs;
+  private int maxEmptyLogFiles = HConstants.RAFT_MAX_NUM_NEW_LOGS;
+
+  /** Log directory where it should create the new files */
+  private final String logDirectory;
+
+  /** Stops the creator from issuing/creating new files */
+  private volatile boolean isClosed = false;
+
+  /** Tells whether to sync the data always on append or not */
+  private final boolean isSync;
+
+  private final Runnable refillTask = new Runnable() {
+    @Override
+    public void run() {
+      while (futureLogs.remainingCapacity() > 0 && !isClosed) {
+        try {
+          tryGenerateNewLogFile();
+        } catch (IOException e) {
+          LOG.error("Failed to create log file in " + logDirectory + " . Will retry in " +
+              HConstants.RETRY_TRANSACTION_LOG_CREATION_DELAY_IN_SECS + " seconds.", e);
+          threadPool.schedule(this, HConstants.RETRY_TRANSACTION_LOG_CREATION_DELAY_IN_SECS, TimeUnit.SECONDS);
+          break;
+        } catch (Throwable t) {
+          LOG.error("Failed to create log file in " + logDirectory +
+              " unexpectedly. Aborting!", t);
+          break;
+        }
+      }
+    }
+  };
+
+  // purely for mocking
+  public TransactionLogCreator(final String logDirectory) {
+    this(logDirectory, false, null);
+  }
+
+  public TransactionLogCreator (String logDirectory, boolean isSync,
+                                Configuration conf) {
+    if (conf != null) {
+      maxEmptyLogFiles = conf.getInt(HConstants.RAFT_MAX_NUM_NEW_LOGS_KEY,
+        HConstants.RAFT_MAX_NUM_NEW_LOGS);
+    }
+
+    StringBuilder logDirectoryBuilder = new StringBuilder(logDirectory);
+    if (!logDirectory.endsWith(HConstants.PATH_SEPARATOR)) {
+      logDirectoryBuilder.append(HConstants.PATH_SEPARATOR);
+    }
+    logDirectoryBuilder.append(HConstants.RAFT_CURRENT_LOG_DIRECTORY_NAME);
+    logDirectoryBuilder.append(HConstants.PATH_SEPARATOR);
+    this.logDirectory = logDirectoryBuilder.toString();
+
+    this.isSync = isSync;
+
+    futureLogs = new LinkedBlockingQueue<>(maxEmptyLogFiles);
+
+    refillFutureLogs();
+  }
+
+  /**
+   * Returns the new log from the currently list of open files.
+   * @return
+   * @throws InterruptedException
+   */
+  public LogFileInterface getNewLogFile() throws InterruptedException {
+    if (isClosed) {
+      return null;
+    }
+
+    RandomAccessLog file = null;
+    while (file == null) {
+      file = futureLogs.poll(100, TimeUnit.MILLISECONDS);
+    }
+    refillFutureLogs();
+    return file;
+  }
+
+  /**
+   * Deletes all the opened files.
+   * @throws IOException
+   */
+  public void close() {
+    isClosed = true;
+    // Delete all the opened files.
+    RandomAccessLog ral;
+    while ((ral = futureLogs.poll()) != null) {
+      try {
+        ral.closeAndDelete();
+      } catch (IOException e) {
+        LOG.warn("Failed to delete log file in " + logDirectory, e);
+      }
+    }
+  }
+
+  /**
+   * Refill future logs queue
+   */
+  private void refillFutureLogs() {
+    threadPool.execute(refillTask);
+  }
+
+  private void tryGenerateNewLogFile() throws IOException {
+    String fileName = generateNewFileName();
+    File newFile = new File(logDirectory + fileName);
+    if (!futureLogs.offer(new RandomAccessLog(newFile, isSync))) {
+      LOG.debug(logDirectory + " is currently full");
+      newFile.delete();
+    }
+  }
+
+  private String generateNewFileName() {
+    return "log_" + System.nanoTime();
+  }
+
+  protected String getLogDirectory() {
+    return logDirectory;
+  }
+
+  public static void setMaxThreadNum(int maxThreadNum) {
+    ((ThreadPoolExecutor)threadPool).setMaximumPoolSize(maxThreadNum);
+  }
+}


[04/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InHeapArena.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InHeapArena.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InHeapArena.java
new file mode 100644
index 0000000..617d799
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InHeapArena.java
@@ -0,0 +1,79 @@
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.io.hfile.bucket.CacheFullException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A basic Arena Class which handles a chunk of contiguous memory using
+ * a Bucket Allocator.
+ */
+public class InHeapArena implements Arena {
+
+  public static final AtomicInteger numOps = new AtomicInteger();
+  public static final AtomicInteger failedOps = new AtomicInteger();
+  public static final AtomicInteger freedOps = new AtomicInteger();
+
+  public static final Logger LOG = LoggerFactory
+    .getLogger(InHeapArena.class);
+
+  private final byte[] buffer;
+  private BucketAllocator allocator;
+
+  public InHeapArena(int[] bucketSizes, int capacity) {
+    try {
+      allocator = new BucketAllocator(bucketSizes, capacity);
+    } catch (BucketAllocatorException e) {
+      LOG.error("Cannot initialize allocator with size " + capacity +
+        ". Reason", e);
+    }
+    buffer = new byte[capacity];
+  }
+
+  @Override
+  public MemoryBuffer allocateByteBuffer(int size) throws
+    CacheFullException, BucketAllocatorException {
+    int offset;
+
+    try {
+      offset = (int) allocator.allocateBlock(size);
+      numOps.incrementAndGet();
+    } catch (CacheFullException | BucketAllocatorException e) {
+      failedOps.incrementAndGet();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Cannot allocate memory buffer of size " + size, e);
+      }
+      throw e;
+    }
+
+    final ByteBuffer chunk = ByteBuffer.wrap(buffer, offset, size);
+    return new MemoryBuffer(chunk, offset, chunk.remaining());
+  }
+
+  @Override
+  public void freeByteBuffer(final MemoryBuffer buffer) {
+    if (buffer.getOffset() != MemoryBuffer.UNDEFINED_OFFSET) {
+      allocator.freeBlock(buffer.getOffset());
+      freedOps.incrementAndGet();
+    }
+  }
+
+  public static int getNumOpsAndReset() {
+    return numOps.getAndSet(0);
+  }
+
+  public static int getFailedOpsAndReset() {
+    return failedOps.getAndSet(0);
+  }
+
+  public static int getFreedOpsAndReset() {
+    return freedOps.getAndSet(0);
+  }
+
+  public BucketAllocator getAllocator() {
+    return allocator;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InfoServer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InfoServer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InfoServer.java
new file mode 100644
index 0000000..4bf49b5
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InfoServer.java
@@ -0,0 +1,148 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.apache.hadoop.http.HttpServer;
+import org.mortbay.jetty.handler.ContextHandlerCollection;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.DefaultServlet;
+import org.mortbay.jetty.webapp.WebAppContext;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Map;
+
+/**
+ * Create a Jetty embedded server to answer http requests. The primary goal
+ * is to serve up status information for the server.
+ * There are three contexts:
+ *   "/stacks/" -> points to stack trace
+ *   "/static/" -> points to common static files (src/hbase-webapps/static)
+ *   "/" -> the jsp server code from (src/hbase-webapps/<name>)
+ */
+public class InfoServer extends HttpServer {
+
+  WebAppContext quorumMonitoring;
+
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/hbase-webapps/<code>name<code>.
+   * @param name The name of the server
+   * @param bindAddress address to bind to
+   * @param port The port to use on the server
+   * @param findPort whether the server should start at the given port and
+   * increment by 1 until it finds a free port.
+   * @throws IOException e
+   */
+  public InfoServer(String name, String bindAddress, int port, 
+      boolean findPort, Configuration conf) throws IOException {
+    super(name, bindAddress, port, findPort, conf);
+    webServer.addHandler(new ContextHandlerCollection());
+  }
+
+  protected void addDefaultApps(ContextHandlerCollection parent, String appDir)
+  throws IOException {
+    super.addDefaultApps(parent, appDir, null);
+    // Must be same as up in hadoop.
+    final String logsContextPath = "/logs";
+    // Now, put my logs in place of hadoops... disable old one first.
+    Context oldLogsContext = null;
+    for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
+      if (e.getKey().getContextPath().equals(logsContextPath)) {
+        oldLogsContext = e.getKey();
+        break;
+      }
+    }
+    if (oldLogsContext != null) {
+      this.defaultContexts.put(oldLogsContext, Boolean.FALSE);
+    }
+    // Now do my logs.
+    // set up the context for "/logs/" if "hadoop.log.dir" property is defined.
+    String logDir = System.getProperty("hbase.log.dir");
+    if (logDir != null) {
+      Context logContext = new Context(parent, "/logs");
+      logContext.setResourceBase(logDir);
+      logContext.addServlet(DefaultServlet.class, "/");
+      defaultContexts.put(logContext, true);
+    }
+    // Now bring up the task monitor
+    WebAppContext taskMonitorContext = 
+      new WebAppContext(parent, "taskmontior", "/taskmonitor");
+    taskMonitorContext.addServlet(DefaultServlet.class, "/");
+    taskMonitorContext.setWar(appDir + "/taskmonitor");
+    defaultContexts.put(taskMonitorContext, true);
+
+    // Add some Quorum monitoring page
+    quorumMonitoring = new WebAppContext(parent,
+        HConstants.QUORUM_MONITORING_PAGE_KEY,
+        HConstants.QUORUM_MONITORING_PAGE_PATH);
+    quorumMonitoring.addServlet(DefaultServlet.class, "/");
+    quorumMonitoring.setWar(appDir + HConstants.QUORUM_MONITORING_PAGE_PATH);
+    defaultContexts.put(quorumMonitoring, true);
+  }
+
+  public void addQuorumMonitoringContext(LocalConsensusServer server) {
+    quorumMonitoring.setAttribute(HConstants.QUORUM_MONITORING_PAGE_KEY, server);
+  }
+
+  /**
+   * Get the pathname to the <code>path</code> files.
+   * @return the pathname as a URL
+   */
+  /*
+  protected String getWebAppsPath(String ignore) throws IOException {
+    // Hack: webapps is not a unique enough element to find in CLASSPATH
+    // We'll more than likely find the hadoop webapps dir.  So, instead
+    // look for the 'master' webapp in the webapps subdir.  That should
+    // get us the hbase context.  Presumption is that place where the
+    // master webapp resides is where we want this InfoServer picking up
+    // web applications.
+    final String master = "master";
+    String p = getWebAppDir(master);
+    // Now strip master + the separator off the end of our context
+    return p.substring(0, p.length() - (master.length() + 1/* The separator* /));
+  }
+*/
+
+  private static String getWebAppsPathStatic(final String path)
+  throws IOException {
+    URL url = InfoServer.class.getClassLoader().getResource(path);
+    if (url == null)
+      throw new IOException("hbase-webapps not found in CLASSPATH: " + path);
+    return url.toString();
+  }
+
+  /**
+   * Get the path for this web app
+   * @param webappName web app
+   * @return path
+   * @throws IOException e
+   */
+  public static String getWebAppDir(final String webappName)
+  throws IOException {
+    String webappDir;
+    webappDir = getWebAppsPathStatic("hbase-webapps/" + webappName);
+    return webappDir;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InjectionEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InjectionEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InjectionEvent.java
new file mode 100644
index 0000000..d550365
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InjectionEvent.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+/**
+ * Enumeration of all injection events.
+ * When defining new events, please PREFIX the name
+ * with the supervised class.
+ *
+ * Please see InjectionHandler.
+ */
+public enum InjectionEvent {
+  HMASTER_CREATE_TABLE,
+  HMASTER_DELETE_TABLE,
+  HMASTER_ALTER_TABLE,
+  HMASTER_ENABLE_TABLE,
+  HMASTER_DISABLE_TABLE,
+  ZKUNASSIGNEDWATCHER_REGION_OPENED,
+  SPLITLOGWORKER_SPLIT_LOG_START,
+  HMASTER_START_PROCESS_DEAD_SERVER,
+  HREGIONSERVER_REPORT_RESPONSE,
+
+  // Injection into Store.java
+  READONLYSTORE_COMPACTION_WHILE_SNAPSHOTTING,
+  STORESCANNER_COMPACTION_RACE,
+  STOREFILE_AFTER_WRITE_CLOSE,
+  STOREFILE_AFTER_RENAME
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InjectionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InjectionHandler.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InjectionHandler.java
new file mode 100644
index 0000000..ff670bc
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/InjectionHandler.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * The InjectionHandler is an object provided to a class,
+ * which can perform custom actions for JUnit testing.
+ * JUnit test can implement custom version of the handler.
+ * For example, let's say we want to supervise FSImage object:
+ *
+ * <code>
+ * // JUnit test code
+ * class MyInjectionHandler extends InjectionHandler {
+ *   protected void _processEvent(InjectionEvent event,
+ *       Object... args) {
+ *     if (event == InjectionEvent.MY_EVENT) {
+ *       LOG.info("Handling my event for fsImage: "
+ *         + args[0].toString());
+ *     }
+ *   }
+ * }
+ *
+ * public void testMyEvent() {
+ *   InjectionHandler ih = new MyInjectionHandler();
+ *   InjectionHandler.set(ih);
+ *   ...
+ *
+ *   InjectionHandler.clear();
+ * }
+ *
+ * // supervised code example
+ *
+ * class FSImage {
+ *
+ *   private doSomething() {
+ *     ...
+ *     if (condition1 && InjectionHandler.trueCondition(MY_EVENT1) {
+ *       ...
+ *     }
+ *     if (condition2 || condition3
+ *       || InjectionHandler.falseCondition(MY_EVENT1) {
+ *       ...
+ *     }
+ *     ...
+ *     InjectionHandler.processEvent(MY_EVENT2, this)
+ *     ...
+ *     try {
+ *       read();
+ *       InjectionHandler.processEventIO(MY_EVENT3, this, object);
+ *       // might throw an exception when testing
+ *     catch (IOEXception) {
+ *       LOG.info("Exception")
+ *     }
+ *     ...
+ *   }
+ *   ...
+ * }
+ * </code>
+ *
+ * Each unit test should use a unique event type.
+ * The types can be defined by adding them to
+ * InjectionEvent class.
+ *
+ * methods:
+ *
+ * // simulate actions
+ * void processEvent()
+ * // simulate exceptions
+ * void processEventIO() throws IOException
+ *
+ * // simulate conditions
+ * boolean trueCondition()
+ * boolean falseCondition()
+ *
+ * The class implementing InjectionHandler must
+ * override respective protected methods
+ * _processEvent()
+ * _processEventIO()
+ * _trueCondition()
+ * _falseCondition()
+ */
+public class InjectionHandler {
+
+  private static final Log LOG = LogFactory.getLog(InjectionHandler.class);
+
+  // the only handler to which everyone reports
+  private static InjectionHandler handler = new InjectionHandler();
+
+  // can not be instantiated outside, unless a testcase extends it
+  protected InjectionHandler() {}
+
+  // METHODS FOR PRODUCTION CODE
+
+  protected void _processEvent(InjectionEvent event, Object... args) {
+    // by default do nothing
+  }
+
+  protected void _processEventIO(InjectionEvent event, Object... args) throws IOException{
+    // by default do nothing
+  }
+
+  protected boolean _trueCondition(InjectionEvent event, Object... args) {
+    return true; // neutral in conjunction
+  }
+
+  protected boolean _falseCondition(InjectionEvent event, Object... args) {
+    return false; // neutral in alternative
+  }
+
+  ////////////////////////////////////////////////////////////
+
+  /**
+   * Set to the empty/production implementation.
+   */
+  public static void clear() {
+    handler = new InjectionHandler();
+  }
+
+  /**
+   * Set custom implementation of the handler.
+   */
+  public static void set(InjectionHandler custom) {
+    LOG.warn("WARNING: SETTING INJECTION HANDLER" +
+      " - THIS SHOULD NOT BE USED IN PRODUCTION !!!");
+    handler = custom;
+  }
+
+  /*
+  * Static methods for reporting to the handler
+  */
+
+  public static void processEvent(InjectionEvent event, Object... args) {
+    handler._processEvent(event, args);
+  }
+
+  public static void processEventIO(InjectionEvent event, Object... args)
+    throws IOException {
+    handler._processEventIO(event, args);
+  }
+
+  public static boolean trueCondition(InjectionEvent event, Object... args) {
+    return handler._trueCondition(event, args);
+  }
+
+  public static boolean falseCondition(InjectionEvent event, Object... args) {
+    return handler._falseCondition(event, args);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/MemoryBuffer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/MemoryBuffer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/MemoryBuffer.java
new file mode 100644
index 0000000..7dfc814
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/MemoryBuffer.java
@@ -0,0 +1,48 @@
+package org.apache.hadoop.hbase.util;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Wraps the memory allocated by the Arena. It provides the abstraction of a
+ * ByteBuffer to the outside world.
+ */
+public class MemoryBuffer {
+
+  private final long offset;
+  private final int size;
+  private final ByteBuffer buffer;
+  public static final int UNDEFINED_OFFSET = -1;
+
+  public MemoryBuffer(final ByteBuffer buffer, long offset, int size) {
+    this.offset = offset;
+    this.size = size;
+    this.buffer = buffer;
+  }
+
+  public MemoryBuffer(ByteBuffer buffer) {
+    this.offset = UNDEFINED_OFFSET;
+    this.size = buffer.remaining();
+    this.buffer = buffer;
+  }
+
+  public ByteBuffer getBuffer() {
+    return buffer;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public int getSize() {
+    return size;
+  }
+
+  public void flip() {
+    if (offset != UNDEFINED_OFFSET) {
+      buffer.limit(buffer.position());
+      buffer.position((int)offset);
+    } else {
+      buffer.flip();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Pair.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Pair.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Pair.java
new file mode 100644
index 0000000..8e9fa36
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Pair.java
@@ -0,0 +1,131 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import java.io.Serializable;
+
+/**
+ * A generic class for pairs.
+ * @param <T1>
+ * @param <T2>
+ */
+public class Pair<T1, T2> implements Serializable
+{
+  private static final long serialVersionUID = -3986244606585552569L;
+  protected T1 first = null;
+  protected T2 second = null;
+
+  /**
+   * Default constructor.
+   */
+  public Pair()
+  {
+  }
+
+  /**
+   * Constructor
+   * @param a operand
+   * @param b operand
+   */
+  public Pair(T1 a, T2 b)
+  {
+    this.first = a;
+    this.second = b;
+  }
+  
+  /**
+   * Constructs a new pair, inferring the type via the passed arguments
+   * @param <T1> type for first
+   * @param <T2> type for second
+   * @param a first element
+   * @param b second element
+   * @return a new pair containing the passed arguments
+   */
+  public static <T1,T2> Pair<T1,T2> newPair(T1 a, T2 b) {
+    return new Pair<T1,T2>(a, b);
+  }
+  
+  /**
+   * Replace the first element of the pair.
+   * @param a operand
+   */
+  public void setFirst(T1 a)
+  {
+    this.first = a;
+  }
+
+  /**
+   * Replace the second element of the pair.
+   * @param b operand
+   */
+  public void setSecond(T2 b)
+  {
+    this.second = b;
+  }
+
+  /**
+   * Return the first element stored in the pair.
+   * @return T1
+   */
+  public T1 getFirst()
+  {
+    return first;
+  }
+
+  /**
+   * Return the second element stored in the pair.
+   * @return T2
+   */
+  public T2 getSecond()
+  {
+    return second;
+  }
+
+  private static boolean equals(Object x, Object y)
+  {
+     return (x == null && y == null) || (x != null && x.equals(y));
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public boolean equals(Object other)
+  {
+    return other instanceof Pair && equals(first, ((Pair)other).first) &&
+      equals(second, ((Pair)other).second);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    if (first == null)
+      return (second == null) ? 0 : second.hashCode() + 1;
+    else if (second == null)
+      return first.hashCode() + 2;
+    else
+      return first.hashCode() * 17 + second.hashCode();
+  }
+
+  @Override
+  public String toString()
+  {
+    return "{" + getFirst() + "," + getSecond() + "}";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Threads.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Threads.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Threads.java
new file mode 100644
index 0000000..4094bc7
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Threads.java
@@ -0,0 +1,264 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import java.io.InterruptedIOException;
+import java.io.PrintWriter;
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.hbase.util.HasThread;
+
+/**
+ * Thread Utility
+ */
+public class Threads {
+  protected static final Log LOG = LogFactory.getLog(Threads.class);
+
+  /**
+   * Utility method that sets name, daemon status and starts passed thread.
+   * @param t thread to frob
+   * @param name new name
+   * @return Returns the passed Thread <code>t</code>.
+   */
+  public static Thread setDaemonThreadRunning(final Thread t,
+    final String name) {
+    return setDaemonThreadRunning(t, name, null);
+  }
+
+  public static HasThread setDaemonThreadRunning(final HasThread t,
+      final String name) {
+    setDaemonThreadRunning(t.getThread(), name, null);
+    return t;
+  }
+
+  /**
+   * Utility method that sets name, daemon status and starts passed thread.
+   * @param t thread to frob
+   * @param name new name
+   * @param handler A handler to set on the thread.  Pass null if want to
+   * use default handler.
+   * @return Returns the passed Thread <code>t</code>.
+   */
+  public static Thread setDaemonThreadRunning(final Thread t,
+    final String name, final UncaughtExceptionHandler handler) {
+    if (name != null) {
+      t.setName(name);
+    }
+    if (handler != null) {
+      t.setUncaughtExceptionHandler(handler);
+    }
+    t.setDaemon(true);
+    t.start();
+    return t;
+  }
+
+  public static HasThread setDaemonThreadRunning(final HasThread t,
+      final String name, final UncaughtExceptionHandler handler) {
+    setDaemonThreadRunning(t.getThread(), name, handler);
+    return t;
+  }
+
+  /**
+   * Shutdown passed thread using isAlive and join.
+   * @param t Thread to shutdown
+   */
+  public static void shutdown(final Thread t) {
+    shutdown(t, 0);
+  }
+
+  public static void shutdown(final HasThread t) {
+    shutdown(t.getThread(), 0);
+  }
+
+  /**
+   * Shutdown passed thread using isAlive and join.
+   * @param joinwait Pass 0 if we're to wait forever.
+   * @param t Thread to shutdown
+   */
+  public static void shutdown(final Thread t, final long joinwait) {
+    if (t == null) return;
+    while (t.isAlive()) {
+      try {
+        t.join(joinwait);
+      } catch (InterruptedException e) {
+        LOG.warn(t.getName() + "; joinwait=" + joinwait, e);
+      }
+    }
+  }
+  
+  public static void shutdown(final HasThread t, final long joinwait) {
+    shutdown(t.getThread(), joinwait);
+  }
+
+  /**
+   * @param t Waits on the passed thread to die dumping a threaddump every
+   * minute while its up.
+   * @throws InterruptedException
+   */
+  public static void threadDumpingIsAlive(final Thread t)
+  throws InterruptedException {
+    if (t == null) {
+      return;
+    }
+    long startTime = System.currentTimeMillis();
+    while (t.isAlive()) {
+      Thread.sleep(1000);
+      if (System.currentTimeMillis() - startTime > 60000) {
+        startTime = System.currentTimeMillis();
+        ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
+            "Automatic Stack Trace every 60 seconds waiting on " +
+            t.getName());
+      }
+    }
+  }
+
+  /**
+   * @param millis How long to sleep for in milliseconds.
+   */
+  public static void sleep(long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (InterruptedException e) {
+      LOG.warn("Interrupted when sleeping for " + millis + "ms, ignoring", e);
+    }
+  }
+
+  /**
+   * Sleeps for the given amount of time. Retains the thread's interruption status. 
+   * @param millis How long to sleep for in milliseconds.
+   */
+  public static void sleepRetainInterrupt(long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (InterruptedException e) {
+      LOG.warn("Interrupted when sleeping for " + millis + "ms");
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  /**
+   * Sleeps for the given amount of time even if interrupted. Preserves
+   * the interrupt status.
+   * @param msToWait the amount of time to sleep in milliseconds
+   */
+  public static void sleepWithoutInterrupt(final long msToWait) {
+    long timeMillis = System.currentTimeMillis();
+    long endTime = timeMillis + msToWait;
+    boolean interrupted = false;
+    while (timeMillis < endTime) {
+      try {
+        Thread.sleep(endTime - timeMillis);
+      } catch (InterruptedException ex) {
+        interrupted = true;
+      }
+      timeMillis = System.currentTimeMillis();
+    }
+
+    if (interrupted) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  /**
+   * Create a new CachedThreadPool with a bounded number as the maximum 
+   * thread size in the pool.
+   * 
+   * @param maxCachedThread the maximum thread could be created in the pool
+   * @param timeout the maximum time to wait
+   * @param unit the time unit of the timeout argument
+   * @param threadFactory the factory to use when creating new threads
+   * @return threadPoolExecutor the cachedThreadPool with a bounded number 
+   * as the maximum thread size in the pool. 
+   */
+  public static ThreadPoolExecutor getBoundedCachedThreadPool(
+      int maxCachedThread, long timeout, TimeUnit unit,
+      ThreadFactory threadFactory) {
+    ThreadPoolExecutor boundedCachedThreadPool =
+      new ThreadPoolExecutor(maxCachedThread, maxCachedThread, timeout,
+        TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(), threadFactory);
+    // allow the core pool threads timeout and terminate
+    boundedCachedThreadPool.allowCoreThreadTimeOut(true);
+    return boundedCachedThreadPool;
+  }
+
+  /**
+   * Creates a ThreadPoolExecutor which has a bound on the number of tasks that can be 
+   * submitted to it, determined by the blockingLimit parameter. Excess tasks 
+   * submitted will block on the calling thread till space frees up.
+   * 
+   * @param blockingLimit max number of tasks that can be submitted
+   * @param timeout time value after which unused threads are killed
+   * @param unit time unit for killing unused threads
+   * @param threadFactory thread factory to use to spawn threads
+   * @return the ThreadPoolExecutor
+   */
+  public static ThreadPoolExecutor getBlockingThreadPool(
+      int blockingLimit, long timeout, TimeUnit unit,
+      ThreadFactory threadFactory) {
+    ThreadPoolExecutor blockingThreadPool = 
+      new ThreadPoolExecutor( 
+        1, blockingLimit, timeout, TimeUnit.SECONDS,
+        new SynchronousQueue<Runnable>(),
+        threadFactory, 
+        new RejectedExecutionHandler() {  
+          @Override 
+          public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {  
+            try { 
+              // The submitting thread will block until the thread pool frees up. 
+              executor.getQueue().put(r); 
+            } catch (InterruptedException e) {  
+              throw new RejectedExecutionException( 
+                  "Failed to requeue the rejected request because of ", e); 
+            } 
+          } 
+        });
+    blockingThreadPool.allowCoreThreadTimeOut(true);
+    return blockingThreadPool;
+  }
+
+  public static void renameThread(Thread t, String newName) {
+    String oldName = t.getName();
+    if (!oldName.equals(newName)) {
+      LOG.info("Thread '" + oldName + "' is now known as '" + newName + "'");
+      t.setName(newName);
+    }
+  }
+
+  /**
+   * A helper function to check if the current thread was interrupted.
+   *
+   * @throws InterruptedIOException if the thread had been interrupted.
+   **/
+  public static void checkInterrupted() throws InterruptedIOException {
+    if (Thread.interrupted()) {
+      throw new InterruptedIOException("Thread was interrupted");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/serial/AsyncSerialExecutorServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/serial/AsyncSerialExecutorServiceImpl.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/serial/AsyncSerialExecutorServiceImpl.java
new file mode 100644
index 0000000..9ea0f4b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/serial/AsyncSerialExecutorServiceImpl.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.hbase.util.serial;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+/**
+ * Library that can be used when we want to schedule many async events but such that execution of a
+ * certain event depends on completing a previous event. One doesn't need to know on which thread
+ * the execution will happen - since the next free thread from the pool will be chosen and the
+ * constraint will be preserved.
+ */
+public class AsyncSerialExecutorServiceImpl implements SerialExecutorService {
+  static final Log LOG = LogFactory.getLog(AsyncSerialExecutorServiceImpl.class);
+
+  private final Executor executorService;
+
+  public AsyncSerialExecutorServiceImpl(int numThreads, String poolName) {
+    this.executorService =
+        Executors.newFixedThreadPool(numThreads, new DaemonThreadFactory(poolName));
+  }
+
+  public AsyncSerialExecutorServiceImpl(Executor executorService) {
+    this.executorService = executorService;
+  }
+
+  @Override
+  public SerialExecutionStream createStream() {
+    return new SerialExecutionStream() {
+      /**
+       * Future that completes when last scheduled action finishes, representing last
+       * future in the stream/list.
+       * So, each future needs to have only one listener attached to it.
+       * We enforce that by accessing lastExecuted only through getAndSet method,
+       * which guarantees each future is both stored and returned by this AtomicReference
+       * only once.
+       *
+       * At start it has already been finished, so first command doesn't have to wait.
+       */
+      private AtomicReference<ListenableFuture<Void>> lastExecuted = new AtomicReference<>(
+          Futures.<Void>immediateFuture(null));
+
+      @Override
+      public ListenableFuture<Void> execute(Callable<ListenableFuture<Void>> command) {
+        SettableFuture<Void> commandDone = SettableFuture.create();
+        ListenableFuture<Void> last = lastExecuted.getAndSet(commandDone);
+        last.addListener(new CommandRunAndCompleteCurrent(command, commandDone), executorService);
+        return commandDone;
+      }
+    };
+  }
+
+  /**
+   * Wrapper around runnable and settable future
+   */
+  private class CommandRunAndCompleteCurrent implements Runnable {
+    private Callable<ListenableFuture<Void>> command;
+    private SettableFuture<Void> currentFuture;
+
+    public CommandRunAndCompleteCurrent(Callable<ListenableFuture<Void>> command, SettableFuture<Void> currentFuture) {
+      this.command = command;
+      this.currentFuture = currentFuture;
+    }
+
+    @Override
+    public void run() {
+      try {
+        ListenableFuture<Void> commandFuture = command.call();
+        if (commandFuture == null) {
+          currentFuture.set(null);
+        } else {
+          Futures.addCallback(commandFuture, new FutureCallback<Void>() {
+            @Override
+            public void onSuccess(Void result) {
+              currentFuture.set(null);
+            }
+            @Override
+            public void onFailure(Throwable t) {
+              LOG.error("exception in the command future", t);
+              currentFuture.set(null);
+            }
+          }, executorService);
+        }
+      } catch (Exception e) {
+        // Log exception
+        LOG.error("exception while executing command", e);
+        currentFuture.set(null);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/serial/SerialExecutorService.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/serial/SerialExecutorService.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/serial/SerialExecutorService.java
new file mode 100644
index 0000000..3f26b36
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/serial/SerialExecutorService.java
@@ -0,0 +1,34 @@
+package org.apache.hadoop.hbase.util.serial;
+
+import java.util.concurrent.Callable;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * Executor service which allows creation of arbitrary number of serial streams, all to be executed
+ * on a single thread pool. All commands scheduled on a single serial stream are executed serially
+ * (not necessarily on the same thread).
+ */
+public interface SerialExecutorService {
+  /** Creates new serial stream */
+  SerialExecutionStream createStream();
+
+  /**
+   * Executor, that makes sure commands are executed serially, in
+   * order they are scheduled. No two commands will be running at the same time.
+   */
+  public interface SerialExecutionStream {
+    /**
+     * Executes the given command on this stream, returning future.
+     * If future command return is not null, next command in the stream
+     * will not be executed before future completes.
+     *
+     * I.e. async commands should return future when it completes,
+     * and non-async commands should return null.
+     *
+     * @return Returns future which is going to be completed when command and
+     *   future it returns both finish.
+     */
+    ListenableFuture<Void> execute(Callable<ListenableFuture<Void>> command);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/LocalTestBed.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/LocalTestBed.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/LocalTestBed.java
new file mode 100644
index 0000000..9f7a7c3
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/LocalTestBed.java
@@ -0,0 +1,944 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.exceptions.LeaderNotReadyException;
+import org.apache.hadoop.hbase.consensus.exceptions.NewLeaderException;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumAgent;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.InstrumentedConsensusServiceImpl;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.apache.hadoop.hbase.consensus.server.peer.states.PeerHandleAppendResponse;
+
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A test utility for Unit Testing RAFT protocol by itself.
+ */
+public class LocalTestBed {
+  private static final Logger LOG = LoggerFactory.getLogger(LocalTestBed.class);
+
+  private static int DEFAULT_RAFT_CONSENSUS_START_PORT_NUMBER = 60080;
+  private static int QUORUM_SIZE = 5;
+  private static final int QUORUM_MAJORITY = 3;
+  private static int nextPortNumber;
+
+  private Configuration conf;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private final List<int[]> mockLogs;
+  private boolean checkLeaderCount = false;
+  private boolean usePersistentLog = true;
+  private boolean autoRestartThriftServer = true;
+  private AtomicLong commitSuccessCount = new AtomicLong(0);
+  private AtomicLong commitFailureCount = new AtomicLong(0);
+  private HealthChecker checker = null;
+
+  public static String LOCAL_HOST = "127.0.0.1";
+  ConcurrentHashMap<String, LocalConsensusServer> servers;
+
+  public final long BigBang = System.currentTimeMillis();
+
+  public static class HealthChecker extends Thread {
+    private HRegionInfo   regionInfo;
+    private RaftTestUtil  testUtil;
+    private boolean       autoRestartThriftServer;
+    private long          checkInterval;
+    private AtomicBoolean time2die = new AtomicBoolean(false);
+
+    public HealthChecker(HRegionInfo regionInfo, RaftTestUtil testUtil, boolean autoRestartThriftServer, long checkInterval) {
+      this.regionInfo = regionInfo;
+      this.testUtil = testUtil;
+      this.autoRestartThriftServer = autoRestartThriftServer;
+      this.checkInterval = checkInterval;
+    }
+
+    @Override
+    public void run() {
+      long previousCheckTime = 0L;
+      while (!time2die.get()) {
+        try {
+          long now = System.currentTimeMillis();
+          if (now >= previousCheckTime + checkInterval) {
+            LOG.info("checking the health of all quorum members ......");
+            testUtil.checkHealth(regionInfo, autoRestartThriftServer);
+            previousCheckTime = now = System.currentTimeMillis();
+          }
+          long sleepTime = previousCheckTime + checkInterval - now;
+          if (sleepTime > 0) {
+            Thread.sleep(sleepTime);
+          }
+        } catch (InterruptedException ex) {
+          LOG.info("Time to Die!!");
+          time2die.set(true);
+        } catch (Throwable stone) {
+          LOG.warn("Caught an exception while checking health", stone);
+        }
+      }
+    }
+
+    public void shutdown() {
+      time2die.set(true);
+    }
+  }
+
+
+  public static class LinkState {
+    private HServerAddress  src;
+    private HServerAddress  dst;
+    private State           state;
+    private long            delay = 0L;
+    private long            hiccupStopTime = 0L;
+    private long            hiccupStartTime = 0L;
+
+    public static enum State {
+      UP, DOWN
+    }
+
+    public LinkState(HServerAddress src, HServerAddress dst, State state) {
+      this.src = src;
+      this.dst = dst;
+      this.state = state;
+    }
+
+    public HServerAddress getSrc() {
+      return src;
+    }
+
+    public HServerAddress getDst() {
+      return dst;
+    }
+
+    public void setDelay(long delay) {
+      this.delay = delay;
+    }
+
+    public boolean inHiccup(long now) {
+      return hiccupStartTime <= now && now <= hiccupStopTime;
+    }
+
+    public long getDelay(long now) {
+      return delay;
+    }
+
+    public void setHiccupTime(long start, long stop) {
+      hiccupStartTime = start;
+      hiccupStopTime = stop;
+    }
+
+    public long getHiccupStartTime() {
+      return hiccupStartTime;
+    }
+
+    public long getHiccupStopTime() {
+      return hiccupStopTime;
+    }
+
+    @Override
+    public String toString() {
+      return "[" + src + "--" + dst + "]";
+    }
+  }
+
+  public static class NodeState {
+    private State state;
+    public static enum State {
+      UP, DOWN, INTERMITTENT, SLOW
+    }
+
+    public NodeState(State state) {
+    }
+
+    public NodeState changeState(State state) {
+      return this;
+    }
+  }
+
+  public static class Chaos {
+    private TestConfig config;
+    private Random prng;
+
+    // the map is maintained per pair of nodes
+    private ConcurrentHashMap<HServerAddress, ConcurrentHashMap<HServerAddress, LinkState>> links
+      = new ConcurrentHashMap<HServerAddress, ConcurrentHashMap<HServerAddress, LinkState>>();
+
+    private ConcurrentHashMap<String, NodeState> nodes =
+      new ConcurrentHashMap<String, NodeState>();
+
+    public Chaos() {
+      this(new TestConfig());
+    }
+
+    public Chaos(TestConfig config) {
+      this.config = config;
+      prng = new Random(config.getLong("seed", System.currentTimeMillis()));
+    }
+
+    public LinkState getLinkState(HServerAddress src, HServerAddress dst) {
+      if (src == null || dst == null || src.equals(dst)) {
+        return null;
+      }
+      if (src.toString().compareTo(dst.toString()) > 0) {
+        HServerAddress tmp = src;
+        src = dst;
+        dst = tmp;
+      }
+      if (links.get(src) == null) {
+        links.put(src, new ConcurrentHashMap<HServerAddress, LinkState>());
+      }
+      if (links.get(src).get(dst) == null) {
+        links.get(src).put(dst, new LinkState(src, dst, LinkState.State.UP));
+      }
+      return links.get(src).get(dst);
+    }
+
+    public void updateNetworkStates(long now, boolean clear) {
+      double defaultDropRate = config.getDouble("packet-drop-rate", 0.0);
+      double maxRandomDelay = config.getDouble("max-random-delay", 20.0);
+      double minRandomDelay = config.getDouble("min-random-delay", 00.0);
+
+      long maxHiccupDuration = config.getLong("max-hiccup-duration", 10000L);
+      long minHiccupDuration = config.getLong("min-hiccup-duration",  1000L);
+      long maxHiccupLinks = config.getLong("max-hiccup-links", 1L);
+      long hiccupLambda = config.getLong("hiccup-lambda",  -1L);
+
+      ConcurrentHashMap<String, LocalConsensusServer> servers = RAFT_TEST_UTIL.getServers();
+      Set<LinkState> currentHiccups = new HashSet<LinkState>();
+      Set<LinkState> allHiccups = new HashSet<LinkState>(); // include future hiccups
+      Set<LinkState> nohiccups = new HashSet<LinkState>();
+      for (HServerAddress dst : regionInfo.getPeersWithRank().keySet()) {
+        dst = RaftUtil.getLocalConsensusAddress(dst);
+        for (HServerAddress src : regionInfo.getPeersWithRank().keySet()) {
+          src = RaftUtil.getLocalConsensusAddress(src);
+          if (src.equals(dst)) {
+            continue;
+          }
+          LinkState link = getLinkState(src, dst);
+          if (link.inHiccup(now)) {
+            currentHiccups.add(link);
+            allHiccups.add(link);
+          } else if (link.getHiccupStartTime() > now) {
+            allHiccups.add(link);
+          } else {
+            // We know link.getHiccupStopTime() < now, too
+            nohiccups.add(link);
+          }
+        }
+      }
+
+      if (currentHiccups.size() > 0) {
+        LOG.debug("The following links are in hiccup: " + currentHiccups);
+      }
+
+      List<LinkState> futureHiccups = new ArrayList<LinkState>(nohiccups);
+
+      // Spread the love; otherwise, the first few links are always in hiccups (now
+      // and in future).
+      Collections.shuffle(futureHiccups);
+
+      for (LinkState link : futureHiccups) {
+        // Calculate the next hiccup time
+        long delta = poissonRandomInterarrivalDelay(prng, (double)hiccupLambda);
+        long hiccupStartTime = link.getHiccupStopTime() == 0 ? now + delta : link.getHiccupStopTime() + delta;
+        long duration = (long)(prng.nextDouble()*(maxHiccupDuration-minHiccupDuration) + minHiccupDuration);
+        long hiccupStopTime = hiccupStartTime + duration;
+
+        // before we schedule this hiccup, make sure we don't have too many
+        // hiccups at the same time (both now and in future).
+        LinkState tmp = new LinkState(link.getSrc(), link.getDst(), LinkState.State.UP);
+        tmp.setHiccupTime(hiccupStartTime, hiccupStopTime);
+        allHiccups.add(tmp);
+        int nhiccups =computeMaxHiccupLinks(allHiccups);
+        allHiccups.remove(tmp);
+
+        if (nhiccups < maxHiccupLinks) {
+          LOG.debug("---- scheduling a future hiccup for " + link + " [" + hiccupStartTime + ", " + hiccupStopTime + "]");
+          link.setHiccupTime(hiccupStartTime, hiccupStopTime);
+          allHiccups.add(link);
+        } else {
+          LOG.debug("---- too many hiccups right now; not scheduling a future hiccup for " + link + " [" + hiccupStartTime + ", " + hiccupStopTime + "]");
+        }
+      }
+
+      for (HServerAddress dst : regionInfo.getPeersWithRank().keySet()) {
+        dst = RaftUtil.getLocalConsensusAddress(dst);
+        InstrumentedConsensusServiceImpl service = (InstrumentedConsensusServiceImpl)
+          (servers.get(dst.getHostAddressWithPort()).getHandler());
+        for (HServerAddress src : regionInfo.getPeersWithRank().keySet()) {
+          src = RaftUtil.getLocalConsensusAddress(src);
+          if (!src.equals(dst) && !clear) {
+            long delay = (long)(prng.nextDouble()*(maxRandomDelay-minRandomDelay) + minRandomDelay);
+            double dropRate = defaultDropRate;
+            boolean inHiccup = false;
+            if (hiccupLambda > 0) {
+              LinkState link = getLinkState(src, dst);
+              if (link.inHiccup(now)) {
+                LOG.debug("---- " + link + " in hiccup right now!");
+                dropRate = 1.1; // make sure we take care of floating-point inaccuracies
+                inHiccup = true;
+              }
+            }
+            service.setHiccup(src.toString(), inHiccup);
+            service.setPacketDropRate(src.toString(), dropRate);
+            service.setPacketDelay(src.toString(), delay);
+          } else {
+            service.setHiccup(src.toString(), false);
+            service.setPacketDropRate(src.toString(), 0.0);
+            service.setPacketDelay(src.toString(), 0L);
+          }
+        }
+      }
+    }
+
+    /**
+     *  computeMaxHiccupLinks
+     *
+     *  Compute the maximum number of overlapping hiccups.
+     *
+     *  We can really use a unit test here.
+     */
+    public int computeMaxHiccupLinks(Set<LinkState> hiccups) {
+      if (hiccups == null || hiccups.size() == 0) {
+        return 0;
+      }
+      List<Long> endpoints = new ArrayList<Long>();
+      for (LinkState x : hiccups) {
+        endpoints.add(x.getHiccupStartTime());
+        endpoints.add(x.getHiccupStopTime());
+      }
+      Collections.sort(endpoints);
+
+      // we are guaranteed here that endpoints.size() >= 2
+      int maxCount = 0;
+      for (int i=1; i<endpoints.size(); i++) {
+        Long start = endpoints.get(i-1);
+        Long stop = endpoints.get(i);
+        int count = 0;
+        for (LinkState link : hiccups) {
+          // Two intervals are disjoint iff one's start > the other's end
+          // because there exists a point that partitions the one-dimensional
+          // space.
+          boolean disjoint = link.getHiccupStartTime() > stop
+            || start > link.getHiccupStopTime();
+          if (!disjoint) {
+            count ++;
+          }
+        }
+        maxCount = Math.max(maxCount, count);
+      }
+      return maxCount;
+    }
+  }
+
+  public static class Adversary {
+    private Chaos       chaos;
+    private long        previousNetworkChangeTime = 0L;
+    private TestConfig  config;
+
+    public Adversary(Chaos chaos, TestConfig config) {
+      this.chaos = chaos;
+      this.config = config;
+    }
+
+    public Chaos getChaos() {
+      return chaos;
+    }
+
+    public List<TestEvent> getEvents(long now) {
+      if (now - previousNetworkChangeTime > 1000L) {
+        LOG.info("Changing network delays ......");
+        chaos.updateNetworkStates(now, false);
+        previousNetworkChangeTime = now;
+      }
+      return new ArrayList<TestEvent>();
+    }
+  }
+
+
+  public static class TestEvent {
+    private EventType     type;
+    private long          when;
+    private List<String>  nodes;
+    private List<WALEdit> edits;
+    private long          sleepAmount;
+    private Map<String, Map<String, Long>> delayMap;
+
+    public static enum EventType {
+      START, STOP, NOOP, SLEEP, NEW_COMMITS, UPDATE_DELAYS, CRASH_NODES, START_NODES, DONE;
+    }
+
+    public TestEvent(EventType type) {
+      this(type, System.currentTimeMillis());
+    }
+
+    public TestEvent(EventType type, long when) {
+      this.type = type;
+      this.when = when;
+    }
+
+    public EventType type() {
+      return type;
+    }
+
+    public long when() {
+      return when;
+    }
+
+    public long sleepAmount() {
+      return sleepAmount;
+    }
+
+    public List<String> nodes() {
+      return nodes;
+    }
+
+    public Map<String, Map<String, Long>> delayMap() {
+      return delayMap;
+    }
+
+    public List<WALEdit> edits() {
+      return edits;
+    }
+
+    public TestEvent sleep(long ms) {
+      this.sleepAmount = ms;
+      return this;
+    }
+
+    public TestEvent crash(List<String> nodes) {
+      this.nodes = nodes;
+      return this;
+    }
+
+    public TestEvent start(List<String> nodes) {
+      this.nodes = nodes;
+      return this;
+    }
+
+    public TestEvent updateDelays(Map<String, Map<String, Long>> delayMap) {
+      this.delayMap = delayMap;
+      return this;
+    }
+
+    public TestEvent newCommits(List<WALEdit> edits) {
+      this.edits = edits;
+      return this;
+    }
+  }
+
+  public static class TestRequest implements Runnable {
+    public static enum State {
+      READY, RUNNING, DONE, FAILED
+    }
+    private LocalTestBed    testbed;
+    private List<WALEdit>   edits;
+    private State           state = State.READY;
+
+    public TestRequest(LocalTestBed testbed, List<WALEdit> edits) {
+      this.testbed = testbed;
+      this.edits = edits;
+    }
+
+    public synchronized State state() {
+      return state;
+    }
+
+    public void run() {
+      synchronized (this) { state = State.RUNNING; }
+      LOG.info("commiting ......");
+      testbed.dumpStates();
+      boolean done = testbed.testSingleCommit(edits);
+      synchronized (this) { state = done ? State.DONE : State.FAILED; }
+      testbed.incrementCommitCount(done);
+      LOG.info("DONE commiting ......");
+      testbed.dumpStates();
+    }
+  }
+
+
+  public static class TestConfig {
+
+    private Map<String, String> kvs = new HashMap<String, String>();
+
+    public TestConfig(String... args) {
+      kvs.put("seed", "" + System.currentTimeMillis());
+      kvs.put("packet-arrival-lambda", "100.0");
+      kvs.put("max-test-time", "3600000");
+      kvs.put("max-commits", "100");
+      kvs.put("event-loop-step", "1");
+      kvs.put("min-random-delay", "10");
+      kvs.put("max-random-delay", "100");
+      kvs.put("packet-drop-rate", "0.00");
+
+      for (String arg : args) {
+        String [] pair = arg.trim().split("=");
+        if (pair.length == 2) {
+          kvs.put(pair[0], pair[1]);
+        } else if (pair.length == 1) {
+          kvs.put(pair[0], "");
+        }
+      }
+    }
+
+    public String get(String key) {
+      return kvs.get(key);
+    }
+
+    public String get(String key, String defaultValue) {
+      if (kvs.get(key) == null) {
+        return defaultValue;
+      } else {
+        return kvs.get(key);
+      }
+    }
+
+    public double getDouble(String key, double defaultValue) {
+      String value = kvs.get(key);
+      if (value == null) {
+        return defaultValue;
+      }
+      try {
+        return Double.parseDouble(value);
+      } catch (Exception ex) {
+        return defaultValue;
+      }
+    }
+
+    public long getLong(String key, long defaultValue) {
+      String value = kvs.get(key);
+      if (value == null) {
+        return defaultValue;
+      }
+      try {
+        return Long.parseLong(value);
+      } catch (Exception ex) {
+        return defaultValue;
+      }
+    }
+
+    public int getInt(String key, int defaultValue) {
+      String value = kvs.get(key);
+      if (value == null) {
+        return defaultValue;
+      }
+      try {
+        return Integer.parseInt(value);
+      } catch (Exception ex) {
+        return defaultValue;
+      }
+    }
+  }
+
+  public static class Test implements Runnable {
+    private LocalTestBed  testbed;
+    private TestConfig    config;
+    private long          testStartTime;
+    private long          previousTick;
+    private int           ncommits = 0;
+    private Adversary     adversary;
+    private long          testEndTime;
+    private long          nextCommitTime;
+    private Random        prng;
+
+    public Test(TestConfig config, LocalTestBed testbed) {
+      this.config = config;
+      this.testbed = testbed;
+    }
+
+    public void init(Adversary adversary) {
+      prng = new Random(config.getLong("seed", System.currentTimeMillis()));
+      testStartTime = System.currentTimeMillis();
+      previousTick = 0L;
+      this.adversary = adversary;
+      nextCommitTime = testStartTime  + 5000L; // one second to settle
+      LOG.info("nextCommitTime = " + nextCommitTime);
+    }
+
+    public void incrementSuccessfulCommits() {
+    }
+
+    public List<TestEvent> getNextEvents(long now) {
+      List<TestEvent> events = new ArrayList<TestEvent>();
+      long wakeupTime = previousTick + config.getLong("event-loop-step", 50L);
+      long delay = Math.max(wakeupTime, now) - now;
+      if (delay > 0) {
+        noThrowSleep(delay);
+      }
+      previousTick = Math.max(wakeupTime, now);
+
+      for (TestEvent event : adversary.getEvents(now)) {
+        events.add(event);
+      }
+      // flip a coin
+      if (now >= nextCommitTime) {
+        events.add(new TestEvent(TestEvent.EventType.NEW_COMMITS).newCommits(generateTestingWALEdit()));
+        nextCommitTime += poissonRandomInterarrivalDelay(prng, config.getDouble("packet-arrival-lambda", 100));
+        ncommits ++;
+      }
+      return events;
+    }
+
+    public void run() {
+      BlockingQueue<Runnable> workQueue = new LinkedBlockingQueue<Runnable>();
+      ThreadPoolExecutor fakeClient = new ThreadPoolExecutor(1, 1, 60, TimeUnit.SECONDS, workQueue);
+      long previousStateDumpTime = 0L;
+      List<TestRequest> requests = new ArrayList<TestRequest>();
+      List<TestRequest> processedRequests = new ArrayList<TestRequest>();
+  mainloop:
+      while ((long)processedRequests.size() < config.getLong("max-commits", 10L)) {
+        List<TestEvent> events = getNextEvents(System.currentTimeMillis());
+        for (TestEvent event : events) {
+          switch (event.type()) {
+            case SLEEP:
+              noThrowSleep(event.sleepAmount());
+              break;
+
+            case NEW_COMMITS:
+              if (requests.size() <= 0) {
+                TestRequest request = new TestRequest(testbed, event.edits());
+                requests.add(request);
+                fakeClient.execute(request);
+              }
+              break;
+
+            case UPDATE_DELAYS:
+              break;
+
+            case CRASH_NODES:
+              break;
+
+            case START_NODES:
+              break;
+
+            default:
+              break;
+          }
+          if (testbed.checkLeaderCount()) {
+            Assert.assertTrue(testbed.getLeaderCount() < 2);
+          }
+        }
+        List<TestRequest> tmp = new ArrayList<TestRequest>();
+        for (TestRequest request : requests) {
+          switch(request.state()) {
+            case FAILED:
+            case DONE:
+              processedRequests.add(request);
+              break;
+            default:
+              tmp.add(request);
+          }
+        }
+        requests = tmp;
+        if (previousStateDumpTime + 2000L < System.currentTimeMillis()) {
+          long t1 = System.nanoTime();
+          testbed.dumpStates();
+          long t2 = System.nanoTime();
+          LOG.info("Dumping States took " + (t2-t1)/1000L + " us");
+          previousStateDumpTime = System.currentTimeMillis();
+        }
+      }
+
+      LOG.info(
+          "\n\n"
+          + "-------------------------------------------\n"
+          + "SHUTTING DOWN THE SIMULATOR ......\n"
+          + "-------------------------------------------\n"
+      );
+
+      fakeClient.shutdown();
+
+      try {
+        fakeClient.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);
+      } catch (InterruptedException ex) {
+        LOG.error("Time to die!");
+        System.exit(1);
+      }
+
+      int ndone = 0;
+      int nfail = 0;
+      for (TestRequest request : processedRequests) {
+        switch (request.state()) {
+          case FAILED:
+            nfail ++;
+            break;
+          case DONE:
+            ndone ++;
+            break;
+
+          default:
+            LOG.error("There are requests that have not been processed!");
+            System.exit(1);
+        }
+      }
+      LOG.info("\n\n"
+          + "--------------------------------------------------------\n"
+          + "Successfully committed " + ndone  + " client requests\n"
+          + "    Stats\n"
+          + "    - droppped " + testbed.getPacketDropCount() + " packets (" + testbed.getHiccupPacketDropCount() + " due to hiccups)\n"
+          + "--------------------------------------------------------\n"
+      );
+      if (nfail > 0) {
+        LOG.error("There are failed requests!");
+        System.exit(1);
+      }
+
+      LOG.info("Clearing network states ......");
+      adversary.getChaos().updateNetworkStates(System.currentTimeMillis(), true);
+      LOG.info("-------- Verifying log consistency amongst all quorum members");
+      while (!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE, true)) {
+        testbed.dumpStates();
+        if (testbed.checkLeaderCount()) {
+          Assert.assertTrue(testbed.getLeaderCount() < 2);
+        }
+        noThrowSleep(5000L);
+      }
+      LOG.info("-------- Done verifying log consistency amongst all quorum members");
+      testbed.dumpStates();
+    }
+  }
+
+
+  public void start() throws Exception {
+    start(new TestConfig());
+  }
+
+  public void start(TestConfig config) throws Exception {
+    PeerHandleAppendResponse.setGlobalMaxBatchLogs((int)config.getLong("max-batch-logs", 64));
+    int port  = config.getInt("regionserver-port", -1);
+    if (port > 0) {
+      LOG.info("Setting the HBASE region server port to " + port);
+      RAFT_TEST_UTIL.getConf().setInt(HConstants.REGIONSERVER_PORT, port);
+    }
+
+    long timeout = config.getLong("handle-rpc-timeout", -1);
+    if (timeout > 0) {
+      LOG.info("Setting the RPC Error timeout to " + timeout + " ms");
+      RAFT_TEST_UTIL.getConf().setLong(HConstants.RAFT_PEERSERVER_HANDLE_RPC_TIMEOUT_MS, timeout);
+    }
+
+    RAFT_TEST_UTIL.setUsePeristentLog(usePersistentLog);
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, mockLogs);
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    checker = new HealthChecker(regionInfo, RAFT_TEST_UTIL, autoRestartThriftServer, 30000L);
+    checker.start();
+  }
+
+  public void stop() throws Exception {
+    if (checker != null) {
+      checker.shutdown();
+    }
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  public LocalTestBed() {
+    this(null);
+  }
+
+  public LocalTestBed(List<int[]> logs) {
+    if (logs == null) {
+      logs = new ArrayList<int[]>();
+      for (int i=0; i<QUORUM_SIZE; i++) {
+        logs.add(new int[] {});
+      }
+    }
+    mockLogs = logs;
+  }
+
+  private static List<WALEdit> generateTestingWALEdit() {
+    KeyValue kv = KeyValue.createFirstOnRow(Bytes.toBytes("TestQuorum"));
+    return Arrays.asList(new WALEdit(Arrays.asList(kv)));
+  }
+
+  public void dumpStates() {
+    RAFT_TEST_UTIL.dumpStates(regionInfo);
+    LOG.info("Total Commit = " + commitSuccessCount.get()+ " successes and " + commitFailureCount.get() + " failures "
+        + " with " + getPacketDropCount()  + "(" + getHiccupPacketDropCount() + ") total-dropped (hiccup) packets "
+        + " and " + RAFT_TEST_UTIL.getServerRestartCount() + " server restarts "
+        + " in " + (System.currentTimeMillis() - BigBang)/1000L + " secs");
+  }
+
+  public void incrementCommitCount(boolean success) {
+    if (success) {
+      commitSuccessCount.incrementAndGet();
+    } else {
+      commitFailureCount.incrementAndGet();
+    }
+  }
+
+  private boolean doCommit(final WALEdit edit) {
+    if (edit == null) {
+      return true;
+    }
+    retry_loop:
+    while (true) {
+      try {
+        QuorumAgent agent = getLeader();
+        if (agent == null) {
+          LOG.info("NULL leader. Sleep for some time ......");
+          try {
+            Thread.sleep(1000L);
+          } catch (InterruptedException ex) {
+            LOG.info("Time to Die!");
+            return false;
+          }
+          continue;
+        }
+        agent.syncAppend(edit);
+        // Verify all the logs across the majority are the same
+        RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_MAJORITY, false);
+        return true;
+      } catch (NewLeaderException e) {
+        LOG.warn("Got a new leader in the quorum: " + e.getNewLeaderAddress());
+        RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_MAJORITY, false);
+      } catch (Exception e) {
+        Throwable cause = e;
+        while (cause != null) {
+          if (cause instanceof NewLeaderException || cause instanceof LeaderNotReadyException) {
+            continue retry_loop;
+          }
+          cause = cause.getCause();
+        }
+        LOG.error("Errors: ", e);
+      }
+    }
+  }
+
+  public boolean testSingleCommit(List<WALEdit> edits) {
+    boolean success = true;
+    if (edits.size() == 0) {
+      return true;
+    }
+
+    for (final WALEdit edit : edits) {
+      if ((success = doCommit(edit)) == false) {
+        break;
+      }
+    }
+    return success;
+  }
+
+  public QuorumAgent getLeader() {
+    RaftQuorumContext leaderQuorum = null;
+    do {
+      int leaderCnt = 0;
+      for (LocalConsensusServer server : RAFT_TEST_UTIL.getServers().values()) {
+        RaftQuorumContext c = server.getHandler().getRaftQuorumContext(regionInfo.getEncodedName());
+        if (c != null && c.isLeader()) {
+          leaderQuorum = c;
+          leaderCnt++;
+        }
+      }
+      if (checkLeaderCount()) {
+        Assert.assertTrue(getLeaderCount() < 2);
+      }
+    } while (leaderQuorum == null );
+    return leaderQuorum.getQuorumAgentInstance();
+  }
+
+  public int getLeaderCount() {
+    int leaderCnt = 0;
+    for (LocalConsensusServer server : RAFT_TEST_UTIL.getServers().values()) {
+      RaftQuorumContext c = server.getHandler().getRaftQuorumContext(regionInfo.getEncodedName());
+      if (c != null && c.isLeader()) {
+        leaderCnt++;
+      }
+    }
+    return leaderCnt;
+  }
+
+
+
+  public static void noThrowSleep(long ms) {
+    try {
+      Thread.sleep(ms);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  public void runConsensusProtocol() throws Exception {
+    runConsensusProtocol(new TestConfig());
+  }
+
+  public void runConsensusProtocol(TestConfig config) throws Exception {
+    start(config);
+
+    try {
+      Chaos chaos = new Chaos(config);
+      Test test = new Test(config, this);
+      Adversary adversary = new Adversary(chaos, config);
+      test.init(adversary);
+      test.run();
+      LOG.info("Stopping the experiment ......");
+      stop();
+    } catch (Throwable fit) {
+      LOG.error("runConsensusProtocol caught a fit", fit);
+      System.exit(1);
+    }
+  }
+
+  public boolean checkLeaderCount() {
+    return checkLeaderCount;
+  }
+
+  public void parseOptions(String[] args) {
+    for (String arg : args) {
+      if (arg.equalsIgnoreCase("--check-leader-count")) {
+        checkLeaderCount = true;
+      } else if (arg.equalsIgnoreCase("--no-check-leader-count")) {
+        checkLeaderCount = false;
+      } else if (arg.equalsIgnoreCase("--persistent-log")) {
+        usePersistentLog = true;
+      } else if (arg.equalsIgnoreCase("--no-persistent-log")) {
+        usePersistentLog = false;
+      } else if (arg.equalsIgnoreCase("--auto-restart-thrift")) {
+        autoRestartThriftServer = true;
+      } else if (arg.equalsIgnoreCase("--no-auto-restart-thrift")) {
+        autoRestartThriftServer = false;
+      }
+    }
+  }
+
+  public long getPacketDropCount() {
+    return RAFT_TEST_UTIL.getPacketDropCount(regionInfo);
+  }
+
+  public long getHiccupPacketDropCount() {
+    return RAFT_TEST_UTIL.getHiccupPacketDropCount(regionInfo);
+  }
+
+  // shoplifted from http://preshing.com/20111007/how-to-generate-random-timings-for-a-poisson-process/
+  public static long poissonRandomInterarrivalDelay(Random prng, double lambda) {
+    return Math.max(0L, (long)(-Math.log(1.0 - prng.nextDouble()) * lambda));
+  }
+
+  public static void main(String[] args) throws Exception {
+    LocalTestBed testbed = new LocalTestBed();
+    testbed.parseOptions(args);
+    testbed.runConsensusProtocol(new TestConfig(args));
+    LOG.info("\n\n"
+        + "--------------------------------------------------------\n"
+        + "SUCCESS\n"
+        + "    Stats\n"
+        + "    - droppped " + testbed.getPacketDropCount() + " packets (" + testbed.getHiccupPacketDropCount() + " due to hiccups)\n"
+        + "--------------------------------------------------------\n"
+    );
+    System.exit(0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/RaftTestDataProvider.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/RaftTestDataProvider.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/RaftTestDataProvider.java
new file mode 100644
index 0000000..9efefb9
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/RaftTestDataProvider.java
@@ -0,0 +1,130 @@
+package org.apache.hadoop.hbase.consensus;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class RaftTestDataProvider {
+
+  public static List<Object[]> getRaftBasicLogTestData() {
+    List<int[]> test1 = Arrays.asList(new int[][]{
+      {1},
+      {1},
+      {1},
+      {1},
+      {1}
+    });
+
+    List<int[]> test2 = Arrays.asList(new int[][] {
+      {1, 2},
+      {1},
+      {1},
+      {1},
+      {1}
+    });
+
+    List<int[]> test3 = Arrays.asList(new int[][] {
+      {1, 2},
+      {1, 2},
+      {1},
+      {1},
+      {1}
+    });
+
+    List<int[]> test4 = Arrays.asList(new int[][] {
+      {1, 2},
+      {1, 2},
+      {1, 2},
+      {1},
+      {1}
+    });
+
+
+    List<int[]> test5 = Arrays.asList(new int[][] {
+      {1, 2},
+      {1, 2},
+      {1, 2},
+      {1, 2},
+      {1}
+    });
+
+
+    List<int[]> test6 = Arrays.asList(new int[][] {
+      {1, 2},
+      {1, 2},
+      {1, 3},
+      {1, 3},
+      {1}
+    });
+
+    List<int[]> test7 = Arrays.asList(new int[][] {
+      {1, 2},
+      {1, 2},
+      {1, 3},
+      {1, 3},
+      {1, 3}
+    });
+
+    List<int[]> test8 = Arrays.asList(new int[][] {
+      {1, 1, 2},
+      {1, 1, 1},
+      {1, 1, 2},
+      {1, 1, 1},
+      {}
+    });
+
+    List<int[]> test9 = Arrays.asList(new int[][] {
+      {1, 1},
+      {1},
+      {1},
+      {},
+      {}
+    });
+
+    List<int[]> test10 = Arrays.asList(new int[][] {
+      {1, 1, 1, 4, 4, 5, 5, 6, 6, 6},
+      {1, 1, 1, 4},
+      {1, 1, 1, 4, 4, 5, 5, 6, 6, 6, 7},
+      {},
+      {}
+    });
+
+
+    Object[][] data = new Object[][] {
+      { test1 },
+      { test2 },
+      { test3 },
+      { test4 },
+      { test5 },
+      { test6 },
+      { test7 },
+      { test8 },
+      { test9 },
+      { test10 }
+    };
+    return Arrays.asList(data);
+  }
+
+  public static List<Object[]> getRaftBasicLogTestSeedData() {
+    List<int[]> test1 = Arrays.asList(new int[][]{
+      {1, 1, 1, 1, 1, 1 ,1},
+      {1, 1, 1},
+      {1, 1, 1, 1, 1, 1, 1},
+      {1, 1, 1, 1},
+      {1, 1, 1, 1, 1, 1, 1}
+    });
+
+    List<int[]> test2 = Arrays.asList(new int[][] {
+      {1, 1, 1, 1, 1, 1 ,1},
+      {1, 1, 1, 1, 1, 1 ,1},
+      {1, 1, 2},
+      {1},
+      {1, 1, 1, 1, 1, 1 ,1}
+    });
+
+    Object[][] data = new Object[][] {
+      { test1 },
+      { test2 },
+    };
+    return Arrays.asList(data);
+  }
+}


[12/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/TransactionLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/TransactionLogManager.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/TransactionLogManager.java
new file mode 100644
index 0000000..0c343a8
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/TransactionLogManager.java
@@ -0,0 +1,1827 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.consensus.exceptions.NotEnoughMemoryException;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.apache.hadoop.hbase.consensus.rpc.LogState;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.io.hfile.bucket.CacheFullException;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.BucketAllocatorException;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileLock;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * Manages the logs for the Consensus Protocol and provides interfaces to
+ * perform read/write/truncate operations to the logs.
+ * <p/>
+ * This is and should be the single point of entry to access any consensus logs.
+ * <p/>
+ * Layout:
+ * <p/>
+ * All the logs will be stored in the 'TRANSACTION_LOG_DIRECTORY/quorum-name/'
+ * directory. The directory will have 2 folders:
+ * 1. /current : contains the current random access log
+ * 2. /finalized : contains all the logs  for which the entries have been
+ *                 committed to the data-store
+ * <p/>
+ * Life cycle of a log file:
+ * <p/>
+ * 1. Every log (apart from recovery) will start in a Random-Access mode in
+ * '/current' directory. All the append() operations will be going to this log.
+ * The log could be potentially truncated during this phase.
+ * 2. Once the size of the log reaches the LOG_ROLL_SIZE and all the entries
+ * are committed to the data-store the log will be finalized and moved to the
+ * 'finalized' directory. Now the log is available in ReadOnly mode and cannot
+ * be truncated, but still can be deleted.
+ * 3. Once all the entries from the log are flushed by the data store, the log
+ * will be then a part of retiredLogs and will be eventually deleted depending
+ * upon the LOG_RETENTION_POLICY.
+ *
+ * There are two sets of logs maintained by the log manager:
+ * 1. uncommittedLogs => log files which reside in the /current dir and contains
+ * all the uncommitted edits.
+ * 2. committedLogs => logs files which reside in the /finalized dir and contains
+ * all the committed edits.
+ *
+ * * The log roller will always grab the write lock on the logRollerLock before
+ * moving the file from /current to /finalized directory.
+ *
+ * * All the read operations grab the read lock across the entire operation to
+ * prevent the log rolling from happening.
+ *
+ * * append/truncate will always be writing to files which will never be rolled
+ * at the same time. Hence, they do not grab any logRollLock.
+ *
+ * The invariants are:
+ * 1. For any given index, there should be only ONE entry in ONE log file.
+ * 2. There should never be a gap in index and hence it should always be
+ * monotonically increasing.
+ *
+ * CANDIDATE LOGS
+ *
+ * In addition there is an optional list of "Candidate Logs" which hold committed
+ * transactions and are retrieved from other members during recovery. These logs
+ * files are held by candidateLogsManager. On every append/truncate operation,
+ * candidateLogsManager will be asked to check if some files it holds can be
+ * incorporated into the uncommittedLogs set.
+ *
+ * Please refer to the comments before incorporateCandidateLogs for the correctness
+ * analysis.
+ */
+public class TransactionLogManager implements CommitLogManagerInterface {
+
+  private final Logger LOG = LoggerFactory.getLogger(
+    TransactionLogManager.class);
+
+  private String logDirectory;
+
+  /** Helps debugging. */
+  private final String contextName;
+
+  private final Configuration conf;
+
+  private final boolean isSync;
+  private volatile boolean isAccessible = true;
+  private ImmutableRaftContext context;
+
+  /** RW lock to manage the log roll and read access to the log */
+  private final ReentrantReadWriteLock logRollLock;
+
+  /** Log roller thread pool */
+  private final static ScheduledExecutorService logRollerThreadPool =
+    Executors.newScheduledThreadPool(HConstants.RAFT_LOG_ROLL_POOL_SIZE, new DaemonThreadFactory("logRoller-"));
+
+  /** Log deleter thread pool */
+  private final static ScheduledExecutorService logDeleteThreadPool =
+    Executors.newScheduledThreadPool(HConstants.RAFT_LOG_ROLL_POOL_SIZE, new DaemonThreadFactory("logDeleter-"));
+
+  // TODO: make it volatile if we want to update it via online configuration change
+  /** If current log exceeds this value, roll to a new log */
+  private long raftLogRollSize;
+
+  /** last known committed index */
+  private volatile long committedIndex;
+
+  /** Seed point for the transaction log manager. The transaction log manager
+   * will bootstrap from this index. It will basically create a empty file, which
+   * represents the missing indexes.
+   *
+   * In case any attempt is made to fetch the transaction for index from this
+   * empty log file will lead into an error.
+   */
+  private long seedIndex;
+
+  private long currentFileSize;
+
+  private boolean isCheckInvariants = false;
+
+  /**
+   * LogReader may prefetch logs to reduce the latency of reading entries.
+   * This specifies how much to prefetch.
+   */
+  private static int logPrefetchSize;
+  /**
+   * Map of log files which have been rolled but the entries might not have
+   * been committed to the data store.
+   */
+  ConcurrentSkipListMap<Long, LogFileInterface> uncommittedLogs;
+
+  /**
+   * Map of logs for which entries have been committed by the DataStore.
+   */
+  ConcurrentSkipListMap<Long, LogFileInterface> committedLogs;
+
+  /**
+   *  Candidate transaction logs that hold already committed transactions;
+   *  they are fetched from other hosts.
+   */
+  CandidateLogsManager candidateLogsManager;
+
+  /** Log creator for this log manager. */
+  TransactionLogCreator logCreator;
+
+  ConcurrentSkipListMap<Long, ByteBuffer> uncommittedTransactions;
+
+  /** Retention time for committed logs */
+  private long transactionLogRetentionTime;
+
+  /** Deletion Task */
+  private ScheduledFuture logDeletionTask;
+
+  /** Log roll task */
+  private ScheduledFuture logRollTask;
+
+  public static enum FindLogType {
+    UNCOMMITTED_ONLY, COMMITTED_ONLY, ALL
+  }
+
+  public TransactionLogManager(Configuration conf, String contextName, long seedIndex) {
+    this.conf = conf;
+    String [] logDirectories = conf.get(
+        HConstants.RAFT_TRANSACTION_LOG_DIRECTORY_KEY,
+        HConstants.DEFAULT_TRANSACTION_LOG_DIRECTORY
+    ).split(",");
+    int idx = Math.abs(contextName.hashCode()) % logDirectories.length;
+
+
+    // need to jump down to decide on a logDirectory
+
+    logDirectory = logDirectories[idx];
+    if (!this.logDirectory.endsWith(HConstants.PATH_SEPARATOR)) {
+      logDirectory = logDirectory + HConstants.PATH_SEPARATOR;
+    }
+
+    logDirectory += contextName + "/";
+
+    transactionLogRetentionTime = conf.getLong(
+      HConstants.CONSENSUS_TRANCTION_LOG_RETENTION_TIME_KEY,
+      HConstants.CONSENSUS_TRANCTION_LOG_RETENTION_TIME_DEFAULT_VALUE);
+
+    isSync = conf.getBoolean(HConstants.RAFT_TRANSACTION_LOG_IS_SYNC_KEY,
+        HConstants.RAFT_TRANSACTION_LOG_IS_SYNC_DEFAULT);
+    isCheckInvariants = conf.getBoolean("hbase.consensus.log.isCheckInvariants",
+      false);
+    this.contextName = contextName;
+
+    logRollLock = new ReentrantReadWriteLock();
+    uncommittedLogs = new ConcurrentSkipListMap<>();
+    committedLogs = new ConcurrentSkipListMap<>();
+
+    raftLogRollSize = conf.getLong(HConstants.RAFT_LOG_ROLL_SIZE_KEY, HConstants.DEFAULT_RAFT_LOG_ROLL_SIZE);
+
+    uncommittedTransactions = new ConcurrentSkipListMap<>();
+
+    long logRollInterval = conf.getInt(
+      HConstants.RAFT_LOG_ROLL_INTERVAL_KEY,
+      HConstants.DEFAULT_RAFT_LOG_ROLL_INTERVAL);
+
+    logPrefetchSize = conf.getInt(HConstants.RAFT_LOG_READER_PREFETCH_KEY,
+        HConstants.DEFAULT_RAFT_LOG_READER_PREFETCH_SIZE);
+
+    logRollTask = logRollerThreadPool.scheduleAtFixedRate(new Runnable() {
+      @Override public void run() {
+        rollCommittedLogs();
+      }
+    }, logRollInterval, logRollInterval, TimeUnit.MILLISECONDS);
+
+    long logDeletionInterval = conf.getInt(
+      HConstants.RAFT_LOG_DELETION_INTERVAL_KEY,
+      HConstants.DEFAULT_RAFT_LOG_DELETION_INTERVAL);
+
+    logDeletionTask = logDeleteThreadPool.scheduleAtFixedRate(new Runnable() {
+      @Override public void run() {
+        deleteOldLogs();
+      }
+    }, logDeletionInterval, logDeletionInterval, TimeUnit.MILLISECONDS);
+
+    candidateLogsManager = createCandidateLogsManager(conf, contextName,
+      logDirectory);
+
+    this.seedIndex = seedIndex;
+  }
+
+  public CandidateLogsManager getCandidateLogsManager() {
+    return candidateLogsManager;
+  }
+
+  @Override
+  public String toString() {
+    return "@" + hashCode() + "[" + contextName + "]";
+  }
+
+  @Override
+  public void initialize(final ImmutableRaftContext c) {
+    try {
+      context = c;
+      initializeDirectories();
+      scanFinalizedLogs();
+      scanCurrentLogs();
+      logCreator = createTransactionLogCreator(logDirectory, isSync);
+//      scanCandidateLogs();
+    } catch (IOException e) {
+      isAccessible = false;
+      error("Initialization failed. Error ", e);
+    }
+  }
+
+  /**
+   * Will check if there is any gap between the seed index and the latest index
+   * present in the logs.
+   */
+  @Override
+  public void fillLogGap(long seedIndex) throws IOException {
+    LOG.info("Filling the log gap upto " + seedIndex + " for " + contextName);
+
+    this.seedIndex = seedIndex;
+
+    long latestIndex = this.getLastEditID().getIndex();
+
+    if (latestIndex >= seedIndex) {
+      // It might happen that the old logs are deleted and we are just left with
+      // the last file, which might not have any entries committed. In that case
+      // the committed index is -1, and we can assume that the whatever the data
+      // store returns the seedIndex is the minimum committed index
+      if (committedIndex < seedIndex) {
+        committedIndex = seedIndex;
+      }
+      return;
+    }
+
+    // At this point the seedIndex is greater than the latest index in the
+    // transaction logs.
+    //
+    // We can safely assume that all the entries up to lastValidTransaction are
+    // committed successfully.
+
+    // 1. Truncate all the entries beyond lastValidTransaction.
+    truncate(getLastValidTransactionId());
+
+    latestIndex = this.getLastEditID().getIndex();
+
+    // 2. Roll the uncommitted logs as all the entries are committed.
+    Iterator<Map.Entry<Long, LogFileInterface>> itr =
+      uncommittedLogs.entrySet().iterator();
+
+    Map.Entry<Long, LogFileInterface> entry;
+    while (itr.hasNext()) {
+      entry = itr.next();
+      performRollLog((RandomAccessLog)entry.getValue());
+    }
+    if (!uncommittedLogs.isEmpty()) {
+      throw new IOException("Cannot roll all the uncommitted logs");
+    }
+
+    // 3. Create a SeedLogFile with range {lastIndex + 1, seedIndex}
+
+    // Create a parse seed file with the range of latestIndex + 1 to seedIndex.
+    // The Automatic Log File Fetcher will fetch the entries over time and fill
+    // in this gap.
+    StringBuilder fileName = new StringBuilder();
+    fileName.append(HConstants.SEED_TERM);
+    fileName.append("_");
+    fileName.append(latestIndex + 1);
+    fileName.append("_");
+    fileName.append(seedIndex);
+
+    File seedFile = new File(logDirectory +
+      HConstants.RAFT_FINALIZED_LOG_DIRECTORY_NAME + HConstants.PATH_SEPARATOR +
+      fileName.toString());
+    if (!seedFile.createNewFile()) {
+      LOG.error("Cannot create a seed file " + fileName);
+      throw new IOException("Cannot create a seed file " + fileName);
+    }
+
+    if (!seedFile.exists()) {
+      LOG.error("Cannot verify the seed file " + fileName);
+      throw new IOException("Seed file not present " + fileName);
+    }
+
+    seedFile.setLastModified(System.currentTimeMillis());
+
+    // Make the entry for the dummy file
+    committedLogs.put(latestIndex + 1, new SeedLogFile(seedFile));
+
+    // Set the committed index to be the seed index
+    committedIndex = seedIndex;
+
+    if (LOG.isDebugEnabled()) {
+      LOG.error("Created a seed file " + fileName);
+    }
+  }
+
+  protected void initializeDirectories() throws IOException {
+    createLogDirectory(logDirectory);
+    createLogDirectory(logDirectory +
+      HConstants.RAFT_CURRENT_LOG_DIRECTORY_NAME);
+    createLogDirectory(logDirectory +
+        HConstants.RAFT_FINALIZED_LOG_DIRECTORY_NAME);
+  }
+
+  /**
+   *  createLogDirectory
+   *
+   *  Creates the directory (including the parent dir).
+   *
+   *  Mockable
+   */
+  protected void createLogDirectory(String path) throws IOException {
+    File currentDirectory = new File (path);
+
+    if (!currentDirectory.exists()) {
+      if (!currentDirectory.mkdirs()) {
+        throw new IOException("Cannot create dir at " +
+          currentDirectory.getAbsolutePath());
+      }
+    }
+  }
+
+  /**
+   * Scans the finalized directory for this quorum and updates the maps
+   * accordingly.
+   * @throws IOException
+   */
+  private void scanFinalizedLogs() throws IOException {
+    List<File> files = null;
+    try {
+      files = getFinalizedLogsList();
+    } catch (IOException x) {
+      error("Cannot read the finalized directory. Error ", x);
+    }
+    if (files == null) {
+      return;
+    }
+    for (File f : files) {
+      try {
+        LogFileInterface logFile = createReadOnlyLog(f, this.toString());
+        long index = logFile.getInitialIndex();
+        assert index != HConstants.UNDEFINED_TERM_INDEX;
+
+        committedLogs.put(index, logFile);
+      } catch (Exception ex) {
+        error("failed to add finalized log " + f.getAbsolutePath(), ex);
+      }
+    }
+  }
+
+  /**
+   *  scanCurrentLogs
+   *
+   *  Scans the logs in the current directory and updates the in-memory maps.
+   *  @throws IOException
+   */
+  private void scanCurrentLogs() throws IOException {
+    List<File> files = null;
+    try {
+      files = getCurrentLogsList();
+    } catch (IOException x) {
+      error("Cannot read the current directory. Error ", x);
+    }
+    if (files == null) {
+      return;
+    }
+    for (File f : files) {
+      try {
+        // Close the files which are not initialized
+        RandomAccessLog logFile = createRandomAccessLog(f, isSync);
+        try {
+          // Rebuild the index by scanning the log file
+          logFile.rebuild(toString());
+        } catch (IOException e) {
+          LOG.warn(this.contextName + " is unable to rebuild the current log file " +
+            logFile.getFileName() + ". Deleting it..");
+        }
+
+        if (logFile.getInitialIndex() == HConstants.UNDEFINED_TERM_INDEX) {
+          logFile.closeAndDelete();
+        } else {
+          uncommittedLogs.put(logFile.getInitialIndex(), logFile);
+        }
+      } catch (IOException ex) {
+        error("Cannot add current log file " + f.getAbsolutePath(), ex);
+      }
+    }
+    committedIndex = findLastValidTransactionId().getIndex();
+  }
+
+  /**
+   *  getFinalizedLogsList
+   *
+   *  Returns a list of files in finalized logs directory.
+   *
+   *  Mockable
+   */
+  protected List<File> getFinalizedLogsList() throws IOException {
+    Path finalizedDirectory = Paths.get(logDirectory +
+      HConstants.RAFT_FINALIZED_LOG_DIRECTORY_NAME);
+
+    DirectoryStream<Path> stream = Files.newDirectoryStream(finalizedDirectory);
+    List<File> files = new ArrayList<>();
+    for (Path entry: stream) {
+      files.add(entry.toFile());
+    }
+    return files;
+  }
+
+  /**
+   *
+   * getCurrentLogsList, Mockable
+   *
+   * @return Returns a list of files in the current logs directory.
+   * @throws IOException
+   */
+  protected List<File> getCurrentLogsList() throws IOException {
+    Path currentDirectory = Paths.get(logDirectory +
+      HConstants.RAFT_CURRENT_LOG_DIRECTORY_NAME);
+    // Gets the list of files in the current directory
+    DirectoryStream<Path> stream = Files.newDirectoryStream(currentDirectory);
+    List<File> files = new ArrayList<>();
+    for (Path entry: stream) {
+      files.add(entry.toFile());
+    }
+    return files;
+  }
+
+  /**
+   * Append the requested entry to the current log
+   * @param editId Edit Id to append
+   * @param commitIndex the new commit index
+   * @param txns transaction for the commit index
+   * @return true if successful
+   */
+  @Override
+  public boolean append(final EditId editId, long commitIndex,
+                        final ByteBuffer txns) {
+    try {
+      append(editId.getTerm(), editId.getIndex(), commitIndex, txns);
+    } catch (IOException | InterruptedException | IllegalArgumentException e) {
+      error("Failed to append " + editId + ". Reason: ", e);
+      isAccessible = false;
+    }
+    return isAccessible;
+  }
+
+  /**
+   * Tells whether the log is accessible or not.
+   * @return true if the log is still accessible
+   */
+  public boolean isAccessible() {
+    return this.isAccessible;
+  }
+
+  /**
+   * Returns the previous edit id.
+   * @param editId return the id previous to the given edit id
+   * @return UNDEFINED_EDIT_ID, if no previous edit id
+   */
+  @Override
+  public EditId getPreviousEditID(EditId editId) {
+
+    logRollLock.readLock().lock();
+    try {
+      long term = getTermInternal(editId.getIndex() - 1);
+      if (term == HConstants.UNDEFINED_TERM_INDEX) {
+        return UNDEFINED_EDIT_ID;
+      }
+      return new EditId(term, editId.getIndex() - 1);
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+  }
+
+  /**
+   *  getNextEditIdTransaction
+   *
+   *  Returns the next EditId and the transaction associated with it.
+   *
+   *  @param  sessionKey  String reader's session key
+   *  @param  currentIndex   EditId edit id
+   *  @param  arena       Arena to use to allocate Memory. If Null is passed it
+   *  @return Pair<EditId, MemoryBuffer> the edit id and the corresponding
+   *          transaction
+   *
+   */
+  @Override
+  public Pair<EditId, MemoryBuffer> getNextEditIdTransaction(
+    final String sessionKey,
+    final long currentIndex,
+    final Arena arena) throws IOException {
+    EditId nextId = null;
+    LogFileInterface log = null;
+    ByteBuffer transaction = null;
+
+    logRollLock.readLock().lock();
+    long start = System.nanoTime();
+    try {
+      long term = getTermInternal(currentIndex + 1);
+
+      // return a valid transaction only. In case, it's an edit in Seed File
+      // return null
+      if (term >= 0) {
+        nextId = new EditId(term, currentIndex + 1);
+        if ((transaction =
+          uncommittedTransactions.get(nextId.getIndex())) == null) {
+          log = getLogFileForEdit(nextId.getIndex());
+        }
+      }
+
+      if (transaction != null) {
+        MemoryBuffer buffer = allocateBuffer(arena, transaction.remaining());
+
+        buffer.getBuffer().put(transaction.array(),
+          transaction.arrayOffset() + transaction.position(),
+          transaction.remaining());
+        buffer.flip();
+        return new Pair<>(nextId, buffer);
+      }
+
+      if (log == null) {
+        return null;
+      }
+
+      // At this point, the entry you are reading is supposed to be
+      // committed entry
+
+      return new Pair<>(nextId, getTransactionFromLog(log, nextId, sessionKey, arena));
+    } finally {
+      long end = System.nanoTime();
+      logRollLock.readLock().unlock();
+      long timeTaken = end - start;
+      long threshold = 10000000; // 10 ms
+      if (timeTaken > threshold) {
+        LOG.debug("Fetching nextEditId took too long. It took {} ns. currentIndex {}", timeTaken, currentIndex);
+      }
+    }
+  }
+
+  /**
+   * Returns the last valid transaction from the log.
+   *
+   * @return the last valid transaction
+   * @throws IOException
+   */
+  @Override
+  public EditId getLastValidTransactionId() {
+    return getEditId(committedIndex);
+  }
+
+  /**
+   * It tries to find the latest term for which there are at least two entries
+   * committed to the log and returns the second last entry for that term.
+   *
+   * @return
+   */
+  private EditId findLastValidTransactionId() {
+    logRollLock.readLock().lock();
+    try {
+      // Check the current log first.
+      // Get the latest log for which there are more than 1 entries.
+      for (long index : uncommittedLogs.descendingKeySet()) {
+        LogFileInterface file = uncommittedLogs.get(index);
+        if (file.getTxnCount() > 1) {
+          return new EditId(file.getCurrentTerm(), file.getLastIndex() - 1);
+        }
+      }
+
+      // Since all the entries in committedLogs are already committed to the
+      // data store, return the last entry of the latest log file in this
+      // list.
+      if (!committedLogs.isEmpty()) {
+        LogFileInterface file = committedLogs.lastEntry().getValue();
+        return new EditId(file.getCurrentTerm(), file.getLastIndex());
+      }
+      return UNDEFINED_EDIT_ID;
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+  }
+
+  /**
+   * @return the first index in the log
+   * @throws IOException
+   */
+  @Override
+  public long getFirstIndex() {
+    logRollLock.readLock().lock();
+    try {
+      ConcurrentSkipListMap.Entry<Long, ? extends LogFileInterface> entry;
+      ConcurrentSkipListMap.Entry<Long, ByteBuffer> txn;
+
+      if ((entry = committedLogs.firstEntry()) != null) {
+        return entry.getValue().getInitialIndex();
+      } else if ((entry = uncommittedLogs.firstEntry()) != null) {
+        return entry.getValue().getInitialIndex();
+      } else if ((txn = this.uncommittedTransactions.firstEntry()) != null) {
+        return txn.getKey();
+      }
+
+      return HConstants.UNDEFINED_TERM_INDEX;
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+  }
+
+  /**
+   * Checks whether the given transaction is present in the log or not.
+   * @param editId
+   * @return
+   */
+  @Override
+  public boolean isExist(EditId editId) {
+    LogFileInterface log = null;
+    logRollLock.readLock().lock();
+    try {
+      log = getLogFileForEdit(editId.getIndex());
+      if (log != null) {
+        return log.getCurrentTerm() == editId.getTerm();
+      }
+      return false;
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+  }
+
+  public boolean exists(EditId editId, FindLogType findType) {
+    logRollLock.readLock().lock();
+    try {
+      LogFileInterface log = getLogFileForEdit(editId.getIndex(), findType);
+      if (log != null) {
+        return log.getCurrentTerm() == editId.getTerm();
+      } else {
+        return false;
+      }
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+  }
+
+  /**
+   * Truncates all the log entries from the current editId. Currently, this API
+   * assumes that there is no interleaving between the append() and truncate()
+   * call.
+   *
+   * There can be a read operation trying to read the same uncommitted transactions
+   * which we are trying to truncate. The order is as follows:
+   *
+   * truncate() -> file.truncate() -> file.delete() -> uncommittedTransactions.delete()
+   *
+   * read() -> uncommittedTransactions.get() -> file.getTransaction()
+   *
+   * So, the uncommittedTransactions should be able to satisfy all the queries
+   * and the entries will be deleted ONLY after they are deleted from the log file.
+   *
+   * Returning a transaction which is truncate should be fine.
+   *
+   * @param editId
+   * @return
+   * @throws IOException
+   */
+  @Override
+  public boolean truncate(EditId editId) throws IOException {
+    final long toBeTruncatedIndex = editId.getIndex() + 1;
+    try {
+      if (toBeTruncatedIndex <= committedIndex &&
+        !UNDEFINED_EDIT_ID.equals(editId)) {
+        LOG.error("The entry " + editId +
+          " is already committed. Current commit index: " + committedIndex);
+        return false;
+      }
+      // Get the RandomAccessFile which has the edit.
+      RandomAccessLog log = getRandomAccessFile(toBeTruncatedIndex);
+
+      if (log == null) {
+        return true;
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("Truncate the WAL %s up to: %d",
+            log.getFileName(), toBeTruncatedIndex));
+      }
+
+      // If the current log has no entries left after truncate or if current
+      // log has entries that should be truncated then roll the current log
+      log.truncate(toBeTruncatedIndex);
+
+      // Delete the rest of the logs with index >= toBeTruncatedIndex
+      Iterator<Map.Entry<Long, LogFileInterface>> filesToDelete =
+        uncommittedLogs.tailMap(toBeTruncatedIndex).entrySet().iterator();
+
+      Map.Entry<Long, LogFileInterface> fileEntry;
+      while(filesToDelete.hasNext()) {
+        fileEntry = filesToDelete.next();
+        if (LOG.isDebugEnabled()) {
+          debug("Deleting because of truncate, log file " +
+            fileEntry.getValue().getFileName());
+        }
+        filesToDelete.remove();
+        fileEntry.getValue().closeAndDelete();
+      }
+
+      // Update the uncommitted index cache
+      final Iterator<Map.Entry<Long, ByteBuffer>> invalidEdits =
+        uncommittedTransactions.tailMap(toBeTruncatedIndex).entrySet().iterator();
+      Map.Entry<Long, ByteBuffer> invalidEditEntry;
+      while(invalidEdits.hasNext()) {
+        invalidEditEntry = invalidEdits.next();
+        if (LOG.isDebugEnabled()) {
+          debug("Removing " + invalidEditEntry.getKey() + " from the" +
+            " uncommitted transactions map. Truncate Index " +
+            toBeTruncatedIndex);
+        }
+        invalidEdits.remove();
+      }
+    } finally {
+      if (isCheckInvariants) {
+        checkInvariants();
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Returns the last edit id successfully written to the log.
+   * @return
+   */
+  @Override
+  public EditId getLastEditID() {
+
+    logRollLock.readLock().lock();
+    try {
+      if (!uncommittedLogs.isEmpty()) {
+        LogFileInterface file = uncommittedLogs.lastEntry().getValue();
+        return new EditId (file.getCurrentTerm(), file.getLastIndex());
+      } else if (!committedLogs.isEmpty()) {
+        LogFileInterface file = committedLogs.lastEntry().getValue();
+        return new EditId (file.getCurrentTerm(), file.getLastIndex());
+      }
+      return UNDEFINED_EDIT_ID;
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+  }
+
+  @Override
+  public LogState getLogState() {
+    LogState logState = new LogState(null);
+
+    logState.setLastCommittedEdit(getEditId(committedIndex));
+
+    // Make sure its a consistent view of the logs at this point in time.
+
+    logRollLock.readLock().lock();
+    try {
+      if (!uncommittedLogs.isEmpty()) {
+        for (LogFileInterface file : uncommittedLogs.values()) {
+          logState.addUncommittedLogFile(new LogFileInfo(
+            file.getFileAbsolutePath(),
+            file.getFileSize(),
+            new EditId(file.getCurrentTerm(), file.getInitialIndex()),
+            new EditId(file.getCurrentTerm(), file.getLastIndex()),
+            file.getLastModificationTime(), file.getCreationTime()));
+        }
+      }
+
+      if (!committedLogs.isEmpty()) {
+        for (LogFileInterface file : committedLogs.values()) {
+          logState.addCommittedLogFile(new LogFileInfo(
+            file.getFileAbsolutePath(),
+            file.getFileSize(),
+            new EditId(file.getCurrentTerm(), file.getInitialIndex()),
+            new EditId(file.getCurrentTerm(), file.getLastIndex()),
+            file.getLastModificationTime(), file.getCreationTime()));
+        }
+      }
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+
+    return logState;
+  }
+
+  @Override
+  public List<LogFileInfo> getCommittedLogStatus(long minIndex) {
+    List<LogFileInfo> logFileInfos = new ArrayList<>();
+
+    for (LogFileInterface rol : committedLogs.values()) {
+      if (rol.getLastIndex() > minIndex) {
+        logFileInfos.add(new LogFileInfo(
+            rol.getFileAbsolutePath(),
+            rol.getFileSize(),
+            new EditId(rol.getCurrentTerm(), rol.getInitialIndex()),
+            new EditId(rol.getCurrentTerm(), rol.getLastIndex()),
+            rol.getLastModificationTime(),
+            rol.getCreationTime()
+        ));
+      }
+    }
+
+    return logFileInfos;
+  }
+
+  public String dumpLogs(int n) {
+    if (n <= 0) {
+      n = Integer.MAX_VALUE;
+    }
+    List<String> ids = new ArrayList<>();
+    for (EditId id = getLastEditID();
+         n > 0 && !id.equals(UNDEFINED_EDIT_ID);
+         n--, id = getPreviousEditID(id)) {
+      ids.add("" + id.getTerm() + "/" + id.getIndex());
+    }
+    Collections.reverse(ids);
+    return RaftUtil.listToString(ids);
+  }
+
+  /**
+   * Returns the log file which has the requested edit
+   * @param index
+   * @return
+   */
+  private LogFileInterface getLogFileForEdit(final long index, FindLogType findType) {
+
+    LogFileInterface log = null;
+
+    assert logRollLock.getReadHoldCount() > 0;
+
+    if (findType == null) {
+      findType = FindLogType.ALL;
+    }
+
+    switch (findType) {
+      case ALL:
+      case UNCOMMITTED_ONLY:
+        log = getRandomAccessFile(index);
+        break;
+    }
+    if (log != null) {
+      return log;
+    }
+    switch (findType) {
+      case ALL:
+      case COMMITTED_ONLY:
+        log = getReadOnlyFile(index);
+        break;
+    }
+    return log;
+  }
+
+  private LogFileInterface getLogFileForEdit(final long index) {
+    return getLogFileForEdit(index, FindLogType.ALL);
+  }
+
+  /**
+   * Returns the actual transaction by reading it from the log
+   * @param log log file to read from
+   * @param editId edit id of the transaction
+   * @param sessionKey reader's session key
+   * @return
+   * @throws IOException
+   */
+  private MemoryBuffer getTransactionFromLog(
+    final LogFileInterface log, final EditId editId, final String sessionKey,
+    final Arena arena)
+    throws IOException {
+    try {
+      return log.getTransaction(editId.getTerm(),
+        editId.getIndex(), sessionKey, arena);
+    } catch (NoSuchElementException e) {
+      // This is not an IO error. We are not disabling the log manager.
+      error("Failed to get transaction " + editId + " from " + log, e);
+      // We re-throw this exception as an IOException in order not to
+      // change the caller.
+      throw new IOException("Failed to get transaction " + editId +
+        " from readonly logs", e);
+    } catch (NotEnoughMemoryException e) {
+      error("Not enough memory to read the transaction.", e);
+      throw e;
+    } catch (Exception e) {
+      error("Failed to get transaction " + editId + " from " + log, e);
+      isAccessible = false;
+      throw e;
+    }
+  }
+
+  /**
+   * Updates the in-memory map to move the given log file from unrolledLogs to
+   * unflushedLogs map.
+   * @param log
+   */
+  private void logRollCompleted(final ReadOnlyLog log) {
+    committedLogs.put(log.getInitialIndex(), log);
+    uncommittedLogs.remove(log.getInitialIndex());
+  }
+
+  /**
+   * Performs the append to the log.
+   * @param term
+   * @param index
+   * @param commitIndex
+   * @param data
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void append(long term, long index, long commitIndex,
+                      final ByteBuffer data)
+    throws IOException, InterruptedException {
+
+    boolean mustCheckInvariants = false;
+
+    try {
+      LogFileInterface currentLog;
+
+      if (!uncommittedLogs.isEmpty()) {
+        currentLog = uncommittedLogs.lastEntry().getValue();
+      } else {
+        currentLog = logCreator.getNewLogFile();
+      }
+
+      if ((currentLog.getCurrentTerm() != HConstants.UNDEFINED_TERM_INDEX &&
+        currentLog.getCurrentTerm() < term) ||
+        currentFileSize > raftLogRollSize) {
+        try {
+          mustCheckInvariants = true;
+          debug(String.format("Moving away from current log %s for quorum %s. new edit " +
+              "{term=%d, index=%d}, current edit {term=%d,index=%d}",
+              currentLog.getFileName(), contextName,
+              term, index, currentLog.getCurrentTerm(), currentLog.getLastIndex()));
+          currentLog = rollCurrentLog(currentLog);
+          debug("The new log file is " + currentLog.getFileAbsolutePath());
+          currentFileSize = 0;
+        } catch (InterruptedException e) {
+          error("Unable to roll log " + currentLog.getFileName(), e);
+          isAccessible = false;
+          throw e;
+        }
+      }
+
+      uncommittedTransactions.put(index, data);
+
+      // Append the current edit.
+      currentFileSize = ((RandomAccessLog)currentLog).append(term, index, data);
+
+      // If its a new log, add it to the map after a successful append
+      if (currentLog.getTxnCount() == 1) {
+        uncommittedLogs.put(index, currentLog);
+      }
+
+      committedIndex = commitIndex;
+
+      // At this point, all the edits upto the committed index should be synced
+      // to disk. Adding TODO to track this!
+
+      final Iterator<Map.Entry<Long, ByteBuffer>> newlyCommittedEdits =
+        uncommittedTransactions.headMap(committedIndex, true).entrySet().iterator();
+      while(newlyCommittedEdits.hasNext()) {
+        newlyCommittedEdits.next();
+        newlyCommittedEdits.remove();
+      }
+    } finally {
+      if (isCheckInvariants && mustCheckInvariants) {
+        checkInvariants();
+      }
+    }
+  }
+
+  /**
+   * Rolls the current committed log.
+   * @param file file to roll the log
+   * @return
+   */
+  private boolean performRollLog(final RandomAccessLog file) {
+
+    try {
+      info("Performing log roll for " + file.getFileName());
+
+      long initialIndex = file.getInitialIndex();
+
+      String newName =
+        file.getCurrentTerm() + "_" + initialIndex + "_" + file.getLastIndex();
+
+      File renamedFile = new File(logDirectory +
+        HConstants.RAFT_FINALIZED_LOG_DIRECTORY_NAME + HConstants.PATH_SEPARATOR +
+        newName);
+
+      // There can be instances where the roll request is queued multiple times.
+      // Lets check that if the rolled file exists then just return.
+      if (renamedFile.exists()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("File " + file.getFileName() + " is already rolled to the " +
+            "new file " + renamedFile.getName());
+        }
+        return true;
+      }
+
+      try {
+        file.finalizeForWrite();
+      } catch (IOException e) {
+        error("Unable to finalize the " + file.getFileName() +
+          " for roll.", e);
+        isAccessible = false;
+        return false;
+      }
+
+      assert file.isFinalized() == true;
+
+      logRollLock.writeLock().lock();
+
+      if (!isAccessible()) {
+        error("Log is not accessible, returning.");
+        return false;
+      }
+
+      // Close all the reader sessions
+      try {
+        file.removeAllReaders();
+      } catch (IOException e) {
+        error("Unable to close the readers for " + file.getFileName() +
+          " for roll.", e);
+        return false;
+      }
+
+      if (!renamedFile.exists() && !renameFile(file.getFile(), renamedFile)) {
+        LOG.warn("Cannot rename the log " + file.getFileName() + " to " +
+          renamedFile.getAbsoluteFile().getName() + ". Aborting.");
+        return false;
+      }
+
+      logRollCompleted(
+        createReadOnlyLog(renamedFile, file.getCurrentTerm(), initialIndex, file.getLastIndex()));
+
+      info("Log roll for " + file.getFileName() + " completed successfully." +
+        " The new file created is at " + renamedFile.getAbsoluteFile().getName());
+      return true;
+    } finally {
+      logRollLock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * Rolls the current RandomAccessLog in case it reached the size limit or
+   * there was a change in term.
+   *
+   * @return The new log file
+   *
+   * @throws InterruptedException
+   */
+
+  private LogFileInterface rollCurrentLog(final LogFileInterface currentLog)
+    throws InterruptedException {
+
+    LogFileInterface oldLog = currentLog;
+    LogFileInterface newLog;
+    try {
+      newLog = logCreator.getNewLogFile();
+    } catch (InterruptedException e) {
+      error("Unable to get a new log file.", e);
+      isAccessible = false;
+      throw e;
+    }
+
+    assert newLog != null;
+    uncommittedLogs.put(oldLog.getInitialIndex(), oldLog);
+    // we don't check invariants here because checkInvariants is only
+    // called by append.
+    return newLog;
+  }
+
+  /**
+   * For unit test only!
+   */
+  protected void forceRollLog() throws InterruptedException {
+    try {
+      logCreator.getNewLogFile();
+    } catch (InterruptedException e) {
+      error("Unable to get a new log file.", e);
+      throw e;
+    }
+  }
+
+  /**
+   * Depending upon the committedIndex, get the list of logs which can now be
+   * rolled and submit jobs to roll them. Returns the number of logs rolled.
+   *
+   * For the read operations, the log manager also takes logRollLock read lock
+   * and for roll the Log Roller will take the write lock. Hence
+   * roll/read operation file cannot happen simultaneously.
+   *
+   * The log manager grabs the writeLock before making any updates to the file
+   * maps. So after the log roll happens it will also have to grab the write lock
+   * before updating the file maps. This prevents the race conditions between
+   * the log roller and file location lookup operation done by the transaction
+   * log manager.
+   *
+   */
+  protected int rollCommittedLogs() {
+    int nRolled = 0;
+    Long initialIndexOfLastUncommittedLog =
+      uncommittedLogs.floorKey(committedIndex);
+
+    if (initialIndexOfLastUncommittedLog != null) {
+      Iterator<Map.Entry<Long, LogFileInterface>> iter =
+        uncommittedLogs.headMap(
+          initialIndexOfLastUncommittedLog).entrySet().iterator();
+      while (iter.hasNext()) {
+        final LogFileInterface log = iter.next().getValue();
+        assert committedIndex >= log.getLastIndex();
+        nRolled ++;
+        debug("Rolling log file " + log.getFileName());
+        if (!performRollLog((RandomAccessLog)log)) {
+          debug("Log roll failed for " + log);
+        } else {
+          debug("Log roll succeeded for " + log);
+        }
+      }
+    }
+    return nRolled;
+    // no checking of invariants here because we check in the
+    // caller append
+  }
+
+  /**
+   * Deletes the old committed logs.
+   */
+  protected void deleteOldLogs() {
+
+    if (context == null | !isAccessible()) {
+      return;
+    }
+
+    long deleteIndex = context.getPurgeIndex();
+    long currentTimeInMillis = System.currentTimeMillis();
+    Iterator<Map.Entry<Long, LogFileInterface>> iter =
+      committedLogs.headMap(deleteIndex).entrySet().iterator();
+
+    LogFileInterface log;
+    while (iter.hasNext()) {
+      Map.Entry<Long, LogFileInterface> entry = iter.next();
+      log = entry.getValue();
+
+      if (deleteIndex >= log.getLastIndex() &&
+        (currentTimeInMillis - log.getLastModificationTime()) >
+          transactionLogRetentionTime) {
+        LOG.info("Deleting log file " + entry.getValue().getFileName() +
+          ", term= " + log.getCurrentTerm() + ", startIndex=" +
+          log.getInitialIndex() + ", lastIndex=" +
+          log.getLastIndex() + " deleteIndex=" + deleteIndex + " mod time " +
+          log.getLastModificationTime());
+        logRollLock.readLock().lock();
+        try {
+          iter.remove();
+          log.closeAndDelete();
+        } catch (IOException e) {
+          LOG.error("Cannot delete log file " + entry.getValue(), e);
+        } finally {
+          logRollLock.readLock().unlock();
+        }
+      }
+    }
+  }
+
+  /**
+   * Get the EditId from the log for the given index
+   * @param index
+   * @return the EditId from the log for the given index;
+   *         The term will be set to -1 in case there is no entry in the log.
+   */
+  @Override
+  public EditId getEditId(final long index) {
+    logRollLock.readLock().lock();
+    try {
+      return new EditId(getTermInternal(index), index);
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+  }
+
+  /**
+   * Returns the term for the given index, assuming the caller has acquired the
+   * read lock of logRollLock.
+   * @param index
+   * @return
+   */
+  private long getTermInternal(final long index) {
+    long term = HConstants.UNDEFINED_TERM_INDEX;
+    LogFileInterface log = getLogFileForEdit(index);
+    if (log != null) {
+      term = log.getCurrentTerm();
+    }
+
+    return term;
+  }
+
+  /**
+   * Goes through all the ReadOnlyFiles and returns the file which should have
+   * the edit.
+   *
+   * @param index
+   * @return ReadOnlyLog file if edit id exists, else null;
+   */
+  private ReadOnlyLog getReadOnlyFile(long index) {
+
+    LogFileInterface file = null;
+    long oldestIndex = HConstants.UNDEFINED_TERM_INDEX;
+    long newestIndex = HConstants.UNDEFINED_TERM_INDEX;
+
+    // Asking for a very old index, which has been already rolled and committed
+    if (committedLogs.firstEntry() != null) {
+      oldestIndex = committedLogs.firstKey();
+      newestIndex = committedLogs.lastEntry().getValue().getLastIndex();
+    }
+
+    if (oldestIndex == HConstants.UNDEFINED_TERM_INDEX ||
+      newestIndex == HConstants.UNDEFINED_TERM_INDEX) {
+      return (ReadOnlyLog)file;
+    }
+
+    if (index >= oldestIndex && index <= newestIndex) {
+      file = committedLogs.floorEntry(index).getValue();
+    }
+    return (ReadOnlyLog)file;
+  }
+
+  public static MemoryBuffer allocateBuffer(final Arena arena, final int size) {
+    MemoryBuffer buffer;
+    try {
+      if (arena != null) {
+        buffer = arena.allocateByteBuffer(size);
+      } else {
+        buffer = new MemoryBuffer(ByteBuffer.allocate(size));
+      }
+    } catch (CacheFullException | BucketAllocatorException e) {
+      buffer = new MemoryBuffer(ByteBuffer.allocate(size));
+    }
+    return buffer;
+  }
+
+  /**
+   * Goes through all the RandomAccessFiles and returns the file which should
+   * have the edit.
+   *
+   * This assumes that the caller as held the read lock.
+   *
+   * @param index
+   * @return RandomAccessLog file if edit id exists, else null;
+   */
+  private RandomAccessLog getRandomAccessFile(long index) {
+
+    long oldestIndex = HConstants.UNDEFINED_TERM_INDEX;
+    long newestIndex = HConstants.UNDEFINED_TERM_INDEX;
+    LogFileInterface file = null;
+
+    // Get oldest index
+    if (!uncommittedLogs.isEmpty()) {
+      oldestIndex = uncommittedLogs.firstKey();
+      newestIndex = uncommittedLogs.lastEntry().getValue().getLastIndex();
+    }
+
+    if (oldestIndex == HConstants.UNDEFINED_TERM_INDEX ||
+      newestIndex == HConstants.UNDEFINED_TERM_INDEX) {
+      return (RandomAccessLog)file;
+    }
+
+    if (index >= oldestIndex && index <= newestIndex) {
+      file = uncommittedLogs.floorEntry(index).getValue();
+    }
+
+    return (RandomAccessLog)file;
+  }
+
+  public long getSeedIndex() {
+    return seedIndex;
+  }
+
+  private void debug(String str) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(this.toString() + ":" + str);
+    }
+  }
+
+  private void info(String str) {
+    LOG.info(this.toString() + ":" + str);
+  }
+
+  private void error(String str) {
+    LOG.error(this.toString() + ":" + str);
+  }
+
+  private void error(String str, Throwable stone) {
+    LOG.error(this.toString() + ":" + str, stone);
+  }
+
+  private void warn(String str) {
+    LOG.warn(this.toString() + ":" + str);
+  }
+
+  private void warn(String str, Throwable stone) {
+    LOG.warn(this.toString() + ":" + str, stone);
+  }
+
+  private <T> Long validate(String name, ConcurrentSkipListMap<Long, T> logs,
+                            Long expectedNextIndex) throws IOException {
+    for (Long key : logs.descendingKeySet()) {
+      LogFileInterface log = (LogFileInterface) logs.get(key);
+      if (log.getLastIndex() < log.getInitialIndex()) {
+        throw new IOException(
+            "Incorrect index range ["
+            + log.getInitialIndex() + ", " + log.getLastIndex()
+            + "] found in " + name + " log " + log
+        );
+      }
+      if (expectedNextIndex != null && expectedNextIndex != log.getLastIndex()) {
+        throw new IOException(
+            "A gap is found between " + expectedNextIndex + " and the " + name +
+              " log " + log + " [" + log.getInitialIndex() + ", "
+              + log.getLastIndex()+ "]"
+        );
+      }
+      expectedNextIndex = log.getInitialIndex() - 1;
+    }
+    return expectedNextIndex;
+  }
+
+  public void checkInvariants() throws IOException {
+    // we assume we have the write lock
+    long t0 = System.nanoTime();
+
+    logRollLock.readLock().lock();
+    try {
+      Long expectedNextIndex = null;
+
+      if (!uncommittedTransactions.isEmpty() &&
+        uncommittedTransactions.firstKey() <= committedIndex) {
+        throw new IOException("Stale committed edit left in uncommitted list." +
+          " First Key: " + uncommittedLogs.firstKey() + ", committed index: " +
+        committedIndex);
+      }
+
+      expectedNextIndex = validate("uncommitted", uncommittedLogs, expectedNextIndex);
+      expectedNextIndex = validate("committed", committedLogs, expectedNextIndex);
+    } finally {
+      logRollLock.readLock().unlock();
+      debug("checkInvariants() took " + ((System.nanoTime() - t0)/1000) + " us");
+    }
+  }
+
+  @Override
+  public String getPath() {
+    return this.logDirectory;
+  }
+
+  public void scanCandidateLogs() throws IOException {
+    candidateLogsManager.scan();
+  }
+
+  /**
+   *  greedyIncorporateCandidateLogs
+   *
+   *  @param    sessionKey            String
+   *  @return   Pair<EditId, EditId>  The pivot Id and the last EditId in the
+   *                                  promoted candidate logs
+   *
+   *  We try to keep as much of uncommitted logs intact possible by finding a
+   *  usable pivot log from the most recent logs. We will stop incorporating any logs
+   *  beyond lastLogIndex if it's set.
+   */
+  public Pair<EditId, EditId> greedyIncorporateCandidateLogs(
+      String    sessionKey
+  ) {
+    return greedyIncorporateCandidateLogs(sessionKey, Long.MAX_VALUE);
+  }
+
+  public Pair<EditId, EditId> greedyIncorporateCandidateLogs(
+      String    sessionKey,
+      long      lastLogIndex
+  ) {
+    EditId pivotLogId = null;
+    pivotLogId = getLastValidTransactionId();
+
+    if (pivotLogId != null && pivotLogId != UNDEFINED_EDIT_ID) {
+      return new Pair<EditId, EditId>(pivotLogId, incorporateCandidateLogs(sessionKey, pivotLogId, lastLogIndex));
+    }
+    return null;
+  }
+
+  /**
+   *  incorporateCandidateLogs
+   *
+   *
+   *  @param    sessionKey    String
+   *  @param    pivotLogId    EditId
+   *  @param    lastLogIndex  long
+   *
+   *  @return   EditId      the last EditId in the promoted candidate logs
+   *
+   *  WARNING:
+   *
+   *  We must be very cautious here because a false leader may send AppendRequest
+   *  that triggers incorporateCandidateLogs().
+   *
+   *  Pre-conditions
+   *  P1. A set of contiguous candidate logs S with index range [S_1, S_2] contain
+   *      all valid RAFT transactions in the range [S_1, S_2]. These transactions
+   *      may not be committed yet.
+   *  P2. The local logs state is valid with respect to the quorum up to a certain
+   *      head subset, which may be empty, of uncommitted logs.
+   *
+   *  Input
+   *      pivotLogId <= last valid transaction Id
+   *
+   *  Incorporation Criteria
+   *  We only incorporate a set of contiguous candidate logs S with index range
+   *  [A, B] iff
+   *  C1. The pivotLogId is found in both uncommittedLogs and S.
+   *
+   *  What this means is that
+   *    1) uncommitedLogs.headSet(pivotLogId) is valid
+   *    2) uncommitedLogs.headSet(pivotLogId) + S is valid
+   *
+   *  Procedure
+   *  1. We find a set of contiguous candidate logs that cover the pivot ID as well
+   *     as pivotID.next().
+   *
+   *  2. We truncate the head log at pivotId.next() and the existing uncommitted logs
+   *     up to and including pivotId.next().
+   *
+   *  3. We then merge the set of candidate logs with existing uncommitted logs.
+   */
+  private EditId incorporateCandidateLogs(
+      String    sessionKey,
+      EditId    pivotLogId,
+      long      lastLogIndex
+  ) {
+    if (candidateLogsManager.isEmpty()) {
+      return null;
+    }
+    String whoami = "incorporateCandidateLogs("
+      + sessionKey
+      + ", pivot=" + pivotLogId
+      + ") ";
+
+    // a quick test
+    if (!exists(pivotLogId, FindLogType.UNCOMMITTED_ONLY)) {
+      warn(whoami + " failed to locate pivotLogId " + pivotLogId + " in local logs");
+      return null;
+    }
+
+    if (!candidateLogsManager.exists(pivotLogId)) {
+      warn(whoami + " failed to locate pivotLogId " + pivotLogId + " in candidate logs");
+      return null;
+    }
+
+    // This variable records the last edit id of the most recent promotion
+    EditId lastIncorporatedEditId = null;
+
+    // In future we may want to change the locking discipline to reduce the amount of time
+    // spent in the critical section.
+    logRollLock.writeLock().lock();
+    try {
+      // Checking criterion C1
+      if (!exists(pivotLogId, FindLogType.UNCOMMITTED_ONLY)) {
+        warn(whoami + " failed to locate pivotLogId " + pivotLogId);
+        return null;
+      }
+
+      List<LogFileInfo> candidates = candidateLogsManager.getContiguousLogsContaining(pivotLogId, lastLogIndex);
+      if (candidates == null || candidates.isEmpty()) {
+        debug(whoami + " found no suitable candidate logs containing [" + pivotLogId + ", " + lastLogIndex + "]");
+        return null;
+      }
+
+      // sanity-check the list
+      for (int i=1; i<candidates.size(); i++) {
+        if (candidates.get(i).getInitialIndex() != candidates.get(i-1).getLastIndex() + 1) {
+          error(whoami + " found a gap or overlap between candidates " + candidates.get(i-1) + " and " + candidates.get(i));
+          return null;
+        }
+      }
+
+      boolean foundContainingLog = false;
+loop_over_candidates:
+      for (LogFileInfo candidateInfo : candidates) {
+        RandomAccessLog candidateLog = createRandomAccessLog(new File(candidateInfo.getAbsolutePath()), isSync);
+        debug(whoami + " locking file " + candidateLog.getFile().getAbsolutePath());
+        FileLock filelock = lockRandomAccessLog(candidateLog);
+        try {
+          RandomAccessLog targetLog = candidateLog;
+
+          // check the modification time
+          if (getModificationTime(candidateLog.getFile()) != candidateInfo.getLastVerifiedModificationTime()) {
+            error(whoami + " found file " + candidateLog.getFile().getAbsolutePath()
+                + " modified since " + candidateInfo.getLastVerifiedModificationTime()
+                + "; the lastest modification time is " +
+                getModificationTime(candidateLog.getFile()));
+            return lastIncorporatedEditId;
+          }
+
+          // the file is already locked; it's safe to rebuild
+          candidateLog.rebuild(toString());
+
+          if (candidateLog.getInitialIndex() != candidateInfo.getInitialIndex()) {
+            error("InitialIndex mismatch between RandomAccessLog " + candidateLog
+                + " and LogFileInfo " + candidateInfo);
+            return lastIncorporatedEditId;
+          } else if (candidateLog.getLastIndex() != candidateInfo.getLastIndex()) {
+            error("LastIndex mismatch between RandomAccessLog " + candidateLog
+                + " and LogFileInfo " + candidateInfo);
+            return lastIncorporatedEditId;
+          }
+
+          // again, sanity check
+          if (candidateLog.getLastIndex() <= pivotLogId.getIndex()) {
+            candidateLogsManager.removeFromCollection(candidateInfo);
+            continue loop_over_candidates;
+          } else if (candidateLog.getInitialIndex() > pivotLogId.getIndex()) {
+            if (!foundContainingLog) {
+              error(whoami + " failed to find a candidateInfo log containing " + pivotLogId + " before " + candidateInfo);
+              return null;
+            }
+            // else fall through
+          } else {
+            // FACT: candidateLog.getInitialIndex() <= pivotLogId.getIndex() < candidateLog.getLastIndex()
+
+            if (foundContainingLog) {
+              // this should never ever happen because we have already checked the list.
+              // but it doesn't hurt to check here.
+              error(whoami + " found more than one candidates containing " + pivotLogId);
+              return null;
+            }
+            try {
+              MemoryBuffer buffer = candidateLog.getTransaction(
+                pivotLogId.getTerm(), pivotLogId.getIndex(), sessionKey, null);
+            } catch (BufferOverflowException ex) {
+              // it's OK; we should really let the reader go through the entire message to ascertain
+              // its integrity. Perhaps LogReader.seekAndRead() can handle null buffer.
+            } catch (NoSuchElementException ex) {
+              error(whoami + " failed to find pivotLogId " + pivotLogId + " in " + candidateLog);
+              return null;
+            } catch (IOException ex) {
+              error(whoami + " failed to find pivotLogId " + pivotLogId + " in " + candidateLog);
+              return null;
+            }
+
+            foundContainingLog = true;
+
+            Pair<EditId, MemoryBuffer> pair = getNextEditIdTransaction(sessionKey,
+              pivotLogId.getIndex(), null);
+            if (pair == null || pair.getFirst() == null) {
+              error("Failed to find the next edit of the pivot " + pivotLogId);
+              return null;
+            }
+
+            // We use a very strict criterion
+            EditId nextLogId = pair.getFirst();
+            if (candidateLog.getLastIndex() < nextLogId.getIndex() || candidateLog.getCurrentTerm() != nextLogId.getTerm()) {
+              warn(whoami + " The edit ID after the pivot (" + pivotLogId + " -> " + nextLogId
+                  + " does not exist in the same log file " + candidateLog);
+              return null;
+            }
+            if (!nextLogId.equals(getLastEditID())) {
+              debug(whoami + " the nextLogId " + nextLogId + " is not the last one; truncating logs up to " + nextLogId);
+              if (!truncate(nextLogId)) {
+                error(whoami + " failed to truncate @ " + nextLogId);
+                return null;
+              }
+            }
+
+            // we may have to truncate this file
+            if (candidateLog.getInitialIndex() < nextLogId.getIndex()) {
+              long newInitialIndex = nextLogId.getIndex()+1;
+              info(whoami + " We have to head-truncate " + candidateLog + " @ " + newInitialIndex);
+              File newFile = new File(candidateInfo.getAbsolutePath() + ".subcopy_to_"
+                  + candidateLog.getCurrentTerm() + "_" + newInitialIndex + "_" + candidateLog.getLastIndex());
+
+              RandomAccessLog newLog = candidateLogsManager.subCopyRandomAccessLog(
+                  candidateLog, newInitialIndex, candidateLog.getLastIndex(), newFile);
+              if (newLog == null) {
+                warn(whoami + " Failed to subcopy " + candidateLog + " to " + newFile.getAbsolutePath());
+                return null;
+              }
+              newLog.rebuild(toString());
+              targetLog = newLog;
+            }
+          }
+
+          // the big job
+          // candidateLogsManager.removeFromCollection(candidateInfo);
+
+          // TODO: use the appropriate naming scheme
+          String newName = getCanidatePromotionFilename(
+              targetLog.getFile().getName(),
+              targetLog.getCurrentTerm(),
+              targetLog.getInitialIndex(),
+              targetLog.getLastIndex()
+          );
+          File newFile = new File(logDirectory + HConstants.RAFT_CURRENT_LOG_DIRECTORY_NAME + "/" + newName);
+          debug(whoami + " moving " + targetLog.getFile().getAbsolutePath() + " to " + newFile.getAbsolutePath());
+          if (!renameFile(targetLog.getFile(), newFile)) {
+            error(whoami + " failed to move " + targetLog.getFile().getAbsolutePath() + " to " + newFile.getAbsolutePath());
+            return lastIncorporatedEditId;
+          }
+
+          // candidateLog.getFile() should point to the new path now
+          info(whoami + " moved " + targetLog.getFile().getAbsolutePath() + " to " + newFile.getAbsolutePath());
+
+          uncommittedLogs.put(targetLog.getInitialIndex(), targetLog);
+          lastIncorporatedEditId = new EditId(targetLog.getCurrentTerm(), targetLog.getLastIndex());
+          targetLog = null;
+        } finally {
+          unlockRandomAccessLog(filelock);
+        }
+      }
+    } catch (IOException ex) {
+      error(whoami + " caught an exception", ex);
+    } finally {
+      logRollLock.writeLock().unlock();
+      if (lastIncorporatedEditId != null) {
+        try {
+          checkInvariants();
+        } catch (IOException ex) {
+          error(whoami + " failed to check invariants after incorporating candidate logs up till " + lastIncorporatedEditId, ex);
+          isAccessible = false;
+        }
+        candidateLogsManager.prune(lastIncorporatedEditId.getIndex());
+      }
+    }
+    return lastIncorporatedEditId;
+  }
+
+  protected String getCanidatePromotionFilename(
+      final String      oldFilename,
+      final long        term,
+      final long        initialIndex,
+      final long        lastIndex
+  ) {
+    return oldFilename + ".promoted_to_" + term + "_" + initialIndex + "_" + lastIndex;
+  }
+
+  /*  lockRandomAccessLog
+   *
+   *  @param  log     RandomAccessLog
+   *  @return         FileLock
+   *
+   *  Mockable
+   */
+  protected FileLock lockRandomAccessLog(RandomAccessLog logFile) throws IOException {
+    RandomAccessFile raf = logFile.getRandomAccessFile();
+    return raf.getChannel().lock();
+  }
+
+  /**
+   *  unlockRandomAccessLog
+   *
+   *  @param  lock    FileLock
+   *
+   *  Mockable
+   */
+  protected void unlockRandomAccessLog(FileLock lock) throws IOException {
+    lock.release();
+  }
+
+  /**
+   *  getModificationTime
+   *
+   *  Mockable
+   */
+  protected long getModificationTime(File file) {
+    return file.lastModified();
+  }
+
+  /**
+   *  renameFile
+   *
+   *  Mockable
+   */
+  protected boolean renameFile(File oldFile, File newFile) {
+    return oldFile.renameTo(newFile);
+  }
+
+  /**
+   *  createReadOnlyLog
+   *
+   *  Mockable
+   */
+  protected ReadOnlyLog createReadOnlyLog(File f, long t, long initialIndex, long lastIndex) {
+    return new ReadOnlyLog(f, t, initialIndex);
+  }
+
+  /**
+   *  createReadOnlyLog
+   *
+   *  Mockable
+   */
+  protected ReadOnlyLog createReadOnlyLog(File f, String sessionKey) throws IOException {
+    if (SeedLogFile.isSeedFile(f)) {
+      return new SeedLogFile(f);
+    } else {
+      return new ReadOnlyLog(f, sessionKey);
+    }
+  }
+
+  /**
+   *  createRandomAccessLog
+   *
+   *  Mockable
+   */
+  protected RandomAccessLog createRandomAccessLog(File f, boolean isSync) throws IOException {
+    return new RandomAccessLog(f, isSync);
+  }
+
+  /**
+   *  createTransactionLogCreator
+   *
+   *  Mockable
+   */
+  protected TransactionLogCreator createTransactionLogCreator(
+      String            logDirectory,
+      boolean           isSync
+  ) {
+    return new TransactionLogCreator(logDirectory, isSync, conf);
+  }
+
+  /**
+   * createCandidateLogsManager
+   *
+   * Mockable
+   */
+  protected CandidateLogsManager createCandidateLogsManager(
+      Configuration     conf,
+      String            contextName,
+      String            logsDirectory
+  ) {
+    return new CandidateLogsManager(conf, contextName, logDirectory);
+  }
+
+  public void setRaftLogRollSize(long size) {
+    raftLogRollSize = size;
+  }
+
+  public void stop() {
+    logRollLock.readLock().lock();
+    try {
+      isAccessible = false;
+      logRollTask.cancel(true);
+      logDeletionTask.cancel(true);
+    } finally {
+      logRollLock.readLock().unlock();
+    }
+  }
+
+  public static int getLogPrefetchSize() {
+    return logPrefetchSize;
+  }
+
+  /**
+   * Unit test only
+   */
+  public ConcurrentSkipListMap<Long, LogFileInterface> getUncommittedLogs() {
+    return uncommittedLogs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/metrics/ConsensusMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/metrics/ConsensusMetrics.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/metrics/ConsensusMetrics.java
new file mode 100644
index 0000000..d956563
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/metrics/ConsensusMetrics.java
@@ -0,0 +1,379 @@
+package org.apache.hadoop.hbase.consensus.metrics;
+
+import io.airlift.stats.CounterStat;
+import io.airlift.stats.Distribution;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.metrics.MetricsBase;
+import org.apache.hadoop.hbase.metrics.TimeStat;
+import org.weakref.jmx.JmxException;
+import org.weakref.jmx.MBeanExporter;
+import org.weakref.jmx.Managed;
+import org.weakref.jmx.Nested;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * A class used to expose various metrics around the consensus, such as the
+ * number of leader election, number of stepping downs, append latencies, etc.
+ */
+@ThreadSafe
+public class ConsensusMetrics extends MetricsBase {
+  /**
+   * Type string used when exporting an MBean for these metrics
+   */
+  public static final String TYPE = "ConsensusMetrics";
+  /**
+   * Domain string used when exporting an MBean for these metrics
+   */
+  public static final String DOMAIN = "org.apache.hadoop.hbase.consensus";
+
+  private final String name;
+  private final String procId;
+  private final List<PeerMetrics> peerMetrics = new ArrayList<>();
+
+  /**
+   * Keep track of the Raft state. This is used to make a decision on which
+   * metrics to publish and how to publish them.
+   */
+  private volatile PeerStatus.RAFT_STATE raftState =
+          PeerStatus.RAFT_STATE.INVALID;
+
+  /**
+   * Leader election metrics
+   */
+  private final CounterStat leaderElectionAttempts = new CounterStat();
+  private final CounterStat leaderElectionFailures = new CounterStat();
+  private final TimeStat leaderElectionLatency =
+          new TimeStat(TimeUnit.MILLISECONDS);
+  // This is the metric indicating the number of times a higher rank peer
+  // caught up with a lower rank leader and caused it to step down.
+  private final CounterStat higherRankCaughtUpStepDown = new CounterStat();
+  private final CounterStat appendEntriesStepDown = new CounterStat();
+
+  /**
+   * Timer metrics
+   */
+  private final CounterStat progressTimeouts = new CounterStat();
+  private final CounterStat heartbeatTimeouts = new CounterStat();
+  private final CounterStat heartbeatCanceled = new CounterStat();
+
+  /**
+   * Log metrics
+   */
+  private final TimeStat logWriteLatency = new TimeStat(TimeUnit.MICROSECONDS);
+  private final TimeStat logTruncateLatency =
+          new TimeStat(TimeUnit.MICROSECONDS);
+  private final TimeStat logSerializationLatency =
+          new TimeStat(TimeUnit.NANOSECONDS);
+  private final TimeStat logDeserializationLatency =
+          new TimeStat(TimeUnit.NANOSECONDS);
+  private final TimeStat logAppendGreaterThanSecond =
+      new TimeStat(TimeUnit.MILLISECONDS);
+
+  /**
+   * AppendEntries metrics
+   */
+  private final TimeStat appendEntriesLatency =
+          new TimeStat(TimeUnit.MICROSECONDS);
+  private final Distribution appendEntriesSize = new Distribution();
+  private final Distribution appendEntriesCompressedSize = new Distribution();
+  private final Distribution appendEntriesBatchSize = new Distribution();
+  private final CounterStat appendEntriesRetries = new CounterStat();
+  private final CounterStat appendEntriesDuplicates = new CounterStat();
+  private final CounterStat appendEntriesMissedDeadline = new CounterStat();
+
+  /** Sync Latencies */
+  private final TimeStat fsSyncLatency = new TimeStat(TimeUnit.MICROSECONDS);
+  private final TimeStat fsGSyncLatency = new TimeStat(TimeUnit.MICROSECONDS);
+
+  /**
+   * Commit queue metrics
+   */
+  private final CounterStat commitQueueEntriesLimitExceeded = new CounterStat();
+  private final CounterStat commitQueueSizeLimitExceeded = new CounterStat();
+
+  public ConsensusMetrics(final String name, final String procId,
+                          final MBeanExporter exporter) {
+    super(DOMAIN, TYPE, name, procId, Collections.<String, String>emptyMap(),
+            exporter);
+    this.name = name;
+    this.procId = procId;
+  }
+
+  public ConsensusMetrics(final String name, final String hostId) {
+    this(name, hostId, null);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getProcId() {
+    return procId;
+  }
+
+  public List<PeerMetrics> getPeerMetrics() {
+    synchronized (peerMetrics) {
+      return ImmutableList.copyOf(peerMetrics);
+    }
+  }
+
+  public PeerMetrics createPeerMetrics(final String peerId) {
+    return new PeerMetrics(name, procId, peerId, getMBeanExporter());
+  }
+
+  public void exportPeerMetrics(final PeerMetrics metrics)
+          throws JmxException {
+    MBeanExporter exporter = getMBeanExporter();
+    synchronized (peerMetrics) {
+      peerMetrics.add(metrics);
+    }
+    if (exporter != null && !metrics.isExported()) {
+      metrics.export(exporter);
+    }
+  }
+
+  public void unexportPeerMetrics(final PeerMetrics metrics)
+          throws JmxException {
+    synchronized (peerMetrics) {
+      peerMetrics.remove(metrics);
+    }
+    if (metrics.isExported()) {
+      metrics.unexport();
+    }
+  }
+
+  @Override
+  public void setMBeanExporter(final MBeanExporter exporter) {
+    super.setMBeanExporter(exporter);
+    for (PeerMetrics metrics : getPeerMetrics()) {
+      metrics.setMBeanExporter(exporter);
+    }
+  }
+
+  /**
+   * Convenience method which will set the given {@link MBeanExporter} for this
+   * and the associated {@link LeaderMetrics} and {@link PeerMetrics} objects.
+   * Upon setting the exporter this object and the {@link PeerMetrics} objects
+   * will be exported as MBeans.
+   *
+   * @param exporter exporter to use when exporting the metrics
+   * @throws JmxException if the object could not be exported
+   */
+  @Override
+  public void export(final MBeanExporter exporter) {
+    super.export(exporter);
+    for (PeerMetrics metrics : getPeerMetrics()) {
+      metrics.export(exporter);
+    }
+  }
+
+  @Managed
+  public PeerStatus.RAFT_STATE getRaftState() {
+    return raftState;
+  }
+
+  public void setRaftState(PeerStatus.RAFT_STATE state) {
+    raftState = state;
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getLeaderElectionAttempts() {
+    return leaderElectionAttempts;
+  }
+
+  public void incLeaderElectionAttempts() {
+    leaderElectionAttempts.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getLeaderElectionFailures() {
+    return leaderElectionFailures;
+  }
+
+  public void incLeaderElectionFailures() {
+    leaderElectionFailures.update(1);
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getLeaderElectionLatency() {
+    return leaderElectionLatency;
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getHigherRankCaughtUpStepDown() {
+    return higherRankCaughtUpStepDown;
+  }
+
+  public void incHigherRankCaughtUpStepDown() {
+    higherRankCaughtUpStepDown.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getAppendEntriesStepDown() {
+    return appendEntriesStepDown;
+  }
+
+  public void incAppendEntriesStepDown() {
+    appendEntriesStepDown.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getProgressTimeouts() {
+    return progressTimeouts;
+  }
+
+  public void incProgressTimeouts() {
+    progressTimeouts.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getHeartbeatTimeouts() {
+    return heartbeatTimeouts;
+  }
+
+  public void incHeartBeatTimeouts() {
+    heartbeatTimeouts.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getHeartbeatCanceled() {
+    return heartbeatCanceled;
+  }
+
+  public void incHeartBeatCanceled() {
+    heartbeatCanceled.update(1);
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getLogWriteLatency() {
+    return logWriteLatency;
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getLogTruncateLatency() {
+    return logTruncateLatency;
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getLogSerializationLatency() {
+    return logSerializationLatency;
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getLogDeserializationLatency() {
+    return logDeserializationLatency;
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getAppendEntriesLatency() {
+    return appendEntriesLatency;
+  }
+
+  @Managed
+  @Nested
+  public Distribution getAppendEntriesSize() {
+    return appendEntriesSize;
+  }
+
+  @Managed
+  @Nested
+  public Distribution getAppendEntriesCompressedSize() {
+    return appendEntriesCompressedSize;
+  }
+
+  @Managed
+  @Nested
+  public Distribution getAppendEntriesBatchSize() {
+    return appendEntriesBatchSize;
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getAppendEntriesRetries() {
+    return appendEntriesRetries;
+  }
+
+  public void incAppendEntriesRetries() {
+    appendEntriesRetries.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getAppendEntriesDuplicates() {
+    return appendEntriesDuplicates;
+  }
+
+  public void incAppendEntriesDuplicates() {
+    appendEntriesDuplicates.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getAppendEntriesMissedDeadline() {
+    return appendEntriesMissedDeadline;
+  }
+
+  public void incAppendEntriesMissedDeadline() {
+    appendEntriesMissedDeadline.update(1);
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getFsSyncLatency() {
+    return fsSyncLatency;
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getFsGSyncLatency() {
+    return fsGSyncLatency;
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getCommitQueueEntriesLimitExceeded() {
+    return commitQueueEntriesLimitExceeded;
+  }
+
+  public void incCommitQueueEntriesLimitExceeded() {
+    commitQueueEntriesLimitExceeded.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getCommitQueueSizeLimitExceeded() {
+    return commitQueueSizeLimitExceeded;
+  }
+
+  public void incCommitQueueSizeLimitExceeded() {
+    commitQueueSizeLimitExceeded.update(1);
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getLogAppendGreaterThanSecond() {
+    return logAppendGreaterThanSecond;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/metrics/PeerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/metrics/PeerMetrics.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/metrics/PeerMetrics.java
new file mode 100644
index 0000000..efa9df6
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/metrics/PeerMetrics.java
@@ -0,0 +1,113 @@
+package org.apache.hadoop.hbase.consensus.metrics;
+
+import io.airlift.stats.CounterStat;
+import io.airlift.stats.Distribution;
+import org.apache.hadoop.hbase.metrics.MetricsBase;
+import org.apache.hadoop.hbase.metrics.TimeStat;
+import org.weakref.jmx.MBeanExporter;
+import org.weakref.jmx.Managed;
+import org.weakref.jmx.Nested;
+
+import javax.annotation.concurrent.ThreadSafe;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+@ThreadSafe
+public class PeerMetrics extends MetricsBase {
+  public static final String TYPE = "PeerMetrics";
+
+  /** General metrics */
+  private final CounterStat RPCErrorEvents = new CounterStat();
+
+  /** Leader election metrics */
+  private final CounterStat voteRequestFailures = new CounterStat();
+
+  /** AppendEntries metrics */
+  private final CounterStat appendEntriesFailures = new CounterStat();
+  private final TimeStat appendEntriesLatency =
+          new TimeStat(TimeUnit.MICROSECONDS);
+  private final AtomicLong appendEntriesLag = new AtomicLong(0);
+
+  /** Batch recovery metrics */
+  private final Distribution batchRecoverySize = new Distribution();
+  private final TimeStat batchRecoveryLatency =
+          new TimeStat(TimeUnit.MICROSECONDS);
+
+  public PeerMetrics(final String name, final String procId,
+                     final String peerId, final MBeanExporter exporter) {
+    super(ConsensusMetrics.DOMAIN, TYPE, name, procId,
+            getExtendedAttributes(peerId), exporter);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getRPCErrorEvents() {
+    return RPCErrorEvents;
+  }
+
+  public void incRPCErrorEvents() {
+    RPCErrorEvents.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getVoteRequestFailures() {
+    return voteRequestFailures;
+  }
+
+  public void incVoteRequestFailures() {
+    voteRequestFailures.update(1);
+  }
+
+  @Managed
+  @Nested
+  public CounterStat getAppendEntriesFailures() {
+    return appendEntriesFailures;
+  }
+
+  public void incAppendEntriesFailures() {
+    appendEntriesFailures.update(1);
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getAppendEntriesLatency() {
+    return appendEntriesLatency;
+  }
+
+  @Managed
+  @Nested
+  public Distribution getBatchRecoverySize() {
+    return batchRecoverySize;
+  }
+
+  @Managed
+  @Nested
+  public TimeStat getBatchRecoveryLatency() {
+    return batchRecoveryLatency;
+  }
+
+  @Managed
+  public long getAppendEntriesLag() {
+    return appendEntriesLag.get();
+  }
+
+  public void setAppendEntriesLag(long lag) {
+    appendEntriesLag.set(lag < 0 ? 0 : lag);
+  }
+
+  protected static Map<String, String> getExtendedAttributes(
+          final String peerId) {
+    Map<String, String> extendedAttributes = new TreeMap<>();
+    extendedAttributes.put("peer", peerId);
+    return extendedAttributes;
+  }
+
+  public static String getMBeanName(final String name, final String procId,
+                                    final String peerId) {
+    return getMBeanName(ConsensusMetrics.DOMAIN, TYPE, name, procId,
+            getExtendedAttributes(peerId));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/ConsensusHost.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/ConsensusHost.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/ConsensusHost.java
new file mode 100644
index 0000000..ffa6f12
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/ConsensusHost.java
@@ -0,0 +1,55 @@
+package org.apache.hadoop.hbase.consensus.protocol;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+//import org.apache.http.annotation.Immutable;
+
+@ThriftStruct
+//@Immutable
+public final class ConsensusHost {
+  final long term;
+  final String hostId;
+
+  @ThriftConstructor
+  public ConsensusHost(
+    @ThriftField(1) final long term,
+    @ThriftField(2) String address) {
+    this.term = term;
+    this.hostId = address;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof ConsensusHost)) {
+      return false;
+    }
+
+    ConsensusHost that = (ConsensusHost) o;
+
+    if (term != that.term || !hostId.equals(that.hostId)) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @ThriftField(1)
+  public long getTerm() {
+    return term;
+  }
+
+  @ThriftField(2)
+  public String getHostId() {
+    return hostId;
+  }
+
+  @Override
+  public String toString() {
+    return "{host=" + hostId + ", term=" + term + "}";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/DataStoreCommitEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/DataStoreCommitEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/DataStoreCommitEvent.java
new file mode 100644
index 0000000..fcfd1bb
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/DataStoreCommitEvent.java
@@ -0,0 +1,37 @@
+package org.apache.hadoop.hbase.consensus.protocol;
+
+import com.lmax.disruptor.EventFactory;
+
+/**
+ * Class which contains information about the transaction to commit. This will
+ * be used by the Disruptor Producer.
+ */
+public class DataStoreCommitEvent {
+
+  private Payload value;
+  private long commitIndex;
+
+  public DataStoreCommitEvent() {
+    this.value = null;
+  }
+
+  public Payload getValue() {
+    return value;
+  }
+
+  public long getCommitIndex() {
+    return commitIndex;
+  }
+
+  public void setValue(long commitIndex, Payload value) {
+    this.commitIndex = commitIndex;
+    this.value = value;
+  }
+
+  public final static EventFactory<DataStoreCommitEvent> EVENT_FACTORY =
+    new EventFactory<DataStoreCommitEvent>() {
+    public DataStoreCommitEvent newInstance() {
+      return new DataStoreCommitEvent();
+    }
+  };
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/EditId.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/EditId.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/EditId.java
new file mode 100644
index 0000000..3bb860b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/EditId.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.hbase.consensus.protocol;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import org.apache.hadoop.hbase.HConstants;
+
+@ThriftStruct
+public final class EditId implements Comparable<EditId> {
+
+  private final long term;
+  private final long index;
+
+  @ThriftConstructor
+  public EditId(
+    @ThriftField(1) final long term,
+    @ThriftField(2) final long index) {
+    this.term = term;
+    this.index = index;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof EditId)) {
+      return false;
+    }
+
+    EditId editId = (EditId) o;
+
+    if (index != editId.index || term != editId.term) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @ThriftField(1)
+  public long getTerm() {
+    return term;
+  }
+
+  @ThriftField(2)
+  public long getIndex() {
+    return index;
+  }
+
+  @Override
+  public int compareTo(EditId o) {
+
+    if (this.term < o.term) {
+      return -1;
+    } else if (this.term == o.term) {
+      if (this.index < o.index) {
+        return -1;
+      } else if (this.index > o.index) {
+        return 1;
+      }
+      return 0;
+    }
+
+    return 1;
+  }
+
+  public static EditId getElectionEditID(EditId current, int termDelta, int indexDelta) {
+
+    long currentTerm = current.getTerm();
+
+    // if the current term is the seed term (-2) then set it to undefined (-1)
+    // so that the new election term is >= 0. This is assuming that the termDelta
+    // is +ve, which should always be the case as use minimum rank as 1
+    if (currentTerm == HConstants.SEED_TERM) {
+      currentTerm = HConstants.UNDEFINED_TERM_INDEX;
+    }
+
+    return new EditId(currentTerm + termDelta , current.getIndex() + indexDelta);
+  }
+
+  public static EditId getNewAppendEditID(final EditId current) {
+    return new EditId(current.getTerm(), current.getIndex() + 1);
+  }
+
+  @Override
+  public String toString() {
+    return "{term = " + term + ", index = " + index + "}";
+  }
+
+  @Override
+  public EditId clone() {
+    return new EditId(term, index);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = (int) (term ^ (term >>> 32));
+    result = 31 * result + (int) (index ^ (index >>> 32));
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/Payload.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/Payload.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/Payload.java
new file mode 100644
index 0000000..5c6a4a3
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/protocol/Payload.java
@@ -0,0 +1,25 @@
+package org.apache.hadoop.hbase.consensus.protocol;
+
+import com.google.common.util.concurrent.SettableFuture;
+import java.nio.ByteBuffer;
+
+/**
+ * Wrap the data to commit into a object along with the result.
+ */
+public class Payload {
+  final ByteBuffer entries;
+  final SettableFuture<Long> result;
+
+  public Payload(ByteBuffer entries, SettableFuture<Long> result) {
+    this.entries = entries;
+    this.result = result;
+  }
+
+  public ByteBuffer getEntries() {
+    return entries;
+  }
+
+  public SettableFuture<Long> getResult() {
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AbstractPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AbstractPeerManager.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AbstractPeerManager.java
new file mode 100644
index 0000000..e81578d
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/AbstractPeerManager.java
@@ -0,0 +1,134 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerConsensusServer;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServer;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public abstract class AbstractPeerManager implements PeerManagerInterface {
+  // Map of last acked edits of each peer
+  protected Map<String, Long> lastAckedIndexMap;
+
+  // Min acked id across all peers
+  protected volatile long minAckedIndexAcrossAllPeers =
+    HConstants.UNDEFINED_TERM_INDEX;
+  protected final MutableRaftContext c;
+
+  public long getMinAckedIndexAcrossAllPeers() {
+    return minAckedIndexAcrossAllPeers;
+  }
+
+  public Map<String, Long> getLastAckedIndexMap() {
+    return lastAckedIndexMap;
+  }
+
+  public void updateLastPeerAckedIdMap(Map<String, Long> currentMap) {
+    for (String peer : this.lastAckedIndexMap.keySet()) {
+      if (currentMap.containsKey(peer)) {
+        lastAckedIndexMap.put(peer, currentMap.get(peer));
+      }
+    }
+  }
+
+  public AbstractPeerManager(final MutableRaftContext c) {
+    this.c = c;
+    minAckedIndexAcrossAllPeers = HConstants.UNDEFINED_TERM_INDEX;
+    lastAckedIndexMap = new HashMap<>();
+  }
+
+  public void updatePeerAckedId(String address, EditId remoteEdit) {
+    lastAckedIndexMap.put(address, remoteEdit.getIndex());
+
+    minAckedIndexAcrossAllPeers =
+      Collections.min(lastAckedIndexMap.values());
+  }
+
+  /**
+   * Returns the min across all the persisted index across peers in the quorum,
+   * and the locally flushed edits.
+   */
+  public long getMinUnPersistedIndexAcrossQuorum() {
+    return minAckedIndexAcrossAllPeers;
+  }
+
+  public void setMinAckedIndexAcrossAllPeers(long index) {
+    minAckedIndexAcrossAllPeers = index;
+  }
+
+  // Utility functions
+  protected void resetPeers(final Map<String, PeerServer> peers) {
+    for (PeerServer server : peers.values()) {
+      server.resetPeerContext();
+    }
+  }
+
+  // Utility functions
+  protected Map<String, PeerServer> initializePeers(final Map<HServerAddress, Integer> peers) {
+
+    Map<String, PeerServer> peerServers = new HashMap<>();
+    // Initialize the RaftQuorum by setting up the PeerServer
+    for (HServerAddress hostAddress : peers.keySet()) {
+      if (!hostAddress.equals(c.getServerAddress())) {
+        // Generate the PeerServer port: RegionServer Port + fixed port jump
+        HServerAddress peerAddress = RaftUtil
+          .getLocalConsensusAddress(hostAddress);
+        int peerRank = peers.get(hostAddress);
+
+        peerServers.put(peerAddress.getHostAddressWithPort(),
+          new PeerConsensusServer(peerAddress, peerRank, c, c.getConf()));
+        lastAckedIndexMap.put(peerAddress.getHostAddressWithPort(),
+          HConstants.UNDEFINED_TERM_INDEX);
+      }
+    }
+
+    for (PeerServer peer : peerServers.values()) {
+      peer.initialize();
+      if (c.getDataStoreEventListener() != null) {
+        peer.registerDataStoreEventListener(c.getDataStoreEventListener());
+      }
+    }
+    return peerServers;
+  }
+
+  protected void broadcastVoteRequest(final Map<String, PeerServer> peerServers,
+                                      final VoteRequest request) {
+    for (PeerServer peerServer : peerServers.values()) {
+      peerServer.sendRequestVote(request);
+    }
+  }
+
+  protected void broadcastAppendRequest(final Map<String, PeerServer> peerServers,
+                                        AppendRequest request) {
+    for (PeerServer peerServer : peerServers.values()) {
+      peerServer.sendAppendEntries(request);
+    }
+  }
+
+  protected void stop(final Map<String, PeerServer> peerServers) {
+    for (PeerServer peer : peerServers.values()) {
+      peer.stop();
+    }
+  }
+
+  protected String getState(final Map<String, PeerServer> peerServers) {
+    StringBuilder sb = new StringBuilder();
+    for (PeerServer peer : peerServers.values()) {
+      sb.append(peer.getRank()).append("->").append(peer.getLastEditID().getIndex())
+          .append(" ; ");
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public boolean isInJointQuorumMode() {
+    return false;
+  }
+}


[09/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/QuorumMembershipChangeEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/QuorumMembershipChangeEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/QuorumMembershipChangeEvent.java
new file mode 100644
index 0000000..511030c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/QuorumMembershipChangeEvent.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumMembershipChangeRequest;
+
+public class QuorumMembershipChangeEvent extends Event {
+
+  private final QuorumMembershipChangeRequest request;
+
+  public QuorumMembershipChangeEvent(final QuorumMembershipChangeRequest request) {
+    super(RaftEventType.QUORUM_MEMBERSHIP_CHANGE);
+    this.request = request;
+  }
+
+  public QuorumMembershipChangeRequest getRequest() {
+    return request;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/QuorumMembershipChangeInProgress.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/QuorumMembershipChangeInProgress.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/QuorumMembershipChangeInProgress.java
new file mode 100644
index 0000000..20b8857
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/QuorumMembershipChangeInProgress.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class QuorumMembershipChangeInProgress implements Conditional {
+  private static Logger LOG = LoggerFactory.getLogger(AppendNotCompleted.class);
+
+  ImmutableRaftContext c;
+
+  public QuorumMembershipChangeInProgress(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    return c.getUpdateMembershipRequest() != null;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/RaftEventType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/RaftEventType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/RaftEventType.java
new file mode 100644
index 0000000..7d971a3
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/RaftEventType.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.EventType;
+
+public enum RaftEventType implements EventType {
+  NONE,
+  START,
+  VOTE_REQUEST_RECEIVED,
+  VOTE_RESPONSE_RECEIVED,
+  APPEND_REQUEST_RECEIVED,
+  APPEND_RESPONSE_RECEIVED,
+  RESEED_REQUEST_RECEIVED,
+  REPLICATE_ENTRIES,
+  PROGRESS_TIMEOUT,
+  TERM_TIMEOUT,
+  HALT,
+  QUORUM_MEMBERSHIP_CHANGE,
+  MAX
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ReplicateEntriesEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ReplicateEntriesEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ReplicateEntriesEvent.java
new file mode 100644
index 0000000..0b8988d
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ReplicateEntriesEvent.java
@@ -0,0 +1,90 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import com.google.common.util.concurrent.SettableFuture;
+
+import org.apache.hadoop.hbase.consensus.protocol.Payload;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
+
+import java.nio.ByteBuffer;
+
+public class ReplicateEntriesEvent extends Event {
+  private static final Logger LOG = LoggerFactory.getLogger(
+          ReplicateEntriesEvent.class);
+  private final boolean isHeartBeat;
+
+  private final Payload payload;
+
+  public ReplicateEntriesEvent(final boolean isHeartBeat, final ByteBuffer entries) {
+    this (isHeartBeat, entries, null);
+  }
+
+  public ReplicateEntriesEvent(final boolean isHeartBeat,
+                               final ByteBuffer entries,
+                               final SettableFuture<Long> result) {
+    super(RaftEventType.REPLICATE_ENTRIES);
+    this.isHeartBeat = isHeartBeat;
+    this.payload = new Payload(entries, result);
+  }
+
+  public Payload getPayload() {
+    return payload;
+  }
+
+  public ByteBuffer getEntries() {
+    return payload.getEntries();
+  }
+
+  public boolean isHeartBeat() {
+    return this.isHeartBeat;
+  }
+
+  public SettableFuture<Long> getFutureResult() {
+    // won't create the future result for the heart beat msg
+    assert !isHeartBeat;
+    return payload.getResult();
+  }
+
+  public void setReplicationSucceeded(long commitIndex) {
+    assert !isHeartBeat;
+    if (!isHeartBeat() && payload.getResult() != null) {
+      payload.getResult().set(commitIndex);
+    }
+  }
+
+  public void setReplicationFailed(Throwable reason) {
+    assert !isHeartBeat;
+    if (!isHeartBeat() && payload.getResult() != null) {
+      payload.getResult().setException(reason);
+    }
+  }
+
+  @Override
+  public void abort(final String message) {
+    LOG.error(String.format("Aborted %s event: %s", this, message));
+
+    if (!isHeartBeat() && payload.getResult() != null) {
+      payload.getResult().setException(new ThriftHBaseException(new Exception(
+        String.format("Cannot complete the replication request. Reason %s",
+          message))));
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    boolean equals = false;
+    if (this == o) {
+      equals = true;
+    } else {
+      if (o instanceof ReplicateEntriesEvent) {
+        ReplicateEntriesEvent that = (ReplicateEntriesEvent)o;
+        equals = super.equals(that) &&
+                isHeartBeat == that.isHeartBeat &&
+                payload.getEntries().equals(that.payload.getEntries());
+      }
+    }
+    return equals;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ReseedRequestEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ReseedRequestEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ReseedRequestEvent.java
new file mode 100644
index 0000000..65f3907
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ReseedRequestEvent.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.ReseedRequest;
+import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
+
+public class ReseedRequestEvent extends Event {
+  private ReseedRequest request;
+
+  public ReseedRequestEvent(ReseedRequest request) {
+    super(RaftEventType.RESEED_REQUEST_RECEIVED);
+    this.request = request;
+  }
+
+  public ReseedRequest getRequest() {
+    return request;
+  }
+
+  public void abort(final String message) {
+    request.getResult().setException(
+      new ThriftHBaseException(new Exception("Aborted AppendRequestEvent: " + message)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/StartEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/StartEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/StartEvent.java
new file mode 100644
index 0000000..f5f1792
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/StartEvent.java
@@ -0,0 +1,9 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+
+public class StartEvent extends Event {
+  public StartEvent() {
+    super(RaftEventType.START);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteFailed.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteFailed.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteFailed.java
new file mode 100644
index 0000000..e59e1ec
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteFailed.java
@@ -0,0 +1,28 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.*;
+
+public class VoteFailed implements Conditional {
+  ImmutableRaftContext c;
+
+  public VoteFailed(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    // Get the current outstanding election session
+    VoteConsensusSessionInterface session = this.c.getElectionSession(this.c.getCurrentEdit());
+
+    // Handle the stale request
+    if (session == null) {
+      return false;
+    }
+
+    // return true if the majority has sent the step down response
+    return session.getResult().equals(SessionResult.STEP_DOWN);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteNotCompleted.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteNotCompleted.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteNotCompleted.java
new file mode 100644
index 0000000..ae9f336
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteNotCompleted.java
@@ -0,0 +1,31 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSession;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.VoteConsensusSession;
+import org.apache.hadoop.hbase.consensus.quorum.VoteConsensusSessionInterface;
+
+public class VoteNotCompleted implements Conditional {
+  ImmutableRaftContext c;
+
+  public VoteNotCompleted(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    // Get the current outstanding election session
+    VoteConsensusSessionInterface session =
+      c.getElectionSession(this.c.getCurrentEdit());
+
+    // Handle the stale request
+    if (session == null) {
+      return true;
+    }
+
+    return !session.isComplete();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteRequestEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteRequestEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteRequestEvent.java
new file mode 100644
index 0000000..e990a6f
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteRequestEvent.java
@@ -0,0 +1,24 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
+
+public class VoteRequestEvent extends Event {
+
+  private final VoteRequest request;
+
+  public VoteRequestEvent(VoteRequest request) {
+    super(RaftEventType.VOTE_REQUEST_RECEIVED);
+    this.request = request;
+  }
+
+  public VoteRequest getVoteRequest() {
+    return this.request;
+  }
+
+  public void abort(final String message) {
+    request.setError(new ThriftHBaseException(
+      new Exception("Aborted VoteRequestEvent: " + message)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteResponseEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteResponseEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteResponseEvent.java
new file mode 100644
index 0000000..786109e
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteResponseEvent.java
@@ -0,0 +1,17 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.VoteResponse;
+
+public class VoteResponseEvent extends Event {
+  VoteResponse response;
+
+  public VoteResponseEvent(final VoteResponse response) {
+    super(RaftEventType.VOTE_RESPONSE_RECEIVED);
+    this.response = response;
+  }
+
+  public VoteResponse getResponse() {
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteSucceeded.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteSucceeded.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteSucceeded.java
new file mode 100644
index 0000000..39e5d2f
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/VoteSucceeded.java
@@ -0,0 +1,29 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.*;
+
+public class VoteSucceeded implements Conditional {
+  ImmutableRaftContext c;
+
+  public VoteSucceeded(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    // Get the current outstanding election session
+    VoteConsensusSessionInterface session =
+      c.getElectionSession(this.c.getCurrentEdit());
+
+    // Handle the stale request
+    if (session == null) {
+      return false;
+    }
+
+    // return true if the majority has sent the Ack response
+    return session.getResult().equals(SessionResult.MAJORITY_ACKED);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/AckClient.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/AckClient.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/AckClient.java
new file mode 100644
index 0000000..0526060
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/AckClient.java
@@ -0,0 +1,28 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSession;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+
+public class AckClient extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(AckClient.class);
+
+  public AckClient(MutableRaftContext context) {
+    super(RaftStateType.ACK_CLIENT, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+
+    AppendConsensusSessionInterface session = c.getAppendSession(c.getCurrentEdit());
+
+    assert session != null;
+
+    c.advanceCommittedIndex(session.getSessionId());
+    c.setAppendSession(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/BecomeFollower.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/BecomeFollower.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/BecomeFollower.java
new file mode 100644
index 0000000..ed2d509
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/BecomeFollower.java
@@ -0,0 +1,27 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+public class BecomeFollower extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(BecomeFollower.class);
+
+  public BecomeFollower(MutableRaftContext context) {
+    super(RaftStateType.BECOME_FOLLOWER, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+
+    c.leaderStepDown();
+    c.candidateStepDown();
+
+    assert c.getOutstandingAppendSession() == null ;
+    assert c.getOutstandingElectionSession() == null ;
+
+    c.getHeartbeatTimer().stop();
+    c.getProgressTimer().start();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/BecomeLeader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/BecomeLeader.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/BecomeLeader.java
new file mode 100644
index 0000000..e2f13d2
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/BecomeLeader.java
@@ -0,0 +1,73 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.quorum.JointConsensusPeerManager;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumMembershipChangeRequest;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class BecomeLeader extends RaftAsyncState {
+  private static Logger LOG = LoggerFactory.getLogger(BecomeLeader.class);
+
+  private ListenableFuture<?> sendAppendRequestFuture = null;
+
+  public BecomeLeader(MutableRaftContext context) {
+    super(RaftStateType.BECOME_LEADER, context);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return sendAppendRequestFuture == null || sendAppendRequestFuture.isDone();
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    // Clear the election session and sanity check the append session
+    c.setElectionSession(null);
+    assert c.getOutstandingAppendSession() == null;
+
+    // Set up as the leader
+    this.c.setLeader(new ConsensusHost(this.c.getCurrentEdit().getTerm(), this.c.getMyAddress()));
+    this.c.getConsensusMetrics().setRaftState(PeerStatus.RAFT_STATE.LEADER);
+    assert c.isLeader();
+
+    if (LOG.isInfoEnabled()) {
+      LOG.info(c + " is leader with edit: " + c.getCurrentEdit());
+    }
+
+    c.getProgressTimer().stop();
+    c.resetPeers();
+    c.getHeartbeatTimer().start();
+
+    if (c.getDataStoreEventListener() != null) {
+      // Notify the data store to start serving reads/writes
+      try {
+        sendAppendRequestFuture = c.sendAppendRequest(new ReplicateEntriesEvent(false,
+          c.getDataStoreEventListener().becameLeader()));
+      } catch (IOException ioe) {
+        LOG.error(String.format(
+                "%s Caught IOException while generating AppendEntries."
+                + " This is very unexpected, so stepping down.", c), ioe);
+        c.clearLeader();
+      }
+    } else {
+      sendAppendRequestFuture = c.sendEmptyAppendRequest();
+    }
+
+    // We are in middle of Quorum Membership Change, lets continue it
+    if (c.getPeerManager() instanceof JointConsensusPeerManager) {
+      QuorumMembershipChangeRequest request =
+        new QuorumMembershipChangeRequest(c.getPeerManager().getConfigs().get(1));
+      c.setUpdateMembershipRequest(request);
+      request.setCurrentState(
+        QuorumMembershipChangeRequest.QuorumMembershipChangeState.JOINT_CONFIG_COMMIT_IN_PROGRESS);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Candidate.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Candidate.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Candidate.java
new file mode 100644
index 0000000..b2b7029
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Candidate.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+public class Candidate extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(Follower.class);
+
+  public Candidate(MutableRaftContext context) {
+    super(RaftStateType.CANDIDATE, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    assert c.isCandidate();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ChangeQuorumMembership.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ChangeQuorumMembership.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ChangeQuorumMembership.java
new file mode 100644
index 0000000..1a84bb5
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ChangeQuorumMembership.java
@@ -0,0 +1,167 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumMembershipChangeRequest;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumMembershipChangeRequest.QuorumMembershipChangeState;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.regionserver.RaftEventListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+public class ChangeQuorumMembership extends RaftAsyncState {
+  private static Logger LOG = LoggerFactory.getLogger(Leader.class);
+  private SettableFuture<?> changeQuorumMembershipFuture;
+
+  public ChangeQuorumMembership(MutableRaftContext context) {
+    super(RaftStateType.CHANGE_QUORUM_MEMBERSHIP, context);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return changeQuorumMembershipFuture == null || changeQuorumMembershipFuture.isDone();
+  }
+
+  @Override
+  public ListenableFuture<?> getAsyncCompletion() {
+    return changeQuorumMembershipFuture;
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    if (changeQuorumMembershipFuture != null) {
+      assert changeQuorumMembershipFuture.isDone();
+    }
+    changeQuorumMembershipFuture = null;
+    final QuorumMembershipChangeRequest request = c.getUpdateMembershipRequest();
+
+    if (request == null) {
+      LOG.warn("We should not be in this state if there is no membership change" +
+        " in progress. ");
+      return;
+    }
+
+    QuorumMembershipChangeRequest.QuorumMembershipChangeState currState =
+      request.getCurrentState();
+
+    if (currState == QuorumMembershipChangeRequest.QuorumMembershipChangeState.PENDING) {
+      changeQuorumMembershipFuture = SettableFuture.create();
+
+      LOG.debug("Updating the quorum " + c.getQuorumInfo().getQuorumName()
+          + " to a joint quorum membership.");
+      final List<QuorumInfo> configs = Arrays.asList(c.getQuorumInfo(), request.getConfig());
+      // We need to send the new configuration to the quorum. And move to joint
+      // membership state.
+
+      try {
+        c.updateToJointQuorumMembership(configs.get(1));
+      } catch (IOException ioe) {
+        LOG.error("Cannot upgrade to joint quorum membership. Will retry. " +
+          "New config: " + configs.get(1) + " Error :", ioe);
+      }
+
+      // Send the config change entry
+      ListenableFuture<Void> sendAppendReqFuture =
+          (ListenableFuture<Void>) c.sendAppendRequest(new ReplicateEntriesEvent(false, QuorumInfo
+              .serializeToBuffer(configs)));
+      Futures.addCallback(sendAppendReqFuture, new FutureCallback<Void>() {
+
+        @Override
+        public void onSuccess(Void result) {
+          request.setCurrentState(QuorumMembershipChangeState.JOINT_CONFIG_COMMIT_IN_PROGRESS);
+          changeQuorumMembershipFuture.set(null);
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+          if (t instanceof InterruptedException) {
+            LOG.info("Exception while sending append request to update configuration", t);
+            Thread.currentThread().interrupt();
+          } else if (t instanceof ExecutionException) {
+            LOG.info("Exception while sending append request to update configuration", t);
+          }
+          changeQuorumMembershipFuture.set(null);
+        }
+
+      });
+    } else if (currState == QuorumMembershipChangeRequest.QuorumMembershipChangeState.JOINT_CONFIG_COMMIT_IN_PROGRESS) {
+      changeQuorumMembershipFuture = SettableFuture.create();
+      LOG.debug("Updating the quorum " + c.getQuorumInfo().getQuorumName()
+          + " to the new quorum membership.");
+
+      // Send the new config now to both old and new set of peers.
+      ListenableFuture<Void> sendAppendReqFuture =
+          (ListenableFuture<Void>) c.sendAppendRequest(new ReplicateEntriesEvent(false, QuorumInfo
+              .serializeToBuffer(Arrays.asList(request.getConfig()))));
+
+      Futures.addCallback(sendAppendReqFuture, new FutureCallback<Void>() {
+
+        @Override
+        public void onSuccess(Void result) {
+          LOG.info("Sent the append entries request");
+          try {
+            c.updateToNewQuorumMembership(request.getConfig());
+          } catch (IOException e) {
+            LOG.error("Cannot upgrade to new membership. Will retry. New config: "
+                + request.getConfig() + " Error: " + e);
+          }
+          request.setCurrentState(QuorumMembershipChangeState.NEW_CONFIG_COMMIT_IN_PROGRESS);
+          changeQuorumMembershipFuture.set(null);
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+
+          if (t instanceof InterruptedException) {
+            LOG.info("Exception while sending append request to update configuration", t);
+            Thread.currentThread().interrupt();
+          } else if (t instanceof ExecutionException) {
+            LOG.info("Exception while sending append request to update configuration", t);
+          }
+          try {
+            c.updateToNewQuorumMembership(request.getConfig());
+          } catch (IOException e) {
+            LOG.error("Cannot upgrade to new membership. Will retry. New config: "
+                + request.getConfig() + " Error: " + e);
+          }
+          changeQuorumMembershipFuture.set(null);
+        }
+      });
+
+    } else if (currState ==
+      QuorumMembershipChangeRequest.QuorumMembershipChangeState.NEW_CONFIG_COMMIT_IN_PROGRESS) {
+      LOG.debug("Updating the quorum " + c.getQuorumInfo().getQuorumName() +
+        " to the new quorum membership by setting the quorum membership " +
+        " change request to COMPLETE.");
+      request.setResponse(true);
+      // We have completed the quorum membership change, lets go back to normal
+      c.setUpdateMembershipRequest(null);
+      c.cleanUpJointStates();
+
+      request.setCurrentState(
+        QuorumMembershipChangeRequest.QuorumMembershipChangeState.COMPLETE);
+
+      // The leader is no longer the part of the quorum. HALT
+      if (c.getQuorumInfo().getPeersWithRank().get(c.getServerAddress()) == null) {
+        c.stop(false);
+
+        RaftEventListener eventListener = c.getDataStoreEventListener();
+        if (eventListener != null) {
+          eventListener.closeDataStore();
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Follower.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Follower.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Follower.java
new file mode 100644
index 0000000..89e9318
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Follower.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+public class Follower extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(Follower.class);
+
+  public Follower(MutableRaftContext context) {
+    super(RaftStateType.FOLLOWER, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    //assert c.isFollower();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Halt.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Halt.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Halt.java
new file mode 100644
index 0000000..abab02c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Halt.java
@@ -0,0 +1,26 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+public class Halt extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(Halt.class);
+
+  public Halt(MutableRaftContext context) {
+    super(RaftStateType.HALT, context);
+  }
+
+  public void onEntry(final Event e) {
+    LOG.error("Entering HALT state " + c + " Current State: " + t + ", OnEntryEvent: " + e);
+    super.onEntry(e);
+    c.clearLeader();
+    c.leaderStepDown();
+    c.candidateStepDown();
+    c.getProgressTimer().stop();
+    c.getHeartbeatTimer().stop();
+    c.getConsensusMetrics().setRaftState(PeerStatus.RAFT_STATE.HALT);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleAppendRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleAppendRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleAppendRequest.java
new file mode 100644
index 0000000..727fd66
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleAppendRequest.java
@@ -0,0 +1,402 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.fsm.FSMLargeOpsExecutorService;
+import org.apache.hadoop.hbase.consensus.log.CommitLogManagerInterface;
+import org.apache.hadoop.hbase.consensus.log.LogFileInterface;
+import org.apache.hadoop.hbase.consensus.log.TransactionLogManager;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+import org.apache.hadoop.hbase.consensus.raft.events.AppendRequestEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+import org.apache.hadoop.hbase.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+
+public class HandleAppendRequest extends RaftAsyncState {
+  private static Logger LOG = LoggerFactory.getLogger(HandleAppendRequest.class);
+  private final boolean candidateLogPromotionEnabled;
+  private final long raftBatchAppendTryCandidateLogsPromotionThresholdDefault;
+  private volatile SettableFuture<Void> handleAppendReqFuture;
+
+  public HandleAppendRequest(MutableRaftContext context) {
+    super(RaftStateType.HANDLE_APPEND_REQUEST, context);
+    candidateLogPromotionEnabled = c.getConf().getBoolean(
+        HConstants.RAFT_CANDIDATE_LOG_PROMOTION_ENABLED_KEY,
+        HConstants.RAFT_CANDIDATE_LOG_PROMOTION_ENABLED_DEFAULT);
+    raftBatchAppendTryCandidateLogsPromotionThresholdDefault =
+        c.getConf().getLong(
+          HConstants.RAFT_BATCH_APPEND_TRY_CANDIDATE_LOGS_PROMOTION_THRESHOLD_KEY,
+          HConstants.RAFT_BATCH_APPEND_TRY_CANDIDATE_LOGS_PROMOTION_THRESHOLD_DEFAULT);
+  }
+
+  private boolean candidateLogPromotionEnabled() {
+    return candidateLogPromotionEnabled;
+  }
+
+  private static class SingleAppendResult {
+    private int which;
+    private AppendResponse.Result result;
+    private EditId  validId;
+
+    public SingleAppendResult(int which, AppendResponse.Result result, EditId validId) {
+      update(which, result, validId);
+    }
+    public void update(int which, AppendResponse.Result result, EditId validId) {
+      this.which = which;
+      this.result = result;
+      this.validId = validId;
+    }
+
+    public AppendResponse.Result getResult() {
+      return result;
+    }
+
+    public EditId validId() {
+      return validId;
+    }
+
+    public int getWhich() {
+      return which;
+    }
+  }
+
+  @Override
+  public boolean isComplete() {
+    return handleAppendReqFuture == null || handleAppendReqFuture.isDone();
+  }
+
+  @Override
+  public ListenableFuture<?> getAsyncCompletion() {
+    return handleAppendReqFuture;
+  }
+
+  @Override
+  public void onEntry(final Event event) {
+    super.onEntry(event);
+    handleAppendReqFuture = null;
+    final long start = System.nanoTime();
+    AppendRequestEvent e = (AppendRequestEvent) event;
+    final AppendRequest request = e.getRequest();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(c + " processing AppendRequest " + request);
+    }
+
+    assert request != null;
+    assert request.validateFields();
+
+    EditId prevLogId = request.getPrevLogId();
+
+    boolean promotedSomeCandidateLogs = false;
+    // We poke the Candidate Logs Manager first.
+    // Note we use the global commit index here. Current LogId could be very, very far
+    // behind the global commit index.
+    if (candidateLogPromotionEnabled() &&
+      prevLogId.getIndex() + getTryPromotingCandidateLogsThreshold() < request.getCommitIndex()) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(c + " . try promoting candidate logs before appends");
+      }
+      promotedSomeCandidateLogs = tryToIncorporateCandidateLogs(request.getCommitIndex());
+    } else if (LOG.isTraceEnabled()) {
+      LOG.trace(c + " . not promoting candidate logs.");
+    }
+
+    final SingleAppendResult lastResult = new SingleAppendResult(0, AppendResponse.Result.LAGGING, null);
+
+    // Check if we can apply the edits.
+    if (canApplyEdit(request, 0, prevLogId, lastResult)) {
+      handleAppendReqFuture = SettableFuture.create();
+      // Stop the timer, because we are actually making some progress.
+      c.getProgressTimer().stop();
+      final boolean promotedSomeCandidatesLogsFinal = false;
+      ListenableFuture<EditId> applyAllEditsFuture = applyAllEdits(request, prevLogId, lastResult);
+      Futures.addCallback(applyAllEditsFuture, new FutureCallback<EditId>() {
+
+        @Override
+        public void onSuccess(EditId result) {
+          c.getProgressTimer().start();
+          finish(start, request, result, promotedSomeCandidatesLogsFinal, lastResult);
+          handleAppendReqFuture.set(null);
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+          c.getProgressTimer().start();
+          request.setError(t);
+          LOG.error("applyAllEdits failed for quorum: " + c.getQuorumInfo().getQuorumName(), t);
+          handleAppendReqFuture.set(null);
+        }
+      });
+    } else {
+      finish(start, request, prevLogId, promotedSomeCandidateLogs, lastResult);
+    }
+  }
+
+  /**
+   * @param start
+   * @param request
+   * @param prevLogId
+   * @param promotedSomeCandidateLogs
+   * @param lastResult
+   */
+  private void finish(long start, final AppendRequest request, EditId prevLogId,
+      boolean promotedSomeCandidateLogs, SingleAppendResult lastResult) {
+    request.setResponse(
+      new AppendResponse(
+        c.getMyAddress(),
+        request.getLogId(0), // 1st EditID in the request is the identifier for the AppendSession
+        lastResult.validId(),
+        lastResult.getResult(),
+        c.getRanking(),
+        c.canTakeOver() // can takeover
+      ));
+
+    // Add request latency
+    c.getConsensusMetrics().getAppendEntriesLatency().add(
+      System.nanoTime() - start, TimeUnit.NANOSECONDS);
+
+    // We may have skipped log promotion because of some gaps. If that's the case, after we append
+    // a few edits, we may be able to promote some candidate logs. Do this now and we will save
+    // another round trip from the leader.
+    if (lastResult.getResult() == AppendResponse.Result.SUCCESS &&
+      candidateLogPromotionEnabled() &&
+      !promotedSomeCandidateLogs &&
+      prevLogId.getIndex() + getTryPromotingCandidateLogsThreshold() < request.getCommitIndex()
+      ) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(c + " . try promoting candidate logs after successful appends");
+      }
+      promotedSomeCandidateLogs = tryToIncorporateCandidateLogs(request.getCommitIndex());
+    }
+  }
+
+  /**
+   * @param request
+   * @param prevLogId
+   * @param lastResult
+   * @return
+   * @throws Throwable
+   */
+  private ListenableFuture<EditId> applyAllEdits(final AppendRequest request, final EditId prevLogId,
+      final SingleAppendResult lastResult) {
+    return FSMLargeOpsExecutorService.fsmWriteOpsExecutorService.submit(new Callable<EditId>() {
+      @Override
+      public EditId call() throws Exception {
+        EditId prevLogIdNonFinal = prevLogId;
+        try {
+          // we take each log in the list and try to apply it locally.
+          for (int i = 0; i < request.logCount(); i++) {
+            EditId logId = request.getLogId(i);
+            if (!applySingleEdit(request, i, prevLogIdNonFinal, lastResult)) {
+              break;
+            }
+            prevLogIdNonFinal = logId;
+          }
+        } catch (Throwable t) {
+          throw t;
+        }
+        return prevLogIdNonFinal;
+      }
+    });
+
+  }
+
+  private boolean canApplyEdit(
+    final AppendRequest request,
+    final int which,
+    final EditId prevLogId,
+    SingleAppendResult result) {
+    boolean appendResult = true;
+    if (request.getLeaderId().getTerm() < c.getCurrentEdit().getTerm() ||
+      !c.validateLogEntry(prevLogId)) {
+      // When the current replica is lagging or higher term
+      EditId lastLoggedEditID;
+      // Reset the timer only if you are lagging entries. In case the leader's
+      // term is less than yours, than don't count this request towards progress.
+      if (request.getLeaderId().getTerm() >= c.getCurrentEdit().getTerm()) {
+        c.getProgressTimer().reset();
+      }
+
+      // If old term, or previous edit does not match
+      appendResult = false;
+      lastLoggedEditID = c.getLogManager().getLastValidTransactionId();
+
+      if (request.getLeaderId().getTerm() < c.getCurrentEdit().getTerm()) {
+        // when the current peer has a higher term
+        result.update(which, AppendResponse.Result.HIGHER_TERM, lastLoggedEditID);
+      } else {
+        // when the current peer is lagging
+        result.update(which, AppendResponse.Result.LAGGING, lastLoggedEditID);
+      }
+    }
+
+    EditId prevEdit = request.getLogId(0);
+    for (int i = 1; i < request.logCount(); i++) {
+      EditId curEdit = request.getLogId(i);
+      if (curEdit.getIndex() != prevEdit.getIndex() + 1) {
+        result.update(which, AppendResponse.Result.MISSING_EDITS,
+          c.getLogManager().getLastValidTransactionId());
+        LOG.error("Missing edits in request " + request +
+          ". The next edit after " + prevEdit + " was " + curEdit);
+        appendResult = false;
+        break;
+      }
+      prevEdit = curEdit;
+    }
+
+    return appendResult;
+  }
+
+  /*
+   *  applySingleEdit
+   *
+   *  Apply only one log from AppendRequest.
+   */
+  private boolean applySingleEdit(
+      final AppendRequest       request,
+      final int                 which,
+      final EditId              prevLogId,
+            SingleAppendResult  result   // simulate C-style passing by reference
+  ) {
+    boolean appendResult = true;
+    EditId currEditID = request.getLogId(which);
+
+    // request.getLeaderId().getTerm() >= c.getCurrentEdit().getTerm() &&
+    // c.validateLogEntry(prevLogId)
+
+    TransactionLogManager logManager = (TransactionLogManager)c.getLogManager();
+    boolean needCheckInvariants = false;
+    // If the leader receives a valid append request with higher term, print debug message and
+    // check integrity of its hlogs.
+    if (c.isLeader() && !c.getLeader().equals(request.getLeaderId())) {
+      Map.Entry<Long, LogFileInterface> currentLogEntry = logManager.getUncommittedLogs().lastEntry();
+      LogFileInterface currentLog = null;
+      if (currentLogEntry != null) {
+        currentLog = currentLogEntry.getValue();
+      }
+      LOG.warn(
+          String.format("Received a higher term append request from %s when I'm the leader. "
+                      + "Current edit: %s. Request: %s. Current log info: %s",
+              request.getLeaderId(), c.getCurrentEdit(), request,
+              currentLog == null ? "null" : currentLog.toString()));
+      try {
+        logManager.checkInvariants();
+      } catch (IOException e) {
+        LOG.error("Logs are messed up!", e);
+        return false;
+      }
+      needCheckInvariants = true;
+    }
+    c.setLeader(request.getLeaderId());
+    c.clearVotedFor();
+
+    c.setLastAppendRequestReceivedTime(System.currentTimeMillis());
+    c.setMinAckedIndexAcrossAllPeers(request.getPersistedIndex());
+
+    // The commit index in the request is the highest in the entire batch. Hence
+    // for this request, we should take the min of current logId and commit index.
+    long commitIndex = Math.min(currEditID.getIndex(), request.getCommitIndex());
+
+    // isExist() takes the read flow which grabs a lock while doing a lookup. One
+    // small optimization is to check if the current index is less than the
+    // input index. We are safe to do the check here as we have already verified
+    // that the previous edit id is present in the log
+    if (c.getCurrentEdit().getIndex() < currEditID.getIndex() ||
+      !c.getLogManager().isExist(currEditID)) {
+      // Reset the leader which will ask the current leader/candidate to stepdown in the
+      // BecomeFollower state
+      if (!request.isHeartBeat()) {
+        try {
+          c.truncateLogEntries(prevLogId);
+          ByteBuffer edit = request.getEdit(which);
+          c.appendToLog(currEditID, commitIndex, edit);
+          c.setPreviousEditId(currEditID);
+          c.setCurrentEditId(currEditID);
+          if (QuorumInfo.isQuorumChangeRequest(edit)) {
+            LOG.debug("Received a quorum change request. " +
+              "Caching it for now, will handle later. EditId: " + currEditID);
+          }
+        } catch (IOException ex) {
+          LOG.error("Cannot append edit " + request.getLogId(which), ex);
+          appendResult = false;
+        }
+        if (appendResult == false) {
+          // return the last edit present in the log manager
+          currEditID = c.getLogManager().getLastEditID();
+        }
+      }
+    } else {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(c + " . Transaction already exists: " + currEditID);
+      }
+      if (!request.isHeartBeat()) {
+        c.getConsensusMetrics().incAppendEntriesDuplicates();
+      }
+    }
+
+    if (needCheckInvariants) {
+      // Check invariants again after the append is done
+      try {
+        logManager.checkInvariants();
+      } catch (IOException e) {
+        LOG.error("Logs are messed up!", e);
+        return false;
+      }
+    }
+
+    // Move forward the commit index. This should be done regardless of
+    // whether the current message is a heartbeat or not. The information
+    // about the commitIndex is piggy backed on the current message. But,
+    // does not depend on the current message in any way.
+    c.advanceCommittedIndex(new EditId(request.getLogId(which).getTerm(),
+      commitIndex));
+
+    if (appendResult) {
+      result.update(which, AppendResponse.Result.SUCCESS, currEditID);
+    } else {
+      result.update(which, AppendResponse.Result.LAGGING, currEditID);
+    }
+    return appendResult;
+  }
+
+
+  /**
+   *  tryToIncorporateCandidateLogs
+   *
+   *  Trying to incorporate some candidate log files given current append request.
+   */
+  public boolean tryToIncorporateCandidateLogs(long lastCommittedIndex) {
+    CommitLogManagerInterface logman = c.getLogManager();
+    long t0 = System.currentTimeMillis();
+    Pair<EditId, EditId> result = logman.greedyIncorporateCandidateLogs(c.toString(), lastCommittedIndex);
+    if (result != null && result.getSecond() != null) {
+      EditId prevEditId = result.getFirst();
+      EditId lastEditId = result.getSecond();
+      long prevIndex = prevEditId != null ?
+        (prevEditId.getIndex() != HConstants.UNDEFINED_TERM_INDEX ? prevEditId.getIndex() : 0) : 0;
+      LOG.info("Incorporated " + (lastEditId.getIndex() - prevIndex) + " edits in "
+          + (System.currentTimeMillis() - t0) + " milliseconds.");
+      c.setPreviousEditId(lastEditId);
+      c.setCurrentEditId(lastEditId);
+      return true;
+    }
+    return false;
+  }
+
+  public long getTryPromotingCandidateLogsThreshold() {
+    return raftBatchAppendTryCandidateLogsPromotionThresholdDefault;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleAppendResponse.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleAppendResponse.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleAppendResponse.java
new file mode 100644
index 0000000..ce4da56
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleAppendResponse.java
@@ -0,0 +1,64 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.raft.events.AppendResponseEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HandleAppendResponse extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(HandleAppendResponse.class);
+
+  public HandleAppendResponse(MutableRaftContext context) {
+    super(RaftStateType.HANDLE_APPEND_RESPONSE, context);
+  }
+
+  public void onEntry(final Event event) {
+    super.onEntry(event);
+
+    AppendResponseEvent e = (AppendResponseEvent)event;
+    final AppendResponse response = e.getResponse();
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(c.toString() + " handling " + response);
+    }
+
+    EditId currentEdit = c.getCurrentEdit();
+    EditId remoteCurrentEdit = response.getId();
+
+    // Ignore the old response
+    if (!currentEdit.equals(remoteCurrentEdit)) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Older response " + response.getId());
+      }
+      return;
+    }
+
+    // Verify the session is valid
+    AppendConsensusSessionInterface session = c.getAppendSession(response.getId());
+    if (session == null) {
+      return;
+    }
+
+    switch (response.getResult()) {
+      case SUCCESS:
+        session.incrementAck(
+          remoteCurrentEdit, response.getAddress(), response.getRank(), response.canTakeover());
+        c.updatePeerAckedId(response.getAddress(), remoteCurrentEdit);
+        break;
+      case HIGHER_TERM:
+        session.incrementHighTermCnt(remoteCurrentEdit, response.getAddress());
+        break;
+      case LAGGING:
+        session.incrementLagCnt(remoteCurrentEdit, response.getAddress());
+        break;
+      default:
+        LOG.error("[Error] AppendSession received an unexpected response. Current edit is "
+          + currentEdit + " , remote edit is " + remoteCurrentEdit + "; " +
+          "And response is " + response);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleQuorumMembershipChangeRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleQuorumMembershipChangeRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleQuorumMembershipChangeRequest.java
new file mode 100644
index 0000000..2c627b1
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleQuorumMembershipChangeRequest.java
@@ -0,0 +1,108 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumMembershipChangeRequest;
+import org.apache.hadoop.hbase.consensus.raft.events
+  .QuorumMembershipChangeEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HandleQuorumMembershipChangeRequest extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(Leader.class);
+
+  public HandleQuorumMembershipChangeRequest(MutableRaftContext context) {
+    super(RaftStateType.HANDLE_QUORUM_MEMBERSHIP_CHANGE_REQUEST, context);
+  }
+
+  public void onEntry(final Event e) {
+    QuorumMembershipChangeRequest request = null;
+    if (e instanceof QuorumMembershipChangeEvent) {
+      request = ((QuorumMembershipChangeEvent)e).getRequest();
+    }
+    if (request == null) {
+      if (LOG.isWarnEnabled()) {
+        LOG.warn("Change event did not include a request");
+      }
+      return;
+    }
+
+    // Make sure we have a valid change request for this FSM before moving on.
+    final QuorumInfo newConfig = request.getConfig();
+    final QuorumInfo currConfig = c.getQuorumInfo();
+    if (currConfig == null || newConfig == null ||
+            !currConfig.getQuorumName().equals(newConfig.getQuorumName())) {
+      LOG.warn("Quorum name in current and new info does not match");
+      request.setResponse(false);
+      return;
+    }
+
+    // Make sure there is no existing Quorum Membership Change request in flight
+    if (c.getUpdateMembershipRequest() != null) {
+      if (LOG.isWarnEnabled()) {
+        LOG.warn("There is an existing quorum membership change request in " +
+          "progress. " + c.getUpdateMembershipRequest() +
+          " Cannot accept a new request.");
+      }
+      request.setResponse(false);
+      return;
+    }
+
+    // If the given QuorumInfo is not different from the current QuorumInfo
+    // assume a duplicate and just move on.
+    if (currConfig.equals(newConfig)) {
+      if (LOG.isWarnEnabled()) {
+        LOG.warn("Current and new QuorumInfo are equal." +
+                " Ignoring the request as it is most likely a duplicate");
+      }
+      request.setResponse(true);
+      return;
+    }
+
+    // If the replica set does not change the quorum info can be updated
+    // without going through the protocol.
+    if (currConfig.hasEqualReplicaSet(newConfig)) {
+      c.setQuorumInfo(newConfig);
+      if (LOG.isInfoEnabled()) {
+        LOG.info(String.format("Updating %s, new config: %s", c.getQuorumName(),
+                newConfig.getPeersWithRank()));
+      }
+      request.setResponse(true);
+      return;
+    }
+
+    // Do not allow change of majority of peers in the quorum. This can lead to
+    // issues where we can encounter data loss.
+    int disjointPeerSetCount = 0;
+    for (String peer : newConfig.getPeersAsString()) {
+      if (!currConfig.getPeersAsString().contains(peer)) {
+        ++disjointPeerSetCount;
+      }
+    }
+
+    if (disjointPeerSetCount >= c.getMajorityCnt()) {
+      if (LOG.isWarnEnabled()) {
+        LOG.warn("Cannot change majority of the peers in quorum at the same time.");
+      }
+      request.setResponse(false);
+      return;
+    }
+
+    // Only if this replica is leader should an info change be requested through
+    // the protocol.
+    if (c.isLeader()) {
+      if (LOG.isInfoEnabled()) {
+        LOG.info(String.format("Updating %s, new config: %s", c.getQuorumName(),
+                newConfig.getPeersWithRank()));
+      }
+      c.setUpdateMembershipRequest(request);
+      return;
+    }
+
+    // At this point this request is either not valid or this replica is
+    // follower or candidate and it is probably safe to reject at this point.
+    LOG.debug("Cannot process quorum request change. The request is either invalid, or replica is a follower/candidate");
+    request.setResponse(false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleReseedRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleReseedRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleReseedRequest.java
new file mode 100644
index 0000000..f1a152d
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleReseedRequest.java
@@ -0,0 +1,40 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.quorum.ReseedRequest;
+import org.apache.hadoop.hbase.consensus.raft.events.ReseedRequestEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+import java.io.IOException;
+
+public class HandleReseedRequest extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(HandleVoteRequest.class);
+
+  public HandleReseedRequest(MutableRaftContext context) {
+    super(RaftStateType.HANDLE_RESEED_REQUEST, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+
+    ReseedRequest request = ((ReseedRequestEvent)e).getRequest();
+
+    // In case you are the leader, just acknowledge it and move on
+    if (c.isLeader()) {
+      request.setResponse(true);
+      return;
+    }
+
+    try {
+      c.reseedStartIndex(request.getReseedIndex());
+    } catch (IOException e1) {
+      LOG.error("Cannot complete the reseed request ", e1);
+      request.getResult().setException(e1);
+    }
+
+    request.getResult().set(true);
+    c.candidateStepDown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleVoteRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleVoteRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleVoteRequest.java
new file mode 100644
index 0000000..1e50e8b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleVoteRequest.java
@@ -0,0 +1,105 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.raft.events.VoteRequestEvent;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+public class HandleVoteRequest extends RaftAsyncState {
+  private static Logger LOG = LoggerFactory.getLogger(HandleVoteRequest.class);
+  private SettableFuture<Void> handleVoteRequestFuture;
+
+  public HandleVoteRequest(MutableRaftContext context) {
+    super(RaftStateType.HANDLE_VOTE_REQUEST, context);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return handleVoteRequestFuture == null || handleVoteRequestFuture.isDone();
+  }
+
+  @Override
+  public ListenableFuture<?> getAsyncCompletion() {
+    return handleVoteRequestFuture;
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    handleVoteRequestFuture = null;
+    super.onEntry(e);
+    // Assume there is only type of event here
+    final VoteRequest request =  ((VoteRequestEvent)e).getVoteRequest();
+
+    // First check if we received a VoteRequest from a non-member. This can
+    // happen in cases when some peers did not transition properly to a new
+    // quorum during a quorum membership change.
+    if (!c.getPeerManager().getPeerServers().keySet()
+      .contains(request.getAddress())) {
+      LOG.info("Received VoteRequest from " + request.getAddress() +
+        ", even though it is not part of the quorum: " +
+        c.getPeerManager().getPeerServers().keySet());
+      finishRequest(request, VoteResponse.VoteResult.WRONGQUORUM);
+    } else if (c.getCurrentEdit().getTerm() < request.getTerm() &&
+      c.getLogManager().getLastEditID().compareTo(request.getPrevEditID()) <= 0) {
+
+      handleVoteRequestFuture = SettableFuture.create();
+      c.getProgressTimer().reset();
+
+      if (c.isLeader()) {
+        LOG.debug(c + " has stepped down from leader state due to vote request: " + request);
+      }
+
+      c.clearLeader();
+      ListenableFuture<Void> votedForDone = c.setVotedFor(new ConsensusHost(request.getTerm(), request.getAddress()));
+      Futures.addCallback(votedForDone, new FutureCallback<Void>() {
+
+        @Override
+        public void onSuccess(Void result) {
+          c.setCurrentEditId(
+            new EditId(request.getTerm(), c.getCurrentEdit().getIndex()));
+          finishRequest(request, VoteResponse.VoteResult.SUCCESS);
+          handleVoteRequestFuture.set(null);
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+          request.setError(t);
+          // Set the remote machine reachable when receiving any RPC request
+          c.setPeerReachable(request.getAddress());
+          LOG.error("setVotedFor failed for quorum: " + c.getQuorumInfo().getQuorumName(), t);
+          handleVoteRequestFuture.set(null);
+        }
+      });
+    } else {
+      finishRequest(request, VoteResponse.VoteResult.FAILURE);
+    }
+  }
+
+  /**
+   * @param request
+   * @param voteResult
+   */
+  private void finishRequest(final VoteRequest request, VoteResponse.VoteResult voteResult) {
+    // Set the response for this request
+    VoteResponse response = new VoteResponse(this.c.getMyAddress(),
+      request.getTerm(), voteResult);
+    request.setResponse(response);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(c + " is sending vote response: " + response +
+        " for request " + request);
+    }
+
+    // Set the remote machine reachable when receiving any RPC request
+    c.setPeerReachable(request.getAddress());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleVoteResponse.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleVoteResponse.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleVoteResponse.java
new file mode 100644
index 0000000..ad9e38c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/HandleVoteResponse.java
@@ -0,0 +1,59 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.quorum.VoteConsensusSessionInterface;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.raft.events.VoteResponseEvent;
+import org.apache.hadoop.hbase.consensus.rpc.VoteResponse;
+
+public class HandleVoteResponse extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(HandleVoteResponse.class);
+
+  public HandleVoteResponse(MutableRaftContext context) {
+    super(RaftStateType.HANDLE_VOTE_RESPONSE, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+
+    // Get the vote response from the event
+    final VoteResponse response = ((VoteResponseEvent)e).getResponse();
+
+    // Update the ElectionSession. If the currentEdit has been increased,
+    // then there must be a new leader or candidate with much higher term in the quorum.
+    VoteConsensusSessionInterface session =
+      c.getElectionSession(this.c.getCurrentEdit());
+    if (session != null) {
+      if (response.isSuccess()) {
+        session.incrementAck(this.c.getCurrentEdit(), response.getAddress());
+      } else {
+        if (response.isWrongQuorum()) {
+          LOG.warn("As per the VoteResponse from " + response.getAddress() +
+            ", it is possible that I am in the wrong quorum. ");
+          if (!c.isPartOfNewQuorum()) {
+            LOG.info("There was a Joint Quorum Configuration change in the " +
+              "past, wherein I would not be a part of the new quorum. " +
+              "Closing myself.");
+            session.setVoteSessionFailed(this.c.getCurrentEdit());
+            c.stop(false);
+            if (c.getDataStoreEventListener() != null) {
+              LOG.debug("Issuing a request to close the quorum: " +
+                c.getQuorumName());
+              c.getDataStoreEventListener().closeDataStore();
+            } else {
+              LOG.debug("No event listener registered, so can't stop the quorum: " +
+                c.getQuorumName());
+            }
+          }
+        }
+        session.incrementNack(this.c.getCurrentEdit(), response.getAddress());
+      }
+    } else {
+      if (LOG.isWarnEnabled()) {
+        LOG.warn(c.toString() + ": VoteConsensusSession is null for " + c.getCurrentEdit());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Leader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Leader.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Leader.java
new file mode 100644
index 0000000..02c1d2d
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Leader.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+public class Leader extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(Leader.class);
+
+  public Leader(MutableRaftContext context) {
+    super(RaftStateType.LEADER, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+    assert c.isLeader();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ProgressTimeout.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ProgressTimeout.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ProgressTimeout.java
new file mode 100644
index 0000000..3b81623
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ProgressTimeout.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+public class ProgressTimeout extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(ProgressTimeout.class);
+
+  public ProgressTimeout(MutableRaftContext context) {
+    super(RaftStateType.PROGRESS_TIMEOUT, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftAsyncState.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftAsyncState.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftAsyncState.java
new file mode 100644
index 0000000..9e0a9ba
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftAsyncState.java
@@ -0,0 +1,17 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+/**
+ * A {@link RaftState} which has async tasks.
+ */
+public class RaftAsyncState extends RaftState {
+  public RaftAsyncState(final RaftStateType t, MutableRaftContext c) {
+    super(t, c);
+  }
+
+  @Override
+  public boolean isAsyncState() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftState.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftState.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftState.java
new file mode 100644
index 0000000..79df1a7
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftState.java
@@ -0,0 +1,29 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.fsm.State;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+public class RaftState extends State {
+  private static Logger LOG = LoggerFactory.getLogger(RaftState.class);
+
+  protected MutableRaftContext c;
+  RaftState(final RaftStateType t, MutableRaftContext c) {
+    super(t);
+    this.c = c;
+  }
+
+  public void onEntry(final Event e) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(c + " Current State: " + t + ", OnEntryEvent: " + e);
+    }
+  }
+
+  public void onExit(final Event e) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(c + " Current State: " + t + ", OnExitEvent: " + e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftStateType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftStateType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftStateType.java
new file mode 100644
index 0000000..d4302d9
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/RaftStateType.java
@@ -0,0 +1,35 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.fsm.StateType;
+
+public enum RaftStateType implements StateType {
+  NONE,
+
+  START,
+  BECOME_FOLLOWER,
+  FOLLOWER,
+
+  CANDIDATE,
+
+  BECOME_LEADER,
+  LEADER,
+  CHANGE_QUORUM_MEMBERSHIP,
+
+  SEND_VOTE_REQUEST,
+  HANDLE_VOTE_REQUEST,
+  HANDLE_VOTE_RESPONSE,
+  HANDLE_QUORUM_MEMBERSHIP_CHANGE_REQUEST,
+
+  SEND_APPEND_REQUEST,
+  RESEND_APPEND_REQUEST,
+  HANDLE_APPEND_REQUEST,
+  HANDLE_APPEND_RESPONSE,
+
+  HANDLE_RESEED_REQUEST,
+
+  PROGRESS_TIMEOUT,
+
+  ACK_CLIENT,
+
+  HALT
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ReSendAppendRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ReSendAppendRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ReSendAppendRequest.java
new file mode 100644
index 0000000..ce71204
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/ReSendAppendRequest.java
@@ -0,0 +1,31 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSession;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.SessionResult;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+
+public class ReSendAppendRequest extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(ReSendAppendRequest.class);
+
+  public ReSendAppendRequest(MutableRaftContext context) {
+    super(RaftStateType.RESEND_APPEND_REQUEST, context);
+  }
+
+  public void onEntry(final Event event) {
+    super.onEntry(event);
+
+    // Get and check the current outstanding append session
+    AppendConsensusSessionInterface session = c.getAppendSession(c.getCurrentEdit());
+    assert session != null && session.getResult().equals(SessionResult.RETRY);
+
+    if (!session.isTimeout()) {
+      c.resendOutstandingAppendRequest();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/SendAppendRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/SendAppendRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/SendAppendRequest.java
new file mode 100644
index 0000000..19479e8
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/SendAppendRequest.java
@@ -0,0 +1,82 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.consensus.exceptions.LeaderNotReadyException;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ *
+ * The basic logic here is to ensure there is only one outstanding AppendRequest
+ * in the quorum (HB is one special AppendRequest as well). The following table
+ * described the mechanism of handling the next AppendRequest if the current Append
+ * session has not been completed yet.
+ *
+ * ----------------------------------------------------------
+ * |next\current |     HB               |     Append        |
+ * ----------------------------------------------------------
+ * |   HB        |  resend HB           | resend Append     |
+ * ----------------------------------------------------------
+ * |  Append     |  send immediately    | NotReadyException |
+ * ----------------------------------------------------------
+ */
+public class SendAppendRequest extends RaftAsyncState {
+  public static final Log LOG = LogFactory.getLog(SendAppendRequest.class);
+  private ListenableFuture<?> sendAppendRequestFuture;
+
+  public SendAppendRequest(MutableRaftContext context) {
+    super(RaftStateType.SEND_APPEND_REQUEST, context);
+  }
+
+  @Override
+  public boolean isComplete() {
+    return sendAppendRequestFuture == null || sendAppendRequestFuture.isDone();
+  }
+
+  @Override
+  public ListenableFuture<?> getAsyncCompletion() {
+    return sendAppendRequestFuture;
+  }
+
+  @Override
+  public void onEntry(final Event event) {
+    super.onEntry(event);
+    sendAppendRequestFuture = null;
+
+    final EditId currentEdit = c.getCurrentEdit();
+    ReplicateEntriesEvent rEvent = (ReplicateEntriesEvent)event;
+
+    AppendConsensusSessionInterface session = c.getAppendSession(currentEdit);
+
+    if (session != null && !session.isComplete()) {
+      // Handling the case where the current append session has NOT completed
+
+      if (rEvent.isHeartBeat() && !session.isTimeout()) {
+        // Resend the heartbeat
+        c.resendOutstandingAppendRequest();
+      } else {
+        if (session.getAppendRequest().isHeartBeat()) {
+          // Cancel the original AppendSession
+          session.cancel();
+          // Resend the ReplicateEntriesEvent
+          c.getHeartbeatTimer().reset();
+          c.sendAppendRequest(rEvent);
+        } else {
+          // Throw the LeaderNotReadyException
+          rEvent.setReplicationFailed(new LeaderNotReadyException(
+            c.getLeaderNotReadyMsg()));
+        }
+      }
+    } else { // Handling the case where the current session has completed
+      // Resend the ReplicateEntriesEvent
+      c.getHeartbeatTimer().reset();
+      sendAppendRequestFuture = c.sendAppendRequest(rEvent);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/SendVoteRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/SendVoteRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/SendVoteRequest.java
new file mode 100644
index 0000000..e2523d22
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/SendVoteRequest.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.VoteConsensusSessionInterface;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+public class SendVoteRequest extends RaftAsyncState {
+  private SettableFuture<Void> handleSendVoteRequestFuture;
+
+  @Override
+  public boolean isComplete() {
+    return handleSendVoteRequestFuture == null || handleSendVoteRequestFuture.isDone();
+  }
+
+  @Override
+  public ListenableFuture<?> getAsyncCompletion() {
+    return handleSendVoteRequestFuture;
+  }
+
+  public SendVoteRequest(MutableRaftContext context) {
+    super(RaftStateType.SEND_VOTE_REQUEST, context);
+  }
+
+  @Override
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+
+    handleSendVoteRequestFuture = null;
+    EditId prevEditID = c.getCurrentEdit();
+    EditId nextEditId = prevEditID;
+
+    final VoteConsensusSessionInterface oldVoteConsensusSession =
+      c.getElectionSession(prevEditID);
+
+    // In case the current election session is not complete, retry the election
+    // with the same edit id, else get a new election id
+    if (oldVoteConsensusSession != null &&
+      !oldVoteConsensusSession.isComplete()) {
+      finish(oldVoteConsensusSession.getRequest());
+    } else {
+      handleSendVoteRequestFuture = SettableFuture.create();
+
+      // Increment the current term and index
+      nextEditId = EditId.getElectionEditID(prevEditID, c.getRanking(), 0);
+      c.setCurrentEditId(nextEditId);
+
+      // Set the VoteFor and reset the leader
+      c.clearLeader();
+      final EditId finalNextEditId = nextEditId;
+      ListenableFuture<Void> votedForDone = c.setVotedFor(new ConsensusHost(nextEditId.getTerm(), c.getMyAddress()));
+      Futures.addCallback(votedForDone, new FutureCallback<Void>() {
+
+        @Override
+        public void onSuccess(Void result) {
+          // Prepare the VoteRequest
+          VoteRequest request = new VoteRequest(c.getQuorumName(), c.getMyAddress(),
+            finalNextEditId.getTerm(), c.getPreviousEdit());
+
+          // Prepare the vote session
+          VoteConsensusSessionInterface electionSession =
+            c.getPeerManager().createVoteConsensusSession(
+              c.getMajorityCnt(), request, c.getConsensusMetrics());
+
+          // Set the outstanding the election session
+          c.setElectionSession(electionSession);
+
+          // Increment the ack for the local server
+          electionSession.incrementAck(finalNextEditId, c.getMyAddress());
+
+          finish(request);
+          handleSendVoteRequestFuture.set(null);
+
+        }
+
+        @Override
+        public void onFailure(Throwable t) {
+          handleSendVoteRequestFuture.set(null);
+        }
+      });
+    }
+  }
+
+  /**
+   * @param request
+   */
+  private void finish(final VoteRequest request) {
+    // Send the vote request for each peer server
+    c.sendVoteRequestToQuorum(request);
+
+    c.getConsensusMetrics().setRaftState(PeerStatus.RAFT_STATE.CANDIDATE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Start.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Start.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Start.java
new file mode 100644
index 0000000..8422309
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/states/Start.java
@@ -0,0 +1,18 @@
+package org.apache.hadoop.hbase.consensus.raft.states;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+
+public class Start extends RaftState {
+  private static Logger LOG = LoggerFactory.getLogger(Start.class);
+
+  public Start(MutableRaftContext context) {
+    super(RaftStateType.START, context);
+  }
+
+  public void onEntry(final Event e) {
+    super.onEntry(e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/transitions/RaftTransitionType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/transitions/RaftTransitionType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/transitions/RaftTransitionType.java
new file mode 100644
index 0000000..9943dd6
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/transitions/RaftTransitionType.java
@@ -0,0 +1,44 @@
+package org.apache.hadoop.hbase.consensus.raft.transitions;
+
+import org.apache.hadoop.hbase.consensus.fsm.TransitionType;
+
+public enum RaftTransitionType implements TransitionType {
+  NONE,
+  UNCONDITIONAL,
+
+  ON_START,
+  ON_PROGRESS_TIMEOUT,
+
+  ON_VOTE_REQUEST,
+  ON_VOTE_RESPONSE,
+
+  ON_VOTE_SUCCEEDED,
+  ON_VOTE_FAILED,
+  ON_VOTE_NOT_COMPLETED,
+
+  ON_APPEND_REQUEST,
+  ON_APPEND_RESPONSE,
+
+  ON_APPEND_SUCCEEDED,
+  ON_NEED_STEPDOWN,
+  ON_APPEND_RETRY,
+  ON_APPEND_NOT_COMPLETED,
+  ON_APPEND_TIMEOUT,
+
+  ON_REPLICATED_ENTRIES,
+
+  IS_LEADER,
+  IS_CANDIDATE,
+  IS_FOLLOWER,
+
+  ON_QUORUM_MEMBERSHIP_CHANGE_REQUEST,
+  IS_QUORUM_MEMBERSHIP_CHANGE_IN_PROGRESS,
+  QUORUM_MEMBERSHIP_CHANGE_NOT_IN_PROGRESS,
+
+  ON_TRANSACTION_LOG_NOT_ACCESSIBLE,
+
+  ON_RESEED_REQUEST,
+
+  ON_HALT,
+  MAX
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/GetHydraBaseRegionInfoUtil.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/GetHydraBaseRegionInfoUtil.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/GetHydraBaseRegionInfoUtil.java
new file mode 100644
index 0000000..1dd4b03
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/GetHydraBaseRegionInfoUtil.java
@@ -0,0 +1,67 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.log4j.Level;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.List;
+
+/**
+ * Takes comma-separated list of (full/partial) region-names and output the
+ * required information about that region
+ */
+public class GetHydraBaseRegionInfoUtil {
+  private static Logger LOG = LoggerFactory.getLogger(
+    GetHydraBaseRegionInfoUtil.class);
+
+  public static void main(String[] args) throws IOException, RMapException {
+
+    // Silent the noisy o/p
+    org.apache.log4j.Logger.getLogger(
+      "org.apache.zookeeper").setLevel(Level.ERROR);
+    org.apache.log4j.Logger.getLogger(
+      "org.apache.hadoop.conf.ClientConfigurationUtil").setLevel(Level.ERROR);
+    org.apache.log4j.Logger.getLogger(
+      "org.apache.hadoop.fs").setLevel(Level.ERROR);
+    org.apache.log4j.Logger.getLogger(
+      "org.apache.hadoop.util.NativeCodeLoader").setLevel(Level.ERROR);
+
+    String[] regions = args[0].split(",");
+    Configuration conf = HBaseConfiguration.create();
+    RMapConfiguration rMapConfiguration = new RMapConfiguration(conf);
+
+    Map<String, HRegionInfo> regionInfoMap = new HashMap<>();
+    List<HRegionInfo> regionInfoList;
+
+    URI uri = rMapConfiguration.getRMapSubscription(conf);
+    if (uri != null) {
+      rMapConfiguration.readRMap(uri);
+      regionInfoList = rMapConfiguration.getRegions(uri);
+      for (HRegionInfo r : regionInfoList) {
+        regionInfoMap.put(r.getEncodedName(), r);
+      }
+    }
+
+    HRegionInfo region;
+    for (String regionName : regions) {
+      if ((region = regionInfoMap.get(regionName)) != null) {
+        LOG.info(String.format("%s:[table: %s, start_key: %s, " +
+          "end_key: %s, peers: %s]", regionName,
+          region.getTableDesc().getNameAsString(),
+          Bytes.toStringBinary(region.getStartKey()),
+          Bytes.toStringBinary(region.getEndKey()),
+          region.getQuorumInfo().getPeersAsString()));
+      } else {
+        LOG.error("No region found with encoded name " + regionName);
+      }
+    }
+  }
+}


[10/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/RaftQuorumContext.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/RaftQuorumContext.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/RaftQuorumContext.java
new file mode 100644
index 0000000..5ad454a
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/RaftQuorumContext.java
@@ -0,0 +1,1415 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.exceptions.NewLeaderException;
+import org.apache.hadoop.hbase.consensus.fsm.ConstitutentFSMService;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.fsm.FSMLargeOpsExecutorService;
+import org.apache.hadoop.hbase.consensus.fsm.FiniteStateMachine;
+import org.apache.hadoop.hbase.consensus.fsm.FiniteStateMachineService;
+import org.apache.hadoop.hbase.consensus.fsm.FiniteStateMachineServiceImpl;
+import org.apache.hadoop.hbase.consensus.fsm.State;
+import org.apache.hadoop.hbase.consensus.fsm.Util;
+import org.apache.hadoop.hbase.consensus.log.CommitLogManagerInterface;
+import org.apache.hadoop.hbase.consensus.log.LogFileInfo;
+import org.apache.hadoop.hbase.consensus.log.TransactionLogManager;
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.protocol.Payload;
+import org.apache.hadoop.hbase.consensus.raft.RaftStateMachine;
+import org.apache.hadoop.hbase.consensus.raft.events.ProgressTimeoutEvent;
+import org.apache.hadoop.hbase.consensus.raft.events.RaftEventType;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.raft.events.ReseedRequestEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.LogState;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServer;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.metrics.TimeStat;
+import org.apache.hadoop.hbase.regionserver.DataStoreState;
+import org.apache.hadoop.hbase.regionserver.RaftEventListener;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.BucketAllocator;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.InHeapArena;
+import org.apache.hadoop.hbase.util.serial.SerialExecutorService;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Class to hold RAFT Consensus related data for each replica/peer.
+ */
+public class RaftQuorumContext implements ImmutableRaftContext,
+  MutableRaftContext {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    RaftQuorumContext.class);
+
+  /** Single global timer to schedule events */
+  private final AggregateTimer aggregateTimer;
+
+  /** Event execution stream engine */
+  private final SerialExecutorService serialExecutor;
+
+  /** Lowest progress timeout value in the millis */
+  volatile long progressTimeoutForLowestRankMillis;
+
+  /** The follower/candidate progress timer */
+  private Timer progressTimer;
+
+  /** Total time in ms to retry the append entry */
+  private int appendEntriesTimeoutInMilliseconds;
+
+  /** The transaction logManager */
+  private CommitLogManagerInterface logManager;
+
+  /** Current leader */
+  private ConsensusHost leader;
+
+  /** Last host we voted for */
+  private ConsensusHost lastVotedFor;
+
+  /** Consensus Server address */
+  private final HServerAddress localConsensusServerAddress;
+
+  /** HRegionServer address */
+  protected final HServerAddress regionServerAddress;
+
+  /** Peer Address */
+  private final String myAddress;
+
+  /** Current Edit */
+  private EditId currentEdit;
+
+  /** Latest committed Edit */
+  private EditId committedEdit;
+
+  /** Previous Edit */
+  private EditId previousLogEdit;
+
+  /** Time we last received an append request */
+  private long lastAppendRequestTimeMillis = -1;
+
+  /** Session object for the current outstanding Append Session */
+  private AppendConsensusSessionInterface outstandingAppendSession;
+
+  /** Session object for the current outstanding Election Session */
+  private VoteConsensusSessionInterface outstandingElectionSession;
+
+  /** Map of the last set of AppendEntries applied to log but not committed */
+  private HashMap<Long, Payload> uncommittedEntries;
+
+  /** Heartbeat Timer */
+  private Timer heartbeatTimer;
+
+  /** The majority cnt in the quorum including the current server */
+  private int majorityCnt;
+
+  /** Manages the set of peers and their associated states */
+  protected PeerManagerInterface peerManager;
+
+  /** After successfully moving into the new quorum, close agents to
+   * old peers because we no longer need them */
+  JointConsensusPeerManager oldManager;
+
+  /** Information about the HBase Region and
+   * peers in quorum.
+   */
+  protected QuorumInfo quorumInfo;
+
+  /** Current update Membership Change Request */
+  private QuorumMembershipChangeRequest updateMembershipRequest;
+
+  /**
+   * If there is an impending quorum change, and if we are no longer going to be
+   * a part of the quorum eventually, note it down. Used when the quorum moves
+   * to the new configuration, without letting us know (so that we can
+   * shutdown).
+   */
+  private volatile boolean partOfNewQuorum = true;
+
+  /** Data Store to committ entries to */
+  protected RaftEventListener dataStoreEventListener;
+
+  /** HRegion Server configuration object */
+  private final Configuration conf;
+
+  /** Agent others should use to send requests to this replica */
+  private QuorumAgent agent;
+
+  /** State Machine */
+  private FiniteStateMachineService stateMachine;
+
+  /** Memory Allocator. Disabled by default */
+  private final Arena arena;
+
+  /** The rank is occasionally read outside the FSM so it should be volatile */
+  protected volatile int rank;
+
+  /** Consensus Metrics Object */
+  protected ConsensusMetrics metrics;
+
+  private final Random random;
+
+  /** Location of the file which store the lastVotedFor persistently on disk */
+  private Path lastVotedForFilePath;
+
+  /** Handle to lastVoteFor file */
+  private RandomAccessFile lastVotedForFile;
+
+  /** Allow balancing in case higher rank replica is available */
+  private final boolean enableStepdownOnHigherRankConf;
+
+  /** Whether the current replica is ready to take over the leadership or not */
+  private volatile boolean canTakeOver = false;
+
+  /** Amount of time to wait before we become ready to takeover */
+  private long takeOverReadyTime = 0;
+
+  private boolean useAggregateTimer;
+
+  private long lastAppendRequestReceivedTime = 0;
+
+  private ExecutorService execServiceForThriftClients;
+
+  public RaftQuorumContext(final QuorumInfo info,
+                           final Configuration config,
+                           final HServerAddress consensusServerAddress,
+                           final String metricsMBeanNamePrefix,
+                           final AggregateTimer aggregateTimer,
+                           final SerialExecutorService serialExecutor,
+                           final ExecutorService execServiceForThriftClients) {
+
+    quorumInfo = info;
+    conf = config;
+    this.metrics = new ConsensusMetrics(
+            metricsMBeanNamePrefix + info.getQuorumName(),
+            consensusServerAddress.getHostNameWithPort().replace(':', '.'));
+    this.aggregateTimer = aggregateTimer;
+    this.serialExecutor = serialExecutor;
+    random = new Random(System.currentTimeMillis());
+    localConsensusServerAddress = consensusServerAddress;
+    this.updateMembershipRequest = null;
+    regionServerAddress = RaftUtil.getHRegionServerAddress(
+      new HServerAddress(localConsensusServerAddress));
+    myAddress = localConsensusServerAddress.getHostAddressWithPort();
+
+    majorityCnt = info.getPeersWithRank().size() / 2 + 1 ;
+    initializeRank();
+
+    if (conf.getBoolean(HConstants.USE_ARENA_KEY, HConstants.USE_ARENA_DEFAULT)) {
+      int capacity = config.getInt(HConstants.ARENA_CAPACITY_KEY,
+        HConstants.ARENA_CAPACITY_DEFAULT);
+      arena = new InHeapArena(BucketAllocator.DEFAULT_BUCKETS, capacity);
+    } else {
+      arena = null;
+    }
+
+    FSMLargeOpsExecutorService.initialize(config);
+
+    this.enableStepdownOnHigherRankConf =
+        conf.getBoolean(HConstants.ENABLE_STEPDOWN_ON_HIGHER_RANK_CAUGHT_UP, true);
+
+    useAggregateTimer = conf.getBoolean(
+      HConstants.QUORUM_USE_AGGREGATE_TIMER_KEY,
+      HConstants.QUORUM_USE_AGGREGATE_TIMER_DEFAULT);
+    this.execServiceForThriftClients = execServiceForThriftClients;
+    LOG.debug("RaftQuorumContext for quorum " + getQuorumName() +
+      " initialized with rank: " + getRanking());
+    this.uncommittedEntries = new HashMap<>();
+  }
+
+  public void startStateMachines() {
+    initializeMetaData();
+    initializeAgent();
+    initializePeers();
+    initializeTimers();
+    initializeStateMachine();
+  }
+
+  @Override
+  public CommitLogManagerInterface getLogManager() {
+    return logManager;
+  }
+
+  public void setStateMachineService(
+          final FiniteStateMachineService stateMachineService) {
+    stateMachine = stateMachineService;
+  }
+
+  public State getCurrentRaftState() {
+    return stateMachine.getCurrentState();
+  }
+
+  /**
+   * Tells whether the current peer is ready to takeover or not.
+   * @return boolean ready to takeover or not
+   */
+  @Override
+  public boolean canTakeOver() {
+    if (!canTakeOver) {
+      if (System.currentTimeMillis() > takeOverReadyTime) {
+        canTakeOver = true;
+        // now that we are ready for take over let us make sure to set our
+        // timeout correctly.
+        if (this.progressTimer != null) {
+          this.progressTimer.setDelay(this.getProgressTimeoutForMeMillis(),
+            TimeUnit.MILLISECONDS);
+          LOG.info(String.format("%s Region is now ready to become the ACTIVE." +
+              " Setting timeout to %d.", quorumInfo.getQuorumName(),
+            this.getProgressTimeoutForMeMillis()));
+        }
+        return true;
+      }
+    }
+    return canTakeOver;
+  }
+
+  /**
+   * Initializes the log manager and updates the current and previous Edit info
+   */
+  public synchronized void initializeLog() {
+    if (logManager == null) {
+      logManager = new TransactionLogManager(
+        conf, quorumInfo.getQuorumName(), HConstants.UNDEFINED_TERM_INDEX);
+    }
+
+    logManager.initialize(this);
+    refreshLogState();
+  }
+
+  /**
+   * Tells the replica to use the given index as the starting index in the log.
+   *
+   * @param index bootstrap index
+   * @throws IOException
+   */
+  @Override
+  public void reseedStartIndex(long index) throws IOException {
+    logManager.fillLogGap(index);
+    refreshLogState();
+  }
+
+  /**
+   * Creates and initializes the peer state machines
+   */
+  protected void initializePeers() {
+    if (peerManager == null) {
+      peerManager = new SingleConsensusPeerManager(this);
+    }
+    peerManager.initializePeers();
+  }
+
+  /**
+   * Initialize the QuorumAgent responsible for communicating with the clients
+   */
+  public void initializeAgent() {
+    if (agent == null) {
+      agent = new QuorumAgent(this);
+    }
+  }
+
+  public void setLogManager(final CommitLogManagerInterface manager) {
+    logManager = manager;
+  }
+
+  public void registerRaftEventListener(RaftEventListener listener) {
+    this.dataStoreEventListener = listener;
+  }
+
+  @Override
+  public void resetPeers() {
+    peerManager.resetPeers();
+  }
+
+  @Override
+  public void setPeerReachable(String address) {
+    peerManager.setPeerReachable(address);
+  }
+
+  @Override
+  public HServerAddress getServerAddress() {
+    return regionServerAddress;
+  }
+
+  /**
+   * Upgrade to Joint Quorum Membership from single Quorum membership.
+   *
+   * @param newConfig the new config
+   * @throws IOException
+   */
+  @Override
+  public void updateToJointQuorumMembership(final QuorumInfo newConfig)
+    throws IOException {
+    if (peerManager.isInJointQuorumMode()) {
+      throw new IOException(String.format("Region %s is already in joint " +
+        "quorum mode.", getQuorumName()));
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(getQuorumName() + " Updating to joint quorum membership");
+    }
+
+    SingleConsensusPeerManager oldManager = (SingleConsensusPeerManager)peerManager;
+    JointConsensusPeerManager peerManager =
+      new JointConsensusPeerManager(this, newConfig);
+    peerManager.setOldPeerServers(oldManager.getPeerServers());
+    peerManager.initializePeers();
+    this.peerManager = peerManager;
+    checkIfNotPartOfNewQuorum(newConfig);
+  }
+
+  /**
+   * Move to the new quorum configuration from the current joint quorum
+   * configuration.
+   *
+   * @param newConfig The new config to move to.
+   * @throws IOException
+   */
+  @Override
+  public void updateToNewQuorumMembership(final QuorumInfo newConfig)
+    throws IOException {
+    LOG.debug(getQuorumName() + " Updating to new quorum membership");
+    if (!peerManager.isInJointQuorumMode()) {
+      throw new IOException(String.format("Cannot upgrade %s to new quorum, " +
+        "as it isn't in joint quorum mode.", getQuorumName()));
+    }
+    oldManager = (JointConsensusPeerManager)peerManager;
+
+    // Verify that all the peers in the new config are in the new peer servers list
+    for (HServerAddress peer : newConfig.getPeersWithRank().keySet()) {
+      if (peer.getHostAddressWithPort().equals(
+        this.regionServerAddress.getHostAddressWithPort())) {
+        continue;
+      }
+      if (oldManager.getNewPeerServers().get(
+        RaftUtil.getLocalConsensusAddress(peer).getHostAddressWithPort()) == null) {
+        throw new IOException("Invalid list of new peers. Cannot update the" +
+          " quorum to new config. Reason: Peer " + peer + " not present.");
+      }
+    }
+
+    // Perform the swap
+    setQuorumInfo(newConfig);
+    SingleConsensusPeerManager newManager = new SingleConsensusPeerManager(this);
+
+    newManager.setPeerServers(oldManager.getNewPeerServers());
+
+    peerManager = newManager;
+  }
+
+  /**
+   * This method checks if we are not a part of the new quorum. If we are not,
+   * then it marks the 'partOfNewQuorum' boolean to false. This helps in
+   * avoiding conditions where we miss any of the quorum membership change
+   * messages.
+   *
+   * @param newQuorumConfig
+   */
+  public void checkIfNotPartOfNewQuorum(QuorumInfo newQuorumConfig) {
+    if (!newQuorumConfig.getPeersWithRank().keySet().contains(getServerAddress())) {
+      LOG.warn(
+        String.format("%s on %s is not a member of the new quorum. " +
+          "Will be closing in future. New Quorum Members: %s. " +
+          "Setting partOfNewQuorum = false.",
+          getQuorumName(), getMyAddress(),
+          newQuorumConfig.getPeersWithRank().keySet())
+      );
+      partOfNewQuorum = false;
+    }
+  }
+
+  @Override
+  public void cleanUpJointStates() {
+    // Stop the old peers
+    oldManager.stopOldPeers();
+    oldManager = null;
+  }
+
+  @Override
+  public void handleQuorumChangeRequest(ByteBuffer buffer)
+    throws IOException {
+    List<QuorumInfo> configs = QuorumInfo.deserializeFromByteBuffer(buffer);
+    if (configs == null) {
+      return;
+    }
+    if (configs.size() == 1) {
+      updateToNewQuorumMembership(configs.get(0));
+    } else {
+      updateToJointQuorumMembership(configs.get(1));
+    }
+
+    // Close the data store if this replica is not part of the replica
+    // set anymore after this change request.
+    if (getQuorumInfo().getPeersWithRank().get(getServerAddress()) == null) {
+      LOG.info("Current host " + getServerAddress() + " is no longer a member"
+        + " of the quorum " + getQuorumName() + ". Stopping the context!");
+      stop(false);
+      if (getDataStoreEventListener() != null) {
+        getDataStoreEventListener().closeDataStore();
+      } else {
+        LOG.warn("Data store not closed as no event listener could be found");
+      }
+    }
+  }
+
+  protected void initializeTimers() {
+    progressTimeoutForLowestRankMillis = conf.getInt(
+            HConstants.PROGRESS_TIMEOUT_INTERVAL_KEY,
+            HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS);
+
+    appendEntriesTimeoutInMilliseconds = conf.getInt(
+      HConstants.APPEND_ENTRIES_TIMEOUT_KEY,
+      HConstants.DEFAULT_APPEND_ENTRIES_TIMEOUT_IN_MILLISECONDS);
+
+    // Check if we can takeover from start itself.
+    canTakeOver();
+
+    if (heartbeatTimer == null) {
+      setHeartbeatTimer(
+        RaftUtil.createTimer(
+          useAggregateTimer,
+          "heartbeat",
+          getHeartbeatTimeoutInMilliseconds(),
+          TimeUnit.MILLISECONDS,
+          new HeartbeatTimeoutCallback(this),
+          aggregateTimer
+        )
+      );
+    }
+
+    if (progressTimer == null) {
+      setProgressTimer(
+        RaftUtil.createTimer(
+          useAggregateTimer,
+          "progress",
+          getProgressTimeoutForMeMillis(),
+          TimeUnit.MILLISECONDS,
+          new ProgressTimeoutCallback(this),
+          aggregateTimer
+        )
+      );
+    }
+  }
+
+  /**
+   * Initialize the consensus with the quorum configuration, last term,
+   * last index, last committed index, vote_for and so on.
+   *
+   * Initialize the livenessTimer thread.
+   *
+   * Make a transition to Follower state.
+   */
+  public void initializeAll(long seedIndex) {
+    initializeLog();
+    startStateMachines();
+  }
+
+  public QuorumAgent getQuorumAgentInstance() {
+    return agent;
+  }
+
+  @Override
+  public Arena getArena() {
+    return arena;
+  }
+
+  @Override
+  public boolean isLeader() {
+    return (
+      (leader != null) &&
+      (lastVotedFor != null) &&
+      myAddress.equals(leader.getHostId()) &&
+      leader.equals(lastVotedFor) &&
+      (currentEdit.getTerm() == leader.getTerm()) &&
+      logManager.isAccessible()
+    );
+  }
+
+  @Override
+  public boolean isCandidate() {
+    return (
+      (lastVotedFor != null) &&
+      myAddress.equals(lastVotedFor.getHostId()) &&
+      (leader == null || !leader.getHostId().equals(lastVotedFor.getHostId())) &&
+      (currentEdit.getTerm() == lastVotedFor.getTerm()) &&
+      logManager.isAccessible()
+    );
+  }
+
+  @Override
+  public boolean isFollower() {
+    return (!isLeader() && !isCandidate() && logManager.isAccessible());
+ }
+
+  @Override
+  public String getMyAddress() {
+    return this.myAddress;
+  }
+
+  @Override
+  public EditId getCurrentEdit() {
+    return currentEdit;
+  }
+
+  @Override
+  public EditId getCommittedEdit() {
+    return committedEdit;
+  }
+
+  @Override
+  public EditId getPreviousEdit() {
+    return previousLogEdit;
+  }
+
+  @Override
+  public ConsensusHost getLeader() {
+    return leader;
+  }
+
+  @Override
+  public ConsensusHost getLastVotedFor() {
+    return lastVotedFor;
+  }
+
+  @Override
+  public ConsensusMetrics getConsensusMetrics() {
+    return metrics;
+  }
+
+  @Override
+  public Map<HServerAddress, Integer> getNewConfiguration() {
+    if (updateMembershipRequest != null) {
+      return updateMembershipRequest.getConfig().getPeersWithRank();
+    }
+    return null;
+  }
+
+  @Override
+  public AppendConsensusSessionInterface getAppendSession(final EditId id) {
+    if (outstandingAppendSession != null &&
+      outstandingAppendSession.getSessionId().equals(id)) {
+      return outstandingAppendSession;
+    }
+    return null;
+  }
+
+  @Override
+  public VoteConsensusSessionInterface getElectionSession(final EditId id) {
+    if (outstandingElectionSession != null &&
+      outstandingElectionSession.getSessionId().equals(id)) {
+      return outstandingElectionSession;
+    }
+    return null;
+  }
+
+  @Override
+  public AppendConsensusSessionInterface getOutstandingAppendSession() {
+    return outstandingAppendSession;
+  }
+
+  @Override
+  public VoteConsensusSessionInterface getOutstandingElectionSession() {
+    return outstandingElectionSession;
+  }
+
+  @Override
+  public boolean isLogAccessible() {
+    return logManager.isAccessible();
+  }
+
+  @Override
+  public void setCurrentEditId(final EditId edit) {
+    currentEdit = edit;
+  }
+
+  /**
+   * Moves the committed index to request index.
+   *
+   * It will mark all the index from the current committedIndex to the input index
+   * as committed. It also forward the request to the data store (if present)
+   *
+   * @param edit
+   */
+  @Override
+  public void advanceCommittedIndex(final EditId edit) {
+
+    // if the commit index is not initialized or if the new commit index is
+    // greater than the current one then commit all the uncommitted entries so
+    // far we have.
+    if (committedEdit == null ||
+      (edit.getIndex() >= committedEdit.getIndex() &&
+       edit.getIndex() != HConstants.UNDEFINED_TERM_INDEX)) {
+      // apply commits
+      long nextIndex = committedEdit.getIndex() + 1;
+
+      Payload payload;
+      while (nextIndex <= edit.getIndex()) {
+        payload = uncommittedEntries.get(nextIndex);
+        if (payload != null) {
+          if (QuorumInfo.isQuorumChangeRequest(payload.getEntries())) {
+            // TODO @gauravm:
+            // Make the state machine go to HALT if there is an exception.
+            // Need to figure out why exactly can't we remove the
+            // updateToNewQuorumMembership() call from the
+            // ChangeQuorumMembership state, and handle all changes here.
+            try {
+              handleQuorumChangeRequest(payload.getEntries());
+            } catch (IOException ioe) {
+              LOG.error(String.format(
+                "Could not apply the config change for quorum %s, because of: ",
+                getQuorumName()), ioe);
+            }
+          }
+
+          if (dataStoreEventListener != null) {
+            this.dataStoreEventListener.commit(nextIndex, payload);
+          } else if (payload.getResult() != null) {
+            payload.getResult().set(nextIndex);
+          }
+          uncommittedEntries.remove(nextIndex);
+        }
+        ++nextIndex;
+      }
+      committedEdit = edit;
+    }
+  }
+
+  @Override
+  public void setLeader(ConsensusHost host) {
+    clearLeader();
+    leader = host;
+  }
+
+  @Override
+  public void setPreviousEditId(final EditId id) {
+    previousLogEdit = id;
+  }
+
+  @Override
+  public ListenableFuture<Void> setVotedFor(ConsensusHost votedFor) {
+    lastVotedFor = votedFor;
+    return (ListenableFuture<Void>) FSMLargeOpsExecutorService.submitToWriteOpsThreadPool(
+      new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          persistLastVotedFor();
+          return null;
+        }
+      });
+  }
+
+  @Override
+  public void clearLeader() {
+    leader = null;
+  }
+
+  @Override
+  public void clearVotedFor() {
+    lastVotedFor = null;
+    persistLastVotedFor();
+  }
+
+  @Override
+  public void setAppendSession(AppendConsensusSessionInterface session) {
+    outstandingAppendSession = session;
+  }
+
+  @Override
+  public void setElectionSession(VoteConsensusSessionInterface session) {
+    outstandingElectionSession = session;
+  }
+
+  /**
+   * Appends the data which the given edit id to the transaction log.
+   *
+   * @param currLogId Edit Id for the data
+   * @param commitIndex current committed Index
+   * @param data the payload
+   */
+  @Override
+  public void appendToLog(final EditId currLogId, final long commitIndex, final ByteBuffer data) {
+    long now = System.currentTimeMillis();
+    int sizeToWrite = data.remaining();
+    try (TimeStat.BlockTimer timer = metrics.getLogWriteLatency().time()) {
+      logManager.append(currLogId, commitIndex, data);
+      final Payload payload =
+          getAppendSession(currLogId) == null ? new Payload(data, null) : getAppendSession(
+            currLogId).getReplicateEntriesEvent().getPayload();
+      uncommittedEntries.put(currLogId.getIndex(), payload);
+    } catch (Exception e) {
+      LOG.error(this + "Error while appending entries to logManager ", e);
+      assert !logManager.isAccessible();
+    } finally {
+      long timeTakenToAppendToLog = System.currentTimeMillis() - now;
+      if (timeTakenToAppendToLog > 1000) {
+        metrics.getLogAppendGreaterThanSecond().add(timeTakenToAppendToLog);
+        LOG.warn("Time taken to append to log: " + timeTakenToAppendToLog
+            + " is > 1000ms, and we are writing: " + sizeToWrite + " quorum: "
+            + quorumInfo.getQuorumName());
+      }
+    }
+  }
+
+  @Override
+  public int getMajorityCnt() {
+    return majorityCnt;
+  }
+
+  @Override
+  public void sendVoteRequestToQuorum(VoteRequest request) {
+    peerManager.sendVoteRequestToQuorum(request);
+  }
+
+  /**
+   * Broadcasts the AppendRequest to the peers in the Quorum
+   * @param request
+   */
+  public void sendAppendRequestToQuorum(AppendRequest request) {
+    peerManager.sendAppendRequestToQuorum(request);
+  }
+
+  /**
+   * Truncates all the entries the transaction log greater that the given
+   * valid edit.
+   *
+   * @param lastValidEntryId last valid EditId
+   * @throws IOException
+   */
+  @Override
+  public void truncateLogEntries(final EditId lastValidEntryId) throws IOException {
+    try (TimeStat.BlockTimer timer = metrics.getLogTruncateLatency().time()) {
+      if (!logManager.truncate(lastValidEntryId)) {
+        throw new IOException("Failed to truncate logs since " + lastValidEntryId + "; rank "
+            + rank);
+      }
+      long currentIndex = currentEdit.getIndex();
+      while (currentIndex > lastValidEntryId.getIndex()) {
+        uncommittedEntries.remove(currentIndex--);
+      }
+    }
+  }
+
+  @Override
+  public int getRanking() {
+    return rank;
+  }
+
+  /**
+   * Tells whether the log has the request edit or not
+   *
+   * @param id EditId to check
+   * @return
+   */
+  @Override
+  public boolean validateLogEntry(final EditId id) {
+    // If we are told to verify the start of log, or if the id is the same
+    // as the in-memory previous log edit. In that case, we don't have to touch
+    // the log manager.
+    if (id.equals(TransactionLogManager.UNDEFINED_EDIT_ID) ||
+        id.equals(previousLogEdit)) {
+      return true;
+    }
+    return logManager.isExist(id);
+  }
+
+  @Override
+  public boolean offerEvent(final Event e) {
+    return stateMachine.offer(e);
+  }
+
+  @Override
+  public Timer getHeartbeatTimer() {
+    return heartbeatTimer;
+  }
+
+  public void setHeartbeatTimer(final Timer heartbeatTimer) {
+    this.heartbeatTimer = heartbeatTimer;
+  }
+
+  @Override
+  public Timer getProgressTimer() {
+    return progressTimer;
+  }
+
+  public void setProgressTimer(final Timer progressTimer) {
+    this.progressTimer = progressTimer;
+  }
+
+  @Override
+  public String getQuorumName(){
+    return getRegionNameAsQuorumName();
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public EditId getLastLogIndex() {
+    return previousLogEdit;
+  }
+
+  /**
+   * Triggers the shutdown of the peer
+   *
+   * @param wait
+   */
+  @Override
+  public void stop(boolean wait) {
+    // Send a HALT event, causing the FSM to abort future events.
+    LOG.debug("Offering the HALT event.");
+    stateMachine.offer(new Event(RaftEventType.HALT));
+    stateMachine.shutdown();
+
+    progressTimer.shutdown();
+    heartbeatTimer.shutdown();
+    peerManager.stop();
+    logManager.stop();
+
+    // With the RaftStateMachine in HALT state the queue should be drained and
+    // shutdown should happen quickly.
+    if (wait && !Util.awaitTermination(
+        stateMachine, 3, 3, TimeUnit.SECONDS)) {
+      LOG.error("State Machine Service " + stateMachine.getName() +
+              " did not shutdown");
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "[QuorumContext: " + myAddress + ", quorum: " + getQuorumName() +
+      ", rank: " + rank + "]";
+  }
+
+  private String getRegionNameAsQuorumName(){
+    return quorumInfo.getQuorumName();
+  }
+
+  @Override
+  public QuorumInfo getQuorumInfo() {
+    return quorumInfo;
+  }
+
+  @Override
+  public ListenableFuture<?> sendAppendRequest(ReplicateEntriesEvent event) {
+    return
+      createAndSendAppendRequest(event);
+  }
+
+  @Override
+  public ListenableFuture<?> sendEmptyAppendRequest() {
+    ReplicateEntriesEvent event = new ReplicateEntriesEvent(
+      false, ByteBuffer.allocate(1));
+    return createAndSendAppendRequest(event);
+  }
+
+  /**
+   * Will stepdown from the being the leader.
+   *
+   * It will also abort any outstanding AppendConsensusSessions.
+   *
+   */
+  @Override
+  public void leaderStepDown() {
+    AppendConsensusSessionInterface session = this.getOutstandingAppendSession();
+
+    if (session != null) {
+      String leaderId = "";
+      if (getLeader() != null && !getLeader().getHostId().equals(this.getMyAddress())) {
+        leaderId = getLeader().getHostId();
+      } else {
+        // If the leader voluntarily step down, then clear the leader variable here.
+        clearLeader();
+      }
+
+      if (dataStoreEventListener != null) {
+        dataStoreEventListener.becameNonLeader();
+      }
+
+      // Set the replication event failed
+      ReplicateEntriesEvent event = session.getReplicateEntriesEvent();
+      if (event != null && !event.isHeartBeat()) {
+        event.setReplicationFailed(new NewLeaderException(leaderId));
+        uncommittedEntries.clear();
+      }
+
+      // We lost/gave up leadership, lets backs off a little bit
+      if (session.getResult().equals(SessionResult.STEP_DOWN)) {
+       long timeOut = progressTimeoutForLowestRankMillis +
+          Math.abs(random.nextLong()) % progressTimeoutForLowestRankMillis;
+
+        LOG.warn(this + " has stepped down from leader state and its last request " +
+          session.getAppendRequest() + " and back off the timeout for " + timeOut + " ms");
+
+        this.progressTimer.backoff(timeOut, TimeUnit.MILLISECONDS);
+      }
+
+      // Reset the outstanding AppendConsensusSession
+      this.setAppendSession(null);
+    }
+
+    // Should not honor quorum change request in follower mode
+    if (updateMembershipRequest != null) {
+      updateMembershipRequest.setResponse(false);
+      setUpdateMembershipRequest(null);
+    }
+
+    getConsensusMetrics().setRaftState(PeerStatus.RAFT_STATE.FOLLOWER);
+  }
+
+  /**
+   * Steps down from Candidate Role.
+   *
+   * It will abort any outstanding VoteConsensusSession.
+   *
+   */
+  @Override
+  public void candidateStepDown() {
+
+    final VoteConsensusSessionInterface session = this.getElectionSession(
+      this.getCurrentEdit());
+
+    /**
+     * Handle the scenario where you were a candidate and you didn't get the
+     * majority as your log is lagging as compared to majority, but no one else
+     * has higher term so far; We need to step down and decrease the
+     * term to the previous one, so that we can join the quorum.
+     */
+    if (session != null) {
+      this.setCurrentEditId(logManager.getLastEditID());
+
+      // We lost an election, let's back off a little bit
+      long timeout = Math.abs(random.nextLong()) % progressTimeoutForLowestRankMillis;
+      LOG.info(this + " has stepped down from candidate state and its last request " +
+        session.getRequest() + " and back off the timeout for " + timeout + " ms");
+      this.progressTimer.backoff(timeout, TimeUnit.MILLISECONDS);
+    }
+
+    /** Reset election session */
+    this.setElectionSession(null);
+
+    getConsensusMetrics().setRaftState(PeerStatus.RAFT_STATE.FOLLOWER);
+  }
+
+  /**
+   * Resends the pending AppendRequest to the peers.
+   */
+  @Override
+  public void resendOutstandingAppendRequest() {
+    // Reset the heart beat timer
+    getHeartbeatTimer().reset();
+
+    // Reset the session and re-send the AppendRequest one more time
+    outstandingAppendSession.reset();
+
+    // increment ack count for yourself
+    outstandingAppendSession.incrementAck(currentEdit, myAddress, rank, false);
+    sendAppendRequestToQuorum(outstandingAppendSession.getAppendRequest());
+  }
+
+  public PeerStatus.RAFT_STATE getPaxosState() {
+    return getConsensusMetrics().getRaftState();
+  }
+
+  public DataStoreState getStoreState() {
+    return dataStoreEventListener.getState();
+  }
+
+  public LogState getLogState() {
+    LogState logState = this.logManager.getLogState();
+    logState.setPeerState(peerManager.getState());
+
+    return logState;
+  }
+
+  public List<LogFileInfo> getCommittedLogStatus(long minIndex) {
+    return logManager.getCommittedLogStatus(minIndex);
+  }
+
+  public Map<String, PeerServer> getPeerServers() {
+    return peerManager.getPeerServers();
+  }
+
+  @Override
+  public QuorumMembershipChangeRequest getUpdateMembershipRequest() {
+    return updateMembershipRequest;
+  }
+
+  /**
+   * Get the max safest index before which all the logs can be purged.
+   *
+   * It is basically is the
+   * min(minIndexPersistedAcrossAllPeers, prevFlushMaxSeqId)
+   *
+   * @return long max safest index
+   */
+  @Override public long getPurgeIndex() {
+    long minUncommittedIndexInMemstores = Long.MAX_VALUE;
+    if (dataStoreEventListener != null) {
+      minUncommittedIndexInMemstores = dataStoreEventListener
+        .getMinUnpersistedIndex();
+      if (minUncommittedIndexInMemstores == Long.MAX_VALUE) {
+        minUncommittedIndexInMemstores = -1;
+      }
+    }
+    return Math.min(peerManager.getMinUnPersistedIndexAcrossQuorum(),
+      minUncommittedIndexInMemstores - 1);
+  }
+
+  @Override public PeerStatus getStatus() {
+    return new PeerStatus(getQuorumName(), getRanking(), getCurrentEdit().getTerm(),
+      getPaxosState(), getLogState(), "", getStoreState());
+  }
+
+  private int getHeartbeatTimeoutInMilliseconds() {
+    return Math.round(progressTimeoutForLowestRankMillis / 20);
+  }
+
+  @Override
+  public int getAppendEntriesMaxTries() {
+    return Math.max(1, appendEntriesTimeoutInMilliseconds /
+      getHeartbeatTimeoutInMilliseconds());
+  }
+
+  @Override
+  public void setUpdateMembershipRequest(
+    QuorumMembershipChangeRequest request) {
+    this.updateMembershipRequest = request;
+  }
+
+  @Override
+  public PeerManagerInterface getPeerManager() {
+    return peerManager;
+  }
+
+  @Override
+  public String getLeaderNotReadyMsg() {
+    return "The current server " + this.getMyAddress() +
+      " is not the leader for the quorum " +
+      this.getQuorumName();
+  }
+
+  public void timeoutNow() {
+    LOG.info("Timing out immediately because we are asked to");
+    this.offerEvent(new ProgressTimeoutEvent());
+  }
+
+  @Override
+  public void updatePeerAckedId(String address, EditId remoteEdit) {
+    peerManager.updatePeerAckedId(address, remoteEdit);
+  }
+
+  @Override
+  public final RaftEventListener getDataStoreEventListener() {
+    return dataStoreEventListener;
+  }
+
+  /**
+   * Returns the min across all the persisted index across peers in the quorum,
+   * and the locally flushed edits.
+   */
+  @Override
+  public long getMinUnPersistedIndexAcrossQuorum() {
+    return peerManager.getMinUnPersistedIndexAcrossQuorum();
+  }
+
+  @Override
+  public void setMinAckedIndexAcrossAllPeers(long index) {
+    peerManager.setMinAckedIndexAcrossAllPeers(index);
+  }
+
+  public void reseedIndex(long seqid)
+    throws ExecutionException, InterruptedException, IOException {
+    // If it is initialized go through the state machine to avoid race
+    // conditions, else directly update it.
+    if (stateMachine != null) {
+      ReseedRequest request = new ReseedRequest(seqid);
+      // Create replicate event and add the event to the state machine
+      ReseedRequestEvent event = new ReseedRequestEvent(request);
+
+      // Get the future object and wait for the result
+      ListenableFuture<Boolean> futureResult = event.getRequest().getResult();
+      if (!offerEvent(event)) {
+        event.getRequest().getResult().setException(new Exception("Cannot" +
+          "offer the event to the state machine."));
+      }
+      futureResult.get();
+    } else {
+      reseedStartIndex(seqid);
+    }
+  }
+
+  @Override
+  public void setQuorumInfo(final QuorumInfo quorumInfo) {
+    this.quorumInfo = quorumInfo;
+    // Update the rank if needed, making sure this replica is still part of the
+    // set before doing so.
+    int expectedRank = quorumInfo.getRank(regionServerAddress);
+    if (expectedRank != rank && expectedRank > 0) {
+      LOG.info("Updating my quorumInfo, old rank: " + getRanking() +
+        ", new rank: " + expectedRank);
+      initializeRank();
+      progressTimeoutForLowestRankMillis = conf.getInt(
+              HConstants.PROGRESS_TIMEOUT_INTERVAL_KEY,
+              HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS);
+      progressTimer.setDelay(getProgressTimeoutForMeMillis(),
+        TimeUnit.MILLISECONDS);
+    }
+  }
+
+  @Override
+  public void setLastAppendRequestReceivedTime(long timeMillis) {
+    this.lastAppendRequestTimeMillis = timeMillis;
+  }
+
+  @Override
+  public long getLastAppendRequestReceivedTime() {
+    return lastAppendRequestTimeMillis;
+  }
+
+  @Override
+  public int getNumPendingEvents() {
+    return stateMachine.getNumPendingEvents();
+  }
+
+  @Override
+  public boolean isPartOfNewQuorum() {
+    return partOfNewQuorum;
+  }
+
+  /**
+   * Returns the progress timeout for the curren replica.
+   * @return timeout values in ms.
+   */
+  public long getProgressTimeoutForMeMillis() {
+    if (this.canTakeOver) {
+      return Math.round(
+        progressTimeoutForLowestRankMillis * 1.0 / rank);
+    } else {
+      return progressTimeoutForLowestRankMillis;
+    }
+  }
+
+  public FiniteStateMachineService createFiniteStateMachineService(FiniteStateMachine fsm) {
+    return new ConstitutentFSMService(fsm, serialExecutor.createStream());
+  }
+
+  @Override
+  public ExecutorService getExecServiceForThriftClients() {
+    return execServiceForThriftClients;
+  }
+
+  private void initializeMetaData() {
+    String consensusMetadataDirectoryStr = conf.get(
+      HConstants.RAFT_METADATA_DIRECTORY_KEY,
+      HConstants.DEFAULT_RAFT_METADATA_DIRECTORY
+    );
+
+    if (!consensusMetadataDirectoryStr.endsWith(HConstants.PATH_SEPARATOR)) {
+      consensusMetadataDirectoryStr =
+        consensusMetadataDirectoryStr + HConstants.PATH_SEPARATOR;
+    }
+    consensusMetadataDirectoryStr +=
+      HConstants.PATH_SEPARATOR + getQuorumName() + HConstants.PATH_SEPARATOR;
+
+    lastVotedForFilePath = Paths.get(
+      consensusMetadataDirectoryStr + "lastVotedFor");
+
+    try {
+      // Create the consensus meta-data directory, as it might have not
+      // existed, causing this exception. If the directory already exists,
+      // there will not be any exception.
+      Files.createDirectories(Paths.get(consensusMetadataDirectoryStr));
+      lastVotedForFile =
+        new RandomAccessFile(lastVotedForFilePath.toString(), "rws");
+    } catch (FileNotFoundException fnf) {
+      // This exception should not happen if the directory is created
+      // successfully, because we use the "rw" option, which creates the file
+      // if it does not exist.
+      LOG.error("Could not open metadata files: ", fnf);
+    } catch (IOException ioe) {
+      LOG.error("Could not initialize metadata: ", ioe);
+    }
+
+    try {
+      int fileLength = (int)lastVotedForFile.length();
+      if (fileLength == 0) {
+        lastVotedFor = null;
+        return;
+      }
+
+      byte[] lastVotedForSerialized = new byte[fileLength];
+      lastVotedForFile.readFully(lastVotedForSerialized);
+      lastVotedFor = Bytes.readThriftBytes(lastVotedForSerialized,
+        ConsensusHost.class, TCompactProtocol.class);
+    } catch (Exception e) {
+      LOG.info("An error while reading the metadata file: ", e);
+    }
+  }
+
+  private void persistLastVotedFor() {
+    // This case would never happen, unless the file was not opened
+    // successfully, which is also unlikely (see the initializeMetaData()
+    // method). In this case, its better to not persist lastVotedFor, than
+    // get an NPE.
+    if (lastVotedForFile == null) {
+      throw new IllegalStateException("lastVotedFor file was not created");
+    }
+
+    if (lastVotedFor == null) {
+      // If the file exists, we will clear it
+      try {
+        lastVotedForFile.getChannel().truncate(0);
+      } catch (IOException ioe) {
+        LOG.error("Could not write to the lastVotedFor file", ioe);
+        return;
+      }
+      return;
+    }
+
+    try {
+      ByteBuffer lastVotedForSerialized =
+        Bytes.writeThriftBytes(lastVotedFor, ConsensusHost.class,
+          TCompactProtocol.class);
+      lastVotedForFile.write(Bytes.getBytes(lastVotedForSerialized));
+    } catch (Exception e) {
+      LOG.error("Could not write to the lastVotedFor file ", e);
+    }
+  }
+
+  private void initializeStateMachine() {
+    if (stateMachine == null) {
+      FiniteStateMachine fsm = new RaftStateMachine(
+        "RaftFSM-" + getQuorumName() + "-Rank:" +
+          getRanking() + "-" + getMyAddress(), this);
+      FiniteStateMachineService fsmService;
+      if (conf.getBoolean(HConstants.USE_FSMMUX_FOR_RSM,
+        HConstants.USE_FSMMUX_FOR_RSM_DEFAULT)) {
+        fsmService = createFiniteStateMachineService(fsm);
+      } else {
+        fsmService = new FiniteStateMachineServiceImpl(fsm);
+      }
+      setStateMachineService(fsmService);
+    }
+    stateMachine.offer(new Event(RaftEventType.START));
+  }
+
+  private void initializeRank() {
+    rank = quorumInfo.getRank(regionServerAddress);
+    // Whenever we set the rank, update the thread name, because debugging
+    // will be hard if there is a quorum membership change.
+    Thread.currentThread()
+      .setName("RaftFSM-" + getQuorumName() + "-Rank:" +
+        rank + "-" + getMyAddress());
+    assert rank > 0 ;
+  }
+
+  private void refreshLogState() {
+    previousLogEdit = logManager.getLastEditID();
+    currentEdit = previousLogEdit.clone();
+    committedEdit = logManager.getLastValidTransactionId();
+  }
+
+  /**
+   * Initiates the replication of the given transaction.
+   *
+   * @param event
+   * @return
+   */
+  private ListenableFuture<?> createAndSendAppendRequest(ReplicateEntriesEvent event) {
+
+    // 1. Create a new Edit Id if its not a heartbeat
+    if (!event.isHeartBeat()) {
+      // Create a new appendEditId for a non heartbeat msg
+      currentEdit = EditId.getNewAppendEditID(currentEdit);
+      // Update the previous and the current edit id
+    }
+
+    // 2. Create an append request
+    final AppendRequest request =
+      AppendRequest.createSingleAppendRequest(quorumInfo.getQuorumName(),
+        leader, currentEdit, previousLogEdit, committedEdit.getIndex(),
+        getMinUnPersistedIndexAcrossQuorum(), event.isHeartBeat(), event.getEntries());
+
+    // 3. Create a new session for this append request
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("creating an AppendConsensusSession from " + request);
+    }
+
+    final AppendConsensusSessionInterface appendSession =
+      peerManager.createAppendConsensusSession(majorityCnt, request, event,
+        metrics, rank, safeToStepDown());
+
+    outstandingAppendSession = appendSession;
+
+    // 4. Send the request to peers
+    peerManager.sendAppendRequestToQuorum(request);
+
+    // 5. If this is not a hearbeat, then execute the appendToLog in a separate
+    // thread pool.
+    if (!event.isHeartBeat()) {
+      previousLogEdit = currentEdit;
+      return FSMLargeOpsExecutorService.submitToWriteOpsThreadPool(new Callable<Void>() {
+
+        @Override
+        public Void call() throws Exception {
+          appendToLog(request.getLogId(0), committedEdit.getIndex(),
+            request.getEdit(0));
+
+          if (isLogAccessible()) {
+            // Ack yourself if append succeeded
+            appendSession.incrementAck(currentEdit, myAddress, rank, false);
+          }
+          return null;
+        }
+      });
+    } else {
+      if (isLogAccessible()) {
+        appendSession.incrementAck(currentEdit, myAddress, rank, false);
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Tells whether its safe to step down from leadership or not.
+   * @return
+   */
+  private boolean safeToStepDown() {
+    if (dataStoreEventListener == null) {
+      return true;
+    }
+
+    boolean bool = enableStepdownOnHigherRankConf &&
+      this.dataStoreEventListener.canStepDown();
+    if (!bool) {
+      LOG.debug("Not ready to step down now");
+    }
+    return bool;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/RepeatingTimer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/RepeatingTimer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/RepeatingTimer.java
new file mode 100644
index 0000000..b7bf1c5
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/RepeatingTimer.java
@@ -0,0 +1,93 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+
+/**
+ * A single threaded Timer implementation. In production we use the
+ * AggregateTimer + ConsitutentTimer combination.
+ */
+public class RepeatingTimer implements Timer, Runnable {
+  private volatile long delayMillis;
+  private volatile long backOffInterval;
+  private TimeoutEventHandler callBack;
+  private ExecutorService executor;
+
+  private volatile boolean isStopped = true;
+  private volatile boolean isTaskSubmitted = false;
+  private volatile boolean shouldReset = false;
+
+  public RepeatingTimer(final String name, final long delay, TimeUnit unit,
+                           final TimeoutEventHandler callback) {
+    this.delayMillis = TimeUnit.MILLISECONDS.convert(delay, unit);
+    this.callBack = callback;
+    this.executor = Executors.newSingleThreadExecutor(
+            new DaemonThreadFactory(name + "-timer"));
+    this.backOffInterval = 0;
+  }
+
+  @Override
+  public synchronized void start() {
+    isStopped = false;
+    if (!isTaskSubmitted) {
+      executor.submit(this);
+      isTaskSubmitted = true;
+    }
+  }
+
+  @Override
+  public synchronized void stop() {
+    if (isTaskSubmitted) {
+      isStopped = true;
+      this.notifyAll();
+    }
+  }
+
+  @Override
+  public synchronized void reset() {
+    backOffInterval = 0;
+    if (isTaskSubmitted) {
+      shouldReset = true;
+      this.notifyAll();
+    }
+  }
+
+  @Override
+  public synchronized void shutdown() {
+    executor.shutdown();
+    stop();
+  }
+
+  @Override public synchronized void backoff(long backOffTime, TimeUnit units) {
+    backOffInterval = TimeUnit.MILLISECONDS.convert(backOffTime, units);
+  }
+
+  @Override
+  public synchronized void run() {
+    try {
+      while (!isStopped) {
+        wait(delayMillis + backOffInterval);
+        if (!isStopped) {  // The timer might have been stopped.
+          if (!shouldReset) {
+            callBack.onTimeout();
+          } else {
+            shouldReset = false;
+          }
+        }
+      }
+    } catch (InterruptedException ex) {
+      // This should not happen under normal circumstances. If the waiting
+      // thread is interrupted, assume something bad happened and let the
+      // task complete.
+    } finally {
+      isTaskSubmitted = false;
+    }
+  }
+
+  public synchronized void setDelay(final long delay, TimeUnit unit) {
+    delayMillis = TimeUnit.MILLISECONDS.convert(delay, unit);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ReseedRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ReseedRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ReseedRequest.java
new file mode 100644
index 0000000..0175864
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/ReseedRequest.java
@@ -0,0 +1,27 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import com.google.common.util.concurrent.SettableFuture;
+
+public class ReseedRequest {
+
+  private SettableFuture<Boolean> result;
+
+  private long reseedIndex;
+
+  public ReseedRequest(long reseedIndex) {
+    this.reseedIndex = reseedIndex;
+    result = SettableFuture.create();
+  }
+
+  public void setResponse(Boolean response) {
+    result.set(response);
+  }
+
+  public SettableFuture<Boolean> getResult() {
+    return result;
+  }
+
+  public long getReseedIndex() {
+    return reseedIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/SessionResult.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/SessionResult.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/SessionResult.java
new file mode 100644
index 0000000..144d9b2
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/SessionResult.java
@@ -0,0 +1,9 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+public enum SessionResult {
+  MAJORITY_ACKED,
+  STEP_DOWN,
+  RETRY,
+  NOT_COMPLETED,
+  CANCELED
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/SingleConsensusPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/SingleConsensusPeerManager.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/SingleConsensusPeerManager.java
new file mode 100644
index 0000000..873676e
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/SingleConsensusPeerManager.java
@@ -0,0 +1,83 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.raft.events.ReplicateEntriesEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServer;
+import org.apache.hadoop.hbase.consensus.server.peer.events.PeerServerEventType;
+
+import java.util.*;
+
+public class SingleConsensusPeerManager extends AbstractPeerManager {
+
+  /** The map of the peer servers, excluding the current server */
+  private Map<String, PeerServer> peerServers;
+
+  public SingleConsensusPeerManager(final MutableRaftContext c) {
+    super(c);
+  }
+
+  public void initializePeers() {
+    if (peerServers == null) {
+      peerServers = super.initializePeers(c.getQuorumInfo().getPeersWithRank());
+    }
+  }
+
+  public void setPeerServers(final Map<String, PeerServer> peers) {
+    peerServers = peers;
+  }
+
+  public Map<String, PeerServer> getPeerServers() {
+    return peerServers;
+  }
+
+  public void resetPeers() {
+    super.resetPeers(peerServers);
+  }
+
+  public void setPeerReachable(String address) {
+    PeerServer server;
+    if ((server = peerServers.get(address)) != null) {
+      server.enqueueEvent(
+        new Event(PeerServerEventType.PEER_REACHABLE));
+    }
+  }
+
+  public void sendVoteRequestToQuorum(VoteRequest request) {
+    super.broadcastVoteRequest(peerServers, request);
+  }
+
+  public void sendAppendRequestToQuorum(AppendRequest request) {
+    super.broadcastAppendRequest(peerServers, request);
+  }
+
+  public void stop() {
+    super.stop(peerServers);
+  }
+
+  public String getState() {
+    return super.getState(peerServers);
+  }
+
+  @Override
+  public AppendConsensusSessionInterface createAppendConsensusSession(
+          int majorityCount, AppendRequest request, ReplicateEntriesEvent event,
+          ConsensusMetrics metrics, int rank,
+    boolean enableStepDownOnHigherRankCaughtUp) {
+    return new AppendConsensusSession(c, majorityCount, request, event,
+            metrics, rank, enableStepDownOnHigherRankCaughtUp,
+            c.getAppendEntriesMaxTries(), c.getQuorumInfo().getPeersAsString());
+  }
+
+  @Override
+  public VoteConsensusSessionInterface createVoteConsensusSession(
+          int majorityCount, VoteRequest request, ConsensusMetrics metrics) {
+    return new VoteConsensusSession(majorityCount, request, metrics, c.getQuorumInfo().getPeersAsString());
+  }
+
+  @Override public List<QuorumInfo> getConfigs() {
+    return Arrays.asList(c.getQuorumInfo());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/TimeoutEventHandler.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/TimeoutEventHandler.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/TimeoutEventHandler.java
new file mode 100644
index 0000000..85c1607
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/TimeoutEventHandler.java
@@ -0,0 +1,5 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+public interface TimeoutEventHandler {
+  void onTimeout();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/Timer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/Timer.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/Timer.java
new file mode 100644
index 0000000..76c61fc
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/Timer.java
@@ -0,0 +1,12 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import java.util.concurrent.TimeUnit;
+
+public interface Timer {
+  public void start();
+  public void stop();
+  public void reset();
+  public void shutdown();
+  public void backoff(long backOffTime, TimeUnit units);
+  public void setDelay(long delay, TimeUnit unit);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/VoteConsensusSession.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/VoteConsensusSession.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/VoteConsensusSession.java
new file mode 100644
index 0000000..1a3fed1
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/VoteConsensusSession.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.metrics.ConsensusMetrics;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public class VoteConsensusSession implements VoteConsensusSessionInterface {
+
+  private final int majorityCount;
+  private final VoteRequest request;
+  private final EditId sessionId;
+
+  private final Set<String> ackSet;
+  private final Set<String> nackSet;
+  private final Set<String> peers;
+
+  private final long sessionStartTime;
+  private final ConsensusMetrics metrics;
+
+  private SessionResult currentResult = SessionResult.NOT_COMPLETED;
+
+  public VoteConsensusSession(int majorityCount, final VoteRequest request,
+          final ConsensusMetrics metrics, final Set<String> peers) {
+    this.majorityCount = majorityCount;
+    this.request = request;
+    sessionId = new EditId(request.getTerm(), request.getPrevEditID().getIndex());
+    ackSet = new HashSet<>();
+    nackSet = new HashSet<>();
+    this.metrics = metrics;
+    this.metrics.incLeaderElectionAttempts();
+    sessionStartTime = System.currentTimeMillis();
+    this.peers = peers;
+  }
+
+  @Override
+  public boolean isComplete() {
+    return getResult().equals(SessionResult.NOT_COMPLETED) ? false : true;
+  }
+
+  @Override
+  public SessionResult getResult() {
+    generateResult();
+    return currentResult;
+  }
+
+  private void generateResult() {
+    if (!currentResult.equals(SessionResult.NOT_COMPLETED)) {
+      return;
+    }
+
+    if (ackSet.size() >= majorityCount) {
+      // Leader election succeeded
+      long elapsed = System.currentTimeMillis() - sessionStartTime;
+      metrics.getLeaderElectionLatency().add(elapsed, TimeUnit.MILLISECONDS);
+      currentResult = SessionResult.MAJORITY_ACKED;
+    } else if (nackSet.size() >= majorityCount) {
+      // Leader election failed
+      metrics.incLeaderElectionFailures();
+      currentResult = SessionResult.STEP_DOWN;
+    }
+
+  }
+
+  @Override
+  public EditId getSessionId() {
+    return sessionId;
+  }
+
+  @Override
+  public void incrementAck(final EditId id, final String address) {
+    assert this.sessionId.equals(id);
+    if (peers.contains(address)) {
+      ackSet.add(address);
+    }
+  }
+
+  @Override
+  public void incrementNack(final EditId id, final String address) {
+    assert this.sessionId.equals(id);
+    if (peers.contains(address)) {
+      nackSet.add(address);
+    }
+  }
+
+  @Override
+  public void setVoteSessionFailed(EditId id) {
+    assert this.sessionId.equals(id);
+    for (String peer : peers) {
+      incrementNack(id, peer);
+    }
+    generateResult();
+  }
+
+  public final VoteRequest getRequest() {
+    return request;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/VoteConsensusSessionInterface.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/VoteConsensusSessionInterface.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/VoteConsensusSessionInterface.java
new file mode 100644
index 0000000..3215392
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/quorum/VoteConsensusSessionInterface.java
@@ -0,0 +1,11 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+
+public interface VoteConsensusSessionInterface extends ConsensusSession {
+  void incrementAck(final EditId id, final String address);
+  void incrementNack(final EditId id, final String address);
+  VoteRequest getRequest();
+  void setVoteSessionFailed(final EditId id);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/RaftStateMachine.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/RaftStateMachine.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/RaftStateMachine.java
new file mode 100644
index 0000000..3e30dcd
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/RaftStateMachine.java
@@ -0,0 +1,231 @@
+package org.apache.hadoop.hbase.consensus.raft;
+
+import org.apache.hadoop.hbase.consensus.fsm.*;
+import org.apache.hadoop.hbase.consensus.quorum.MutableRaftContext;
+import org.apache.hadoop.hbase.consensus.raft.events.*;
+import org.apache.hadoop.hbase.consensus.raft.states.*;
+import org.apache.hadoop.hbase.consensus.raft.transitions.RaftTransitionType;
+
+public class RaftStateMachine extends FiniteStateMachine {
+  public RaftStateMachine(final String name, final MutableRaftContext c) {
+    super(name);
+
+    State start = new Start(c);
+    State halt = new Halt(c);
+
+    State becomeFollower = new BecomeFollower(c);
+    State follower = new Follower(c);
+
+    State becomeLeader = new BecomeLeader(c);
+    State leader = new Leader(c);
+
+    State candidate = new Candidate(c);
+
+    State handleVoteRequest = new HandleVoteRequest(c);
+    State handleVoteResponse = new HandleVoteResponse(c);
+    State handleAppendRequest = new HandleAppendRequest(c);
+    State handleAppendResponse = new HandleAppendResponse(c);
+    State handleQuorumMembershipChangeRequest = new HandleQuorumMembershipChangeRequest(c);
+
+    State progressTimeout = new ProgressTimeout(c);
+
+    State sendAppendRequest = new SendAppendRequest(c);
+    State reSendAppendRequest = new ReSendAppendRequest(c);
+    State sendVoteRequest = new SendVoteRequest(c);
+    State ackClient = new AckClient(c);
+    State quorumMembershipChange = new ChangeQuorumMembership(c);
+    State handleReseedRequest = new HandleReseedRequest(c);
+
+    Transition onStart =
+            new Transition(RaftTransitionType.ON_START,
+                    new OnEvent(RaftEventType.START));
+
+    Transition isLeader =
+            new Transition(RaftTransitionType.IS_LEADER,
+                    new IsLeader(c));
+
+    Transition isCandidate =
+            new Transition(RaftTransitionType.IS_CANDIDATE,
+                    new IsCandidate(c));
+
+    Transition isFollower =
+            new Transition(RaftTransitionType.IS_FOLLOWER,
+                    new IsFollower(c));
+
+    Transition unConditional =
+            new Transition(RaftTransitionType.UNCONDITIONAL,
+                    new Unconditional());
+
+    Transition onVoteRequestReceived =
+            new Transition(RaftTransitionType.ON_VOTE_REQUEST,
+                    new OnEvent(RaftEventType.VOTE_REQUEST_RECEIVED));
+
+    Transition onVoteResponseReceived =
+            new Transition(RaftTransitionType.ON_VOTE_RESPONSE,
+                    new OnEvent(RaftEventType.VOTE_RESPONSE_RECEIVED));
+
+    Transition onAppendRequestReceived =
+            new Transition(RaftTransitionType.ON_APPEND_REQUEST,
+                    new OnEvent(RaftEventType.APPEND_REQUEST_RECEIVED));
+
+    Transition onAppendResponseReceived =
+            new Transition(RaftTransitionType.ON_APPEND_RESPONSE,
+                    new OnEvent(RaftEventType.APPEND_RESPONSE_RECEIVED));
+
+    Transition onProgressTimeout =
+            new Transition(RaftTransitionType.ON_PROGRESS_TIMEOUT,
+                    new OnEvent(RaftEventType.PROGRESS_TIMEOUT));
+
+    Transition onSendAppendEntries =
+            new Transition(RaftTransitionType.ON_REPLICATED_ENTRIES,
+                    new OnEvent(RaftEventType.REPLICATE_ENTRIES));
+
+    Transition onAppendSucceeded =
+            new Transition(RaftTransitionType.ON_APPEND_SUCCEEDED,
+                    new AppendSucceeded(c));
+
+    Transition onAppendTimeout =
+            new Transition(RaftTransitionType.ON_APPEND_TIMEOUT,
+                    new AppendRequestTimeout(c));
+
+    Transition onNeedStepDown =
+            new Transition(RaftTransitionType.ON_NEED_STEPDOWN,
+                    new NeedStepDown(c));
+
+    Transition onAppendNotCompleted =
+            new Transition(RaftTransitionType.ON_APPEND_NOT_COMPLETED,
+                    new AppendNotCompleted(c));
+
+    Transition onAppendRetry =
+            new Transition(RaftTransitionType.ON_APPEND_RETRY,
+                    new AppendRetry(c));
+
+    Transition onVoteSucceeded =
+            new Transition(RaftTransitionType.ON_VOTE_SUCCEEDED,
+                    new VoteSucceeded(c));
+
+    Transition onVoteFailed =
+            new Transition(RaftTransitionType.ON_VOTE_FAILED,
+                    new VoteFailed(c));
+
+    Transition onVoteNotCompleted =
+            new Transition(RaftTransitionType.ON_VOTE_NOT_COMPLETED,
+                    new VoteNotCompleted(c));
+
+    Transition onTransactionLogNotAccessible =
+            new Transition(RaftTransitionType.ON_TRANSACTION_LOG_NOT_ACCESSIBLE,
+                    new IsTransactionLogNotAccessible(c));
+
+    Transition onHalt = new Transition(RaftTransitionType.ON_HALT,
+            new OnEvent(RaftEventType.HALT));
+    Transition onQuorumMembershipChangeRequest = new Transition(RaftTransitionType.ON_QUORUM_MEMBERSHIP_CHANGE_REQUEST,
+      new OnEvent(RaftEventType.QUORUM_MEMBERSHIP_CHANGE));
+
+    Transition quorumMembershipChangeInProgress = new Transition(RaftTransitionType.IS_QUORUM_MEMBERSHIP_CHANGE_IN_PROGRESS,
+      new QuorumMembershipChangeInProgress(c));
+    Transition noQuorumMembershipChangeNotInProgress = new Transition(RaftTransitionType.QUORUM_MEMBERSHIP_CHANGE_NOT_IN_PROGRESS,
+      new NoQuorumMembershipChangeInProgress(c));
+
+    Transition onReseedRequest = new Transition(RaftTransitionType.ON_RESEED_REQUEST,
+      new OnEvent(RaftEventType.RESEED_REQUEST_RECEIVED));
+
+    // BEGIN
+    addTransition(start, becomeFollower, onStart);
+
+    // BECOME_FOLLOWER
+    addTransition(becomeFollower, follower, unConditional);
+
+    // FOLLOWER
+    addTransition(follower, progressTimeout, onProgressTimeout);
+    addTransition(follower, handleAppendRequest, onAppendRequestReceived);
+    addTransition(follower, handleVoteRequest, onVoteRequestReceived);
+    addTransition(follower, handleQuorumMembershipChangeRequest, onQuorumMembershipChangeRequest);
+    addTransition(follower, halt, onHalt);
+    addTransition(follower, handleReseedRequest, onReseedRequest);
+
+    // CANDIDATE
+    addTransition(candidate, handleVoteRequest, onVoteRequestReceived);
+    addTransition(candidate, handleAppendRequest, onAppendRequestReceived);
+    addTransition(candidate, handleVoteResponse, onVoteResponseReceived);
+    addTransition(candidate, progressTimeout, onProgressTimeout);
+    addTransition(candidate, handleQuorumMembershipChangeRequest, onQuorumMembershipChangeRequest);
+    addTransition(candidate, halt, onHalt);
+    addTransition(candidate, handleReseedRequest, onReseedRequest);
+
+    // BECOME_LEADER
+    addTransition(becomeLeader, leader, isLeader);
+    addTransition(becomeLeader, halt, onTransactionLogNotAccessible);
+
+    // LEADER
+    addTransition(leader, handleVoteRequest, onVoteRequestReceived);
+    addTransition(leader, handleAppendRequest, onAppendRequestReceived);
+    addTransition(leader, sendAppendRequest, onSendAppendEntries);
+    addTransition(leader, becomeFollower, onAppendTimeout);
+    addTransition(leader, handleAppendResponse, onAppendResponseReceived);
+    addTransition(leader, handleQuorumMembershipChangeRequest, onQuorumMembershipChangeRequest);
+    addTransition(leader, halt, onHalt);
+    addTransition(leader, handleReseedRequest, onReseedRequest);
+
+    // SEND_APPEND_REQUEST
+    addTransition(sendAppendRequest, leader, isLeader);
+    addTransition(sendAppendRequest, follower, isFollower);
+    addTransition(sendAppendRequest, candidate, isCandidate);
+    addTransition(sendAppendRequest, halt, onTransactionLogNotAccessible);
+
+    // HANDLE_APPEND_REQUEST
+    addTransition(handleAppendRequest, leader, isLeader);
+    addTransition(handleAppendRequest, becomeFollower, isFollower);
+    addTransition(handleAppendRequest, candidate, isCandidate);
+    addTransition(handleAppendRequest, halt, onTransactionLogNotAccessible);
+
+    // HANDLE_APPEND_RESPONSE
+    addTransition(handleAppendResponse, ackClient, onAppendSucceeded);
+    addTransition(handleAppendResponse, leader, onAppendNotCompleted);
+    addTransition(handleAppendResponse, reSendAppendRequest, onAppendRetry);
+    addTransition(handleAppendResponse, becomeFollower, onNeedStepDown);
+    addTransition(handleAppendResponse, halt, onTransactionLogNotAccessible);
+
+    // SEND_VOTE_REQUEST
+    addTransition(sendVoteRequest, candidate, isCandidate);
+    addTransition(sendVoteRequest, becomeFollower, isFollower);
+    addTransition(sendVoteRequest, halt, onTransactionLogNotAccessible);
+
+    // HANDLE_VOTE_REQUEST
+    addTransition(handleVoteRequest, leader, isLeader);
+    addTransition(handleVoteRequest, candidate, isCandidate);
+    addTransition(handleVoteRequest, becomeFollower, isFollower);
+    addTransition(handleVoteRequest, halt, onTransactionLogNotAccessible);
+
+    // HANDLE_VOTE_RESPONSE
+    addTransition(handleVoteResponse, becomeLeader, onVoteSucceeded);
+    addTransition(handleVoteResponse, candidate, onVoteNotCompleted);
+    addTransition(handleVoteResponse, becomeFollower, onVoteFailed);
+
+    // HANDLE_QUORUM_MEMBERSHIP_CHANGE_REQUEST
+    addTransition(handleQuorumMembershipChangeRequest, leader, isLeader);
+    addTransition(handleQuorumMembershipChangeRequest, follower, isFollower);
+    addTransition(handleQuorumMembershipChangeRequest, candidate, isCandidate);
+
+    // ACK_CLIENT
+    addTransition(ackClient, quorumMembershipChange, quorumMembershipChangeInProgress);
+    addTransition(ackClient, leader, noQuorumMembershipChangeNotInProgress);
+
+    // QUORUM_MEMBERSHIP_CHANGE
+    addTransition(quorumMembershipChange, leader, isLeader);
+    addTransition(quorumMembershipChange, follower, isFollower);
+    addTransition(quorumMembershipChange, candidate, isCandidate);
+
+    // PROGRESS_TIMEOUT
+    addTransition(progressTimeout, sendVoteRequest, unConditional);
+
+    // RESEND_APPEND_REQUEST
+    addTransition(reSendAppendRequest, leader, unConditional);
+
+    // HANDLE_RESEED_REQUEST
+    addTransition(handleReseedRequest, leader, isLeader);
+    addTransition(handleReseedRequest, candidate, isCandidate);
+    addTransition(handleReseedRequest, follower, isFollower);
+
+    setStartState(start);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendNotCompleted.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendNotCompleted.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendNotCompleted.java
new file mode 100644
index 0000000..b76387c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendNotCompleted.java
@@ -0,0 +1,32 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSession;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+
+public class AppendNotCompleted implements Conditional {
+  private static Logger LOG = LoggerFactory.getLogger(AppendNotCompleted.class);
+
+  ImmutableRaftContext c;
+
+  public AppendNotCompleted(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    AppendConsensusSessionInterface session = c.getAppendSession(c.getCurrentEdit());
+
+    // Handle the stale request
+    if (session == null) {
+      return true;
+    }
+
+    return !session.isComplete();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRequestEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRequestEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRequestEvent.java
new file mode 100644
index 0000000..2619f05
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRequestEvent.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
+
+public class AppendRequestEvent extends Event {
+  private AppendRequest request;
+
+  public AppendRequestEvent(AppendRequest request) {
+    super(RaftEventType.APPEND_REQUEST_RECEIVED);
+    this.request = request;
+  }
+
+  public AppendRequest getRequest() {
+    return request;
+  }
+
+  public void abort(final String message) {
+    request.setError(new ThriftHBaseException(new Exception("Aborted AppendRequestEvent: "
+        + message)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRequestTimeout.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRequestTimeout.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRequestTimeout.java
new file mode 100644
index 0000000..ceda7e2
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRequestTimeout.java
@@ -0,0 +1,24 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+
+public class AppendRequestTimeout implements Conditional {
+  final ImmutableRaftContext c;
+
+  public AppendRequestTimeout(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(final Event e) {
+    AppendConsensusSessionInterface appendSession =
+            c.getOutstandingAppendSession();
+    if (appendSession != null) {
+      return appendSession.isTimeout();
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendResponseEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendResponseEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendResponseEvent.java
new file mode 100644
index 0000000..b6e8e9d
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendResponseEvent.java
@@ -0,0 +1,17 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+
+public class AppendResponseEvent extends Event {
+  private AppendResponse response;
+
+  public AppendResponseEvent(final AppendResponse response) {
+    super(RaftEventType.APPEND_RESPONSE_RECEIVED);
+    this.response = response;
+  }
+
+  public AppendResponse getResponse() {
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRetry.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRetry.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRetry.java
new file mode 100644
index 0000000..59fd043
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendRetry.java
@@ -0,0 +1,28 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSession;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.SessionResult;
+
+public class AppendRetry implements Conditional {
+  ImmutableRaftContext c;
+
+  public AppendRetry(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    AppendConsensusSessionInterface session = c.getAppendSession(c.getCurrentEdit());
+
+    // Handle the stale request
+    if (session == null) {
+      return false;
+    }
+
+    return session.getResult().equals(SessionResult.RETRY);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendSucceeded.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendSucceeded.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendSucceeded.java
new file mode 100644
index 0000000..c967cff
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/AppendSucceeded.java
@@ -0,0 +1,28 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSession;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.SessionResult;
+
+public class AppendSucceeded implements Conditional {
+  ImmutableRaftContext c;
+
+  public AppendSucceeded(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    AppendConsensusSessionInterface session = c.getAppendSession(c.getCurrentEdit());
+
+    // Handle the stale request
+    if (session == null) {
+      return false;
+    }
+
+    return session.getResult().equals(SessionResult.MAJORITY_ACKED);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsCandidate.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsCandidate.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsCandidate.java
new file mode 100644
index 0000000..40d51b9
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsCandidate.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.apache.hadoop.hbase.consensus.raft.states.Follower;
+
+public class IsCandidate implements Conditional {
+  private static Logger LOG = LoggerFactory.getLogger(IsCandidate.class);
+
+  ImmutableRaftContext c;
+
+  public IsCandidate(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    return c.isCandidate();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsFollower.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsFollower.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsFollower.java
new file mode 100644
index 0000000..52b6e99
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsFollower.java
@@ -0,0 +1,22 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+
+public class IsFollower implements Conditional {
+  private static Logger LOG = LoggerFactory.getLogger(IsFollower.class);
+
+  ImmutableRaftContext c;
+
+  public IsFollower(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    return c.isFollower();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsLeader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsLeader.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsLeader.java
new file mode 100644
index 0000000..3464976
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsLeader.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+
+public class IsLeader implements Conditional {
+  private static Logger LOG = LoggerFactory.getLogger(IsLeader.class);
+
+  ImmutableRaftContext c;
+
+  public IsLeader(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    return c.isLeader();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsTransactionLogNotAccessible.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsTransactionLogNotAccessible.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsTransactionLogNotAccessible.java
new file mode 100644
index 0000000..1a905d1
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/IsTransactionLogNotAccessible.java
@@ -0,0 +1,19 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+
+public class IsTransactionLogNotAccessible implements Conditional {
+  private ImmutableRaftContext context;
+
+  public IsTransactionLogNotAccessible(final ImmutableRaftContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    return !context.isLogAccessible();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/NeedStepDown.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/NeedStepDown.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/NeedStepDown.java
new file mode 100644
index 0000000..73d7ace
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/NeedStepDown.java
@@ -0,0 +1,29 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSession;
+import org.apache.hadoop.hbase.consensus.quorum.AppendConsensusSessionInterface;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.apache.hadoop.hbase.consensus.quorum.SessionResult;
+
+public class NeedStepDown implements Conditional {
+  ImmutableRaftContext c;
+
+  public NeedStepDown(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    AppendConsensusSessionInterface session = c.getAppendSession(c.getCurrentEdit());
+
+    // Handle the stale request
+    if (session == null) {
+      return false;
+    }
+
+    return session.getResult().equals(SessionResult.STEP_DOWN);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/NoQuorumMembershipChangeInProgress.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/NoQuorumMembershipChangeInProgress.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/NoQuorumMembershipChangeInProgress.java
new file mode 100644
index 0000000..3cf8515
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/NoQuorumMembershipChangeInProgress.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Conditional;
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+import org.apache.hadoop.hbase.consensus.quorum.ImmutableRaftContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NoQuorumMembershipChangeInProgress implements Conditional {
+  private static Logger LOG = LoggerFactory.getLogger(AppendNotCompleted.class);
+
+  ImmutableRaftContext c;
+
+  public NoQuorumMembershipChangeInProgress(final ImmutableRaftContext c) {
+    this.c = c;
+  }
+
+  @Override
+  public boolean isMet(Event e) {
+    return c.getUpdateMembershipRequest() == null;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ProgressTimeoutEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ProgressTimeoutEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ProgressTimeoutEvent.java
new file mode 100644
index 0000000..28ca257
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/raft/events/ProgressTimeoutEvent.java
@@ -0,0 +1,9 @@
+package org.apache.hadoop.hbase.consensus.raft.events;
+
+import org.apache.hadoop.hbase.consensus.fsm.Event;
+
+public class ProgressTimeoutEvent extends Event {
+  public ProgressTimeoutEvent() {
+    super(RaftEventType.PROGRESS_TIMEOUT);
+  }
+}


[05/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/BucketAllocator.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/BucketAllocator.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/BucketAllocator.java
new file mode 100644
index 0000000..62ca6cd
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/BucketAllocator.java
@@ -0,0 +1,455 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.hfile.bucket.CacheFullException;
+
+/**
+ * This class is used to allocate a block with specified size and free the block
+ * when evicting. It manages an array of buckets, each bucket is associated with
+ * a size and caches elements up to this size. For completely empty bucket, this
+ * size could be re-specified dynamically.
+ *
+ * This class is not thread safe.
+ */
+public final class BucketAllocator {
+  static final Log LOG = LogFactory.getLog(BucketAllocator.class);
+
+  public static final int[] DEFAULT_BUCKETS = { 4 * 1024 + 1024,
+    8 * 1024 + 1024 };
+
+  final private static class Bucket {
+    private final int[] bucketSizes;
+    private final long bucketCapacity;
+    private long baseOffset;
+    private int itemAllocationSize, sizeIndex;
+    private int itemCount;
+    private int freeList[];
+    private int freeCount, usedCount;
+
+    public Bucket(int[] bucketSizes, long bucketCapacity, long offset) {
+      this.bucketSizes = bucketSizes;
+      this.bucketCapacity = bucketCapacity;
+      baseOffset = offset;
+      sizeIndex = -1;
+    }
+
+    void reconfigure(int sizeIndex) {
+      this.sizeIndex = Preconditions.checkPositionIndex(sizeIndex,
+          bucketSizes.length);
+      itemAllocationSize = bucketSizes[sizeIndex];
+      itemCount = (int) ((bucketCapacity) / (long) itemAllocationSize);
+      freeCount = itemCount;
+      usedCount = 0;
+      freeList = new int[itemCount];
+      for (int i = 0; i < freeCount; ++i)
+        freeList[i] = i;
+    }
+
+    public boolean isUninstantiated() {
+      return sizeIndex == -1;
+    }
+
+    public int sizeIndex() {
+      return sizeIndex;
+    }
+
+    public int itemAllocationSize() {
+      return itemAllocationSize;
+    }
+
+    public boolean hasFreeSpace() {
+      return freeCount > 0;
+    }
+
+    public boolean isCompletelyFree() {
+      return usedCount == 0;
+    }
+
+    public int freeCount() {
+      return freeCount;
+    }
+
+    public int usedCount() {
+      return usedCount;
+    }
+
+    public int freeBytes() {
+      return freeCount * itemAllocationSize;
+    }
+
+    public int usedBytes() {
+      return usedCount * itemAllocationSize;
+    }
+
+    public long baseOffset() {
+      return baseOffset;
+    }
+
+    /**
+     * Allocate a block in this bucket, return the offset representing the
+     * position in physical space
+     * @return the offset in the IOEngine
+     */
+    public long allocate() {
+      Preconditions.checkState(freeCount > 0, "No space to allocate!");
+      Preconditions.checkState(sizeIndex != -1);
+      ++usedCount;
+      return ConditionUtil
+          .checkPositiveOffset(baseOffset + (freeList[--freeCount] * itemAllocationSize));
+    }
+
+    private void free(long offset) {
+      Preconditions.checkState(usedCount > 0);
+      Preconditions.checkState(freeCount < itemCount,
+          "duplicate free, offset: " + offset);
+      offset = ConditionUtil.checkOffset(offset - baseOffset,
+          itemCount * itemAllocationSize);
+      Preconditions.checkState(offset % itemAllocationSize == 0);
+      int item = (int) (offset / (long) itemAllocationSize);
+      Preconditions.checkState(!freeListContains(item), "Item at " + offset +
+          " already on freelist!");
+
+      --usedCount;
+      freeList[freeCount++] = item;
+    }
+
+    private boolean freeListContains(int blockNo) {
+      for (int i = 0; i < freeCount; ++i) {
+        if (freeList[i] == blockNo) return true;
+      }
+      return false;
+    }
+  }
+
+  public final class BucketSizeInfo {
+    // Free bucket means it has space to allocate a block;
+    // Completely free bucket means it has no block.
+    private List<Bucket> bucketList, freeBuckets, completelyFreeBuckets;
+    private int sizeIndex;
+
+    BucketSizeInfo(int sizeIndex) {
+      bucketList = new ArrayList<Bucket>();
+      freeBuckets = new ArrayList<Bucket>();
+      completelyFreeBuckets = new ArrayList<Bucket>();
+      this.sizeIndex = sizeIndex;
+    }
+
+    public void instantiateBucket(Bucket b) {
+      Preconditions.checkArgument(b.isUninstantiated() || b.isCompletelyFree());
+      b.reconfigure(sizeIndex);
+      bucketList.add(b);
+      freeBuckets.add(b);
+      completelyFreeBuckets.add(b);
+    }
+
+    public int sizeIndex() {
+      return sizeIndex;
+    }
+
+    /**
+     * Find a bucket to allocate a block
+     * @return the offset in the IOEngine
+     */
+    public long allocateBlock() {
+      Bucket b = null;
+      if (freeBuckets.size() > 0) // Use up an existing one first...
+        b = freeBuckets.get(freeBuckets.size() - 1);
+      if (b == null) {
+        b = grabGlobalCompletelyFreeBucket();
+        if (b != null) instantiateBucket(b);
+      }
+      if (b == null) return -1;
+      long result = b.allocate();
+      blockAllocated(b);
+      return result;
+    }
+
+    void blockAllocated(Bucket b) {
+      if (!b.isCompletelyFree()) completelyFreeBuckets.remove(b);
+      if (!b.hasFreeSpace()) freeBuckets.remove(b);
+    }
+
+    public Bucket findAndRemoveCompletelyFreeBucket() {
+      Bucket b = null;
+      Preconditions.checkState(bucketList.size() > 0);
+      if (bucketList.size() == 1) {
+        // So we never get complete starvation of a bucket for a size
+        return null;
+      }
+
+      if (completelyFreeBuckets.size() > 0) {
+        b = completelyFreeBuckets.get(0);
+        removeBucket(b);
+      }
+      return b;
+    }
+
+    private void removeBucket(Bucket b) {
+      Preconditions.checkArgument(b.isCompletelyFree());
+      bucketList.remove(b);
+      freeBuckets.remove(b);
+      completelyFreeBuckets.remove(b);
+    }
+
+    public void freeBlock(Bucket b, long offset) {
+      Preconditions.checkArgument(bucketList.contains(b));
+      // else we shouldn't have anything to free...
+      Preconditions.checkArgument(!completelyFreeBuckets.contains(b),
+          "nothing to free!");
+      b.free(offset);
+      if (!freeBuckets.contains(b)) freeBuckets.add(b);
+      if (b.isCompletelyFree()) completelyFreeBuckets.add(b);
+    }
+
+    public IndexStatistics statistics() {
+      long free = 0, used = 0;
+      for (Bucket b : bucketList) {
+        free += b.freeCount();
+        used += b.usedCount();
+      }
+      return new IndexStatistics(free, used, bucketSizes[sizeIndex]);
+    }
+  }
+
+  private final int bucketSizes[];
+
+  /**
+   * Round up the given block size to bucket size, and get the corresponding
+   * BucketSizeInfo
+   * @param blockSize
+   * @return BucketSizeInfo
+   */
+  public BucketSizeInfo roundUpToBucketSizeInfo(int blockSize) {
+    for (int i = 0; i < bucketSizes.length; ++i)
+      if (blockSize <= bucketSizes[i])
+        return bucketSizeInfos[i];
+    return null;
+  }
+
+
+  static public final int FEWEST_ITEMS_IN_BUCKET = 4;
+  // The capacity size for each bucket
+
+  private final long bucketCapacity;
+
+  private final Bucket[] buckets;
+  private final BucketSizeInfo[] bucketSizeInfos;
+  private final long totalSize;
+
+  private long usedSize = 0;
+
+  public BucketAllocator(int[] bucketSizes, long availableSpace) throws
+    BucketAllocatorException {
+    this.bucketSizes = bucketSizes;
+    int bigItemSize = bucketSizes[bucketSizes.length - 1];
+    bucketCapacity = FEWEST_ITEMS_IN_BUCKET * bigItemSize;
+    buckets = new Bucket[(int) (availableSpace / bucketCapacity)];
+    if (buckets.length < bucketSizes.length)
+      throw new BucketAllocatorException(
+          "Bucket allocator size too small - must have room for at least "
+              + bucketSizes.length + " buckets");
+    bucketSizeInfos = new BucketSizeInfo[bucketSizes.length];
+    for (int i = 0; i < bucketSizes.length; ++i) {
+      bucketSizeInfos[i] = new BucketSizeInfo(i);
+    }
+    for (int i = 0; i < buckets.length; ++i) {
+      buckets[i] = new Bucket(bucketSizes, bucketCapacity, bucketCapacity * i);
+      bucketSizeInfos[i < bucketSizes.length ? i : bucketSizes.length - 1]
+          .instantiateBucket(buckets[i]);
+    }
+    this.totalSize = ((long) buckets.length) * bucketCapacity;
+  }
+
+  public String getInfo() {
+    StringBuilder sb = new StringBuilder(1024);
+    for (int i = 0; i < buckets.length; ++i) {
+      Bucket b = buckets[i];
+      sb.append("    Bucket ").append(i).append(": ").append(b.itemAllocationSize());
+      sb.append(" freeCount=").append(b.freeCount()).append(" used=")
+          .append(b.usedCount());
+      sb.append('\n');
+    }
+    return sb.toString();
+  }
+
+  public long getUsedSize() {
+    return this.usedSize;
+  }
+
+  public long getFreeSize() {
+    return this.totalSize - getUsedSize();
+  }
+
+  public long getTotalSize() {
+    return this.totalSize;
+  }
+
+  /**
+   * Allocate a block with specified size. Return the offset
+   * @param blockSize size of block
+   * @throws BucketAllocatorException,org.apache.hadoop.hbase.io.hfile.bucket.CacheFullException
+   * @return the offset in the IOEngine
+   */
+  public synchronized long allocateBlock(int blockSize) throws
+    CacheFullException,
+      BucketAllocatorException {
+    Preconditions.checkArgument(blockSize > 0);
+    BucketSizeInfo bsi = roundUpToBucketSizeInfo(blockSize);
+    if (bsi == null) {
+      throw new BucketAllocatorException("Allocation too big size=" + blockSize);
+    }
+    long offset = bsi.allocateBlock();
+
+    // Ask caller to free up space and try again!
+    if (offset < 0)
+      throw new CacheFullException(blockSize, bsi.sizeIndex());
+    usedSize += bucketSizes[bsi.sizeIndex()];
+    return offset;
+  }
+
+  private Bucket grabGlobalCompletelyFreeBucket() {
+    for (BucketSizeInfo bsi : bucketSizeInfos) {
+      Bucket b = bsi.findAndRemoveCompletelyFreeBucket();
+      if (b != null) return b;
+    }
+    return null;
+  }
+
+  /**
+   * Free a block with the offset
+   * @param offset block's offset
+   * @return size freed
+   */
+  public synchronized int freeBlock(long offset) {
+    int bucketNo = (int) (offset / bucketCapacity);
+    Preconditions.checkPositionIndex(bucketNo, buckets.length);
+    Bucket targetBucket = buckets[bucketNo];
+    bucketSizeInfos[targetBucket.sizeIndex()].freeBlock(targetBucket, offset);
+    usedSize -= targetBucket.itemAllocationSize();
+    return targetBucket.itemAllocationSize();
+  }
+
+  public int sizeOfAllocation(long offset) {
+    int bucketNo = (int) (offset / bucketCapacity);
+    Preconditions.checkPositionIndex(bucketNo, buckets.length);
+    Bucket targetBucket = buckets[bucketNo];
+    return targetBucket.itemAllocationSize();
+  }
+
+  public static class IndexStatistics {
+    private long freeCount, usedCount, itemSize, totalCount;
+
+    public long freeCount() {
+      return freeCount;
+    }
+
+    public long usedCount() {
+      return usedCount;
+    }
+
+    public long totalCount() {
+      return totalCount;
+    }
+
+    public long freeBytes() {
+      return freeCount * itemSize;
+    }
+
+    public long usedBytes() {
+      return usedCount * itemSize;
+    }
+
+    public long totalBytes() {
+      return totalCount * itemSize;
+    }
+
+    public long itemSize() {
+      return itemSize;
+    }
+
+    public IndexStatistics(long free, long used, long itemSize) {
+      setTo(free, used, itemSize);
+    }
+
+    public IndexStatistics() {
+      setTo(-1, -1, 0);
+    }
+
+    public void setTo(long free, long used, long itemSize) {
+      this.itemSize = itemSize;
+      this.freeCount = free;
+      this.usedCount = used;
+      this.totalCount = free + used;
+    }
+  }
+
+  public void dumpToLog() {
+    logStatistics();
+    StringBuilder sb = new StringBuilder();
+    for (Bucket b : buckets) {
+      sb.append("Bucket:").append(b.baseOffset).append('\n');
+      sb.append("  Size index: " + b.sizeIndex() + "; Free:" + b.freeCount
+          + "; used:" + b.usedCount + "; freelist\n");
+      for (int i = 0; i < b.freeCount(); ++i)
+        sb.append(b.freeList[i]).append(',');
+      sb.append('\n');
+    }
+    LOG.info(sb);
+  }
+
+  public void logStatistics() {
+    IndexStatistics total = new IndexStatistics();
+    IndexStatistics[] stats = getIndexStatistics(total);
+    LOG.info("Bucket allocator statistics follow:\n");
+    LOG.info("  Free bytes=" + total.freeBytes() + "+; used bytes="
+        + total.usedBytes() + "; total bytes=" + total.totalBytes());
+    for (IndexStatistics s : stats) {
+      LOG.info("  Object size " + s.itemSize() + " used=" + s.usedCount()
+          + "; free=" + s.freeCount() + "; total=" + s.totalCount());
+    }
+  }
+
+  public IndexStatistics[] getIndexStatistics(IndexStatistics grandTotal) {
+    IndexStatistics[] stats = getIndexStatistics();
+    long totalfree = 0, totalused = 0;
+    for (IndexStatistics stat : stats) {
+      totalfree += stat.freeBytes();
+      totalused += stat.usedBytes();
+    }
+    grandTotal.setTo(totalfree, totalused, 1);
+    return stats;
+  }
+
+  public IndexStatistics[] getIndexStatistics() {
+    IndexStatistics[] stats = new IndexStatistics[bucketSizes.length];
+    for (int i = 0; i < stats.length; ++i)
+      stats[i] = bucketSizeInfos[i].statistics();
+    return stats;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/BucketAllocatorException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/BucketAllocatorException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/BucketAllocatorException.java
new file mode 100644
index 0000000..932a64f
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/BucketAllocatorException.java
@@ -0,0 +1,34 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import java.io.IOException;
+
+
+/**
+ * Thrown by {@link org.apache.hadoop.hbase.util.BucketAllocator}
+ */
+public class BucketAllocatorException extends IOException {
+  private static final long serialVersionUID = 2479119906660788096L;
+
+  BucketAllocatorException(String reason) {
+    super(reason);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Bytes.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
new file mode 100644
index 0000000..2fdbb47
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
@@ -0,0 +1,1612 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.apache.thrift.transport.TMemoryInputTransport;
+import org.apache.thrift.transport.TTransport;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * Utility class that handles byte arrays, conversions to/from other types,
+ * comparisons, hash code generation, manufacturing keys for HashMaps or
+ * HashSets, etc.
+ */
+public class Bytes {
+
+  private static final Log LOG = LogFactory.getLog(Bytes.class);
+
+  /**
+   * Size of boolean in bytes
+   */
+  public static final int SIZEOF_BOOLEAN = Byte.SIZE / Byte.SIZE;
+
+  /**
+   * Size of byte in bytes
+   */
+  public static final int SIZEOF_BYTE = SIZEOF_BOOLEAN;
+
+  /**
+   * Size of char in bytes
+   */
+  public static final int SIZEOF_CHAR = Character.SIZE / Byte.SIZE;
+
+  /**
+   * Size of double in bytes
+   */
+  public static final int SIZEOF_DOUBLE = Double.SIZE / Byte.SIZE;
+
+  /**
+   * Size of float in bytes
+   */
+  public static final int SIZEOF_FLOAT = Float.SIZE / Byte.SIZE;
+
+  /**
+   * Size of int in bytes
+   */
+  public static final int SIZEOF_INT = Integer.SIZE / Byte.SIZE;
+
+  /**
+   * Size of long in bytes
+   */
+  public static final int SIZEOF_LONG = Long.SIZE / Byte.SIZE;
+
+  /**
+   * Size of short in bytes
+   */
+  public static final int SIZEOF_SHORT = Short.SIZE / Byte.SIZE;
+
+
+  /**
+   * Estimate of size cost to pay beyond payload in jvm for instance of byte [].
+   * Estimate based on study of jhat and jprofiler numbers.
+   */
+  // JHat says BU is 56 bytes.
+  // SizeOf which uses java.lang.instrument says 24 bytes. (3 longs?)
+  public static final int ESTIMATED_HEAP_TAX = 16;
+
+  /**
+   * Byte array comparator class.
+   */
+  public static class ByteArrayComparator implements RawComparator<byte []> {
+    /**
+     * Constructor
+     */
+    public ByteArrayComparator() {
+      super();
+    }
+    @Override
+    public int compare(byte [] left, byte [] right) {
+      return compareTo(left, right);
+    }
+    @Override
+    public int compare(byte [] b1, int s1, int l1, byte [] b2, int s2, int l2) {
+      return compareTo(b1, s1, l1, b2, s2, l2);
+    }
+  }
+
+  /**
+   * Pass this to TreeMaps where byte [] are keys.
+   */
+  public static final Comparator<byte []> BYTES_COMPARATOR =
+    new ByteArrayComparator();
+
+  /**
+   * Use comparing byte arrays, byte-by-byte
+   */
+  public static final RawComparator<byte []> BYTES_RAWCOMPARATOR =
+    new ByteArrayComparator();
+
+  public static final Comparator<ByteBuffer> BYTE_BUFFER_COMPARATOR =
+      new Comparator<ByteBuffer>() {
+        @Override
+        public int compare(ByteBuffer left, ByteBuffer right) {
+          int lpos = left.position();
+          int rpos = right.position();
+          return compareTo(left.array(), left.arrayOffset() + lpos, left.limit() - lpos,
+              right.array(), right.arrayOffset() + rpos, right.limit() - rpos);
+        }
+      };
+
+  public static final ThriftCodecManager THRIFT_CODEC_MANAGER =
+    new ThriftCodecManager();
+
+  /**
+   * Read byte-array written with a WritableableUtils.vint prefix.
+   * @param in Input to read from.
+   * @return byte array read off <code>in</code>
+   * @throws IOException e
+   */
+  public static byte [] readByteArray(final DataInput in)
+  throws IOException {
+    int len = WritableUtils.readVInt(in);
+    if (len < 0) {
+      throw new NegativeArraySizeException(Integer.toString(len));
+    }
+    byte [] result = new byte[len];
+    in.readFully(result, 0, len);
+    return result;
+  }
+
+  /**
+   * Read byte-array written with a WritableableUtils.vint prefix.
+   * IOException is converted to a RuntimeException.
+   * @param in Input to read from.
+   * @return byte array read off <code>in</code>
+   */
+  public static byte [] readByteArrayThrowsRuntime(final DataInput in) {
+    try {
+      return readByteArray(in);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Write byte-array with a WritableableUtils.vint prefix.
+   * @param out output stream to be written to
+   * @param b array to write
+   * @throws IOException e
+   */
+  public static void writeByteArray(final DataOutput out, final byte [] b)
+  throws IOException {
+    if(b == null) {
+      WritableUtils.writeVInt(out, 0);
+    } else {
+      writeByteArray(out, b, 0, b.length);
+    }
+  }
+
+  /**
+   * Write byte-array to out with a vint length prefix.
+   * @param out output stream
+   * @param b array
+   * @param offset offset into array
+   * @param length length past offset
+   * @throws IOException e
+   */
+  public static void writeByteArray(final DataOutput out, final byte [] b,
+      final int offset, final int length)
+  throws IOException {
+    WritableUtils.writeVInt(out, length);
+    out.write(b, offset, length);
+  }
+
+  /**
+   * Write byte-array from src to tgt with a vint length prefix.
+   * @param tgt target array
+   * @param tgtOffset offset into target array
+   * @param src source array
+   * @param srcOffset source offset
+   * @param srcLength source length
+   * @return New offset in src array.
+   */
+  public static int writeByteArray(final byte [] tgt, final int tgtOffset,
+      final byte [] src, final int srcOffset, final int srcLength) {
+    byte [] vint = vintToBytes(srcLength);
+    System.arraycopy(vint, 0, tgt, tgtOffset, vint.length);
+    int offset = tgtOffset + vint.length;
+    System.arraycopy(src, srcOffset, tgt, offset, srcLength);
+    return offset + srcLength;
+  }
+
+  /**
+   * Put bytes at the specified byte array position.
+   * @param tgtBytes the byte array
+   * @param tgtOffset position in the array
+   * @param srcBytes array to write out
+   * @param srcOffset source offset
+   * @param srcLength source length
+   * @return incremented offset
+   */
+  public static int putBytes(byte[] tgtBytes, int tgtOffset, byte[] srcBytes,
+      int srcOffset, int srcLength) {
+    System.arraycopy(srcBytes, srcOffset, tgtBytes, tgtOffset, srcLength);
+    return tgtOffset + srcLength;
+  }
+
+  /**
+   * Write a single byte out to the specified byte array position.
+   * @param bytes the byte array
+   * @param offset position in the array
+   * @param b byte to write out
+   * @return incremented offset
+   */
+  public static int putByte(byte[] bytes, int offset, byte b) {
+    bytes[offset] = b;
+    return offset + 1;
+  }
+
+  /**
+   * Returns a new byte array, copied from the passed ByteBuffer. Starts from the array offset
+   * of the buffer and copies bytes to the limit of the buffer.
+   * @param bb A ByteBuffer
+   * @return the byte array
+   */
+  public static byte[] toBytes(ByteBuffer bb) {
+    int length = bb.limit();
+    byte [] result = new byte[length];
+    System.arraycopy(bb.array(), bb.arrayOffset(), result, 0, length);
+    return result;
+  }
+
+  /**
+   * Returns a new byte array, copied from the passed ByteBuffer. Starts from the current position
+   * in the buffer and copies all the remaining bytes to the limit of the buffer.
+   * @param bb A ByteBuffer
+   * @return the byte array
+   */
+  public static byte[] toBytesRemaining(ByteBuffer bb) {
+    int length = bb.remaining();
+    byte [] result = new byte[length];
+    System.arraycopy(bb.array(), bb.arrayOffset() + bb.position(), result, 0, length);
+    return result;
+  }
+
+  /**
+   * @param b Presumed UTF-8 encoded byte array.
+   * @return String made from <code>b</code>
+   */
+  public static String toString(final byte [] b) {
+    if (b == null) {
+      return null;
+    }
+    return toString(b, 0, b.length);
+  }
+
+  /**
+   * Joins two byte arrays together using a separator.
+   * @param b1 The first byte array.
+   * @param sep The separator to use.
+   * @param b2 The second byte array.
+   */
+  public static String toString(final byte [] b1,
+                                String sep,
+                                final byte [] b2) {
+    return toString(b1, 0, b1.length) + sep + toString(b2, 0, b2.length);
+  }
+
+  /**
+   * This method will convert utf8 encoded bytes into a string. If
+   * an UnsupportedEncodingException occurs, this method will eat it
+   * and return null instead.
+   *
+   * @param b Presumed UTF-8 encoded byte array.
+   * @param off offset into array
+   * @param len length of utf-8 sequence
+   * @return String made from <code>b</code> or null
+   */
+  public static String toString(final byte [] b, int off, int len) {
+    if (b == null) {
+      return null;
+    }
+    if (len == 0) {
+      return "";
+    }
+    try {
+      return new String(b, off, len, HConstants.UTF8_ENCODING);
+    } catch (UnsupportedEncodingException e) {
+      LOG.error("UTF-8 not supported?", e);
+      return null;
+    }
+  }
+
+  /**
+   * Write a printable representation of a byte array.
+   *
+   * @param b byte array
+   * @return string
+   * @see #toStringBinary(byte[], int, int)
+   */
+  public static String toStringBinary(final byte [] b) {
+    if (b == null)
+      return "null";
+    return toStringBinary(b, 0, b.length);
+  }
+
+  /**
+   * Converts the given byte buffer, from its array offset to its limit, to
+   * a string. The position and the mark are ignored.
+   *
+   * @param buf a byte buffer
+   * @return a string representation of the buffer's binary contents
+   */
+  public static String toStringBinary(ByteBuffer buf) {
+    if (buf == null)
+      return "null";
+    return toStringBinary(buf.array(), buf.arrayOffset(), buf.limit());
+  }
+
+  /**
+   * Similar to {@link #toStringBinary(byte[])}, but converts the portion of the buffer from the
+   * current position to the limit to string.
+   *
+   * @param buf a byte buffer
+   * @return a string representation of the buffer's remaining contents
+   */
+  public static String toStringBinaryRemaining(ByteBuffer buf) {
+    if (buf == null) {
+      return "null";
+    }
+    int offset = buf.arrayOffset();
+    int pos = buf.position();
+    return toStringBinary(buf.array(), offset + pos, buf.limit() - pos);
+  }
+
+  /**
+   * Write a printable representation of a byte array. Non-printable
+   * characters are hex escaped in the format \\x%02X, eg:
+   * \x00 \x05 etc
+   *
+   * @param b array to write out
+   * @param off offset to start at
+   * @param len length to write
+   * @return string output
+   */
+  public static String toStringBinary(final byte [] b, int off, int len) {
+    StringBuilder result = new StringBuilder();
+    try {
+      String first = new String(b, off, len, "ISO-8859-1");
+      for (int i = 0; i < first.length() ; ++i ) {
+        int ch = first.charAt(i) & 0xFF;
+        if ( (ch >= '0' && ch <= '9')
+            || (ch >= 'A' && ch <= 'Z')
+            || (ch >= 'a' && ch <= 'z')
+            || " `~!@#$%^&*()-_=+[]{}\\|;:'\",.<>/?".indexOf(ch) >= 0 ) {
+          result.append(first.charAt(i));
+        } else {
+          result.append(String.format("\\x%02X", ch));
+        }
+      }
+    } catch (UnsupportedEncodingException e) {
+      LOG.error("ISO-8859-1 not supported?", e);
+    }
+    return result.toString();
+  }
+
+  private static boolean isHexDigit(char c) {
+    return
+        (c >= 'A' && c <= 'F') ||
+        (c >= '0' && c <= '9');
+  }
+
+  /**
+   * Takes a ASCII digit in the range A-F0-9 and returns
+   * the corresponding integer/ordinal value.
+   * @param ch  The hex digit.
+   * @return The converted hex value as a byte.
+   */
+  public static byte toBinaryFromHex(byte ch) {
+    if ( ch >= 'A' && ch <= 'F' )
+      return (byte) ((byte)10 + (byte) (ch - 'A'));
+    // else
+    return (byte) (ch - '0');
+  }
+
+  public static byte [] toBytesBinary(String in) {
+    // this may be bigger than we need, but lets be safe.
+    byte [] b = new byte[in.length()];
+    int size = 0;
+    for (int i = 0; i < in.length(); ++i) {
+      char ch = in.charAt(i);
+      if (ch == '\\') {
+        // begin hex escape:
+        char next = in.charAt(i+1);
+        if (next != 'x') {
+          // invalid escape sequence, ignore this one.
+          b[size++] = (byte)ch;
+          continue;
+        }
+        // ok, take next 2 hex digits.
+        char hd1 = in.charAt(i+2);
+        char hd2 = in.charAt(i+3);
+
+        // they need to be A-F0-9:
+        if (!isHexDigit(hd1) ||
+            !isHexDigit(hd2)) {
+          // bogus escape code, ignore:
+          continue;
+        }
+        // turn hex ASCII digit -> number
+        byte d = (byte) ((toBinaryFromHex((byte)hd1) << 4) + toBinaryFromHex((byte)hd2));
+
+        b[size++] = d;
+        i += 3; // skip 3
+      } else {
+        b[size++] = (byte) ch;
+      }
+    }
+    // resize:
+    byte [] b2 = new byte[size];
+    System.arraycopy(b, 0, b2, 0, size);
+    return b2;
+  }
+
+  /**
+   * Converts a string to a UTF-8 byte array.
+   * @param s string
+   * @return the byte array
+   */
+  public static byte[] toBytes(String s) {
+    try {
+      return s.getBytes(HConstants.UTF8_ENCODING);
+    } catch (UnsupportedEncodingException e) {
+      LOG.error("UTF-8 not supported?", e);
+      return null;
+    }
+  }
+
+  /**
+   * Convert a boolean to a byte array. True becomes -1
+   * and false becomes 0.
+   *
+   * @param b value
+   * @return <code>b</code> encoded in a byte array.
+   */
+  public static byte [] toBytes(final boolean b) {
+    return new byte[] { b ? (byte) -1 : (byte) 0 };
+  }
+
+  /**
+   * Reverses {@link #toBytes(boolean)}
+   * @param b array
+   * @return True or false.
+   */
+  public static boolean toBoolean(final byte [] b) {
+    if (b.length != 1) {
+      throw new IllegalArgumentException("Array has wrong size: " + b.length);
+    }
+    return b[0] != (byte) 0;
+  }
+
+  /**
+   * Convert a long value to a byte array using big-endian.
+   *
+   * @param val value to convert
+   * @return the byte array
+   */
+  public static byte[] toBytes(long val) {
+    byte [] b = new byte[8];
+    for (int i = 7; i > 0; i--) {
+      b[i] = (byte) val;
+      val >>>= 8;
+    }
+    b[0] = (byte) val;
+    return b;
+  }
+
+  /**
+   * Converts a byte array to a long value. Reverses
+   * {@link #toBytes(long)}
+   * @param bytes array
+   * @return the long value
+   */
+  public static long toLong(byte[] bytes) {
+    return toLong(bytes, 0, SIZEOF_LONG);
+  }
+
+  /**
+   * Converts a byte array to a long value. Assumes there will be
+   * {@link #SIZEOF_LONG} bytes available.
+   *
+   * @param bytes bytes
+   * @param offset offset
+   * @return the long value
+   */
+  public static long toLong(byte[] bytes, int offset) {
+    return toLong(bytes, offset, SIZEOF_LONG);
+  }
+
+  /**
+   * Converts a byte array to a long value.
+   *
+   * @param bytes array of bytes
+   * @param offset offset into array
+   * @param length length of data (must be {@link #SIZEOF_LONG})
+   * @return the long value
+   * @throws IllegalArgumentException if length is not {@link #SIZEOF_LONG} or
+   * if there's not enough room in the array at the offset indicated.
+   */
+  public static long toLong(byte[] bytes, int offset, final int length) {
+    if (length != SIZEOF_LONG || offset + length > bytes.length) {
+      throw explainWrongLengthOrOffset(bytes, offset, length, SIZEOF_LONG);
+    }
+    long l = 0;
+    for(int i = offset; i < offset + length; i++) {
+      l <<= 8;
+      l ^= bytes[i] & 0xFF;
+    }
+    return l;
+  }
+
+  private static IllegalArgumentException
+    explainWrongLengthOrOffset(final byte[] bytes,
+                               final int offset,
+                               final int length,
+                               final int expectedLength) {
+    String reason;
+    if (length != expectedLength) {
+      reason = "Wrong length: " + length + ", expected " + expectedLength;
+    } else {
+     reason = "offset (" + offset + ") + length (" + length + ") exceed the"
+        + " capacity of the array: " + bytes.length;
+    }
+    return new IllegalArgumentException(reason);
+  }
+
+  /**
+   * Put a long value out to the specified byte array position.
+   * @param bytes the byte array
+   * @param offset position in the array
+   * @param val long to write out
+   * @return incremented offset
+   * @throws IllegalArgumentException if the byte array given doesn't have
+   * enough room at the offset specified.
+   */
+  public static int putLong(byte[] bytes, int offset, long val) {
+    if (bytes.length - offset < SIZEOF_LONG) {
+      throw new IllegalArgumentException("Not enough room to put a long at"
+          + " offset " + offset + " in a " + bytes.length + " byte array");
+    }
+    for(int i = offset + 7; i > offset; i--) {
+      bytes[i] = (byte) val;
+      val >>>= 8;
+    }
+    bytes[offset] = (byte) val;
+    return offset + SIZEOF_LONG;
+  }
+
+  /**
+   * Presumes float encoded as IEEE 754 floating-point "single format"
+   * @param bytes byte array
+   * @return Float made from passed byte array.
+   */
+  public static float toFloat(byte [] bytes) {
+    return toFloat(bytes, 0);
+  }
+
+  /**
+   * Presumes float encoded as IEEE 754 floating-point "single format"
+   * @param bytes array to convert
+   * @param offset offset into array
+   * @return Float made from passed byte array.
+   */
+  public static float toFloat(byte [] bytes, int offset) {
+    return Float.intBitsToFloat(toInt(bytes, offset, SIZEOF_INT));
+  }
+
+  /**
+   * @param bytes byte array
+   * @param offset offset to write to
+   * @param f float value
+   * @return New offset in <code>bytes</code>
+   */
+  public static int putFloat(byte [] bytes, int offset, float f) {
+    return putInt(bytes, offset, Float.floatToRawIntBits(f));
+  }
+
+  /**
+   * @param f float value
+   * @return the float represented as byte []
+   */
+  public static byte [] toBytes(final float f) {
+    // Encode it as int
+    return Bytes.toBytes(Float.floatToRawIntBits(f));
+  }
+
+  /**
+   * @param bytes byte array
+   * @return Return double made from passed bytes.
+   */
+  public static double toDouble(final byte [] bytes) {
+    return toDouble(bytes, 0);
+  }
+
+  /**
+   * @param bytes byte array
+   * @param offset offset where double is
+   * @return Return double made from passed bytes.
+   */
+  public static double toDouble(final byte [] bytes, final int offset) {
+    return Double.longBitsToDouble(toLong(bytes, offset, SIZEOF_LONG));
+  }
+
+  /**
+   * @param bytes byte array
+   * @param offset offset to write to
+   * @param d value
+   * @return New offset into array <code>bytes</code>
+   */
+  public static int putDouble(byte [] bytes, int offset, double d) {
+    return putLong(bytes, offset, Double.doubleToLongBits(d));
+  }
+
+  /**
+   * Serialize a double as the IEEE 754 double format output. The resultant
+   * array will be 8 bytes long.
+   *
+   * @param d value
+   * @return the double represented as byte []
+   */
+  public static byte [] toBytes(final double d) {
+    // Encode it as a long
+    return Bytes.toBytes(Double.doubleToRawLongBits(d));
+  }
+
+  /**
+   * Convert an int value to a byte array
+   * @param val value
+   * @return the byte array
+   */
+  public static byte[] toBytes(int val) {
+    byte [] b = new byte[4];
+    for(int i = 3; i > 0; i--) {
+      b[i] = (byte) val;
+      val >>>= 8;
+    }
+    b[0] = (byte) val;
+    return b;
+  }
+
+  /**
+   * Converts a byte array to an int value
+   * @param bytes byte array
+   * @return the int value
+   */
+  public static int toInt(byte[] bytes) {
+    return toInt(bytes, 0, SIZEOF_INT);
+  }
+
+  /**
+   * Converts a byte array to an int value
+   * @param bytes byte array
+   * @param offset offset into array
+   * @return the int value
+   */
+  public static int toInt(byte[] bytes, int offset) {
+    return toInt(bytes, offset, SIZEOF_INT);
+  }
+
+  /**
+   * Converts a byte array to an int value
+   * @param bytes byte array
+   * @param offset offset into array
+   * @param length length of int (has to be {@link #SIZEOF_INT})
+   * @return the int value
+   * @throws IllegalArgumentException if length is not {@link #SIZEOF_INT} or
+   * if there's not enough room in the array at the offset indicated.
+   */
+  public static int toInt(byte[] bytes, int offset, final int length) {
+    if (length != SIZEOF_INT || offset + length > bytes.length) {
+      throw explainWrongLengthOrOffset(bytes, offset, length, SIZEOF_INT);
+    }
+    int n = 0;
+    for(int i = offset; i < (offset + length); i++) {
+      n <<= 8;
+      n ^= bytes[i] & 0xFF;
+    }
+    return n;
+  }
+
+  /**
+   * Put an int value out to the specified byte array position.
+   * @param bytes the byte array
+   * @param offset position in the array
+   * @param val int to write out
+   * @return incremented offset
+   * @throws IllegalArgumentException if the byte array given doesn't have
+   * enough room at the offset specified.
+   */
+  public static int putInt(byte[] bytes, int offset, int val) {
+    if (bytes.length - offset < SIZEOF_INT) {
+      throw new IllegalArgumentException("Not enough room to put an int at"
+          + " offset " + offset + " in a " + bytes.length + " byte array");
+    }
+    for(int i= offset + 3; i > offset; i--) {
+      bytes[i] = (byte) val;
+      val >>>= 8;
+    }
+    bytes[offset] = (byte) val;
+    return offset + SIZEOF_INT;
+  }
+
+  /**
+   * Convert a short value to a byte array of {@link #SIZEOF_SHORT} bytes long.
+   * @param val value
+   * @return the byte array
+   */
+  public static byte[] toBytes(short val) {
+    byte[] b = new byte[SIZEOF_SHORT];
+    b[1] = (byte) val;
+    val >>= 8;
+    b[0] = (byte) val;
+    return b;
+  }
+
+  /**
+   * Converts a byte array to a short value
+   * @param bytes byte array
+   * @return the short value
+   */
+  public static short toShort(byte[] bytes) {
+    return toShort(bytes, 0, SIZEOF_SHORT);
+  }
+
+  /**
+   * Converts a byte array to a short value
+   * @param bytes byte array
+   * @param offset offset into array
+   * @return the short value
+   */
+  public static short toShort(byte[] bytes, int offset) {
+    return toShort(bytes, offset, SIZEOF_SHORT);
+  }
+
+  /**
+   * Converts a byte array to a short value
+   * @param bytes byte array
+   * @param offset offset into array
+   * @param length length, has to be {@link #SIZEOF_SHORT}
+   * @return the short value
+   * @throws IllegalArgumentException if length is not {@link #SIZEOF_SHORT}
+   * or if there's not enough room in the array at the offset indicated.
+   */
+  public static short toShort(byte[] bytes, int offset, final int length) {
+    if (length != SIZEOF_SHORT || offset + length > bytes.length) {
+      throw explainWrongLengthOrOffset(bytes, offset, length, SIZEOF_SHORT);
+    }
+    short n = 0;
+    n ^= bytes[offset] & 0xFF;
+    n <<= 8;
+    n ^= bytes[offset+1] & 0xFF;
+    return n;
+  }
+
+  public static byte[] getBytes(ByteBuffer buf) {
+    if (buf == null) {
+      return HConstants.EMPTY_BYTE_ARRAY;
+    }
+
+    if (buf.arrayOffset() == 0 && buf.position() == 0) {
+      byte[] arr = buf.array();
+      if (buf.limit() == arr.length) {
+        // We already have the exact array we need, just return it.
+        return arr;
+      }
+    }
+
+    int savedPos = buf.position();
+    byte [] newBytes = new byte[buf.remaining()];
+    buf.get(newBytes);
+    buf.position(savedPos);
+    return newBytes;
+  }
+
+  /**
+   * Put a short value out to the specified byte array position.
+   * @param bytes the byte array
+   * @param offset position in the array
+   * @param val short to write out
+   * @return incremented offset
+   * @throws IllegalArgumentException if the byte array given doesn't have
+   * enough room at the offset specified.
+   */
+  public static int putShort(byte[] bytes, int offset, short val) {
+    if (bytes.length - offset < SIZEOF_SHORT) {
+      throw new IllegalArgumentException("Not enough room to put a short at"
+          + " offset " + offset + " in a " + bytes.length + " byte array");
+    }
+    bytes[offset+1] = (byte) val;
+    val >>= 8;
+    bytes[offset] = (byte) val;
+    return offset + SIZEOF_SHORT;
+  }
+
+  /**
+   * @param vint Integer to make a vint of.
+   * @return Vint as bytes array.
+   */
+  public static byte [] vintToBytes(final long vint) {
+    long i = vint;
+    int size = WritableUtils.getVIntSize(i);
+    byte [] result = new byte[size];
+    int offset = 0;
+    if (i >= -112 && i <= 127) {
+      result[offset] = (byte) i;
+      return result;
+    }
+
+    int len = -112;
+    if (i < 0) {
+      i ^= -1L; // take one's complement'
+      len = -120;
+    }
+
+    long tmp = i;
+    while (tmp != 0) {
+      tmp = tmp >> 8;
+      len--;
+    }
+
+    result[offset++] = (byte) len;
+
+    len = (len < -120) ? -(len + 120) : -(len + 112);
+
+    for (int idx = len; idx != 0; idx--) {
+      int shiftbits = (idx - 1) * 8;
+      long mask = 0xFFL << shiftbits;
+      result[offset++] = (byte)((i & mask) >> shiftbits);
+    }
+    return result;
+  }
+
+  /**
+   * @param buffer buffer to convert
+   * @return vint bytes as an integer.
+   */
+  public static long bytesToVint(final byte [] buffer) {
+    int offset = 0;
+    byte firstByte = buffer[offset++];
+    int len = WritableUtils.decodeVIntSize(firstByte);
+    if (len == 1) {
+      return firstByte;
+    }
+    long i = 0;
+    for (int idx = 0; idx < len-1; idx++) {
+      byte b = buffer[offset++];
+      i = i << 8;
+      i = i | (b & 0xFF);
+    }
+    return (WritableUtils.isNegativeVInt(firstByte) ? ~i : i);
+  }
+
+  /**
+   * Reads a zero-compressed encoded long from input stream and returns it.
+   * @param buffer Binary array
+   * @param offset Offset into array at which vint begins.
+   * @throws java.io.IOException e
+   * @return deserialized long from stream.
+   */
+  public static long readVLong(final byte [] buffer, final int offset)
+  throws IOException {
+    byte firstByte = buffer[offset];
+    int len = WritableUtils.decodeVIntSize(firstByte);
+    if (len == 1) {
+      return firstByte;
+    }
+    long i = 0;
+    for (int idx = 0; idx < len-1; idx++) {
+      byte b = buffer[offset + 1 + idx];
+      i = i << 8;
+      i = i | (b & 0xFF);
+    }
+    return (WritableUtils.isNegativeVInt(firstByte) ? ~i : i);
+  }
+
+  /**
+   * @param left left operand
+   * @param right right operand
+   * @return 0 if equal, < 0 if left is less than right, etc.
+   */
+  public static int compareTo(final byte [] left, final byte [] right) {
+    return compareTo(left, 0, left.length, right, 0, right.length);
+  }
+
+  /**
+   * Lexographically compare two arrays.
+   *
+   * @param buffer1 left operand
+   * @param buffer2 right operand
+   * @param offset1 Where to start comparing in the left buffer
+   * @param offset2 Where to start comparing in the right buffer
+   * @param length1 How much to compare from the left buffer
+   * @param length2 How much to compare from the right buffer
+   * @return 0 if equal, < 0 if left is less than right, etc.
+   */
+  public static int compareTo(byte[] buffer1, int offset1, int length1,
+      byte[] buffer2, int offset2, int length2) {
+    // Bring WritableComparator code local
+    int end1 = offset1 + length1;
+    int end2 = offset2 + length2;
+    for (int i = offset1, j = offset2; i < end1 && j < end2; i++, j++) {
+      int a = (buffer1[i] & 0xff);
+      int b = (buffer2[j] & 0xff);
+      if (a != b) {
+        return a - b;
+      }
+    }
+    return length1 - length2;
+  }
+
+  /**
+   * @param left left operand
+   * @param right right operand
+   * @return True if equal
+   */
+  public static boolean equals(final byte [] left, final byte [] right) {
+    // Could use Arrays.equals?
+    //noinspection SimplifiableConditionalExpression
+    if (left == null && right == null) {
+      return true;
+    }
+    return (left == null || right == null || (left.length != right.length)
+            ? false : compareTo(left, right) == 0);
+  }
+
+  public static boolean equals(final byte[] left, int leftOffset, int leftLength,
+      final byte[] right, int rightOffset, int rightLength) {
+    if (left == null && right == null) {
+      return true;
+    }
+    return (left == null || right == null || (leftLength != rightLength) ? false : compareTo(left,
+        leftOffset, leftLength, right, rightOffset, rightLength) == 0);
+  }
+
+  /**
+   * Return true if the byte array on the right is a prefix of the byte
+   * array on the left.
+   */
+  public static boolean startsWith(byte[] bytes, byte[] prefix) {
+    return bytes != null && prefix != null &&
+      bytes.length >= prefix.length &&
+      compareTo(bytes, 0, prefix.length, prefix, 0, prefix.length) == 0;
+  }
+
+  /**
+   * @param b bytes to hash
+   * @return Runs {@link WritableComparator#hashBytes(byte[], int)} on the
+   * passed in array.  This method is what {@link org.apache.hadoop.io.Text} and
+   * {@link ImmutableBytesWritable} use calculating hash code.
+   */
+  public static int hashCode(final byte [] b) {
+    return hashCode(b, b.length);
+  }
+
+  /**
+   * @param b value
+   * @param length length of the value
+   * @return Runs {@link WritableComparator#hashBytes(byte[], int)} on the
+   * passed in array.  This method is what {@link org.apache.hadoop.io.Text} and
+   * {@link ImmutableBytesWritable} use calculating hash code.
+   */
+  public static int hashCode(final byte [] b, final int length) {
+    return WritableComparator.hashBytes(b, length);
+  }
+
+  /**
+   * @param b bytes to hash
+   * @return A hash of <code>b</code> as an Integer that can be used as key in
+   * Maps.
+   */
+  public static Integer mapKey(final byte [] b) {
+    return hashCode(b);
+  }
+
+  /**
+   * @param b bytes to hash
+   * @param length length to hash
+   * @return A hash of <code>b</code> as an Integer that can be used as key in
+   * Maps.
+   */
+  public static Integer mapKey(final byte [] b, final int length) {
+    return hashCode(b, length);
+  }
+
+  /**
+   * @param a lower half
+   * @param b upper half
+   * @return New array that has a in lower half and b in upper half.
+   */
+  public static byte [] add(final byte [] a, final byte [] b) {
+    return add(a, b, HConstants.EMPTY_BYTE_ARRAY);
+  }
+
+  /**
+   * @param a first third
+   * @param b second third
+   * @param c third third
+   * @return New array made from a, b and c
+   */
+  public static byte [] add(final byte [] a, final byte [] b, final byte [] c) {
+    byte [] result = new byte[a.length + b.length + c.length];
+    System.arraycopy(a, 0, result, 0, a.length);
+    System.arraycopy(b, 0, result, a.length, b.length);
+    System.arraycopy(c, 0, result, a.length + b.length, c.length);
+    return result;
+  }
+
+  /**
+   * @param a first third
+   * @param b second third
+   * @param c third third
+   * @return New array made from a, b and c
+   */
+  public static byte [] add(final byte [] a, int aOffset, int aLength,
+      final byte [] b, int bOffset, int bLength,
+      final byte [] c, int cOffset, int cLength) {
+    byte [] result = new byte[aLength + bLength + cLength];
+    System.arraycopy(a, aOffset, result, 0, aLength);
+    System.arraycopy(b, bOffset, result, aLength, bLength);
+    System.arraycopy(c, cOffset, result, aLength + bLength, cLength);
+    return result;
+  }
+
+  /**
+   * @param a array
+   * @param length amount of bytes to grab
+   * @return First <code>length</code> bytes from <code>a</code>
+   */
+  public static byte [] head(final byte [] a, final int length) {
+    if (a.length < length) {
+      return null;
+    }
+    byte [] result = new byte[length];
+    System.arraycopy(a, 0, result, 0, length);
+    return result;
+  }
+
+  /**
+   * @param a array
+   * @param length amount of bytes to snarf
+   * @return Last <code>length</code> bytes from <code>a</code>
+   */
+  public static byte [] tail(final byte [] a, final int length) {
+    if (a.length < length) {
+      return null;
+    }
+    byte [] result = new byte[length];
+    System.arraycopy(a, a.length - length, result, 0, length);
+    return result;
+  }
+
+  /**
+   * @param a array
+   * @param length new array size
+   * @return Value in <code>a</code> plus <code>length</code> prepended 0 bytes
+   */
+  public static byte [] padHead(final byte [] a, final int length) {
+    byte [] padding = new byte[length];
+    for (int i = 0; i < length; i++) {
+      padding[i] = 0;
+    }
+    return add(padding,a);
+  }
+
+  /**
+   * @param a array
+   * @param length new array size
+   * @return Value in <code>a</code> plus <code>length</code> appended 0 bytes
+   */
+  public static byte [] padTail(final byte [] a, final int length) {
+    return appendToTail(a, length, (byte)0);
+  }
+
+  /**
+   * Appends length bytes to the end of the array and returns the new array
+   * Fills byte b in the newly allocated space in the byte[].
+   * @param a array
+   * @param length new array size
+   * @param b byte to write to the tail.
+   * @return Value in <code>a</code> plus <code>length</code> appended 0 bytes
+   */
+  public static byte [] appendToTail(final byte [] a, final int length, byte b)
+  {
+    byte [] padding = new byte[length];
+    for (int i = 0; i < length; i++) {
+      padding[i] = b;
+    }
+    return add(a,padding);
+  }
+
+  /**
+   * Split passed range.  Expensive operation relatively.  Uses BigInteger math.
+   * Useful splitting ranges for MapReduce jobs.
+   * @param a Beginning of range
+   * @param b End of range
+   * @param num Number of times to split range.  Pass 1 if you want to split
+   * the range in two; i.e. one split.
+   * @return Array of dividing values
+   */
+  public static byte [][] split(final byte [] a, final byte [] b, final int num) {
+    return split(a, b, false, num);
+  }
+
+  /**
+   * Split passed range.  Expensive operation relatively.  Uses BigInteger math.
+   * Useful splitting ranges for MapReduce jobs.
+   * @param a Beginning of range
+   * @param b End of range
+   * @param inclusive Whether the end of range is prefix-inclusive or is
+   * considered an exclusive boundary.  Automatic splits are generally exclusive
+   * and manual splits with an explicit range utilize an inclusive end of range.
+   * @param num Number of times to split range.  Pass 1 if you want to split
+   * the range in two; i.e. one split.
+   * @return Array of dividing values
+   */
+  public static byte[][] split(final byte[] a, final byte[] b,
+      boolean inclusive, final int num) {
+    byte[][] ret = new byte[num + 2][];
+    int i = 0;
+    Iterable<byte[]> iter = iterateOnSplits(a, b, inclusive, num);
+    if (iter == null)
+      return null;
+    for (byte[] elem : iter) {
+      ret[i++] = elem;
+    }
+    return ret;
+  }
+
+  /**
+   * Iterate over keys within the passed range, splitting at an [a,b) boundary.
+   */
+  public static Iterable<byte[]> iterateOnSplits(final byte[] a,
+      final byte[] b, final int num)
+  {
+    return iterateOnSplits(a, b, false, num);
+  }
+
+  /**
+   * Iterate over keys within the passed range.
+   */
+  public static Iterable<byte[]> iterateOnSplits(
+      final byte[] a, final byte[]b, boolean inclusive, final int num)
+  {
+    byte [] aPadded;
+    byte [] bPadded;
+    if (a.length < b.length) {
+      aPadded = padTail(a, b.length - a.length);
+      bPadded = b;
+    } else if (b.length < a.length) {
+      aPadded = a;
+      bPadded = padTail(b, a.length - b.length);
+    } else {
+      aPadded = a;
+      bPadded = b;
+    }
+    if (compareTo(aPadded,bPadded) >= 0) {
+      throw new IllegalArgumentException("b <= a");
+    }
+    if (num < 0) {
+      throw new IllegalArgumentException("num cannot be < 0");
+    }
+    byte [] prependHeader = {1, 0};
+    final BigInteger startBI = new BigInteger(add(prependHeader, aPadded));
+    final BigInteger stopBI = new BigInteger(add(prependHeader, bPadded));
+    BigInteger diffBI = stopBI.subtract(startBI);
+    if (inclusive) {
+      diffBI = diffBI.add(BigInteger.ONE);
+    }
+    final BigInteger splitsBI = BigInteger.valueOf(num + 1);
+    if(diffBI.compareTo(splitsBI) < 0) {
+      return null;
+    }
+    final BigInteger intervalBI;
+    try {
+      intervalBI = diffBI.divide(splitsBI);
+    } catch(Exception e) {
+      LOG.error("Exception caught during division", e);
+      return null;
+    }
+
+    final Iterator<byte[]> iterator = new Iterator<byte[]>() {
+      private int i = -1;
+      private BigInteger curBI = startBI;
+
+      @Override
+      public boolean hasNext() {
+        return i < num+1;
+      }
+
+      @Override
+      public byte[] next() {
+        i++;
+        if (i == 0) return a;
+        if (i == num + 1) return b;
+
+        curBI = curBI.add(intervalBI);
+        byte [] padded = curBI.toByteArray();
+        if (padded[1] == 0)
+          padded = tail(padded, padded.length - 2);
+        else
+          padded = tail(padded, padded.length - 1);
+        return padded;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+
+    };
+
+    return new Iterable<byte[]>() {
+      @Override
+      public Iterator<byte[]> iterator() {
+        return iterator;
+      }
+    };
+  }
+
+  /**
+   * Calculate the next <code>num</code> elements in arithemetic
+   * progression sequence.
+   *
+   * @param a First element.
+   * @param b Second element.
+   * @param num Number of next elements to find.
+   * @return <code>num</code> byte arrays each having the same interval
+   *         from <code>a</code> to <code>b</code>, starting from b. In
+   *         other words, it returns an array consists of b+(b-a)*(i+1),
+   *         where i is the index of the resulting array of size <code>
+   *         num</code>. Uses BigInteger math.
+   */
+  public static byte[][] arithmeticProgSeq(byte[] a, byte[] b, int num) {
+    byte [][] result = new byte[num][];
+    byte [] aPadded;
+    byte [] bPadded;
+    if (a.length < b.length) {
+      aPadded = padTail(a, b.length - a.length);
+      bPadded = b;
+    } else if (b.length < a.length) {
+      aPadded = a;
+      bPadded = padTail(b, a.length - b.length);
+    } else {
+      aPadded = a;
+      bPadded = b;
+    }
+    if (num < 0) {
+      throw new IllegalArgumentException("num cannot be < 0");
+    }
+    byte [] prependHeader = {1, 0};
+    BigInteger startBI = new BigInteger(add(prependHeader, aPadded));
+    BigInteger stopBI = new BigInteger(add(prependHeader, bPadded));
+    BigInteger diffBI = stopBI.subtract(startBI);
+    BigInteger curBI = stopBI;
+    for (int i = 0; i < num; i++) {
+      curBI = curBI.add(diffBI);
+      byte [] padded = curBI.toByteArray();
+      if (padded[1] == 0)
+        padded = tail(padded, padded.length - 2);
+      else
+        padded = tail(padded, padded.length - 1);
+      result[i] = padded;
+    }
+    return result;
+  }
+
+  /**
+   * @param t operands
+   * @return Array of byte arrays made from passed array of Text
+   */
+  public static byte [][] toByteArrays(final String [] t) {
+    byte [][] result = new byte[t.length][];
+    for (int i = 0; i < t.length; i++) {
+      result[i] = Bytes.toBytes(t[i]);
+    }
+    return result;
+  }
+
+  /**
+   * @param column operand
+   * @return A byte array of a byte array where first and only entry is
+   * <code>column</code>
+   */
+  public static byte [][] toByteArrays(final String column) {
+    return toByteArrays(toBytes(column));
+  }
+
+  /**
+   * @param column operand
+   * @return A byte array of a byte array where first and only entry is
+   * <code>column</code>
+   */
+  public static byte [][] toByteArrays(final byte [] column) {
+    byte [][] result = new byte[1][];
+    result[0] = column;
+    return result;
+  }
+
+  /**
+   * Binary search for keys in indexes.
+   *
+   * @param arr array of byte arrays to search for
+   * @param key the key you want to find
+   * @param offset the offset in the key you want to find
+   * @param length the length of the key
+   * @param comparator a comparator to compare.
+   * @return zero-based index of the key, if the key is present in the array.
+   *         Otherwise, a value -(i + 1) such that the key is between arr[i -
+   *         1] and arr[i] non-inclusively, where i is in [0, i], if we define
+   *         arr[-1] = -Inf and arr[N] = Inf for an N-element array. The above
+   *         means that this function can return 2N + 1 different values
+   *         ranging from -(N + 1) to N - 1.
+   */
+  public static int binarySearch(byte [][]arr, byte []key, int offset,
+      int length, RawComparator<byte []> comparator) {
+    int low = 0;
+    int high = arr.length - 1;
+
+    while (low <= high) {
+      int mid = (low+high) >>> 1;
+      // we have to compare in this order, because the comparator order
+      // has special logic when the 'left side' is a special key.
+      int cmp = comparator.compare(key, offset, length,
+          arr[mid], 0, arr[mid].length);
+      // key lives above the midpoint
+      if (cmp > 0)
+        low = mid + 1;
+      // key lives below the midpoint
+      else if (cmp < 0)
+        high = mid - 1;
+      // BAM. how often does this really happen?
+      else
+        return mid;
+    }
+    return - (low+1);
+  }
+
+  /**
+   * Bytewise binary increment/deincrement of long contained in byte array
+   * on given amount.
+   *
+   * @param value - array of bytes containing long (length <= SIZEOF_LONG)
+   * @param amount value will be incremented on (deincremented if negative)
+   * @return array of bytes containing incremented long (length == SIZEOF_LONG)
+   * @throws IOException - if value.length > SIZEOF_LONG
+   */
+  public static byte [] incrementBytes(byte[] value, long amount)
+  throws IOException {
+    byte[] val = value;
+    if (val.length < SIZEOF_LONG) {
+      // Hopefully this doesn't happen too often.
+      byte [] newvalue;
+      if (val[0] < 0) {
+        newvalue = new byte[]{-1, -1, -1, -1, -1, -1, -1, -1};
+      } else {
+        newvalue = new byte[SIZEOF_LONG];
+      }
+      System.arraycopy(val, 0, newvalue, newvalue.length - val.length,
+        val.length);
+      val = newvalue;
+    } else if (val.length > SIZEOF_LONG) {
+      throw new IllegalArgumentException("Increment Bytes - value too big: " +
+        val.length);
+    }
+    if(amount == 0) return val;
+    if(val[0] < 0){
+      return binaryIncrementNeg(val, amount);
+    }
+    return binaryIncrementPos(val, amount);
+  }
+
+  /* increment/deincrement for positive value */
+  private static byte [] binaryIncrementPos(byte [] value, long amount) {
+    long amo = amount;
+    int sign = 1;
+    if (amount < 0) {
+      amo = -amount;
+      sign = -1;
+    }
+    for(int i=0;i<value.length;i++) {
+      int cur = ((int)amo % 256) * sign;
+      amo = (amo >> 8);
+      int val = value[value.length-i-1] & 0x0ff;
+      int total = val + cur;
+      if(total > 255) {
+        amo += sign;
+        total %= 256;
+      } else if (total < 0) {
+        amo -= sign;
+      }
+      value[value.length-i-1] = (byte)total;
+      if (amo == 0) return value;
+    }
+    return value;
+  }
+
+  /* increment/deincrement for negative value */
+  private static byte [] binaryIncrementNeg(byte [] value, long amount) {
+    long amo = amount;
+    int sign = 1;
+    if (amount < 0) {
+      amo = -amount;
+      sign = -1;
+    }
+    for(int i=0;i<value.length;i++) {
+      int cur = ((int)amo % 256) * sign;
+      amo = (amo >> 8);
+      int val = ((~value[value.length-i-1]) & 0x0ff) + 1;
+      int total = cur - val;
+      if(total >= 0) {
+        amo += sign;
+      } else if (total < -256) {
+        amo -= sign;
+        total %= 256;
+      }
+      value[value.length-i-1] = (byte)total;
+      if (amo == 0) return value;
+    }
+    return value;
+  }
+
+  /**
+   * Writes a string as a fixed-size field, padded with zeros.
+   */
+  public static void writeStringFixedSize(final DataOutput out, String s,
+      int size) throws IOException {
+    byte[] b = toBytes(s);
+    if (b.length > size) {
+      throw new IOException("Trying to write " + b.length + " bytes (" +
+          toStringBinary(b) + ") into a field of length " + size);
+    }
+
+    out.writeBytes(s);
+    for (int i = 0; i < size - s.length(); ++i)
+      out.writeByte(0);
+  }
+
+  /**
+   * Reads a fixed-size field and interprets it as a string padded with zeros.
+   */
+  public static String readStringFixedSize(final DataInput in, int size)
+      throws IOException {
+    byte[] b = new byte[size];
+    in.readFully(b);
+    int n = b.length;
+    while (n > 0 && b[n - 1] == 0)
+      --n;
+
+    return toString(b, 0, n);
+  }
+
+  /**
+   * @param b a byte buffer
+   * @return true if the given byte buffer is non-null and non-empty (has remaining bytes)
+   */
+  public static boolean isNonEmpty(ByteBuffer b) {
+    return b != null && b.remaining() > 0;
+  }
+
+  /**
+   * This is a utility method, that serializes a Swift annotated class' object
+   * into a byte array. This is equivalent to Writable.getBytes().
+   *
+   * @param t The object to be serialized.
+   * @param clazz The class of the object to be serialized
+   * @param <T>
+   * @return The byte array corresponding to the serialized object.
+   * @throws Exception
+   */
+  public static <T> ByteBuffer writeThriftBytes(T t, Class<T> clazz,
+    Class<? extends TProtocol> protocolClass)
+    throws Exception {
+    TMemoryBuffer buffer =
+      writeThriftBytesAndGetBuffer(t, clazz, protocolClass);
+    return ByteBuffer.wrap(buffer.getArray(), 0, buffer.length());
+  }
+
+  /**
+   * @param t
+   * @param clazz
+   * @return
+   * @throws Exception
+   */
+  public static <T> TMemoryBuffer writeThriftBytesAndGetBuffer(T t,
+    Class<T> clazz, Class<? extends TProtocol> protocolClass) throws Exception {
+    ThriftCodec<T> codec = THRIFT_CODEC_MANAGER.getCodec(clazz);
+    TMemoryBuffer buffer = new TMemoryBuffer(0);
+    TProtocol protocol =
+      protocolClass.getConstructor(TTransport.class).newInstance(buffer);
+    codec.write(t, protocol);
+    return buffer;
+  }
+
+  /**
+   * This is a utility method, that deserializes a Swift annotated class' object
+   * from a byte array. This is equivalent to Writable.getWritable().
+   *
+   * @param buff
+   * @param clazz
+   * @param <T>
+   * @return
+   * @throws Exception
+   */
+  public static <T> T readThriftBytes(byte[] buff,
+    Class<T> clazz, Class<? extends TProtocol> protocolClass)
+    throws Exception {
+    ThriftCodec<T> codec = THRIFT_CODEC_MANAGER.getCodec(clazz);
+    TMemoryInputTransport buffer = new TMemoryInputTransport(buff);
+    TProtocol protocol =
+      protocolClass.getConstructor(TTransport.class).newInstance(buffer);
+    return codec.read(protocol);
+  }
+
+  public static byte[] copyOfByteArray(byte[] arr) {
+    byte[] tmp = new byte[arr.length];
+    System.arraycopy(arr, 0, tmp, 0, arr.length);
+    return tmp;
+  }
+
+  /**
+   * Returns a byte array next to a given one, i.e. it is the smallest byte
+   * array among all byte arrays that is strictly greater than the give array.
+   * Greater and smaller are defined by Bytes.compareTo.
+   *
+   * @param b
+   *          the give array
+   */
+  public static byte[] nextOf(byte[] b) {
+    byte[] res = new byte[b.length + 1];
+    System.arraycopy(b, 0, res, 0, b.length);
+    return res;
+  }
+
+  /**
+   * Return whether b equals nextOf(a)
+   */
+  public static boolean isNext(byte[] a, byte[] b) {
+    return isNext(a, 0, a.length, b, 0, b.length);
+  }
+
+  /**
+   * Return whether b[...] equals nextOf(a[...])
+   */
+  public static boolean isNext(byte[] a, int aOffs, int aLen, byte[] b,
+      int bOffs, int bLen) {
+    if (a == null || b == null) {
+      return false;
+    }
+
+    if (bLen != aLen + 1) {
+      return false;
+    }
+    if (b[bOffs + aLen] != 0) {
+      return false;
+    }
+    return Bytes.compareTo(a, aOffs, aLen, b, bOffs, aLen) == 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/ConditionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/ConditionUtil.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/ConditionUtil.java
new file mode 100644
index 0000000..a86264d
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/ConditionUtil.java
@@ -0,0 +1,62 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Extends the patterns in {@link Preconditions}
+ */
+public class ConditionUtil {
+
+  /**
+   * Checks if a specified offset is >= 0
+   * @param offset The offset to check
+   * @return The specified offset if it is >= 0
+   * @throws IndexOutOfBoundsException If specified offset is negative
+   */
+  public static long checkPositiveOffset(long offset) {
+    return checkOffset(offset, -1);
+  }
+
+  /**
+   * Check if an offset is >= 0 but less than a maximum limit (if one is
+   * specified).
+   * @see {@link Preconditions#checkPositionIndex(int, int)}
+   * @param offset The offset to check
+   * @param limit The maximum limit or -1 if none
+   * @return The specified offset if it is positive and if the a limit is
+   *         specified lower than that limit.
+   * @throws IllegalStateException If the offset is negative, or if a limit
+   *         is specified and the offset is greater than the limit.
+   */
+  public static long checkOffset(long offset, long limit) {
+    if (offset < 0) {
+      throw new IndexOutOfBoundsException("Negative offset: " + offset);
+    }
+    if (limit != -1 && offset >= limit) {
+      throw new IndexOutOfBoundsException("Offset (" + offset +
+          ") is greater than limit (" + limit + ")");
+    }
+    return offset;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/DaemonThreadFactory.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/DaemonThreadFactory.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/DaemonThreadFactory.java
new file mode 100644
index 0000000..9733fdb
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/DaemonThreadFactory.java
@@ -0,0 +1,46 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import java.util.concurrent.ThreadFactory;
+
+public class DaemonThreadFactory implements ThreadFactory {
+  final ThreadGroup group;
+  int threadNumber = 1;
+  final String namePrefix;
+
+  public DaemonThreadFactory(String namePrefix) {
+      SecurityManager s = System.getSecurityManager();
+      group = (s != null)? s.getThreadGroup() :
+                           Thread.currentThread().getThreadGroup();
+      this.namePrefix = namePrefix;
+  }
+
+  public Thread newThread(Runnable r) {
+      Thread t = new Thread(group, r,
+                            namePrefix + (threadNumber++),
+                            0);
+      if (!t.isDaemon())
+          t.setDaemon(true);
+      if (t.getPriority() != Thread.NORM_PRIORITY)
+          t.setPriority(Thread.NORM_PRIORITY);
+      return t;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/HasThread.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/HasThread.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/HasThread.java
new file mode 100644
index 0000000..058d64c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/HasThread.java
@@ -0,0 +1,101 @@
+/**
+ * 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.hadoop.hbase.util;
+
+import java.lang.Thread.UncaughtExceptionHandler;
+
+/**
+ * Abstract class which contains a Thread and delegates the common Thread
+ * methods to that instance.
+ *
+ * The purpose of this class is to workaround Sun JVM bug #6915621, in which
+ * something internal to the JDK uses Thread.currentThread() as a monitor
+ * lock. This can produce deadlocks like HBASE-4367, HBASE-4101, etc.
+ */
+public abstract class HasThread implements Runnable {
+  private final Thread thread;
+
+  public HasThread() {
+    this.thread = new Thread(this);
+  }
+
+  public HasThread(String name) {
+    this.thread = new Thread(this, name);
+  }
+
+  public Thread getThread() {
+    return thread;
+  }
+
+  public abstract void run();
+
+  //// Begin delegation to Thread
+
+  public final String getName() {
+    return thread.getName();
+  }
+
+  public void interrupt() {
+    thread.interrupt();
+  }
+
+  public final boolean isAlive() {
+    return thread.isAlive();
+  }
+
+  public boolean isInterrupted() {
+    return thread.isInterrupted();
+  }
+
+  public final void setDaemon(boolean on) {
+    thread.setDaemon(on);
+  }
+
+  public final void setName(String name) {
+    thread.setName(name);
+  }
+
+  public final void setPriority(int newPriority) {
+    thread.setPriority(newPriority);
+  }
+
+  public void setUncaughtExceptionHandler(UncaughtExceptionHandler eh) {
+    thread.setUncaughtExceptionHandler(eh);
+  }
+
+  public void start() {
+    thread.start();
+  }
+
+  public final void join() throws InterruptedException {
+    thread.join();
+  }
+
+  public final void join(long millis, int nanos) throws InterruptedException {
+    thread.join(millis, nanos);
+  }
+
+  public final void join(long millis) throws InterruptedException {
+    thread.join(millis);
+  }
+
+  public static void sleep(long millis) throws InterruptedException {
+    Thread.sleep(millis);
+  }
+  //// End delegation to Thread
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Histogram.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Histogram.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Histogram.java
new file mode 100644
index 0000000..adab560
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Histogram.java
@@ -0,0 +1,390 @@
+/**
+ * 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.hadoop.hbase.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.regionserver.metrics.PercentileMetric;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * The Histogram class provides a mechanism to sample data points and perform
+ * estimates about percentile metrics.
+ * Percentile metric is defined as the follows :
+ *  A P99 value is the 99th percentile value among the given data points.
+ *
+ * Usage :
+ * Refer to RegionServerMetrics to see how this Histogram can be used to find
+ * percentile estimates.
+ *
+ * The general expected workflow of a Histogram class is as follows:
+ * [<Initialize Histogram> [[<addValue>]* [<getPercentileEstimate>]+ <refresh>]*]
+ *
+ * In the above sequence addValue can be called from different threads, but
+ * getPercentileEstimate and refresh should be called from the same thread since
+ * they are not mutually thread safe.
+ */
+
+public class Histogram {
+  private List<Bucket> buckets;
+  private int numBuckets;
+  private double minValue;
+  private double maxValue;
+
+  public static class Stats {
+    public double min = 0;
+    public double max = 0;
+    public double avg = 0;
+    public double count = 0;
+  }
+
+  private List<Double> underloadSampleList; // We serve the under loaded cases
+  // from this list.
+  final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+  private boolean underload = true;
+  public static final Log LOG = LogFactory.getLog(Histogram.class.getName());
+
+  // We will consider the case when we have more than 100 samples as
+  // overloaded case and use the Histogram only in such scenarios. In other
+  // cases, we can serve the percentile queries using the underloadSampleList
+  private static final int DEFAULT_MINIMUM_LOAD = 100;
+
+  /**
+   * Create a histogram with the default values of number of buckets,
+   * and min/max for the values.
+   */
+  public Histogram() {
+    this(PercentileMetric.HISTOGRAM_NUM_BUCKETS_DEFAULT,
+          PercentileMetric.HISTOGRAM_MINVALUE_DEFAULT,
+          PercentileMetric.HISTOGRAM_MAXVALUE_DEFAULT);
+  }
+
+  // Bucket indexing is from 1 to N
+  public Histogram(int numBuckets, double lowBound, double maxValue) {
+    if (numBuckets < 1 || lowBound >= maxValue) {
+      throw new UnsupportedOperationException();
+    }
+    buckets = new ArrayList<Bucket>(numBuckets);
+    underloadSampleList = Collections.synchronizedList(new ArrayList<Double>());
+    refresh(numBuckets, lowBound, maxValue);
+  }
+
+  public Stats getBasicStats() {
+    Stats stats = new Stats();
+
+    this.lock.writeLock().lock();
+    try {
+      if (underloadSampleList.size() == 0) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Too few data points. Consider increasing the sampling time.");
+        }
+        return stats;
+      } else if (underload) {
+        Collections.sort(underloadSampleList);
+        stats.min = underloadSampleList.get(0);
+        stats.max = underloadSampleList.get(underloadSampleList.size() - 1);
+        stats.count = underloadSampleList.size();
+        double sum = 0;
+        for (double d : underloadSampleList) {
+          sum += d;
+        }
+        stats.avg = sum / stats.count;
+
+        return stats;
+      } else {
+        int totalCount = 0;
+        double totalSum = 0;
+        boolean minFound = false;
+        stats.max = Double.MIN_VALUE;
+
+        for (int i = 0; i < this.buckets.size(); i++) {
+          Bucket b = this.buckets.get(i);
+          if (b.count > 0) {
+            if (!minFound) {
+              stats.min = b.getMinValue();
+              minFound = true;
+            }
+            stats.max = Math.max(stats.max, b.getMaxValue());
+            totalCount += b.getCount();
+            totalSum += b.getSum();
+          }
+        }
+        stats.count = totalCount;
+        stats.avg = (totalCount == 0) ? 0 : (totalSum / (double) totalCount);
+      }
+    } catch(Exception e) {
+      LOG.error("Unknown Exception : " + e.getMessage());
+    } finally {
+      this.lock.writeLock().unlock();
+    }
+
+    return stats;
+  }
+
+  // This is included in the bucket
+  private double getBucketStartValue(int bucketIndex) {
+    if (bucketIndex < 1 || bucketIndex > this.numBuckets) {
+      throw new IndexOutOfBoundsException();
+    }
+    double gap = this.maxValue - this.minValue;
+    double slice = gap/this.numBuckets;
+    return this.minValue + (bucketIndex - 1.0)*slice;
+  }
+
+  //This is not included in the bucket
+  private double getBucketEndValue(int bucketIndex) {
+    if (bucketIndex < 1 || bucketIndex > this.numBuckets) {
+      throw new IndexOutOfBoundsException();
+    }
+    double gap = this.maxValue - this.minValue;
+    double slice = gap/this.numBuckets;
+    return this.minValue + (bucketIndex)*slice;
+  }
+
+  private int getBucketIndex(double value) {
+    if (value < this.minValue) {
+      return 0;
+    } else if (value >= this.maxValue) {
+      return this.numBuckets + 1;
+    } else {
+      double gap = value - this.minValue;
+      double idx = this.numBuckets * gap / (this.maxValue - this.minValue);
+      int i = (int)idx + 1;
+      // Check if the index falls in the margin somehow.
+      if (value < this.getBucketStartValue(i)) i--;
+      else if (value >= this.getBucketEndValue(i)) i++;
+      return i; // Due to 1 indexing
+    }
+  }
+
+  public void refresh() {
+    this.refresh(this.numBuckets);
+  }
+
+  public void refresh(int numBuckets) {
+    this.lock.writeLock().lock();
+    try {
+      double lowBound = this.minValue;
+      double upBound = this.maxValue;
+      for (Bucket bucket : this.buckets) {
+        if (bucket.count > 0) {
+          lowBound = bucket.getMinValue();
+          break;
+        }
+      }
+      for (int i = this.buckets.size() - 1; i>=0; i--) {
+        Bucket bucket = this.buckets.get(i);
+        if (bucket.count > 0) {
+          upBound = bucket.getMaxValue();
+          break;
+        }
+      }
+      this.refresh(numBuckets, lowBound, upBound);
+    } catch (Exception e) {
+      LOG.error("Unknown Exception : " + e.getMessage());
+    } finally {
+      this.lock.writeLock().unlock();
+    }
+  }
+
+  private void refresh(int numBuckets, double minValue, double maxValue) {
+    this.numBuckets = numBuckets;
+    this.minValue = minValue;
+    this.maxValue = maxValue;
+
+    this.buckets.clear();
+    underloadSampleList.clear();
+    underload = true;
+    buckets.add(new Bucket(Double.MIN_VALUE, this.getBucketStartValue(1)));
+    for (int i = 1; i<=this.numBuckets; i++) {
+      this.buckets.add(new Bucket(this.getBucketStartValue(i),
+          this.getBucketEndValue(i)));
+    }
+    buckets.add(new Bucket(this.getBucketEndValue(this.numBuckets),
+        Double.MAX_VALUE));
+  }
+
+  public double getPercentileEstimate(double prcntyl) {
+    // We scan from the end of the table since our use case is to find the
+    // p99, p95 latencies.
+    double originalPrcntyl = prcntyl;
+    if (prcntyl > 100.0 || prcntyl < 0.0) {
+      throw new IllegalArgumentException("Percentile input value not in range.");
+    } else {
+      prcntyl = 100.0 - prcntyl;
+    }
+    double ret = 0.0;
+    this.lock.writeLock().lock();
+    try {
+      if (underloadSampleList.size() == 0) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Too few data points. Consider increasing the sampling time.");
+        }
+        return ret;
+      } else if (underload) {
+        Collections.sort(underloadSampleList);
+        // Since the use case is to clear the list after a call to this
+        // function, we can afford to sort this list here and enjoy O(1) the
+        // rest of the time.
+        return underloadSampleList.get(
+            (int)(originalPrcntyl * underloadSampleList.size()/100.0));
+      }
+      int totalCount = 0;
+      for (Bucket bucket : this.buckets) {
+        totalCount += bucket.count;
+      }
+      double countToCoverdouble = (totalCount * prcntyl / 100.0);
+      int countToCover = (int)countToCoverdouble;
+      for (int i=this.buckets.size() - 1; i >= 0; i--) {
+        Bucket bucket = this.buckets.get(i);
+        if (bucket.getCount() >= countToCover) {
+          return bucket.getGreaterValue(bucket.getCount() - countToCover);
+        } else {
+          countToCover -= bucket.getCount();
+        }
+      }
+      ret = this.maxValue;
+    } catch(Exception e) {
+      LOG.error("Unknown Exception : " + e.getMessage());
+    } finally {
+      this.lock.writeLock().unlock();
+    }
+    return ret;
+  }
+
+  public void addValue(double value) {
+    this.lock.readLock().lock();
+    try {
+      if (underloadSampleList.size() >= Histogram.DEFAULT_MINIMUM_LOAD) {
+        if (underload) {
+          synchronized (underloadSampleList) {
+            if (underload) {
+              for (double val : underloadSampleList) {
+                Bucket bucket = buckets.get(this.getBucketIndex(val));
+                bucket.addValue(val);
+              }
+            }
+            underload = false;
+          }
+        }
+        Bucket bucket = buckets.get(this.getBucketIndex(value));
+        bucket.addValue(value);
+      } else {
+        underloadSampleList.add(value);
+      }
+    } catch (Exception e) {
+      LOG.error("Unknown Exception : " + e.getMessage());
+    } finally {
+      this.lock.readLock().unlock();
+    }
+  }
+
+  public void addValue(Long value) {
+    addValue(value.doubleValue());
+  }
+
+  public class Bucket {
+    private double sum;
+    private int count;
+    private double minValue;
+    private double maxValue;
+    private double startValue;
+    private double endValue;
+    public Bucket(double startValue, double endValue) {
+      this.sum = 0.0;
+      this.count = 0;
+      this.minValue = endValue;
+      this.maxValue = startValue;
+      this.startValue = startValue;
+      this.endValue = endValue;
+    }
+
+    public void addValue(double value) {
+      this.sum = this.sum + value;
+      count++;
+      this.minValue = Math.min(this.minValue, value);
+      this.maxValue = Math.max(this.maxValue, value);
+    }
+
+    /**
+     * This function gives the count of the number of items in the bucket
+     * which are smaller than the given value;
+     * For the purpose of this calculation, the distribution over the bucket
+     * is assumed to be uniformly distributed between minValue and maxValue
+     */
+    public int getGreaterCount(double value) {
+      if (!(this.minValue < value && this.maxValue >= value)) {
+        throw new IllegalArgumentException();
+      }
+      if (this.count == 0) return 0;
+      else if (this.count == 1) {
+        if (this.minValue > value) return 0;
+        else return 1;
+      }
+      double gap = value - this.minValue;
+      double ret = this.count * gap / (this.maxValue - this.minValue);
+      return (int)ret;
+    }
+
+    /**
+     * This function gives the value which is more than a certain count in this
+     * bucket.
+     * */
+    public double getGreaterValue(int count) {
+      if (count > this.count) {
+        throw new IllegalArgumentException();
+      }
+      double gap = this.maxValue - this.minValue;
+      double ret = this.minValue + gap * count / this.count;
+      return ret;
+    }
+
+    public double getSum() {
+      return this.sum;
+    }
+
+    public int getCount() {
+      return this.count;
+    }
+
+    public double getMinValue() {
+      return this.minValue;
+    }
+
+    public double getMaxValue() {
+      return this.maxValue;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder s = new StringBuilder();
+      s.append("Bucket Details :");
+      s.append(" count : " + this.count);
+      s.append(" sum : " + this.sum);
+      s.append(" minValue : " + this.minValue);
+      s.append(" maxValue : " + this.maxValue);
+      s.append(" startValue : " + this.startValue);
+      s.append(" endValue : " + this.endValue);
+      return s.toString();
+    }
+  }
+}


[15/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumClient.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumClient.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumClient.java
new file mode 100644
index 0000000..61c042d
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumClient.java
@@ -0,0 +1,285 @@
+package org.apache.hadoop.hbase.consensus.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.client.NoLeaderForRegionException;
+import org.apache.hadoop.hbase.consensus.log.LogFileInfo;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+import org.apache.hadoop.hbase.consensus.rpc.LogState;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.regionserver.DataStoreState;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Threads;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+/**
+ * QuorumClient is the user facing agent for replicating commits (WALEdits) on
+ * a per-quorum level. Once set up, the replicateCommits(List<WALEdit>) method
+ * will transparently handle contacting the leader, sending the edits to be
+ * replicated, handling the retries / errors, etc.
+ */
+public class QuorumClient {
+  private final static Logger LOG = LoggerFactory.getLogger(QuorumClient.class);
+  private Map<String, QuorumThriftClientAgent> agentMap;
+  private List<QuorumThriftClientAgent> agents;
+  private final String regionId;
+  private final Configuration conf;
+  private final long maxOperationLatencyInMillis;
+  private final long sleepIntervalInMillis;
+  private QuorumThriftClientAgent leader = null;
+  private ExecutorService pool;
+
+  /**
+   * Creates the QuorumClient.
+   * @param regionInfo The QuorumInfo which describes the quorum.
+   * @param conf The configuration object.
+   * @param pool The thread pool to be used.
+   * @throws IOException
+   */
+  public QuorumClient(QuorumInfo regionInfo, Configuration conf,
+    ExecutorService pool) throws IOException {
+    this.regionId = regionInfo.getQuorumName();
+    this.conf = conf;
+    this.maxOperationLatencyInMillis = conf.getLong(
+            HConstants.QUORUM_CLIENT_COMMIT_DEADLINE_KEY,
+            HConstants.QUORUM_CLIENT_COMMIT_DEADLINE_DEFAULT);
+    this.sleepIntervalInMillis = conf.getLong(
+      HConstants.QUORUM_CLIENT_RETRY_SLEEP_INTERVAL,
+      HConstants.QUORUM_CLIENT_RETRY_SLEEP_INTERVAL_DEFAULT
+    );
+    updateConfig(regionInfo);
+    this.pool = pool;
+  }
+
+  protected QuorumClient(String regionId, final Configuration conf,
+                         ExecutorService pool) throws IOException {
+    this(RaftUtil.createDummyRegionInfo(regionId).getQuorumInfo(), conf, pool);
+  }
+
+  public synchronized long replicateCommits(List<WALEdit> txns)
+    throws IOException {
+    int numRetries = 0;
+    long endTime = System.currentTimeMillis() + maxOperationLatencyInMillis;
+    while (System.currentTimeMillis() <= endTime) {
+      try {
+        if (agents == null || agents.isEmpty()) {
+          LOG.error("No quorum agent is running");
+          return HConstants.UNDEFINED_TERM_INDEX;
+        }
+        if (leader == null) {
+          waitForLeader(endTime);
+        }
+
+        if (leader == null) {
+          ++numRetries;
+          LOG.error(String.format("%s cannot find leader. Retry : %d",
+            regionId, numRetries));
+          Threads.sleep(this.sleepIntervalInMillis);
+        } else {
+          return leader.replicateCommit(this.regionId, txns);
+        }
+      } catch (Exception e) {
+        ++numRetries;
+        LOG.error(String.format("%s Unable to replicate the commit. Retry number" +
+          " %d", regionId, numRetries), e);
+        Threads.sleep(this.sleepIntervalInMillis);
+        resetLeader();
+      }
+    }
+    throw new IOException(String.format("%s Unable to replicate the commit." +
+        " Retried %s times",
+      regionId, numRetries));
+  }
+
+  /**
+   * Request the quorum to change to a new config.
+   * @param config the new config
+   */
+  public synchronized boolean changeQuorum(final QuorumInfo config)
+    throws IOException {
+
+    int numRetries = 0;
+    long endTime = System.currentTimeMillis() + this.maxOperationLatencyInMillis;
+    while (System.currentTimeMillis() <= endTime) {
+      try {
+        if (agents == null || agents.isEmpty()) {
+          LOG.error(String.format("%s : No quorum agent is running", regionId));
+          return false;
+        }
+        if (leader == null) {
+          waitForLeader(endTime);
+        }
+
+        if (leader != null) {
+          return leader.changeQuorum(config.getQuorumName(),
+            QuorumInfo.serializeToBuffer(Arrays.asList(config)));
+        } else {
+          ++numRetries;
+          LOG.error(String.format("%s cannot find leader. Retry number: %d",
+            regionId, numRetries));
+          Threads.sleep(this.sleepIntervalInMillis);
+        }
+      } catch (Exception e) {
+        ++numRetries;
+        LOG.error(String.format("%s Unable to send membership change request." +
+          "Retry number %d.", regionId, numRetries), e);
+        Threads.sleep(this.sleepIntervalInMillis);
+        resetLeader();
+      }
+    }
+    throw new IOException(String.format(
+      "%s Unable to send membership" + " change request. Retry number %d.",
+      regionId, numRetries));
+  }
+
+  private void resetLeader() {
+    this.leader = null;
+  }
+
+  private void waitForLeader(long endTime) throws NoLeaderForRegionException {
+    ExecutorCompletionService<String> service =
+      new ExecutorCompletionService<>(pool);
+    while (System.currentTimeMillis() < endTime) {
+      for (final QuorumThriftClientAgent agent : agents) {
+        service.submit(new Callable<String>() {
+          @Override
+          public String call() throws Exception {
+            try {
+              return agent.getLeader(regionId);
+            } catch (Exception e) {
+              LOG.error(String.format("%s: Cannot talk to quorum server: %s",
+                regionId, agent.getServerAddress()), e);
+              throw e;
+            }
+          }
+        });
+      }
+      // as soon as the first future is finished we are done
+      for (int index = 0; index < agents.size(); index++) {
+        try {
+          final Future<String> future = service.take();
+          String leaderAddrString = future.get();
+          if (leaderAddrString != null && !leaderAddrString.isEmpty()) {
+            HServerAddress leaderAddr = new HServerAddress(leaderAddrString);
+            leader = agentMap.get(leaderAddr.getHostNameWithPort());
+            if (leader != null) {
+              LOG.debug(String.format(
+                "The current leader for the region %s is %s.",
+                regionId, leader.getServerAddress()));
+              return;
+            }
+          }
+        } catch (ExecutionException e) {
+          LOG.error(String.format("%s Error appeared:", regionId), e);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          LOG.error(String.format("%s Error appeared:", regionId), e);
+        }
+      }
+    }
+    throw new NoLeaderForRegionException(
+      String.format("No leader found for region %s", regionId));
+  }
+
+  public synchronized PeerStatus[] getPeerStatus() {
+    PeerStatus[] statuses = new PeerStatus[this.agents.size()];
+    for (int i = 0; i < agents.size(); i++) {
+      try {
+        statuses[i] = agents.get(i).getPeerStatus(regionId);
+      } catch (Throwable e) {
+        statuses[i] = new PeerStatus(regionId, -1, HConstants.UNDEFINED_TERM_INDEX,
+          PeerStatus.RAFT_STATE.INVALID, new LogState(e.toString()),
+          "ERROR", new DataStoreState("ERROR"));
+      }
+      statuses[i].setPeerAddress(agents.get(i).getServerAddress());
+    }
+
+    Arrays.sort(statuses);
+    return statuses;
+  }
+
+  public synchronized List<Pair<String, List<LogFileInfo>>> getPeerCommittedLogStatus(
+      HServerAddress localServerAddr,
+      String quorumName,
+      long minIndex) throws Exception {
+    List<Pair<String, List<LogFileInfo>>> result = new ArrayList<>();
+    if (agents == null || agents.isEmpty()) {
+      return result;
+    }
+    String localConsensusAddr =
+        RaftUtil.getLocalConsensusAddress(localServerAddr).getHostNameWithPort();
+    for (QuorumThriftClientAgent agent : agents) {
+      if (!agent.getServerAddress().equals(localConsensusAddr)) {
+        List<LogFileInfo> logFiles = agent.getCommittedLogStatus(quorumName, minIndex);
+        if (logFiles != null) {
+          result.add(new Pair<>(agent.getServerAddress(), logFiles));
+        }
+      }
+    }
+
+    return result;
+  }
+
+  public synchronized void close() {
+    for (QuorumThriftClientAgent agent : agents) {
+      agent.close();
+    }
+  }
+
+  public synchronized QuorumThriftClientAgent getLeader() {
+    return leader;
+  }
+
+  private void updateConfig(final QuorumInfo config) {
+    // Close the current agents
+    if (agents != null) {
+      close();
+    }
+
+    int connectionTimeout =
+      conf.getInt("hbase.regionserver.quorum.client.connection.timeout", 10000);
+    int readTimeout =
+      conf.getInt("hbase.regionserver.quorum.client.read.timeout", 10000);
+    int writeTimeout =
+      conf.getInt("hbase.regionserver.quorum.client.write.timeout", 10000);
+    int retryCount =
+      conf.getInt("hbase.regionserver.quorum.client.retry.cnt", 1);
+    this.agents = new ArrayList<>(config.getPeersWithRank().size());
+    this.agentMap = new HashMap<>();
+    for (HServerAddress address : config.getPeersWithRank().keySet()) {
+      try {
+        QuorumThriftClientAgent agent = new QuorumThriftClientAgent(
+            RaftUtil.getLocalConsensusAddress(address).getHostAddressWithPort(),
+            connectionTimeout, readTimeout, writeTimeout, retryCount);
+        this.agents.add(agent);
+        agentMap.put(agent.getServerAddress(), agent);
+      } catch (IOException e) {
+        LOG.error("Unable to initialize quorum thrift agent for " + address);
+      }
+    }
+  }
+
+  /**
+   * Updated the ThriftClientAgents based on the given quorum info.
+   * @param config
+   */
+  public synchronized void refreshConfig(final QuorumInfo config) {
+    updateConfig(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumLoadTestClient.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumLoadTestClient.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumLoadTestClient.java
new file mode 100644
index 0000000..4b9d8e0
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumLoadTestClient.java
@@ -0,0 +1,161 @@
+package org.apache.hadoop.hbase.consensus.client;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class QuorumLoadTestClient {
+  private static final Logger LOG = LoggerFactory.getLogger(
+          QuorumLoadTestClient.class);
+
+  private static int status_interval = 1000;
+  private static int progress_interval = 100;
+  private static int maxBatchSize = 5;
+
+  private static QuorumClient client;
+  private final static ExecutorService EXEC_SERVICE_FOR_THRIFT_CLIENTS =
+      Executors.newFixedThreadPool(5, new DaemonThreadFactory("QuorumClient-"));
+
+  private static int[] payloadSize = {
+    512, 1 * 1024, 4 * 1024, 16 * 1024, 64 * 1024,
+    128 * 1024, 256 * 1024, 512 * 1024,
+  };
+
+  public static void loadTest() {
+    // generate the test data
+    int testDataIndex = 0;
+    List<WALEdit> testData = new ArrayList<>();
+
+    while(true) {
+
+      for (int batchSize = 1; batchSize <= maxBatchSize; batchSize++) {
+        testData.clear();
+        String payloadSizeName = FileUtils.byteCountToDisplaySize(payloadSize[testDataIndex]);
+
+        for (int j = 0; j < batchSize; j++) {
+          KeyValue kv = KeyValue.generateKeyValue(
+            payloadSizeName,payloadSize[testDataIndex]);
+          testData.add(new WALEdit(Arrays.asList(kv)));
+        }
+
+        long start = System.nanoTime();
+        for (int s = 0; s < status_interval; s++) {
+          try {
+            client.replicateCommits(testData);
+            if (s % progress_interval == 0) {
+              System.out.print("+");
+            }
+          } catch (Exception e) {
+            System.out.println("Unexpected exception " + e);
+            e.printStackTrace();
+          }
+        }
+
+        float timeInMicroSec =  (System.nanoTime() - start) / (float) 1000;
+        float avgLatency = timeInMicroSec / status_interval;
+        float size = payloadSize[testDataIndex] * batchSize;
+        float avgThroughput = size / (timeInMicroSec / 1000/ 1000) ;
+
+        String latency = new DecimalFormat("##.##").format(avgLatency);
+        String throughput = FileUtils.byteCountToDisplaySize((long)avgThroughput) + "/sec";
+
+        System.out.println("[kv_size: " + payloadSizeName +
+          ", batch_size: " + batchSize +
+          ", avg_latency: " + latency + " microsec" +
+          ", throughput: " + throughput + " ]");
+      }
+
+      if (++testDataIndex == payloadSize.length) {
+        testDataIndex = 0;
+      }
+
+    }
+  }
+
+  public static void main(String[] args) throws IOException {
+
+    Options opt = new Options();
+    opt.addOption("region", true, "the region id for the load test");
+    opt.addOption("servers", true, "A list of quorum server delimited by ,");
+
+    opt.addOption("max_batch_size", true, "The max number of kvs in each transaction");
+    opt.addOption("status_interval", true, "To print the status at very $status_interval " +
+      "transactions");
+    opt.addOption("progress_interval", true,
+      "To print the progress at very $progress_interval transactions");
+
+    String regionId = null;
+    String serverList;
+    String servers[] = null;
+
+    try {
+      CommandLine cmd = new GnuParser().parse(opt, args);
+
+      if (cmd.hasOption("region")) {
+        regionId = cmd.getOptionValue("region");
+      }
+
+      if (cmd.hasOption("servers")) {
+        serverList = cmd.getOptionValue("servers");
+        servers = serverList.split(",");
+      }
+
+      if (cmd.hasOption("max_batch_size")) {
+        maxBatchSize = Integer.valueOf(cmd.getOptionValue("max_batch_size"));
+      }
+
+      if (cmd.hasOption("status_interval")) {
+        status_interval = Integer.valueOf(cmd.getOptionValue("status_interval"));
+      }
+
+      if (cmd.hasOption("progress_interval")) {
+        progress_interval = Integer.valueOf(cmd.getOptionValue("progress_interval"));
+      }
+
+      if (regionId == null || regionId.isEmpty() || servers == null || servers.length == 0) {
+        LOG.error("Wrong args !");
+        printHelp(opt);
+        System.exit(-1);
+      }
+
+      System.out.println("*******************");
+      System.out.println("region: " + regionId);
+
+      Configuration conf = HBaseConfiguration.create();
+      client = new QuorumClient(regionId, conf, EXEC_SERVICE_FOR_THRIFT_CLIENTS);
+
+      System.out.println("Initialized the quorum client");
+      System.out.println("*******************");
+    } catch (Exception e) {
+      e.printStackTrace();
+      printHelp(opt);
+      System.exit(-1);
+    }
+
+    System.out.println("Starting client for load testing");
+    loadTest();
+  }
+
+  private static void printHelp(Options opt) {
+    new HelpFormatter().printHelp(
+      "QuorumLoadTestClient -region regionID -servers h1:port,h2:port,h3:port..." +
+        "[-status_interval interval] [-progress_interval interval]", opt);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumThriftClientAgent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumThriftClientAgent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumThriftClientAgent.java
new file mode 100644
index 0000000..2d39705
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumThriftClientAgent.java
@@ -0,0 +1,342 @@
+package org.apache.hadoop.hbase.consensus.client;
+
+import com.facebook.nifty.client.FramedClientConnector;
+import com.facebook.nifty.duplex.TDuplexProtocolFactory;
+import com.facebook.swift.service.ThriftClient;
+import com.facebook.swift.service.ThriftClientConfig;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import io.airlift.units.Duration;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.log.LogFileInfo;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.consensus.server.ConsensusService;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
+import java.io.IOException;
+import java.net.NoRouteToHostException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Maintains a connection to a single peer in a quorum.
+ */
+@NotThreadSafe
+public class QuorumThriftClientAgent {
+  private final static Logger LOG = LoggerFactory.getLogger(QuorumThriftClientAgent.class);
+  private final HServerAddress address;
+
+  private ThriftClient<ConsensusService> thriftClient;
+  private ListenableFuture<ConsensusService> futureConnection;
+  private ThriftClientConfig thriftClientConf;
+  private volatile ConsensusService agent = null;
+  private  int connectionRetry = 1;
+
+  public QuorumThriftClientAgent(String address, int connectionTimeout,
+                                 int readTimeout, int writeTimeout,
+                                 int connectionRetry) throws IOException {
+    this.address = new HServerAddress(address);
+    this.thriftClientConf = new ThriftClientConfig()
+      .setConnectTimeout(new Duration(connectionTimeout, TimeUnit.MILLISECONDS))
+      .setReadTimeout(new Duration(readTimeout, TimeUnit.MILLISECONDS))
+      .setWriteTimeout(new Duration(writeTimeout, TimeUnit.MILLISECONDS));
+    this.connectionRetry = connectionRetry;
+  }
+
+  /**
+   * Return the current term if the remote consensus server is the leader,
+   * otherwise return -1;
+   * @param regionId
+   * @return the current term if the remote consensus server is the leader,
+   *         otherwise return -1;
+   */
+  public synchronized String getLeader(final String regionId) throws Exception {
+    final ConsensusService localAgent = getConsensusServiceAgent();
+
+    final SettableFuture<String> result = SettableFuture.create();
+    final Runnable runnable = new Runnable() {
+      @Override public void run() {
+        try {
+          Futures.addCallback(localAgent.getLeader(regionId),
+            new FutureCallback<String>() {
+              @Override
+              public void onSuccess(@Nullable String r) {
+                result.set(r);
+              }
+              @Override
+              public void onFailure(Throwable t) {
+                handleFailure(localAgent, t, result);
+              }
+            });
+        } catch (Exception e) {
+          LOG.error(String.format("%s. Cannot send replicate commit to %s",
+            regionId, address.getHostAddressWithPort()));
+          handleFailure(localAgent, e, result);
+        }
+      }
+    };
+    executeRequest(localAgent, runnable, result);
+    return result.get();
+  }
+
+  /**
+   * Replicates a list of WALEdits, on a given quorum.
+   * @param regionId The region where we want to replicate these edits.
+   * @param txns The actual edits
+   * @return The commit index of the committed edits
+   * @throws Exception
+   */
+  public synchronized long replicateCommit(final String regionId,
+                                           final List<WALEdit> txns)
+    throws Exception {
+    final ConsensusService localAgent = getConsensusServiceAgent();
+
+    final SettableFuture<Long> result = SettableFuture.create();
+    final Runnable runnable = new Runnable() {
+      @Override
+      public void run() {
+        try {
+          Futures.addCallback(localAgent.replicateCommit(regionId, txns),
+            new FutureCallback<Long>() {
+              @Override
+              public void onSuccess(@Nullable Long r) {
+                result.set(r);
+              }
+              @Override
+              public void onFailure(Throwable t) {
+                handleFailure(localAgent, t, result);
+              }
+            });
+        } catch (Exception e) {
+          LOG.error(String.format("%s. Cannot send replicate commit to %s",
+            regionId, address.getHostAddressWithPort()));
+          handleFailure(localAgent, e, result);
+        }
+      }
+    };
+    executeRequest(localAgent, runnable, result);
+    return result.get();
+  }
+
+  public synchronized boolean changeQuorum(final String regionId,
+                                           final ByteBuffer config)
+    throws Exception {
+    final ConsensusService localAgent = getConsensusServiceAgent();
+    final SettableFuture<Boolean> result = SettableFuture.create();
+    final Runnable runnable = new Runnable() {
+      @Override public void run() {
+        try {
+          Futures.addCallback(localAgent.changeQuorum(regionId, config),
+            new FutureCallback<Boolean>() {
+              @Override
+              public void onSuccess(@Nullable Boolean r) {
+                result.set(r);
+              }
+              @Override
+              public void onFailure(Throwable t) {
+                handleFailure(localAgent, t, result);
+              }
+            });
+        } catch (Exception e) {
+          LOG.error(String.format(
+            "%s Cannot send the request to change the " +
+            " quorum to server %s", regionId,
+            address.getHostAddressWithPort()));
+          handleFailure(localAgent, e, result);
+        }
+      }
+    };
+    executeRequest(localAgent, runnable, result);
+    return result.get();
+  }
+
+  public synchronized PeerStatus getPeerStatus(final String regionId) throws Exception {
+    final ConsensusService localAgent = getConsensusServiceAgent();
+    final SettableFuture<PeerStatus> result = SettableFuture.create();
+    final Runnable runnable = new Runnable() {
+      @Override public void run() {
+        try {
+          Futures.addCallback(localAgent.getPeerStatus(regionId),
+            new FutureCallback<PeerStatus>() {
+              @Override
+              public void onSuccess(@Nullable PeerStatus r) {
+                result.set(r);
+              }
+              @Override
+              public void onFailure(Throwable t) {
+                handleFailure(localAgent, t, result);
+              }
+            });
+        } catch (Exception e) {
+          LOG.error(String.format("%s. Cannot send replicate commit to %s",
+            regionId, address.getHostAddressWithPort()));
+          handleFailure(localAgent, e, result);
+        }
+      }
+    };
+    executeRequest(localAgent, runnable, result);
+    return result.get();
+  }
+
+  public synchronized List<LogFileInfo> getCommittedLogStatus(final String quorumName,
+                                                 final long minIndex)
+    throws Exception {
+    final ConsensusService localAgent = getConsensusServiceAgent();
+    final SettableFuture<List<LogFileInfo>> result = SettableFuture.create();
+    final Runnable runnable = new Runnable() {
+      @Override public void run() {
+        try {
+          Futures.addCallback(localAgent.getCommittedLogStatus(quorumName, minIndex),
+            new FutureCallback<List<LogFileInfo>>() {
+              @Override
+              public void onSuccess(@Nullable List<LogFileInfo> r) {
+                result.set(r);
+              }
+              @Override
+              public void onFailure(Throwable t) {
+                handleFailure(localAgent, t, result);
+              }
+            });
+        } catch (Exception e) {
+          LOG.error(String.format("%s. Cannot send replicate commit to %s",
+            quorumName, address.getHostAddressWithPort()));
+          handleFailure(localAgent, e, result);
+        }
+      }
+    };
+    executeRequest(localAgent, runnable, result);
+    return result.get();
+  }
+
+  public synchronized List<PeerStatus> getAllPeerStatuses() throws Exception {
+    final ConsensusService localAgent = getConsensusServiceAgent();
+    final SettableFuture<List<PeerStatus>> result = SettableFuture.create();
+    final Runnable runnable = new Runnable() {
+      @Override public void run() {
+        try {
+          Futures.addCallback(localAgent.getAllPeerStatuses(),
+            new FutureCallback<List<PeerStatus>>() {
+              @Override
+              public void onSuccess(@Nullable List<PeerStatus> r) {
+                result.set(r);
+              }
+              @Override
+              public void onFailure(Throwable t) {
+                handleFailure(localAgent, t, result);
+              }
+            });
+        } catch (Exception e) {
+          LOG.error(String.format("Cannot send replicate commit to %s",
+            address.getHostAddressWithPort()));
+          handleFailure(localAgent, e, result);
+        }
+      }
+    };
+    executeRequest(localAgent, runnable, result);
+    return result.get();
+  }
+
+  public synchronized String getServerAddress() {
+    return this.address.getHostNameWithPort();
+  }
+
+  private ConsensusService getConsensusServiceAgent()
+    throws NoRouteToHostException {
+    if (agent == null) {
+      synchronized (this) {
+        if (agent == null) {
+          for (int i = 0; i < connectionRetry; i++) {
+            try {
+              resetConnection();
+              if (agent != null) {
+                LOG.debug(String
+                  .format("New connection established to server %s.",
+                    getServerAddress()));
+                return agent;
+              }
+            } catch (Throwable t) {
+              LOG.debug(String.format("Exception occurred while resetting the" +
+                " connection to server %s. Error %s", getServerAddress(), t));
+              if (i == connectionRetry - 1) {
+                agent = null;
+                throw new NoRouteToHostException(String.format("Exception" +
+                  "occurred while resetting the connection to server %s." +
+                  " Error %s", getServerAddress(), t));
+              }
+            }
+          }
+        }
+      }
+    }
+    return agent;
+  }
+
+  private void resetConnection()
+    throws ExecutionException, InterruptedException, TimeoutException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("Resetting connection to %s", getServerAddress()));
+    }
+
+    close();
+    thriftClient = new ThriftClient<>(
+      RaftUtil.getThriftClientManager(), ConsensusService.class, thriftClientConf,
+      this.toString());
+    futureConnection = thriftClient.open(new FramedClientConnector(
+      address.getInetSocketAddress(),
+      TDuplexProtocolFactory.fromSingleFactory(
+        new TCompactProtocol.Factory())));
+    agent = futureConnection.get();
+    if (agent == null) {
+      LOG.error(String.format("Failed to resetConnection() to %s.",
+        getServerAddress()));
+    }
+  }
+
+  public void close() {
+    if (agent != null) {
+      try {
+        agent.close();
+      } catch (Exception e) {
+        LOG.error(String.format("Unable to close the agent for server %s.",
+          getServerAddress()), e);
+      }
+    }
+  }
+
+  private void executeRequest(final ConsensusService localAgent, final Runnable call,
+                              final SettableFuture<?> result) {
+    try {
+      RaftUtil.getThriftClientManager().getNiftyChannel(localAgent)
+        .executeInIoThread(call);
+    } catch (Exception e) {
+      handleFailure(localAgent, e, result);
+    }
+  }
+
+  private void handleFailure(final ConsensusService localAgent, final Throwable t,
+                             final SettableFuture<?> future) {
+    future.setException(t);
+
+    if (!RaftUtil.isNetworkError(t)) {
+      return;
+    }
+    LOG.error(String.format("Ran into error while talking to %s.",
+      address.getHostAddressWithPort()), t);
+    synchronized (this) {
+      if (agent == localAgent) {
+        agent = null;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumThriftClientCLI.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumThriftClientCLI.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumThriftClientCLI.java
new file mode 100644
index 0000000..04399f1
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/QuorumThriftClientCLI.java
@@ -0,0 +1,72 @@
+package org.apache.hadoop.hbase.consensus.client;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+
+import com.facebook.swift.service.ThriftClient;
+import com.facebook.swift.service.ThriftClientConfig;
+import com.facebook.swift.service.ThriftClientManager;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.consensus.server.ConsensusService;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+
+import java.io.IOException;
+import java.util.List;
+
+
+public class QuorumThriftClientCLI {
+
+  public static void main(String[] args) throws Exception {
+
+    Options opts = new Options();
+    Option opt;
+    opt = new Option("r", "region", true, "The region ID");
+    opt.setRequired(true);
+    opts.addOption(opt);
+
+    opt = new Option("s", "servers", true, "A comma-separated list of address:port");
+    opt.setRequired(true);
+    opts.addOption(opt);
+
+    opt = new Option("t", "timeout", true, "Controls connection, read and write timeouts");
+    opts.addOption(opt);
+
+    try {
+      CommandLine cmd = new GnuParser().parse(opts, args);
+
+      String serverList = cmd.getOptionValue("servers");
+      String regionId = regionId = cmd.getOptionValue("region");
+      String [] servers = serverList.split(",");
+
+      int connectionRetry = 3;
+      int timeout = Integer.parseInt(cmd.getOptionValue("timeout", "5000"));
+      for (String server : servers) {
+        server = server.trim();
+        System.out.println("Getting QuorumThriftClientAgent for " + server);
+        QuorumThriftClientAgent agent = new QuorumThriftClientAgent(
+            server, timeout, timeout, timeout, connectionRetry);
+        System.out.println("QuorumThriftClientAgent for " + server + " = " + agent);
+        PeerStatus status = agent.getPeerStatus(regionId);
+        System.out.println("PeerStatus for " + server + " : " + status);
+      }
+    } catch (ParseException ex) {
+      System.err.println("Failed to parse the command line: " + ex);
+      ex.printStackTrace();
+      printHelp(opts);
+      System.exit(1);
+    }
+  }
+
+  private static void printHelp(Options opts) {
+    new HelpFormatter().printHelp(
+      "QuorumLoadTestClient -r regionID -s h1:port,...,h3:port", opts
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/CommitQueueOverloadedException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/CommitQueueOverloadedException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/CommitQueueOverloadedException.java
new file mode 100644
index 0000000..47d830a
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/CommitQueueOverloadedException.java
@@ -0,0 +1,9 @@
+package org.apache.hadoop.hbase.consensus.exceptions;
+
+import org.apache.hadoop.hbase.regionserver.RegionOverloadedException;
+
+public class CommitQueueOverloadedException extends RegionOverloadedException {
+  public CommitQueueOverloadedException(final String msg, long waitMillis) {
+    super(msg, waitMillis);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/LeaderNotReadyException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/LeaderNotReadyException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/LeaderNotReadyException.java
new file mode 100644
index 0000000..ad30cbc
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/LeaderNotReadyException.java
@@ -0,0 +1,9 @@
+package org.apache.hadoop.hbase.consensus.exceptions;
+
+import java.io.IOException;
+
+public class LeaderNotReadyException extends IOException {
+  public LeaderNotReadyException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/NewLeaderException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/NewLeaderException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/NewLeaderException.java
new file mode 100644
index 0000000..e73d537
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/NewLeaderException.java
@@ -0,0 +1,20 @@
+package org.apache.hadoop.hbase.consensus.exceptions;
+
+import java.io.IOException;
+
+/**
+ * Thrown by the ConsensusState whenever a request is made to a non Leader
+ * State.
+ */
+public class NewLeaderException extends IOException {
+  private String newLeaderAddress;
+
+  public NewLeaderException(String newLeaderAddress) {
+    super("The new leader in the quorum is " + newLeaderAddress);
+    this.newLeaderAddress = newLeaderAddress;
+  }
+
+  public String getNewLeaderAddress() {
+    return newLeaderAddress;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/NotEnoughMemoryException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/NotEnoughMemoryException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/NotEnoughMemoryException.java
new file mode 100644
index 0000000..0e79bbe4
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/exceptions/NotEnoughMemoryException.java
@@ -0,0 +1,16 @@
+package org.apache.hadoop.hbase.consensus.exceptions;
+
+import java.io.IOException;
+
+public class NotEnoughMemoryException extends IOException {
+
+  final int requiredBytes;
+  public NotEnoughMemoryException(String msg, int bytes) {
+    super(msg);
+    this.requiredBytes = bytes;
+  }
+
+  public int getRequiredBytes() {
+    return requiredBytes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Conditional.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Conditional.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Conditional.java
new file mode 100644
index 0000000..aabe9f9
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Conditional.java
@@ -0,0 +1,9 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+/**
+ * A Conditional is used in the State Machine to check if an event meets a
+ * particular condition before making a transition.
+ */
+public interface Conditional {
+  boolean isMet(final Event e);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/ConstitutentFSMService.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/ConstitutentFSMService.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/ConstitutentFSMService.java
new file mode 100644
index 0000000..e68b58e
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/ConstitutentFSMService.java
@@ -0,0 +1,178 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.serial.SerialExecutorService.SerialExecutionStream;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Implementation of the FiniteStateMachineService where we have multiple
+ * state machines being handled by a single thread. Used along with a
+ * SerialExecutionStream object, which handles the multiplexing of the
+ * events of multiple state machines.
+ */
+public class ConstitutentFSMService implements FiniteStateMachineService {
+  private static final Log LOG = LogFactory.getLog(ConstitutentFSMService.class);
+  private static final int MAX_EVENTS = 10;
+  private static final long EVENT_PROCESSING_LATENCY_OUTLIER_THRESHOLD = 1000;
+  private static final long MAX_TIME_TO_SPEND_IN_ASYNC_MS = 2000;
+  FiniteStateMachine finiteStateMachine;
+  SerialExecutionStream executionStream;
+
+  private final AtomicInteger countPending = new AtomicInteger();
+
+  // For all practical purposes, when this flag is set to true, we assume the
+  // FSM to be shutdown / terminated. The multiplexer will lazily remove this
+  // FSM from its queue.
+  volatile boolean shutdownRequested = false;
+
+  public ConstitutentFSMService(FiniteStateMachine fsm,
+                                SerialExecutionStream executionStream) {
+    this.finiteStateMachine = fsm;
+    this.executionStream = executionStream;
+  }
+
+  protected void handleEvent(Event e) {
+    finiteStateMachine.handleEvent(e);
+  }
+
+  @Override
+  public boolean isShutdown() {
+    return shutdownRequested;
+  }
+
+  @Override
+  public boolean isTerminated() {
+    return shutdownRequested;
+  }
+
+  @Override
+  public String getName() {
+    return finiteStateMachine.getName();
+  }
+
+  @Override
+  public State getCurrentState() {
+    return finiteStateMachine.getCurrentState();
+  }
+
+  @Override
+  public boolean offer(final Event e) {
+    if (shutdownRequested) {
+      return false;
+    }
+    if (countPending.incrementAndGet() > MAX_EVENTS) {
+      LOG.warn(String.format("%s has %d pending events in queue. Current" +
+          " event:%s and current state:%s",
+        finiteStateMachine.getName(), countPending.get(), e,
+        finiteStateMachine.getCurrentState()));
+    }
+
+    Callable<ListenableFuture<Void>> command = new Callable<ListenableFuture<Void>>() {
+      @Override
+      public ListenableFuture<Void> call() {
+        final long start = EnvironmentEdgeManager.currentTime();
+        countPending.decrementAndGet();
+
+        final String currentState = finiteStateMachine.getCurrentState().toString();
+        handleEvent(e);
+        ListenableFuture<?> future = finiteStateMachine.getAsyncCompletion();
+        final long delta = EnvironmentEdgeManager.currentTime() - start;
+
+        if (delta > EVENT_PROCESSING_LATENCY_OUTLIER_THRESHOLD) {
+          LOG.warn(String.format("%s took %d ms from %s to %s.",
+            finiteStateMachine.getName(), delta, currentState,
+            finiteStateMachine.getCurrentState()));
+        }
+        if (future == null) {
+          completeAsyncState();
+          return null;
+        } else {
+          final long timestamp = System.currentTimeMillis();
+          final SettableFuture<Void> readyForComplete = SettableFuture.create();
+          Futures.addCallback(future, new FutureCallback<Object>() {
+            @Override
+            public void onFailure(Throwable t) {
+              long taken = System.currentTimeMillis() - timestamp;
+              if (taken > MAX_TIME_TO_SPEND_IN_ASYNC_MS) {
+                LOG.error("action took too long to fail " + taken + ", event we handled: " + e
+                    + ", current state: " + currentState);
+              }
+              if (!completeAsyncState()) {
+                LOG.fatal("async state couldn't be completed after future completion");
+              }
+              LOG.error("Future failure ", t);
+              readyForComplete.setException(t);
+            }
+            @Override
+            public void onSuccess(Object result) {
+              long taken = System.currentTimeMillis() - timestamp;
+              if (taken > MAX_TIME_TO_SPEND_IN_ASYNC_MS) {
+                LOG.error("action took too long to succeed " + taken + ", event we handled: " + e
+                    + ", current state: " + currentState);
+              }
+
+              if (!completeAsyncState()) {
+                LOG.fatal("async state couldn't be completed after future completion");
+              }
+              readyForComplete.set(null);
+            }
+          });
+          return readyForComplete;
+        }
+      }
+
+      @Override
+      public String toString() {
+        return "Command for event " + e;
+      }
+    };
+    executionStream.execute(command);
+    return true;
+  }
+
+  @Override
+  public void shutdown() {
+    // Setting the shutdownRequest = true will let the multiplexer lazily
+    // reap the FSM when the event queue has drained.
+    shutdownRequested = true;
+  }
+
+  @Override
+  public void shutdownNow() {
+    // There is no executor to call shutdownNow() on.
+    shutdown();
+  }
+
+  @Override
+  public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
+    // We cannot actually terminate the executor, since other FSMs might be
+    // using it.
+    return true;
+  }
+
+  @Override
+  public int getNumPendingEvents() {
+    return countPending.get();
+  }
+
+  protected boolean completeAsyncState() {
+    if (finiteStateMachine.isInAnIncompleteAsyncState()) {
+      return false;
+    }
+
+    if (!shutdownRequested && finiteStateMachine.isInACompletedAsyncState()) {
+      finiteStateMachine.finishAsyncStateCompletionHandling();
+    }
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Event.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Event.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Event.java
new file mode 100644
index 0000000..cc7475a
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Event.java
@@ -0,0 +1,52 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class represents an event within a state machine.
+ */
+public class Event {
+  protected static Logger LOG = LoggerFactory.getLogger(Event.class);
+
+  protected EventType t;
+
+  public Event(final EventType t) {
+    this.t = t;
+  }
+
+  public EventType getEventType() {
+    return t;
+  }
+
+  /**
+   * This method is called when the event is aborted. If your event has state,
+   * such as a future, make sure to override this method to handle that state
+   * appropriately on an abort.
+   * @param message
+   */
+  public void abort(final String message) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(String.format("Aborted %s event: %s", this, message));
+    }
+  }
+
+  @Override
+  public String toString() {
+    return t.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    boolean equals = false;
+    if (this == o) {
+      equals = true;
+    } else {
+      if (o instanceof Event) {
+        Event that = (Event)o;
+        equals = this.t.equals(that.t);
+      }
+    }
+    return equals;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/EventType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/EventType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/EventType.java
new file mode 100644
index 0000000..8cbafc5
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/EventType.java
@@ -0,0 +1,3 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+public interface EventType {}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FSMLargeOpsExecutorService.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FSMLargeOpsExecutorService.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FSMLargeOpsExecutorService.java
new file mode 100644
index 0000000..246bd8e
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FSMLargeOpsExecutorService.java
@@ -0,0 +1,91 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+
+/**
+ * This is an executor service, which has a thread pool to execute IO heavy
+ * parts of state transition code (the {@link State#onEntry(Event)} methods).
+ */
+public class FSMLargeOpsExecutorService {
+  public static ListeningExecutorService fsmWriteOpsExecutorService = null;
+  public static ListeningExecutorService fsmReadOpsExecutorService = null;
+
+  public static synchronized void initialize(Configuration config) {
+    if (fsmWriteOpsExecutorService == null) {
+      fsmWriteOpsExecutorService = createWriteOpsExecutorService(config);
+    }
+
+    if (fsmReadOpsExecutorService == null) {
+      fsmReadOpsExecutorService = createReadOpsExecutorService(config);
+    }
+  }
+
+  /**
+   * This method should only be used for testing
+   */
+  public static synchronized void initializeForTesting(
+    ExecutorService writeOpsService, ExecutorService readOpsService) {
+    fsmWriteOpsExecutorService = MoreExecutors.listeningDecorator(writeOpsService);
+    fsmReadOpsExecutorService = MoreExecutors.listeningDecorator(readOpsService);
+  }
+
+  public static ListeningExecutorService createWriteOpsExecutorService(
+    Configuration conf) {
+    return MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(
+      conf.getInt(HConstants.FSM_WRITEOPS_THREADPOOL_SIZE_KEY,
+        HConstants.FSM_WRITEOPS_THREADPOOL_SIZE_DEFAULT),
+      new DaemonThreadFactory("fsmWriteOpsExecutor")));
+  }
+
+  public static ListeningExecutorService createReadOpsExecutorService(
+    Configuration conf) {
+    return MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(
+      conf.getInt(HConstants.FSM_READOPS_THREADPOOL_SIZE_KEY,
+        HConstants.FSM_READOPS_THREADPOOL_SIZE_DEFAULT),
+      new DaemonThreadFactory("fsmReadOpsExecutor")));
+  }
+
+  private static ListenableFuture<?> submit(ListeningExecutorService executorService, Runnable r) {
+    // We should be calling initialize() in the RaftQuorumContext constructor,
+    // but just in case we don't initialize the thread pool with the defaults.
+    if (executorService == null) {
+      initialize(new Configuration());
+    }
+    return executorService.submit(r);
+  }
+
+  public static ListenableFuture<?> submitToWriteOpsThreadPool(Runnable r) {
+    return submit(fsmWriteOpsExecutorService, r);
+  }
+
+  public static ListenableFuture<?>  submitToReadOpsThreadPool(Runnable r) {
+    return submit(fsmReadOpsExecutorService, r);
+  }
+
+  private static ListenableFuture<?> submit(ListeningExecutorService executorService, Callable<?> c) {
+    // We should be calling initialize() in the RaftQuorumContext constructor,
+    // but just in case we don't initialize the thread pool with the defaults.
+    if (executorService == null) {
+      initialize(new Configuration());
+    }
+    return executorService.submit(c);
+  }
+
+  public static ListenableFuture<?> submitToWriteOpsThreadPool(Callable<?> c) {
+    return submit(fsmWriteOpsExecutorService, c);
+  }
+
+  public static ListenableFuture<?>  submitToReadOpsThreadPool(Callable<?> c) {
+    return submit(fsmReadOpsExecutorService, c);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FSMMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FSMMetrics.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FSMMetrics.java
new file mode 100644
index 0000000..84b28d9
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FSMMetrics.java
@@ -0,0 +1,24 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import org.apache.hadoop.hbase.metrics.MetricsBase;
+import org.weakref.jmx.MBeanExporter;
+
+import java.util.Collections;
+
+public class FSMMetrics extends MetricsBase {
+  /** Type string used when exporting an MBean for these metrics */
+  public static final String TYPE = "FSMMetrics";
+  /** Domain string used when exporting an MBean for these metrics */
+  public static final String DOMAIN = "org.apache.hadoop.hbase.consensus.fsm";
+
+  String name;
+  String procId;
+
+  public FSMMetrics(final String name, final String procId,
+    final MBeanExporter exporter) {
+      super(DOMAIN, TYPE, name, procId, Collections.<String, String>emptyMap(),
+        exporter);
+      this.name = name;
+      this.procId = procId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachine.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachine.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachine.java
new file mode 100644
index 0000000..963e797
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachine.java
@@ -0,0 +1,200 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * This is an implementation of a finite state machine. The state machine also
+ * handles async states.
+ */
+public class FiniteStateMachine implements FiniteStateMachineIf {
+  protected static Logger LOG = LoggerFactory.getLogger(
+          FiniteStateMachine.class);
+
+  private String name;
+  private State currentState;
+  private HashMap<State, HashMap<Transition, State>> stateTransitionMap;
+  private boolean needsHandlingOnAsyncTaskCompletion = false;
+
+  public FiniteStateMachine(final String name) {
+    this.name = name;
+    this.stateTransitionMap = new HashMap<>();
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public State getCurrentState() {
+    return currentState;
+  }
+
+  @Override
+  public String toString() {
+    return "[" + this.getClass().getSimpleName() + ": " + name + ", state: " +
+            currentState + "]";
+  }
+
+  @Override
+  public void addTransition(final State s1, final State s2,
+                            final Transition t) {
+    HashMap<Transition, State> transitionMap = getTransitionMap(s1);
+    if (transitionMap == null) {
+      transitionMap = new HashMap<>();
+    }
+    transitionMap.put(t, s2);
+    stateTransitionMap.put(s1, transitionMap);
+  }
+
+  @Override
+  public boolean setStartState(State s) {
+    if (currentState == null) {
+      currentState = s;
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public boolean isInAnIncompleteAsyncState() {
+    return currentState.isAsyncState() && !currentState.isComplete();
+  }
+
+  @Override
+  public boolean isInACompletedAsyncState() {
+    return currentState.isAsyncState() && currentState.isComplete();
+  }
+
+  @Override
+  public void finishAsyncStateCompletionHandling() {
+    if (isInACompletedAsyncState()) {
+      applyValidConditionalTransitions();
+    }
+  }
+
+  /**
+   * Check if we can apply any conditional transitions. This is usually
+   * used when an async state finishes, and any valid conditional transitions
+   * that were not applicable earlier, can be applied now.
+   */
+  private void applyValidConditionalTransitions() {
+    if (isInACompletedAsyncState() && needsHandlingOnAsyncTaskCompletion) {
+      needsHandlingOnAsyncTaskCompletion = false;
+      State nextState = getNextState(null);
+      if (nextState != null) {
+        currentState = nextState;
+      }
+    }
+  }
+
+  @Override
+  public void handleEvent(final Event e) {
+    try {
+      State nextState = getNextState(e);
+      if (nextState != null) {
+        currentState = nextState;
+      } else {
+        e.abort(String.format("No transition found from state %s:%s", getName(),
+          currentState));
+      }
+    } catch (Exception ex) {
+      LOG.warn(String.format("%s ran into an error while processing event " +
+        "%s. Current State : %s", this.getName(), e, currentState), ex);
+    }
+  }
+
+  @Override
+  public ListenableFuture<?> getAsyncCompletion() {
+    if (currentState.isAsyncState()) {
+      Assert.assertTrue(needsHandlingOnAsyncTaskCompletion);
+      return currentState.getAsyncCompletion();
+    }
+    return null;
+  }
+
+  public boolean waitForAsyncToComplete(long timeout, TimeUnit unit) {
+    if (isInAnIncompleteAsyncState()) {
+      Future<?> future = getAsyncCompletion();
+      if (future == null) {
+        return true;
+      }
+      try {
+        future.get(timeout, unit);
+        return true;
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return false;
+      } catch (TimeoutException e) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("Timed out while waiting for the async " +
+            "state %s to complete.", getCurrentState()));
+        }
+        return false;
+      } catch (ExecutionException e) {
+        return true;
+      }
+    }
+    return true;
+  }
+
+  protected HashMap<Transition, State> getTransitionMap(final State s) {
+    return stateTransitionMap.get(s);
+  }
+
+  protected Transition getNextTransition(final State s, final Event e) {
+    if (needsHandlingOnAsyncTaskCompletion) {
+      return null;
+    }
+
+    Transition nextTransition = null;
+    HashMap<Transition, State> transitionMap = getTransitionMap(s);
+    if (transitionMap != null) {
+      Iterator<Entry<Transition, State>> it = transitionMap.entrySet().iterator();
+      while (nextTransition == null && it.hasNext()) {
+        Entry<Transition, State> entry = it.next();
+        Transition t = entry.getKey();
+        if (t.getCondition().isMet(e)) {
+          nextTransition = t;
+        }
+      }
+    }
+    return nextTransition;
+  }
+
+  protected State getNextState(Event e) {
+    State nextState = null;
+
+    Transition nextTransition = getNextTransition(currentState, e);
+    while (nextTransition != null) {
+      nextState = stateTransitionMap.get(currentState).get(nextTransition);
+
+      if (nextState != null && nextState != currentState) {
+        currentState.onExit(e);
+
+        currentState = nextState;
+        if (nextState.isAsyncState()) {
+          needsHandlingOnAsyncTaskCompletion = true;
+        } else {
+          needsHandlingOnAsyncTaskCompletion = false;
+        }
+        nextState.onEntry(e);
+      }
+      e = null; // Only pass the event on the first loop.
+      nextTransition = getNextTransition(nextState, e);
+    }
+    return nextState;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineIf.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineIf.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineIf.java
new file mode 100644
index 0000000..d9d2f66
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineIf.java
@@ -0,0 +1,57 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * An interface for FiniteStateMachine implementations.
+ */
+public interface FiniteStateMachineIf {
+  /**
+   * @return The name of the State Machine.
+   */
+  public String getName();
+
+  /**
+   * @return The current state of the state machine.
+   */
+  public State getCurrentState();
+
+  /**
+   * Set the start state of the state machine.
+   * @param s The starting state.
+   * @return Return false if we could not set the state. Otherwise return true.
+   */
+  public boolean setStartState(State s);
+
+  /**
+   * Add a transition t from the state s1, to the state s2.
+   * @param s1
+   * @param s2
+   * @param t
+   */
+  public void addTransition(final State s1, final State s2, final Transition t);
+
+  /**
+   * Request the FSM to handle a particular Event.
+   * @param e
+   */
+  public void handleEvent(final Event e);
+
+  /**
+   * @return Returns true, if the FSM is in an async state, that has not yet
+   *         completed.
+   */
+  public boolean isInAnIncompleteAsyncState();
+
+  /**
+   * @return Returns true, if the FSM is in an async state, that has completed.
+   */
+  public boolean isInACompletedAsyncState();
+
+  public ListenableFuture<?> getAsyncCompletion();
+
+  /**
+   * Finish the handling of an async state's onEntry() method completion.
+   */
+  public void finishAsyncStateCompletionHandling();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineService.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineService.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineService.java
new file mode 100644
index 0000000..562d2a8
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineService.java
@@ -0,0 +1,67 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import java.util.concurrent.TimeUnit;
+
+public interface FiniteStateMachineService {
+  /**
+   * @return true if this service is shutdown, false otherwise
+   */
+  public boolean isShutdown();
+
+  /**
+   * @return true if the service has completed processing events,
+   * false otherwise.
+   */
+  public boolean isTerminated();
+
+  /**
+   * @return the name of the state machine
+   */
+  public String getName();
+
+  /**
+   * Return the current state of the state machine. Note: this is not thread
+   * safe, so unless this is called from within the state machine this will
+   * return a stale result.
+   *
+   * @return the current state of the state machine
+   */
+  public State getCurrentState();
+
+  /**
+   * Submits an event to the state machine for execution.
+   *
+   * @param e the event to be executed
+   * @return true if the event was added to the event queue, false otherwise
+   */
+  public boolean offer(final Event e);
+
+  /**
+   * Initiates an orderly shutdown in which no new events will be accepted and
+   * the service will shutdown as soon as the currently executing event was
+   * completed.
+   */
+  public void shutdown();
+
+  /**
+   * Attempts to interrupt the actively executing event and halts the process of
+   * waiting for events.
+   */
+  public void shutdownNow();
+
+  /**
+   * Blocks until the current event has completed execution, or the timeout
+   * occurs, or the current thread is interrupted, whichever happens first.
+   * @param timeout the maximum time to wait
+   * @param unit the time unit of the timeout argument
+   * @return true if this service terminated and false if the timeout elapsed before
+   * termination
+   */
+  public boolean awaitTermination(final long timeout, TimeUnit unit)
+          throws InterruptedException;
+
+  /**
+   * @return Return the number of pending events in the FSM's event queue.
+   */
+  public int getNumPendingEvents();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineServiceImpl.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineServiceImpl.java
new file mode 100644
index 0000000..29052db
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/FiniteStateMachineServiceImpl.java
@@ -0,0 +1,132 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A single threaded implementation of the FiniteStateMachineService
+ */
+public class FiniteStateMachineServiceImpl
+        implements FiniteStateMachineService {
+  protected static Logger LOG = LoggerFactory.getLogger(
+          FiniteStateMachineServiceImpl.class);
+
+  private FiniteStateMachine fsm;
+  private ExecutorService executor;
+  protected LinkedBlockingDeque<Event> eventQueue;
+  protected volatile boolean shutdownRequested = false;
+  private static final int SECONDS_TOWAIT_FOR_ASYNC_TO_COMPLETE = 1;
+
+  public FiniteStateMachineServiceImpl(final FiniteStateMachine fsm) {
+    this.fsm = fsm;
+    this.eventQueue = new LinkedBlockingDeque<>();
+    this.executor = Executors.newSingleThreadExecutor(
+            new DaemonThreadFactory(String.format("%s-", fsm.getName())));
+    start();
+  }
+
+  @Override
+  public boolean isShutdown() {
+    return shutdownRequested && executor.isShutdown();
+  }
+
+  @Override
+  public boolean isTerminated() {
+    return shutdownRequested && executor.isTerminated();
+  }
+
+  @Override
+  public String getName() {
+    return fsm.getName();
+  }
+
+  @Override
+  public State getCurrentState() {
+    return fsm.getCurrentState();
+  }
+
+  protected void start() {
+    if (getCurrentState() != null) {
+      executor.submit(new Runnable() {
+        @Override
+        public void run() {
+          while (!(shutdownRequested && eventQueue.isEmpty())) {
+            try {
+              // Check if we are in an incomplete async state. If yes,
+              // we will busy wait in this 'inner loop'.
+              completeAsyncState();
+
+              Event e = eventQueue.poll(200, TimeUnit.MILLISECONDS);
+              if (e != null) {
+                // At this place, we are guaranteed that the state, if async,
+                // is complete, and any conditional transitions, if applicable,
+                // have been taken. We can safely handle this event.
+                fsm.handleEvent(e);
+              }
+            } catch (InterruptedException ex) {
+              // This is most likely caused by a call to shutdownNow on the
+              // executor, meaning shutdown took too long. If this is the case
+              // shutdownRequested will be false on the next cycle and this task
+              // should properly end.
+              Thread.currentThread().interrupt();
+              continue;
+            } catch (AssertionError ex) {
+              LOG.error("Assertion error ", ex);
+              // Terminate the process
+              System.exit(-1);
+            } catch (Throwable ex) {
+              LOG.error("Unexpected exception: ", ex);
+            }
+          }
+        }
+      });
+    }
+  }
+
+  private void completeAsyncState() {
+    while (!shutdownRequested) {
+      if (fsm.waitForAsyncToComplete(
+          SECONDS_TOWAIT_FOR_ASYNC_TO_COMPLETE, TimeUnit.SECONDS)) {
+        break;
+      }
+    }
+    fsm.finishAsyncStateCompletionHandling();
+  }
+
+  @Override
+  public boolean offer(final Event e) {
+    if (!shutdownRequested) {
+      return eventQueue.offer(e);
+    }
+    return false;
+  }
+
+  @Override
+  public void shutdown() {
+    shutdownRequested = true;
+    executor.shutdown();
+  }
+
+  @Override
+  public void shutdownNow() {
+    shutdownRequested = true;
+    executor.shutdownNow();
+  }
+
+  @Override
+  public boolean awaitTermination(final long timeout, TimeUnit unit)
+          throws InterruptedException {
+    return executor.awaitTermination(timeout, unit);
+  }
+
+  @Override
+  public int getNumPendingEvents() {
+    return eventQueue.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/MutableContext.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/MutableContext.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/MutableContext.java
new file mode 100644
index 0000000..5b983ca
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/MutableContext.java
@@ -0,0 +1,3 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+public interface MutableContext {}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/OnEvent.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/OnEvent.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/OnEvent.java
new file mode 100644
index 0000000..338e981
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/OnEvent.java
@@ -0,0 +1,13 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+public class OnEvent implements Conditional {
+  protected EventType t;
+
+  public OnEvent(final EventType t) {
+    this.t = t;
+  }
+
+  public boolean isMet(final Event e) {
+    return e != null && e.getEventType() == t;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/State.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/State.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/State.java
new file mode 100644
index 0000000..6f9ef46
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/State.java
@@ -0,0 +1,67 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+/**
+ * Represents a state in the state machine implementation.
+ */
+public abstract class State {
+  protected StateType t;
+
+  public State(final StateType t) {
+    this.t = t;
+  }
+
+  public StateType getStateType() {
+    return t;
+  }
+
+  @Override
+  public String toString() {
+    return t.toString();
+  }
+
+  abstract public void onEntry(final Event e);
+  abstract public void onExit(final Event e);
+
+  /**
+   * @return Return true if the state is an async state. This means, that the
+   * state machine would invoke the onEntry() method for this state, but it is
+   * possible that the state might not be complete, because we spawned a
+   * background operation, such as writing to disk.
+   */
+  public boolean isAsyncState() {
+    return false;
+  }
+
+  /**
+   * If this state is async, this method returns the future to be completed,
+   * before we can declare an async state to be complete.
+   * @return
+   */
+  public ListenableFuture<?> getAsyncCompletion() {
+    return null;
+  }
+
+  /**
+   * This method dictates whether the onEntry() method hasn't completed
+   * logically, even if the call has actually returned.
+   *
+   * The FSM checks for this method before making any transition. If this method
+   * returns false, then any arriving events will not be applied/aborted, even
+   * if the call to onEntry() method has returned.
+   *
+   * This is a way to do FSM thread-blocking work in an async fashion in the
+   * onEntry method, and make the FSM wait the state to complete before
+   * transitioning off to another state.
+   *
+   * By default, the method returns true, which means the FSM will not wait for
+   * any async ops that you would have issued. States which require waiting,
+   * will need to override this method, and make this method return true when
+   * they are done.
+   * @return
+   */
+  public boolean isComplete() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/StateType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/StateType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/StateType.java
new file mode 100644
index 0000000..921f51c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/StateType.java
@@ -0,0 +1,3 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+public interface StateType {}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Transition.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Transition.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Transition.java
new file mode 100644
index 0000000..45e08e5
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Transition.java
@@ -0,0 +1,33 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+/**
+ * Represents a transition in the state machine.
+ */
+public class Transition {
+  protected TransitionType t;
+  protected Conditional c;
+
+  /**
+   * Create a transition
+   * @param t type of transition
+   * @param c the Conditional object, whose isMet() should return true, for
+   *          this transition to happen.
+   */
+  public Transition(final TransitionType t, final Conditional c) {
+    this.t = t;
+    this.c = c;
+  }
+
+  public TransitionType getTransitionType() {
+    return t;
+  }
+
+  public Conditional getCondition() {
+    return c;
+  }
+
+  @Override
+  public String toString() {
+    return t.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/TransitionType.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/TransitionType.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/TransitionType.java
new file mode 100644
index 0000000..e5b4a07
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/TransitionType.java
@@ -0,0 +1,3 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+public interface TransitionType {}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Unconditional.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Unconditional.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Unconditional.java
new file mode 100644
index 0000000..570e01e
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Unconditional.java
@@ -0,0 +1,9 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+public class Unconditional implements Conditional {
+
+  @Override
+  public boolean isMet(Event e){
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Util.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Util.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Util.java
new file mode 100644
index 0000000..2e3633c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/fsm/Util.java
@@ -0,0 +1,27 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import java.util.concurrent.TimeUnit;
+
+public class Util {
+  public static boolean awaitTermination(
+      final FiniteStateMachineService service,
+      final long shutdownTimeout,
+      final long shutdownNowTimeout,
+      TimeUnit unit) {
+    boolean isTerminated = false;
+    try {
+      isTerminated = service.awaitTermination(shutdownTimeout, unit);
+      if (!isTerminated) {
+        service.shutdownNow();
+        isTerminated = service.awaitTermination(shutdownNowTimeout, unit);
+      }
+    } catch (InterruptedException e) {
+      // Interrupted while waiting for termination. Something is bad, so
+      // interrupt the current event.
+      service.shutdownNow();
+      // Honor the interrupt
+      Thread.currentThread().interrupt();
+    }
+    return isTerminated;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CachedFileChannel.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CachedFileChannel.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CachedFileChannel.java
new file mode 100644
index 0000000..b4c26c5
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/log/CachedFileChannel.java
@@ -0,0 +1,232 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileChannel.MapMode;
+import java.nio.channels.FileLock;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.channels.spi.AbstractInterruptibleChannel;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class CachedFileChannel extends FileChannel {
+  private RandomAccessFile raf;
+  private long startPosMBFF = -1;
+  private long countMBFF = 0;
+  private long curPosition = 0;
+  private MappedByteBuffer mBff = null;
+  private FileChannel fileChannel;
+  private long maxSizeToPrefetch = 1000000;
+  public static final Log LOG = LogFactory.getLog(CachedFileChannel.class);
+
+  public CachedFileChannel(RandomAccessFile raf, long countMBFF) {
+    this.raf = raf;
+    maxSizeToPrefetch = countMBFF;
+
+    this.fileChannel = this.raf.getChannel();
+  }
+
+  /**
+   * Tries to read from the mapped buffer.
+   *
+   * If the requested range is already mapped, we can read it right away. If not,
+   * the method will try to map the portion that is being read, if the size is smaller
+   * than the MAX_SIZE. Otherwise, the requested size is too big, or it is 0, we return
+   * false. (The caller will then have to fetch the data from the fileChannel)
+   *
+   * @param buffer
+   * @param fileOffset
+   * @param size
+   * @return true if the read was successful. false otherwise.
+   * @throws IOException
+   */
+  private boolean tryReadingFromMemoryBuffer(ByteBuffer buffer,
+      long fileOffset, int size) throws IOException {
+    if (size > maxSizeToPrefetch
+        || size <= 0
+        || fileOffset + size > size()) {
+      LOG.debug("Did not preload " + size
+          + "  bytes from log file starting at offset " + fileOffset
+          + ". Memory currently contains: " + this.startPosMBFF + " to "
+          + (this.startPosMBFF + this.countMBFF) + " looking for "
+          + fileOffset + " to " + (fileOffset + size)
+          + " file size is " + size());
+      return false;
+    }
+
+    if (fileOffset < this.startPosMBFF
+        || fileOffset + size > this.startPosMBFF + this.countMBFF ) {
+      loadMemoryBufferFrom(fileOffset);
+    }
+
+    readEntryFromMemoryBuffer(buffer,
+        (int)(fileOffset - this.startPosMBFF),
+        size);
+    this.curPosition = this.startPosMBFF + this.mBff.position();
+
+    return true;
+  }
+
+  /**
+   * Map a portion of the file into the MappedByteBuffer
+   * @param fileOffset offset in the file to map from
+   * @throws IOException when FileChannel.map has issues
+   */
+  private void loadMemoryBufferFrom(long fileOffset) throws IOException {
+    countMBFF = Math.min(maxSizeToPrefetch, size() - fileOffset);
+    startPosMBFF = fileOffset;
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Preloading " + countMBFF
+          + "  bytes from log file starting at offset " + startPosMBFF);
+    }
+    mBff = raf.getChannel().map(MapMode.READ_ONLY, startPosMBFF, countMBFF);
+  }
+
+  /**
+   * Reads the specified data from the mapped buffer into the given buffer.
+   * @param bb buffer to read the data into
+   * @param memOffset memory offset in the mapped buffer to start reading from.
+   *        (THIS is NOT necessarily the offset in the file.)
+   * @param size
+   */
+  private void readEntryFromMemoryBuffer(ByteBuffer bb, int memOffset, int size) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Loading into memory. Memory currently contains: "
+          + startPosMBFF + " to " + (startPosMBFF + countMBFF)
+          + " looking for " + (startPosMBFF + memOffset) + " to "
+          + (startPosMBFF + memOffset + size));
+    }
+    // temporarily modify the limit to the size of data we want to read.
+    int oldLimit = this.mBff.limit();
+    this.mBff.position(memOffset);
+    int newLimit = memOffset + size;
+    this.mBff.limit(newLimit);
+
+    bb.put(mBff);
+
+    // restore old limit
+    this.mBff.limit(oldLimit);
+  }
+
+  @Override
+  public long position() throws IOException {
+    return this.curPosition;
+  }
+
+  @Override
+  public FileChannel position(long newPosition) throws IOException {
+    this.curPosition = newPosition;
+    return this;
+  }
+
+  @Override
+  public long size() throws IOException {
+    return this.fileChannel.size();
+  }
+
+  @Override
+  public int read(ByteBuffer dst) throws IOException {
+    return read(dst, this.curPosition);
+  }
+
+  @Override
+  public int read(ByteBuffer buffer, long offset) throws IOException {
+    int ret = 0;
+    if (!tryReadingFromMemoryBuffer(buffer, offset, buffer.remaining())) {
+      // This is too large to fit in mBFF anyways
+
+      if (offset != fileChannel.position()) {
+        fileChannel.position(offset);
+      }
+
+      ret = fileChannel.read(buffer);
+      this.curPosition = fileChannel.position();
+    }
+    return ret;
+  }
+
+  @Override
+  public long read(ByteBuffer[] buffer, int offset, int length)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "CachedFileChannel does not support this operation");
+  }
+
+  @Override
+  protected void implCloseChannel() throws IOException {
+    this.mBff = null;
+    this.curPosition = 0;
+    this.startPosMBFF = -1;
+    this.countMBFF = 0;
+    fileChannel.close();
+  }
+
+  public RandomAccessFile getRandomAccessFile() {
+    return this.raf;
+  }
+
+  @Override
+  public int write(ByteBuffer src) throws IOException {
+    throw new UnsupportedOperationException("CachedFileChannel is read-only");
+  }
+
+  @Override
+  public long write(ByteBuffer[] srcs, int offset, int length)
+      throws IOException {
+    throw new UnsupportedOperationException("CachedFileChannel is read-only");
+  }
+
+  @Override
+  public FileChannel truncate(long size) throws IOException {
+    throw new UnsupportedOperationException("CachedFileChannel is read-only");
+  }
+
+  @Override
+  public void force(boolean metaData) throws IOException {
+    throw new UnsupportedOperationException("CachedFileChannel is read-only");
+  }
+
+  @Override
+  public long transferTo(long position, long count, WritableByteChannel target)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "CachedFileChannel does not support this operation");
+  }
+
+  @Override
+  public long transferFrom(ReadableByteChannel src, long position, long count)
+      throws IOException {
+    throw new UnsupportedOperationException("CachedFileChannel is read-only");
+  }
+
+  @Override
+  public int write(ByteBuffer src, long position) throws IOException {
+    throw new UnsupportedOperationException("CachedFileChannel is read-only");
+  }
+
+  @Override
+  public MappedByteBuffer map(MapMode mode, long position, long size)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "CachedFileChannel does not support this operation");
+  }
+
+  @Override
+  public FileLock lock(long position, long size, boolean shared)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "CachedFileChannel does not support this operation");
+  }
+
+  @Override
+  public FileLock tryLock(long position, long size, boolean shared)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "CachedFileChannel does not support this operation");
+  }
+}


[17/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HServerAddress.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HServerAddress.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HServerAddress.java
new file mode 100644
index 0000000..b091ad5
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HServerAddress.java
@@ -0,0 +1,243 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * HServerAddress is a "label" for a HBase server made of host and port number.
+ */
+public class HServerAddress implements WritableComparable<HServerAddress> {
+
+  private static final Log LOG = LogFactory.getLog(HServerAddress.class);
+
+  private InetSocketAddress address;
+  private String stringValue;
+  private String hostAddress;
+
+  /** 
+   * We don't expect the IP addresses of HBase servers to change, so we cache them
+   * indefinitely. At this level we only do positive caching.
+   */
+  private static ConcurrentMap<String, InetSocketAddress> addressCache =
+      new ConcurrentHashMap<String, InetSocketAddress>();
+
+  public HServerAddress() {
+    this.address = null;
+    this.stringValue = null;
+    this.hostAddress = null;
+  }
+
+  /**
+   * Construct an instance from an {@link InetSocketAddress}.
+   * @param address InetSocketAddress of server
+   */
+  public HServerAddress(InetSocketAddress address) {
+    this.address = address;
+    this.stringValue = getHostAddressWithPort();
+    checkBindAddressCanBeResolved();
+  }
+
+  /**
+   * @param hostAndPort Hostname and port formatted as <code>&lt;hostname> ':' &lt;port></code>
+   */
+  public HServerAddress(String hostAndPort) {
+    int colonIndex = hostAndPort.lastIndexOf(':');
+    if (colonIndex < 0) {
+      throw new IllegalArgumentException("Not a host:port pair: " + hostAndPort);
+    }
+    String host = hostAndPort.substring(0, colonIndex);
+    int port = Integer.parseInt(hostAndPort.substring(colonIndex + 1));
+    address = addressCache.get(hostAndPort);
+    if (address == null) {
+      this.address = new InetSocketAddress(host, port);
+      if (getBindAddress() != null) {
+        // Resolved the hostname successfully, cache it.
+        InetSocketAddress existingAddress = addressCache.putIfAbsent(hostAndPort, address);
+        if (existingAddress != null) {
+          // Another thread cached the address ahead of us, reuse it.
+          this.address = existingAddress;
+        }
+      }
+    }
+    this.stringValue = getHostAddressWithPort();
+    checkBindAddressCanBeResolved();
+  }
+
+  /**
+   * @param bindAddress Hostname
+   * @param port Port number
+   */
+  public HServerAddress(String bindAddress, int port) {
+    this.address = new InetSocketAddress(bindAddress, port);
+    this.stringValue = getHostAddressWithPort();
+    checkBindAddressCanBeResolved();
+  }
+
+  /**
+   * Copy-constructor.
+   * @param other HServerAddress to copy from
+   */
+  public HServerAddress(HServerAddress other) {
+    String bindAddress = other.getBindAddress();
+    int port = other.getPort();
+    this.address = new InetSocketAddress(bindAddress, port);
+    stringValue = other.stringValue;
+    checkBindAddressCanBeResolved();
+  }
+  
+  /**
+   * Get the normalized hostAddress:port as a string format
+   * @param address
+   * @return the normalized hostAddress:port as a string format
+   */
+  public String getHostAddressWithPort() {
+    if (address == null) return null;
+    return this.getBindAddress() + ":" + address.getPort();
+  }
+
+  /**
+   * Get the normalized hostName:port as a string format
+   * @param address
+   * @return the normalized hostName:port as a string format
+   */
+  public String getHostNameWithPort() {
+    if (address == null) return null;
+    return address.getHostName() + ":" +
+      address.getPort();
+  }
+
+  /** @return Bind address */
+  public String getBindAddress() {
+    if (this.hostAddress != null)
+      return hostAddress;
+    
+    final InetAddress addr = address.getAddress();
+    if (addr != null) {
+      return addr.getHostAddress();
+    } else {
+      LOG.error("Could not resolve the DNS name of " + stringValue);
+      return null;
+    }
+  }
+
+  private void checkBindAddressCanBeResolved() {
+    if ((this.hostAddress = getBindAddress()) == null) {
+      throw new IllegalArgumentException("Could not resolve the"
+          + " DNS name of " + stringValue);
+    }
+  }
+
+  /** @return Port number */
+  public int getPort() {
+    return address.getPort();
+  }
+
+  /** @return Hostname */
+  public String getHostname() {
+    return address.getHostName();
+  }
+
+  /** @return The InetSocketAddress */
+  public InetSocketAddress getInetSocketAddress() {
+    return address;
+  }
+
+  /**
+   * @return String formatted as <code>&lt;bind address> ':' &lt;port></code>
+   */
+  @Override
+  public String toString() {
+    return stringValue == null ? "" : stringValue;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null) {
+      return false;
+    }
+    if (getClass() != o.getClass()) {
+      return false;
+    }
+    return compareTo((HServerAddress) o) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return address.hashCode();
+  }
+
+  //
+  // Writable
+  //
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    String bindAddress = in.readUTF();
+    int port = in.readInt();
+
+    if (bindAddress == null || bindAddress.length() == 0) {
+      address = null;
+      stringValue = null;
+    } else {
+      address = new InetSocketAddress(bindAddress, port);
+      stringValue = getHostAddressWithPort();
+      checkBindAddressCanBeResolved();
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    if (address == null) {
+      out.writeUTF("");
+      out.writeInt(0);
+    } else {
+      out.writeUTF(address.getAddress().getHostAddress());
+      out.writeInt(address.getPort());
+    }
+  }
+
+  //
+  // Comparable
+  //
+
+  @Override
+  public int compareTo(HServerAddress o) {
+    if (address == null) return -1;
+    // Addresses as Strings may not compare though address is for the one
+    // server with only difference being that one address has hostname
+    // resolved whereas other only has IP.
+    if (address.equals(o.address)) return 0;
+    return toString().compareTo(o.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
new file mode 100644
index 0000000..380dfaa
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -0,0 +1,792 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * HTableDescriptor contains the name of an HTable, and its
+ * column families.
+ */
+public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
+
+  // Changes prior to version 3 were not recorded here.
+  // Version 3 adds metadata as a map where keys and values are byte[].
+  // Version 4 adds indexes
+  // Version 5 removed transactional pollution -- e.g. indexes
+  public static final byte TABLE_DESCRIPTOR_VERSION = 5;
+  
+  private byte [] name = HConstants.EMPTY_BYTE_ARRAY;
+  private String nameAsString = "";
+
+  // Table metadata
+  protected final Map<ImmutableBytesWritable, ImmutableBytesWritable> values =
+    new HashMap<ImmutableBytesWritable, ImmutableBytesWritable>();
+
+  public static final String FAMILIES = "FAMILIES";
+  public static final ImmutableBytesWritable FAMILIES_KEY =
+    new ImmutableBytesWritable(Bytes.toBytes(FAMILIES));
+  public static final String MAX_FILESIZE = "MAX_FILESIZE";
+  public static final ImmutableBytesWritable MAX_FILESIZE_KEY =
+    new ImmutableBytesWritable(Bytes.toBytes(MAX_FILESIZE));
+  public static final String READONLY = "READONLY";
+  public static final ImmutableBytesWritable READONLY_KEY =
+    new ImmutableBytesWritable(Bytes.toBytes(READONLY));
+  public static final String MEMSTORE_FLUSHSIZE = "MEMSTORE_FLUSHSIZE";
+  public static final ImmutableBytesWritable MEMSTORE_FLUSHSIZE_KEY =
+    new ImmutableBytesWritable(Bytes.toBytes(MEMSTORE_FLUSHSIZE));
+  public static final String IS_ROOT = "IS_ROOT";
+  public static final ImmutableBytesWritable IS_ROOT_KEY =
+    new ImmutableBytesWritable(Bytes.toBytes(IS_ROOT));
+  public static final String IS_META = "IS_META";
+
+  public static final ImmutableBytesWritable IS_META_KEY =
+    new ImmutableBytesWritable(Bytes.toBytes(IS_META));
+
+  public static final String DEFERRED_LOG_FLUSH = "DEFERRED_LOG_FLUSH";
+  public static final ImmutableBytesWritable DEFERRED_LOG_FLUSH_KEY =
+    new ImmutableBytesWritable(Bytes.toBytes(DEFERRED_LOG_FLUSH));
+
+  public static final String DISABLE_WAL = "DISABLE_WAL";
+  public static final ImmutableBytesWritable DISABLE_WAL_KEY =
+    new ImmutableBytesWritable(Bytes.toBytes(DISABLE_WAL));
+
+
+  // The below are ugly but better than creating them each time till we
+  // replace booleans being saved as Strings with plain booleans.  Need a
+  // migration script to do this.  TODO.
+  private static final ImmutableBytesWritable FALSE =
+    new ImmutableBytesWritable(Bytes.toBytes(Boolean.FALSE.toString()));
+  private static final ImmutableBytesWritable TRUE =
+    new ImmutableBytesWritable(Bytes.toBytes(Boolean.TRUE.toString()));
+
+  public static final boolean DEFAULT_READONLY = false;
+
+  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = 1024*1024*64L;
+
+  public static final long DEFAULT_MEMSTORE_COLUMNFAMILY_FLUSH_SIZE =
+    1024*1024*16L;
+
+  public static final long DEFAULT_MAX_FILESIZE = 1024*1024*256L;
+
+  public static final boolean DEFAULT_DEFERRED_LOG_FLUSH = true;
+
+  private final static Map<String, String> DEFAULT_VALUES
+    = new HashMap<String, String>();
+  private final static Set<ImmutableBytesWritable> RESERVED_KEYWORDS
+    = new HashSet<ImmutableBytesWritable>();
+  static {
+    DEFAULT_VALUES.put(MAX_FILESIZE, String.valueOf(DEFAULT_MAX_FILESIZE));
+    DEFAULT_VALUES.put(READONLY, String.valueOf(DEFAULT_READONLY));
+    DEFAULT_VALUES.put(MEMSTORE_FLUSHSIZE,
+        String.valueOf(DEFAULT_MEMSTORE_FLUSH_SIZE));
+    DEFAULT_VALUES.put(DEFERRED_LOG_FLUSH,
+        String.valueOf(DEFAULT_DEFERRED_LOG_FLUSH));
+    for (String s : DEFAULT_VALUES.keySet()) {
+      RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(s)));
+    }
+    RESERVED_KEYWORDS.add(IS_ROOT_KEY);
+    RESERVED_KEYWORDS.add(IS_META_KEY);
+  }
+
+  private volatile Boolean meta = null;
+  private volatile Boolean root = null;
+  private Boolean isDeferredLog = null;
+
+  // Key is hash of the family name.
+  public final Map<byte [], HColumnDescriptor> families =
+    new TreeMap<byte [], HColumnDescriptor>(Bytes.BYTES_RAWCOMPARATOR);
+
+  /**
+   * Private constructor used internally creating table descriptors for
+   * catalog tables: e.g. .META. and -ROOT-.
+   */
+  public HTableDescriptor(final byte [] name, HColumnDescriptor[] families) {
+    this.name = name.clone();
+    this.nameAsString = Bytes.toString(this.name);
+    setMetaFlags(name);
+    for(HColumnDescriptor descriptor : families) {
+      this.families.put(descriptor.getName(), descriptor);
+    }
+  }
+
+  /**
+   * Private constructor used internally creating table descriptors for
+   * catalog tables: e.g. .META. and -ROOT-.
+   */
+  protected HTableDescriptor(final byte [] name, HColumnDescriptor[] families,
+      Map<ImmutableBytesWritable,ImmutableBytesWritable> values) {
+    this.name = name.clone();
+    this.nameAsString = Bytes.toString(this.name);
+    setMetaFlags(name);
+    for(HColumnDescriptor descriptor : families) {
+      this.families.put(descriptor.getName(), descriptor);
+    }
+    for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> entry:
+        values.entrySet()) {
+      this.values.put(entry.getKey(), entry.getValue());
+    }
+  }
+
+
+  /**
+   * Constructs an empty object.
+   * For deserializing an HTableDescriptor instance only.
+   * @see #HTableDescriptor(byte[])
+   */
+  public HTableDescriptor() {
+    super();
+  }
+
+  /**
+   * Constructor.
+   * @param name Table name.
+   * @throws IllegalArgumentException if passed a table name
+   * that is made of other than 'word' characters, underscore or period: i.e.
+   * <code>[a-zA-Z_0-9.].
+   * @see <a href="HADOOP-1581">HADOOP-1581 HBASE: Un-openable tablename bug</a>
+   */
+  public HTableDescriptor(final String name) {
+    this(Bytes.toBytes(name));
+  }
+
+  /**
+   * Constructor.
+   * @param name Table name.
+   * @throws IllegalArgumentException if passed a table name
+   * that is made of other than 'word' characters, underscore or period: i.e.
+   * <code>[a-zA-Z_0-9-.].
+   * @see <a href="HADOOP-1581">HADOOP-1581 HBASE: Un-openable tablename bug</a>
+   */
+  public HTableDescriptor(final byte [] name) {
+    super();
+    setMetaFlags(name);
+    this.name = this.isMetaRegion()? name: isLegalTableName(name);
+    this.nameAsString = Bytes.toString(this.name);
+  }
+
+  /**
+   * Constructor.
+   * <p>
+   * Makes a deep copy of the supplied descriptor.
+   * Can make a modifiable descriptor from an UnmodifyableHTableDescriptor.
+   * @param desc The descriptor.
+   */
+  public HTableDescriptor(final HTableDescriptor desc) {
+    super();
+    this.name = desc.name.clone();
+    this.nameAsString = Bytes.toString(this.name);
+    setMetaFlags(this.name);
+    for (HColumnDescriptor c: desc.families.values()) {
+      this.families.put(c.getName(), new HColumnDescriptor(c));
+    }
+    for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
+        desc.values.entrySet()) {
+      this.values.put(e.getKey(), e.getValue());
+    }
+  }
+
+  /*
+   * Set meta flags on this table.
+   * Called by constructors.
+   * @param name
+   */
+  private void setMetaFlags(final byte [] name) {
+    setRootRegion(Bytes.equals(name, HConstants.ROOT_TABLE_NAME));
+    setMetaRegion(isRootRegion() ||
+      Bytes.equals(name, HConstants.META_TABLE_NAME));
+  }
+
+  /** @return true if this is the root region */
+  public boolean isRootRegion() {
+    if (this.root == null) {
+      this.root = isSomething(IS_ROOT_KEY, false)? Boolean.TRUE: Boolean.FALSE;
+    }
+    return this.root.booleanValue();
+  }
+
+  /** @param isRoot true if this is the root region */
+  protected void setRootRegion(boolean isRoot) {
+    // TODO: Make the value a boolean rather than String of boolean.
+    values.put(IS_ROOT_KEY, isRoot? TRUE: FALSE);
+  }
+
+  /** @return true if this is a meta region (part of the root or meta tables) */
+  public boolean isMetaRegion() {
+    if (this.meta == null) {
+      this.meta = calculateIsMetaRegion();
+    }
+    return this.meta.booleanValue();
+  }
+
+  private synchronized Boolean calculateIsMetaRegion() {
+    byte [] value = getValue(IS_META_KEY);
+    return (value != null)? Boolean.valueOf(Bytes.toString(value)): Boolean.FALSE;
+  }
+
+  private boolean isSomething(final ImmutableBytesWritable key,
+      final boolean valueIfNull) {
+    byte [] value = getValue(key);
+    if (value != null) {
+      // TODO: Make value be a boolean rather than String of boolean.
+      return Boolean.valueOf(Bytes.toString(value)).booleanValue();
+    }
+    return valueIfNull;
+  }
+
+  /**
+   * @param isMeta true if this is a meta region (part of the root or meta
+   * tables) */
+  protected void setMetaRegion(boolean isMeta) {
+    values.put(IS_META_KEY, isMeta? TRUE: FALSE);
+  }
+
+  /** @return true if table is the meta table */
+  public boolean isMetaTable() {
+    return isMetaRegion() && !isRootRegion();
+  }
+
+  /**
+   * Check passed buffer is legal user-space table name.
+   * @param b Table name.
+   * @return Returns passed <code>b</code> param
+   * @throws NullPointerException If passed <code>b</code> is null
+   * @throws IllegalArgumentException if passed a table name
+   * that is made of other than 'word' characters or underscores: i.e.
+   * <code>[a-zA-Z_0-9].
+   */
+  public static byte [] isLegalTableName(final byte [] b) {
+    if (b == null || b.length <= 0) {
+      throw new IllegalArgumentException("Name is null or empty");
+    }
+    if (b[0] == '.' || b[0] == '-') {
+      throw new IllegalArgumentException("Illegal first character <" + b[0] +
+          "> at 0. User-space table names can only start with 'word " +
+          "characters': i.e. [a-zA-Z_0-9]: " + Bytes.toString(b));
+    }
+    for (int i = 0; i < b.length; i++) {
+      if (Character.isLetterOrDigit(b[i]) || b[i] == '_' || b[i] == '-' ||
+          b[i] == '.') {
+        continue;
+      }
+      throw new IllegalArgumentException("Illegal character <" + b[i] +
+        "> at " + i + ". User-space table names can only contain " +
+        "'word characters': i.e. [a-zA-Z_0-9-.]: " + Bytes.toString(b));
+    }
+    return b;
+  }
+
+  /**
+   * @param key The key.
+   * @return The value.
+   */
+  public byte[] getValue(byte[] key) {
+    return getValue(new ImmutableBytesWritable(key));
+  }
+
+  private byte[] getValue(final ImmutableBytesWritable key) {
+    ImmutableBytesWritable ibw = values.get(key);
+    if (ibw == null)
+      return null;
+    return ibw.get();
+  }
+
+  /**
+   * @param key The key.
+   * @return The value as a string.
+   */
+  public String getValue(String key) {
+    byte[] value = getValue(Bytes.toBytes(key));
+    if (value == null)
+      return null;
+    return Bytes.toString(value);
+  }
+
+  /**
+   * @return All values.
+   */
+  public Map<ImmutableBytesWritable,ImmutableBytesWritable> getValues() {
+    // shallow pointer copy
+    return Collections.unmodifiableMap(values);
+  }
+
+  /**
+   * @param key The key.
+   * @param value The value.
+   */
+  public void setValue(byte[] key, byte[] value) {
+    setValue(new ImmutableBytesWritable(key), value);
+  }
+
+  /*
+   * @param key The key.
+   * @param value The value.
+   */
+  private void setValue(final ImmutableBytesWritable key,
+      final byte[] value) {
+    values.put(key, new ImmutableBytesWritable(value));
+  }
+
+  /*
+   * @param key The key.
+   * @param value The value.
+   */
+  private void setValue(final ImmutableBytesWritable key,
+      final ImmutableBytesWritable value) {
+    values.put(key, value);
+  }
+
+  /**
+   * @param key The key.
+   * @param value The value.
+   */
+  public void setValue(String key, String value) {
+    if (value == null) {
+      remove(Bytes.toBytes(key));
+    } else {
+      setValue(Bytes.toBytes(key), Bytes.toBytes(value));
+    }
+  }
+
+  /**
+   * @param key Key whose key and value we're to remove from HTD parameters.
+   */
+  public void remove(final byte [] key) {
+    values.remove(new ImmutableBytesWritable(key));
+  }
+
+  /**
+   * @return true if all columns in the table should be read only
+   */
+  public boolean isReadOnly() {
+    return isSomething(READONLY_KEY, DEFAULT_READONLY);
+  }
+
+  /**
+   * @param readOnly True if all of the columns in the table should be read
+   * only.
+   */
+  public void setReadOnly(final boolean readOnly) {
+    setValue(READONLY_KEY, readOnly? TRUE: FALSE);
+  }
+
+  /**
+   * @return true if that table's log is hflush by other means
+   */
+  public synchronized boolean isDeferredLogFlush() {
+    if(this.isDeferredLog == null) {
+      this.isDeferredLog =
+          isSomething(DEFERRED_LOG_FLUSH_KEY, DEFAULT_DEFERRED_LOG_FLUSH);
+    }
+    return this.isDeferredLog;
+  }
+
+  /**
+   * @param isDeferredLogFlush true if that table's log is hlfush by oter means
+   * only.
+   */
+  public void setDeferredLogFlush(final boolean isDeferredLogFlush) {
+    setValue(DEFERRED_LOG_FLUSH_KEY, isDeferredLogFlush? TRUE: FALSE);
+  }
+
+  /** @return name of table */
+  public byte [] getName() {
+    return name;
+  }
+
+  /** @return name of table */
+  public String getNameAsString() {
+    return this.nameAsString;
+  }
+
+  /** @return max hregion size for table */
+  public long getMaxFileSize() {
+    byte [] value = getValue(MAX_FILESIZE_KEY);
+    if (value != null)
+      return Long.valueOf(Bytes.toString(value)).longValue();
+    return HConstants.DEFAULT_MAX_FILE_SIZE;
+  }
+
+  /** @param name name of table */
+  public void setName(byte[] name) {
+    this.name = name;
+  }
+
+  /**
+   * @param maxFileSize The maximum file size that a store file can grow to
+   * before a split is triggered.
+   */
+  public void setMaxFileSize(long maxFileSize) {
+    setValue(MAX_FILESIZE_KEY, Bytes.toBytes(Long.toString(maxFileSize)));
+  }
+
+  /**
+   * @return memory cache flush size for each hregion
+   */
+  public long getMemStoreFlushSize() {
+    byte [] value = getValue(MEMSTORE_FLUSHSIZE_KEY);
+    if (value != null)
+      return Long.valueOf(Bytes.toString(value)).longValue();
+    return DEFAULT_MEMSTORE_FLUSH_SIZE;
+  }
+
+  /**
+   * @param memstoreFlushSize memory cache flush size for each hregion
+   */
+  public void setMemStoreFlushSize(long memstoreFlushSize) {
+    setValue(MEMSTORE_FLUSHSIZE_KEY,
+      Bytes.toBytes(Long.toString(memstoreFlushSize)));
+  }
+
+  /**
+   * Adds a column family.
+   * @param family HColumnDescriptor of familyto add.
+   */
+  public void addFamily(final HColumnDescriptor family) {
+    if (family.getName() == null || family.getName().length <= 0) {
+      throw new NullPointerException("Family name cannot be null or empty");
+    }
+    this.families.put(family.getName(), family);
+  }
+
+  /**
+   * Checks to see if this table contains the given column family
+   * @param c Family name or column name.
+   * @return true if the table contains the specified family name
+   */
+  public boolean hasFamily(final byte [] c) {
+    return families.containsKey(c);
+  }
+
+  /**
+   * @return Name of this table and then a map of all of the column family
+   * descriptors.
+   * @see #getNameAsString()
+   */
+  @Override
+  public String toString() {
+    StringBuilder s = new StringBuilder();
+    s.append('\'').append(Bytes.toString(name)).append('\'');
+    s.append(getValues(true));
+    for (HColumnDescriptor f : families.values()) {
+      s.append(", ").append(f);
+    }
+    return s.toString();
+  }
+
+  public String toStringCustomizedValues() {
+    StringBuilder s = new StringBuilder();
+    s.append('\'').append(Bytes.toString(name)).append('\'');
+    s.append(getValues(false));
+    for(HColumnDescriptor hcd : families.values()) {
+      s.append(", ").append(hcd.toStringCustomizedValues());
+    }
+    return s.toString();
+  }
+
+  private StringBuilder getValues(boolean printDefaults) {
+    StringBuilder s = new StringBuilder();
+
+    // step 1: set partitioning and pruning
+    Set<ImmutableBytesWritable> reservedKeys = new TreeSet<ImmutableBytesWritable>();
+    Set<ImmutableBytesWritable> configKeys = new TreeSet<ImmutableBytesWritable>();
+    for (ImmutableBytesWritable k : values.keySet()) {
+      if (!RESERVED_KEYWORDS.contains(k)) {
+        configKeys.add(k);
+        continue;
+      }
+      // only print out IS_ROOT/IS_META if true
+      String key = Bytes.toString(k.get());
+      String value = Bytes.toString(values.get(k).get());
+      if (key.equalsIgnoreCase(IS_ROOT) || key.equalsIgnoreCase(IS_META)) {
+        if (Boolean.valueOf(value) == false) continue;
+      }
+      if (printDefaults
+          || !DEFAULT_VALUES.containsKey(key)
+          || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
+        reservedKeys.add(k);
+      }
+    }
+
+    // early exit optimization
+    if (reservedKeys.isEmpty() && configKeys.isEmpty()) return s;
+
+    // step 2: printing
+    s.append(", {METHOD => 'table_att'");
+
+    // print all reserved keys first
+    for (ImmutableBytesWritable k : reservedKeys) {
+      String key = Bytes.toString(k.get());
+      String value = Bytes.toString(values.get(k).get());
+      s.append(", ");
+      s.append(key);
+      s.append(" => ");
+      s.append('\'').append(value).append('\'');
+    }
+
+    if (!configKeys.isEmpty()) {
+      // print all non-reserved, advanced config keys as a separate subset
+      s.append(", ");
+      s.append(HConstants.CONFIG).append(" => ");
+      s.append("{");
+      boolean printComma = false;
+      for (ImmutableBytesWritable k : configKeys) {
+        String key = Bytes.toString(k.get());
+        String value = Bytes.toString(values.get(k).get());
+        if (printComma) s.append(", ");
+        printComma = true;
+        s.append('\'').append(key).append('\'');
+        s.append(" => ");
+        s.append('\'').append(value).append('\'');
+      }
+      s.append('}');
+    }
+
+    s.append('}'); // end METHOD
+
+    return s;
+  }
+
+  public static Map<String, String> getDefaultValues() {
+    return Collections.unmodifiableMap(DEFAULT_VALUES);
+  }
+
+  /**
+   * @see java.lang.Object#equals(java.lang.Object)
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof HTableDescriptor)) {
+      return false;
+    }
+    return compareTo((HTableDescriptor)obj) == 0;
+  }
+
+  /**
+   * @see java.lang.Object#hashCode()
+   */
+  @Override
+  public int hashCode() {
+    int result = Bytes.hashCode(this.name);
+    result ^= Byte.valueOf(TABLE_DESCRIPTOR_VERSION).hashCode();
+    if (this.families != null && this.families.size() > 0) {
+      for (HColumnDescriptor e: this.families.values()) {
+        result ^= e.hashCode();
+      }
+    }
+    result ^= values.hashCode();
+    return result;
+  }
+
+  // Writable
+
+  public void readFields(DataInput in) throws IOException {
+    int version = in.readInt();
+    if (version < 3)
+      throw new IOException("versions < 3 are not supported (and never existed!?)");
+    // version 3+
+    name = Bytes.readByteArray(in);
+    nameAsString = Bytes.toString(this.name);
+    setRootRegion(in.readBoolean());
+    setMetaRegion(in.readBoolean());
+    values.clear();
+    int numVals = in.readInt();
+    for (int i = 0; i < numVals; i++) {
+      ImmutableBytesWritable key = new ImmutableBytesWritable();
+      ImmutableBytesWritable value = new ImmutableBytesWritable();
+      key.readFields(in);
+      value.readFields(in);
+      values.put(key, value);
+    }
+    families.clear();
+    int numFamilies = in.readInt();
+    for (int i = 0; i < numFamilies; i++) {
+      HColumnDescriptor c = new HColumnDescriptor();
+      c.readFields(in);
+      families.put(c.getName(), c);
+    }
+    if (version < 4) {
+      return;
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(TABLE_DESCRIPTOR_VERSION);
+    Bytes.writeByteArray(out, name);
+    out.writeBoolean(isRootRegion());
+    out.writeBoolean(isMetaRegion());
+    out.writeInt(values.size());
+    for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
+        values.entrySet()) {
+      e.getKey().write(out);
+      e.getValue().write(out);
+    }
+    out.writeInt(families.size());
+    for(Iterator<HColumnDescriptor> it = families.values().iterator();
+        it.hasNext(); ) {
+      HColumnDescriptor family = it.next();
+      family.write(out);
+    }
+  }
+
+  // Comparable
+
+  public int compareTo(final HTableDescriptor other) {
+    int result = Bytes.compareTo(this.name, other.name);
+    if (result == 0) {
+      result = families.size() - other.families.size();
+    }
+    if (result == 0 && families.size() != other.families.size()) {
+      result = Integer.valueOf(families.size()).compareTo(
+          Integer.valueOf(other.families.size()));
+    }
+    if (result == 0) {
+      for (Iterator<HColumnDescriptor> it = families.values().iterator(),
+          it2 = other.families.values().iterator(); it.hasNext(); ) {
+        result = it.next().compareTo(it2.next());
+        if (result != 0) {
+          break;
+        }
+      }
+    }
+    if (result == 0) {
+      // punt on comparison for ordering, just calculate difference
+      result = this.values.hashCode() - other.values.hashCode();
+      if (result < 0)
+        result = -1;
+      else if (result > 0)
+        result = 1;
+    }
+    return result;
+  }
+
+  /**
+   * @return Immutable sorted map of families.
+   */
+  public Collection<HColumnDescriptor> getFamilies() {
+    return Collections.unmodifiableCollection(this.families.values());
+  }
+
+  /**
+   * @return Immutable sorted set of the keys of the families.
+   */
+  public Set<byte[]> getFamiliesKeys() {
+    return Collections.unmodifiableSet(this.families.keySet());
+  }
+
+  public HColumnDescriptor[] getColumnFamilies() {
+    return getFamilies().toArray(new HColumnDescriptor[0]);
+  }
+
+  /**
+   * @param column
+   * @return Column descriptor for the passed family name or the family on
+   * passed in column.
+   */
+  public HColumnDescriptor getFamily(final byte [] column) {
+    return this.families.get(column);
+  }
+
+  /**
+   * @param column
+   * @return Column descriptor for the passed family name or the family on
+   * passed in column.
+   */
+  public HColumnDescriptor removeFamily(final byte [] column) {
+    return this.families.remove(column);
+  }
+
+  /**
+   * @param rootdir qualified path of HBase root directory
+   * @param tableName name of table
+   * @return path for table
+   */
+  public static Path getTableDir(Path rootdir, final byte [] tableName) {
+    return new Path(rootdir, Bytes.toString(tableName));
+  }
+
+  /** Table descriptor for <core>-ROOT-</code> catalog table */
+  public static final HTableDescriptor ROOT_TABLEDESC = new HTableDescriptor(
+      HConstants.ROOT_TABLE_NAME,
+      new HColumnDescriptor[] {
+          new HColumnDescriptor(HConstants.CATALOG_FAMILY)
+              // Ten is arbitrary number.  Keep versions to help debugging.
+              .setMaxVersions(10)
+              .setInMemory(true)
+              .setBlocksize(8 * 1024)
+              .setTimeToLive(HConstants.FOREVER)
+              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+      });
+
+  /** Table descriptor for <code>.META.</code> catalog table */
+  public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor(
+      HConstants.META_TABLE_NAME, new HColumnDescriptor[] {
+          new HColumnDescriptor(HConstants.CATALOG_FAMILY)
+              // Ten is arbitrary number.  Keep versions to help debugging.
+              .setMaxVersions(10)
+              .setInMemory(true)
+              .setBlocksize(8 * 1024)
+              .setScope(HConstants.REPLICATION_SCOPE_LOCAL),
+          new HColumnDescriptor(HConstants.CATALOG_HISTORIAN_FAMILY)
+              .setMaxVersions(HConstants.ALL_VERSIONS)
+              .setBlocksize(8 * 1024)
+              .setTimeToLive(HConstants.WEEK_IN_SECONDS)
+              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+      });
+
+  /**
+   * @return true if all columns in the table should be read only
+   */
+  public boolean isWALDisabled() {
+    return isSomething(DISABLE_WAL_KEY, false);
+  }
+
+  /**
+   * @param readOnly True if all of the columns in the table should be read
+   * only.
+   */
+  public void setWALDisabled(final boolean disable) {
+    setValue(DISABLE_WAL_KEY, disable? TRUE: FALSE);
+  }
+
+}


[08/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/HDFSReader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/HDFSReader.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/HDFSReader.java
new file mode 100644
index 0000000..7d6b0f7
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/HDFSReader.java
@@ -0,0 +1,140 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class HDFSReader extends RMapReader {
+  protected static final Logger LOG = LoggerFactory.getLogger(HDFSReader.class);
+
+  private Configuration conf;
+
+  public HDFSReader(final Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public List<Long> getVersions(URI uri) throws IOException {
+    Path path = new Path(getSchemeAndPath(uri));
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus[] statuses = fs.globStatus(new Path(path.toString() + ".*"));
+
+    List<Long> versions = new ArrayList<>(statuses.length);
+    for (FileStatus status : statuses) {
+      long version = getVersionFromPath(status.getPath().toString());
+      if (version > 0) {
+        versions.add(version);
+      }
+    }
+    Collections.sort(versions);
+    return versions;
+  }
+
+  @Override
+  public URI resolveSymbolicVersion(URI uri) throws URISyntaxException {
+    long version = getVersion(uri);
+    String schemeAndPath = getSchemeAndPath(uri);
+
+    if (version == RMapReader.CURRENT || version == RMapReader.NEXT) {
+      Path link = new Path(String.format("%s.%s", schemeAndPath,
+              version == RMapReader.CURRENT ? "CURRENT" : "NEXT"));
+      // Resolve to an explicit version, or UNKNOWN
+      try {
+        Path target = getLinkTarget(link);
+        version = target != null ? getVersionFromPath(target.toString()) :
+                RMapReader.UNKNOWN;
+      } catch (IOException e) {
+        LOG.error("Failed to look up version from link:", e);
+        version = RMapReader.UNKNOWN;
+      }
+    }
+
+    if (version > 0) {
+      return new URI(String.format("%s?version=%d", schemeAndPath, version));
+    }
+    return new URI(schemeAndPath);
+  }
+
+  @Override
+  public String readRMapAsString(final URI uri) throws IOException {
+    // Get file status, throws IOException if the path does not exist.
+    Path path = getPathWithVersion(uri);
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus status = fs.getFileStatus(path);
+
+    long n = status.getLen();
+    if (n < 0 || n > MAX_SIZE_BYTES) {
+      throw new IOException(String.format("Invalid RMap file size " +
+              "(expected between 0 and %d but got %d bytes)",
+              MAX_SIZE_BYTES, n));
+    }
+
+    byte[] buf = new byte[(int)n];
+    FSDataInputStream stream = fs.open(path);
+    try {
+      stream.readFully(buf);
+    } finally {
+      stream.close();
+    }
+    return Bytes.toString(buf);
+  }
+
+  public Path getPathWithVersion(final URI uri) throws IOException {
+    long version = RMapReader.UNKNOWN;
+    try {
+      version = getVersion(resolveSymbolicVersion(uri));
+    } catch (URISyntaxException e) {
+      // Ignore invalid URIs and assume version UNKNOWN
+    }
+
+    if (version > 0) {
+      return new Path(String.format("%s.%d", getSchemeAndPath(uri), version));
+    }
+    return new Path(uri.toString());
+  }
+
+  private long getVersionFromPath(final String path) {
+    String[] tokens = path.split("[\\.]");
+    try {
+      return Long.parseLong(tokens[tokens.length - 1]);
+    } catch (NumberFormatException e) {
+      // Skip if token not numerical
+    }
+    return RMapReader.UNKNOWN;
+  }
+
+  private Path getLinkTarget(final Path path) throws IOException {
+    FileSystem fs = path.getFileSystem(conf);
+
+    // The getHardLinkedFiles call is a bit tricky, as it effectively returns
+    // all other paths to the inode shared with the given path. In order to
+    // guard against erroneous links, only consider those where the paths
+    // are the same, up to the version.
+    String pathWithoutVersion = path.toString().substring(0,
+            path.toString().lastIndexOf('.'));
+    /*
+TODO: FIXME: Amit: this code works with the internal hdfs. might not work with the
+OSS version.
+
+    for (String link : fs.getHardLinkedFiles(path)) {
+      if (path.toString().startsWith(pathWithoutVersion) &&
+              getVersionFromPath(link) > 0) {
+        return new Path(link);
+      }
+    }
+    */
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/LocalReader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/LocalReader.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/LocalReader.java
new file mode 100644
index 0000000..fc1e877
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/LocalReader.java
@@ -0,0 +1,96 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class LocalReader extends RMapReader {
+  protected static final Logger LOG = LoggerFactory.getLogger(
+          LocalReader.class);
+
+  @Override
+  public List<Long> getVersions(final URI uri) throws IOException {
+    Path path = Paths.get(uri);
+    List<Long> versions = new ArrayList<>();
+
+    for (Path match : Files.newDirectoryStream(path.getParent(),
+            path.getFileName() + ".*")) {
+      long version = getVersionFromPath(match.toString());
+      if (version > 0) {
+        versions.add(version);
+      }
+    }
+    Collections.sort(versions);
+    return versions;
+  }
+
+  @Override
+  public URI resolveSymbolicVersion(URI uri) throws URISyntaxException {
+    long version = getVersion(uri);
+    String schemeAndPath = getSchemeAndPath(uri);
+
+    if (version == RMapReader.CURRENT || version == RMapReader.NEXT) {
+      Path link = Paths.get(String.format("%s.%s", schemeAndPath,
+              version == RMapReader.CURRENT ? "CURRENT" : "NEXT"));
+      // Resolve to an explicit version, or UNKNOWN
+      try {
+        version = getVersionFromPath(Files.readSymbolicLink(link).toString());
+      } catch (IOException e) {
+        LOG.error("Failed to look up version from link:", e);
+        version = RMapReader.UNKNOWN;
+      }
+    }
+
+    if (version > 0) {
+      return new URI(String.format("%s?version=%d", schemeAndPath, version));
+    }
+    return new URI(schemeAndPath);
+  }
+
+  @Override
+  public String readRMapAsString(final URI uri) throws IOException {
+    Path path = getPathWithVersion(uri);
+
+    long n = Files.size(path);
+    if (n < 0 || n > MAX_SIZE_BYTES) {
+      throw new IOException(String.format("Invalid RMap file size " +
+              "(expected between 0 and %d but got %d bytes)",
+              MAX_SIZE_BYTES, n));
+    }
+
+    return new String(Files.readAllBytes(path));
+  }
+
+  private long getVersionFromPath(final String path) {
+    String[] tokens = path.split("[\\.]");
+    try {
+      return Long.parseLong(tokens[tokens.length - 1]);
+    } catch (NumberFormatException e) {
+      // Skip if token not numerical
+    }
+    return RMapReader.UNKNOWN;
+  }
+
+  private Path getPathWithVersion(final URI uri) {
+    long version = RMapReader.UNKNOWN;
+    try {
+      version = getVersion(resolveSymbolicVersion(uri));
+    } catch (URISyntaxException e) {
+      // Ignore invalid URIs and assume version UNKNOWN
+    }
+
+    if (version > 0) {
+      return Paths.get(String.format("%s.%d", uri.getPath(), version));
+    }
+    return Paths.get(uri);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/NoSuchRMapException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/NoSuchRMapException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/NoSuchRMapException.java
new file mode 100644
index 0000000..6136063
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/NoSuchRMapException.java
@@ -0,0 +1,10 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import java.io.IOException;
+import java.net.URI;
+
+public class NoSuchRMapException extends IOException {
+  public NoSuchRMapException(final URI uri) {
+    super("No RMap found with URI " + uri);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/Parser.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/Parser.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/Parser.java
new file mode 100644
index 0000000..f345b1a
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/Parser.java
@@ -0,0 +1,146 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import java.net.InetSocketAddress;
+import java.util.*;
+
+public class Parser {
+  private Configuration conf;
+
+  public Parser(final Configuration conf) {
+    this.conf = conf;
+  }
+
+  public List<HRegionInfo> parseEncodedRMap(JSONObject encodedRMap)
+          throws JSONException {
+    List<HRegionInfo> regions = new ArrayList<>();
+    JSONObject tables = encodedRMap.getJSONObject("tables");
+
+    for (Iterator<String> names = tables.keys(); names.hasNext();) {
+      String name = names.next();
+      regions.addAll(parseTable(name, tables.getJSONObject(name)));
+    }
+
+    return regions;
+  }
+
+  public List<HRegionInfo> parseTable(String name, JSONObject table)
+          throws JSONException {
+    HTableDescriptor tableDesc = new HTableDescriptor(name);
+    List<HRegionInfo> regions = Collections.emptyList();
+    Iterator<String> keys = table.keys();
+    while (keys.hasNext()) {
+      String key = keys.next();
+      if (key.equals("families")) {
+        JSONObject families = table.getJSONObject(key);
+        Iterator<String> familyKeys = families.keys();
+        while (familyKeys.hasNext()) {
+          String familyName = familyKeys.next();
+          JSONObject familyJson = families.getJSONObject(familyName);
+          tableDesc.addFamily(parseFamily(familyName, familyJson));
+        }
+      } else if (key.equals("regions")) {
+        JSONArray regionsJson = table.getJSONArray(key);
+        int length = regionsJson.length();
+        regions = new ArrayList<>(length);
+        for (int i = 0; i < length; ++i) {
+          regions.add(parseRegion(tableDesc, regionsJson.getJSONObject(i)));
+        }
+      } else {
+        String value = table.get(key).toString();
+        tableDesc.setValue(key, value);
+      }
+    }
+    return regions;
+  }
+
+  public HColumnDescriptor parseFamily(String name, JSONObject family)
+          throws JSONException {
+    HColumnDescriptor columnDesc = new HColumnDescriptor();
+    columnDesc.setName(Bytes.toBytes(name));
+    Iterator<String> keys = family.keys();
+    while (keys.hasNext()) {
+      String key = keys.next();
+      String value = family.get(key).toString();
+      columnDesc.setValue(key, value);
+    }
+    return columnDesc;
+  }
+
+  public HRegionInfo parseRegion(HTableDescriptor table, JSONObject region)
+          throws JSONException {
+    long id = region.getLong("id");
+    byte[] startKey = Bytes.toBytes(region.getString("start_key"));
+    byte[] endKey = Bytes.toBytes(region.getString("end_key"));
+    Map<String, Map<HServerAddress, Integer>> peers = parsePeers(region
+            .getJSONObject("peers"));
+    Map<String, InetSocketAddress[]> favoredNodesMap = parseFavoredNodesMap(region
+            .getJSONObject("favored_nodes"));
+    return new HRegionInfo(table, startKey, endKey, false, id, peers,
+            favoredNodesMap);
+  }
+
+  public Map<String, Map<HServerAddress, Integer>> parsePeers(JSONObject peersJson)
+          throws JSONException {
+    Map<String, Map<HServerAddress, Integer>> peers = new LinkedHashMap<>();
+    Iterator<String> keys = peersJson.keys();
+    while (keys.hasNext()) {
+      String cellName = keys.next();
+      JSONArray peersWithRank = peersJson.getJSONArray(cellName);
+      peers.put(cellName, parsePeersWithRank(peersWithRank));
+    }
+    return peers;
+  }
+
+  public Map<HServerAddress, Integer> parsePeersWithRank(JSONArray peersJson)
+          throws JSONException {
+    Map<HServerAddress, Integer> peers = new LinkedHashMap<HServerAddress, Integer>();
+    for (int i = 0; i < peersJson.length(); ++i) {
+      String peer = peersJson.getString(i);
+      int colonIndex = peer.lastIndexOf(':');
+      peers.put(new HServerAddress(peer.substring(0, colonIndex)),
+              Integer.valueOf(peer.substring(colonIndex + 1)));
+    }
+    return peers;
+  }
+
+  Map<String, InetSocketAddress[]> parseFavoredNodesMap(JSONObject favoredNodesJson)
+          throws JSONException {
+    Iterator<String> keys = favoredNodesJson.keys();
+
+    HashMap<String, InetSocketAddress[]> favoredNodesMap = new HashMap<>();
+    while (keys.hasNext()) {
+      String cellName = keys.next();
+      JSONArray peersWithRank = favoredNodesJson.getJSONArray(cellName);
+      favoredNodesMap.put(cellName, parseFavoredNodes(peersWithRank));
+    }
+    return favoredNodesMap;
+  }
+
+  public InetSocketAddress[] parseFavoredNodes(JSONArray favoredNodesInCell)
+          throws JSONException {
+    if (favoredNodesInCell == null) {
+      return null;
+    } else {
+      int length = favoredNodesInCell.length();
+      InetSocketAddress[] favoredNodes = new InetSocketAddress[length];
+      for (int i = 0; i < length; ++i) {
+        String node = favoredNodesInCell.getString(i);
+        int colonIndex = node.lastIndexOf(':');
+        favoredNodes[i] = new InetSocketAddress(node.substring(0, colonIndex),
+                Integer.parseInt(node.substring(colonIndex + 1)));
+
+      }
+      return favoredNodes;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapConfiguration.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapConfiguration.java
new file mode 100644
index 0000000..00306dc
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapConfiguration.java
@@ -0,0 +1,330 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.json.JSONException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+public class RMapConfiguration {
+  private static final Logger LOG = LoggerFactory.getLogger(RMapConfiguration.class);
+
+  private Configuration conf;
+
+  private Map<String, RMap> appliedRMaps;
+  private Map<URI, RMap> cachedRMaps;
+
+  public RMapConfiguration(final Configuration conf) {
+    this.conf = conf;
+    this.appliedRMaps = new HashMap<>();
+    this.cachedRMaps = new HashMap<>();
+  }
+
+  public static URI getRMapSubscription(final Configuration conf) {
+    String[] subscriptionsList =
+      conf.get(HConstants.RMAP_SUBSCRIPTION, "").split(",");
+    if (subscriptionsList.length >= 1) {
+      if (subscriptionsList.length > 1) {
+        LOG.warn(String.format("We do not support multiple RMaps. " +
+          "Using the first RMap as the correct one: %s", subscriptionsList[0]));
+      }
+      else if (!subscriptionsList[0].equals("")) {
+        try {
+          return new URI(subscriptionsList[0]);
+        } catch (URISyntaxException e) {
+          LOG.warn(String.format("Failed to parse URI for subscription %s: ",
+            subscriptionsList[0]), e);
+        }
+      }
+    }
+    return null;
+  }
+
+  public static RMapReader getRMapReader(final Configuration conf,
+          final URI uri) throws RMapException {
+    switch (uri.getScheme()) {
+      case "file":
+        return new LocalReader();
+      case "hdfs":
+        return new HDFSReader(conf);
+      default:
+        throw new RMapException("No reader found for RMap: " + uri);
+    }
+  }
+
+  public synchronized RMap getRMap(URI uri)
+      throws IOException, RMapException {
+    return getRMap(uri, false);
+  }
+
+  public synchronized RMap getRMap(URI uri, boolean reload)
+      throws IOException, RMapException {
+    try {
+      RMapReader reader = getRMapReader(conf, uri);
+      URI nonSymbolicURI = reader.resolveSymbolicVersion(uri);
+      // Try to get a cached instance of the RMap.
+      RMap rmap = cachedRMaps.get(nonSymbolicURI);
+      if (reload || rmap == null) {
+        // No cached instance was found, read it using the reader.
+        RMapJSON encodedRMap = reader.readRMap(nonSymbolicURI);
+        rmap = new RMap(encodedRMap.uri,
+            new Parser(conf).parseEncodedRMap(encodedRMap.getEncodedRMap()),
+            encodedRMap.signature);
+        cachedRMaps.put(rmap.uri, rmap);
+      }
+      return rmap;
+    } catch (URISyntaxException e) {
+      throw new RMapException("URI syntax invalid for RMap: " + uri, e);
+    } catch (JSONException e) {
+      throw new RMapException("Failed to decode JSON for RMap: " + uri, e);
+    }
+  }
+
+  /**
+   * Reads and caches the RMap from the given URI and returns its signature.
+   *
+   * @param uri
+   * @return
+   */
+  public synchronized String readRMap(final URI uri) throws IOException,
+          RMapException {
+    return getRMap(uri).signature;
+  }
+
+  public synchronized String readRMap(URI uri, boolean reload)
+      throws IOException, RMapException {
+    return getRMap(uri, reload).signature;
+  }
+
+  /**
+   * Get the list of regions which need to be updated in order to transition to
+   * this (version) of the RMap by the given URI.
+   *
+   * @param uri of the RMap
+   * @return a list of regions
+   */
+  public synchronized Collection<HRegionInfo> getTransitionDelta(final URI uri)
+          throws IOException, RMapException {
+    RMap nextRMap = getRMap(uri);
+    RMap currentRMap = appliedRMaps.get(RMapReader.getSchemeAndPath(uri));
+
+    // The standard Set implementations seem to be using compareTo() for their
+    // operations. On the HRegionInfo objects compareTo() and equals() have
+    // different properties where equals() is needed here. What follows is a
+    // poor mans Set comparison to determine which regions need to be modified
+    // to make the RMap transition.
+    if (nextRMap != null) {
+      HashMap<String, HRegionInfo> delta = new HashMap<>();
+      for (HRegionInfo next : nextRMap.regions) {
+        delta.put(next.getEncodedName(), next);
+      }
+
+      if (currentRMap != null) {
+        // Remove all regions already present in the current RMap from the
+        // delta. This should use the {@link HRegionInfo.equals} method as it
+        // should consider the favored nodes and replicas.
+        for (HRegionInfo current : currentRMap.regions) {
+          HRegionInfo next = delta.get(current.getEncodedName());
+          if (next != null) {
+            if (next.equals(current)) {
+              delta.remove(next.getEncodedName());
+            }
+          }
+        }
+      }
+
+      return delta.values();
+    }
+
+    return Collections.emptyList();
+  }
+
+  public synchronized void appliedRMap(final URI uri) throws IOException,
+          RMapException {
+    RMap previous = appliedRMaps.put(RMapReader.getSchemeAndPath(uri),
+        getRMap(uri));
+    // Purge the earlier version of the RMap from cache.
+    if (previous != null) {
+      cachedRMaps.remove(previous.uri);
+    }
+  }
+
+  public synchronized boolean isRMapApplied(final URI uri) {
+    RMap active = appliedRMaps.get(RMapReader.getSchemeAndPath(uri));
+    if (active != null) {
+      return active.uri.equals(uri);
+    }
+    return false;
+  }
+
+  public synchronized RMap getAppliedRMap(String uri) {
+    return appliedRMaps.get(uri);
+  }
+
+  public synchronized List<HRegionInfo> getRegions(final URI uri)
+          throws IOException, RMapException {
+    RMap rmap = getRMap(uri);
+    if (rmap == null) {
+      return Collections.emptyList();
+    }
+    return Collections.unmodifiableList(rmap.regions);
+  }
+
+  public synchronized void clearFromRMapCache(URI uri) {
+    cachedRMaps.remove(uri);
+  }
+
+  /**
+   * Replace the content of cached RMap. For testing only!
+   *
+   * @param uri
+   * @param rMap
+   */
+  public synchronized void cacheCustomRMap(URI uri, RMap rMap) {
+    cachedRMaps.put(uri, rMap);
+    appliedRMaps.put(uri.toString(), rMap);
+  }
+
+  public class RMap {
+    public final URI uri;
+    public final List<HRegionInfo> regions;
+    public final String signature;
+
+    RMap(final URI uri, final List<HRegionInfo> regions,
+         final String signature) {
+      this.uri = uri;
+      this.regions = regions;
+      this.signature = signature;
+    }
+
+    /**
+     * Return the quorum size in the RMap.
+     * @return
+     */
+    public int getQuorumSize() {
+      if (regions.size() == 0) {
+        return 0;
+      }
+      return regions.get(0).getQuorumInfo().getQuorumSize();
+    }
+
+    /**
+     * Return the list of regions that are served by the specified server.
+     * @param hServerAddress
+     * @return
+     */
+    public List<HRegionInfo> getRegionsForServer(HServerAddress hServerAddress) {
+      List<HRegionInfo> ret = new ArrayList<HRegionInfo>();
+      for (HRegionInfo region: regions) {
+        if (region.getPeersWithRank().containsKey(hServerAddress)) {
+          ret.add(region);
+        }
+      }
+      return ret;
+    }
+
+    /**
+     * Returns the set of servers that are hosting any of the regions in the RMap.
+     * @return
+     */
+    public Set<HServerAddress> getAllServers() {
+      Set<HServerAddress> ret = new HashSet<>();
+      for (HRegionInfo region: regions) {
+        ret.addAll(region.getPeersWithRank().keySet());
+      }
+      return ret;
+    }
+
+    /**
+     * Create a customized RMap for test use only!
+     *
+     * @param uri
+     * @param regions
+     * @param signature
+     * @return
+     */
+    public RMap createCustomizedRMap(URI uri,
+                                     List<HRegionInfo> regions,
+                                     String signature) {
+      return new RMapConfiguration.RMap(
+          uri == null ? this.uri : uri,
+          regions == null ? this.regions : regions,
+          signature == null ? this.signature : signature
+      );
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == null || !(obj instanceof RMap)) {
+        return false;
+      }
+      RMap that = (RMap)obj;
+      if (this.regions == null || that.regions == null || this.regions.size() != that.regions.size()) {
+        return false;
+      }
+      Set<HRegionInfo> regionInfos = new TreeSet<>();
+      regionInfos.addAll(regions);
+      for (HRegionInfo region : that.regions) {
+        if (!regionInfos.contains(region)) {
+          return false;
+        }
+        regionInfos.remove(region);
+      }
+      return regionInfos.isEmpty();
+    }
+  }
+
+  /**
+   * Creates a temporary name for an RMap, based on the date and time.
+   * @return
+   */
+  public static String createRMapName() {
+    SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd-HHmmss");
+    return "rmap.json." + format.format(System.currentTimeMillis());
+  }
+
+  /**
+   * View information about an RMap. Currently only prints its signature.
+   * @param args
+   */
+  public static void main(String[] args) throws ParseException,
+    URISyntaxException, RMapException, IOException {
+    Options options = new Options();
+    options.addOption("r", "rmap", true, "Name of the rmap");
+
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd = parser.parse(options, args);
+
+    if (!cmd.hasOption("r")) {
+      System.out.println("Please specify the rmap with -r");
+      return;
+    }
+
+    String rmapUriStr = cmd.getOptionValue("r");
+    RMapConfiguration conf = new RMapConfiguration(new Configuration());
+    String rmapStr = conf.readRMap(new URI(rmapUriStr));
+    LOG.debug("RMap Signature: " + rmapStr);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapException.java
new file mode 100644
index 0000000..31621ab
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapException.java
@@ -0,0 +1,11 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+public class RMapException extends Exception {
+  public RMapException(final String message) {
+    super(message);
+  }
+
+  public RMapException(final String message, final Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapJSON.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapJSON.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapJSON.java
new file mode 100644
index 0000000..6d06123
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapJSON.java
@@ -0,0 +1,34 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.json.JSONObject;
+
+import java.net.URI;
+
+public class RMapJSON {
+  final URI uri;
+  final JSONObject rmap;
+  final String signature;
+
+  public RMapJSON(final URI uri, final JSONObject rmap,
+                  final String signature) {
+    this.uri = uri;
+    this.rmap = rmap;
+    this.signature = signature;
+  }
+
+  public long getVersion() {
+    return RMapReader.getVersion(uri);
+  }
+
+  public URI getURI() {
+    return uri;
+  }
+
+  public JSONObject getEncodedRMap() {
+    return rmap;
+  }
+
+  public String getSignature() {
+    return signature;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapReader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapReader.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapReader.java
new file mode 100644
index 0000000..dc81d34
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RMapReader.java
@@ -0,0 +1,205 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.apache.commons.codec.binary.Hex;
+//import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.http.NameValuePair;
+import org.apache.http.client.utils.URLEncodedUtils;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+
+public abstract class RMapReader {
+  /** Max file sisze of a single file containing a RMap */
+  public static long MAX_SIZE_BYTES = 16 * 1024 * 1204; // 16 MB
+
+  /** RMap version special values */
+  public static long NEXT = -2;
+  public static long CURRENT = -1;
+  public static long UNKNOWN = 0;
+
+  /**
+   * Return a naturally sorted list of available versions of a given RMap URI.
+   *
+   * @param uri URI of the RMap
+   * @return a naturally sorted list of versions of the given RMap URI
+   * @throws IOException if an exception occurs while reading versions
+   */
+  public abstract List<Long> getVersions(final URI uri) throws IOException;
+
+  /**
+   * Resolve a URI containing a symbolic version into a URI with an absolute
+   * value which can be requested from the reader.
+   *
+   * @param uri URI containing a symbolic version
+   * @return a URI containing an absolute version
+   * @throws URISyntaxException if the given URI is malformed
+   */
+  public abstract URI resolveSymbolicVersion(final URI uri)
+          throws URISyntaxException;
+
+  /**
+   * Return the contents of the RMap at given URI as a string.
+   *
+   * @param uri URI of the RMap
+   * @return contents of the RMap as String
+   * @throws IOException if an exception occurs while reading the RMap
+   */
+  public abstract String readRMapAsString(final URI uri) throws IOException;
+
+  /**
+   * Return the version number of the RMap specified in the given URI.
+   *
+   * @param uri URI of the RMap
+   * @return the version number of the RMap or 0 if no version was found
+   */
+  public static long getVersion(final URI uri) {
+    for (NameValuePair param : URLEncodedUtils.parse(uri, "UTF-8")) {
+      if (param.getName().equals("version")) {
+        switch (param.getValue().toUpperCase()) {
+          case "NEXT":
+            return NEXT;
+          case "CURRENT":
+            return CURRENT;
+          default:
+            try {
+              return Long.parseLong(param.getValue());
+            } catch (NumberFormatException e) {
+              /* Ignore if NaN */
+            }
+        }
+      }
+    }
+    return UNKNOWN;
+  }
+
+  public static boolean isSymbolicVersion(final URI uri) {
+    return getVersion(uri) < 0;
+  }
+
+  /**
+   * Read and return a {@link RMapJSON} of the RMap at the given URI.
+   *
+   * @param uri URI of the RMap
+   * @return a JSON representation of the RMap
+   * @throws IOException if an (possible transient) exception occurs while
+   *        reading the RMap
+   * @throws RMapException if any other exception occurs while reading the RMap
+   */
+  public RMapJSON readRMap(final URI uri) throws IOException, RMapException {
+    URI nonSymbolicURI;
+    try {
+      nonSymbolicURI = resolveSymbolicVersion(uri);
+      String encodedRMap = readRMapAsString(nonSymbolicURI);
+      return new RMapJSON(nonSymbolicURI, new JSONObject(encodedRMap),
+              getSignature(encodedRMap));
+    } catch (URISyntaxException e) {
+      throw new RMapException("URI syntax invalid for RMap: " + uri, e);
+    } catch (JSONException e) {
+      throw new RMapException(
+              "Failed to decode JSON string for RMap: " + uri, e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new RMapException(
+              "Failed to generate signature for RMap: " + uri, e);
+    }
+  }
+
+  /**
+   * Get a MD5 hash of the given string.
+   *
+   * @param s string to be hashed
+   * @return a hex String representation of the hash
+   * @throws NoSuchAlgorithmException if MD5 message digest is unavailable
+   */
+  public static String getSignature(final String s)
+          throws NoSuchAlgorithmException {
+    return new String(Hex.encodeHex(
+            MessageDigest.getInstance("MD5").digest(s.getBytes())));
+  }
+
+  /**
+   * Get a MD5 hash of the given string.
+   *
+   * @param s string to be hashed
+   * @return a hex String representation of the hash
+   * @throws NoSuchAlgorithmException if MD5 message digest is unavailable
+   */
+  public String getSignature(final URI uri) throws IOException, RMapException {
+    URI nonSymbolicURI;
+    try {
+      nonSymbolicURI = resolveSymbolicVersion(uri);
+      String encodedRMap = readRMapAsString(nonSymbolicURI);
+      return getSignature(encodedRMap);
+    } catch (URISyntaxException e) {
+      throw new RMapException("URI syntax invalid for RMap: " + uri, e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new RMapException(
+              "Failed to generate signature for RMap: " + uri, e);
+    }
+  }
+
+  /**
+   * Get the scheme, authority (if present) and path of a given URI as a string.
+   * @param uri
+   * @return a string containing just the scheme, authority and path
+   */
+  public static String getSchemeAndPath(final URI uri) {
+    return String.format("%s:%s%s", uri.getScheme(),
+            uri.getAuthority() != null ?
+                    String.format("//%s", uri.getAuthority()) : "",
+            uri.getPath());
+  }
+
+  /**
+   * Get a versioned URI for the RMap with given scheme, path and version.
+   * @param schemeAndPath
+   * @param version
+   * @return a URI of the form [scheme]:[authority]//[path]?version=[version]
+   * @throws URISyntaxException
+   */
+  public static URI getVersionedURI(final String schemeAndPath,
+          final long version) throws URISyntaxException {
+    String token = "UNKNOWN";
+
+    if (version > 0) {
+      token = String.format("%d", version);
+    } else if (version == CURRENT) {
+      token = "CURRENT";
+    } else if (version == NEXT) {
+      token = "NEXT";
+    }
+
+    return new URI(String.format("%s?version=%s", schemeAndPath, token));
+  }
+
+  /**
+   * Get a versioned URI for the RMap with given base URI and version. If the
+   * given URI already contains a version it is overwritten by the given
+   * version.
+   * @param uri
+   * @param version
+   * @return a URI of the form [scheme]:[authority]//[path]?version=[version]
+   * @throws URISyntaxException
+   */
+  public static URI getVersionedURI(final URI uri, final long version)
+          throws URISyntaxException {
+    return getVersionedURI(getSchemeAndPath(uri), version);
+  }
+
+  public long getCurrentVersion(final String schemeAndPath)
+          throws URISyntaxException {
+    return getVersion(resolveSymbolicVersion(
+            getVersionedURI(schemeAndPath, CURRENT)));
+  }
+
+  public long getNextVersion(final String schemeAndPath)
+          throws URISyntaxException {
+    return getVersion(resolveSymbolicVersion(
+            getVersionedURI(schemeAndPath, NEXT)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RegionLocator.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RegionLocator.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RegionLocator.java
new file mode 100644
index 0000000..6dfaa57
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rmap/RegionLocator.java
@@ -0,0 +1,142 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.json.JSONException;
+import org.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+public class RegionLocator {
+  private static final Logger LOG = LoggerFactory.getLogger(
+          RegionLocator.class);
+
+  private Configuration conf;
+
+  // regionInfoMap is a mapping from table name to region start key to
+  // HRegionInfo. This will be used in locateRegion and in turn in
+  // HConnection.locateRegion, so it needs to be thread-safe as the same
+  // HConnection can be used from multiple threads at the same time
+  ConcurrentHashMap<String, ConcurrentSkipListMap<byte[], HRegionInfo>>
+          regionInfoMap = new ConcurrentHashMap<>();
+
+  public RegionLocator(final Configuration conf) {
+    this.conf = conf;
+  }
+
+  public HRegionInfo findRegion(byte[] tableName, byte[] row) {
+    ConcurrentSkipListMap<byte[], HRegionInfo> regions =
+            regionInfoMap.get(Bytes.toString(tableName));
+    if (regions != null) {
+      Map.Entry<byte[], HRegionInfo> entry = regions.floorEntry(row);
+      if (entry != null) {
+        return entry.getValue();
+      }
+    }
+    return null;
+  }
+
+  public List<HTableDescriptor> getAllTables() {
+    List<HTableDescriptor> tables = new ArrayList<>(regionInfoMap.size());
+    for (ConcurrentSkipListMap<byte[], HRegionInfo> regionMapForTable :
+            regionInfoMap.values()) {
+      if (regionMapForTable.size() > 0) {
+        tables.add(regionMapForTable.firstEntry().getValue().getTableDesc());
+      }
+    }
+    return tables;
+  }
+
+  public List<List<HRegionInfo>> getAllRegionsGroupByTable() {
+    List<List<HRegionInfo>> regions = new ArrayList<>(regionInfoMap.size());
+    for (ConcurrentSkipListMap<byte[], HRegionInfo> regionMapForTable :
+            regionInfoMap.values()) {
+      regions.add(new ArrayList<>(regionMapForTable.values()));
+    }
+    return regions;
+  }
+
+  /**
+   * Get all servers found in the regionInfo map. This method iterates over all
+   * HRegionInfo entries and thus might be expensive.
+   *
+   * @return a set containing all servers found in the region map
+   */
+  public Set<HServerAddress> getAllServers() {
+    Set<HServerAddress> servers = new HashSet<>();
+    for (ConcurrentSkipListMap<byte[], HRegionInfo> regionMapForTable :
+            regionInfoMap.values()) {
+      for (HRegionInfo region : regionMapForTable.values()) {
+        for (HServerAddress server : region.getPeersWithRank().keySet()) {
+          servers.add(server);
+        }
+      }
+    }
+    return servers;
+  }
+
+  public List<HRegionInfo> getRegionsForTable(byte[] tableName) {
+    ConcurrentSkipListMap<byte[], HRegionInfo> regions =
+            regionInfoMap.get(Bytes.toString(tableName));
+    if (regions != null) {
+      return new ArrayList<>(regions.values());
+    } else {
+      return null;
+    }
+  }
+
+  public List<HRegionInfo> getRegionsForServer(final HServerAddress address) {
+    List<HRegionInfo> regions = new ArrayList<>();
+    for (ConcurrentSkipListMap<byte[], HRegionInfo> regionMapForTable :
+            regionInfoMap.values()) {
+      for (HRegionInfo region : regionMapForTable.values()) {
+        if (region.getPeersWithRank().containsKey(address)) {
+          regions.add(region);
+        }
+      }
+    }
+    return regions;
+  }
+
+  private void updateRegionInfoMap(final List<HRegionInfo> regions) {
+    for (HRegionInfo region : regions) {
+      String tableName = region.getTableDesc().getNameAsString();
+      ConcurrentSkipListMap<byte[], HRegionInfo> regionMapForTable
+              = regionInfoMap.get(tableName);
+      if (regionMapForTable == null) {
+        regionMapForTable = new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
+        regionInfoMap.put(tableName, regionMapForTable);
+      }
+      regionMapForTable.put(region.getStartKey(), region);
+    }
+  }
+
+  public void refresh() throws IOException, RMapException {
+    Parser parser = new Parser(conf);
+
+    URI uri = RMapConfiguration.getRMapSubscription(conf);
+    if (uri != null) {
+      RMapReader reader = RMapConfiguration.getRMapReader(conf, uri);
+
+      try {
+        JSONObject encodedRMap = reader.readRMap(uri).getEncodedRMap();
+        updateRegionInfoMap(parser.parseEncodedRMap(encodedRMap));
+      } catch (JSONException e) {
+        throw new RMapException("Failed to decode JSON for RMap: " + uri, e);
+      }
+    }
+  }
+
+  public boolean isEmpty() {
+    return regionInfoMap.isEmpty();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/AppendRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/AppendRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/AppendRequest.java
new file mode 100644
index 0000000..c2bd496
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/AppendRequest.java
@@ -0,0 +1,217 @@
+package org.apache.hadoop.hbase.consensus.rpc;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+@ThriftStruct
+public final class AppendRequest extends Request<AppendResponse> {
+
+  private final String regionId;
+
+  private final ConsensusHost leaderId;
+
+  private final boolean isHeartBeat;
+
+  private final long commitIndex;
+
+  private final long persistedIndex;
+
+  private final EditId prevLogId;
+
+  private final List<EditId> logIds;
+
+  private final List<ByteBuffer> listOfEdits;
+
+  private boolean isTraceable = false;
+
+  private SettableFuture<AppendResponse> response;
+
+  @ThriftConstructor
+  public AppendRequest(
+      @ThriftField(1) final String regionId,
+      @ThriftField(2) final ConsensusHost id,
+      @ThriftField(3) final boolean isHeartBeat,
+      @ThriftField(4) final long commitIndex,
+      @ThriftField(5) final long persistedIndex,
+      @ThriftField(6) final EditId prevLogId,
+      @ThriftField(7) final List<EditId> logIds,
+      @ThriftField(8) final List<ByteBuffer> listOfEdits
+  ) {
+    this.regionId = regionId;
+    this.leaderId = id;
+    this.isHeartBeat = isHeartBeat;
+    this.commitIndex = commitIndex;
+    this.persistedIndex = persistedIndex;
+    this.prevLogId = prevLogId;
+    this.logIds = logIds;
+    this.listOfEdits = listOfEdits;
+    assert logIds.size() == listOfEdits.size();
+  }
+
+  public AppendRequest(final AppendRequest r) {
+    this.regionId = r.regionId;
+    this.leaderId = r.leaderId;
+    this.isHeartBeat = r.isHeartBeat;
+    this.commitIndex = r.commitIndex;
+    this.persistedIndex = r.persistedIndex;
+    this.prevLogId = r.prevLogId;
+    this.logIds = r.logIds;
+    this.listOfEdits = r.listOfEdits;
+  }
+
+  @ThriftField(1)
+  public String getRegionId() {
+    return regionId;
+  }
+
+  @ThriftField(2)
+  public ConsensusHost getLeaderId() {
+    return leaderId;
+  }
+
+  @ThriftField(3)
+  public boolean isHeartBeat() {
+    return this.isHeartBeat;
+  }
+
+  @ThriftField(4)
+  public long getCommitIndex() {
+    return commitIndex;
+  }
+
+  @ThriftField(5)
+  public long getPersistedIndex() {
+    return persistedIndex;
+  }
+
+  @ThriftField(6)
+  public EditId getPrevLogId() {
+    return prevLogId;
+  }
+
+  @ThriftField(7)
+  public List<EditId> getLogIds() {
+    return logIds;
+  }
+
+  @ThriftField(8)
+  public List<ByteBuffer> getListOfEdits() {
+    return listOfEdits;
+  }
+
+  public EditId getLogId(int index) {
+    return logIds.get(index);
+  }
+
+  public ByteBuffer getEdit(int index) {
+    return listOfEdits.get(index);
+  }
+
+  public void createAppendResponse() {
+    if (response == null) {
+      response = SettableFuture.create();
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb = sb.append("AppendRequest{")
+      .append("region = ").append(regionId)
+      .append(", address = ").append(leaderId)
+      .append(", heartbeat = ").append(isHeartBeat)
+      .append(", commit index = ").append(commitIndex)
+      .append(", persisted index = ").append(persistedIndex)
+      .append(", prev log edit = ").append(prevLogId);
+
+    sb.append(", current edit logs = ")
+      .append(RaftUtil.listToString(logIds));
+
+    if (listOfEdits != null) {
+      sb.append(", edit sizes = [");
+
+      for (int i=0; i < listOfEdits.size(); i++) {
+        if (i > 0) {
+          sb.append(", ");
+        }
+        if (listOfEdits.get(i) != null) {
+          sb.append("" + listOfEdits.get(i));
+        }
+      }
+      sb.append("] ");
+    }
+
+    sb.append('}');
+    return sb.toString();
+  }
+
+  public ListenableFuture<AppendResponse> getResponse() {
+    return response;
+  }
+
+  public void setResponse(AppendResponse r) {
+    if (response != null) {
+      response.set(r);
+    }
+  }
+
+  public void setError(final Throwable exception) {
+    if (response != null) {
+      response.setException(exception);
+    }
+  }
+
+  public boolean validateFields() {
+    assert getLogIds() != null;
+    assert getListOfEdits() != null;
+    assert getLogIds().size() == getListOfEdits().size();
+    return true;
+  }
+
+  public int logCount() {
+    assert validateFields();
+    return getLogIds().size();
+  }
+
+  public static AppendRequest createSingleAppendRequest(
+    final String regionId,
+    final ConsensusHost id,
+    final EditId logId,
+    final EditId prevLogId,
+    final long commitIndex,
+    final long persistedIndex,
+    final boolean isHeartBeat,
+    final ByteBuffer edits) {
+    return new AppendRequest(
+      regionId,
+      id,
+      isHeartBeat,
+      commitIndex,
+      persistedIndex,
+      prevLogId,
+      Arrays.asList(logId),
+      Arrays.asList(edits));
+  }
+
+  public void enableTraceable() {
+    this.isTraceable = true;
+  }
+
+  public void disableTraceable() {
+    this.isTraceable = false;
+  }
+
+  public boolean isTraceable() {
+    return this.isTraceable;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/AppendResponse.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/AppendResponse.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/AppendResponse.java
new file mode 100644
index 0000000..8e7c1e0
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/AppendResponse.java
@@ -0,0 +1,94 @@
+package org.apache.hadoop.hbase.consensus.rpc;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+
+@ThriftStruct
+public final class AppendResponse {
+
+  public enum Result
+  {
+    SUCCESS, HIGHER_TERM, LAGGING, MISSING_EDITS
+  }
+
+  private final String address;
+
+  /** The identifier to associate the AppendRequest and AppendResponse. */
+  private final EditId id;
+
+  private final EditId prevEditID;
+
+  private final Result result;
+
+  private final int rank;
+
+  private final boolean canTakeover;
+
+  @ThriftConstructor
+  public AppendResponse (String address,
+                         final EditId id,
+                         final EditId prevEditID,
+                         Result result,
+                         int rank,
+                         boolean canTakeover) {
+    this.address = address;
+    this.id = id;
+    this.prevEditID = prevEditID;
+    this.result = result;
+    this.rank = rank;
+    this.canTakeover = canTakeover;
+  }
+
+  @ThriftField(1)
+  public String getAddress() {
+    return address;
+  }
+
+  /**
+   *
+   * @return the identifier to associate the AppendRequest and AppendResponse.
+   */
+  @ThriftField(2)
+  public EditId getId() {
+    return id;
+  }
+
+  @ThriftField(3)
+  public EditId getPrevEditID() {
+    return prevEditID;
+  }
+
+  @ThriftField(4)
+  public Result getResult() {
+    return result;
+  }
+
+  @ThriftField(5)
+  public int getRank() {
+    return rank;
+  }
+
+  @ThriftField(6)
+  public boolean canTakeover() {
+    return canTakeover;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder s = new StringBuilder();
+
+    s.append("AppendResponse{")
+      .append("address= ").append(address)
+      .append(", id=").append(id)
+      .append(", prevEditID=").append(prevEditID)
+      .append(", result=").append(result)
+      .append(", rank=").append(rank)
+      .append(", canTakeOver=").append(canTakeover)
+      .append('}');
+
+    return s.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/LogState.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/LogState.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/LogState.java
new file mode 100644
index 0000000..0d2dae3
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/LogState.java
@@ -0,0 +1,151 @@
+package org.apache.hadoop.hbase.consensus.rpc;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import org.apache.hadoop.hbase.consensus.log.LogFileInfo;
+import org.apache.hadoop.hbase.consensus.log.SeedLogFile;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.util.Pair;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+@ThriftStruct
+public final class LogState {
+  private List<LogFileInfo> committedLogFiles;
+  private List<LogFileInfo> uncommittedLogFiles;
+  private String peerState;
+  private EditId lastCommittedEdit;
+
+  // Used to pass error message. If not null, other fields are considered
+  // invalid and should be discarded.
+  private String errMsg;
+
+  public LogState(String errMsg) {
+    this.errMsg = errMsg;
+    this.peerState = null;
+    lastCommittedEdit = null;
+
+    if (this.errMsg == null) {
+      this.committedLogFiles = new ArrayList<>();
+      this.uncommittedLogFiles = new ArrayList<>();
+    }
+  }
+
+  @ThriftConstructor
+  public LogState(
+      @ThriftField(1) List<LogFileInfo> committedLogFiles,
+      @ThriftField(2) List<LogFileInfo> uncommittedLogFiles,
+      @ThriftField(3) String peerState,
+      @ThriftField(4) String errMsg,
+      @ThriftField(5) EditId lastCommittedEdit) {
+    this.committedLogFiles = committedLogFiles;
+    this.uncommittedLogFiles = uncommittedLogFiles;
+    this.peerState = peerState;
+    this.errMsg = errMsg;
+    this.lastCommittedEdit = lastCommittedEdit;
+  }
+
+  @ThriftField(1)
+  public List<LogFileInfo> getCommittedLogFiles() {
+    return this.committedLogFiles;
+  }
+
+  @ThriftField(2)
+  public List<LogFileInfo> getUncommittedLogFiles() {
+    return this.uncommittedLogFiles;
+  }
+
+  @ThriftField(3)
+  public String getPeerState() {
+    return this.peerState;
+  }
+
+  @ThriftField(4)
+  public String getErrMsg() {
+    return this.errMsg;
+  }
+
+  @ThriftField(5)
+  public EditId getLastCommittedEdit() {
+    return this.lastCommittedEdit;
+  }
+
+  public void setPeerState(String peerState) {
+    this.peerState = peerState;
+  }
+
+  public void addCommittedLogFile(LogFileInfo info) {
+    this.committedLogFiles.add(info);
+  }
+
+  public void addUncommittedLogFile(LogFileInfo info) {
+    this.uncommittedLogFiles.add(info);
+  }
+
+  public void setErrMsg(String errMsg) {
+    this.errMsg = errMsg;
+  }
+
+  public void setLastCommittedEdit(EditId lastCommittedEdit) {
+    this.lastCommittedEdit = lastCommittedEdit;
+  }
+
+  public boolean isErrorState() {
+    return errMsg != null;
+  }
+
+  public void sortLogFiles() {
+    if (committedLogFiles != null && !committedLogFiles.isEmpty()) {
+      Collections.sort(committedLogFiles);
+    }
+    if (uncommittedLogFiles != null && !uncommittedLogFiles.isEmpty()) {
+      Collections.sort(uncommittedLogFiles);
+    }
+  }
+
+  private Pair<Long, Long> getIndexRange(List<LogFileInfo> logFiles) {
+    if (logFiles == null || logFiles.isEmpty()) {
+      return null;
+    }
+    Long startIndex = logFiles.get(0).getInitialIndex();
+    Long lastIndex = logFiles.get(logFiles.size() - 1).getLastIndex();
+    return new Pair<>(startIndex, lastIndex);
+  }
+
+  @Override
+  public String toString() {
+    if (errMsg != null) {
+      return errMsg;
+    }
+
+    sortLogFiles();
+
+    StringBuilder sb = new StringBuilder();
+    sb.append("{ ");
+
+    Pair<Long, Long> uncommittedRange = getIndexRange(uncommittedLogFiles);
+    if (uncommittedRange != null) {
+      sb.append("Uncommitted [").append(uncommittedRange.getFirst())
+          .append(", ").append(uncommittedRange.getSecond()).append("] ");
+    }
+
+    Pair<Long, Long> committedRange = getIndexRange(committedLogFiles);
+    if (committedRange != null) {
+      sb.append("Committed [").append(committedRange.getFirst())
+          .append(", ").append(committedRange.getSecond()).append("] ");
+    }
+    for (LogFileInfo info : committedLogFiles) {
+      if (SeedLogFile.isSeedFile(info.getFilename())) {
+        sb.append("Seed File [").append(info.getInitialIndex()).append(", ")
+            .append(info.getLastIndex()).append("]");
+      }
+    }
+    sb.append(" } ; {Peers: ");
+    sb.append(peerState);
+    sb.append(" }");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/PeerStatus.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/PeerStatus.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/PeerStatus.java
new file mode 100644
index 0000000..cbd28ce
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/PeerStatus.java
@@ -0,0 +1,107 @@
+package org.apache.hadoop.hbase.consensus.rpc;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftEnum;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import org.apache.hadoop.hbase.regionserver.DataStoreState;
+
+import javax.annotation.concurrent.Immutable;
+
+@Immutable
+@ThriftStruct
+public final class PeerStatus implements Comparable<PeerStatus> {
+
+  @ThriftEnum
+  public static enum RAFT_STATE {
+    INVALID,
+    LEADER,
+    FOLLOWER,
+    CANDIDATE,
+    HALT
+  }
+
+  private final String id;
+  private final int rank;
+  private final long term;
+  private final RAFT_STATE paxosState;
+  private final LogState logState;
+  private final String metrics;
+  private final DataStoreState dataStoreState;
+
+  private String peerAddress;
+
+  @ThriftConstructor
+  public PeerStatus(
+    @ThriftField(1) final String id,
+    @ThriftField(2) final int rank,
+    @ThriftField(3) final long term,
+    @ThriftField(4) final RAFT_STATE paxosState,
+    @ThriftField(5) final LogState logState,
+    @ThriftField(6) final String metrics,
+    @ThriftField(7) final DataStoreState dataStoreState) {
+    this.id = id;
+    this.rank = rank;
+    this.term = term;
+    this.paxosState = paxosState;
+    this.logState = logState;
+    this.metrics = metrics;
+    this.dataStoreState = dataStoreState;
+  }
+
+  @ThriftField(1)
+  public String getId() {
+    return id;
+  }
+
+  @ThriftField(2)
+  public int getRank() {
+    return rank;
+  }
+
+  @ThriftField(3)
+  public long getTerm() {
+    return term;
+  }
+
+  @ThriftField(4)
+  public RAFT_STATE getPaxosState() {
+    return paxosState;
+  }
+
+  @ThriftField(5)
+  public LogState getLogState() {
+    return logState;
+  }
+
+  @ThriftField(6)
+  public String getMetrics() {
+    return metrics;
+  }
+
+  @ThriftField(7)
+  public DataStoreState getDataStoreState() {
+    return dataStoreState;
+  }
+
+  public String getPeerAddress() {
+    return peerAddress;
+  }
+
+  public void setPeerAddress(String peerAddress) {
+    this.peerAddress = peerAddress;
+  }
+
+  @Override
+  public String toString() {
+    return "Peer : " + peerAddress + " {" + "id=" + id + "-" + rank
+           + "term=" + term + ", " + "paxosState=" + paxosState + ", "
+           + "logState=" + logState + ", " + "dataStoreState=" + dataStoreState
+           + "}";
+  }
+
+  @Override
+  public int compareTo(PeerStatus peer) {
+    return Integer.compare(this.rank, peer.getRank());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/Request.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/Request.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/Request.java
new file mode 100644
index 0000000..a90af7b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/Request.java
@@ -0,0 +1,8 @@
+package org.apache.hadoop.hbase.consensus.rpc;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+public abstract class Request<T> {
+  public abstract void setResponse(final T response);
+  public abstract ListenableFuture<T> getResponse();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/VoteRequest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/VoteRequest.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/VoteRequest.java
new file mode 100644
index 0000000..bb7660a
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/VoteRequest.java
@@ -0,0 +1,94 @@
+package org.apache.hadoop.hbase.consensus.rpc;
+
+import javax.annotation.concurrent.Immutable;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+
+@Immutable
+@ThriftStruct
+public final class VoteRequest extends Request<VoteResponse> {
+
+  private final String regionId;
+  private final String address;
+  private final long term;
+  private final EditId prevEditID;
+
+  private SettableFuture<VoteResponse> response;
+
+  @ThriftConstructor
+  public VoteRequest(
+      @ThriftField(1)final String regionId,
+      @ThriftField(2)final String address,
+      @ThriftField(3)final long term,
+      @ThriftField(4)final EditId prevEditID) {
+    this.regionId = regionId;
+    this.address = address;
+    this.term = term;
+    this.prevEditID = prevEditID;
+  }
+
+  public VoteRequest(final VoteRequest r) {
+    this.regionId = r.regionId;
+    this.address = r.address;
+    this.term = r.term;
+    this.prevEditID = r.prevEditID;
+  }
+
+  @ThriftField(1)
+  public String getRegionId() {
+    return regionId;
+  }
+
+  @ThriftField(2)
+  public String getAddress() {
+    return address;
+  }
+
+  @ThriftField(3)
+  public long getTerm() {
+    return term;
+  }
+
+  @ThriftField(4)
+  public EditId getPrevEditID() {
+    return prevEditID;
+  }
+  
+  @Override
+  public String toString() {
+    return "VoteRequest{" +
+        "region=" + regionId +
+        ", address='" + address + '\'' +
+        ", term=" + term +
+        ", prevEditID=" + prevEditID +
+        '}';
+  }
+
+  public void createVoteResponse() {
+    if (response == null) {
+      response = SettableFuture.create();
+    }
+  }
+
+  public void setResponse(VoteResponse r) {
+    if (response != null) {
+      response.set(r);
+    }
+  }
+
+  @Override
+  public ListenableFuture<VoteResponse> getResponse() {
+    return response;
+  }
+
+  public void setError(final Throwable exception) {
+    if (response != null) {
+      response.setException(exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/VoteResponse.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/VoteResponse.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/VoteResponse.java
new file mode 100644
index 0000000..3986f3c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/rpc/VoteResponse.java
@@ -0,0 +1,61 @@
+package org.apache.hadoop.hbase.consensus.rpc;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+
+@ThriftStruct
+public final class VoteResponse {
+  final String address;
+  final long term;
+  final VoteResult voteResult;
+
+  @ThriftStruct
+  public enum VoteResult {
+    SUCCESS,
+    FAILURE,
+    WRONGQUORUM
+  };
+
+  @ThriftConstructor
+  public VoteResponse(
+      @ThriftField(1) final String address,
+      @ThriftField(2) final long term,
+      @ThriftField(3) final VoteResult voteResult) {
+    this.address = address;
+    this.term = term;
+    this.voteResult = voteResult;
+  }
+
+  @ThriftField(1)
+  public String getAddress() {
+    return address;
+  }
+
+  @ThriftField(2)
+  public long getTerm() {
+    return term;
+  }
+
+  @ThriftField(3)
+  public VoteResult voteResult() {
+    return voteResult;
+  }
+
+  public boolean isSuccess() {
+    return voteResult.equals(VoteResult.SUCCESS);
+  }
+
+  public boolean isWrongQuorum() {
+    return voteResult.equals(VoteResult.WRONGQUORUM);
+  }
+
+  @Override
+  public String toString() {
+    return "VoteResponse{" +
+        "address=" + address +
+        " term=" + term +
+        ", voteResult=" + voteResult +
+        '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/ConsensusService.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/ConsensusService.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/ConsensusService.java
new file mode 100644
index 0000000..e061cc3
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/ConsensusService.java
@@ -0,0 +1,70 @@
+package org.apache.hadoop.hbase.consensus.server;
+
+import com.facebook.swift.service.ThriftException;
+import com.facebook.swift.service.ThriftMethod;
+import com.facebook.swift.service.ThriftService;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.hbase.consensus.client.FetchTask;
+import org.apache.hadoop.hbase.consensus.log.LogFileInfo;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteResponse;
+import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+@ThriftService
+public interface ConsensusService extends AutoCloseable {
+
+  @ThriftMethod(value = "appendEntries", exception = {
+      @ThriftException(type = ThriftHBaseException.class, id = 1) })
+  ListenableFuture<AppendResponse> appendEntries(AppendRequest appendRequest);
+
+  @ThriftMethod(value = "requestVote", exception = {
+      @ThriftException(type = ThriftHBaseException.class, id = 1) })
+  ListenableFuture<VoteResponse> requestVote(VoteRequest request);
+
+  @ThriftMethod
+  ListenableFuture<PeerStatus> getPeerStatus(String quorumName);
+
+  @ThriftMethod(value = "replicateCommit", exception = {
+        @ThriftException(type = ThriftHBaseException.class, id = 1) })
+  ListenableFuture<Long> replicateCommit(String regionId, List<WALEdit> txns)
+    throws ThriftHBaseException;
+
+  @ThriftMethod
+  ListenableFuture<Boolean> changeQuorum(String regionId, final ByteBuffer config);
+
+  @ThriftMethod
+  ListenableFuture<String> getLeader(String regionId);
+
+  @ThriftMethod
+  ListenableFuture<List<LogFileInfo>> getCommittedLogStatus(String quorumName,
+                                                            long minIndex);
+
+  @ThriftMethod
+  // TODO @gauravm
+  // Remove?
+  ListenableFuture<Void> fetchLogs(List<FetchTask> tasks, String regionId);
+
+  @ThriftMethod
+  ListenableFuture<List<PeerStatus>> getAllPeerStatuses();
+
+  ImmutableMap<String, RaftQuorumContext> getQuorumContextMapSnapshot();
+
+  RaftQuorumContext getRaftQuorumContext(String regionId);
+
+  RaftQuorumContext addRaftQuorumContext(final RaftQuorumContext c);
+
+  // TODO @gauravm
+  // Remove?
+  boolean removeRaftQuorumContext(final String regionName);
+
+  void stopService();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/ConsensusServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/ConsensusServiceImpl.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/ConsensusServiceImpl.java
new file mode 100644
index 0000000..6bb9eac
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/ConsensusServiceImpl.java
@@ -0,0 +1,248 @@
+package org.apache.hadoop.hbase.consensus.server;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.consensus.client.FetchTask;
+import org.apache.hadoop.hbase.consensus.log.LogFileInfo;
+import org.apache.hadoop.hbase.consensus.log.RemoteLogFetcher;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumAgent;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumMembershipChangeRequest;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.raft.events.AppendRequestEvent;
+import org.apache.hadoop.hbase.consensus.raft.events.QuorumMembershipChangeEvent;
+import org.apache.hadoop.hbase.consensus.raft.events.VoteRequestEvent;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+import org.apache.hadoop.hbase.consensus.rpc.LogState;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteResponse;
+import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
+import org.apache.hadoop.hbase.regionserver.DataStoreState;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class ConsensusServiceImpl implements ConsensusService {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    ConsensusServiceImpl.class);
+
+  /** TODO: replace this with HRegionServer.getRegions() API once we
+   *  integrate HBase with the protocol
+   */
+  private ConcurrentHashMap<String, RaftQuorumContext> quorumContextMap = new ConcurrentHashMap<>();
+
+  protected ConsensusServiceImpl() {}
+
+  public static ConsensusService createConsensusServiceImpl() {
+    return new ConsensusServiceImpl();
+  }
+
+  public static ConsensusService createTestConsensusServiceImpl() {
+    return new InstrumentedConsensusServiceImpl();
+  }
+
+  @Override
+  public ListenableFuture<AppendResponse> appendEntries(AppendRequest appendRequest) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Receiving " + appendRequest);
+    }
+    boolean success = false;
+
+    appendRequest.createAppendResponse();
+
+    RaftQuorumContext c = getRaftQuorumContext(appendRequest.getRegionId());
+    if (c != null) {
+      // Add an event for the correct state machine
+      success = c.offerEvent(new AppendRequestEvent(appendRequest));
+    }
+
+    if (!success) {
+      appendRequest.setError(new ThriftHBaseException(new Exception(
+          "Unable to complete AppendEntries: " + appendRequest)));
+    }
+    return appendRequest.getResponse();
+  }
+
+  @Override
+  public ListenableFuture<VoteResponse> requestVote(VoteRequest voteRequest) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Receiving " + voteRequest);
+    }
+
+    voteRequest.createVoteResponse();
+
+    RaftQuorumContext c = getRaftQuorumContext(voteRequest.getRegionId());
+
+    boolean success = false;
+    if (c != null) {
+      // Add an event for the correct state machine
+      success = c.offerEvent(new VoteRequestEvent(voteRequest));
+    } else {
+      LOG.error("There is no such region " + voteRequest.getRegionId() +
+        " in the current quorum server");
+    }
+
+    if (!success) {
+      voteRequest.setError(new ThriftHBaseException(
+          new Exception("Unable to complete Vote Request: " + voteRequest)));
+    }
+
+    return voteRequest.getResponse();
+  }
+
+  @Override
+  public ListenableFuture<Long> replicateCommit(String regionId, List<WALEdit> txns) throws ThriftHBaseException {
+    RaftQuorumContext c = getRaftQuorumContext(regionId);
+    if (c != null) {
+      QuorumAgent agent = c.getQuorumAgentInstance();
+      try {
+        return agent.asyncAppend(txns);
+      } catch (IOException e) {
+        throw new ThriftHBaseException(e);
+      }
+    } else {
+      Exception e = new Exception("Unable to find the " +
+              "region information for " + regionId);
+      LOG.error(e.getMessage());
+      return Futures.immediateFailedFuture(new ThriftHBaseException(e));
+    }
+  }
+
+  @Override
+  public ListenableFuture<Boolean> changeQuorum(String regionId, ByteBuffer config) {
+
+    RaftQuorumContext c = getRaftQuorumContext(regionId);
+
+    if (c != null) {
+      List<QuorumInfo> configs = QuorumInfo.deserializeFromByteBuffer(config);
+      if (configs == null || configs.size() != 1) {
+        return Futures.immediateCheckedFuture(false);
+      }
+      // Make sure the config request change is really needed.
+      if (configs.get(0).equals(c.getQuorumInfo())) {
+        return Futures.immediateCheckedFuture(true);
+      }
+
+      final QuorumMembershipChangeRequest request =
+        new QuorumMembershipChangeRequest(configs.get(0));
+      // Add an event for the correct state machine
+      boolean success = c.offerEvent(new QuorumMembershipChangeEvent(request));
+      if (!success) {
+        return Futures.immediateCheckedFuture(false);
+      }
+      return request;
+    } else {
+      LOG.debug("Ignored " + config);
+    }
+
+    return Futures.immediateCheckedFuture(false);
+  }
+
+  @Override
+  public ListenableFuture<String> getLeader(String regionId) {
+    RaftQuorumContext c = getRaftQuorumContext(regionId);
+    String leader = "";
+    if (c != null) {
+      ConsensusHost host = c.getLeader();
+      if (host != null) {
+        leader = host.getHostId();
+      }
+    }
+    if (leader.isEmpty() && LOG.isTraceEnabled()) {
+      LOG.trace("Leader is unknown for " + regionId);
+    }
+    return Futures.immediateFuture(leader);
+  }
+
+  @Override
+  public ListenableFuture<PeerStatus> getPeerStatus(String quorum) {
+    RaftQuorumContext c = getRaftQuorumContext(quorum);
+
+    if (c != null) {
+     return Futures.immediateFuture(c.getStatus());
+    } else {
+      return Futures.immediateFuture(new PeerStatus(quorum, -1, HConstants.UNDEFINED_TERM_INDEX,
+        PeerStatus.RAFT_STATE.INVALID, new LogState("Wrong Quorum"), "Error",
+        new DataStoreState("Wrong Quorum")));
+    }
+  }
+
+  @Override
+  public ListenableFuture<List<LogFileInfo>> getCommittedLogStatus(String quorumName, long minIndex) {
+    RaftQuorumContext c = getRaftQuorumContext(quorumName);
+    List<LogFileInfo> info = null;
+
+    if (c != null) {
+      info = c.getCommittedLogStatus(minIndex);
+    } else {
+      info = new ArrayList<>();
+    }
+
+    return Futures.immediateFuture(info);
+  }
+
+  @Override
+  public void stopService() {
+    LOG.info("Clear all the RaftQuorum from the quorumContextMap !");
+    // stop all the context before clearing
+    for (RaftQuorumContext context : quorumContextMap.values()) {
+      context.stop(true);
+    }
+
+    this.quorumContextMap.clear();
+  }
+
+  @Override
+  public ImmutableMap<String, RaftQuorumContext> getQuorumContextMapSnapshot() {
+    return ImmutableMap.copyOf(this.quorumContextMap);
+  }
+
+  @Override
+  public RaftQuorumContext addRaftQuorumContext(final RaftQuorumContext c) {
+    LOG.info("Adding RaftQuorumContext " + c + " to the quorumContextMap");
+    return this.quorumContextMap.put(c.getQuorumName(), c);
+  }
+
+  @Override
+  public boolean removeRaftQuorumContext(String regionName) {
+    LOG.info("Removing quorum for the region " + regionName + " from the quorumContextMap");
+    return this.quorumContextMap.remove(regionName) != null;
+  }
+
+  @Override
+  public RaftQuorumContext getRaftQuorumContext(String regionId) {
+    return this.quorumContextMap.get(regionId);
+  }
+
+  @Override public void close() throws Exception {
+    // Do nothing. Added the AutoCloseable only for the client cleanup
+  }
+
+  @Override
+  public ListenableFuture<Void> fetchLogs(List<FetchTask> tasks, String regionId) {
+    return new RemoteLogFetcher().executeTasks(tasks, regionId);
+  }
+
+  @Override public ListenableFuture<List<PeerStatus>> getAllPeerStatuses() {
+    List<PeerStatus> statuses = new ArrayList<>();
+    Map<String, RaftQuorumContext> quorumMap = getQuorumContextMapSnapshot();
+
+    for (RaftQuorumContext quorum : quorumMap.values()) {
+      statuses.add(quorum.getStatus());
+    }
+
+    return Futures.immediateFuture(statuses);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/InstrumentedConsensusServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/InstrumentedConsensusServiceImpl.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/InstrumentedConsensusServiceImpl.java
new file mode 100644
index 0000000..7b10e7d
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/server/InstrumentedConsensusServiceImpl.java
@@ -0,0 +1,241 @@
+package org.apache.hadoop.hbase.consensus.server;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.hadoop.hbase.consensus.rpc.AppendRequest;
+import org.apache.hadoop.hbase.consensus.rpc.AppendResponse;
+import org.apache.hadoop.hbase.consensus.rpc.VoteRequest;
+import org.apache.hadoop.hbase.consensus.rpc.VoteResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MarkerFactory;
+
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class InstrumentedConsensusServiceImpl extends ConsensusServiceImpl {
+  private final Logger LOG = LoggerFactory.getLogger(InstrumentedConsensusServiceImpl.class);
+
+  private ConcurrentHashMap<String, Double> packetDropRateMap = new ConcurrentHashMap<String, Double>();
+  private ConcurrentHashMap<String, Long> packetDelayMap = new ConcurrentHashMap<String, Long>();
+  private ConcurrentHashMap<String, Boolean> hiccupMap = new ConcurrentHashMap<String, Boolean>();
+  private Random random = new Random(System.currentTimeMillis());
+  private RelayThread relayThread;
+  private String identity = "None";
+  private AtomicLong normalPacketDropCount = new AtomicLong(0);
+  private AtomicLong hiccupPacketDropCount = new AtomicLong(0);
+
+  protected InstrumentedConsensusServiceImpl() {
+    relayThread = new RelayThread(this);
+    relayThread.start();
+  }
+
+  public synchronized void setIdentity(String str) {
+    identity = str;
+  }
+
+  public synchronized String getIdentity() {
+    return identity;
+  }
+
+  public enum PacketDropStyle {
+    NONE,
+    ALWAYS,
+    RANDOM
+  };
+
+  public class RelayThread extends Thread {
+    private InstrumentedConsensusServiceImpl cs;
+    private DelayQueue<DelayedRequest> queue = new DelayQueue<DelayedRequest>();
+
+    public class DelayedRequest implements Delayed {
+      private long    deadline;
+      private Object  request;
+
+      public DelayedRequest(Object request, long delay) {
+        deadline = delay + System.currentTimeMillis();
+        this.request = request;
+      }
+
+      @Override
+      public long getDelay(TimeUnit unit) {
+        return unit.convert(deadline - System.currentTimeMillis(), TimeUnit.MILLISECONDS);
+      }
+
+      public long getDeadline() {
+        return deadline;
+      }
+
+      public int compareTo(Delayed other) {
+        long diff = getDelay(TimeUnit.MILLISECONDS) - other.getDelay(TimeUnit.MILLISECONDS);
+        return diff < 0 ? -1 : (diff > 0 ? +1 : 0);
+      }
+
+      public Object request() {
+        return request;
+      }
+    }
+
+    public RelayThread(InstrumentedConsensusServiceImpl cs) {
+      this.cs = cs;
+    }
+
+    public void run() {
+      while (true) {
+        try {
+          DelayedRequest request = queue.take();
+          LOG.info("-------- [" + getIdentity() + "] draining request " + request.request());
+          if (request.request() instanceof AppendRequest) {
+            cs.reallyAppendEntries((AppendRequest) request.request());
+          } else if (request.request() instanceof VoteRequest) {
+            cs.reallyRequestVote((VoteRequest) request.request());
+          } else {
+            LOG.error(MarkerFactory.getMarker("FATAL"),
+                    "Incorrect request type found : " + request.request());
+            System.exit(1);
+          }
+        } catch (InterruptedException ex) {
+          LOG.warn("RelayThread is interrupted; time to die!");
+          return;
+        } catch (Exception ex) {
+          LOG.warn("Caught exception:\n" + ex);
+        }
+      }
+    }
+
+    public <T extends Object> T queueRequest(T request, long delay) {
+      try {
+        LOG.info("-------- [" + getIdentity() + "] queueing request " + request + " with delay " + delay + " ms");
+        queue.offer(new DelayedRequest((Object)request, delay));
+      } catch (NullPointerException ex) {
+      }
+      return request;
+    }
+  }
+
+  public void setPacketDelay(long delay) {
+    packetDelayMap.clear();
+    packetDelayMap.put("*", delay);
+  }
+
+  public void setPacketDelay(String src, long delay) {
+    packetDelayMap.put(src, delay);
+  }
+
+  public long getPacketDelay(String src) {
+    Long delay = packetDelayMap.get(src);
+    if (delay == null) {
+      delay = packetDelayMap.get("*");
+      if (delay == null) {
+        return 0L;
+      } else {
+        return delay;
+      }
+    } else {
+      return delay;
+    }
+  }
+
+  public synchronized void setHiccup(
+      final String src, boolean inHiccup
+  ) {
+    hiccupMap.put(src, inHiccup);
+  }
+
+  public synchronized boolean inHiccup(final String src) {
+    Boolean inHiccup = hiccupMap.get(src);
+    return inHiccup != null ? (boolean) inHiccup : false;
+  }
+
+  public synchronized void setPacketDropRate(
+      final String src, final double rate
+  ) {
+    packetDropRateMap.put(src, rate);
+  }
+
+  public synchronized void setPacketDropStyle(final PacketDropStyle style) {
+    packetDropRateMap.clear();
+    if (style == null) {
+      return;
+    }
+    switch (style) {
+      case ALWAYS:
+        packetDropRateMap.put("*", 2.0);
+        break;
+      case RANDOM:
+        packetDropRateMap.put("*", 0.5);
+        break;
+      case NONE:
+        packetDropRateMap.put("*", 0.0);
+        break;
+    }
+  }
+
+  private Double getPacketDropRate(String src) {
+    Double dropRate = packetDropRateMap.get(src);
+    if (dropRate == null) {
+      return packetDropRateMap.get("*");
+    } else {
+      return dropRate;
+    }
+  }
+
+  public ListenableFuture<AppendResponse> reallyAppendEntries(AppendRequest appendRequest) {
+    return super.appendEntries(appendRequest);
+  }
+
+  public ListenableFuture<VoteResponse> reallyRequestVote(VoteRequest voteRequest) {
+    return super.requestVote(voteRequest);
+  }
+
+  public long getNormalPacketDropCount() {
+    return normalPacketDropCount.get();
+  }
+
+  public long getHiccupPacketDropCount() {
+    return hiccupPacketDropCount.get();
+  }
+
+  public long getPacketDropCount() {
+    return getHiccupPacketDropCount() + getNormalPacketDropCount();
+  }
+
+  @Override
+  public ListenableFuture<AppendResponse> appendEntries(AppendRequest appendRequest) {
+    appendRequest.createAppendResponse();
+    String src = appendRequest.getLeaderId().getHostId();
+    if (inHiccup(src)) {
+        LOG.debug("-------- [" + getIdentity() + "] Dropping packet due to hiccup: " + appendRequest);
+        hiccupPacketDropCount.incrementAndGet();
+        return appendRequest.getResponse();
+    }
+    Double dropRate = getPacketDropRate(src);
+    if (dropRate != null && random.nextDouble() < dropRate) {
+        LOG.debug("-------- [" + getIdentity() + "] Dropping packet " + appendRequest);
+        normalPacketDropCount.incrementAndGet();
+        return appendRequest.getResponse();
+    }
+    return relayThread.queueRequest(appendRequest, getPacketDelay(src)).getResponse();
+  }
+
+  @Override
+  public ListenableFuture<VoteResponse> requestVote(VoteRequest voteRequest) {
+    voteRequest.createVoteResponse();
+    String src = voteRequest.getAddress();
+    if (inHiccup(src)) {
+        LOG.debug("-------- [" + getIdentity() + "] Dropping packet due to hiccup: " + voteRequest);
+        hiccupPacketDropCount.incrementAndGet();
+        return voteRequest.getResponse();
+    }
+    Double dropRate = getPacketDropRate(src);
+    if (dropRate != null && random.nextDouble() < dropRate) {
+        normalPacketDropCount.incrementAndGet();
+        LOG.debug("-------- [" + getIdentity() + "] Dropping packet " + voteRequest);
+        return voteRequest.getResponse();
+    }
+    return relayThread.queueRequest(voteRequest, getPacketDelay(src)).getResponse();
+  }
+}


[03/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/RaftTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/RaftTestUtil.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/RaftTestUtil.java
new file mode 100644
index 0000000..d259308
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/RaftTestUtil.java
@@ -0,0 +1,751 @@
+package org.apache.hadoop.hbase.consensus;
+
+import com.facebook.swift.service.ThriftEventHandler;
+import com.facebook.swift.service.ThriftServerConfig;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.client.QuorumThriftClientAgent;
+import org.apache.hadoop.hbase.consensus.log.CommitLogManagerInterface;
+import org.apache.hadoop.hbase.consensus.log.InMemoryLogManager;
+import org.apache.hadoop.hbase.consensus.log.TransactionLogManager;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.quorum.AggregateTimer;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.rpc.PeerStatus;
+import org.apache.hadoop.hbase.consensus.server.ConsensusServiceImpl;
+import org.apache.hadoop.hbase.consensus.server.InstrumentedConsensusServiceImpl;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.DaemonThreadFactory;
+import org.apache.hadoop.hbase.util.serial.SerialExecutorService;
+import org.apache.hadoop.net.DNS;
+import org.apache.log4j.Level;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * A test utility for Unit Testing RAFT protocol by itself.
+ */
+public class RaftTestUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(RaftTestUtil.class);
+
+  ConcurrentHashMap<String, LocalConsensusServer> servers;
+
+  private static int DEFAULT_RAFT_CONSENSUS_START_PORT_NUMBER = 60080;
+
+  private static int nextPortNumber;
+  private Configuration conf = HBaseConfiguration.create();
+  ExecutorService pool;
+  private String raftDirectory;
+  private AtomicLong serverRestartCount = new AtomicLong(0);
+  private Class<? extends RaftQuorumContext> raftQuorumContextClass =
+    RaftQuorumContext.class;
+
+  private boolean usePersistentLog = false;
+
+  public static String LOCAL_HOST;
+
+  static {
+    try {
+      LOCAL_HOST = DNS.getDefaultHost("default", "default");
+    } catch (UnknownHostException e) {
+      e.printStackTrace();
+    }
+    ;
+  }
+
+  private long seedIndex = HConstants.UNDEFINED_TERM_INDEX;
+
+  /**
+   * Creates a cluster of num nodes and starts the Consensus Server on each of
+   * the nodes.
+   * @param num number of nodes in the cluster
+   * @return true if cluster was successfully created, else false
+   */
+  public boolean createRaftCluster(int num) {
+
+    raftDirectory = "/tmp/" + System.currentTimeMillis();
+
+    if (num % 2 == 0) {
+      return false;
+    }
+    nextPortNumber =
+      conf.getInt(HConstants.REGIONSERVER_PORT, HConstants.DEFAULT_REGIONSERVER_PORT) +
+        HConstants.CONSENSUS_SERVER_PORT_JUMP;
+
+    for (int i = 0; i < num; i++) {
+      addServer(nextPortNumber++);
+    }
+
+    for (LocalConsensusServer server : servers.values()) {
+      server.startService();
+    }
+    pool = Executors.newFixedThreadPool(
+      HConstants.DEFAULT_QUORUM_CLIENT_NUM_WORKERS,
+      new DaemonThreadFactory("QuorumClient-"));
+    return true;
+  }
+
+  public LocalConsensusServer addServer(int port) {
+    List<ThriftEventHandler> handlers = new ArrayList<>();
+    conf.setLong(HConstants.QUORUM_CLIENT_COMMIT_DEADLINE_KEY, 20000);
+    ThriftServerConfig config =  new ThriftServerConfig()
+      .setWorkerThreads(conf.getInt(HConstants.CONSENSUS_SERVER_WORKER_THREAD,
+        HConstants.DEFAULT_CONSENSUS_SERVER_WORKER_THREAD))
+      .setPort(port)
+      .setIoThreadCount(conf.getInt(HConstants.CONSENSUS_SERVER_IO_THREAD,
+        HConstants.DEFAULT_CONSENSUS_SERVER_IO_THREAD));
+    InstrumentedConsensusServiceImpl service =
+      (InstrumentedConsensusServiceImpl) ConsensusServiceImpl.
+        createTestConsensusServiceImpl();
+
+    LocalConsensusServer server = new LocalConsensusServer(service, handlers, conf);
+    server.initialize(config);
+    HServerAddress addr = new HServerAddress(LOCAL_HOST + ":" + port);
+    getServers().put(addr.getHostAddressWithPort(), server);
+    service.setIdentity(LOCAL_HOST + ":" + port);
+    server.startService();
+    return server;
+  }
+
+  public static int getNextPortNumber() {
+    return nextPortNumber;
+  }
+
+  public static void setNextPortNumber(int nextPortNumber) {
+    RaftTestUtil.nextPortNumber = nextPortNumber;
+  }
+
+  public void setRaftQuorumContextClass(Class<? extends RaftQuorumContext> clazz) {
+    raftQuorumContextClass = clazz;
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  public ConcurrentHashMap<String, LocalConsensusServer> getServers() {
+    if (servers == null) {
+      servers = new ConcurrentHashMap<>();
+    }
+    return servers;
+  }
+
+  /**
+   * Creates a Raft context.
+   *
+   * Allows override.
+   */
+  public RaftQuorumContext createRaftQuorumContext(QuorumInfo info, Configuration conf,
+      HServerAddress address, LocalConsensusServer server) {
+    String mBeansPrefix = "Test.";
+    try {
+      return raftQuorumContextClass.getConstructor(QuorumInfo.class, Configuration.class,
+        HServerAddress.class, String.class, AggregateTimer.class, SerialExecutorService.class,
+        ExecutorService.class).newInstance(info, conf, address,
+        mBeansPrefix, server.aggregateTimer, server.serialExecutorService,
+        server.getExecServiceForThriftClients());
+    } catch (Exception e) {
+      LOG.error("Could not construct a RaftQuorumContext of type: " + raftQuorumContextClass
+          + ", because of: ", e);
+      return null;
+    }
+  }
+
+  /**
+   * Creates a new quorum from the given quorum info object.
+   * @param regionInfo
+   */
+  public void addQuorum(final HRegionInfo regionInfo, List<int[]> mockLogs)
+          throws IOException {
+    int i = 0;
+    int[] mockLog;
+    for (LocalConsensusServer server : servers.values()) {
+      mockLog = null;
+      if (mockLogs != null) {
+        mockLog = mockLogs.get(i++);
+      }
+      addQuorumForServer(server, regionInfo.getQuorumInfo(), mockLog);
+    }
+  }
+
+  public void addQuorumForServer(final LocalConsensusServer server,
+                                 final QuorumInfo info, int[] mockLog)
+          throws IOException {
+    HServerAddress consensusServerAddress = new HServerAddress(LOCAL_HOST,
+      server.getThriftServer().getPort());
+
+    Configuration conf = HBaseConfiguration.create(getConf());
+    conf.set(HConstants.RAFT_TRANSACTION_LOG_DIRECTORY_KEY,
+      raftDirectory + "/" +
+        consensusServerAddress.getHostAddressWithPort() + "/wal");
+    conf.set(HConstants.RAFT_METADATA_DIRECTORY_KEY,
+      raftDirectory + "/" +
+        consensusServerAddress.getHostAddressWithPort() + "/metadata");
+
+    conf.setInt(
+      HConstants.RAFT_LOG_DELETION_INTERVAL_KEY,
+      100);
+    conf.setLong(HConstants.CONSENSUS_PUSH_APPEND_MAX_BATCH_BYTES_KEY,
+      256 * 1024L);
+    conf.setInt(HConstants.CONSENSUS_PUSH_APPEND_MAX_BATCH_LOGS_KEY, 32);
+
+    conf.setLong(HConstants.RAFT_PEERSERVER_HANDLE_RPC_TIMEOUT_MS, 100);
+
+    RaftQuorumContext context = createRaftQuorumContext(info, conf,
+      consensusServerAddress, server);
+    RaftQuorumContext originalContext = server.getHandler().addRaftQuorumContext(context);
+    if (originalContext != null) {
+      // Stop the original consensus quorum
+      originalContext.stop(true);
+    }
+
+    if (mockLog != null) {
+      mockTransactionLog(context, mockLog);
+    }
+  }
+
+  public long getServerRestartCount() {
+    return serverRestartCount.get();
+  }
+
+  public void checkHealth(HRegionInfo regionInfo, boolean reset) {
+    for (LocalConsensusServer server : servers.values()) {
+      boolean healthy = false;
+      LOG.info("Checking the health of ThriftServer for " + server + " ......");
+      try {
+        healthy = checkHealth(regionInfo, server);
+      } catch (Exception ex) {
+        LOG.error("Failed to check the status for " + server, ex);
+      }
+      LOG.info("ThriftServer for " + server + " is " + (healthy ? "healthy" : "unhealthy"));
+      if (!healthy && reset) {
+        try {
+          LOG.info("Restarting an unhealthy ThriftServer for " + server);
+          serverRestartCount.incrementAndGet();
+          server.restartService();
+        } catch (Throwable stone) {
+          LOG.error("Caught an error while trying to restart " + server, stone);
+        }
+      }
+    }
+  }
+
+  public boolean checkHealth(HRegionInfo regionInfo, LocalConsensusServer server) throws Exception {
+    HServerAddress consensusServerAddress = new HServerAddress(LOCAL_HOST,
+      server.getThriftServer().getPort());
+    int timeout = 5000;
+    LOG.info("Getting QuorumThriftClientAgent for " + consensusServerAddress);
+    QuorumThriftClientAgent agent = new QuorumThriftClientAgent(
+        consensusServerAddress.toString(), timeout, timeout, timeout, 3);
+    LOG.info("QuorumThriftClientAgent for " + consensusServerAddress + " = " + agent);
+    PeerStatus status = agent.getPeerStatus(regionInfo.getEncodedName());
+    LOG.info("PeerStatus for " + consensusServerAddress + ": " + status);
+    return status != null;
+  }
+
+  public RaftQuorumContext restartLocalConsensusServer(LocalConsensusServer server,
+                                                       final HRegionInfo regionInfo,
+                                                       final String contextAddress)
+
+    throws IOException {
+    server.restartService();
+    getServers().put(contextAddress, server);
+
+
+    HServerAddress consensusServerAddress = new HServerAddress(LOCAL_HOST,
+      server.getThriftServer().getPort());
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(HConstants.RAFT_TRANSACTION_LOG_DIRECTORY_KEY, raftDirectory +
+      "/"+ consensusServerAddress.getHostAddressWithPort() + "/wal");
+    conf.set(HConstants.RAFT_METADATA_DIRECTORY_KEY,
+      raftDirectory + "/" +
+        consensusServerAddress.getHostAddressWithPort() + "/metadata");
+    conf.setInt(
+      HConstants.RAFT_LOG_DELETION_INTERVAL_KEY,
+      100);
+    RaftQuorumContext context = createRaftQuorumContext(
+            regionInfo.getQuorumInfo(), conf, consensusServerAddress, server);
+    context.initializeLog();
+    context.reseedStartIndex(seedIndex);
+    context.startStateMachines();
+    server.getHandler().addRaftQuorumContext(context);
+    return context;
+  }
+
+  public void startQuorum(final HRegionInfo regionInfo)
+    throws IOException {
+    for (LocalConsensusServer server : servers.values()) {
+      RaftQuorumContext context =
+        server.getHandler().getRaftQuorumContext(regionInfo.getEncodedName());
+      context.initializeLog();
+      context.reseedStartIndex(seedIndex);
+      context.startStateMachines();
+    }
+  }
+
+  /**
+   * Stops all the consensus servers.
+   */
+  public void shutdown() {
+    if (servers != null) {
+      for (LocalConsensusServer server : servers.values()) {
+        server.stopService();
+
+        for (RaftQuorumContext context :
+          server.getHandler().getQuorumContextMapSnapshot().values()) {
+          context.stop(true);
+        }
+      }
+      servers = null;
+    }
+    if (pool != null) {
+      pool.shutdownNow();
+    }
+  }
+
+  public static void setLogging(final Level level) {
+    List<String> modules = Arrays.asList(
+      "com.facebook.nifty.core",
+      "org.apache.hadoop.hbase.consensus.fsm",
+      "org.apache.hadoop.hbase.consensus.raft",
+      "org.apache.hadoop.hbase.consensus.server",
+      "org.apache.hadoop.hbase.consensus.server.peer",
+      "org.apache.hadoop.hbase.consensus.server.peer.states",
+      "org.apache.hadoop.hbase.consensus.log",
+      "org.apache.hadoop.hbase.consensus.quorum"
+    );
+    for (String module : modules) {
+      org.apache.log4j.Logger.getLogger(module).setLevel(level);
+    }
+  }
+
+  public static void disableVerboseLogging() {
+    setLogging(Level.ERROR);
+  }
+
+  public static void enableVerboseLogging() {
+    setLogging(Level.DEBUG);
+  }
+
+  public QuorumClient getQuorumClient(QuorumInfo regionInfo) throws IOException {
+    return new QuorumClient(regionInfo, conf, pool);
+  }
+
+  private void mockTransactionLog(final RaftQuorumContext context, int[] terms)
+          throws IOException {
+
+    CommitLogManagerInterface log = null;
+    if (usePersistentLog) {
+      log = new TransactionLogManager(context.getConf(), context.getQuorumName()
+      , HConstants.UNDEFINED_TERM_INDEX);
+    } else {
+      log = new InMemoryLogManager();
+      context.setLogManager(log);
+    }
+
+    log.initialize(context);
+    int index = 0;
+    List<KeyValue> kvs = new ArrayList<>();
+    kvs.add(KeyValue.LOWESTKEY);
+    WALEdit edit = new WALEdit(kvs);
+    List<WALEdit> entries = new ArrayList<>();
+    entries.add(edit);
+
+    long prevTerm = HConstants.UNDEFINED_TERM_INDEX;
+    long committedIndex = HConstants.UNDEFINED_TERM_INDEX;
+    for (int term : terms) {
+      // Entry can only be considered committed to the Data Store if the current
+      // term has at least more than one entry committed to the transaction log
+      if (prevTerm == term) {
+        committedIndex = index - 1;
+      }
+      log.append(new EditId(term, index), committedIndex,
+              WALEdit.serializeToByteBuffer(entries, System.currentTimeMillis(),
+                      Compression.Algorithm.NONE));
+      ++index;
+      prevTerm = term;
+    }
+  }
+
+  // shop-lifted from http://en.wikibooks.org/wiki/Algorithm_Implementation/Strings/Longest_common_substring
+  // we should really use longest common subsequence of EditIds. what the heck.
+  // let's get it running first.
+
+  private static String longestCommonSubstring(String S1, String S2)
+  {
+      int Start = 0;
+      int Max = 0;
+      for (int i = 0; i < S1.length(); i++)
+      {
+          for (int j = 0; j < S2.length(); j++)
+          {
+              int x = 0;
+              while (S1.charAt(i + x) == S2.charAt(j + x))
+              {
+                  x++;
+                  if (((i + x) >= S1.length()) || ((j + x) >= S2.length())) {
+                    break;
+                  }
+              }
+              if (x > Max)
+              {
+                  Max = x;
+                  Start = i;
+              }
+           }
+      }
+      return S1.substring(Start, (Start + Max));
+  }
+
+  public static String summarizeInterval(String str) {
+    String[] list = str.split(",");
+    if (list == null || list.length == 0) {
+      return "[]";
+    } else if (list.length == 1) {
+      return "[" + list[0].trim() + "]";
+    } else {
+      return "[" + list[0].trim() + ", ..., " + list[list.length-1].trim() + "]";
+    }
+  }
+
+  public void dumpStates(final HRegionInfo info) {
+    LOG.info("---- logs for region " + info + ":");
+    List<String> logs = new ArrayList<String>();
+    List<Integer> ports = new ArrayList<Integer>();
+    for (LocalConsensusServer server : servers.values()) {
+      final CommitLogManagerInterface log =
+        server.getHandler().getRaftQuorumContext(info.getEncodedName()).getLogManager();
+      logs.add(log.dumpLogs(-1));
+      ports.add(server.getThriftServer().getPort());
+    }
+    String common = "";
+    if (logs.size() > 1) {
+      common = longestCommonSubstring(logs.get(0), logs.get(1));
+      for (int i=2; i<logs.size(); i++) {
+        common = longestCommonSubstring(common, logs.get(i));
+      }
+    }
+    String newcommon = summarizeInterval(common);
+    for (int i=0; i<logs.size(); i++) {
+      String str = logs.get(i);
+      if (common.length() > 10) {
+        logs.set(i, str.replace(common, newcommon + " "));
+      }
+    }
+    // try again
+    for (int i=0; i<logs.size(); i++) {
+      LOG.info("    ---- logs for " + LOCAL_HOST + ":" + ports.get(i) + " = " + logs.get(i));
+    }
+  }
+
+  public boolean verifyLogs(final QuorumInfo info, int majorityCount) {
+    return verifyLogs(info, majorityCount, true);
+  }
+
+  public boolean verifyLogs(final QuorumInfo info, int majorityCount, boolean verbose) {
+    return verifyLogs(servers.values(), info, majorityCount, verbose);
+  }
+
+
+  public static boolean verifyLogs(final Collection<LocalConsensusServer> consensusServers,
+      final QuorumInfo info, int majorityCount, boolean verbose) {
+    if (verbose) {
+      LOG.debug("Verify logs for " + info.getQuorumName());
+    }
+
+    int quorumSize = info.getPeersWithRank().size();
+    List<CommitLogManagerInterface> logs = new ArrayList<>(quorumSize);
+    for (LocalConsensusServer server : consensusServers) {
+      RaftQuorumContext context =
+        server.getHandler().getRaftQuorumContext(info.getQuorumName());
+
+      if (context == null ||
+        info.getPeersWithRank().get(RaftUtil.getHRegionServerAddress(
+          new HServerAddress(RaftTestUtil.LOCAL_HOST,
+            server.getThriftServer().getPort())
+        )) == null) {
+        if (verbose) {
+          if (context == null) {
+            LOG.debug("Context is null for " + server);
+          } else {
+            LOG.debug("Given Server is not a part of the quorum: " + server);
+          }
+        }
+        continue;
+      }
+
+      final CommitLogManagerInterface log = context.getLogManager();
+      logs.add(log);
+
+      System.out.println(context + " ; " + context.getPaxosState() + " ; " + log.getLogState());
+    }
+
+    for (int ii = 0; ii < quorumSize - 1; ii++) {
+      int numMatches = 1;
+      for (int jj = ii + 1; jj < quorumSize; jj++) {
+        numMatches += verifyLog(logs.get(ii), logs.get(jj));
+        if (numMatches == majorityCount) {
+          if (verbose) {
+            LOG.debug(numMatches + " logs match for " + info.getQuorumName());
+          }
+          return true;
+        }
+      }
+    }
+    LOG.debug("Logs don't match for " + info.getQuorumName());
+    return false;
+  }
+
+  public void assertAllServersRunning() throws Exception {
+    for (LocalConsensusServer server : servers.values()) {
+      assert server.getThriftServer().isRunning() == true;
+    }
+  }
+
+  public HRegionInfo initializePeers() {
+    Map<HServerAddress, Integer> peers = new HashMap<>();
+    int rank = servers.size();
+    for (LocalConsensusServer server : servers.values()) {
+      int regionServerPort =
+        server.getThriftServer().getPort() - HConstants.CONSENSUS_SERVER_PORT_JUMP;
+      peers.put(new HServerAddress(RaftTestUtil.LOCAL_HOST, regionServerPort),
+        Math.max(rank--, 0));
+    }
+
+    HTableDescriptor table = new HTableDescriptor(RaftTestUtil.class.getName());
+    byte [] FAMILY = Bytes.toBytes("family");
+    HColumnDescriptor hcd = new HColumnDescriptor(FAMILY).setMaxVersions(Integer.MAX_VALUE);
+    table.addFamily(hcd);
+
+    HRegionInfo regionInfo = new HRegionInfo(table, Bytes.toBytes("00000000"),
+      Bytes.toBytes("ffffffff"));
+    Map<String, Map<HServerAddress, Integer>> peerMap = new HashMap<>();
+    peerMap.put(HRegionInfo.LOCAL_DC_KEY, peers);
+    regionInfo.setPeers(peerMap);
+
+    return regionInfo;
+  }
+
+  public HRegionInfo resetPeers(HRegionInfo regionInfo, List<int[]> logs) throws Exception {
+
+    addQuorum(regionInfo, logs);
+    return regionInfo;
+  }
+
+  public void setSeedIndex(long seedIndex) {
+    this.seedIndex = seedIndex;
+  }
+
+  public static List<int[]> getScratchSetup(int quorumSize) {
+    List<int[]> logs = new ArrayList<>(quorumSize);
+    for (int i = 0; i < quorumSize; i++) {
+      logs.add(i, new int[] {1});
+    }
+    return logs;
+  }
+
+  public static List<WALEdit> generateTransaction(int size) {
+    List<WALEdit> testData = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      String payloadSizeName = FileUtils.byteCountToDisplaySize(size);
+      KeyValue kv = KeyValue.generateKeyValue(payloadSizeName, size);
+
+      List<KeyValue> kvs = new ArrayList<>();
+      for (int j = 0; j < 10; j++) {
+        kvs.add(kv);
+      }
+
+      testData.add(new WALEdit(kvs));
+    }
+    return testData;
+  }
+
+  /**
+   * Verifies whether the two logs are same or not
+   * @param log1
+   * @param log2
+   * @return 1 if they are same, else 0
+   */
+  private static int verifyLog(final CommitLogManagerInterface log1,
+                               final CommitLogManagerInterface log2) {
+
+    EditId log1Marker = log1.getLastEditID();
+    EditId log2Marker = log2.getLastEditID();
+
+    while (log1Marker != TransactionLogManager.UNDEFINED_EDIT_ID &&
+      log2Marker != TransactionLogManager.UNDEFINED_EDIT_ID) {
+
+      // Handle the seed file case, where the index is the same but the term
+      // is SEED_TERM.
+      if ((log1Marker.getTerm() != HConstants.SEED_TERM &&
+        log2Marker.getTerm() != HConstants.SEED_TERM &&
+        !log1Marker.equals(log2Marker)) ||
+        log1Marker.getIndex() != log2Marker.getIndex()) {
+        return 0;
+      }
+
+      log1Marker = log1.getPreviousEditID(log1Marker);
+      log2Marker = log2.getPreviousEditID(log2Marker);
+    }
+
+    if (log1Marker != TransactionLogManager.UNDEFINED_EDIT_ID ||
+      log2Marker != TransactionLogManager.UNDEFINED_EDIT_ID) {
+      return 0;
+    }
+    return 1;
+  }
+
+  public boolean simulatePacketDropForServer(final HRegionInfo regionInfo, int rank,
+                                             final InstrumentedConsensusServiceImpl.PacketDropStyle style) {
+    HServerAddress server = null;
+
+    for (HServerAddress s : regionInfo.getPeersWithRank().keySet()) {
+      if (regionInfo.getPeersWithRank().get(s) == rank) {
+        server = s;
+        break;
+      }
+    }
+
+    if (server == null) {
+      return false;
+    }
+
+    InstrumentedConsensusServiceImpl service = (InstrumentedConsensusServiceImpl)
+      (servers.get(RaftUtil.getLocalConsensusAddress(server).getHostAddressWithPort()).
+        getHandler());
+    service.setPacketDropStyle(style);
+    return true;
+  }
+
+  public long getHiccupPacketDropCount(final HRegionInfo regionInfo) {
+    long count = 0;
+    for (HServerAddress server : regionInfo.getPeersWithRank().keySet()) {
+      InstrumentedConsensusServiceImpl service = (InstrumentedConsensusServiceImpl)
+              (servers.get(RaftUtil.getLocalConsensusAddress(server).getHostAddressWithPort()).getHandler());
+      count += service.getHiccupPacketDropCount();
+    }
+    return count;
+  }
+
+  public long getPacketDropCount(final HRegionInfo regionInfo) {
+    long count = 0;
+    for (HServerAddress server : regionInfo.getPeersWithRank().keySet()) {
+      InstrumentedConsensusServiceImpl service = (InstrumentedConsensusServiceImpl)
+              (servers.get(RaftUtil.getLocalConsensusAddress(server).getHostAddressWithPort()).getHandler());
+      count += service.getPacketDropCount();
+    }
+    return count;
+  }
+
+  public void setUsePeristentLog(boolean usePersistentLog) {
+    this.usePersistentLog = usePersistentLog;
+  }
+
+  public RaftQuorumContext getRaftQuorumContextByAddress(HRegionInfo regionInfo,
+    String address) {
+    return getServers().get(address).getHandler()
+      .getRaftQuorumContext(regionInfo.getEncodedName());
+  }
+
+  public RaftQuorumContext getRaftQuorumContextByRank(HRegionInfo regionInfo, int rank) {
+    String peerAddress = null;
+    for (HServerAddress addr : regionInfo.getPeersWithRank().keySet()) {
+      if (regionInfo.getPeersWithRank().get(addr) == rank) {
+        peerAddress = RaftUtil.getLocalConsensusAddress(addr).getHostAddressWithPort();
+      }
+    }
+
+    return getServers().get(peerAddress).getHandler().
+      getRaftQuorumContext(regionInfo.getEncodedName());
+  }
+
+  public LocalConsensusServer stopLocalConsensusServer(HRegionInfo regionInfo, int rank) {
+    String peerAddress = null;
+    for (HServerAddress addr : regionInfo.getPeersWithRank().keySet()) {
+      if (regionInfo.getPeersWithRank().get(addr) == rank) {
+        peerAddress = RaftUtil.getLocalConsensusAddress(addr).getHostAddressWithPort();
+      }
+    }
+
+    return stopLocalConsensusServer(peerAddress);
+  }
+
+  public LocalConsensusServer stopLocalConsensusServer(String peerAddress) {
+    LocalConsensusServer server = getServers().remove(peerAddress);
+    server.stopService();
+
+    return server;
+  }
+
+  public void printStatusOfQuorum(HRegionInfo regionInfo) {
+    System.out.println(" ======= Status Update =========");
+    for (LocalConsensusServer server : getServers().values()) {
+      RaftQuorumContext context =
+        server.getHandler().getRaftQuorumContext(regionInfo.getEncodedName());
+      System.out.println(context + " ; " + context.getPaxosState() + " ; " + context.getLogState());
+    }
+    System.out.println(" ================");
+  }
+
+  public List<RaftQuorumContext> getQuorumContexts(
+    final HRegionInfo regionInfo) {
+      Set<HServerAddress> replias = regionInfo.getPeersWithRank().keySet();
+      List<RaftQuorumContext> contexts = new ArrayList<>(replias.size());
+
+      for (HServerAddress address : replias) {
+        String consensusServerAddress =
+                RaftUtil.getLocalConsensusAddress(address).
+                        getHostAddressWithPort();
+        if (getServers().containsKey(consensusServerAddress)) {
+          contexts.add(getRaftQuorumContextByAddress(regionInfo,
+                  consensusServerAddress));
+        }
+      }
+      return contexts;
+    }
+
+  public RaftQuorumContext getLeaderQuorumContext(HRegionInfo regionInfo) {
+    RaftQuorumContext leader = null;
+    for (RaftQuorumContext context : getQuorumContexts(regionInfo)) {
+      if (context.isLeader()) {
+        leader = context;
+      }
+    }
+    return leader;
+  }
+
+  public void waitForLeader(final HRegionInfo regionInfo)
+          throws InterruptedException {
+    while (getLeaderQuorumContext(regionInfo) == null) {
+      Thread.sleep(500);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/ReplicationLoadForUnitTest.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/ReplicationLoadForUnitTest.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/ReplicationLoadForUnitTest.java
new file mode 100644
index 0000000..e4cf944
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/ReplicationLoadForUnitTest.java
@@ -0,0 +1,90 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public class ReplicationLoadForUnitTest {
+
+  private volatile int transactionNums = 0;
+  private ThreadPoolExecutor loadGeneratorExecutor;
+  private volatile boolean stop = false;
+
+  private RaftTestUtil util;
+  private HRegionInfo regionInfo;
+  private QuorumClient client;
+  private int quorumSize = 5;
+  private int majoritySize = 3;
+
+  private volatile long sleepTime = 50;
+
+  public ReplicationLoadForUnitTest(HRegionInfo regionInfo, QuorumClient client,
+                                    RaftTestUtil util, int quorumSize, int majoritySize) {
+    this.regionInfo = regionInfo;
+    this.client = client;
+    this.util = util;
+    this.quorumSize = quorumSize;
+    this.majoritySize = majoritySize;
+  }
+
+
+  public int makeProgress(long sleepTime, int prevLoad) throws InterruptedException {
+    System.out.println("Let the client load fly for " + sleepTime + " ms");
+    Thread.sleep(sleepTime);
+    util.printStatusOfQuorum(regionInfo);
+
+    while (transactionNums <= prevLoad) {
+      System.out.println("No Progress ! prev " + prevLoad + " current " + transactionNums);
+      util.printStatusOfQuorum(regionInfo);
+      Thread.sleep(sleepTime);
+    }
+
+    return transactionNums;
+  }
+
+  public void startReplicationLoad(final int progressInterval) {
+    loadGeneratorExecutor = new ThreadPoolExecutor(1, 1,
+      0L, TimeUnit.MILLISECONDS,
+      new LinkedBlockingQueue<Runnable>());
+
+    loadGeneratorExecutor.submit(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          while (!stop) {
+            try {
+              client.replicateCommits(RaftTestUtil.generateTransaction(1 * 1024));
+              if ((++transactionNums) % progressInterval == 0) {
+                System.out.println("Sent " + transactionNums + " transactions to the quorum");
+                util.printStatusOfQuorum(regionInfo);
+              }
+
+            } catch (Exception e) {
+              System.out.println("Failed to replicate transactions due to " + e);
+            }
+
+            Thread.sleep(sleepTime);
+          }
+        } catch (InterruptedException e) {
+          System.out.println("Failed to replicate transactions due to  " + e);
+        }
+      }
+    });
+  }
+
+  public void stopReplicationLoad() throws InterruptedException {
+    stop = true;
+    loadGeneratorExecutor.shutdownNow();
+    loadGeneratorExecutor.awaitTermination(10, TimeUnit.SECONDS);
+    System.out.println("Shutdown the replication load and " + transactionNums + " transactions " +
+      "have been successfully replicated");
+  }
+
+  public void slowDownReplicationLoad() throws InterruptedException {
+    sleepTime = 200;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/SimulatedNetworkTester.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/SimulatedNetworkTester.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/SimulatedNetworkTester.java
new file mode 100644
index 0000000..64889d9
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/SimulatedNetworkTester.java
@@ -0,0 +1,48 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * A test utility for Unit Testing RAFT protocol by itself.
+ */
+//@RunWith(value = Parameterized.class)
+public class SimulatedNetworkTester {
+  private static final Logger LOG = LoggerFactory.getLogger(
+          SimulatedNetworkTester.class);
+  private final LocalTestBed testbed;
+
+  @Before
+  public void setUpBeforeClass() throws Exception {
+    testbed.start();
+  }
+
+  @After
+  public void tearDownAfterClass() throws Exception {
+    testbed.stop();
+  }
+
+  /**
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return RaftTestDataProvider.getRaftBasicLogTestData();
+  }
+  */
+
+  public SimulatedNetworkTester() {
+    testbed = new LocalTestBed();
+  }
+
+  /**
+   * This test function is to test the protocol is able to make progress within a period of time
+   * based on the on-disk transaction log
+   */
+  @Test(timeout=1000000)
+  public void testConsensusProtocol() throws Exception {
+    testbed.runConsensusProtocol();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicCommit.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicCommit.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicCommit.java
new file mode 100644
index 0000000..b54c2a6
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicCommit.java
@@ -0,0 +1,121 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class TestBasicCommit {
+  private static int QUORUM_SIZE = 3;
+  private static int QUORUM_MAJORITY = 2;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private static QuorumClient client;
+  private static volatile int transactionNum = 0;
+  private ReplicationLoadForUnitTest loader;
+
+  private int failureInterval = 1;
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.disableVerboseLogging();
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, RAFT_TEST_UTIL.getScratchSetup(QUORUM_SIZE));
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+    transactionNum = 0;
+    loader = new ReplicationLoadForUnitTest(regionInfo, client, RAFT_TEST_UTIL, QUORUM_SIZE,
+      QUORUM_MAJORITY);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Test(timeout=120000)
+  public void testAllPeerFailureAndRecovery()
+    throws InterruptedException, IOException {
+    int failureCnt = 0;
+    final long sleepTime =
+      2 * HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS * failureInterval;
+
+    // Start the client load
+    loader.startReplicationLoad(100);
+
+    // Let the traffic fly for a while
+    transactionNum = loader.makeProgress(sleepTime, transactionNum);
+
+    // Get all the quorum contexts from rank 3 to rank 1
+    RaftQuorumContext c3 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 3);
+    RaftQuorumContext c2 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 2);
+    RaftQuorumContext c1 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 1);
+
+    // Shutdown 1st quorum member whose rank is 1.
+    System.out.println("Stopping one quorum member: " + c1);
+    LocalConsensusServer s1 = RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, 1);
+
+    // Shutdown 2nd quorum member whose rank 2
+    System.out.println("Stopping another quorum member: " + c2);
+    LocalConsensusServer s2 = RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, 2);
+
+    // Sleep for some time to make sure the leader is stuck in retry
+
+    Thread.sleep(HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS);
+
+    // Shutdown 3rd quorum member whose rank 3
+    System.out.println("Stopping another quorum member: " + c3);
+    LocalConsensusServer s3 = RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, 3);
+
+    // Restart 3
+    c3 = RAFT_TEST_UTIL.restartLocalConsensusServer(s3, regionInfo, c3.getMyAddress());
+    System.out.println("Restarted one quorum member: " + c3);
+
+    // Restart 2
+    c2 = RAFT_TEST_UTIL.restartLocalConsensusServer(s2, regionInfo, c2.getMyAddress());
+    System.out.println("Restarted one quorum member: " + c2);
+
+    // Restart 1
+    c1 = RAFT_TEST_UTIL.restartLocalConsensusServer(s1, regionInfo, c1.getMyAddress());
+    System.out.println("Restarted one quorum member: " + c1);
+
+    // Let the traffic fly for a while
+    if ((++failureCnt % failureInterval) == 0) {
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+
+      while(!c3.isLeader()) {
+        System.out.println("Wait for the rank 3 to take over the leadership");
+        Thread.sleep(sleepTime);
+      }
+      Assert.assertTrue("Rank 3 shall be the leader of the quorum", c3.isLeader());
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+    }
+
+    loader.slowDownReplicationLoad();
+
+    // Verify logs are identical across all the quorum members
+    while (!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE)) {
+      Thread.sleep(10 * 1000);
+      System.out.println("Verifying logs ....");
+      Assert.assertTrue("Rank 3 shall be the leader of the quorum", c3.isLeader());
+    }
+
+    // Stop the client load
+    loader.stopReplicationLoad();
+
+    System.out.println(transactionNum + " transactions have been successfully replicated");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicLeaderElection.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicLeaderElection.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicLeaderElection.java
new file mode 100644
index 0000000..de31eea
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicLeaderElection.java
@@ -0,0 +1,74 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Collection;
+import java.util.List;
+
+import static junit.framework.Assert.assertEquals;
+
+@RunWith(value = Parameterized.class)
+public class TestBasicLeaderElection {
+
+  private static int QUORUM_SIZE = 5;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private final List<int[]> mockLogs;
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, mockLogs);
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return RaftTestDataProvider.getRaftBasicLogTestData();
+  }
+
+  public TestBasicLeaderElection(List<int[]> logs) {
+    this.mockLogs = logs;
+  }
+
+  /**
+   * This test function is to test the protocol is able to make progress within a period of time
+   * based on the on-disk transaction log
+   */
+  @Test(timeout=50000)
+  public void testConsensusProtocol() {
+    int leaderCnt;
+    do {
+      leaderCnt = 0;
+      try {
+      // Sleep for MAX_TIMEOUT time for leader election to complete
+      Thread.sleep(HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS * 2);
+      } catch (InterruptedException e) {}
+
+      for (LocalConsensusServer server : RAFT_TEST_UTIL.getServers().values()) {
+        RaftQuorumContext c = server.getHandler().getRaftQuorumContext(regionInfo.getEncodedName());
+        if (c.isLeader()) {
+          leaderCnt++;
+        }
+      }
+    }
+    while(!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE) && leaderCnt != 1);
+    assertEquals("There should be only one leader", 1, leaderCnt);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerFailure.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerFailure.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerFailure.java
new file mode 100644
index 0000000..ceab1f5
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerFailure.java
@@ -0,0 +1,170 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class TestBasicPeerFailure {
+  private static final Logger LOG = LoggerFactory.getLogger(
+          TestBasicPeerFailure.class);
+  private static int QUORUM_SIZE = 5;
+  private static int QUORUM_MAJORITY = 3;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private static QuorumClient client;
+  private static volatile int transactionNum = 0;
+  private ReplicationLoadForUnitTest loader;
+
+  @Before
+   public void setUp() throws Exception {
+    RAFT_TEST_UTIL.disableVerboseLogging();
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, RAFT_TEST_UTIL.getScratchSetup(QUORUM_SIZE));
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+    transactionNum = 0;
+    loader = new ReplicationLoadForUnitTest(regionInfo, client, RAFT_TEST_UTIL, QUORUM_SIZE,
+      QUORUM_MAJORITY);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Test(timeout=500000)
+  public void testSinglePeerFailureAndRecovery()
+    throws InterruptedException, IOException {
+    simulateFailureEvent(1);
+  }
+
+  @Test(timeout=500000)
+  public void testMultiplePeerFailureAndRecovery()
+    throws InterruptedException, IOException {
+    simulateFailureEvent(2);
+  }
+
+  @Test(timeout=60000)
+  public void testStepDownOnNoProgress() throws InterruptedException {
+    final long sleepTime =
+            2 * HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS;
+    RaftQuorumContext c5 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 5);
+
+    // Start the client load
+    loader.startReplicationLoad(100);
+
+    // Let the traffic fly for a while
+    transactionNum = loader.makeProgress(sleepTime, transactionNum);
+    Assert.assertTrue(c5.isLeader());
+
+    // Stop the majority of replicas
+    for (int i = 0; i < QUORUM_MAJORITY; i++) {
+      System.out.println("Stopping replica with rank " + (i + 1));
+      RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, i + 1);
+    }
+
+    Thread.sleep(2 * HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS);
+    Assert.assertTrue(c5.isLeader());
+
+    // Let the request timeout
+    System.out.printf("Waiting %d seconds for the request to time out\n",
+            HConstants.DEFAULT_APPEND_ENTRIES_TIMEOUT_IN_MILLISECONDS / 1000);
+    Thread.sleep(HConstants.DEFAULT_APPEND_ENTRIES_TIMEOUT_IN_MILLISECONDS);
+    Assert.assertTrue("Leader should step down", c5.isFollower());
+  }
+
+  private void simulateFailureEvent(final int failureInterval)
+    throws InterruptedException, IOException {
+    int failureCnt = 0;
+    final long sleepTime =
+       2 * HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS * failureInterval;
+
+    // Start the client load
+    loader.startReplicationLoad(100);
+
+    // Let the traffic fly for a while
+    transactionNum = loader.makeProgress(sleepTime, transactionNum);
+
+    // Get all the quorum contexts from rank 5 to rank 3
+    RaftQuorumContext c5 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 5);
+    RaftQuorumContext c4 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 4);
+    RaftQuorumContext c3 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 3);
+
+    // Shutdown 1st quorum member whose rank is 5.
+    System.out.println("Stopping one quorum member: " + c5);
+    LocalConsensusServer s5 = RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, 5);
+
+    // Let the traffic fly for a while
+    if ((++failureCnt % failureInterval) == 0) {
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+      Assert.assertTrue("Rank 4 shall be the leader of the quorum", c4.isLeader());
+    }
+
+    // Shutdown 2nd quorum member whose rank 4
+    System.out.println("Stopping another quorum member: " + c4);
+    LocalConsensusServer s4 = RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, 4);
+
+    // Let the traffic fly for a while
+    if ((++failureCnt % failureInterval) == 0) {
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+      Assert.assertTrue("Rank 3 shall be the leader of the quorum", c3.isLeader());
+    }
+
+    // Restart the quorum member whose rank is 4
+    c4 = RAFT_TEST_UTIL.restartLocalConsensusServer(s4, regionInfo, c4.getMyAddress());
+    System.out.println("Restarted one quorum member: " + c4);
+
+    // Let the traffic fly for a while
+    if ((++failureCnt % failureInterval) == 0) {
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+
+      while(!c4.isLeader()) {
+        System.out.println("Wait for the rank 4 to take over the leadership");
+        Thread.sleep(sleepTime);
+      }
+      Assert.assertTrue("Rank 4 shall be the leader of the quorum", c4.isLeader());
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+    }
+    // Restart the quorum member whose rank is 5
+    c5 = RAFT_TEST_UTIL.restartLocalConsensusServer(s5, regionInfo, c5.getMyAddress());
+    System.out.println("Restarted another quorum member: " + c5);
+
+    // Let the traffic fly for a while
+    if (++failureCnt % failureInterval == 0) {
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+      while(!c5.isLeader()) {
+
+        System.out.println("Wait for the rank 5 to take over the leadership");
+        Thread.sleep(sleepTime);
+      }
+      Assert.assertTrue("Rank 5 shall be the leader of the quorum", c5.isLeader());
+    }
+
+    loader.slowDownReplicationLoad();
+
+    // Verify logs are identical across all the quorum members
+    while (!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE)) {
+      Thread.sleep(10 * 1000);
+      System.out.println("Verifying logs ....");
+      Assert.assertTrue("Rank 5 shall be the leader of the quorum", c5.isLeader());
+    }
+
+    // Stop the client load
+    loader.stopReplicationLoad();
+
+    System.out.println(transactionNum + " transactions have been successfully replicated");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerSeeding.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerSeeding.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerSeeding.java
new file mode 100644
index 0000000..02dc112
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerSeeding.java
@@ -0,0 +1,100 @@
+package org.apache.hadoop.hbase.consensus;
+
+import static junit.framework.Assert.fail;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(value = Parameterized.class)
+public class TestBasicPeerSeeding {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    TestBasicPeerSeeding.class);
+  private static final int MULTIPLE_COMMIT_NUM = 10;
+
+  private static final int QUORUM_SIZE = 5;
+  private static final int QUORUM_MAJORITY = 3;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private final List<int[]> mockLogs;
+  private final long seedIndex = 4;
+  private QuorumClient client;
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, mockLogs);
+    RAFT_TEST_UTIL.setSeedIndex(seedIndex);
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return RaftTestDataProvider.getRaftBasicLogTestSeedData();
+  }
+
+  public TestBasicPeerSeeding(List<int[]> logs) {
+    this.mockLogs = logs;
+  }
+
+  @Test(timeout=50000)
+  public void testMultipleCommit() {
+    for (int i = 0 ; i <= MULTIPLE_COMMIT_NUM; i++) {
+      testSingleCommit();
+      LOG.info("Passed the " + i + " commit !");
+    }
+    // Verify all the logs across the quorum are the same
+    while(!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE)) {
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+      try {
+        // Sleep for MAX_TIMEOUT time for leader election to complete
+        Thread.sleep(HConstants.QUORUM_CLIENT_COMMIT_DEADLINE_DEFAULT);
+      } catch (InterruptedException e) {
+        LOG.warn("We are told to exit.");
+        System.exit(1);
+      }
+    }
+  }
+
+  private void testSingleCommit() {
+
+    try {
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+      client.replicateCommits(Arrays.asList(generateTestingWALEdit()));
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+      // Verify all the logs across the majority are the same
+      RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_MAJORITY);
+    } catch (Exception e) {
+      LOG.error("Errors: ", e);
+      fail("Unexpected exception: e");
+    }
+  }
+
+  private static WALEdit generateTestingWALEdit() {
+    KeyValue kv = KeyValue.createFirstOnRow(Bytes.toBytes("TestQuorum"));
+    return new WALEdit(Arrays.asList(kv));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerSlow.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerSlow.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerSlow.java
new file mode 100644
index 0000000..138c674
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicPeerSlow.java
@@ -0,0 +1,230 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.InstrumentedConsensusServiceImpl;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+@RunWith(value = Parameterized.class)
+public class TestBasicPeerSlow {
+  private static int QUORUM_SIZE = 5;
+  private static int QUORUM_MAJORITY = 3;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private static QuorumClient client;
+  private static volatile int transactionNums = 0;
+  private static ThreadPoolExecutor loadGeneratorExecutor;
+  private static volatile boolean stop = false;
+  private static volatile long clientTrafficFrequency;
+  private List<int[]> testEvents;
+
+  @Before
+  public void setUp() throws Exception {
+    clientTrafficFrequency = 10;
+    RaftTestUtil.disableVerboseLogging();
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, RaftTestUtil.getScratchSetup(QUORUM_SIZE));
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+
+    transactionNums = 0;
+    stop = false;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+
+    List<int[]> test1 = Arrays.asList(new int[][]{
+      {1, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+      {2, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+    });
+
+    List<int[]> test2 = Arrays.asList(new int[][]{
+      {3, InstrumentedConsensusServiceImpl.PacketDropStyle.ALWAYS.ordinal()},
+      {4, InstrumentedConsensusServiceImpl.PacketDropStyle.ALWAYS.ordinal()},
+    });
+
+    List<int[]> test3= Arrays.asList(new int[][]{
+      {3, InstrumentedConsensusServiceImpl.PacketDropStyle.ALWAYS.ordinal()},
+      {4, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+    });
+
+    List<int[]> test4 = Arrays.asList(new int[][]{
+      {1, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+      {2, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+      {3, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+    });
+
+    List<int[]> test5= Arrays.asList(new int[][]{
+      {2, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+      {3, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+      {4, InstrumentedConsensusServiceImpl.PacketDropStyle.ALWAYS.ordinal()},
+    });
+
+    List<int[]> test6= Arrays.asList(new int[][]{
+      {1, InstrumentedConsensusServiceImpl.PacketDropStyle.ALWAYS.ordinal()},
+      {2, InstrumentedConsensusServiceImpl.PacketDropStyle.ALWAYS.ordinal()},
+      {3, InstrumentedConsensusServiceImpl.PacketDropStyle.RANDOM.ordinal()},
+    });
+
+    Object[][] data = new Object[][] {
+      { test1 },
+      { test2 },
+      { test3 },
+      { test4 },
+      { test5 },
+      { test6 }
+    };
+    return Arrays.asList(data);
+
+  }
+
+  public TestBasicPeerSlow(List<int[]> testEvents) {
+    this.testEvents = testEvents;
+  }
+
+  @Test(timeout=180000)
+  public void testSingleSlowFollower() throws InterruptedException {
+    final long sleepTime = HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS;
+
+    // Start the client load
+    startReplicationLoad(100);
+
+    // Let the traffic fly for a while
+    makeProgress(sleepTime, transactionNums, true);
+
+    // Simulate all the events
+    for (int [] event: testEvents) {
+      // Get the rank as well as the style
+      assert event.length == 2;
+      int rank = event[0];
+      InstrumentedConsensusServiceImpl.PacketDropStyle style =
+        InstrumentedConsensusServiceImpl.PacketDropStyle.values()[event[1]];
+
+      RaftQuorumContext context = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, rank);
+      RAFT_TEST_UTIL.simulatePacketDropForServer(regionInfo, rank, style);
+
+      System.out.println("Set package drop for the quorum: " + context + " as " + style);
+    }
+
+
+    boolean waitForProgress = true;
+
+    // In case we are affecting majority or more number of peers, then don't
+    // wait for progress in the quorum. Just make sure that we recover, once
+    // things are back to normal
+    if (testEvents.size() >= QUORUM_MAJORITY) {
+      waitForProgress = false;
+    }
+    // Let the traffic fly for a while
+    makeProgress(sleepTime, transactionNums, waitForProgress);
+
+    // Stop the package drop
+    for (int [] event: testEvents) {
+      // Get the rank as well as the style
+      assert event.length == 2;
+      int rank = event[0];
+      InstrumentedConsensusServiceImpl.PacketDropStyle nodrop =
+        InstrumentedConsensusServiceImpl.PacketDropStyle.NONE;
+
+      RaftQuorumContext context = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, rank);
+      RAFT_TEST_UTIL.simulatePacketDropForServer(regionInfo, rank, nodrop);
+
+      System.out.println("Reset package drop for the quorum: " + context + " as " + nodrop);
+    }
+
+    // Slow down the client traffic;
+    clientTrafficFrequency = clientTrafficFrequency * 5;
+
+    // Let the traffic fly for a while
+    makeProgress(sleepTime, transactionNums, true);
+
+    // Verify logs are identical across all the quorum members
+    while (!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE)) {
+      Thread.sleep(5 * 1000);
+      clientTrafficFrequency = clientTrafficFrequency * 10;
+      System.out.println("Verifying logs ....");
+    }
+
+    // Stop the replication load
+    stopReplicationLoad();
+
+    System.out.println(transactionNums + " transactions have been successfully replicated");
+  }
+
+  private int makeProgress(long sleepTime, int prevLoad, boolean waitForProgress)
+    throws InterruptedException {
+    System.out.println("Let the client load fly for " + sleepTime + " ms");
+    Thread.sleep(sleepTime);
+    RAFT_TEST_UTIL.printStatusOfQuorum(regionInfo);
+
+    int i = 0;
+    while ((waitForProgress && transactionNums <= prevLoad) ||
+      (!waitForProgress && (++i <= 1))) {
+      System.out.println("No Progress ! prev " + prevLoad + " current " + transactionNums);
+      RAFT_TEST_UTIL.printStatusOfQuorum(regionInfo);
+      Thread.sleep(sleepTime);
+    }
+
+    return transactionNums;
+  }
+
+  private void startReplicationLoad(final int progressInterval) {
+    loadGeneratorExecutor = new ThreadPoolExecutor(1, 1,
+      0L, TimeUnit.MILLISECONDS,
+      new LinkedBlockingQueue<Runnable>());
+
+    loadGeneratorExecutor.submit(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          while (!stop) {
+            try {
+              client
+                .replicateCommits(RaftTestUtil.generateTransaction(1 * 1024));
+              if ((++transactionNums) % progressInterval == 0) {
+                System.out.println("Sent " + transactionNums +
+                  "transactions to the quorum");
+                RAFT_TEST_UTIL.printStatusOfQuorum(regionInfo);
+              }
+            } catch (Exception e) {
+              System.out.print(String.format("Cannot replicate transaction" + e));
+            }
+            Thread.sleep(clientTrafficFrequency);
+          }
+        } catch (InterruptedException e) {
+          System.out.println("Failed to replicate transactions due to  " + e);
+        }
+      }
+    });
+  }
+
+  private void stopReplicationLoad() throws InterruptedException {
+    stop = true;
+    loadGeneratorExecutor.shutdownNow();
+    loadGeneratorExecutor.awaitTermination(10, TimeUnit.SECONDS);
+    System.out.println("Shutdown the replication load and " + transactionNums + " transactions " +
+      "have been successfully replicated");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicQuorumCommit.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicQuorumCommit.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicQuorumCommit.java
new file mode 100644
index 0000000..1c63a47
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicQuorumCommit.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumAgent;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static junit.framework.Assert.fail;
+
+@RunWith(value = Parameterized.class)
+public class TestBasicQuorumCommit {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    TestBasicQuorumCommit.class);
+  private static final int MULTIPLE_COMMIT_NUM = 10;
+
+  private static final int QUORUM_SIZE = 5;
+  private static final int QUORUM_MAJORITY = 3;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private final List<int[]> mockLogs;
+  private QuorumClient client;
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, mockLogs);
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return RaftTestDataProvider.getRaftBasicLogTestData();
+  }
+
+  public TestBasicQuorumCommit(List<int[]> logs) {
+    this.mockLogs = logs;
+  }
+
+  @Test(timeout=50000)
+  public void testMultipleCommit() {
+    for (int i = 0 ; i <= MULTIPLE_COMMIT_NUM; i++) {
+      testSingleCommit();
+      LOG.info("Passed the " + i + " commit !");
+    }
+    // Verify all the logs across the quorum are the same
+    while(!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE)) {
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+      try {
+        // Sleep for MAX_TIMEOUT time for leader election to complete
+        Thread.sleep(HConstants.QUORUM_CLIENT_COMMIT_DEADLINE_DEFAULT);
+      } catch (InterruptedException e) {
+        LOG.warn("We are told to exit.");
+        System.exit(1);
+      }
+    }
+  }
+
+  private void testSingleCommit() {
+    try {
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+      client.replicateCommits(Arrays.asList(generateTestingWALEdit()));
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+      // Verify all the logs across the majority are the same
+      RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_MAJORITY);
+    } catch (Exception e) {
+      LOG.error("Errors: ", e);
+      fail("Unexpected exception: " + e.getMessage());
+    }
+  }
+
+  private static WALEdit generateTestingWALEdit() {
+    KeyValue kv = KeyValue.createFirstOnRow(Bytes.toBytes("TestQuorum"));
+    return new WALEdit(Arrays.asList(kv));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicQuorumMembershipChange.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicQuorumMembershipChange.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicQuorumMembershipChange.java
new file mode 100644
index 0000000..e75e4af
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicQuorumMembershipChange.java
@@ -0,0 +1,219 @@
+package org.apache.hadoop.hbase.consensus;
+
+import junit.framework.Assert;
+ import org.apache.hadoop.hbase.HServerAddress;
+import org.apache.hadoop.hbase.client.NoLeaderForRegionException;
+import org.apache.hadoop.hbase.consensus.quorum.QuorumInfo;
+ import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.apache.hadoop.hbase.consensus.util.RaftUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.runner.RunWith;
+ import org.junit.runners.Parameterized;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ import org.apache.hadoop.hbase.HConstants;
+ import org.apache.hadoop.hbase.HRegionInfo;
+ import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+ import org.junit.After;
+ import org.junit.Before;
+ import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadPoolExecutor;
+
+@RunWith(value = Parameterized.class)
+ public class TestBasicQuorumMembershipChange {
+   private static final Logger LOG = LoggerFactory.getLogger(
+     TestBasicPeerFailure.class);
+   private static int QUORUM_SIZE = 5;
+   private static int QUORUM_MAJORITY = 3;
+   private static HRegionInfo regionInfo;
+   private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+   private static QuorumClient client;
+   private static volatile int transactionNums = 0;
+   private static ThreadPoolExecutor loadGeneratorExecutor;
+   private static volatile boolean stop = false;
+   private static volatile long clientTrafficFrequency = 10;
+   private final int numPeersToChange;
+   private ReplicationLoadForUnitTest loader;
+   private boolean replaceLeader = false;
+
+   @Before
+   public void setUp() throws Exception {
+     RAFT_TEST_UTIL.disableVerboseLogging();
+     RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+     RAFT_TEST_UTIL.setUsePeristentLog(true);
+     RAFT_TEST_UTIL.assertAllServersRunning();
+     regionInfo = RAFT_TEST_UTIL.initializePeers();
+     RAFT_TEST_UTIL.addQuorum(regionInfo, null);
+     RAFT_TEST_UTIL.startQuorum(regionInfo);
+     client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+
+     loader = new ReplicationLoadForUnitTest(regionInfo, client, RAFT_TEST_UTIL,
+       QUORUM_SIZE, QUORUM_MAJORITY);
+
+     transactionNums = 0;
+     stop = false;
+   }
+
+   @Parameterized.Parameters
+   public static Collection<Object[]> data() {
+     return Arrays.asList(new Object[][] {
+       {1, false},
+       {1, true},
+       {2, false},
+       {2, true},
+       {3, true}
+     });
+   }
+
+   @After
+   public void tearDown() throws Exception {
+     RAFT_TEST_UTIL.shutdown();
+   }
+
+   public TestBasicQuorumMembershipChange(int numPeersToChange, boolean replaceLeader) {
+     this.replaceLeader = replaceLeader;
+     this.numPeersToChange = numPeersToChange;
+   }
+
+   @Test(timeout=600000)
+   public void testSingleMemberChange()
+           throws InterruptedException, ExecutionException, IOException {
+     final long sleepTime = HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS;
+
+     Thread.sleep(sleepTime);
+
+     // Start the client load
+     loader.startReplicationLoad(100);
+
+     // Let the traffic fly for a while
+     loader.makeProgress(sleepTime, transactionNums);
+
+     int currPort = RAFT_TEST_UTIL.getNextPortNumber();
+
+     int[] newPorts = new int[numPeersToChange];
+     for (int i = 0; i < numPeersToChange; i++) {
+       newPorts[i] = currPort++;
+     }
+
+     RAFT_TEST_UTIL.setNextPortNumber(++currPort);
+
+     // Get the new config
+     QuorumInfo newConfig = this.createNewQuorumInfo(newPorts);
+
+     LOG.debug("Old Config " + regionInfo.getPeersWithRank());
+
+     LOG.debug("New Config " + newConfig.getPeersWithRank());
+
+     // Add servers with new config
+     addServers(newPorts, newConfig);
+
+     // Send down the request
+     // We are not allowed to change majority of more number of peers at the
+     // same time.
+     if (numPeersToChange >= this.QUORUM_MAJORITY) {
+      Assert.assertFalse(client.changeQuorum(newConfig));
+      newConfig = regionInfo.getQuorumInfo();
+     } else {
+       Assert.assertTrue(client.changeQuorum(newConfig));
+       // Tell the quorum client about the new config
+       client.refreshConfig(newConfig);
+     }
+
+     // Simulate all the events
+     loader.makeProgress(sleepTime, transactionNums);
+
+     // Slow down the client traffic;
+     clientTrafficFrequency = clientTrafficFrequency * 5;
+
+     // Let the traffic fly for a while
+     loader.makeProgress(sleepTime, transactionNums);
+
+     // Verify logs are identical across all the quorum members
+     while (!RAFT_TEST_UTIL.verifyLogs(newConfig, QUORUM_SIZE)) {
+       Thread.sleep(5 * 1000);
+       clientTrafficFrequency = clientTrafficFrequency * 10;
+       System.out.println("Verifying logs ....");
+     }
+
+     // Stop the replication load
+     loader.stopReplicationLoad();
+
+     System.out.println(transactionNums + " transactions have been successfully replicated");
+   }
+
+   private void addServers(int[] ports, final QuorumInfo info)
+           throws IOException {
+     for (int port : ports) {
+       // Spin up a new server
+       final LocalConsensusServer server = RAFT_TEST_UTIL.addServer(port);
+       server.startService();
+
+       // Add the new config to the new server map
+       RAFT_TEST_UTIL.addQuorumForServer(server, info, null);
+
+       // Start the raft protocol on the server
+       server.getHandler().getRaftQuorumContext(
+         regionInfo.getQuorumInfo().getQuorumName()).initializeAll(
+         HConstants.UNDEFINED_TERM_INDEX);
+     }
+   }
+
+   private QuorumInfo createNewQuorumInfo(int[] ports)
+     throws NoLeaderForRegionException {
+
+     // Make a copy
+
+     QuorumInfo info = new QuorumInfo(regionInfo.getQuorumInfo());
+     boolean leaderReplaced = false;
+     List<HServerAddress> peerToReplaceAddr;
+     peerToReplaceAddr = new ArrayList<>();
+     List<Pair<HServerAddress, Integer>> newServers = new ArrayList<>();
+
+     Map<HServerAddress, Integer> currentPeers =
+       info.getPeers().get(HRegionInfo.LOCAL_DC_KEY);
+
+     HServerAddress oldPeer, newPeer;
+     for (int newServerPort : ports) {
+
+       newPeer = new HServerAddress(RAFT_TEST_UTIL.LOCAL_HOST,
+         newServerPort - HConstants.CONSENSUS_SERVER_PORT_JUMP);
+
+       LOG.debug("Adding new server with address " + newPeer);
+       if (replaceLeader && !leaderReplaced) {
+         oldPeer = RaftUtil.getHRegionServerAddress(
+           new HServerAddress(client.getLeader().getServerAddress()));
+         leaderReplaced = true;
+         System.out.println(
+           "Replacing leader " + oldPeer + " with port " + newPeer);
+
+       } else {
+         oldPeer = currentPeers.keySet().iterator().next();
+         System.out.println("Replacing non-leader " + oldPeer + " with port " +
+           newPeer);
+       }
+
+       peerToReplaceAddr.add(oldPeer);
+       int rank = currentPeers.remove(oldPeer);
+       newServers.add(new Pair<HServerAddress, Integer>(newPeer, rank));
+     }
+
+     // Make sure we actually removed the required number of peers
+     Assert.assertTrue(info.getPeers().get(HRegionInfo.LOCAL_DC_KEY).size() ==
+       QUORUM_SIZE - ports.length);
+
+     for (Pair<HServerAddress, Integer> server : newServers) {
+       // Update the config
+       info.getPeers().get(HRegionInfo.LOCAL_DC_KEY).put(server.getFirst(),
+               server.getSecond());
+     }
+     info.refresh();
+
+     Assert.assertTrue(info.getPeersWithRank().size() == QUORUM_SIZE);
+
+     return info;
+   }
+ }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicSeedCommitIndex.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicSeedCommitIndex.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicSeedCommitIndex.java
new file mode 100644
index 0000000..e7c8644
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestBasicSeedCommitIndex.java
@@ -0,0 +1,115 @@
+package org.apache.hadoop.hbase.consensus;
+
+import static junit.framework.Assert.fail;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestBasicSeedCommitIndex {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    TestBasicPeerSeeding.class);
+
+  private static final int QUORUM_SIZE = 5;
+  private static final int QUORUM_MAJORITY = 3;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private final long seedIndex = 100;
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.getConf().setLong(
+      HConstants.CONSENSUS_TRANCTION_LOG_RETENTION_TIME_KEY, 1000);
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, null);
+    RAFT_TEST_UTIL.setSeedIndex(seedIndex);
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Test(timeout=50000)
+  public void testSingleCommit() throws IOException {
+
+    // Wait for leader
+
+    RaftQuorumContext c5 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 5);
+    while (!c5.isLeader()) {
+      Threads.sleep(1000);
+    }
+
+    // Wait till we purge the seed log file
+    int count = 0;
+    List<RaftQuorumContext> peers = RAFT_TEST_UTIL.getQuorumContexts(regionInfo);
+
+    while (count != QUORUM_SIZE) {
+      Threads.sleep(1000);
+      count = 0;
+      for (RaftQuorumContext p : peers) {
+        if (p.getLogManager().getFirstIndex() >= 101) {
+          ++count;
+        }
+      }
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+    }
+
+    // At this point the state should
+    // [rank: 5] ; LEADER ; { Uncommitted [101, 101] }
+    // [rank: 4] ; FOLLOWER ; { Uncommitted [101, 101] }
+    // [rank: 3] ; FOLLOWER ; { Uncommitted [101, 101] }
+    // [rank: 2] ; FOLLOWER ; { Uncommitted [101, 101] }
+    // [rank: 1] ; FOLLOWER ; { Uncommitted [101, 101] }
+
+    // Let's stop the leader
+    LocalConsensusServer s5 = RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo,
+      5);
+
+    RaftQuorumContext c4 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 4);
+
+    while (!c4.isLeader()) {
+      Threads.sleep(1000);
+    }
+
+    c5 = RAFT_TEST_UTIL.restartLocalConsensusServer(s5, regionInfo,
+      c5.getMyAddress());
+
+    while (!c5.isLeader()) {
+      Threads.sleep(1000);
+    }
+
+    // Wait for logs to be verified
+    // Verify logs are identical across all the quorum members
+    while (!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE)) {
+      Threads.sleep(1000);
+      System.out.println("Verifying logs ....");
+      Assert
+        .assertTrue("Rank 5 shall be the leader of the quorum", c5.isLeader());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestCommitDeadline.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestCommitDeadline.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestCommitDeadline.java
new file mode 100644
index 0000000..883a2ab
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestCommitDeadline.java
@@ -0,0 +1,85 @@
+package org.apache.hadoop.hbase.consensus;
+
+import com.google.common.base.Stopwatch;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.client.QuorumThriftClientAgent;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestCommitDeadline {
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private static int NUM_REPLICAS = 3;
+  private static int QUORUM_SIZE = 2;
+
+  private HRegionInfo regionInfo;
+  private QuorumClient client;
+  private QuorumThriftClientAgent leader;
+
+  @Test
+  public void testQuorumAgentCommitDeadline() throws Exception {
+    long deadline = HConstants.QUORUM_AGENT_COMMIT_DEADLINE_DEFAULT;
+
+    // Do a transaction to make the QuorumClient lookup the leader.
+    client.replicateCommits(RAFT_TEST_UTIL.generateTransaction(1024));
+    leader = client.getLeader();
+    assertNotNull(leader);
+
+    // A successful commit should complete within the set deadline.
+    Stopwatch stopwatch = new Stopwatch(); //Stopwatch.createStarted();
+    assertTrue(leader.replicateCommit(regionInfo.getEncodedName(),
+            RAFT_TEST_UTIL.generateTransaction(1024)) > 0);
+    assertTrue("The commit should complete within the deadline",
+            stopwatch.elapsedTime(TimeUnit.MILLISECONDS) < deadline);
+
+    // Stop the majority of the replicas. The leader should remain leader.
+    for (int i = 0; i < QUORUM_SIZE; i++) {
+      RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, i + 1);
+    }
+    leader = client.getLeader();
+    assertNotNull(leader);
+
+    // An unsuccessful commit should throw after the deadline expires.
+    stopwatch.reset().start();
+    Exception expectedException = null;
+    try {
+      leader.replicateCommit(regionInfo.getEncodedName(),
+              RAFT_TEST_UTIL.generateTransaction(1024));
+    } catch (Exception e) {
+      expectedException = e;
+    }
+    long elapsed = stopwatch.elapsedTime(TimeUnit.MILLISECONDS);
+    assertTrue("Elapsed time should be within 10% of the deadline",
+            Math.abs(deadline - elapsed) < 0.1 * deadline);
+    assertNotNull("A TimeoutException should have been thrown",
+            expectedException);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.disableVerboseLogging();
+    RAFT_TEST_UTIL.createRaftCluster(NUM_REPLICAS);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+
+    RAFT_TEST_UTIL.addQuorum(regionInfo,
+            RAFT_TEST_UTIL.getScratchSetup(NUM_REPLICAS));
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+  }
+
+  @After
+  public void tearDown() {
+    RAFT_TEST_UTIL.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLogFileViewer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLogFileViewer.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLogFileViewer.java
new file mode 100644
index 0000000..7024583
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLogFileViewer.java
@@ -0,0 +1,79 @@
+package org.apache.hadoop.hbase.consensus;
+
+import junit.framework.Assert;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.log.LogFileViewer;
+import org.apache.hadoop.hbase.consensus.log.LogReader;
+import org.apache.hadoop.hbase.consensus.log.LogWriter;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.Arrays;
+import java.util.List;
+
+public class TestLogFileViewer {
+  private static final Logger
+    LOG = LoggerFactory.getLogger(TestLogFileViewer.class);
+
+  @Test
+  public void testViewer() throws IOException {
+    final int numTXNs = 100;
+    final KeyValue.KVComparator comparator = new KeyValue.KVComparator();
+
+    // Initialize the writer
+    File file = new File("TestLogFileViewer");
+    RandomAccessFile raf = new RandomAccessFile(file, "rw");
+    LogWriter writer = new LogWriter(raf, false);
+
+    // Generate the header
+    final long initialIndex = 0;
+    final long term = 1;
+    writer.writeFileHeader(term, initialIndex);
+
+    // Write the numTXNs to the log file
+    long curIndex, filePosition;
+    List<WALEdit> txns;
+    WALEdit edit;
+    for (curIndex = initialIndex; curIndex < numTXNs; curIndex++) {
+      edit = new WALEdit();
+      edit.add(new KeyValue(Bytes.toBytes(curIndex), curIndex));
+      txns = Arrays.asList(edit);
+      writer.append(curIndex, WALEdit.serializeToByteBuffer(txns, 1234567890L,
+              Compression.Algorithm.NONE));
+
+      // Test the truncate for every 10 entries;
+      if (curIndex % 10 == 0) {
+        // Write some dummy data to be truncated
+        filePosition = writer.getCurrentPosition();
+
+        edit = new WALEdit();
+        edit.add(new KeyValue(Bytes.toBytes("ToBeTruncated"), System.currentTimeMillis()));
+        txns = Arrays.asList(edit);
+
+        long tmpIndex = curIndex + 1;
+        long tmpOffset = writer.append(tmpIndex,
+                WALEdit.serializeToByteBuffer(txns, 1234567890L,
+                        Compression.Algorithm.NONE));
+
+
+        Assert.assertEquals(filePosition, tmpOffset);
+        writer.truncate(tmpOffset);
+        Assert.assertEquals(tmpOffset, raf.getChannel().size());
+
+        LOG.info("Truncate the log at the offset of " + tmpOffset + " for the index " + tmpIndex);
+      }
+    }
+
+    // Close the writer
+    writer.close();
+
+    LogFileViewer.dumpFileInfo(file, false);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLogWriterAndReader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLogWriterAndReader.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLogWriterAndReader.java
new file mode 100644
index 0000000..0d1439c
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLogWriterAndReader.java
@@ -0,0 +1,105 @@
+package org.apache.hadoop.hbase.consensus;
+
+import junit.framework.Assert;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.util.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.log.LogReader;
+import org.apache.hadoop.hbase.consensus.log.LogWriter;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+public class TestLogWriterAndReader {
+  private static final Logger LOG = LoggerFactory.getLogger(
+          TestLogWriterAndReader.class);
+
+  private final Arena arena = new InHeapArena(BucketAllocator.DEFAULT_BUCKETS,
+    HConstants.ARENA_CAPACITY_DEFAULT);
+
+  @Test
+  public void testSequentialWriteAndRead() throws IOException {
+    final int numTXNs = 100;
+    final KeyValue.KVComparator comparator = new KeyValue.KVComparator();
+
+    // Initialize the writer
+    File file = new File("testSequentialWriteAndRead");
+    RandomAccessFile raf = new RandomAccessFile(file, "rw");
+    LogWriter writer = new LogWriter(raf, false);
+
+    // Generate the header
+    final long initialIndex = 0;
+    final long term = 1;
+    writer.writeFileHeader(term, initialIndex);
+
+    // Write the numTXNs to the log file
+    long curIndex, filePosition;
+    List<WALEdit> txns;
+    WALEdit edit;
+    for (curIndex = initialIndex; curIndex < numTXNs; curIndex++) {
+      edit = new WALEdit();
+      edit.add(new KeyValue(Bytes.toBytes(curIndex), curIndex));
+      txns = Arrays.asList(edit);
+      writer.append(curIndex, WALEdit.serializeToByteBuffer(txns, 1234567890L,
+              Compression.Algorithm.NONE));
+
+      // Test the truncate for every 10 entries;
+      if (curIndex % 10 == 0) {
+        // Write some dummy data to be truncated
+        filePosition = writer.getCurrentPosition();
+
+        edit = new WALEdit();
+        edit.add(new KeyValue(Bytes.toBytes("ToBeTruncated"), System.currentTimeMillis()));
+        txns = Arrays.asList(edit);
+
+        long tmpIndex = curIndex + 1;
+        long tmpOffset = writer.append(tmpIndex,
+                WALEdit.serializeToByteBuffer(txns, 1234567890L,
+                        Compression.Algorithm.NONE));
+
+        Assert.assertEquals(filePosition, tmpOffset);
+        writer.truncate(tmpOffset);
+        Assert.assertEquals(tmpOffset, raf.getChannel().size());
+
+        LOG.info("Truncate the log at the offset of " + tmpOffset + " for the index " + tmpIndex);
+      }
+    }
+
+    // Close the writer
+    writer.close();
+
+    // Initialize the reader
+    LogReader reader = new LogReader(file);
+    reader.initialize();
+
+    // Verify the header data
+    Assert.assertEquals(term, reader.getCurrentTerm());
+    Assert.assertEquals(initialIndex, reader.getInitialIndex());
+
+    // Read the transactions and verify
+    for (long i = initialIndex; i < numTXNs; i++) {
+      MemoryBuffer buffer = reader.seekAndRead(i, arena);
+      txns = WALEdit.deserializeFromByteBuffer(buffer.getBuffer());
+      edit = txns.get(0);
+      KeyValue kv = edit.getKeyValues().get(0);
+      KeyValue expectedKV = new KeyValue(Bytes.toBytes(i), i);
+
+      Assert.assertEquals(1, txns.size());
+      Assert.assertEquals(1, edit.size());
+      Assert.assertEquals(0, comparator.compare(expectedKV, kv));
+      arena.freeByteBuffer(buffer);
+    }
+
+    LOG.info("Verified all " + numTXNs + " entries from the log file: " + file.getAbsoluteFile());
+    file.delete();
+  }
+}


[16/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/KeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/KeyValue.java
new file mode 100644
index 0000000..8a9dce6
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -0,0 +1,2300 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import javax.annotation.concurrent.Immutable;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import com.google.common.primitives.Longs;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.Writable;
+
+import com.google.common.primitives.Longs;
+
+/**
+ * An HBase Key/Value.
+ *
+ * <p>If being used client-side, the primary methods to access individual fields
+ * are {@link #getRow()}, {@link #getFamily()}, {@link #getQualifier()},
+ * {@link #getTimestamp()}, and {@link #getValue()}.  These methods allocate new
+ * byte arrays and return copies so they should be avoided server-side.
+ *
+ * <p>Instances of this class are immutable.  They are not
+ * comparable but Comparators are provided.  Comparators change with context,
+ * whether user table or a catalog table comparison context.  Its
+ * important that you use the appropriate comparator comparing rows in
+ * particular.  There are Comparators for KeyValue instances and then for
+ * just the Key portion of a KeyValue used mostly in {@link HFile}.
+ *
+ * <p>KeyValue wraps a byte array and has offset and length for passed array
+ * at where to start interpreting the content as a KeyValue blob.  The KeyValue
+ * blob format inside the byte array is:
+ * <code>&lt;keylength> &lt;valuelength> &lt;key> &lt;value></code>
+ * Key is decomposed as:
+ * <code>&lt;rowlength> &lt;row> &lt;columnfamilylength> &lt;columnfamily> &lt;columnqualifier> &lt;timestamp> &lt;keytype></code>
+ * Rowlength maximum is Short.MAX_SIZE, column family length maximum is
+ * Byte.MAX_SIZE, and column qualifier + key length must be < Integer.MAX_SIZE.
+ * The column does not contain the family/qualifier delimiter.
+ *
+ * <p>TODO: Group Key-only comparators and operations into a Key class, just
+ * for neatness sake, if can figure what to call it.
+ */
+
+@Immutable
+@ThriftStruct
+public final class KeyValue implements Writable, HeapSize, Cloneable {
+  static final Log LOG = LogFactory.getLog(KeyValue.class);
+
+  /**
+   * Colon character in UTF-8
+   */
+  public static final char COLUMN_FAMILY_DELIMITER = ':';
+
+  public static final byte[] COLUMN_FAMILY_DELIM_ARRAY =
+    new byte[]{COLUMN_FAMILY_DELIMITER};
+
+  /**
+   * Comparator for plain key/values; i.e. non-catalog table key/values.
+   */
+  public static final KVComparator COMPARATOR = new KVComparator();
+
+  /**
+   * Comparator for plain key; i.e. non-catalog table key.  Works on Key portion
+   * of KeyValue only.
+   */
+  public static final KeyComparator KEY_COMPARATOR = new KeyComparator();
+
+  /**
+   * A {@link KVComparator} for <code>.META.</code> catalog table
+   * {@link KeyValue}s.
+   */
+  public static final KVComparator META_COMPARATOR = new MetaComparator();
+
+  /**
+   * A {@link KVComparator} for <code>.META.</code> catalog table
+   * {@link KeyValue} keys.
+   */
+  public static final KeyComparator META_KEY_COMPARATOR = new MetaKeyComparator();
+
+  /**
+   * A {@link KVComparator} for <code>-ROOT-</code> catalog table
+   * {@link KeyValue}s.
+   */
+  public static final KVComparator ROOT_COMPARATOR = new RootComparator();
+
+  /**
+   * A {@link KVComparator} for <code>-ROOT-</code> catalog table
+   * {@link KeyValue} keys.
+   */
+  public static final KeyComparator ROOT_KEY_COMPARATOR = new RootKeyComparator();
+
+  /**
+   * Get the appropriate row comparator for the specified table.
+   *
+   * Hopefully we can get rid of this, I added this here because it's replacing
+   * something in HSK.  We should move completely off of that.
+   *
+   * @param tableName  The table name.
+   * @return The comparator.
+   */
+  public static KeyComparator getRowComparator(byte [] tableName) {
+    if(Bytes.equals(HTableDescriptor.ROOT_TABLEDESC.getName(),tableName)) {
+      return ROOT_COMPARATOR.getRawComparator();
+    }
+    if(Bytes.equals(HTableDescriptor.META_TABLEDESC.getName(), tableName)) {
+      return META_COMPARATOR.getRawComparator();
+    }
+    return COMPARATOR.getRawComparator();
+  }
+
+  /** Size of the key length field in bytes*/
+  public static final int KEY_LENGTH_SIZE = Bytes.SIZEOF_INT;
+
+  /** Size of the key type field in bytes */
+  public static final int TYPE_SIZE = Bytes.SIZEOF_BYTE;
+
+  /** Size of the row length field in bytes */
+  public static final int ROW_LENGTH_SIZE = Bytes.SIZEOF_SHORT;
+
+  /** Size of the family length field in bytes */
+  public static final int FAMILY_LENGTH_SIZE = Bytes.SIZEOF_BYTE;
+
+  /** Size of the timestamp field in bytes */
+  public static final int TIMESTAMP_SIZE = Bytes.SIZEOF_LONG;
+
+  // Size of the timestamp and type byte on end of a key -- a long + a byte.
+  public static final int TIMESTAMP_TYPE_SIZE = TIMESTAMP_SIZE + TYPE_SIZE;
+
+  // Size of the length shorts and bytes in key.
+  public static final int KEY_INFRASTRUCTURE_SIZE = ROW_LENGTH_SIZE
+      + FAMILY_LENGTH_SIZE + TIMESTAMP_TYPE_SIZE;
+
+  // How far into the key the row starts at. First thing to read is the short
+  // that says how long the row is.
+  public static final int ROW_OFFSET =
+    Bytes.SIZEOF_INT /*keylength*/ +
+    Bytes.SIZEOF_INT /*valuelength*/;
+
+  // Size of the length ints in a KeyValue datastructure.
+  public static final int KEYVALUE_INFRASTRUCTURE_SIZE = ROW_OFFSET;
+
+  /**
+   * Key type.
+   * Has space for other key types to be added later.  Cannot rely on
+   * enum ordinals . They change if item is removed or moved.  Do our own codes.
+   */
+  public static enum Type {
+    /**
+     * The minimum type. The latest type in the sorted order out of all
+     * key-values for the same row/column/timestamp combination. See
+     * {@link #createLastOnRow} functions. The minimum key type is actually
+     * greater than all other types, as compared by
+     * {@link KeyComparator#compare(byte[], int, int, byte[], int, int)}.
+     */
+    Minimum((byte)0),
+    Put((byte)4),
+
+    Delete((byte)8),
+    DeleteColumn((byte)12),
+    DeleteFamily((byte)14),
+
+    /**
+     * Maximum is used when searching; you look from maximum on down. The
+     * earliest type in the sorted order for the same row/column/timestamp. See
+     * {@link #createFirstOnRow} functions. The maximum key type is actually
+     * smaller than all other types, as compared by
+     * {@link KeyComparator#compare(byte[], int, int, byte[], int, int)}.
+     */
+    Maximum((byte)255);
+
+    private final byte code;
+
+    Type(final byte c) {
+      this.code = c;
+    }
+
+    public byte getCode() {
+      return this.code;
+    }
+
+    /**
+     * Cannot rely on enum ordinals . They change if item is removed or moved.
+     * Do our own codes.
+     * @param b
+     * @return Type associated with passed code.
+     */
+    public static Type codeToType(final byte b) {
+      for (Type t : Type.values()) {
+        if (t.getCode() == b) {
+          return t;
+        }
+      }
+      throw new RuntimeException("Unknown code " + b);
+    }
+  }
+
+  /**
+   * Lowest possible key.
+   * Makes a Key with highest possible Timestamp, empty row and column.  No
+   * key can be equal or lower than this one in memstore or in store file.
+   */
+  public static final KeyValue LOWESTKEY =
+    new KeyValue(HConstants.EMPTY_BYTE_ARRAY, HConstants.LATEST_TIMESTAMP);
+
+  private byte [] bytes = null;
+  private int offset = 0;
+  private int length = 0;
+
+  /** Here be dragons **/
+
+  // used to achieve atomic operations in the memstore.
+  public long getMemstoreTS() {
+    return memstoreTS;
+  }
+
+  public void setMemstoreTS(long memstoreTS) {
+    this.memstoreTS = memstoreTS;
+  }
+
+  // default value is 0, aka DNC
+  private long memstoreTS = 0;
+
+  /** Dragon time over, return to normal business */
+
+
+  /** Writable Constructor -- DO NOT USE */
+  public KeyValue() {}
+
+  @ThriftConstructor
+  public KeyValue(
+      @ThriftField(1) final ByteBuffer buffer) {
+    this.bytes = buffer.array();
+    this.length = buffer.limit() - buffer.position();
+    this.offset = buffer.position() + buffer.arrayOffset();
+  }
+
+  @ThriftField(1)
+  public ByteBuffer getByteBuffer() {
+    return ByteBuffer.wrap(bytes, offset, length);
+  }
+
+  /**
+   * Creates a KeyValue from the start of the specified byte array.
+   * Presumes <code>bytes</code> content is formatted as a KeyValue blob.
+   * @param bytes byte array
+   */
+  public KeyValue(final byte [] bytes) {
+    this(bytes, 0);
+  }
+
+  /**
+   * Creates a KeyValue from the specified byte array and offset.
+   * Presumes <code>bytes</code> content starting at <code>offset</code> is
+   * formatted as a KeyValue blob.
+   * @param bytes byte array
+   * @param offset offset to start of KeyValue
+   */
+  public KeyValue(final byte [] bytes, final int offset) {
+    this(bytes, offset, getLength(bytes, offset));
+  }
+
+  /**
+   * Creates a KeyValue from the specified byte array, starting at offset, and
+   * for length <code>length</code>.
+   * @param bytes byte array
+   * @param offset offset to start of the KeyValue
+   * @param length length of the KeyValue
+   */
+  public KeyValue(final byte [] bytes, final int offset, final int length) {
+    this.bytes = bytes;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  /** Constructors that build a new backing byte array from fields */
+
+  /**
+   * Constructs KeyValue structure filled with null value.
+   * Sets type to {@link KeyValue.Type#Maximum}
+   * @param row - row key (arbitrary byte array)
+   * @param timestamp
+   */
+  public KeyValue(final byte [] row, final long timestamp) {
+    this(row, timestamp, Type.Maximum);
+  }
+
+  /**
+   * Constructs KeyValue structure filled with null value.
+   * @param row - row key (arbitrary byte array)
+   * @param timestamp
+   */
+  public KeyValue(final byte [] row, final long timestamp, Type type) {
+    this(row, null, null, timestamp, type, null);
+  }
+
+  /**
+   * Constructs KeyValue structure filled with null value.
+   * Sets type to {@link KeyValue.Type#Maximum}
+   * @param row - row key (arbitrary byte array)
+   * @param family family name
+   * @param qualifier column qualifier
+   */
+  public KeyValue(final byte [] row, final byte [] family,
+      final byte [] qualifier) {
+    this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
+  }
+
+  /**
+   * Constructs KeyValue structure filled with null value.
+   * @param row - row key (arbitrary byte array)
+   * @param family family name
+   * @param qualifier column qualifier
+   */
+  public KeyValue(final byte [] row, final byte [] family,
+      final byte [] qualifier, final byte [] value) {
+    this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Put, value);
+  }
+
+  /**
+   * Constructs KeyValue structure filled with specified values.
+   * @param row row key
+   * @param family family name
+   * @param qualifier column qualifier
+   * @param timestamp version timestamp
+   * @param type key type
+   * @throws IllegalArgumentException
+   */
+  public KeyValue(final byte[] row, final byte[] family,
+      final byte[] qualifier, final long timestamp, Type type) {
+    this(row, family, qualifier, timestamp, type, null);
+  }
+
+  /**
+   * Constructs KeyValue structure filled with specified values.
+   * @param row row key
+   * @param family family name
+   * @param qualifier column qualifier
+   * @param timestamp version timestamp
+   * @param value column value
+   * @throws IllegalArgumentException
+   */
+  public KeyValue(final byte[] row, final byte[] family,
+      final byte[] qualifier, final long timestamp, final byte[] value) {
+    this(row, family, qualifier, timestamp, Type.Put, value);
+  }
+
+  /**
+   * Constructs KeyValue structure filled with specified values.
+   * @param row row key
+   * @param family family name
+   * @param qualifier column qualifier
+   * @param timestamp version timestamp
+   * @param type key type
+   * @param value column value
+   * @throws IllegalArgumentException
+   */
+  public KeyValue(final byte[] row, final byte[] family,
+      final byte[] qualifier, final long timestamp, Type type,
+      final byte[] value) {
+    this(row, family, qualifier, 0, qualifier==null ? 0 : qualifier.length,
+        timestamp, type, value, 0, value==null ? 0 : value.length);
+  }
+
+  /**
+   * Constructs KeyValue structure filled with specified values.
+   * @param row row key
+   * @param family family name
+   * @param qualifier column qualifier
+   * @param qoffset qualifier offset
+   * @param qlength qualifier length
+   * @param timestamp version timestamp
+   * @param type key type
+   * @param value column value
+   * @param voffset value offset
+   * @param vlength value length
+   * @throws IllegalArgumentException
+   */
+  public KeyValue(byte [] row, byte [] family,
+      byte [] qualifier, int qoffset, int qlength, long timestamp, Type type,
+      byte [] value, int voffset, int vlength) {
+    this(row, 0, row==null ? 0 : row.length,
+        family, 0, family==null ? 0 : family.length,
+        qualifier, qoffset, qlength, timestamp, type,
+        value, voffset, vlength);
+  }
+
+  /**
+   * Constructs KeyValue structure filled with specified values.
+   * <p>
+   * Column is split into two fields, family and qualifier.
+   * @param row row key
+   * @param roffset row offset
+   * @param rlength row length
+   * @param family family name
+   * @param foffset family offset
+   * @param flength family length
+   * @param qualifier column qualifier
+   * @param qoffset qualifier offset
+   * @param qlength qualifier length
+   * @param timestamp version timestamp
+   * @param type key type
+   * @param value column value
+   * @param voffset value offset
+   * @param vlength value length
+   * @throws IllegalArgumentException
+   */
+  public KeyValue(final byte [] row, final int roffset, final int rlength,
+      final byte [] family, final int foffset, final int flength,
+      final byte [] qualifier, final int qoffset, final int qlength,
+      final long timestamp, final Type type,
+      final byte [] value, final int voffset, final int vlength) {
+    this.bytes = createByteArray(row, roffset, rlength,
+        family, foffset, flength, qualifier, qoffset, qlength,
+        timestamp, type, value, voffset, vlength);
+    this.length = bytes.length;
+    this.offset = 0;
+  }
+
+  /**
+   * Write KeyValue format into a byte array.
+   *
+   * @param row row key
+   * @param roffset row offset
+   * @param rlength row length
+   * @param family family name
+   * @param foffset family offset
+   * @param flength family length
+   * @param qualifier column qualifier
+   * @param qoffset qualifier offset
+   * @param qlength qualifier length
+   * @param timestamp version timestamp
+   * @param type key type
+   * @param value column value
+   * @param voffset value offset
+   * @param vlength value length
+   * @return The newly created byte array.
+   */
+  static byte [] createByteArray(final byte [] row, final int roffset,
+      final int rlength, final byte [] family, final int foffset, int flength,
+      final byte [] qualifier, final int qoffset, int qlength,
+      final long timestamp, final Type type,
+      final byte [] value, final int voffset, int vlength) {
+    if (rlength > Short.MAX_VALUE) {
+      throw new IllegalArgumentException("Row > " + Short.MAX_VALUE);
+    }
+    if (row == null) {
+      throw new IllegalArgumentException("Row is null");
+    }
+    // Family length
+    flength = family == null ? 0 : flength;
+    if (flength > Byte.MAX_VALUE) {
+      throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE);
+    }
+    // Qualifier length
+    qlength = qualifier == null ? 0 : qlength;
+    if (qlength > Integer.MAX_VALUE - rlength - flength) {
+      throw new IllegalArgumentException("Qualifier > " + Integer.MAX_VALUE);
+    }
+    // Key length
+    long longkeylength = KEY_INFRASTRUCTURE_SIZE + rlength + flength + qlength;
+    if (longkeylength > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("keylength " + longkeylength + " > " +
+        Integer.MAX_VALUE);
+    }
+    int keylength = (int)longkeylength;
+    // Value length
+    vlength = value == null? 0 : vlength;
+    if (vlength > HConstants.MAXIMUM_VALUE_LENGTH) { // FindBugs INT_VACUOUS_COMPARISON
+      throw new IllegalArgumentException("Valuer > " +
+          HConstants.MAXIMUM_VALUE_LENGTH);
+    }
+
+    // Allocate right-sized byte array.
+    byte [] bytes = new byte[KEYVALUE_INFRASTRUCTURE_SIZE + keylength + vlength];
+    // Write key, value and key row length.
+    int pos = 0;
+    pos = Bytes.putInt(bytes, pos, keylength);
+    pos = Bytes.putInt(bytes, pos, vlength);
+    pos = Bytes.putShort(bytes, pos, (short)(rlength & 0x0000ffff));
+    pos = Bytes.putBytes(bytes, pos, row, roffset, rlength);
+    pos = Bytes.putByte(bytes, pos, (byte)(flength & 0x0000ff));
+    if(flength != 0) {
+      pos = Bytes.putBytes(bytes, pos, family, foffset, flength);
+    }
+    if(qlength != 0) {
+      pos = Bytes.putBytes(bytes, pos, qualifier, qoffset, qlength);
+    }
+    pos = Bytes.putLong(bytes, pos, timestamp);
+    pos = Bytes.putByte(bytes, pos, type.getCode());
+    if (value != null && value.length > 0) {
+      pos = Bytes.putBytes(bytes, pos, value, voffset, vlength);
+    }
+    return bytes;
+  }
+
+  /**
+   * Write KeyValue format into a byte array.
+   * <p>
+   * Takes column in the form <code>family:qualifier</code>
+   * @param row - row key (arbitrary byte array)
+   * @param roffset
+   * @param rlength
+   * @param column
+   * @param coffset
+   * @param clength
+   * @param timestamp
+   * @param type
+   * @param value
+   * @param voffset
+   * @param vlength
+   * @return The newly created byte array.
+   */
+  static byte [] createByteArray(final byte [] row, final int roffset,
+        final int rlength,
+      final byte [] column, final int coffset, int clength,
+      final long timestamp, final Type type,
+      final byte [] value, final int voffset, int vlength) {
+    // If column is non-null, figure where the delimiter is at.
+    int delimiteroffset = 0;
+    if (column != null && column.length > 0) {
+      delimiteroffset = getFamilyDelimiterIndex(column, coffset, clength);
+      if (delimiteroffset > Byte.MAX_VALUE) {
+        throw new IllegalArgumentException("Family > " + Byte.MAX_VALUE);
+      }
+    } else {
+      return createByteArray(row,roffset,rlength,null,0,0,null,0,0,timestamp,
+          type,value,voffset,vlength);
+    }
+    int flength = delimiteroffset-coffset;
+    int qlength = clength - flength - 1;
+    return createByteArray(row, roffset, rlength, column, coffset,
+        flength, column, delimiteroffset+1, qlength, timestamp, type,
+        value, voffset, vlength);
+  }
+
+  /**
+   * Needed doing 'contains' on List. Only compares the key portion, not the
+   * value.
+   */
+  public boolean equals(Object other) {
+    if (!(other instanceof KeyValue)) {
+      return false;
+    }
+    KeyValue kv = (KeyValue)other;
+    // Comparing bytes should be fine doing equals test.  Shouldn't have to
+    // worry about special .META. comparators doing straight equals.
+    boolean result = Bytes.BYTES_RAWCOMPARATOR.compare(getBuffer(),
+        getKeyOffset(), getKeyLength(),
+      kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength()) == 0;
+    return result;
+  }
+
+  public int hashCode() {
+    byte[] b = getBuffer();
+    int start = getOffset(), end = getOffset() + getLength();
+    int h = b[start++];
+    for (int i = start; i < end; i++) {
+      h = (h * 13) ^ b[i];
+    }
+    return h;
+  }
+
+  //---------------------------------------------------------------------------
+  //
+  //  KeyValue cloning
+  //
+  //---------------------------------------------------------------------------
+
+  /**
+   * Clones a KeyValue.  This creates a copy, re-allocating the buffer.
+   * @return Fully copied clone of this KeyValue
+   */
+  public KeyValue clone() {
+    byte [] b = new byte[this.length];
+    System.arraycopy(this.bytes, this.offset, b, 0, this.length);
+    KeyValue ret = new KeyValue(b, 0, b.length);
+    // Important to clone the memstoreTS as well - otherwise memstore's
+    // update-in-place methods (eg increment) will end up creating
+    // new entries
+    ret.setMemstoreTS(memstoreTS);
+    return ret;
+  }
+
+  /**
+   * Creates a shallow copy of this KeyValue, reusing the data byte buffer.
+   * http://en.wikipedia.org/wiki/Object_copy
+   * @return Shallow copy of this KeyValue
+   */
+  public KeyValue shallowCopy() {
+    KeyValue shallowCopy = new KeyValue(this.bytes, this.offset, this.length);
+    shallowCopy.setMemstoreTS(this.memstoreTS);
+    return shallowCopy;
+  }
+
+  //---------------------------------------------------------------------------
+  //
+  //  String representation
+  //
+  //---------------------------------------------------------------------------
+
+  public String toString() {
+    if (this.bytes == null || this.bytes.length == 0) {
+      return "empty";
+    }
+    return keyToString(this.bytes, this.offset + ROW_OFFSET, getKeyLength()) +
+      "/vlen=" + getValueLength();
+  }
+
+  /**
+   * @param k Key portion of a KeyValue.
+   * @return Key as a String.
+   */
+  public static String keyToString(final byte [] k) {
+    return keyToString(k, 0, k.length);
+  }
+
+  /**
+   * Produces a string map for this key/value pair. Useful for programmatic use
+   * and manipulation of the data stored in an HLogKey, for example, printing
+   * as JSON. Values are left out due to their tendency to be large. If needed,
+   * they can be added manually.
+   *
+   * @return the Map<String,?> containing data from this key
+   */
+  public Map<String, Object> toStringMap() {
+    Map<String, Object> stringMap = new HashMap<String, Object>();
+    stringMap.put("row", Bytes.toStringBinary(getRow()));
+    stringMap.put("family", Bytes.toStringBinary(getFamily()));
+    stringMap.put("qualifier", Bytes.toStringBinary(getQualifier()));
+    stringMap.put("timestamp", getTimestamp());
+    stringMap.put("vlen", getValueLength());
+    return stringMap;
+  }
+
+  public static String keyToString(final byte [] b, final int o, final int l) {
+    if (b == null) return "";
+    int rowlength = Bytes.toShort(b, o);
+    String row = Bytes.toStringBinary(b, o + Bytes.SIZEOF_SHORT, rowlength);
+    int columnoffset = o + Bytes.SIZEOF_SHORT + 1 + rowlength;
+    int familylength = b[columnoffset - 1];
+    int columnlength = l - ((columnoffset - o) + TIMESTAMP_TYPE_SIZE);
+    String family = familylength == 0? "":
+      Bytes.toStringBinary(b, columnoffset, familylength);
+    String qualifier = columnlength == 0? "":
+      Bytes.toStringBinary(b, columnoffset + familylength,
+      columnlength - familylength);
+    long timestamp = Bytes.toLong(b, o + (l - TIMESTAMP_TYPE_SIZE));
+    String timestampStr = humanReadableTimestamp(timestamp);
+    byte type = b[o + l - 1];
+    return row + "/" + family +
+      (family != null && family.length() > 0? ":" :"") +
+      qualifier + "/" + timestampStr + "/" + Type.codeToType(type);
+  }
+
+  public static String humanReadableTimestamp(final long timestamp) {
+    if (timestamp == HConstants.LATEST_TIMESTAMP) {
+      return "LATEST_TIMESTAMP";
+    }
+    if (timestamp == HConstants.OLDEST_TIMESTAMP) {
+      return "OLDEST_TIMESTAMP";
+    }
+    return String.valueOf(timestamp);
+  }
+
+  //---------------------------------------------------------------------------
+  //
+  //  Public Member Accessors
+  //
+  //---------------------------------------------------------------------------
+
+  /**
+   * @return The byte array backing this KeyValue.
+   */
+  public byte [] getBuffer() {
+    return this.bytes;
+  }
+
+  /**
+   * @return Offset into {@link #getBuffer()} at which this KeyValue starts.
+   */
+  public int getOffset() {
+    return this.offset;
+  }
+
+  /**
+   * @return Length of bytes this KeyValue occupies in {@link #getBuffer()}.
+   */
+  public int getLength() {
+    return length;
+  }
+
+  //---------------------------------------------------------------------------
+  //
+  //  Length and Offset Calculators
+  //
+  //---------------------------------------------------------------------------
+
+  /**
+   * Determines the total length of the KeyValue stored in the specified
+   * byte array and offset.  Includes all headers.
+   * @param bytes byte array
+   * @param offset offset to start of the KeyValue
+   * @return length of entire KeyValue, in bytes
+   */
+  private static int getLength(byte [] bytes, int offset) {
+    return ROW_OFFSET +
+        Bytes.toInt(bytes, offset) +
+        Bytes.toInt(bytes, offset + Bytes.SIZEOF_INT);
+  }
+
+  /**
+   * @return Key offset in backing buffer..
+   */
+  public int getKeyOffset() {
+    return this.offset + ROW_OFFSET;
+  }
+
+  public String getKeyString() {
+    return Bytes.toStringBinary(getBuffer(), getKeyOffset(), getKeyLength());
+  }
+
+  /**
+   * @return Length of key portion.
+   */
+  public int getKeyLength() {
+    return Bytes.toInt(this.bytes, this.offset);
+  }
+
+  /**
+   * @return Value offset
+   */
+  public int getValueOffset() {
+    return getKeyOffset() + getKeyLength();
+  }
+
+  /**
+   * @return Value length
+   */
+  public int getValueLength() {
+    return Bytes.toInt(this.bytes, this.offset + Bytes.SIZEOF_INT);
+  }
+
+  /**
+   * @return Row offset
+   */
+  public int getRowOffset() {
+    return getKeyOffset() + Bytes.SIZEOF_SHORT;
+  }
+
+  /**
+   * @return Row length
+   */
+  public short getRowLength() {
+    return Bytes.toShort(this.bytes, getKeyOffset());
+  }
+
+  /**
+   * @return Family offset
+   */
+  public int getFamilyOffset() {
+    return getFamilyOffset(getRowLength());
+  }
+
+  /**
+   * @return Family offset
+   */
+  public int getFamilyOffset(int rlength) {
+    return this.offset + ROW_OFFSET + Bytes.SIZEOF_SHORT + rlength + Bytes.SIZEOF_BYTE;
+  }
+
+  /**
+   * @return Family length
+   */
+  public byte getFamilyLength() {
+    return getFamilyLength(getFamilyOffset());
+  }
+
+  /**
+   * @return Family length
+   */
+  public byte getFamilyLength(int foffset) {
+    return this.bytes[foffset-1];
+  }
+
+  /**
+   * @return Qualifier offset
+   */
+  public int getQualifierOffset() {
+    return getQualifierOffset(getFamilyOffset());
+  }
+
+  /**
+   * @return Qualifier offset
+   */
+  public int getQualifierOffset(int foffset) {
+    return foffset + getFamilyLength(foffset);
+  }
+
+  /**
+   * @return Qualifier length
+   */
+  public int getQualifierLength() {
+    return getQualifierLength(getRowLength(),getFamilyLength());
+  }
+
+  /**
+   * @return Qualifier length
+   */
+  public int getQualifierLength(int rlength, int flength) {
+    return getKeyLength() -
+      (KEY_INFRASTRUCTURE_SIZE + rlength + flength);
+  }
+
+  /**
+   * @return Column (family + qualifier) length
+   */
+  public int getTotalColumnLength() {
+    int rlength = getRowLength();
+    int foffset = getFamilyOffset(rlength);
+    return getTotalColumnLength(rlength,foffset);
+  }
+
+  /**
+   * @return Column (family + qualifier) length
+   */
+  public int getTotalColumnLength(int rlength, int foffset) {
+    int flength = getFamilyLength(foffset);
+    int qlength = getQualifierLength(rlength,flength);
+    return flength + qlength;
+  }
+
+  /**
+   * @return Timestamp offset
+   */
+  public int getTimestampOffset() {
+    return getTimestampOffset(getKeyLength());
+  }
+
+  /**
+   * @param keylength Pass if you have it to save on a int creation.
+   * @return Timestamp offset
+   */
+  public int getTimestampOffset(final int keylength) {
+    return getKeyOffset() + keylength - TIMESTAMP_TYPE_SIZE;
+  }
+
+  /**
+   * @return True if this KeyValue has a LATEST_TIMESTAMP timestamp.
+   */
+  public boolean isLatestTimestamp() {
+    return  Bytes.compareTo(getBuffer(), getTimestampOffset(), Bytes.SIZEOF_LONG,
+      HConstants.LATEST_TIMESTAMP_BYTES, 0, Bytes.SIZEOF_LONG) == 0;
+  }
+
+  /**
+   * @param now Time to set into <code>this</code> IFF timestamp ==
+   * {@link HConstants#LATEST_TIMESTAMP} (else, its a noop).
+   * @return True is we modified this.
+   */
+  public boolean updateLatestStamp(final byte [] now) {
+    if (this.isLatestTimestamp()) {
+      int tsOffset = getTimestampOffset();
+      System.arraycopy(now, 0, this.bytes, tsOffset, Bytes.SIZEOF_LONG);
+      return true;
+    }
+    return false;
+  }
+
+  //---------------------------------------------------------------------------
+  //
+  //  Methods that return copies of fields
+  //
+  //---------------------------------------------------------------------------
+
+  /**
+   * Do not use unless you have to.  Used internally for compacting and testing.
+   *
+   * Use {@link #getRow()}, {@link #getFamily()}, {@link #getQualifier()}, and
+   * {@link #getValue()} if accessing a KeyValue client-side.
+   * @return Copy of the key portion only.
+   */
+  public byte [] getKey() {
+    int keylength = getKeyLength();
+    byte [] key = new byte[keylength];
+    System.arraycopy(getBuffer(), getKeyOffset(), key, 0, keylength);
+    return key;
+  }
+
+  /**
+   * Returns value in a new byte array.
+   * Primarily for use client-side. If server-side, use
+   * {@link #getBuffer()} with appropriate offsets and lengths instead to
+   * save on allocations.
+   * @return Value in a new byte array.
+   */
+  public byte [] getValue() {
+    int o = getValueOffset();
+    int l = getValueLength();
+    byte [] result = new byte[l];
+    System.arraycopy(getBuffer(), o, result, 0, l);
+    return result;
+  }
+
+  /**
+   * Primarily for use client-side.  Returns the row of this KeyValue in a new
+   * byte array.<p>
+   *
+   * If server-side, use {@link #getBuffer()} with appropriate offsets and
+   * lengths instead.
+   * @return Row in a new byte array.
+   */
+  public byte [] getRow() {
+    int o = getRowOffset();
+    short l = getRowLength();
+    byte [] result = new byte[l];
+    System.arraycopy(getBuffer(), o, result, 0, l);
+    return result;
+  }
+
+  /**
+   *
+   * @return Timestamp
+   */
+  public long getTimestamp() {
+    return getTimestamp(getKeyLength());
+  }
+
+  /**
+   * @param keylength Pass if you have it to save on a int creation.
+   * @return Timestamp
+   */
+  long getTimestamp(final int keylength) {
+    int tsOffset = getTimestampOffset(keylength);
+    return Bytes.toLong(this.bytes, tsOffset);
+  }
+
+  /**
+   * @return Type of this KeyValue.
+   */
+  public byte getType() {
+    return getType(getKeyLength());
+  }
+
+  /**
+   * @param keylength Pass if you have it to save on a int creation.
+   * @return Type of this KeyValue.
+   */
+  byte getType(final int keylength) {
+    return this.bytes[this.offset + keylength - 1 + ROW_OFFSET];
+  }
+
+  /**
+   * @return True if a delete type, a {@link KeyValue.Type#Delete} or
+   * a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
+   * KeyValue type.
+   */
+  public boolean isDelete() {
+    int t = getType();
+    return Type.Delete.getCode() <= t && t <= Type.DeleteFamily.getCode();
+  }
+
+  /**
+   * @return return True if Put type.
+   */
+  public boolean isPut() {
+    int t = getType();
+    return (t == Type.Put.getCode()) ? true : false;
+  }
+
+  /**
+   * @return True if this KV is a {@link KeyValue.Type#Delete} type.
+   */
+  public boolean isDeleteType() {
+    return getType() == Type.Delete.getCode();
+  }
+
+  /**
+   * @return True if this KV is a delete family type.
+   */
+  public boolean isDeleteFamily() {
+    return getType() == Type.DeleteFamily.getCode();
+  }
+
+  /**
+   * @return True if this KV is a delete column type.
+   */
+  public boolean isDeleteColumn() {
+    return getType() == Type.DeleteColumn.getCode();
+  }
+
+  /**
+   *
+   * @return True if this KV is a delete family or column type.
+   */
+  public boolean isDeleteColumnOrFamily() {
+    int t = getType();
+    return t == Type.DeleteColumn.getCode() || t == Type.DeleteFamily.getCode();
+  }
+
+  /**
+   * Primarily for use client-side.  Returns the family of this KeyValue in a
+   * new byte array.<p>
+   *
+   * If server-side, use {@link #getBuffer()} with appropriate offsets and
+   * lengths instead.
+   * @return Returns family. Makes a copy.
+   */
+  public byte [] getFamily() {
+    int o = getFamilyOffset();
+    int l = getFamilyLength(o);
+    byte [] result = new byte[l];
+    System.arraycopy(this.bytes, o, result, 0, l);
+    return result;
+  }
+
+  /**
+   * Primarily for use client-side.  Returns the column qualifier of this
+   * KeyValue in a new byte array.<p>
+   *
+   * If server-side, use {@link #getBuffer()} with appropriate offsets and
+   * lengths instead.
+   * Use {@link #getBuffer()} with appropriate offsets and lengths instead.
+   * @return Returns qualifier. Makes a copy.
+   */
+  public byte [] getQualifier() {
+    int o = getQualifierOffset();
+    int l = getQualifierLength();
+    byte [] result = new byte[l];
+    System.arraycopy(this.bytes, o, result, 0, l);
+    return result;
+  }
+
+  //---------------------------------------------------------------------------
+  //
+  //  KeyValue splitter
+  //
+  //---------------------------------------------------------------------------
+
+  /**
+   * Utility class that splits a KeyValue buffer into separate byte arrays.
+   * <p>
+   * Should get rid of this if we can, but is very useful for debugging.
+   */
+  public static class SplitKeyValue {
+    private byte [][] split;
+    SplitKeyValue() {
+      this.split = new byte[6][];
+    }
+    public void setRow(byte [] value) { this.split[0] = value; }
+    public void setFamily(byte [] value) { this.split[1] = value; }
+    public void setQualifier(byte [] value) { this.split[2] = value; }
+    public void setTimestamp(byte [] value) { this.split[3] = value; }
+    public void setType(byte [] value) { this.split[4] = value; }
+    public void setValue(byte [] value) { this.split[5] = value; }
+    public byte [] getRow() { return this.split[0]; }
+    public byte [] getFamily() { return this.split[1]; }
+    public byte [] getQualifier() { return this.split[2]; }
+    public byte [] getTimestamp() { return this.split[3]; }
+    public byte [] getType() { return this.split[4]; }
+    public byte [] getValue() { return this.split[5]; }
+  }
+
+  public void verify() {
+    int splitOffset = this.offset;
+    int keyLen = Bytes.toInt(bytes, splitOffset);
+    splitOffset += Bytes.SIZEOF_INT;
+    int valLen = Bytes.toInt(bytes, splitOffset);
+    splitOffset += Bytes.SIZEOF_INT;
+    short rowLen = Bytes.toShort(bytes, splitOffset);
+    splitOffset += Bytes.SIZEOF_SHORT;
+    splitOffset += rowLen;
+    byte famLen = bytes[splitOffset];
+    if (!(keyLen >= 0 && valLen >=0
+        && keyLen + valLen + KEYVALUE_INFRASTRUCTURE_SIZE == this.length
+        && this.length + this.offset <= this.bytes.length
+        && rowLen >=0 && rowLen <= keyLen
+        && famLen >=0 && famLen <= keyLen
+        && rowLen + famLen <= keyLen
+        )) {
+      String msg = "Malformed key value: "
+         + ", offset ="  + offset
+         + ", keyLen ="  + keyLen
+         + ", valLen ="  + valLen
+         + ", length ="  + length
+         + ", rowLen ="  + rowLen
+         + ", famLen ="  + famLen
+         + ", bytes[] is " + Bytes.toStringBinary(bytes, offset, length);
+      LOG.error(msg);
+      throw new IllegalArgumentException(msg);
+    };
+  }
+
+  public SplitKeyValue split() {
+    SplitKeyValue split = new SplitKeyValue();
+    int splitOffset = this.offset;
+    int keyLen = Bytes.toInt(bytes, splitOffset);
+    splitOffset += Bytes.SIZEOF_INT;
+    int valLen = Bytes.toInt(bytes, splitOffset);
+    splitOffset += Bytes.SIZEOF_INT;
+    short rowLen = Bytes.toShort(bytes, splitOffset);
+    splitOffset += Bytes.SIZEOF_SHORT;
+    byte [] row = new byte[rowLen];
+    System.arraycopy(bytes, splitOffset, row, 0, rowLen);
+    splitOffset += rowLen;
+    split.setRow(row);
+    byte famLen = bytes[splitOffset];
+    splitOffset += Bytes.SIZEOF_BYTE;
+    byte [] family = new byte[famLen];
+    System.arraycopy(bytes, splitOffset, family, 0, famLen);
+    splitOffset += famLen;
+    split.setFamily(family);
+    int colLen = keyLen -
+      (rowLen + famLen + Bytes.SIZEOF_SHORT + Bytes.SIZEOF_BYTE +
+      Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE);
+    byte [] qualifier = new byte[colLen];
+    System.arraycopy(bytes, splitOffset, qualifier, 0, colLen);
+    splitOffset += colLen;
+    split.setQualifier(qualifier);
+    byte [] timestamp = new byte[Bytes.SIZEOF_LONG];
+    System.arraycopy(bytes, splitOffset, timestamp, 0, Bytes.SIZEOF_LONG);
+    splitOffset += Bytes.SIZEOF_LONG;
+    split.setTimestamp(timestamp);
+    byte [] type = new byte[1];
+    type[0] = bytes[splitOffset];
+    splitOffset += Bytes.SIZEOF_BYTE;
+    split.setType(type);
+    byte [] value = new byte[valLen];
+    System.arraycopy(bytes, splitOffset, value, 0, valLen);
+    split.setValue(value);
+    return split;
+  }
+
+  //---------------------------------------------------------------------------
+  //
+  //  Compare specified fields against those contained in this KeyValue
+  //
+  //---------------------------------------------------------------------------
+
+  /**
+   * @param family
+   * @return True if matching families.
+   */
+  public boolean matchingFamily(final byte [] family) {
+    if (this.length == 0 || this.bytes.length == 0) {
+      return false;
+    }
+    int o = getFamilyOffset();
+    int l = getFamilyLength(o);
+    return Bytes.compareTo(family, 0, family.length, this.bytes, o, l) == 0;
+  }
+
+  /**
+   * @param qualifier
+   * @return True if matching qualifiers.
+   */
+  public boolean matchingQualifier(final byte [] qualifier) {
+    int o = getQualifierOffset();
+    int l = getQualifierLength();
+    return Bytes.compareTo(qualifier, 0, qualifier.length,
+        this.bytes, o, l) == 0;
+  }
+
+  public boolean matchingRow(final byte [] row) {
+    return matchingRow(row, 0, row.length);
+  }
+
+  public boolean matchingRow(final byte[] row, int offset, int length) {
+    return Bytes.compareTo(row, offset, length,
+        this.bytes, getRowOffset(), getRowLength()) == 0;
+  }
+
+  /**
+   * @param column Column minus its delimiter
+   * @return True if column matches.
+   */
+  public boolean matchingColumnNoDelimiter(final byte [] column) {
+    int rl = getRowLength();
+    int o = getFamilyOffset(rl);
+    int fl = getFamilyLength(o);
+    int l = fl + getQualifierLength(rl,fl);
+    return Bytes.compareTo(column, 0, column.length, this.bytes, o, l) == 0;
+  }
+
+  /**
+   *
+   * @param family column family
+   * @param qualifier column qualifier
+   * @return True if column matches
+   */
+  public boolean matchingColumn(final byte[] family, final byte[] qualifier) {
+    int rl = getRowLength();
+    int o = getFamilyOffset(rl);
+    int fl = getFamilyLength(o);
+    int ql = getQualifierLength(rl,fl);
+    if(Bytes.compareTo(family, 0, family.length, this.bytes, o, family.length)
+        != 0) {
+      return false;
+    }
+    if(qualifier == null || qualifier.length == 0) {
+      if(ql == 0) {
+        return true;
+      }
+      return false;
+    }
+    return Bytes.compareTo(qualifier, 0, qualifier.length,
+        this.bytes, o + fl, ql) == 0;
+  }
+
+  /**
+   * @param left
+   * @param loffset
+   * @param llength
+   * @param lfamilylength Offset of family delimiter in left column.
+   * @param right
+   * @param roffset
+   * @param rlength
+   * @param rfamilylength Offset of family delimiter in right column.
+   * @return The result of the comparison.
+   */
+  static int compareColumns(final byte [] left, final int loffset,
+      final int llength, final int lfamilylength,
+      final byte [] right, final int roffset, final int rlength,
+      final int rfamilylength) {
+    // Compare family portion first.
+    int diff = Bytes.compareTo(left, loffset, lfamilylength,
+      right, roffset, rfamilylength);
+    if (diff != 0) {
+      return diff;
+    }
+    // Compare qualifier portion
+    return Bytes.compareTo(left, loffset + lfamilylength,
+      llength - lfamilylength,
+      right, roffset + rfamilylength, rlength - rfamilylength);
+  }
+
+  /**
+   * @return True if non-null row and column.
+   */
+  public boolean nonNullRowAndColumn() {
+    return getRowLength() > 0 && !isEmptyColumn();
+  }
+
+  /**
+   * @return True if column is empty.
+   */
+  public boolean isEmptyColumn() {
+    return getQualifierLength() == 0;
+  }
+
+  /**
+   * Converts this KeyValue to only contain the key portion (the value is
+   * changed to be null).  This method does a full copy of the backing byte
+   * array and does not modify the original byte array of this KeyValue.
+   * <p>
+   * This method is used by {@link KeyOnlyFilter} and is an advanced feature of
+   * KeyValue, proceed with caution.
+   * @param lenAsVal replace value with the actual value length (false=empty)
+   */
+  public void convertToKeyOnly(boolean lenAsVal) {
+    // KV format:  <keylen:4><valuelen:4><key:keylen><value:valuelen>
+    // Rebuild as: <keylen:4><0:4><key:keylen>
+    int dataLen = lenAsVal? Bytes.SIZEOF_INT : 0;
+    byte [] newBuffer = new byte[getKeyLength() + ROW_OFFSET + dataLen];
+    System.arraycopy(this.bytes, this.offset, newBuffer, 0,
+        Math.min(newBuffer.length,this.length));
+    Bytes.putInt(newBuffer, Bytes.SIZEOF_INT, dataLen);
+    if (lenAsVal) {
+      Bytes.putInt(newBuffer, newBuffer.length - dataLen, this.getValueLength());
+    }
+    this.bytes = newBuffer;
+    this.offset = 0;
+    this.length = newBuffer.length;
+  }
+
+  /**
+   * Splits a column in family:qualifier form into separate byte arrays.
+   * <p>
+   * Not recommend to be used as this is old-style API.
+   * @param c  The column.
+   * @return The parsed column.
+   */
+  public static byte [][] parseColumn(byte [] c) {
+    final int index = getDelimiter(c, 0, c.length, COLUMN_FAMILY_DELIMITER);
+    if (index == -1) {
+      // If no delimiter, return array of size 1
+      return new byte [][] { c };
+    } else if(index == c.length - 1) {
+      // Only a family, return array size 1
+      byte [] family = new byte[c.length-1];
+      System.arraycopy(c, 0, family, 0, family.length);
+      return new byte [][] { family };
+    }
+    // Family and column, return array size 2
+    final byte [][] result = new byte [2][];
+    result[0] = new byte [index];
+    System.arraycopy(c, 0, result[0], 0, index);
+    final int len = c.length - (index + 1);
+    result[1] = new byte[len];
+    System.arraycopy(c, index + 1 /*Skip delimiter*/, result[1], 0,
+      len);
+    return result;
+  }
+
+  /**
+   * Makes a column in family:qualifier form from separate byte arrays.
+   * <p>
+   * Not recommended for usage as this is old-style API.
+   * @param family
+   * @param qualifier
+   * @return family:qualifier
+   */
+  public static byte [] makeColumn(byte [] family, byte [] qualifier) {
+    return Bytes.add(family, COLUMN_FAMILY_DELIM_ARRAY, qualifier);
+  }
+
+  public byte[] makeColumn() {
+    return Bytes.add(bytes, getFamilyOffset(), getFamilyLength(),
+        COLUMN_FAMILY_DELIM_ARRAY, 0, COLUMN_FAMILY_DELIM_ARRAY.length,
+        bytes, getQualifierOffset(), getQualifierLength());
+  }
+
+  /**
+   * @param b
+   * @return Index of the family-qualifier colon delimiter character in passed
+   * buffer.
+   */
+  public static int getFamilyDelimiterIndex(final byte [] b, final int offset,
+      final int length) {
+    return getRequiredDelimiter(b, offset, length, COLUMN_FAMILY_DELIMITER);
+  }
+
+  private static int getRequiredDelimiter(final byte [] b,
+      final int offset, final int length, final int delimiter) {
+    int index = getDelimiter(b, offset, length, delimiter);
+    if (index < 0) {
+      throw new IllegalArgumentException("No '" + (char)delimiter + "' in <" +
+        Bytes.toString(b) + ">" + ", length=" + length + ", offset=" + offset);
+    }
+    return index;
+  }
+
+  /**
+   * This function is only used in Meta key comparisons so its error message
+   * is specific for meta key errors.
+   */
+  static int getRequiredDelimiterInReverse(final byte [] b,
+      final int offset, final int length, final int delimiter) {
+    int index = getDelimiterInReverse(b, offset, length, delimiter);
+    if (index < 0) {
+      throw new IllegalArgumentException("No '" + ((char) delimiter) + "' in <" +
+        Bytes.toString(b) + ">" + ", length=" + length + ", offset=" + offset);
+    }
+    return index;
+  }
+
+  /**
+   * @param b
+   * @param delimiter
+   * @return Index of delimiter having started from start of <code>b</code>
+   * moving rightward.
+   */
+  public static int getDelimiter(final byte [] b, int offset, final int length,
+      final int delimiter) {
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int result = -1;
+    for (int i = offset; i < length + offset; i++) {
+      if (b[i] == delimiter) {
+        result = i;
+        break;
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Find index of passed delimiter walking from end of buffer backwards.
+   * @param b
+   * @param delimiter
+   * @return Index of delimiter
+   */
+  public static int getDelimiterInReverse(final byte [] b, final int offset,
+      final int length, final int delimiter) {
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int result = -1;
+    for (int i = (offset + length) - 1; i >= offset; i--) {
+      if (b[i] == delimiter) {
+        result = i;
+        break;
+      }
+    }
+    return result;
+  }
+
+  /**
+   * A {@link KVComparator} for <code>-ROOT-</code> catalog table
+   * {@link KeyValue}s.
+   */
+  public static class RootComparator extends MetaComparator {
+    private final KeyComparator rawcomparator = new RootKeyComparator();
+
+    public KeyComparator getRawComparator() {
+      return this.rawcomparator;
+    }
+
+    @Override
+    protected Object clone() throws CloneNotSupportedException {
+      return new RootComparator();
+    }
+  }
+
+  /**
+   * A {@link KVComparator} for <code>.META.</code> catalog table
+   * {@link KeyValue}s.
+   */
+  public static class MetaComparator extends KVComparator {
+    private final KeyComparator rawcomparator = new MetaKeyComparator();
+
+    public KeyComparator getRawComparator() {
+      return this.rawcomparator;
+    }
+
+    @Override
+    protected Object clone() throws CloneNotSupportedException {
+      return new MetaComparator();
+    }
+  }
+
+  /**
+   * Compare KeyValues.  When we compare KeyValues, we only compare the Key
+   * portion.  This means two KeyValues with same Key but different Values are
+   * considered the same as far as this Comparator is concerned.
+   * Hosts a {@link KeyComparator}.
+   */
+  public static class KVComparator implements java.util.Comparator<KeyValue> {
+    private final KeyComparator rawcomparator = new KeyComparator();
+
+    /**
+     * @return RawComparator that can compare the Key portion of a KeyValue.
+     * Used in hfile where indices are the Key portion of a KeyValue.
+     */
+    public KeyComparator getRawComparator() {
+      return this.rawcomparator;
+    }
+
+    public int compare(final KeyValue left, final KeyValue right) {
+      int ret = getRawComparator().compare(left.getBuffer(),
+          left.getOffset() + ROW_OFFSET, left.getKeyLength(),
+          right.getBuffer(), right.getOffset() + ROW_OFFSET,
+          right.getKeyLength());
+      if (ret != 0) return ret;
+      // Negate this comparison so later edits show up first
+      return -Longs.compare(left.getMemstoreTS(), right.getMemstoreTS());
+    }
+
+    public int compareTimestamps(final KeyValue left, final KeyValue right) {
+      return compareTimestamps(left, left.getKeyLength(), right,
+        right.getKeyLength());
+    }
+
+    int compareTimestamps(final KeyValue left, final int lkeylength,
+        final KeyValue right, final int rkeylength) {
+      // Compare timestamps
+      long ltimestamp = left.getTimestamp(lkeylength);
+      long rtimestamp = right.getTimestamp(rkeylength);
+      return getRawComparator().compareTimestamps(ltimestamp, rtimestamp);
+    }
+
+    /**
+     * @param left
+     * @param right
+     * @return Result comparing rows.
+     */
+    public int compareRows(final KeyValue left, final KeyValue right) {
+      return compareRows(left, left.getRowLength(), right,
+          right.getRowLength());
+    }
+
+    /**
+     * @param left
+     * @param lrowlength Length of left row.
+     * @param right
+     * @param rrowlength Length of right row.
+     * @return Result comparing rows.
+     */
+    public int compareRows(final KeyValue left, final short lrowlength,
+        final KeyValue right, final short rrowlength) {
+      return getRawComparator().compareRows(left.getBuffer(),
+          left.getRowOffset(), lrowlength,
+        right.getBuffer(), right.getRowOffset(), rrowlength);
+    }
+
+    /**
+     * @param left
+     * @param row - row key (arbitrary byte array)
+     * @return RawComparator
+     */
+    public int compareRows(final KeyValue left, final byte [] row) {
+      return getRawComparator().compareRows(left.getBuffer(),
+          left.getRowOffset(), left.getRowLength(), row, 0, row.length);
+    }
+
+    public int compareRows(byte [] left, int loffset, int llength,
+        byte [] right, int roffset, int rlength) {
+      return getRawComparator().compareRows(left, loffset, llength,
+        right, roffset, rlength);
+    }
+
+    public int compareColumns(final KeyValue left, final byte [] right,
+        final int roffset, final int rlength, final int rfamilyoffset) {
+      int offset = left.getFamilyOffset();
+      int length = left.getFamilyLength() + left.getQualifierLength();
+      return getRawComparator().compareColumns(left.getBuffer(), offset, length,
+        left.getFamilyLength(offset),
+        right, roffset, rlength, rfamilyoffset);
+    }
+
+    int compareColumns(final KeyValue left, final short lrowlength,
+        final KeyValue right, final short rrowlength) {
+      int lfoffset = left.getFamilyOffset(lrowlength);
+      int rfoffset = right.getFamilyOffset(rrowlength);
+      int lclength = left.getTotalColumnLength(lrowlength,lfoffset);
+      int rclength = right.getTotalColumnLength(rrowlength, rfoffset);
+      int lfamilylength = left.getFamilyLength(lfoffset);
+      int rfamilylength = right.getFamilyLength(rfoffset);
+      return getRawComparator().compareColumns(left.getBuffer(), lfoffset,
+          lclength, lfamilylength,
+        right.getBuffer(), rfoffset, rclength, rfamilylength);
+    }
+
+    /**
+     * Compares the row and column of two keyvalues for equality
+     * @param left
+     * @param right
+     * @return True if same row and column.
+     */
+    public boolean matchingRowColumn(final KeyValue left,
+        final KeyValue right) {
+      short lrowlength = left.getRowLength();
+      short rrowlength = right.getRowLength();
+      // TsOffset = end of column data. just comparing Row+CF length of each
+      return ((left.getTimestampOffset() - left.getOffset()) ==
+              (right.getTimestampOffset() - right.getOffset())) &&
+        matchingRows(left, lrowlength, right, rrowlength) &&
+        compareColumns(left, lrowlength, right, rrowlength) == 0;
+    }
+
+    /**
+     * @param left
+     * @param right
+     * @return True if rows match.
+     */
+    public boolean matchingRows(final KeyValue left, final byte [] right) {
+      return compareRows(left, right) == 0;
+    }
+
+    /**
+     * Compares the row of two keyvalues for equality
+     * @param left
+     * @param right
+     * @return True if rows match.
+     */
+    public boolean matchingRows(final KeyValue left, final KeyValue right) {
+      short lrowlength = left.getRowLength();
+      short rrowlength = right.getRowLength();
+      return matchingRows(left, lrowlength, right, rrowlength);
+    }
+
+    /**
+     * @param left
+     * @param lrowlength
+     * @param right
+     * @param rrowlength
+     * @return True if rows match.
+     */
+    public boolean matchingRows(final KeyValue left, final short lrowlength,
+        final KeyValue right, final short rrowlength) {
+      return lrowlength == rrowlength &&
+        compareRows(left, lrowlength, right, rrowlength) == 0;
+    }
+
+    public boolean matchingRows(final byte [] left, final int loffset,
+        final int llength,
+        final byte [] right, final int roffset, final int rlength) {
+      int compare = compareRows(left, loffset, llength,
+          right, roffset, rlength);
+      if (compare != 0) {
+        return false;
+      }
+      return true;
+    }
+
+    /**
+     * Compares the row and timestamp of two keys
+     * Was called matchesWithoutColumn in HStoreKey.
+     * @param right Key to compare against.
+     * @return True if same row and timestamp is greater than the timestamp in
+     * <code>right</code>
+     */
+    public boolean matchingRowsGreaterTimestamp(final KeyValue left,
+        final KeyValue right) {
+      short lrowlength = left.getRowLength();
+      short rrowlength = right.getRowLength();
+      if (!matchingRows(left, lrowlength, right, rrowlength)) {
+        return false;
+      }
+      return left.getTimestamp() >= right.getTimestamp();
+    }
+
+    @Override
+    protected Object clone() throws CloneNotSupportedException {
+      return new KVComparator();
+    }
+
+    /**
+     * @return Comparator that ignores timestamps; useful counting versions.
+     */
+    public KVComparator getComparatorIgnoringTimestamps() {
+      KVComparator c = null;
+      try {
+        c = (KVComparator)this.clone();
+        c.getRawComparator().ignoreTimestamp = true;
+      } catch (CloneNotSupportedException e) {
+        LOG.error("Not supported", e);
+      }
+      return c;
+    }
+
+    /**
+     * @return Comparator that ignores key type; useful checking deletes
+     */
+    public KVComparator getComparatorIgnoringType() {
+      KVComparator c = null;
+      try {
+        c = (KVComparator)this.clone();
+        c.getRawComparator().ignoreType = true;
+      } catch (CloneNotSupportedException e) {
+        LOG.error("Not supported", e);
+      }
+      return c;
+    }
+  }
+
+  /**
+   * Creates a KeyValue that is last on the specified row id. That is,
+   * every other possible KeyValue for the given row would compareTo()
+   * less than the result of this call.
+   * @param row row key
+   * @return Last possible KeyValue on passed <code>row</code>
+   */
+  public static KeyValue createLastOnRow(final byte[] row) {
+    return new KeyValue(row, null, null, HConstants.LATEST_TIMESTAMP, Type.Minimum);
+  }
+
+  /**
+   * Create a KeyValue that is smaller than all other possible KeyValues
+   * for the given row. That is any (valid) KeyValue on 'row' would sort
+   * _after_ the result.
+   *
+   * @param row - row key (arbitrary byte array)
+   * @return First possible KeyValue on passed <code>row</code>
+   */
+  public static KeyValue createFirstOnRow(final byte [] row) {
+    return createFirstOnRow(row, HConstants.LATEST_TIMESTAMP);
+  }
+
+  /**
+   * Creates a KeyValue that is smaller than all other KeyValues that
+   * are older than the passed timestamp.
+   * @param row - row key (arbitrary byte array)
+   * @param ts - timestamp
+   * @return First possible key on passed <code>row</code> and timestamp.
+   */
+  public static KeyValue createFirstOnRow(final byte [] row,
+      final long ts) {
+    return new KeyValue(row, null, null, ts, Type.Maximum);
+  }
+
+  /**
+   * @param row - row key (arbitrary byte array)
+   * @param c column - {@link #parseColumn(byte[])} is called to split
+   * the column.
+   * @param ts - timestamp
+   * @return First possible key on passed <code>row</code>, column and timestamp
+   * @deprecated
+   */
+  public static KeyValue createFirstOnRow(final byte [] row, final byte [] c,
+      final long ts) {
+    byte [][] split = parseColumn(c);
+    return new KeyValue(row, split[0], split[1], ts, Type.Maximum);
+  }
+
+  /**
+   * Create a KeyValue for the specified row, family and qualifier that would be
+   * smaller than all other possible KeyValues that have the same row,family,qualifier.
+   * Used for seeking.
+   * @param row - row key (arbitrary byte array)
+   * @param family - family name
+   * @param qualifier - column qualifier
+   * @return First possible key on passed <code>row</code>, and column.
+   */
+  public static KeyValue createFirstOnRow(final byte [] row, final byte [] family,
+      final byte [] qualifier) {
+    return new KeyValue(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
+  }
+
+  public static KeyValue createFirstDeleteFamilyOnRow(final byte[] row,
+      final byte[] family) {
+    return createDeleteFamilyOnRow(row, family,
+        HConstants.LATEST_TIMESTAMP);
+  }
+
+  /**
+   * Create a Delete Family KeyValue for the specified row and family that would
+   * be smaller than all other possible Delete Family KeyValues that have the
+   * same row and family. Used for seeking.
+   *
+   * @param row
+   *          - row key (arbitrary byte array)
+   * @param family
+   *          - family name
+   * @param ts
+   *          - timestamp
+   * @return the Delete Family possible key on passed <code>row</code>
+   * and <code>ts</code>.
+   */
+  public static KeyValue createDeleteFamilyOnRow(final byte [] row,
+      final byte[] family, long ts) {
+    return new KeyValue(row, family, null, ts,
+        Type.DeleteFamily);
+  }
+
+  /**
+   * @param row - row key (arbitrary byte array)
+   * @param f - family name
+   * @param q - column qualifier
+   * @param ts - timestamp
+   * @return First possible key on passed <code>row</code>, column and timestamp
+   */
+  public static KeyValue createFirstOnRow(final byte [] row, final byte [] f,
+      final byte [] q, final long ts) {
+    return new KeyValue(row, f, q, ts, Type.Maximum);
+  }
+
+  /**
+   * Create a KeyValue for the specified row, family and qualifier that would be
+   * smaller than all other possible KeyValues that have the same row,
+   * family, qualifier.
+   * Used for seeking.
+   * @param row row key
+   * @param roffset row offset
+   * @param rlength row length
+   * @param family family name
+   * @param foffset family offset
+   * @param flength family length
+   * @param qualifier column qualifier
+   * @param qoffset qualifier offset
+   * @param qlength qualifier length
+   * @return First possible key on passed Row, Family, Qualifier.
+   */
+  public static KeyValue createFirstOnRow(final byte [] row,
+      final int roffset, final int rlength, final byte [] family,
+      final int foffset, final int flength, final byte [] qualifier,
+      final int qoffset, final int qlength) {
+    return new KeyValue(row, roffset, rlength, family,
+        foffset, flength, qualifier, qoffset, qlength,
+        HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0);
+  }
+
+  /**
+   * Create a KeyValue for the specified row, family, qualifier, timestamp that
+   * would be smaller than or equal to all other possible KeyValues that have
+   * the same row, family, qualifier, timestamp.
+   * Used for seeking.
+   * @param row row key
+   * @param roffset row offset
+   * @param rlength row length
+   * @param family family name
+   * @param foffset family offset
+   * @param flength family length
+   * @param qualifier column qualifier
+   * @param qoffset qualifier offset
+   * @param qlength qualifier length
+   * @param timestamp timestamp
+   * @return First possible key on passed row, family, qualifier, timestamp
+   */
+  public static KeyValue createFirstOnRow(final byte [] row,
+      final int roffset, final int rlength, final byte [] family,
+      final int foffset, final int flength, final byte [] qualifier,
+      final int qoffset, final int qlength, long timestamp) {
+    return new KeyValue(row, roffset, rlength, family,
+        foffset, flength, qualifier, qoffset, qlength,
+        timestamp, Type.Maximum, null, 0, 0);
+  }
+
+  /**
+   * Create a KeyValue for the specified row, family and qualifier that would be
+   * larger than or equal to all other possible KeyValues that have the same
+   * row, family, qualifier.
+   * Used for reseeking.
+   * @param row row key
+   * @param roffset row offset
+   * @param rlength row length
+   * @param family family name
+   * @param foffset family offset
+   * @param flength family length
+   * @param qualifier column qualifier
+   * @param qoffset qualifier offset
+   * @param qlength qualifier length
+   * @return Last possible key on passed row, family, qualifier.
+   */
+  public static KeyValue createLastOnRow(final byte [] row,
+      final int roffset, final int rlength, final byte [] family,
+      final int foffset, final int flength, final byte [] qualifier,
+      final int qoffset, final int qlength) {
+    return new KeyValue(row, roffset, rlength, family,
+        foffset, flength, qualifier, qoffset, qlength,
+        HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
+  }
+
+  /**
+   * Similar to {@link #createLastOnRow(byte[], int, int, byte[], int, int,
+   * byte[], int, int)} but takes a {@link KeyValue}.
+   *
+   * @param kv the key-value pair to take row and column from
+   * @return the last key on the row/column of the given key-value pair
+   */
+  public KeyValue createLastOnRowCol() {
+    return new KeyValue(
+        bytes, getRowOffset(), getRowLength(),
+        bytes, getFamilyOffset(), getFamilyLength(),
+        bytes, getQualifierOffset(), getQualifierLength(),
+        HConstants.OLDEST_TIMESTAMP, Type.Minimum, null, 0, 0);
+  }
+
+  /**
+   * Creates the first KV with the row/family/qualifier of this KV and the
+   * given timestamp. Uses the "maximum" KV type that guarantees that the new
+   * KV is the lowest possible for this combination of row, family, qualifier,
+   * and timestamp. This KV's own timestamp is ignored. While this function
+   * copies the value from this KV, it is normally used on key-only KVs.
+   */
+  public KeyValue createFirstOnRowColTS(long ts) {
+    return new KeyValue(
+        bytes, getRowOffset(), getRowLength(),
+        bytes, getFamilyOffset(), getFamilyLength(),
+        bytes, getQualifierOffset(), getQualifierLength(),
+        ts, Type.Maximum, bytes, getValueOffset(), getValueLength());
+  }
+
+  /**
+   * @param b
+   * @return A KeyValue made of a byte array that holds the key-only part.
+   * Needed to convert hfile index members to KeyValues.
+   */
+  public static KeyValue createKeyValueFromKey(final byte [] b) {
+    return createKeyValueFromKey(b, 0, b.length);
+  }
+
+  /**
+   * @param bb
+   * @return A KeyValue made of a byte buffer that holds the key-only part.
+   * Needed to convert hfile index members to KeyValues.
+   */
+  public static KeyValue createKeyValueFromKey(final ByteBuffer bb) {
+    return createKeyValueFromKey(bb.array(), bb.arrayOffset(), bb.limit());
+  }
+
+  /**
+   * @param b
+   * @param o
+   * @param l
+   * @return A KeyValue made of a byte array that holds the key-only part.
+   * Needed to convert hfile index members to KeyValues.
+   */
+  public static KeyValue createKeyValueFromKey(final byte [] b, final int o,
+      final int l) {
+    byte [] newb = new byte[l + ROW_OFFSET];
+    System.arraycopy(b, o, newb, ROW_OFFSET, l);
+    Bytes.putInt(newb, 0, l);
+    Bytes.putInt(newb, Bytes.SIZEOF_INT, 0);
+    return new KeyValue(newb);
+  }
+
+  /**
+   * Compare key portion of a {@link KeyValue} for keys in <code>-ROOT-<code>
+   * table.
+   */
+  public static class RootKeyComparator extends MetaKeyComparator {
+    public int compareRows(byte [] left, int loffset, int llength,
+        byte [] right, int roffset, int rlength) {
+      // Rows look like this: .META.,ROW_FROM_META,RID
+      //        LOG.info("ROOT " + Bytes.toString(left, loffset, llength) +
+      //          "---" + Bytes.toString(right, roffset, rlength));
+      final int metalength = 7; // '.META.' length
+      int lmetaOffsetPlusDelimiter = loffset + metalength;
+      int leftFarDelimiter = getDelimiterInReverse(left,
+          lmetaOffsetPlusDelimiter,
+          llength - metalength, HRegionInfo.DELIMITER);
+      int rmetaOffsetPlusDelimiter = roffset + metalength;
+      int rightFarDelimiter = getDelimiterInReverse(right,
+          rmetaOffsetPlusDelimiter, rlength - metalength,
+          HRegionInfo.DELIMITER);
+      if (leftFarDelimiter < 0 && rightFarDelimiter >= 0) {
+        // Nothing between .META. and regionid.  Its first key.
+        return -1;
+      } else if (rightFarDelimiter < 0 && leftFarDelimiter >= 0) {
+        return 1;
+      } else if (leftFarDelimiter < 0 && rightFarDelimiter < 0) {
+        return 0;
+      }
+      int result = super.compareRows(left, lmetaOffsetPlusDelimiter,
+          leftFarDelimiter - lmetaOffsetPlusDelimiter,
+          right, rmetaOffsetPlusDelimiter,
+          rightFarDelimiter - rmetaOffsetPlusDelimiter);
+      if (result != 0) {
+        return result;
+      }
+      // Compare last part of row, the rowid.
+      leftFarDelimiter++;
+      rightFarDelimiter++;
+      result = compareRowid(left, leftFarDelimiter,
+          llength - (leftFarDelimiter - loffset),
+          right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
+      return result;
+    }
+  }
+
+  /**
+   * Comparator that compares row component only of a KeyValue.
+   */
+  public static class RowComparator implements Comparator<KeyValue> {
+    final KVComparator comparator;
+
+    public RowComparator(final KVComparator c) {
+      this.comparator = c;
+    }
+
+    public int compare(KeyValue left, KeyValue right) {
+      return comparator.compareRows(left, right);
+    }
+  }
+
+  /**
+   * Compare key portion of a {@link KeyValue} for keys in <code>.META.</code>
+   * table.
+   */
+  public static class MetaKeyComparator extends KeyComparator {
+    public int compareRows(byte [] left, int loffset, int llength,
+        byte [] right, int roffset, int rlength) {
+      //        LOG.info("META " + Bytes.toString(left, loffset, llength) +
+      //          "---" + Bytes.toString(right, roffset, rlength));
+      int leftDelimiter = getDelimiter(left, loffset, llength,
+          HRegionInfo.DELIMITER);
+      int rightDelimiter = getDelimiter(right, roffset, rlength,
+          HRegionInfo.DELIMITER);
+      if (leftDelimiter < 0 && rightDelimiter >= 0) {
+        // Nothing between .META. and regionid.  Its first key.
+        return -1;
+      } else if (rightDelimiter < 0 && leftDelimiter >= 0) {
+        return 1;
+      } else if (leftDelimiter < 0 && rightDelimiter < 0) {
+        return 0;
+      }
+      // Compare up to the delimiter
+      int result = Bytes.compareTo(left, loffset, leftDelimiter - loffset,
+          right, roffset, rightDelimiter - roffset);
+      if (result != 0) {
+        return result;
+      }
+      // Compare middle bit of the row.
+      // Move past delimiter
+      leftDelimiter++;
+      rightDelimiter++;
+      int leftFarDelimiter = getRequiredDelimiterInReverse(left, leftDelimiter,
+          llength - (leftDelimiter - loffset), HRegionInfo.DELIMITER);
+      int rightFarDelimiter = getRequiredDelimiterInReverse(right,
+          rightDelimiter, rlength - (rightDelimiter - roffset),
+          HRegionInfo.DELIMITER);
+      // Now compare middlesection of row.
+      result = super.compareRows(left, leftDelimiter,
+          leftFarDelimiter - leftDelimiter, right, rightDelimiter,
+          rightFarDelimiter - rightDelimiter);
+      if (result != 0) {
+        return result;
+      }
+      // Compare last part of row, the rowid.
+      leftFarDelimiter++;
+      rightFarDelimiter++;
+      result = compareRowid(left, leftFarDelimiter,
+          llength - (leftFarDelimiter - loffset),
+          right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
+      return result;
+    }
+
+    protected int compareRowid(byte[] left, int loffset, int llength,
+        byte[] right, int roffset, int rlength) {
+      return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
+    }
+  }
+
+  /**
+   * Avoids redundant comparisons for better performance.
+   */
+  public static interface SamePrefixComparator<T> {
+    /**
+     * Compare two keys assuming that the first n bytes are the same.
+     * @param commonPrefix How many bytes are the same.
+     */
+    public int compareIgnoringPrefix(int commonPrefix,
+        T left, int loffset, int llength,
+        T right, int roffset, int rlength);
+  }
+
+  /**
+   * Compare key portion of a {@link KeyValue}.
+   */
+  public static class KeyComparator
+      implements RawComparator<byte []>, SamePrefixComparator<byte[]> {
+    volatile boolean ignoreTimestamp = false;
+    volatile boolean ignoreType = false;
+
+    public int compare(byte[] left, int loffset, int llength, byte[] right,
+        int roffset, int rlength) {
+      // Compare row
+      short lrowlength = Bytes.toShort(left, loffset);
+      short rrowlength = Bytes.toShort(right, roffset);
+      int compare = compareRows(left, loffset + Bytes.SIZEOF_SHORT,
+          lrowlength, right, roffset + Bytes.SIZEOF_SHORT, rrowlength);
+      if (compare != 0) {
+        return compare;
+      }
+
+      // Compare the rest of the two KVs without making any assumptions about
+      // the common prefix. This function will not compare rows anyway, so we
+      // don't need to tell it that the common prefix includes the row.
+      return compareWithoutRow(0, left, loffset, llength, right, roffset,
+          rlength, rrowlength);
+    }
+
+    /**
+     * Compare the two key-values, ignoring the prefix of the given length
+     * that is known to be the same between the two.
+     * @param commonPrefix the prefix length to ignore
+     */
+    @Override
+    public int compareIgnoringPrefix(int commonPrefix, byte[] left,
+        int loffset, int llength, byte[] right, int roffset, int rlength) {
+      // Compare row
+      short lrowlength = Bytes.toShort(left, loffset);
+      short rrowlength;
+
+      int comparisonResult = 0;
+      if (commonPrefix < ROW_LENGTH_SIZE) {
+        // almost nothing in common
+        rrowlength = Bytes.toShort(right, roffset);
+        comparisonResult = compareRows(left, loffset + ROW_LENGTH_SIZE,
+            lrowlength, right, roffset + ROW_LENGTH_SIZE, rrowlength);
+      } else { // the row length is the same
+        rrowlength = lrowlength;
+        if (commonPrefix < ROW_LENGTH_SIZE + rrowlength) {
+          // The rows are not the same. Exclude the common prefix and compare
+          // the rest of the two rows.
+          int common = commonPrefix - ROW_LENGTH_SIZE;
+          comparisonResult = compareRows(
+              left, loffset + common + ROW_LENGTH_SIZE, lrowlength - common,
+              right, roffset + common + ROW_LENGTH_SIZE, rrowlength - common);
+        }
+      }
+      if (comparisonResult != 0) {
+        return comparisonResult;
+      }
+
+      assert lrowlength == rrowlength;
+
+      return compareWithoutRow(commonPrefix, left, loffset, llength, right,
+          roffset, rlength, lrowlength);
+    }
+
+    /**
+     * Compare column, timestamp, and key type (everything except the row).
+     * This method is used both in the normal comparator and the "same-prefix"
+     * comparator. Note that we are assuming that row portions of both KVs have
+     * already been parsed and found identical, and we don't validate that
+     * assumption here.
+     * @param commonPrefix the length of the common prefix of the two
+     *          key-values being compared, including row length and row
+     */
+    private int compareWithoutRow(int commonPrefix, byte[] left, int loffset,
+        int llength, byte[] right, int roffset, int rlength, short rowlength) {
+      // Compare column family. Start comparing past row and family length.
+      int lcolumnoffset = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE +
+          rowlength + loffset;
+      int rcolumnoffset = ROW_LENGTH_SIZE + FAMILY_LENGTH_SIZE +
+          rowlength + roffset;
+      int lcolumnlength = llength - TIMESTAMP_TYPE_SIZE -
+          (lcolumnoffset - loffset);
+      int rcolumnlength = rlength - TIMESTAMP_TYPE_SIZE -
+          (rcolumnoffset - roffset);
+
+      // If row matches, and no column in the 'left' AND put type is 'minimum',
+      // then return that left is larger than right.
+
+      // This supports 'last key on a row' - the magic is if there is no column
+      // in the left operand, and the left operand has a type of '0' - magical
+      // value, then we say the left is bigger.  This will let us seek to the
+      // last key in a row.
+
+      byte ltype = left[loffset + (llength - 1)];
+      byte rtype = right[roffset + (rlength - 1)];
+
+      // If the column is not specified, the "minimum" key type appears the
+      // latest in the sorted order, regardless of the timestamp. This is used
+      // for specifying the last key/value in a given row, because there is no
+      // "lexicographically last column" (it would be infinitely long).  The
+      // "maximum" key type does not need this behavior.
+      if (lcolumnlength == 0 && ltype == Type.Minimum.getCode()) {
+        // left is "bigger", i.e. it appears later in the sorted order
+        return 1;
+      }
+      if (rcolumnlength == 0 && rtype == Type.Minimum.getCode()) {
+        return -1;
+      }
+
+      int common = 0;
+      if (commonPrefix > 0) {
+        common = Math.max(0, commonPrefix -
+            rowlength - ROW_LENGTH_SIZE - FAMILY_LENGTH_SIZE);
+        common = Math.min(common, Math.min(lcolumnlength, rcolumnlength));
+      }
+
+      final int comparisonResult = Bytes.compareTo(
+          left, lcolumnoffset + common, lcolumnlength - common,
+          right, rcolumnoffset + common, rcolumnlength - common);
+      if (comparisonResult != 0) {
+        return comparisonResult;
+      }
+
+      return compareTimestampAndType(left, loffset, llength, right, roffset,
+          rlength, ltype, rtype);
+    }
+
+    private int compareTimestampAndType(byte[] left, int loffset, int llength,
+        byte[] right, int roffset, int rlength, byte ltype, byte rtype) {
+      int compare;
+      if (!this.ignoreTimestamp) {
+        // Get timestamps.
+        long ltimestamp = Bytes.toLong(left,
+            loffset + (llength - TIMESTAMP_TYPE_SIZE));
+        long rtimestamp = Bytes.toLong(right,
+            roffset + (rlength - TIMESTAMP_TYPE_SIZE));
+        compare = compareTimestamps(ltimestamp, rtimestamp);
+        if (compare != 0) {
+          return compare;
+        }
+      }
+
+      if (!this.ignoreType) {
+        // Compare types. Let the delete types sort ahead of puts; i.e. types
+        // of higher numbers sort before those of lesser numbers. Maximum (255)
+        // appears ahead of everything, and minimum (0) appears after
+        // everything.
+        return (0xff & rtype) - (0xff & ltype);
+      }
+      return 0;
+    }
+
+    public int compare(byte[] left, byte[] right) {
+      return compare(left, 0, left.length, right, 0, right.length);
+    }
+
+    public int compareRows(byte [] left, int loffset, int llength,
+        byte [] right, int roffset, int rlength) {
+      return Bytes.compareTo(left, loffset, llength, right, roffset, rlength);
+    }
+
+    protected int compareColumns(
+        byte [] left, int loffset, int llength, final int lfamilylength,
+        byte [] right, int roffset, int rlength, final int rfamilylength) {
+      return KeyValue.compareColumns(left, loffset, llength, lfamilylength,
+        right, roffset, rlength, rfamilylength);
+    }
+
+    int compareTimestamps(final long ltimestamp, final long rtimestamp) {
+      // The below older timestamps sorting ahead of newer timestamps looks
+      // wrong but it is intentional. This way, newer timestamps are first
+      // found when we iterate over a memstore and newer versions are the
+      // first we trip over when reading from a store file.
+      if (ltimestamp < rtimestamp) {
+        return 1;
+      } else if (ltimestamp > rtimestamp) {
+        return -1;
+      }
+      return 0;
+    }
+  }
+
+  // HeapSize
+  public long heapSize() {
+    return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE +
+        ClassSize.align(ClassSize.ARRAY + length) +
+        (2 * Bytes.SIZEOF_INT) +
+        Bytes.SIZEOF_LONG);
+  }
+
+  // this overload assumes that the length bytes have already been read,
+  // and it expects the length of the KeyValue to be explicitly passed
+  // to it.
+  public void readFields(int length, final DataInput in) throws IOException {
+    this.length = length;
+    this.offset = 0;
+    this.bytes = new byte[this.length];
+    in.readFully(this.bytes, 0, this.length);
+    this.verify();
+  }
+
+  // Writable
+  public void readFields(final DataInput in) throws IOException {
+    int length = in.readInt();
+    readFields(length, in);
+  }
+
+  public void write(final DataOutput out) throws IOException {
+    this.verify();
+    out.writeInt(this.length);
+    out.write(this.bytes, this.offset, this.length);
+  }
+
+  /**
+   * Returns the size of a key/value pair in bytes
+   * @param keyLength length of the key in bytes
+   * @param valueLength length of the value in bytes
+   * @return key/value pair size in bytes
+   */
+  public static int getKVSize(final int keyLength,
+      final int valueLength) {
+    return ROW_OFFSET + keyLength + valueLength;
+  }
+
+  /**
+   * Returns new keyValue which has all properties same as the passed ones
+   * except the value
+   *
+   * @param newValue
+   *          - new value
+   * @return new KeyValue
+   */
+  public KeyValue modifyValueAndClone(byte[] newValue) {
+    KeyValue newKV = new KeyValue(bytes, getRowOffset(), getRowLength(), bytes,
+        getFamilyOffset(), getFamilyLength(), bytes, getQualifierOffset(),
+        getQualifierLength(), this.getTimestamp(), Type.codeToType(this
+            .getType()), newValue, 0, newValue == null ? 0 : newValue.length);
+    newKV.setMemstoreTS(this.getMemstoreTS());
+    return newKV;
+  }
+
+  public static KeyValue generateKeyValue(String key, int dataSize) {
+    Random seededRandom = new Random(System.nanoTime());
+    byte[] randomBytes = new byte[dataSize];
+    seededRandom.nextBytes(randomBytes);
+
+    return new KeyValue(Bytes.toBytes(key), Bytes.toBytes(""),
+      Bytes.toBytes(""), randomBytes);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/RegionException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/RegionException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/RegionException.java
new file mode 100644
index 0000000..6687fb6
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/RegionException.java
@@ -0,0 +1,56 @@
+/**
+ * Copyright 2008 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase;
+
+import java.io.IOException;
+
+
+/**
+ * Thrown when something happens related to region handling.
+ * Subclasses have to be more specific.
+ */
+public class RegionException extends IOException {
+  private long waitMillis = 0;
+
+  private static final long serialVersionUID = 1473510258071111371L;
+
+  /** default constructor */
+  public RegionException() {
+    super();
+  }
+
+  /**
+   * Constructor
+   * @param s message
+   */
+  public RegionException(String s) {
+    super(s);
+  }
+
+  public RegionException(String s, long waitMillis) {
+    super(s);
+    this.waitMillis = waitMillis;
+  }
+
+  public long getBackoffTimeMillis() {
+    return this.waitMillis;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/client/NoLeaderForRegionException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/client/NoLeaderForRegionException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/client/NoLeaderForRegionException.java
new file mode 100644
index 0000000..8076873
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/client/NoLeaderForRegionException.java
@@ -0,0 +1,22 @@
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.RegionException;
+
+/**
+ * Thrown when the quorum client cannot find the leader for a particular
+ * region.
+ */
+public class NoLeaderForRegionException extends RegionException {
+  /** default constructor */
+  public NoLeaderForRegionException() {
+    super();
+  }
+
+  /**
+   * Constructor
+   * @param s message
+   */
+  public NoLeaderForRegionException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationObserver.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationObserver.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationObserver.java
new file mode 100644
index 0000000..2f9db79
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/conf/ConfigurationObserver.java
@@ -0,0 +1,36 @@
+/**
+ * Copyright 2013 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.conf;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Every class that wants to observe online changes in Configuration properties,
+ * must implement interface (and also, register itself with the
+ * <code>ConfigurationManager</code> object.
+ */
+public interface ConfigurationObserver {
+
+  /**
+   * This method would be called by the <code>ConfigurationManager</code>
+   * object when the <code>Configuration</code> object is reloaded from disk.
+   */
+  void notifyOnChange(Configuration conf);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/FetchTask.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/FetchTask.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/FetchTask.java
new file mode 100644
index 0000000..6399634
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/consensus/client/FetchTask.java
@@ -0,0 +1,65 @@
+package org.apache.hadoop.hbase.consensus.client;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import org.apache.hadoop.hbase.consensus.log.LogFileInfo;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Fetch log files from a peer
+ */
+@ThriftStruct
+public final class FetchTask implements Runnable {
+  // The peer's address
+  private final String addr;
+
+  // List of files that are going to be downloaded from the peer
+  private List<LogFileInfo> fileInfos = new ArrayList<>();
+
+  public FetchTask(String address) {
+    addr = address;
+  }
+
+  @ThriftConstructor
+  public FetchTask(
+      @ThriftField(1) String address,
+      @ThriftField(2) List<LogFileInfo> infos) {
+    addr = address;
+    fileInfos = infos;
+  }
+
+  @ThriftField(1)
+  public String getAddr() {
+    return addr;
+  }
+
+  @ThriftField(2)
+  public List<LogFileInfo> getFileInfos() {
+    return fileInfos;
+  }
+
+  public void addTask(LogFileInfo info) {
+    fileInfos.add(info);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Requests for peer ").append(addr).append(" :");
+    for (LogFileInfo info : fileInfos) {
+      sb.append(" {").append(info.getFilename()).append(",[")
+          .append(info.getInitialIndex()).append(",")
+          .append(info.getLastIndex()).append("]}");
+    }
+
+    return sb.toString();
+  }
+
+  @Override
+  public void run() {
+    // TODO in part 2: fetch log files from the peer!
+  }
+}


[06/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java
new file mode 100644
index 0000000..7c828db
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/Compression.java
@@ -0,0 +1,345 @@
+/**
+ * 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.hadoop.hbase.io.hfile;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.hbase.util.InjectionEvent;
+import org.apache.hadoop.hbase.util.InjectionHandler;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionInputStream;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Compression related stuff.
+ * Copied from hadoop-3315 tfile.
+ */
+public final class Compression {
+  static final Log LOG = LogFactory.getLog(Compression.class);
+
+  /**
+   * Prevent the instantiation of class.
+   */
+  private Compression() {
+    super();
+  }
+
+  static class FinishOnFlushCompressionStream extends FilterOutputStream {
+    public FinishOnFlushCompressionStream(CompressionOutputStream cout) {
+      super(cout);
+    }
+
+    @Override
+    public void write(byte b[], int off, int len) throws IOException {
+      out.write(b, off, len);
+    }
+
+    @Override
+    public void flush() throws IOException {
+      CompressionOutputStream cout = (CompressionOutputStream) out;
+      cout.finish();
+      cout.flush();
+      cout.resetState();
+    }
+  }
+
+  /**
+   * Compression algorithms. Don't change the order of codecs here, and add new codecs to the end
+   * of the list, because we are using ordinal numbers of elements of this enum in some of our
+   * persistent data formats.
+   */
+  public static enum Algorithm {
+    LZO("lzo") {
+      // Use base type to avoid compile-time dependencies.
+      private volatile transient CompressionCodec lzoCodec;
+      @Override
+      protected void deleteCodec() {
+        lzoCodec = null;
+      }
+      @Override
+      public CompressionCodec getCodec(Configuration conf) {
+        if (lzoCodec == null) {
+          try {
+            Class<?> externalCodec = ClassLoader.getSystemClassLoader().loadClass(
+                "com.hadoop.compression.lzo.LzoCodec");
+            lzoCodec = (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf);
+          } catch (ClassNotFoundException e) {
+            throw new RuntimeException(e);
+          }
+        }
+        return lzoCodec;
+      }
+    },
+    GZ("gz") {
+      private volatile transient GzipCodec codec;
+      private transient Object lock = new Object();
+
+      @Override
+      public DefaultCodec getCodec(Configuration conf) {
+        if (codec == null) {
+          synchronized (lock) {
+            if (codec == null) {
+              GzipCodec tmpCodec = new ReusableStreamGzipCodec();
+              tmpCodec.setConf(new Configuration(conf));
+              codec = tmpCodec;
+            }
+          }
+        }
+        return codec;
+      }
+      @Override
+      protected void deleteCodec() {
+        codec = null;
+      }
+    },
+    NONE("none") {
+      @Override
+      public DefaultCodec getCodec(Configuration conf) {
+        return null;
+      }
+      @Override
+      protected void deleteCodec() {
+        return;
+      }
+
+      @Override
+      public synchronized InputStream createDecompressionStream(
+          InputStream downStream, Decompressor decompressor,
+          int downStreamBufferSize) throws IOException {
+        if (downStreamBufferSize > 0) {
+          return new BufferedInputStream(downStream, downStreamBufferSize);
+        }
+        // else {
+          // Make sure we bypass FSInputChecker buffer.
+        // return new BufferedInputStream(downStream, 1024);
+        // }
+        // }
+        return downStream;
+      }
+
+      @Override
+      public synchronized OutputStream createCompressionStream(
+          OutputStream downStream, Compressor compressor,
+          int downStreamBufferSize) throws IOException {
+        if (downStreamBufferSize > 0) {
+          return new BufferedOutputStream(downStream, downStreamBufferSize);
+        }
+
+        return downStream;
+      }
+    },
+    SNAPPY("snappy") {
+      private volatile transient CompressionCodec snappyCodec;
+      private transient Object lock = new Object();
+
+      @Override
+      protected void deleteCodec() {
+        snappyCodec = null;
+      }
+      @SuppressWarnings("unchecked")
+      @Override
+      public CompressionCodec getCodec(Configuration conf) {
+        if (snappyCodec == null) {
+          synchronized (lock) {
+            if (snappyCodec == null) {
+              CompressionCodec tmpCodec;
+              try {
+                Class<? extends CompressionCodec> snappyCodecClass =
+                    (Class<? extends CompressionCodec>)
+                    Class.forName(CompressionCodec.class.getPackage().getName() + ".SnappyCodec");
+                tmpCodec = snappyCodecClass.newInstance();
+              } catch (InstantiationException e) {
+                LOG.error(e);
+                throw new RuntimeException(e);
+              } catch (IllegalAccessException e) {
+                LOG.error(e);
+                throw new RuntimeException(e);
+              } catch (ClassNotFoundException e) {
+                LOG.error(e);
+                throw new RuntimeException(e);
+              }
+              ((Configurable) tmpCodec).setConf(new Configuration(conf));
+              snappyCodec = tmpCodec;
+            }
+          }
+        }
+        return (CompressionCodec) snappyCodec;
+      }
+    };
+
+    private final Configuration conf;
+    private final String compressName;
+    // data input buffer size to absorb small reads from application.
+    private static final int DATA_IBUF_SIZE = 1 * 1024;
+    // data output buffer size to absorb small writes from application.
+    private static final int DATA_OBUF_SIZE = 4 * 1024;
+
+    Algorithm(String name) {
+      this.conf = new Configuration();
+      this.conf.setBoolean("hadoop.native.lib", true);
+      this.compressName = name;
+    }
+
+    public abstract CompressionCodec getCodec(Configuration conf);
+
+    public InputStream createDecompressionStream(
+        InputStream downStream, Decompressor decompressor,
+        int downStreamBufferSize) throws IOException {
+      CompressionCodec codec = getCodec(conf);
+      // Set the internal buffer size to read from down stream.
+      if (downStreamBufferSize > 0) {
+        Configurable c = (Configurable) codec;
+        c.getConf().setInt("io.file.buffer.size", downStreamBufferSize);
+      }
+      CompressionInputStream cis =
+          codec.createInputStream(downStream, decompressor);
+      BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
+      return bis2;
+
+    }
+
+    public OutputStream createCompressionStream(
+        OutputStream downStream, Compressor compressor, int downStreamBufferSize)
+        throws IOException {
+      OutputStream bos1 = null;
+      if (downStreamBufferSize > 0) {
+        bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
+      }
+      else {
+        bos1 = downStream;
+      }
+      CompressionOutputStream cos =
+          createPlainCompressionStream(bos1, compressor);
+      BufferedOutputStream bos2 =
+          new BufferedOutputStream(new FinishOnFlushCompressionStream(cos),
+              DATA_OBUF_SIZE);
+      return bos2;
+    }
+
+    /**
+     * Creates a compression stream without any additional wrapping into
+     * buffering streams.
+     */
+    public CompressionOutputStream createPlainCompressionStream(
+        OutputStream downStream, Compressor compressor) throws IOException {
+      CompressionCodec codec = getCodec(conf);
+      ((Configurable)codec).getConf().setInt("io.file.buffer.size", 32 * 1024);
+      return codec.createOutputStream(downStream, compressor);
+    }
+
+    public Compressor getCompressor() {
+      CompressionCodec codec = getCodec(conf);
+      if (codec != null) {
+        Compressor compressor = CodecPool.getCompressor(codec);
+        if (compressor != null) {
+          if (compressor.finished()) {
+            // Somebody returns the compressor to CodecPool but is still using
+            // it.
+            LOG
+                .warn("Compressor obtained from CodecPool is already finished()");
+            // throw new AssertionError(
+            // "Compressor obtained from CodecPool is already finished()");
+          }
+          compressor.reset();
+        }
+        return compressor;
+      }
+      return null;
+    }
+
+    public void returnCompressor(Compressor compressor) {
+      if (compressor != null) {
+        CodecPool.returnCompressor(compressor);
+      }
+    }
+
+    public Decompressor getDecompressor() {
+      CompressionCodec codec = getCodec(conf);
+      if (codec != null) {
+        Decompressor decompressor = CodecPool.getDecompressor(codec);
+        if (decompressor != null) {
+          if (decompressor.finished()) {
+            // Somebody returns the decompressor to CodecPool but is still using
+            // it.
+            LOG
+                .warn("Deompressor obtained from CodecPool is already finished()");
+            // throw new AssertionError(
+            // "Decompressor obtained from CodecPool is already finished()");
+          }
+          decompressor.reset();
+        }
+        return decompressor;
+      }
+
+      return null;
+    }
+
+    public void returnDecompressor(Decompressor decompressor) {
+      if (decompressor != null) {
+        CodecPool.returnDecompressor(decompressor);
+      }
+    }
+
+    public String getName() {
+      return compressName;
+    }
+
+    protected abstract void deleteCodec();
+  }
+
+  public static Algorithm getCompressionAlgorithmByName(String compressName) {
+    Algorithm[] algos = Algorithm.class.getEnumConstants();
+
+    for (Algorithm a : algos) {
+      if (a.getName().equals(compressName)) {
+        return a;
+      }
+    }
+
+    throw new IllegalArgumentException(
+        "Unsupported compression algorithm name: " + compressName);
+  }
+
+  static String[] getSupportedAlgorithms() {
+    Algorithm[] algos = Algorithm.class.getEnumConstants();
+
+    String[] ret = new String[algos.length];
+    int i = 0;
+    for (Algorithm a : algos) {
+      ret[i++] = a.getName();
+    }
+
+    return ret;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/ReusableStreamGzipCodec.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/ReusableStreamGzipCodec.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/ReusableStreamGzipCodec.java
new file mode 100644
index 0000000..2b1d48b
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/ReusableStreamGzipCodec.java
@@ -0,0 +1,133 @@
+/*
+ * 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.hadoop.hbase.io.hfile;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.zip.GZIPOutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.CompressorStream;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.compress.zlib.ZlibFactory;
+
+/**
+ * Fixes an inefficiency in Hadoop's Gzip codec, allowing to reuse compression
+ * streams.
+ */
+public class ReusableStreamGzipCodec extends GzipCodec {
+
+  private static final Log LOG = LogFactory.getLog(Compression.class);
+
+  /**
+   * A bridge that wraps around a DeflaterOutputStream to make it a
+   * CompressionOutputStream.
+   */
+  protected static class ReusableGzipOutputStream extends CompressorStream {
+
+    private static final int GZIP_HEADER_LENGTH = 10;
+
+    /**
+     * Fixed ten-byte gzip header. See {@link GZIPOutputStream}'s source for
+     * details.
+     */
+    private static final byte[] GZIP_HEADER;
+
+    static {
+      // Capture the fixed ten-byte header hard-coded in GZIPOutputStream.
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      byte[] header = null;
+      GZIPOutputStream gzipStream = null;
+      try {
+        gzipStream  = new GZIPOutputStream(baos);
+        gzipStream.finish();
+        header = Arrays.copyOfRange(baos.toByteArray(), 0, GZIP_HEADER_LENGTH);
+      } catch (IOException e) {
+        throw new RuntimeException("Could not create gzip stream", e);
+      } finally {
+        if (gzipStream != null) {
+          try {
+            gzipStream.close();
+          } catch (IOException e) {
+            LOG.error(e);
+          }
+        }
+      }
+      GZIP_HEADER = header;
+    }
+
+    private static class ResetableGZIPOutputStream extends GZIPOutputStream {
+      public ResetableGZIPOutputStream(OutputStream out) throws IOException {
+        super(out);
+      }
+
+      public void resetState() throws IOException {
+        def.reset();
+        crc.reset();
+        out.write(GZIP_HEADER);
+      }
+    }
+
+    public ReusableGzipOutputStream(OutputStream out) throws IOException {
+      super(new ResetableGZIPOutputStream(out));
+    }
+
+    @Override
+    public void close() throws IOException {
+      out.close();
+    }
+
+    @Override
+    public void flush() throws IOException {
+      out.flush();
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      out.write(b);
+    }
+
+    @Override
+    public void write(byte[] data, int offset, int length) throws IOException {
+      out.write(data, offset, length);
+    }
+
+    @Override
+    public void finish() throws IOException {
+      ((GZIPOutputStream) out).finish();
+    }
+
+    @Override
+    public void resetState() throws IOException {
+      ((ResetableGZIPOutputStream) out).resetState();
+    }
+  }
+
+  @Override
+  public CompressionOutputStream createOutputStream(OutputStream out)
+      throws IOException {
+    if (ZlibFactory.isNativeZlibLoaded(getConf())) {
+      return super.createOutputStream(out);
+    }
+    return new ReusableGzipOutputStream(out);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CacheFullException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CacheFullException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CacheFullException.java
new file mode 100644
index 0000000..cfb3297
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/CacheFullException.java
@@ -0,0 +1,54 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.io.hfile.bucket;
+
+import java.io.IOException;
+
+
+/**
+ * Thrown by {@link org.apache.hadoop.hbase.util.BucketAllocator#allocateBlock(int)} when cache is full for
+ * the requested size
+ */
+public class CacheFullException extends IOException {
+  private static final long serialVersionUID = 3265127301824638920L;
+  private int requestedSize, bucketIndex;
+
+  public CacheFullException(int requestedSize, int bucketIndex) {
+    super();
+    this.requestedSize = requestedSize;
+    this.bucketIndex = bucketIndex;
+  }
+
+  public int bucketIndex() {
+    return bucketIndex;
+  }
+
+  public int requestedSize() {
+    return requestedSize;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(1024);
+    sb.append("Allocator requested size ").append(requestedSize);
+    sb.append(" for bucket ").append(bucketIndex);
+    return sb.toString();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/ipc/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/ipc/ByteBufferOutputStream.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/ipc/ByteBufferOutputStream.java
new file mode 100644
index 0000000..92856ce
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/ipc/ByteBufferOutputStream.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.ipc;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+/**
+ * Not thread safe!
+ */
+public class ByteBufferOutputStream extends OutputStream {
+
+  protected ByteBuffer buf;
+
+  public ByteBufferOutputStream(int capacity) {
+    this(capacity, false);
+  }
+
+  public ByteBufferOutputStream(int capacity, boolean useDirectByteBuffer) {
+    if (useDirectByteBuffer) {
+      buf = ByteBuffer.allocateDirect(capacity);
+    } else {
+      buf = ByteBuffer.allocate(capacity);
+    }
+  }
+
+  public int size() {
+    return buf.position();
+  }
+
+  /**
+   * This flips the underlying BB so be sure to use it _last_!
+   * @return
+   */
+  public ByteBuffer getByteBuffer() {
+    buf.flip();
+    return buf;
+  }
+
+  private void checkSizeAndGrow(int extra) {
+    if ( (buf.position() + extra) > buf.limit()) {
+      int newSize = (int)Math.min((((long)buf.capacity()) * 2),
+                                  (long)(Integer.MAX_VALUE));
+      newSize = Math.max(newSize, buf.position() + extra);
+      ByteBuffer newBuf = ByteBuffer.allocate(newSize);
+      buf.flip();
+      newBuf.put(buf);
+      buf = newBuf;
+    }
+  }
+
+  // OutputStream
+  @Override
+  public void write(int b) throws IOException {
+    checkSizeAndGrow(Bytes.SIZEOF_BYTE);
+
+    buf.put((byte)b);
+  }
+
+  @Override
+  public void write(byte[] b) throws IOException {
+    checkSizeAndGrow(b.length);
+
+    buf.put(b);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    checkSizeAndGrow(len);
+
+    buf.put(b, off, len);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    // noop
+  }
+
+  @Override
+  public void close() throws IOException {
+    // noop again. heh
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/ipc/thrift/exceptions/ThriftHBaseException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/ipc/thrift/exceptions/ThriftHBaseException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/ipc/thrift/exceptions/ThriftHBaseException.java
new file mode 100644
index 0000000..ce37023
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/ipc/thrift/exceptions/ThriftHBaseException.java
@@ -0,0 +1,151 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.ipc.thrift.exceptions;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Arrays;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import com.google.common.base.Objects;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+
+@ThriftStruct
+public final class ThriftHBaseException extends Exception {
+
+  private static final long serialVersionUID = -1294763299320737511L;
+
+  public static final Log LOG = LogFactory.getLog(ThriftHBaseException.class);
+
+  private String message;
+  private String exceptionClass;
+  private byte[] serializedServerJavaEx;
+
+  /**
+   * Swift Contructor used for serialization
+   *
+   * @param message - the message of the exception
+   * @param exceptionClass - the class of which instance the exception is
+   * @param serializedServerJavaEx - serialized java exception
+   */
+  @ThriftConstructor
+  public ThriftHBaseException(@ThriftField(1) String message,
+      @ThriftField(2) String exceptionClass,
+      @ThriftField(3) byte[] serializedServerJavaEx) {
+    this.message = message;
+    this.exceptionClass = exceptionClass;
+    this.serializedServerJavaEx = serializedServerJavaEx;
+  }
+
+  public ThriftHBaseException(){}
+
+  public ThriftHBaseException(Exception serverJavaException) {
+    try {
+      ByteArrayOutputStream out = new ByteArrayOutputStream();
+      ObjectOutputStream oout = new ObjectOutputStream(out);
+      oout.writeObject(serverJavaException);
+      this.message = serverJavaException.getMessage();
+      this.exceptionClass = serverJavaException.getClass().getCanonicalName();
+      serializedServerJavaEx = out.toByteArray();
+    } catch (IOException e) {
+      // Should never happen in reality
+      LOG.error("Exception happened during serialization of java server exception");
+    }
+  }
+
+  public Exception getServerJavaException() {
+    Exception ex = null;
+    try {
+      ByteArrayInputStream in = new ByteArrayInputStream(serializedServerJavaEx);
+      ObjectInputStream oin = new ObjectInputStream(in);
+      ex = (Exception) oin.readObject();
+    } catch (Exception e) {
+      // Should never happen in reality
+      LOG.error("Exception happened during serialization of java server exception");
+    }
+    return ex;
+  }
+
+  @ThriftField(1)
+  public String getMessage() {
+    return message;
+  }
+
+  @ThriftField(2)
+  public String getExceptionClass() {
+    return exceptionClass;
+  }
+
+  @ThriftField(3)
+  public byte[] getSerializedServerJavaEx() {
+    return serializedServerJavaEx;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(message, exceptionClass, serializedServerJavaEx);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+
+    if (other == this) {
+      return true;
+    }
+
+    if (other.getClass() != this.getClass()) {
+      return false;
+    }
+
+    ThriftHBaseException otherException = (ThriftHBaseException) other;
+    if (!this.getMessage().equals(otherException.getMessage())) {
+      return false;
+    }
+
+    if (!this.getExceptionClass().equals(otherException.getExceptionClass())) {
+      return false;
+    }
+
+    if (!Bytes.equals(this.getSerializedServerJavaEx(),
+                      otherException.getSerializedServerJavaEx())) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return getClass() + " [message=" + message + ", exceptionClass="
+        + exceptionClass + ", serializedServerJavaEx Hash="
+        + Arrays.toString(serializedServerJavaEx).hashCode() + "]";
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/metrics/MetricsBase.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/metrics/MetricsBase.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/metrics/MetricsBase.java
new file mode 100644
index 0000000..c85944c
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/metrics/MetricsBase.java
@@ -0,0 +1,160 @@
+package org.apache.hadoop.hbase.metrics;
+
+import com.google.common.base.Joiner;
+import org.weakref.jmx.JmxException;
+import org.weakref.jmx.MBeanExporter;
+
+import javax.annotation.concurrent.ThreadSafe;
+import java.util.Collections;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * This class provides a skeleton implementation of class which can be
+ * exported as a {@link javax.management.DynamicMBean}, using the
+ * {@link org.weakref.jmx} package.
+ */
+@ThreadSafe
+public abstract class MetricsBase {
+  private final String mbeanName;
+  private MBeanExporter exporter;
+
+  /**
+   * Construct an object which can be exported as MBean with the given name and
+   * exporter. The given name will be used to construct an
+   * {@link javax.management.ObjectName}, which has specific requirements.
+   * The caller is responsible to pass a valid name.
+   * @param mbeanName name to be used to export the MBean
+   * @param exporter exporter to be used to export this object
+   */
+  public MetricsBase(final String mbeanName, final MBeanExporter exporter) {
+    this.mbeanName = checkNotNull(mbeanName, "Name can not be null");
+    this.exporter = exporter;
+  }
+
+
+  /**
+   * Construct an object which will exported as MBean using a
+   * {@link javax.management.ObjectName} that follows the following pattern:
+   * <domain>:type=<type>,name=<name>,proc=<prodId>,<extendedAttributes> where
+   * the extended attributes are a map of key value strings.
+   *
+   * @param domain the domain this MBean should belong to
+   * @param type the type of the MBean
+   * @param name the name of the MBean
+   * @param procId a identifier making this MBean unique, for example the PID of
+   *               the running JVM
+   * @param extendedAttributes a key value map of strings containing additional
+   *                           attributes to be added
+   * @param exporter the exporter to be used to export this MBean
+   */
+  public MetricsBase(final String domain, final String type, final String name,
+          final String procId, final Map<String, String> extendedAttributes,
+          final MBeanExporter exporter) {
+    this(getMBeanName(domain, type, name, procId, extendedAttributes),
+            exporter);
+  }
+
+  /**
+   * Get the {@link javax.management.ObjectName} as a string of the MBean
+   * backed by this object.
+   */
+  public String getMBeanName() {
+    return mbeanName;
+  }
+
+  /**
+   * Get the {@link MBeanExporter} used to export this object.
+   */
+  public synchronized MBeanExporter getMBeanExporter() {
+    return exporter;
+  }
+
+  /**
+   * Set the given {@link MBeanExporter} as the exporter to be used to
+   * export/un-export this object.
+   * @param exporter exporter to be used to export this object
+   */
+  public synchronized void setMBeanExporter(final MBeanExporter exporter) {
+    this.exporter = exporter;
+  }
+
+  /**
+   * Check if this object is exported as MBean by the set {@link MBeanExporter}.
+   * @return true if this object is exported as MBean
+   */
+  public boolean isExported() {
+    MBeanExporter exporter = getMBeanExporter();
+    Map<String, Object> exportedObjects = Collections.emptyMap();
+    if (exporter != null) {
+      exportedObjects = exporter.getExportedObjects();
+    }
+    return exportedObjects.containsKey(mbeanName);
+  }
+
+  /**
+   * Export this object as MBean.
+   * @throws JmxException if the object could not be exported
+   */
+  public void export() throws JmxException {
+    MBeanExporter exporter = getMBeanExporter();
+    if (exporter != null) {
+      exporter.export(mbeanName, this);
+    }
+  }
+
+  /**
+   * Convenience method which will set the given {@link MBeanExporter} and
+   * export this object as MBean.
+   * @param exporter MBeanExporter to use when exporting the object
+   * @throws JmxException if the object could not be exported
+   */
+  public synchronized void export(final MBeanExporter exporter)
+          throws JmxException {
+    setMBeanExporter(checkNotNull(exporter, "MBeanExporter can not be null"));
+    export();
+  }
+
+  /**
+   * Un-export the MBean backed by this object.
+   * @throws JmxException if the MBean could not be un-exported
+   */
+  public void unexport() throws JmxException {
+    MBeanExporter exporter = getMBeanExporter();
+    if (exporter != null) {
+      exporter.unexport(mbeanName);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return mbeanName;
+  }
+
+  /**
+   * Get an MBean name that follows the following pattern:
+   * <domain>:type=<type>,name=<name>,proc=<prodId>,<extendedAttributes> where
+   * the extended attributes are a map of key value strings.
+
+   * @param domain the domain this MBean should belong to
+   * @param type the type of the MBean
+   * @param name the name of the MBean
+   * @param procId a identifier making this MBean unique, such as the PID of
+   *               the running JVM
+   * @param extendedAttributes a key value map of strings containing additional
+   *                           attributes to be added
+   * @return the MBean name as string
+   */
+  public static String getMBeanName(final String domain, final String type,
+          final String name, final String procId,
+          final Map<String, String> extendedAttributes) {
+    if (!extendedAttributes.isEmpty()) {
+      return String.format("%s:type=%s,name=%s,proc=%s,%s", domain, type, name,
+              procId, Joiner.on(",").withKeyValueSeparator("=").join(
+                      extendedAttributes));
+    }
+    return String.format("%s:type=%s,name=%s,proc=%s", domain, type, name,
+            procId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/metrics/TimeStat.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/metrics/TimeStat.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/metrics/TimeStat.java
new file mode 100644
index 0000000..6f26099
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/metrics/TimeStat.java
@@ -0,0 +1,88 @@
+package org.apache.hadoop.hbase.metrics;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import com.google.common.base.Ticker;
+import io.airlift.stats.ExponentialDecay;
+import io.airlift.stats.TimeDistribution;
+
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+public class TimeStat extends TimeDistribution {
+  private final Ticker ticker;
+
+  public TimeStat(TimeUnit unit) {
+    this(ExponentialDecay.oneMinute(), unit);
+  }
+
+  public TimeStat(double alpha, TimeUnit unit) {
+    this(alpha, unit, Ticker.systemTicker());
+  }
+
+  public TimeStat(double alpha, TimeUnit unit, Ticker ticker) {
+    super(alpha, unit);
+    this.ticker = ticker;
+  }
+
+  public synchronized void add(long value, TimeUnit unit) {
+    add(unit.toNanos(value));
+  }
+
+  public BlockTimer time() {
+    return new BlockTimer();
+  }
+
+  public class BlockTimer implements AutoCloseable {
+    private final long start = ticker.read();
+
+    @Override
+    public void close() {
+      add(ticker.read() - start);
+    }
+  }
+
+  public static void main(String [] args) {
+    final TimeStat stat = new TimeStat(TimeUnit.MICROSECONDS);
+    for (int i = 0; i < 100; ++i) {
+      benchmark(stat);
+    }
+  }
+
+  private static void benchmark(final TimeStat stat) {
+    long n = 1000000;
+    long[] randomValues = new long[1000];
+    Stopwatch stopwatch = new Stopwatch();
+    //Stopwatch.createUnstarted();
+
+    long elapsedTotal = 0;
+    long cycleMax = 0;
+    for (int i = 0; i < n / randomValues.length; ++i) {
+      generateValues(randomValues, 3000000L, 4000000000L);
+      stopwatch.start();
+      for (int j = 0; j < randomValues.length; ++j) {
+        stat.add(randomValues[j]);
+      }
+
+      long elapsed = stopwatch.elapsedTime(TimeUnit.NANOSECONDS);
+      elapsedTotal += elapsed;
+      if (elapsed > cycleMax) {
+        cycleMax = elapsed;
+      }
+
+      stopwatch.reset();
+    }
+    System.out.printf("Elapsed: %dns, max cycle: %dns\n", elapsedTotal,
+            cycleMax);
+  }
+
+  private static void generateValues(final long[]a, long start, long end) {
+    Preconditions.checkArgument(start < end, "Start should be less than end");
+    long delta = end - start;
+    Random random = new Random();
+
+    for (int i = 0; i < a.length; ++i) {
+      a[i] = (random.nextLong() % delta) + start;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/DataStoreState.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/DataStoreState.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/DataStoreState.java
new file mode 100644
index 0000000..f7100e7
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/DataStoreState.java
@@ -0,0 +1,101 @@
+package org.apache.hadoop.hbase.regionserver;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftEnum;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+
+@ThriftStruct
+public final class DataStoreState {
+
+  @ThriftEnum
+  public static enum RegionMode {
+    UNASSIGNED,
+    ACTIVE,  // does memstore updates, and flushing/compaction.
+    WITNESS, // does memstore updates. No flushing/compaction.
+    // Prunes the memstore whenever the active replica flushes.
+    SHADOW_WITNESS; // does not update memstore. No flushing/compaction.
+
+    boolean isWitness() {
+      return this.equals(WITNESS);
+    }
+
+    boolean isActive() {
+      return this.equals(ACTIVE);
+    }
+
+    boolean isShadowWitness() {
+      return this.equals(SHADOW_WITNESS);
+    }
+
+    boolean isUnassigned() {
+      return this.equals(UNASSIGNED);
+    }
+  };
+
+  private final String dataStoreId;
+  private volatile long committedUpto;
+  private volatile long canCommitUpto;
+  private volatile RegionMode mode;
+
+  public DataStoreState(final String dataStoreId) {
+    this.dataStoreId = dataStoreId;
+    committedUpto = -1;
+    canCommitUpto = -1;
+    mode = RegionMode.UNASSIGNED;
+  }
+
+  @ThriftConstructor
+  public DataStoreState(@ThriftField(1) String dataStoreId,
+                        @ThriftField(2) long committedUpto,
+                        @ThriftField(3) long canCommitUpto,
+                        @ThriftField(4) RegionMode mode) {
+    this.dataStoreId = dataStoreId;
+    this.committedUpto = committedUpto;
+    this.canCommitUpto = canCommitUpto;
+    this.mode = mode;
+  }
+
+  @ThriftField(1)
+  public String getDataStoreId() {
+    return dataStoreId;
+  }
+
+  @ThriftField(2)
+  public long getCommittedUpto() {
+    return committedUpto;
+  }
+
+  @ThriftField(3)
+  public long getCanCommitUpto() {
+    return canCommitUpto;
+  }
+
+  @ThriftField(4)
+  public RegionMode getMode() {
+    return mode;
+  }
+
+  public void setCommittedUpto(long committedUpto) {
+    this.committedUpto = committedUpto;
+    this.canCommitUpto = committedUpto;
+  }
+
+  public void setMode(RegionMode mode) {
+    this.mode = mode;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+
+    builder.append("{");
+    builder.append("id=" + dataStoreId + ", ");
+    builder.append("mode=" + mode + ", ");
+    builder.append("canCommitUpto=" + canCommitUpto + ", ");
+    builder.append("committedUptoIndex=" + committedUpto);
+    builder.append("}");
+
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/RaftEventListener.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/RaftEventListener.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/RaftEventListener.java
new file mode 100644
index 0000000..04f4a77
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/RaftEventListener.java
@@ -0,0 +1,17 @@
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.consensus.protocol.Payload;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public interface RaftEventListener {
+  ByteBuffer becameLeader() throws IOException;
+  void becameNonLeader();
+  void commit(final long index, final Payload payload);
+  boolean canStepDown();
+  long getMinUnpersistedIndex();
+  DataStoreState getState();
+  void updatePeerAvailabilityStatus(String peerAddress, boolean isAvailable);
+  void closeDataStore();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java
new file mode 100644
index 0000000..f5cec76
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/RegionOverloadedException.java
@@ -0,0 +1,47 @@
+package org.apache.hadoop.hbase.regionserver;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.RegionException;
+
+public class RegionOverloadedException extends RegionException {
+  private static final long serialVersionUID = -8436877560512061623L;
+
+  /** default constructor */
+  public RegionOverloadedException() {
+    super();
+  }
+
+  /** @param s message
+   *  @param waitMillis -- request client to backoff for waitMillis
+   */
+  public RegionOverloadedException(String s, long waitMillis) {
+    super(s, waitMillis);
+  }
+
+  /**
+   * Create a RegionOverloadedException from another one, attaching a set of related exceptions
+   * from a batch operation. The new exception reuses the original exception's stack trace.
+   *  
+   * @param roe the original exception
+   * @param exceptions other exceptions that happened in the same batch operation
+   * @param waitMillis remaining time for the client to wait in milliseconds
+   * @return the new exception with complete information
+   */
+  public static RegionOverloadedException create(RegionOverloadedException roe,
+      List<Throwable> exceptions, int waitMillis) {
+    StringBuilder sb = new StringBuilder(roe.getMessage());
+    for (Throwable t : exceptions) {
+      if (t != roe) {
+        sb.append(t.toString());
+        sb.append("\n");
+      }
+    }
+    RegionOverloadedException e = new RegionOverloadedException(sb.toString(), waitMillis);
+    if (roe != null) {  // Safety check
+      e.setStackTrace(roe.getStackTrace());
+    }
+    return e;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/PercentileMetric.java/PercentileMetric.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/PercentileMetric.java/PercentileMetric.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/PercentileMetric.java/PercentileMetric.java
new file mode 100644
index 0000000..0ad6031
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/PercentileMetric.java/PercentileMetric.java
@@ -0,0 +1,115 @@
+/**
+ * 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.hadoop.hbase.regionserver.metrics;
+import org.apache.hadoop.hbase.util.Histogram;
+import org.apache.hadoop.metrics.util.MetricsLongValue;
+import org.apache.hadoop.metrics.util.MetricsRegistry;
+import org.apache.hadoop.metrics.MetricsRecord;
+
+/**
+ * Used the org.apache.hadoop.hbase.util.Histogram to maintain time varying
+ * metrics. The histogram class can provide various approximate details about
+ * a stream of data supplied to the PercentileMetric without actually
+ * storing the data.
+ */
+public class PercentileMetric extends MetricsLongValue {
+  public static final int HISTOGRAM_NUM_BUCKETS_DEFAULT = 20;
+  public static final double HISTOGRAM_MINVALUE_DEFAULT = 0.0;
+  public static final double HISTOGRAM_MAXVALUE_DEFAULT = 1000000000.0;
+  public static final double DEFAULT_PERCENTILE = 99.0;
+  public static final long DEFAULT_SAMPLE_WINDOW = 60;
+  public static final double P99 = 99.0;
+  public static final double P95 = 95.0;
+  public static final double P75 = 75.0;
+  public static final double P50 = 50.0;
+
+  private int numBuckets;
+  private double percentile;
+  private Histogram underlyingHistogram;
+
+  /**
+   * This constructor provides a way to create a HistogramMetric which uses a
+   * Histogram to maintain the statistics of a metric stream.
+   */
+  public PercentileMetric(final String nam, final MetricsRegistry registry,
+    Histogram histogram) {
+    super(nam, registry);
+    underlyingHistogram = histogram;
+  }
+
+  public PercentileMetric(String nam, MetricsRegistry registry,
+      Histogram histogram, double percentile, int numBuckets) {
+    super(nam, registry);
+    this.underlyingHistogram = histogram;
+    this.percentile = percentile;
+    this.numBuckets = numBuckets;
+  }
+
+  /**
+   * The histogram which has the values updated.
+   */
+  public void setHistogram(final Histogram hist) {
+    this.underlyingHistogram = hist;
+  }
+
+  /**
+   * numBuckets : This denotes the number of buckets used to sample the data.
+   * the updateMetric and refresh calls will run in O(numBuckets).
+   */
+  public void setNumBuckets(final int numBuckets) {
+    this.numBuckets = numBuckets;
+  }
+
+  /**
+   * percentile : The percentile estimate of the metric that will be seeked
+   * using this metric. The value should be between 0 and 100,
+   * else it will throw and exception.
+   */
+  public void setPercentile(final double prcntyl) {
+    this.percentile = prcntyl;
+  }
+
+  public double getValue() {
+    return this.get();
+  }
+
+  public void updateMetric() {
+    this.set((long)underlyingHistogram.getPercentileEstimate(percentile));
+  }
+
+  public void refresh() {
+    underlyingHistogram.refresh(this.numBuckets);
+  }
+
+  /**
+   * Add a value in the underlying histogram.
+   * @param value The value to be added.
+   */
+  public void addValueInHistogram(long value) {
+    underlyingHistogram.addValue(value);
+  }
+
+  /**
+   * Push the metric value to the <code>MetricsRecord</code> object
+   * @param mr
+   */
+  public void pushMetric(final MetricsRecord mr) {
+    this.updateMetric();
+    mr.setMetric(getName(), (long)getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java
new file mode 100644
index 0000000..2ae84d1
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractWAL.java
@@ -0,0 +1,69 @@
+package org.apache.hadoop.hbase.regionserver.wal;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.regionserver.metrics.PercentileMetric;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Histogram;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+public abstract class AbstractWAL {
+  public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
+  public static final byte [] METAROW = Bytes.toBytes("METAROW");
+
+  // For measuring size of each transaction
+  protected static Histogram writeSize = new Histogram(
+    PercentileMetric.HISTOGRAM_NUM_BUCKETS_DEFAULT,
+    PercentileMetric.HISTOGRAM_MINVALUE_DEFAULT,
+    PercentileMetric.HISTOGRAM_MAXVALUE_DEFAULT);
+
+  // For measure the sync time for each HLog.append operation;
+  protected static Histogram syncTime = new Histogram(
+    PercentileMetric.HISTOGRAM_NUM_BUCKETS_DEFAULT,
+    PercentileMetric.HISTOGRAM_MINVALUE_DEFAULT,
+    PercentileMetric.HISTOGRAM_MAXVALUE_DEFAULT);
+
+  // For measuring the internal group commit time
+  protected static Histogram gsyncTime = new Histogram(
+    PercentileMetric.HISTOGRAM_NUM_BUCKETS_DEFAULT,
+    PercentileMetric.HISTOGRAM_MINVALUE_DEFAULT,
+    PercentileMetric.HISTOGRAM_MAXVALUE_DEFAULT);
+
+  public abstract long append(HRegionInfo info, byte [] tableName, WALEdit edits,
+                     final long now)
+    throws IOException, ExecutionException, InterruptedException;
+
+  public abstract long startMemStoreFlush(final byte[] regionName);
+  public abstract void completeMemStoreFlush(final byte[] regionName, final byte[] tableName,
+                                    final long logSeqId, final boolean isMetaRegion);
+  public abstract void abortMemStoreFlush(byte[] regionName);
+  public abstract long startMemStoreFlush(final byte[] regionName,
+                                 long firstSeqIdInStoresToFlush,
+                                 long firstSeqIdInStoresNotToFlush);
+
+  public abstract long obtainNextSequenceNumber()
+    throws IOException, ExecutionException, InterruptedException;
+  public abstract long getSequenceNumber();
+  public abstract void initSequenceNumber(long seqid)
+    throws IOException, ExecutionException, InterruptedException;
+
+  public abstract void close() throws IOException;
+  public abstract void closeAndDelete() throws IOException;
+  public abstract String getPath();
+
+  public static Histogram getWriteSizeHistogram() {
+    return writeSize;
+  }
+
+  public static Histogram getSyncTimeHistogram() {
+    return syncTime;
+  }
+
+  public static Histogram getGSyncTimeHistogram() {
+    return gsyncTime;
+  }
+
+  public abstract long getLastCommittedIndex();
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
new file mode 100644
index 0000000..b1222bf
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -0,0 +1,521 @@
+/**
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.regionserver.wal;
+
+import com.facebook.swift.codec.ThriftConstructor;
+import com.facebook.swift.codec.ThriftField;
+import com.facebook.swift.codec.ThriftStruct;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.ipc.ByteBufferOutputStream;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+/**
+ * WALEdit: Used in HBase's transaction log (WAL) to represent
+ * the collection of edits (KeyValue objects) corresponding to a
+ * single transaction. The class implements "Writable" interface
+ * for serializing/deserializing a set of KeyValue items.
+ *
+ * Previously, if a transaction contains 3 edits to c1, c2, c3 for a row R,
+ * the HLog would have three log entries as follows:
+ *
+ *    <logseq1-for-edit1>:<KeyValue-for-edit-c1>
+ *    <logseq2-for-edit2>:<KeyValue-for-edit-c2>
+ *    <logseq3-for-edit3>:<KeyValue-for-edit-c3>
+ *
+ * This presents problems because row level atomicity of transactions
+ * was not guaranteed. If we crash after few of the above appends make
+ * it, then recovery will restore a partial transaction.
+ *
+ * In the new world, all the edits for a given transaction are written
+ * out as a single record, for example:
+ *
+ *   <logseq#-for-entire-txn>:<WALEdit-for-entire-txn>
+ *
+ * where, the WALEdit is serialized as:
+ *   <-1, # of edits, <KeyValue>, <KeyValue>, ... >
+ * For example:
+ *   <-1, 3, <Keyvalue-for-edit-c1>, <KeyValue-for-edit-c2>, <KeyValue-for-edit-c3>>
+ *
+ * The -1 marker is just a special way of being backward compatible with
+ * an old HLog which would have contained a single <KeyValue>.
+ *
+ * The deserializer for WALEdit backward compatibly detects if the record
+ * is an old style KeyValue or the new style WALEdit.
+ *
+ */
+
+@ThriftStruct
+public final class WALEdit implements Writable, HeapSize {
+  public enum PayloadHeaderField {
+    MAGIC(0, Bytes.SIZEOF_BYTE),
+    TYPE(1, Bytes.SIZEOF_BYTE),
+    VERSION(2, Bytes.SIZEOF_BYTE),
+    TIMESTAMP(3, Bytes.SIZEOF_LONG),
+    COMPRESSION_CODEC(11, Bytes.SIZEOF_BYTE),
+    UNCOMPRESSED_LENGTH(12, Bytes.SIZEOF_INT),
+    NUM_WALEDITS(16, Bytes.SIZEOF_INT);
+
+    final int offset;
+    final int length;
+
+    private PayloadHeaderField(final int offset, final int length) {
+      this.offset = offset;
+      this.length = length;
+    }
+  }
+
+  public final static int PAYLOAD_HEADER_SIZE =
+          PayloadHeaderField.NUM_WALEDITS.offset +
+                  PayloadHeaderField.NUM_WALEDITS.length;
+
+  private final int VERSION_2 = -1;
+
+  private final List<KeyValue> kvs;
+
+  private NavigableMap<byte[], Integer> scopes;
+  
+  private long length = 0;
+
+  private SettableFuture<Long> commitFuture;
+
+  @ThriftConstructor
+  public WALEdit(
+      @ThriftField(1) final List<KeyValue> kvs) {
+    this.kvs = kvs;
+    for (KeyValue k : kvs) {
+      length += k.getLength();
+    }
+  }
+
+  public WALEdit() {
+    kvs = new ArrayList<KeyValue>();
+  }
+
+  public SettableFuture<Long> getCommitFuture() {
+    return commitFuture;
+  }
+
+  public void add(KeyValue kv) {
+    this.kvs.add(kv);
+    length += kv.getLength();
+  }
+
+  public boolean isEmpty() {
+    return kvs.isEmpty();
+  }
+  
+  public long getTotalKeyValueLength() {
+    return length;
+  }
+
+  public int size() {
+    return kvs.size();
+  }
+
+  @ThriftField(1)
+  public List<KeyValue> getKeyValues() {
+    return kvs;
+  }
+
+  public NavigableMap<byte[], Integer> getScopes() {
+    return scopes;
+  }
+
+  public void setScopes (NavigableMap<byte[], Integer> scopes) {
+    // We currently process the map outside of WALEdit,
+    // TODO revisit when replication is part of core
+    this.scopes = scopes;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    kvs.clear();
+    if (scopes != null) {
+      scopes.clear();
+    }
+    int versionOrLength = in.readInt();
+    if (versionOrLength == VERSION_2) {
+      // this is new style HLog entry containing multiple KeyValues.
+      int numEdits = in.readInt();
+      for (int idx = 0; idx < numEdits; idx++) {
+        KeyValue kv = new KeyValue();
+        kv.readFields(in);
+        this.add(kv);
+      }
+      int numFamilies = in.readInt();
+      if (numFamilies > 0) {
+        if (scopes == null) {
+          scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+        }
+        for (int i = 0; i < numFamilies; i++) {
+          byte[] fam = Bytes.readByteArray(in);
+          int scope = in.readInt();
+          scopes.put(fam, scope);
+        }
+      }
+    } else {
+      // this is an old style HLog entry. The int that we just
+      // read is actually the length of a single KeyValue.
+      KeyValue kv = new KeyValue();
+      kv.readFields(versionOrLength, in);
+      this.add(kv);
+    }
+
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(VERSION_2);
+    out.writeInt(kvs.size());
+    // We interleave the two lists for code simplicity
+    for (KeyValue kv : kvs) {
+      kv.write(out);
+    }
+    if (scopes == null) {
+      out.writeInt(0);
+    } else {
+      out.writeInt(scopes.size());
+      for (byte[] key : scopes.keySet()) {
+        Bytes.writeByteArray(out, key);
+        out.writeInt(scopes.get(key));
+      }
+    }
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+
+    sb.append("[#edits: " + kvs.size() + " = <");
+    for (KeyValue kv : kvs) {
+      sb.append(kv.toString());
+      sb.append("; ");
+    }
+    if (scopes != null) {
+      sb.append(" scopes: " + scopes.toString());
+    }
+    sb.append(">]");
+    return sb.toString();
+  }
+
+  @Override
+  public long heapSize() {
+    long ret = 0;
+    for (KeyValue kv : kvs) {
+      ret += kv.heapSize();
+    }
+    if (scopes != null) {
+      ret += ClassSize.TREEMAP;
+      ret += ClassSize.align(scopes.size() * ClassSize.MAP_ENTRY);
+    }
+    return ret;
+  }
+
+  /**
+   * Serialize the given list of WALEdits to an OutputStream.
+   * @param edits the WALEdits to be serialized
+   * @param os the {@link DataOutputStream} to write to
+   * @throws IOException if the output could not be written to the stream
+   */
+  private static void serializeWALEdits(final List<WALEdit> edits,
+          final DataOutputStream os) throws IOException {
+    for (final WALEdit e : edits) {
+      os.writeInt(e.getKeyValues().size());
+      for (final KeyValue k : e.getKeyValues()) {
+        os.writeInt(k.getLength());
+        os.write(k.getBuffer(), k.getOffset(), k.getLength());
+      }
+    }
+  }
+
+  /**
+   * Serialize the given list of WALEdits edits to a {@link ByteBuffer},
+   * optionally compressing the WALEdit data using the given compression codec.
+   *
+   * @param edits the list of WALEdits
+   * @return a {@link ByteBuffer} containing a serialized representation of the
+   *          WALEdits.
+   * @throws java.io.IOException if the WALEdits could not be serialized
+   */
+  public static ByteBuffer serializeToByteBuffer(final List<WALEdit> edits,
+          long timestamp, Compression.Algorithm codec) throws IOException {
+    Preconditions.checkNotNull(codec);
+    if (edits == null) {
+      return null;
+    }
+
+    int totalPayloadSize = getTotalPayloadSize(edits);
+    ByteBufferOutputStream buffer = new ByteBufferOutputStream(
+            totalPayloadSize);
+    try (DataOutputStream os = new DataOutputStream(buffer)) {
+      // Write the magic value
+      os.write(HConstants.CONSENSUS_PAYLOAD_MAGIC_VALUE);
+
+      // Write that the payload is WALEdit
+      os.write(HConstants.BATCHED_WALEDIT_TYPE);
+
+      // Write the version of WALEdit
+      os.write(HConstants.BATCHED_WALEDIT_VERSION);
+
+      // Write the timestamp
+      os.writeLong(timestamp);
+
+      // Write compression algorithm
+      os.write((byte) codec.ordinal());
+
+      // Write uncompressed size of the list of WALEdits
+      os.writeInt(totalPayloadSize - PAYLOAD_HEADER_SIZE);
+
+      // Write the number of WALEdits in the list
+      os.writeInt(edits.size());
+    }
+
+    // Turn on compression if requested when serializing the list of WALEdits.
+    boolean compressed = !codec.equals(Compression.Algorithm.NONE);
+    Compressor compressor = codec.getCompressor();
+    try (DataOutputStream os = new DataOutputStream(compressed ?
+                    codec.createCompressionStream(buffer, compressor,
+                            totalPayloadSize - PAYLOAD_HEADER_SIZE) : buffer)) {
+      serializeWALEdits(edits, os);
+    } finally {
+      codec.returnCompressor(compressor);
+    }
+
+    // Flip and return the byte buffer.
+    return buffer.getByteBuffer();
+  }
+
+  public static int getWALEditsSize(final List<WALEdit> edits) {
+    int size = 0;
+    for (final WALEdit e : edits) {
+      size += Bytes.SIZEOF_INT + e.getKeyValues().size() * Bytes.SIZEOF_INT
+              + e.getTotalKeyValueLength();
+    }
+    return size;
+  }
+
+  public static int getTotalPayloadSize(final List<WALEdit> edits) {
+    return PAYLOAD_HEADER_SIZE + getWALEditsSize(edits);
+  }
+
+  public static boolean isBatchedWALEdit(final ByteBuffer data) {
+    // Read the Magic Value
+    if (data.get(data.position() + PayloadHeaderField.MAGIC.offset) !=
+            HConstants.CONSENSUS_PAYLOAD_MAGIC_VALUE) {
+      return false;
+    }
+
+    // Read the payload type
+    if (data.get(data.position() + PayloadHeaderField.TYPE.offset) !=
+            HConstants.BATCHED_WALEDIT_TYPE) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Get the timestamp of the batched WALEdit. This method assumes the given
+   * ByteBuffer contains a valid batched WALEdits which can be verified using
+   * {@link #isBatchedWALEdit}.
+   */
+  public static long getTimestamp(final ByteBuffer data) {
+    return data.getLong(data.position() + PayloadHeaderField.TIMESTAMP.offset);
+  }
+
+  /**
+   * Get the compression codec used to compress the serialized WALEdits
+   * contained in the given {@link ByteBuffer}. This method assumes the position
+   * of the buffer to point to the does not change the
+   * position value of the buffer and assumes the caller has performed a version
+   * check on the buffer to ensure the
+   *
+   * @param data a {@link java.nio.ByteBuffer} containing a serialized list of
+   *             WALEdits
+   * @return the compression codec or the NONE codec if the WALEdit was written
+   *              with a version which does not support compression
+   */
+  public static Compression.Algorithm getCompressionCodec(
+          final ByteBuffer data) {
+    byte codecValue = data.get(data.position() +
+            PayloadHeaderField.COMPRESSION_CODEC.offset);
+    Compression.Algorithm[] codecs = Compression.Algorithm.values();
+    if (codecValue >= 0 && codecValue < codecs.length) {
+      return codecs[codecValue];
+    }
+    return Compression.Algorithm.NONE;
+  }
+
+  /**
+   * Wrap the array backing the given ByteBuffer with a ByteArrayInputStream.
+   * Since this InputStream works on the underlying array the state of the given
+   * ByteBuffer is guaranteed to remain unchanged.
+   *
+   * @param buffer an array backed {@link ByteBuffer}
+   * @param position the position in the buffer from where to start the stream
+   * @param length length of the input stream
+   * @return an {@link java.io.InputStream} wrapping the underlying array of
+   *          the given {@link ByteBuffer}
+   */
+  private static ByteArrayInputStream getByteArrayInputStream(
+          final ByteBuffer buffer, final int position, final int length) {
+    Preconditions.checkArgument(buffer.hasArray(),
+            "An array backed buffer is required");
+    Preconditions.checkArgument(position >= buffer.position(),
+            "Position can not be behind buffer.position()");
+    Preconditions.checkArgument(
+            position - buffer.position() + length <= buffer.remaining(),
+            "Length can not be past the remainder of the buffer");
+    return new ByteArrayInputStream(buffer.array(),
+            buffer.arrayOffset() + position, length);
+  }
+
+  /**
+   * Read a list of serialized WALEdits from the given
+   * {@link DataInputStream}, instantiating them backed by the given
+   * {@link ByteBuffer}.
+   *
+   * @param numEdits the number of WALEdits expected in the stream
+   * @param is the {@link InputStream} containing serialized WALEdits
+   * @param buffer the {@link ByteBuffer} to be used to back the KVs
+   * @param offset the offset in the buffer from where to copy the KVs
+   * @param copyToBuffer copy from the stream to the buffer if true, assume the
+   *                     steam data is already in the buffer otherwise
+   * @return a list of WALEdits
+   * @throws IOException if an exception occurs while reading from the stream
+   */
+  private static List<WALEdit> deserializeWALEdits(final int numEdits,
+          final DataInputStream is, final ByteBuffer buffer, final int offset,
+          final boolean copyToBuffer) throws IOException {
+    List<WALEdit> edits = new ArrayList<>(numEdits);
+    byte[] array = buffer.array();
+    int cursor = buffer.arrayOffset() + offset;
+
+    for (int editIdx = 0; editIdx < numEdits; ++editIdx) {
+      WALEdit edit = new WALEdit();
+      int numKVs = is.readInt();
+      cursor += Bytes.SIZEOF_INT;
+
+      for (int kvIdx = 0; kvIdx < numKVs; ++kvIdx) {
+        int kvLen = is.readInt();
+        cursor += Bytes.SIZEOF_INT;
+
+        if (copyToBuffer) {
+          // If the buffer does not contain the data yet (which would be the
+          // case if it is compressed), copy from the InputStream to the buffer.
+          is.read(array, cursor, kvLen);
+        } else {
+          // Do not copy to the buffer and advance the stream cursor.
+          is.skipBytes(kvLen);
+        }
+
+        // Instantiate the KV backed by the ByteBuffer
+        edit.add(new KeyValue(array, cursor, kvLen));
+        // Move the ByteBuffer write cursor
+        cursor += kvLen;
+      }
+
+      edits.add(edit);
+    }
+
+    return edits;
+  }
+
+  /**
+   * De-serializes a ByteBuffer to list of WALEdits. If the serialized WALEdits
+   * are not compressed, the resulting list of KVs will be backed by the array
+   * backing the ByteBuffer instead of allocating fresh buffers. As a
+   * consequence of this method assumes the state of the ByteBuffer is never
+   * modified.
+   *
+   * @param data a {@link ByteBuffer} containing serialized WALEdits
+   * @return a list of WALEdits
+   * @throws java.io.IOException if the WALEdits could not be deserialized
+   */
+  public static List<WALEdit> deserializeFromByteBuffer(final ByteBuffer data)
+          throws IOException {
+    if (!isBatchedWALEdit(data)) {
+      return null;
+    }
+
+    int firstBytePosition = data.position();
+    int bufferLength = data.remaining();
+
+    // The check above already read the magic value and type fields, so move on
+    // to the version field.
+    byte version = data.get(firstBytePosition +
+            PayloadHeaderField.VERSION.offset);
+    if (version != HConstants.BATCHED_WALEDIT_VERSION) {
+      return null;
+    }
+
+    // Get the compression codec and uncompressed size of the list of WALEdits.
+    // Use the remainder of the current buffer as a hint.
+    Compression.Algorithm codec = getCompressionCodec(data);
+
+    int uncompressedEditsLen = data.getInt(firstBytePosition +
+            PayloadHeaderField.UNCOMPRESSED_LENGTH.offset);
+    int numEdits = data.getInt(firstBytePosition +
+            PayloadHeaderField.NUM_WALEDITS.offset);
+
+    if (numEdits == 0) {
+      return Collections.emptyList();
+    }
+
+    // Wrap the remainder of the given ByteBuffer with a DataInputStream and
+    // de-serialize the list of WALEdits.
+    //
+    // If the WALEdits are compressed, wrap the InputStream by a decompression
+    // stream and allocate a new buffer to store the uncompressed data.
+    int cursor = firstBytePosition + PAYLOAD_HEADER_SIZE;
+    InputStream is = getByteArrayInputStream(data, cursor,
+            bufferLength - PAYLOAD_HEADER_SIZE);
+    ByteBuffer deserializedData = data;
+
+    final boolean compressed = !codec.equals(Compression.Algorithm.NONE);
+    Decompressor decompressor = codec.getDecompressor();
+    try {
+      if (compressed) {
+        int compressedEditsLen = bufferLength - PAYLOAD_HEADER_SIZE;
+        is = codec.createDecompressionStream(is, decompressor,
+                compressedEditsLen);
+        // Allocate a new ByteBuffer for the uncompressed data.
+        deserializedData = ByteBuffer.allocate(uncompressedEditsLen);
+        cursor = 0;
+      }
+
+      try (DataInputStream dis = new DataInputStream(is)) {
+        return deserializeWALEdits(numEdits, dis, deserializedData, cursor,
+                compressed);
+      }
+    } finally {
+      codec.returnDecompressor(decompressor);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
new file mode 100644
index 0000000..9633d01
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
@@ -0,0 +1,400 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.1)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An IllegalArgument exception indicates an illegal or invalid
+ * argument was passed into a procedure.
+ */
+public class IllegalArgument extends TException implements org.apache.thrift.TBase<IllegalArgument, IllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<IllegalArgument> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument");
+
+  private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new IllegalArgumentStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new IllegalArgumentTupleSchemeFactory());
+  }
+
+  public String message; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MESSAGE((short)1, "message");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MESSAGE
+          return MESSAGE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("message", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(IllegalArgument.class, metaDataMap);
+  }
+
+  public IllegalArgument() {
+  }
+
+  public IllegalArgument(
+    String message)
+  {
+    this();
+    this.message = message;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public IllegalArgument(IllegalArgument other) {
+    if (other.isSetMessage()) {
+      this.message = other.message;
+    }
+  }
+
+  public IllegalArgument deepCopy() {
+    return new IllegalArgument(this);
+  }
+
+  @Override
+  public void clear() {
+    this.message = null;
+  }
+
+  public String getMessage() {
+    return this.message;
+  }
+
+  public IllegalArgument setMessage(String message) {
+    this.message = message;
+    return this;
+  }
+
+  public void unsetMessage() {
+    this.message = null;
+  }
+
+  /** Returns true if field message is set (has been assigned a value) and false otherwise */
+  public boolean isSetMessage() {
+    return this.message != null;
+  }
+
+  public void setMessageIsSet(boolean value) {
+    if (!value) {
+      this.message = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MESSAGE:
+      if (value == null) {
+        unsetMessage();
+      } else {
+        setMessage((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MESSAGE:
+      return getMessage();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MESSAGE:
+      return isSetMessage();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof IllegalArgument)
+      return this.equals((IllegalArgument)that);
+    return false;
+  }
+
+  public boolean equals(IllegalArgument that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_message = true && this.isSetMessage();
+    boolean that_present_message = true && that.isSetMessage();
+    if (this_present_message || that_present_message) {
+      if (!(this_present_message && that_present_message))
+        return false;
+      if (!this.message.equals(that.message))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_message = true && (isSetMessage());
+    builder.append(present_message);
+    if (present_message)
+      builder.append(message);
+
+    return builder.toHashCode();
+  }
+
+  @Override
+  public int compareTo(IllegalArgument other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetMessage()).compareTo(other.isSetMessage());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMessage()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message, other.message);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("IllegalArgument(");
+    boolean first = true;
+
+    sb.append("message:");
+    if (this.message == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.message);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class IllegalArgumentStandardSchemeFactory implements SchemeFactory {
+    public IllegalArgumentStandardScheme getScheme() {
+      return new IllegalArgumentStandardScheme();
+    }
+  }
+
+  private static class IllegalArgumentStandardScheme extends StandardScheme<IllegalArgument> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, IllegalArgument struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MESSAGE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.message = iprot.readString();
+              struct.setMessageIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+
+      // check for required fields of primitive type, which can't be checked in the validate method
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, IllegalArgument struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.message != null) {
+        oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
+        oprot.writeString(struct.message);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class IllegalArgumentTupleSchemeFactory implements SchemeFactory {
+    public IllegalArgumentTupleScheme getScheme() {
+      return new IllegalArgumentTupleScheme();
+    }
+  }
+
+  private static class IllegalArgumentTupleScheme extends TupleScheme<IllegalArgument> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, IllegalArgument struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetMessage()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetMessage()) {
+        oprot.writeString(struct.message);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, IllegalArgument struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.message = iprot.readString();
+        struct.setMessageIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Arena.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Arena.java b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Arena.java
new file mode 100644
index 0000000..e27dd1f
--- /dev/null
+++ b/hbase-consensus/src/main/java/org/apache/hadoop/hbase/util/Arena.java
@@ -0,0 +1,12 @@
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.io.hfile.bucket.CacheFullException;
+
+public interface Arena {
+
+  MemoryBuffer allocateByteBuffer(int size) throws
+    CacheFullException, BucketAllocatorException;
+
+  void freeByteBuffer(final MemoryBuffer buffer);
+
+}


[02/19] hbase git commit: HBASE-12476 HydraBase consensus protocol

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLowerRankBecomingLeader.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLowerRankBecomingLeader.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLowerRankBecomingLeader.java
new file mode 100644
index 0000000..a72ebf9
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestLowerRankBecomingLeader.java
@@ -0,0 +1,124 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server
+  .InstrumentedConsensusServiceImpl;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class TestLowerRankBecomingLeader {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    TestLowerRankBecomingLeader.class);
+  private static int QUORUM_SIZE = 3;
+  private static int QUORUM_MAJORITY = 2;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private static QuorumClient client;
+  private static volatile int transactionNum = 0;
+  private ReplicationLoadForUnitTest loader;
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.disableVerboseLogging();
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, RAFT_TEST_UTIL.getScratchSetup(QUORUM_SIZE));
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+    transactionNum = 0;
+    loader = new ReplicationLoadForUnitTest(regionInfo, client, RAFT_TEST_UTIL, QUORUM_SIZE,
+      QUORUM_MAJORITY);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Test(timeout=500000)
+  public void testSinglePeerFailureAndRecovery()
+    throws InterruptedException, IOException {
+    simulateFailureEvent(1);
+  }
+
+  private void simulateFailureEvent(final int failureInterval)
+    throws InterruptedException, IOException {
+    int failureCnt = 0;
+    final long sleepTime =
+      2 * HConstants.PROGRESS_TIMEOUT_INTERVAL_IN_MILLISECONDS * failureInterval;
+
+    // Start the client load
+    loader.startReplicationLoad(100);
+
+    // Let the traffic fly for a while
+    transactionNum = loader.makeProgress(sleepTime, transactionNum);
+
+    RAFT_TEST_UTIL.simulatePacketDropForServer(regionInfo, 2,
+        InstrumentedConsensusServiceImpl.PacketDropStyle.ALWAYS);
+
+    transactionNum = loader.makeProgress(sleepTime, transactionNum);
+
+    RAFT_TEST_UTIL.simulatePacketDropForServer(regionInfo, 2,
+        InstrumentedConsensusServiceImpl.PacketDropStyle.NONE);
+
+    // At this point, rank 1 and rank 3 peers are up to date
+
+    // Get all the quorum contexts from rank 3 to rank 2
+    RaftQuorumContext c3 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 3);
+    RaftQuorumContext c2 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 2);
+
+    // Shutdown 1st quorum member whose rank is 3
+    System.out.println("Stopping one quorum member: " + c3);
+    LocalConsensusServer s3 = RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, 3);
+
+    // Let the traffic fly for a while
+    if ((++failureCnt % failureInterval) == 0) {
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+      while(!c2.isLeader()) {
+        System.out.println("Wait for the rank 2 to take over the leadership");
+        Thread.sleep(sleepTime);
+      }
+      Assert.assertTrue("Rank 2 shall be the leader of the quorum", c2.isLeader());
+    }
+
+    // Restart the quorum member whose rank is 3
+    c3 = RAFT_TEST_UTIL.restartLocalConsensusServer(s3, regionInfo, c3.getMyAddress());
+    System.out.println("Restarted another quorum member: " + c3);
+
+    // Let the traffic fly for a while
+    if (++failureCnt % failureInterval == 0) {
+      transactionNum = loader.makeProgress(sleepTime, transactionNum);
+      while(!c3.isLeader()) {
+        System.out.println("Wait for the rank 3 to take over the leadership");
+        Thread.sleep(sleepTime);
+      }
+      Assert.assertTrue("Rank 3 shall be the leader of the quorum", c3.isLeader());
+    }
+
+    loader.slowDownReplicationLoad();
+
+    // Verify logs are identical across all the quorum members
+    while (!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE)) {
+      Thread.sleep(10 * 1000);
+      System.out.println("Verifying logs ....");
+      Assert.assertTrue("Rank 3 shall be the leader of the quorum", c3.isLeader());
+    }
+
+    // Stop the client load
+    loader.stopReplicationLoad();
+
+    System.out.println(transactionNum + " transactions have been successfully replicated");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestPersistLastVotedFor.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestPersistLastVotedFor.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestPersistLastVotedFor.java
new file mode 100644
index 0000000..73683bb
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestPersistLastVotedFor.java
@@ -0,0 +1,83 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.protocol.ConsensusHost;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class TestPersistLastVotedFor {
+  private static int QUORUM_SIZE = 3;
+  private static int QUORUM_MAJORITY = 2;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private static QuorumClient client;
+  private static volatile int transactionNum = 0;
+  private ReplicationLoadForUnitTest loader;
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.disableVerboseLogging();
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, RAFT_TEST_UTIL.getScratchSetup(QUORUM_SIZE));
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+    transactionNum = 0;
+    loader = new ReplicationLoadForUnitTest(regionInfo, client, RAFT_TEST_UTIL, QUORUM_SIZE,
+      QUORUM_MAJORITY);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Test
+  public void testLastVotedForIsPersisted() throws IOException, InterruptedException {
+    // Add some transactions
+    loader.startReplicationLoad(1000);
+    transactionNum = loader.makeProgress(100, transactionNum);
+    loader.stopReplicationLoad();
+
+    RaftQuorumContext leader =
+      RAFT_TEST_UTIL.getLeaderQuorumContext(regionInfo);
+    // What is the current lastVotedFor
+    ConsensusHost initialLastVotedFor = leader.getLastVotedFor();
+
+    // Stop the consensusServer. lastVotedFor should have been persisted.
+    LocalConsensusServer consensusServer =
+      RAFT_TEST_UTIL.stopLocalConsensusServer(leader.getMyAddress());
+
+    RaftQuorumContext newQuorumContext =
+      RAFT_TEST_UTIL.restartLocalConsensusServer(consensusServer, regionInfo,
+      leader.getMyAddress());
+    ConsensusHost lastVotedForAsReadFromDisk =
+      newQuorumContext.getLastVotedFor();
+    Assert.assertEquals("Last Voted For was not persisted properly",
+      initialLastVotedFor, lastVotedForAsReadFromDisk);
+
+    // Let us try if the persisting works, if the lastVotedFor is null.
+    newQuorumContext.clearVotedFor();
+    consensusServer =
+      RAFT_TEST_UTIL.stopLocalConsensusServer(newQuorumContext.getMyAddress());
+    RaftQuorumContext newQuorumContextAfterSecondRestart =
+      RAFT_TEST_UTIL.restartLocalConsensusServer(consensusServer, regionInfo,
+        newQuorumContext.getMyAddress());
+
+    ConsensusHost emptyLastVotedFor =
+      newQuorumContextAfterSecondRestart.getLastVotedFor();
+    Assert.assertNull(emptyLastVotedFor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestRaftEventListener.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestRaftEventListener.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestRaftEventListener.java
new file mode 100644
index 0000000..7da8372
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestRaftEventListener.java
@@ -0,0 +1,130 @@
+package org.apache.hadoop.hbase.consensus;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.protocol.EditId;
+import org.apache.hadoop.hbase.consensus.protocol.Payload;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServer;
+import org.apache.hadoop.hbase.regionserver.DataStoreState;
+import org.apache.hadoop.hbase.regionserver.RaftEventListener;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestRaftEventListener {
+
+  private static int QUORUM_SIZE = 5;
+  private static int QUORUM_MAJORITY = 3;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private static QuorumClient client;
+  private int transactionNum = 0;
+
+  private ReplicationLoadForUnitTest loader;
+
+  /**
+   * A set keeps track of the current unavailable peers
+   */
+  private ConcurrentSkipListSet<String> unavailablePeerSet = new ConcurrentSkipListSet<>();
+
+  public class MockedRaftEventListener implements RaftEventListener {
+
+    public ByteBuffer becameLeader() {
+      return null;
+    }
+
+    public void becameNonLeader() {}
+    public void commit(final long index, final Payload payload) {}
+    public long getMinUnpersistedIndex() { return -1; }
+
+    @Override
+    public DataStoreState getState() {
+      return null;
+    }
+
+    @Override
+    public void updatePeerAvailabilityStatus(String peerAddress, boolean isAvailable) {
+      if (isAvailable) {
+        unavailablePeerSet.remove(peerAddress);
+      } else {
+        unavailablePeerSet.add(peerAddress);
+      }
+    }
+
+    @Override
+    public void closeDataStore() {}
+
+    @Override
+    public boolean canStepDown() {
+      return false;
+    }
+  }
+
+  private MockedRaftEventListener listener = new MockedRaftEventListener();
+
+  @Before
+  public void setUp() throws Exception {
+    RAFT_TEST_UTIL.disableVerboseLogging();
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, RAFT_TEST_UTIL.getScratchSetup(QUORUM_SIZE));
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+
+    // Register the listener for the highest rank, which is equal to QUORUM_SIZE;
+    for (Map.Entry<String, PeerServer> entry :
+        RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, QUORUM_SIZE).getPeerServers().entrySet()) {
+      entry.getValue().registerDataStoreEventListener(listener);
+    }
+
+    loader = new ReplicationLoadForUnitTest(regionInfo, client, RAFT_TEST_UTIL, QUORUM_SIZE,
+      QUORUM_MAJORITY);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Test(timeout=500000)
+  public void testRaftEventListenerForAvailability()
+    throws InterruptedException, IOException {
+    // Start the client load
+    loader.startReplicationLoad(100);
+
+    // Sleep for 5 sec
+    transactionNum = loader.makeProgress(1000, transactionNum);
+
+    // Stop the replica whose rank is 4
+    RaftQuorumContext c4 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 4);
+    System.out.println("Stopping one quorum member: " + c4);
+    LocalConsensusServer s4 = RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, 4);
+
+    // Sleep for 1 sec
+    transactionNum = loader.makeProgress(2000, transactionNum);
+    assertEquals(1, unavailablePeerSet.size());
+
+    // Start the replica whose rank is 4
+    RAFT_TEST_UTIL.restartLocalConsensusServer(s4, regionInfo, c4.getMyAddress());
+    System.out.println("Restarted one quorum member: " + c4);
+
+    // Sleep for 5 sec
+    transactionNum = loader.makeProgress(3000, transactionNum);
+    assertEquals(0, unavailablePeerSet.size());
+    System.out.println("There is no element in the unavailablePeerSet !");
+    loader.stopReplicationLoad();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestRandomAccessLog.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestRandomAccessLog.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestRandomAccessLog.java
new file mode 100644
index 0000000..37910b1
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestRandomAccessLog.java
@@ -0,0 +1,93 @@
+package org.apache.hadoop.hbase.consensus;
+
+import junit.framework.Assert;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.log.RandomAccessLog;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.BucketAllocator;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.InHeapArena;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+public class TestRandomAccessLog {
+
+  File file;
+
+  private final Arena arena = new InHeapArena(BucketAllocator.DEFAULT_BUCKETS,
+    HConstants.ARENA_CAPACITY_DEFAULT);
+
+  @Before
+  public void setUp() throws Exception {
+    file = new File("testBasics");
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (file != null) {
+      file.delete();
+    }
+  }
+
+  @Test
+  public void testBasics() throws IOException {
+    RandomAccessLog log = new RandomAccessLog(file, false);
+
+    Assert.assertTrue(log.getCreationTime() !=
+      RandomAccessLog.UNKNOWN_CREATION_TIME);
+
+    final int term = 1;
+    final int startIndex = 1;
+    final int middleIndex = 500;
+    final int endIndex = 1000;
+    final String readSessionKey = "test";
+
+    for (int i = startIndex; i <= endIndex; i++) {
+      WALEdit edit = new WALEdit();
+      edit.add(new KeyValue(Bytes.toBytes("test" + i), System.currentTimeMillis()));
+      log.append(term, i, WALEdit.serializeToByteBuffer(Arrays.asList(edit),
+              1234567890L, Compression.Algorithm.NONE));
+    }
+
+    Assert.assertEquals(term, log.getCurrentTerm());
+    Assert.assertEquals(startIndex, log.getInitialIndex());
+    Assert.assertEquals(endIndex, log.getLastIndex());
+    Assert.assertEquals(endIndex, log.getTxnCount());
+
+    log.truncate(middleIndex + 1);
+    Assert.assertEquals(term, log.getCurrentTerm());
+    Assert.assertEquals(startIndex, log.getInitialIndex());
+    Assert.assertEquals(middleIndex, log.getLastIndex());
+    Assert.assertEquals(middleIndex, log.getTxnCount());
+
+    log.finalizeForWrite();
+
+    RandomAccessLog log2 = new RandomAccessLog(file, false);
+    log2.rebuild(readSessionKey);
+    Assert.assertEquals(log.getCurrentTerm(), log2.getCurrentTerm());
+    Assert.assertEquals(log.getInitialIndex(), log2.getInitialIndex());
+    Assert.assertEquals(log.getLastIndex(), log2.getLastIndex());
+    Assert.assertEquals(log.getTxnCount(), log2.getTxnCount());
+
+
+    for (int i = startIndex; i <= middleIndex; i++) {
+      MemoryBuffer buffer = log2.getTransaction(term, i, readSessionKey, arena);
+      List<WALEdit> txns = WALEdit.deserializeFromByteBuffer(buffer.getBuffer());
+      Assert.assertEquals(1, txns.size());
+      Assert.assertEquals(1, txns.get(0).getKeyValues().size());
+      byte[] row = txns.get(0).getKeyValues().get(0).getRow();
+      Assert.assertEquals(0, Bytes.compareTo(Bytes.toBytes("test" + i), row));
+      arena.freeByteBuffer(buffer);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestReadOnlyLog.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestReadOnlyLog.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestReadOnlyLog.java
new file mode 100644
index 0000000..66a1e36
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/TestReadOnlyLog.java
@@ -0,0 +1,147 @@
+package org.apache.hadoop.hbase.consensus;
+
+import junit.framework.Assert;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.log.LogWriter;
+import org.apache.hadoop.hbase.consensus.log.RandomAccessLog;
+import org.apache.hadoop.hbase.consensus.log.ReadOnlyLog;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Arena;
+import org.apache.hadoop.hbase.util.BucketAllocator;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.InHeapArena;
+import org.apache.hadoop.hbase.util.MemoryBuffer;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class TestReadOnlyLog {
+  private static final Logger LOG = LoggerFactory.getLogger(
+          TestReadOnlyLog.class);
+  private static final int TOTAL_COMMIT = 100;
+  private static final long INITIAL_TERM = 1;
+  private static final long INITIAL_INDEX = 1;
+  private static final KeyValue.KVComparator comparator = new KeyValue.KVComparator();
+  private static final int CONCURRENT_READER_CNT = 10;
+  private final AtomicInteger SUCCESS_CNT = new AtomicInteger(0);
+
+  private static Random random;
+  private static File file;
+  private static ReadOnlyLog readOnlyLog;
+
+  private final Arena arena = new InHeapArena(BucketAllocator.DEFAULT_BUCKETS,
+    HConstants.ARENA_CAPACITY_DEFAULT);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    random = new Random();
+    file = new File("TestReadOnlyLog_" + INITIAL_INDEX + "_" + INITIAL_INDEX);
+    file.createNewFile();
+    readOnlyLog = new ReadOnlyLog(file, INITIAL_TERM, INITIAL_INDEX);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    readOnlyLog.closeAndDelete();
+  }
+
+  @Test(timeout=50000)
+  public void testConcurrentReader() throws Exception {
+    prepareLogFile();
+
+    Assert.assertTrue(readOnlyLog.getCreationTime() !=
+      RandomAccessLog.UNKNOWN_CREATION_TIME);
+
+    ExecutorService service = Executors.newFixedThreadPool(CONCURRENT_READER_CNT);
+
+    for (int i = 0 ; i < CONCURRENT_READER_CNT; i++) {
+      final String sessionKey = Integer.toString(i);
+      service.submit(new Runnable() {
+        @Override
+        public void run() {
+          readLogFile(sessionKey);
+        }
+      });
+    }
+
+    service.shutdown();
+    try {
+      service.awaitTermination(50000, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {}
+
+    Assert.assertEquals(CONCURRENT_READER_CNT, SUCCESS_CNT.get());
+
+  }
+
+  private void readLogFile(String sessionKey) {
+    // Choose a random read point
+    long readPointer = (long)random.nextInt(TOTAL_COMMIT);
+
+    try {
+      for (long i = readPointer; i < TOTAL_COMMIT; i++) {
+        MemoryBuffer buffer = readOnlyLog.getTransaction(INITIAL_TERM, i, sessionKey, arena);
+
+        // Read the commit entry
+        List<WALEdit> txns = WALEdit.deserializeFromByteBuffer(buffer.getBuffer());
+
+        WALEdit edit = txns.get(0);
+        KeyValue kv = edit.getKeyValues().get(0);
+        KeyValue expectedKV = new KeyValue(Bytes.toBytes(i), i);
+
+        // Verify the commit entry
+        Assert.assertEquals(1, txns.size());
+        Assert.assertEquals(1, edit.size());
+        Assert.assertEquals(0, comparator.compare(expectedKV, kv));
+        arena.freeByteBuffer(buffer);
+      }
+
+      // Increase the success cnt
+      SUCCESS_CNT.incrementAndGet();
+      LOG.info("Reader #" + sessionKey + " has read and verified all the commits from " +
+        + readPointer + " to " + TOTAL_COMMIT);
+
+    } catch (Exception e) {
+      // Fail the unit test if any exception caught
+      LOG.error("Unexpected exception: ", e);
+      Assert.fail("Unexpected exception: " + e);
+    }
+  }
+
+  private void prepareLogFile() throws IOException {
+    LogWriter writer = new LogWriter(new RandomAccessFile(file, "rw"), false);
+
+    // Generate the header
+    final long initialIndex = 0;
+    final long term = 1;
+    writer.writeFileHeader(term, initialIndex);
+
+    // Write the numTXNs to the log file
+    List<WALEdit> txns;
+    WALEdit edit;
+    for (long i = initialIndex; i < TOTAL_COMMIT; i++) {
+      edit = new WALEdit();
+      edit.add(new KeyValue(Bytes.toBytes(i), i));
+      txns = Arrays.asList(edit);
+      writer.append(i, WALEdit.serializeToByteBuffer(txns, 1234567890L,
+              Compression.Algorithm.NONE));
+    }
+
+    // Close the writer
+    writer.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestAsyncStatesInRaftStateMachine.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestAsyncStatesInRaftStateMachine.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestAsyncStatesInRaftStateMachine.java
new file mode 100644
index 0000000..17d7ace
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestAsyncStatesInRaftStateMachine.java
@@ -0,0 +1,242 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import com.google.common.util.concurrent.SettableFuture;
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.consensus.RaftTestUtil;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.consensus.raft.states.RaftStateType;
+import org.apache.hadoop.hbase.consensus.server.LocalConsensusServer;
+import org.apache.hadoop.hbase.consensus.server.peer.AbstractPeer;
+import org.apache.hadoop.hbase.consensus.server.peer.PeerServer;
+import org.apache.hadoop.hbase.consensus.server.peer.states.PeerServerStateType;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.junit.After;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.AbstractExecutorService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertTrue;
+import static junit.framework.Assert.fail;
+
+public class TestAsyncStatesInRaftStateMachine {
+  private static final Logger LOG = LoggerFactory.getLogger(
+    TestAsyncStatesInRaftStateMachine.class);
+
+  private static final int QUORUM_SIZE = 5;
+  private static final int QUORUM_MAJORITY = 3;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private Configuration conf;
+  private QuorumClient client;
+  private ExecutorService executorService;
+
+  class DummyExecutorService extends AbstractExecutorService {
+    @Override
+    public void shutdown() {
+    }
+
+    @Override
+    public List<Runnable> shutdownNow() {
+      return null;
+    }
+
+    @Override
+    public boolean isShutdown() {
+      return false;
+    }
+
+    @Override
+    public boolean isTerminated() {
+      return false;
+    }
+
+    @Override
+    public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
+      return false;
+    }
+
+    @Override
+    public Future<?> submit(Runnable task) {
+      return SettableFuture.create();
+    }
+
+    @Override
+    public void execute(Runnable command) {
+    }
+  }
+
+  public void setUpExecutors(boolean useDummyWriteOpsPool,
+                             boolean useDummyReadOpsPool) {
+    FSMLargeOpsExecutorService
+      .initializeForTesting(
+        (useDummyWriteOpsPool ?
+          new DummyExecutorService() :
+          FSMLargeOpsExecutorService.createWriteOpsExecutorService(conf)),
+        (useDummyReadOpsPool ?
+          new DummyExecutorService() :
+          FSMLargeOpsExecutorService.createReadOpsExecutorService(conf)));
+  }
+
+  public void setUp(boolean useDummyWriteOpsPool,
+                    boolean useDummyReadOpsPool) throws Exception {
+    conf = RAFT_TEST_UTIL.getConf();
+    setUpExecutors(useDummyWriteOpsPool, useDummyReadOpsPool);
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, null);
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (executorService != null) {
+      while (!executorService.isTerminated()) {
+        executorService.shutdownNow();
+        Threads.sleep(1000);
+      }
+    }
+    LOG.info("Shutting down the FSM");
+    RAFT_TEST_UTIL.setRaftQuorumContextClass(RaftQuorumContext.class);
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Test(expected = TimeoutException.class)
+  public void ensureNoProgressIfSendAppendRequestIsNotComplete()
+    throws Exception {
+    setUp(true, false);
+    try {
+      testReplicatingCommitsAsync(1).get(3000, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      // the peer with the lowest timeout will try to write the
+      // votedFor and should get stuck.
+      RaftQuorumContext r = RAFT_TEST_UTIL.getRaftQuorumContextByRank(
+        regionInfo, 5);
+      assertEquals(RaftStateType.SEND_VOTE_REQUEST,
+        r.getCurrentRaftState().getStateType());
+      throw e;
+    }
+  }
+
+  public PeerServer killAndRevivePeerWhileReplicatingEdits(
+    boolean blockOnHandleAppendResponse) throws Exception {
+    setUp(false, blockOnHandleAppendResponse);
+    // Make one single commit.
+    testReplicatingCommits(1);
+
+    RaftQuorumContext leader =
+      RAFT_TEST_UTIL.getLeaderQuorumContext(regionInfo);
+
+    // Stop the peer with rank = 1.
+    RaftQuorumContext peer =
+      RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 1);
+
+    PeerServer peerServer = leader.getPeerServers().get(peer.getMyAddress());
+    LocalConsensusServer peerConsensusServer =
+      RAFT_TEST_UTIL.stopLocalConsensusServer(regionInfo, 1);
+
+    // Replicate some other commits, the dead server will miss out.
+    testReplicatingCommits(10);
+
+    // Restart that dead server.
+    RAFT_TEST_UTIL.restartLocalConsensusServer(peerConsensusServer,
+      regionInfo, peer.getMyAddress());
+
+    // Wait for dead server to come back
+    long start = System.currentTimeMillis();
+    while (!RAFT_TEST_UTIL.verifyLogs(regionInfo.getQuorumInfo(), QUORUM_SIZE, true) &&
+      !blockOnHandleAppendResponse) {
+      Thread.sleep(1000);
+      // stop if we waited for more than 10 seconds
+      if (System.currentTimeMillis() - start > 100000) {
+        Assert.fail("Timed out while waiting for dead server to come back");
+      }
+    }
+
+    return peerServer;
+  }
+
+  // TODO
+  // The two tests below are unstable, and can be flaky dependent on the killing
+  // of the server, and restarting it. I could fix it by adding sleep, but
+  // that is no guarantee that the test won't break in the future.
+  /**
+  @Test
+  public void ensureNoProgressIfPeerHandleAppendResponseIsNotComplete()
+    throws Exception {
+    PeerServer s = killAndRevivePeerWhileReplicatingEdits(true);
+    assertTrue(
+      ((AbstractPeer) s).getStateMachineService().getCurrentState().getStateType().equals(
+        PeerServerStateType.HANDLE_APPEND_RESPONSE));
+  }
+
+  @Test
+  public void ensureProgressIfPeerHandleAppendResponseIsComplete()
+    throws Exception {
+    PeerServer s = killAndRevivePeerWhileReplicatingEdits(false);
+    assertTrue(
+      ((AbstractPeer) s).getStateMachineService().getCurrentState().getStateType().equals(
+        PeerServerStateType.PEER_FOLLOWER));
+  }
+   */
+
+  @Test
+  public void ensureProgressWhenSendAppendRequestCompletes() throws Exception {
+    setUp(false, false);
+    testReplicatingCommitsAsync(1).get(3000, TimeUnit.MILLISECONDS);
+  }
+
+  private Future testReplicatingCommitsAsync(final int numCommits)
+    throws Exception {
+    Runnable r = new Runnable() {
+      @Override
+      public void run() {
+        testReplicatingCommits(numCommits);
+      }
+    };
+    executorService = Executors.newSingleThreadExecutor();
+    return executorService.submit(r);
+  }
+
+  private void testReplicatingCommits(int numCommits) {
+    try {
+      RAFT_TEST_UTIL.waitForLeader(regionInfo);
+      RaftQuorumContext leader =
+        RAFT_TEST_UTIL.getLeaderQuorumContext(regionInfo);
+      Assert.assertNotNull(leader);
+
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+      for (int i = 0; i < numCommits; i++) {
+        client.replicateCommits(Arrays.asList(generateTestingWALEdit()));
+
+      }
+      RAFT_TEST_UTIL.dumpStates(regionInfo);
+    } catch (Exception e) {
+      LOG.error("Errors: ", e);
+      fail("Unexpected exception: " + e);
+    }
+  }
+
+  private static WALEdit generateTestingWALEdit() {
+    KeyValue kv = KeyValue.createFirstOnRow(Bytes.toBytes("TestQuorum"));
+    return new WALEdit(Arrays.asList(kv));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestFiniteStateMachine.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestFiniteStateMachine.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestFiniteStateMachine.java
new file mode 100644
index 0000000..3125c30
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestFiniteStateMachine.java
@@ -0,0 +1,194 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * A test class wrapping FiniteStateMachineImpl, exposing some of its internals
+ * in order to increase testability.
+ */
+@InterfaceAudience.Private
+class TestableFiniteStateMachine extends FiniteStateMachine {
+
+  public TestableFiniteStateMachine(State defaultStartState) {
+    super(TestableFiniteStateMachine.class.getName());
+    setStartState(defaultStartState);
+  }
+
+  public HashMap<Transition, State> getTransitionMap(
+          final State s) {
+    return super.getTransitionMap(s);
+  }
+
+  public Transition getNextTransition(final State s, final Event e) {
+    return super.getNextTransition(s, e);
+  }
+}
+
+interface ImmutableTestContext {
+  public boolean getA();
+}
+
+interface MutableTestContext extends ImmutableTestContext, MutableContext {
+  public void setA(boolean x);
+}
+
+class TestContext implements ImmutableTestContext, MutableTestContext {
+  private boolean a;
+
+  public TestContext() {
+    this.a = false;
+  }
+
+  public boolean getA() { return a; }
+  public void setA(boolean x) { a = x; }
+}
+
+enum States implements StateType {
+  NONE,
+  START,
+  STOP,
+  A,
+  B,
+  MAX
+}
+
+class TestState extends State {
+  protected MutableTestContext c;
+
+  public TestState(final States t, final MutableTestContext c) {
+    super(t);
+    this.c = c;
+  }
+  public void onEntry(final Event e) {}
+  public void onExit(final Event e) {}
+}
+
+class Start extends TestState {
+  public Start(final MutableTestContext c) {
+    super(States.START, c);
+  }
+}
+
+class Stop extends TestState {
+  public Stop(final MutableTestContext c) {
+    super(States.STOP, c);
+  }
+}
+
+class A extends TestState {
+  public A(final MutableTestContext c) {
+    super(States.A, c);
+  }
+  public void onEntry(final Event e) {
+    if (c != null) c.setA(!c.getA());
+  }
+}
+
+enum Events implements EventType {
+  NONE,
+  TYPE1,
+  TYPE2,
+  MAX
+}
+
+enum Transitions implements TransitionType {
+  NONE,
+  NOOP,
+  IS_A,
+  ON_TYPE1,
+  MAX
+}
+
+abstract class TestCondition implements Conditional {
+  protected ImmutableTestContext c;
+
+  public TestCondition(final ImmutableTestContext c) {
+    this.c = c;
+  }
+
+  public abstract boolean isMet(final Event e);
+}
+
+class IsNoop extends TestCondition {
+  public IsNoop(final ImmutableTestContext c) {
+    super(c);
+  }
+  public boolean isMet(final Event e) { return true; }
+}
+
+class IsA extends TestCondition {
+  public IsA(final ImmutableTestContext c) {
+    super(c);
+  }
+  public boolean isMet(final Event e) { return c.getA(); }
+}
+
+public class TestFiniteStateMachine {
+  TestableFiniteStateMachine s;
+  TestContext c = new TestContext();
+
+  State START = new Start(c);
+  State STOP = new Stop(c);
+  State A = new A(c);
+
+  Transition NOOP = new Transition(Transitions.NOOP, new IsNoop(c));
+  Transition IS_A = new Transition(Transitions.IS_A, new IsA(c));
+  Transition ON_TYPE1 = new Transition(Transitions.ON_TYPE1,
+          new OnEvent(Events.TYPE1));
+
+  Event TYPE1 = new Event(Events.TYPE1);
+
+  @Before
+  public void createNewStateMachine() {
+    s = new TestableFiniteStateMachine(START);
+  }
+
+  @Test
+  public void shouldAddTransitions() throws Exception {
+    s.addTransition(START, STOP, NOOP);
+    HashMap<Transition, State> transitionMap = s.getTransitionMap(START);
+    assertNotNull(transitionMap);
+    assertEquals(STOP, transitionMap.get(NOOP));
+  }
+
+  @Test
+  public void shouldTransitionOnMetCondition() throws Exception {
+    s.addTransition(START, STOP, NOOP);
+    assertEquals(NOOP, s.getNextTransition(START, null));
+  }
+
+  @Test
+  public void shouldIgnoreUnmetConditionals() throws Exception {
+    s.addTransition(START, STOP, IS_A);
+    assertNull(s.getNextTransition(START, null));
+  }
+
+  @Test
+  public void shouldTransitionOnEvent() throws Exception {
+    s.addTransition(START, STOP, ON_TYPE1);
+    assertEquals(ON_TYPE1, s.getNextTransition(START, TYPE1));
+  }
+
+  @Test
+  public void shouldIgnoreInvalidEvents() throws Exception {
+    Event e = new Event(Events.TYPE2);
+    s.addTransition(START, STOP, ON_TYPE1);
+    assertNull(s.getNextTransition(START, e));
+  }
+
+  @Test
+  public void shouldBeGreedy() throws Exception {
+    s.addTransition(START, A, ON_TYPE1);
+    s.addTransition(A, STOP, IS_A);
+    s.setStartState(START);
+    assertEquals(STOP, s.getNextState(TYPE1));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestFiniteStateMachineService.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestFiniteStateMachineService.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestFiniteStateMachineService.java
new file mode 100644
index 0000000..d864737
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestFiniteStateMachineService.java
@@ -0,0 +1,144 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.serial.AsyncSerialExecutorServiceImpl;
+import org.apache.hadoop.hbase.util.serial.SerialExecutorService;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFiniteStateMachineService {
+  CountDownLatch latch;
+  CountDownMachine fsm;
+  FiniteStateMachineService service;
+  boolean multiplexedFSM;
+
+  public TestFiniteStateMachineService(boolean multiplexedFSM) {
+    this.multiplexedFSM = multiplexedFSM;
+  }
+
+  @SuppressWarnings("serial")
+  @Parameterized.Parameters
+  public static Collection<Boolean[]> parameters() {
+    return new ArrayList<Boolean[]>() {{
+      add(new Boolean[]{ true });
+      add(new Boolean[]{ false });
+    }};
+  }
+
+  @Before
+  public void setUp() {
+    latch = new CountDownLatch(3);
+    fsm = new CountDownMachine("CountDownMachine", latch);
+    if (multiplexedFSM) {
+      SerialExecutorService serialService =
+          new AsyncSerialExecutorServiceImpl(HConstants.DEFAULT_FSM_MUX_THREADPOOL_SIZE,
+              "serialScheduler");
+      service = new ConstitutentFSMService(fsm, serialService.createStream());
+    } else {
+      service = new FiniteStateMachineServiceImpl(fsm);
+    }
+  }
+
+  @Test
+  public void shouldDrainEventQueueOnShutdown() throws InterruptedException {
+    assertFalse("Service should not be shutdown", service.isShutdown());
+    for (int i = 0; i < latch.getCount(); i++) {
+      assertTrue("Event should be scheduled",
+        service.offer(new Event(Events.PUSH)));
+    }
+    service.shutdown();
+    fsm.start();  // Allow the CountDownMachine to handle events.
+    assertTrue("Event queue should be drained after shutdown",
+      latch.await(100, TimeUnit.MILLISECONDS));
+    assertTrue("Service should be terminated",
+      service.awaitTermination(100, TimeUnit.MILLISECONDS));
+  }
+
+  enum States implements StateType {
+    NONE,
+    WAIT,
+    PUSH,
+    MAX
+  }
+
+  enum Events implements EventType {
+    NONE,
+    PUSH,
+    MAX
+  }
+
+  enum Transitions implements TransitionType {
+    NONE,
+    UNCONDITIONAL,
+    ON_PUSH,
+    MAX
+  }
+
+  private class CountDownMachine extends FiniteStateMachine {
+    CountDownLatch startDelay;
+
+    public CountDownMachine(String name, CountDownLatch latch) {
+      super(name);
+      this.startDelay = new CountDownLatch(1);
+
+      State WAIT = new CountDownState(States.WAIT, null);
+      State PUSH = new CountDownState(States.PUSH, latch);
+
+      addTransition(WAIT, PUSH,
+        new Transition(Transitions.ON_PUSH, new OnEvent(Events.PUSH)));
+      addTransition(PUSH, WAIT,
+        new Transition(Transitions.UNCONDITIONAL, new Unconditional()));
+      setStartState(WAIT);
+    }
+
+    public void start() {
+      startDelay.countDown();
+    }
+
+    /**
+     * Delay handling events until start has been called. This works around
+     * the race condition between adding events to the queue and shutdown
+     * of the FSM service.
+     * @param e Event to be handled
+     */
+    @Override
+    public void handleEvent(final Event e) {
+      try {
+        startDelay.await();
+        super.handleEvent(e);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    private class CountDownState extends State {
+      CountDownLatch latch;
+
+      public CountDownState(StateType t, CountDownLatch latch) {
+        super(t);
+        this.latch = latch;
+      }
+
+      @Override
+      public void onEntry(Event e) {
+        if (latch != null) {
+          latch.countDown();
+        }
+      }
+
+      @Override
+      public void onExit(Event e) {}
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestIncompleteStates.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestIncompleteStates.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestIncompleteStates.java
new file mode 100644
index 0000000..22c62e1
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/fsm/TestIncompleteStates.java
@@ -0,0 +1,291 @@
+package org.apache.hadoop.hbase.consensus.fsm;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import junit.framework.Assert;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.serial.AsyncSerialExecutorServiceImpl;
+import org.apache.hadoop.hbase.util.serial.SerialExecutorService;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIncompleteStates {
+  enum States implements StateType {
+    S1,
+    S2,
+    S3,
+    S4,
+    S5,
+    S6
+  }
+
+  enum Events implements EventType {
+    E1,
+    E2,
+    E3
+  }
+
+  enum Transitions implements TransitionType {
+    T1,
+    T2,
+    T3,
+    T4,
+    T5
+  }
+
+  private class TestableState extends State {
+    boolean entered = false;
+
+    public TestableState(StateType stateType) {
+      super(stateType);
+    }
+
+    @Override
+    public void onEntry(Event e) {
+      entered = true;
+    }
+
+    @Override
+    public void onExit(Event e) {
+      countDownLatch.countDown();
+    }
+
+    public boolean isEntered() {
+      return entered;
+    }
+  }
+
+  /**
+   * This type of state mocks the behavior of a state which doesn't complete
+   * its onEntry() method's logic with the return of the onEntry() method call.
+   */
+  private class TestableStateWithAsyncWork extends TestableState {
+    boolean isComplete = false;
+    SettableFuture<?> pendingFuture;
+
+    public TestableStateWithAsyncWork(StateType stateType) {
+      super(stateType);
+    }
+
+    @Override
+    public boolean isAsyncState() {
+      return true;
+    }
+
+    @Override
+    public void onEntry(Event e) {
+      entered = true;
+      pendingFuture = SettableFuture.create();
+    }
+
+    @Override
+    public boolean isComplete() {
+      return isComplete;
+    }
+
+    @Override
+    public ListenableFuture<?> getAsyncCompletion() {
+      return pendingFuture;
+    }
+
+    public void setComplete() {
+      if (entered) {
+        isComplete = true;
+        pendingFuture.set(null);
+      }
+    }
+  }
+
+  // States
+  TestableState S1 = new TestableState(States.S1);
+  TestableStateWithAsyncWork S2 = new TestableStateWithAsyncWork(States.S2);
+  TestableState S3 = new TestableState(States.S3);
+  TestableState S4 = new TestableState(States.S4);
+  TestableState S5 = new TestableState(States.S5);
+  TestableStateWithAsyncWork S6 = new TestableStateWithAsyncWork(States.S6);
+
+  // Transitions
+  Transition T1 = new Transition(Transitions.T1, new OnEvent(Events.E1));
+  Transition T2 = new Transition(Transitions.T2, new OnEvent(Events.E2));
+  Transition T3 = new Transition(Transitions.T3, new Unconditional());
+  Transition T4 = new Transition(Transitions.T4, new Unconditional());
+  Transition T5 = new Transition(Transitions.T5, new OnEvent(Events.E3));
+
+  // Events
+  Event E1 = new Event(Events.E1);
+  Event E2 = new Event(Events.E2);
+  Event E3 = new Event(Events.E3);
+
+  CountDownLatch countDownLatch;
+  FiniteStateMachine fsm;
+  FiniteStateMachineService fsmService;
+  boolean multiplexedFSM;
+
+  public TestIncompleteStates(boolean multiplexedFSM) {
+    this.multiplexedFSM = multiplexedFSM;
+  }
+
+  @SuppressWarnings("serial")
+  @Parameterized.Parameters
+  public static Collection<Boolean[]> parameters() {
+    return new ArrayList<Boolean[]>() {{
+      add(new Boolean[]{ true });
+      add(new Boolean[]{ false });
+    }};
+  }
+
+  @Before
+  public void setUp() {
+    fsm = new FiniteStateMachine("fsm");
+    fsm.setStartState(S1);
+    if (multiplexedFSM) {
+      SerialExecutorService serialService =
+          new AsyncSerialExecutorServiceImpl(HConstants.DEFAULT_FSM_MUX_THREADPOOL_SIZE,
+              "serialScheduler");
+      fsmService = new ConstitutentFSMService(fsm, serialService.createStream());
+    } else {
+      fsmService = new FiniteStateMachineServiceImpl(fsm);
+    }
+  }
+
+  public void countDown() {
+    try {
+      countDownLatch.await(500, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+    Assert.assertEquals(0, countDownLatch.getCount());
+  }
+
+  public void resetLatch(int expectedCount) {
+    countDownLatch = new CountDownLatch(expectedCount);
+  }
+
+  @Test
+  public void testIncompleteStates() throws InterruptedException {
+    fsm.addTransition(S1, S2, T1);
+    fsm.addTransition(S2, S3, T2);
+
+    // Offer the event E1.
+    resetLatch(1);
+    fsmService.offer(E1);
+    countDown();
+
+    // Check that the state S1 is complete, since it is a regular state.
+    Assert.assertEquals(true, S1.isComplete());
+
+    // Check that we land up in S2.
+    Assert.assertEquals(S2, fsmService.getCurrentState());
+    Assert.assertEquals(0, fsmService.getNumPendingEvents());
+
+    // Issue an event to transition to S3. Note that S2 mimics a state which
+    // has pending async work.
+    resetLatch(0);
+    fsmService.offer(E2);
+    countDown();
+
+    Assert.assertEquals(1, fsmService.getNumPendingEvents());
+    Assert.assertEquals(S2, fsmService.getCurrentState());
+
+    // Now set the state to be completed
+    resetLatch(1);
+    S2.setComplete();
+    Assert.assertEquals(true, S2.isComplete());
+    countDown();
+
+    Assert.assertEquals(0, fsmService.getNumPendingEvents());
+    Assert.assertEquals(S3, fsmService.getCurrentState());
+  }
+
+  /**
+   * This tests the case when you have a scenario like (S2 is an async state):
+   * S2--(Conditional)-->S3--(Conditional)-->S4--(Event E)-->S5.
+   *
+   * If we are in S2, and the async task is not complete, we should not
+   * transition to S3, and then to S4. Only when the task is complete, should we
+   * automatically transition to S3, and then to S4.
+   *
+   * Upon offering the event E, we should transition to S3, and then to S5.
+   * E should not be discarded.
+   */
+  @Test
+  public void testStateTransitionFromAsyncStatesWithConditions() {
+    fsm.addTransition(S1, S2, T1);
+    fsm.addTransition(S2, S3, T3);
+    fsm.addTransition(S3, S4, T4);
+    fsm.addTransition(S4, S5, T5);
+
+    // Offer the event E1.
+    resetLatch(1);
+    fsmService.offer(E1);
+    countDown();
+
+    // Check that the state S1 is complete, since it is a regular state.
+    Assert.assertEquals(true, S1.isComplete());
+
+    // Check that the current state is S2, and NOT S3, even though there is an
+    // unconditional transition from S2 to S3.
+    Assert.assertEquals(S2, fsm.getCurrentState());
+
+    // Now set the state S2 to be completed.
+    resetLatch(2);
+    S2.setComplete();
+    countDown();
+
+    // We should now be in S4, by taking a conditional transition to S3, and
+    // then to S4.
+    Assert.assertEquals(S4, fsmService.getCurrentState());
+
+    // Also check that we visited S3.
+    Assert.assertTrue(S3.isEntered());
+
+
+    // Now offer E3, we should transition unconditionally from S2 to S3, and
+    // unconditionally from S3 to S4. Then because E3 was offered, we should
+    // transition to S5.
+    resetLatch(1);
+    fsmService.offer(E3);
+    countDown();
+
+    Assert.assertEquals(S5, fsm.getCurrentState());
+  }
+
+  /**
+   * We should not abort an event when we are in an async state.
+   */
+  @Test
+  public void testEventNotAbortedWhenInAsyncState() {
+    // S2 and S6 are async states, which transition on E2 and E3 respectively.
+    fsm.addTransition(S1, S2, T1);
+    fsm.addTransition(S2, S6, T2);
+    fsm.addTransition(S6, S3, T5);
+
+    resetLatch(1);
+    fsmService.offer(E1);
+    countDown();
+
+    // We offer E3 before S2 is complete, so that it is in the event queue.
+    // After E2 we transition to S6, but E3 is not applicable yet because the
+    // state is not complete. We should not discard E3.
+    resetLatch(1);
+    fsmService.offer(E2);
+    fsmService.offer(E3);
+    S2.setComplete();
+    countDown();
+
+    resetLatch(1);
+    S6.setComplete();
+    countDown();
+
+    Assert.assertEquals(S3, fsm.getCurrentState());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestCachedFileChannel.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestCachedFileChannel.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestCachedFileChannel.java
new file mode 100644
index 0000000..eab543c
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestCachedFileChannel.java
@@ -0,0 +1,76 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Test;
+
+public class TestCachedFileChannel {
+  final static Log LOG = LogFactory.getLog(TestCachedFileChannel.class);
+
+  @Test
+  public void testNormalFile() throws IOException {
+    final int size = 100000;
+    final int max = 10000;
+    final String path = "/tmp/testNormalFile";
+    writeToFile(path, size);
+    CachedFileChannel cfc = new CachedFileChannel(new RandomAccessFile(path, "r"), max);
+
+    final int trials = 1000;
+    Random rand = new Random();
+    long now = System.currentTimeMillis();
+    LOG.debug("Setting the seed to " + now);
+    rand.setSeed(now);
+    for(int i = 0; i < trials; i++) {
+      int offset = rand.nextInt(size);
+      int length;
+      if (rand.nextBoolean()) {
+        // read something small that fits in memory.
+        length = rand.nextInt(Math.min(max, size-offset));
+      } else {
+        length = rand.nextInt(size - offset);
+      }
+
+      verifyData(cfc, offset, length);
+    }
+
+    // do some reads reading all the way to the end.
+    int more = 100;
+    for(int i = 0; i < more; i++) {
+      int length = rand.nextInt((int)(1.5 * max));
+      int offset = size - length;
+      verifyData(cfc, offset, length);
+    }
+
+    new File(path).delete();
+  }
+
+  private void verifyData(CachedFileChannel cfc, int offset, int length) throws IOException {
+    LOG.debug("Verifying data " + length + " bytes, starting from " + offset);
+    ByteBuffer bb = ByteBuffer.allocate(length);
+    cfc.read(bb, offset);
+    bb.flip();
+
+    for(int i = 0; i < length; ++i) {
+      Assert.assertEquals("Mismatch at location " + (offset + i),
+          (byte)(offset + i), bb.get());
+    }
+  }
+
+  private void writeToFile(String path, int size) throws IOException {
+    FileOutputStream fsOut = new FileOutputStream(path);
+    for(int i = 0; i < size; ++i) {
+      fsOut.write(i);
+    }
+    fsOut.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestRemoteLogFetcher.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestRemoteLogFetcher.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestRemoteLogFetcher.java
new file mode 100644
index 0000000..0605222
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestRemoteLogFetcher.java
@@ -0,0 +1,167 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.consensus.RaftTestUtil;
+import org.apache.hadoop.hbase.consensus.ReplicationLoadForUnitTest;
+import org.apache.hadoop.hbase.consensus.client.FetchTask;
+import org.apache.hadoop.hbase.consensus.client.QuorumClient;
+import org.apache.hadoop.hbase.consensus.quorum.RaftQuorumContext;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestRemoteLogFetcher {
+  private static final Logger LOG = LoggerFactory.getLogger(TestRemoteLogFetcher.class);
+  private static int QUORUM_SIZE = 3;
+  private static int QUORUM_MAJORITY = 2;
+  private static HRegionInfo regionInfo;
+  private static RaftTestUtil RAFT_TEST_UTIL = new RaftTestUtil();
+  private static QuorumClient client;
+  private static volatile int transactionNum = 0;
+  private ReplicationLoadForUnitTest loader;
+
+  @Before
+  public void setUp() throws Exception {
+//    RAFT_TEST_UTIL.disableVerboseLogging();
+    RAFT_TEST_UTIL.createRaftCluster(QUORUM_SIZE);
+    RAFT_TEST_UTIL.setUsePeristentLog(true);
+    RAFT_TEST_UTIL.assertAllServersRunning();
+    regionInfo = RAFT_TEST_UTIL.initializePeers();
+    RAFT_TEST_UTIL.addQuorum(regionInfo, RAFT_TEST_UTIL.getScratchSetup(QUORUM_SIZE));
+    RAFT_TEST_UTIL.startQuorum(regionInfo);
+    client = RAFT_TEST_UTIL.getQuorumClient(regionInfo.getQuorumInfo());
+    transactionNum = 0;
+    loader = new ReplicationLoadForUnitTest(regionInfo, client, RAFT_TEST_UTIL, QUORUM_SIZE,
+        QUORUM_MAJORITY);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    RAFT_TEST_UTIL.shutdown();
+  }
+
+  @Test(timeout=60000)
+  public void testLogFileStatusRetrieval() throws Exception {
+    RaftQuorumContext c3 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 3);
+    RaftQuorumContext c2 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 2);
+    RaftQuorumContext c1 = RAFT_TEST_UTIL.getRaftQuorumContextByRank(regionInfo, 1);
+
+    TransactionLogManager l3 = (TransactionLogManager)c3.getLogManager();
+    // Around 60 indices per log file on peer 3
+    l3.setRaftLogRollSize(1024 * 3000);
+    TransactionLogManager l1 = (TransactionLogManager)c1.getLogManager();
+    // Around 40 indices per log file on peer 1
+    l1.setRaftLogRollSize(1024 * 2000);
+
+    // Start the client load
+    loader.startReplicationLoad(100);
+
+    // Let the traffic fly for a while
+    transactionNum = loader.makeProgress(10000, transactionNum);
+    l3.rollCommittedLogs();
+    l1.rollCommittedLogs();
+
+    RemoteLogFetcher fetcher = new RemoteLogFetcher(c2);
+    fetcher.initializeQuorumClients();
+    List<Pair<String, List<LogFileInfo>>> statuses = fetcher.getPeerCommittedLogStatus(0);
+    assertFalse(statuses.isEmpty());
+    long minIndex = Long.MAX_VALUE;
+    long maxIndex = Long.MIN_VALUE;
+    for (Pair<String, List<LogFileInfo>> status : statuses) {
+      for (LogFileInfo info : status.getSecond()) {
+        if (info.getLastIndex() > maxIndex) {
+          maxIndex = info.getLastIndex();
+        }
+        if (info.getInitialIndex() < minIndex) {
+          minIndex = info.getInitialIndex();
+        }
+      }
+    }
+    assertTrue(minIndex == 1);
+    assertTrue(maxIndex > 1);
+    LOG.debug("Fetch log files for range [" + minIndex + ", " + maxIndex + "]");
+
+    Collection<FetchTask> tasks = fetcher.createFetchTasks(statuses, minIndex);
+    for (FetchTask task : tasks) {
+      LOG.debug(task.toString());
+    }
+    validateIndexCoverage(minIndex, maxIndex, tasks);
+
+    // Stop the client load
+    loader.stopReplicationLoad();
+  }
+
+  /**
+   * Validate indexes in all fetch tasks can be merged into one range, which matches
+   * the maximum index on any other peer
+   */
+  private void validateIndexCoverage(long minIndex, long maxIndex, Collection<FetchTask> tasks) {
+    List<Interval> intervals = new ArrayList<>();
+    for (FetchTask task : tasks) {
+      List<LogFileInfo> fileInfos = task.getFileInfos();
+      for (LogFileInfo info : fileInfos) {
+        Interval interval = new Interval(info.getInitialIndex(), info.getLastIndex());
+        intervals.add(interval);
+      }
+    }
+
+    Collections.sort(intervals, new Comparator<Interval>() {
+      @Override
+      public int compare(Interval i1, Interval i2) {
+        if (i1.start < i2.start) {
+          return -1;
+        } else if (i1.start > i2.start) {
+          return 1;
+        } else {
+          if (i1.end < i2.end) {
+            return -1;
+          } else if (i1.end > i2.end) {
+            return 1;
+          } else {
+            return 0;
+          }
+        }
+      }
+    });
+
+    // Merge sorted intervals into a set of minimum discrete intervals
+    ArrayList<Interval> res = new ArrayList<>();
+    for (Interval cur : intervals) {
+      if (res.isEmpty()) {
+        res.add(cur);
+      } else {
+        Interval last = res.get(res.size() - 1);
+        if (last.end >= cur.start - 1) {
+          last.end = Math.max(last.end, cur.end);
+        } else {
+          res.add(cur);
+        }
+      }
+    }
+
+    assertEquals("Indices should merge into one interval", 1, res.size());
+    Interval interval = res.get(0);
+    assertEquals("Min index should match", minIndex, interval.start);
+    assertEquals("Max index should match", maxIndex, interval.end);
+  }
+
+  private class Interval {
+    public long start;
+    public long end;
+    public Interval(long s, long e) { start = s; end = e; }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestTransactionLogCreator.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestTransactionLogCreator.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestTransactionLogCreator.java
new file mode 100644
index 0000000..bdc7d92
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/log/TestTransactionLogCreator.java
@@ -0,0 +1,40 @@
+package org.apache.hadoop.hbase.consensus.log;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.junit.Test;
+
+import java.io.File;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTransactionLogCreator {
+
+  @Test
+  public void testNewLogfileCreation() throws Exception {
+    String logDir = "/tmp/testNewLogfileCreation";
+    File logDirFile = new File(logDir);
+    if (logDirFile.exists()) {
+      FileUtils.deleteDirectory(logDirFile);
+    }
+    logDirFile.mkdir();
+    Configuration conf = new Configuration();
+    conf.set(HConstants.RAFT_TRANSACTION_LOG_DIRECTORY_KEY, logDir);
+    TransactionLogManager logManager =
+        new TransactionLogManager(conf, "test", HConstants.UNDEFINED_TERM_INDEX);
+    logManager.initialize(null);
+    // Wait for new logs to be created
+    Thread.sleep(1000);
+    File currentLogDir = new File(logDir + "/test/current/");
+    File[] files = currentLogDir.listFiles();
+    int expected = HConstants.RAFT_MAX_NUM_NEW_LOGS;
+    assertEquals("# of new log files", expected, files.length);
+
+    logManager.forceRollLog();
+    Thread.sleep(1000);
+    files = currentLogDir.listFiles();
+    expected++;
+    assertEquals("# of new log files", expected, files.length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/metrics/TestPeerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/metrics/TestPeerMetrics.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/metrics/TestPeerMetrics.java
new file mode 100644
index 0000000..85dcd72
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/metrics/TestPeerMetrics.java
@@ -0,0 +1,23 @@
+package org.apache.hadoop.hbase.consensus.metrics;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestPeerMetrics {
+  PeerMetrics metrics;
+
+  @Before
+  public void setUp() {
+    metrics = new PeerMetrics("TestTable.region", "proc1", "peer1", null);
+  }
+
+  @Test
+  public void shoudReturnName() {
+    String expectedName = String.format("%s:type=%s,name=%s,proc=%s,peer=%s",
+            "org.apache.hadoop.hbase.consensus", PeerMetrics.TYPE,
+            "TestTable.region", "proc1", "peer1");
+    assertEquals(expectedName, metrics.getMBeanName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestAggregateTimer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestAggregateTimer.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestAggregateTimer.java
new file mode 100644
index 0000000..84e53f4
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestAggregateTimer.java
@@ -0,0 +1,221 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import junit.framework.Assert;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * This test checks if we can run multiple timers at the same time.
+ */
+public class TestAggregateTimer {
+  public static final Log LOG = LogFactory.getLog(TestAggregateTimer.class);
+
+  class TimeoutHandlerWithCounter implements TimeoutEventHandler {
+    public AtomicInteger counter = new AtomicInteger(0);
+
+    @Override
+    public void onTimeout() {
+      counter.incrementAndGet();
+    }
+  }
+
+  AggregateTimer aggregateTimer;
+  TimeoutHandlerWithCounter c1, c2;
+  ConstituentTimer timer1, timer2;
+  long timeout1 = 100, timeout2 = 200;
+
+
+  @Before
+  public void setUp() {
+    aggregateTimer = new AggregateTimer();
+    c1 = new TimeoutHandlerWithCounter();
+    c2 = new TimeoutHandlerWithCounter();
+
+    // Default values.
+    // timer1 is supposed to be faster than timer2.
+    timeout1 = 100;
+    timeout2 = 200;
+
+    // Create the timers
+    timer1 = aggregateTimer.createTimer(
+      "timer1",
+      timeout1,
+      TimeUnit.MILLISECONDS,
+      c1
+    );
+
+    timer2 = aggregateTimer.createTimer(
+      "timer2",
+      timeout2,
+      TimeUnit.MILLISECONDS,
+      c2
+    );
+  }
+
+  @Test
+  public void testNothingFiresWhenTimersAreNotStarted() {
+    LOG.debug("Checking for timeouts not firing when the timers weren't started");
+    // Sleep for 2 * (timeout1 + timeout2). None of them should have fired
+    try {
+      Thread.sleep(2 * (timeout1 + timeout2));
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+
+    Assert.assertEquals("Timeout for Timer 1 should not have fired",
+      0, c1.counter.get());
+    Assert.assertEquals("Timeout for Timer 2 should not have fired",
+      0, c2.counter.get());
+  }
+
+  @Test
+  public void testOnlyOneTimerFiresIfOnlyOneWasStarted() {
+    // Starting the first timer thread
+    timer1.start();
+
+    LOG.debug("Checking for timeouts not firing when only one of the timers were started");
+
+    // Sleep for 2 * (timeout1 + timeout2).
+    // Only the callback for timer1 should have fired, since only timer1 was
+    // started.
+    try {
+      Thread.sleep(2 * (timeout1 + timeout2));
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+
+    // Checking that the timeout fires for timer 1, and resetting the counter
+    Assert.assertTrue("Timeout for Timer 1 should have fired",
+      c1.counter.getAndSet(0) > 0);
+    Assert.assertEquals("Timeout for Timer 2 should not have fired",
+      0, c2.counter.get());
+
+  }
+
+  @Test
+  public void testTimersFireSufficiently() {
+    // Start both the timer threads
+    timer1.start();
+    timer2.start();
+
+    LOG.debug("Checking for timeouts when both timers were started");
+
+    // Sleep for 3 * (timeout1 + timeout2). Both should have fired at least
+    // (total / individual timeout - 1). Subtracting one to avoid edge cases.
+    try {
+      Thread.sleep(3 * (timeout1 + timeout2));
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+
+    long totalSleepTime = (3 * (timeout1 + timeout2));
+    long targetTimeoutsForTimer1 = (totalSleepTime / timeout1) - 1;
+    long targetTimeoutsForTimer2 = (totalSleepTime / timeout2) - 1;
+    Assert.assertTrue("Timeout for Timer 1 did not fire enough",
+      c1.counter.getAndSet(0) >= targetTimeoutsForTimer1);
+    Assert.assertTrue("Timeout for Timer 2 did not fire enough",
+      c2.counter.getAndSet(0) >= targetTimeoutsForTimer2);
+  }
+
+  @Test
+  public void testResettingATimerWorks() {
+    // Start both the timer threads
+    timer1.start();
+    timer2.start();
+
+    LOG.debug("Checking timeouts when we reset one of the timers");
+    for (int i = 0; i < 100; i++) {
+      try {
+        Thread.sleep(timeout2 / 4);
+      } catch (InterruptedException e) {
+        LOG.error(e);
+        Thread.currentThread().interrupt();
+      }
+      timer2.reset();
+    }
+    LOG.debug("Done with the reset of the timers");
+
+    // As expected, timeout for timer 2 should have not fired
+    Assert.assertEquals(
+      "Timeout for Timer 2 should not be firing because we were resetting it",
+      0, c2.counter.getAndSet(0));
+
+    // Timer 1 wasn't touched, so it should still have fired
+    Assert.assertTrue(
+      "Timeout for Timer 1 should have fired, since we did not reset it",
+      c1.counter.getAndSet(0) > 0
+    );
+  }
+
+  @Test
+  public void testStartAndStopOfATimer() {
+    testTimersFireSufficiently();
+
+    // Stop timer 1 now
+    timer1.stop();
+    c1.counter.getAndSet(0);
+
+    // Sleep for 3 * (timeout1 + timeout2).
+    // Only callback for timer1 should not have fired.
+    try {
+      Thread.sleep(3 * (timeout1 + timeout2));
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+
+    // As expected, timeout for timer 1 should have not fired
+    Assert.assertEquals(
+      "Timeout for Timer 1 should not be firing because we have stopped it",
+      0, c1.counter.getAndSet(0));
+
+    // Timer 2 wasn't touched, so it should still have fired
+    Assert.assertTrue(
+      "Timeout for Timer 2 should have fired, since we did not stop it",
+      c2.counter.getAndSet(0) > 0
+    );
+  }
+
+  @Test
+  public void testStopAndStartWithBackoff() {
+    // Making the second timer slower to reproduce this case.
+    timeout2 = 500;
+    timer2.setDelay(timeout2, TimeUnit.MILLISECONDS);
+
+    timer1.start();
+    timer2.start();
+
+    // Assuming right now is t=0,
+    // The event for timer2 has been queued to occur at t=timeout2,
+    // t=2*timeout2, and so on.
+
+    // Stop timer2.
+    // The old event should not be processed for some time, since timer1's
+    // event would be queued at t=timeout1, which is before t=timeout2.
+    // If we change the backoff for timer2, and start it again before the old
+    // event for timer2 is processed, the old event should be discarded.
+    LOG.debug("Stopping timer2");
+    timer2.stop();
+
+    // Set the backoff to a large value
+    timer2.backoff(timeout2 * 100, TimeUnit.MILLISECONDS);
+
+    // Now resume the timer
+    timer2.start();
+
+    try {
+      Thread.sleep(timeout2 * 10);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+    Assert.assertEquals(
+      "The second timer should not have fired, since the backoff is very large",
+      0,
+      c2.counter.get()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestConstituentTimer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestConstituentTimer.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestConstituentTimer.java
new file mode 100644
index 0000000..1f56536
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestConstituentTimer.java
@@ -0,0 +1,147 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+class TimeoutWithLatchForConstituentTimer implements TimeoutEventHandler {
+  private AtomicInteger count = new AtomicInteger(0);
+  private final TestConstituentTimer test;
+
+  public TimeoutWithLatchForConstituentTimer(final TestConstituentTimer test) {
+    this.test = test;
+  }
+
+  public int getCount() {
+    return count.get();
+  }
+
+  @Override
+  public void onTimeout() {
+    count.incrementAndGet();
+    test.latch.countDown();
+  }
+}
+
+/**
+ * Test that the ConstituentTimer behaves like a regular RepeatingTimer.
+ */
+public class TestConstituentTimer {
+  public CountDownLatch latch;
+  public ConstituentTimer timer;
+  public TimeoutWithLatchForConstituentTimer callback;
+
+  @Before
+  public void setUp() {
+    latch = new CountDownLatch(1);
+    callback = new TimeoutWithLatchForConstituentTimer(this);
+    timer = new AggregateTimer().createTimer("timer", 100,
+      TimeUnit.MILLISECONDS, callback);
+  }
+
+  @Test(timeout=1000)
+  public void shouldFireOnTimeout() throws InterruptedException {
+    timer.start();
+    latch.await();
+    assertEquals(1, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void startShouldBeIdempotent() throws InterruptedException {
+    latch = new CountDownLatch(3);
+    timer.start();
+    timer.start();
+    timer.start();
+    // Wait 50 ms to make sure the task is running.
+    assertFalse("Timeout expected", latch.await(50, TimeUnit.MILLISECONDS));
+    timer.stop();
+    // Wait several cycles to pass.
+    assertFalse("Latch should not reach zero",
+      latch.await(500, TimeUnit.MILLISECONDS));
+    assertEquals(0, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldNotFireOnStop() throws InterruptedException {
+    timer.start();
+    latch.await(50, TimeUnit.MILLISECONDS); // Make sure the timer is running.
+    timer.stop();
+    assertFalse("Latch should not reach zero",
+      latch.await(200, TimeUnit.MILLISECONDS));
+    assertEquals(0, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldFireRepeatedlyIfNotStopped() throws InterruptedException {
+    latch = new CountDownLatch(3);
+    timer.start();
+    assertTrue("Latch should reach zero",
+      latch.await(500, TimeUnit.MILLISECONDS));
+    assertEquals(3, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldDelayCallbackOnReset() throws InterruptedException {
+    long begin = System.currentTimeMillis();
+    timer.start();
+    latch.await(50, TimeUnit.MILLISECONDS);
+    timer.reset();
+    latch.await();
+    long end = System.currentTimeMillis();
+    // Elapsed time should be >=150 milliseconds by now.
+    assertTrue(end - begin >= 150);
+    assertEquals(1, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldNotFireOnShutdown() throws InterruptedException {
+    timer.start();
+    timer.shutdown();
+    assertFalse("Latch should not reach zero",
+      latch.await(200, TimeUnit.MILLISECONDS));
+    assertEquals(0, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldRunTheUsualFollowerPattern() throws InterruptedException {
+    timer.start();
+    assertFalse("Timeout expected", latch.await(50, TimeUnit.MILLISECONDS));
+    timer.reset();
+    assertFalse("Timeout expected", latch.await(50, TimeUnit.MILLISECONDS));
+    timer.stop();
+    assertFalse("Timeout expected", latch.await(50, TimeUnit.MILLISECONDS));
+    assertEquals("onTimeout handler should not have executed yet", 0,
+      callback.getCount());
+    timer.start();
+    latch.await();
+    timer.stop();
+    synchronized (this) {
+      this.wait(300);  // Wait for three more timer cycles.
+    }
+    assertEquals(1, callback.getCount());
+  }
+
+  @Test
+  public void shouldBackoff() throws InterruptedException {
+    long delay = 200;
+    timer.setDelay(delay, TimeUnit.MILLISECONDS);
+    timer.start();
+    timer.stop();
+    timer.backoff(delay * 10, TimeUnit.MILLISECONDS);
+    timer.start();
+    synchronized (this) {
+      this.wait(delay * 5);
+    }
+    // We started and stopped the timer, followed by starting it with a large
+    // backoff again. The timer shouldn't have fired before the back off
+    // duration.
+    assertEquals(0, callback.getCount());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestRepeatingTimer.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestRepeatingTimer.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestRepeatingTimer.java
new file mode 100644
index 0000000..0fc2b03
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/quorum/TestRepeatingTimer.java
@@ -0,0 +1,143 @@
+package org.apache.hadoop.hbase.consensus.quorum;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+class TimeoutWithLatch implements TimeoutEventHandler {
+  private AtomicInteger count = new AtomicInteger(0);
+  private final TestRepeatingTimer test;
+
+  public TimeoutWithLatch(final TestRepeatingTimer test) {
+    this.test = test;
+  }
+
+  public int getCount() {
+    return count.get();
+  }
+
+  @Override
+  public void onTimeout() {
+    count.incrementAndGet();
+    test.latch.countDown();
+  }
+}
+
+public class TestRepeatingTimer {
+  public CountDownLatch latch;
+  public RepeatingTimer timer;
+  public TimeoutWithLatch callback;
+
+  @Before
+  public void setUp() {
+    latch = new CountDownLatch(1);
+    callback = new TimeoutWithLatch(this);
+    timer = new RepeatingTimer("test", 100, TimeUnit.MILLISECONDS, callback);
+  }
+
+  @Test(timeout=1000)
+  public void shouldFireOnTimeout() throws InterruptedException {
+    timer.start();
+    latch.await();
+    assertEquals(1, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void startShouldBeIdempotent() throws InterruptedException {
+    latch = new CountDownLatch(3);
+    timer.start();
+    timer.start();
+    timer.start();
+    // Wait 50 ms to make sure the task is running.
+    assertFalse("Timeout expected", latch.await(50, TimeUnit.MILLISECONDS));
+    timer.stop();
+    // Wait several cycles to pass.
+    assertFalse("Latch should not reach zero",
+            latch.await(500, TimeUnit.MILLISECONDS));
+    assertEquals(0, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldNotFireOnStop() throws InterruptedException {
+    timer.start();
+    latch.await(50, TimeUnit.MILLISECONDS); // Make sure the timer is running.
+    timer.stop();
+    assertFalse("Latch should not reach zero",
+            latch.await(200, TimeUnit.MILLISECONDS));
+    assertEquals(0, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldFireRepeatedlyIfNotStopped() throws InterruptedException {
+    latch = new CountDownLatch(3);
+    timer.start();
+    assertTrue("Latch should reach zero",
+            latch.await(500, TimeUnit.MILLISECONDS));
+    assertEquals(3, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldDelayCallbackOnReset() throws InterruptedException {
+    long begin = System.currentTimeMillis();
+    timer.start();
+    latch.await(50, TimeUnit.MILLISECONDS);
+    timer.reset();
+    latch.await();
+    long end = System.currentTimeMillis();
+    // Elapsed time should be >150 milliseconds by now.
+    assertTrue(end - begin > 150);
+    assertEquals(1, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldNotFireOnShutdown() throws InterruptedException {
+    timer.start();
+    timer.shutdown();
+    assertFalse("Latch should not reach zero",
+            latch.await(200, TimeUnit.MILLISECONDS));
+    assertEquals(0, callback.getCount());
+  }
+
+  @Test(timeout=1000)
+  public void shouldRunTheUsualFollowerPattern() throws InterruptedException {
+    timer.start();
+    assertFalse("Timeout expected", latch.await(50, TimeUnit.MILLISECONDS));
+    timer.reset();
+    assertFalse("Timeout expected", latch.await(50, TimeUnit.MILLISECONDS));
+    timer.stop();
+    assertFalse("Timeout expected", latch.await(50, TimeUnit.MILLISECONDS));
+    assertEquals("onTimeout handler should not have executed yet", 0,
+            callback.getCount());
+    timer.start();
+    latch.await();
+    timer.stop();
+    synchronized (this) {
+      this.wait(300);  // Wait for three more timer cycles.
+    }
+    assertEquals(1, callback.getCount());
+  }
+
+  @Test
+  public void shouldBackoff() throws InterruptedException {
+    long delay = 200;
+    timer.setDelay(delay, TimeUnit.MILLISECONDS);
+    timer.start();
+    timer.stop();
+    timer.backoff(delay * 10, TimeUnit.MILLISECONDS);
+    timer.start();
+    synchronized (this) {
+      this.wait(delay * 5);
+    }
+    // We started and stopped the timer, followed by starting it with a large
+    // backoff again. The timer shouldn't have fired before the back off
+    // duration.
+    assertEquals(0, callback.getCount());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestParser.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestParser.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestParser.java
new file mode 100644
index 0000000..334fe09
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestParser.java
@@ -0,0 +1,93 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.json.JSONException;
+import org.json.JSONObject;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class TestParser {
+  Configuration conf;
+  Parser parser;
+  JSONObject rmapAsJSON;
+
+  @Before
+  public void setUp() throws IOException, JSONException {
+    conf = HBaseConfiguration.create();
+    conf.set(HConstants.HYDRABASE_DCNAME, "DUMMYCLUSTER1");
+
+    parser = new Parser(conf);
+    rmapAsJSON = new JSONObject(new String(Files.readAllBytes(
+            Paths.get(getClass().getResource("rmap.json").getPath()))));
+  }
+
+  @Test
+  public void testParseRMap() throws IOException, JSONException {
+    List<HRegionInfo> regions = parser.parseTable("RPCBenchmarkingTable",
+            getTableObjectFromJSON("RPCBenchmarkingTable"));
+    assertEquals(3, regions.size());
+    HRegionInfo region = regions.get(0);
+    HTableDescriptor table = region.getTableDesc();
+    assertEquals("RPCBenchmarkingTable", table.getNameAsString());
+    assertFalse(table.isMetaTable());
+    assertFalse(table.isRootRegion());
+    HColumnDescriptor[] columnFamilies = table.getColumnFamilies();
+    assertEquals(1, columnFamilies.length);
+    HColumnDescriptor cf0 = columnFamilies[0];
+    assertEquals("cf", cf0.getNameAsString());
+    assertEquals("true", cf0.getValue("BLOCKCACHE"));
+    assertEquals("65536", cf0.getValue("BLOCKSIZE"));
+    assertEquals("NONE", cf0.getValue("BLOOMFILTER"));
+    assertEquals("0.01", cf0.getValue("BLOOMFILTER_ERRORRATE"));
+    assertEquals("NONE", cf0.getValue("COMPRESSION"));
+    assertEquals("NONE", cf0.getValue("DATA_BLOCK_ENCODING"));
+    assertEquals("true", cf0.getValue("ENCODE_ON_DISK"));
+    assertEquals("false", cf0.getValue("IN_MEMORY"));
+    assertEquals("0", cf0.getValue("REPLICATION_SCOPE"));
+    assertEquals("2147483647", cf0.getValue("TTL"));
+    assertEquals("2147483647", cf0.getValue("VERSIONS"));
+
+    assertEquals("aeeb54dc6fbca609443bd35796b59da5", region.getEncodedName());
+    assertEquals("", Bytes.toString(region.getStartKey()));
+    assertEquals("2aaaaaaa", Bytes.toString(region.getEndKey()));
+    assertEquals(1373324048180L, region.getRegionId());
+
+    InetSocketAddress[] favoredNodes =
+            region.getFavoredNodesMap().get("DUMMYCLUSTER1");
+    assertEquals(3, favoredNodes.length);
+    assertEquals(new InetSocketAddress("10.159.9.49", 60020), favoredNodes[0]);
+    assertEquals(new InetSocketAddress("10.159.9.45", 60020), favoredNodes[1]);
+    assertEquals(new InetSocketAddress("10.159.9.47", 60020), favoredNodes[2]);
+
+    Map<String, Map<HServerAddress, Integer>> peers = region.getPeers();
+    assertEquals(1, peers.size());
+    Map<HServerAddress, Integer> peersWithRank = region.getPeersWithRank();
+    assertEquals(3, peersWithRank.size());
+    assertEquals(new Integer(1),
+            peersWithRank.get(new HServerAddress("10.159.9.41:60020")));
+    assertEquals(new Integer(2),
+            peersWithRank.get(new HServerAddress("10.159.9.45:60020")));
+    assertEquals(new Integer(3),
+            peersWithRank.get(new HServerAddress("10.159.9.47:60020")));
+    assertEquals(peers.get("DUMMYCLUSTER1"), peersWithRank);
+
+    assertEquals(null, peersWithRank.get(new HServerAddress("1.1.1.1:11111")));
+  }
+
+  private JSONObject getTableObjectFromJSON(final String name)
+          throws JSONException {
+    return rmapAsJSON.getJSONObject("tables").getJSONObject(name);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/eca32aa4/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRMapConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRMapConfiguration.java b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRMapConfiguration.java
new file mode 100644
index 0000000..4c22dc1
--- /dev/null
+++ b/hbase-consensus/src/test/java/org/apache/hadoop/hbase/consensus/rmap/TestRMapConfiguration.java
@@ -0,0 +1,55 @@
+package org.apache.hadoop.hbase.consensus.rmap;
+
+import static org.junit.Assert.*;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestRMapConfiguration {
+  private Configuration conf;
+  private RMapConfiguration rmapConf;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = HBaseConfiguration.create();
+    conf.set(HConstants.RMAP_SUBSCRIPTION,
+            getClass().getResource("rmap.json").toURI().toString());
+    conf.set(HConstants.HYDRABASE_DCNAME, "DUMMYCLUSTER1");
+
+    rmapConf = new RMapConfiguration(conf);
+    URI uri = RMapConfiguration.getRMapSubscription(conf);
+    rmapConf.readRMap(uri);
+    rmapConf.appliedRMap(uri);
+  }
+
+  @Test
+  public void testReadingEmptyRMapSubscription() {
+    conf.set(HConstants.RMAP_SUBSCRIPTION, "");
+    assertNull("RMap subscription should be empty",
+      rmapConf.getRMapSubscription(conf));
+  }
+
+  @Test
+  public void testReadingNonEmptyRMapSubscription()
+          throws URISyntaxException {
+    conf.set(HConstants.RMAP_SUBSCRIPTION,
+            "hbase/rmaps/map1");
+    URI expectedRMapSubscription = new URI("hbase/rmaps/map1");
+    assertEquals(expectedRMapSubscription,
+            rmapConf.getRMapSubscription(conf));
+  }
+
+  @Test
+  public void shouldApplyRMap() {
+    URI subscription = RMapConfiguration.getRMapSubscription(conf);
+    assertTrue(rmapConf.isRMapApplied(subscription));
+  }
+}