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.io.OutputStream;
022import java.nio.ByteBuffer;
023import org.apache.hadoop.hbase.util.ByteBufferUtils;
024import org.apache.hadoop.hbase.util.Bytes;
025import org.apache.hadoop.hbase.util.ClassSize;
026import org.apache.yetus.audience.InterfaceAudience;
027
028import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
029
030/**
031 * This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
032 * off heap/ on heap ByteBuffer
033 */
034@InterfaceAudience.Private
035public class ByteBufferKeyValue extends ByteBufferExtendedCell {
036
037  protected final ByteBuffer buf;
038  protected final int offset;
039  protected final int length;
040  private long seqId = 0;
041
042  public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
043      + (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_LONG;
044
045  public ByteBufferKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
046    this.buf = buf;
047    this.offset = offset;
048    this.length = length;
049    this.seqId = seqId;
050  }
051
052  public ByteBufferKeyValue(ByteBuffer buf, int offset, int length) {
053    this.buf = buf;
054    this.offset = offset;
055    this.length = length;
056  }
057
058  @VisibleForTesting
059  public ByteBuffer getBuffer() {
060    return this.buf;
061  }
062
063  @VisibleForTesting
064  public int getOffset() {
065    return this.offset;
066  }
067
068  @Override
069  public byte[] getRowArray() {
070    return CellUtil.cloneRow(this);
071  }
072
073  @Override
074  public int getRowOffset() {
075    return 0;
076  }
077
078  @Override
079  public short getRowLength() {
080    return ByteBufferUtils.toShort(this.buf, this.offset + KeyValue.ROW_OFFSET);
081  }
082
083  @Override
084  public byte[] getFamilyArray() {
085    return CellUtil.cloneFamily(this);
086  }
087
088  @Override
089  public int getFamilyOffset() {
090    return 0;
091  }
092
093  @Override
094  public byte getFamilyLength() {
095    return getFamilyLength(getFamilyLengthPosition());
096  }
097
098  int getFamilyLengthPosition() {
099    return getFamilyLengthPosition(getRowLength());
100  }
101
102  int getFamilyLengthPosition(int rowLength) {
103    return this.offset + KeyValue.ROW_KEY_OFFSET + rowLength;
104  }
105
106  byte getFamilyLength(int famLenPos) {
107    return ByteBufferUtils.toByte(this.buf, famLenPos);
108  }
109
110  @Override
111  public byte[] getQualifierArray() {
112    return CellUtil.cloneQualifier(this);
113  }
114
115  @Override
116  public int getQualifierOffset() {
117    return 0;
118  }
119
120  @Override
121  public int getQualifierLength() {
122    return getQualifierLength(getKeyLength(), getRowLength(), getFamilyLength());
123  }
124
125  int getQualifierLength(int keyLength, int rlength, int flength) {
126    return keyLength - (int) KeyValue.getKeyDataStructureSize(rlength, flength, 0);
127  }
128
129  @Override
130  public long getTimestamp() {
131    return getTimestamp(getKeyLength());
132  }
133
134  long getTimestamp(int keyLength) {
135    int offset = getTimestampOffset(keyLength);
136    return ByteBufferUtils.toLong(this.buf, offset);
137  }
138
139  int getKeyLength() {
140    return ByteBufferUtils.toInt(this.buf, this.offset);
141  }
142
143  private int getTimestampOffset(int keyLen) {
144    return this.offset + KeyValue.ROW_OFFSET + keyLen - KeyValue.TIMESTAMP_TYPE_SIZE;
145  }
146
147  @Override
148  public byte getTypeByte() {
149    return getTypeByte(getKeyLength());
150  }
151
152  byte getTypeByte(int keyLen) {
153    return ByteBufferUtils.toByte(this.buf, this.offset + keyLen - 1 + KeyValue.ROW_OFFSET);
154  }
155
156  @Override
157  public long getSequenceId() {
158    return this.seqId;
159  }
160
161  @Override
162  public void setSequenceId(long seqId) {
163    this.seqId = seqId;
164  }
165
166  @Override
167  public byte[] getValueArray() {
168    return CellUtil.cloneValue(this);
169  }
170
171  @Override
172  public int getValueOffset() {
173    return 0;
174  }
175
176  @Override
177  public int getValueLength() {
178    return ByteBufferUtils.toInt(this.buf, this.offset + Bytes.SIZEOF_INT);
179  }
180
181  @Override
182  public byte[] getTagsArray() {
183    return CellUtil.cloneTags(this);
184  }
185
186  @Override
187  public int getTagsOffset() {
188    return 0;
189  }
190
191  @Override
192  public int getTagsLength() {
193    int tagsLen = this.length - (getKeyLength() + getValueLength()
194        + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE);
195    if (tagsLen > 0) {
196      // There are some Tag bytes in the byte[]. So reduce 2 bytes which is
197      // added to denote the tags
198      // length
199      tagsLen -= KeyValue.TAGS_LENGTH_SIZE;
200    }
201    return tagsLen;
202  }
203
204  @Override
205  public ByteBuffer getRowByteBuffer() {
206    return this.buf;
207  }
208
209  @Override
210  public int getRowPosition() {
211    return this.offset + KeyValue.ROW_KEY_OFFSET;
212  }
213
214  @Override
215  public ByteBuffer getFamilyByteBuffer() {
216    return this.buf;
217  }
218
219  @Override
220  public int getFamilyPosition() {
221    return getFamilyPosition(getFamilyLengthPosition());
222  }
223
224  public int getFamilyPosition(int familyLengthPosition) {
225    return familyLengthPosition + Bytes.SIZEOF_BYTE;
226  }
227
228  @Override
229  public ByteBuffer getQualifierByteBuffer() {
230    return this.buf;
231  }
232
233  @Override
234  public int getQualifierPosition() {
235    return getQualifierPosition(getFamilyPosition(), getFamilyLength());
236  }
237
238  int getQualifierPosition(int familyPosition, int familyLength) {
239    return familyPosition + familyLength;
240  }
241
242  @Override
243  public ByteBuffer getValueByteBuffer() {
244    return this.buf;
245  }
246
247  @Override
248  public int getValuePosition() {
249    return this.offset + KeyValue.ROW_OFFSET + getKeyLength();
250  }
251
252  @Override
253  public ByteBuffer getTagsByteBuffer() {
254    return this.buf;
255  }
256
257  @Override
258  public int getTagsPosition() {
259    int tagsLen = getTagsLength();
260    if (tagsLen == 0) {
261      return this.offset + this.length;
262    }
263    return this.offset + this.length - tagsLen;
264  }
265
266  @Override
267  public long heapSize() {
268    if (this.buf.hasArray()) {
269      return ClassSize.align(FIXED_OVERHEAD + length);
270    }
271    return ClassSize.align(FIXED_OVERHEAD) + KeyValueUtil.length(this);
272  }
273
274  @Override
275  public int write(OutputStream out, boolean withTags) throws IOException {
276    int length = getSerializedSize(withTags);
277    ByteBufferUtils.copyBufferToStream(out, this.buf, this.offset, length);
278    return length;
279  }
280
281  @Override
282  public int getSerializedSize(boolean withTags) {
283    if (withTags) {
284      return this.length;
285    }
286    return getKeyLength() + this.getValueLength() + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
287  }
288
289  @Override
290  public void write(ByteBuffer buf, int offset) {
291    ByteBufferUtils.copyFromBufferToBuffer(this.buf, buf, this.offset, offset, this.length);
292  }
293
294  @Override
295  public String toString() {
296    return CellUtil.toString(this, true);
297  }
298
299  @Override
300  public void setTimestamp(long ts) throws IOException {
301    ByteBufferUtils.copyFromArrayToBuffer(this.buf, this.getTimestampOffset(), Bytes.toBytes(ts), 0,
302      Bytes.SIZEOF_LONG);
303  }
304
305  private int getTimestampOffset() {
306    return this.offset + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE
307        + getKeyLength() - KeyValue.TIMESTAMP_TYPE_SIZE;
308  }
309
310  @Override
311  public void setTimestamp(byte[] ts) throws IOException {
312    ByteBufferUtils.copyFromArrayToBuffer(this.buf, this.getTimestampOffset(), ts, 0,
313        Bytes.SIZEOF_LONG);
314  }
315
316  @Override
317  public ExtendedCell deepClone() {
318    byte[] copy = new byte[this.length];
319    ByteBufferUtils.copyFromBufferToArray(copy, this.buf, this.offset, 0, this.length);
320    KeyValue kv = new KeyValue(copy, 0, copy.length);
321    kv.setSequenceId(this.getSequenceId());
322    return kv;
323  }
324
325  /**
326   * Needed doing 'contains' on List. Only compares the key portion, not the value.
327   */
328  @Override
329  public boolean equals(Object other) {
330    if (!(other instanceof Cell)) {
331      return false;
332    }
333    return CellUtil.equals(this, (Cell) other);
334  }
335
336  /**
337   * In line with {@link #equals(Object)}, only uses the key portion, not the value.
338   */
339  @Override
340  public int hashCode() {
341    return calculateHashForKey(this);
342  }
343
344  private int calculateHashForKey(ByteBufferExtendedCell cell) {
345    int rowHash = ByteBufferUtils.hashCode(cell.getRowByteBuffer(), cell.getRowPosition(),
346      cell.getRowLength());
347    int familyHash = ByteBufferUtils.hashCode(cell.getFamilyByteBuffer(), cell.getFamilyPosition(),
348      cell.getFamilyLength());
349    int qualifierHash = ByteBufferUtils.hashCode(cell.getQualifierByteBuffer(),
350      cell.getQualifierPosition(), cell.getQualifierLength());
351
352    int hash = 31 * rowHash + familyHash;
353    hash = 31 * hash + qualifierHash;
354    hash = 31 * hash + (int) cell.getTimestamp();
355    hash = 31 * hash + cell.getTypeByte();
356    return hash;
357  }
358}