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