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  package org.apache.hadoop.hbase.protobuf;
19  
20  
21  import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
22  
23  import java.io.ByteArrayOutputStream;
24  import java.io.IOException;
25  import java.lang.reflect.Constructor;
26  import java.lang.reflect.InvocationTargetException;
27  import java.lang.reflect.Method;
28  import java.lang.reflect.ParameterizedType;
29  import java.lang.reflect.Type;
30  import java.nio.ByteBuffer;
31  import java.util.ArrayList;
32  import java.util.Collection;
33  import java.util.HashMap;
34  import java.util.List;
35  import java.util.Map;
36  import java.util.Map.Entry;
37  import java.util.NavigableSet;
38  
39  import org.apache.hadoop.conf.Configuration;
40  import org.apache.hadoop.fs.Path;
41  import org.apache.hadoop.hbase.Cell;
42  import org.apache.hadoop.hbase.CellScanner;
43  import org.apache.hadoop.hbase.CellUtil;
44  import org.apache.hadoop.hbase.DoNotRetryIOException;
45  import org.apache.hadoop.hbase.HBaseConfiguration;
46  import org.apache.hadoop.hbase.HConstants;
47  import org.apache.hadoop.hbase.HRegionInfo;
48  import org.apache.hadoop.hbase.HTableDescriptor;
49  import org.apache.hadoop.hbase.KeyValue;
50  import org.apache.hadoop.hbase.KeyValueUtil;
51  import org.apache.hadoop.hbase.NamespaceDescriptor;
52  import org.apache.hadoop.hbase.ServerName;
53  import org.apache.hadoop.hbase.TableName;
54  import org.apache.hadoop.hbase.Tag;
55  import org.apache.hadoop.hbase.client.Append;
56  import org.apache.hadoop.hbase.client.Consistency;
57  import org.apache.hadoop.hbase.client.Delete;
58  import org.apache.hadoop.hbase.client.Durability;
59  import org.apache.hadoop.hbase.client.Get;
60  import org.apache.hadoop.hbase.client.Increment;
61  import org.apache.hadoop.hbase.client.Mutation;
62  import org.apache.hadoop.hbase.client.Put;
63  import org.apache.hadoop.hbase.client.Result;
64  import org.apache.hadoop.hbase.client.Scan;
65  import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
66  import org.apache.hadoop.hbase.exceptions.DeserializationException;
67  import org.apache.hadoop.hbase.filter.ByteArrayComparable;
68  import org.apache.hadoop.hbase.filter.Filter;
69  import org.apache.hadoop.hbase.io.TimeRange;
70  import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
71  import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
72  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
73  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
74  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
75  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
76  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
77  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
78  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
79  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest;
80  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
81  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRequest;
82  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileResponse;
83  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest;
84  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
85  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
86  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
87  import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
88  import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
89  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
90  import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
91  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
92  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
93  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
94  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
95  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
96  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
97  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
98  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
99  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
100 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
101 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
102 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
103 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
104 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
105 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
106 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
107 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
108 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
109 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
110 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
111 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
112 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
113 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
114 import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
115 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
116 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
117 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
118 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
119 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
120 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
121 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
122 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
123 import org.apache.hadoop.hbase.security.access.Permission;
124 import org.apache.hadoop.hbase.security.access.TablePermission;
125 import org.apache.hadoop.hbase.security.access.UserPermission;
126 import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
127 import org.apache.hadoop.hbase.security.visibility.Authorizations;
128 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
129 import org.apache.hadoop.hbase.util.ByteStringer;
130 import org.apache.hadoop.hbase.util.Bytes;
131 import org.apache.hadoop.hbase.util.DynamicClassLoader;
132 import org.apache.hadoop.hbase.util.ExceptionUtil;
133 import org.apache.hadoop.hbase.util.Methods;
134 import org.apache.hadoop.hbase.util.Pair;
135 import org.apache.hadoop.io.Text;
136 import org.apache.hadoop.ipc.RemoteException;
137 import org.apache.hadoop.security.token.Token;
138 
139 import com.google.common.collect.ArrayListMultimap;
140 import com.google.common.collect.ListMultimap;
141 import com.google.common.collect.Lists;
142 import com.google.protobuf.ByteString;
143 import com.google.protobuf.InvalidProtocolBufferException;
144 import com.google.protobuf.Message;
145 import com.google.protobuf.Parser;
146 import com.google.protobuf.RpcChannel;
147 import com.google.protobuf.Service;
148 import com.google.protobuf.ServiceException;
149 import com.google.protobuf.TextFormat;
150 
151 /**
152  * Protobufs utility.
153  */
154 public final class ProtobufUtil {
155 
156   private ProtobufUtil() {
157   }
158 
159   /**
160    * Primitive type to class mapping.
161    */
162   private final static Map<String, Class<?>>
163     PRIMITIVES = new HashMap<String, Class<?>>();
164 
165 
166   /**
167    * Many results are simple: no cell, exists true or false. To save on object creations,
168    *  we reuse them across calls.
169    */
170   private final static Cell[] EMPTY_CELL_ARRAY = new Cell[]{};
171   private final static Result EMPTY_RESULT = Result.create(EMPTY_CELL_ARRAY);
172   private final static Result EMPTY_RESULT_EXISTS_TRUE = Result.create(null, true);
173   private final static Result EMPTY_RESULT_EXISTS_FALSE = Result.create(null, false);
174   private final static Result EMPTY_RESULT_STALE = Result.create(EMPTY_CELL_ARRAY, null, true);
175   private final static Result EMPTY_RESULT_EXISTS_TRUE_STALE
176     = Result.create((Cell[])null, true, true);
177   private final static Result EMPTY_RESULT_EXISTS_FALSE_STALE
178     = Result.create((Cell[])null, false, true);
179 
180   private final static ClientProtos.Result EMPTY_RESULT_PB;
181   private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_TRUE;
182   private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_FALSE;
183   private final static ClientProtos.Result EMPTY_RESULT_PB_STALE;
184   private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_TRUE_STALE;
185   private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_FALSE_STALE;
186 
187 
188   static {
189     ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
190 
191     builder.setExists(true);
192     builder.setAssociatedCellCount(0);
193     EMPTY_RESULT_PB_EXISTS_TRUE =  builder.build();
194 
195     builder.setStale(true);
196     EMPTY_RESULT_PB_EXISTS_TRUE_STALE = builder.build();
197     builder.clear();
198 
199     builder.setExists(false);
200     builder.setAssociatedCellCount(0);
201     EMPTY_RESULT_PB_EXISTS_FALSE =  builder.build();
202     builder.setStale(true);
203     EMPTY_RESULT_PB_EXISTS_FALSE_STALE = builder.build();
204 
205     builder.clear();
206     builder.setAssociatedCellCount(0);
207     EMPTY_RESULT_PB =  builder.build();
208     builder.setStale(true);
209     EMPTY_RESULT_PB_STALE = builder.build();
210   }
211 
212   /**
213    * Dynamic class loader to load filter/comparators
214    */
215   private final static ClassLoader CLASS_LOADER;
216 
217   static {
218     ClassLoader parent = ProtobufUtil.class.getClassLoader();
219     Configuration conf = HBaseConfiguration.create();
220     CLASS_LOADER = new DynamicClassLoader(conf, parent);
221 
222     PRIMITIVES.put(Boolean.TYPE.getName(), Boolean.TYPE);
223     PRIMITIVES.put(Byte.TYPE.getName(), Byte.TYPE);
224     PRIMITIVES.put(Character.TYPE.getName(), Character.TYPE);
225     PRIMITIVES.put(Short.TYPE.getName(), Short.TYPE);
226     PRIMITIVES.put(Integer.TYPE.getName(), Integer.TYPE);
227     PRIMITIVES.put(Long.TYPE.getName(), Long.TYPE);
228     PRIMITIVES.put(Float.TYPE.getName(), Float.TYPE);
229     PRIMITIVES.put(Double.TYPE.getName(), Double.TYPE);
230     PRIMITIVES.put(Void.TYPE.getName(), Void.TYPE);
231   }
232 
233   /**
234    * Magic we put ahead of a serialized protobuf message.
235    * For example, all znode content is protobuf messages with the below magic
236    * for preamble.
237    */
238   public static final byte [] PB_MAGIC = new byte [] {'P', 'B', 'U', 'F'};
239   private static final String PB_MAGIC_STR = Bytes.toString(PB_MAGIC);
240 
241   /**
242    * Prepend the passed bytes with four bytes of magic, {@link #PB_MAGIC}, to flag what
243    * follows as a protobuf in hbase.  Prepend these bytes to all content written to znodes, etc.
244    * @param bytes Bytes to decorate
245    * @return The passed <code>bytes</codes> with magic prepended (Creates a new
246    * byte array that is <code>bytes.length</code> plus {@link #PB_MAGIC}.length.
247    */
248   public static byte [] prependPBMagic(final byte [] bytes) {
249     return Bytes.add(PB_MAGIC, bytes);
250   }
251 
252   /**
253    * @param bytes Bytes to check.
254    * @return True if passed <code>bytes</code> has {@link #PB_MAGIC} for a prefix.
255    */
256   public static boolean isPBMagicPrefix(final byte [] bytes) {
257     return isPBMagicPrefix(bytes, 0, bytes.length);
258   }
259 
260   /**
261    * @param bytes Bytes to check.
262    * @return True if passed <code>bytes</code> has {@link #PB_MAGIC} for a prefix.
263    */
264   public static boolean isPBMagicPrefix(final byte [] bytes, int offset, int len) {
265     if (bytes == null || len < PB_MAGIC.length) return false;
266     return Bytes.compareTo(PB_MAGIC, 0, PB_MAGIC.length, bytes, offset, PB_MAGIC.length) == 0;
267   }
268 
269   /**
270    * @param bytes
271    * @throws DeserializationException if we are missing the pb magic prefix
272    */
273   public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
274     if (!isPBMagicPrefix(bytes)) {
275       throw new DeserializationException("Missing pb magic " + PB_MAGIC_STR + " prefix");
276     }
277   }
278 
279   /**
280    * @return Length of {@link #PB_MAGIC}
281    */
282   public static int lengthOfPBMagic() {
283     return PB_MAGIC.length;
284   }
285 
286   /**
287    * Return the IOException thrown by the remote server wrapped in
288    * ServiceException as cause.
289    *
290    * @param se ServiceException that wraps IO exception thrown by the server
291    * @return Exception wrapped in ServiceException or
292    *   a new IOException that wraps the unexpected ServiceException.
293    */
294   public static IOException getRemoteException(ServiceException se) {
295     Throwable e = se.getCause();
296     if (e == null) {
297       return new IOException(se);
298     }
299     if (ExceptionUtil.isInterrupt(e)) {
300       return ExceptionUtil.asInterrupt(e);
301     }
302     if (e instanceof RemoteException) {
303       e = ((RemoteException) e).unwrapRemoteException();
304     }
305     return e instanceof IOException ? (IOException) e : new IOException(se);
306   }
307 
308   /**
309    * Convert a ServerName to a protocol buffer ServerName
310    *
311    * @param serverName the ServerName to convert
312    * @return the converted protocol buffer ServerName
313    * @see #toServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName)
314    */
315   public static HBaseProtos.ServerName
316       toServerName(final ServerName serverName) {
317     if (serverName == null) return null;
318     HBaseProtos.ServerName.Builder builder =
319       HBaseProtos.ServerName.newBuilder();
320     builder.setHostName(serverName.getHostname());
321     if (serverName.getPort() >= 0) {
322       builder.setPort(serverName.getPort());
323     }
324     if (serverName.getStartcode() >= 0) {
325       builder.setStartCode(serverName.getStartcode());
326     }
327     return builder.build();
328   }
329 
330   /**
331    * Convert a protocol buffer ServerName to a ServerName
332    *
333    * @param proto the protocol buffer ServerName to convert
334    * @return the converted ServerName
335    */
336   public static ServerName toServerName(final HBaseProtos.ServerName proto) {
337     if (proto == null) return null;
338     String hostName = proto.getHostName();
339     long startCode = -1;
340     int port = -1;
341     if (proto.hasPort()) {
342       port = proto.getPort();
343     }
344     if (proto.hasStartCode()) {
345       startCode = proto.getStartCode();
346     }
347     return ServerName.valueOf(hostName, port, startCode);
348   }
349 
350   /**
351    * Get HTableDescriptor[] from GetTableDescriptorsResponse protobuf
352    *
353    * @param proto the GetTableDescriptorsResponse
354    * @return HTableDescriptor[]
355    */
356   public static HTableDescriptor[] getHTableDescriptorArray(GetTableDescriptorsResponse proto) {
357     if (proto == null) return null;
358 
359     HTableDescriptor[] ret = new HTableDescriptor[proto.getTableSchemaCount()];
360     for (int i = 0; i < proto.getTableSchemaCount(); ++i) {
361       ret[i] = HTableDescriptor.convert(proto.getTableSchema(i));
362     }
363     return ret;
364   }
365 
366   /**
367    * get the split keys in form "byte [][]" from a CreateTableRequest proto
368    *
369    * @param proto the CreateTableRequest
370    * @return the split keys
371    */
372   public static byte [][] getSplitKeysArray(final CreateTableRequest proto) {
373     byte [][] splitKeys = new byte[proto.getSplitKeysCount()][];
374     for (int i = 0; i < proto.getSplitKeysCount(); ++i) {
375       splitKeys[i] = proto.getSplitKeys(i).toByteArray();
376     }
377     return splitKeys;
378   }
379 
380   /**
381    * Convert a protobuf Durability into a client Durability
382    */
383   public static Durability toDurability(
384       final ClientProtos.MutationProto.Durability proto) {
385     switch(proto) {
386     case USE_DEFAULT:
387       return Durability.USE_DEFAULT;
388     case SKIP_WAL:
389       return Durability.SKIP_WAL;
390     case ASYNC_WAL:
391       return Durability.ASYNC_WAL;
392     case SYNC_WAL:
393       return Durability.SYNC_WAL;
394     case FSYNC_WAL:
395       return Durability.FSYNC_WAL;
396     default:
397       return Durability.USE_DEFAULT;
398     }
399   }
400 
401   /**
402    * Convert a client Durability into a protbuf Durability
403    */
404   public static ClientProtos.MutationProto.Durability toDurability(
405       final Durability d) {
406     switch(d) {
407     case USE_DEFAULT:
408       return ClientProtos.MutationProto.Durability.USE_DEFAULT;
409     case SKIP_WAL:
410       return ClientProtos.MutationProto.Durability.SKIP_WAL;
411     case ASYNC_WAL:
412       return ClientProtos.MutationProto.Durability.ASYNC_WAL;
413     case SYNC_WAL:
414       return ClientProtos.MutationProto.Durability.SYNC_WAL;
415     case FSYNC_WAL:
416       return ClientProtos.MutationProto.Durability.FSYNC_WAL;
417     default:
418       return ClientProtos.MutationProto.Durability.USE_DEFAULT;
419     }
420   }
421 
422   /**
423    * Convert a protocol buffer Get to a client Get
424    *
425    * @param proto the protocol buffer Get to convert
426    * @return the converted client Get
427    * @throws IOException
428    */
429   public static Get toGet(
430       final ClientProtos.Get proto) throws IOException {
431     if (proto == null) return null;
432     byte[] row = proto.getRow().toByteArray();
433     Get get = new Get(row);
434     if (proto.hasCacheBlocks()) {
435       get.setCacheBlocks(proto.getCacheBlocks());
436     }
437     if (proto.hasMaxVersions()) {
438       get.setMaxVersions(proto.getMaxVersions());
439     }
440     if (proto.hasStoreLimit()) {
441       get.setMaxResultsPerColumnFamily(proto.getStoreLimit());
442     }
443     if (proto.hasStoreOffset()) {
444       get.setRowOffsetPerColumnFamily(proto.getStoreOffset());
445     }
446     if (proto.hasTimeRange()) {
447       HBaseProtos.TimeRange timeRange = proto.getTimeRange();
448       long minStamp = 0;
449       long maxStamp = Long.MAX_VALUE;
450       if (timeRange.hasFrom()) {
451         minStamp = timeRange.getFrom();
452       }
453       if (timeRange.hasTo()) {
454         maxStamp = timeRange.getTo();
455       }
456       get.setTimeRange(minStamp, maxStamp);
457     }
458     if (proto.hasFilter()) {
459       FilterProtos.Filter filter = proto.getFilter();
460       get.setFilter(ProtobufUtil.toFilter(filter));
461     }
462     for (NameBytesPair attribute: proto.getAttributeList()) {
463       get.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
464     }
465     if (proto.getColumnCount() > 0) {
466       for (Column column: proto.getColumnList()) {
467         byte[] family = column.getFamily().toByteArray();
468         if (column.getQualifierCount() > 0) {
469           for (ByteString qualifier: column.getQualifierList()) {
470             get.addColumn(family, qualifier.toByteArray());
471           }
472         } else {
473           get.addFamily(family);
474         }
475       }
476     }
477     if (proto.hasExistenceOnly() && proto.getExistenceOnly()){
478       get.setCheckExistenceOnly(true);
479     }
480     if (proto.hasClosestRowBefore() && proto.getClosestRowBefore()){
481       get.setClosestRowBefore(true);
482     }
483     if (proto.hasConsistency()) {
484       get.setConsistency(toConsistency(proto.getConsistency()));
485     }
486     return get;
487   }
488 
489   public static Consistency toConsistency(ClientProtos.Consistency consistency) {
490     switch (consistency) {
491       case STRONG : return Consistency.STRONG;
492       case TIMELINE : return Consistency.TIMELINE;
493       default : return Consistency.STRONG;
494     }
495   }
496 
497   public static ClientProtos.Consistency toConsistency(Consistency consistency) {
498     switch (consistency) {
499       case STRONG : return ClientProtos.Consistency.STRONG;
500       case TIMELINE : return ClientProtos.Consistency.TIMELINE;
501       default : return ClientProtos.Consistency.STRONG;
502     }
503   }
504 
505   /**
506    * Convert a protocol buffer Mutate to a Put.
507    *
508    * @param proto The protocol buffer MutationProto to convert
509    * @return A client Put.
510    * @throws IOException
511    */
512   public static Put toPut(final MutationProto proto)
513   throws IOException {
514     return toPut(proto, null);
515   }
516 
517   /**
518    * Convert a protocol buffer Mutate to a Put.
519    *
520    * @param proto The protocol buffer MutationProto to convert
521    * @param cellScanner If non-null, the Cell data that goes with this proto.
522    * @return A client Put.
523    * @throws IOException
524    */
525   public static Put toPut(final MutationProto proto, final CellScanner cellScanner)
526   throws IOException {
527     // TODO: Server-side at least why do we convert back to the Client types?  Why not just pb it?
528     MutationType type = proto.getMutateType();
529     assert type == MutationType.PUT: type.name();
530     long timestamp = proto.hasTimestamp()? proto.getTimestamp(): HConstants.LATEST_TIMESTAMP;
531     Put put = null;
532     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
533     if (cellCount > 0) {
534       // The proto has metadata only and the data is separate to be found in the cellScanner.
535       if (cellScanner == null) {
536         throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
537             toShortString(proto));
538       }
539       for (int i = 0; i < cellCount; i++) {
540         if (!cellScanner.advance()) {
541           throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
542             " no cell returned: " + toShortString(proto));
543         }
544         Cell cell = cellScanner.current();
545         if (put == null) {
546           put = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
547         }
548         put.add(cell);
549       }
550     } else {
551       if (proto.hasRow()) {
552         put = new Put(proto.getRow().asReadOnlyByteBuffer(), timestamp);
553       } else {
554         throw new IllegalArgumentException("row cannot be null");
555       }
556       // The proto has the metadata and the data itself
557       for (ColumnValue column: proto.getColumnValueList()) {
558         byte[] family = column.getFamily().toByteArray();
559         for (QualifierValue qv: column.getQualifierValueList()) {
560           if (!qv.hasValue()) {
561             throw new DoNotRetryIOException(
562                 "Missing required field: qualifier value");
563           }
564           ByteBuffer qualifier =
565               qv.hasQualifier() ? qv.getQualifier().asReadOnlyByteBuffer() : null;
566           ByteBuffer value =
567               qv.hasValue() ? qv.getValue().asReadOnlyByteBuffer() : null;
568           long ts = timestamp;
569           if (qv.hasTimestamp()) {
570             ts = qv.getTimestamp();
571           }
572           byte[] tags;
573           if (qv.hasTags()) {
574             tags = qv.getTags().toByteArray();
575             Object[] array = Tag.asList(tags, 0, (short)tags.length).toArray();
576             Tag[] tagArray = new Tag[array.length];
577             for(int i = 0; i< array.length; i++) {
578               tagArray[i] = (Tag)array[i];
579             }
580             put.addImmutable(family, qualifier, ts, value, tagArray);
581           } else {
582             put.addImmutable(family, qualifier, ts, value);
583           }
584         }
585       }
586     }
587     put.setDurability(toDurability(proto.getDurability()));
588     for (NameBytesPair attribute: proto.getAttributeList()) {
589       put.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
590     }
591     return put;
592   }
593 
594   /**
595    * Convert a protocol buffer Mutate to a Delete
596    *
597    * @param proto the protocol buffer Mutate to convert
598    * @return the converted client Delete
599    * @throws IOException
600    */
601   public static Delete toDelete(final MutationProto proto)
602   throws IOException {
603     return toDelete(proto, null);
604   }
605 
606   /**
607    * Convert a protocol buffer Mutate to a Delete
608    *
609    * @param proto the protocol buffer Mutate to convert
610    * @param cellScanner if non-null, the data that goes with this delete.
611    * @return the converted client Delete
612    * @throws IOException
613    */
614   public static Delete toDelete(final MutationProto proto, final CellScanner cellScanner)
615   throws IOException {
616     MutationType type = proto.getMutateType();
617     assert type == MutationType.DELETE : type.name();
618     byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
619     long timestamp = HConstants.LATEST_TIMESTAMP;
620     if (proto.hasTimestamp()) {
621       timestamp = proto.getTimestamp();
622     }
623     Delete delete = null;
624     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
625     if (cellCount > 0) {
626       // The proto has metadata only and the data is separate to be found in the cellScanner.
627       if (cellScanner == null) {
628         // TextFormat should be fine for a Delete since it carries no data, just coordinates.
629         throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
630           TextFormat.shortDebugString(proto));
631       }
632       for (int i = 0; i < cellCount; i++) {
633         if (!cellScanner.advance()) {
634           // TextFormat should be fine for a Delete since it carries no data, just coordinates.
635           throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
636             " no cell returned: " + TextFormat.shortDebugString(proto));
637         }
638         Cell cell = cellScanner.current();
639         if (delete == null) {
640           delete =
641             new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
642         }
643         delete.addDeleteMarker(KeyValueUtil.ensureKeyValue(cell));
644       }
645     } else {
646       delete = new Delete(row, timestamp);
647       for (ColumnValue column: proto.getColumnValueList()) {
648         byte[] family = column.getFamily().toByteArray();
649         for (QualifierValue qv: column.getQualifierValueList()) {
650           DeleteType deleteType = qv.getDeleteType();
651           byte[] qualifier = null;
652           if (qv.hasQualifier()) {
653             qualifier = qv.getQualifier().toByteArray();
654           }
655           long ts = HConstants.LATEST_TIMESTAMP;
656           if (qv.hasTimestamp()) {
657             ts = qv.getTimestamp();
658           }
659           if (deleteType == DeleteType.DELETE_ONE_VERSION) {
660             delete.deleteColumn(family, qualifier, ts);
661           } else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) {
662             delete.deleteColumns(family, qualifier, ts);
663           } else if (deleteType == DeleteType.DELETE_FAMILY_VERSION) {
664             delete.deleteFamilyVersion(family, ts);
665           } else {
666             delete.deleteFamily(family, ts);
667           }
668         }
669       }
670     }
671     delete.setDurability(toDurability(proto.getDurability()));
672     for (NameBytesPair attribute: proto.getAttributeList()) {
673       delete.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
674     }
675     return delete;
676   }
677 
678   /**
679    * Convert a protocol buffer Mutate to an Append
680    * @param cellScanner
681    * @param proto the protocol buffer Mutate to convert
682    * @return the converted client Append
683    * @throws IOException
684    */
685   public static Append toAppend(final MutationProto proto, final CellScanner cellScanner)
686   throws IOException {
687     MutationType type = proto.getMutateType();
688     assert type == MutationType.APPEND : type.name();
689     byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
690     Append append = null;
691     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
692     if (cellCount > 0) {
693       // The proto has metadata only and the data is separate to be found in the cellScanner.
694       if (cellScanner == null) {
695         throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
696           toShortString(proto));
697       }
698       for (int i = 0; i < cellCount; i++) {
699         if (!cellScanner.advance()) {
700           throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
701             " no cell returned: " + toShortString(proto));
702         }
703         Cell cell = cellScanner.current();
704         if (append == null) {
705           append = new Append(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
706         }
707         append.add(KeyValueUtil.ensureKeyValue(cell));
708       }
709     } else {
710       append = new Append(row);
711       for (ColumnValue column: proto.getColumnValueList()) {
712         byte[] family = column.getFamily().toByteArray();
713         for (QualifierValue qv: column.getQualifierValueList()) {
714           byte[] qualifier = qv.getQualifier().toByteArray();
715           if (!qv.hasValue()) {
716             throw new DoNotRetryIOException(
717               "Missing required field: qualifier value");
718           }
719           byte[] value = qv.getValue().toByteArray();
720           byte[] tags = null;
721           if (qv.hasTags()) {
722             tags = qv.getTags().toByteArray();
723           }
724           append.add(CellUtil.createCell(row, family, qualifier, qv.getTimestamp(),
725               KeyValue.Type.Put, value, tags));
726         }
727       }
728     }
729     append.setDurability(toDurability(proto.getDurability()));
730     for (NameBytesPair attribute: proto.getAttributeList()) {
731       append.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
732     }
733     return append;
734   }
735 
736   /**
737    * Convert a MutateRequest to Mutation
738    *
739    * @param proto the protocol buffer Mutate to convert
740    * @return the converted Mutation
741    * @throws IOException
742    */
743   public static Mutation toMutation(final MutationProto proto) throws IOException {
744     MutationType type = proto.getMutateType();
745     if (type == MutationType.APPEND) {
746       return toAppend(proto, null);
747     }
748     if (type == MutationType.DELETE) {
749       return toDelete(proto, null);
750     }
751     if (type == MutationType.PUT) {
752       return toPut(proto, null);
753     }
754     throw new IOException("Unknown mutation type " + type);
755   }
756 
757   /**
758    * Convert a protocol buffer Mutate to an Increment
759    *
760    * @param proto the protocol buffer Mutate to convert
761    * @return the converted client Increment
762    * @throws IOException
763    */
764   public static Increment toIncrement(final MutationProto proto, final CellScanner cellScanner)
765   throws IOException {
766     MutationType type = proto.getMutateType();
767     assert type == MutationType.INCREMENT : type.name();
768     byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
769     Increment increment = null;
770     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
771     if (cellCount > 0) {
772       // The proto has metadata only and the data is separate to be found in the cellScanner.
773       if (cellScanner == null) {
774         throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
775           TextFormat.shortDebugString(proto));
776       }
777       for (int i = 0; i < cellCount; i++) {
778         if (!cellScanner.advance()) {
779           throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
780             " no cell returned: " + TextFormat.shortDebugString(proto));
781         }
782         Cell cell = cellScanner.current();
783         if (increment == null) {
784           increment = new Increment(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
785         }
786         increment.add(KeyValueUtil.ensureKeyValue(cell));
787       }
788     } else {
789       increment = new Increment(row);
790       for (ColumnValue column: proto.getColumnValueList()) {
791         byte[] family = column.getFamily().toByteArray();
792         for (QualifierValue qv: column.getQualifierValueList()) {
793           byte[] qualifier = qv.getQualifier().toByteArray();
794           if (!qv.hasValue()) {
795             throw new DoNotRetryIOException("Missing required field: qualifier value");
796           }
797           byte[] value = qv.getValue().toByteArray();
798           byte[] tags = null;
799           if (qv.hasTags()) {
800             tags = qv.getTags().toByteArray();
801           }
802           increment.add(CellUtil.createCell(row, family, qualifier, qv.getTimestamp(),
803               KeyValue.Type.Put, value, tags));
804         }
805       }
806     }
807     if (proto.hasTimeRange()) {
808       HBaseProtos.TimeRange timeRange = proto.getTimeRange();
809       long minStamp = 0;
810       long maxStamp = Long.MAX_VALUE;
811       if (timeRange.hasFrom()) {
812         minStamp = timeRange.getFrom();
813       }
814       if (timeRange.hasTo()) {
815         maxStamp = timeRange.getTo();
816       }
817       increment.setTimeRange(minStamp, maxStamp);
818     }
819     increment.setDurability(toDurability(proto.getDurability()));
820     for (NameBytesPair attribute : proto.getAttributeList()) {
821       increment.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
822     }
823     return increment;
824   }
825 
826   /**
827    * Convert a client Scan to a protocol buffer Scan
828    *
829    * @param scan the client Scan to convert
830    * @return the converted protocol buffer Scan
831    * @throws IOException
832    */
833   public static ClientProtos.Scan toScan(
834       final Scan scan) throws IOException {
835     ClientProtos.Scan.Builder scanBuilder =
836       ClientProtos.Scan.newBuilder();
837     scanBuilder.setCacheBlocks(scan.getCacheBlocks());
838     if (scan.getBatch() > 0) {
839       scanBuilder.setBatchSize(scan.getBatch());
840     }
841     if (scan.getMaxResultSize() > 0) {
842       scanBuilder.setMaxResultSize(scan.getMaxResultSize());
843     }
844     if (scan.isSmall()) {
845       scanBuilder.setSmall(scan.isSmall());
846     }
847     Boolean loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue();
848     if (loadColumnFamiliesOnDemand != null) {
849       scanBuilder.setLoadColumnFamiliesOnDemand(loadColumnFamiliesOnDemand.booleanValue());
850     }
851     scanBuilder.setMaxVersions(scan.getMaxVersions());
852     TimeRange timeRange = scan.getTimeRange();
853     if (!timeRange.isAllTime()) {
854       HBaseProtos.TimeRange.Builder timeRangeBuilder =
855         HBaseProtos.TimeRange.newBuilder();
856       timeRangeBuilder.setFrom(timeRange.getMin());
857       timeRangeBuilder.setTo(timeRange.getMax());
858       scanBuilder.setTimeRange(timeRangeBuilder.build());
859     }
860     Map<String, byte[]> attributes = scan.getAttributesMap();
861     if (!attributes.isEmpty()) {
862       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
863       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
864         attributeBuilder.setName(attribute.getKey());
865         attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
866         scanBuilder.addAttribute(attributeBuilder.build());
867       }
868     }
869     byte[] startRow = scan.getStartRow();
870     if (startRow != null && startRow.length > 0) {
871       scanBuilder.setStartRow(ByteStringer.wrap(startRow));
872     }
873     byte[] stopRow = scan.getStopRow();
874     if (stopRow != null && stopRow.length > 0) {
875       scanBuilder.setStopRow(ByteStringer.wrap(stopRow));
876     }
877     if (scan.hasFilter()) {
878       scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
879     }
880     if (scan.hasFamilies()) {
881       Column.Builder columnBuilder = Column.newBuilder();
882       for (Map.Entry<byte[],NavigableSet<byte []>>
883           family: scan.getFamilyMap().entrySet()) {
884         columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
885         NavigableSet<byte []> qualifiers = family.getValue();
886         columnBuilder.clearQualifier();
887         if (qualifiers != null && qualifiers.size() > 0) {
888           for (byte [] qualifier: qualifiers) {
889             columnBuilder.addQualifier(ByteStringer.wrap(qualifier));
890           }
891         }
892         scanBuilder.addColumn(columnBuilder.build());
893       }
894     }
895     if (scan.getMaxResultsPerColumnFamily() >= 0) {
896       scanBuilder.setStoreLimit(scan.getMaxResultsPerColumnFamily());
897     }
898     if (scan.getRowOffsetPerColumnFamily() > 0) {
899       scanBuilder.setStoreOffset(scan.getRowOffsetPerColumnFamily());
900     }
901     if (scan.isReversed()) {
902       scanBuilder.setReversed(scan.isReversed());
903     }
904     if (scan.getConsistency() == Consistency.TIMELINE) {
905       scanBuilder.setConsistency(toConsistency(scan.getConsistency()));
906     }
907     return scanBuilder.build();
908   }
909 
910   /**
911    * Convert a protocol buffer Scan to a client Scan
912    *
913    * @param proto the protocol buffer Scan to convert
914    * @return the converted client Scan
915    * @throws IOException
916    */
917   public static Scan toScan(
918       final ClientProtos.Scan proto) throws IOException {
919     byte [] startRow = HConstants.EMPTY_START_ROW;
920     byte [] stopRow  = HConstants.EMPTY_END_ROW;
921     if (proto.hasStartRow()) {
922       startRow = proto.getStartRow().toByteArray();
923     }
924     if (proto.hasStopRow()) {
925       stopRow = proto.getStopRow().toByteArray();
926     }
927     Scan scan = new Scan(startRow, stopRow);
928     if (proto.hasCacheBlocks()) {
929       scan.setCacheBlocks(proto.getCacheBlocks());
930     }
931     if (proto.hasMaxVersions()) {
932       scan.setMaxVersions(proto.getMaxVersions());
933     }
934     if (proto.hasStoreLimit()) {
935       scan.setMaxResultsPerColumnFamily(proto.getStoreLimit());
936     }
937     if (proto.hasStoreOffset()) {
938       scan.setRowOffsetPerColumnFamily(proto.getStoreOffset());
939     }
940     if (proto.hasLoadColumnFamiliesOnDemand()) {
941       scan.setLoadColumnFamiliesOnDemand(proto.getLoadColumnFamiliesOnDemand());
942     }
943     if (proto.hasTimeRange()) {
944       HBaseProtos.TimeRange timeRange = proto.getTimeRange();
945       long minStamp = 0;
946       long maxStamp = Long.MAX_VALUE;
947       if (timeRange.hasFrom()) {
948         minStamp = timeRange.getFrom();
949       }
950       if (timeRange.hasTo()) {
951         maxStamp = timeRange.getTo();
952       }
953       scan.setTimeRange(minStamp, maxStamp);
954     }
955     if (proto.hasFilter()) {
956       FilterProtos.Filter filter = proto.getFilter();
957       scan.setFilter(ProtobufUtil.toFilter(filter));
958     }
959     if (proto.hasBatchSize()) {
960       scan.setBatch(proto.getBatchSize());
961     }
962     if (proto.hasMaxResultSize()) {
963       scan.setMaxResultSize(proto.getMaxResultSize());
964     }
965     if (proto.hasSmall()) {
966       scan.setSmall(proto.getSmall());
967     }
968     for (NameBytesPair attribute: proto.getAttributeList()) {
969       scan.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
970     }
971     if (proto.getColumnCount() > 0) {
972       for (Column column: proto.getColumnList()) {
973         byte[] family = column.getFamily().toByteArray();
974         if (column.getQualifierCount() > 0) {
975           for (ByteString qualifier: column.getQualifierList()) {
976             scan.addColumn(family, qualifier.toByteArray());
977           }
978         } else {
979           scan.addFamily(family);
980         }
981       }
982     }
983     if (proto.hasReversed()) {
984       scan.setReversed(proto.getReversed());
985     }
986     if (proto.hasConsistency()) {
987       scan.setConsistency(toConsistency(proto.getConsistency()));
988     }
989     return scan;
990   }
991 
992   /**
993    * Create a protocol buffer Get based on a client Get.
994    *
995    * @param get the client Get
996    * @return a protocol buffer Get
997    * @throws IOException
998    */
999   public static ClientProtos.Get toGet(
1000       final Get get) throws IOException {
1001     ClientProtos.Get.Builder builder =
1002       ClientProtos.Get.newBuilder();
1003     builder.setRow(ByteStringer.wrap(get.getRow()));
1004     builder.setCacheBlocks(get.getCacheBlocks());
1005     builder.setMaxVersions(get.getMaxVersions());
1006     if (get.getFilter() != null) {
1007       builder.setFilter(ProtobufUtil.toFilter(get.getFilter()));
1008     }
1009     TimeRange timeRange = get.getTimeRange();
1010     if (!timeRange.isAllTime()) {
1011       HBaseProtos.TimeRange.Builder timeRangeBuilder =
1012         HBaseProtos.TimeRange.newBuilder();
1013       timeRangeBuilder.setFrom(timeRange.getMin());
1014       timeRangeBuilder.setTo(timeRange.getMax());
1015       builder.setTimeRange(timeRangeBuilder.build());
1016     }
1017     Map<String, byte[]> attributes = get.getAttributesMap();
1018     if (!attributes.isEmpty()) {
1019       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
1020       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
1021         attributeBuilder.setName(attribute.getKey());
1022         attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
1023         builder.addAttribute(attributeBuilder.build());
1024       }
1025     }
1026     if (get.hasFamilies()) {
1027       Column.Builder columnBuilder = Column.newBuilder();
1028       Map<byte[], NavigableSet<byte[]>> families = get.getFamilyMap();
1029       for (Map.Entry<byte[], NavigableSet<byte[]>> family: families.entrySet()) {
1030         NavigableSet<byte[]> qualifiers = family.getValue();
1031         columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
1032         columnBuilder.clearQualifier();
1033         if (qualifiers != null && qualifiers.size() > 0) {
1034           for (byte[] qualifier: qualifiers) {
1035             columnBuilder.addQualifier(ByteStringer.wrap(qualifier));
1036           }
1037         }
1038         builder.addColumn(columnBuilder.build());
1039       }
1040     }
1041     if (get.getMaxResultsPerColumnFamily() >= 0) {
1042       builder.setStoreLimit(get.getMaxResultsPerColumnFamily());
1043     }
1044     if (get.getRowOffsetPerColumnFamily() > 0) {
1045       builder.setStoreOffset(get.getRowOffsetPerColumnFamily());
1046     }
1047     if (get.isCheckExistenceOnly()){
1048       builder.setExistenceOnly(true);
1049     }
1050     if (get.isClosestRowBefore()){
1051       builder.setClosestRowBefore(true);
1052     }
1053     if (get.getConsistency() != null && get.getConsistency() != Consistency.STRONG) {
1054       builder.setConsistency(toConsistency(get.getConsistency()));
1055     }
1056 
1057     return builder.build();
1058   }
1059 
1060   /**
1061    * Convert a client Increment to a protobuf Mutate.
1062    *
1063    * @param increment
1064    * @return the converted mutate
1065    */
1066   public static MutationProto toMutation(
1067     final Increment increment, final MutationProto.Builder builder, long nonce) {
1068     builder.setRow(ByteStringer.wrap(increment.getRow()));
1069     builder.setMutateType(MutationType.INCREMENT);
1070     builder.setDurability(toDurability(increment.getDurability()));
1071     if (nonce != HConstants.NO_NONCE) {
1072       builder.setNonce(nonce);
1073     }
1074     TimeRange timeRange = increment.getTimeRange();
1075     if (!timeRange.isAllTime()) {
1076       HBaseProtos.TimeRange.Builder timeRangeBuilder =
1077         HBaseProtos.TimeRange.newBuilder();
1078       timeRangeBuilder.setFrom(timeRange.getMin());
1079       timeRangeBuilder.setTo(timeRange.getMax());
1080       builder.setTimeRange(timeRangeBuilder.build());
1081     }
1082     ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
1083     QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
1084     for (Map.Entry<byte[], List<Cell>> family: increment.getFamilyCellMap().entrySet()) {
1085       columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
1086       columnBuilder.clearQualifierValue();
1087       List<Cell> values = family.getValue();
1088       if (values != null && values.size() > 0) {
1089         for (Cell cell: values) {
1090           KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
1091           valueBuilder.setQualifier(ByteStringer.wrap(
1092               kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()));
1093           valueBuilder.setValue(ByteStringer.wrap(
1094               kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
1095           if (kv.getTagsLength() > 0) {
1096             valueBuilder.setTags(ByteStringer.wrap(kv.getTagsArray(),
1097                 kv.getTagsOffset(), kv.getTagsLength()));
1098           }
1099           columnBuilder.addQualifierValue(valueBuilder.build());
1100         }
1101       }
1102       builder.addColumnValue(columnBuilder.build());
1103     }
1104     Map<String, byte[]> attributes = increment.getAttributesMap();
1105     if (!attributes.isEmpty()) {
1106       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
1107       for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) {
1108         attributeBuilder.setName(attribute.getKey());
1109         attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
1110         builder.addAttribute(attributeBuilder.build());
1111       }
1112     }
1113     return builder.build();
1114   }
1115 
1116   public static MutationProto toMutation(final MutationType type, final Mutation mutation)
1117     throws IOException {
1118     return toMutation(type, mutation, HConstants.NO_NONCE);
1119   }
1120 
1121   /**
1122    * Create a protocol buffer Mutate based on a client Mutation
1123    *
1124    * @param type
1125    * @param mutation
1126    * @return a protobuf'd Mutation
1127    * @throws IOException
1128    */
1129   public static MutationProto toMutation(final MutationType type, final Mutation mutation,
1130     final long nonce) throws IOException {
1131     return toMutation(type, mutation, MutationProto.newBuilder(), nonce);
1132   }
1133 
1134   public static MutationProto toMutation(final MutationType type, final Mutation mutation,
1135       MutationProto.Builder builder) throws IOException {
1136     return toMutation(type, mutation, builder, HConstants.NO_NONCE);
1137   }
1138 
1139   public static MutationProto toMutation(final MutationType type, final Mutation mutation,
1140       MutationProto.Builder builder, long nonce)
1141   throws IOException {
1142     builder = getMutationBuilderAndSetCommonFields(type, mutation, builder);
1143     if (nonce != HConstants.NO_NONCE) {
1144       builder.setNonce(nonce);
1145     }
1146     ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
1147     QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
1148     for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
1149       columnBuilder.clear();
1150       columnBuilder.setFamily(ByteStringer.wrap(family.getKey()));
1151       for (Cell cell: family.getValue()) {
1152         KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
1153         valueBuilder.setQualifier(ByteStringer.wrap(
1154             kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()));
1155         valueBuilder.setValue(ByteStringer.wrap(
1156             kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
1157         valueBuilder.setTimestamp(kv.getTimestamp());
1158         if(cell.getTagsLength() > 0) {
1159           valueBuilder.setTags(ByteStringer.wrap(kv.getTagsArray(), kv.getTagsOffset(),
1160               kv.getTagsLength()));
1161         }
1162         if (type == MutationType.DELETE) {
1163           KeyValue.Type keyValueType = KeyValue.Type.codeToType(kv.getType());
1164           valueBuilder.setDeleteType(toDeleteType(keyValueType));
1165         }
1166         columnBuilder.addQualifierValue(valueBuilder.build());
1167       }
1168       builder.addColumnValue(columnBuilder.build());
1169     }
1170     return builder.build();
1171   }
1172 
1173   /**
1174    * Create a protocol buffer MutationProto based on a client Mutation. Does NOT include data.
1175    * Understanding is that the Cell will be transported other than via protobuf.
1176    * @param type
1177    * @param mutation
1178    * @param builder
1179    * @return a protobuf'd Mutation
1180    * @throws IOException
1181    */
1182   public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation,
1183       final MutationProto.Builder builder)  throws IOException {
1184     return toMutationNoData(type, mutation, builder, HConstants.NO_NONCE);
1185   }
1186 
1187   /**
1188    * Create a protocol buffer MutationProto based on a client Mutation.  Does NOT include data.
1189    * Understanding is that the Cell will be transported other than via protobuf.
1190    * @param type
1191    * @param mutation
1192    * @return a protobuf'd Mutation
1193    * @throws IOException
1194    */
1195   public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation)
1196   throws IOException {
1197     MutationProto.Builder builder =  MutationProto.newBuilder();
1198     return toMutationNoData(type, mutation, builder);
1199   }
1200 
1201   public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation,
1202       final MutationProto.Builder builder, long nonce) throws IOException {
1203     getMutationBuilderAndSetCommonFields(type, mutation, builder);
1204     builder.setAssociatedCellCount(mutation.size());
1205     if (nonce != HConstants.NO_NONCE) {
1206       builder.setNonce(nonce);
1207     }
1208     return builder.build();
1209   }
1210 
1211   /**
1212    * Code shared by {@link #toMutation(MutationType, Mutation)} and
1213    * {@link #toMutationNoData(MutationType, Mutation)}
1214    * @param type
1215    * @param mutation
1216    * @return A partly-filled out protobuf'd Mutation.
1217    */
1218   private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type,
1219       final Mutation mutation, MutationProto.Builder builder) {
1220     builder.setRow(ByteStringer.wrap(mutation.getRow()));
1221     builder.setMutateType(type);
1222     builder.setDurability(toDurability(mutation.getDurability()));
1223     builder.setTimestamp(mutation.getTimeStamp());
1224     Map<String, byte[]> attributes = mutation.getAttributesMap();
1225     if (!attributes.isEmpty()) {
1226       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
1227       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
1228         attributeBuilder.setName(attribute.getKey());
1229         attributeBuilder.setValue(ByteStringer.wrap(attribute.getValue()));
1230         builder.addAttribute(attributeBuilder.build());
1231       }
1232     }
1233     return builder;
1234   }
1235 
1236   /**
1237    * Convert a client Result to a protocol buffer Result
1238    *
1239    * @param result the client Result to convert
1240    * @return the converted protocol buffer Result
1241    */
1242   public static ClientProtos.Result toResult(final Result result) {
1243     if (result.getExists() != null) {
1244       return toResult(result.getExists(), result.isStale());
1245     }
1246 
1247     Cell[] cells = result.rawCells();
1248     if (cells == null || cells.length == 0) {
1249       return result.isStale() ? EMPTY_RESULT_PB_STALE : EMPTY_RESULT_PB;
1250     }
1251 
1252     ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
1253     for (Cell c : cells) {
1254       builder.addCell(toCell(c));
1255     }
1256 
1257     builder.setStale(result.isStale());
1258 
1259     return builder.build();
1260   }
1261 
1262   /**
1263    * Convert a client Result to a protocol buffer Result
1264    *
1265    * @param existence the client existence to send
1266    * @return the converted protocol buffer Result
1267    */
1268   public static ClientProtos.Result toResult(final boolean existence, boolean stale) {
1269     if (stale){
1270       return existence ? EMPTY_RESULT_PB_EXISTS_TRUE_STALE : EMPTY_RESULT_PB_EXISTS_FALSE_STALE;
1271     } else {
1272       return existence ? EMPTY_RESULT_PB_EXISTS_TRUE : EMPTY_RESULT_PB_EXISTS_FALSE;
1273     }
1274   }
1275 
1276   /**
1277    * Convert a client Result to a protocol buffer Result.
1278    * The pb Result does not include the Cell data.  That is for transport otherwise.
1279    *
1280    * @param result the client Result to convert
1281    * @return the converted protocol buffer Result
1282    */
1283   public static ClientProtos.Result toResultNoData(final Result result) {
1284     if (result.getExists() != null) return toResult(result.getExists(), result.isStale());
1285     int size = result.size();
1286     if (size == 0) return result.isStale() ? EMPTY_RESULT_PB_STALE : EMPTY_RESULT_PB;
1287     ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
1288     builder.setAssociatedCellCount(size);
1289     builder.setStale(result.isStale());
1290     return builder.build();
1291   }
1292 
1293   /**
1294    * Convert a protocol buffer Result to a client Result
1295    *
1296    * @param proto the protocol buffer Result to convert
1297    * @return the converted client Result
1298    */
1299   public static Result toResult(final ClientProtos.Result proto) {
1300     if (proto.hasExists()) {
1301       if (proto.getStale()) {
1302         return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE_STALE :EMPTY_RESULT_EXISTS_FALSE_STALE;
1303       }
1304       return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE : EMPTY_RESULT_EXISTS_FALSE;
1305     }
1306 
1307     List<CellProtos.Cell> values = proto.getCellList();
1308     if (values.isEmpty()){
1309       return proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT;
1310     }
1311 
1312     List<Cell> cells = new ArrayList<Cell>(values.size());
1313     for (CellProtos.Cell c : values) {
1314       cells.add(toCell(c));
1315     }
1316     return Result.create(cells, null, proto.getStale());
1317   }
1318 
1319   /**
1320    * Convert a protocol buffer Result to a client Result
1321    *
1322    * @param proto the protocol buffer Result to convert
1323    * @param scanner Optional cell scanner.
1324    * @return the converted client Result
1325    * @throws IOException
1326    */
1327   public static Result toResult(final ClientProtos.Result proto, final CellScanner scanner)
1328   throws IOException {
1329     List<CellProtos.Cell> values = proto.getCellList();
1330 
1331     if (proto.hasExists()) {
1332       if ((values != null && !values.isEmpty()) ||
1333           (proto.hasAssociatedCellCount() && proto.getAssociatedCellCount() > 0)) {
1334         throw new IllegalArgumentException("bad proto: exists with cells is no allowed " + proto);
1335       }
1336       if (proto.getStale()) {
1337         return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE_STALE :EMPTY_RESULT_EXISTS_FALSE_STALE;
1338       }
1339       return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE : EMPTY_RESULT_EXISTS_FALSE;
1340     }
1341 
1342     // TODO: Unit test that has some Cells in scanner and some in the proto.
1343     List<Cell> cells = null;
1344     if (proto.hasAssociatedCellCount()) {
1345       int count = proto.getAssociatedCellCount();
1346       cells = new ArrayList<Cell>(count + values.size());
1347       for (int i = 0; i < count; i++) {
1348         if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count);
1349         cells.add(scanner.current());
1350       }
1351     }
1352 
1353     if (!values.isEmpty()){
1354       if (cells == null) cells = new ArrayList<Cell>(values.size());
1355       for (CellProtos.Cell c: values) {
1356         cells.add(toCell(c));
1357       }
1358     }
1359 
1360     return (cells == null || cells.isEmpty())
1361         ? (proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT)
1362         : Result.create(cells, null, proto.getStale());
1363   }
1364 
1365 
1366   /**
1367    * Convert a ByteArrayComparable to a protocol buffer Comparator
1368    *
1369    * @param comparator the ByteArrayComparable to convert
1370    * @return the converted protocol buffer Comparator
1371    */
1372   public static ComparatorProtos.Comparator toComparator(ByteArrayComparable comparator) {
1373     ComparatorProtos.Comparator.Builder builder = ComparatorProtos.Comparator.newBuilder();
1374     builder.setName(comparator.getClass().getName());
1375     builder.setSerializedComparator(ByteStringer.wrap(comparator.toByteArray()));
1376     return builder.build();
1377   }
1378 
1379   /**
1380    * Convert a protocol buffer Comparator to a ByteArrayComparable
1381    *
1382    * @param proto the protocol buffer Comparator to convert
1383    * @return the converted ByteArrayComparable
1384    */
1385   @SuppressWarnings("unchecked")
1386   public static ByteArrayComparable toComparator(ComparatorProtos.Comparator proto)
1387   throws IOException {
1388     String type = proto.getName();
1389     String funcName = "parseFrom";
1390     byte [] value = proto.getSerializedComparator().toByteArray();
1391     try {
1392       Class<? extends ByteArrayComparable> c =
1393         (Class<? extends ByteArrayComparable>)Class.forName(type, true, CLASS_LOADER);
1394       Method parseFrom = c.getMethod(funcName, byte[].class);
1395       if (parseFrom == null) {
1396         throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
1397       }
1398       return (ByteArrayComparable)parseFrom.invoke(null, value);
1399     } catch (Exception e) {
1400       throw new IOException(e);
1401     }
1402   }
1403 
1404   /**
1405    * Convert a protocol buffer Filter to a client Filter
1406    *
1407    * @param proto the protocol buffer Filter to convert
1408    * @return the converted Filter
1409    */
1410   @SuppressWarnings("unchecked")
1411   public static Filter toFilter(FilterProtos.Filter proto) throws IOException {
1412     String type = proto.getName();
1413     final byte [] value = proto.getSerializedFilter().toByteArray();
1414     String funcName = "parseFrom";
1415     try {
1416       Class<? extends Filter> c =
1417         (Class<? extends Filter>)Class.forName(type, true, CLASS_LOADER);
1418       Method parseFrom = c.getMethod(funcName, byte[].class);
1419       if (parseFrom == null) {
1420         throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
1421       }
1422       return (Filter)parseFrom.invoke(c, value);
1423     } catch (Exception e) {
1424       throw new IOException(e);
1425     }
1426   }
1427 
1428   /**
1429    * Convert a client Filter to a protocol buffer Filter
1430    *
1431    * @param filter the Filter to convert
1432    * @return the converted protocol buffer Filter
1433    */
1434   public static FilterProtos.Filter toFilter(Filter filter) throws IOException {
1435     FilterProtos.Filter.Builder builder = FilterProtos.Filter.newBuilder();
1436     builder.setName(filter.getClass().getName());
1437     builder.setSerializedFilter(ByteStringer.wrap(filter.toByteArray()));
1438     return builder.build();
1439   }
1440 
1441   /**
1442    * Convert a delete KeyValue type to protocol buffer DeleteType.
1443    *
1444    * @param type
1445    * @return protocol buffer DeleteType
1446    * @throws IOException
1447    */
1448   public static DeleteType toDeleteType(
1449       KeyValue.Type type) throws IOException {
1450     switch (type) {
1451     case Delete:
1452       return DeleteType.DELETE_ONE_VERSION;
1453     case DeleteColumn:
1454       return DeleteType.DELETE_MULTIPLE_VERSIONS;
1455     case DeleteFamily:
1456       return DeleteType.DELETE_FAMILY;
1457     case DeleteFamilyVersion:
1458       return DeleteType.DELETE_FAMILY_VERSION;
1459     default:
1460         throw new IOException("Unknown delete type: " + type);
1461     }
1462   }
1463 
1464   /**
1465    * Convert a stringified protocol buffer exception Parameter to a Java Exception
1466    *
1467    * @param parameter the protocol buffer Parameter to convert
1468    * @return the converted Exception
1469    * @throws IOException if failed to deserialize the parameter
1470    */
1471   @SuppressWarnings("unchecked")
1472   public static Throwable toException(final NameBytesPair parameter) throws IOException {
1473     if (parameter == null || !parameter.hasValue()) return null;
1474     String desc = parameter.getValue().toStringUtf8();
1475     String type = parameter.getName();
1476     try {
1477       Class<? extends Throwable> c =
1478         (Class<? extends Throwable>)Class.forName(type, true, CLASS_LOADER);
1479       Constructor<? extends Throwable> cn = null;
1480       try {
1481         cn = c.getDeclaredConstructor(String.class);
1482         return cn.newInstance(desc);
1483       } catch (NoSuchMethodException e) {
1484         // Could be a raw RemoteException. See HBASE-8987.
1485         cn = c.getDeclaredConstructor(String.class, String.class);
1486         return cn.newInstance(type, desc);
1487       }
1488     } catch (Exception e) {
1489       throw new IOException(e);
1490     }
1491   }
1492 
1493 // Start helpers for Client
1494 
1495   /**
1496    * A helper to get a row of the closet one before using client protocol.
1497    *
1498    * @param client
1499    * @param regionName
1500    * @param row
1501    * @param family
1502    * @return the row or the closestRowBefore if it doesn't exist
1503    * @throws IOException
1504    * @deprecated since 0.99 - use reversed scanner instead.
1505    */
1506   @Deprecated
1507   public static Result getRowOrBefore(final ClientService.BlockingInterface client,
1508       final byte[] regionName, final byte[] row,
1509       final byte[] family) throws IOException {
1510     GetRequest request =
1511       RequestConverter.buildGetRowOrBeforeRequest(
1512         regionName, row, family);
1513     try {
1514       GetResponse response = client.get(null, request);
1515       if (!response.hasResult()) return null;
1516       return toResult(response.getResult());
1517     } catch (ServiceException se) {
1518       throw getRemoteException(se);
1519     }
1520   }
1521 
1522   /**
1523    * A helper to bulk load a list of HFiles using client protocol.
1524    *
1525    * @param client
1526    * @param familyPaths
1527    * @param regionName
1528    * @param assignSeqNum
1529    * @return true if all are loaded
1530    * @throws IOException
1531    */
1532   public static boolean bulkLoadHFile(final ClientService.BlockingInterface client,
1533       final List<Pair<byte[], String>> familyPaths,
1534       final byte[] regionName, boolean assignSeqNum) throws IOException {
1535     BulkLoadHFileRequest request =
1536       RequestConverter.buildBulkLoadHFileRequest(familyPaths, regionName, assignSeqNum);
1537     try {
1538       BulkLoadHFileResponse response =
1539         client.bulkLoadHFile(null, request);
1540       return response.getLoaded();
1541     } catch (ServiceException se) {
1542       throw getRemoteException(se);
1543     }
1544   }
1545 
1546   public static CoprocessorServiceResponse execService(final ClientService.BlockingInterface client,
1547       final CoprocessorServiceCall call, final byte[] regionName) throws IOException {
1548     CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
1549         .setCall(call).setRegion(
1550             RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build();
1551     try {
1552       CoprocessorServiceResponse response =
1553           client.execService(null, request);
1554       return response;
1555     } catch (ServiceException se) {
1556       throw getRemoteException(se);
1557     }
1558   }
1559 
1560   public static CoprocessorServiceResponse execService(
1561     final MasterService.BlockingInterface client, final CoprocessorServiceCall call)
1562   throws IOException {
1563     CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
1564         .setCall(call).setRegion(
1565             RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY)).build();
1566     try {
1567       CoprocessorServiceResponse response =
1568           client.execMasterService(null, request);
1569       return response;
1570     } catch (ServiceException se) {
1571       throw getRemoteException(se);
1572     }
1573   }
1574 
1575   @SuppressWarnings("unchecked")
1576   public static <T extends Service> T newServiceStub(Class<T> service, RpcChannel channel)
1577       throws Exception {
1578     return (T)Methods.call(service, null, "newStub",
1579         new Class[]{ RpcChannel.class }, new Object[]{ channel });
1580   }
1581 
1582 // End helpers for Client
1583 // Start helpers for Admin
1584 
1585   /**
1586    * A helper to retrieve region info given a region name
1587    * using admin protocol.
1588    *
1589    * @param admin
1590    * @param regionName
1591    * @return the retrieved region info
1592    * @throws IOException
1593    */
1594   public static HRegionInfo getRegionInfo(final AdminService.BlockingInterface admin,
1595       final byte[] regionName) throws IOException {
1596     try {
1597       GetRegionInfoRequest request =
1598         RequestConverter.buildGetRegionInfoRequest(regionName);
1599       GetRegionInfoResponse response =
1600         admin.getRegionInfo(null, request);
1601       return HRegionInfo.convert(response.getRegionInfo());
1602     } catch (ServiceException se) {
1603       throw getRemoteException(se);
1604     }
1605   }
1606 
1607   /**
1608    * A helper to close a region given a region name
1609    * using admin protocol.
1610    *
1611    * @param admin
1612    * @param regionName
1613    * @param transitionInZK
1614    * @throws IOException
1615    */
1616   public static void closeRegion(final AdminService.BlockingInterface admin,
1617       final ServerName server, final byte[] regionName, final boolean transitionInZK) throws IOException {
1618     CloseRegionRequest closeRegionRequest =
1619       RequestConverter.buildCloseRegionRequest(server, regionName, transitionInZK);
1620     try {
1621       admin.closeRegion(null, closeRegionRequest);
1622     } catch (ServiceException se) {
1623       throw getRemoteException(se);
1624     }
1625   }
1626 
1627   /**
1628    * A helper to close a region given a region name
1629    * using admin protocol.
1630    *
1631    * @param admin
1632    * @param regionName
1633    * @param versionOfClosingNode
1634    * @return true if the region is closed
1635    * @throws IOException
1636    */
1637   public static boolean closeRegion(final AdminService.BlockingInterface admin,
1638       final ServerName server,
1639       final byte[] regionName,
1640       final int versionOfClosingNode, final ServerName destinationServer,
1641       final boolean transitionInZK) throws IOException {
1642     CloseRegionRequest closeRegionRequest =
1643       RequestConverter.buildCloseRegionRequest(server,
1644         regionName, versionOfClosingNode, destinationServer, transitionInZK);
1645     try {
1646       CloseRegionResponse response = admin.closeRegion(null, closeRegionRequest);
1647       return ResponseConverter.isClosed(response);
1648     } catch (ServiceException se) {
1649       throw getRemoteException(se);
1650     }
1651   }
1652 
1653 
1654   /**
1655    * A helper to open a region using admin protocol.
1656    * @param admin
1657    * @param region
1658    * @throws IOException
1659    */
1660   public static void openRegion(final AdminService.BlockingInterface admin,
1661       ServerName server, final HRegionInfo region) throws IOException {
1662     OpenRegionRequest request =
1663       RequestConverter.buildOpenRegionRequest(server, region, -1, null, null);
1664     try {
1665       admin.openRegion(null, request);
1666     } catch (ServiceException se) {
1667       throw ProtobufUtil.getRemoteException(se);
1668     }
1669   }
1670 
1671   /**
1672    * A helper to get the all the online regions on a region
1673    * server using admin protocol.
1674    *
1675    * @param admin
1676    * @return a list of online region info
1677    * @throws IOException
1678    */
1679   public static List<HRegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
1680   throws IOException {
1681     GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
1682     GetOnlineRegionResponse response = null;
1683     try {
1684       response = admin.getOnlineRegion(null, request);
1685     } catch (ServiceException se) {
1686       throw getRemoteException(se);
1687     }
1688     return getRegionInfos(response);
1689   }
1690 
1691   /**
1692    * Get the list of region info from a GetOnlineRegionResponse
1693    *
1694    * @param proto the GetOnlineRegionResponse
1695    * @return the list of region info or null if <code>proto</code> is null
1696    */
1697   static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
1698     if (proto == null) return null;
1699     List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
1700     for (RegionInfo regionInfo: proto.getRegionInfoList()) {
1701       regionInfos.add(HRegionInfo.convert(regionInfo));
1702     }
1703     return regionInfos;
1704   }
1705 
1706   /**
1707    * A helper to get the info of a region server using admin protocol.
1708    *
1709    * @param admin
1710    * @return the server name
1711    * @throws IOException
1712    */
1713   public static ServerInfo getServerInfo(final AdminService.BlockingInterface admin)
1714   throws IOException {
1715     GetServerInfoRequest request = RequestConverter.buildGetServerInfoRequest();
1716     try {
1717       GetServerInfoResponse response = admin.getServerInfo(null, request);
1718       return response.getServerInfo();
1719     } catch (ServiceException se) {
1720       throw getRemoteException(se);
1721     }
1722   }
1723 
1724   /**
1725    * A helper to get the list of files of a column family
1726    * on a given region using admin protocol.
1727    *
1728    * @param admin
1729    * @param regionName
1730    * @param family
1731    * @return the list of store files
1732    * @throws IOException
1733    */
1734   public static List<String> getStoreFiles(final AdminService.BlockingInterface admin,
1735       final byte[] regionName, final byte[] family)
1736   throws IOException {
1737     GetStoreFileRequest request =
1738       RequestConverter.buildGetStoreFileRequest(regionName, family);
1739     try {
1740       GetStoreFileResponse response = admin.getStoreFile(null, request);
1741       return response.getStoreFileList();
1742     } catch (ServiceException se) {
1743       throw ProtobufUtil.getRemoteException(se);
1744     }
1745   }
1746 
1747   /**
1748    * A helper to split a region using admin protocol.
1749    *
1750    * @param admin
1751    * @param hri
1752    * @param splitPoint
1753    * @throws IOException
1754    */
1755   public static void split(final AdminService.BlockingInterface admin,
1756       final HRegionInfo hri, byte[] splitPoint) throws IOException {
1757     SplitRegionRequest request =
1758       RequestConverter.buildSplitRegionRequest(hri.getRegionName(), splitPoint);
1759     try {
1760       admin.splitRegion(null, request);
1761     } catch (ServiceException se) {
1762       throw ProtobufUtil.getRemoteException(se);
1763     }
1764   }
1765 
1766   /**
1767    * A helper to merge regions using admin protocol. Send request to
1768    * regionserver.
1769    * @param admin
1770    * @param region_a
1771    * @param region_b
1772    * @param forcible true if do a compulsory merge, otherwise we will only merge
1773    *          two adjacent regions
1774    * @throws IOException
1775    */
1776   public static void mergeRegions(final AdminService.BlockingInterface admin,
1777       final HRegionInfo region_a, final HRegionInfo region_b,
1778       final boolean forcible) throws IOException {
1779     MergeRegionsRequest request = RequestConverter.buildMergeRegionsRequest(
1780         region_a.getRegionName(), region_b.getRegionName(),forcible);
1781     try {
1782       admin.mergeRegions(null, request);
1783     } catch (ServiceException se) {
1784       throw ProtobufUtil.getRemoteException(se);
1785     }
1786   }
1787 
1788 // End helpers for Admin
1789 
1790   /*
1791    * Get the total (read + write) requests from a RegionLoad pb
1792    * @param rl - RegionLoad pb
1793    * @return total (read + write) requests
1794    */
1795   public static long getTotalRequestsCount(RegionLoad rl) {
1796     if (rl == null) {
1797       return 0;
1798     }
1799 
1800     return rl.getReadRequestsCount() + rl.getWriteRequestsCount();
1801   }
1802 
1803 
1804   /**
1805    * @param m Message to get delimited pb serialization of (with pb magic prefix)
1806    */
1807   public static byte [] toDelimitedByteArray(final Message m) throws IOException {
1808     // Allocate arbitrary big size so we avoid resizing.
1809     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
1810     baos.write(PB_MAGIC);
1811     m.writeDelimitedTo(baos);
1812     return baos.toByteArray();
1813   }
1814 
1815   /**
1816    * Converts a Permission proto to a client Permission object.
1817    *
1818    * @param proto the protobuf Permission
1819    * @return the converted Permission
1820    */
1821   public static Permission toPermission(AccessControlProtos.Permission proto) {
1822     if (proto.getType() != AccessControlProtos.Permission.Type.Global) {
1823       return toTablePermission(proto);
1824     } else {
1825       List<Permission.Action> actions = toPermissionActions(proto.getGlobalPermission().getActionList());
1826       return new Permission(actions.toArray(new Permission.Action[actions.size()]));
1827     }
1828   }
1829 
1830   /**
1831    * Converts a Permission proto to a client TablePermission object.
1832    *
1833    * @param proto the protobuf Permission
1834    * @return the converted TablePermission
1835    */
1836   public static TablePermission toTablePermission(AccessControlProtos.Permission proto) {
1837     if(proto.getType() == AccessControlProtos.Permission.Type.Global) {
1838       AccessControlProtos.GlobalPermission perm = proto.getGlobalPermission();
1839       List<Permission.Action> actions = toPermissionActions(perm.getActionList());
1840 
1841       return new TablePermission(null, null, null,
1842           actions.toArray(new Permission.Action[actions.size()]));
1843     }
1844     if(proto.getType() == AccessControlProtos.Permission.Type.Namespace) {
1845       AccessControlProtos.NamespacePermission perm = proto.getNamespacePermission();
1846       List<Permission.Action> actions = toPermissionActions(perm.getActionList());
1847 
1848       if(!proto.hasNamespacePermission()) {
1849         throw new IllegalStateException("Namespace must not be empty in NamespacePermission");
1850       }
1851       String namespace = perm.getNamespaceName().toStringUtf8();
1852       return new TablePermission(namespace, actions.toArray(new Permission.Action[actions.size()]));
1853     }
1854     if(proto.getType() == AccessControlProtos.Permission.Type.Table) {
1855       AccessControlProtos.TablePermission perm = proto.getTablePermission();
1856       List<Permission.Action> actions = toPermissionActions(perm.getActionList());
1857 
1858       byte[] qualifier = null;
1859       byte[] family = null;
1860       TableName table = null;
1861 
1862       if (!perm.hasTableName()) {
1863         throw new IllegalStateException("TableName cannot be empty");
1864       }
1865       table = ProtobufUtil.toTableName(perm.getTableName());
1866 
1867       if (perm.hasFamily()) family = perm.getFamily().toByteArray();
1868       if (perm.hasQualifier()) qualifier = perm.getQualifier().toByteArray();
1869 
1870       return new TablePermission(table, family, qualifier,
1871           actions.toArray(new Permission.Action[actions.size()]));
1872     }
1873     throw new IllegalStateException("Unrecognize Perm Type: "+proto.getType());
1874   }
1875 
1876   /**
1877    * Convert a client Permission to a Permission proto
1878    *
1879    * @param perm the client Permission
1880    * @return the protobuf Permission
1881    */
1882   public static AccessControlProtos.Permission toPermission(Permission perm) {
1883     AccessControlProtos.Permission.Builder ret = AccessControlProtos.Permission.newBuilder();
1884     if (perm instanceof TablePermission) {
1885       TablePermission tablePerm = (TablePermission)perm;
1886       if(tablePerm.hasNamespace()) {
1887         ret.setType(AccessControlProtos.Permission.Type.Namespace);
1888 
1889         AccessControlProtos.NamespacePermission.Builder builder =
1890             AccessControlProtos.NamespacePermission.newBuilder();
1891         builder.setNamespaceName(ByteString.copyFromUtf8(tablePerm.getNamespace()));
1892         Permission.Action actions[] = perm.getActions();
1893         if (actions != null) {
1894           for (Permission.Action a : actions) {
1895             builder.addAction(toPermissionAction(a));
1896           }
1897 	}
1898         ret.setNamespacePermission(builder);
1899         return ret.build();
1900       } else if (tablePerm.hasTable()) {
1901         ret.setType(AccessControlProtos.Permission.Type.Table);
1902 
1903         AccessControlProtos.TablePermission.Builder builder =
1904             AccessControlProtos.TablePermission.newBuilder();
1905         builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTableName()));
1906         if (tablePerm.hasFamily()) {
1907           builder.setFamily(ByteStringer.wrap(tablePerm.getFamily()));
1908         }
1909         if (tablePerm.hasQualifier()) {
1910           builder.setQualifier(ByteStringer.wrap(tablePerm.getQualifier()));
1911         }
1912         Permission.Action actions[] = perm.getActions();
1913         if (actions != null) {
1914           for (Permission.Action a : actions) {
1915             builder.addAction(toPermissionAction(a));
1916           }
1917         }
1918         ret.setTablePermission(builder);
1919         return ret.build();
1920       }
1921     }
1922 
1923     ret.setType(AccessControlProtos.Permission.Type.Global);
1924 
1925     AccessControlProtos.GlobalPermission.Builder builder =
1926         AccessControlProtos.GlobalPermission.newBuilder();
1927     Permission.Action actions[] = perm.getActions();
1928     if (actions != null) {
1929       for (Permission.Action a: actions) {
1930         builder.addAction(toPermissionAction(a));
1931       }
1932     }
1933     ret.setGlobalPermission(builder);
1934     return ret.build();
1935   }
1936 
1937   /**
1938    * Converts a list of Permission.Action proto to a list of client Permission.Action objects.
1939    *
1940    * @param protoActions the list of protobuf Actions
1941    * @return the converted list of Actions
1942    */
1943   public static List<Permission.Action> toPermissionActions(
1944       List<AccessControlProtos.Permission.Action> protoActions) {
1945     List<Permission.Action> actions = new ArrayList<Permission.Action>(protoActions.size());
1946     for (AccessControlProtos.Permission.Action a : protoActions) {
1947       actions.add(toPermissionAction(a));
1948     }
1949     return actions;
1950   }
1951 
1952   /**
1953    * Converts a Permission.Action proto to a client Permission.Action object.
1954    *
1955    * @param action the protobuf Action
1956    * @return the converted Action
1957    */
1958   public static Permission.Action toPermissionAction(
1959       AccessControlProtos.Permission.Action action) {
1960     switch (action) {
1961       case READ:
1962         return Permission.Action.READ;
1963       case WRITE:
1964         return Permission.Action.WRITE;
1965       case EXEC:
1966         return Permission.Action.EXEC;
1967       case CREATE:
1968         return Permission.Action.CREATE;
1969       case ADMIN:
1970         return Permission.Action.ADMIN;
1971     }
1972     throw new IllegalArgumentException("Unknown action value "+action.name());
1973   }
1974 
1975   /**
1976    * Convert a client Permission.Action to a Permission.Action proto
1977    *
1978    * @param action the client Action
1979    * @return the protobuf Action
1980    */
1981   public static AccessControlProtos.Permission.Action toPermissionAction(
1982       Permission.Action action) {
1983     switch (action) {
1984       case READ:
1985         return AccessControlProtos.Permission.Action.READ;
1986       case WRITE:
1987         return AccessControlProtos.Permission.Action.WRITE;
1988       case EXEC:
1989         return AccessControlProtos.Permission.Action.EXEC;
1990       case CREATE:
1991         return AccessControlProtos.Permission.Action.CREATE;
1992       case ADMIN:
1993         return AccessControlProtos.Permission.Action.ADMIN;
1994     }
1995     throw new IllegalArgumentException("Unknown action value "+action.name());
1996   }
1997 
1998   /**
1999    * Convert a client user permission to a user permission proto
2000    *
2001    * @param perm the client UserPermission
2002    * @return the protobuf UserPermission
2003    */
2004   public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) {
2005     return AccessControlProtos.UserPermission.newBuilder()
2006         .setUser(ByteStringer.wrap(perm.getUser()))
2007         .setPermission(toPermission(perm))
2008         .build();
2009   }
2010 
2011   /**
2012    * Converts a user permission proto to a client user permission object.
2013    *
2014    * @param proto the protobuf UserPermission
2015    * @return the converted UserPermission
2016    */
2017   public static UserPermission toUserPermission(AccessControlProtos.UserPermission proto) {
2018     return new UserPermission(proto.getUser().toByteArray(),
2019         toTablePermission(proto.getPermission()));
2020   }
2021 
2022   /**
2023    * Convert a ListMultimap<String, TablePermission> where key is username
2024    * to a protobuf UserPermission
2025    *
2026    * @param perm the list of user and table permissions
2027    * @return the protobuf UserTablePermissions
2028    */
2029   public static AccessControlProtos.UsersAndPermissions toUserTablePermissions(
2030       ListMultimap<String, TablePermission> perm) {
2031     AccessControlProtos.UsersAndPermissions.Builder builder =
2032                   AccessControlProtos.UsersAndPermissions.newBuilder();
2033     for (Map.Entry<String, Collection<TablePermission>> entry : perm.asMap().entrySet()) {
2034       AccessControlProtos.UsersAndPermissions.UserPermissions.Builder userPermBuilder =
2035                   AccessControlProtos.UsersAndPermissions.UserPermissions.newBuilder();
2036       userPermBuilder.setUser(ByteString.copyFromUtf8(entry.getKey()));
2037       for (TablePermission tablePerm: entry.getValue()) {
2038         userPermBuilder.addPermissions(toPermission(tablePerm));
2039       }
2040       builder.addUserPermissions(userPermBuilder.build());
2041     }
2042     return builder.build();
2043   }
2044 
2045   /**
2046    * A utility used to grant a user global permissions.
2047    * <p>
2048    * It's also called by the shell, in case you want to find references.
2049    *
2050    * @param protocol the AccessControlService protocol proxy
2051    * @param userShortName the short name of the user to grant permissions
2052    * @param actions the permissions to be granted
2053    * @throws ServiceException
2054    */
2055   public static void grant(AccessControlService.BlockingInterface protocol,
2056       String userShortName, Permission.Action... actions) throws ServiceException {
2057     List<AccessControlProtos.Permission.Action> permActions =
2058         Lists.newArrayListWithCapacity(actions.length);
2059     for (Permission.Action a : actions) {
2060       permActions.add(ProtobufUtil.toPermissionAction(a));
2061     }
2062     AccessControlProtos.GrantRequest request = RequestConverter.
2063       buildGrantRequest(userShortName, permActions.toArray(
2064         new AccessControlProtos.Permission.Action[actions.length]));
2065     protocol.grant(null, request);
2066   }
2067 
2068   /**
2069    * A utility used to grant a user table permissions. The permissions will
2070    * be for a table table/column family/qualifier.
2071    * <p>
2072    * It's also called by the shell, in case you want to find references.
2073    *
2074    * @param protocol the AccessControlService protocol proxy
2075    * @param userShortName the short name of the user to grant permissions
2076    * @param tableName optional table name
2077    * @param f optional column family
2078    * @param q optional qualifier
2079    * @param actions the permissions to be granted
2080    * @throws ServiceException
2081    */
2082   public static void grant(AccessControlService.BlockingInterface protocol,
2083       String userShortName, TableName tableName, byte[] f, byte[] q,
2084       Permission.Action... actions) throws ServiceException {
2085     List<AccessControlProtos.Permission.Action> permActions =
2086         Lists.newArrayListWithCapacity(actions.length);
2087     for (Permission.Action a : actions) {
2088       permActions.add(ProtobufUtil.toPermissionAction(a));
2089     }
2090     AccessControlProtos.GrantRequest request = RequestConverter.
2091       buildGrantRequest(userShortName, tableName, f, q, permActions.toArray(
2092         new AccessControlProtos.Permission.Action[actions.length]));
2093     protocol.grant(null, request);
2094   }
2095 
2096   /**
2097    * A utility used to grant a user namespace permissions.
2098    * <p>
2099    * It's also called by the shell, in case you want to find references.
2100    *
2101    * @param protocol the AccessControlService protocol proxy
2102    * @param namespace the short name of the user to grant permissions
2103    * @param actions the permissions to be granted
2104    * @throws ServiceException
2105    */
2106   public static void grant(AccessControlService.BlockingInterface protocol,
2107       String userShortName, String namespace,
2108       Permission.Action... actions) throws ServiceException {
2109     List<AccessControlProtos.Permission.Action> permActions =
2110         Lists.newArrayListWithCapacity(actions.length);
2111     for (Permission.Action a : actions) {
2112       permActions.add(ProtobufUtil.toPermissionAction(a));
2113     }
2114     AccessControlProtos.GrantRequest request = RequestConverter.
2115       buildGrantRequest(userShortName, namespace, permActions.toArray(
2116         new AccessControlProtos.Permission.Action[actions.length]));
2117     protocol.grant(null, request);
2118   }
2119 
2120   /**
2121    * A utility used to revoke a user's global permissions.
2122    * <p>
2123    * It's also called by the shell, in case you want to find references.
2124    *
2125    * @param protocol the AccessControlService protocol proxy
2126    * @param userShortName the short name of the user to revoke permissions
2127    * @param actions the permissions to be revoked
2128    * @throws ServiceException
2129    */
2130   public static void revoke(AccessControlService.BlockingInterface protocol,
2131       String userShortName, Permission.Action... actions) throws ServiceException {
2132     List<AccessControlProtos.Permission.Action> permActions =
2133         Lists.newArrayListWithCapacity(actions.length);
2134     for (Permission.Action a : actions) {
2135       permActions.add(ProtobufUtil.toPermissionAction(a));
2136     }
2137     AccessControlProtos.RevokeRequest request = RequestConverter.
2138       buildRevokeRequest(userShortName, permActions.toArray(
2139         new AccessControlProtos.Permission.Action[actions.length]));
2140     protocol.revoke(null, request);
2141   }
2142 
2143   /**
2144    * A utility used to revoke a user's table permissions. The permissions will
2145    * be for a table/column family/qualifier.
2146    * <p>
2147    * It's also called by the shell, in case you want to find references.
2148    *
2149    * @param protocol the AccessControlService protocol proxy
2150    * @param userShortName the short name of the user to revoke permissions
2151    * @param tableName optional table name
2152    * @param f optional column family
2153    * @param q optional qualifier
2154    * @param actions the permissions to be revoked
2155    * @throws ServiceException
2156    */
2157   public static void revoke(AccessControlService.BlockingInterface protocol,
2158       String userShortName, TableName tableName, byte[] f, byte[] q,
2159       Permission.Action... actions) throws ServiceException {
2160     List<AccessControlProtos.Permission.Action> permActions =
2161         Lists.newArrayListWithCapacity(actions.length);
2162     for (Permission.Action a : actions) {
2163       permActions.add(ProtobufUtil.toPermissionAction(a));
2164     }
2165     AccessControlProtos.RevokeRequest request = RequestConverter.
2166       buildRevokeRequest(userShortName, tableName, f, q, permActions.toArray(
2167         new AccessControlProtos.Permission.Action[actions.length]));
2168     protocol.revoke(null, request);
2169   }
2170 
2171   /**
2172    * A utility used to revoke a user's namespace permissions.
2173    * <p>
2174    * It's also called by the shell, in case you want to find references.
2175    *
2176    * @param protocol the AccessControlService protocol proxy
2177    * @param userShortName the short name of the user to revoke permissions
2178    * @param namespace optional table name
2179    * @param actions the permissions to be revoked
2180    * @throws ServiceException
2181    */
2182   public static void revoke(AccessControlService.BlockingInterface protocol,
2183       String userShortName, String namespace,
2184       Permission.Action... actions) throws ServiceException {
2185     List<AccessControlProtos.Permission.Action> permActions =
2186         Lists.newArrayListWithCapacity(actions.length);
2187     for (Permission.Action a : actions) {
2188       permActions.add(ProtobufUtil.toPermissionAction(a));
2189     }
2190     AccessControlProtos.RevokeRequest request = RequestConverter.
2191       buildRevokeRequest(userShortName, namespace, permActions.toArray(
2192         new AccessControlProtos.Permission.Action[actions.length]));
2193     protocol.revoke(null, request);
2194   }
2195 
2196   /**
2197    * A utility used to get user's global permissions.
2198    * <p>
2199    * It's also called by the shell, in case you want to find references.
2200    *
2201    * @param protocol the AccessControlService protocol proxy
2202    * @throws ServiceException
2203    */
2204   public static List<UserPermission> getUserPermissions(
2205       AccessControlService.BlockingInterface protocol) throws ServiceException {
2206     AccessControlProtos.GetUserPermissionsRequest.Builder builder =
2207       AccessControlProtos.GetUserPermissionsRequest.newBuilder();
2208     builder.setType(AccessControlProtos.Permission.Type.Global);
2209     AccessControlProtos.GetUserPermissionsRequest request = builder.build();
2210     AccessControlProtos.GetUserPermissionsResponse response =
2211       protocol.getUserPermissions(null, request);
2212     List<UserPermission> perms = new ArrayList<UserPermission>();
2213     for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
2214       perms.add(ProtobufUtil.toUserPermission(perm));
2215     }
2216     return perms;
2217   }
2218 
2219   /**
2220    * A utility used to get user table permissions.
2221    * <p>
2222    * It's also called by the shell, in case you want to find references.
2223    *
2224    * @param protocol the AccessControlService protocol proxy
2225    * @param t optional table name
2226    * @throws ServiceException
2227    */
2228   public static List<UserPermission> getUserPermissions(
2229       AccessControlService.BlockingInterface protocol,
2230       TableName t) throws ServiceException {
2231     AccessControlProtos.GetUserPermissionsRequest.Builder builder =
2232       AccessControlProtos.GetUserPermissionsRequest.newBuilder();
2233     if (t != null) {
2234       builder.setTableName(ProtobufUtil.toProtoTableName(t));
2235     }
2236     builder.setType(AccessControlProtos.Permission.Type.Table);
2237     AccessControlProtos.GetUserPermissionsRequest request = builder.build();
2238     AccessControlProtos.GetUserPermissionsResponse response =
2239       protocol.getUserPermissions(null, request);
2240     List<UserPermission> perms = new ArrayList<UserPermission>();
2241     for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
2242       perms.add(ProtobufUtil.toUserPermission(perm));
2243     }
2244     return perms;
2245   }
2246 
2247   /**
2248    * A utility used to get permissions for selected namespace.
2249    * <p>
2250    * It's also called by the shell, in case you want to find references.
2251    *
2252    * @param protocol the AccessControlService protocol proxy
2253    * @param namespace name of the namespace
2254    * @throws ServiceException
2255    */
2256   public static List<UserPermission> getUserPermissions(
2257       AccessControlService.BlockingInterface protocol,
2258       byte[] namespace) throws ServiceException {
2259     AccessControlProtos.GetUserPermissionsRequest.Builder builder =
2260       AccessControlProtos.GetUserPermissionsRequest.newBuilder();
2261     if (namespace != null) {
2262       builder.setNamespaceName(ByteStringer.wrap(namespace));
2263     }
2264     builder.setType(AccessControlProtos.Permission.Type.Namespace);
2265     AccessControlProtos.GetUserPermissionsRequest request = builder.build();
2266     AccessControlProtos.GetUserPermissionsResponse response =
2267       protocol.getUserPermissions(null, request);
2268     List<UserPermission> perms = new ArrayList<UserPermission>();
2269     for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
2270       perms.add(ProtobufUtil.toUserPermission(perm));
2271     }
2272     return perms;
2273   }
2274 
2275   /**
2276    * Convert a protobuf UserTablePermissions to a
2277    * ListMultimap<String, TablePermission> where key is username.
2278    *
2279    * @param proto the protobuf UserPermission
2280    * @return the converted UserPermission
2281    */
2282   public static ListMultimap<String, TablePermission> toUserTablePermissions(
2283       AccessControlProtos.UsersAndPermissions proto) {
2284     ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
2285     AccessControlProtos.UsersAndPermissions.UserPermissions userPerm;
2286 
2287     for (int i = 0; i < proto.getUserPermissionsCount(); i++) {
2288       userPerm = proto.getUserPermissions(i);
2289       for (int j = 0; j < userPerm.getPermissionsCount(); j++) {
2290         TablePermission tablePerm = toTablePermission(userPerm.getPermissions(j));
2291         perms.put(userPerm.getUser().toStringUtf8(), tablePerm);
2292       }
2293     }
2294 
2295     return perms;
2296   }
2297 
2298   /**
2299    * Converts a Token instance (with embedded identifier) to the protobuf representation.
2300    *
2301    * @param token the Token instance to copy
2302    * @return the protobuf Token message
2303    */
2304   public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
2305     AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder();
2306     builder.setIdentifier(ByteStringer.wrap(token.getIdentifier()));
2307     builder.setPassword(ByteStringer.wrap(token.getPassword()));
2308     if (token.getService() != null) {
2309       builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
2310     }
2311     return builder.build();
2312   }
2313 
2314   /**
2315    * Converts a protobuf Token message back into a Token instance.
2316    *
2317    * @param proto the protobuf Token message
2318    * @return the Token instance
2319    */
2320   public static Token<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) {
2321     return new Token<AuthenticationTokenIdentifier>(
2322         proto.hasIdentifier() ? proto.getIdentifier().toByteArray() : null,
2323         proto.hasPassword() ? proto.getPassword().toByteArray() : null,
2324         AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE,
2325         proto.hasService() ? new Text(proto.getService().toStringUtf8()) : null);
2326   }
2327 
2328   /**
2329    * Find the HRegion encoded name based on a region specifier
2330    *
2331    * @param regionSpecifier the region specifier
2332    * @return the corresponding region's encoded name
2333    * @throws DoNotRetryIOException if the specifier type is unsupported
2334    */
2335   public static String getRegionEncodedName(
2336       final RegionSpecifier regionSpecifier) throws DoNotRetryIOException {
2337     byte[] value = regionSpecifier.getValue().toByteArray();
2338     RegionSpecifierType type = regionSpecifier.getType();
2339     switch (type) {
2340       case REGION_NAME:
2341         return HRegionInfo.encodeRegionName(value);
2342       case ENCODED_REGION_NAME:
2343         return Bytes.toString(value);
2344       default:
2345         throw new DoNotRetryIOException(
2346           "Unsupported region specifier type: " + type);
2347     }
2348   }
2349 
2350   public static ScanMetrics toScanMetrics(final byte[] bytes) {
2351     Parser<MapReduceProtos.ScanMetrics> parser = MapReduceProtos.ScanMetrics.PARSER;
2352     MapReduceProtos.ScanMetrics pScanMetrics = null;
2353     try {
2354       pScanMetrics = parser.parseFrom(bytes);
2355     } catch (InvalidProtocolBufferException e) {
2356       //Ignored there are just no key values to add.
2357     }
2358     ScanMetrics scanMetrics = new ScanMetrics();
2359     if (pScanMetrics != null) {
2360       for (HBaseProtos.NameInt64Pair pair : pScanMetrics.getMetricsList()) {
2361         if (pair.hasName() && pair.hasValue()) {
2362           scanMetrics.setCounter(pair.getName(), pair.getValue());
2363         }
2364       }
2365     }
2366     return scanMetrics;
2367   }
2368 
2369   public static MapReduceProtos.ScanMetrics toScanMetrics(ScanMetrics scanMetrics) {
2370     MapReduceProtos.ScanMetrics.Builder builder = MapReduceProtos.ScanMetrics.newBuilder();
2371     Map<String, Long> metrics = scanMetrics.getMetricsMap();
2372     for (Entry<String, Long> e : metrics.entrySet()) {
2373       HBaseProtos.NameInt64Pair nameInt64Pair =
2374           HBaseProtos.NameInt64Pair.newBuilder()
2375               .setName(e.getKey())
2376               .setValue(e.getValue())
2377               .build();
2378       builder.addMetrics(nameInt64Pair);
2379     }
2380     return builder.build();
2381   }
2382 
2383   /**
2384    * Unwraps an exception from a protobuf service into the underlying (expected) IOException.
2385    * This method will <strong>always</strong> throw an exception.
2386    * @param se the {@code ServiceException} instance to convert into an {@code IOException}
2387    */
2388   public static void toIOException(ServiceException se) throws IOException {
2389     if (se == null) {
2390       throw new NullPointerException("Null service exception passed!");
2391     }
2392 
2393     Throwable cause = se.getCause();
2394     if (cause != null && cause instanceof IOException) {
2395       throw (IOException)cause;
2396     }
2397     throw new IOException(se);
2398   }
2399 
2400   public static CellProtos.Cell toCell(final Cell kv) {
2401     // Doing this is going to kill us if we do it for all data passed.
2402     // St.Ack 20121205
2403     CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder();
2404     kvbuilder.setRow(ByteStringer.wrap(kv.getRowArray(), kv.getRowOffset(),
2405         kv.getRowLength()));
2406     kvbuilder.setFamily(ByteStringer.wrap(kv.getFamilyArray(),
2407         kv.getFamilyOffset(), kv.getFamilyLength()));
2408     kvbuilder.setQualifier(ByteStringer.wrap(kv.getQualifierArray(),
2409         kv.getQualifierOffset(), kv.getQualifierLength()));
2410     kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte()));
2411     kvbuilder.setTimestamp(kv.getTimestamp());
2412     kvbuilder.setValue(ByteStringer.wrap(kv.getValueArray(), kv.getValueOffset(),
2413         kv.getValueLength()));
2414     return kvbuilder.build();
2415   }
2416 
2417   public static Cell toCell(final CellProtos.Cell cell) {
2418     // Doing this is going to kill us if we do it for all data passed.
2419     // St.Ack 20121205
2420     return CellUtil.createCell(cell.getRow().toByteArray(),
2421       cell.getFamily().toByteArray(),
2422       cell.getQualifier().toByteArray(),
2423       cell.getTimestamp(),
2424       (byte)cell.getCellType().getNumber(),
2425       cell.getValue().toByteArray());
2426   }
2427 
2428   public static HBaseProtos.NamespaceDescriptor toProtoNamespaceDescriptor(NamespaceDescriptor ns) {
2429     HBaseProtos.NamespaceDescriptor.Builder b =
2430         HBaseProtos.NamespaceDescriptor.newBuilder()
2431             .setName(ByteString.copyFromUtf8(ns.getName()));
2432     for(Map.Entry<String, String> entry: ns.getConfiguration().entrySet()) {
2433       b.addConfiguration(HBaseProtos.NameStringPair.newBuilder()
2434           .setName(entry.getKey())
2435           .setValue(entry.getValue()));
2436     }
2437     return b.build();
2438   }
2439 
2440   public static NamespaceDescriptor toNamespaceDescriptor(
2441       HBaseProtos.NamespaceDescriptor desc) throws IOException {
2442     NamespaceDescriptor.Builder b =
2443       NamespaceDescriptor.create(desc.getName().toStringUtf8());
2444     for(HBaseProtos.NameStringPair prop : desc.getConfigurationList()) {
2445       b.addConfiguration(prop.getName(), prop.getValue());
2446     }
2447     return b.build();
2448   }
2449 
2450   /**
2451    * Get an instance of the argument type declared in a class's signature. The
2452    * argument type is assumed to be a PB Message subclass, and the instance is
2453    * created using parseFrom method on the passed ByteString.
2454    * @param runtimeClass the runtime type of the class
2455    * @param position the position of the argument in the class declaration
2456    * @param b the ByteString which should be parsed to get the instance created
2457    * @return the instance
2458    * @throws IOException
2459    */
2460   @SuppressWarnings("unchecked")
2461   public static <T extends Message>
2462   T getParsedGenericInstance(Class<?> runtimeClass, int position, ByteString b)
2463       throws IOException {
2464     Type type = runtimeClass.getGenericSuperclass();
2465     Type argType = ((ParameterizedType)type).getActualTypeArguments()[position];
2466     Class<T> classType = (Class<T>)argType;
2467     T inst;
2468     try {
2469       Method m = classType.getMethod("parseFrom", ByteString.class);
2470       inst = (T)m.invoke(null, b);
2471       return inst;
2472     } catch (SecurityException e) {
2473       throw new IOException(e);
2474     } catch (NoSuchMethodException e) {
2475       throw new IOException(e);
2476     } catch (IllegalArgumentException e) {
2477       throw new IOException(e);
2478     } catch (InvocationTargetException e) {
2479       throw new IOException(e);
2480     } catch (IllegalAccessException e) {
2481       throw new IOException(e);
2482     }
2483   }
2484 
2485   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
2486       List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
2487     // compaction descriptor contains relative paths.
2488     // input / output paths are relative to the store dir
2489     // store dir is relative to region dir
2490     CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
2491         .setTableName(ByteStringer.wrap(info.getTableName()))
2492         .setEncodedRegionName(ByteStringer.wrap(info.getEncodedNameAsBytes()))
2493         .setFamilyName(ByteStringer.wrap(family))
2494         .setStoreHomeDir(storeDir.getName()); //make relative
2495     for (Path inputPath : inputPaths) {
2496       builder.addCompactionInput(inputPath.getName()); //relative path
2497     }
2498     for (Path outputPath : outputPaths) {
2499       builder.addCompactionOutput(outputPath.getName());
2500     }
2501     builder.setRegionName(ByteStringer.wrap(info.getRegionName()));
2502     return builder.build();
2503   }
2504 
2505   public static FlushDescriptor toFlushDescriptor(FlushAction action, HRegionInfo hri,
2506       long flushSeqId, Map<byte[], List<Path>> committedFiles) {
2507     FlushDescriptor.Builder desc = FlushDescriptor.newBuilder()
2508         .setAction(action)
2509         .setEncodedRegionName(ByteStringer.wrap(hri.getEncodedNameAsBytes()))
2510         .setFlushSequenceNumber(flushSeqId)
2511         .setTableName(ByteStringer.wrap(hri.getTable().getName()));
2512 
2513     for (Map.Entry<byte[], List<Path>> entry : committedFiles.entrySet()) {
2514       WALProtos.FlushDescriptor.StoreFlushDescriptor.Builder builder =
2515           WALProtos.FlushDescriptor.StoreFlushDescriptor.newBuilder()
2516           .setFamilyName(ByteStringer.wrap(entry.getKey()))
2517           .setStoreHomeDir(Bytes.toString(entry.getKey())); //relative to region
2518       if (entry.getValue() != null) {
2519         for (Path path : entry.getValue()) {
2520           builder.addFlushOutput(path.getName());
2521         }
2522       }
2523       desc.addStoreFlushes(builder);
2524     }
2525     return desc.build();
2526   }
2527 
2528   /**
2529    * Return short version of Message toString'd, shorter than TextFormat#shortDebugString.
2530    * Tries to NOT print out data both because it can be big but also so we do not have data in our
2531    * logs. Use judiciously.
2532    * @param m
2533    * @return toString of passed <code>m</code>
2534    */
2535   public static String getShortTextFormat(Message m) {
2536     if (m == null) return "null";
2537     if (m instanceof ScanRequest) {
2538       // This should be small and safe to output.  No data.
2539       return TextFormat.shortDebugString(m);
2540     } else if (m instanceof RegionServerReportRequest) {
2541       // Print a short message only, just the servername and the requests, not the full load.
2542       RegionServerReportRequest r = (RegionServerReportRequest)m;
2543       return "server " + TextFormat.shortDebugString(r.getServer()) +
2544         " load { numberOfRequests: " + r.getLoad().getNumberOfRequests() + " }";
2545     } else if (m instanceof RegionServerStartupRequest) {
2546       // Should be small enough.
2547       return TextFormat.shortDebugString(m);
2548     } else if (m instanceof MutationProto) {
2549       return toShortString((MutationProto)m);
2550     } else if (m instanceof GetRequest) {
2551       GetRequest r = (GetRequest) m;
2552       return "region= " + getStringForByteString(r.getRegion().getValue()) +
2553           ", row=" + getStringForByteString(r.getGet().getRow());
2554     } else if (m instanceof ClientProtos.MultiRequest) {
2555       ClientProtos.MultiRequest r = (ClientProtos.MultiRequest) m;
2556       // Get first set of Actions.
2557       ClientProtos.RegionAction actions = r.getRegionActionList().get(0);
2558       String row = actions.getActionCount() <= 0? "":
2559         getStringForByteString(actions.getAction(0).hasGet()?
2560           actions.getAction(0).getGet().getRow():
2561           actions.getAction(0).getMutation().getRow());
2562       return "region= " + getStringForByteString(actions.getRegion().getValue()) +
2563           ", for " + r.getRegionActionCount() +
2564           " actions and 1st row key=" + row;
2565     } else if (m instanceof ClientProtos.MutateRequest) {
2566       ClientProtos.MutateRequest r = (ClientProtos.MutateRequest) m;
2567       return "region= " + getStringForByteString(r.getRegion().getValue()) +
2568           ", row=" + getStringForByteString(r.getMutation().getRow());
2569     }
2570     return "TODO: " + m.getClass().toString();
2571   }
2572 
2573   private static String getStringForByteString(ByteString bs) {
2574     return Bytes.toStringBinary(bs.toByteArray());
2575   }
2576 
2577   /**
2578    * Print out some subset of a MutationProto rather than all of it and its data
2579    * @param proto Protobuf to print out
2580    * @return Short String of mutation proto
2581    */
2582   static String toShortString(final MutationProto proto) {
2583     return "row=" + Bytes.toString(proto.getRow().toByteArray()) +
2584         ", type=" + proto.getMutateType().toString();
2585   }
2586 
2587   public static TableName toTableName(HBaseProtos.TableName tableNamePB) {
2588     return TableName.valueOf(tableNamePB.getNamespace().asReadOnlyByteBuffer(),
2589         tableNamePB.getQualifier().asReadOnlyByteBuffer());
2590   }
2591 
2592   public static HBaseProtos.TableName toProtoTableName(TableName tableName) {
2593     return HBaseProtos.TableName.newBuilder()
2594         .setNamespace(ByteStringer.wrap(tableName.getNamespace()))
2595         .setQualifier(ByteStringer.wrap(tableName.getQualifier())).build();
2596   }
2597 
2598   public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) {
2599     if (tableNamesList == null) {
2600       return new TableName[0];
2601     }
2602     TableName[] tableNames = new TableName[tableNamesList.size()];
2603     for (int i = 0; i < tableNamesList.size(); i++) {
2604       tableNames[i] = toTableName(tableNamesList.get(i));
2605     }
2606     return tableNames;
2607   }
2608 
2609   /**
2610    * Convert a protocol buffer CellVisibility to a client CellVisibility
2611    *
2612    * @param proto
2613    * @return the converted client CellVisibility
2614    */
2615   public static CellVisibility toCellVisibility(ClientProtos.CellVisibility proto) {
2616     if (proto == null) return null;
2617     return new CellVisibility(proto.getExpression());
2618   }
2619 
2620   /**
2621    * Convert a protocol buffer CellVisibility bytes to a client CellVisibility
2622    *
2623    * @param protoBytes
2624    * @return the converted client CellVisibility
2625    * @throws DeserializationException
2626    */
2627   public static CellVisibility toCellVisibility(byte[] protoBytes) throws DeserializationException {
2628     if (protoBytes == null) return null;
2629     ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder();
2630     ClientProtos.CellVisibility proto = null;
2631     try {
2632       proto = builder.mergeFrom(protoBytes).build();
2633     } catch (InvalidProtocolBufferException e) {
2634       throw new DeserializationException(e);
2635     }
2636     return toCellVisibility(proto);
2637   }
2638 
2639   /**
2640    * Create a protocol buffer CellVisibility based on a client CellVisibility.
2641    *
2642    * @param cellVisibility
2643    * @return a protocol buffer CellVisibility
2644    */
2645   public static ClientProtos.CellVisibility toCellVisibility(CellVisibility cellVisibility) {
2646     ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder();
2647     builder.setExpression(cellVisibility.getExpression());
2648     return builder.build();
2649   }
2650 
2651   /**
2652    * Convert a protocol buffer Authorizations to a client Authorizations
2653    *
2654    * @param proto
2655    * @return the converted client Authorizations
2656    */
2657   public static Authorizations toAuthorizations(ClientProtos.Authorizations proto) {
2658     if (proto == null) return null;
2659     return new Authorizations(proto.getLabelList());
2660   }
2661 
2662   /**
2663    * Convert a protocol buffer Authorizations bytes to a client Authorizations
2664    *
2665    * @param protoBytes
2666    * @return the converted client Authorizations
2667    * @throws DeserializationException
2668    */
2669   public static Authorizations toAuthorizations(byte[] protoBytes) throws DeserializationException {
2670     if (protoBytes == null) return null;
2671     ClientProtos.Authorizations.Builder builder = ClientProtos.Authorizations.newBuilder();
2672     ClientProtos.Authorizations proto = null;
2673     try {
2674       proto = builder.mergeFrom(protoBytes).build();
2675     } catch (InvalidProtocolBufferException e) {
2676       throw new DeserializationException(e);
2677     }
2678     return toAuthorizations(proto);
2679   }
2680 
2681   /**
2682    * Create a protocol buffer Authorizations based on a client Authorizations.
2683    *
2684    * @param authorizations
2685    * @return a protocol buffer Authorizations
2686    */
2687   public static ClientProtos.Authorizations toAuthorizations(Authorizations authorizations) {
2688     ClientProtos.Authorizations.Builder builder = ClientProtos.Authorizations.newBuilder();
2689     for (String label : authorizations.getLabels()) {
2690       builder.addLabel(label);
2691     }
2692     return builder.build();
2693   }
2694 
2695   public static AccessControlProtos.UsersAndPermissions toUsersAndPermissions(String user,
2696       Permission perms) {
2697     return AccessControlProtos.UsersAndPermissions.newBuilder()
2698       .addUserPermissions(AccessControlProtos.UsersAndPermissions.UserPermissions.newBuilder()
2699         .setUser(ByteString.copyFromUtf8(user))
2700         .addPermissions(toPermission(perms))
2701         .build())
2702       .build();
2703   }
2704 
2705   public static AccessControlProtos.UsersAndPermissions toUsersAndPermissions(
2706       ListMultimap<String, Permission> perms) {
2707     AccessControlProtos.UsersAndPermissions.Builder builder =
2708         AccessControlProtos.UsersAndPermissions.newBuilder();
2709     for (Map.Entry<String, Collection<Permission>> entry : perms.asMap().entrySet()) {
2710       AccessControlProtos.UsersAndPermissions.UserPermissions.Builder userPermBuilder =
2711         AccessControlProtos.UsersAndPermissions.UserPermissions.newBuilder();
2712       userPermBuilder.setUser(ByteString.copyFromUtf8(entry.getKey()));
2713       for (Permission perm: entry.getValue()) {
2714         userPermBuilder.addPermissions(toPermission(perm));
2715       }
2716       builder.addUserPermissions(userPermBuilder.build());
2717     }
2718     return builder.build();
2719   }
2720 
2721   public static ListMultimap<String, Permission> toUsersAndPermissions(
2722       AccessControlProtos.UsersAndPermissions proto) {
2723     ListMultimap<String, Permission> result = ArrayListMultimap.create();
2724     for (AccessControlProtos.UsersAndPermissions.UserPermissions userPerms:
2725         proto.getUserPermissionsList()) {
2726       String user = userPerms.getUser().toStringUtf8();
2727       for (AccessControlProtos.Permission perm: userPerms.getPermissionsList()) {
2728         result.put(user, toPermission(perm));
2729       }
2730     }
2731     return result;
2732   }
2733 }