From 2fd6f2a829a6ee3641d287aa713631027e512572 Mon Sep 17 00:00:00 2001 From: Sameet Agarwal Date: Thu, 12 Mar 2015 10:56:58 -0700 Subject: [PATCH] HBASE-13316 Reduce the downtime on planned moves of regions Summary: The current behavior of a region move shuts down a region and then starts is up in another regionserver. This causes increased latency and possibly timeouts till the new region's cache is fully warmed up. We can make a region move less disruptive by warming the cache in the destination region server before shutting dow the old region. See https://issues.apache.org/jira/browse/HBASE-13316 Test Plan: 1. Unit Tests 2. Added test for concurrent moves and warmups 3. Manually tested reads/writes happening with concurrent moves Subscribers: tedyu Differential Revision: https://reviews.facebook.net/D35967 Signed-off-by: Elliott Clark --- .../hadoop/hbase/protobuf/ProtobufUtil.java | 22 + .../hbase/protobuf/RequestConverter.java | 11 + .../hbase/protobuf/generated/AdminProtos.java | 1282 +++++++++++++++-- hbase-protocol/src/main/protobuf/Admin.proto | 11 + .../hbase/master/AssignmentManager.java | 1 + .../apache/hadoop/hbase/master/HMaster.java | 5 + .../hadoop/hbase/master/ServerManager.java | 21 + .../hadoop/hbase/regionserver/HRegion.java | 47 +- .../hbase/regionserver/RSRpcServices.java | 53 + .../hadoop/hbase/master/MockRegionServer.java | 8 + .../hadoop/hbase/master/TestWarmupRegion.java | 163 +++ 11 files changed, 1476 insertions(+), 148 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 55311e2d6d9..fd9da05c467 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsReques import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; import org.apache.hadoop.hbase.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; @@ -1730,6 +1731,26 @@ public final class ProtobufUtil { } } + /** + * A helper to warmup a region given a region name + * using admin protocol + * + * @param admin + * @param regionInfo + * + */ + public static void warmupRegion(final AdminService.BlockingInterface admin, + final HRegionInfo regionInfo) throws IOException { + + try { + WarmupRegionRequest warmupRegionRequest = + RequestConverter.buildWarmupRegionRequest(regionInfo); + + admin.warmupRegion(null, warmupRegionRequest); + } catch (ServiceException e) { + throw getRemoteException(e); + } + } /** * A helper to open a region using admin protocol. @@ -1748,6 +1769,7 @@ public final class ProtobufUtil { } } + /** * A helper to get the all the online regions on a region * server using admin protocol. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java index abd9c0fe1bc..085b6e84997 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest; @@ -866,6 +867,16 @@ public final class RequestConverter { return builder.build(); } + /** + * Create a WarmupRegionRequest for a given region name + * + * @param regionInfo Region we are warming up + */ + public static WarmupRegionRequest buildWarmupRegionRequest(final HRegionInfo regionInfo) { + WarmupRegionRequest.Builder builder = WarmupRegionRequest.newBuilder(); + builder.setRegionInfo(HRegionInfo.convert(regionInfo)); + return builder.build(); + } /** * Create a CloseRegionRequest for a given encoded region name * diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java index ea022b5a84e..a76936d935f 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java @@ -6440,6 +6440,905 @@ public final class AdminProtos { // @@protoc_insertion_point(class_scope:OpenRegionResponse) } + public interface WarmupRegionRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .RegionInfo regionInfo = 1; + /** + * required .RegionInfo regionInfo = 1; + */ + boolean hasRegionInfo(); + /** + * required .RegionInfo regionInfo = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(); + /** + * required .RegionInfo regionInfo = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(); + } + /** + * Protobuf type {@code WarmupRegionRequest} + */ + public static final class WarmupRegionRequest extends + com.google.protobuf.GeneratedMessage + implements WarmupRegionRequestOrBuilder { + // Use WarmupRegionRequest.newBuilder() to construct. + private WarmupRegionRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private WarmupRegionRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final WarmupRegionRequest defaultInstance; + public static WarmupRegionRequest getDefaultInstance() { + return defaultInstance; + } + + public WarmupRegionRequest getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private WarmupRegionRequest( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = regionInfo_.toBuilder(); + } + regionInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(regionInfo_); + regionInfo_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public WarmupRegionRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new WarmupRegionRequest(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .RegionInfo regionInfo = 1; + public static final int REGIONINFO_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo regionInfo_; + /** + * required .RegionInfo regionInfo = 1; + */ + public boolean hasRegionInfo() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() { + return regionInfo_; + } + /** + * required .RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() { + return regionInfo_; + } + + private void initFields() { + regionInfo_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRegionInfo()) { + memoizedIsInitialized = 0; + return false; + } + if (!getRegionInfo().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, regionInfo_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, regionInfo_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest other = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest) obj; + + boolean result = true; + result = result && (hasRegionInfo() == other.hasRegionInfo()); + if (hasRegionInfo()) { + result = result && getRegionInfo() + .equals(other.getRegionInfo()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRegionInfo()) { + hash = (37 * hash) + REGIONINFO_FIELD_NUMBER; + hash = (53 * hash) + getRegionInfo().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code WarmupRegionRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getRegionInfoFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (regionInfoBuilder_ == null) { + regionInfo_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance(); + } else { + regionInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionRequest_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest build() { + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest result = new org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (regionInfoBuilder_ == null) { + result.regionInfo_ = regionInfo_; + } else { + result.regionInfo_ = regionInfoBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.getDefaultInstance()) return this; + if (other.hasRegionInfo()) { + mergeRegionInfo(other.getRegionInfo()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasRegionInfo()) { + + return false; + } + if (!getRegionInfo().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .RegionInfo regionInfo = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_; + /** + * required .RegionInfo regionInfo = 1; + */ + public boolean hasRegionInfo() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() { + if (regionInfoBuilder_ == null) { + return regionInfo_; + } else { + return regionInfoBuilder_.getMessage(); + } + } + /** + * required .RegionInfo regionInfo = 1; + */ + public Builder setRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) { + if (regionInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + regionInfo_ = value; + onChanged(); + } else { + regionInfoBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .RegionInfo regionInfo = 1; + */ + public Builder setRegionInfo( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) { + if (regionInfoBuilder_ == null) { + regionInfo_ = builderForValue.build(); + onChanged(); + } else { + regionInfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .RegionInfo regionInfo = 1; + */ + public Builder mergeRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) { + if (regionInfoBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + regionInfo_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) { + regionInfo_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(regionInfo_).mergeFrom(value).buildPartial(); + } else { + regionInfo_ = value; + } + onChanged(); + } else { + regionInfoBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .RegionInfo regionInfo = 1; + */ + public Builder clearRegionInfo() { + if (regionInfoBuilder_ == null) { + regionInfo_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance(); + onChanged(); + } else { + regionInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getRegionInfoFieldBuilder().getBuilder(); + } + /** + * required .RegionInfo regionInfo = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() { + if (regionInfoBuilder_ != null) { + return regionInfoBuilder_.getMessageOrBuilder(); + } else { + return regionInfo_; + } + } + /** + * required .RegionInfo regionInfo = 1; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> + getRegionInfoFieldBuilder() { + if (regionInfoBuilder_ == null) { + regionInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>( + regionInfo_, + getParentForChildren(), + isClean()); + regionInfo_ = null; + } + return regionInfoBuilder_; + } + + // @@protoc_insertion_point(builder_scope:WarmupRegionRequest) + } + + static { + defaultInstance = new WarmupRegionRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:WarmupRegionRequest) + } + + public interface WarmupRegionResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + /** + * Protobuf type {@code WarmupRegionResponse} + */ + public static final class WarmupRegionResponse extends + com.google.protobuf.GeneratedMessage + implements WarmupRegionResponseOrBuilder { + // Use WarmupRegionResponse.newBuilder() to construct. + private WarmupRegionResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private WarmupRegionResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final WarmupRegionResponse defaultInstance; + public static WarmupRegionResponse getDefaultInstance() { + return defaultInstance; + } + + public WarmupRegionResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private WarmupRegionResponse( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public WarmupRegionResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new WarmupRegionResponse(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse other = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code WarmupRegionResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.class, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.internal_static_WarmupRegionResponse_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse build() { + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse result = new org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + // @@protoc_insertion_point(builder_scope:WarmupRegionResponse) + } + + static { + defaultInstance = new WarmupRegionResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:WarmupRegionResponse) + } + public interface CloseRegionRequestOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -20819,6 +21718,14 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest request, com.google.protobuf.RpcCallback done); + /** + * rpc WarmupRegion(.WarmupRegionRequest) returns (.WarmupRegionResponse); + */ + public abstract void warmupRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest request, + com.google.protobuf.RpcCallback done); + /** * rpc CloseRegion(.CloseRegionRequest) returns (.CloseRegionResponse); */ @@ -20952,6 +21859,14 @@ public final class AdminProtos { impl.openRegion(controller, request, done); } + @java.lang.Override + public void warmupRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest request, + com.google.protobuf.RpcCallback done) { + impl.warmupRegion(controller, request, done); + } + @java.lang.Override public void closeRegion( com.google.protobuf.RpcController controller, @@ -21079,28 +21994,30 @@ public final class AdminProtos { case 3: return impl.openRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest)request); case 4: - return impl.closeRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest)request); + return impl.warmupRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest)request); case 5: - return impl.flushRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest)request); + return impl.closeRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest)request); case 6: - return impl.splitRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest)request); + return impl.flushRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest)request); case 7: - return impl.compactRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest)request); + return impl.splitRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest)request); case 8: - return impl.mergeRegions(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest)request); + return impl.compactRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest)request); case 9: - return impl.replicateWALEntry(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request); + return impl.mergeRegions(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest)request); case 10: - return impl.replay(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request); + return impl.replicateWALEntry(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request); case 11: - return impl.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request); + return impl.replay(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request); case 12: - return impl.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request); + return impl.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request); case 13: - return impl.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request); + return impl.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request); case 14: - return impl.updateFavoredNodes(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest)request); + return impl.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request); case 15: + return impl.updateFavoredNodes(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest)request); + case 16: return impl.updateConfiguration(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest)request); default: throw new java.lang.AssertionError("Can't get here."); @@ -21125,28 +22042,30 @@ public final class AdminProtos { case 3: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.getDefaultInstance(); case 4: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.getDefaultInstance(); case 5: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest.getDefaultInstance(); case 6: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest.getDefaultInstance(); case 7: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest.getDefaultInstance(); case 8: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest.getDefaultInstance(); case 9: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest.getDefaultInstance(); case 10: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); case 11: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); case 12: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); case 13: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); case 14: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance(); case 15: + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.getDefaultInstance(); + case 16: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -21171,28 +22090,30 @@ public final class AdminProtos { case 3: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.getDefaultInstance(); case 4: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance(); case 5: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance(); case 6: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance(); case 7: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse.getDefaultInstance(); case 8: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse.getDefaultInstance(); case 9: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse.getDefaultInstance(); case 10: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); case 11: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); case 12: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); case 13: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); case 14: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(); case 15: + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(); + case 16: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -21234,6 +22155,14 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest request, com.google.protobuf.RpcCallback done); + /** + * rpc WarmupRegion(.WarmupRegionRequest) returns (.WarmupRegionResponse); + */ + public abstract void warmupRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest request, + com.google.protobuf.RpcCallback done); + /** * rpc CloseRegion(.CloseRegionRequest) returns (.CloseRegionResponse); */ @@ -21373,61 +22302,66 @@ public final class AdminProtos { done)); return; case 4: + this.warmupRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 5: this.closeRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 5: + case 6: this.flushRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 6: + case 7: this.splitRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 7: + case 8: this.compactRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 8: + case 9: this.mergeRegions(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 9: + case 10: this.replicateWALEntry(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 10: + case 11: this.replay(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 11: + case 12: this.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 12: + case 13: this.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 13: + case 14: this.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 14: + case 15: this.updateFavoredNodes(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 15: + case 16: this.updateConfiguration(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); @@ -21455,28 +22389,30 @@ public final class AdminProtos { case 3: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.getDefaultInstance(); case 4: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest.getDefaultInstance(); case 5: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest.getDefaultInstance(); case 6: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest.getDefaultInstance(); case 7: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest.getDefaultInstance(); case 8: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest.getDefaultInstance(); case 9: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest.getDefaultInstance(); case 10: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); case 11: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); case 12: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); case 13: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); case 14: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance(); case 15: + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest.getDefaultInstance(); + case 16: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -21501,28 +22437,30 @@ public final class AdminProtos { case 3: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.getDefaultInstance(); case 4: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance(); case 5: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance(); case 6: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance(); case 7: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse.getDefaultInstance(); case 8: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse.getDefaultInstance(); case 9: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse.getDefaultInstance(); case 10: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); case 11: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); case 12: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); case 13: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); case 14: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(); case 15: + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(); + case 16: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -21605,12 +22543,27 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.getDefaultInstance())); } + public void warmupRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(4), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance())); + } + public void closeRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(4), + getDescriptor().getMethods().get(5), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance(), @@ -21625,7 +22578,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(5), + getDescriptor().getMethods().get(6), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance(), @@ -21640,7 +22593,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(6), + getDescriptor().getMethods().get(7), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse.getDefaultInstance(), @@ -21655,7 +22608,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(7), + getDescriptor().getMethods().get(8), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse.getDefaultInstance(), @@ -21670,7 +22623,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(8), + getDescriptor().getMethods().get(9), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse.getDefaultInstance(), @@ -21685,7 +22638,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(9), + getDescriptor().getMethods().get(10), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(), @@ -21700,7 +22653,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(10), + getDescriptor().getMethods().get(11), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(), @@ -21715,7 +22668,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(11), + getDescriptor().getMethods().get(12), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(), @@ -21730,7 +22683,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(12), + getDescriptor().getMethods().get(13), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(), @@ -21745,7 +22698,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(13), + getDescriptor().getMethods().get(14), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(), @@ -21760,7 +22713,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(14), + getDescriptor().getMethods().get(15), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance(), @@ -21775,7 +22728,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(15), + getDescriptor().getMethods().get(16), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance(), @@ -21812,6 +22765,11 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest request) throws com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse warmupRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest request) + throws com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse closeRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest request) @@ -21928,12 +22886,24 @@ public final class AdminProtos { } + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse warmupRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(4), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse.getDefaultInstance()); + } + + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse closeRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(4), + getDescriptor().getMethods().get(5), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse.getDefaultInstance()); @@ -21945,7 +22915,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(5), + getDescriptor().getMethods().get(6), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance()); @@ -21957,7 +22927,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(6), + getDescriptor().getMethods().get(7), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse.getDefaultInstance()); @@ -21969,7 +22939,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(7), + getDescriptor().getMethods().get(8), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse.getDefaultInstance()); @@ -21981,7 +22951,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(8), + getDescriptor().getMethods().get(9), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse.getDefaultInstance()); @@ -21993,7 +22963,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(9), + getDescriptor().getMethods().get(10), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance()); @@ -22005,7 +22975,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(10), + getDescriptor().getMethods().get(11), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance()); @@ -22017,7 +22987,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(11), + getDescriptor().getMethods().get(12), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance()); @@ -22029,7 +22999,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(12), + getDescriptor().getMethods().get(13), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance()); @@ -22041,7 +23011,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(13), + getDescriptor().getMethods().get(14), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance()); @@ -22053,7 +23023,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(14), + getDescriptor().getMethods().get(15), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateFavoredNodesResponse.getDefaultInstance()); @@ -22065,7 +23035,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(15), + getDescriptor().getMethods().get(16), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationResponse.getDefaultInstance()); @@ -22121,6 +23091,16 @@ public final class AdminProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_OpenRegionResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_WarmupRegionRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_WarmupRegionRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_WarmupRegionResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_WarmupRegionResponse_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_CloseRegionRequest_descriptor; private static @@ -22279,12 +23259,14 @@ public final class AdminProtos { "\ropening_state\030\001 \003(\0162&.OpenRegionRespons" + "e.RegionOpeningState\"H\n\022RegionOpeningSta" + "te\022\n\n\006OPENED\020\000\022\022\n\016ALREADY_OPENED\020\001\022\022\n\016FA" + - "ILED_OPENING\020\002\"\271\001\n\022CloseRegionRequest\022 \n" + + "ILED_OPENING\020\002\"6\n\023WarmupRegionRequest\022\037\n" + + "\nregionInfo\030\001 \002(\0132\013.RegionInfo\"\026\n\024Warmup" + + "RegionResponse\"\271\001\n\022CloseRegionRequest\022 \n" + "\006region\030\001 \002(\0132\020.RegionSpecifier\022\037\n\027versi" + "on_of_closing_node\030\002 \001(\r\022\036\n\020transition_i" + - "n_ZK\030\003 \001(\010:\004true\022\'\n\022destination_server\030\004" + + "n_ZK\030\003 \001(\010:\004true\022\'\n\022destination_server\030\004", " \001(\0132\013.ServerName\022\027\n\017serverStartCode\030\005 \001" + - "(\004\"%\n\023CloseRegionResponse\022\016\n\006closed\030\001 \002(", + "(\004\"%\n\023CloseRegionResponse\022\016\n\006closed\030\001 \002(" + "\010\"p\n\022FlushRegionRequest\022 \n\006region\030\001 \002(\0132" + "\020.RegionSpecifier\022\030\n\020if_older_than_ts\030\002 " + "\001(\004\022\036\n\026write_flush_wal_marker\030\003 \001(\010\"_\n\023F" + @@ -22292,9 +23274,9 @@ public final class AdminProtos { "\002(\004\022\017\n\007flushed\030\002 \001(\010\022\036\n\026wrote_flush_wal_" + "marker\030\003 \001(\010\"K\n\022SplitRegionRequest\022 \n\006re" + "gion\030\001 \002(\0132\020.RegionSpecifier\022\023\n\013split_po" + - "int\030\002 \001(\014\"\025\n\023SplitRegionResponse\"W\n\024Comp" + + "int\030\002 \001(\014\"\025\n\023SplitRegionResponse\"W\n\024Comp", "actRegionRequest\022 \n\006region\030\001 \002(\0132\020.Regio" + - "nSpecifier\022\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003 \001(", + "nSpecifier\022\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003 \001(" + "\014\"\027\n\025CompactRegionResponse\"\262\001\n\031UpdateFav" + "oredNodesRequest\022@\n\013update_info\030\001 \003(\0132+." + "UpdateFavoredNodesRequest.RegionUpdateIn" + @@ -22302,9 +23284,9 @@ public final class AdminProtos { ".RegionInfo\022\"\n\rfavored_nodes\030\002 \003(\0132\013.Ser" + "verName\".\n\032UpdateFavoredNodesResponse\022\020\n" + "\010response\030\001 \001(\r\"v\n\023MergeRegionsRequest\022\"" + - "\n\010region_a\030\001 \002(\0132\020.RegionSpecifier\022\"\n\010re" + + "\n\010region_a\030\001 \002(\0132\020.RegionSpecifier\022\"\n\010re", "gion_b\030\002 \002(\0132\020.RegionSpecifier\022\027\n\010forcib" + - "le\030\003 \001(\010:\005false\"\026\n\024MergeRegionsResponse\"", + "le\030\003 \001(\010:\005false\"\026\n\024MergeRegionsResponse\"" + "X\n\010WALEntry\022\024\n\003key\030\001 \002(\0132\007.WALKey\022\027\n\017key" + "_value_bytes\030\002 \003(\014\022\035\n\025associated_cell_co" + "unt\030\003 \001(\005\"4\n\030ReplicateWALEntryRequest\022\030\n" + @@ -22312,40 +23294,42 @@ public final class AdminProtos { "ntryResponse\"\026\n\024RollWALWriterRequest\"0\n\025" + "RollWALWriterResponse\022\027\n\017region_to_flush" + "\030\001 \003(\014\"#\n\021StopServerRequest\022\016\n\006reason\030\001 " + - "\002(\t\"\024\n\022StopServerResponse\"\026\n\024GetServerIn" + + "\002(\t\"\024\n\022StopServerResponse\"\026\n\024GetServerIn", "foRequest\"B\n\nServerInfo\022 \n\013server_name\030\001" + - " \002(\0132\013.ServerName\022\022\n\nwebui_port\030\002 \001(\r\"9\n", + " \002(\0132\013.ServerName\022\022\n\nwebui_port\030\002 \001(\r\"9\n" + "\025GetServerInfoResponse\022 \n\013server_info\030\001 " + "\002(\0132\013.ServerInfo\"\034\n\032UpdateConfigurationR" + - "equest\"\035\n\033UpdateConfigurationResponse2\230\010" + + "equest\"\035\n\033UpdateConfigurationResponse2\325\010" + "\n\014AdminService\022>\n\rGetRegionInfo\022\025.GetReg" + "ionInfoRequest\032\026.GetRegionInfoResponse\022;" + "\n\014GetStoreFile\022\024.GetStoreFileRequest\032\025.G" + "etStoreFileResponse\022D\n\017GetOnlineRegion\022\027" + - ".GetOnlineRegionRequest\032\030.GetOnlineRegio" + + ".GetOnlineRegionRequest\032\030.GetOnlineRegio", "nResponse\0225\n\nOpenRegion\022\022.OpenRegionRequ" + - "est\032\023.OpenRegionResponse\0228\n\013CloseRegion\022", - "\023.CloseRegionRequest\032\024.CloseRegionRespon" + - "se\0228\n\013FlushRegion\022\023.FlushRegionRequest\032\024" + - ".FlushRegionResponse\0228\n\013SplitRegion\022\023.Sp" + - "litRegionRequest\032\024.SplitRegionResponse\022>" + - "\n\rCompactRegion\022\025.CompactRegionRequest\032\026" + - ".CompactRegionResponse\022;\n\014MergeRegions\022\024" + - ".MergeRegionsRequest\032\025.MergeRegionsRespo" + - "nse\022J\n\021ReplicateWALEntry\022\031.ReplicateWALE" + - "ntryRequest\032\032.ReplicateWALEntryResponse\022" + - "?\n\006Replay\022\031.ReplicateWALEntryRequest\032\032.R", - "eplicateWALEntryResponse\022>\n\rRollWALWrite" + - "r\022\025.RollWALWriterRequest\032\026.RollWALWriter" + - "Response\022>\n\rGetServerInfo\022\025.GetServerInf" + - "oRequest\032\026.GetServerInfoResponse\0225\n\nStop" + - "Server\022\022.StopServerRequest\032\023.StopServerR" + - "esponse\022M\n\022UpdateFavoredNodes\022\032.UpdateFa" + - "voredNodesRequest\032\033.UpdateFavoredNodesRe" + - "sponse\022P\n\023UpdateConfiguration\022\033.UpdateCo" + - "nfigurationRequest\032\034.UpdateConfiguration" + - "ResponseBA\n*org.apache.hadoop.hbase.prot", - "obuf.generatedB\013AdminProtosH\001\210\001\001\240\001\001" + "est\032\023.OpenRegionResponse\022;\n\014WarmupRegion" + + "\022\024.WarmupRegionRequest\032\025.WarmupRegionRes" + + "ponse\0228\n\013CloseRegion\022\023.CloseRegionReques" + + "t\032\024.CloseRegionResponse\0228\n\013FlushRegion\022\023" + + ".FlushRegionRequest\032\024.FlushRegionRespons" + + "e\0228\n\013SplitRegion\022\023.SplitRegionRequest\032\024." + + "SplitRegionResponse\022>\n\rCompactRegion\022\025.C" + + "ompactRegionRequest\032\026.CompactRegionRespo" + + "nse\022;\n\014MergeRegions\022\024.MergeRegionsReques", + "t\032\025.MergeRegionsResponse\022J\n\021ReplicateWAL" + + "Entry\022\031.ReplicateWALEntryRequest\032\032.Repli" + + "cateWALEntryResponse\022?\n\006Replay\022\031.Replica" + + "teWALEntryRequest\032\032.ReplicateWALEntryRes" + + "ponse\022>\n\rRollWALWriter\022\025.RollWALWriterRe" + + "quest\032\026.RollWALWriterResponse\022>\n\rGetServ" + + "erInfo\022\025.GetServerInfoRequest\032\026.GetServe" + + "rInfoResponse\0225\n\nStopServer\022\022.StopServer" + + "Request\032\023.StopServerResponse\022M\n\022UpdateFa" + + "voredNodes\022\032.UpdateFavoredNodesRequest\032\033", + ".UpdateFavoredNodesResponse\022P\n\023UpdateCon" + + "figuration\022\033.UpdateConfigurationRequest\032" + + "\034.UpdateConfigurationResponseBA\n*org.apa" + + "che.hadoop.hbase.protobuf.generatedB\013Adm" + + "inProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -22406,56 +23390,68 @@ public final class AdminProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OpenRegionResponse_descriptor, new java.lang.String[] { "OpeningState", }); - internal_static_CloseRegionRequest_descriptor = + internal_static_WarmupRegionRequest_descriptor = getDescriptor().getMessageTypes().get(8); + internal_static_WarmupRegionRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_WarmupRegionRequest_descriptor, + new java.lang.String[] { "RegionInfo", }); + internal_static_WarmupRegionResponse_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_WarmupRegionResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_WarmupRegionResponse_descriptor, + new java.lang.String[] { }); + internal_static_CloseRegionRequest_descriptor = + getDescriptor().getMessageTypes().get(10); internal_static_CloseRegionRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CloseRegionRequest_descriptor, new java.lang.String[] { "Region", "VersionOfClosingNode", "TransitionInZK", "DestinationServer", "ServerStartCode", }); internal_static_CloseRegionResponse_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(11); internal_static_CloseRegionResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CloseRegionResponse_descriptor, new java.lang.String[] { "Closed", }); internal_static_FlushRegionRequest_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(12); internal_static_FlushRegionRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_FlushRegionRequest_descriptor, new java.lang.String[] { "Region", "IfOlderThanTs", "WriteFlushWalMarker", }); internal_static_FlushRegionResponse_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(13); internal_static_FlushRegionResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_FlushRegionResponse_descriptor, new java.lang.String[] { "LastFlushTime", "Flushed", "WroteFlushWalMarker", }); internal_static_SplitRegionRequest_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(14); internal_static_SplitRegionRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SplitRegionRequest_descriptor, new java.lang.String[] { "Region", "SplitPoint", }); internal_static_SplitRegionResponse_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(15); internal_static_SplitRegionResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SplitRegionResponse_descriptor, new java.lang.String[] { }); internal_static_CompactRegionRequest_descriptor = - getDescriptor().getMessageTypes().get(14); + getDescriptor().getMessageTypes().get(16); internal_static_CompactRegionRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CompactRegionRequest_descriptor, new java.lang.String[] { "Region", "Major", "Family", }); internal_static_CompactRegionResponse_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(17); internal_static_CompactRegionResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CompactRegionResponse_descriptor, new java.lang.String[] { }); internal_static_UpdateFavoredNodesRequest_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(18); internal_static_UpdateFavoredNodesRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UpdateFavoredNodesRequest_descriptor, @@ -22467,91 +23463,91 @@ public final class AdminProtos { internal_static_UpdateFavoredNodesRequest_RegionUpdateInfo_descriptor, new java.lang.String[] { "Region", "FavoredNodes", }); internal_static_UpdateFavoredNodesResponse_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(19); internal_static_UpdateFavoredNodesResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UpdateFavoredNodesResponse_descriptor, new java.lang.String[] { "Response", }); internal_static_MergeRegionsRequest_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(20); internal_static_MergeRegionsRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MergeRegionsRequest_descriptor, new java.lang.String[] { "RegionA", "RegionB", "Forcible", }); internal_static_MergeRegionsResponse_descriptor = - getDescriptor().getMessageTypes().get(19); + getDescriptor().getMessageTypes().get(21); internal_static_MergeRegionsResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MergeRegionsResponse_descriptor, new java.lang.String[] { }); internal_static_WALEntry_descriptor = - getDescriptor().getMessageTypes().get(20); + getDescriptor().getMessageTypes().get(22); internal_static_WALEntry_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_WALEntry_descriptor, new java.lang.String[] { "Key", "KeyValueBytes", "AssociatedCellCount", }); internal_static_ReplicateWALEntryRequest_descriptor = - getDescriptor().getMessageTypes().get(21); + getDescriptor().getMessageTypes().get(23); internal_static_ReplicateWALEntryRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ReplicateWALEntryRequest_descriptor, new java.lang.String[] { "Entry", }); internal_static_ReplicateWALEntryResponse_descriptor = - getDescriptor().getMessageTypes().get(22); + getDescriptor().getMessageTypes().get(24); internal_static_ReplicateWALEntryResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ReplicateWALEntryResponse_descriptor, new java.lang.String[] { }); internal_static_RollWALWriterRequest_descriptor = - getDescriptor().getMessageTypes().get(23); + getDescriptor().getMessageTypes().get(25); internal_static_RollWALWriterRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RollWALWriterRequest_descriptor, new java.lang.String[] { }); internal_static_RollWALWriterResponse_descriptor = - getDescriptor().getMessageTypes().get(24); + getDescriptor().getMessageTypes().get(26); internal_static_RollWALWriterResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RollWALWriterResponse_descriptor, new java.lang.String[] { "RegionToFlush", }); internal_static_StopServerRequest_descriptor = - getDescriptor().getMessageTypes().get(25); + getDescriptor().getMessageTypes().get(27); internal_static_StopServerRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_StopServerRequest_descriptor, new java.lang.String[] { "Reason", }); internal_static_StopServerResponse_descriptor = - getDescriptor().getMessageTypes().get(26); + getDescriptor().getMessageTypes().get(28); internal_static_StopServerResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_StopServerResponse_descriptor, new java.lang.String[] { }); internal_static_GetServerInfoRequest_descriptor = - getDescriptor().getMessageTypes().get(27); + getDescriptor().getMessageTypes().get(29); internal_static_GetServerInfoRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_GetServerInfoRequest_descriptor, new java.lang.String[] { }); internal_static_ServerInfo_descriptor = - getDescriptor().getMessageTypes().get(28); + getDescriptor().getMessageTypes().get(30); internal_static_ServerInfo_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ServerInfo_descriptor, new java.lang.String[] { "ServerName", "WebuiPort", }); internal_static_GetServerInfoResponse_descriptor = - getDescriptor().getMessageTypes().get(29); + getDescriptor().getMessageTypes().get(31); internal_static_GetServerInfoResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_GetServerInfoResponse_descriptor, new java.lang.String[] { "ServerInfo", }); internal_static_UpdateConfigurationRequest_descriptor = - getDescriptor().getMessageTypes().get(30); + getDescriptor().getMessageTypes().get(32); internal_static_UpdateConfigurationRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UpdateConfigurationRequest_descriptor, new java.lang.String[] { }); internal_static_UpdateConfigurationResponse_descriptor = - getDescriptor().getMessageTypes().get(31); + getDescriptor().getMessageTypes().get(33); internal_static_UpdateConfigurationResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UpdateConfigurationResponse_descriptor, diff --git a/hbase-protocol/src/main/protobuf/Admin.proto b/hbase-protocol/src/main/protobuf/Admin.proto index 5f0572a540f..1df0958ac76 100644 --- a/hbase-protocol/src/main/protobuf/Admin.proto +++ b/hbase-protocol/src/main/protobuf/Admin.proto @@ -90,6 +90,14 @@ message OpenRegionResponse { } } +message WarmupRegionRequest { + + required RegionInfo regionInfo = 1; +} + +message WarmupRegionResponse { +} + /** * Closes the specified region and will use or not use ZK during the close * according to the specified flag. @@ -253,6 +261,9 @@ service AdminService { rpc OpenRegion(OpenRegionRequest) returns(OpenRegionResponse); + rpc WarmupRegion(WarmupRegionRequest) + returns(WarmupRegionResponse); + rpc CloseRegion(CloseRegionRequest) returns(CloseRegionResponse); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index b1a1cc73a4d..0aebe0fa3bc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java @@ -3394,6 +3394,7 @@ public class AssignmentManager extends ZooKeeperListener { * @param plan Plan to execute. */ public void balance(final RegionPlan plan) { + HRegionInfo hri = plan.getRegionInfo(); TableName tableName = hri.getTable(); if (tableStateManager.isTableState(tableName, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 7984d1b47ed..87774eaa384 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1283,6 +1283,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server { return; } } + // warmup the region on the destination before initiating the move. this call + // is synchronous and takes some time. doing it before the source region gets + // closed + serverManager.sendRegionWarmup(rp.getDestination(), hri); + LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer"); this.assignmentManager.balance(rp); if (this.cpHost != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index 9f4aa08104b..ab0bcdb95bd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -815,6 +815,27 @@ public class ServerManager { return sendRegionClose(server, region, versionOfClosingNode, null, true); } + /** + * Sends a WARMUP RPC to the specified server to warmup the specified region. + *

+ * A region server could reject the close request because it either does not + * have the specified region or the region is being split. + * @param server server to warmup a region + * @param region region to warmup + */ + public void sendRegionWarmup(ServerName server, + HRegionInfo region) { + if (server == null) return; + try { + AdminService.BlockingInterface admin = getRsAdmin(server); + ProtobufUtil.warmupRegion(admin, region); + } catch (IOException e) { + LOG.error("Received exception in RPC for warmup server:" + + server + "region: " + region + + "exception: " + e); + } + } + /** * Contacts a region server and waits up to timeout ms * to close the region. This bypasses the active hmaster. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 704947daf32..92332756519 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -849,11 +849,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // status.setStatus("Writing region info on filesystem"); fs.checkRegionInfoOnFilesystem(); - - // Initialize all the HStores status.setStatus("Initializing all the Stores"); - long maxSeqId = initializeRegionStores(reporter, status); + long maxSeqId = initializeRegionStores(reporter, status, false); this.lastReplayedOpenRegionSeqId = maxSeqId; this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this)); @@ -916,8 +914,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return nextSeqid; } - private long initializeRegionStores(final CancelableProgressable reporter, MonitoredTask status) + private long initializeRegionStores(final CancelableProgressable reporter, MonitoredTask status, + boolean warmupOnly) throws IOException { + // Load in all the HStores. long maxSeqId = -1; @@ -979,7 +979,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // } } } - if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) { + if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this) && !warmupOnly) { // Recover any edits if available. maxSeqId = Math.max(maxSeqId, replayRecoveredEditsIfAny( this.fs.getRegionDir(), maxSeqIdInStores, reporter, status)); @@ -989,6 +989,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return maxSeqId; } + private void initializeWarmup(final CancelableProgressable reporter) throws IOException { + MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this); + + // Initialize all the HStores + status.setStatus("Warming up all the Stores"); + initializeRegionStores(reporter, status, true); + } + private void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException { Map> storeFiles = new TreeMap>(Bytes.BYTES_COMPARATOR); @@ -6333,6 +6341,35 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return this; } + public static void warmupHRegion(final HRegionInfo info, + final HTableDescriptor htd, final WAL wal, final Configuration conf, + final RegionServerServices rsServices, + final CancelableProgressable reporter) + throws IOException { + + if (info == null) throw new NullPointerException("Passed region info is null"); + + if (LOG.isDebugEnabled()) { + LOG.debug("HRegion.Warming up region: " + info); + } + + Path rootDir = FSUtils.getRootDir(conf); + Path tableDir = FSUtils.getTableDir(rootDir, info.getTable()); + + FileSystem fs = null; + if (rsServices != null) { + fs = rsServices.getFileSystem(); + } + if (fs == null) { + fs = FileSystem.get(conf); + } + + HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices); + r.initializeWarmup(reporter); + r.close(); + } + + private void checkCompressionCodecs() throws IOException { for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) { CompressionTest.testCompression(fam.getCompression()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index ba7b70ca23c..b8b2eb012c9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -108,6 +108,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest; @@ -1452,6 +1454,57 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return builder.build(); } + /** + * Wamrmup a region on this server. + * + * This method should only be called by Master. It synchrnously opens the region and + * closes the region bringing the most important pages in cache. + *

+ * + * @param controller the RPC controller + * @param request the request + * @throws ServiceException + */ + public WarmupRegionResponse warmupRegion(final RpcController controller, + final WarmupRegionRequest request) throws ServiceException { + + RegionInfo regionInfo = request.getRegionInfo(); + final HRegionInfo region = HRegionInfo.convert(regionInfo); + HTableDescriptor htd; + WarmupRegionResponse response = WarmupRegionResponse.getDefaultInstance(); + + try { + String encodedName = region.getEncodedName(); + byte[] encodedNameBytes = region.getEncodedNameAsBytes(); + final HRegion onlineRegion = regionServer.getFromOnlineRegions(encodedName); + + if (onlineRegion != null) { + LOG.info("Region already online. Skipping warming up " + region); + return response; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Warming up Region " + region.getRegionNameAsString()); + } + + htd = regionServer.tableDescriptors.get(region.getTable()); + + if (regionServer.getRegionsInTransitionInRS().containsKey(encodedNameBytes)) { + LOG.info("Region is in transition. Skipping warmup " + region); + return response; + } + + HRegion.warmupHRegion(region, htd, regionServer.getWAL(region), + regionServer.getConfiguration(), regionServer, null); + + } catch (IOException ie) { + LOG.error("Failed warming up region " + region.getRegionNameAsString(), ie); + throw new ServiceException(ie); + } + + return response; + } + /** * Replay the given changes when distributedLogReplay WAL edits from a failed RS. The guarantee is * that the given mutations will be durable on the receiving RS if this method returns without any diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 7b5c4942b87..00f8509f01b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -65,6 +65,8 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsReques import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.MergeRegionsResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WarmupRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest; @@ -452,6 +454,12 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { return null; } + @Override + public WarmupRegionResponse warmupRegion(RpcController controller, + WarmupRegionRequest request) throws ServiceException { + //TODO Auto-generated method stub + return null; + } @Override public CloseRegionResponse closeRegion(RpcController controller, CloseRegionRequest request) throws ServiceException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java new file mode 100644 index 00000000000..ab08ef02335 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java @@ -0,0 +1,163 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master; + +import static org.apache.hadoop.hbase.regionserver.HRegion.warmupHRegion; +import java.io.IOException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.experimental.categories.Category; +import org.junit.BeforeClass; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.After; +import org.junit.Test; + +/** + * Run tests that use the HBase clients; {@link HTable}. + * Sets up the HBase mini cluster once at start and runs through all client tests. + * Each creates a table named for the method and does its stuff against that. + */ +@Category(LargeTests.class) +@SuppressWarnings ("deprecation") +public class TestWarmupRegion { + final Log LOG = LogFactory.getLog(getClass()); + protected TableName TABLENAME = TableName.valueOf("testPurgeFutureDeletes"); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static byte [] ROW = Bytes.toBytes("testRow"); + private static byte [] FAMILY = Bytes.toBytes("testFamily"); + private static byte [] QUALIFIER = Bytes.toBytes("testQualifier"); + private static byte [] VALUE = Bytes.toBytes("testValue"); + private static byte[] COLUMN = Bytes.toBytes("column"); + private static int numRows = 10000; + protected static int SLAVES = 3; + private static MiniHBaseCluster myCluster; + private static Table table; + + /** + * @throws java.lang.Exception + */ + @BeforeClass + public static void setUpBeforeClass() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + TEST_UTIL.startMiniCluster(SLAVES); + } + + /** + * @throws java.lang.Exception + */ + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * @throws java.lang.Exception + */ + @Before + public void setUp() throws Exception { + table = TEST_UTIL.createTable(TABLENAME, FAMILY); + + // future timestamp + for (int i = 0; i < numRows; i++) { + long ts = System.currentTimeMillis() * 2; + Put put = new Put(ROW, ts); + put.add(FAMILY, COLUMN, VALUE); + table.put(put); + } + + // major compaction, purged future deletes + TEST_UTIL.getHBaseAdmin().flush(TABLENAME); + TEST_UTIL.getHBaseAdmin().majorCompact(TABLENAME); + + // waiting for the major compaction to complete + TEST_UTIL.waitFor(6000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws IOException { + return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) == + AdminProtos.GetRegionInfoResponse.CompactionState.NONE; + } + }); + + table.close(); + } + + + /** + * @throws java.lang.Exception + */ + @After + public void tearDown() throws Exception { + // Nothing to do. + } + + protected void runwarmup() throws InterruptedException{ + Thread thread = new Thread(new Runnable() { + @Override + public void run() { + HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0); + HRegion region = TEST_UTIL.getMiniHBaseCluster().getRegions(TABLENAME).get(0); + HRegionInfo info = region.getRegionInfo(); + + try { + HTableDescriptor htd = table.getTableDescriptor(); + for (int i = 0; i < 10; i++) { + warmupHRegion(info, htd, rs.getWAL(info), rs.getConfiguration(), rs, null); + } + + } catch (IOException ie) { + LOG.error("Failed warming up region " + info.getRegionNameAsString(), ie); + } + } + }); + thread.start(); + thread.join(); + } + + /** + * Basic client side validation of HBASE-4536 + */ + @Test + public void testWarmup() throws Exception { + int serverid = 0; + HRegion region = TEST_UTIL.getMiniHBaseCluster().getRegions(TABLENAME).get(0); + HRegionInfo info = region.getRegionInfo(); + runwarmup(); + for (int i = 0; i < 10; i++) { + HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(serverid); + byte [] destName = Bytes.toBytes(rs.getServerName().toString()); + TEST_UTIL.getMiniHBaseCluster().getMaster().move(info.getEncodedNameAsBytes(), destName); + serverid = (serverid + 1) % 2; + } + } +}