1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.protobuf;
19  
20  
21  import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
22  
23  import java.io.ByteArrayOutputStream;
24  import java.io.IOException;
25  import java.lang.reflect.Constructor;
26  import java.lang.reflect.InvocationTargetException;
27  import java.lang.reflect.Method;
28  import java.lang.reflect.ParameterizedType;
29  import java.lang.reflect.Type;
30  import java.util.ArrayList;
31  import java.util.Collection;
32  import java.util.HashMap;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.Map.Entry;
36  import java.util.NavigableSet;
37  
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.Cell;
41  import org.apache.hadoop.hbase.CellScanner;
42  import org.apache.hadoop.hbase.CellUtil;
43  import org.apache.hadoop.hbase.HBaseConfiguration;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HRegionInfo;
46  import org.apache.hadoop.hbase.HTableDescriptor;
47  import org.apache.hadoop.hbase.KeyValue;
48  import org.apache.hadoop.hbase.KeyValueUtil;
49  import org.apache.hadoop.hbase.ServerName;
50  import org.apache.hadoop.hbase.client.Append;
51  import org.apache.hadoop.hbase.client.Delete;
52  import org.apache.hadoop.hbase.client.Durability;
53  import org.apache.hadoop.hbase.client.Get;
54  import org.apache.hadoop.hbase.client.Increment;
55  import org.apache.hadoop.hbase.client.Mutation;
56  import org.apache.hadoop.hbase.client.Put;
57  import org.apache.hadoop.hbase.client.Result;
58  import org.apache.hadoop.hbase.client.Scan;
59  import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
60  import org.apache.hadoop.hbase.exceptions.DeserializationException;
61  import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
62  import org.apache.hadoop.hbase.filter.ByteArrayComparable;
63  import org.apache.hadoop.hbase.filter.Filter;
64  import org.apache.hadoop.hbase.io.TimeRange;
65  import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
66  import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
67  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
68  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
69  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
70  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
71  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
72  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
73  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
74  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest;
75  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse;
76  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRequest;
77  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileResponse;
78  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest;
79  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
80  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
81  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
82  import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
83  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
84  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
85  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
86  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
87  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
88  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
89  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
90  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
91  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
92  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
93  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
94  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
95  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
96  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
97  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
98  import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
99  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
100 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
101 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
102 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad;
103 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
104 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
105 import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos;
106 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest;
107 import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService;
108 import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsResponse;
109 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
110 import org.apache.hadoop.hbase.security.access.Permission;
111 import org.apache.hadoop.hbase.security.access.TablePermission;
112 import org.apache.hadoop.hbase.security.access.UserPermission;
113 import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
114 import org.apache.hadoop.hbase.util.Bytes;
115 import org.apache.hadoop.hbase.util.DynamicClassLoader;
116 import org.apache.hadoop.hbase.util.Methods;
117 import org.apache.hadoop.hbase.util.Pair;
118 import org.apache.hadoop.io.Text;
119 import org.apache.hadoop.ipc.RemoteException;
120 import org.apache.hadoop.security.token.Token;
121 
122 import com.google.common.collect.ArrayListMultimap;
123 import com.google.common.collect.ListMultimap;
124 import com.google.common.collect.Lists;
125 import com.google.protobuf.ByteString;
126 import com.google.protobuf.InvalidProtocolBufferException;
127 import com.google.protobuf.Message;
128 import com.google.protobuf.RpcChannel;
129 import com.google.protobuf.Service;
130 import com.google.protobuf.ServiceException;
131 import com.google.protobuf.TextFormat;
132 
133 /**
134  * Protobufs utility.
135  */
136 public final class ProtobufUtil {
137 
138   private ProtobufUtil() {
139   }
140 
141   /**
142    * Primitive type to class mapping.
143    */
144   private final static Map<String, Class<?>>
145     PRIMITIVES = new HashMap<String, Class<?>>();
146 
147   /**
148    * Dynamic class loader to load filter/comparators
149    */
150   private final static ClassLoader CLASS_LOADER;
151 
152   static {
153     ClassLoader parent = ProtobufUtil.class.getClassLoader();
154     Configuration conf = HBaseConfiguration.create();
155     CLASS_LOADER = new DynamicClassLoader(conf, parent);
156 
157     PRIMITIVES.put(Boolean.TYPE.getName(), Boolean.TYPE);
158     PRIMITIVES.put(Byte.TYPE.getName(), Byte.TYPE);
159     PRIMITIVES.put(Character.TYPE.getName(), Character.TYPE);
160     PRIMITIVES.put(Short.TYPE.getName(), Short.TYPE);
161     PRIMITIVES.put(Integer.TYPE.getName(), Integer.TYPE);
162     PRIMITIVES.put(Long.TYPE.getName(), Long.TYPE);
163     PRIMITIVES.put(Float.TYPE.getName(), Float.TYPE);
164     PRIMITIVES.put(Double.TYPE.getName(), Double.TYPE);
165     PRIMITIVES.put(Void.TYPE.getName(), Void.TYPE);
166   }
167 
168   /**
169    * Magic we put ahead of a serialized protobuf message.
170    * For example, all znode content is protobuf messages with the below magic
171    * for preamble.
172    */
173   public static final byte [] PB_MAGIC = new byte [] {'P', 'B', 'U', 'F'};
174   private static final String PB_MAGIC_STR = Bytes.toString(PB_MAGIC);
175 
176   /**
177    * Prepend the passed bytes with four bytes of magic, {@link #PB_MAGIC}, to flag what
178    * follows as a protobuf in hbase.  Prepend these bytes to all content written to znodes, etc.
179    * @param bytes Bytes to decorate
180    * @return The passed <code>bytes</codes> with magic prepended (Creates a new
181    * byte array that is <code>bytes.length</code> plus {@link #PB_MAGIC}.length.
182    */
183   public static byte [] prependPBMagic(final byte [] bytes) {
184     return Bytes.add(PB_MAGIC, bytes);
185   }
186 
187   /**
188    * @param bytes Bytes to check.
189    * @return True if passed <code>bytes</code> has {@link #PB_MAGIC} for a prefix.
190    */
191   public static boolean isPBMagicPrefix(final byte [] bytes) {
192     if (bytes == null || bytes.length < PB_MAGIC.length) return false;
193     return Bytes.compareTo(PB_MAGIC, 0, PB_MAGIC.length, bytes, 0, PB_MAGIC.length) == 0;
194   }
195 
196   /**
197    * @param bytes
198    * @throws DeserializationException if we are missing the pb magic prefix
199    */
200   public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
201     if (!isPBMagicPrefix(bytes)) {
202       throw new DeserializationException("Missing pb magic " + PB_MAGIC_STR + " prefix");
203     }
204   }
205 
206   /**
207    * @return Length of {@link #PB_MAGIC}
208    */
209   public static int lengthOfPBMagic() {
210     return PB_MAGIC.length;
211   }
212 
213   /**
214    * Return the IOException thrown by the remote server wrapped in
215    * ServiceException as cause.
216    *
217    * @param se ServiceException that wraps IO exception thrown by the server
218    * @return Exception wrapped in ServiceException or
219    *   a new IOException that wraps the unexpected ServiceException.
220    */
221   public static IOException getRemoteException(ServiceException se) {
222     Throwable e = se.getCause();
223     if (e == null) {
224       return new IOException(se);
225     }
226     if (e instanceof RemoteException) {
227       e = ((RemoteException)e).unwrapRemoteException();
228     }
229     return e instanceof IOException ? (IOException) e : new IOException(se);
230   }
231 
232   /**
233    * Convert a ServerName to a protocol buffer ServerName
234    *
235    * @param serverName the ServerName to convert
236    * @return the converted protocol buffer ServerName
237    * @see #toServerName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName)
238    */
239   public static HBaseProtos.ServerName
240       toServerName(final ServerName serverName) {
241     if (serverName == null) return null;
242     HBaseProtos.ServerName.Builder builder =
243       HBaseProtos.ServerName.newBuilder();
244     builder.setHostName(serverName.getHostname());
245     if (serverName.getPort() >= 0) {
246       builder.setPort(serverName.getPort());
247     }
248     if (serverName.getStartcode() >= 0) {
249       builder.setStartCode(serverName.getStartcode());
250     }
251     return builder.build();
252   }
253 
254   /**
255    * Convert a protocol buffer ServerName to a ServerName
256    *
257    * @param proto the protocol buffer ServerName to convert
258    * @return the converted ServerName
259    */
260   public static ServerName toServerName(final HBaseProtos.ServerName proto) {
261     if (proto == null) return null;
262     String hostName = proto.getHostName();
263     long startCode = -1;
264     int port = -1;
265     if (proto.hasPort()) {
266       port = proto.getPort();
267     }
268     if (proto.hasStartCode()) {
269       startCode = proto.getStartCode();
270     }
271     return new ServerName(hostName, port, startCode);
272   }
273 
274   /**
275    * Get HTableDescriptor[] from GetTableDescriptorsResponse protobuf
276    *
277    * @param proto the GetTableDescriptorsResponse
278    * @return HTableDescriptor[]
279    */
280   public static HTableDescriptor[] getHTableDescriptorArray(GetTableDescriptorsResponse proto) {
281     if (proto == null) return null;
282 
283     HTableDescriptor[] ret = new HTableDescriptor[proto.getTableSchemaCount()];
284     for (int i = 0; i < proto.getTableSchemaCount(); ++i) {
285       ret[i] = HTableDescriptor.convert(proto.getTableSchema(i));
286     }
287     return ret;
288   }
289 
290   /**
291    * get the split keys in form "byte [][]" from a CreateTableRequest proto
292    *
293    * @param proto the CreateTableRequest
294    * @return the split keys
295    */
296   public static byte [][] getSplitKeysArray(final CreateTableRequest proto) {
297     byte [][] splitKeys = new byte[proto.getSplitKeysCount()][];
298     for (int i = 0; i < proto.getSplitKeysCount(); ++i) {
299       splitKeys[i] = proto.getSplitKeys(i).toByteArray();
300     }
301     return splitKeys;
302   }
303 
304   /**
305    * Convert a protobuf Durability into a client Durability
306    */
307   public static Durability toDurability(
308       final ClientProtos.MutationProto.Durability proto) {
309     switch(proto) {
310     case USE_DEFAULT:
311       return Durability.USE_DEFAULT;
312     case SKIP_WAL:
313       return Durability.SKIP_WAL;
314     case ASYNC_WAL:
315       return Durability.ASYNC_WAL;
316     case SYNC_WAL:
317       return Durability.SYNC_WAL;
318     case FSYNC_WAL:
319       return Durability.FSYNC_WAL;
320     default:
321       return Durability.USE_DEFAULT;
322     }
323   }
324 
325   /**
326    * Convert a client Durability into a protbuf Durability
327    */
328   public static ClientProtos.MutationProto.Durability toDurability(
329       final Durability d) {
330     switch(d) {
331     case USE_DEFAULT:
332       return ClientProtos.MutationProto.Durability.USE_DEFAULT;
333     case SKIP_WAL:
334       return ClientProtos.MutationProto.Durability.SKIP_WAL;
335     case ASYNC_WAL:
336       return ClientProtos.MutationProto.Durability.ASYNC_WAL;
337     case SYNC_WAL:
338       return ClientProtos.MutationProto.Durability.SYNC_WAL;
339     case FSYNC_WAL:
340       return ClientProtos.MutationProto.Durability.FSYNC_WAL;
341     default:
342       return ClientProtos.MutationProto.Durability.USE_DEFAULT;
343     }
344   }
345 
346   /**
347    * Convert a protocol buffer Get to a client Get
348    *
349    * @param proto the protocol buffer Get to convert
350    * @return the converted client Get
351    * @throws IOException
352    */
353   public static Get toGet(
354       final ClientProtos.Get proto) throws IOException {
355     if (proto == null) return null;
356     byte[] row = proto.getRow().toByteArray();
357     Get get = new Get(row);
358     if (proto.hasCacheBlocks()) {
359       get.setCacheBlocks(proto.getCacheBlocks());
360     }
361     if (proto.hasMaxVersions()) {
362       get.setMaxVersions(proto.getMaxVersions());
363     }
364     if (proto.hasStoreLimit()) {
365       get.setMaxResultsPerColumnFamily(proto.getStoreLimit());
366     }
367     if (proto.hasStoreOffset()) {
368       get.setRowOffsetPerColumnFamily(proto.getStoreOffset());
369     }
370     if (proto.hasTimeRange()) {
371       HBaseProtos.TimeRange timeRange = proto.getTimeRange();
372       long minStamp = 0;
373       long maxStamp = Long.MAX_VALUE;
374       if (timeRange.hasFrom()) {
375         minStamp = timeRange.getFrom();
376       }
377       if (timeRange.hasTo()) {
378         maxStamp = timeRange.getTo();
379       }
380       get.setTimeRange(minStamp, maxStamp);
381     }
382     if (proto.hasFilter()) {
383       HBaseProtos.Filter filter = proto.getFilter();
384       get.setFilter(ProtobufUtil.toFilter(filter));
385     }
386     for (NameBytesPair attribute: proto.getAttributeList()) {
387       get.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
388     }
389     if (proto.getColumnCount() > 0) {
390       for (Column column: proto.getColumnList()) {
391         byte[] family = column.getFamily().toByteArray();
392         if (column.getQualifierCount() > 0) {
393           for (ByteString qualifier: column.getQualifierList()) {
394             get.addColumn(family, qualifier.toByteArray());
395           }
396         } else {
397           get.addFamily(family);
398         }
399       }
400     }
401     return get;
402   }
403 
404   /**
405    * Convert a protocol buffer Mutate to a Put.
406    *
407    * @param proto The protocol buffer MutationProto to convert
408    * @return A client Put.
409    * @throws IOException
410    */
411   public static Put toPut(final MutationProto proto)
412   throws IOException {
413     return toPut(proto, null);
414   }
415 
416   /**
417    * Convert a protocol buffer Mutate to a Put.
418    *
419    * @param proto The protocol buffer MutationProto to convert
420    * @param cellScanner If non-null, the Cell data that goes with this proto.
421    * @return A client Put.
422    * @throws IOException
423    */
424   public static Put toPut(final MutationProto proto, final CellScanner cellScanner)
425   throws IOException {
426     // TODO: Server-side at least why do we convert back to the Client types?  Why not just pb it?
427     MutationType type = proto.getMutateType();
428     assert type == MutationType.PUT: type.name();
429     byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
430     long timestamp = proto.hasTimestamp()? proto.getTimestamp(): HConstants.LATEST_TIMESTAMP;
431     Put put = null;
432     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
433     if (cellCount > 0) {
434       // The proto has metadata only and the data is separate to be found in the cellScanner.
435       if (cellScanner == null) {
436         throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
437           TextFormat.shortDebugString(proto));
438       }
439       for (int i = 0; i < cellCount; i++) {
440         if (!cellScanner.advance()) {
441           throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
442             " no cell returned: " + TextFormat.shortDebugString(proto));
443         }
444         Cell cell = cellScanner.current();
445         if (put == null) {
446           put = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
447         }
448         put.add(KeyValueUtil.ensureKeyValue(cell));
449       }
450     } else {
451       put = new Put(row, timestamp);
452       // The proto has the metadata and the data itself
453       for (ColumnValue column: proto.getColumnValueList()) {
454         byte[] family = column.getFamily().toByteArray();
455         for (QualifierValue qv: column.getQualifierValueList()) {
456           byte[] qualifier = qv.getQualifier().toByteArray();
457           if (!qv.hasValue()) {
458             throw new DoNotRetryIOException(
459                 "Missing required field: qualifer value");
460           }
461           byte[] value = qv.getValue().toByteArray();
462           long ts = timestamp;
463           if (qv.hasTimestamp()) {
464             ts = qv.getTimestamp();
465           }
466           put.add(family, qualifier, ts, value);
467         }
468       }
469     }
470     put.setDurability(toDurability(proto.getDurability()));
471     for (NameBytesPair attribute: proto.getAttributeList()) {
472       put.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
473     }
474     return put;
475   }
476 
477   /**
478    * Convert a protocol buffer Mutate to a Delete
479    *
480    * @param proto the protocol buffer Mutate to convert
481    * @return the converted client Delete
482    * @throws IOException
483    */
484   public static Delete toDelete(final MutationProto proto)
485   throws IOException {
486     return toDelete(proto, null);
487   }
488 
489   /**
490    * Convert a protocol buffer Mutate to a Delete
491    *
492    * @param proto the protocol buffer Mutate to convert
493    * @param cellScanner if non-null, the data that goes with this delete.
494    * @return the converted client Delete
495    * @throws IOException
496    */
497   public static Delete toDelete(final MutationProto proto, final CellScanner cellScanner)
498   throws IOException {
499     MutationType type = proto.getMutateType();
500     assert type == MutationType.DELETE : type.name();
501     byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
502     long timestamp = HConstants.LATEST_TIMESTAMP;
503     if (proto.hasTimestamp()) {
504       timestamp = proto.getTimestamp();
505     }
506     Delete delete = null;
507     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
508     if (cellCount > 0) {
509       // The proto has metadata only and the data is separate to be found in the cellScanner.
510       if (cellScanner == null) {
511         throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
512           TextFormat.shortDebugString(proto));
513       }
514       for (int i = 0; i < cellCount; i++) {
515         if (!cellScanner.advance()) {
516           throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
517             " no cell returned: " + TextFormat.shortDebugString(proto));
518         }
519         Cell cell = cellScanner.current();
520         if (delete == null) {
521           delete =
522             new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
523         }
524         delete.addDeleteMarker(KeyValueUtil.ensureKeyValue(cell));
525       }
526     } else {
527       delete = new Delete(row, timestamp);
528       for (ColumnValue column: proto.getColumnValueList()) {
529         byte[] family = column.getFamily().toByteArray();
530         for (QualifierValue qv: column.getQualifierValueList()) {
531           DeleteType deleteType = qv.getDeleteType();
532           byte[] qualifier = null;
533           if (qv.hasQualifier()) {
534             qualifier = qv.getQualifier().toByteArray();
535           }
536           long ts = HConstants.LATEST_TIMESTAMP;
537           if (qv.hasTimestamp()) {
538             ts = qv.getTimestamp();
539           }
540           if (deleteType == DeleteType.DELETE_ONE_VERSION) {
541             delete.deleteColumn(family, qualifier, ts);
542           } else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) {
543             delete.deleteColumns(family, qualifier, ts);
544           } else {
545             delete.deleteFamily(family, ts);
546           }
547         }
548       }
549     }
550     delete.setDurability(toDurability(proto.getDurability()));
551     for (NameBytesPair attribute: proto.getAttributeList()) {
552       delete.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
553     }
554     return delete;
555   }
556 
557   /**
558    * Convert a protocol buffer Mutate to an Append
559    * @param cellScanner
560    * @param proto the protocol buffer Mutate to convert
561    * @return the converted client Append
562    * @throws IOException 
563    */
564   public static Append toAppend(final MutationProto proto, final CellScanner cellScanner)
565   throws IOException {
566     MutationType type = proto.getMutateType();
567     assert type == MutationType.APPEND : type.name();
568     byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
569     Append append = null;
570     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
571     if (cellCount > 0) {
572       // The proto has metadata only and the data is separate to be found in the cellScanner.
573       if (cellScanner == null) {
574         throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
575           TextFormat.shortDebugString(proto));
576       }
577       for (int i = 0; i < cellCount; i++) {
578         if (!cellScanner.advance()) {
579           throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
580             " no cell returned: " + TextFormat.shortDebugString(proto));
581         }
582         Cell cell = cellScanner.current();
583         if (append == null) {
584           append = new Append(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
585         }
586         append.add(KeyValueUtil.ensureKeyValue(cell));
587       }
588     } else {
589       append = new Append(row);
590       for (ColumnValue column: proto.getColumnValueList()) {
591         byte[] family = column.getFamily().toByteArray();
592         for (QualifierValue qv: column.getQualifierValueList()) {
593           byte[] qualifier = qv.getQualifier().toByteArray();
594           if (!qv.hasValue()) {
595             throw new DoNotRetryIOException(
596               "Missing required field: qualifer value");
597           }
598           byte[] value = qv.getValue().toByteArray();
599           append.add(family, qualifier, value);
600         }
601       }
602     }
603     append.setDurability(toDurability(proto.getDurability()));
604     for (NameBytesPair attribute: proto.getAttributeList()) {
605       append.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
606     }
607     return append;
608   }
609 
610   /**
611    * Convert a MutateRequest to Mutation
612    *
613    * @param proto the protocol buffer Mutate to convert
614    * @return the converted Mutation
615    * @throws IOException
616    */
617   public static Mutation toMutation(final MutationProto proto) throws IOException {
618     MutationType type = proto.getMutateType();
619     if (type == MutationType.APPEND) {
620       return toAppend(proto, null);
621     }
622     if (type == MutationType.DELETE) {
623       return toDelete(proto, null);
624     }
625     if (type == MutationType.PUT) {
626       return toPut(proto, null);
627     }
628     throw new IOException("Unknown mutation type " + type);
629   }
630 
631   /**
632    * Convert a protocol buffer Mutate to an Increment
633    *
634    * @param proto the protocol buffer Mutate to convert
635    * @return the converted client Increment
636    * @throws IOException
637    */
638   public static Increment toIncrement(final MutationProto proto, final CellScanner cellScanner)
639   throws IOException {
640     MutationType type = proto.getMutateType();
641     assert type == MutationType.INCREMENT : type.name();
642     byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
643     Increment increment = null;
644     int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
645     if (cellCount > 0) {
646       // The proto has metadata only and the data is separate to be found in the cellScanner.
647       if (cellScanner == null) {
648         throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
649           TextFormat.shortDebugString(proto));
650       }
651       for (int i = 0; i < cellCount; i++) {
652         if (!cellScanner.advance()) {
653           throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
654             " no cell returned: " + TextFormat.shortDebugString(proto));
655         }
656         Cell cell = cellScanner.current();
657         if (increment == null) {
658           increment = new Increment(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
659         }
660         increment.add(KeyValueUtil.ensureKeyValue(cell));
661       }
662     } else {
663       increment = new Increment(row);
664       for (ColumnValue column: proto.getColumnValueList()) {
665         byte[] family = column.getFamily().toByteArray();
666         for (QualifierValue qv: column.getQualifierValueList()) {
667           byte[] qualifier = qv.getQualifier().toByteArray();
668           if (!qv.hasValue()) {
669             throw new DoNotRetryIOException("Missing required field: qualifer value");
670           }
671           long value = Bytes.toLong(qv.getValue().toByteArray());
672           increment.addColumn(family, qualifier, value);
673         }
674       }
675     }
676     if (proto.hasTimeRange()) {
677       HBaseProtos.TimeRange timeRange = proto.getTimeRange();
678       long minStamp = 0;
679       long maxStamp = Long.MAX_VALUE;
680       if (timeRange.hasFrom()) {
681         minStamp = timeRange.getFrom();
682       }
683       if (timeRange.hasTo()) {
684         maxStamp = timeRange.getTo();
685       }
686       increment.setTimeRange(minStamp, maxStamp);
687     }
688     increment.setDurability(toDurability(proto.getDurability()));
689     return increment;
690   }
691 
692   /**
693    * Convert a client Scan to a protocol buffer Scan
694    *
695    * @param scan the client Scan to convert
696    * @return the converted protocol buffer Scan
697    * @throws IOException
698    */
699   public static ClientProtos.Scan toScan(
700       final Scan scan) throws IOException {
701     ClientProtos.Scan.Builder scanBuilder =
702       ClientProtos.Scan.newBuilder();
703     scanBuilder.setCacheBlocks(scan.getCacheBlocks());
704     if (scan.getBatch() > 0) {
705       scanBuilder.setBatchSize(scan.getBatch());
706     }
707     if (scan.getMaxResultSize() > 0) {
708       scanBuilder.setMaxResultSize(scan.getMaxResultSize());
709     }
710     Boolean loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue();
711     if (loadColumnFamiliesOnDemand != null) {
712       scanBuilder.setLoadColumnFamiliesOnDemand(loadColumnFamiliesOnDemand.booleanValue());
713     }
714     scanBuilder.setMaxVersions(scan.getMaxVersions());
715     TimeRange timeRange = scan.getTimeRange();
716     if (!timeRange.isAllTime()) {
717       HBaseProtos.TimeRange.Builder timeRangeBuilder =
718         HBaseProtos.TimeRange.newBuilder();
719       timeRangeBuilder.setFrom(timeRange.getMin());
720       timeRangeBuilder.setTo(timeRange.getMax());
721       scanBuilder.setTimeRange(timeRangeBuilder.build());
722     }
723     Map<String, byte[]> attributes = scan.getAttributesMap();
724     if (!attributes.isEmpty()) {
725       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
726       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
727         attributeBuilder.setName(attribute.getKey());
728         attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
729         scanBuilder.addAttribute(attributeBuilder.build());
730       }
731     }
732     byte[] startRow = scan.getStartRow();
733     if (startRow != null && startRow.length > 0) {
734       scanBuilder.setStartRow(ByteString.copyFrom(startRow));
735     }
736     byte[] stopRow = scan.getStopRow();
737     if (stopRow != null && stopRow.length > 0) {
738       scanBuilder.setStopRow(ByteString.copyFrom(stopRow));
739     }
740     if (scan.hasFilter()) {
741       scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
742     }
743     if (scan.hasFamilies()) {
744       Column.Builder columnBuilder = Column.newBuilder();
745       for (Map.Entry<byte[],NavigableSet<byte []>>
746           family: scan.getFamilyMap().entrySet()) {
747         columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
748         NavigableSet<byte []> qualifiers = family.getValue();
749         columnBuilder.clearQualifier();
750         if (qualifiers != null && qualifiers.size() > 0) {
751           for (byte [] qualifier: qualifiers) {
752             columnBuilder.addQualifier(ByteString.copyFrom(qualifier));
753           }
754         }
755         scanBuilder.addColumn(columnBuilder.build());
756       }
757     }
758     if (scan.getMaxResultsPerColumnFamily() >= 0) {
759       scanBuilder.setStoreLimit(scan.getMaxResultsPerColumnFamily());
760     }
761     if (scan.getRowOffsetPerColumnFamily() > 0) {
762       scanBuilder.setStoreOffset(scan.getRowOffsetPerColumnFamily());
763     }
764     return scanBuilder.build();
765   }
766 
767   /**
768    * Convert a protocol buffer Scan to a client Scan
769    *
770    * @param proto the protocol buffer Scan to convert
771    * @return the converted client Scan
772    * @throws IOException
773    */
774   public static Scan toScan(
775       final ClientProtos.Scan proto) throws IOException {
776     byte [] startRow = HConstants.EMPTY_START_ROW;
777     byte [] stopRow  = HConstants.EMPTY_END_ROW;
778     if (proto.hasStartRow()) {
779       startRow = proto.getStartRow().toByteArray();
780     }
781     if (proto.hasStopRow()) {
782       stopRow = proto.getStopRow().toByteArray();
783     }
784     Scan scan = new Scan(startRow, stopRow);
785     if (proto.hasCacheBlocks()) {
786       scan.setCacheBlocks(proto.getCacheBlocks());
787     }
788     if (proto.hasMaxVersions()) {
789       scan.setMaxVersions(proto.getMaxVersions());
790     }
791     if (proto.hasStoreLimit()) {
792       scan.setMaxResultsPerColumnFamily(proto.getStoreLimit());
793     }
794     if (proto.hasStoreOffset()) {
795       scan.setRowOffsetPerColumnFamily(proto.getStoreOffset());
796     }
797     if (proto.hasLoadColumnFamiliesOnDemand()) {
798       scan.setLoadColumnFamiliesOnDemand(proto.getLoadColumnFamiliesOnDemand());
799     }
800     if (proto.hasTimeRange()) {
801       HBaseProtos.TimeRange timeRange = proto.getTimeRange();
802       long minStamp = 0;
803       long maxStamp = Long.MAX_VALUE;
804       if (timeRange.hasFrom()) {
805         minStamp = timeRange.getFrom();
806       }
807       if (timeRange.hasTo()) {
808         maxStamp = timeRange.getTo();
809       }
810       scan.setTimeRange(minStamp, maxStamp);
811     }
812     if (proto.hasFilter()) {
813       HBaseProtos.Filter filter = proto.getFilter();
814       scan.setFilter(ProtobufUtil.toFilter(filter));
815     }
816     if (proto.hasBatchSize()) {
817       scan.setBatch(proto.getBatchSize());
818     }
819     if (proto.hasMaxResultSize()) {
820       scan.setMaxResultSize(proto.getMaxResultSize());
821     }
822     for (NameBytesPair attribute: proto.getAttributeList()) {
823       scan.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
824     }
825     if (proto.getColumnCount() > 0) {
826       for (Column column: proto.getColumnList()) {
827         byte[] family = column.getFamily().toByteArray();
828         if (column.getQualifierCount() > 0) {
829           for (ByteString qualifier: column.getQualifierList()) {
830             scan.addColumn(family, qualifier.toByteArray());
831           }
832         } else {
833           scan.addFamily(family);
834         }
835       }
836     }
837     return scan;
838   }
839 
840   /**
841    * Create a protocol buffer Get based on a client Get.
842    *
843    * @param get the client Get
844    * @return a protocol buffer Get
845    * @throws IOException
846    */
847   public static ClientProtos.Get toGet(
848       final Get get) throws IOException {
849     ClientProtos.Get.Builder builder =
850       ClientProtos.Get.newBuilder();
851     builder.setRow(ByteString.copyFrom(get.getRow()));
852     builder.setCacheBlocks(get.getCacheBlocks());
853     builder.setMaxVersions(get.getMaxVersions());
854     if (get.getFilter() != null) {
855       builder.setFilter(ProtobufUtil.toFilter(get.getFilter()));
856     }
857     TimeRange timeRange = get.getTimeRange();
858     if (!timeRange.isAllTime()) {
859       HBaseProtos.TimeRange.Builder timeRangeBuilder =
860         HBaseProtos.TimeRange.newBuilder();
861       timeRangeBuilder.setFrom(timeRange.getMin());
862       timeRangeBuilder.setTo(timeRange.getMax());
863       builder.setTimeRange(timeRangeBuilder.build());
864     }
865     Map<String, byte[]> attributes = get.getAttributesMap();
866     if (!attributes.isEmpty()) {
867       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
868       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
869         attributeBuilder.setName(attribute.getKey());
870         attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
871         builder.addAttribute(attributeBuilder.build());
872       }
873     }
874     if (get.hasFamilies()) {
875       Column.Builder columnBuilder = Column.newBuilder();
876       Map<byte[], NavigableSet<byte[]>> families = get.getFamilyMap();
877       for (Map.Entry<byte[], NavigableSet<byte[]>> family: families.entrySet()) {
878         NavigableSet<byte[]> qualifiers = family.getValue();
879         columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
880         columnBuilder.clearQualifier();
881         if (qualifiers != null && qualifiers.size() > 0) {
882           for (byte[] qualifier: qualifiers) {
883             columnBuilder.addQualifier(ByteString.copyFrom(qualifier));
884           }
885         }
886         builder.addColumn(columnBuilder.build());
887       }
888     }
889     if (get.getMaxResultsPerColumnFamily() >= 0) {
890       builder.setStoreLimit(get.getMaxResultsPerColumnFamily());
891     }
892     if (get.getRowOffsetPerColumnFamily() > 0) {
893       builder.setStoreOffset(get.getRowOffsetPerColumnFamily());
894     }
895     return builder.build();
896   }
897 
898   /**
899    * Convert a client Increment to a protobuf Mutate.
900    *
901    * @param increment
902    * @return the converted mutate
903    */
904   public static MutationProto toMutation(final Increment increment) {
905     MutationProto.Builder builder = MutationProto.newBuilder();
906     builder.setRow(ByteString.copyFrom(increment.getRow()));
907     builder.setMutateType(MutationType.INCREMENT);
908     builder.setDurability(toDurability(increment.getDurability()));
909     TimeRange timeRange = increment.getTimeRange();
910     if (!timeRange.isAllTime()) {
911       HBaseProtos.TimeRange.Builder timeRangeBuilder =
912         HBaseProtos.TimeRange.newBuilder();
913       timeRangeBuilder.setFrom(timeRange.getMin());
914       timeRangeBuilder.setTo(timeRange.getMax());
915       builder.setTimeRange(timeRangeBuilder.build());
916     }
917     ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
918     QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
919    for (Map.Entry<byte[], List<? extends Cell>> family: increment.getFamilyMap().entrySet()) {
920       columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
921       columnBuilder.clearQualifierValue();
922       List<? extends Cell> values = family.getValue();
923       if (values != null && values.size() > 0) {
924         for (Cell cell: values) {
925           KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
926           valueBuilder.setQualifier(ByteString.copyFrom(kv.getQualifier()));
927           valueBuilder.setValue(ByteString.copyFrom(kv.getValue()));
928           columnBuilder.addQualifierValue(valueBuilder.build());
929         }
930       }
931       builder.addColumnValue(columnBuilder.build());
932     }
933     return builder.build();
934   }
935 
936   /**
937    * Create a protocol buffer Mutate based on a client Mutation
938    *
939    * @param type
940    * @param mutation
941    * @return a protobuf'd Mutation
942    * @throws IOException
943    */
944   public static MutationProto toMutation(final MutationType type, final Mutation mutation)
945   throws IOException {
946     MutationProto.Builder builder = getMutationBuilderAndSetCommonFields(type, mutation);
947     ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
948     QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
949     for (Map.Entry<byte[],List<? extends Cell>> family: mutation.getFamilyMap().entrySet()) {
950       columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
951       columnBuilder.clearQualifierValue();
952       for (Cell cell: family.getValue()) {
953         KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
954         valueBuilder.setQualifier(ByteString.copyFrom(kv.getQualifier()));
955         valueBuilder.setValue(ByteString.copyFrom(kv.getValue()));
956         valueBuilder.setTimestamp(kv.getTimestamp());
957         if (type == MutationType.DELETE) {
958           KeyValue.Type keyValueType = KeyValue.Type.codeToType(kv.getType());
959           valueBuilder.setDeleteType(toDeleteType(keyValueType));
960         }
961         columnBuilder.addQualifierValue(valueBuilder.build());
962       }
963       builder.addColumnValue(columnBuilder.build());
964     }
965     return builder.build();
966   }
967 
968   /**
969    * Create a protocol buffer MutationProto based on a client Mutation.  Does NOT include data.
970    * Understanding is that the Cell will be transported other than via protobuf.
971    * @param type
972    * @param mutation
973    * @return a protobuf'd Mutation
974    * @throws IOException
975    */
976   public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation)
977   throws IOException {
978     MutationProto.Builder builder = getMutationBuilderAndSetCommonFields(type, mutation);
979     builder.setAssociatedCellCount(mutation.size());
980     return builder.build();
981   }
982 
983   /**
984    * Code shared by {@link #toMutation(MutationType, Mutation)} and
985    * {@link #toMutationNoData(MutationType, Mutation)}
986    * @param type
987    * @param mutation
988    * @return A partly-filled out protobuf'd Mutation.
989    */
990   private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type,
991       final Mutation mutation) {
992     MutationProto.Builder builder = MutationProto.newBuilder();
993     builder.setRow(ByteString.copyFrom(mutation.getRow()));
994     builder.setMutateType(type);
995     builder.setDurability(toDurability(mutation.getDurability()));
996     builder.setTimestamp(mutation.getTimeStamp());
997     Map<String, byte[]> attributes = mutation.getAttributesMap();
998     if (!attributes.isEmpty()) {
999       NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
1000       for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
1001         attributeBuilder.setName(attribute.getKey());
1002         attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
1003         builder.addAttribute(attributeBuilder.build());
1004       }
1005     }
1006     return builder;
1007   }
1008 
1009   /**
1010    * Convert a client Result to a protocol buffer Result
1011    *
1012    * @param result the client Result to convert
1013    * @return the converted protocol buffer Result
1014    */
1015   public static ClientProtos.Result toResult(final Result result) {
1016     ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
1017     Cell [] cells = result.raw();
1018     if (cells != null) {
1019       for (Cell c : cells) {
1020         builder.addCell(toCell(c));
1021       }
1022     }
1023     return builder.build();
1024   }
1025 
1026   /**
1027    * Convert a client Result to a protocol buffer Result.
1028    * The pb Result does not include the Cell data.  That is for transport otherwise.
1029    *
1030    * @param result the client Result to convert
1031    * @return the converted protocol buffer Result
1032    */
1033   public static ClientProtos.Result toResultNoData(final Result result) {
1034     ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
1035     builder.setAssociatedCellCount(result.size());
1036     return builder.build();
1037   }
1038 
1039   /**
1040    * Convert a protocol buffer Result to a client Result
1041    *
1042    * @param proto the protocol buffer Result to convert
1043    * @return the converted client Result
1044    */
1045   public static Result toResult(final ClientProtos.Result proto) {
1046     List<HBaseProtos.Cell> values = proto.getCellList();
1047     List<Cell> cells = new ArrayList<Cell>(values.size());
1048     for (HBaseProtos.Cell c: values) {
1049       cells.add(toCell(c));
1050     }
1051     return new Result(cells);
1052   }
1053 
1054   /**
1055    * Convert a protocol buffer Result to a client Result
1056    *
1057    * @param proto the protocol buffer Result to convert
1058    * @param scanner Optional cell scanner.
1059    * @return the converted client Result
1060    * @throws IOException
1061    */
1062   public static Result toResult(final ClientProtos.Result proto, final CellScanner scanner)
1063   throws IOException {
1064     // TODO: Unit test that has some Cells in scanner and some in the proto.
1065     List<Cell> cells = null;
1066     if (proto.hasAssociatedCellCount()) {
1067       int count = proto.getAssociatedCellCount();
1068       cells = new ArrayList<Cell>(count);
1069       for (int i = 0; i < count; i++) {
1070         if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count);
1071         cells.add(scanner.current());
1072       }
1073     }
1074     List<HBaseProtos.Cell> values = proto.getCellList();
1075     if (cells == null) cells = new ArrayList<Cell>(values.size());
1076     for (HBaseProtos.Cell c: values) {
1077       cells.add(toCell(c));
1078     }
1079     return new Result(cells);
1080   }
1081 
1082   /**
1083    * Convert a ByteArrayComparable to a protocol buffer Comparator
1084    *
1085    * @param comparator the ByteArrayComparable to convert
1086    * @return the converted protocol buffer Comparator
1087    */
1088   public static ComparatorProtos.Comparator toComparator(ByteArrayComparable comparator) {
1089     ComparatorProtos.Comparator.Builder builder = ComparatorProtos.Comparator.newBuilder();
1090     builder.setName(comparator.getClass().getName());
1091     builder.setSerializedComparator(ByteString.copyFrom(comparator.toByteArray()));
1092     return builder.build();
1093   }
1094 
1095   /**
1096    * Convert a protocol buffer Comparator to a ByteArrayComparable
1097    *
1098    * @param proto the protocol buffer Comparator to convert
1099    * @return the converted ByteArrayComparable
1100    */
1101   @SuppressWarnings("unchecked")
1102   public static ByteArrayComparable toComparator(ComparatorProtos.Comparator proto)
1103   throws IOException {
1104     String type = proto.getName();
1105     String funcName = "parseFrom";
1106     byte [] value = proto.getSerializedComparator().toByteArray();
1107     try {
1108       Class<? extends ByteArrayComparable> c =
1109         (Class<? extends ByteArrayComparable>)Class.forName(type, true, CLASS_LOADER);
1110       Method parseFrom = c.getMethod(funcName, byte[].class);
1111       if (parseFrom == null) {
1112         throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
1113       }
1114       return (ByteArrayComparable)parseFrom.invoke(null, value);
1115     } catch (Exception e) {
1116       throw new IOException(e);
1117     }
1118   }
1119 
1120   /**
1121    * Convert a protocol buffer Filter to a client Filter
1122    *
1123    * @param proto the protocol buffer Filter to convert
1124    * @return the converted Filter
1125    */
1126   @SuppressWarnings("unchecked")
1127   public static Filter toFilter(HBaseProtos.Filter proto) throws IOException {
1128     String type = proto.getName();
1129     final byte [] value = proto.getSerializedFilter().toByteArray();
1130     String funcName = "parseFrom";
1131     try {
1132       Class<? extends Filter> c =
1133         (Class<? extends Filter>)Class.forName(type, true, CLASS_LOADER);
1134       Method parseFrom = c.getMethod(funcName, byte[].class);
1135       if (parseFrom == null) {
1136         throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
1137       }
1138       return (Filter)parseFrom.invoke(c, value);
1139     } catch (Exception e) {
1140       throw new IOException(e);
1141     }
1142   }
1143 
1144   /**
1145    * Convert a client Filter to a protocol buffer Filter
1146    *
1147    * @param filter the Filter to convert
1148    * @return the converted protocol buffer Filter
1149    */
1150   public static HBaseProtos.Filter toFilter(Filter filter) throws IOException {
1151     HBaseProtos.Filter.Builder builder = HBaseProtos.Filter.newBuilder();
1152     builder.setName(filter.getClass().getName());
1153     builder.setSerializedFilter(ByteString.copyFrom(filter.toByteArray()));
1154     return builder.build();
1155   }
1156 
1157   /**
1158    * Convert a delete KeyValue type to protocol buffer DeleteType.
1159    *
1160    * @param type
1161    * @return protocol buffer DeleteType
1162    * @throws IOException
1163    */
1164   public static DeleteType toDeleteType(
1165       KeyValue.Type type) throws IOException {
1166     switch (type) {
1167     case Delete:
1168       return DeleteType.DELETE_ONE_VERSION;
1169     case DeleteColumn:
1170       return DeleteType.DELETE_MULTIPLE_VERSIONS;
1171     case DeleteFamily:
1172       return DeleteType.DELETE_FAMILY;
1173       default:
1174         throw new IOException("Unknown delete type: " + type);
1175     }
1176   }
1177 
1178   /**
1179    * Convert a stringified protocol buffer exception Parameter to a Java Exception
1180    *
1181    * @param parameter the protocol buffer Parameter to convert
1182    * @return the converted Exception
1183    * @throws IOException if failed to deserialize the parameter
1184    */
1185   @SuppressWarnings("unchecked")
1186   public static Throwable toException(
1187       final NameBytesPair parameter) throws IOException {
1188     if (parameter == null || !parameter.hasValue()) return null;
1189     String desc = parameter.getValue().toStringUtf8();
1190     String type = parameter.getName();
1191     try {
1192       Class<? extends Throwable> c =
1193         (Class<? extends Throwable>)Class.forName(type, true, CLASS_LOADER);
1194       Constructor<? extends Throwable> cn =
1195         c.getDeclaredConstructor(String.class);
1196       return cn.newInstance(desc);
1197     } catch (Exception e) {
1198       throw new IOException(e);
1199     }
1200   }
1201 
1202 // Start helpers for Client
1203 
1204   /**
1205    * A helper to invoke a Get using client protocol.
1206    *
1207    * @param client
1208    * @param regionName
1209    * @param get
1210    * @return the result of the Get
1211    * @throws IOException
1212    */
1213   public static Result get(final ClientService.BlockingInterface client,
1214       final byte[] regionName, final Get get) throws IOException {
1215     GetRequest request =
1216       RequestConverter.buildGetRequest(regionName, get);
1217     try {
1218       GetResponse response = client.get(null, request);
1219       if (response == null) return null;
1220       return toResult(response.getResult());
1221     } catch (ServiceException se) {
1222       throw getRemoteException(se);
1223     }
1224   }
1225 
1226   /**
1227    * A helper to get a row of the closet one before using client protocol.
1228    *
1229    * @param client
1230    * @param regionName
1231    * @param row
1232    * @param family
1233    * @return the row or the closestRowBefore if it doesn't exist
1234    * @throws IOException
1235    */
1236   public static Result getRowOrBefore(final ClientService.BlockingInterface client,
1237       final byte[] regionName, final byte[] row,
1238       final byte[] family) throws IOException {
1239     GetRequest request =
1240       RequestConverter.buildGetRowOrBeforeRequest(
1241         regionName, row, family);
1242     try {
1243       GetResponse response = client.get(null, request);
1244       if (!response.hasResult()) return null;
1245       return toResult(response.getResult());
1246     } catch (ServiceException se) {
1247       throw getRemoteException(se);
1248     }
1249   }
1250 
1251   /**
1252    * A helper to bulk load a list of HFiles using client protocol.
1253    *
1254    * @param client
1255    * @param familyPaths
1256    * @param regionName
1257    * @param assignSeqNum
1258    * @return true if all are loaded
1259    * @throws IOException
1260    */
1261   public static boolean bulkLoadHFile(final ClientService.BlockingInterface client,
1262       final List<Pair<byte[], String>> familyPaths,
1263       final byte[] regionName, boolean assignSeqNum) throws IOException {
1264     BulkLoadHFileRequest request =
1265       RequestConverter.buildBulkLoadHFileRequest(familyPaths, regionName, assignSeqNum);
1266     try {
1267       BulkLoadHFileResponse response =
1268         client.bulkLoadHFile(null, request);
1269       return response.getLoaded();
1270     } catch (ServiceException se) {
1271       throw getRemoteException(se);
1272     }
1273   }
1274 
1275   public static CoprocessorServiceResponse execService(final ClientService.BlockingInterface client,
1276       final CoprocessorServiceCall call, final byte[] regionName) throws IOException {
1277     CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
1278         .setCall(call).setRegion(
1279             RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build();
1280     try {
1281       CoprocessorServiceResponse response =
1282           client.execService(null, request);
1283       return response;
1284     } catch (ServiceException se) {
1285       throw getRemoteException(se);
1286     }
1287   }
1288 
1289   public static CoprocessorServiceResponse execService(
1290     final MasterAdminService.BlockingInterface client, final CoprocessorServiceCall call)
1291   throws IOException {
1292     CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
1293         .setCall(call).setRegion(
1294             RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY)).build();
1295     try {
1296       CoprocessorServiceResponse response =
1297           client.execMasterService(null, request);
1298       return response;
1299     } catch (ServiceException se) {
1300       throw getRemoteException(se);
1301     }
1302   }
1303 
1304   @SuppressWarnings("unchecked")
1305   public static <T extends Service> T newServiceStub(Class<T> service, RpcChannel channel)
1306       throws Exception {
1307     return (T)Methods.call(service, null, "newStub",
1308         new Class[]{ RpcChannel.class }, new Object[]{ channel });
1309   }
1310 
1311 // End helpers for Client
1312 // Start helpers for Admin
1313 
1314   /**
1315    * A helper to retrieve region info given a region name
1316    * using admin protocol.
1317    *
1318    * @param admin
1319    * @param regionName
1320    * @return the retrieved region info
1321    * @throws IOException
1322    */
1323   public static HRegionInfo getRegionInfo(final AdminService.BlockingInterface admin,
1324       final byte[] regionName) throws IOException {
1325     try {
1326       GetRegionInfoRequest request =
1327         RequestConverter.buildGetRegionInfoRequest(regionName);
1328       GetRegionInfoResponse response =
1329         admin.getRegionInfo(null, request);
1330       return HRegionInfo.convert(response.getRegionInfo());
1331     } catch (ServiceException se) {
1332       throw getRemoteException(se);
1333     }
1334   }
1335 
1336   /**
1337    * A helper to close a region given a region name
1338    * using admin protocol.
1339    *
1340    * @param admin
1341    * @param regionName
1342    * @param transitionInZK
1343    * @throws IOException
1344    */
1345   public static void closeRegion(final AdminService.BlockingInterface admin,
1346       final byte[] regionName, final boolean transitionInZK) throws IOException {
1347     CloseRegionRequest closeRegionRequest =
1348       RequestConverter.buildCloseRegionRequest(regionName, transitionInZK);
1349     try {
1350       admin.closeRegion(null, closeRegionRequest);
1351     } catch (ServiceException se) {
1352       throw getRemoteException(se);
1353     }
1354   }
1355 
1356   /**
1357    * A helper to close a region given a region name
1358    * using admin protocol.
1359    *
1360    * @param admin
1361    * @param regionName
1362    * @param versionOfClosingNode
1363    * @return true if the region is closed
1364    * @throws IOException
1365    */
1366   public static boolean closeRegion(final AdminService.BlockingInterface admin,
1367       final byte[] regionName,
1368       final int versionOfClosingNode, final ServerName destinationServer,
1369       final boolean transitionInZK) throws IOException {
1370     CloseRegionRequest closeRegionRequest =
1371       RequestConverter.buildCloseRegionRequest(
1372         regionName, versionOfClosingNode, destinationServer, transitionInZK);
1373     try {
1374       CloseRegionResponse response = admin.closeRegion(null, closeRegionRequest);
1375       return ResponseConverter.isClosed(response);
1376     } catch (ServiceException se) {
1377       throw getRemoteException(se);
1378     }
1379   }
1380 
1381 
1382   /**
1383    * A helper to open a region using admin protocol.
1384    * @param admin
1385    * @param region
1386    * @throws IOException
1387    */
1388   public static void openRegion(final AdminService.BlockingInterface admin,
1389       final HRegionInfo region) throws IOException {
1390     OpenRegionRequest request =
1391       RequestConverter.buildOpenRegionRequest(region, -1, null);
1392     try {
1393       admin.openRegion(null, request);
1394     } catch (ServiceException se) {
1395       throw ProtobufUtil.getRemoteException(se);
1396     }
1397   }
1398 
1399   /**
1400    * A helper to get the all the online regions on a region
1401    * server using admin protocol.
1402    *
1403    * @param admin
1404    * @return a list of online region info
1405    * @throws IOException
1406    */
1407   public static List<HRegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
1408   throws IOException {
1409     GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
1410     GetOnlineRegionResponse response = null;
1411     try {
1412       response = admin.getOnlineRegion(null, request);
1413     } catch (ServiceException se) {
1414       throw getRemoteException(se);
1415     }
1416     return getRegionInfos(response);
1417   }
1418 
1419   /**
1420    * Get the list of region info from a GetOnlineRegionResponse
1421    *
1422    * @param proto the GetOnlineRegionResponse
1423    * @return the list of region info or null if <code>proto</code> is null
1424    */
1425   static List<HRegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) {
1426     if (proto == null) return null;
1427     List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
1428     for (RegionInfo regionInfo: proto.getRegionInfoList()) {
1429       regionInfos.add(HRegionInfo.convert(regionInfo));
1430     }
1431     return regionInfos;
1432   }
1433 
1434   /**
1435    * A helper to get the info of a region server using admin protocol.
1436    *
1437    * @param admin
1438    * @return the server name
1439    * @throws IOException
1440    */
1441   public static ServerInfo getServerInfo(final AdminService.BlockingInterface admin)
1442   throws IOException {
1443     GetServerInfoRequest request = RequestConverter.buildGetServerInfoRequest();
1444     try {
1445       GetServerInfoResponse response = admin.getServerInfo(null, request);
1446       return response.getServerInfo();
1447     } catch (ServiceException se) {
1448       throw getRemoteException(se);
1449     }
1450   }
1451 
1452   /**
1453    * A helper to get the list of files of a column family
1454    * on a given region using admin protocol.
1455    *
1456    * @param admin
1457    * @param regionName
1458    * @param family
1459    * @return the list of store files
1460    * @throws IOException
1461    */
1462   public static List<String> getStoreFiles(final AdminService.BlockingInterface admin,
1463       final byte[] regionName, final byte[] family)
1464   throws IOException {
1465     GetStoreFileRequest request =
1466       RequestConverter.buildGetStoreFileRequest(regionName, family);
1467     try {
1468       GetStoreFileResponse response = admin.getStoreFile(null, request);
1469       return response.getStoreFileList();
1470     } catch (ServiceException se) {
1471       throw ProtobufUtil.getRemoteException(se);
1472     }
1473   }
1474 
1475   /**
1476    * A helper to split a region using admin protocol.
1477    *
1478    * @param admin
1479    * @param hri
1480    * @param splitPoint
1481    * @throws IOException
1482    */
1483   public static void split(final AdminService.BlockingInterface admin,
1484       final HRegionInfo hri, byte[] splitPoint) throws IOException {
1485     SplitRegionRequest request =
1486       RequestConverter.buildSplitRegionRequest(hri.getRegionName(), splitPoint);
1487     try {
1488       admin.splitRegion(null, request);
1489     } catch (ServiceException se) {
1490       throw ProtobufUtil.getRemoteException(se);
1491     }
1492   }
1493 
1494   /**
1495    * A helper to merge regions using admin protocol. Send request to
1496    * regionserver.
1497    * @param admin
1498    * @param region_a
1499    * @param region_b
1500    * @param forcible true if do a compulsory merge, otherwise we will only merge
1501    *          two adjacent regions
1502    * @throws IOException
1503    */
1504   public static void mergeRegions(final AdminService.BlockingInterface admin,
1505       final HRegionInfo region_a, final HRegionInfo region_b,
1506       final boolean forcible) throws IOException {
1507     MergeRegionsRequest request = RequestConverter.buildMergeRegionsRequest(
1508         region_a.getRegionName(), region_b.getRegionName(),forcible);
1509     try {
1510       admin.mergeRegions(null, request);
1511     } catch (ServiceException se) {
1512       throw ProtobufUtil.getRemoteException(se);
1513     }
1514   }
1515 
1516 // End helpers for Admin
1517 
1518   /*
1519    * Get the total (read + write) requests from a RegionLoad pb
1520    * @param rl - RegionLoad pb
1521    * @return total (read + write) requests
1522    */
1523   public static long getTotalRequestsCount(RegionLoad rl) {
1524     if (rl == null) {
1525       return 0;
1526     }
1527 
1528     return rl.getReadRequestsCount() + rl.getWriteRequestsCount();
1529   }
1530 
1531 
1532   /**
1533    * @param m Message to get delimited pb serialization of (with pb magic prefix)
1534    */
1535   public static byte [] toDelimitedByteArray(final Message m) throws IOException {
1536     // Allocate arbitrary big size so we avoid resizing.
1537     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
1538     m.writeDelimitedTo(baos);
1539     baos.close();
1540     return ProtobufUtil.prependPBMagic(baos.toByteArray());
1541   }
1542 
1543   /**
1544    * Converts a Permission proto to a client Permission object.
1545    *
1546    * @param proto the protobuf Permission
1547    * @return the converted Permission
1548    */
1549   public static Permission toPermission(AccessControlProtos.Permission proto) {
1550     if (proto.hasTable()) {
1551       return toTablePermission(proto);
1552     } else {
1553       List<Permission.Action> actions = toPermissionActions(proto.getActionList());
1554       return new Permission(actions.toArray(new Permission.Action[actions.size()]));
1555     }
1556   }
1557 
1558   /**
1559    * Converts a Permission proto to a client TablePermission object.
1560    *
1561    * @param proto the protobuf Permission
1562    * @return the converted TablePermission
1563    */
1564   public static TablePermission toTablePermission(AccessControlProtos.Permission proto) {
1565     List<Permission.Action> actions = toPermissionActions(proto.getActionList());
1566 
1567     byte[] qualifier = null;
1568     byte[] family = null;
1569     byte[] table = null;
1570 
1571     if (proto.hasTable()) table = proto.getTable().toByteArray();
1572     if (proto.hasFamily()) family = proto.getFamily().toByteArray();
1573     if (proto.hasQualifier()) qualifier = proto.getQualifier().toByteArray();
1574 
1575     return new TablePermission(table, family, qualifier,
1576         actions.toArray(new Permission.Action[actions.size()]));
1577   }
1578 
1579   /**
1580    * Convert a client Permission to a Permission proto
1581    *
1582    * @param perm the client Permission
1583    * @return the protobuf Permission
1584    */
1585   public static AccessControlProtos.Permission toPermission(Permission perm) {
1586     AccessControlProtos.Permission.Builder builder = AccessControlProtos.Permission.newBuilder();
1587     if (perm instanceof TablePermission) {
1588       TablePermission tablePerm = (TablePermission)perm;
1589       if (tablePerm.hasTable()) {
1590         builder.setTable(ByteString.copyFrom(tablePerm.getTable()));
1591       }
1592       if (tablePerm.hasFamily()) {
1593         builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
1594       }
1595       if (tablePerm.hasQualifier()) {
1596         builder.setQualifier(ByteString.copyFrom(tablePerm.getQualifier()));
1597       }
1598     }
1599     for (Permission.Action a : perm.getActions()) {
1600       builder.addAction(toPermissionAction(a));
1601     }
1602     return builder.build();
1603   }
1604 
1605   /**
1606    * Converts a list of Permission.Action proto to a list of client Permission.Action objects.
1607    *
1608    * @param protoActions the list of protobuf Actions
1609    * @return the converted list of Actions
1610    */
1611   public static List<Permission.Action> toPermissionActions(
1612       List<AccessControlProtos.Permission.Action> protoActions) {
1613     List<Permission.Action> actions = new ArrayList<Permission.Action>(protoActions.size());
1614     for (AccessControlProtos.Permission.Action a : protoActions) {
1615       actions.add(toPermissionAction(a));
1616     }
1617     return actions;
1618   }
1619 
1620   /**
1621    * Converts a Permission.Action proto to a client Permission.Action object.
1622    *
1623    * @param action the protobuf Action
1624    * @return the converted Action
1625    */
1626   public static Permission.Action toPermissionAction(
1627       AccessControlProtos.Permission.Action action) {
1628     switch (action) {
1629       case READ:
1630         return Permission.Action.READ;
1631       case WRITE:
1632         return Permission.Action.WRITE;
1633       case EXEC:
1634         return Permission.Action.EXEC;
1635       case CREATE:
1636         return Permission.Action.CREATE;
1637       case ADMIN:
1638         return Permission.Action.ADMIN;
1639     }
1640     throw new IllegalArgumentException("Unknown action value "+action.name());
1641   }
1642 
1643   /**
1644    * Convert a client Permission.Action to a Permission.Action proto
1645    *
1646    * @param action the client Action
1647    * @return the protobuf Action
1648    */
1649   public static AccessControlProtos.Permission.Action toPermissionAction(
1650       Permission.Action action) {
1651     switch (action) {
1652       case READ:
1653         return AccessControlProtos.Permission.Action.READ;
1654       case WRITE:
1655         return AccessControlProtos.Permission.Action.WRITE;
1656       case EXEC:
1657         return AccessControlProtos.Permission.Action.EXEC;
1658       case CREATE:
1659         return AccessControlProtos.Permission.Action.CREATE;
1660       case ADMIN:
1661         return AccessControlProtos.Permission.Action.ADMIN;
1662     }
1663     throw new IllegalArgumentException("Unknown action value "+action.name());
1664   }
1665 
1666   /**
1667    * Convert a client user permission to a user permission proto
1668    *
1669    * @param perm the client UserPermission
1670    * @return the protobuf UserPermission
1671    */
1672   public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) {
1673     AccessControlProtos.Permission.Builder permissionBuilder =
1674         AccessControlProtos.Permission.newBuilder();
1675     for (Permission.Action a : perm.getActions()) {
1676       permissionBuilder.addAction(toPermissionAction(a));
1677     }
1678     if (perm.hasTable()) {
1679       permissionBuilder.setTable(ByteString.copyFrom(perm.getTable()));
1680     }
1681     if (perm.hasFamily()) {
1682       permissionBuilder.setFamily(ByteString.copyFrom(perm.getFamily()));
1683     }
1684     if (perm.hasQualifier()) {
1685       permissionBuilder.setQualifier(ByteString.copyFrom(perm.getQualifier()));
1686     }
1687 
1688     return AccessControlProtos.UserPermission.newBuilder()
1689         .setUser(ByteString.copyFrom(perm.getUser()))
1690         .setPermission(permissionBuilder)
1691         .build();
1692   }
1693 
1694   /**
1695    * Converts a user permission proto to a client user permission object.
1696    *
1697    * @param proto the protobuf UserPermission
1698    * @return the converted UserPermission
1699    */
1700   public static UserPermission toUserPermission(AccessControlProtos.UserPermission proto) {
1701     AccessControlProtos.Permission permission = proto.getPermission();
1702     List<Permission.Action> actions = toPermissionActions(permission.getActionList());
1703 
1704     byte[] qualifier = null;
1705     byte[] family = null;
1706     byte[] table = null;
1707 
1708     if (permission.hasTable()) table = permission.getTable().toByteArray();
1709     if (permission.hasFamily()) family = permission.getFamily().toByteArray();
1710     if (permission.hasQualifier()) qualifier = permission.getQualifier().toByteArray();
1711 
1712     return new UserPermission(proto.getUser().toByteArray(),
1713         table, family, qualifier,
1714         actions.toArray(new Permission.Action[actions.size()]));
1715   }
1716 
1717   /**
1718    * Convert a ListMultimap<String, TablePermission> where key is username
1719    * to a protobuf UserPermission
1720    *
1721    * @param perm the list of user and table permissions
1722    * @return the protobuf UserTablePermissions
1723    */
1724   public static AccessControlProtos.UserTablePermissions toUserTablePermissions(
1725       ListMultimap<String, TablePermission> perm) {
1726     AccessControlProtos.UserTablePermissions.Builder builder =
1727                   AccessControlProtos.UserTablePermissions.newBuilder();
1728     for (Map.Entry<String, Collection<TablePermission>> entry : perm.asMap().entrySet()) {
1729       AccessControlProtos.UserTablePermissions.UserPermissions.Builder userPermBuilder =
1730                   AccessControlProtos.UserTablePermissions.UserPermissions.newBuilder();
1731       userPermBuilder.setUser(ByteString.copyFromUtf8(entry.getKey()));
1732       for (TablePermission tablePerm: entry.getValue()) {
1733         userPermBuilder.addPermissions(toPermission(tablePerm));
1734       }
1735       builder.addPermissions(userPermBuilder.build());
1736     }
1737     return builder.build();
1738   }
1739 
1740   /**
1741    * A utility used to grant a user some permissions. The permissions will
1742    * be global if table is not specified.  Otherwise, they are for those
1743    * table/column family/qualifier only.
1744    * <p>
1745    * It's also called by the shell, in case you want to find references.
1746    *
1747    * @param protocol the AccessControlService protocol proxy
1748    * @param userShortName the short name of the user to grant permissions
1749    * @param t optional table name
1750    * @param f optional column family
1751    * @param q optional qualifier
1752    * @param actions the permissions to be granted
1753    * @throws ServiceException
1754    */
1755   public static void grant(AccessControlService.BlockingInterface protocol,
1756       String userShortName, byte[] t, byte[] f, byte[] q,
1757       Permission.Action... actions) throws ServiceException {
1758     List<AccessControlProtos.Permission.Action> permActions =
1759         Lists.newArrayListWithCapacity(actions.length);
1760     for (Permission.Action a : actions) {
1761       permActions.add(ProtobufUtil.toPermissionAction(a));
1762     }
1763     AccessControlProtos.GrantRequest request = RequestConverter.
1764       buildGrantRequest(userShortName, t, f, q, permActions.toArray(
1765         new AccessControlProtos.Permission.Action[actions.length]));
1766     protocol.grant(null, request);
1767   }
1768 
1769   /**
1770    * A utility used to revoke a user some permissions. The permissions will
1771    * be global if table is not specified.  Otherwise, they are for those
1772    * table/column family/qualifier only.
1773    * <p>
1774    * It's also called by the shell, in case you want to find references.
1775    *
1776    * @param protocol the AccessControlService protocol proxy
1777    * @param userShortName the short name of the user to revoke permissions
1778    * @param t optional table name
1779    * @param f optional column family
1780    * @param q optional qualifier
1781    * @param actions the permissions to be revoked
1782    * @throws ServiceException
1783    */
1784   public static void revoke(AccessControlService.BlockingInterface protocol,
1785       String userShortName, byte[] t, byte[] f, byte[] q,
1786       Permission.Action... actions) throws ServiceException {
1787     List<AccessControlProtos.Permission.Action> permActions =
1788         Lists.newArrayListWithCapacity(actions.length);
1789     for (Permission.Action a : actions) {
1790       permActions.add(ProtobufUtil.toPermissionAction(a));
1791     }
1792     AccessControlProtos.RevokeRequest request = RequestConverter.
1793       buildRevokeRequest(userShortName, t, f, q, permActions.toArray(
1794         new AccessControlProtos.Permission.Action[actions.length]));
1795     protocol.revoke(null, request);
1796   }
1797 
1798   /**
1799    * A utility used to get user permissions.
1800    * <p>
1801    * It's also called by the shell, in case you want to find references.
1802    *
1803    * @param protocol the AccessControlService protocol proxy
1804    * @param t optional table name
1805    * @throws ServiceException
1806    */
1807   public static List<UserPermission> getUserPermissions(
1808       AccessControlService.BlockingInterface protocol,
1809       byte[] t) throws ServiceException {
1810     AccessControlProtos.UserPermissionsRequest.Builder builder =
1811       AccessControlProtos.UserPermissionsRequest.newBuilder();
1812     if (t != null) {
1813       builder.setTable(ByteString.copyFrom(t));
1814     }
1815     AccessControlProtos.UserPermissionsRequest request = builder.build();
1816     AccessControlProtos.UserPermissionsResponse response =
1817       protocol.getUserPermissions(null, request);
1818     List<UserPermission> perms = new ArrayList<UserPermission>();
1819     for (AccessControlProtos.UserPermission perm: response.getPermissionList()) {
1820       perms.add(ProtobufUtil.toUserPermission(perm));
1821     }
1822     return perms;
1823   }
1824 
1825   /**
1826    * Convert a protobuf UserTablePermissions to a
1827    * ListMultimap<String, TablePermission> where key is username.
1828    *
1829    * @param proto the protobuf UserPermission
1830    * @return the converted UserPermission
1831    */
1832   public static ListMultimap<String, TablePermission> toUserTablePermissions(
1833       AccessControlProtos.UserTablePermissions proto) {
1834     ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
1835     AccessControlProtos.UserTablePermissions.UserPermissions userPerm;
1836 
1837     for (int i = 0; i < proto.getPermissionsCount(); i++) {
1838       userPerm = proto.getPermissions(i);
1839       for (int j = 0; j < userPerm.getPermissionsCount(); j++) {
1840         TablePermission tablePerm = toTablePermission(userPerm.getPermissions(j));
1841         perms.put(userPerm.getUser().toStringUtf8(), tablePerm);
1842       }
1843     }
1844 
1845     return perms;
1846   }
1847 
1848   /**
1849    * Converts a Token instance (with embedded identifier) to the protobuf representation.
1850    *
1851    * @param token the Token instance to copy
1852    * @return the protobuf Token message
1853    */
1854   public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
1855     AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder();
1856     builder.setIdentifier(ByteString.copyFrom(token.getIdentifier()));
1857     builder.setPassword(ByteString.copyFrom(token.getPassword()));
1858     if (token.getService() != null) {
1859       builder.setService(ByteString.copyFromUtf8(token.getService().toString()));
1860     }
1861     return builder.build();
1862   }
1863 
1864   /**
1865    * Converts a protobuf Token message back into a Token instance.
1866    *
1867    * @param proto the protobuf Token message
1868    * @return the Token instance
1869    */
1870   public static Token<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) {
1871     return new Token<AuthenticationTokenIdentifier>(
1872         proto.hasIdentifier() ? proto.getIdentifier().toByteArray() : null,
1873         proto.hasPassword() ? proto.getPassword().toByteArray() : null,
1874         AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE,
1875         proto.hasService() ? new Text(proto.getService().toStringUtf8()) : null);
1876   }
1877 
1878   /**
1879    * Find the HRegion encoded name based on a region specifier
1880    *
1881    * @param regionSpecifier the region specifier
1882    * @return the corresponding region's encoded name
1883    * @throws DoNotRetryIOException if the specifier type is unsupported
1884    */
1885   public static String getRegionEncodedName(
1886       final RegionSpecifier regionSpecifier) throws DoNotRetryIOException {
1887     byte[] value = regionSpecifier.getValue().toByteArray();
1888     RegionSpecifierType type = regionSpecifier.getType();
1889     switch (type) {
1890       case REGION_NAME:
1891         return HRegionInfo.encodeRegionName(value);
1892       case ENCODED_REGION_NAME:
1893         return Bytes.toString(value);
1894       default:
1895         throw new DoNotRetryIOException(
1896           "Unsupported region specifier type: " + type);
1897     }
1898   }
1899 
1900   public static ScanMetrics toScanMetrics(final byte[] bytes) {
1901     MapReduceProtos.ScanMetrics.Builder builder = MapReduceProtos.ScanMetrics.newBuilder();
1902     try {
1903       builder.mergeFrom(bytes);
1904     } catch (InvalidProtocolBufferException e) {
1905       //Ignored there are just no key values to add.
1906     }
1907     MapReduceProtos.ScanMetrics pScanMetrics = builder.build();
1908     ScanMetrics scanMetrics = new ScanMetrics();
1909     for (HBaseProtos.NameInt64Pair pair : pScanMetrics.getMetricsList()) {
1910       if (pair.hasName() && pair.hasValue()) {
1911         scanMetrics.setCounter(pair.getName(), pair.getValue());
1912       }
1913     }
1914     return scanMetrics;
1915   }
1916 
1917   public static MapReduceProtos.ScanMetrics toScanMetrics(ScanMetrics scanMetrics) {
1918     MapReduceProtos.ScanMetrics.Builder builder = MapReduceProtos.ScanMetrics.newBuilder();
1919     Map<String, Long> metrics = scanMetrics.getMetricsMap();
1920     for (Entry<String, Long> e : metrics.entrySet()) {
1921       HBaseProtos.NameInt64Pair nameInt64Pair =
1922           HBaseProtos.NameInt64Pair.newBuilder()
1923               .setName(e.getKey())
1924               .setValue(e.getValue())
1925               .build();
1926       builder.addMetrics(nameInt64Pair);
1927     }
1928     return builder.build();
1929   }
1930 
1931   /**
1932    * Unwraps an exception from a protobuf service into the underlying (expected) IOException.
1933    * This method will <strong>always</strong> throw an exception.
1934    * @param se the {@code ServiceException} instance to convert into an {@code IOException}
1935    */
1936   public static void toIOException(ServiceException se) throws IOException {
1937     if (se == null) {
1938       throw new NullPointerException("Null service exception passed!");
1939     }
1940 
1941     Throwable cause = se.getCause();
1942     if (cause != null && cause instanceof IOException) {
1943       throw (IOException)cause;
1944     }
1945     throw new IOException(se);
1946   }
1947 
1948   public static HBaseProtos.Cell toCell(final Cell kv) {
1949     // Doing this is going to kill us if we do it for all data passed.
1950     // St.Ack 20121205
1951     HBaseProtos.Cell.Builder kvbuilder = HBaseProtos.Cell.newBuilder();
1952     kvbuilder.setRow(ByteString.copyFrom(kv.getRowArray(), kv.getRowOffset(),
1953       kv.getRowLength()));
1954     kvbuilder.setFamily(ByteString.copyFrom(kv.getFamilyArray(),
1955       kv.getFamilyOffset(), kv.getFamilyLength()));
1956     kvbuilder.setQualifier(ByteString.copyFrom(kv.getQualifierArray(),
1957       kv.getQualifierOffset(), kv.getQualifierLength()));
1958     kvbuilder.setCellType(HBaseProtos.CellType.valueOf(kv.getTypeByte()));
1959     kvbuilder.setTimestamp(kv.getTimestamp());
1960     kvbuilder.setValue(ByteString.copyFrom(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
1961     return kvbuilder.build();
1962   }
1963 
1964   public static Cell toCell(final HBaseProtos.Cell cell) {
1965     // Doing this is going to kill us if we do it for all data passed.
1966     // St.Ack 20121205
1967     return CellUtil.createCell(cell.getRow().toByteArray(),
1968       cell.getFamily().toByteArray(),
1969       cell.getQualifier().toByteArray(),
1970       cell.getTimestamp(),
1971       (byte)cell.getCellType().getNumber(),
1972       cell.getValue().toByteArray());
1973   }
1974 
1975   /**
1976    * Get an instance of the argument type declared in a class's signature. The
1977    * argument type is assumed to be a PB Message subclass, and the instance is
1978    * created using parseFrom method on the passed ByteString.
1979    * @param runtimeClass the runtime type of the class
1980    * @param position the position of the argument in the class declaration
1981    * @param b the ByteString which should be parsed to get the instance created
1982    * @return the instance
1983    * @throws IOException
1984    */
1985   @SuppressWarnings("unchecked")
1986   public static <T extends Message>
1987   T getParsedGenericInstance(Class<?> runtimeClass, int position, ByteString b)
1988       throws IOException {
1989     Type type = runtimeClass.getGenericSuperclass();
1990     Type argType = ((ParameterizedType)type).getActualTypeArguments()[position];
1991     Class<T> classType = (Class<T>)argType;
1992     T inst;
1993     try {
1994       Method m = classType.getMethod("parseFrom", ByteString.class);
1995       inst = (T)m.invoke(null, b);
1996       return inst;
1997     } catch (SecurityException e) {
1998       throw new IOException(e);
1999     } catch (NoSuchMethodException e) {
2000       throw new IOException(e);
2001     } catch (IllegalArgumentException e) {
2002       throw new IOException(e);
2003     } catch (InvocationTargetException e) {
2004       throw new IOException(e);
2005     } catch (IllegalAccessException e) {
2006       throw new IOException(e);
2007     }
2008   }
2009 
2010   public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
2011       List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
2012     // compaction descriptor contains relative paths.
2013     // input / output paths are relative to the store dir
2014     // store dir is relative to region dir
2015     CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder()
2016         .setTableName(ByteString.copyFrom(info.getTableName()))
2017         .setEncodedRegionName(ByteString.copyFrom(info.getEncodedNameAsBytes()))
2018         .setFamilyName(ByteString.copyFrom(family))
2019         .setStoreHomeDir(storeDir.getName()); //make relative
2020     for (Path inputPath : inputPaths) {
2021       builder.addCompactionInput(inputPath.getName()); //relative path
2022     }
2023     for (Path outputPath : outputPaths) {
2024       builder.addCompactionOutput(outputPath.getName());
2025     }
2026     return builder.build();
2027   }
2028 }