View Javadoc

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