HBASE-24163 MOB compactor implementations should use format specifiers when calling...
[hbase.git] / hbase-common / src / main / java / org / apache / hadoop / hbase / ExtendedCell.java
blob20f8e59ef08b630f29939d906577d0dce0b357fb
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase;
20 import java.io.IOException;
21 import java.io.OutputStream;
22 import java.nio.ByteBuffer;
24 import org.apache.hadoop.hbase.io.HeapSize;
25 import org.apache.hadoop.hbase.util.ByteBufferUtils;
26 import org.apache.yetus.audience.InterfaceAudience;
28 /**
29 * Extension to {@link Cell} with server side required functions. Server side Cell implementations
30 * must implement this.
32 @InterfaceAudience.Private
33 public interface ExtendedCell extends RawCell, HeapSize {
34 int CELL_NOT_BASED_ON_CHUNK = -1;
36 /**
37 * Write this cell to an OutputStream in a {@link KeyValue} format.
38 * <br> KeyValue format <br>
39 * <code>&lt;4 bytes keylength&gt; &lt;4 bytes valuelength&gt; &lt;2 bytes rowlength&gt;
40 * &lt;row&gt; &lt;1 byte columnfamilylength&gt; &lt;columnfamily&gt; &lt;columnqualifier&gt;
41 * &lt;8 bytes timestamp&gt; &lt;1 byte keytype&gt; &lt;value&gt; &lt;2 bytes tagslength&gt;
42 * &lt;tags&gt;</code>
43 * @param out Stream to which cell has to be written
44 * @param withTags Whether to write tags.
45 * @return how many bytes are written.
46 * @throws IOException
48 // TODO remove the boolean param once HBASE-16706 is done.
49 default int write(OutputStream out, boolean withTags) throws IOException {
50 // Key length and then value length
51 ByteBufferUtils.putInt(out, KeyValueUtil.keyLength(this));
52 ByteBufferUtils.putInt(out, getValueLength());
54 // Key
55 PrivateCellUtil.writeFlatKey(this, out);
57 if (getValueLength() > 0) {
58 // Value
59 out.write(getValueArray(), getValueOffset(), getValueLength());
62 // Tags length and tags byte array
63 if (withTags && getTagsLength() > 0) {
64 // Tags length
65 out.write((byte)(0xff & (getTagsLength() >> 8)));
66 out.write((byte)(0xff & getTagsLength()));
68 // Tags byte array
69 out.write(getTagsArray(), getTagsOffset(), getTagsLength());
72 return getSerializedSize(withTags);
75 /**
76 * @param withTags Whether to write tags.
77 * @return Bytes count required to serialize this Cell in a {@link KeyValue} format.
78 * <br> KeyValue format <br>
79 * <code>&lt;4 bytes keylength&gt; &lt;4 bytes valuelength&gt; &lt;2 bytes rowlength&gt;
80 * &lt;row&gt; &lt;1 byte columnfamilylength&gt; &lt;columnfamily&gt; &lt;columnqualifier&gt;
81 * &lt;8 bytes timestamp&gt; &lt;1 byte keytype&gt; &lt;value&gt; &lt;2 bytes tagslength&gt;
82 * &lt;tags&gt;</code>
84 // TODO remove the boolean param once HBASE-16706 is done.
85 default int getSerializedSize(boolean withTags) {
86 return KeyValueUtil.length(getRowLength(), getFamilyLength(), getQualifierLength(),
87 getValueLength(), getTagsLength(), withTags);
90 /**
91 * @return Serialized size (defaults to include tag length).
93 @Override
94 default int getSerializedSize() {
95 return getSerializedSize(true);
98 /**
99 * Write this Cell into the given buf's offset in a {@link KeyValue} format.
100 * @param buf The buffer where to write the Cell.
101 * @param offset The offset within buffer, to write the Cell.
103 default void write(ByteBuffer buf, int offset) {
104 KeyValueUtil.appendTo(this, buf, offset, true);
108 * Does a deep copy of the contents to a new memory area and returns it as a new cell.
109 * @return The deep cloned cell
111 default ExtendedCell deepClone() {
112 // When being added to the memstore, deepClone() is called and KeyValue has less heap overhead.
113 return new KeyValue(this);
117 * Extracts the id of the backing bytebuffer of this cell if it was obtained from fixed sized
118 * chunks as in case of MemstoreLAB
119 * @return the chunk id if the cell is backed by fixed sized Chunks, else return
120 * {@link #CELL_NOT_BASED_ON_CHUNK}; i.e. -1.
122 default int getChunkId() {
123 return CELL_NOT_BASED_ON_CHUNK;
127 * Sets with the given seqId.
128 * @param seqId sequence ID
130 void setSequenceId(long seqId) throws IOException;
133 * Sets with the given timestamp.
134 * @param ts timestamp
136 void setTimestamp(long ts) throws IOException;
139 * Sets with the given timestamp.
140 * @param ts buffer containing the timestamp value
142 void setTimestamp(byte[] ts) throws IOException;
145 * A region-specific unique monotonically increasing sequence ID given to each Cell. It always
146 * exists for cells in the memstore but is not retained forever. It will be kept for
147 * {@link HConstants#KEEP_SEQID_PERIOD} days, but generally becomes irrelevant after the cell's
148 * row is no longer involved in any operations that require strict consistency.
149 * @return seqId (always &gt; 0 if exists), or 0 if it no longer exists
151 long getSequenceId();
154 * Contiguous raw bytes representing tags that may start at any index in the containing array.
155 * @return the tags byte array
157 byte[] getTagsArray();
160 * @return the first offset where the tags start in the Cell
162 int getTagsOffset();
165 * HBase internally uses 2 bytes to store tags length in Cell. As the tags length is always a
166 * non-negative number, to make good use of the sign bit, the max of tags length is defined 2 *
167 * Short.MAX_VALUE + 1 = 65535. As a result, the return type is int, because a short is not
168 * capable of handling that. Please note that even if the return type is int, the max tags length
169 * is far less than Integer.MAX_VALUE.
170 * @return the total length of the tags in the Cell.
172 int getTagsLength();
175 * @return The byte representation of the KeyValue.TYPE of this cell: one of Put, Delete, etc
177 byte getTypeByte();