001 /**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements. See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership. The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License. You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018 package org.apache.hadoop.io.compress;
019
020 import java.util.ArrayList;
021 import java.util.HashMap;
022 import java.util.List;
023 import java.util.Map;
024 import java.util.concurrent.atomic.AtomicInteger;
025
026 import org.apache.commons.logging.Log;
027 import org.apache.commons.logging.LogFactory;
028 import org.apache.hadoop.classification.InterfaceAudience;
029 import org.apache.hadoop.classification.InterfaceStability;
030 import org.apache.hadoop.conf.Configuration;
031 import org.apache.hadoop.util.ReflectionUtils;
032
033 import com.google.common.cache.CacheBuilder;
034 import com.google.common.cache.CacheLoader;
035 import com.google.common.cache.LoadingCache;
036
037 /**
038 * A global compressor/decompressor pool used to save and reuse
039 * (possibly native) compression/decompression codecs.
040 */
041 @InterfaceAudience.Public
042 @InterfaceStability.Evolving
043 public class CodecPool {
044 private static final Log LOG = LogFactory.getLog(CodecPool.class);
045
046 /**
047 * A global compressor pool used to save the expensive
048 * construction/destruction of (possibly native) decompression codecs.
049 */
050 private static final Map<Class<Compressor>, List<Compressor>> compressorPool =
051 new HashMap<Class<Compressor>, List<Compressor>>();
052
053 /**
054 * A global decompressor pool used to save the expensive
055 * construction/destruction of (possibly native) decompression codecs.
056 */
057 private static final Map<Class<Decompressor>, List<Decompressor>> decompressorPool =
058 new HashMap<Class<Decompressor>, List<Decompressor>>();
059
060 private static <T> LoadingCache<Class<T>, AtomicInteger> createCache(
061 Class<T> klass) {
062 return CacheBuilder.newBuilder().build(
063 new CacheLoader<Class<T>, AtomicInteger>() {
064 @Override
065 public AtomicInteger load(Class<T> key) throws Exception {
066 return new AtomicInteger();
067 }
068 });
069 }
070
071 /**
072 * Map to track the number of leased compressors
073 */
074 private static final LoadingCache<Class<Compressor>, AtomicInteger> compressorCounts =
075 createCache(Compressor.class);
076
077 /**
078 * Map to tracks the number of leased decompressors
079 */
080 private static final LoadingCache<Class<Decompressor>, AtomicInteger> decompressorCounts =
081 createCache(Decompressor.class);
082
083 private static <T> T borrow(Map<Class<T>, List<T>> pool,
084 Class<? extends T> codecClass) {
085 T codec = null;
086
087 // Check if an appropriate codec is available
088 synchronized (pool) {
089 if (pool.containsKey(codecClass)) {
090 List<T> codecList = pool.get(codecClass);
091
092 if (codecList != null) {
093 synchronized (codecList) {
094 if (!codecList.isEmpty()) {
095 codec = codecList.remove(codecList.size()-1);
096 }
097 }
098 }
099 }
100 }
101
102 return codec;
103 }
104
105 private static <T> void payback(Map<Class<T>, List<T>> pool, T codec) {
106 if (codec != null) {
107 Class<T> codecClass = ReflectionUtils.getClass(codec);
108 synchronized (pool) {
109 if (!pool.containsKey(codecClass)) {
110 pool.put(codecClass, new ArrayList<T>());
111 }
112
113 List<T> codecList = pool.get(codecClass);
114 synchronized (codecList) {
115 codecList.add(codec);
116 }
117 }
118 }
119 }
120
121 @SuppressWarnings("unchecked")
122 private static <T> int getLeaseCount(
123 LoadingCache<Class<T>, AtomicInteger> usageCounts,
124 Class<? extends T> codecClass) {
125 return usageCounts.getUnchecked((Class<T>) codecClass).get();
126 }
127
128 private static <T> void updateLeaseCount(
129 LoadingCache<Class<T>, AtomicInteger> usageCounts, T codec, int delta) {
130 if (codec != null) {
131 Class<T> codecClass = ReflectionUtils.getClass(codec);
132 usageCounts.getUnchecked(codecClass).addAndGet(delta);
133 }
134 }
135
136 /**
137 * Get a {@link Compressor} for the given {@link CompressionCodec} from the
138 * pool or a new one.
139 *
140 * @param codec the <code>CompressionCodec</code> for which to get the
141 * <code>Compressor</code>
142 * @param conf the <code>Configuration</code> object which contains confs for creating or reinit the compressor
143 * @return <code>Compressor</code> for the given
144 * <code>CompressionCodec</code> from the pool or a new one
145 */
146 public static Compressor getCompressor(CompressionCodec codec, Configuration conf) {
147 Compressor compressor = borrow(compressorPool, codec.getCompressorType());
148 if (compressor == null) {
149 compressor = codec.createCompressor();
150 LOG.info("Got brand-new compressor ["+codec.getDefaultExtension()+"]");
151 } else {
152 compressor.reinit(conf);
153 if(LOG.isDebugEnabled()) {
154 LOG.debug("Got recycled compressor");
155 }
156 }
157 updateLeaseCount(compressorCounts, compressor, 1);
158 return compressor;
159 }
160
161 public static Compressor getCompressor(CompressionCodec codec) {
162 return getCompressor(codec, null);
163 }
164
165 /**
166 * Get a {@link Decompressor} for the given {@link CompressionCodec} from the
167 * pool or a new one.
168 *
169 * @param codec the <code>CompressionCodec</code> for which to get the
170 * <code>Decompressor</code>
171 * @return <code>Decompressor</code> for the given
172 * <code>CompressionCodec</code> the pool or a new one
173 */
174 public static Decompressor getDecompressor(CompressionCodec codec) {
175 Decompressor decompressor = borrow(decompressorPool, codec.getDecompressorType());
176 if (decompressor == null) {
177 decompressor = codec.createDecompressor();
178 LOG.info("Got brand-new decompressor ["+codec.getDefaultExtension()+"]");
179 } else {
180 if(LOG.isDebugEnabled()) {
181 LOG.debug("Got recycled decompressor");
182 }
183 }
184 updateLeaseCount(decompressorCounts, decompressor, 1);
185 return decompressor;
186 }
187
188 /**
189 * Return the {@link Compressor} to the pool.
190 *
191 * @param compressor the <code>Compressor</code> to be returned to the pool
192 */
193 public static void returnCompressor(Compressor compressor) {
194 if (compressor == null) {
195 return;
196 }
197 // if the compressor can't be reused, don't pool it.
198 if (compressor.getClass().isAnnotationPresent(DoNotPool.class)) {
199 return;
200 }
201 compressor.reset();
202 payback(compressorPool, compressor);
203 updateLeaseCount(compressorCounts, compressor, -1);
204 }
205
206 /**
207 * Return the {@link Decompressor} to the pool.
208 *
209 * @param decompressor the <code>Decompressor</code> to be returned to the
210 * pool
211 */
212 public static void returnDecompressor(Decompressor decompressor) {
213 if (decompressor == null) {
214 return;
215 }
216 // if the decompressor can't be reused, don't pool it.
217 if (decompressor.getClass().isAnnotationPresent(DoNotPool.class)) {
218 return;
219 }
220 decompressor.reset();
221 payback(decompressorPool, decompressor);
222 updateLeaseCount(decompressorCounts, decompressor, -1);
223 }
224
225 /**
226 * Return the number of leased {@link Compressor}s for this
227 * {@link CompressionCodec}
228 */
229 public static int getLeasedCompressorsCount(CompressionCodec codec) {
230 return (codec == null) ? 0 : getLeaseCount(compressorCounts,
231 codec.getCompressorType());
232 }
233
234 /**
235 * Return the number of leased {@link Decompressor}s for this
236 * {@link CompressionCodec}
237 */
238 public static int getLeasedDecompressorsCount(CompressionCodec codec) {
239 return (codec == null) ? 0 : getLeaseCount(decompressorCounts,
240 codec.getDecompressorType());
241 }
242 }