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