View Javadoc

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