You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2016/02/08 07:54:04 UTC

[1/2] hbase git commit: HBASE-15221 Reload the cache on re-tried puts in HTableMultiplexer and adds a close() method to HTableMultiplexer

Repository: hbase
Updated Branches:
  refs/heads/branch-1.2 63adcf33b -> d568db837


HBASE-15221 Reload the cache on re-tried puts in HTableMultiplexer and adds a close() method to HTableMultiplexer

When a Put fails due to a NotServingRegionException, the cached location
of that Region is never cleared. Thus, subsequent calls to resubmit
the Put will fail in the same way as the original, never determining
the new location of the Region.

If the Connection is not closed by the user before the Multiplexer
is discarded, it will leak resources and could cause resource
issues.

Signed-off-by: Sean Busbey <bu...@cloudera.com>


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

Branch: refs/heads/branch-1.2
Commit: 25878e1a97ab3d1009ecf799bfa5dabb205ee2fc
Parents: 63adcf3
Author: Josh Elser <el...@apache.org>
Authored: Fri Feb 5 21:23:37 2016 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Mon Feb 8 00:08:26 2016 -0600

----------------------------------------------------------------------
 hbase-client/pom.xml                            |   5 +
 .../hadoop/hbase/client/HTableMultiplexer.java  | 111 +++++++++--
 .../client/TestHTableMultiplexerViaMocks.java   | 193 +++++++++++++++++++
 3 files changed, 292 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/25878e1a/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 3e2c032..57f4f94 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -193,6 +193,11 @@
       <groupId>com.yammer.metrics</groupId>
       <artifactId>metrics-core</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/25878e1a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index f6becc6..744b24b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -19,6 +19,9 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import java.io.IOException;
 import java.util.AbstractMap.SimpleEntry;
 import java.util.ArrayList;
@@ -50,8 +53,6 @@ import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 /**
  * HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
  * Each put will be sharded into different buffer queues based on its destination region server.
@@ -97,7 +98,18 @@ public class HTableMultiplexer {
    */
   public HTableMultiplexer(Configuration conf, int perRegionServerBufferQueueSize)
       throws IOException {
-    this.conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    this(ConnectionFactory.createConnection(conf), conf, perRegionServerBufferQueueSize);
+  }
+
+  /**
+   * @param conn The HBase connection.
+   * @param conf The HBase configuration
+   * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for
+   *          each region server before dropping the request.
+   */
+  public HTableMultiplexer(Connection conn, Configuration conf,
+      int perRegionServerBufferQueueSize) {
+    this.conn = (ClusterConnection) conn;
     this.pool = HTable.getDefaultExecutor(conf);
     this.retryNum = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
@@ -116,6 +128,18 @@ public class HTableMultiplexer {
   }
 
   /**
+   * Closes the internal {@link Connection}. Does nothing if the {@link Connection} has already
+   * been closed.
+   * @throws IOException If there is an error closing the connection.
+   */
+  @SuppressWarnings("deprecation")
+  public synchronized void close() throws IOException {
+    if (!getConnection().isClosed()) {
+      getConnection().close();
+    }
+  }
+
+  /**
    * The put request will be buffered by its corresponding buffer queue. Return false if the queue
    * is already full.
    * @param tableName
@@ -169,13 +193,28 @@ public class HTableMultiplexer {
    * @return true if the request can be accepted by its corresponding buffer queue.
    */
   public boolean put(final TableName tableName, final Put put, int retry) {
+    return _put(tableName, put, retry, false);
+  }
+
+  /**
+   * Internal "put" which exposes a boolean flag to control whether or not the region location
+   * cache should be reloaded when trying to queue the {@link Put}.
+   * @param tableName Destination table for the Put
+   * @param put The Put to send
+   * @param retry Number of attempts to retry the {@code put}
+   * @param reloadCache Should the region location cache be reloaded
+   * @return true if the request was accepted in the queue, otherwise false
+   */
+  boolean _put(final TableName tableName, final Put put, int retry, boolean reloadCache) {
     if (retry <= 0) {
       return false;
     }
 
     try {
       HTable.validatePut(put, maxKeyValueSize);
-      HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), false);
+      // Allow mocking to get at the connection, but don't expose the connection to users.
+      ClusterConnection conn = (ClusterConnection) getConnection();
+      HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), reloadCache);
       if (loc != null) {
         // Add the put pair into its corresponding queue.
         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
@@ -214,7 +253,8 @@ public class HTableMultiplexer {
     return new HTableMultiplexerStatus(serverToFlushWorkerMap);
   }
 
-  private LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
+  @VisibleForTesting
+  LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
     FlushWorker worker = serverToFlushWorkerMap.get(addr);
     if (worker == null) {
       synchronized (this.serverToFlushWorkerMap) {
@@ -231,6 +271,11 @@ public class HTableMultiplexer {
     return worker.getQueue();
   }
 
+  @VisibleForTesting
+  ClusterConnection getConnection() {
+    return this.conn;
+  }
+
   /**
    * HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer.
    * report the number of buffered requests and the number of the failed (dropped) requests
@@ -339,7 +384,8 @@ public class HTableMultiplexer {
     }
   }
 
-  private static class PutStatus {
+  @VisibleForTesting
+  static class PutStatus {
     public final HRegionInfo regionInfo;
     public final Put put;
     public final int retryCount;
@@ -391,7 +437,8 @@ public class HTableMultiplexer {
     }
   }
 
-  private static class FlushWorker implements Runnable {
+  @VisibleForTesting
+  static class FlushWorker implements Runnable {
     private final HRegionLocation addr;
     private final LinkedBlockingQueue<PutStatus> queue;
     private final HTableMultiplexer multiplexer;
@@ -439,7 +486,7 @@ public class HTableMultiplexer {
       return this.maxLatency.getAndSet(0);
     }
 
-    private boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
+    boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
       // Decrease the retry count
       final int retryCount = ps.retryCount - 1;
 
@@ -448,10 +495,10 @@ public class HTableMultiplexer {
         return false;
       }
 
-      int cnt = retryInQueue.incrementAndGet();
-      if (cnt > maxRetryInQueue) {
+      int cnt = getRetryInQueue().incrementAndGet();
+      if (cnt > getMaxRetryInQueue()) {
         // Too many Puts in queue for resubmit, give up this
-        retryInQueue.decrementAndGet();
+        getRetryInQueue().decrementAndGet();
         return false;
       }
 
@@ -459,22 +506,21 @@ public class HTableMultiplexer {
       // The currentPut is failed. So get the table name for the currentPut.
       final TableName tableName = ps.regionInfo.getTable();
 
-      long delayMs = ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
-        multiplexer.retryNum - retryCount - 1);
+      long delayMs = getNextDelay(retryCount);
       if (LOG.isDebugEnabled()) {
         LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
       }
 
-      executor.schedule(new Runnable() {
+      getExecutor().schedule(new Runnable() {
         @Override
         public void run() {
           boolean succ = false;
           try {
-            succ = FlushWorker.this.multiplexer.put(tableName, failedPut, retryCount);
+            succ = FlushWorker.this.getMultiplexer()._put(tableName, failedPut, retryCount, true);
           } finally {
-            FlushWorker.this.retryInQueue.decrementAndGet();
+            FlushWorker.this.getRetryInQueue().decrementAndGet();
             if (!succ) {
-              FlushWorker.this.totalFailedPutCount.incrementAndGet();
+              FlushWorker.this.getTotalFailedPutCount().incrementAndGet();
             }
           }
         }
@@ -482,6 +528,37 @@ public class HTableMultiplexer {
       return true;
     }
 
+    @VisibleForTesting
+    long getNextDelay(int retryCount) {
+      return ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
+          multiplexer.retryNum - retryCount - 1);
+    }
+
+    @VisibleForTesting
+    AtomicInteger getRetryInQueue() {
+      return this.retryInQueue;
+    }
+
+    @VisibleForTesting
+    int getMaxRetryInQueue() {
+      return this.maxRetryInQueue;
+    }
+
+    @VisibleForTesting
+    AtomicLong getTotalFailedPutCount() {
+      return this.totalFailedPutCount;
+    }
+
+    @VisibleForTesting
+    HTableMultiplexer getMultiplexer() {
+      return this.multiplexer;
+    }
+
+    @VisibleForTesting
+    ScheduledExecutorService getExecutor() {
+      return this.executor;
+    }
+
     @Override
     public void run() {
       int failedCount = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/25878e1a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
new file mode 100644
index 0000000..38ddeb9
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerViaMocks.java
@@ -0,0 +1,193 @@
+/*
+ * 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.client;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTableMultiplexer.FlushWorker;
+import org.apache.hadoop.hbase.client.HTableMultiplexer.PutStatus;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@Category(SmallTests.class)
+public class TestHTableMultiplexerViaMocks {
+
+  private static final int NUM_RETRIES = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
+  private HTableMultiplexer mockMultiplexer;
+  private ClusterConnection mockConnection;
+  private HRegionLocation mockRegionLocation;
+  private HRegionInfo mockRegionInfo;
+
+  private TableName tableName;
+  private Put put;
+
+  @Before
+  public void setupTest() {
+    mockMultiplexer = mock(HTableMultiplexer.class);
+    mockConnection = mock(ClusterConnection.class);
+    mockRegionLocation = mock(HRegionLocation.class);
+    mockRegionInfo = mock(HRegionInfo.class);
+
+    tableName = TableName.valueOf("my_table");
+    put = new Put(getBytes("row1"));
+    put.addColumn(getBytes("f1"), getBytes("q1"), getBytes("v11"));
+    put.addColumn(getBytes("f1"), getBytes("q2"), getBytes("v12"));
+    put.addColumn(getBytes("f2"), getBytes("q1"), getBytes("v21"));
+
+    // Call the real put(TableName, Put, int) method
+    when(mockMultiplexer.put(any(TableName.class), any(Put.class), anyInt())).thenCallRealMethod();
+
+    // Return the mocked ClusterConnection
+    when(mockMultiplexer.getConnection()).thenReturn(mockConnection);
+
+    // Return the regionInfo from the region location
+    when(mockRegionLocation.getRegionInfo()).thenReturn(mockRegionInfo);
+
+    // Make sure this RegionInfo points to our table
+    when(mockRegionInfo.getTable()).thenReturn(tableName);
+  }
+
+  @Test public void useCacheOnInitialPut() throws Exception {
+    mockMultiplexer.put(tableName, put, NUM_RETRIES);
+
+    verify(mockMultiplexer)._put(tableName, put, NUM_RETRIES, false);
+  }
+
+  @Test public void nonNullLocationQueuesPut() throws Exception {
+    final LinkedBlockingQueue<PutStatus> queue = new LinkedBlockingQueue<>();
+
+    // Call the real method for _put(TableName, Put, int, boolean)
+    when(mockMultiplexer._put(any(TableName.class), any(Put.class), anyInt(), anyBoolean())).thenCallRealMethod();
+
+    // Return a region location
+    when(mockConnection.getRegionLocation(tableName, put.getRow(), false)).thenReturn(mockRegionLocation);
+    when(mockMultiplexer.getQueue(mockRegionLocation)).thenReturn(queue);
+
+    assertTrue("Put should have been queued", mockMultiplexer.put(tableName, put, NUM_RETRIES));
+
+    assertEquals(1, queue.size());
+    final PutStatus ps = queue.take();
+    assertEquals(put, ps.put);
+    assertEquals(mockRegionInfo, ps.regionInfo);
+  }
+
+  @Test public void ignoreCacheOnRetriedPut() throws Exception {
+    FlushWorker mockFlushWorker = mock(FlushWorker.class);
+    ScheduledExecutorService mockExecutor = mock(ScheduledExecutorService.class);
+    final AtomicInteger retryInQueue = new AtomicInteger(0);
+    final AtomicLong totalFailedPuts = new AtomicLong(0L);
+    final int maxRetryInQueue = 20;
+    final long delay = 100L;
+
+    final PutStatus ps = new PutStatus(mockRegionInfo, put, NUM_RETRIES);
+
+    // Call the real resubmitFailedPut(PutStatus, HRegionLocation) method
+    when(mockFlushWorker.resubmitFailedPut(any(PutStatus.class), any(HRegionLocation.class))).thenCallRealMethod();
+    // Succeed on the re-submit without caching
+    when(mockMultiplexer._put(tableName, put, NUM_RETRIES - 1, true)).thenReturn(true);
+
+    // Stub out the getters for resubmitFailedPut(PutStatus, HRegionLocation)
+    when(mockFlushWorker.getExecutor()).thenReturn(mockExecutor);
+    when(mockFlushWorker.getNextDelay(anyInt())).thenReturn(delay);
+    when(mockFlushWorker.getMultiplexer()).thenReturn(mockMultiplexer);
+    when(mockFlushWorker.getRetryInQueue()).thenReturn(retryInQueue);
+    when(mockFlushWorker.getMaxRetryInQueue()).thenReturn(maxRetryInQueue);
+    when(mockFlushWorker.getTotalFailedPutCount()).thenReturn(totalFailedPuts);
+
+    // When a Runnable is scheduled, run that Runnable
+    when(mockExecutor.schedule(any(Runnable.class), eq(delay), eq(TimeUnit.MILLISECONDS))).thenAnswer(
+        new Answer<Void>() {
+          @Override
+          public Void answer(InvocationOnMock invocation) throws Throwable {
+            // Before we run this, should have one retry in progress.
+            assertEquals(1L, retryInQueue.get());
+
+            Object[] args = invocation.getArguments();
+            assertEquals(3, args.length);
+            assertTrue("Argument should be an instance of Runnable", args[0] instanceof Runnable);
+            Runnable runnable = (Runnable) args[0];
+            runnable.run();
+            return null;
+          }
+        });
+
+    // The put should be rescheduled
+    assertTrue("Put should have been rescheduled", mockFlushWorker.resubmitFailedPut(ps, mockRegionLocation));
+
+    verify(mockMultiplexer)._put(tableName, put, NUM_RETRIES - 1, true);
+    assertEquals(0L, totalFailedPuts.get());
+    // Net result should be zero (added one before rerunning, subtracted one after running).
+    assertEquals(0L, retryInQueue.get());
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test public void testConnectionClosing() throws IOException {
+    doCallRealMethod().when(mockMultiplexer).close();
+    // If the connection is not closed
+    when(mockConnection.isClosed()).thenReturn(false);
+
+    mockMultiplexer.close();
+
+    // We should close it
+    verify(mockConnection).close();
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test public void testClosingAlreadyClosedConnection() throws IOException {
+    doCallRealMethod().when(mockMultiplexer).close();
+    // If the connection is already closed
+    when(mockConnection.isClosed()).thenReturn(true);
+
+    mockMultiplexer.close();
+
+    // We should not close it again
+    verify(mockConnection, times(0)).close();
+  }
+
+  /**
+   * @return UTF-8 byte representation for {@code str}
+   */
+  private static byte[] getBytes(String str) {
+    return str.getBytes(UTF_8);
+  }
+}


[2/2] hbase git commit: HBASE-14025 update CHANGES.txt for 1.2 RC2

Posted by bu...@apache.org.
HBASE-14025 update CHANGES.txt for 1.2 RC2


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

Branch: refs/heads/branch-1.2
Commit: d568db8372a3fbc6b93c5854749c30276ba19ba4
Parents: 25878e1
Author: Sean Busbey <bu...@cloudera.com>
Authored: Sun Feb 7 23:38:10 2016 -0600
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Mon Feb 8 00:46:28 2016 -0600

----------------------------------------------------------------------
 CHANGES.txt | 42 ++++++++++++++++++++++++++++++------------
 1 file changed, 30 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d568db83/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3438d58..e1ae64f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,6 +1,6 @@
 HBase Change Log
 
-Release Notes - HBase - Version 1.2.0 02/05/2016
+Release Notes - HBase - Version 1.2.0 02/15/2016
 
 ** Sub-task
     * [HBASE-12748] - RegionCoprocessorHost.execOperation creates too many iterator objects
@@ -9,7 +9,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-13470] - High level Integration test for master DDL operations
     * [HBASE-13476] - Procedure V2 - Add Replay Order logic for child procedures
     * [HBASE-13497] - Remove MVCC stamps from HFile when that is safe
-    * [HBASE-13536] - Cleanup the handlers that are no longer being used. 
+    * [HBASE-13536] - Cleanup the handlers that are no longer being used.
     * [HBASE-13563] - Add missing table owner to AC tests.
     * [HBASE-13569] - correct errors reported with mvn site
     * [HBASE-13579] - Avoid isCellTTLExpired() for NO-TAG cases
@@ -24,14 +24,14 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-13899] - Jacoco instrumentation fails under jdk8
     * [HBASE-13912] - add branch-1.2 post-commit builds
     * [HBASE-13920] - Exclude Java files generated from protobuf from javadoc
-    * [HBASE-13937] - Partially revert HBASE-13172 
+    * [HBASE-13937] - Partially revert HBASE-13172
     * [HBASE-13950] - Add a NoopProcedureStore for testing
     * [HBASE-13963] - avoid leaking jdk.tools
     * [HBASE-13967] - add jdk profiles for jdk.tools dependency
     * [HBASE-13973] - Update documentation for 10070 Phase 2 changes
     * [HBASE-13983] - Doc how the oddball HTable methods getStartKey, getEndKey, etc. will be removed in 2.0.0
     * [HBASE-13990] - clean up remaining errors for maven site goal
-    * [HBASE-13993] - WALProcedureStore fencing is not effective if new WAL rolls 
+    * [HBASE-13993] - WALProcedureStore fencing is not effective if new WAL rolls
     * [HBASE-14003] - work around jdk8 spec bug in WALPerfEval
     * [HBASE-14013] - Retry when RegionServerNotYetRunningException rather than go ahead with assign so for sure we don't skip WAL replay
     * [HBASE-14017] - Procedure v2 - MasterProcedureQueue fix concurrency issue on table queue deletion
@@ -65,7 +65,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-14488] - Procedure V2 - shell command to abort a procedure
     * [HBASE-14513] - TestBucketCache runs obnoxious 1k threads in a unit test
     * [HBASE-14519] - Purge TestFavoredNodeAssignmentHelper, a test for an abandoned feature that can hang
-    * [HBASE-14535] - Integration test for rpc connection concurrency / deadlock testing 
+    * [HBASE-14535] - Integration test for rpc connection concurrency / deadlock testing
     * [HBASE-14539] - Slight improvement of StoreScanner.optimize
     * [HBASE-14559] - branch-1 test tweeks; disable assert explicit region lands post-restart and up a few handlers
     * [HBASE-14561] - Disable zombie TestReplicationShell
@@ -93,6 +93,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-14794] - Cleanup TestAtomicOperation, TestImportExport, and TestMetaWithReplicas
     * [HBASE-14798] - NPE reporting server load causes regionserver abort; causes TestAcidGuarantee to fail
     * [HBASE-14819] - hbase-it tests failing with OOME; permgen
+    * [HBASE-14837] - Procedure V2 - Procedure Queue Improvement
     * [HBASE-14863] - Add missing test/resources/log4j files in hbase modules
     * [HBASE-14883] - TestSplitTransactionOnCluster#testFailedSplit flakey
     * [HBASE-14908] - TestRowCounter flakey especially on branch-1
@@ -107,6 +108,8 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-15114] - NPE when IPC server ByteBuffer reservoir is turned off
     * [HBASE-15115] - Fix findbugs complaints in hbase-client
     * [HBASE-15118] - Fix findbugs complaint in hbase-server
+    * [HBASE-15157] - Add *PerformanceTest for Append, CheckAnd*
+    * [HBASE-15210] - Undo aggressive load balancer logging at tens of lines per millisecond
 
 ** Bug
     * [HBASE-5878] - Use getVisibleLength public api from HdfsDataInputStream from Hadoop-2.
@@ -183,7 +186,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-13767] - Allow ZKAclReset to set and not just clear ZK ACLs
     * [HBASE-13768] - ZooKeeper znodes are bootstrapped with insecure ACLs in a secure configuration
     * [HBASE-13770] - Programmatic JAAS configuration option for secure zookeeper may be broken
-    * [HBASE-13776] - Setting illegal versions for HColumnDescriptor does not throw IllegalArgumentException 
+    * [HBASE-13776] - Setting illegal versions for HColumnDescriptor does not throw IllegalArgumentException
     * [HBASE-13777] - Table fragmentation display triggers NPE on master status page
     * [HBASE-13778] - BoundedByteBufferPool incorrectly increasing runningAverage buffer length
     * [HBASE-13779] - Calling table.exists() before table.get() end up with an empty Result
@@ -243,7 +246,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-13970] - NPE during compaction in trunk
     * [HBASE-13971] - Flushes stuck since 6 hours on a regionserver.
     * [HBASE-13974] - TestRateLimiter#testFixedIntervalResourceAvailability may fail
-    * [HBASE-13978] - Variable never assigned in SimpleTotalOrderPartitioner.getPartition() 
+    * [HBASE-13978] - Variable never assigned in SimpleTotalOrderPartitioner.getPartition()
     * [HBASE-13982] - Add info for visibility labels/cell TTLs to ImportTsv
     * [HBASE-13988] - Add exception handler for lease thread
     * [HBASE-13989] - Threshold for combined MemStore and BlockCache percentages is not checked
@@ -281,7 +284,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-14206] - MultiRowRangeFilter returns records whose rowKeys are out of allowed ranges
     * [HBASE-14209] - TestShell visibility tests failing
     * [HBASE-14211] - Add more rigorous integration tests of splits
-    * [HBASE-14214] - list_labels shouldn't raise ArgumentError if no labels are defined 
+    * [HBASE-14214] - list_labels shouldn't raise ArgumentError if no labels are defined
     * [HBASE-14219] - src tgz no longer builds after HBASE-14085
     * [HBASE-14224] - Fix coprocessor handling of duplicate classes
     * [HBASE-14228] - Close BufferedMutator and connection in MultiTableOutputFormat
@@ -334,7 +337,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-14469] - Fix some comment, validation and logging around memstore lower limit configuration
     * [HBASE-14471] - Thrift -  HTTP Error 413 full HEAD if using kerberos authentication
     * [HBASE-14473] - Compute region locality in parallel
-    * [HBASE-14474] - DeadLock in RpcClientImpl.Connection.close() 
+    * [HBASE-14474] - DeadLock in RpcClientImpl.Connection.close()
     * [HBASE-14475] - Region split requests are always audited with "hbase" user rather than request user
     * [HBASE-14486] - Disable TestRegionPlacement, a flakey test for an unfinished feature
     * [HBASE-14489] - postScannerFilterRow consumes a lot of CPU
@@ -399,13 +402,14 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-14784] - Port conflict is not resolved in HBaseTestingUtility.randomFreePort()
     * [HBASE-14788] - Splitting a region does not support the hbase.rs.evictblocksonclose config when closing source region
     * [HBASE-14793] - Allow limiting size of block into L1 block cache.
-    * [HBASE-14799] - Commons-collections object deserialization remote command execution vulnerability 
+    * [HBASE-14799] - Commons-collections object deserialization remote command execution vulnerability
     * [HBASE-14802] - Replaying server crash recovery procedure after a failover causes incorrect handling of deadservers
     * [HBASE-14804] - HBase shell's create table command ignores 'NORMALIZATION_ENABLED' attribute
     * [HBASE-14806] - Missing sources.jar for several modules when building HBase
     * [HBASE-14807] - TestWALLockup is flakey
-    * [HBASE-14809] - Grant / revoke Namespace admin permission to group 
+    * [HBASE-14809] - Grant / revoke Namespace admin permission to group
     * [HBASE-14812] - Fix ResultBoundedCompletionService deadlock
+    * [HBASE-14822] - Renewing leases of scanners doesn't work
     * [HBASE-14824] - HBaseAdmin.mergeRegions should recognize both full region names and encoded region names
     * [HBASE-14838] - Clarify that SimpleRegionNormalizer does not merge empty (<1MB) regions
     * [HBASE-14840] - Sink cluster reports data replication request as success though the data is not replicated
@@ -419,7 +423,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-14904] - Mark Base[En|De]coder LimitedPrivate and fix binary compat issue
     * [HBASE-14905] - VerifyReplication does not honour versions option
     * [HBASE-14922] - Delayed flush doesn't work causing flush storms.
-    * [HBASE-14923] - VerifyReplication should not mask the exception during result comparison 
+    * [HBASE-14923] - VerifyReplication should not mask the exception during result comparison
     * [HBASE-14926] - Hung ThriftServer; no timeout on read from client; if client crashes, worker thread gets stuck reading
     * [HBASE-14928] - Start row should be set for query through HBase REST gateway involving globbing option
     * [HBASE-14929] - There is a space missing from Table "foo" is not currently available.
@@ -436,6 +440,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-14968] - ConcurrentModificationException in region close resulting in the region staying in closing state
     * [HBASE-14974] - Total number of Regions in Transition number on UI incorrect
     * [HBASE-14977] - ChoreService.shutdown may result in ConcurrentModificationException
+    * [HBASE-14987] - Compaction marker whose region name doesn't match current region's needs to be handled
     * [HBASE-14989] - Implementation of Mutation.getWriteToWAL() is backwards
     * [HBASE-14999] - Remove ref to org.mortbay.log.Log
     * [HBASE-15001] - Thread Safety issues in ReplicationSinkManager and HBaseInterClusterReplicationEndpoint
@@ -450,9 +455,12 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-15032] - hbase shell scan filter string assumes UTF-8 encoding
     * [HBASE-15035] - bulkloading hfiles with tags that require splits do not preserve tags
     * [HBASE-15039] - HMaster and RegionServers should try to refresh token keys from zk when facing InvalidToken
+    * [HBASE-15052] - Use EnvironmentEdgeManager in ReplicationSource
     * [HBASE-15057] - local-master-backup.sh doesn't start HMaster correctly
+    * [HBASE-15058] - AssignmentManager should account for unsuccessful split correctly which initially passes quota check
     * [HBASE-15065] - SimpleRegionNormalizer should return multiple normalization plans in one run
     * [HBASE-15079] - TestMultiParallel.validateLoadedData AssertionError: null
+    * [HBASE-15083] - Gets from Multiactions are not counted in metrics for gets.
     * [HBASE-15085] - IllegalStateException was thrown when scanning on bulkloaded HFiles
     * [HBASE-15098] - Normalizer switch in configuration is not used
     * [HBASE-15100] - Master WALProcs still never clean up
@@ -464,6 +472,14 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-15146] - Don't block on Reader threads queueing to a scheduler queue
     * [HBASE-15147] - Shell should use Admin.listTableNames() instead of Admin.listTables()
     * [HBASE-15148] - Resolve IS2_INCONSISTENT_SYNC findbugs warning in AuthenticationTokenSecretManager
+    * [HBASE-15152] - Automatically include prefix-tree module in MR jobs if present
+    * [HBASE-15190] - Monkey dies when running on shared cluster (gives up when can't kill the other fellows processes)
+    * [HBASE-15200] - ZooKeeper znode ACL checks should only compare the shortname
+    * [HBASE-15206] - Flakey testSplitDaughtersNotInMeta test
+    * [HBASE-15209] - disable table in HBaseTestingUtility.truncateTable
+    * [HBASE-15214] - Valid mutate Ops fail with RPC Codec in use and region moves across
+    * [HBASE-15218] - On RS crash and replay of WAL, loosing all Tags in Cells
+    * [HBASE-15221] - HTableMultiplexer improvements (stale region locations and resource leaks)
 
 ** Improvement
     * [HBASE-6617] - ReplicationSourceManager should be able to track multiple WAL paths
@@ -479,6 +495,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-13358] - Upgrade VisibilityClient API to accept Connection object.
     * [HBASE-13366] - Throw DoNotRetryIOException instead of read only IOException
     * [HBASE-13375] - Provide HBase superuser higher priority over other users in the RPC handling
+    * [HBASE-13376] - Improvements to Stochastic load balancer
     * [HBASE-13420] - RegionEnvironment.offerExecutionLatency Blocks Threads under Heavy Load
     * [HBASE-13534] - Change HBase master WebUI to explicitly mention if it is a backup master
     * [HBASE-13598] - Make hbase assembly 'attach' to the project
@@ -606,6 +623,7 @@ Release Notes - HBase - Version 1.2.0 02/05/2016
     * [HBASE-14851] - Add test showing how to use TTL from thrift
     * [HBASE-15003] - Remove BoundedConcurrentLinkedQueue and associated test
     * [HBASE-15099] - Move RegionStateListener class out of quotas package
+    * [HBASE-15220] - Change two logs in SimpleRegionNormalizer to INFO level
 
 ** Test
     * [HBASE-13591] - TestHBaseFsck is flakey