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