001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  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,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase;
020
021import java.io.IOException;
022import java.nio.ByteBuffer;
023import java.util.Collections;
024import java.util.Iterator;
025import java.util.Optional;
026import org.apache.hadoop.hbase.util.ByteBufferUtils;
027import org.apache.hadoop.hbase.util.Bytes;
028import org.apache.hadoop.hbase.util.ClassSize;
029import org.apache.yetus.audience.InterfaceAudience;
030
031/**
032 * This is a key only Cell implementation which is identical to {@link KeyValue.KeyOnlyKeyValue}
033 * with respect to key serialization but have its data in the form of Byte buffer
034 * (onheap and offheap).
035 */
036@InterfaceAudience.Private
037public class ByteBufferKeyOnlyKeyValue extends ByteBufferExtendedCell {
038  public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
039      + (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT;
040  private ByteBuffer buf;
041  private int offset = 0; // offset into buffer where key starts at
042  private int length = 0; // length of this.
043  private short rowLen;
044
045  /**
046   * Used in cases where we want to avoid lot of garbage by allocating new objects with different
047   * keys. Use the emtpy construtor and set the keys using {@link #setKey(ByteBuffer, int, int)}
048   */
049  public ByteBufferKeyOnlyKeyValue() {
050  }
051
052  public ByteBufferKeyOnlyKeyValue(ByteBuffer buf, int offset, int length) {
053    setKey(buf, offset, length);
054  }
055
056  /**
057   * A setter that helps to avoid object creation every time and whenever
058   * there is a need to create new OffheapKeyOnlyKeyValue.
059   * @param key
060   * @param offset
061   * @param length
062   */
063  public void setKey(ByteBuffer key, int offset, int length) {
064    setKey(key, offset, length, ByteBufferUtils.toShort(key, offset));
065  }
066
067  /**
068   * A setter that helps to avoid object creation every time and whenever
069   * there is a need to create new OffheapKeyOnlyKeyValue.
070   * @param key - the key part of the cell
071   * @param offset - offset of the cell
072   * @param length - length of the cell
073   * @param rowLen - the rowlen part of the cell
074   */
075  public void setKey(ByteBuffer key, int offset, int length, short rowLen) {
076    this.buf = key;
077    this.offset = offset;
078    this.length = length;
079    this.rowLen = rowLen;
080  }
081
082  @Override
083  public byte[] getRowArray() {
084    if (this.buf.hasArray()) {
085      return this.buf.array();
086    }
087    return CellUtil.cloneRow(this);
088  }
089
090  @Override
091  public int getRowOffset() {
092    if (this.buf.hasArray()) {
093      return getRowPosition() + this.buf.arrayOffset();
094    }
095    return 0;
096  }
097
098  @Override
099  public short getRowLength() {
100    return this.rowLen;
101  }
102
103  @Override
104  public byte[] getFamilyArray() {
105    if (this.buf.hasArray()) {
106      return this.buf.array();
107    }
108    return CellUtil.cloneFamily(this);
109  }
110
111  @Override
112  public int getFamilyOffset() {
113    if (this.buf.hasArray()) {
114      return getFamilyPosition() + this.buf.arrayOffset();
115    }
116    return 0;
117  }
118
119  @Override
120  public byte getFamilyLength() {
121    return getFamilyLength(getFamilyLengthPosition());
122  }
123
124  private byte getFamilyLength(int famLenPos) {
125    return ByteBufferUtils.toByte(this.buf, famLenPos);
126  }
127
128  @Override
129  public byte[] getQualifierArray() {
130    if (this.buf.hasArray()) {
131      return this.buf.array();
132    }
133    return CellUtil.cloneQualifier(this);
134  }
135
136  @Override
137  public int getQualifierOffset() {
138    if (this.buf.hasArray()) {
139      return getQualifierPosition() + this.buf.arrayOffset();
140    }
141    return 0;
142  }
143
144  @Override
145  public int getQualifierLength() {
146    return getQualifierLength(getRowLength(), getFamilyLength());
147  }
148
149  private int getQualifierLength(int rlength, int flength) {
150    return this.length - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0);
151  }
152
153  @Override
154  public long getTimestamp() {
155    return ByteBufferUtils.toLong(this.buf, getTimestampOffset());
156  }
157
158  private int getTimestampOffset() {
159    return this.offset + this.length - KeyValue.TIMESTAMP_TYPE_SIZE;
160  }
161
162  @Override
163  public byte getTypeByte() {
164    return ByteBufferUtils.toByte(this.buf, this.offset + this.length - 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 this.offset + Bytes.SIZEOF_SHORT + getRowLength();
240  }
241
242  @Override
243  public ByteBuffer getQualifierByteBuffer() {
244    return this.buf;
245  }
246
247  @Override
248  public int getQualifierPosition() {
249    int famLenPos = getFamilyLengthPosition();
250    return famLenPos + Bytes.SIZEOF_BYTE + getFamilyLength(famLenPos);
251  }
252
253  @Override
254  public ByteBuffer getValueByteBuffer() {
255    throw new IllegalArgumentException("This is a key only Cell");
256  }
257
258  @Override
259  public int getValuePosition() {
260    return 0;
261  }
262
263  @Override
264  public ByteBuffer getTagsByteBuffer() {
265    throw new IllegalArgumentException("This is a key only Cell");
266  }
267
268  @Override
269  public int getTagsPosition() {
270    return 0;
271  }
272
273  @Override
274  public String toString() {
275    return CellUtil.toString(this, false);
276  }
277
278  @Override
279  public Iterator<Tag> getTags() {
280    return Collections.emptyIterator();
281  }
282
283  @Override
284  public Optional<Tag> getTag(byte type) {
285    return Optional.empty();
286  }
287
288  @Override
289  public long heapSize() {
290    if (this.buf.hasArray()) {
291      return ClassSize.align(FIXED_OVERHEAD + length);
292    }
293    return ClassSize.align(FIXED_OVERHEAD);
294  }
295}