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