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