wchevreuil commented on code in PR #6679:
URL: https://github.com/apache/hbase/pull/6679#discussion_r1950925088
##########
hbase-common/pom.xml:
##########
@@ -110,6 +110,10 @@
<groupId>org.apache.commons</groupId>
<artifactId>commons-crypto</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.github.ben-manes.caffeine</groupId>
+ <artifactId>caffeine</artifactId>
+ </dependency>
Review Comment:
Why we need an extra library? Can't we use `google.common.cache`, just like
the CodecPool from hadoop-commons?
##########
hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/CodecPool.java:
##########
@@ -0,0 +1,215 @@
+/*
+ * 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.Map;
+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(Map<Class<T>, NavigableSet<T>> pool, Class<?
extends T> codecClass) {
Review Comment:
Nit: Maybe enforce usage of ConcurrentMap, since no synchronization within
the method implementation?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]