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