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.hbase.regionserver;
019
020import java.nio.ByteBuffer;
021import java.util.concurrent.atomic.AtomicInteger;
022
023import org.apache.yetus.audience.InterfaceAudience;
024import org.apache.hadoop.hbase.util.Bytes;
025
026import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
027import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
028
029/**
030 * A chunk of memory out of which allocations are sliced.
031 */
032@InterfaceAudience.Private
033public abstract class Chunk {
034  /** Actual underlying data */
035  protected ByteBuffer data;
036
037  protected static final int UNINITIALIZED = -1;
038  protected static final int OOM = -2;
039  /**
040   * Offset for the next allocation, or the sentinel value -1 which implies that the chunk is still
041   * uninitialized.
042   */
043  protected AtomicInteger nextFreeOffset = new AtomicInteger(UNINITIALIZED);
044
045  /** Total number of allocations satisfied from this buffer */
046  protected AtomicInteger allocCount = new AtomicInteger();
047
048  /** Size of chunk in bytes */
049  protected final int size;
050
051  // The unique id associated with the chunk.
052  private final int id;
053
054  // indicates if the chunk is formed by ChunkCreator#MemstorePool
055  private final boolean fromPool;
056
057  /**
058   * Create an uninitialized chunk. Note that memory is not allocated yet, so
059   * this is cheap.
060   * @param size in bytes
061   * @param id the chunk id
062   */
063  public Chunk(int size, int id) {
064    this(size, id, false);
065  }
066
067  /**
068   * Create an uninitialized chunk. Note that memory is not allocated yet, so
069   * this is cheap.
070   * @param size in bytes
071   * @param id the chunk id
072   * @param fromPool if the chunk is formed by pool
073   */
074  public Chunk(int size, int id, boolean fromPool) {
075    this.size = size;
076    this.id = id;
077    this.fromPool = fromPool;
078  }
079
080  int getId() {
081    return this.id;
082  }
083
084  boolean isFromPool() {
085    return this.fromPool;
086  }
087
088  boolean isJumbo() {
089    return size > ChunkCreator.getInstance().getChunkSize();
090  }
091
092  boolean isIndexChunk() {
093    return size == ChunkCreator.getInstance().getChunkSize(ChunkCreator.ChunkType.INDEX_CHUNK);
094  }
095
096  /**
097   * Actually claim the memory for this chunk. This should only be called from the thread that
098   * constructed the chunk. It is thread-safe against other threads calling alloc(), who will block
099   * until the allocation is complete.
100   */
101  public void init() {
102    assert nextFreeOffset.get() == UNINITIALIZED;
103    try {
104      allocateDataBuffer();
105    } catch (OutOfMemoryError e) {
106      boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
107      assert failInit; // should be true.
108      throw e;
109    }
110    // Mark that it's ready for use
111    // Move 4 bytes since the first 4 bytes are having the chunkid in it
112    boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, Bytes.SIZEOF_INT);
113    // We should always succeed the above CAS since only one thread
114    // calls init()!
115    Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
116  }
117
118  abstract void allocateDataBuffer();
119
120  /**
121   * Reset the offset to UNINITIALIZED before before reusing an old chunk
122   */
123  void reset() {
124    if (nextFreeOffset.get() != UNINITIALIZED) {
125      nextFreeOffset.set(UNINITIALIZED);
126      allocCount.set(0);
127    }
128  }
129
130  /**
131   * Try to allocate <code>size</code> bytes from the chunk.
132   * If a chunk is tried to get allocated before init() call, the thread doing the allocation
133   * will be in busy-wait state as it will keep looping till the nextFreeOffset is set.
134   * @return the offset of the successful allocation, or -1 to indicate not-enough-space
135   */
136  public int alloc(int size) {
137    while (true) {
138      int oldOffset = nextFreeOffset.get();
139      if (oldOffset == UNINITIALIZED) {
140        // The chunk doesn't have its data allocated yet.
141        // Since we found this in curChunk, we know that whoever
142        // CAS-ed it there is allocating it right now. So spin-loop
143        // shouldn't spin long!
144        Thread.yield();
145        continue;
146      }
147      if (oldOffset == OOM) {
148        // doh we ran out of ram. return -1 to chuck this away.
149        return -1;
150      }
151
152      if (oldOffset + size > data.capacity()) {
153        return -1; // alloc doesn't fit
154      }
155      // TODO : If seqID is to be written add 8 bytes here for nextFreeOFfset
156      // Try to atomically claim this chunk
157      if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + size)) {
158        // we got the alloc
159        allocCount.incrementAndGet();
160        return oldOffset;
161      }
162      // we raced and lost alloc, try again
163    }
164  }
165
166  /**
167   * @return This chunk's backing data.
168   */
169  ByteBuffer getData() {
170    return this.data;
171  }
172
173  @Override
174  public String toString() {
175    return "Chunk@" + System.identityHashCode(this) + " allocs=" + allocCount.get() + "waste="
176        + (data.capacity() - nextFreeOffset.get());
177  }
178
179  @VisibleForTesting
180  int getNextFreeOffset() {
181    return this.nextFreeOffset.get();
182  }
183}