View Javadoc

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
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
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.
17   */
18  
19  package org.apache.hadoop.hbase.codec.prefixtree;
20  
21  import java.io.IOException;
22  import java.io.InputStream;
23  import java.io.OutputStream;
24  import java.nio.ByteBuffer;
25  
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.codec.prefixtree.encode.other.LongEncoder;
28  import org.apache.hadoop.hbase.util.Bytes;
29  import org.apache.hadoop.hbase.util.vint.UVIntTool;
30  import org.apache.hadoop.hbase.util.vint.UVLongTool;
31  
32  /**
33   * Information about the block.  Stored at the beginning of the byte[].  Contains things
34   * like minimum timestamp and width of FInts in the row tree.
35   *
36   * Most fields stored in VInts that get decoded on the first access of each new block.
37   */
38  @InterfaceAudience.Private
39  public class PrefixTreeBlockMeta {
40  
41    /******************* static fields ********************/
42  
43    public static final int VERSION = 0;
44  
45    public static final int MAX_FAMILY_LENGTH = Byte.MAX_VALUE;// hard-coded in KeyValue
46  
47    public static final int
48      NUM_LONGS = 2,
49      NUM_INTS = 28,
50      NUM_SHORTS = 0,//keyValueTypeWidth not persisted
51      NUM_SINGLE_BYTES = 2,
52      MAX_BYTES = Bytes.SIZEOF_LONG * NUM_LONGS
53          + Bytes.SIZEOF_SHORT * NUM_SHORTS
54          + Bytes.SIZEOF_INT * NUM_INTS
55          + NUM_SINGLE_BYTES;
56  
57  
58    /**************** transient fields *********************/
59  
60    protected int arrayOffset;
61    protected int bufferOffset;
62  
63  
64    /**************** persisted fields **********************/
65  
66    // PrefixTree version to allow future format modifications
67    protected int version;
68    protected int numMetaBytes;
69    protected int numKeyValueBytes;
70    protected boolean includesMvccVersion;//probably don't need this explicitly, but only 1 byte
71  
72    // split the byte[] into 6 sections for the different data types
73    protected int numRowBytes;
74    protected int numFamilyBytes;
75    protected int numQualifierBytes;
76    protected int numTimestampBytes;
77    protected int numMvccVersionBytes;
78    protected int numValueBytes;
79    protected int numTagsBytes;
80  
81    // number of bytes in each section of fixed width FInts
82    protected int nextNodeOffsetWidth;
83    protected int familyOffsetWidth;
84    protected int qualifierOffsetWidth;
85    protected int timestampIndexWidth;
86    protected int mvccVersionIndexWidth;
87    protected int valueOffsetWidth;
88    protected int valueLengthWidth;
89    protected int tagsOffsetWidth;
90  
91    // used to pre-allocate structures for reading
92    protected int rowTreeDepth;
93    protected int maxRowLength;
94    protected int maxQualifierLength;
95    protected int maxTagsLength;
96  
97    // the timestamp from which the deltas are calculated
98    protected long minTimestamp;
99    protected int timestampDeltaWidth;
100   protected long minMvccVersion;
101   protected int mvccVersionDeltaWidth;
102 
103   protected boolean allSameType;
104   protected byte allTypes;
105 
106   protected int numUniqueRows;
107   protected int numUniqueFamilies;
108   protected int numUniqueQualifiers;
109   protected int numUniqueTags;
110 
111 
112   /***************** constructors ********************/
113 
114   public PrefixTreeBlockMeta() {
115   }
116 
117   public PrefixTreeBlockMeta(InputStream is) throws IOException{
118     this.version = VERSION;
119     this.arrayOffset = 0;
120     this.bufferOffset = 0;
121     readVariableBytesFromInputStream(is);
122   }
123 
124   /**
125    * @param buffer positioned at start of PtBlockMeta
126    */
127   public PrefixTreeBlockMeta(ByteBuffer buffer) {
128     initOnBlock(buffer);
129   }
130 
131   public void initOnBlock(ByteBuffer buffer) {
132     arrayOffset = buffer.arrayOffset();
133     bufferOffset = buffer.position();
134     readVariableBytesFromArray(buffer.array(), arrayOffset + bufferOffset);
135   }
136 
137 
138   /**************** operate on each field **********************/
139 
140   public int calculateNumMetaBytes(){
141     int numBytes = 0;
142     numBytes += UVIntTool.numBytes(version);
143     numBytes += UVLongTool.numBytes(numMetaBytes);
144     numBytes += UVIntTool.numBytes(numKeyValueBytes);
145     ++numBytes;//os.write(getIncludesMvccVersion());
146 
147     numBytes += UVIntTool.numBytes(numRowBytes);
148     numBytes += UVIntTool.numBytes(numFamilyBytes);
149     numBytes += UVIntTool.numBytes(numQualifierBytes);
150     numBytes += UVIntTool.numBytes(numTagsBytes);
151     numBytes += UVIntTool.numBytes(numTimestampBytes);
152     numBytes += UVIntTool.numBytes(numMvccVersionBytes);
153     numBytes += UVIntTool.numBytes(numValueBytes);
154 
155     numBytes += UVIntTool.numBytes(nextNodeOffsetWidth);
156     numBytes += UVIntTool.numBytes(familyOffsetWidth);
157     numBytes += UVIntTool.numBytes(qualifierOffsetWidth);
158     numBytes += UVIntTool.numBytes(tagsOffsetWidth);
159     numBytes += UVIntTool.numBytes(timestampIndexWidth);
160     numBytes += UVIntTool.numBytes(mvccVersionIndexWidth);
161     numBytes += UVIntTool.numBytes(valueOffsetWidth);
162     numBytes += UVIntTool.numBytes(valueLengthWidth);
163 
164     numBytes += UVIntTool.numBytes(rowTreeDepth);
165     numBytes += UVIntTool.numBytes(maxRowLength);
166     numBytes += UVIntTool.numBytes(maxQualifierLength);
167     numBytes += UVIntTool.numBytes(maxTagsLength);
168 
169     numBytes += UVLongTool.numBytes(minTimestamp);
170     numBytes += UVIntTool.numBytes(timestampDeltaWidth);
171     numBytes += UVLongTool.numBytes(minMvccVersion);
172     numBytes += UVIntTool.numBytes(mvccVersionDeltaWidth);
173     ++numBytes;//os.write(getAllSameTypeByte());
174     ++numBytes;//os.write(allTypes);
175 
176     numBytes += UVIntTool.numBytes(numUniqueRows);
177     numBytes += UVIntTool.numBytes(numUniqueFamilies);
178     numBytes += UVIntTool.numBytes(numUniqueQualifiers);
179     numBytes += UVIntTool.numBytes(numUniqueTags);
180     return numBytes;
181   }
182 
183   public void writeVariableBytesToOutputStream(OutputStream os) throws IOException{
184       UVIntTool.writeBytes(version, os);
185       UVIntTool.writeBytes(numMetaBytes, os);
186       UVIntTool.writeBytes(numKeyValueBytes, os);
187       os.write(getIncludesMvccVersionByte());
188 
189       UVIntTool.writeBytes(numRowBytes, os);
190       UVIntTool.writeBytes(numFamilyBytes, os);
191       UVIntTool.writeBytes(numQualifierBytes, os);
192       UVIntTool.writeBytes(numTagsBytes, os);
193       UVIntTool.writeBytes(numTimestampBytes, os);
194       UVIntTool.writeBytes(numMvccVersionBytes, os);
195       UVIntTool.writeBytes(numValueBytes, os);
196 
197       UVIntTool.writeBytes(nextNodeOffsetWidth, os);
198       UVIntTool.writeBytes(familyOffsetWidth, os);
199       UVIntTool.writeBytes(qualifierOffsetWidth, os);
200       UVIntTool.writeBytes(tagsOffsetWidth, os);
201       UVIntTool.writeBytes(timestampIndexWidth, os);
202       UVIntTool.writeBytes(mvccVersionIndexWidth, os);
203       UVIntTool.writeBytes(valueOffsetWidth, os);
204       UVIntTool.writeBytes(valueLengthWidth, os);
205 
206       UVIntTool.writeBytes(rowTreeDepth, os);
207       UVIntTool.writeBytes(maxRowLength, os);
208       UVIntTool.writeBytes(maxQualifierLength, os);
209       UVIntTool.writeBytes(maxTagsLength, os);
210 
211       UVLongTool.writeBytes(minTimestamp, os);
212       UVIntTool.writeBytes(timestampDeltaWidth, os);
213       UVLongTool.writeBytes(minMvccVersion, os);
214       UVIntTool.writeBytes(mvccVersionDeltaWidth, os);
215       os.write(getAllSameTypeByte());
216       os.write(allTypes);
217 
218       UVIntTool.writeBytes(numUniqueRows, os);
219       UVIntTool.writeBytes(numUniqueFamilies, os);
220       UVIntTool.writeBytes(numUniqueQualifiers, os);
221       UVIntTool.writeBytes(numUniqueTags, os);
222   }
223 
224   public void readVariableBytesFromInputStream(InputStream is) throws IOException{
225       version = UVIntTool.getInt(is);
226       numMetaBytes = UVIntTool.getInt(is);
227       numKeyValueBytes = UVIntTool.getInt(is);
228       setIncludesMvccVersion((byte) is.read());
229 
230       numRowBytes = UVIntTool.getInt(is);
231       numFamilyBytes = UVIntTool.getInt(is);
232       numQualifierBytes = UVIntTool.getInt(is);
233       numTagsBytes = UVIntTool.getInt(is);
234       numTimestampBytes = UVIntTool.getInt(is);
235       numMvccVersionBytes = UVIntTool.getInt(is);
236       numValueBytes = UVIntTool.getInt(is);
237 
238       nextNodeOffsetWidth = UVIntTool.getInt(is);
239       familyOffsetWidth = UVIntTool.getInt(is);
240       qualifierOffsetWidth = UVIntTool.getInt(is);
241       tagsOffsetWidth = UVIntTool.getInt(is);
242       timestampIndexWidth = UVIntTool.getInt(is);
243       mvccVersionIndexWidth = UVIntTool.getInt(is);
244       valueOffsetWidth = UVIntTool.getInt(is);
245       valueLengthWidth = UVIntTool.getInt(is);
246 
247       rowTreeDepth = UVIntTool.getInt(is);
248       maxRowLength = UVIntTool.getInt(is);
249       maxQualifierLength = UVIntTool.getInt(is);
250       maxTagsLength = UVIntTool.getInt(is);
251 
252       minTimestamp = UVLongTool.getLong(is);
253       timestampDeltaWidth = UVIntTool.getInt(is);
254       minMvccVersion = UVLongTool.getLong(is);
255       mvccVersionDeltaWidth = UVIntTool.getInt(is);
256 
257       setAllSameType((byte) is.read());
258       allTypes = (byte) is.read();
259 
260       numUniqueRows = UVIntTool.getInt(is);
261       numUniqueFamilies = UVIntTool.getInt(is);
262       numUniqueQualifiers = UVIntTool.getInt(is);
263       numUniqueTags = UVIntTool.getInt(is);
264   }
265 
266   public void readVariableBytesFromArray(byte[] bytes, int offset) {
267     int position = offset;
268 
269     version = UVIntTool.getInt(bytes, position);
270     position += UVIntTool.numBytes(version);
271     numMetaBytes = UVIntTool.getInt(bytes, position);
272     position += UVIntTool.numBytes(numMetaBytes);
273     numKeyValueBytes = UVIntTool.getInt(bytes, position);
274     position += UVIntTool.numBytes(numKeyValueBytes);
275     setIncludesMvccVersion(bytes[position]);
276     ++position;
277 
278     numRowBytes = UVIntTool.getInt(bytes, position);
279     position += UVIntTool.numBytes(numRowBytes);
280     numFamilyBytes = UVIntTool.getInt(bytes, position);
281     position += UVIntTool.numBytes(numFamilyBytes);
282     numQualifierBytes = UVIntTool.getInt(bytes, position);
283     position += UVIntTool.numBytes(numQualifierBytes);
284     numTagsBytes = UVIntTool.getInt(bytes, position);
285     position += UVIntTool.numBytes(numTagsBytes);
286     numTimestampBytes = UVIntTool.getInt(bytes, position);
287     position += UVIntTool.numBytes(numTimestampBytes);
288     numMvccVersionBytes = UVIntTool.getInt(bytes, position);
289     position += UVIntTool.numBytes(numMvccVersionBytes);
290     numValueBytes = UVIntTool.getInt(bytes, position);
291     position += UVIntTool.numBytes(numValueBytes);
292 
293     nextNodeOffsetWidth = UVIntTool.getInt(bytes, position);
294     position += UVIntTool.numBytes(nextNodeOffsetWidth);
295     familyOffsetWidth = UVIntTool.getInt(bytes, position);
296     position += UVIntTool.numBytes(familyOffsetWidth);
297     qualifierOffsetWidth = UVIntTool.getInt(bytes, position);
298     position += UVIntTool.numBytes(qualifierOffsetWidth);
299     tagsOffsetWidth = UVIntTool.getInt(bytes, position);
300     position += UVIntTool.numBytes(tagsOffsetWidth);
301     timestampIndexWidth = UVIntTool.getInt(bytes, position);
302     position += UVIntTool.numBytes(timestampIndexWidth);
303     mvccVersionIndexWidth = UVIntTool.getInt(bytes, position);
304     position += UVIntTool.numBytes(mvccVersionIndexWidth);
305     valueOffsetWidth = UVIntTool.getInt(bytes, position);
306     position += UVIntTool.numBytes(valueOffsetWidth);
307     valueLengthWidth = UVIntTool.getInt(bytes, position);
308     position += UVIntTool.numBytes(valueLengthWidth);
309 
310     rowTreeDepth = UVIntTool.getInt(bytes, position);
311     position += UVIntTool.numBytes(rowTreeDepth);
312     maxRowLength = UVIntTool.getInt(bytes, position);
313     position += UVIntTool.numBytes(maxRowLength);
314     maxQualifierLength = UVIntTool.getInt(bytes, position);
315     position += UVIntTool.numBytes(maxQualifierLength);
316     maxTagsLength = UVIntTool.getInt(bytes, position);
317     position += UVIntTool.numBytes(maxTagsLength);
318     minTimestamp = UVLongTool.getLong(bytes, position);
319     position += UVLongTool.numBytes(minTimestamp);
320     timestampDeltaWidth = UVIntTool.getInt(bytes, position);
321     position += UVIntTool.numBytes(timestampDeltaWidth);
322     minMvccVersion = UVLongTool.getLong(bytes, position);
323     position += UVLongTool.numBytes(minMvccVersion);
324     mvccVersionDeltaWidth = UVIntTool.getInt(bytes, position);
325     position += UVIntTool.numBytes(mvccVersionDeltaWidth);
326 
327     setAllSameType(bytes[position]);
328     ++position;
329     allTypes = bytes[position];
330     ++position;
331 
332     numUniqueRows = UVIntTool.getInt(bytes, position);
333     position += UVIntTool.numBytes(numUniqueRows);
334     numUniqueFamilies = UVIntTool.getInt(bytes, position);
335     position += UVIntTool.numBytes(numUniqueFamilies);
336     numUniqueQualifiers = UVIntTool.getInt(bytes, position);
337     position += UVIntTool.numBytes(numUniqueQualifiers);
338     numUniqueTags = UVIntTool.getInt(bytes, position);
339     position += UVIntTool.numBytes(numUniqueTags);
340   }
341 
342   //TODO method that can read directly from ByteBuffer instead of InputStream
343 
344 
345   /*************** methods *************************/
346 
347   public int getKeyValueTypeWidth() {
348     return allSameType ? 0 : 1;
349   }
350 
351   public byte getIncludesMvccVersionByte() {
352     return includesMvccVersion ? (byte) 1 : (byte) 0;
353   }
354 
355   public void setIncludesMvccVersion(byte includesMvccVersionByte) {
356     includesMvccVersion = includesMvccVersionByte != 0;
357   }
358 
359   public byte getAllSameTypeByte() {
360     return allSameType ? (byte) 1 : (byte) 0;
361   }
362 
363   public void setAllSameType(byte allSameTypeByte) {
364     allSameType = allSameTypeByte != 0;
365   }
366 
367   public boolean isAllSameTimestamp() {
368     return timestampIndexWidth == 0;
369   }
370 
371   public boolean isAllSameMvccVersion() {
372     return mvccVersionIndexWidth == 0;
373   }
374 
375   public void setTimestampFields(LongEncoder encoder){
376     this.minTimestamp = encoder.getMin();
377     this.timestampIndexWidth = encoder.getBytesPerIndex();
378     this.timestampDeltaWidth = encoder.getBytesPerDelta();
379     this.numTimestampBytes = encoder.getTotalCompressedBytes();
380   }
381 
382   public void setMvccVersionFields(LongEncoder encoder){
383     this.minMvccVersion = encoder.getMin();
384     this.mvccVersionIndexWidth = encoder.getBytesPerIndex();
385     this.mvccVersionDeltaWidth = encoder.getBytesPerDelta();
386     this.numMvccVersionBytes = encoder.getTotalCompressedBytes();
387   }
388 
389 
390   /*************** Object methods *************************/
391 
392   /**
393    * Generated by Eclipse
394    */
395   @Override
396   public boolean equals(Object obj) {
397     if (this == obj)
398       return true;
399     if (obj == null)
400       return false;
401     if (getClass() != obj.getClass())
402       return false;
403     PrefixTreeBlockMeta other = (PrefixTreeBlockMeta) obj;
404     if (allSameType != other.allSameType)
405       return false;
406     if (allTypes != other.allTypes)
407       return false;
408     if (arrayOffset != other.arrayOffset)
409       return false;
410     if (bufferOffset != other.bufferOffset)
411       return false;
412     if (valueLengthWidth != other.valueLengthWidth)
413       return false;
414     if (valueOffsetWidth != other.valueOffsetWidth)
415       return false;
416     if (familyOffsetWidth != other.familyOffsetWidth)
417       return false;
418     if (includesMvccVersion != other.includesMvccVersion)
419       return false;
420     if (maxQualifierLength != other.maxQualifierLength)
421       return false;
422     if (maxTagsLength != other.maxTagsLength)
423       return false;
424     if (maxRowLength != other.maxRowLength)
425       return false;
426     if (mvccVersionDeltaWidth != other.mvccVersionDeltaWidth)
427       return false;
428     if (mvccVersionIndexWidth != other.mvccVersionIndexWidth)
429       return false;
430     if (minMvccVersion != other.minMvccVersion)
431       return false;
432     if (minTimestamp != other.minTimestamp)
433       return false;
434     if (nextNodeOffsetWidth != other.nextNodeOffsetWidth)
435       return false;
436     if (numValueBytes != other.numValueBytes)
437       return false;
438     if (numFamilyBytes != other.numFamilyBytes)
439       return false;
440     if (numMvccVersionBytes != other.numMvccVersionBytes)
441       return false;
442     if (numMetaBytes != other.numMetaBytes)
443       return false;
444     if (numQualifierBytes != other.numQualifierBytes)
445       return false;
446     if (numTagsBytes != other.numTagsBytes)
447       return false;
448     if (numRowBytes != other.numRowBytes)
449       return false;
450     if (numTimestampBytes != other.numTimestampBytes)
451       return false;
452     if (numUniqueFamilies != other.numUniqueFamilies)
453       return false;
454     if (numUniqueQualifiers != other.numUniqueQualifiers)
455       return false;
456     if (numUniqueTags != other.numUniqueTags)
457       return false;
458     if (numUniqueRows != other.numUniqueRows)
459       return false;
460     if (numKeyValueBytes != other.numKeyValueBytes)
461       return false;
462     if (qualifierOffsetWidth != other.qualifierOffsetWidth)
463       return false;
464     if(tagsOffsetWidth !=  other.tagsOffsetWidth) 
465       return false;
466     if (rowTreeDepth != other.rowTreeDepth)
467       return false;
468     if (timestampDeltaWidth != other.timestampDeltaWidth)
469       return false;
470     if (timestampIndexWidth != other.timestampIndexWidth)
471       return false;
472     if (version != other.version)
473       return false;
474     return true;
475   }
476 
477   /**
478    * Generated by Eclipse
479    */
480   @Override
481   public int hashCode() {
482     final int prime = 31;
483     int result = 1;
484     result = prime * result + (allSameType ? 1231 : 1237);
485     result = prime * result + allTypes;
486     result = prime * result + arrayOffset;
487     result = prime * result + bufferOffset;
488     result = prime * result + valueLengthWidth;
489     result = prime * result + valueOffsetWidth;
490     result = prime * result + familyOffsetWidth;
491     result = prime * result + (includesMvccVersion ? 1231 : 1237);
492     result = prime * result + maxQualifierLength;
493     result = prime * result + maxTagsLength;
494     result = prime * result + maxRowLength;
495     result = prime * result + mvccVersionDeltaWidth;
496     result = prime * result + mvccVersionIndexWidth;
497     result = prime * result + (int) (minMvccVersion ^ (minMvccVersion >>> 32));
498     result = prime * result + (int) (minTimestamp ^ (minTimestamp >>> 32));
499     result = prime * result + nextNodeOffsetWidth;
500     result = prime * result + numValueBytes;
501     result = prime * result + numFamilyBytes;
502     result = prime * result + numMvccVersionBytes;
503     result = prime * result + numMetaBytes;
504     result = prime * result + numQualifierBytes;
505     result = prime * result + numTagsBytes;
506     result = prime * result + numRowBytes;
507     result = prime * result + numTimestampBytes;
508     result = prime * result + numUniqueFamilies;
509     result = prime * result + numUniqueQualifiers;
510     result = prime * result + numUniqueTags;
511     result = prime * result + numUniqueRows;
512     result = prime * result + numKeyValueBytes;
513     result = prime * result + qualifierOffsetWidth;
514     result = prime * result + tagsOffsetWidth;
515     result = prime * result + rowTreeDepth;
516     result = prime * result + timestampDeltaWidth;
517     result = prime * result + timestampIndexWidth;
518     result = prime * result + version;
519     return result;
520   }
521 
522   /**
523    * Generated by Eclipse
524    */
525   @Override
526   public String toString() {
527     StringBuilder builder = new StringBuilder();
528     builder.append("PtBlockMeta [arrayOffset=");
529     builder.append(arrayOffset);
530     builder.append(", bufferOffset=");
531     builder.append(bufferOffset);
532     builder.append(", version=");
533     builder.append(version);
534     builder.append(", numMetaBytes=");
535     builder.append(numMetaBytes);
536     builder.append(", numKeyValueBytes=");
537     builder.append(numKeyValueBytes);
538     builder.append(", includesMvccVersion=");
539     builder.append(includesMvccVersion);
540     builder.append(", numRowBytes=");
541     builder.append(numRowBytes);
542     builder.append(", numFamilyBytes=");
543     builder.append(numFamilyBytes);
544     builder.append(", numQualifierBytes=");
545     builder.append(numQualifierBytes);
546     builder.append(", numTimestampBytes=");
547     builder.append(numTimestampBytes);
548     builder.append(", numMvccVersionBytes=");
549     builder.append(numMvccVersionBytes);
550     builder.append(", numValueBytes=");
551     builder.append(numValueBytes);
552     builder.append(", numTagBytes=");
553     builder.append(numTagsBytes);
554     builder.append(", nextNodeOffsetWidth=");
555     builder.append(nextNodeOffsetWidth);
556     builder.append(", familyOffsetWidth=");
557     builder.append(familyOffsetWidth);
558     builder.append(", qualifierOffsetWidth=");
559     builder.append(qualifierOffsetWidth);
560     builder.append(", tagOffsetWidth=");
561     builder.append(tagsOffsetWidth);
562     builder.append(", timestampIndexWidth=");
563     builder.append(timestampIndexWidth);
564     builder.append(", mvccVersionIndexWidth=");
565     builder.append(mvccVersionIndexWidth);
566     builder.append(", valueOffsetWidth=");
567     builder.append(valueOffsetWidth);
568     builder.append(", valueLengthWidth=");
569     builder.append(valueLengthWidth);
570     builder.append(", rowTreeDepth=");
571     builder.append(rowTreeDepth);
572     builder.append(", maxRowLength=");
573     builder.append(maxRowLength);
574     builder.append(", maxQualifierLength=");
575     builder.append(maxQualifierLength);
576     builder.append(", maxTagLength=");
577     builder.append(maxTagsLength);
578     builder.append(", minTimestamp=");
579     builder.append(minTimestamp);
580     builder.append(", timestampDeltaWidth=");
581     builder.append(timestampDeltaWidth);
582     builder.append(", minMvccVersion=");
583     builder.append(minMvccVersion);
584     builder.append(", mvccVersionDeltaWidth=");
585     builder.append(mvccVersionDeltaWidth);
586     builder.append(", allSameType=");
587     builder.append(allSameType);
588     builder.append(", allTypes=");
589     builder.append(allTypes);
590     builder.append(", numUniqueRows=");
591     builder.append(numUniqueRows);
592     builder.append(", numUniqueFamilies=");
593     builder.append(numUniqueFamilies);
594     builder.append(", numUniqueQualifiers=");
595     builder.append(numUniqueQualifiers);
596     builder.append(", numUniqueTags=");
597     builder.append(numUniqueTags);
598     builder.append("]");
599     return builder.toString();
600   }
601 
602 
603   /************** absolute getters *******************/
604 
605   public int getAbsoluteMetaOffset() {
606     return arrayOffset + bufferOffset;
607   }
608 
609   public int getAbsoluteRowOffset() {
610     return getAbsoluteMetaOffset() + numMetaBytes;
611   }
612 
613   public int getAbsoluteFamilyOffset() {
614     return getAbsoluteRowOffset() + numRowBytes;
615   }
616 
617   public int getAbsoluteQualifierOffset() {
618     return getAbsoluteFamilyOffset() + numFamilyBytes;
619   }
620 
621   public int getAbsoluteTagsOffset() {
622     return getAbsoluteQualifierOffset() + numQualifierBytes;
623   }
624 
625   public int getAbsoluteTimestampOffset() {
626     return getAbsoluteTagsOffset() + numTagsBytes;
627   }
628 
629   public int getAbsoluteMvccVersionOffset() {
630     return getAbsoluteTimestampOffset() + numTimestampBytes;
631   }
632 
633   public int getAbsoluteValueOffset() {
634     return getAbsoluteMvccVersionOffset() + numMvccVersionBytes;
635   }
636 
637 
638   /*************** get/set ***************************/
639 
640   public int getTimestampDeltaWidth() {
641     return timestampDeltaWidth;
642   }
643 
644   public void setTimestampDeltaWidth(int timestampDeltaWidth) {
645     this.timestampDeltaWidth = timestampDeltaWidth;
646   }
647 
648   public int getValueOffsetWidth() {
649     return valueOffsetWidth;
650   }
651 
652   public int getTagsOffsetWidth() {
653     return tagsOffsetWidth;
654   }
655 
656   public void setValueOffsetWidth(int dataOffsetWidth) {
657     this.valueOffsetWidth = dataOffsetWidth;
658   }
659 
660   public void setTagsOffsetWidth(int dataOffsetWidth) {
661     this.tagsOffsetWidth = dataOffsetWidth;
662   }
663 
664   public int getValueLengthWidth() {
665     return valueLengthWidth;
666   }
667 
668   public void setValueLengthWidth(int dataLengthWidth) {
669     this.valueLengthWidth = dataLengthWidth;
670   }
671 
672   public int getMaxRowLength() {
673     return maxRowLength;
674   }
675 
676   public void setMaxRowLength(int maxRowLength) {
677     this.maxRowLength = maxRowLength;
678   }
679 
680   public long getMinTimestamp() {
681     return minTimestamp;
682   }
683 
684   public void setMinTimestamp(long minTimestamp) {
685     this.minTimestamp = minTimestamp;
686   }
687 
688   public byte getAllTypes() {
689     return allTypes;
690   }
691 
692   public void setAllTypes(byte allTypes) {
693     this.allTypes = allTypes;
694   }
695 
696   public boolean isAllSameType() {
697     return allSameType;
698   }
699 
700   public void setAllSameType(boolean allSameType) {
701     this.allSameType = allSameType;
702   }
703 
704   public int getNextNodeOffsetWidth() {
705     return nextNodeOffsetWidth;
706   }
707 
708   public void setNextNodeOffsetWidth(int nextNodeOffsetWidth) {
709     this.nextNodeOffsetWidth = nextNodeOffsetWidth;
710   }
711 
712   public int getNumRowBytes() {
713     return numRowBytes;
714   }
715 
716   public void setNumRowBytes(int numRowBytes) {
717     this.numRowBytes = numRowBytes;
718   }
719 
720   public int getNumTimestampBytes() {
721     return numTimestampBytes;
722   }
723 
724   public void setNumTimestampBytes(int numTimestampBytes) {
725     this.numTimestampBytes = numTimestampBytes;
726   }
727 
728   public int getNumValueBytes() {
729     return numValueBytes;
730   }
731 
732   public int getNumTagsBytes() {
733     return numTagsBytes;
734   }
735 
736   public void setNumTagsBytes(int numTagBytes){
737     this.numTagsBytes = numTagBytes;
738   }
739 
740   public void setNumValueBytes(int numValueBytes) {
741     this.numValueBytes = numValueBytes;
742   }
743 
744   public int getNumMetaBytes() {
745     return numMetaBytes;
746   }
747 
748   public void setNumMetaBytes(int numMetaBytes) {
749     this.numMetaBytes = numMetaBytes;
750   }
751 
752   public int getArrayOffset() {
753     return arrayOffset;
754   }
755 
756   public void setArrayOffset(int arrayOffset) {
757     this.arrayOffset = arrayOffset;
758   }
759 
760   public int getBufferOffset() {
761     return bufferOffset;
762   }
763 
764   public void setBufferOffset(int bufferOffset) {
765     this.bufferOffset = bufferOffset;
766   }
767 
768   public int getNumKeyValueBytes() {
769     return numKeyValueBytes;
770   }
771 
772   public void setNumKeyValueBytes(int numKeyValueBytes) {
773     this.numKeyValueBytes = numKeyValueBytes;
774   }
775 
776   public int getRowTreeDepth() {
777     return rowTreeDepth;
778   }
779 
780   public void setRowTreeDepth(int rowTreeDepth) {
781     this.rowTreeDepth = rowTreeDepth;
782   }
783 
784   public int getNumMvccVersionBytes() {
785     return numMvccVersionBytes;
786   }
787 
788   public void setNumMvccVersionBytes(int numMvccVersionBytes) {
789     this.numMvccVersionBytes = numMvccVersionBytes;
790   }
791 
792   public int getMvccVersionDeltaWidth() {
793     return mvccVersionDeltaWidth;
794   }
795 
796   public void setMvccVersionDeltaWidth(int mvccVersionDeltaWidth) {
797     this.mvccVersionDeltaWidth = mvccVersionDeltaWidth;
798   }
799 
800   public long getMinMvccVersion() {
801     return minMvccVersion;
802   }
803 
804   public void setMinMvccVersion(long minMvccVersion) {
805     this.minMvccVersion = minMvccVersion;
806   }
807 
808   public int getNumFamilyBytes() {
809     return numFamilyBytes;
810   }
811 
812   public void setNumFamilyBytes(int numFamilyBytes) {
813     this.numFamilyBytes = numFamilyBytes;
814   }
815 
816   public int getFamilyOffsetWidth() {
817     return familyOffsetWidth;
818   }
819 
820   public void setFamilyOffsetWidth(int familyOffsetWidth) {
821     this.familyOffsetWidth = familyOffsetWidth;
822   }
823 
824   public int getNumUniqueRows() {
825     return numUniqueRows;
826   }
827 
828   public void setNumUniqueRows(int numUniqueRows) {
829     this.numUniqueRows = numUniqueRows;
830   }
831 
832   public int getNumUniqueFamilies() {
833     return numUniqueFamilies;
834   }
835 
836   public void setNumUniqueFamilies(int numUniqueFamilies) {
837     this.numUniqueFamilies = numUniqueFamilies;
838   }
839 
840   public int getNumUniqueQualifiers() {
841     return numUniqueQualifiers;
842   }
843 
844   public void setNumUniqueQualifiers(int numUniqueQualifiers) {
845     this.numUniqueQualifiers = numUniqueQualifiers;
846   }
847 
848   public void setNumUniqueTags(int numUniqueTags) {
849     this.numUniqueTags = numUniqueTags;
850   }
851 
852   public int getNumUniqueTags() {
853     return numUniqueTags;
854   }
855   public int getNumQualifierBytes() {
856     return numQualifierBytes;
857   }
858 
859   public void setNumQualifierBytes(int numQualifierBytes) {
860     this.numQualifierBytes = numQualifierBytes;
861   }
862 
863   public int getQualifierOffsetWidth() {
864     return qualifierOffsetWidth;
865   }
866 
867   public void setQualifierOffsetWidth(int qualifierOffsetWidth) {
868     this.qualifierOffsetWidth = qualifierOffsetWidth;
869   }
870 
871   public int getMaxQualifierLength() {
872     return maxQualifierLength;
873   }
874 
875   // TODO : decide on some max value for this ? INTEGER_MAX?
876   public void setMaxQualifierLength(int maxQualifierLength) {
877     this.maxQualifierLength = maxQualifierLength;
878   }
879 
880   public int getMaxTagsLength() {
881     return this.maxTagsLength;
882   }
883 
884   public void setMaxTagsLength(int maxTagLength) {
885     this.maxTagsLength = maxTagLength;
886   }
887 
888   public int getTimestampIndexWidth() {
889     return timestampIndexWidth;
890   }
891 
892   public void setTimestampIndexWidth(int timestampIndexWidth) {
893     this.timestampIndexWidth = timestampIndexWidth;
894   }
895 
896   public int getMvccVersionIndexWidth() {
897     return mvccVersionIndexWidth;
898   }
899 
900   public void setMvccVersionIndexWidth(int mvccVersionIndexWidth) {
901     this.mvccVersionIndexWidth = mvccVersionIndexWidth;
902   }
903 
904   public int getVersion() {
905     return version;
906   }
907 
908   public void setVersion(int version) {
909     this.version = version;
910   }
911 
912   public boolean isIncludesMvccVersion() {
913     return includesMvccVersion;
914   }
915 
916   public void setIncludesMvccVersion(boolean includesMvccVersion) {
917     this.includesMvccVersion = includesMvccVersion;
918   }
919 
920 }