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