http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 095f4bd..d56d6ec 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -116,7 +116,6 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterAllFilter;
 import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.NullComparator;
@@ -4932,7 +4931,6 @@ public class TestHRegion {
       String callingMethod, Configuration conf, boolean isReadOnly, byte[]... 
families)
       throws IOException {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
     HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);
     return initHRegion(tableName, startKey, stopKey, isReadOnly,

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 6eed7df..0054642 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -153,7 +153,7 @@ public class TestHRegionReplayEvents {
     }
 
     time = System.currentTimeMillis();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
+
     primaryHri = new HRegionInfo(htd.getTableName(),
       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
       false, time, 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
index 1768801..37a7664 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreChunkPool.java
@@ -48,30 +48,30 @@ import static org.junit.Assert.assertTrue;
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestMemStoreChunkPool {
   private final static Configuration conf = new Configuration();
-  private static ChunkCreator chunkCreator;
+  private static MemStoreChunkPool chunkPool;
   private static boolean chunkPoolDisabledBeforeTest;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
     conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
-    chunkPoolDisabledBeforeTest = ChunkCreator.chunkPoolDisabled;
-    ChunkCreator.chunkPoolDisabled = false;
+    chunkPoolDisabledBeforeTest = MemStoreChunkPool.chunkPoolDisabled;
+    MemStoreChunkPool.chunkPoolDisabled = false;
     long globalMemStoreLimit = (long) 
(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    chunkCreator = ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 
false,
-      globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
-    assertTrue(chunkCreator != null);
+    chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
+        MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, 
MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
+    assertTrue(chunkPool != null);
   }
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    ChunkCreator.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
+    MemStoreChunkPool.chunkPoolDisabled = chunkPoolDisabledBeforeTest;
   }
 
   @Before
   public void tearDown() throws Exception {
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
   }
 
   @Test
@@ -90,7 +90,7 @@ public class TestMemStoreChunkPool {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        expectedOff = 8;
+        expectedOff = 0;
         lastBuffer = newKv.getBuffer();
       }
       assertEquals(expectedOff, newKv.getOffset());
@@ -100,14 +100,14 @@ public class TestMemStoreChunkPool {
     }
     // chunks will be put back to pool after close
     mslab.close();
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
     // reconstruct mslab
     mslab = new MemStoreLABImpl(conf);
     // chunk should be got from the pool, so we can reuse it.
     KeyValue kv = new KeyValue(rk, cf, q, new byte[10]);
     mslab.copyCellInto(kv);
-    assertEquals(chunkCount - 1, chunkCreator.getPoolSize());
+    assertEquals(chunkCount - 1, chunkPool.getPoolSize());
   }
 
   @Test
@@ -143,7 +143,7 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    int chunkCount = chunkCreator.getPoolSize();
+    int chunkCount = chunkPool.getPoolSize();
     assertTrue(chunkCount > 0);
 
   }
@@ -189,16 +189,16 @@ public class TestMemStoreChunkPool {
     }
     memstore.clearSnapshot(snapshot.getId());
 
-    assertTrue(chunkCreator.getPoolSize() == 0);
+    assertTrue(chunkPool.getPoolSize() == 0);
 
     // Chunks will be put back to pool after close scanners;
     for (KeyValueScanner scanner : scanners) {
       scanner.close();
     }
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
 
     // clear chunks
-    chunkCreator.clearChunksInPool();
+    chunkPool.clearChunks();
 
     // Creating another snapshot
     snapshot = memstore.snapshot();
@@ -218,20 +218,20 @@ public class TestMemStoreChunkPool {
       scanner.close();
     }
     memstore.clearSnapshot(snapshot.getId());
-    assertTrue(chunkCreator.getPoolSize() > 0);
+    assertTrue(chunkPool.getPoolSize() > 0);
   }
 
   @Test
   public void testPutbackChunksMultiThreaded() throws Exception {
+    MemStoreChunkPool oldPool = MemStoreChunkPool.GLOBAL_INSTANCE;
     final int maxCount = 10;
     final int initialCount = 5;
-    final int chunkSize = 40;
+    final int chunkSize = 30;
     final int valSize = 7;
-    ChunkCreator oldCreator = ChunkCreator.getInstance();
-    ChunkCreator newCreator = new ChunkCreator(chunkSize, false, 400, 1, 0.5f, 
null);
-    assertEquals(initialCount, newCreator.getPoolSize());
-    assertEquals(maxCount, newCreator.getMaxCount());
-    ChunkCreator.INSTANCE = newCreator;// Replace the global ref with the new 
one we created.
+    MemStoreChunkPool pool = new MemStoreChunkPool(chunkSize, maxCount, 
initialCount, 1, false);
+    assertEquals(initialCount, pool.getPoolSize());
+    assertEquals(maxCount, pool.getMaxCount());
+    MemStoreChunkPool.GLOBAL_INSTANCE = pool;// Replace the global ref with 
the new one we created.
                                              // Used it for the testing. Later 
in finally we put
                                              // back the original
     final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), 
Bytes.toBytes("q"),
@@ -258,9 +258,9 @@ public class TestMemStoreChunkPool {
       t1.join();
       t2.join();
       t3.join();
-      assertTrue(newCreator.getPoolSize() <= maxCount);
+      assertTrue(pool.getPoolSize() <= maxCount);
     } finally {
-      ChunkCreator.INSTANCE = oldCreator;
+      MemStoreChunkPool.GLOBAL_INSTANCE = oldPool;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
index 6696e43..141b802 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStoreLAB.java
@@ -63,8 +63,8 @@ public class TestMemStoreLAB {
   public static void setUpBeforeClass() throws Exception {
     long globalMemStoreLimit = (long) 
(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
         .getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 
globalMemStoreLimit,
-      0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
+    MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f, 
MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT,
+        MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
   }
 
   /**
@@ -76,7 +76,6 @@ public class TestMemStoreLAB {
     MemStoreLAB mslab = new MemStoreLABImpl();
     int expectedOff = 0;
     ByteBuffer lastBuffer = null;
-    long lastChunkId = -1;
     // 100K iterations by 0-1K alloc -> 50MB expected
     // should be reasonable for unit test and also cover wraparound
     // behavior
@@ -86,13 +85,8 @@ public class TestMemStoreLAB {
       int size = KeyValueUtil.length(kv);
       ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
       if (newKv.getBuffer() != lastBuffer) {
-        // since we add the chunkID at the 0th offset of the chunk and the
-        // chunkid is a long we need to account for those 8 bytes
-        expectedOff = Bytes.SIZEOF_LONG;
+        expectedOff = 0;
         lastBuffer = newKv.getBuffer();
-        long chunkId = newKv.getBuffer().getLong(0);
-        assertTrue("chunkid should be different", chunkId != lastChunkId);
-        lastChunkId = chunkId;
       }
       assertEquals(expectedOff, newKv.getOffset());
       assertTrue("Allocation overruns buffer",
@@ -142,21 +136,23 @@ public class TestMemStoreLAB {
       };
       ctx.addThread(t);
     }
-
+    
     ctx.startThreads();
     while (totalAllocated.get() < 50*1024*1024 && ctx.shouldRun()) {
       Thread.sleep(10);
     }
     ctx.stop();
+    
     // Partition the allocations by the actual byte[] they point into,
     // make sure offsets are unique for each chunk
     Map<ByteBuffer, Map<Integer, AllocRecord>> mapsByChunk =
       Maps.newHashMap();
-
+    
     int sizeCounted = 0;
     for (AllocRecord rec : Iterables.concat(allocations)) {
       sizeCounted += rec.size;
       if (rec.size == 0) continue;
+      
       Map<Integer, AllocRecord> mapForThisByteArray =
         mapsByChunk.get(rec.alloc);
       if (mapForThisByteArray == null) {
@@ -171,9 +167,7 @@ public class TestMemStoreLAB {
     
     // Now check each byte array to make sure allocations don't overlap
     for (Map<Integer, AllocRecord> allocsInChunk : mapsByChunk.values()) {
-      // since we add the chunkID at the 0th offset of the chunk and the
-      // chunkid is a long we need to account for those 8 bytes
-      int expectedOff = Bytes.SIZEOF_LONG;
+      int expectedOff = 0;
       for (AllocRecord alloc : allocsInChunk.values()) {
         assertEquals(expectedOff, alloc.offset);
         assertTrue("Allocation overruns buffer",
@@ -181,6 +175,7 @@ public class TestMemStoreLAB {
         expectedOff += alloc.size;
       }
     }
+
   }
 
   /**
@@ -199,7 +194,7 @@ public class TestMemStoreLAB {
     // set chunk size to default max alloc size, so we could easily trigger 
chunk retirement
     conf.setLong(MemStoreLABImpl.CHUNK_SIZE_KEY, 
MemStoreLABImpl.MAX_ALLOC_DEFAULT);
     // reconstruct mslab
-    ChunkCreator.clearDisableFlag();
+    MemStoreChunkPool.clearDisableFlag();
     mslab = new MemStoreLABImpl(conf);
     // launch multiple threads to trigger frequent chunk retirement
     List<Thread> threads = new ArrayList<>();
@@ -228,8 +223,6 @@ public class TestMemStoreLAB {
     }
     // close the mslab
     mslab.close();
-    // none of the chunkIds would have been returned back
-    assertTrue("All the chunks must have been cleared", 
ChunkCreator.INSTANCE.size() != 0);
     // make sure all chunks reclaimed or removed from chunk queue
     int queueLength = mslab.getPooledChunks().size();
     assertTrue("All chunks in chunk queue should be reclaimed or removed"

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
deleted file mode 100644
index f38a75e..0000000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemstoreLABWithoutPool.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/**
- *
- * 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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.lang.management.ManagementFactory;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({RegionServerTests.class, SmallTests.class})
-public class TestMemstoreLABWithoutPool {
-  private final static Configuration conf = new Configuration();
-
-  private static final byte[] rk = Bytes.toBytes("r1");
-  private static final byte[] cf = Bytes.toBytes("f");
-  private static final byte[] q = Bytes.toBytes("q");
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    long globalMemStoreLimit = (long) 
(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
-        .getMax() * 0.8);
-    // disable pool
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT + 
Bytes.SIZEOF_LONG, false, globalMemStoreLimit,
-      0.0f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, null);
-  }
-
-  /**
-   * Test a bunch of random allocations
-   */
-  @Test
-  public void testLABRandomAllocation() {
-    Random rand = new Random();
-    MemStoreLAB mslab = new MemStoreLABImpl();
-    int expectedOff = 0;
-    ByteBuffer lastBuffer = null;
-    long lastChunkId = -1;
-    // 100K iterations by 0-1K alloc -> 50MB expected
-    // should be reasonable for unit test and also cover wraparound
-    // behavior
-    for (int i = 0; i < 100000; i++) {
-      int valSize = rand.nextInt(1000);
-      KeyValue kv = new KeyValue(rk, cf, q, new byte[valSize]);
-      int size = KeyValueUtil.length(kv);
-      ByteBufferKeyValue newKv = (ByteBufferKeyValue) mslab.copyCellInto(kv);
-      if (newKv.getBuffer() != lastBuffer) {
-        // since we add the chunkID at the 0th offset of the chunk and the
-        // chunkid is a long we need to account for those 8 bytes
-        expectedOff = Bytes.SIZEOF_LONG;
-        lastBuffer = newKv.getBuffer();
-        long chunkId = newKv.getBuffer().getLong(0);
-        assertTrue("chunkid should be different", chunkId != lastChunkId);
-        lastChunkId = chunkId;
-      }
-      assertEquals(expectedOff, newKv.getOffset());
-      assertTrue("Allocation overruns buffer",
-          newKv.getOffset() + size <= newKv.getBuffer().capacity());
-      expectedOff += size;
-    }
-  }
-
-  /**
-   * Test frequent chunk retirement with chunk pool triggered by lots of 
threads, making sure
-   * there's no memory leak (HBASE-16195)
-   * @throws Exception if any error occurred
-   */
-  @Test
-  public void testLABChunkQueueWithMultipleMSLABs() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    MemStoreLABImpl[] mslab = new MemStoreLABImpl[10];
-    for (int i = 0; i < 10; i++) {
-      mslab[i] = new MemStoreLABImpl(conf);
-    }
-    // launch multiple threads to trigger frequent chunk retirement
-    List<Thread> threads = new ArrayList<>();
-    // create smaller sized kvs
-    final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), 
Bytes.toBytes("q"),
-        new byte[0]);
-    for (int i = 0; i < 10; i++) {
-      for (int j = 0; j < 10; j++) {
-        threads.add(getChunkQueueTestThread(mslab[i], "testLABChunkQueue-" + 
j, kv));
-      }
-    }
-    for (Thread thread : threads) {
-      thread.start();
-    }
-    // let it run for some time
-    Thread.sleep(3000);
-    for (Thread thread : threads) {
-      thread.interrupt();
-    }
-    boolean threadsRunning = true;
-    boolean alive = false;
-    while (threadsRunning) {
-      alive = false;
-      for (Thread thread : threads) {
-        if (thread.isAlive()) {
-          alive = true;
-          break;
-        }
-      }
-      if (!alive) {
-        threadsRunning = false;
-      }
-    }
-    // close the mslab
-    for (int i = 0; i < 10; i++) {
-      mslab[i].close();
-    }
-    // all of the chunkIds would have been returned back
-    assertTrue("All the chunks must have been cleared", 
ChunkCreator.INSTANCE.size() == 0);
-  }
-
-  private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String 
threadName,
-      Cell cellToCopyInto) {
-    Thread thread = new Thread() {
-      boolean stopped = false;
-
-      @Override
-      public void run() {
-        while (!stopped) {
-          // keep triggering chunk retirement
-          mslab.copyCellInto(cellToCopyInto);
-        }
-      }
-
-      @Override
-      public void interrupt() {
-        this.stopped = true;
-      }
-    };
-    thread.setName(threadName);
-    thread.setDaemon(true);
-    return thread;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index 7160e5e..4315bd4 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -108,7 +108,6 @@ public class TestRecoveredEdits {
       }
     };
     Path hbaseRootDir = TEST_UTIL.getDataTestDir();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
     Path tableDir = FSUtils.getTableDir(hbaseRootDir, htd.getTableName());
     HRegionFileSystem hrfs =

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
index ad56081..5d11c0e 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
@@ -83,7 +83,6 @@ public class TestRegionIncrement {
   private HRegion getRegion(final Configuration conf, final String tableName) 
throws IOException {
     WAL wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
       TEST_UTIL.getDataTestDir().toString(), conf);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
     return (HRegion)TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName),
       HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, tableName, 
conf,
       false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
index 5355c77..0d339b1 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
@@ -178,7 +178,6 @@ public class TestStore {
     } else {
       htd.addFamily(hcd);
     }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 99dd00d..3cdb227 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -111,7 +111,6 @@ public class TestStoreFileRefresherChore {
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
     HRegion region =
         new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), 
info.getTable().getNamespace()),
             conf, htd, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 51260a6..4f247b0 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -588,7 +588,6 @@ public class TestWALLockup {
    */
   public static HRegion initHRegion(TableName tableName, byte[] startKey, 
byte[] stopKey, WAL wal)
   throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, false, 
Durability.SYNC_WAL,
       wal, COLUMN_FAMILY_BYTES);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index e63bad9..994779f 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -98,7 +98,6 @@ public class TestWALMonotonicallyIncreasingSeqId {
     FSUtils.setRootDir(walConf, tableDir);
     this.walConf = walConf;
     wals = new WALFactory(walConf, null, "log_" + replicaId);
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
     HRegion region = new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(),
         info.getTable().getNamespace()), conf, htd, null);
     region.initialize();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ecdfb823/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index 057b9bf..f976b49 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -37,9 +37,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -290,7 +288,6 @@ public class TestDurability {
           throw new IOException("Failed delete of " + path);
         }
       }
-      ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 
0, null);
       return HRegion.createHRegion(info, path, CONF, htd, log);
     }
 

Reply via email to