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