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  @Override
094  default int getSerializedSize() {
095    return getSerializedSize(true);
096  }
097
098  /**
099   * 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.
102   */
103  default void write(ByteBuffer buf, int offset) {
104    KeyValueUtil.appendTo(this, buf, offset, true);
105  }
106
107  /**
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
110   */
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);
114  }
115
116  /**
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.
121   */
122  default int getChunkId() {
123    return CELL_NOT_BASED_ON_CHUNK;
124  }
125
126  /**
127   * Sets with the given seqId.
128   * @param seqId sequence ID
129   */
130  void setSequenceId(long seqId) throws IOException;
131
132  /**
133   * Sets with the given timestamp.
134   * @param ts timestamp
135   */
136  void setTimestamp(long ts) throws IOException;
137
138  /**
139   * Sets with the given timestamp.
140   * @param ts buffer containing the timestamp value
141   */
142  void setTimestamp(byte[] ts) throws IOException;
143
144  /**
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
150   */
151  long getSequenceId();
152
153  /**
154   * Contiguous raw bytes representing tags that may start at any index in the containing array.
155   * @return the tags byte array
156   */
157  byte[] getTagsArray();
158
159  /**
160   * @return the first offset where the tags start in the Cell
161   */
162  int getTagsOffset();
163
164  /**
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.
171   */
172  int getTagsLength();
173
174  /**
175   * @return The byte representation of the KeyValue.TYPE of this cell: one of Put, Delete, etc
176   */
177  byte getTypeByte();
178}