View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.io.hfile.slab;
20  
21  import java.nio.ByteBuffer;
22  import java.util.concurrent.ConcurrentLinkedQueue;
23  import java.util.concurrent.LinkedBlockingQueue;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.util.ClassSize;
29  import org.apache.hadoop.hbase.util.DirectMemoryUtils;
30  import com.google.common.base.Preconditions;
31  
32  /**
33   * Slab is a class which is designed to allocate blocks of a certain size.
34   * Constructor creates a number of DirectByteBuffers and slices them into the
35   * requisite size, then puts them all in a buffer.
36   *
37   * @deprecated As of 1.0, replaced by {@link org.apache.hadoop.hbase.io.hfile.bucket.BucketCache}.
38   */
39  @InterfaceAudience.Private
40  @Deprecated
41  class Slab implements org.apache.hadoop.hbase.io.HeapSize {
42    static final Log LOG = LogFactory.getLog(Slab.class);
43  
44    /** This is where our items, or blocks of the slab, are stored. */
45    private LinkedBlockingQueue<ByteBuffer> buffers;
46  
47    /** This is where our Slabs are stored */
48    private ConcurrentLinkedQueue<ByteBuffer> slabs;
49  
50    private final int blockSize;
51    private final int numBlocks;
52    private long heapSize;
53  
54    Slab(int blockSize, int numBlocks) {
55      buffers = new LinkedBlockingQueue<ByteBuffer>();
56      slabs = new ConcurrentLinkedQueue<ByteBuffer>();
57  
58      this.blockSize = blockSize;
59      this.numBlocks = numBlocks;
60  
61      this.heapSize = ClassSize.estimateBase(this.getClass(), false);
62  
63      int maxBlocksPerSlab = Integer.MAX_VALUE / blockSize;
64      int maxSlabSize = maxBlocksPerSlab * blockSize;
65  
66      int numFullSlabs = numBlocks / maxBlocksPerSlab;
67      int partialSlabSize = (numBlocks % maxBlocksPerSlab) * blockSize;
68      for (int i = 0; i < numFullSlabs; i++) {
69        allocateAndSlice(maxSlabSize, blockSize);
70      }
71  
72      if (partialSlabSize > 0) {
73        allocateAndSlice(partialSlabSize, blockSize);
74      }
75    }
76  
77    private void allocateAndSlice(int size, int sliceSize) {
78      ByteBuffer newSlab = ByteBuffer.allocateDirect(size);
79      slabs.add(newSlab);
80      for (int j = 0; j < newSlab.capacity(); j += sliceSize) {
81        newSlab.limit(j + sliceSize).position(j);
82        ByteBuffer aSlice = newSlab.slice();
83        buffers.add(aSlice);
84        heapSize += ClassSize.estimateBase(aSlice.getClass(), false);
85      }
86    }
87  
88    /*
89     * Shutdown deallocates the memory for all the DirectByteBuffers. Each
90     * DirectByteBuffer has a "cleaner" method, which is similar to a
91     * deconstructor in C++.
92     */
93    void shutdown() {
94      for (ByteBuffer aSlab : slabs) {
95        try {
96          DirectMemoryUtils.destroyDirectByteBuffer(aSlab);
97        } catch (Exception e) {
98          LOG.warn("Unable to deallocate direct memory during shutdown", e);
99        }
100     }
101   }
102 
103   int getBlockSize() {
104     return this.blockSize;
105   }
106 
107   int getBlockCapacity() {
108     return this.numBlocks;
109   }
110 
111   int getBlocksRemaining() {
112     return this.buffers.size();
113   }
114 
115   /*
116    * Throws an exception if you try to allocate a
117    * bigger size than the allocator can handle. Alloc will block until a buffer is available.
118    */
119   ByteBuffer alloc(int bufferSize) throws InterruptedException {
120     int newCapacity = Preconditions.checkPositionIndex(bufferSize, blockSize);
121 
122     ByteBuffer returnedBuffer = buffers.take();
123 
124     returnedBuffer.clear().limit(newCapacity);
125     return returnedBuffer;
126   }
127 
128   void free(ByteBuffer toBeFreed) {
129     Preconditions.checkArgument(toBeFreed.capacity() == blockSize);
130     buffers.add(toBeFreed);
131   }
132 
133   @Override
134   public long heapSize() {
135     return heapSize;
136   }
137 }