Skip to content

Commit ec200e7

Browse files
committed
HBASE-27028 Add a shell command for flushing master local region
1 parent bc21967 commit ec200e7

File tree

19 files changed

+140
-2
lines changed

19 files changed

+140
-2
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2539,4 +2539,9 @@ default BalanceResponse balanceRSGroup(String groupName) throws IOException {
25392539
*/
25402540
List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType,
25412541
int limit, Map<String, Object> filterParams) throws IOException;
2542+
2543+
/**
2544+
* Flush master local region
2545+
*/
2546+
void flushMasterStore() throws IOException;
25422547
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1078,4 +1078,9 @@ public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
10781078
ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException {
10791079
return get(admin.getLogEntries(serverNames, logType, serverType, limit, filterParams));
10801080
}
1081+
1082+
@Override
1083+
public void flushMasterStore() throws IOException {
1084+
get(admin.flushMasterStore());
1085+
}
10811086
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1785,4 +1785,9 @@ default CompletableFuture<BalanceResponse> balanceRSGroup(String groupName) {
17851785
*/
17861786
CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType,
17871787
ServerType serverType, int limit, Map<String, Object> filterParams);
1788+
1789+
/**
1790+
* Flush master local region
1791+
*/
1792+
CompletableFuture<Void> flushMasterStore();
17881793
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -949,4 +949,9 @@ public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNam
949949
String logType, ServerType serverType, int limit, Map<String, Object> filterParams) {
950950
return wrap(rawAdmin.getLogEntries(serverNames, logType, serverType, limit, filterParams));
951951
}
952+
953+
@Override
954+
public CompletableFuture<Void> flushMasterStore() {
955+
return wrap(rawAdmin.flushMasterStore());
956+
}
952957
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -178,6 +178,8 @@
178178
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
179179
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
180180
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
181+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreRequest;
182+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreResponse;
181183
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
182184
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
183185
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
@@ -4280,4 +4282,14 @@ public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNam
42804282
return CompletableFuture.completedFuture(Collections.emptyList());
42814283
}
42824284
}
4285+
4286+
@Override
4287+
public CompletableFuture<Void> flushMasterStore() {
4288+
FlushMasterStoreRequest.Builder request = FlushMasterStoreRequest.newBuilder();
4289+
return this.<Void> newMasterCaller()
4290+
.action(((controller, stub) -> this.<FlushMasterStoreRequest, FlushMasterStoreResponse,
4291+
Void> call(controller, stub, request.build(),
4292+
(s, c, req, done) -> s.flushMasterStore(c, req, done), resp -> null)))
4293+
.call();
4294+
}
42834295
}

hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -758,6 +758,9 @@ message ModifyColumnStoreFileTrackerResponse {
758758
optional uint64 proc_id = 1;
759759
}
760760

761+
message FlushMasterStoreRequest {}
762+
message FlushMasterStoreResponse {}
763+
761764
service MasterService {
762765
/** Used by the client to get the number of regions that have received the updated schema */
763766
rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
@@ -1197,6 +1200,9 @@ service MasterService {
11971200

11981201
rpc ModifyColumnStoreFileTracker(ModifyColumnStoreFileTrackerRequest)
11991202
returns(ModifyColumnStoreFileTrackerResponse);
1203+
1204+
rpc FlushMasterStore(FlushMasterStoreRequest)
1205+
returns(FlushMasterStoreResponse);
12001206
}
12011207

12021208
// HBCK Service definitions.

hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4201,6 +4201,11 @@ public List<HRegionLocation> getMetaLocations() {
42014201
return metaRegionLocationCache.getMetaRegionLocations();
42024202
}
42034203

4204+
@Override
4205+
public void flushMasterStore() {
4206+
masterRegion.rpcRequestFlush();
4207+
}
4208+
42044209
public Collection<ServerName> getLiveRegionServers() {
42054210
return regionServerTracker.getRegionServers();
42064211
}

hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -229,6 +229,8 @@
229229
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
230230
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
231231
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
232+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreRequest;
233+
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreResponse;
232234
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
233235
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
234236
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
@@ -3468,4 +3470,12 @@ public ReplicateWALEntryResponse replicateToReplica(RpcController controller,
34683470
ReplicateWALEntryRequest request) throws ServiceException {
34693471
throw new ServiceException(new DoNotRetryIOException("Unsupported method on master"));
34703472
}
3473+
3474+
@Override
3475+
public FlushMasterStoreResponse flushMasterStore(RpcController controller,
3476+
FlushMasterStoreRequest request) throws ServiceException {
3477+
rpcPreCheck("flushMasterStore");
3478+
server.flushMasterStore();
3479+
return FlushMasterStoreResponse.newBuilder().build();
3480+
}
34713481
}

hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -510,4 +510,9 @@ boolean normalizeRegions(final NormalizeTableFilterParams ntfp, final boolean is
510510
* We need to get this in MTP to tell the syncer the new meta replica count.
511511
*/
512512
MetaLocationSyncer getMetaLocationSyncer();
513+
514+
/**
515+
* Flush master local region
516+
*/
517+
void flushMasterStore();
513518
}

hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -154,6 +154,10 @@ public RegionScanner getRegionScanner(Scan scan) throws IOException {
154154
return region.getScanner(scan);
155155
}
156156

157+
public void rpcRequestFlush() {
158+
flusherAndCompactor.requestFlush();
159+
}
160+
157161
public FlushResult flush(boolean force) throws IOException {
158162
return region.flush(force);
159163
}

0 commit comments

Comments
 (0)