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