View Javadoc

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