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