001/**
002 * Copyright The Apache Software Foundation
003 *
004 * Licensed to the Apache Software Foundation (ASF) under one or more
005 * contributor license agreements. See the NOTICE file distributed with this
006 * work for additional information regarding copyright ownership. The ASF
007 * licenses this file to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance with the License.
009 * You may obtain a copy of the License at
010 *
011 * http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
015 * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
016 * License for the specific language governing permissions and limitations
017 * under the License.
018 */
019package org.apache.hadoop.hbase.util;
020
021import java.io.IOException;
022import java.nio.ByteBuffer;
023import java.util.ArrayList;
024import java.util.Iterator;
025import java.util.List;
026import java.util.concurrent.ExecutorService;
027import java.util.concurrent.Executors;
028import java.util.concurrent.Future;
029import java.util.function.BiConsumer;
030
031import org.apache.hadoop.hbase.nio.ByteBuff;
032import org.apache.hadoop.util.StringUtils;
033import org.apache.yetus.audience.InterfaceAudience;
034import org.slf4j.Logger;
035import org.slf4j.LoggerFactory;
036
037/**
038 * This class manages an array of ByteBuffers with a default size 4MB. These buffers are sequential
039 * and could be considered as a large buffer.It supports reading/writing data from this large buffer
040 * with a position and offset
041 */
042@InterfaceAudience.Private
043public class ByteBufferArray {
044  private static final Logger LOG = LoggerFactory.getLogger(ByteBufferArray.class);
045
046  public static final int DEFAULT_BUFFER_SIZE = 4 * 1024 * 1024;
047  private final int bufferSize;
048  private final int bufferCount;
049  final ByteBuffer[] buffers;
050
051  /**
052   * We allocate a number of byte buffers as the capacity.
053   * @param capacity total size of the byte buffer array
054   * @param allocator the ByteBufferAllocator that will create the buffers
055   * @throws IOException throws IOException if there is an exception thrown by the allocator
056   */
057  public ByteBufferArray(long capacity, ByteBufferAllocator allocator) throws IOException {
058    this(getBufferSize(capacity), getBufferCount(capacity),
059        Runtime.getRuntime().availableProcessors(), capacity, allocator);
060  }
061
062  ByteBufferArray(int bufferSize, int bufferCount, int threadCount, long capacity,
063      ByteBufferAllocator alloc) throws IOException {
064    this.bufferSize = bufferSize;
065    this.bufferCount = bufferCount;
066    LOG.info("Allocating buffers total={}, sizePerBuffer={}, count={}",
067      StringUtils.byteDesc(capacity), StringUtils.byteDesc(bufferSize), bufferCount);
068    this.buffers = new ByteBuffer[bufferCount];
069    createBuffers(threadCount, alloc);
070  }
071
072  private void createBuffers(int threadCount, ByteBufferAllocator alloc) throws IOException {
073    ExecutorService pool = Executors.newFixedThreadPool(threadCount);
074    int perThreadCount = bufferCount / threadCount;
075    int reminder = bufferCount % threadCount;
076    try {
077      List<Future<ByteBuffer[]>> futures = new ArrayList<>(threadCount);
078      // Dispatch the creation task to each thread.
079      for (int i = 0; i < threadCount; i++) {
080        final int chunkSize = perThreadCount + ((i == threadCount - 1) ? reminder : 0);
081        futures.add(pool.submit(() -> {
082          ByteBuffer[] chunk = new ByteBuffer[chunkSize];
083          for (int k = 0; k < chunkSize; k++) {
084            chunk[k] = alloc.allocate(bufferSize);
085          }
086          return chunk;
087        }));
088      }
089      // Append the buffers created by each thread.
090      int bufferIndex = 0;
091      try {
092        for (Future<ByteBuffer[]> f : futures) {
093          for (ByteBuffer b : f.get()) {
094            this.buffers[bufferIndex++] = b;
095          }
096        }
097        assert bufferIndex == bufferCount;
098      } catch (Exception e) {
099        LOG.error("Buffer creation interrupted", e);
100        throw new IOException(e);
101      }
102    } finally {
103      pool.shutdownNow();
104    }
105  }
106
107  static int getBufferSize(long capacity) {
108    int bufferSize = DEFAULT_BUFFER_SIZE;
109    if (bufferSize > (capacity / 16)) {
110      bufferSize = (int) roundUp(capacity / 16, 32768);
111    }
112    return bufferSize;
113  }
114
115  private static int getBufferCount(long capacity) {
116    int bufferSize = getBufferSize(capacity);
117    return (int) (roundUp(capacity, bufferSize) / bufferSize);
118  }
119
120  private static long roundUp(long n, long to) {
121    return ((n + to - 1) / to) * to;
122  }
123
124  /**
125   * Transfers bytes from this buffers array into the given destination {@link ByteBuff}
126   * @param offset start position in this big logical array.
127   * @param dst the destination ByteBuff. Notice that its position will be advanced.
128   * @return number of bytes read
129   */
130  public int read(long offset, ByteBuff dst) {
131    return internalTransfer(offset, dst, READER);
132  }
133
134  /**
135   * Transfers bytes from the given source {@link ByteBuff} into this buffer array
136   * @param offset start offset of this big logical array.
137   * @param src the source ByteBuff. Notice that its position will be advanced.
138   * @return number of bytes write
139   */
140  public int write(long offset, ByteBuff src) {
141    return internalTransfer(offset, src, WRITER);
142  }
143
144  /**
145   * Transfer bytes from source {@link ByteBuff} to destination {@link ByteBuffer}. Position of both
146   * source and destination will be advanced.
147   */
148  private static final BiConsumer<ByteBuffer, ByteBuff> WRITER = (dst, src) -> {
149    int off = src.position(), len = dst.remaining();
150    src.get(dst, off, len);
151    src.position(off + len);
152  };
153
154  /**
155   * Transfer bytes from source {@link ByteBuffer} to destination {@link ByteBuff}, Position of both
156   * source and destination will be advanced.
157   */
158  private static final BiConsumer<ByteBuffer, ByteBuff> READER = (src, dst) -> {
159    int off = dst.position(), len = src.remaining(), srcOff = src.position();
160    dst.put(off, ByteBuff.wrap(src), srcOff, len);
161    src.position(srcOff + len);
162    dst.position(off + len);
163  };
164
165  /**
166   * Transferring all remaining bytes from b to the buffers array starting at offset, or
167   * transferring bytes from the buffers array at offset to b until b is filled. Notice that
168   * position of ByteBuff b will be advanced.
169   * @param offset where we start in the big logical array.
170   * @param b the ByteBuff to transfer from or to
171   * @param transfer the transfer interface.
172   * @return the length of bytes we transferred.
173   */
174  private int internalTransfer(long offset, ByteBuff b, BiConsumer<ByteBuffer, ByteBuff> transfer) {
175    int expectedTransferLen = b.remaining();
176    if (expectedTransferLen == 0) {
177      return 0;
178    }
179    BufferIterator it = new BufferIterator(offset, expectedTransferLen);
180    while (it.hasNext()) {
181      ByteBuffer a = it.next();
182      transfer.accept(a, b);
183      assert !a.hasRemaining();
184    }
185    assert expectedTransferLen == it.getSum() : "Expected transfer length (=" + expectedTransferLen
186        + ") don't match the actual transfer length(=" + it.getSum() + ")";
187    return expectedTransferLen;
188  }
189
190  /**
191   * Creates a sub-array from a given array of ByteBuffers from the given offset to the length
192   * specified. For eg, if there are 4 buffers forming an array each with length 10 and if we call
193   * asSubByteBuffers(5, 10) then we will create an sub-array consisting of two BBs and the first
194   * one be a BB from 'position' 5 to a 'length' 5 and the 2nd BB will be from 'position' 0 to
195   * 'length' 5.
196   * @param offset the position in the whole array which is composited by multiple byte buffers.
197   * @param len the length of bytes
198   * @return the underlying ByteBuffers, each ByteBuffer is a slice from the backend and will have a
199   *         zero position.
200   */
201  public ByteBuffer[] asSubByteBuffers(long offset, final int len) {
202    BufferIterator it = new BufferIterator(offset, len);
203    ByteBuffer[] mbb = new ByteBuffer[it.getBufferCount()];
204    for (int i = 0; i < mbb.length; i++) {
205      assert it.hasNext();
206      mbb[i] = it.next();
207    }
208    assert it.getSum() == len;
209    return mbb;
210  }
211
212  /**
213   * Iterator to fetch ByteBuffers from offset with given length in this big logical array.
214   */
215  private class BufferIterator implements Iterator<ByteBuffer> {
216    private final int len;
217    private int startBuffer, startOffset, endBuffer, endOffset;
218    private int curIndex, sum = 0;
219
220    private int index(long pos) {
221      return (int) (pos / bufferSize);
222    }
223
224    private int offset(long pos) {
225      return (int) (pos % bufferSize);
226    }
227
228    public BufferIterator(long offset, int len) {
229      assert len >= 0 && offset >= 0;
230      this.len = len;
231
232      this.startBuffer = index(offset);
233      this.startOffset = offset(offset);
234
235      this.endBuffer = index(offset + len);
236      this.endOffset = offset(offset + len);
237      if (startBuffer < endBuffer && endOffset == 0) {
238        endBuffer--;
239        endOffset = bufferSize;
240      }
241      assert startBuffer >= 0 && startBuffer < bufferCount;
242      assert endBuffer >= 0 && endBuffer < bufferCount;
243
244      // initialize the index to the first buffer index.
245      this.curIndex = startBuffer;
246    }
247
248    @Override
249    public boolean hasNext() {
250      return this.curIndex <= endBuffer;
251    }
252
253    /**
254     * The returned ByteBuffer is an sliced one, it won't affect the position or limit of the
255     * original one.
256     */
257    @Override
258    public ByteBuffer next() {
259      ByteBuffer bb = buffers[curIndex].duplicate();
260      if (curIndex == startBuffer) {
261        bb.position(startOffset).limit(Math.min(bufferSize, startOffset + len));
262      } else if (curIndex == endBuffer) {
263        bb.position(0).limit(endOffset);
264      } else {
265        bb.position(0).limit(bufferSize);
266      }
267      curIndex++;
268      sum += bb.remaining();
269      // Make sure that its pos is zero, it's important because MBB will count from zero for all nio
270      // ByteBuffers.
271      return bb.slice();
272    }
273
274    int getSum() {
275      return sum;
276    }
277
278    int getBufferCount() {
279      return this.endBuffer - this.startBuffer + 1;
280    }
281  }
282}