HBASE-23590 : Update maxStoreFileRefCount to maxCompactedStoreFileRefCount for auto region recovery based on old reader references
Signed-off-by: Anoop Sam John <anoopsamjohn@apache.org>
This commit is contained in:
parent
27fc96e076
commit
1451063115
|
@ -198,8 +198,8 @@ public class RegionLoad {
|
|||
* @return the max reference count for any store file among all stores files
|
||||
* of this region
|
||||
*/
|
||||
public int getMaxStoreFileRefCount() {
|
||||
return regionLoadPB.getMaxStoreFileRefCount();
|
||||
public int getMaxCompactedStoreFileRefCount() {
|
||||
return regionLoadPB.getMaxCompactedStoreFileRefCount();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -212,7 +212,8 @@ public class RegionLoad {
|
|||
sb = Strings.appendKeyValue(sb, "numberOfStorefiles",
|
||||
this.getStorefiles());
|
||||
sb = Strings.appendKeyValue(sb, "storeRefCount", this.getStoreRefCount());
|
||||
sb = Strings.appendKeyValue(sb, "maxStoreFileRefCount", this.getMaxStoreFileRefCount());
|
||||
sb = Strings.appendKeyValue(sb, "maxCompactedStoreFileRefCount",
|
||||
this.getMaxCompactedStoreFileRefCount());
|
||||
sb = Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
|
||||
this.getStoreUncompressedSizeMB());
|
||||
sb = Strings.appendKeyValue(sb, "lastMajorCompactionTimestamp",
|
||||
|
|
|
@ -1657,19 +1657,25 @@ possible configurations would overwhelm and obscure the important.
|
|||
<name>hbase.regions.recovery.store.file.ref.count</name>
|
||||
<value>-1</value>
|
||||
<description>
|
||||
Very large ref count on a file indicates
|
||||
that it is a ref leak on that object. Such files
|
||||
can not be removed even after it is invalidated
|
||||
via compaction. Only way to recover in such
|
||||
scenario is to reopen the region which can
|
||||
release all resources, like the refcount, leases, etc.
|
||||
This config represents Store files Ref Count threshold
|
||||
value considered for reopening regions.
|
||||
Any region with store files ref count > this value
|
||||
would be eligible for reopening by master.
|
||||
Default value -1 indicates this feature is turned off.
|
||||
Only positive integer value should be provided to enable
|
||||
this feature.
|
||||
Very large number of ref count on a compacted
|
||||
store file indicates that it is a ref leak
|
||||
on that object(compacted store file).
|
||||
Such files can not be removed after
|
||||
it is invalidated via compaction.
|
||||
Only way to recover in such scenario is to
|
||||
reopen the region which can release
|
||||
all resources, like the refcount,
|
||||
leases, etc. This config represents Store files Ref
|
||||
Count threshold value considered for reopening
|
||||
regions. Any region with compacted store files
|
||||
ref count > this value would be eligible for
|
||||
reopening by master. Here, we get the max
|
||||
refCount among all refCounts on all
|
||||
compacted away store files that belong to a
|
||||
particular region. Default value -1 indicates
|
||||
this feature is turned off. Only positive
|
||||
integer value should be provided to
|
||||
enable this feature.
|
||||
</description>
|
||||
</property>
|
||||
</configuration>
|
||||
|
|
|
@ -231,7 +231,7 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo
|
|||
String STOREFILE_COUNT_DESC = "Number of Store Files";
|
||||
String STORE_REF_COUNT = "storeRefCount";
|
||||
String STORE_REF_COUNT_DESC = "Store reference count";
|
||||
String MAX_STORE_FILE_REF_COUNT = "maxStoreFileRefCount";
|
||||
String MAX_COMPACTED_STORE_FILE_REF_COUNT = "maxCompactedStoreFileRefCount";
|
||||
String MEMSTORE_SIZE = "memStoreSize";
|
||||
String MEMSTORE_SIZE_DESC = "Size of the memstore";
|
||||
String STOREFILE_SIZE = "storeFileSize";
|
||||
|
|
|
@ -149,8 +149,8 @@ public interface MetricsRegionWrapper {
|
|||
long getStoreRefCount();
|
||||
|
||||
/**
|
||||
* @return the max reference count for any store file among all stores files
|
||||
* of this region
|
||||
* @return the max number of references active on any store file among
|
||||
* all compacted store files that belong to this region
|
||||
*/
|
||||
int getMaxStoreFileRefCount();
|
||||
int getMaxCompactedStoreFileRefCount();
|
||||
}
|
||||
|
|
|
@ -218,9 +218,10 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
|
|||
MetricsRegionServerSource.STORE_REF_COUNT),
|
||||
this.regionWrapper.getStoreRefCount());
|
||||
mrb.addGauge(Interns.info(
|
||||
regionNamePrefix + MetricsRegionServerSource.MAX_STORE_FILE_REF_COUNT,
|
||||
MetricsRegionServerSource.MAX_STORE_FILE_REF_COUNT),
|
||||
this.regionWrapper.getMaxStoreFileRefCount());
|
||||
regionNamePrefix + MetricsRegionServerSource.MAX_COMPACTED_STORE_FILE_REF_COUNT,
|
||||
MetricsRegionServerSource.MAX_COMPACTED_STORE_FILE_REF_COUNT),
|
||||
this.regionWrapper.getMaxCompactedStoreFileRefCount()
|
||||
);
|
||||
mrb.addGauge(Interns.info(
|
||||
regionNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE,
|
||||
MetricsRegionServerSource.MEMSTORE_SIZE_DESC),
|
||||
|
|
|
@ -97,7 +97,7 @@ public class TestMetricsRegionSourceImpl {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int getMaxStoreFileRefCount() {
|
||||
public int getMaxCompactedStoreFileRefCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -3632,27 +3632,27 @@ public final class ClusterStatusProtos {
|
|||
*/
|
||||
int getStoreRefCount();
|
||||
|
||||
// optional int32 max_store_file_ref_count = 22 [default = 0];
|
||||
// optional int32 max_compacted_store_file_ref_count = 22 [default = 0];
|
||||
/**
|
||||
* <code>optional int32 max_store_file_ref_count = 22 [default = 0];</code>
|
||||
* <code>optional int32 max_compacted_store_file_ref_count = 22 [default = 0];</code>
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
* </pre>
|
||||
*/
|
||||
boolean hasMaxStoreFileRefCount();
|
||||
boolean hasMaxCompactedStoreFileRefCount();
|
||||
/**
|
||||
* <code>optional int32 max_store_file_ref_count = 22 [default = 0];</code>
|
||||
* <code>optional int32 max_compacted_store_file_ref_count = 22 [default = 0];</code>
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
* </pre>
|
||||
*/
|
||||
int getMaxStoreFileRefCount();
|
||||
int getMaxCompactedStoreFileRefCount();
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code hbase.pb.RegionLoad}
|
||||
|
@ -3813,7 +3813,7 @@ public final class ClusterStatusProtos {
|
|||
}
|
||||
case 176: {
|
||||
bitField0_ |= 0x00040000;
|
||||
maxStoreFileRefCount_ = input.readInt32();
|
||||
maxCompactedStoreFileRefCount_ = input.readInt32();
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -4357,32 +4357,32 @@ public final class ClusterStatusProtos {
|
|||
return storeRefCount_;
|
||||
}
|
||||
|
||||
// optional int32 max_store_file_ref_count = 22 [default = 0];
|
||||
public static final int MAX_STORE_FILE_REF_COUNT_FIELD_NUMBER = 22;
|
||||
private int maxStoreFileRefCount_;
|
||||
// optional int32 max_compacted_store_file_ref_count = 22 [default = 0];
|
||||
public static final int MAX_COMPACTED_STORE_FILE_REF_COUNT_FIELD_NUMBER = 22;
|
||||
private int maxCompactedStoreFileRefCount_;
|
||||
/**
|
||||
* <code>optional int32 max_store_file_ref_count = 22 [default = 0];</code>
|
||||
* <code>optional int32 max_compacted_store_file_ref_count = 22 [default = 0];</code>
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
* </pre>
|
||||
*/
|
||||
public boolean hasMaxStoreFileRefCount() {
|
||||
public boolean hasMaxCompactedStoreFileRefCount() {
|
||||
return ((bitField0_ & 0x00040000) == 0x00040000);
|
||||
}
|
||||
/**
|
||||
* <code>optional int32 max_store_file_ref_count = 22 [default = 0];</code>
|
||||
* <code>optional int32 max_compacted_store_file_ref_count = 22 [default = 0];</code>
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
* </pre>
|
||||
*/
|
||||
public int getMaxStoreFileRefCount() {
|
||||
return maxStoreFileRefCount_;
|
||||
public int getMaxCompactedStoreFileRefCount() {
|
||||
return maxCompactedStoreFileRefCount_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
|
@ -4405,7 +4405,7 @@ public final class ClusterStatusProtos {
|
|||
lastMajorCompactionTs_ = 0L;
|
||||
storeCompleteSequenceId_ = java.util.Collections.emptyList();
|
||||
storeRefCount_ = 0;
|
||||
maxStoreFileRefCount_ = 0;
|
||||
maxCompactedStoreFileRefCount_ = 0;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -4491,7 +4491,7 @@ public final class ClusterStatusProtos {
|
|||
output.writeInt32(21, storeRefCount_);
|
||||
}
|
||||
if (((bitField0_ & 0x00040000) == 0x00040000)) {
|
||||
output.writeInt32(22, maxStoreFileRefCount_);
|
||||
output.writeInt32(22, maxCompactedStoreFileRefCount_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
@ -4580,7 +4580,7 @@ public final class ClusterStatusProtos {
|
|||
}
|
||||
if (((bitField0_ & 0x00040000) == 0x00040000)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeInt32Size(22, maxStoreFileRefCount_);
|
||||
.computeInt32Size(22, maxCompactedStoreFileRefCount_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
|
@ -4696,10 +4696,10 @@ public final class ClusterStatusProtos {
|
|||
result = result && (getStoreRefCount()
|
||||
== other.getStoreRefCount());
|
||||
}
|
||||
result = result && (hasMaxStoreFileRefCount() == other.hasMaxStoreFileRefCount());
|
||||
if (hasMaxStoreFileRefCount()) {
|
||||
result = result && (getMaxStoreFileRefCount()
|
||||
== other.getMaxStoreFileRefCount());
|
||||
result = result && (hasMaxCompactedStoreFileRefCount() == other.hasMaxCompactedStoreFileRefCount());
|
||||
if (hasMaxCompactedStoreFileRefCount()) {
|
||||
result = result && (getMaxCompactedStoreFileRefCount()
|
||||
== other.getMaxCompactedStoreFileRefCount());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
|
@ -4791,9 +4791,9 @@ public final class ClusterStatusProtos {
|
|||
hash = (37 * hash) + STORE_REF_COUNT_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getStoreRefCount();
|
||||
}
|
||||
if (hasMaxStoreFileRefCount()) {
|
||||
hash = (37 * hash) + MAX_STORE_FILE_REF_COUNT_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getMaxStoreFileRefCount();
|
||||
if (hasMaxCompactedStoreFileRefCount()) {
|
||||
hash = (37 * hash) + MAX_COMPACTED_STORE_FILE_REF_COUNT_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getMaxCompactedStoreFileRefCount();
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
memoizedHashCode = hash;
|
||||
|
@ -4952,7 +4952,7 @@ public final class ClusterStatusProtos {
|
|||
}
|
||||
storeRefCount_ = 0;
|
||||
bitField0_ = (bitField0_ & ~0x00040000);
|
||||
maxStoreFileRefCount_ = 0;
|
||||
maxCompactedStoreFileRefCount_ = 0;
|
||||
bitField0_ = (bitField0_ & ~0x00080000);
|
||||
return this;
|
||||
}
|
||||
|
@ -5070,7 +5070,7 @@ public final class ClusterStatusProtos {
|
|||
if (((from_bitField0_ & 0x00080000) == 0x00080000)) {
|
||||
to_bitField0_ |= 0x00040000;
|
||||
}
|
||||
result.maxStoreFileRefCount_ = maxStoreFileRefCount_;
|
||||
result.maxCompactedStoreFileRefCount_ = maxCompactedStoreFileRefCount_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -5167,8 +5167,8 @@ public final class ClusterStatusProtos {
|
|||
if (other.hasStoreRefCount()) {
|
||||
setStoreRefCount(other.getStoreRefCount());
|
||||
}
|
||||
if (other.hasMaxStoreFileRefCount()) {
|
||||
setMaxStoreFileRefCount(other.getMaxStoreFileRefCount());
|
||||
if (other.hasMaxCompactedStoreFileRefCount()) {
|
||||
setMaxCompactedStoreFileRefCount(other.getMaxCompactedStoreFileRefCount());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
|
@ -6509,59 +6509,59 @@ public final class ClusterStatusProtos {
|
|||
return this;
|
||||
}
|
||||
|
||||
// optional int32 max_store_file_ref_count = 22 [default = 0];
|
||||
private int maxStoreFileRefCount_ ;
|
||||
// optional int32 max_compacted_store_file_ref_count = 22 [default = 0];
|
||||
private int maxCompactedStoreFileRefCount_ ;
|
||||
/**
|
||||
* <code>optional int32 max_store_file_ref_count = 22 [default = 0];</code>
|
||||
* <code>optional int32 max_compacted_store_file_ref_count = 22 [default = 0];</code>
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
* </pre>
|
||||
*/
|
||||
public boolean hasMaxStoreFileRefCount() {
|
||||
public boolean hasMaxCompactedStoreFileRefCount() {
|
||||
return ((bitField0_ & 0x00080000) == 0x00080000);
|
||||
}
|
||||
/**
|
||||
* <code>optional int32 max_store_file_ref_count = 22 [default = 0];</code>
|
||||
* <code>optional int32 max_compacted_store_file_ref_count = 22 [default = 0];</code>
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
* </pre>
|
||||
*/
|
||||
public int getMaxStoreFileRefCount() {
|
||||
return maxStoreFileRefCount_;
|
||||
public int getMaxCompactedStoreFileRefCount() {
|
||||
return maxCompactedStoreFileRefCount_;
|
||||
}
|
||||
/**
|
||||
* <code>optional int32 max_store_file_ref_count = 22 [default = 0];</code>
|
||||
* <code>optional int32 max_compacted_store_file_ref_count = 22 [default = 0];</code>
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
* </pre>
|
||||
*/
|
||||
public Builder setMaxStoreFileRefCount(int value) {
|
||||
public Builder setMaxCompactedStoreFileRefCount(int value) {
|
||||
bitField0_ |= 0x00080000;
|
||||
maxStoreFileRefCount_ = value;
|
||||
maxCompactedStoreFileRefCount_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional int32 max_store_file_ref_count = 22 [default = 0];</code>
|
||||
* <code>optional int32 max_compacted_store_file_ref_count = 22 [default = 0];</code>
|
||||
*
|
||||
* <pre>
|
||||
**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
* </pre>
|
||||
*/
|
||||
public Builder clearMaxStoreFileRefCount() {
|
||||
public Builder clearMaxCompactedStoreFileRefCount() {
|
||||
bitField0_ = (bitField0_ & ~0x00080000);
|
||||
maxStoreFileRefCount_ = 0;
|
||||
maxCompactedStoreFileRefCount_ = 0;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
@ -14983,7 +14983,7 @@ public final class ClusterStatusProtos {
|
|||
"e\030\001 \002(\014\022\023\n\013sequence_id\030\002 \002(\004\"p\n\026RegionSt" +
|
||||
"oreSequenceIds\022 \n\030last_flushed_sequence_" +
|
||||
"id\030\001 \002(\004\0224\n\021store_sequence_id\030\002 \003(\0132\031.hb" +
|
||||
"ase.pb.StoreSequenceId\"\225\005\n\nRegionLoad\0223\n" +
|
||||
"ase.pb.StoreSequenceId\"\237\005\n\nRegionLoad\0223\n" +
|
||||
"\020region_specifier\030\001 \002(\0132\031.hbase.pb.Regio" +
|
||||
"nSpecifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstorefiles",
|
||||
"\030\003 \001(\r\022\"\n\032store_uncompressed_size_MB\030\004 \001" +
|
||||
|
@ -14999,38 +14999,39 @@ public final class ClusterStatusProtos {
|
|||
"\002\022#\n\030last_major_compaction_ts\030\021 \001(\004:\0010\022=" +
|
||||
"\n\032store_complete_sequence_id\030\022 \003(\0132\031.hba" +
|
||||
"se.pb.StoreSequenceId\022\032\n\017store_ref_count" +
|
||||
"\030\025 \001(\005:\0010\022#\n\030max_store_file_ref_count\030\026 " +
|
||||
"\001(\005:\0010\"T\n\023ReplicationLoadSink\022\032\n\022ageOfLa" +
|
||||
"stAppliedOp\030\001 \002(\004\022!\n\031timeStampsOfLastApp" +
|
||||
"liedOp\030\002 \002(\004\"\225\001\n\025ReplicationLoadSource\022\016" +
|
||||
"\n\006peerID\030\001 \002(\t\022\032\n\022ageOfLastShippedOp\030\002 \002" +
|
||||
"(\004\022\026\n\016sizeOfLogQueue\030\003 \002(\r\022 \n\030timeStampO" +
|
||||
"fLastShippedOp\030\004 \002(\004\022\026\n\016replicationLag\030\005",
|
||||
" \002(\004\"\212\003\n\nServerLoad\022\032\n\022number_of_request" +
|
||||
"s\030\001 \001(\004\022 \n\030total_number_of_requests\030\002 \001(" +
|
||||
"\004\022\024\n\014used_heap_MB\030\003 \001(\r\022\023\n\013max_heap_MB\030\004" +
|
||||
" \001(\r\022*\n\014region_loads\030\005 \003(\0132\024.hbase.pb.Re" +
|
||||
"gionLoad\022+\n\014coprocessors\030\006 \003(\0132\025.hbase.p" +
|
||||
"b.Coprocessor\022\031\n\021report_start_time\030\007 \001(\004" +
|
||||
"\022\027\n\017report_end_time\030\010 \001(\004\022\030\n\020info_server" +
|
||||
"_port\030\t \001(\r\0227\n\016replLoadSource\030\n \003(\0132\037.hb" +
|
||||
"ase.pb.ReplicationLoadSource\0223\n\014replLoad" +
|
||||
"Sink\030\013 \001(\0132\035.hbase.pb.ReplicationLoadSin",
|
||||
"k\"a\n\016LiveServerInfo\022$\n\006server\030\001 \002(\0132\024.hb" +
|
||||
"ase.pb.ServerName\022)\n\013server_load\030\002 \002(\0132\024" +
|
||||
".hbase.pb.ServerLoad\"\250\003\n\rClusterStatus\0228" +
|
||||
"\n\rhbase_version\030\001 \001(\0132!.hbase.pb.HBaseVe" +
|
||||
"rsionFileContent\022.\n\014live_servers\030\002 \003(\0132\030" +
|
||||
".hbase.pb.LiveServerInfo\022*\n\014dead_servers" +
|
||||
"\030\003 \003(\0132\024.hbase.pb.ServerName\022;\n\025regions_" +
|
||||
"in_transition\030\004 \003(\0132\034.hbase.pb.RegionInT" +
|
||||
"ransition\022\'\n\ncluster_id\030\005 \001(\0132\023.hbase.pb" +
|
||||
".ClusterId\0222\n\023master_coprocessors\030\006 \003(\0132",
|
||||
"\025.hbase.pb.Coprocessor\022$\n\006master\030\007 \001(\0132\024" +
|
||||
".hbase.pb.ServerName\022,\n\016backup_masters\030\010" +
|
||||
" \003(\0132\024.hbase.pb.ServerName\022\023\n\013balancer_o" +
|
||||
"n\030\t \001(\010BF\n*org.apache.hadoop.hbase.proto" +
|
||||
"buf.generatedB\023ClusterStatusProtosH\001\240\001\001"
|
||||
"\030\025 \001(\005:\0010\022-\n\"max_compacted_store_file_re" +
|
||||
"f_count\030\026 \001(\005:\0010\"T\n\023ReplicationLoadSink\022" +
|
||||
"\032\n\022ageOfLastAppliedOp\030\001 \002(\004\022!\n\031timeStamp" +
|
||||
"sOfLastAppliedOp\030\002 \002(\004\"\225\001\n\025ReplicationLo" +
|
||||
"adSource\022\016\n\006peerID\030\001 \002(\t\022\032\n\022ageOfLastShi" +
|
||||
"ppedOp\030\002 \002(\004\022\026\n\016sizeOfLogQueue\030\003 \002(\r\022 \n\030" +
|
||||
"timeStampOfLastShippedOp\030\004 \002(\004\022\026\n\016replic",
|
||||
"ationLag\030\005 \002(\004\"\212\003\n\nServerLoad\022\032\n\022number_" +
|
||||
"of_requests\030\001 \001(\004\022 \n\030total_number_of_req" +
|
||||
"uests\030\002 \001(\004\022\024\n\014used_heap_MB\030\003 \001(\r\022\023\n\013max" +
|
||||
"_heap_MB\030\004 \001(\r\022*\n\014region_loads\030\005 \003(\0132\024.h" +
|
||||
"base.pb.RegionLoad\022+\n\014coprocessors\030\006 \003(\013" +
|
||||
"2\025.hbase.pb.Coprocessor\022\031\n\021report_start_" +
|
||||
"time\030\007 \001(\004\022\027\n\017report_end_time\030\010 \001(\004\022\030\n\020i" +
|
||||
"nfo_server_port\030\t \001(\r\0227\n\016replLoadSource\030" +
|
||||
"\n \003(\0132\037.hbase.pb.ReplicationLoadSource\0223" +
|
||||
"\n\014replLoadSink\030\013 \001(\0132\035.hbase.pb.Replicat",
|
||||
"ionLoadSink\"a\n\016LiveServerInfo\022$\n\006server\030" +
|
||||
"\001 \002(\0132\024.hbase.pb.ServerName\022)\n\013server_lo" +
|
||||
"ad\030\002 \002(\0132\024.hbase.pb.ServerLoad\"\250\003\n\rClust" +
|
||||
"erStatus\0228\n\rhbase_version\030\001 \001(\0132!.hbase." +
|
||||
"pb.HBaseVersionFileContent\022.\n\014live_serve" +
|
||||
"rs\030\002 \003(\0132\030.hbase.pb.LiveServerInfo\022*\n\014de" +
|
||||
"ad_servers\030\003 \003(\0132\024.hbase.pb.ServerName\022;" +
|
||||
"\n\025regions_in_transition\030\004 \003(\0132\034.hbase.pb" +
|
||||
".RegionInTransition\022\'\n\ncluster_id\030\005 \001(\0132" +
|
||||
"\023.hbase.pb.ClusterId\0222\n\023master_coprocess",
|
||||
"ors\030\006 \003(\0132\025.hbase.pb.Coprocessor\022$\n\006mast" +
|
||||
"er\030\007 \001(\0132\024.hbase.pb.ServerName\022,\n\016backup" +
|
||||
"_masters\030\010 \003(\0132\024.hbase.pb.ServerName\022\023\n\013" +
|
||||
"balancer_on\030\t \001(\010BF\n*org.apache.hadoop.h" +
|
||||
"base.protobuf.generatedB\023ClusterStatusPr" +
|
||||
"otosH\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -15066,7 +15067,7 @@ public final class ClusterStatusProtos {
|
|||
internal_static_hbase_pb_RegionLoad_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_hbase_pb_RegionLoad_descriptor,
|
||||
new java.lang.String[] { "RegionSpecifier", "Stores", "Storefiles", "StoreUncompressedSizeMB", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", "ReadRequestsCount", "WriteRequestsCount", "TotalCompactingKVs", "CurrentCompactedKVs", "RootIndexSizeKB", "TotalStaticIndexSizeKB", "TotalStaticBloomSizeKB", "CompleteSequenceId", "DataLocality", "LastMajorCompactionTs", "StoreCompleteSequenceId", "StoreRefCount", "MaxStoreFileRefCount", });
|
||||
new java.lang.String[] { "RegionSpecifier", "Stores", "Storefiles", "StoreUncompressedSizeMB", "StorefileSizeMB", "MemstoreSizeMB", "StorefileIndexSizeMB", "ReadRequestsCount", "WriteRequestsCount", "TotalCompactingKVs", "CurrentCompactedKVs", "RootIndexSizeKB", "TotalStaticIndexSizeKB", "TotalStaticBloomSizeKB", "CompleteSequenceId", "DataLocality", "LastMajorCompactionTs", "StoreCompleteSequenceId", "StoreRefCount", "MaxCompactedStoreFileRefCount", });
|
||||
internal_static_hbase_pb_ReplicationLoadSink_descriptor =
|
||||
getDescriptor().getMessageTypes().get(5);
|
||||
internal_static_hbase_pb_ReplicationLoadSink_fieldAccessorTable = new
|
||||
|
|
|
@ -147,10 +147,10 @@ message RegionLoad {
|
|||
optional int32 store_ref_count = 21 [ default = 0 ];
|
||||
|
||||
/**
|
||||
* The max number of references active on single store file among all store files
|
||||
* The max number of references active on single store file among all compacted store files
|
||||
* that belong to given region
|
||||
*/
|
||||
optional int32 max_store_file_ref_count = 22 [default = 0];
|
||||
optional int32 max_compacted_store_file_ref_count = 22 [default = 0];
|
||||
}
|
||||
|
||||
/* Server-level protobufs */
|
||||
|
|
|
@ -137,17 +137,18 @@ public class RegionsRecoveryChore extends ScheduledChore {
|
|||
for (ServerLoad serverLoad : serverMetricsMap.values()) {
|
||||
Map<byte[], RegionLoad> regionLoadsMap = serverLoad.getRegionsLoad();
|
||||
for (RegionLoad regionLoad : regionLoadsMap.values()) {
|
||||
// For each region, each store file can have different ref counts
|
||||
// We need to find maximum of all such ref counts and if that max count
|
||||
// is beyond a threshold value, we should reopen the region.
|
||||
// Here, we take max ref count of all store files and not the cumulative
|
||||
// count of all store files
|
||||
final int maxStoreFileRefCount = regionLoad.getMaxStoreFileRefCount();
|
||||
// For each region, each compacted store file can have different ref counts
|
||||
// We need to find maximum of all such ref counts and if that max count of compacted
|
||||
// store files is beyond a threshold value, we should reopen the region.
|
||||
// Here, we take max ref count of all compacted store files and not the cumulative
|
||||
// count of all compacted store files
|
||||
final int maxCompactedStoreFileRefCount = regionLoad
|
||||
.getMaxCompactedStoreFileRefCount();
|
||||
|
||||
if (maxStoreFileRefCount > storeFileRefCountThreshold) {
|
||||
if (maxCompactedStoreFileRefCount > storeFileRefCountThreshold) {
|
||||
final byte[] regionName = regionLoad.getName();
|
||||
prepareTableToReopenRegionsMap(tableToReopenRegionsMap, regionName,
|
||||
maxStoreFileRefCount);
|
||||
maxCompactedStoreFileRefCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1580,7 +1580,7 @@ public class HRegionServer extends HasThread implements
|
|||
int stores = 0;
|
||||
int storefiles = 0;
|
||||
int storeRefCount = 0;
|
||||
int maxStoreFileRefCount = 0;
|
||||
int maxCompactedStoreFileRefCount = 0;
|
||||
int storeUncompressedSizeMB = 0;
|
||||
int storefileSizeMB = 0;
|
||||
int memstoreSizeMB = (int) (r.getMemstoreSize() / 1024 / 1024);
|
||||
|
@ -1598,8 +1598,9 @@ public class HRegionServer extends HasThread implements
|
|||
HStore hStore = (HStore) store;
|
||||
int currentStoreRefCount = hStore.getStoreRefCount();
|
||||
storeRefCount += currentStoreRefCount;
|
||||
int currentMaxStoreFileRefCount = hStore.getMaxStoreFileRefCount();
|
||||
maxStoreFileRefCount = Math.max(maxStoreFileRefCount, currentMaxStoreFileRefCount);
|
||||
int currentMaxCompactedStoreFileRefCount = hStore.getMaxCompactedStoreFileRefCount();
|
||||
maxCompactedStoreFileRefCount = Math.max(maxCompactedStoreFileRefCount,
|
||||
currentMaxCompactedStoreFileRefCount);
|
||||
}
|
||||
storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() / 1024 / 1024);
|
||||
storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
|
||||
|
@ -1628,7 +1629,7 @@ public class HRegionServer extends HasThread implements
|
|||
.setStores(stores)
|
||||
.setStorefiles(storefiles)
|
||||
.setStoreRefCount(storeRefCount)
|
||||
.setMaxStoreFileRefCount(maxStoreFileRefCount)
|
||||
.setMaxCompactedStoreFileRefCount(maxCompactedStoreFileRefCount)
|
||||
.setStoreUncompressedSizeMB(storeUncompressedSizeMB)
|
||||
.setStorefileSizeMB(storefileSizeMB)
|
||||
.setMemstoreSizeMB(memstoreSizeMB)
|
||||
|
|
|
@ -2873,20 +2873,21 @@ public class HStore implements Store {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return get maximum ref count of storeFile among all HStore Files
|
||||
* @return get maximum ref count of storeFile among all compacted HStore Files
|
||||
* for the HStore
|
||||
*/
|
||||
public int getMaxStoreFileRefCount() {
|
||||
int maxStoreFileRefCount = 0;
|
||||
for (StoreFile store : storeEngine.getStoreFileManager().getStorefiles()) {
|
||||
public int getMaxCompactedStoreFileRefCount() {
|
||||
int maxCompactedStoreFileRefCount = 0;
|
||||
for (StoreFile store : storeEngine.getStoreFileManager().getCompactedfiles()) {
|
||||
if (store.isHFile()) {
|
||||
StoreFile.Reader storeReader = store.getReader();
|
||||
if (storeReader != null) {
|
||||
maxStoreFileRefCount = Math.max(maxStoreFileRefCount, storeReader.getRefCount());
|
||||
maxCompactedStoreFileRefCount = Math.max(maxCompactedStoreFileRefCount,
|
||||
storeReader.getRefCount());
|
||||
}
|
||||
}
|
||||
}
|
||||
return maxStoreFileRefCount;
|
||||
return maxCompactedStoreFileRefCount;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -55,7 +55,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
|||
private long numReferenceFiles;
|
||||
private long maxFlushQueueSize;
|
||||
private long maxCompactionQueueSize;
|
||||
private int maxStoreFileRefCount;
|
||||
private int maxCompactedStoreFileRefCount;
|
||||
|
||||
private ScheduledFuture<?> regionMetricsUpdateTask;
|
||||
|
||||
|
@ -125,8 +125,8 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
|||
}
|
||||
|
||||
@Override
|
||||
public int getMaxStoreFileRefCount() {
|
||||
return maxStoreFileRefCount;
|
||||
public int getMaxCompactedStoreFileRefCount() {
|
||||
return maxCompactedStoreFileRefCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -222,7 +222,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
|||
public void run() {
|
||||
long tempNumStoreFiles = 0;
|
||||
int tempStoreRefCount = 0;
|
||||
int tempMaxStoreFileRefCount = 0;
|
||||
int tempMaxCompactedStoreFileRefCount = 0;
|
||||
long tempMemstoreSize = 0;
|
||||
long tempStoreFileSize = 0;
|
||||
long tempMaxStoreFileAge = 0;
|
||||
|
@ -256,16 +256,16 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
|||
// Cast here to avoid interface changes to Store
|
||||
HStore hStore = ((HStore) store);
|
||||
tempStoreRefCount += hStore.getStoreRefCount();
|
||||
int currentMaxStoreFileRefCount = hStore.getMaxStoreFileRefCount();
|
||||
tempMaxStoreFileRefCount = Math.max(tempMaxStoreFileRefCount,
|
||||
currentMaxStoreFileRefCount);
|
||||
int currentMaxCompactedStoreFileRefCount = hStore.getMaxCompactedStoreFileRefCount();
|
||||
tempMaxCompactedStoreFileRefCount = Math.max(tempMaxCompactedStoreFileRefCount,
|
||||
currentMaxCompactedStoreFileRefCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
numStoreFiles = tempNumStoreFiles;
|
||||
storeRefCount = tempStoreRefCount;
|
||||
maxStoreFileRefCount = tempMaxStoreFileRefCount;
|
||||
maxCompactedStoreFileRefCount = tempMaxCompactedStoreFileRefCount;
|
||||
memstoreSize = tempMemstoreSize;
|
||||
storeFileSize = tempStoreFileSize;
|
||||
maxStoreFileAge = tempMaxStoreFileAge;
|
||||
|
|
|
@ -66,7 +66,7 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int getMaxStoreFileRefCount() {
|
||||
public int getMaxCompactedStoreFileRefCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -2230,19 +2230,25 @@ The percent of region server RPC threads failed to abort RS.
|
|||
+
|
||||
.Description
|
||||
|
||||
Very large ref count on a file indicates
|
||||
that it is a ref leak on that object. Such files
|
||||
can not be removed even after it is invalidated
|
||||
via compaction. Only way to recover in such
|
||||
scenario is to reopen the region which can
|
||||
release all resources, like the refcount, leases, etc.
|
||||
This config represents Store files Ref Count threshold
|
||||
value considered for reopening regions.
|
||||
Any region with store files ref count > this value
|
||||
would be eligible for reopening by master.
|
||||
Default value -1 indicates this feature is turned off.
|
||||
Only positive integer value should be provided to enable
|
||||
this feature.
|
||||
Very large number of ref count on a compacted
|
||||
store file indicates that it is a ref leak
|
||||
on that object(compacted store file).
|
||||
Such files can not be removed after
|
||||
it is invalidated via compaction.
|
||||
Only way to recover in such scenario is to
|
||||
reopen the region which can release
|
||||
all resources, like the refcount,
|
||||
leases, etc. This config represents Store files Ref
|
||||
Count threshold value considered for reopening
|
||||
regions. Any region with compacted store files
|
||||
ref count > this value would be eligible for
|
||||
reopening by master. Here, we get the max
|
||||
refCount among all refCounts on all
|
||||
compacted away store files that belong to a
|
||||
particular region. Default value -1 indicates
|
||||
this feature is turned off. Only positive
|
||||
integer value should be provided to
|
||||
enable this feature.
|
||||
|
||||
+
|
||||
.Default
|
||||
|
|
Loading…
Reference in New Issue