This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new c7c64c092d1 HBASE-29123: A faster CodecPool for HBase (#6679)
c7c64c092d1 is described below

commit c7c64c092d1d450adfff49472ae9f7c7afbf5c00
Author: Charles Connell <[email protected]>
AuthorDate: Tue Feb 18 04:35:07 2025 -0500

    HBASE-29123: A faster CodecPool for HBase (#6679)
    
    Signed-off-by: Wellington Ramos Chevreuil <[email protected]>
    Signed-off-by: Ray Mattingly <[email protected]>
    Signed-off-by: Nick Dimiduk <[email protected]>
---
 .../apache/hadoop/hbase/ipc/CellBlockBuilder.java  |   2 +-
 hbase-common/pom.xml                               |   4 +
 .../apache/hadoop/hbase/io/compress/CodecPool.java | 235 +++++++++++++++++++
 .../hadoop/hbase/io/compress/Compression.java      |   1 -
 .../hadoop/hbase/io/compress/TestCodecPool.java    | 255 +++++++++++++++++++++
 .../hadoop/hbase/HFilePerformanceEvaluation.java   |   5 +-
 6 files changed, 498 insertions(+), 4 deletions(-)

diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
index e7364ca3b42..4156a91c9e5 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CellBlockBuilder.java
@@ -34,10 +34,10 @@ import org.apache.hadoop.hbase.io.ByteBuffInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
+import org.apache.hadoop.hbase.io.compress.CodecPool;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
 import org.apache.hadoop.hbase.util.ClassSize;
-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.Compressor;
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index cc0260cdee9..3d009654140 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -109,6 +109,10 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-crypto</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.github.ben-manes.caffeine</groupId>
+      <artifactId>caffeine</artifactId>
+    </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/CodecPool.java 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/CodecPool.java
new file mode 100644
index 00000000000..b43ef12ae99
--- /dev/null
+++ 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/CodecPool.java
@@ -0,0 +1,235 @@
+/*
+ * 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.compress;
+
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.Comparator;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DoNotPool;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A global compressor/decompressor pool used to save and reuse (possibly 
native)
+ * compression/decompression codecs. Copied from the class of the same name in 
hadoop-common and
+ * augmented to improve borrow/return performance.
+ */
[email protected]
+public class CodecPool {
+  private static final Logger LOG = LoggerFactory.getLogger(CodecPool.class);
+
+  private static final ConcurrentMap<Class<Compressor>, 
NavigableSet<Compressor>> COMPRESSOR_POOL =
+    new ConcurrentHashMap<>();
+
+  private static final ConcurrentMap<Class<Decompressor>,
+    NavigableSet<Decompressor>> DECOMPRESSOR_POOL = new ConcurrentHashMap<>();
+
+  private static <T> LoadingCache<Class<T>, AtomicInteger> createCache() {
+    return Caffeine.newBuilder().build(key -> new AtomicInteger());
+  }
+
+  /**
+   * Map to track the number of leased compressors. Only used in unit tests, 
kept null otherwise.
+   */
+  @Nullable
+  private static LoadingCache<Class<Compressor>, AtomicInteger> 
compressorCounts = null;
+
+  /**
+   * Map to tracks the number of leased decompressors. Only used in unit 
tests, kept null otherwise.
+   */
+  @Nullable
+  private static LoadingCache<Class<Decompressor>, AtomicInteger> 
decompressorCounts = null;
+
+  /**
+   * Call if you want lease counting to be enabled. Only used in unit tests.
+   */
+  static void initLeaseCounting() {
+    compressorCounts = createCache();
+    decompressorCounts = createCache();
+  }
+
+  private static <T> T borrow(ConcurrentMap<Class<T>, NavigableSet<T>> pool,
+    Class<? extends T> codecClass) {
+    if (codecClass == null) {
+      return null;
+    }
+
+    NavigableSet<T> codecSet = pool.get(codecClass);
+    if (codecSet != null) {
+      // If a copy of the codec is available, pollFirst() will grab one.
+      // If not, it will return null.
+      return codecSet.pollFirst();
+    } else {
+      return null;
+    }
+  }
+
+  private static <T> boolean payback(ConcurrentMap<Class<T>, NavigableSet<T>> 
pool, T codec) {
+    if (codec != null) {
+      Class<T> codecClass = ReflectionUtils.getClass(codec);
+      Set<T> codecSet = pool.computeIfAbsent(codecClass,
+        k -> new 
ConcurrentSkipListSet<>(Comparator.comparingInt(System::identityHashCode)));
+      return codecSet.add(codec);
+    }
+    return false;
+  }
+
+  /**
+   * Copied from hadoop-common without significant modification.
+   */
+  @SuppressWarnings("unchecked")
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+      value = "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE",
+      justification = "LoadingCache will compute value if absent")
+  private static <T> int getLeaseCount(LoadingCache<Class<T>, AtomicInteger> 
usageCounts,
+    Class<? extends T> codecClass) {
+    return usageCounts.get((Class<T>) codecClass).get();
+  }
+
+  /**
+   * Copied from hadoop-common without significant modification.
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+      value = "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE",
+      justification = "LoadingCache will compute value if absent")
+  private static <T> void updateLeaseCount(LoadingCache<Class<T>, 
AtomicInteger> usageCounts,
+    T codec, int delta) {
+    if (codec != null && usageCounts != null) {
+      Class<T> codecClass = ReflectionUtils.getClass(codec);
+      usageCounts.get(codecClass).addAndGet(delta);
+    }
+  }
+
+  /**
+   * Get a {@link Compressor} for the given {@link CompressionCodec} from the 
pool, or get a new one
+   * if the pool is empty. Copied from hadoop-common without significant 
modification.
+   */
+  public static Compressor getCompressor(CompressionCodec codec, Configuration 
conf) {
+    Compressor compressor = borrow(COMPRESSOR_POOL, codec.getCompressorType());
+    if (compressor == null) {
+      compressor = codec.createCompressor();
+      LOG.info("Got brand-new compressor [" + codec.getDefaultExtension() + 
"]");
+    } else {
+      compressor.reinit(conf);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Got recycled compressor");
+      }
+    }
+    if (compressor != null && 
!compressor.getClass().isAnnotationPresent(DoNotPool.class)) {
+      updateLeaseCount(compressorCounts, compressor, 1);
+    }
+    return compressor;
+  }
+
+  public static Compressor getCompressor(CompressionCodec codec) {
+    return getCompressor(codec, null);
+  }
+
+  /**
+   * Get a {@link Decompressor} for the given {@link CompressionCodec} from 
the pool, or get a new
+   * one if the pool is empty. Copied from hadoop-common without significant 
modification.
+   */
+  public static Decompressor getDecompressor(CompressionCodec codec) {
+    Decompressor decompressor = borrow(DECOMPRESSOR_POOL, 
codec.getDecompressorType());
+    if (decompressor == null) {
+      decompressor = codec.createDecompressor();
+      LOG.info("Got brand-new decompressor [" + codec.getDefaultExtension() + 
"]");
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Got recycled decompressor");
+      }
+    }
+    if (decompressor != null && 
!decompressor.getClass().isAnnotationPresent(DoNotPool.class)) {
+      updateLeaseCount(decompressorCounts, decompressor, 1);
+    }
+    return decompressor;
+  }
+
+  /**
+   * Return the {@link Compressor} to the pool. Copied from hadoop-common 
without significant
+   * modification.
+   */
+  public static void returnCompressor(Compressor compressor) {
+    if (compressor == null) {
+      return;
+    }
+    // if the compressor can't be reused, don't pool it.
+    if (compressor.getClass().isAnnotationPresent(DoNotPool.class)) {
+      compressor.end();
+      return;
+    }
+    compressor.reset();
+    if (payback(COMPRESSOR_POOL, compressor)) {
+      updateLeaseCount(compressorCounts, compressor, -1);
+    }
+  }
+
+  /**
+   * Return the {@link Decompressor} to the pool. Copied from hadoop-common 
without significant
+   * modification.
+   */
+  public static void returnDecompressor(Decompressor decompressor) {
+    if (decompressor == null) {
+      return;
+    }
+    // if the decompressor can't be reused, don't pool it.
+    if (decompressor.getClass().isAnnotationPresent(DoNotPool.class)) {
+      decompressor.end();
+      return;
+    }
+    decompressor.reset();
+    if (payback(DECOMPRESSOR_POOL, decompressor)) {
+      updateLeaseCount(decompressorCounts, decompressor, -1);
+    }
+  }
+
+  /**
+   * Returns the number of leased {@link Compressor}s for this {@link 
CompressionCodec}. Copied from
+   * hadoop-common without significant modification.
+   */
+  static int getLeasedCompressorsCount(@Nullable CompressionCodec codec) {
+    if (compressorCounts == null) {
+      throw new IllegalStateException("initLeaseCounting() not called to set 
up lease counting");
+    }
+    return (codec == null) ? 0 : getLeaseCount(compressorCounts, 
codec.getCompressorType());
+  }
+
+  /**
+   * Returns the number of leased {@link Decompressor}s for this {@link 
CompressionCodec}. Copied
+   * from hadoop-common without significant modification.
+   */
+  static int getLeasedDecompressorsCount(@Nullable CompressionCodec codec) {
+    if (decompressorCounts == null) {
+      throw new IllegalStateException("initLeaseCounting() not called to set 
up lease counting");
+    }
+    return (codec == null) ? 0 : getLeaseCount(decompressorCounts, 
codec.getDecompressorType());
+  }
+}
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
index 7f73cd2f004..d4ca5af8649 100644
--- 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
+++ 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
@@ -26,7 +26,6 @@ import java.io.OutputStream;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-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;
diff --git 
a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/compress/TestCodecPool.java
 
b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/compress/TestCodecPool.java
new file mode 100644
index 00000000000..166c12a658c
--- /dev/null
+++ 
b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/compress/TestCodecPool.java
@@ -0,0 +1,255 @@
+/*
+ * 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.compress;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.io.compress.CompressionInputStream;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Along with CodecPool, this is copied from the class of the same name in 
hadoop-common. Modified
+ * to accommodate changes to HBase's CodecPool.
+ */
+@Category({ MiscTests.class, SmallTests.class })
+public class TestCodecPool {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestCodecPool.class);
+
+  private final String LEASE_COUNT_ERR = "Incorrect number of leased 
(de)compressors";
+  DefaultCodec codec;
+
+  @BeforeClass
+  public static void beforeClass() {
+    CodecPool.initLeaseCounting();
+  }
+
+  @Before
+  public void setup() {
+    this.codec = new DefaultCodec();
+    this.codec.setConf(new Configuration());
+  }
+
+  @Test(timeout = 10000)
+  public void testCompressorPoolCounts() {
+    // Get two compressors and return them
+    Compressor comp1 = CodecPool.getCompressor(codec);
+    Compressor comp2 = CodecPool.getCompressor(codec);
+    assertEquals(LEASE_COUNT_ERR, 2, 
CodecPool.getLeasedCompressorsCount(codec));
+
+    CodecPool.returnCompressor(comp2);
+    assertEquals(LEASE_COUNT_ERR, 1, 
CodecPool.getLeasedCompressorsCount(codec));
+
+    CodecPool.returnCompressor(comp1);
+    assertEquals(LEASE_COUNT_ERR, 0, 
CodecPool.getLeasedCompressorsCount(codec));
+
+    CodecPool.returnCompressor(comp1);
+    assertEquals(LEASE_COUNT_ERR, 0, 
CodecPool.getLeasedCompressorsCount(codec));
+  }
+
+  @Test(timeout = 10000)
+  public void testCompressorNotReturnSameInstance() {
+    Compressor comp = CodecPool.getCompressor(codec);
+    CodecPool.returnCompressor(comp);
+    CodecPool.returnCompressor(comp);
+    Set<Compressor> compressors = new HashSet<Compressor>();
+    for (int i = 0; i < 10; ++i) {
+      compressors.add(CodecPool.getCompressor(codec));
+    }
+    assertEquals(10, compressors.size());
+    for (Compressor compressor : compressors) {
+      CodecPool.returnCompressor(compressor);
+    }
+  }
+
+  @Test(timeout = 10000)
+  public void testDecompressorPoolCounts() {
+    // Get two decompressors and return them
+    Decompressor decomp1 = CodecPool.getDecompressor(codec);
+    Decompressor decomp2 = CodecPool.getDecompressor(codec);
+    assertEquals(LEASE_COUNT_ERR, 2, 
CodecPool.getLeasedDecompressorsCount(codec));
+
+    CodecPool.returnDecompressor(decomp2);
+    assertEquals(LEASE_COUNT_ERR, 1, 
CodecPool.getLeasedDecompressorsCount(codec));
+
+    CodecPool.returnDecompressor(decomp1);
+    assertEquals(LEASE_COUNT_ERR, 0, 
CodecPool.getLeasedDecompressorsCount(codec));
+
+    CodecPool.returnDecompressor(decomp1);
+    assertEquals(LEASE_COUNT_ERR, 0, 
CodecPool.getLeasedCompressorsCount(codec));
+  }
+
+  @Test(timeout = 10000)
+  public void testMultiThreadedCompressorPool() throws InterruptedException {
+    final int iterations = 4;
+    ExecutorService threadpool = Executors.newFixedThreadPool(3);
+    final LinkedBlockingDeque<Compressor> queue =
+      new LinkedBlockingDeque<Compressor>(2 * iterations);
+
+    Callable<Boolean> consumer = new Callable<Boolean>() {
+      @Override
+      public Boolean call() throws Exception {
+        Compressor c = queue.take();
+        CodecPool.returnCompressor(c);
+        return c != null;
+      }
+    };
+
+    Callable<Boolean> producer = new Callable<Boolean>() {
+      @Override
+      public Boolean call() throws Exception {
+        Compressor c = CodecPool.getCompressor(codec);
+        queue.put(c);
+        return c != null;
+      }
+    };
+
+    for (int i = 0; i < iterations; i++) {
+      threadpool.submit(consumer);
+      threadpool.submit(producer);
+    }
+
+    // wait for completion
+    threadpool.shutdown();
+    threadpool.awaitTermination(1000, TimeUnit.SECONDS);
+
+    assertEquals(LEASE_COUNT_ERR, 0, 
CodecPool.getLeasedCompressorsCount(codec));
+  }
+
+  @Test(timeout = 10000)
+  public void testMultiThreadedDecompressorPool() throws InterruptedException {
+    final int iterations = 4;
+    ExecutorService threadpool = Executors.newFixedThreadPool(3);
+    final LinkedBlockingDeque<Decompressor> queue =
+      new LinkedBlockingDeque<Decompressor>(2 * iterations);
+
+    Callable<Boolean> consumer = new Callable<Boolean>() {
+      @Override
+      public Boolean call() throws Exception {
+        Decompressor dc = queue.take();
+        CodecPool.returnDecompressor(dc);
+        return dc != null;
+      }
+    };
+
+    Callable<Boolean> producer = new Callable<Boolean>() {
+      @Override
+      public Boolean call() throws Exception {
+        Decompressor c = CodecPool.getDecompressor(codec);
+        queue.put(c);
+        return c != null;
+      }
+    };
+
+    for (int i = 0; i < iterations; i++) {
+      threadpool.submit(consumer);
+      threadpool.submit(producer);
+    }
+
+    // wait for completion
+    threadpool.shutdown();
+    threadpool.awaitTermination(1000, TimeUnit.SECONDS);
+
+    assertEquals(LEASE_COUNT_ERR, 0, 
CodecPool.getLeasedDecompressorsCount(codec));
+  }
+
+  @Test(timeout = 10000)
+  public void testDecompressorNotReturnSameInstance() {
+    Decompressor decomp = CodecPool.getDecompressor(codec);
+    CodecPool.returnDecompressor(decomp);
+    CodecPool.returnDecompressor(decomp);
+    Set<Decompressor> decompressors = new HashSet<Decompressor>();
+    for (int i = 0; i < 10; ++i) {
+      decompressors.add(CodecPool.getDecompressor(codec));
+    }
+    assertEquals(10, decompressors.size());
+    for (Decompressor decompressor : decompressors) {
+      CodecPool.returnDecompressor(decompressor);
+    }
+  }
+
+  @Test(timeout = 10000)
+  public void testDoNotPoolDecompressorNotUseableAfterReturn() throws 
Exception {
+
+    final GzipCodec gzipCodec = new GzipCodec();
+    gzipCodec.setConf(new Configuration());
+
+    final Random random = new Random();
+    final byte[] bytes = new byte[1024];
+    random.nextBytes(bytes);
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try (OutputStream outputStream = gzipCodec.createOutputStream(baos)) {
+      outputStream.write(bytes);
+    }
+
+    final byte[] gzipBytes = baos.toByteArray();
+    final ByteArrayInputStream bais = new ByteArrayInputStream(gzipBytes);
+
+    // BuiltInGzipDecompressor is an explicit example of a Decompressor
+    // with the @DoNotPool annotation
+    final Decompressor decompressor = new BuiltInGzipDecompressor();
+    CodecPool.returnDecompressor(decompressor);
+
+    final CompressionInputStream inputStream = 
gzipCodec.createInputStream(bais, decompressor);
+    boolean passed = false;
+    try {
+      inputStream.read();
+    } catch (Exception e) {
+      if (
+        e.getMessage().contains("decompress called on closed decompressor")
+          || e.getMessage().contains("Inflater has been closed")
+      ) {
+        passed = true;
+      }
+    }
+
+    if (!passed) {
+      fail("Decompressor from Codec with @DoNotPool should not be "
+        + "useable after returning to CodecPool");
+    }
+  }
+
+}
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
index 60ba0f85c00..de1a45c0c6a 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
@@ -52,12 +52,13 @@ public class HFilePerformanceEvaluation {
   private static final int RFILE_BLOCKSIZE = 8 * 1024;
   private static StringBuilder testSummary = new StringBuilder();
 
-  // Disable verbose INFO logging from org.apache.hadoop.io.compress.CodecPool
+  // Disable verbose INFO logging from 
org.apache.hadoop.hbase.io.compress.CodecPool
   static {
     System.setProperty("org.apache.commons.logging.Log",
       "org.apache.commons.logging.impl.SimpleLog");
     System.setProperty(
-      
"org.apache.commons.logging.simplelog.log.org.apache.hadoop.io.compress.CodecPool",
 "WARN");
+      
"org.apache.commons.logging.simplelog.log.org.apache.hadoop.hbase.io.compress.CodecPool",
+      "WARN");
   }
 
   private static final Logger LOG =

Reply via email to