001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.shaded.protobuf; 019 020import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC; 021 022import java.io.ByteArrayOutputStream; 023import java.io.IOException; 024import java.io.InputStream; 025import java.lang.reflect.Constructor; 026import java.lang.reflect.Method; 027import java.nio.ByteBuffer; 028import java.security.AccessController; 029import java.security.PrivilegedAction; 030import java.util.ArrayList; 031import java.util.Collections; 032import java.util.List; 033import java.util.Locale; 034import java.util.Map; 035import java.util.Map.Entry; 036import java.util.NavigableSet; 037import java.util.Optional; 038import java.util.concurrent.Callable; 039import java.util.concurrent.TimeUnit; 040import java.util.function.Function; 041import java.util.regex.Pattern; 042import java.util.stream.Collectors; 043 044import org.apache.commons.io.input.BoundedInputStream; 045import org.apache.hadoop.conf.Configuration; 046import org.apache.hadoop.fs.Path; 047import org.apache.hadoop.hbase.ByteBufferExtendedCell; 048import org.apache.hadoop.hbase.CacheEvictionStats; 049import org.apache.hadoop.hbase.CacheEvictionStatsBuilder; 050import org.apache.hadoop.hbase.Cell; 051import org.apache.hadoop.hbase.Cell.Type; 052import org.apache.hadoop.hbase.CellBuilderType; 053import org.apache.hadoop.hbase.CellScanner; 054import org.apache.hadoop.hbase.CellUtil; 055import org.apache.hadoop.hbase.DoNotRetryIOException; 056import org.apache.hadoop.hbase.ExtendedCellBuilder; 057import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; 058import org.apache.hadoop.hbase.HBaseConfiguration; 059import org.apache.hadoop.hbase.HBaseIOException; 060import org.apache.hadoop.hbase.HConstants; 061import org.apache.hadoop.hbase.HRegionLocation; 062import org.apache.hadoop.hbase.KeyValue; 063import org.apache.hadoop.hbase.NamespaceDescriptor; 064import org.apache.hadoop.hbase.ServerName; 065import org.apache.hadoop.hbase.TableName; 066import org.apache.hadoop.hbase.client.Append; 067import org.apache.hadoop.hbase.client.ClientUtil; 068import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 069import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; 070import org.apache.hadoop.hbase.client.CompactionState; 071import org.apache.hadoop.hbase.client.Consistency; 072import org.apache.hadoop.hbase.client.Cursor; 073import org.apache.hadoop.hbase.client.Delete; 074import org.apache.hadoop.hbase.client.Durability; 075import org.apache.hadoop.hbase.client.Get; 076import org.apache.hadoop.hbase.client.Increment; 077import org.apache.hadoop.hbase.client.Mutation; 078import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor; 079import org.apache.hadoop.hbase.client.Put; 080import org.apache.hadoop.hbase.client.RegionInfoBuilder; 081import org.apache.hadoop.hbase.client.RegionLoadStats; 082import org.apache.hadoop.hbase.client.Result; 083import org.apache.hadoop.hbase.client.Scan; 084import org.apache.hadoop.hbase.client.SnapshotDescription; 085import org.apache.hadoop.hbase.client.SnapshotType; 086import org.apache.hadoop.hbase.client.TableDescriptor; 087import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 088import org.apache.hadoop.hbase.client.metrics.ScanMetrics; 089import org.apache.hadoop.hbase.client.security.SecurityCapability; 090import org.apache.hadoop.hbase.exceptions.DeserializationException; 091import org.apache.hadoop.hbase.filter.ByteArrayComparable; 092import org.apache.hadoop.hbase.filter.Filter; 093import org.apache.hadoop.hbase.io.TimeRange; 094import org.apache.hadoop.hbase.protobuf.ProtobufMagic; 095import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter; 096import org.apache.hadoop.hbase.quotas.QuotaScope; 097import org.apache.hadoop.hbase.quotas.QuotaType; 098import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy; 099import org.apache.hadoop.hbase.quotas.ThrottleType; 100import org.apache.hadoop.hbase.replication.ReplicationLoadSink; 101import org.apache.hadoop.hbase.replication.ReplicationLoadSource; 102import org.apache.hadoop.hbase.security.visibility.Authorizations; 103import org.apache.hadoop.hbase.security.visibility.CellVisibility; 104import org.apache.hadoop.hbase.util.Addressing; 105import org.apache.hadoop.hbase.util.Bytes; 106import org.apache.hadoop.hbase.util.DynamicClassLoader; 107import org.apache.hadoop.hbase.util.ExceptionUtil; 108import org.apache.hadoop.hbase.util.Methods; 109import org.apache.hadoop.hbase.util.VersionInfo; 110import org.apache.hadoop.ipc.RemoteException; 111import org.apache.yetus.audience.InterfaceAudience; 112 113import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 114import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams; 115import org.apache.hbase.thirdparty.com.google.gson.JsonArray; 116import org.apache.hbase.thirdparty.com.google.gson.JsonElement; 117import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; 118import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream; 119import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; 120import org.apache.hbase.thirdparty.com.google.protobuf.Message; 121import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel; 122import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; 123import org.apache.hbase.thirdparty.com.google.protobuf.Service; 124import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; 125import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; 126import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; 127import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; 128import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest; 129import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest; 130import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse; 131import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest; 132import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; 133import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse; 134import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoRequest; 135import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse; 136import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest; 137import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse; 138import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest; 139import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo; 140import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WarmupRegionRequest; 141import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos; 142import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; 143import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.Column; 144import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest; 145import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto; 146import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue; 147import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue; 148import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.DeleteType; 149import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; 150import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; 151import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos; 152import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad; 153import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos; 154import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; 155import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; 156import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair; 157import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema; 158import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameBytesPair; 159import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair; 160import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; 161import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionExceptionMessage; 162import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo; 163import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier; 164import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; 165import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema; 166import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos; 167import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos; 168import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos; 169import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest; 170import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse; 171import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse; 172import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse; 173import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse; 174import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse; 175import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; 176import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos; 177import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; 178import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; 179import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; 180import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; 181import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor; 182import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; 183import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor; 184import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction; 185import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor; 186import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType; 187import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; 188import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; 189 190/** 191 * Protobufs utility. 192 * Be aware that a class named org.apache.hadoop.hbase.protobuf.ProtobufUtil (i.e. no 'shaded' in 193 * the package name) carries a COPY of a subset of this class for non-shaded 194 * users; e.g. Coprocessor Endpoints. If you make change in here, be sure to make change in 195 * the companion class too (not the end of the world, especially if you are adding new functionality 196 * but something to be aware of. 197 * @see ProtobufUtil 198 */ 199// TODO: Generate the non-shaded protobufutil from this one. 200@InterfaceAudience.Private // TODO: some clients (Hive, etc) use this class 201public final class ProtobufUtil { 202 private ProtobufUtil() { 203 } 204 205 /** 206 * Many results are simple: no cell, exists true or false. To save on object creations, 207 * we reuse them across calls. 208 */ 209 private final static Cell[] EMPTY_CELL_ARRAY = new Cell[]{}; 210 private final static Result EMPTY_RESULT = Result.create(EMPTY_CELL_ARRAY); 211 final static Result EMPTY_RESULT_EXISTS_TRUE = Result.create(null, true); 212 final static Result EMPTY_RESULT_EXISTS_FALSE = Result.create(null, false); 213 private final static Result EMPTY_RESULT_STALE = Result.create(EMPTY_CELL_ARRAY, null, true); 214 private final static Result EMPTY_RESULT_EXISTS_TRUE_STALE 215 = Result.create((Cell[])null, true, true); 216 private final static Result EMPTY_RESULT_EXISTS_FALSE_STALE 217 = Result.create((Cell[])null, false, true); 218 219 private final static ClientProtos.Result EMPTY_RESULT_PB; 220 private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_TRUE; 221 private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_FALSE; 222 private final static ClientProtos.Result EMPTY_RESULT_PB_STALE; 223 private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_TRUE_STALE; 224 private final static ClientProtos.Result EMPTY_RESULT_PB_EXISTS_FALSE_STALE; 225 226 227 static { 228 ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder(); 229 230 builder.setExists(true); 231 builder.setAssociatedCellCount(0); 232 EMPTY_RESULT_PB_EXISTS_TRUE = builder.build(); 233 234 builder.setStale(true); 235 EMPTY_RESULT_PB_EXISTS_TRUE_STALE = builder.build(); 236 builder.clear(); 237 238 builder.setExists(false); 239 builder.setAssociatedCellCount(0); 240 EMPTY_RESULT_PB_EXISTS_FALSE = builder.build(); 241 builder.setStale(true); 242 EMPTY_RESULT_PB_EXISTS_FALSE_STALE = builder.build(); 243 244 builder.clear(); 245 builder.setAssociatedCellCount(0); 246 EMPTY_RESULT_PB = builder.build(); 247 builder.setStale(true); 248 EMPTY_RESULT_PB_STALE = builder.build(); 249 } 250 251 private static volatile boolean classLoaderLoaded = false; 252 253 /** 254 * Dynamic class loader to load filter/comparators 255 */ 256 private final static class ClassLoaderHolder { 257 private final static ClassLoader CLASS_LOADER; 258 259 static { 260 ClassLoader parent = ProtobufUtil.class.getClassLoader(); 261 Configuration conf = HBaseConfiguration.create(); 262 CLASS_LOADER = AccessController.doPrivileged((PrivilegedAction<ClassLoader>) 263 () -> new DynamicClassLoader(conf, parent) 264 ); 265 classLoaderLoaded = true; 266 } 267 } 268 269 @VisibleForTesting 270 public static boolean isClassLoaderLoaded() { 271 return classLoaderLoaded; 272 } 273 274 /** 275 * Prepend the passed bytes with four bytes of magic, {@link ProtobufMagic#PB_MAGIC}, 276 * to flag what follows as a protobuf in hbase. Prepend these bytes to all content written to 277 * znodes, etc. 278 * @param bytes Bytes to decorate 279 * @return The passed <code>bytes</code> with magic prepended (Creates a new 280 * byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length. 281 */ 282 public static byte [] prependPBMagic(final byte [] bytes) { 283 return Bytes.add(PB_MAGIC, bytes); 284 } 285 286 /** 287 * @param bytes Bytes to check. 288 * @return True if passed <code>bytes</code> has {@link ProtobufMagic#PB_MAGIC} for a prefix. 289 */ 290 public static boolean isPBMagicPrefix(final byte [] bytes) { 291 return ProtobufMagic.isPBMagicPrefix(bytes); 292 } 293 294 /** 295 * @param bytes Bytes to check. 296 * @param offset offset to start at 297 * @param len length to use 298 * @return True if passed <code>bytes</code> has {@link ProtobufMagic#PB_MAGIC} for a prefix. 299 */ 300 public static boolean isPBMagicPrefix(final byte [] bytes, int offset, int len) { 301 return ProtobufMagic.isPBMagicPrefix(bytes, offset, len); 302 } 303 304 /** 305 * @param bytes bytes to check 306 * @throws DeserializationException if we are missing the pb magic prefix 307 */ 308 public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException { 309 if (!isPBMagicPrefix(bytes)) { 310 String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length); 311 throw new DeserializationException( 312 "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix" + ", bytes: " + bytesPrefix); 313 } 314 } 315 316 /** 317 * @return Length of {@link ProtobufMagic#lengthOfPBMagic()} 318 */ 319 public static int lengthOfPBMagic() { 320 return ProtobufMagic.lengthOfPBMagic(); 321 } 322 323 public static ComparatorProtos.ByteArrayComparable toByteArrayComparable(final byte [] value) { 324 ComparatorProtos.ByteArrayComparable.Builder builder = 325 ComparatorProtos.ByteArrayComparable.newBuilder(); 326 if (value != null) builder.setValue(UnsafeByteOperations.unsafeWrap(value)); 327 return builder.build(); 328 } 329 330 /** 331 * Return the IOException thrown by the remote server wrapped in 332 * ServiceException as cause. 333 * 334 * @param se ServiceException that wraps IO exception thrown by the server 335 * @return Exception wrapped in ServiceException or 336 * a new IOException that wraps the unexpected ServiceException. 337 */ 338 public static IOException getRemoteException(ServiceException se) { 339 return makeIOExceptionOfException(se); 340 } 341 342 /** 343 * Like {@link #getRemoteException(ServiceException)} but more generic, able to handle more than 344 * just {@link ServiceException}. Prefer this method to 345 * {@link #getRemoteException(ServiceException)} because trying to 346 * contain direct protobuf references. 347 * @param e 348 */ 349 public static IOException handleRemoteException(Exception e) { 350 return makeIOExceptionOfException(e); 351 } 352 353 private static IOException makeIOExceptionOfException(Exception e) { 354 Throwable t = e; 355 if (e instanceof ServiceException) { 356 t = e.getCause(); 357 } 358 if (ExceptionUtil.isInterrupt(t)) { 359 return ExceptionUtil.asInterrupt(t); 360 } 361 if (t instanceof RemoteException) { 362 t = ((RemoteException)t).unwrapRemoteException(); 363 } 364 return t instanceof IOException? (IOException)t: new HBaseIOException(t); 365 } 366 367 /** 368 * Convert a ServerName to a protocol buffer ServerName 369 * 370 * @param serverName the ServerName to convert 371 * @return the converted protocol buffer ServerName 372 * @see #toServerName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName) 373 */ 374 public static HBaseProtos.ServerName toServerName(final ServerName serverName) { 375 if (serverName == null) return null; 376 HBaseProtos.ServerName.Builder builder = 377 HBaseProtos.ServerName.newBuilder(); 378 builder.setHostName(serverName.getHostname()); 379 if (serverName.getPort() >= 0) { 380 builder.setPort(serverName.getPort()); 381 } 382 if (serverName.getStartcode() >= 0) { 383 builder.setStartCode(serverName.getStartcode()); 384 } 385 return builder.build(); 386 } 387 388 /** 389 * Convert a protocol buffer ServerName to a ServerName 390 * 391 * @param proto the protocol buffer ServerName to convert 392 * @return the converted ServerName 393 */ 394 public static ServerName toServerName(final HBaseProtos.ServerName proto) { 395 if (proto == null) return null; 396 String hostName = proto.getHostName(); 397 long startCode = -1; 398 int port = -1; 399 if (proto.hasPort()) { 400 port = proto.getPort(); 401 } 402 if (proto.hasStartCode()) { 403 startCode = proto.getStartCode(); 404 } 405 return ServerName.valueOf(hostName, port, startCode); 406 } 407 408 /** 409 * Convert a list of protocol buffer ServerName to a list of ServerName 410 * @param proto protocol buffer ServerNameList 411 * @return a list of ServerName 412 */ 413 public static List<ServerName> toServerNameList( 414 List<HBaseProtos.ServerName> proto) { 415 return proto.stream().map(ProtobufUtil::toServerName) 416 .collect(Collectors.toList()); 417 } 418 419 /** 420 * Get a list of NamespaceDescriptor from ListNamespaceDescriptorsResponse protobuf 421 * @param proto the ListNamespaceDescriptorsResponse 422 * @return a list of NamespaceDescriptor 423 */ 424 public static List<NamespaceDescriptor> toNamespaceDescriptorList( 425 ListNamespaceDescriptorsResponse proto) { 426 return proto.getNamespaceDescriptorList().stream().map(ProtobufUtil::toNamespaceDescriptor) 427 .collect(Collectors.toList()); 428 } 429 430 /** 431 * Get a list of TableDescriptor from GetTableDescriptorsResponse protobuf 432 * @param proto the GetTableDescriptorsResponse 433 * @return a list of TableDescriptor 434 */ 435 public static List<TableDescriptor> toTableDescriptorList(GetTableDescriptorsResponse proto) { 436 if (proto == null) { 437 return new ArrayList<>(); 438 } 439 return proto.getTableSchemaList().stream().map(ProtobufUtil::toTableDescriptor) 440 .collect(Collectors.toList()); 441 } 442 443 /** 444 * Get a list of TableDescriptor from ListTableDescriptorsByNamespaceResponse protobuf 445 * @param proto the ListTableDescriptorsByNamespaceResponse 446 * @return a list of TableDescriptor 447 */ 448 public static List<TableDescriptor> 449 toTableDescriptorList(ListTableDescriptorsByNamespaceResponse proto) { 450 if (proto == null) return new ArrayList<>(); 451 return proto.getTableSchemaList().stream().map(ProtobufUtil::toTableDescriptor) 452 .collect(Collectors.toList()); 453 } 454 455 /** 456 * get the split keys in form "byte [][]" from a CreateTableRequest proto 457 * 458 * @param proto the CreateTableRequest 459 * @return the split keys 460 */ 461 public static byte [][] getSplitKeysArray(final CreateTableRequest proto) { 462 byte [][] splitKeys = new byte[proto.getSplitKeysCount()][]; 463 for (int i = 0; i < proto.getSplitKeysCount(); ++i) { 464 splitKeys[i] = proto.getSplitKeys(i).toByteArray(); 465 } 466 return splitKeys; 467 } 468 469 /** 470 * Convert a protobuf Durability into a client Durability 471 */ 472 public static Durability toDurability( 473 final ClientProtos.MutationProto.Durability proto) { 474 switch(proto) { 475 case USE_DEFAULT: 476 return Durability.USE_DEFAULT; 477 case SKIP_WAL: 478 return Durability.SKIP_WAL; 479 case ASYNC_WAL: 480 return Durability.ASYNC_WAL; 481 case SYNC_WAL: 482 return Durability.SYNC_WAL; 483 case FSYNC_WAL: 484 return Durability.FSYNC_WAL; 485 default: 486 return Durability.USE_DEFAULT; 487 } 488 } 489 490 /** 491 * Convert a client Durability into a protbuf Durability 492 */ 493 public static ClientProtos.MutationProto.Durability toDurability( 494 final Durability d) { 495 switch(d) { 496 case USE_DEFAULT: 497 return ClientProtos.MutationProto.Durability.USE_DEFAULT; 498 case SKIP_WAL: 499 return ClientProtos.MutationProto.Durability.SKIP_WAL; 500 case ASYNC_WAL: 501 return ClientProtos.MutationProto.Durability.ASYNC_WAL; 502 case SYNC_WAL: 503 return ClientProtos.MutationProto.Durability.SYNC_WAL; 504 case FSYNC_WAL: 505 return ClientProtos.MutationProto.Durability.FSYNC_WAL; 506 default: 507 return ClientProtos.MutationProto.Durability.USE_DEFAULT; 508 } 509 } 510 511 /** 512 * Convert a protocol buffer Get to a client Get 513 * 514 * @param proto the protocol buffer Get to convert 515 * @return the converted client Get 516 * @throws IOException 517 */ 518 public static Get toGet(final ClientProtos.Get proto) throws IOException { 519 if (proto == null) return null; 520 byte[] row = proto.getRow().toByteArray(); 521 Get get = new Get(row); 522 if (proto.hasCacheBlocks()) { 523 get.setCacheBlocks(proto.getCacheBlocks()); 524 } 525 if (proto.hasMaxVersions()) { 526 get.setMaxVersions(proto.getMaxVersions()); 527 } 528 if (proto.hasStoreLimit()) { 529 get.setMaxResultsPerColumnFamily(proto.getStoreLimit()); 530 } 531 if (proto.hasStoreOffset()) { 532 get.setRowOffsetPerColumnFamily(proto.getStoreOffset()); 533 } 534 if (proto.getCfTimeRangeCount() > 0) { 535 for (HBaseProtos.ColumnFamilyTimeRange cftr : proto.getCfTimeRangeList()) { 536 TimeRange timeRange = toTimeRange(cftr.getTimeRange()); 537 get.setColumnFamilyTimeRange(cftr.getColumnFamily().toByteArray(), 538 timeRange.getMin(), timeRange.getMax()); 539 } 540 } 541 if (proto.hasTimeRange()) { 542 TimeRange timeRange = toTimeRange(proto.getTimeRange()); 543 get.setTimeRange(timeRange.getMin(), timeRange.getMax()); 544 } 545 if (proto.hasFilter()) { 546 FilterProtos.Filter filter = proto.getFilter(); 547 get.setFilter(ProtobufUtil.toFilter(filter)); 548 } 549 for (NameBytesPair attribute: proto.getAttributeList()) { 550 get.setAttribute(attribute.getName(), attribute.getValue().toByteArray()); 551 } 552 if (proto.getColumnCount() > 0) { 553 for (Column column: proto.getColumnList()) { 554 byte[] family = column.getFamily().toByteArray(); 555 if (column.getQualifierCount() > 0) { 556 for (ByteString qualifier: column.getQualifierList()) { 557 get.addColumn(family, qualifier.toByteArray()); 558 } 559 } else { 560 get.addFamily(family); 561 } 562 } 563 } 564 if (proto.hasExistenceOnly() && proto.getExistenceOnly()){ 565 get.setCheckExistenceOnly(true); 566 } 567 if (proto.hasConsistency()) { 568 get.setConsistency(toConsistency(proto.getConsistency())); 569 } 570 if (proto.hasLoadColumnFamiliesOnDemand()) { 571 get.setLoadColumnFamiliesOnDemand(proto.getLoadColumnFamiliesOnDemand()); 572 } 573 return get; 574 } 575 576 public static Consistency toConsistency(ClientProtos.Consistency consistency) { 577 switch (consistency) { 578 case STRONG : return Consistency.STRONG; 579 case TIMELINE : return Consistency.TIMELINE; 580 default : return Consistency.STRONG; 581 } 582 } 583 584 public static ClientProtos.Consistency toConsistency(Consistency consistency) { 585 switch (consistency) { 586 case STRONG : return ClientProtos.Consistency.STRONG; 587 case TIMELINE : return ClientProtos.Consistency.TIMELINE; 588 default : return ClientProtos.Consistency.STRONG; 589 } 590 } 591 592 /** 593 * Convert a protocol buffer Mutate to a Put. 594 * 595 * @param proto The protocol buffer MutationProto to convert 596 * @return A client Put. 597 * @throws IOException 598 */ 599 public static Put toPut(final MutationProto proto) 600 throws IOException { 601 return toPut(proto, null); 602 } 603 604 /** 605 * Convert a protocol buffer Mutate to a Put. 606 * 607 * @param proto The protocol buffer MutationProto to convert 608 * @param cellScanner If non-null, the Cell data that goes with this proto. 609 * @return A client Put. 610 * @throws IOException 611 */ 612 public static Put toPut(final MutationProto proto, final CellScanner cellScanner) 613 throws IOException { 614 // TODO: Server-side at least why do we convert back to the Client types? Why not just pb it? 615 MutationType type = proto.getMutateType(); 616 assert type == MutationType.PUT: type.name(); 617 long timestamp = proto.hasTimestamp()? proto.getTimestamp(): HConstants.LATEST_TIMESTAMP; 618 Put put = proto.hasRow() ? new Put(proto.getRow().toByteArray(), timestamp) : null; 619 int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0; 620 if (cellCount > 0) { 621 // The proto has metadata only and the data is separate to be found in the cellScanner. 622 if (cellScanner == null) { 623 throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " + 624 toShortString(proto)); 625 } 626 for (int i = 0; i < cellCount; i++) { 627 if (!cellScanner.advance()) { 628 throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i + 629 " no cell returned: " + toShortString(proto)); 630 } 631 Cell cell = cellScanner.current(); 632 if (put == null) { 633 put = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp); 634 } 635 put.add(cell); 636 } 637 } else { 638 if (put == null) { 639 throw new IllegalArgumentException("row cannot be null"); 640 } 641 // The proto has the metadata and the data itself 642 ExtendedCellBuilder cellBuilder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY); 643 for (ColumnValue column: proto.getColumnValueList()) { 644 byte[] family = column.getFamily().toByteArray(); 645 for (QualifierValue qv: column.getQualifierValueList()) { 646 if (!qv.hasValue()) { 647 throw new DoNotRetryIOException( 648 "Missing required field: qualifier value"); 649 } 650 long ts = timestamp; 651 if (qv.hasTimestamp()) { 652 ts = qv.getTimestamp(); 653 } 654 byte[] allTagsBytes; 655 if (qv.hasTags()) { 656 allTagsBytes = qv.getTags().toByteArray(); 657 if(qv.hasDeleteType()) { 658 put.add(cellBuilder.clear() 659 .setRow(proto.getRow().toByteArray()) 660 .setFamily(family) 661 .setQualifier(qv.hasQualifier() ? qv.getQualifier().toByteArray() : null) 662 .setTimestamp(ts) 663 .setType(fromDeleteType(qv.getDeleteType()).getCode()) 664 .setTags(allTagsBytes) 665 .build()); 666 } else { 667 put.add(cellBuilder.clear() 668 .setRow(put.getRow()) 669 .setFamily(family) 670 .setQualifier(qv.hasQualifier() ? qv.getQualifier().toByteArray() : null) 671 .setTimestamp(ts) 672 .setType(Cell.Type.Put) 673 .setValue(qv.hasValue() ? qv.getValue().toByteArray() : null) 674 .setTags(allTagsBytes) 675 .build()); 676 } 677 } else { 678 if(qv.hasDeleteType()) { 679 put.add(cellBuilder.clear() 680 .setRow(put.getRow()) 681 .setFamily(family) 682 .setQualifier(qv.hasQualifier() ? qv.getQualifier().toByteArray() : null) 683 .setTimestamp(ts) 684 .setType(fromDeleteType(qv.getDeleteType()).getCode()) 685 .build()); 686 } else{ 687 put.add(cellBuilder.clear() 688 .setRow(put.getRow()) 689 .setFamily(family) 690 .setQualifier(qv.hasQualifier() ? qv.getQualifier().toByteArray() : null) 691 .setTimestamp(ts) 692 .setType(Type.Put) 693 .setValue(qv.hasValue() ? qv.getValue().toByteArray() : null) 694 .build()); 695 } 696 } 697 } 698 } 699 } 700 put.setDurability(toDurability(proto.getDurability())); 701 for (NameBytesPair attribute: proto.getAttributeList()) { 702 put.setAttribute(attribute.getName(), attribute.getValue().toByteArray()); 703 } 704 return put; 705 } 706 707 /** 708 * Convert a protocol buffer Mutate to a Delete 709 * 710 * @param proto the protocol buffer Mutate to convert 711 * @return the converted client Delete 712 * @throws IOException 713 */ 714 public static Delete toDelete(final MutationProto proto) 715 throws IOException { 716 return toDelete(proto, null); 717 } 718 719 /** 720 * Convert a protocol buffer Mutate to a Delete 721 * 722 * @param proto the protocol buffer Mutate to convert 723 * @param cellScanner if non-null, the data that goes with this delete. 724 * @return the converted client Delete 725 * @throws IOException 726 */ 727 public static Delete toDelete(final MutationProto proto, final CellScanner cellScanner) 728 throws IOException { 729 MutationType type = proto.getMutateType(); 730 assert type == MutationType.DELETE : type.name(); 731 long timestamp = proto.hasTimestamp() ? proto.getTimestamp() : HConstants.LATEST_TIMESTAMP; 732 Delete delete = proto.hasRow() ? new Delete(proto.getRow().toByteArray(), timestamp) : null; 733 int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0; 734 if (cellCount > 0) { 735 // The proto has metadata only and the data is separate to be found in the cellScanner. 736 if (cellScanner == null) { 737 // TextFormat should be fine for a Delete since it carries no data, just coordinates. 738 throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " + 739 TextFormat.shortDebugString(proto)); 740 } 741 for (int i = 0; i < cellCount; i++) { 742 if (!cellScanner.advance()) { 743 // TextFormat should be fine for a Delete since it carries no data, just coordinates. 744 throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i + 745 " no cell returned: " + TextFormat.shortDebugString(proto)); 746 } 747 Cell cell = cellScanner.current(); 748 if (delete == null) { 749 delete = 750 new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp); 751 } 752 delete.add(cell); 753 } 754 } else { 755 if (delete == null) { 756 throw new IllegalArgumentException("row cannot be null"); 757 } 758 for (ColumnValue column: proto.getColumnValueList()) { 759 byte[] family = column.getFamily().toByteArray(); 760 for (QualifierValue qv: column.getQualifierValueList()) { 761 DeleteType deleteType = qv.getDeleteType(); 762 byte[] qualifier = null; 763 if (qv.hasQualifier()) { 764 qualifier = qv.getQualifier().toByteArray(); 765 } 766 long ts = HConstants.LATEST_TIMESTAMP; 767 if (qv.hasTimestamp()) { 768 ts = qv.getTimestamp(); 769 } 770 if (deleteType == DeleteType.DELETE_ONE_VERSION) { 771 delete.addColumn(family, qualifier, ts); 772 } else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) { 773 delete.addColumns(family, qualifier, ts); 774 } else if (deleteType == DeleteType.DELETE_FAMILY_VERSION) { 775 delete.addFamilyVersion(family, ts); 776 } else { 777 delete.addFamily(family, ts); 778 } 779 } 780 } 781 } 782 delete.setDurability(toDurability(proto.getDurability())); 783 for (NameBytesPair attribute: proto.getAttributeList()) { 784 delete.setAttribute(attribute.getName(), attribute.getValue().toByteArray()); 785 } 786 return delete; 787 } 788 @FunctionalInterface 789 private interface ConsumerWithException <T, U> { 790 void accept(T t, U u) throws IOException; 791 } 792 793 private static <T extends Mutation> T toDelta(Function<Bytes, T> supplier, ConsumerWithException<T, Cell> consumer, 794 final MutationProto proto, final CellScanner cellScanner) throws IOException { 795 byte[] row = proto.hasRow() ? proto.getRow().toByteArray() : null; 796 T mutation = row == null ? null : supplier.apply(new Bytes(row)); 797 int cellCount = proto.hasAssociatedCellCount() ? proto.getAssociatedCellCount() : 0; 798 if (cellCount > 0) { 799 // The proto has metadata only and the data is separate to be found in the cellScanner. 800 if (cellScanner == null) { 801 throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " + 802 toShortString(proto)); 803 } 804 for (int i = 0; i < cellCount; i++) { 805 if (!cellScanner.advance()) { 806 throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i + 807 " no cell returned: " + toShortString(proto)); 808 } 809 Cell cell = cellScanner.current(); 810 if (mutation == null) { 811 mutation = supplier.apply(new Bytes(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())); 812 } 813 consumer.accept(mutation, cell); 814 } 815 } else { 816 if (mutation == null) { 817 throw new IllegalArgumentException("row cannot be null"); 818 } 819 for (ColumnValue column : proto.getColumnValueList()) { 820 byte[] family = column.getFamily().toByteArray(); 821 for (QualifierValue qv : column.getQualifierValueList()) { 822 byte[] qualifier = qv.getQualifier().toByteArray(); 823 if (!qv.hasValue()) { 824 throw new DoNotRetryIOException( 825 "Missing required field: qualifier value"); 826 } 827 byte[] value = qv.getValue().toByteArray(); 828 byte[] tags = null; 829 if (qv.hasTags()) { 830 tags = qv.getTags().toByteArray(); 831 } 832 consumer.accept(mutation, ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) 833 .setRow(mutation.getRow()) 834 .setFamily(family) 835 .setQualifier(qualifier) 836 .setTimestamp(qv.getTimestamp()) 837 .setType(KeyValue.Type.Put.getCode()) 838 .setValue(value) 839 .setTags(tags) 840 .build()); 841 } 842 } 843 } 844 mutation.setDurability(toDurability(proto.getDurability())); 845 for (NameBytesPair attribute : proto.getAttributeList()) { 846 mutation.setAttribute(attribute.getName(), attribute.getValue().toByteArray()); 847 } 848 return mutation; 849 } 850 851 /** 852 * Convert a protocol buffer Mutate to an Append 853 * @param cellScanner 854 * @param proto the protocol buffer Mutate to convert 855 * @return the converted client Append 856 * @throws IOException 857 */ 858 public static Append toAppend(final MutationProto proto, final CellScanner cellScanner) 859 throws IOException { 860 MutationType type = proto.getMutateType(); 861 assert type == MutationType.APPEND : type.name(); 862 Append append = toDelta((Bytes row) -> new Append(row.get(), row.getOffset(), row.getLength()), 863 Append::add, proto, cellScanner); 864 if (proto.hasTimeRange()) { 865 TimeRange timeRange = toTimeRange(proto.getTimeRange()); 866 append.setTimeRange(timeRange.getMin(), timeRange.getMax()); 867 } 868 return append; 869 } 870 871 /** 872 * Convert a protocol buffer Mutate to an Increment 873 * 874 * @param proto the protocol buffer Mutate to convert 875 * @return the converted client Increment 876 * @throws IOException 877 */ 878 public static Increment toIncrement(final MutationProto proto, final CellScanner cellScanner) 879 throws IOException { 880 MutationType type = proto.getMutateType(); 881 assert type == MutationType.INCREMENT : type.name(); 882 Increment increment = toDelta((Bytes row) -> new Increment(row.get(), row.getOffset(), row.getLength()), 883 Increment::add, proto, cellScanner); 884 if (proto.hasTimeRange()) { 885 TimeRange timeRange = toTimeRange(proto.getTimeRange()); 886 increment.setTimeRange(timeRange.getMin(), timeRange.getMax()); 887 } 888 return increment; 889 } 890 891 /** 892 * Convert a MutateRequest to Mutation 893 * 894 * @param proto the protocol buffer Mutate to convert 895 * @return the converted Mutation 896 * @throws IOException 897 */ 898 public static Mutation toMutation(final MutationProto proto) throws IOException { 899 MutationType type = proto.getMutateType(); 900 if (type == MutationType.APPEND) { 901 return toAppend(proto, null); 902 } 903 if (type == MutationType.DELETE) { 904 return toDelete(proto, null); 905 } 906 if (type == MutationType.PUT) { 907 return toPut(proto, null); 908 } 909 throw new IOException("Unknown mutation type " + type); 910 } 911 912 /** 913 * Convert a protocol buffer Mutate to a Get. 914 * @param proto the protocol buffer Mutate to convert. 915 * @param cellScanner 916 * @return the converted client get. 917 * @throws IOException 918 */ 919 public static Get toGet(final MutationProto proto, final CellScanner cellScanner) 920 throws IOException { 921 MutationType type = proto.getMutateType(); 922 assert type == MutationType.INCREMENT || type == MutationType.APPEND : type.name(); 923 byte[] row = proto.hasRow() ? proto.getRow().toByteArray() : null; 924 Get get = null; 925 int cellCount = proto.hasAssociatedCellCount() ? proto.getAssociatedCellCount() : 0; 926 if (cellCount > 0) { 927 // The proto has metadata only and the data is separate to be found in the cellScanner. 928 if (cellScanner == null) { 929 throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " 930 + TextFormat.shortDebugString(proto)); 931 } 932 for (int i = 0; i < cellCount; i++) { 933 if (!cellScanner.advance()) { 934 throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i 935 + " no cell returned: " + TextFormat.shortDebugString(proto)); 936 } 937 Cell cell = cellScanner.current(); 938 if (get == null) { 939 get = new Get(CellUtil.cloneRow(cell)); 940 } 941 get.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell)); 942 } 943 } else { 944 get = new Get(row); 945 for (ColumnValue column : proto.getColumnValueList()) { 946 byte[] family = column.getFamily().toByteArray(); 947 for (QualifierValue qv : column.getQualifierValueList()) { 948 byte[] qualifier = qv.getQualifier().toByteArray(); 949 if (!qv.hasValue()) { 950 throw new DoNotRetryIOException("Missing required field: qualifier value"); 951 } 952 get.addColumn(family, qualifier); 953 } 954 } 955 } 956 if (proto.hasTimeRange()) { 957 TimeRange timeRange = toTimeRange(proto.getTimeRange()); 958 get.setTimeRange(timeRange.getMin(), timeRange.getMax()); 959 } 960 for (NameBytesPair attribute : proto.getAttributeList()) { 961 get.setAttribute(attribute.getName(), attribute.getValue().toByteArray()); 962 } 963 return get; 964 } 965 966 public static ClientProtos.Scan.ReadType toReadType(Scan.ReadType readType) { 967 switch (readType) { 968 case DEFAULT: 969 return ClientProtos.Scan.ReadType.DEFAULT; 970 case STREAM: 971 return ClientProtos.Scan.ReadType.STREAM; 972 case PREAD: 973 return ClientProtos.Scan.ReadType.PREAD; 974 default: 975 throw new IllegalArgumentException("Unknown ReadType: " + readType); 976 } 977 } 978 979 public static Scan.ReadType toReadType(ClientProtos.Scan.ReadType readType) { 980 switch (readType) { 981 case DEFAULT: 982 return Scan.ReadType.DEFAULT; 983 case STREAM: 984 return Scan.ReadType.STREAM; 985 case PREAD: 986 return Scan.ReadType.PREAD; 987 default: 988 throw new IllegalArgumentException("Unknown ReadType: " + readType); 989 } 990 } 991 992 /** 993 * Convert a client Scan to a protocol buffer Scan 994 * 995 * @param scan the client Scan to convert 996 * @return the converted protocol buffer Scan 997 * @throws IOException 998 */ 999 public static ClientProtos.Scan toScan( 1000 final Scan scan) throws IOException { 1001 ClientProtos.Scan.Builder scanBuilder = 1002 ClientProtos.Scan.newBuilder(); 1003 scanBuilder.setCacheBlocks(scan.getCacheBlocks()); 1004 if (scan.getBatch() > 0) { 1005 scanBuilder.setBatchSize(scan.getBatch()); 1006 } 1007 if (scan.getMaxResultSize() > 0) { 1008 scanBuilder.setMaxResultSize(scan.getMaxResultSize()); 1009 } 1010 if (scan.isSmall()) { 1011 scanBuilder.setSmall(scan.isSmall()); 1012 } 1013 if (scan.getAllowPartialResults()) { 1014 scanBuilder.setAllowPartialResults(scan.getAllowPartialResults()); 1015 } 1016 Boolean loadColumnFamiliesOnDemand = scan.getLoadColumnFamiliesOnDemandValue(); 1017 if (loadColumnFamiliesOnDemand != null) { 1018 scanBuilder.setLoadColumnFamiliesOnDemand(loadColumnFamiliesOnDemand); 1019 } 1020 scanBuilder.setMaxVersions(scan.getMaxVersions()); 1021 scan.getColumnFamilyTimeRange().forEach((cf, timeRange) -> { 1022 scanBuilder.addCfTimeRange(HBaseProtos.ColumnFamilyTimeRange.newBuilder() 1023 .setColumnFamily(UnsafeByteOperations.unsafeWrap(cf)) 1024 .setTimeRange(toTimeRange(timeRange)) 1025 .build()); 1026 }); 1027 scanBuilder.setTimeRange(ProtobufUtil.toTimeRange(scan.getTimeRange())); 1028 Map<String, byte[]> attributes = scan.getAttributesMap(); 1029 if (!attributes.isEmpty()) { 1030 NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder(); 1031 for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) { 1032 attributeBuilder.setName(attribute.getKey()); 1033 attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue())); 1034 scanBuilder.addAttribute(attributeBuilder.build()); 1035 } 1036 } 1037 byte[] startRow = scan.getStartRow(); 1038 if (startRow != null && startRow.length > 0) { 1039 scanBuilder.setStartRow(UnsafeByteOperations.unsafeWrap(startRow)); 1040 } 1041 byte[] stopRow = scan.getStopRow(); 1042 if (stopRow != null && stopRow.length > 0) { 1043 scanBuilder.setStopRow(UnsafeByteOperations.unsafeWrap(stopRow)); 1044 } 1045 if (scan.hasFilter()) { 1046 scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter())); 1047 } 1048 if (scan.hasFamilies()) { 1049 Column.Builder columnBuilder = Column.newBuilder(); 1050 for (Map.Entry<byte[],NavigableSet<byte []>> 1051 family: scan.getFamilyMap().entrySet()) { 1052 columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey())); 1053 NavigableSet<byte []> qualifiers = family.getValue(); 1054 columnBuilder.clearQualifier(); 1055 if (qualifiers != null && qualifiers.size() > 0) { 1056 for (byte [] qualifier: qualifiers) { 1057 columnBuilder.addQualifier(UnsafeByteOperations.unsafeWrap(qualifier)); 1058 } 1059 } 1060 scanBuilder.addColumn(columnBuilder.build()); 1061 } 1062 } 1063 if (scan.getMaxResultsPerColumnFamily() >= 0) { 1064 scanBuilder.setStoreLimit(scan.getMaxResultsPerColumnFamily()); 1065 } 1066 if (scan.getRowOffsetPerColumnFamily() > 0) { 1067 scanBuilder.setStoreOffset(scan.getRowOffsetPerColumnFamily()); 1068 } 1069 if (scan.isReversed()) { 1070 scanBuilder.setReversed(scan.isReversed()); 1071 } 1072 if (scan.getConsistency() == Consistency.TIMELINE) { 1073 scanBuilder.setConsistency(toConsistency(scan.getConsistency())); 1074 } 1075 if (scan.getCaching() > 0) { 1076 scanBuilder.setCaching(scan.getCaching()); 1077 } 1078 long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan); 1079 if (mvccReadPoint > 0) { 1080 scanBuilder.setMvccReadPoint(mvccReadPoint); 1081 } 1082 if (!scan.includeStartRow()) { 1083 scanBuilder.setIncludeStartRow(false); 1084 } 1085 scanBuilder.setIncludeStopRow(scan.includeStopRow()); 1086 if (scan.getReadType() != Scan.ReadType.DEFAULT) { 1087 scanBuilder.setReadType(toReadType(scan.getReadType())); 1088 } 1089 if (scan.isNeedCursorResult()) { 1090 scanBuilder.setNeedCursorResult(true); 1091 } 1092 return scanBuilder.build(); 1093 } 1094 1095 /** 1096 * Convert a protocol buffer Scan to a client Scan 1097 * 1098 * @param proto the protocol buffer Scan to convert 1099 * @return the converted client Scan 1100 * @throws IOException 1101 */ 1102 public static Scan toScan( 1103 final ClientProtos.Scan proto) throws IOException { 1104 byte[] startRow = HConstants.EMPTY_START_ROW; 1105 byte[] stopRow = HConstants.EMPTY_END_ROW; 1106 boolean includeStartRow = true; 1107 boolean includeStopRow = false; 1108 if (proto.hasStartRow()) { 1109 startRow = proto.getStartRow().toByteArray(); 1110 } 1111 if (proto.hasStopRow()) { 1112 stopRow = proto.getStopRow().toByteArray(); 1113 } 1114 if (proto.hasIncludeStartRow()) { 1115 includeStartRow = proto.getIncludeStartRow(); 1116 } 1117 if (proto.hasIncludeStopRow()) { 1118 includeStopRow = proto.getIncludeStopRow(); 1119 } else { 1120 // old client without this flag, we should consider start=end as a get. 1121 if (ClientUtil.areScanStartRowAndStopRowEqual(startRow, stopRow)) { 1122 includeStopRow = true; 1123 } 1124 } 1125 Scan scan = 1126 new Scan().withStartRow(startRow, includeStartRow).withStopRow(stopRow, includeStopRow); 1127 if (proto.hasCacheBlocks()) { 1128 scan.setCacheBlocks(proto.getCacheBlocks()); 1129 } 1130 if (proto.hasMaxVersions()) { 1131 scan.setMaxVersions(proto.getMaxVersions()); 1132 } 1133 if (proto.hasStoreLimit()) { 1134 scan.setMaxResultsPerColumnFamily(proto.getStoreLimit()); 1135 } 1136 if (proto.hasStoreOffset()) { 1137 scan.setRowOffsetPerColumnFamily(proto.getStoreOffset()); 1138 } 1139 if (proto.hasLoadColumnFamiliesOnDemand()) { 1140 scan.setLoadColumnFamiliesOnDemand(proto.getLoadColumnFamiliesOnDemand()); 1141 } 1142 if (proto.getCfTimeRangeCount() > 0) { 1143 for (HBaseProtos.ColumnFamilyTimeRange cftr : proto.getCfTimeRangeList()) { 1144 TimeRange timeRange = toTimeRange(cftr.getTimeRange()); 1145 scan.setColumnFamilyTimeRange(cftr.getColumnFamily().toByteArray(), 1146 timeRange.getMin(), timeRange.getMax()); 1147 } 1148 } 1149 if (proto.hasTimeRange()) { 1150 TimeRange timeRange = toTimeRange(proto.getTimeRange()); 1151 scan.setTimeRange(timeRange.getMin(), timeRange.getMax()); 1152 } 1153 if (proto.hasFilter()) { 1154 FilterProtos.Filter filter = proto.getFilter(); 1155 scan.setFilter(ProtobufUtil.toFilter(filter)); 1156 } 1157 if (proto.hasBatchSize()) { 1158 scan.setBatch(proto.getBatchSize()); 1159 } 1160 if (proto.hasMaxResultSize()) { 1161 scan.setMaxResultSize(proto.getMaxResultSize()); 1162 } 1163 if (proto.hasSmall()) { 1164 scan.setSmall(proto.getSmall()); 1165 } 1166 if (proto.hasAllowPartialResults()) { 1167 scan.setAllowPartialResults(proto.getAllowPartialResults()); 1168 } 1169 for (NameBytesPair attribute: proto.getAttributeList()) { 1170 scan.setAttribute(attribute.getName(), attribute.getValue().toByteArray()); 1171 } 1172 if (proto.getColumnCount() > 0) { 1173 for (Column column: proto.getColumnList()) { 1174 byte[] family = column.getFamily().toByteArray(); 1175 if (column.getQualifierCount() > 0) { 1176 for (ByteString qualifier: column.getQualifierList()) { 1177 scan.addColumn(family, qualifier.toByteArray()); 1178 } 1179 } else { 1180 scan.addFamily(family); 1181 } 1182 } 1183 } 1184 if (proto.hasReversed()) { 1185 scan.setReversed(proto.getReversed()); 1186 } 1187 if (proto.hasConsistency()) { 1188 scan.setConsistency(toConsistency(proto.getConsistency())); 1189 } 1190 if (proto.hasCaching()) { 1191 scan.setCaching(proto.getCaching()); 1192 } 1193 if (proto.hasMvccReadPoint()) { 1194 PackagePrivateFieldAccessor.setMvccReadPoint(scan, proto.getMvccReadPoint()); 1195 } 1196 if (scan.isSmall()) { 1197 scan.setReadType(Scan.ReadType.PREAD); 1198 } else if (proto.hasReadType()) { 1199 scan.setReadType(toReadType(proto.getReadType())); 1200 } 1201 if (proto.getNeedCursorResult()) { 1202 scan.setNeedCursorResult(true); 1203 } 1204 return scan; 1205 } 1206 1207 public static ClientProtos.Cursor toCursor(Cursor cursor) { 1208 ClientProtos.Cursor.Builder builder = ClientProtos.Cursor.newBuilder(); 1209 ClientProtos.Cursor.newBuilder().setRow(ByteString.copyFrom(cursor.getRow())); 1210 return builder.build(); 1211 } 1212 1213 public static ClientProtos.Cursor toCursor(Cell cell) { 1214 return ClientProtos.Cursor.newBuilder() 1215 .setRow(ByteString.copyFrom(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) 1216 .build(); 1217 } 1218 1219 public static Cursor toCursor(ClientProtos.Cursor cursor) { 1220 return ClientUtil.createCursor(cursor.getRow().toByteArray()); 1221 } 1222 1223 /** 1224 * Create a protocol buffer Get based on a client Get. 1225 * 1226 * @param get the client Get 1227 * @return a protocol buffer Get 1228 * @throws IOException 1229 */ 1230 public static ClientProtos.Get toGet( 1231 final Get get) throws IOException { 1232 ClientProtos.Get.Builder builder = 1233 ClientProtos.Get.newBuilder(); 1234 builder.setRow(UnsafeByteOperations.unsafeWrap(get.getRow())); 1235 builder.setCacheBlocks(get.getCacheBlocks()); 1236 builder.setMaxVersions(get.getMaxVersions()); 1237 if (get.getFilter() != null) { 1238 builder.setFilter(ProtobufUtil.toFilter(get.getFilter())); 1239 } 1240 get.getColumnFamilyTimeRange().forEach((cf, timeRange) -> { 1241 builder.addCfTimeRange(HBaseProtos.ColumnFamilyTimeRange.newBuilder() 1242 .setColumnFamily(UnsafeByteOperations.unsafeWrap(cf)) 1243 .setTimeRange(toTimeRange(timeRange)) 1244 .build()); 1245 }); 1246 builder.setTimeRange(ProtobufUtil.toTimeRange(get.getTimeRange())); 1247 Map<String, byte[]> attributes = get.getAttributesMap(); 1248 if (!attributes.isEmpty()) { 1249 NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder(); 1250 for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) { 1251 attributeBuilder.setName(attribute.getKey()); 1252 attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue())); 1253 builder.addAttribute(attributeBuilder.build()); 1254 } 1255 } 1256 if (get.hasFamilies()) { 1257 Column.Builder columnBuilder = Column.newBuilder(); 1258 Map<byte[], NavigableSet<byte[]>> families = get.getFamilyMap(); 1259 for (Map.Entry<byte[], NavigableSet<byte[]>> family: families.entrySet()) { 1260 NavigableSet<byte[]> qualifiers = family.getValue(); 1261 columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey())); 1262 columnBuilder.clearQualifier(); 1263 if (qualifiers != null && qualifiers.size() > 0) { 1264 for (byte[] qualifier: qualifiers) { 1265 columnBuilder.addQualifier(UnsafeByteOperations.unsafeWrap(qualifier)); 1266 } 1267 } 1268 builder.addColumn(columnBuilder.build()); 1269 } 1270 } 1271 if (get.getMaxResultsPerColumnFamily() >= 0) { 1272 builder.setStoreLimit(get.getMaxResultsPerColumnFamily()); 1273 } 1274 if (get.getRowOffsetPerColumnFamily() > 0) { 1275 builder.setStoreOffset(get.getRowOffsetPerColumnFamily()); 1276 } 1277 if (get.isCheckExistenceOnly()){ 1278 builder.setExistenceOnly(true); 1279 } 1280 if (get.getConsistency() != null && get.getConsistency() != Consistency.STRONG) { 1281 builder.setConsistency(toConsistency(get.getConsistency())); 1282 } 1283 1284 Boolean loadColumnFamiliesOnDemand = get.getLoadColumnFamiliesOnDemandValue(); 1285 if (loadColumnFamiliesOnDemand != null) { 1286 builder.setLoadColumnFamiliesOnDemand(loadColumnFamiliesOnDemand); 1287 } 1288 return builder.build(); 1289 } 1290 1291 public static MutationProto toMutation(final MutationType type, final Mutation mutation) 1292 throws IOException { 1293 return toMutation(type, mutation, HConstants.NO_NONCE); 1294 } 1295 1296 /** 1297 * Create a protocol buffer Mutate based on a client Mutation 1298 * 1299 * @param type 1300 * @param mutation 1301 * @return a protobuf'd Mutation 1302 * @throws IOException 1303 */ 1304 public static MutationProto toMutation(final MutationType type, final Mutation mutation, 1305 final long nonce) throws IOException { 1306 return toMutation(type, mutation, MutationProto.newBuilder(), nonce); 1307 } 1308 1309 public static MutationProto toMutation(final MutationType type, final Mutation mutation, 1310 MutationProto.Builder builder) throws IOException { 1311 return toMutation(type, mutation, builder, HConstants.NO_NONCE); 1312 } 1313 1314 public static MutationProto toMutation(final MutationType type, final Mutation mutation, 1315 MutationProto.Builder builder, long nonce) 1316 throws IOException { 1317 builder = getMutationBuilderAndSetCommonFields(type, mutation, builder); 1318 if (nonce != HConstants.NO_NONCE) { 1319 builder.setNonce(nonce); 1320 } 1321 if (type == MutationType.INCREMENT) { 1322 builder.setTimeRange(ProtobufUtil.toTimeRange(((Increment) mutation).getTimeRange())); 1323 } 1324 if (type == MutationType.APPEND) { 1325 builder.setTimeRange(ProtobufUtil.toTimeRange(((Append) mutation).getTimeRange())); 1326 } 1327 ColumnValue.Builder columnBuilder = ColumnValue.newBuilder(); 1328 QualifierValue.Builder valueBuilder = QualifierValue.newBuilder(); 1329 for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) { 1330 columnBuilder.clear(); 1331 columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family.getKey())); 1332 for (Cell cell: family.getValue()) { 1333 valueBuilder.clear(); 1334 valueBuilder.setQualifier(UnsafeByteOperations.unsafeWrap( 1335 cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength())); 1336 valueBuilder.setValue(UnsafeByteOperations.unsafeWrap( 1337 cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())); 1338 valueBuilder.setTimestamp(cell.getTimestamp()); 1339 if (type == MutationType.DELETE || (type == MutationType.PUT && CellUtil.isDelete(cell))) { 1340 KeyValue.Type keyValueType = KeyValue.Type.codeToType(cell.getTypeByte()); 1341 valueBuilder.setDeleteType(toDeleteType(keyValueType)); 1342 } 1343 columnBuilder.addQualifierValue(valueBuilder.build()); 1344 } 1345 builder.addColumnValue(columnBuilder.build()); 1346 } 1347 return builder.build(); 1348 } 1349 1350 /** 1351 * Create a protocol buffer MutationProto based on a client Mutation. Does NOT include data. 1352 * Understanding is that the Cell will be transported other than via protobuf. 1353 * @param type 1354 * @param mutation 1355 * @param builder 1356 * @return a protobuf'd Mutation 1357 * @throws IOException 1358 */ 1359 public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation, 1360 final MutationProto.Builder builder) throws IOException { 1361 return toMutationNoData(type, mutation, builder, HConstants.NO_NONCE); 1362 } 1363 1364 /** 1365 * Create a protocol buffer MutationProto based on a client Mutation. Does NOT include data. 1366 * Understanding is that the Cell will be transported other than via protobuf. 1367 * @param type 1368 * @param mutation 1369 * @return a protobuf'd Mutation 1370 * @throws IOException 1371 */ 1372 public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation) 1373 throws IOException { 1374 MutationProto.Builder builder = MutationProto.newBuilder(); 1375 return toMutationNoData(type, mutation, builder); 1376 } 1377 1378 public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation, 1379 final MutationProto.Builder builder, long nonce) throws IOException { 1380 getMutationBuilderAndSetCommonFields(type, mutation, builder); 1381 builder.setAssociatedCellCount(mutation.size()); 1382 if (mutation instanceof Increment) { 1383 builder.setTimeRange(ProtobufUtil.toTimeRange(((Increment) mutation).getTimeRange())); 1384 } 1385 if (mutation instanceof Append) { 1386 builder.setTimeRange(ProtobufUtil.toTimeRange(((Append) mutation).getTimeRange())); 1387 } 1388 if (nonce != HConstants.NO_NONCE) { 1389 builder.setNonce(nonce); 1390 } 1391 return builder.build(); 1392 } 1393 1394 /** 1395 * Code shared by {@link #toMutation(MutationType, Mutation)} and 1396 * {@link #toMutationNoData(MutationType, Mutation)} 1397 * @param type 1398 * @param mutation 1399 * @return A partly-filled out protobuf'd Mutation. 1400 */ 1401 private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type, 1402 final Mutation mutation, MutationProto.Builder builder) { 1403 builder.setRow(UnsafeByteOperations.unsafeWrap(mutation.getRow())); 1404 builder.setMutateType(type); 1405 builder.setDurability(toDurability(mutation.getDurability())); 1406 builder.setTimestamp(mutation.getTimestamp()); 1407 Map<String, byte[]> attributes = mutation.getAttributesMap(); 1408 if (!attributes.isEmpty()) { 1409 NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder(); 1410 for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) { 1411 attributeBuilder.setName(attribute.getKey()); 1412 attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue())); 1413 builder.addAttribute(attributeBuilder.build()); 1414 } 1415 } 1416 return builder; 1417 } 1418 1419 /** 1420 * Convert a client Result to a protocol buffer Result 1421 * 1422 * @param result the client Result to convert 1423 * @return the converted protocol buffer Result 1424 */ 1425 public static ClientProtos.Result toResult(final Result result) { 1426 if (result.getExists() != null) { 1427 return toResult(result.getExists(), result.isStale()); 1428 } 1429 1430 Cell[] cells = result.rawCells(); 1431 if (cells == null || cells.length == 0) { 1432 return result.isStale() ? EMPTY_RESULT_PB_STALE : EMPTY_RESULT_PB; 1433 } 1434 1435 ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder(); 1436 for (Cell c : cells) { 1437 builder.addCell(toCell(c)); 1438 } 1439 1440 builder.setStale(result.isStale()); 1441 builder.setPartial(result.mayHaveMoreCellsInRow()); 1442 1443 return builder.build(); 1444 } 1445 1446 /** 1447 * Convert a client Result to a protocol buffer Result 1448 * 1449 * @param existence the client existence to send 1450 * @return the converted protocol buffer Result 1451 */ 1452 public static ClientProtos.Result toResult(final boolean existence, boolean stale) { 1453 if (stale){ 1454 return existence ? EMPTY_RESULT_PB_EXISTS_TRUE_STALE : EMPTY_RESULT_PB_EXISTS_FALSE_STALE; 1455 } else { 1456 return existence ? EMPTY_RESULT_PB_EXISTS_TRUE : EMPTY_RESULT_PB_EXISTS_FALSE; 1457 } 1458 } 1459 1460 /** 1461 * Convert a client Result to a protocol buffer Result. 1462 * The pb Result does not include the Cell data. That is for transport otherwise. 1463 * 1464 * @param result the client Result to convert 1465 * @return the converted protocol buffer Result 1466 */ 1467 public static ClientProtos.Result toResultNoData(final Result result) { 1468 if (result.getExists() != null) return toResult(result.getExists(), result.isStale()); 1469 int size = result.size(); 1470 if (size == 0) return result.isStale() ? EMPTY_RESULT_PB_STALE : EMPTY_RESULT_PB; 1471 ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder(); 1472 builder.setAssociatedCellCount(size); 1473 builder.setStale(result.isStale()); 1474 return builder.build(); 1475 } 1476 1477 /** 1478 * Convert a protocol buffer Result to a client Result 1479 * 1480 * @param proto the protocol buffer Result to convert 1481 * @return the converted client Result 1482 */ 1483 public static Result toResult(final ClientProtos.Result proto) { 1484 if (proto.hasExists()) { 1485 if (proto.getStale()) { 1486 return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE_STALE :EMPTY_RESULT_EXISTS_FALSE_STALE; 1487 } 1488 return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE : EMPTY_RESULT_EXISTS_FALSE; 1489 } 1490 1491 List<CellProtos.Cell> values = proto.getCellList(); 1492 if (values.isEmpty()){ 1493 return proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT; 1494 } 1495 1496 List<Cell> cells = new ArrayList<>(values.size()); 1497 ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY); 1498 for (CellProtos.Cell c : values) { 1499 cells.add(toCell(builder, c)); 1500 } 1501 return Result.create(cells, null, proto.getStale(), proto.getPartial()); 1502 } 1503 1504 /** 1505 * Convert a protocol buffer Result to a client Result 1506 * 1507 * @param proto the protocol buffer Result to convert 1508 * @param scanner Optional cell scanner. 1509 * @return the converted client Result 1510 * @throws IOException 1511 */ 1512 public static Result toResult(final ClientProtos.Result proto, final CellScanner scanner) 1513 throws IOException { 1514 List<CellProtos.Cell> values = proto.getCellList(); 1515 1516 if (proto.hasExists()) { 1517 if ((values != null && !values.isEmpty()) || 1518 (proto.hasAssociatedCellCount() && proto.getAssociatedCellCount() > 0)) { 1519 throw new IllegalArgumentException("bad proto: exists with cells is no allowed " + proto); 1520 } 1521 if (proto.getStale()) { 1522 return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE_STALE :EMPTY_RESULT_EXISTS_FALSE_STALE; 1523 } 1524 return proto.getExists() ? EMPTY_RESULT_EXISTS_TRUE : EMPTY_RESULT_EXISTS_FALSE; 1525 } 1526 1527 // TODO: Unit test that has some Cells in scanner and some in the proto. 1528 List<Cell> cells = null; 1529 if (proto.hasAssociatedCellCount()) { 1530 int count = proto.getAssociatedCellCount(); 1531 cells = new ArrayList<>(count + values.size()); 1532 for (int i = 0; i < count; i++) { 1533 if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count); 1534 cells.add(scanner.current()); 1535 } 1536 } 1537 1538 if (!values.isEmpty()){ 1539 if (cells == null) cells = new ArrayList<>(values.size()); 1540 ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY); 1541 for (CellProtos.Cell c: values) { 1542 cells.add(toCell(builder, c)); 1543 } 1544 } 1545 1546 return (cells == null || cells.isEmpty()) 1547 ? (proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT) 1548 : Result.create(cells, null, proto.getStale()); 1549 } 1550 1551 1552 /** 1553 * Convert a ByteArrayComparable to a protocol buffer Comparator 1554 * 1555 * @param comparator the ByteArrayComparable to convert 1556 * @return the converted protocol buffer Comparator 1557 */ 1558 public static ComparatorProtos.Comparator toComparator(ByteArrayComparable comparator) { 1559 ComparatorProtos.Comparator.Builder builder = ComparatorProtos.Comparator.newBuilder(); 1560 builder.setName(comparator.getClass().getName()); 1561 builder.setSerializedComparator(UnsafeByteOperations.unsafeWrap(comparator.toByteArray())); 1562 return builder.build(); 1563 } 1564 1565 /** 1566 * Convert a protocol buffer Comparator to a ByteArrayComparable 1567 * 1568 * @param proto the protocol buffer Comparator to convert 1569 * @return the converted ByteArrayComparable 1570 */ 1571 @SuppressWarnings("unchecked") 1572 public static ByteArrayComparable toComparator(ComparatorProtos.Comparator proto) 1573 throws IOException { 1574 String type = proto.getName(); 1575 String funcName = "parseFrom"; 1576 byte [] value = proto.getSerializedComparator().toByteArray(); 1577 try { 1578 Class<?> c = Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER); 1579 Method parseFrom = c.getMethod(funcName, byte[].class); 1580 if (parseFrom == null) { 1581 throw new IOException("Unable to locate function: " + funcName + " in type: " + type); 1582 } 1583 return (ByteArrayComparable)parseFrom.invoke(null, value); 1584 } catch (Exception e) { 1585 throw new IOException(e); 1586 } 1587 } 1588 1589 /** 1590 * Convert a protocol buffer Filter to a client Filter 1591 * 1592 * @param proto the protocol buffer Filter to convert 1593 * @return the converted Filter 1594 */ 1595 @SuppressWarnings("unchecked") 1596 public static Filter toFilter(FilterProtos.Filter proto) throws IOException { 1597 String type = proto.getName(); 1598 final byte [] value = proto.getSerializedFilter().toByteArray(); 1599 String funcName = "parseFrom"; 1600 try { 1601 Class<?> c = Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER); 1602 Method parseFrom = c.getMethod(funcName, byte[].class); 1603 if (parseFrom == null) { 1604 throw new IOException("Unable to locate function: " + funcName + " in type: " + type); 1605 } 1606 return (Filter)parseFrom.invoke(c, value); 1607 } catch (Exception e) { 1608 // Either we couldn't instantiate the method object, or "parseFrom" failed. 1609 // In either case, let's not retry. 1610 throw new DoNotRetryIOException(e); 1611 } 1612 } 1613 1614 /** 1615 * Convert a client Filter to a protocol buffer Filter 1616 * 1617 * @param filter the Filter to convert 1618 * @return the converted protocol buffer Filter 1619 */ 1620 public static FilterProtos.Filter toFilter(Filter filter) throws IOException { 1621 FilterProtos.Filter.Builder builder = FilterProtos.Filter.newBuilder(); 1622 builder.setName(filter.getClass().getName()); 1623 builder.setSerializedFilter(UnsafeByteOperations.unsafeWrap(filter.toByteArray())); 1624 return builder.build(); 1625 } 1626 1627 /** 1628 * Convert a delete KeyValue type to protocol buffer DeleteType. 1629 * 1630 * @param type 1631 * @return protocol buffer DeleteType 1632 * @throws IOException 1633 */ 1634 public static DeleteType toDeleteType( 1635 KeyValue.Type type) throws IOException { 1636 switch (type) { 1637 case Delete: 1638 return DeleteType.DELETE_ONE_VERSION; 1639 case DeleteColumn: 1640 return DeleteType.DELETE_MULTIPLE_VERSIONS; 1641 case DeleteFamily: 1642 return DeleteType.DELETE_FAMILY; 1643 case DeleteFamilyVersion: 1644 return DeleteType.DELETE_FAMILY_VERSION; 1645 default: 1646 throw new IOException("Unknown delete type: " + type); 1647 } 1648 } 1649 1650 /** 1651 * Convert a protocol buffer DeleteType to delete KeyValue type. 1652 * 1653 * @param type The DeleteType 1654 * @return The type. 1655 * @throws IOException 1656 */ 1657 public static KeyValue.Type fromDeleteType( 1658 DeleteType type) throws IOException { 1659 switch (type) { 1660 case DELETE_ONE_VERSION: 1661 return KeyValue.Type.Delete; 1662 case DELETE_MULTIPLE_VERSIONS: 1663 return KeyValue.Type.DeleteColumn; 1664 case DELETE_FAMILY: 1665 return KeyValue.Type.DeleteFamily; 1666 case DELETE_FAMILY_VERSION: 1667 return KeyValue.Type.DeleteFamilyVersion; 1668 default: 1669 throw new IOException("Unknown delete type: " + type); 1670 } 1671 } 1672 1673 /** 1674 * Convert a stringified protocol buffer exception Parameter to a Java Exception 1675 * 1676 * @param parameter the protocol buffer Parameter to convert 1677 * @return the converted Exception 1678 * @throws IOException if failed to deserialize the parameter 1679 */ 1680 @SuppressWarnings("unchecked") 1681 public static Throwable toException(final NameBytesPair parameter) throws IOException { 1682 if (parameter == null || !parameter.hasValue()) return null; 1683 String desc = parameter.getValue().toStringUtf8(); 1684 String type = parameter.getName(); 1685 try { 1686 Class<? extends Throwable> c = 1687 (Class<? extends Throwable>)Class.forName(type, true, ClassLoaderHolder.CLASS_LOADER); 1688 Constructor<? extends Throwable> cn = null; 1689 try { 1690 cn = c.getDeclaredConstructor(String.class); 1691 return cn.newInstance(desc); 1692 } catch (NoSuchMethodException e) { 1693 // Could be a raw RemoteException. See HBASE-8987. 1694 cn = c.getDeclaredConstructor(String.class, String.class); 1695 return cn.newInstance(type, desc); 1696 } 1697 } catch (Exception e) { 1698 throw new IOException(e); 1699 } 1700 } 1701 1702// Start helpers for Client 1703 1704 @SuppressWarnings("unchecked") 1705 public static <T extends Service> T newServiceStub(Class<T> service, RpcChannel channel) 1706 throws Exception { 1707 return (T)Methods.call(service, null, "newStub", 1708 new Class[]{ RpcChannel.class }, new Object[]{ channel }); 1709 } 1710 1711// End helpers for Client 1712// Start helpers for Admin 1713 1714 /** 1715 * A helper to retrieve region info given a region name 1716 * using admin protocol. 1717 * 1718 * @param admin 1719 * @param regionName 1720 * @return the retrieved region info 1721 * @throws IOException 1722 */ 1723 public static org.apache.hadoop.hbase.client.RegionInfo getRegionInfo(final RpcController controller, 1724 final AdminService.BlockingInterface admin, final byte[] regionName) throws IOException { 1725 try { 1726 GetRegionInfoRequest request = 1727 RequestConverter.buildGetRegionInfoRequest(regionName); 1728 GetRegionInfoResponse response = 1729 admin.getRegionInfo(controller, request); 1730 return toRegionInfo(response.getRegionInfo()); 1731 } catch (ServiceException se) { 1732 throw getRemoteException(se); 1733 } 1734 } 1735 1736 public static List<org.apache.hadoop.hbase.RegionLoad> getRegionLoadInfo( 1737 GetRegionLoadResponse regionLoadResponse) { 1738 List<org.apache.hadoop.hbase.RegionLoad> regionLoadList = 1739 new ArrayList<>(regionLoadResponse.getRegionLoadsCount()); 1740 for (RegionLoad regionLoad : regionLoadResponse.getRegionLoadsList()) { 1741 regionLoadList.add(new org.apache.hadoop.hbase.RegionLoad(regionLoad)); 1742 } 1743 return regionLoadList; 1744 } 1745 1746 /** 1747 * A helper to close a region given a region name 1748 * using admin protocol. 1749 * 1750 * @param admin 1751 * @param regionName 1752 * @throws IOException 1753 */ 1754 public static void closeRegion(final RpcController controller, 1755 final AdminService.BlockingInterface admin, final ServerName server, final byte[] regionName) 1756 throws IOException { 1757 CloseRegionRequest closeRegionRequest = 1758 ProtobufUtil.buildCloseRegionRequest(server, regionName); 1759 try { 1760 admin.closeRegion(controller, closeRegionRequest); 1761 } catch (ServiceException se) { 1762 throw getRemoteException(se); 1763 } 1764 } 1765 1766 /** 1767 * A helper to warmup a region given a region name 1768 * using admin protocol 1769 * 1770 * @param admin 1771 * @param regionInfo 1772 * 1773 */ 1774 public static void warmupRegion(final RpcController controller, 1775 final AdminService.BlockingInterface admin, final org.apache.hadoop.hbase.client.RegionInfo regionInfo) throws IOException { 1776 1777 try { 1778 WarmupRegionRequest warmupRegionRequest = 1779 RequestConverter.buildWarmupRegionRequest(regionInfo); 1780 1781 admin.warmupRegion(controller, warmupRegionRequest); 1782 } catch (ServiceException e) { 1783 throw getRemoteException(e); 1784 } 1785 } 1786 1787 /** 1788 * A helper to open a region using admin protocol. 1789 * @param admin 1790 * @param region 1791 * @throws IOException 1792 */ 1793 public static void openRegion(final RpcController controller, 1794 final AdminService.BlockingInterface admin, ServerName server, final org.apache.hadoop.hbase.client.RegionInfo region) 1795 throws IOException { 1796 OpenRegionRequest request = 1797 RequestConverter.buildOpenRegionRequest(server, region, null); 1798 try { 1799 admin.openRegion(controller, request); 1800 } catch (ServiceException se) { 1801 throw ProtobufUtil.getRemoteException(se); 1802 } 1803 } 1804 1805 /** 1806 * A helper to get the all the online regions on a region 1807 * server using admin protocol. 1808 * 1809 * @param admin 1810 * @return a list of online region info 1811 * @throws IOException 1812 */ 1813 public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin) 1814 throws IOException { 1815 return getOnlineRegions(null, admin); 1816 } 1817 1818 /** 1819 * A helper to get the all the online regions on a region 1820 * server using admin protocol. 1821 * @return a list of online region info 1822 */ 1823 public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(final RpcController controller, 1824 final AdminService.BlockingInterface admin) 1825 throws IOException { 1826 GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest(); 1827 GetOnlineRegionResponse response = null; 1828 try { 1829 response = admin.getOnlineRegion(controller, request); 1830 } catch (ServiceException se) { 1831 throw getRemoteException(se); 1832 } 1833 return getRegionInfos(response); 1834 } 1835 1836 /** 1837 * Get the list of region info from a GetOnlineRegionResponse 1838 * 1839 * @param proto the GetOnlineRegionResponse 1840 * @return the list of region info or empty if <code>proto</code> is null 1841 */ 1842 public static List<org.apache.hadoop.hbase.client.RegionInfo> getRegionInfos(final GetOnlineRegionResponse proto) { 1843 if (proto == null) return Collections.EMPTY_LIST; 1844 List<org.apache.hadoop.hbase.client.RegionInfo> regionInfos = new ArrayList<>(proto.getRegionInfoList().size()); 1845 for (RegionInfo regionInfo: proto.getRegionInfoList()) { 1846 regionInfos.add(toRegionInfo(regionInfo)); 1847 } 1848 return regionInfos; 1849 } 1850 1851 /** 1852 * A helper to get the info of a region server using admin protocol. 1853 * @return the server name 1854 */ 1855 public static ServerInfo getServerInfo(final RpcController controller, 1856 final AdminService.BlockingInterface admin) 1857 throws IOException { 1858 GetServerInfoRequest request = RequestConverter.buildGetServerInfoRequest(); 1859 try { 1860 GetServerInfoResponse response = admin.getServerInfo(controller, request); 1861 return response.getServerInfo(); 1862 } catch (ServiceException se) { 1863 throw getRemoteException(se); 1864 } 1865 } 1866 1867 /** 1868 * A helper to get the list of files of a column family 1869 * on a given region using admin protocol. 1870 * 1871 * @return the list of store files 1872 */ 1873 public static List<String> getStoreFiles(final AdminService.BlockingInterface admin, 1874 final byte[] regionName, final byte[] family) 1875 throws IOException { 1876 return getStoreFiles(null, admin, regionName, family); 1877 } 1878 1879 /** 1880 * A helper to get the list of files of a column family 1881 * on a given region using admin protocol. 1882 * 1883 * @return the list of store files 1884 */ 1885 public static List<String> getStoreFiles(final RpcController controller, 1886 final AdminService.BlockingInterface admin, final byte[] regionName, final byte[] family) 1887 throws IOException { 1888 GetStoreFileRequest request = 1889 ProtobufUtil.buildGetStoreFileRequest(regionName, family); 1890 try { 1891 GetStoreFileResponse response = admin.getStoreFile(controller, request); 1892 return response.getStoreFileList(); 1893 } catch (ServiceException se) { 1894 throw ProtobufUtil.getRemoteException(se); 1895 } 1896 } 1897 1898// End helpers for Admin 1899 1900 /* 1901 * Get the total (read + write) requests from a RegionLoad pb 1902 * @param rl - RegionLoad pb 1903 * @return total (read + write) requests 1904 */ 1905 public static long getTotalRequestsCount(RegionLoad rl) { 1906 if (rl == null) { 1907 return 0; 1908 } 1909 1910 return rl.getReadRequestsCount() + rl.getWriteRequestsCount(); 1911 } 1912 1913 1914 /** 1915 * @param m Message to get delimited pb serialization of (with pb magic prefix) 1916 */ 1917 public static byte [] toDelimitedByteArray(final Message m) throws IOException { 1918 // Allocate arbitrary big size so we avoid resizing. 1919 ByteArrayOutputStream baos = new ByteArrayOutputStream(4096); 1920 baos.write(PB_MAGIC); 1921 m.writeDelimitedTo(baos); 1922 return baos.toByteArray(); 1923 } 1924 1925 /** 1926 * Find the HRegion encoded name based on a region specifier 1927 * 1928 * @param regionSpecifier the region specifier 1929 * @return the corresponding region's encoded name 1930 * @throws DoNotRetryIOException if the specifier type is unsupported 1931 */ 1932 public static String getRegionEncodedName( 1933 final RegionSpecifier regionSpecifier) throws DoNotRetryIOException { 1934 ByteString value = regionSpecifier.getValue(); 1935 RegionSpecifierType type = regionSpecifier.getType(); 1936 switch (type) { 1937 case REGION_NAME: 1938 return org.apache.hadoop.hbase.client.RegionInfo.encodeRegionName(value.toByteArray()); 1939 case ENCODED_REGION_NAME: 1940 return value.toStringUtf8(); 1941 default: 1942 throw new DoNotRetryIOException( 1943 "Unsupported region specifier type: " + type); 1944 } 1945 } 1946 1947 public static ScanMetrics toScanMetrics(final byte[] bytes) { 1948 MapReduceProtos.ScanMetrics pScanMetrics = null; 1949 try { 1950 pScanMetrics = MapReduceProtos.ScanMetrics.parseFrom(bytes); 1951 } catch (InvalidProtocolBufferException e) { 1952 // Ignored there are just no key values to add. 1953 } 1954 ScanMetrics scanMetrics = new ScanMetrics(); 1955 if (pScanMetrics != null) { 1956 for (HBaseProtos.NameInt64Pair pair : pScanMetrics.getMetricsList()) { 1957 if (pair.hasName() && pair.hasValue()) { 1958 scanMetrics.setCounter(pair.getName(), pair.getValue()); 1959 } 1960 } 1961 } 1962 return scanMetrics; 1963 } 1964 1965 public static MapReduceProtos.ScanMetrics toScanMetrics(ScanMetrics scanMetrics, boolean reset) { 1966 MapReduceProtos.ScanMetrics.Builder builder = MapReduceProtos.ScanMetrics.newBuilder(); 1967 Map<String, Long> metrics = scanMetrics.getMetricsMap(reset); 1968 for (Entry<String, Long> e : metrics.entrySet()) { 1969 HBaseProtos.NameInt64Pair nameInt64Pair = 1970 HBaseProtos.NameInt64Pair.newBuilder().setName(e.getKey()).setValue(e.getValue()).build(); 1971 builder.addMetrics(nameInt64Pair); 1972 } 1973 return builder.build(); 1974 } 1975 1976 /** 1977 * Unwraps an exception from a protobuf service into the underlying (expected) IOException. 1978 * This method will <strong>always</strong> throw an exception. 1979 * @param se the {@code ServiceException} instance to convert into an {@code IOException} 1980 */ 1981 public static void toIOException(ServiceException se) throws IOException { 1982 if (se == null) { 1983 throw new NullPointerException("Null service exception passed!"); 1984 } 1985 1986 Throwable cause = se.getCause(); 1987 if (cause != null && cause instanceof IOException) { 1988 throw (IOException)cause; 1989 } 1990 throw new IOException(se); 1991 } 1992 1993 public static CellProtos.Cell toCell(final Cell kv) { 1994 // Doing this is going to kill us if we do it for all data passed. 1995 // St.Ack 20121205 1996 CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder(); 1997 if (kv instanceof ByteBufferExtendedCell) { 1998 kvbuilder.setRow(wrap(((ByteBufferExtendedCell) kv).getRowByteBuffer(), 1999 ((ByteBufferExtendedCell) kv).getRowPosition(), kv.getRowLength())); 2000 kvbuilder.setFamily(wrap(((ByteBufferExtendedCell) kv).getFamilyByteBuffer(), 2001 ((ByteBufferExtendedCell) kv).getFamilyPosition(), kv.getFamilyLength())); 2002 kvbuilder.setQualifier(wrap(((ByteBufferExtendedCell) kv).getQualifierByteBuffer(), 2003 ((ByteBufferExtendedCell) kv).getQualifierPosition(), kv.getQualifierLength())); 2004 kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte())); 2005 kvbuilder.setTimestamp(kv.getTimestamp()); 2006 kvbuilder.setValue(wrap(((ByteBufferExtendedCell) kv).getValueByteBuffer(), 2007 ((ByteBufferExtendedCell) kv).getValuePosition(), kv.getValueLength())); 2008 // TODO : Once tags become first class then we may have to set tags to kvbuilder. 2009 } else { 2010 kvbuilder.setRow( 2011 UnsafeByteOperations.unsafeWrap(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength())); 2012 kvbuilder.setFamily(UnsafeByteOperations.unsafeWrap(kv.getFamilyArray(), kv.getFamilyOffset(), 2013 kv.getFamilyLength())); 2014 kvbuilder.setQualifier(UnsafeByteOperations.unsafeWrap(kv.getQualifierArray(), 2015 kv.getQualifierOffset(), kv.getQualifierLength())); 2016 kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte())); 2017 kvbuilder.setTimestamp(kv.getTimestamp()); 2018 kvbuilder.setValue(UnsafeByteOperations.unsafeWrap(kv.getValueArray(), kv.getValueOffset(), 2019 kv.getValueLength())); 2020 } 2021 return kvbuilder.build(); 2022 } 2023 2024 private static ByteString wrap(ByteBuffer b, int offset, int length) { 2025 ByteBuffer dup = b.duplicate(); 2026 dup.position(offset); 2027 dup.limit(offset + length); 2028 return UnsafeByteOperations.unsafeWrap(dup); 2029 } 2030 2031 public static Cell toCell(ExtendedCellBuilder cellBuilder, final CellProtos.Cell cell) { 2032 return cellBuilder.clear() 2033 .setRow(cell.getRow().toByteArray()) 2034 .setFamily(cell.getFamily().toByteArray()) 2035 .setQualifier(cell.getQualifier().toByteArray()) 2036 .setTimestamp(cell.getTimestamp()) 2037 .setType((byte) cell.getCellType().getNumber()) 2038 .setValue(cell.getValue().toByteArray()) 2039 .build(); 2040 } 2041 2042 public static HBaseProtos.NamespaceDescriptor toProtoNamespaceDescriptor(NamespaceDescriptor ns) { 2043 HBaseProtos.NamespaceDescriptor.Builder b = 2044 HBaseProtos.NamespaceDescriptor.newBuilder() 2045 .setName(ByteString.copyFromUtf8(ns.getName())); 2046 for(Map.Entry<String, String> entry: ns.getConfiguration().entrySet()) { 2047 b.addConfiguration(HBaseProtos.NameStringPair.newBuilder() 2048 .setName(entry.getKey()) 2049 .setValue(entry.getValue())); 2050 } 2051 return b.build(); 2052 } 2053 2054 public static NamespaceDescriptor toNamespaceDescriptor(HBaseProtos.NamespaceDescriptor desc) { 2055 NamespaceDescriptor.Builder b = NamespaceDescriptor.create(desc.getName().toStringUtf8()); 2056 for (HBaseProtos.NameStringPair prop : desc.getConfigurationList()) { 2057 b.addConfiguration(prop.getName(), prop.getValue()); 2058 } 2059 return b.build(); 2060 } 2061 2062 public static CompactionDescriptor toCompactionDescriptor( 2063 org.apache.hadoop.hbase.client.RegionInfo info, byte[] family, 2064 List<Path> inputPaths, List<Path> outputPaths, Path storeDir) { 2065 return toCompactionDescriptor(info, null, family, inputPaths, outputPaths, storeDir); 2066 } 2067 2068 public static CompactionDescriptor toCompactionDescriptor( 2069 org.apache.hadoop.hbase.client.RegionInfo info, byte[] regionName, 2070 byte[] family, List<Path> inputPaths, List<Path> outputPaths, Path storeDir) { 2071 // compaction descriptor contains relative paths. 2072 // input / output paths are relative to the store dir 2073 // store dir is relative to region dir 2074 CompactionDescriptor.Builder builder = CompactionDescriptor.newBuilder() 2075 .setTableName(UnsafeByteOperations.unsafeWrap(info.getTable().toBytes())) 2076 .setEncodedRegionName(UnsafeByteOperations.unsafeWrap( 2077 regionName == null ? info.getEncodedNameAsBytes() : regionName)) 2078 .setFamilyName(UnsafeByteOperations.unsafeWrap(family)) 2079 .setStoreHomeDir(storeDir.getName()); //make relative 2080 for (Path inputPath : inputPaths) { 2081 builder.addCompactionInput(inputPath.getName()); //relative path 2082 } 2083 for (Path outputPath : outputPaths) { 2084 builder.addCompactionOutput(outputPath.getName()); 2085 } 2086 builder.setRegionName(UnsafeByteOperations.unsafeWrap(info.getRegionName())); 2087 return builder.build(); 2088 } 2089 2090 public static FlushDescriptor toFlushDescriptor(FlushAction action, org.apache.hadoop.hbase.client.RegionInfo hri, 2091 long flushSeqId, Map<byte[], List<Path>> committedFiles) { 2092 FlushDescriptor.Builder desc = FlushDescriptor.newBuilder() 2093 .setAction(action) 2094 .setEncodedRegionName(UnsafeByteOperations.unsafeWrap(hri.getEncodedNameAsBytes())) 2095 .setRegionName(UnsafeByteOperations.unsafeWrap(hri.getRegionName())) 2096 .setFlushSequenceNumber(flushSeqId) 2097 .setTableName(UnsafeByteOperations.unsafeWrap(hri.getTable().getName())); 2098 2099 for (Map.Entry<byte[], List<Path>> entry : committedFiles.entrySet()) { 2100 WALProtos.FlushDescriptor.StoreFlushDescriptor.Builder builder = 2101 WALProtos.FlushDescriptor.StoreFlushDescriptor.newBuilder() 2102 .setFamilyName(UnsafeByteOperations.unsafeWrap(entry.getKey())) 2103 .setStoreHomeDir(Bytes.toString(entry.getKey())); //relative to region 2104 if (entry.getValue() != null) { 2105 for (Path path : entry.getValue()) { 2106 builder.addFlushOutput(path.getName()); 2107 } 2108 } 2109 desc.addStoreFlushes(builder); 2110 } 2111 return desc.build(); 2112 } 2113 2114 public static RegionEventDescriptor toRegionEventDescriptor( 2115 EventType eventType, org.apache.hadoop.hbase.client.RegionInfo hri, long seqId, ServerName server, 2116 Map<byte[], List<Path>> storeFiles) { 2117 final byte[] tableNameAsBytes = hri.getTable().getName(); 2118 final byte[] encodedNameAsBytes = hri.getEncodedNameAsBytes(); 2119 final byte[] regionNameAsBytes = hri.getRegionName(); 2120 return toRegionEventDescriptor(eventType, 2121 tableNameAsBytes, 2122 encodedNameAsBytes, 2123 regionNameAsBytes, 2124 seqId, 2125 2126 server, 2127 storeFiles); 2128 } 2129 2130 public static RegionEventDescriptor toRegionEventDescriptor(EventType eventType, 2131 byte[] tableNameAsBytes, 2132 byte[] encodedNameAsBytes, 2133 byte[] regionNameAsBytes, 2134 long seqId, 2135 2136 ServerName server, 2137 Map<byte[], List<Path>> storeFiles) { 2138 RegionEventDescriptor.Builder desc = RegionEventDescriptor.newBuilder() 2139 .setEventType(eventType) 2140 .setTableName(UnsafeByteOperations.unsafeWrap(tableNameAsBytes)) 2141 .setEncodedRegionName(UnsafeByteOperations.unsafeWrap(encodedNameAsBytes)) 2142 .setRegionName(UnsafeByteOperations.unsafeWrap(regionNameAsBytes)) 2143 .setLogSequenceNumber(seqId) 2144 .setServer(toServerName(server)); 2145 2146 for (Entry<byte[], List<Path>> entry : storeFiles.entrySet()) { 2147 StoreDescriptor.Builder builder = StoreDescriptor.newBuilder() 2148 .setFamilyName(UnsafeByteOperations.unsafeWrap(entry.getKey())) 2149 .setStoreHomeDir(Bytes.toString(entry.getKey())); 2150 for (Path path : entry.getValue()) { 2151 builder.addStoreFile(path.getName()); 2152 } 2153 2154 desc.addStores(builder); 2155 } 2156 return desc.build(); 2157 } 2158 2159 /** 2160 * Return short version of Message toString'd, shorter than TextFormat#shortDebugString. 2161 * Tries to NOT print out data both because it can be big but also so we do not have data in our 2162 * logs. Use judiciously. 2163 * @param m 2164 * @return toString of passed <code>m</code> 2165 */ 2166 public static String getShortTextFormat(Message m) { 2167 if (m == null) return "null"; 2168 if (m instanceof ScanRequest) { 2169 // This should be small and safe to output. No data. 2170 return TextFormat.shortDebugString(m); 2171 } else if (m instanceof RegionServerReportRequest) { 2172 // Print a short message only, just the servername and the requests, not the full load. 2173 RegionServerReportRequest r = (RegionServerReportRequest)m; 2174 return "server " + TextFormat.shortDebugString(r.getServer()) + 2175 " load { numberOfRequests: " + r.getLoad().getNumberOfRequests() + " }"; 2176 } else if (m instanceof RegionServerStartupRequest) { 2177 // Should be small enough. 2178 return TextFormat.shortDebugString(m); 2179 } else if (m instanceof MutationProto) { 2180 return toShortString((MutationProto)m); 2181 } else if (m instanceof GetRequest) { 2182 GetRequest r = (GetRequest) m; 2183 return "region= " + getStringForByteString(r.getRegion().getValue()) + 2184 ", row=" + getStringForByteString(r.getGet().getRow()); 2185 } else if (m instanceof ClientProtos.MultiRequest) { 2186 ClientProtos.MultiRequest r = (ClientProtos.MultiRequest) m; 2187 2188 // Get the number of Actions 2189 int actionsCount = r.getRegionActionList() 2190 .stream() 2191 .mapToInt(ClientProtos.RegionAction::getActionCount) 2192 .sum(); 2193 2194 // Get first set of Actions. 2195 ClientProtos.RegionAction actions = r.getRegionActionList().get(0); 2196 String row = actions.getActionCount() <= 0? "": 2197 getStringForByteString(actions.getAction(0).hasGet()? 2198 actions.getAction(0).getGet().getRow(): 2199 actions.getAction(0).getMutation().getRow()); 2200 return "region= " + getStringForByteString(actions.getRegion().getValue()) + 2201 ", for " + actionsCount + " action(s) and 1st row key=" + row; 2202 } else if (m instanceof ClientProtos.MutateRequest) { 2203 ClientProtos.MutateRequest r = (ClientProtos.MutateRequest) m; 2204 return "region= " + getStringForByteString(r.getRegion().getValue()) + 2205 ", row=" + getStringForByteString(r.getMutation().getRow()); 2206 } else if (m instanceof ClientProtos.CoprocessorServiceRequest) { 2207 ClientProtos.CoprocessorServiceRequest r = (ClientProtos.CoprocessorServiceRequest) m; 2208 return "coprocessorService= " + r.getCall().getServiceName() + ":" + r.getCall().getMethodName(); 2209 } 2210 return "TODO: " + m.getClass().toString(); 2211 } 2212 2213 private static String getStringForByteString(ByteString bs) { 2214 return Bytes.toStringBinary(bs.toByteArray()); 2215 } 2216 2217 /** 2218 * Print out some subset of a MutationProto rather than all of it and its data 2219 * @param proto Protobuf to print out 2220 * @return Short String of mutation proto 2221 */ 2222 static String toShortString(final MutationProto proto) { 2223 return "row=" + Bytes.toString(proto.getRow().toByteArray()) + 2224 ", type=" + proto.getMutateType().toString(); 2225 } 2226 2227 public static TableName toTableName(HBaseProtos.TableName tableNamePB) { 2228 return TableName.valueOf(tableNamePB.getNamespace().asReadOnlyByteBuffer(), 2229 tableNamePB.getQualifier().asReadOnlyByteBuffer()); 2230 } 2231 2232 public static HBaseProtos.TableName toProtoTableName(TableName tableName) { 2233 return HBaseProtos.TableName.newBuilder() 2234 .setNamespace(UnsafeByteOperations.unsafeWrap(tableName.getNamespace())) 2235 .setQualifier(UnsafeByteOperations.unsafeWrap(tableName.getQualifier())).build(); 2236 } 2237 2238 public static List<TableName> toTableNameList(List<HBaseProtos.TableName> tableNamesList) { 2239 if (tableNamesList == null) { 2240 return new ArrayList<>(); 2241 } 2242 return tableNamesList.stream().map(ProtobufUtil::toTableName).collect(Collectors.toList()); 2243 } 2244 2245 public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) { 2246 if (tableNamesList == null) { 2247 return new TableName[0]; 2248 } 2249 TableName[] tableNames = new TableName[tableNamesList.size()]; 2250 for (int i = 0; i < tableNamesList.size(); i++) { 2251 tableNames[i] = toTableName(tableNamesList.get(i)); 2252 } 2253 return tableNames; 2254 } 2255 2256 /** 2257 * Convert a protocol buffer CellVisibility to a client CellVisibility 2258 * 2259 * @param proto 2260 * @return the converted client CellVisibility 2261 */ 2262 public static CellVisibility toCellVisibility(ClientProtos.CellVisibility proto) { 2263 if (proto == null) return null; 2264 return new CellVisibility(proto.getExpression()); 2265 } 2266 2267 /** 2268 * Convert a protocol buffer CellVisibility bytes to a client CellVisibility 2269 * 2270 * @param protoBytes 2271 * @return the converted client CellVisibility 2272 * @throws DeserializationException 2273 */ 2274 public static CellVisibility toCellVisibility(byte[] protoBytes) throws DeserializationException { 2275 if (protoBytes == null) return null; 2276 ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder(); 2277 ClientProtos.CellVisibility proto = null; 2278 try { 2279 ProtobufUtil.mergeFrom(builder, protoBytes); 2280 proto = builder.build(); 2281 } catch (IOException e) { 2282 throw new DeserializationException(e); 2283 } 2284 return toCellVisibility(proto); 2285 } 2286 2287 /** 2288 * Create a protocol buffer CellVisibility based on a client CellVisibility. 2289 * 2290 * @param cellVisibility 2291 * @return a protocol buffer CellVisibility 2292 */ 2293 public static ClientProtos.CellVisibility toCellVisibility(CellVisibility cellVisibility) { 2294 ClientProtos.CellVisibility.Builder builder = ClientProtos.CellVisibility.newBuilder(); 2295 builder.setExpression(cellVisibility.getExpression()); 2296 return builder.build(); 2297 } 2298 2299 /** 2300 * Convert a protocol buffer Authorizations to a client Authorizations 2301 * 2302 * @param proto 2303 * @return the converted client Authorizations 2304 */ 2305 public static Authorizations toAuthorizations(ClientProtos.Authorizations proto) { 2306 if (proto == null) return null; 2307 return new Authorizations(proto.getLabelList()); 2308 } 2309 2310 /** 2311 * Convert a protocol buffer Authorizations bytes to a client Authorizations 2312 * 2313 * @param protoBytes 2314 * @return the converted client Authorizations 2315 * @throws DeserializationException 2316 */ 2317 public static Authorizations toAuthorizations(byte[] protoBytes) throws DeserializationException { 2318 if (protoBytes == null) return null; 2319 ClientProtos.Authorizations.Builder builder = ClientProtos.Authorizations.newBuilder(); 2320 ClientProtos.Authorizations proto = null; 2321 try { 2322 ProtobufUtil.mergeFrom(builder, protoBytes); 2323 proto = builder.build(); 2324 } catch (IOException e) { 2325 throw new DeserializationException(e); 2326 } 2327 return toAuthorizations(proto); 2328 } 2329 2330 /** 2331 * Create a protocol buffer Authorizations based on a client Authorizations. 2332 * 2333 * @param authorizations 2334 * @return a protocol buffer Authorizations 2335 */ 2336 public static ClientProtos.Authorizations toAuthorizations(Authorizations authorizations) { 2337 ClientProtos.Authorizations.Builder builder = ClientProtos.Authorizations.newBuilder(); 2338 for (String label : authorizations.getLabels()) { 2339 builder.addLabel(label); 2340 } 2341 return builder.build(); 2342 } 2343 2344 /** 2345 * Convert a protocol buffer TimeUnit to a client TimeUnit 2346 * 2347 * @param proto 2348 * @return the converted client TimeUnit 2349 */ 2350 public static TimeUnit toTimeUnit(final HBaseProtos.TimeUnit proto) { 2351 switch (proto) { 2352 case NANOSECONDS: return TimeUnit.NANOSECONDS; 2353 case MICROSECONDS: return TimeUnit.MICROSECONDS; 2354 case MILLISECONDS: return TimeUnit.MILLISECONDS; 2355 case SECONDS: return TimeUnit.SECONDS; 2356 case MINUTES: return TimeUnit.MINUTES; 2357 case HOURS: return TimeUnit.HOURS; 2358 case DAYS: return TimeUnit.DAYS; 2359 } 2360 throw new RuntimeException("Invalid TimeUnit " + proto); 2361 } 2362 2363 /** 2364 * Convert a client TimeUnit to a protocol buffer TimeUnit 2365 * 2366 * @param timeUnit 2367 * @return the converted protocol buffer TimeUnit 2368 */ 2369 public static HBaseProtos.TimeUnit toProtoTimeUnit(final TimeUnit timeUnit) { 2370 switch (timeUnit) { 2371 case NANOSECONDS: return HBaseProtos.TimeUnit.NANOSECONDS; 2372 case MICROSECONDS: return HBaseProtos.TimeUnit.MICROSECONDS; 2373 case MILLISECONDS: return HBaseProtos.TimeUnit.MILLISECONDS; 2374 case SECONDS: return HBaseProtos.TimeUnit.SECONDS; 2375 case MINUTES: return HBaseProtos.TimeUnit.MINUTES; 2376 case HOURS: return HBaseProtos.TimeUnit.HOURS; 2377 case DAYS: return HBaseProtos.TimeUnit.DAYS; 2378 } 2379 throw new RuntimeException("Invalid TimeUnit " + timeUnit); 2380 } 2381 2382 /** 2383 * Convert a protocol buffer ThrottleType to a client ThrottleType 2384 * 2385 * @param proto 2386 * @return the converted client ThrottleType 2387 */ 2388 public static ThrottleType toThrottleType(final QuotaProtos.ThrottleType proto) { 2389 switch (proto) { 2390 case REQUEST_NUMBER: return ThrottleType.REQUEST_NUMBER; 2391 case REQUEST_SIZE: return ThrottleType.REQUEST_SIZE; 2392 case WRITE_NUMBER: return ThrottleType.WRITE_NUMBER; 2393 case WRITE_SIZE: return ThrottleType.WRITE_SIZE; 2394 case READ_NUMBER: return ThrottleType.READ_NUMBER; 2395 case READ_SIZE: return ThrottleType.READ_SIZE; 2396 } 2397 throw new RuntimeException("Invalid ThrottleType " + proto); 2398 } 2399 2400 /** 2401 * Convert a client ThrottleType to a protocol buffer ThrottleType 2402 * 2403 * @param type 2404 * @return the converted protocol buffer ThrottleType 2405 */ 2406 public static QuotaProtos.ThrottleType toProtoThrottleType(final ThrottleType type) { 2407 switch (type) { 2408 case REQUEST_NUMBER: return QuotaProtos.ThrottleType.REQUEST_NUMBER; 2409 case REQUEST_SIZE: return QuotaProtos.ThrottleType.REQUEST_SIZE; 2410 case WRITE_NUMBER: return QuotaProtos.ThrottleType.WRITE_NUMBER; 2411 case WRITE_SIZE: return QuotaProtos.ThrottleType.WRITE_SIZE; 2412 case READ_NUMBER: return QuotaProtos.ThrottleType.READ_NUMBER; 2413 case READ_SIZE: return QuotaProtos.ThrottleType.READ_SIZE; 2414 } 2415 throw new RuntimeException("Invalid ThrottleType " + type); 2416 } 2417 2418 /** 2419 * Convert a protocol buffer QuotaScope to a client QuotaScope 2420 * 2421 * @param proto 2422 * @return the converted client QuotaScope 2423 */ 2424 public static QuotaScope toQuotaScope(final QuotaProtos.QuotaScope proto) { 2425 switch (proto) { 2426 case CLUSTER: return QuotaScope.CLUSTER; 2427 case MACHINE: return QuotaScope.MACHINE; 2428 } 2429 throw new RuntimeException("Invalid QuotaScope " + proto); 2430 } 2431 2432 /** 2433 * Convert a client QuotaScope to a protocol buffer QuotaScope 2434 * 2435 * @param scope 2436 * @return the converted protocol buffer QuotaScope 2437 */ 2438 public static QuotaProtos.QuotaScope toProtoQuotaScope(final QuotaScope scope) { 2439 switch (scope) { 2440 case CLUSTER: return QuotaProtos.QuotaScope.CLUSTER; 2441 case MACHINE: return QuotaProtos.QuotaScope.MACHINE; 2442 } 2443 throw new RuntimeException("Invalid QuotaScope " + scope); 2444 } 2445 2446 /** 2447 * Convert a protocol buffer QuotaType to a client QuotaType 2448 * 2449 * @param proto 2450 * @return the converted client QuotaType 2451 */ 2452 public static QuotaType toQuotaScope(final QuotaProtos.QuotaType proto) { 2453 switch (proto) { 2454 case THROTTLE: return QuotaType.THROTTLE; 2455 case SPACE: return QuotaType.SPACE; 2456 } 2457 throw new RuntimeException("Invalid QuotaType " + proto); 2458 } 2459 2460 /** 2461 * Convert a client QuotaType to a protocol buffer QuotaType 2462 * 2463 * @param type 2464 * @return the converted protocol buffer QuotaType 2465 */ 2466 public static QuotaProtos.QuotaType toProtoQuotaScope(final QuotaType type) { 2467 switch (type) { 2468 case THROTTLE: return QuotaProtos.QuotaType.THROTTLE; 2469 case SPACE: return QuotaProtos.QuotaType.SPACE; 2470 default: throw new RuntimeException("Invalid QuotaType " + type); 2471 } 2472 } 2473 2474 /** 2475 * Converts a protocol buffer SpaceViolationPolicy to a client SpaceViolationPolicy. 2476 * 2477 * @param proto The protocol buffer space violation policy. 2478 * @return The corresponding client SpaceViolationPolicy. 2479 */ 2480 public static SpaceViolationPolicy toViolationPolicy( 2481 final QuotaProtos.SpaceViolationPolicy proto) { 2482 switch (proto) { 2483 case DISABLE: return SpaceViolationPolicy.DISABLE; 2484 case NO_WRITES_COMPACTIONS: return SpaceViolationPolicy.NO_WRITES_COMPACTIONS; 2485 case NO_WRITES: return SpaceViolationPolicy.NO_WRITES; 2486 case NO_INSERTS: return SpaceViolationPolicy.NO_INSERTS; 2487 } 2488 throw new RuntimeException("Invalid SpaceViolationPolicy " + proto); 2489 } 2490 2491 /** 2492 * Converts a client SpaceViolationPolicy to a protocol buffer SpaceViolationPolicy. 2493 * 2494 * @param policy The client SpaceViolationPolicy object. 2495 * @return The corresponding protocol buffer SpaceViolationPolicy. 2496 */ 2497 public static QuotaProtos.SpaceViolationPolicy toProtoViolationPolicy( 2498 final SpaceViolationPolicy policy) { 2499 switch (policy) { 2500 case DISABLE: return QuotaProtos.SpaceViolationPolicy.DISABLE; 2501 case NO_WRITES_COMPACTIONS: return QuotaProtos.SpaceViolationPolicy.NO_WRITES_COMPACTIONS; 2502 case NO_WRITES: return QuotaProtos.SpaceViolationPolicy.NO_WRITES; 2503 case NO_INSERTS: return QuotaProtos.SpaceViolationPolicy.NO_INSERTS; 2504 } 2505 throw new RuntimeException("Invalid SpaceViolationPolicy " + policy); 2506 } 2507 2508 /** 2509 * Build a protocol buffer TimedQuota 2510 * 2511 * @param limit the allowed number of request/data per timeUnit 2512 * @param timeUnit the limit time unit 2513 * @param scope the quota scope 2514 * @return the protocol buffer TimedQuota 2515 */ 2516 public static QuotaProtos.TimedQuota toTimedQuota(final long limit, final TimeUnit timeUnit, 2517 final QuotaScope scope) { 2518 return QuotaProtos.TimedQuota.newBuilder() 2519 .setSoftLimit(limit) 2520 .setTimeUnit(toProtoTimeUnit(timeUnit)) 2521 .setScope(toProtoQuotaScope(scope)) 2522 .build(); 2523 } 2524 2525 /** 2526 * Builds a protocol buffer SpaceQuota. 2527 * 2528 * @param limit The maximum space usage for the quota in bytes. 2529 * @param violationPolicy The policy to apply when the quota is violated. 2530 * @return The protocol buffer SpaceQuota. 2531 */ 2532 public static QuotaProtos.SpaceQuota toProtoSpaceQuota( 2533 final long limit, final SpaceViolationPolicy violationPolicy) { 2534 return QuotaProtos.SpaceQuota.newBuilder() 2535 .setSoftLimit(limit) 2536 .setViolationPolicy(toProtoViolationPolicy(violationPolicy)) 2537 .build(); 2538 } 2539 2540 /** 2541 * Generates a marker for the WAL so that we propagate the notion of a bulk region load 2542 * throughout the WAL. 2543 * 2544 * @param tableName The tableName into which the bulk load is being imported into. 2545 * @param encodedRegionName Encoded region name of the region which is being bulk loaded. 2546 * @param storeFiles A set of store files of a column family are bulk loaded. 2547 * @param storeFilesSize Map of store files and their lengths 2548 * @param bulkloadSeqId sequence ID (by a force flush) used to create bulk load hfile 2549 * name 2550 * @return The WAL log marker for bulk loads. 2551 */ 2552 public static WALProtos.BulkLoadDescriptor toBulkLoadDescriptor(TableName tableName, 2553 ByteString encodedRegionName, Map<byte[], List<Path>> storeFiles, 2554 Map<String, Long> storeFilesSize, long bulkloadSeqId) { 2555 BulkLoadDescriptor.Builder desc = 2556 BulkLoadDescriptor.newBuilder() 2557 .setTableName(ProtobufUtil.toProtoTableName(tableName)) 2558 .setEncodedRegionName(encodedRegionName).setBulkloadSeqNum(bulkloadSeqId); 2559 2560 for (Map.Entry<byte[], List<Path>> entry : storeFiles.entrySet()) { 2561 WALProtos.StoreDescriptor.Builder builder = StoreDescriptor.newBuilder() 2562 .setFamilyName(UnsafeByteOperations.unsafeWrap(entry.getKey())) 2563 .setStoreHomeDir(Bytes.toString(entry.getKey())); // relative to region 2564 for (Path path : entry.getValue()) { 2565 String name = path.getName(); 2566 builder.addStoreFile(name); 2567 Long size = storeFilesSize.get(name) == null ? (Long) 0L : storeFilesSize.get(name); 2568 builder.setStoreFileSizeBytes(size); 2569 } 2570 desc.addStores(builder); 2571 } 2572 2573 return desc.build(); 2574 } 2575 2576 /** 2577 * This version of protobuf's mergeDelimitedFrom avoid the hard-coded 64MB limit for decoding 2578 * buffers 2579 * @param builder current message builder 2580 * @param in Inputsream with delimited protobuf data 2581 * @throws IOException 2582 */ 2583 public static void mergeDelimitedFrom(Message.Builder builder, InputStream in) 2584 throws IOException { 2585 // This used to be builder.mergeDelimitedFrom(in); 2586 // but is replaced to allow us to bump the protobuf size limit. 2587 final int firstByte = in.read(); 2588 if (firstByte != -1) { 2589 final int size = CodedInputStream.readRawVarint32(firstByte, in); 2590 final InputStream limitedInput = ByteStreams.limit(in, size); 2591 final CodedInputStream codedInput = CodedInputStream.newInstance(limitedInput); 2592 codedInput.setSizeLimit(size); 2593 builder.mergeFrom(codedInput); 2594 codedInput.checkLastTagWas(0); 2595 } 2596 } 2597 2598 /** 2599 * This version of protobuf's mergeFrom avoids the hard-coded 64MB limit for decoding 2600 * buffers where the message size is known 2601 * @param builder current message builder 2602 * @param in InputStream containing protobuf data 2603 * @param size known size of protobuf data 2604 * @throws IOException 2605 */ 2606 public static void mergeFrom(Message.Builder builder, InputStream in, int size) 2607 throws IOException { 2608 final CodedInputStream codedInput = CodedInputStream.newInstance(in); 2609 codedInput.setSizeLimit(size); 2610 builder.mergeFrom(codedInput); 2611 codedInput.checkLastTagWas(0); 2612 } 2613 2614 /** 2615 * This version of protobuf's mergeFrom avoids the hard-coded 64MB limit for decoding 2616 * buffers where the message size is not known 2617 * @param builder current message builder 2618 * @param in InputStream containing protobuf data 2619 * @throws IOException 2620 */ 2621 public static void mergeFrom(Message.Builder builder, InputStream in) 2622 throws IOException { 2623 final CodedInputStream codedInput = CodedInputStream.newInstance(in); 2624 codedInput.setSizeLimit(Integer.MAX_VALUE); 2625 builder.mergeFrom(codedInput); 2626 codedInput.checkLastTagWas(0); 2627 } 2628 2629 /** 2630 * This version of protobuf's mergeFrom avoids the hard-coded 64MB limit for decoding 2631 * buffers when working with ByteStrings 2632 * @param builder current message builder 2633 * @param bs ByteString containing the 2634 * @throws IOException 2635 */ 2636 public static void mergeFrom(Message.Builder builder, ByteString bs) throws IOException { 2637 final CodedInputStream codedInput = bs.newCodedInput(); 2638 codedInput.setSizeLimit(bs.size()); 2639 builder.mergeFrom(codedInput); 2640 codedInput.checkLastTagWas(0); 2641 } 2642 2643 /** 2644 * This version of protobuf's mergeFrom avoids the hard-coded 64MB limit for decoding 2645 * buffers when working with byte arrays 2646 * @param builder current message builder 2647 * @param b byte array 2648 * @throws IOException 2649 */ 2650 public static void mergeFrom(Message.Builder builder, byte[] b) throws IOException { 2651 final CodedInputStream codedInput = CodedInputStream.newInstance(b); 2652 codedInput.setSizeLimit(b.length); 2653 builder.mergeFrom(codedInput); 2654 codedInput.checkLastTagWas(0); 2655 } 2656 2657 /** 2658 * This version of protobuf's mergeFrom avoids the hard-coded 64MB limit for decoding 2659 * buffers when working with byte arrays 2660 * @param builder current message builder 2661 * @param b byte array 2662 * @param offset 2663 * @param length 2664 * @throws IOException 2665 */ 2666 public static void mergeFrom(Message.Builder builder, byte[] b, int offset, int length) 2667 throws IOException { 2668 final CodedInputStream codedInput = CodedInputStream.newInstance(b, offset, length); 2669 codedInput.setSizeLimit(length); 2670 builder.mergeFrom(codedInput); 2671 codedInput.checkLastTagWas(0); 2672 } 2673 2674 public static void mergeFrom(Message.Builder builder, CodedInputStream codedInput, int length) 2675 throws IOException { 2676 codedInput.resetSizeCounter(); 2677 int prevLimit = codedInput.setSizeLimit(length); 2678 2679 int limit = codedInput.pushLimit(length); 2680 builder.mergeFrom(codedInput); 2681 codedInput.popLimit(limit); 2682 2683 codedInput.checkLastTagWas(0); 2684 codedInput.setSizeLimit(prevLimit); 2685 } 2686 2687 public static ReplicationLoadSink toReplicationLoadSink( 2688 ClusterStatusProtos.ReplicationLoadSink rls) { 2689 return new ReplicationLoadSink(rls.getAgeOfLastAppliedOp(), rls.getTimeStampsOfLastAppliedOp()); 2690 } 2691 2692 public static ReplicationLoadSource toReplicationLoadSource( 2693 ClusterStatusProtos.ReplicationLoadSource rls) { 2694 return new ReplicationLoadSource(rls.getPeerID(), rls.getAgeOfLastShippedOp(), 2695 rls.getSizeOfLogQueue(), rls.getTimeStampOfLastShippedOp(), rls.getReplicationLag()); 2696 } 2697 2698 /** 2699 * Get a protocol buffer VersionInfo 2700 * 2701 * @return the converted protocol buffer VersionInfo 2702 */ 2703 public static HBaseProtos.VersionInfo getVersionInfo() { 2704 HBaseProtos.VersionInfo.Builder builder = HBaseProtos.VersionInfo.newBuilder(); 2705 String version = VersionInfo.getVersion(); 2706 builder.setVersion(version); 2707 String[] components = version.split("\\."); 2708 if (components != null && components.length > 2) { 2709 builder.setVersionMajor(Integer.parseInt(components[0])); 2710 builder.setVersionMinor(Integer.parseInt(components[1])); 2711 } 2712 builder.setUrl(VersionInfo.getUrl()); 2713 builder.setRevision(VersionInfo.getRevision()); 2714 builder.setUser(VersionInfo.getUser()); 2715 builder.setDate(VersionInfo.getDate()); 2716 builder.setSrcChecksum(VersionInfo.getSrcChecksum()); 2717 return builder.build(); 2718 } 2719 2720 /** 2721 * Convert SecurityCapabilitiesResponse.Capability to SecurityCapability 2722 * @param capabilities capabilities returned in the SecurityCapabilitiesResponse message 2723 * @return the converted list of SecurityCapability elements 2724 */ 2725 public static List<SecurityCapability> toSecurityCapabilityList( 2726 List<MasterProtos.SecurityCapabilitiesResponse.Capability> capabilities) { 2727 List<SecurityCapability> scList = new ArrayList<>(capabilities.size()); 2728 for (MasterProtos.SecurityCapabilitiesResponse.Capability c: capabilities) { 2729 try { 2730 scList.add(SecurityCapability.valueOf(c.getNumber())); 2731 } catch (IllegalArgumentException e) { 2732 // Unknown capability, just ignore it. We don't understand the new capability 2733 // but don't care since by definition we cannot take advantage of it. 2734 } 2735 } 2736 return scList; 2737 } 2738 2739 public static TimeRange toTimeRange(HBaseProtos.TimeRange timeRange) { 2740 return timeRange == null ? 2741 TimeRange.allTime() : 2742 new TimeRange(timeRange.hasFrom() ? timeRange.getFrom() : 0, 2743 timeRange.hasTo() ? timeRange.getTo() : Long.MAX_VALUE); 2744 } 2745 2746 /** 2747 * Converts an ColumnFamilyDescriptor to ColumnFamilySchema 2748 * @param hcd the ColumnFamilySchema 2749 * @return Convert this instance to a the pb column family type 2750 */ 2751 public static ColumnFamilySchema toColumnFamilySchema(ColumnFamilyDescriptor hcd) { 2752 ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder(); 2753 builder.setName(UnsafeByteOperations.unsafeWrap(hcd.getName())); 2754 for (Map.Entry<Bytes, Bytes> e : hcd.getValues().entrySet()) { 2755 BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder(); 2756 aBuilder.setFirst(UnsafeByteOperations.unsafeWrap(e.getKey().get())); 2757 aBuilder.setSecond(UnsafeByteOperations.unsafeWrap(e.getValue().get())); 2758 builder.addAttributes(aBuilder.build()); 2759 } 2760 for (Map.Entry<String, String> e : hcd.getConfiguration().entrySet()) { 2761 NameStringPair.Builder aBuilder = NameStringPair.newBuilder(); 2762 aBuilder.setName(e.getKey()); 2763 aBuilder.setValue(e.getValue()); 2764 builder.addConfiguration(aBuilder.build()); 2765 } 2766 return builder.build(); 2767 } 2768 2769 /** 2770 * Converts a ColumnFamilySchema to ColumnFamilyDescriptor 2771 * @param cfs the ColumnFamilySchema 2772 * @return An {@link ColumnFamilyDescriptor} made from the passed in <code>cfs</code> 2773 */ 2774 public static ColumnFamilyDescriptor toColumnFamilyDescriptor(final ColumnFamilySchema cfs) { 2775 // Use the empty constructor so we preserve the initial values set on construction for things 2776 // like maxVersion. Otherwise, we pick up wrong values on deserialization which makes for 2777 // unrelated-looking test failures that are hard to trace back to here. 2778 ColumnFamilyDescriptorBuilder builder 2779 = ColumnFamilyDescriptorBuilder.newBuilder(cfs.getName().toByteArray()); 2780 cfs.getAttributesList().forEach(a -> builder.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray())); 2781 cfs.getConfigurationList().forEach(a -> builder.setConfiguration(a.getName(), a.getValue())); 2782 return builder.build(); 2783 } 2784 2785 /** 2786 * Converts an TableDescriptor to TableSchema 2787 * @param htd the TableDescriptor 2788 * @return Convert the current {@link TableDescriptor} into a pb TableSchema instance. 2789 */ 2790 public static TableSchema toTableSchema(TableDescriptor htd) { 2791 TableSchema.Builder builder = TableSchema.newBuilder(); 2792 builder.setTableName(toProtoTableName(htd.getTableName())); 2793 for (Map.Entry<Bytes, Bytes> e : htd.getValues().entrySet()) { 2794 BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder(); 2795 aBuilder.setFirst(UnsafeByteOperations.unsafeWrap(e.getKey().get())); 2796 aBuilder.setSecond(UnsafeByteOperations.unsafeWrap(e.getValue().get())); 2797 builder.addAttributes(aBuilder.build()); 2798 } 2799 for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { 2800 builder.addColumnFamilies(toColumnFamilySchema(hcd)); 2801 } 2802 return builder.build(); 2803 } 2804 2805 /** 2806 * Converts a TableSchema to TableDescriptor 2807 * @param ts A pb TableSchema instance. 2808 * @return An {@link TableDescriptor} made from the passed in pb <code>ts</code>. 2809 */ 2810 public static TableDescriptor toTableDescriptor(final TableSchema ts) { 2811 TableDescriptorBuilder builder 2812 = TableDescriptorBuilder.newBuilder(ProtobufUtil.toTableName(ts.getTableName())); 2813 ts.getColumnFamiliesList() 2814 .stream() 2815 .map(ProtobufUtil::toColumnFamilyDescriptor) 2816 .forEach(builder::setColumnFamily); 2817 ts.getAttributesList() 2818 .forEach(a -> builder.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray())); 2819 ts.getConfigurationList() 2820 .forEach(a -> builder.setValue(a.getName(), a.getValue())); 2821 return builder.build(); 2822 } 2823 2824 /** 2825 * Creates {@link CompactionState} from 2826 * {@link org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState} 2827 * state 2828 * @param state the protobuf CompactionState 2829 * @return CompactionState 2830 */ 2831 public static CompactionState createCompactionState(GetRegionInfoResponse.CompactionState state) { 2832 return CompactionState.valueOf(state.toString()); 2833 } 2834 2835 public static GetRegionInfoResponse.CompactionState createCompactionState(CompactionState state) { 2836 return GetRegionInfoResponse.CompactionState.valueOf(state.toString()); 2837 } 2838 2839 public static Optional<Long> toOptionalTimestamp(MajorCompactionTimestampResponse resp) { 2840 long timestamp = resp.getCompactionTimestamp(); 2841 return timestamp == 0 ? Optional.empty() : Optional.of(timestamp); 2842 } 2843 2844 /** 2845 * Creates {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type} 2846 * from {@link SnapshotType} 2847 * @param type the SnapshotDescription type 2848 * @return the protobuf SnapshotDescription type 2849 */ 2850 public static SnapshotProtos.SnapshotDescription.Type 2851 createProtosSnapShotDescType(SnapshotType type) { 2852 return SnapshotProtos.SnapshotDescription.Type.valueOf(type.name()); 2853 } 2854 2855 /** 2856 * Creates {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type} 2857 * from the type of SnapshotDescription string 2858 * @param snapshotDesc string representing the snapshot description type 2859 * @return the protobuf SnapshotDescription type 2860 */ 2861 public static SnapshotProtos.SnapshotDescription.Type 2862 createProtosSnapShotDescType(String snapshotDesc) { 2863 return SnapshotProtos.SnapshotDescription.Type.valueOf(snapshotDesc.toUpperCase(Locale.ROOT)); 2864 } 2865 2866 /** 2867 * Creates {@link SnapshotType} from the type of 2868 * {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription} 2869 * @param type the snapshot description type 2870 * @return the protobuf SnapshotDescription type 2871 */ 2872 public static SnapshotType createSnapshotType(SnapshotProtos.SnapshotDescription.Type type) { 2873 return SnapshotType.valueOf(type.toString()); 2874 } 2875 2876 /** 2877 * Convert from {@link SnapshotDescription} to 2878 * {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription} 2879 * @param snapshotDesc the POJO SnapshotDescription 2880 * @return the protobuf SnapshotDescription 2881 */ 2882 public static SnapshotProtos.SnapshotDescription 2883 createHBaseProtosSnapshotDesc(SnapshotDescription snapshotDesc) { 2884 SnapshotProtos.SnapshotDescription.Builder builder = SnapshotProtos.SnapshotDescription.newBuilder(); 2885 if (snapshotDesc.getTableName() != null) { 2886 builder.setTable(snapshotDesc.getTableNameAsString()); 2887 } 2888 if (snapshotDesc.getName() != null) { 2889 builder.setName(snapshotDesc.getName()); 2890 } 2891 if (snapshotDesc.getOwner() != null) { 2892 builder.setOwner(snapshotDesc.getOwner()); 2893 } 2894 if (snapshotDesc.getCreationTime() != -1L) { 2895 builder.setCreationTime(snapshotDesc.getCreationTime()); 2896 } 2897 if (snapshotDesc.getVersion() != -1) { 2898 builder.setVersion(snapshotDesc.getVersion()); 2899 } 2900 builder.setType(ProtobufUtil.createProtosSnapShotDescType(snapshotDesc.getType())); 2901 SnapshotProtos.SnapshotDescription snapshot = builder.build(); 2902 return snapshot; 2903 } 2904 2905 /** 2906 * Convert from 2907 * {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription} to 2908 * {@link SnapshotDescription} 2909 * @param snapshotDesc the protobuf SnapshotDescription 2910 * @return the POJO SnapshotDescription 2911 */ 2912 public static SnapshotDescription 2913 createSnapshotDesc(SnapshotProtos.SnapshotDescription snapshotDesc) { 2914 return new SnapshotDescription(snapshotDesc.getName(), 2915 snapshotDesc.hasTable() ? TableName.valueOf(snapshotDesc.getTable()) : null, 2916 createSnapshotType(snapshotDesc.getType()), snapshotDesc.getOwner(), 2917 snapshotDesc.getCreationTime(), snapshotDesc.getVersion()); 2918 } 2919 2920 public static RegionLoadStats createRegionLoadStats(ClientProtos.RegionLoadStats stats) { 2921 return new RegionLoadStats(stats.getMemStoreLoad(), stats.getHeapOccupancy(), 2922 stats.getCompactionPressure()); 2923 } 2924 2925 /** 2926 * @param msg 2927 * @return A String version of the passed in <code>msg</code> 2928 */ 2929 public static String toText(Message msg) { 2930 return TextFormat.shortDebugString(msg); 2931 } 2932 2933 public static byte [] toBytes(ByteString bs) { 2934 return bs.toByteArray(); 2935 } 2936 2937 /** 2938 * Contain ServiceException inside here. Take a callable that is doing our pb rpc and run it. 2939 * @throws IOException 2940 */ 2941 public static <T> T call(Callable<T> callable) throws IOException { 2942 try { 2943 return callable.call(); 2944 } catch (Exception e) { 2945 throw ProtobufUtil.handleRemoteException(e); 2946 } 2947 } 2948 2949 /** 2950 * Create a protocol buffer GetStoreFileRequest for a given region name 2951 * 2952 * @param regionName the name of the region to get info 2953 * @param family the family to get store file list 2954 * @return a protocol buffer GetStoreFileRequest 2955 */ 2956 public static GetStoreFileRequest 2957 buildGetStoreFileRequest(final byte[] regionName, final byte[] family) { 2958 GetStoreFileRequest.Builder builder = GetStoreFileRequest.newBuilder(); 2959 RegionSpecifier region = RequestConverter.buildRegionSpecifier( 2960 RegionSpecifierType.REGION_NAME, regionName); 2961 builder.setRegion(region); 2962 builder.addFamily(UnsafeByteOperations.unsafeWrap(family)); 2963 return builder.build(); 2964 } 2965 2966 /** 2967 * Create a CloseRegionRequest for a given region name 2968 * 2969 * @param regionName the name of the region to close 2970 * @return a CloseRegionRequest 2971 */ 2972 public static CloseRegionRequest buildCloseRegionRequest(ServerName server, 2973 final byte[] regionName) { 2974 return ProtobufUtil.buildCloseRegionRequest(server, regionName, null); 2975 } 2976 2977 public static CloseRegionRequest buildCloseRegionRequest(ServerName server, 2978 final byte[] regionName, ServerName destinationServer) { 2979 CloseRegionRequest.Builder builder = CloseRegionRequest.newBuilder(); 2980 RegionSpecifier region = RequestConverter.buildRegionSpecifier( 2981 RegionSpecifierType.REGION_NAME, regionName); 2982 builder.setRegion(region); 2983 if (destinationServer != null){ 2984 builder.setDestinationServer(toServerName(destinationServer)); 2985 } 2986 if (server != null) { 2987 builder.setServerStartCode(server.getStartcode()); 2988 } 2989 return builder.build(); 2990 } 2991 2992 public static ProcedureDescription buildProcedureDescription(String signature, String instance, 2993 Map<String, String> props) { 2994 ProcedureDescription.Builder builder = 2995 ProcedureDescription.newBuilder().setSignature(signature).setInstance(instance); 2996 if (props != null && !props.isEmpty()) { 2997 props.entrySet().forEach(entry -> builder.addConfiguration( 2998 NameStringPair.newBuilder().setName(entry.getKey()).setValue(entry.getValue()).build())); 2999 } 3000 return builder.build(); 3001 } 3002 3003 /** 3004 * Get a ServerName from the passed in data bytes. 3005 * @param data Data with a serialize server name in it; can handle the old style 3006 * servername where servername was host and port. Works too with data that 3007 * begins w/ the pb 'PBUF' magic and that is then followed by a protobuf that 3008 * has a serialized {@link ServerName} in it. 3009 * @return Returns null if <code>data</code> is null else converts passed data 3010 * to a ServerName instance. 3011 * @throws DeserializationException 3012 */ 3013 public static ServerName parseServerNameFrom(final byte [] data) throws DeserializationException { 3014 if (data == null || data.length <= 0) return null; 3015 if (ProtobufMagic.isPBMagicPrefix(data)) { 3016 int prefixLen = ProtobufMagic.lengthOfPBMagic(); 3017 try { 3018 ZooKeeperProtos.Master rss = 3019 ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen); 3020 org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName sn = 3021 rss.getMaster(); 3022 return ServerName.valueOf(sn.getHostName(), sn.getPort(), sn.getStartCode()); 3023 } catch (/*InvalidProtocolBufferException*/IOException e) { 3024 // A failed parse of the znode is pretty catastrophic. Rather than loop 3025 // retrying hoping the bad bytes will changes, and rather than change 3026 // the signature on this method to add an IOE which will send ripples all 3027 // over the code base, throw a RuntimeException. This should "never" happen. 3028 // Fail fast if it does. 3029 throw new DeserializationException(e); 3030 } 3031 } 3032 // The str returned could be old style -- pre hbase-1502 -- which was 3033 // hostname and port seperated by a colon rather than hostname, port and 3034 // startcode delimited by a ','. 3035 String str = Bytes.toString(data); 3036 int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR); 3037 if (index != -1) { 3038 // Presume its ServerName serialized with versioned bytes. 3039 return ServerName.parseVersionedServerName(data); 3040 } 3041 // Presume it a hostname:port format. 3042 String hostname = Addressing.parseHostname(str); 3043 int port = Addressing.parsePort(str); 3044 return ServerName.valueOf(hostname, port, -1L); 3045 } 3046 3047 /** 3048 * Helper to convert the protobuf Procedure to JSON String 3049 * @return Convert the current Protocol Buffers Procedure to JSON String 3050 */ 3051 public static String toProcedureJson(List<ProcedureProtos.Procedure> procProtos) { 3052 JsonArray procJsons = new JsonArray(procProtos.size()); 3053 for (ProcedureProtos.Procedure procProto : procProtos) { 3054 try { 3055 JsonElement procJson = ProtobufMessageConverter.toJsonElement(procProto); 3056 procJsons.add(procJson); 3057 } catch (InvalidProtocolBufferException e) { 3058 procJsons.add(e.toString()); 3059 } 3060 } 3061 return procJsons.toString(); 3062 } 3063 3064 public static String toLockJson(List<LockServiceProtos.LockedResource> lockedResourceProtos) { 3065 JsonArray lockedResourceJsons = new JsonArray(lockedResourceProtos.size()); 3066 for (LockServiceProtos.LockedResource lockedResourceProto : lockedResourceProtos) { 3067 try { 3068 JsonElement lockedResourceJson = ProtobufMessageConverter.toJsonElement(lockedResourceProto); 3069 lockedResourceJsons.add(lockedResourceJson); 3070 } catch (InvalidProtocolBufferException e) { 3071 lockedResourceJsons.add(e.toString()); 3072 } 3073 } 3074 return lockedResourceJsons.toString(); 3075 } 3076 3077 /** 3078 * Convert a RegionInfo to a Proto RegionInfo 3079 * 3080 * @param info the RegionInfo to convert 3081 * @return the converted Proto RegionInfo 3082 */ 3083 public static HBaseProtos.RegionInfo toRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) { 3084 if (info == null) return null; 3085 HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder(); 3086 builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable())); 3087 builder.setRegionId(info.getRegionId()); 3088 if (info.getStartKey() != null) { 3089 builder.setStartKey(UnsafeByteOperations.unsafeWrap(info.getStartKey())); 3090 } 3091 if (info.getEndKey() != null) { 3092 builder.setEndKey(UnsafeByteOperations.unsafeWrap(info.getEndKey())); 3093 } 3094 builder.setOffline(info.isOffline()); 3095 builder.setSplit(info.isSplit()); 3096 builder.setReplicaId(info.getReplicaId()); 3097 return builder.build(); 3098 } 3099 3100 /** 3101 * Convert HBaseProto.RegionInfo to a RegionInfo 3102 * 3103 * @param proto the RegionInfo to convert 3104 * @return the converted RegionInfo 3105 */ 3106 public static org.apache.hadoop.hbase.client.RegionInfo toRegionInfo(final HBaseProtos.RegionInfo proto) { 3107 if (proto == null) return null; 3108 TableName tableName = ProtobufUtil.toTableName(proto.getTableName()); 3109 long regionId = proto.getRegionId(); 3110 int defaultReplicaId = org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID; 3111 int replicaId = proto.hasReplicaId()? proto.getReplicaId(): defaultReplicaId; 3112 if (tableName.equals(TableName.META_TABLE_NAME) && replicaId == defaultReplicaId) { 3113 return RegionInfoBuilder.FIRST_META_REGIONINFO; 3114 } 3115 byte[] startKey = null; 3116 byte[] endKey = null; 3117 if (proto.hasStartKey()) { 3118 startKey = proto.getStartKey().toByteArray(); 3119 } 3120 if (proto.hasEndKey()) { 3121 endKey = proto.getEndKey().toByteArray(); 3122 } 3123 boolean split = false; 3124 if (proto.hasSplit()) { 3125 split = proto.getSplit(); 3126 } 3127 RegionInfoBuilder rib = RegionInfoBuilder.newBuilder(tableName) 3128 .setStartKey(startKey) 3129 .setEndKey(endKey) 3130 .setRegionId(regionId) 3131 .setReplicaId(replicaId) 3132 .setSplit(split); 3133 if (proto.hasOffline()) { 3134 rib.setOffline(proto.getOffline()); 3135 } 3136 return rib.build(); 3137 } 3138 3139 public static HBaseProtos.RegionLocation toRegionLocation(HRegionLocation loc) { 3140 HBaseProtos.RegionLocation.Builder builder = HBaseProtos.RegionLocation.newBuilder(); 3141 builder.setRegionInfo(toRegionInfo(loc.getRegion())); 3142 if (loc.getServerName() != null) { 3143 builder.setServerName(toServerName(loc.getServerName())); 3144 } 3145 builder.setSeqNum(loc.getSeqNum()); 3146 return builder.build(); 3147 } 3148 3149 public static HRegionLocation toRegionLocation(HBaseProtos.RegionLocation proto) { 3150 org.apache.hadoop.hbase.client.RegionInfo regionInfo = toRegionInfo(proto.getRegionInfo()); 3151 ServerName serverName = proto.hasServerName() ? toServerName(proto.getServerName()) : null; 3152 return new HRegionLocation(regionInfo, serverName, proto.getSeqNum()); 3153 } 3154 3155 public static List<SnapshotDescription> toSnapshotDescriptionList( 3156 GetCompletedSnapshotsResponse response, Pattern pattern) { 3157 return response.getSnapshotsList().stream().map(ProtobufUtil::createSnapshotDesc) 3158 .filter(snap -> pattern != null ? pattern.matcher(snap.getName()).matches() : true) 3159 .collect(Collectors.toList()); 3160 } 3161 3162 public static CacheEvictionStats toCacheEvictionStats( 3163 HBaseProtos.CacheEvictionStats stats) throws IOException{ 3164 CacheEvictionStatsBuilder builder = CacheEvictionStats.builder(); 3165 builder.withEvictedBlocks(stats.getEvictedBlocks()) 3166 .withMaxCacheSize(stats.getMaxCacheSize()); 3167 if (stats.getExceptionCount() > 0) { 3168 for (HBaseProtos.RegionExceptionMessage exception : stats.getExceptionList()) { 3169 HBaseProtos.RegionSpecifier rs = exception.getRegion(); 3170 byte[] regionName = rs.getValue().toByteArray(); 3171 builder.addException(regionName, ProtobufUtil.toException(exception.getException())); 3172 } 3173 } 3174 return builder.build(); 3175 } 3176 3177 public static HBaseProtos.CacheEvictionStats toCacheEvictionStats( 3178 CacheEvictionStats cacheEvictionStats) { 3179 HBaseProtos.CacheEvictionStats.Builder builder 3180 = HBaseProtos.CacheEvictionStats.newBuilder(); 3181 for (Map.Entry<byte[], Throwable> entry : cacheEvictionStats.getExceptions().entrySet()) { 3182 builder.addException( 3183 RegionExceptionMessage.newBuilder() 3184 .setRegion(RequestConverter.buildRegionSpecifier( 3185 RegionSpecifierType.REGION_NAME, entry.getKey())) 3186 .setException(ResponseConverter.buildException(entry.getValue())) 3187 .build() 3188 ); 3189 } 3190 return builder 3191 .setEvictedBlocks(cacheEvictionStats.getEvictedBlocks()) 3192 .setMaxCacheSize(cacheEvictionStats.getMaxCacheSize()) 3193 .build(); 3194 } 3195 3196 public static ClusterStatusProtos.ReplicationLoadSource toReplicationLoadSource( 3197 ReplicationLoadSource rls) { 3198 return ClusterStatusProtos.ReplicationLoadSource.newBuilder() 3199 .setPeerID(rls.getPeerID()) 3200 .setAgeOfLastShippedOp(rls.getAgeOfLastShippedOp()) 3201 .setSizeOfLogQueue((int) rls.getSizeOfLogQueue()) 3202 .setTimeStampOfLastShippedOp(rls.getTimestampOfLastShippedOp()) 3203 .setReplicationLag(rls.getReplicationLag()) 3204 .build(); 3205 } 3206 3207 public static ClusterStatusProtos.ReplicationLoadSink toReplicationLoadSink( 3208 ReplicationLoadSink rls) { 3209 return ClusterStatusProtos.ReplicationLoadSink.newBuilder() 3210 .setAgeOfLastAppliedOp(rls.getAgeOfLastAppliedOp()) 3211 .setTimeStampsOfLastAppliedOp(rls.getTimestampsOfLastAppliedOp()) 3212 .build(); 3213 } 3214 3215 public static HBaseProtos.TimeRange toTimeRange(TimeRange timeRange) { 3216 if (timeRange == null) { 3217 timeRange = TimeRange.allTime(); 3218 } 3219 return HBaseProtos.TimeRange.newBuilder().setFrom(timeRange.getMin()) 3220 .setTo(timeRange.getMax()) 3221 .build(); 3222 } 3223}