View Javadoc

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