View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.client;
21  
22  import java.io.IOException;
23  import java.nio.BufferOverflowException;
24  import java.nio.ByteBuffer;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.Comparator;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.NavigableMap;
31  import java.util.TreeMap;
32  
33  import org.apache.hadoop.hbase.Cell;
34  import org.apache.hadoop.hbase.CellScannable;
35  import org.apache.hadoop.hbase.CellScanner;
36  import org.apache.hadoop.hbase.CellUtil;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.KeyValue;
39  import org.apache.hadoop.hbase.KeyValueUtil;
40  import org.apache.hadoop.hbase.classification.InterfaceAudience;
41  import org.apache.hadoop.hbase.classification.InterfaceStability;
42  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
43  import org.apache.hadoop.hbase.util.Bytes;
44  
45  /**
46   * Single row result of a {@link Get} or {@link Scan} query.<p>
47   *
48   * This class is <b>NOT THREAD SAFE</b>.<p>
49   *
50   * Convenience methods are available that return various {@link Map}
51   * structures and values directly.<p>
52   *
53   * To get a complete mapping of all cells in the Result, which can include
54   * multiple families and multiple versions, use {@link #getMap()}.<p>
55   *
56   * To get a mapping of each family to its columns (qualifiers and values),
57   * including only the latest version of each, use {@link #getNoVersionMap()}.
58   *
59   * To get a mapping of qualifiers to latest values for an individual family use
60   * {@link #getFamilyMap(byte[])}.<p>
61   *
62   * To get the latest value for a specific family and qualifier use {@link #getValue(byte[], byte[])}.
63   *
64   * A Result is backed by an array of {@link Cell} objects, each representing
65   * an HBase cell defined by the row, family, qualifier, timestamp, and value.<p>
66   *
67   * The underlying {@link Cell} objects can be accessed through the method {@link #listCells()}.
68   * This will create a List from the internal Cell []. Better is to exploit the fact that
69   * a new Result instance is a primed {@link CellScanner}; just call {@link #advance()} and
70   * {@link #current()} to iterate over Cells as you would any {@link CellScanner}.
71   * Call {@link #cellScanner()} to reset should you need to iterate the same Result over again
72   * ({@link CellScanner}s are one-shot).
73   *
74   * If you need to overwrite a Result with another Result instance -- as in the old 'mapred'
75   * RecordReader next invocations -- then create an empty Result with the null constructor and
76   * in then use {@link #copyFrom(Result)}
77   */
78  @InterfaceAudience.Public
79  @InterfaceStability.Stable
80  public class Result implements CellScannable, CellScanner {
81    private Cell[] cells;
82    private Boolean exists; // if the query was just to check existence.
83    private boolean stale = false;
84  
85    /**
86     * Partial results do not contain the full row's worth of cells. The result had to be returned in
87     * parts because the size of the cells in the row exceeded the RPC result size on the server.
88     * Partial results must be combined client side with results representing the remainder of the
89     * row's cells to form the complete result. Partial results and RPC result size allow us to avoid
90     * OOME on the server when servicing requests for large rows. The Scan configuration used to
91     * control the result size on the server is {@link Scan#setMaxResultSize(long)} and the default
92     * value can be seen here: {@link HConstants#DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE}
93     */
94    private boolean partial = false;
95    // We're not using java serialization.  Transient here is just a marker to say
96    // that this is where we cache row if we're ever asked for it.
97    private transient byte [] row = null;
98    // Ditto for familyMap.  It can be composed on fly from passed in kvs.
99    private transient NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> familyMap = null;
100 
101   private static ThreadLocal<byte[]> localBuffer = new ThreadLocal<byte[]>();
102   private static final int PAD_WIDTH = 128;
103   public static final Result EMPTY_RESULT = new Result(true);
104 
105   private final static int INITIAL_CELLSCANNER_INDEX = -1;
106 
107   /**
108    * Index for where we are when Result is acting as a {@link CellScanner}.
109    */
110   private int cellScannerIndex = INITIAL_CELLSCANNER_INDEX;
111   private ClientProtos.RegionLoadStats stats;
112 
113   private final boolean readonly;
114 
115   /**
116    * Creates an empty Result w/ no KeyValue payload; returns null if you call {@link #rawCells()}.
117    * Use this to represent no results if <code>null</code> won't do or in old 'mapred' as oppposed to 'mapreduce' package
118    * MapReduce where you need to overwrite a Result
119    * instance with a {@link #copyFrom(Result)} call.
120    */
121   public Result() {
122     this(false);
123   }
124 
125   /**
126    * Allows to construct special purpose immutable Result objects,
127    * such as EMPTY_RESULT.
128    * @param readonly whether this Result instance is readonly
129    */
130   private Result(boolean readonly) {
131     this.readonly = readonly;
132   }
133 
134   /**
135    * @deprecated Use {@link #create(List)} instead.
136    */
137   @Deprecated
138   public Result(KeyValue [] cells) {
139     this(cells, null, false, false);
140   }
141 
142   /**
143    * @deprecated Use {@link #create(List)} instead.
144    */
145   @Deprecated
146   public Result(List<KeyValue> kvs) {
147     // TODO: Here we presume the passed in Cells are KVs.  One day this won't always be so.
148     this(kvs.toArray(new Cell[kvs.size()]), null, false, false);
149   }
150 
151   /**
152    * Instantiate a Result with the specified List of KeyValues.
153    * <br><strong>Note:</strong> You must ensure that the keyvalues are already sorted.
154    * @param cells List of cells
155    */
156   public static Result create(List<Cell> cells) {
157     return create(cells, null);
158   }
159 
160   public static Result create(List<Cell> cells, Boolean exists) {
161     return create(cells, exists, false);
162   }
163 
164   public static Result create(List<Cell> cells, Boolean exists, boolean stale) {
165     return create(cells, exists, stale, false);
166   }
167 
168   public static Result create(List<Cell> cells, Boolean exists, boolean stale, boolean partial) {
169     if (exists != null){
170       return new Result(null, exists, stale, partial);
171     }
172     return new Result(cells.toArray(new Cell[cells.size()]), null, stale, partial);
173   }
174 
175   /**
176    * Instantiate a Result with the specified array of KeyValues.
177    * <br><strong>Note:</strong> You must ensure that the keyvalues are already sorted.
178    * @param cells array of cells
179    */
180   public static Result create(Cell[] cells) {
181     return create(cells, null, false);
182   }
183 
184   public static Result create(Cell[] cells, Boolean exists, boolean stale) {
185     return create(cells, exists, stale, false);
186   }
187 
188   public static Result create(Cell[] cells, Boolean exists, boolean stale, boolean partial) {
189     if (exists != null){
190       return new Result(null, exists, stale, partial);
191     }
192     return new Result(cells, null, stale, partial);
193   }
194 
195   /** Private ctor. Use {@link #create(Cell[])}. */
196   private Result(Cell[] cells, Boolean exists, boolean stale, boolean partial) {
197     this.cells = cells;
198     this.exists = exists;
199     this.stale = stale;
200     this.partial = partial;
201     this.readonly = false;
202   }
203 
204   /**
205    * Method for retrieving the row key that corresponds to
206    * the row from which this Result was created.
207    * @return row
208    */
209   public byte [] getRow() {
210     if (this.row == null) {
211       this.row = this.cells == null || this.cells.length == 0? null: CellUtil.cloneRow(this.cells[0]);
212     }
213     return this.row;
214   }
215 
216   /**
217    * Return the array of Cells backing this Result instance.
218    *
219    * The array is sorted from smallest -> largest using the
220    * {@link KeyValue#COMPARATOR}.
221    *
222    * The array only contains what your Get or Scan specifies and no more.
223    * For example if you request column "A" 1 version you will have at most 1
224    * Cell in the array. If you request column "A" with 2 version you will
225    * have at most 2 Cells, with the first one being the newer timestamp and
226    * the second being the older timestamp (this is the sort order defined by
227    * {@link KeyValue#COMPARATOR}).  If columns don't exist, they won't be
228    * present in the result. Therefore if you ask for 1 version all columns,
229    * it is safe to iterate over this array and expect to see 1 Cell for
230    * each column and no more.
231    *
232    * This API is faster than using getFamilyMap() and getMap()
233    *
234    * @return array of Cells; can be null if nothing in the result
235    */
236   public Cell[] rawCells() {
237     return cells;
238   }
239 
240   /**
241    * Return an cells of a Result as an array of KeyValues
242    *
243    * WARNING do not use, expensive.  This does an arraycopy of the cell[]'s value.
244    *
245    * Added to ease transition from  0.94 -> 0.96.
246    *
247    * @deprecated as of 0.96, use {@link #rawCells()}
248    * @return array of KeyValues, empty array if nothing in result.
249    */
250   @Deprecated
251   public KeyValue[] raw() {
252     KeyValue[] kvs = new KeyValue[cells.length];
253     for (int i = 0 ; i < kvs.length; i++) {
254       kvs[i] = KeyValueUtil.ensureKeyValue(cells[i]);
255     }
256     return kvs;
257   }
258 
259   /**
260    * Create a sorted list of the Cell's in this result.
261    *
262    * Since HBase 0.20.5 this is equivalent to raw().
263    *
264    * @return sorted List of Cells; can be null if no cells in the result
265    */
266   public List<Cell> listCells() {
267     return isEmpty()? null: Arrays.asList(rawCells());
268   }
269 
270   /**
271    * Return an cells of a Result as an array of KeyValues
272    *
273    * WARNING do not use, expensive.  This does  an arraycopy of the cell[]'s value.
274    *
275    * Added to ease transition from  0.94 -> 0.96.
276    *
277    * @deprecated as of 0.96, use {@link #listCells()}
278    * @return all sorted List of KeyValues; can be null if no cells in the result
279    */
280   @Deprecated
281   public List<KeyValue> list() {
282     return isEmpty() ? null : Arrays.asList(raw());
283   }
284 
285   /**
286    * @deprecated Use {@link #getColumnCells(byte[], byte[])} instead.
287    */
288   @Deprecated
289   public List<KeyValue> getColumn(byte [] family, byte [] qualifier) {
290     return KeyValueUtil.ensureKeyValues(getColumnCells(family, qualifier));
291   }
292 
293   /**
294    * Return the Cells for the specific column.  The Cells are sorted in
295    * the {@link KeyValue#COMPARATOR} order.  That implies the first entry in
296    * the list is the most recent column.  If the query (Scan or Get) only
297    * requested 1 version the list will contain at most 1 entry.  If the column
298    * did not exist in the result set (either the column does not exist
299    * or the column was not selected in the query) the list will be empty.
300    *
301    * Also see getColumnLatest which returns just a Cell
302    *
303    * @param family the family
304    * @param qualifier
305    * @return a list of Cells for this column or empty list if the column
306    * did not exist in the result set
307    */
308   public List<Cell> getColumnCells(byte [] family, byte [] qualifier) {
309     List<Cell> result = new ArrayList<Cell>();
310 
311     Cell [] kvs = rawCells();
312 
313     if (kvs == null || kvs.length == 0) {
314       return result;
315     }
316     int pos = binarySearch(kvs, family, qualifier);
317     if (pos == -1) {
318       return result; // cant find it
319     }
320 
321     for (int i = pos ; i < kvs.length ; i++ ) {
322       if (CellUtil.matchingColumn(kvs[i], family,qualifier)) {
323         result.add(kvs[i]);
324       } else {
325         break;
326       }
327     }
328 
329     return result;
330   }
331 
332   protected int binarySearch(final Cell [] kvs,
333                              final byte [] family,
334                              final byte [] qualifier) {
335     Cell searchTerm =
336         KeyValueUtil.createFirstOnRow(CellUtil.cloneRow(kvs[0]),
337             family, qualifier);
338 
339     // pos === ( -(insertion point) - 1)
340     int pos = Arrays.binarySearch(kvs, searchTerm, KeyValue.COMPARATOR);
341     // never will exact match
342     if (pos < 0) {
343       pos = (pos+1) * -1;
344       // pos is now insertion point
345     }
346     if (pos == kvs.length) {
347       return -1; // doesn't exist
348     }
349     return pos;
350   }
351 
352   /**
353    * Searches for the latest value for the specified column.
354    *
355    * @param kvs the array to search
356    * @param family family name
357    * @param foffset family offset
358    * @param flength family length
359    * @param qualifier column qualifier
360    * @param qoffset qualifier offset
361    * @param qlength qualifier length
362    *
363    * @return the index where the value was found, or -1 otherwise
364    */
365   protected int binarySearch(final Cell [] kvs,
366       final byte [] family, final int foffset, final int flength,
367       final byte [] qualifier, final int qoffset, final int qlength) {
368 
369     double keyValueSize = (double)
370         KeyValue.getKeyValueDataStructureSize(kvs[0].getRowLength(), flength, qlength, 0);
371 
372     byte[] buffer = localBuffer.get();
373     if (buffer == null || keyValueSize > buffer.length) {
374       // pad to the smallest multiple of the pad width
375       buffer = new byte[(int) Math.ceil(keyValueSize / PAD_WIDTH) * PAD_WIDTH];
376       localBuffer.set(buffer);
377     }
378 
379     Cell searchTerm = KeyValueUtil.createFirstOnRow(buffer, 0,
380         kvs[0].getRowArray(), kvs[0].getRowOffset(), kvs[0].getRowLength(),
381         family, foffset, flength,
382         qualifier, qoffset, qlength);
383 
384     // pos === ( -(insertion point) - 1)
385     int pos = Arrays.binarySearch(kvs, searchTerm, KeyValue.COMPARATOR);
386     // never will exact match
387     if (pos < 0) {
388       pos = (pos+1) * -1;
389       // pos is now insertion point
390     }
391     if (pos == kvs.length) {
392       return -1; // doesn't exist
393     }
394     return pos;
395   }
396 
397   /**
398    * @deprecated Use {@link #getColumnLatestCell(byte[], byte[])} instead.
399    */
400   @Deprecated
401   public KeyValue getColumnLatest(byte [] family, byte [] qualifier) {
402     return KeyValueUtil.ensureKeyValue(getColumnLatestCell(family, qualifier));
403   }
404 
405   /**
406    * The Cell for the most recent timestamp for a given column.
407    *
408    * @param family
409    * @param qualifier
410    *
411    * @return the Cell for the column, or null if no value exists in the row or none have been
412    * selected in the query (Get/Scan)
413    */
414   public Cell getColumnLatestCell(byte [] family, byte [] qualifier) {
415     Cell [] kvs = rawCells(); // side effect possibly.
416     if (kvs == null || kvs.length == 0) {
417       return null;
418     }
419     int pos = binarySearch(kvs, family, qualifier);
420     if (pos == -1) {
421       return null;
422     }
423     if (CellUtil.matchingColumn(kvs[pos], family, qualifier)) {
424       return kvs[pos];
425     }
426     return null;
427   }
428 
429   /**
430    * @deprecated Use {@link #getColumnLatestCell(byte[], int, int, byte[], int, int)} instead.
431    */
432   @Deprecated
433   public KeyValue getColumnLatest(byte [] family, int foffset, int flength,
434       byte [] qualifier, int qoffset, int qlength) {
435     return KeyValueUtil.ensureKeyValue(
436         getColumnLatestCell(family, foffset, flength, qualifier, qoffset, qlength));
437   }
438 
439   /**
440    * The Cell for the most recent timestamp for a given column.
441    *
442    * @param family family name
443    * @param foffset family offset
444    * @param flength family length
445    * @param qualifier column qualifier
446    * @param qoffset qualifier offset
447    * @param qlength qualifier length
448    *
449    * @return the Cell for the column, or null if no value exists in the row or none have been
450    * selected in the query (Get/Scan)
451    */
452   public Cell getColumnLatestCell(byte [] family, int foffset, int flength,
453       byte [] qualifier, int qoffset, int qlength) {
454 
455     Cell [] kvs = rawCells(); // side effect possibly.
456     if (kvs == null || kvs.length == 0) {
457       return null;
458     }
459     int pos = binarySearch(kvs, family, foffset, flength, qualifier, qoffset, qlength);
460     if (pos == -1) {
461       return null;
462     }
463     if (CellUtil.matchingColumn(kvs[pos], family, foffset, flength, qualifier, qoffset, qlength)) {
464       return kvs[pos];
465     }
466     return null;
467   }
468 
469   /**
470    * Get the latest version of the specified column.
471    * Note: this call clones the value content of the hosting Cell. See
472    * {@link #getValueAsByteBuffer(byte[], byte[])}, etc., or {@link #listCells()} if you would
473    * avoid the cloning.
474    * @param family family name
475    * @param qualifier column qualifier
476    * @return value of latest version of column, null if none found
477    */
478   public byte[] getValue(byte [] family, byte [] qualifier) {
479     Cell kv = getColumnLatestCell(family, qualifier);
480     if (kv == null) {
481       return null;
482     }
483     return CellUtil.cloneValue(kv);
484   }
485 
486   /**
487    * Returns the value wrapped in a new <code>ByteBuffer</code>.
488    *
489    * @param family family name
490    * @param qualifier column qualifier
491    *
492    * @return the latest version of the column, or <code>null</code> if none found
493    */
494   public ByteBuffer getValueAsByteBuffer(byte [] family, byte [] qualifier) {
495 
496     Cell kv = getColumnLatestCell(family, 0, family.length, qualifier, 0, qualifier.length);
497 
498     if (kv == null) {
499       return null;
500     }
501     return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()).
502       asReadOnlyBuffer();
503   }
504 
505   /**
506    * Returns the value wrapped in a new <code>ByteBuffer</code>.
507    *
508    * @param family family name
509    * @param foffset family offset
510    * @param flength family length
511    * @param qualifier column qualifier
512    * @param qoffset qualifier offset
513    * @param qlength qualifier length
514    *
515    * @return the latest version of the column, or <code>null</code> if none found
516    */
517   public ByteBuffer getValueAsByteBuffer(byte [] family, int foffset, int flength,
518       byte [] qualifier, int qoffset, int qlength) {
519 
520     Cell kv = getColumnLatestCell(family, foffset, flength, qualifier, qoffset, qlength);
521 
522     if (kv == null) {
523       return null;
524     }
525     return ByteBuffer.wrap(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()).
526       asReadOnlyBuffer();
527   }
528 
529   /**
530    * Loads the latest version of the specified column into the provided <code>ByteBuffer</code>.
531    * <p>
532    * Does not clear or flip the buffer.
533    *
534    * @param family family name
535    * @param qualifier column qualifier
536    * @param dst the buffer where to write the value
537    *
538    * @return <code>true</code> if a value was found, <code>false</code> otherwise
539    *
540    * @throws BufferOverflowException there is insufficient space remaining in the buffer
541    */
542   public boolean loadValue(byte [] family, byte [] qualifier, ByteBuffer dst)
543           throws BufferOverflowException {
544     return loadValue(family, 0, family.length, qualifier, 0, qualifier.length, dst);
545   }
546 
547   /**
548    * Loads the latest version of the specified column into the provided <code>ByteBuffer</code>.
549    * <p>
550    * Does not clear or flip the buffer.
551    *
552    * @param family family name
553    * @param foffset family offset
554    * @param flength family length
555    * @param qualifier column qualifier
556    * @param qoffset qualifier offset
557    * @param qlength qualifier length
558    * @param dst the buffer where to write the value
559    *
560    * @return <code>true</code> if a value was found, <code>false</code> otherwise
561    *
562    * @throws BufferOverflowException there is insufficient space remaining in the buffer
563    */
564   public boolean loadValue(byte [] family, int foffset, int flength,
565       byte [] qualifier, int qoffset, int qlength, ByteBuffer dst)
566           throws BufferOverflowException {
567     Cell kv = getColumnLatestCell(family, foffset, flength, qualifier, qoffset, qlength);
568 
569     if (kv == null) {
570       return false;
571     }
572     dst.put(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
573     return true;
574   }
575 
576   /**
577    * Checks if the specified column contains a non-empty value (not a zero-length byte array).
578    *
579    * @param family family name
580    * @param qualifier column qualifier
581    *
582    * @return whether or not a latest value exists and is not empty
583    */
584   public boolean containsNonEmptyColumn(byte [] family, byte [] qualifier) {
585 
586     return containsNonEmptyColumn(family, 0, family.length, qualifier, 0, qualifier.length);
587   }
588 
589   /**
590    * Checks if the specified column contains a non-empty value (not a zero-length byte array).
591    *
592    * @param family family name
593    * @param foffset family offset
594    * @param flength family length
595    * @param qualifier column qualifier
596    * @param qoffset qualifier offset
597    * @param qlength qualifier length
598    *
599    * @return whether or not a latest value exists and is not empty
600    */
601   public boolean containsNonEmptyColumn(byte [] family, int foffset, int flength,
602       byte [] qualifier, int qoffset, int qlength) {
603 
604     Cell kv = getColumnLatestCell(family, foffset, flength, qualifier, qoffset, qlength);
605 
606     return (kv != null) && (kv.getValueLength() > 0);
607   }
608 
609   /**
610    * Checks if the specified column contains an empty value (a zero-length byte array).
611    *
612    * @param family family name
613    * @param qualifier column qualifier
614    *
615    * @return whether or not a latest value exists and is empty
616    */
617   public boolean containsEmptyColumn(byte [] family, byte [] qualifier) {
618 
619     return containsEmptyColumn(family, 0, family.length, qualifier, 0, qualifier.length);
620   }
621 
622   /**
623    * Checks if the specified column contains an empty value (a zero-length byte array).
624    *
625    * @param family family name
626    * @param foffset family offset
627    * @param flength family length
628    * @param qualifier column qualifier
629    * @param qoffset qualifier offset
630    * @param qlength qualifier length
631    *
632    * @return whether or not a latest value exists and is empty
633    */
634   public boolean containsEmptyColumn(byte [] family, int foffset, int flength,
635       byte [] qualifier, int qoffset, int qlength) {
636     Cell kv = getColumnLatestCell(family, foffset, flength, qualifier, qoffset, qlength);
637 
638     return (kv != null) && (kv.getValueLength() == 0);
639   }
640 
641   /**
642    * Checks for existence of a value for the specified column (empty or not).
643    *
644    * @param family family name
645    * @param qualifier column qualifier
646    *
647    * @return true if at least one value exists in the result, false if not
648    */
649   public boolean containsColumn(byte [] family, byte [] qualifier) {
650     Cell kv = getColumnLatestCell(family, qualifier);
651     return kv != null;
652   }
653 
654   /**
655    * Checks for existence of a value for the specified column (empty or not).
656    *
657    * @param family family name
658    * @param foffset family offset
659    * @param flength family length
660    * @param qualifier column qualifier
661    * @param qoffset qualifier offset
662    * @param qlength qualifier length
663    *
664    * @return true if at least one value exists in the result, false if not
665    */
666   public boolean containsColumn(byte [] family, int foffset, int flength,
667       byte [] qualifier, int qoffset, int qlength) {
668 
669     return getColumnLatestCell(family, foffset, flength, qualifier, qoffset, qlength) != null;
670   }
671 
672   /**
673    * Map of families to all versions of its qualifiers and values.
674    * <p>
675    * Returns a three level Map of the form:
676    * <code>Map&amp;family,Map&lt;qualifier,Map&lt;timestamp,value>>></code>
677    * <p>
678    * Note: All other map returning methods make use of this map internally.
679    * @return map from families to qualifiers to versions
680    */
681   public NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> getMap() {
682     if (this.familyMap != null) {
683       return this.familyMap;
684     }
685     if(isEmpty()) {
686       return null;
687     }
688     this.familyMap = new TreeMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>(Bytes.BYTES_COMPARATOR);
689     for(Cell kv : this.cells) {
690       byte [] family = CellUtil.cloneFamily(kv);
691       NavigableMap<byte[], NavigableMap<Long, byte[]>> columnMap =
692         familyMap.get(family);
693       if(columnMap == null) {
694         columnMap = new TreeMap<byte[], NavigableMap<Long, byte[]>>
695           (Bytes.BYTES_COMPARATOR);
696         familyMap.put(family, columnMap);
697       }
698       byte [] qualifier = CellUtil.cloneQualifier(kv);
699       NavigableMap<Long, byte[]> versionMap = columnMap.get(qualifier);
700       if(versionMap == null) {
701         versionMap = new TreeMap<Long, byte[]>(new Comparator<Long>() {
702           @Override
703           public int compare(Long l1, Long l2) {
704             return l2.compareTo(l1);
705           }
706         });
707         columnMap.put(qualifier, versionMap);
708       }
709       Long timestamp = kv.getTimestamp();
710       byte [] value = CellUtil.cloneValue(kv);
711 
712       versionMap.put(timestamp, value);
713     }
714     return this.familyMap;
715   }
716 
717   /**
718    * Map of families to their most recent qualifiers and values.
719    * <p>
720    * Returns a two level Map of the form: <code>Map&amp;family,Map&lt;qualifier,value>></code>
721    * <p>
722    * The most recent version of each qualifier will be used.
723    * @return map from families to qualifiers and value
724    */
725   public NavigableMap<byte[], NavigableMap<byte[], byte[]>> getNoVersionMap() {
726     if(this.familyMap == null) {
727       getMap();
728     }
729     if(isEmpty()) {
730       return null;
731     }
732     NavigableMap<byte[], NavigableMap<byte[], byte[]>> returnMap =
733       new TreeMap<byte[], NavigableMap<byte[], byte[]>>(Bytes.BYTES_COMPARATOR);
734     for(Map.Entry<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>
735       familyEntry : familyMap.entrySet()) {
736       NavigableMap<byte[], byte[]> qualifierMap =
737         new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
738       for(Map.Entry<byte[], NavigableMap<Long, byte[]>> qualifierEntry :
739         familyEntry.getValue().entrySet()) {
740         byte [] value =
741           qualifierEntry.getValue().get(qualifierEntry.getValue().firstKey());
742         qualifierMap.put(qualifierEntry.getKey(), value);
743       }
744       returnMap.put(familyEntry.getKey(), qualifierMap);
745     }
746     return returnMap;
747   }
748 
749   /**
750    * Map of qualifiers to values.
751    * <p>
752    * Returns a Map of the form: <code>Map&lt;qualifier,value></code>
753    * @param family column family to get
754    * @return map of qualifiers to values
755    */
756   public NavigableMap<byte[], byte[]> getFamilyMap(byte [] family) {
757     if(this.familyMap == null) {
758       getMap();
759     }
760     if(isEmpty()) {
761       return null;
762     }
763     NavigableMap<byte[], byte[]> returnMap =
764       new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
765     NavigableMap<byte[], NavigableMap<Long, byte[]>> qualifierMap =
766       familyMap.get(family);
767     if(qualifierMap == null) {
768       return returnMap;
769     }
770     for(Map.Entry<byte[], NavigableMap<Long, byte[]>> entry :
771       qualifierMap.entrySet()) {
772       byte [] value =
773         entry.getValue().get(entry.getValue().firstKey());
774       returnMap.put(entry.getKey(), value);
775     }
776     return returnMap;
777   }
778 
779   /**
780    * Returns the value of the first column in the Result.
781    * @return value of the first column
782    */
783   public byte [] value() {
784     if (isEmpty()) {
785       return null;
786     }
787     return CellUtil.cloneValue(cells[0]);
788   }
789 
790   /**
791    * Check if the underlying Cell [] is empty or not
792    * @return true if empty
793    */
794   public boolean isEmpty() {
795     return this.cells == null || this.cells.length == 0;
796   }
797 
798   /**
799    * @return the size of the underlying Cell []
800    */
801   public int size() {
802     return this.cells == null? 0: this.cells.length;
803   }
804 
805   /**
806    * @return String
807    */
808   @Override
809   public String toString() {
810     StringBuilder sb = new StringBuilder();
811     sb.append("keyvalues=");
812     if(isEmpty()) {
813       sb.append("NONE");
814       return sb.toString();
815     }
816     sb.append("{");
817     boolean moreThanOne = false;
818     for(Cell kv : this.cells) {
819       if(moreThanOne) {
820         sb.append(", ");
821       } else {
822         moreThanOne = true;
823       }
824       sb.append(kv.toString());
825     }
826     sb.append("}");
827     return sb.toString();
828   }
829 
830   /**
831    * Does a deep comparison of two Results, down to the byte arrays.
832    * @param res1 first result to compare
833    * @param res2 second result to compare
834    * @throws Exception Every difference is throwing an exception
835    */
836   public static void compareResults(Result res1, Result res2)
837       throws Exception {
838     if (res2 == null) {
839       throw new Exception("There wasn't enough rows, we stopped at "
840           + Bytes.toStringBinary(res1.getRow()));
841     }
842     if (res1.size() != res2.size()) {
843       throw new Exception("This row doesn't have the same number of KVs: "
844           + res1.toString() + " compared to " + res2.toString());
845     }
846     Cell[] ourKVs = res1.rawCells();
847     Cell[] replicatedKVs = res2.rawCells();
848     for (int i = 0; i < res1.size(); i++) {
849       if (!ourKVs[i].equals(replicatedKVs[i]) ||
850           !Bytes.equals(CellUtil.cloneValue(ourKVs[i]), CellUtil.cloneValue(replicatedKVs[i]))) {
851         throw new Exception("This result was different: "
852             + res1.toString() + " compared to " + res2.toString());
853       }
854     }
855   }
856 
857   /**
858    * Forms a single result from the partial results in the partialResults list. This method is
859    * useful for reconstructing partial results on the client side.
860    * @param partialResults list of partial results
861    * @return The complete result that is formed by combining all of the partial results together
862    * @throws IOException A complete result cannot be formed because the results in the partial list
863    *           come from different rows
864    */
865   public static Result createCompleteResult(List<Result> partialResults)
866       throws IOException {
867     List<Cell> cells = new ArrayList<Cell>();
868     boolean stale = false;
869     byte[] prevRow = null;
870     byte[] currentRow = null;
871 
872     if (partialResults != null && !partialResults.isEmpty()) {
873       for (int i = 0; i < partialResults.size(); i++) {
874         Result r = partialResults.get(i);
875         currentRow = r.getRow();
876         if (prevRow != null && !Bytes.equals(prevRow, currentRow)) {
877           throw new IOException(
878               "Cannot form complete result. Rows of partial results do not match." +
879                   " Partial Results: " + partialResults);
880         }
881 
882         // Ensure that all Results except the last one are marked as partials. The last result
883         // may not be marked as a partial because Results are only marked as partials when
884         // the scan on the server side must be stopped due to reaching the maxResultSize.
885         // Visualizing it makes it easier to understand:
886         // maxResultSize: 2 cells
887         // (-x-) represents cell number x in a row
888         // Example: row1: -1- -2- -3- -4- -5- (5 cells total)
889         // How row1 will be returned by the server as partial Results:
890         // Result1: -1- -2- (2 cells, size limit reached, mark as partial)
891         // Result2: -3- -4- (2 cells, size limit reached, mark as partial)
892         // Result3: -5- (1 cell, size limit NOT reached, NOT marked as partial)
893         if (i != (partialResults.size() - 1) && !r.isPartial()) {
894           throw new IOException(
895               "Cannot form complete result. Result is missing partial flag. " +
896                   "Partial Results: " + partialResults);
897         }
898         prevRow = currentRow;
899         stale = stale || r.isStale();
900         for (Cell c : r.rawCells()) {
901           cells.add(c);
902         }
903       }
904     }
905 
906     return Result.create(cells, null, stale);
907   }
908 
909   /**
910    * Get total size of raw cells
911    * @param result
912    * @return Total size.
913    */
914   public static long getTotalSizeOfCells(Result result) {
915     long size = 0;
916     for (Cell c : result.rawCells()) {
917       size += CellUtil.estimatedHeapSizeOf(c);
918     }
919     return size;
920   }
921 
922   /**
923    * Copy another Result into this one. Needed for the old Mapred framework
924    * @throws UnsupportedOperationException if invoked on instance of EMPTY_RESULT
925    * (which is supposed to be immutable).
926    * @param other
927    */
928   public void copyFrom(Result other) {
929     checkReadonly();
930     this.row = null;
931     this.familyMap = null;
932     this.cells = other.cells;
933   }
934 
935   @Override
936   public CellScanner cellScanner() {
937     // Reset
938     this.cellScannerIndex = INITIAL_CELLSCANNER_INDEX;
939     return this;
940   }
941 
942   @Override
943   public Cell current() {
944     if (cells == null) return null;
945     return (cellScannerIndex < 0)? null: this.cells[cellScannerIndex];
946   }
947 
948   @Override
949   public boolean advance() {
950     if (cells == null) return false;
951     return ++cellScannerIndex < this.cells.length;
952   }
953 
954   public Boolean getExists() {
955     return exists;
956   }
957 
958   public void setExists(Boolean exists) {
959     checkReadonly();
960     this.exists = exists;
961   }
962 
963   /**
964    * Whether or not the results are coming from possibly stale data. Stale results
965    * might be returned if {@link Consistency} is not STRONG for the query.
966    * @return Whether or not the results are coming from possibly stale data.
967    */
968   public boolean isStale() {
969     return stale;
970   }
971 
972   /**
973    * Whether or not the result is a partial result. Partial results contain a subset of the cells
974    * for a row and should be combined with a result representing the remaining cells in that row to
975    * form a complete (non-partial) result.
976    * @return Whether or not the result is a partial result
977    */
978   public boolean isPartial() {
979     return partial;
980   }
981 
982   /**
983    * Add load information about the region to the information about the result
984    * @param loadStats statistics about the current region from which this was returned
985    * @deprecated use {@link #setStatistics(ClientProtos.RegionLoadStats)} instead
986    * @throws UnsupportedOperationException if invoked on instance of EMPTY_RESULT
987    * (which is supposed to be immutable).
988    */
989   @InterfaceAudience.Private
990   @Deprecated
991   public void addResults(ClientProtos.RegionLoadStats loadStats) {
992     checkReadonly();
993     this.stats = loadStats;
994   }
995 
996   /**
997    * Set load information about the region to the information about the result
998    * @param loadStats statistics about the current region from which this was returned
999    */
1000   @InterfaceAudience.Private
1001   public void setStatistics(ClientProtos.RegionLoadStats loadStats) {
1002     this.stats = loadStats;
1003   }
1004 
1005   /**
1006    * @return the associated statistics about the region from which this was returned. Can be
1007    * <tt>null</tt> if stats are disabled.
1008    */
1009   public ClientProtos.RegionLoadStats getStats() {
1010     return stats;
1011   }
1012 
1013   /**
1014    * All methods modifying state of Result object must call this method
1015    * to ensure that special purpose immutable Results can't be accidentally modified.
1016    */
1017   private void checkReadonly() {
1018     if (readonly == true) {
1019       throw new UnsupportedOperationException("Attempting to modify readonly EMPTY_RESULT!");
1020     }
1021   }
1022 }