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><4 bytes keylength> <4 bytes valuelength> <2 bytes rowlength> 040 * <row> <1 byte columnfamilylength> <columnfamily> <columnqualifier> 041 * <8 bytes timestamp> <1 byte keytype> <value> <2 bytes tagslength> 042 * <tags></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><4 bytes keylength> <4 bytes valuelength> <2 bytes rowlength> 080 * <row> <1 byte columnfamilylength> <columnfamily> <columnqualifier> 081 * <8 bytes timestamp> <1 byte keytype> <value> <2 bytes tagslength> 082 * <tags></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 > 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}