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