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