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