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 */
018package org.apache.hadoop.hdfs.util;
019
020import java.util.HashMap;
021import java.util.LinkedList;
022import java.util.Map;
023import java.util.Queue;
024
025import org.apache.commons.logging.Log;
026import org.apache.commons.logging.LogFactory;
027import org.apache.hadoop.HadoopIllegalArgumentException;
028import org.apache.hadoop.classification.InterfaceAudience;
029import org.apache.hadoop.util.Time;
030
031import com.google.common.base.Preconditions;
032
033/**
034 * Manage byte array creation and release. 
035 */
036@InterfaceAudience.Private
037public abstract class ByteArrayManager {
038  static final Log LOG = LogFactory.getLog(ByteArrayManager.class);
039  private static final ThreadLocal<StringBuilder> debugMessage = new ThreadLocal<StringBuilder>() {
040    protected StringBuilder initialValue() {
041      return new StringBuilder();
042    }
043  };
044
045  private static void logDebugMessage() {
046    final StringBuilder b = debugMessage.get();
047    LOG.debug(b);
048    b.setLength(0);
049  }
050
051  static final int MIN_ARRAY_LENGTH = 32;
052  static final byte[] EMPTY_BYTE_ARRAY = {};
053
054  /**
055   * @return the least power of two greater than or equal to n, i.e. return
056   *         the least integer x with x >= n and x a power of two.
057   *
058   * @throws HadoopIllegalArgumentException
059   *           if n <= 0.
060   */
061  public static int leastPowerOfTwo(final int n) {
062    if (n <= 0) {
063      throw new HadoopIllegalArgumentException("n = " + n + " <= 0");
064    }
065
066    final int highestOne = Integer.highestOneBit(n);
067    if (highestOne == n) {
068      return n; // n is a power of two.
069    }
070    final int roundUp = highestOne << 1;
071    if (roundUp < 0) {
072      final long overflow = ((long) highestOne) << 1;
073      throw new ArithmeticException(
074          "Overflow: for n = " + n + ", the least power of two (the least"
075          + " integer x with x >= n and x a power of two) = "
076          + overflow + " > Integer.MAX_VALUE = " + Integer.MAX_VALUE);
077    }
078    return roundUp;
079  }
080
081  /**
082   * A counter with a time stamp so that it is reset automatically
083   * if there is no increment for the time period.
084   */
085  static class Counter {
086    private final long countResetTimePeriodMs;
087    private long count = 0L;
088    private long timestamp = Time.monotonicNow();
089
090    Counter(long countResetTimePeriodMs) {
091      this.countResetTimePeriodMs = countResetTimePeriodMs;
092    }
093
094    synchronized long getCount() {
095      return count;
096    }
097
098    /**
099     * Increment the counter, and reset it if there is no increment
100     * for acertain time period.
101     *
102     * @return the new count.
103     */
104    synchronized long increment() {
105      final long now = Time.monotonicNow();
106      if (now - timestamp > countResetTimePeriodMs) {
107        count = 0; // reset the counter
108      }
109      timestamp = now;
110      return ++count;
111    }
112  }
113
114  /** A map from integers to counters. */
115  static class CounterMap {
116    /** @see ByteArrayManager.Conf#countResetTimePeriodMs */
117    private final long countResetTimePeriodMs;
118    private final Map<Integer, Counter> map = new HashMap<Integer, Counter>();
119
120    private CounterMap(long countResetTimePeriodMs) {
121      this.countResetTimePeriodMs = countResetTimePeriodMs;
122    }
123
124    /**
125     * @return the counter for the given key;
126     *         and create a new counter if it does not exist.
127     */
128    synchronized Counter get(final Integer key, final boolean createIfNotExist) {
129      Counter count = map.get(key);
130      if (count == null && createIfNotExist) {
131        count = new Counter(countResetTimePeriodMs);
132        map.put(key, count);
133      }
134      return count;
135    }
136
137    synchronized void clear() {
138      map.clear();
139    }
140  }
141
142  /** Manage byte arrays with the same fixed length. */
143  static class FixedLengthManager {
144    private final int byteArrayLength;
145    private final int maxAllocated;
146    private final Queue<byte[]> freeQueue = new LinkedList<byte[]>();
147
148    private int numAllocated = 0;
149
150    FixedLengthManager(int arrayLength, int maxAllocated) {
151      this.byteArrayLength = arrayLength;
152      this.maxAllocated = maxAllocated;
153    }
154
155    /**
156     * Allocate a byte array.
157     *
158     * If the number of allocated arrays >= maximum, the current thread is
159     * blocked until the number of allocated arrays drops to below the maximum.
160     * 
161     * The byte array allocated by this method must be returned for recycling
162     * via the {@link FixedLengthManager#recycle(byte[])} method.
163     */
164    synchronized byte[] allocate() throws InterruptedException {
165      if (LOG.isDebugEnabled()) {
166        debugMessage.get().append(", ").append(this);
167      }
168      for(; numAllocated >= maxAllocated;) {
169        if (LOG.isDebugEnabled()) {
170          debugMessage.get().append(": wait ...");
171          logDebugMessage();
172        }
173
174        wait();
175
176        if (LOG.isDebugEnabled()) {
177          debugMessage.get().append("wake up: ").append(this);
178        }
179      }
180      numAllocated++;
181
182      final byte[] array = freeQueue.poll();
183      if (LOG.isDebugEnabled()) {
184        debugMessage.get().append(", recycled? ").append(array != null);
185      }
186      return array != null? array : new byte[byteArrayLength];
187    }
188
189    /**
190     * Recycle the given byte array, which must have the same length as the
191     * array length managed by this object.
192     *
193     * The byte array may or may not be allocated
194     * by the {@link FixedLengthManager#allocate()} method.
195     */
196    synchronized int recycle(byte[] array) {
197      Preconditions.checkNotNull(array);
198      Preconditions.checkArgument(array.length == byteArrayLength);
199      if (LOG.isDebugEnabled()) {
200        debugMessage.get().append(", ").append(this);
201      }
202
203      if (numAllocated == maxAllocated) {
204        if (LOG.isDebugEnabled()) {
205          debugMessage.get().append(", notifyAll");
206        }
207        notifyAll();
208      }
209      numAllocated--;
210      if (numAllocated < 0) {
211        // it is possible to drop below 0 since
212        // some byte arrays may not be created by the allocate() method.
213        numAllocated = 0;
214      }
215
216      if (freeQueue.size() < maxAllocated - numAllocated) {
217        if (LOG.isDebugEnabled()) {
218          debugMessage.get().append(", freeQueue.offer");
219        }
220        freeQueue.offer(array);
221      }
222      return freeQueue.size();
223    }
224
225    @Override
226    public synchronized String toString() {
227      return "[" + byteArrayLength + ": " + numAllocated + "/"
228          + maxAllocated + ", free=" + freeQueue.size() + "]";
229    }
230  }
231
232  /** A map from array lengths to byte array managers. */
233  static class ManagerMap {
234    private final int countLimit;
235    private final Map<Integer, FixedLengthManager> map = new HashMap<Integer, FixedLengthManager>();
236
237    ManagerMap(int countLimit) {
238      this.countLimit = countLimit;
239    }
240
241    /** @return the manager for the given array length. */
242    synchronized FixedLengthManager get(final Integer arrayLength,
243        final boolean createIfNotExist) {
244      FixedLengthManager manager = map.get(arrayLength);
245      if (manager == null && createIfNotExist) {
246        manager = new FixedLengthManager(arrayLength, countLimit);
247        map.put(arrayLength, manager);
248      }
249      return manager;
250    }
251
252    synchronized void clear() {
253      map.clear();
254    }
255  }
256
257  public static class Conf {
258    /**
259     * The count threshold for each array length so that a manager is created
260     * only after the allocation count exceeds the threshold.
261     */
262    private final int countThreshold;
263    /**
264     * The maximum number of arrays allowed for each array length.
265     */
266    private final int countLimit;
267    /**
268     * The time period in milliseconds that the allocation count for each array
269     * length is reset to zero if there is no increment.
270     */
271    private final long countResetTimePeriodMs;
272
273    public Conf(int countThreshold, int countLimit, long countResetTimePeriodMs) {
274      this.countThreshold = countThreshold;
275      this.countLimit = countLimit;
276      this.countResetTimePeriodMs = countResetTimePeriodMs;
277    }
278  }
279
280  /**
281   * Create a byte array for the given length, where the length of
282   * the returned array is larger than or equal to the given length.
283   *
284   * The current thread may be blocked if some resource is unavailable.
285   * 
286   * The byte array created by this method must be released
287   * via the {@link ByteArrayManager#release(byte[])} method.
288   *
289   * @return a byte array with length larger than or equal to the given length.
290   */
291  public abstract byte[] newByteArray(int size) throws InterruptedException;
292  
293  /**
294   * Release the given byte array.
295   * 
296   * The byte array may or may not be created
297   * by the {@link ByteArrayManager#newByteArray(int)} method.
298   * 
299   * @return the number of free array.
300   */
301  public abstract int release(byte[] array);
302
303  public static ByteArrayManager newInstance(Conf conf) {
304    return conf == null? new NewByteArrayWithoutLimit(): new Impl(conf);
305  }
306
307  /**
308   * A dummy implementation which simply calls new byte[].
309   */
310  static class NewByteArrayWithoutLimit extends ByteArrayManager {
311    @Override
312    public byte[] newByteArray(int size) throws InterruptedException {
313      return new byte[size];
314    }
315    
316    @Override
317    public int release(byte[] array) {
318      return 0;
319    }
320  }
321
322  /**
323   * Manage byte array allocation and provide a mechanism for recycling the byte
324   * array objects.
325   */
326  static class Impl extends ByteArrayManager {
327    private final Conf conf;
328  
329    private final CounterMap counters;
330    private final ManagerMap managers;
331  
332    Impl(Conf conf) {
333      this.conf = conf;
334      this.counters = new CounterMap(conf.countResetTimePeriodMs);
335      this.managers = new ManagerMap(conf.countLimit);
336    }
337  
338    /**
339     * Allocate a byte array, where the length of the allocated array
340     * is the least power of two of the given length
341     * unless the given length is less than {@link #MIN_ARRAY_LENGTH}.
342     * In such case, the returned array length is equal to {@link #MIN_ARRAY_LENGTH}.
343     *
344     * If the number of allocated arrays exceeds the capacity,
345     * the current thread is blocked until
346     * the number of allocated arrays drops to below the capacity.
347     * 
348     * The byte array allocated by this method must be returned for recycling
349     * via the {@link ByteArrayManager#recycle(byte[])} method.
350     *
351     * @return a byte array with length larger than or equal to the given length.
352     */
353    @Override
354    public byte[] newByteArray(final int arrayLength) throws InterruptedException {
355      if (LOG.isDebugEnabled()) {
356        debugMessage.get().append("allocate(").append(arrayLength).append(")");
357      }
358  
359      final byte[] array;
360      if (arrayLength == 0) {
361        array = EMPTY_BYTE_ARRAY;
362      } else {
363        final int powerOfTwo = arrayLength <= MIN_ARRAY_LENGTH?
364            MIN_ARRAY_LENGTH: leastPowerOfTwo(arrayLength);
365        final long count = counters.get(powerOfTwo, true).increment();
366        final boolean aboveThreshold = count > conf.countThreshold;
367        // create a new manager only if the count is above threshold.
368        final FixedLengthManager manager = managers.get(powerOfTwo, aboveThreshold);
369  
370        if (LOG.isDebugEnabled()) {
371          debugMessage.get().append(": count=").append(count)
372              .append(aboveThreshold? ", aboveThreshold": ", belowThreshold");
373        }
374        array = manager != null? manager.allocate(): new byte[powerOfTwo];
375      }
376  
377      if (LOG.isDebugEnabled()) {
378        logDebugMessage();
379      }
380      return array;
381    }
382  
383    /**
384     * Recycle the given byte array.
385     * 
386     * The byte array may or may not be allocated
387     * by the {@link ByteArrayManager#allocate(int)} method.
388     */
389    @Override
390    public int release(final byte[] array) {
391      Preconditions.checkNotNull(array);
392      if (LOG.isDebugEnabled()) {
393        debugMessage.get().append("recycle: array.length=").append(array.length);
394      }
395  
396      final int freeQueueSize;
397      if (array.length == 0) {
398        freeQueueSize = -1;
399      } else {
400        final FixedLengthManager manager = managers.get(array.length, false);
401        freeQueueSize = manager == null? -1: manager.recycle(array);
402      }
403  
404      if (LOG.isDebugEnabled()) {
405        debugMessage.get().append(", freeQueueSize=").append(freeQueueSize);
406        logDebugMessage();
407      }
408      return freeQueueSize;
409    }
410  
411    CounterMap getCounters() {
412      return counters;
413    }
414  
415    ManagerMap getManagers() {
416      return managers;
417    }
418  }
419}