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;
019
020import java.io.IOException;
021import java.nio.ByteBuffer;
022import java.util.Collections;
023import java.util.Iterator;
024import java.util.Optional;
025import org.apache.hadoop.hbase.util.ByteBufferUtils;
026import org.apache.hadoop.hbase.util.Bytes;
027import org.apache.hadoop.hbase.util.ClassSize;
028import org.apache.yetus.audience.InterfaceAudience;
029
030/**
031 * This is a key only Cell implementation which is identical to {@link KeyValue.KeyOnlyKeyValue}
032 * with respect to key serialization but have its data in the form of Byte buffer (onheap and
033 * offheap).
034 */
035@InterfaceAudience.Private
036public class ByteBufferKeyOnlyKeyValue extends ByteBufferExtendedCell {
037  public static final int FIXED_OVERHEAD =
038    ClassSize.OBJECT + ClassSize.REFERENCE + (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT;
039  private ByteBuffer buf;
040  private int offset = 0; // offset into buffer where key starts at
041  private int length = 0; // length of this.
042  private short rowLen;
043
044  /**
045   * Used in cases where we want to avoid lot of garbage by allocating new objects with different
046   * keys. Use the emtpy construtor and set the keys using {@link #setKey(ByteBuffer, int, int)}
047   */
048  public ByteBufferKeyOnlyKeyValue() {
049  }
050
051  public ByteBufferKeyOnlyKeyValue(ByteBuffer buf, int offset, int length) {
052    setKey(buf, offset, length);
053  }
054
055  /**
056   * A setter that helps to avoid object creation every time and whenever there is a need to create
057   * new OffheapKeyOnlyKeyValue.
058   */
059  public void setKey(ByteBuffer key, int offset, int length) {
060    setKey(key, offset, length, ByteBufferUtils.toShort(key, offset));
061  }
062
063  /**
064   * A setter that helps to avoid object creation every time and whenever there is a need to create
065   * new OffheapKeyOnlyKeyValue.
066   * @param key    - the key part of the cell
067   * @param offset - offset of the cell
068   * @param length - length of the cell
069   * @param rowLen - the rowlen part of the cell
070   */
071  public void setKey(ByteBuffer key, int offset, int length, short rowLen) {
072    this.buf = key;
073    this.offset = offset;
074    this.length = length;
075    this.rowLen = rowLen;
076  }
077
078  @Override
079  public byte[] getRowArray() {
080    if (this.buf.hasArray()) {
081      return this.buf.array();
082    }
083    return CellUtil.cloneRow(this);
084  }
085
086  @Override
087  public int getRowOffset() {
088    if (this.buf.hasArray()) {
089      return getRowPosition() + this.buf.arrayOffset();
090    }
091    return 0;
092  }
093
094  @Override
095  public short getRowLength() {
096    return this.rowLen;
097  }
098
099  @Override
100  public byte[] getFamilyArray() {
101    if (this.buf.hasArray()) {
102      return this.buf.array();
103    }
104    return CellUtil.cloneFamily(this);
105  }
106
107  @Override
108  public int getFamilyOffset() {
109    if (this.buf.hasArray()) {
110      return getFamilyPosition() + this.buf.arrayOffset();
111    }
112    return 0;
113  }
114
115  @Override
116  public byte getFamilyLength() {
117    return getFamilyLength(getFamilyLengthPosition());
118  }
119
120  private byte getFamilyLength(int famLenPos) {
121    return ByteBufferUtils.toByte(this.buf, famLenPos);
122  }
123
124  @Override
125  public byte[] getQualifierArray() {
126    if (this.buf.hasArray()) {
127      return this.buf.array();
128    }
129    return CellUtil.cloneQualifier(this);
130  }
131
132  @Override
133  public int getQualifierOffset() {
134    if (this.buf.hasArray()) {
135      return getQualifierPosition() + this.buf.arrayOffset();
136    }
137    return 0;
138  }
139
140  @Override
141  public int getQualifierLength() {
142    return getQualifierLength(getRowLength(), getFamilyLength());
143  }
144
145  private int getQualifierLength(int rlength, int flength) {
146    return this.length - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0);
147  }
148
149  @Override
150  public long getTimestamp() {
151    return ByteBufferUtils.toLong(this.buf, getTimestampOffset());
152  }
153
154  private int getTimestampOffset() {
155    return this.offset + this.length - KeyValue.TIMESTAMP_TYPE_SIZE;
156  }
157
158  @Override
159  public byte getTypeByte() {
160    return getTypeByte(this.length);
161  }
162
163  byte getTypeByte(int keyLen) {
164    return ByteBufferUtils.toByte(this.buf, this.offset + keyLen - 1);
165  }
166
167  @Override
168  public void setSequenceId(long seqId) throws IOException {
169    throw new IllegalArgumentException("This is a key only Cell");
170  }
171
172  @Override
173  public void setTimestamp(long ts) throws IOException {
174    throw new IllegalArgumentException("This is a key only Cell");
175  }
176
177  @Override
178  public void setTimestamp(byte[] ts) throws IOException {
179    throw new IllegalArgumentException("This is a key only Cell");
180  }
181
182  @Override
183  public long getSequenceId() {
184    return 0;
185  }
186
187  @Override
188  public byte[] getValueArray() {
189    throw new IllegalArgumentException("This is a key only Cell");
190  }
191
192  @Override
193  public int getValueOffset() {
194    return 0;
195  }
196
197  @Override
198  public int getValueLength() {
199    return 0;
200  }
201
202  @Override
203  public byte[] getTagsArray() {
204    throw new IllegalArgumentException("This is a key only Cell");
205  }
206
207  @Override
208  public int getTagsOffset() {
209    return 0;
210  }
211
212  @Override
213  public int getTagsLength() {
214    return 0;
215  }
216
217  @Override
218  public ByteBuffer getRowByteBuffer() {
219    return this.buf;
220  }
221
222  @Override
223  public int getRowPosition() {
224    return this.offset + Bytes.SIZEOF_SHORT;
225  }
226
227  @Override
228  public ByteBuffer getFamilyByteBuffer() {
229    return this.buf;
230  }
231
232  @Override
233  public int getFamilyPosition() {
234    return getFamilyLengthPosition() + Bytes.SIZEOF_BYTE;
235  }
236
237  // The position in BB where the family length is added.
238  private int getFamilyLengthPosition() {
239    return getFamilyLengthPosition(getRowLength());
240  }
241
242  int getFamilyLengthPosition(int rowLength) {
243    return this.offset + Bytes.SIZEOF_SHORT + rowLength;
244  }
245
246  @Override
247  public ByteBuffer getQualifierByteBuffer() {
248    return this.buf;
249  }
250
251  @Override
252  public int getQualifierPosition() {
253    int famLenPos = getFamilyLengthPosition();
254    return famLenPos + Bytes.SIZEOF_BYTE + getFamilyLength(famLenPos);
255  }
256
257  @Override
258  public ByteBuffer getValueByteBuffer() {
259    throw new IllegalArgumentException("This is a key only Cell");
260  }
261
262  @Override
263  public int getValuePosition() {
264    return 0;
265  }
266
267  @Override
268  public ByteBuffer getTagsByteBuffer() {
269    throw new IllegalArgumentException("This is a key only Cell");
270  }
271
272  @Override
273  public int getTagsPosition() {
274    return 0;
275  }
276
277  @Override
278  public String toString() {
279    return CellUtil.toString(this, false);
280  }
281
282  @Override
283  public Iterator<Tag> getTags() {
284    return Collections.emptyIterator();
285  }
286
287  @Override
288  public Optional<Tag> getTag(byte type) {
289    return Optional.empty();
290  }
291
292  @Override
293  public long heapSize() {
294    if (this.buf.hasArray()) {
295      return ClassSize.align(FIXED_OVERHEAD + length);
296    }
297    return ClassSize.align(FIXED_OVERHEAD);
298  }
299}