View Javadoc

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