HBASE-27028 Add a shell command for flushing master local region (#4539)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
parent
bdaa4486f3
commit
666aa064e7
|
@ -3212,4 +3212,9 @@ public interface Admin extends Abortable, Closeable {
|
|||
*/
|
||||
List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType,
|
||||
int limit, Map<String, Object> filterParams) throws IOException;
|
||||
|
||||
/**
|
||||
* Flush master local region
|
||||
*/
|
||||
void flushMasterStore() throws IOException;
|
||||
}
|
||||
|
|
|
@ -1629,4 +1629,9 @@ public interface AsyncAdmin {
|
|||
*/
|
||||
CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType,
|
||||
ServerType serverType, int limit, Map<String, Object> filterParams);
|
||||
|
||||
/**
|
||||
* Flush master local region
|
||||
*/
|
||||
CompletableFuture<Void> flushMasterStore();
|
||||
}
|
||||
|
|
|
@ -865,4 +865,9 @@ class AsyncHBaseAdmin implements AsyncAdmin {
|
|||
String logType, ServerType serverType, int limit, Map<String, Object> filterParams) {
|
||||
return wrap(rawAdmin.getLogEntries(serverNames, logType, serverType, limit, filterParams));
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> flushMasterStore() {
|
||||
return wrap(rawAdmin.flushMasterStore());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -119,6 +119,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DecommissionRegionServersResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
|
||||
|
@ -1922,6 +1924,12 @@ public class ConnectionImplementation implements ClusterConnection, Closeable {
|
|||
throws ServiceException {
|
||||
return stub.modifyColumnStoreFileTracker(controller, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlushMasterStoreResponse flushMasterStore(RpcController controller,
|
||||
FlushMasterStoreRequest request) throws ServiceException {
|
||||
return stub.flushMasterStore(controller, request);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -165,6 +165,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTabl
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetLocksRequest;
|
||||
|
@ -4401,6 +4402,18 @@ public class HBaseAdmin implements Admin {
|
|||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flushMasterStore() throws IOException {
|
||||
executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
|
||||
@Override
|
||||
protected Void rpcCall() throws Exception {
|
||||
FlushMasterStoreRequest request = FlushMasterStoreRequest.newBuilder().build();
|
||||
master.flushMasterStore(getRpcController(), request);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private List<LogEntry> getBalancerDecisions(final int limit) throws IOException {
|
||||
return executeCallable(
|
||||
new MasterCallable<List<LogEntry>>(getConnection(), getRpcControllerFactory()) {
|
||||
|
|
|
@ -174,6 +174,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTabl
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
|
||||
|
@ -4023,4 +4025,14 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
return CompletableFuture.completedFuture(Collections.emptyList());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> flushMasterStore() {
|
||||
FlushMasterStoreRequest.Builder request = FlushMasterStoreRequest.newBuilder();
|
||||
return this.<Void> newMasterCaller()
|
||||
.action(((controller, stub) -> this.<FlushMasterStoreRequest, FlushMasterStoreResponse,
|
||||
Void> call(controller, stub, request.build(),
|
||||
(s, c, req, done) -> s.flushMasterStore(c, req, done), resp -> null)))
|
||||
.call();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -65,6 +65,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTabl
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
|
||||
|
@ -744,4 +746,10 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
|
|||
ModifyColumnStoreFileTrackerRequest request) throws ServiceException {
|
||||
return stub.modifyColumnStoreFileTracker(controller, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlushMasterStoreResponse flushMasterStore(RpcController controller,
|
||||
FlushMasterStoreRequest request) throws ServiceException {
|
||||
return stub.flushMasterStore(controller, request);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -757,6 +757,9 @@ message ModifyColumnStoreFileTrackerResponse {
|
|||
optional uint64 proc_id = 1;
|
||||
}
|
||||
|
||||
message FlushMasterStoreRequest {}
|
||||
message FlushMasterStoreResponse {}
|
||||
|
||||
service MasterService {
|
||||
/** Used by the client to get the number of regions that have received the updated schema */
|
||||
rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
|
||||
|
@ -1153,6 +1156,9 @@ service MasterService {
|
|||
|
||||
rpc ModifyColumnStoreFileTracker(ModifyColumnStoreFileTrackerRequest)
|
||||
returns(ModifyColumnStoreFileTrackerResponse);
|
||||
|
||||
rpc FlushMasterStore(FlushMasterStoreRequest)
|
||||
returns(FlushMasterStoreResponse);
|
||||
}
|
||||
|
||||
// HBCK Service definitions.
|
||||
|
|
|
@ -1104,6 +1104,22 @@ public interface MasterObserver {
|
|||
final TableName tableName) throws IOException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Called before the master local region memstore is flushed to disk.
|
||||
* @param ctx the environment to interact with the framework and master
|
||||
*/
|
||||
default void preMasterStoreFlush(final ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
throws IOException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Called after the master local region memstore is flushed to disk.
|
||||
* @param ctx the environment to interact with the framework and master
|
||||
*/
|
||||
default void postMasterStoreFlush(final ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
throws IOException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Called before the quota for the user is stored.
|
||||
* @param ctx the environment to interact with the framework and master
|
||||
|
|
|
@ -4163,6 +4163,26 @@ public class HMaster extends HRegionServer implements MasterServices {
|
|||
return metaLocationSyncer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flushMasterStore() throws IOException {
|
||||
LOG.info("Force flush master local region.");
|
||||
if (this.cpHost != null) {
|
||||
try {
|
||||
cpHost.preMasterStoreFlush();
|
||||
} catch (IOException ioe) {
|
||||
LOG.error("Error invoking master coprocessor preMasterStoreFlush()", ioe);
|
||||
}
|
||||
}
|
||||
masterRegion.flush(true);
|
||||
if (this.cpHost != null) {
|
||||
try {
|
||||
cpHost.postMasterStoreFlush();
|
||||
} catch (IOException ioe) {
|
||||
LOG.error("Error invoking master coprocessor postMasterStoreFlush()", ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@RestrictedApi(explanation = "Should only be called in tests", link = "",
|
||||
allowedOnPath = ".*/src/test/.*")
|
||||
public MasterRegion getMasterRegion() {
|
||||
|
|
|
@ -1224,6 +1224,24 @@ public class MasterCoprocessorHost
|
|||
});
|
||||
}
|
||||
|
||||
public void preMasterStoreFlush() throws IOException {
|
||||
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
|
||||
@Override
|
||||
public void call(MasterObserver observer) throws IOException {
|
||||
observer.preMasterStoreFlush(this);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public void postMasterStoreFlush() throws IOException {
|
||||
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
|
||||
@Override
|
||||
public void call(MasterObserver observer) throws IOException {
|
||||
observer.postMasterStoreFlush(this);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public void preSetUserQuota(final String user, final GlobalQuotaSettings quotas)
|
||||
throws IOException {
|
||||
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
|
||||
|
|
|
@ -201,6 +201,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProced
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FixMetaResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.FlushMasterStoreResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
|
||||
|
@ -1377,6 +1379,18 @@ public class MasterRpcServices extends RSRpcServices
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlushMasterStoreResponse flushMasterStore(RpcController controller,
|
||||
FlushMasterStoreRequest request) throws ServiceException {
|
||||
rpcPreCheck("flushMasterStore");
|
||||
try {
|
||||
master.flushMasterStore();
|
||||
} catch (IOException ioe) {
|
||||
throw new ServiceException(ioe);
|
||||
}
|
||||
return FlushMasterStoreResponse.newBuilder().build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ModifyNamespaceResponse modifyNamespace(RpcController controller,
|
||||
ModifyNamespaceRequest request) throws ServiceException {
|
||||
|
|
|
@ -480,4 +480,9 @@ public interface MasterServices extends Server {
|
|||
* We need to get this in MTP to tell the syncer the new meta replica count.
|
||||
*/
|
||||
MetaLocationSyncer getMetaLocationSyncer();
|
||||
|
||||
/**
|
||||
* Flush master local region
|
||||
*/
|
||||
void flushMasterStore() throws IOException;
|
||||
}
|
||||
|
|
|
@ -161,10 +161,11 @@ public final class MasterRegion {
|
|||
return region.getScanner(scan);
|
||||
}
|
||||
|
||||
@RestrictedApi(explanation = "Should only be called in tests", link = "",
|
||||
allowedOnPath = ".*/src/test/.*")
|
||||
public FlushResult flush(boolean force) throws IOException {
|
||||
return region.flush(force);
|
||||
flusherAndCompactor.resetChangesAfterLastFlush();
|
||||
FlushResult flushResult = region.flush(force);
|
||||
flusherAndCompactor.recordLastFlushTime();
|
||||
return flushResult;
|
||||
}
|
||||
|
||||
@RestrictedApi(explanation = "Should only be called in tests", link = "",
|
||||
|
|
|
@ -180,7 +180,7 @@ class MasterRegionFlusherAndCompactor implements Closeable {
|
|||
}
|
||||
|
||||
private void flushLoop() {
|
||||
lastFlushTime = EnvironmentEdgeManager.currentTime();
|
||||
recordLastFlushTime();
|
||||
while (!closed) {
|
||||
flushLock.lock();
|
||||
try {
|
||||
|
@ -202,10 +202,10 @@ class MasterRegionFlusherAndCompactor implements Closeable {
|
|||
flushLock.unlock();
|
||||
}
|
||||
assert flushRequest;
|
||||
changesAfterLastFlush.set(0);
|
||||
resetChangesAfterLastFlush();
|
||||
try {
|
||||
region.flush(true);
|
||||
lastFlushTime = EnvironmentEdgeManager.currentTime();
|
||||
recordLastFlushTime();
|
||||
} catch (IOException e) {
|
||||
LOG.error(HBaseMarkers.FATAL, "Failed to flush master local region, aborting...", e);
|
||||
abortable.abort("Failed to flush master local region", e);
|
||||
|
@ -263,6 +263,14 @@ class MasterRegionFlusherAndCompactor implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
void resetChangesAfterLastFlush() {
|
||||
changesAfterLastFlush.set(0);
|
||||
}
|
||||
|
||||
void recordLastFlushTime() {
|
||||
lastFlushTime = EnvironmentEdgeManager.currentTime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
closed = true;
|
||||
|
|
|
@ -189,6 +189,8 @@ public class TestMasterObserver {
|
|||
private boolean postRequestLockCalled;
|
||||
private boolean preLockHeartbeatCalled;
|
||||
private boolean postLockHeartbeatCalled;
|
||||
private boolean preMasterStoreFlushCalled;
|
||||
private boolean postMasterStoreFlushCalled;
|
||||
|
||||
public void resetStates() {
|
||||
preCreateTableRegionInfosCalled = false;
|
||||
|
@ -280,6 +282,8 @@ public class TestMasterObserver {
|
|||
postRequestLockCalled = false;
|
||||
preLockHeartbeatCalled = false;
|
||||
postLockHeartbeatCalled = false;
|
||||
preMasterStoreFlushCalled = false;
|
||||
postMasterStoreFlushCalled = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1042,6 +1046,18 @@ public class TestMasterObserver {
|
|||
TableName tableName) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preMasterStoreFlush(ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
throws IOException {
|
||||
preMasterStoreFlushCalled = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postMasterStoreFlush(ObserverContext<MasterCoprocessorEnvironment> ctx)
|
||||
throws IOException {
|
||||
postMasterStoreFlushCalled = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
|
||||
final String userName, final GlobalQuotaSettings quotas) throws IOException {
|
||||
|
@ -1676,4 +1692,22 @@ public class TestMasterObserver {
|
|||
ProcedureTestingUtility.waitNoProcedureRunning(master.getMasterProcedureExecutor());
|
||||
ProcedureTestingUtility.assertProcNotFailed(master.getMasterProcedureExecutor(), procId);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMasterStoreOperations() throws Exception {
|
||||
HMaster master = UTIL.getMiniHBaseCluster().getMaster();
|
||||
MasterCoprocessorHost host = master.getMasterCoprocessorHost();
|
||||
CPMasterObserver cp = host.findCoprocessor(CPMasterObserver.class);
|
||||
cp.resetStates();
|
||||
assertFalse("No master store flush call", cp.preMasterStoreFlushCalled);
|
||||
assertFalse("No master store flush call", cp.postMasterStoreFlushCalled);
|
||||
|
||||
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
|
||||
Admin admin = connection.getAdmin()) {
|
||||
admin.flushMasterStore();
|
||||
|
||||
assertTrue("Master store flush called", cp.preMasterStoreFlushCalled);
|
||||
assertTrue("Master store flush called", cp.postMasterStoreFlushCalled);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -480,6 +480,10 @@ public class MockNoopMasterServices implements MasterServices {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flushMasterStore() throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup,
|
||||
long nonce) throws IOException {
|
||||
|
|
|
@ -1795,11 +1795,17 @@ module Hbase
|
|||
@admin.modifyTableStoreFileTracker(tableName, sft)
|
||||
end
|
||||
|
||||
#----------------------------------------------------------------------------------------------
|
||||
#----------------------------------------------------------------------------------------------
|
||||
# Change table column family's sft
|
||||
def modify_table_family_sft(tableName, family_bytes, sft)
|
||||
@admin.modifyColumnFamilyStoreFileTracker(tableName, family_bytes, sft)
|
||||
end
|
||||
|
||||
#----------------------------------------------------------------------------------------------
|
||||
# Flush master local region
|
||||
def flush_master_store()
|
||||
@admin.flushMasterStore()
|
||||
end
|
||||
end
|
||||
# rubocop:enable Metrics/ClassLength
|
||||
end
|
||||
|
|
|
@ -446,6 +446,7 @@ Shell.load_command_group(
|
|||
compact
|
||||
compaction_switch
|
||||
flush
|
||||
flush_master_store
|
||||
get_balancer_decisions
|
||||
get_balancer_rejections
|
||||
get_slowlog_responses
|
||||
|
|
|
@ -0,0 +1,37 @@
|
|||
#
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
module Shell
|
||||
module Commands
|
||||
class FlushMasterStore < Command
|
||||
def help
|
||||
<<-EOF
|
||||
Flush master local region.
|
||||
For example:
|
||||
|
||||
hbase> flush_master_store
|
||||
EOF
|
||||
end
|
||||
|
||||
def command()
|
||||
admin.flush_master_store()
|
||||
end
|
||||
end
|
||||
end
|
||||
end
|
|
@ -1448,6 +1448,11 @@ public class ThriftAdmin implements Admin {
|
|||
throw new NotImplementedException("getLogEntries not supported in ThriftAdmin");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flushMasterStore() throws IOException {
|
||||
throw new NotImplementedException("flushMasterStore not supported in ThriftAdmin");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family,
|
||||
String dstSFT) throws IOException {
|
||||
|
|
Loading…
Reference in New Issue