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;
023
024import org.apache.hadoop.hbase.io.HeapSize;
025import org.apache.hadoop.hbase.util.ByteBufferUtils;
026import org.apache.yetus.audience.InterfaceAudience;
027
028/**
029 * Extension to {@link Cell} with server side required functions. Server side Cell implementations
030 * must implement this.
031 */
032@InterfaceAudience.Private
033public interface ExtendedCell extends RawCell, HeapSize {
034  int CELL_NOT_BASED_ON_CHUNK = -1;
035
036  /**
037   * Write this cell to an OutputStream in a {@link KeyValue} format.
038   * <br> KeyValue format <br>
039   * <code>&lt;4 bytes keylength&gt; &lt;4 bytes valuelength&gt; &lt;2 bytes rowlength&gt;
040   * &lt;row&gt; &lt;1 byte columnfamilylength&gt; &lt;columnfamily&gt; &lt;columnqualifier&gt;
041   * &lt;8 bytes timestamp&gt; &lt;1 byte keytype&gt; &lt;value&gt; &lt;2 bytes tagslength&gt;
042   * &lt;tags&gt;</code>
043   * @param out Stream to which cell has to be written
044   * @param withTags Whether to write tags.
045   * @return how many bytes are written.
046   * @throws IOException
047   */
048  // TODO remove the boolean param once HBASE-16706 is done.
049  default int write(OutputStream out, boolean withTags) throws IOException {
050    // Key length and then value length
051    ByteBufferUtils.putInt(out, KeyValueUtil.keyLength(this));
052    ByteBufferUtils.putInt(out, getValueLength());
053
054    // Key
055    PrivateCellUtil.writeFlatKey(this, out);
056
057    if (getValueLength() > 0) {
058      // Value
059      out.write(getValueArray(), getValueOffset(), getValueLength());
060    }
061
062    // Tags length and tags byte array
063    if (withTags && getTagsLength() > 0) {
064      // Tags length
065      out.write((byte)(0xff & (getTagsLength() >> 8)));
066      out.write((byte)(0xff & getTagsLength()));
067
068      // Tags byte array
069      out.write(getTagsArray(), getTagsOffset(), getTagsLength());
070    }
071
072    return getSerializedSize(withTags);
073  }
074
075  /**
076   * @param withTags Whether to write tags.
077   * @return Bytes count required to serialize this Cell in a {@link KeyValue} format.
078   * <br> KeyValue format <br>
079   * <code>&lt;4 bytes keylength&gt; &lt;4 bytes valuelength&gt; &lt;2 bytes rowlength&gt;
080   * &lt;row&gt; &lt;1 byte columnfamilylength&gt; &lt;columnfamily&gt; &lt;columnqualifier&gt;
081   * &lt;8 bytes timestamp&gt; &lt;1 byte keytype&gt; &lt;value&gt; &lt;2 bytes tagslength&gt;
082   * &lt;tags&gt;</code>
083   */
084  // TODO remove the boolean param once HBASE-16706 is done.
085  default int getSerializedSize(boolean withTags) {
086    return KeyValueUtil.length(getRowLength(), getFamilyLength(), getQualifierLength(),
087        getValueLength(), getTagsLength(), withTags);
088  }
089
090  /**
091   * @return Serialized size (defaults to include tag length).
092   */
093  default int getSerializedSize() {
094    return getSerializedSize(true);
095  }
096
097  /**
098   * Write this Cell into the given buf's offset in a {@link KeyValue} format.
099   * @param buf The buffer where to write the Cell.
100   * @param offset The offset within buffer, to write the Cell.
101   */
102  default void write(ByteBuffer buf, int offset) {
103    KeyValueUtil.appendTo(this, buf, offset, true);
104  }
105
106  /**
107   * Does a deep copy of the contents to a new memory area and returns it as a new cell.
108   * @return The deep cloned cell
109   */
110  default ExtendedCell deepClone() {
111    // When being added to the memstore, deepClone() is called and KeyValue has less heap overhead.
112    return new KeyValue(this);
113  }
114
115  /**
116   * Extracts the id of the backing bytebuffer of this cell if it was obtained from fixed sized
117   * chunks as in case of MemstoreLAB
118   * @return the chunk id if the cell is backed by fixed sized Chunks, else return
119   * {@link #CELL_NOT_BASED_ON_CHUNK}; i.e. -1.
120   */
121  default int getChunkId() {
122    return CELL_NOT_BASED_ON_CHUNK;
123  }
124
125  /**
126   * Sets with the given seqId.
127   * @param seqId sequence ID
128   */
129  void setSequenceId(long seqId) throws IOException;
130
131  /**
132   * Sets with the given timestamp.
133   * @param ts timestamp
134   */
135  void setTimestamp(long ts) throws IOException;
136
137  /**
138   * Sets with the given timestamp.
139   * @param ts buffer containing the timestamp value
140   */
141  void setTimestamp(byte[] ts) throws IOException;
142
143  /**
144   * A region-specific unique monotonically increasing sequence ID given to each Cell. It always
145   * exists for cells in the memstore but is not retained forever. It will be kept for
146   * {@link HConstants#KEEP_SEQID_PERIOD} days, but generally becomes irrelevant after the cell's
147   * row is no longer involved in any operations that require strict consistency.
148   * @return seqId (always &gt; 0 if exists), or 0 if it no longer exists
149   */
150  long getSequenceId();
151
152  /**
153   * Contiguous raw bytes representing tags that may start at any index in the containing array.
154   * @return the tags byte array
155   */
156  byte[] getTagsArray();
157
158  /**
159   * @return the first offset where the tags start in the Cell
160   */
161  int getTagsOffset();
162
163  /**
164   * HBase internally uses 2 bytes to store tags length in Cell. As the tags length is always a
165   * non-negative number, to make good use of the sign bit, the max of tags length is defined 2 *
166   * Short.MAX_VALUE + 1 = 65535. As a result, the return type is int, because a short is not
167   * capable of handling that. Please note that even if the return type is int, the max tags length
168   * is far less than Integer.MAX_VALUE.
169   * @return the total length of the tags in the Cell.
170   */
171  int getTagsLength();
172
173  /**
174   * @return The byte representation of the KeyValue.TYPE of this cell: one of Put, Delete, etc
175   */
176  byte getTypeByte();
177}