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