From b0f826bafae4485d8bd5302ab33c947ec87026fc Mon Sep 17 00:00:00 2001 From: huiruan Date: Sun, 26 Dec 2021 01:01:32 +0800 Subject: [PATCH] using a optional field to introduce the snapshot procedure to the client --- .../org/apache/hadoop/hbase/client/Admin.java | 60 ---------- .../hbase/client/AdminOverAsyncAdmin.java | 6 - .../hadoop/hbase/client/AsyncAdmin.java | 13 --- .../hadoop/hbase/client/AsyncHBaseAdmin.java | 5 - .../hbase/client/RawAsyncHBaseAdmin.java | 104 ++++++++---------- .../main/protobuf/server/master/Master.proto | 18 +-- .../hbase/master/MasterRpcServices.java | 37 +++---- .../master/snapshot/SnapshotManager.java | 48 +++++--- .../procedure/TestSnapshotProcedure.java | 14 +-- .../hbase/rsgroup/VerifyingRSGroupAdmin.java | 6 - .../hbase/thrift2/client/ThriftAdmin.java | 5 - 11 files changed, 100 insertions(+), 216 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index a9389369ecb5..48893ccbcf4f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -1542,66 +1542,6 @@ default void snapshot(String snapshotName, TableName tableName, void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException, IllegalArgumentException; - /** - * Take a snapshot and wait for the server to complete that snapshot (blocking). It's same as - * {@link org.apache.hadoop.hbase.client.Admin#snapshot(String, TableName)} for users. The - * difference between the two methods is that - * {@link org.apache.hadoop.hbase.client.Admin#snapshotTable(String, TableName)} is based on - * proc-v2. - * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other - * snapshots stored on the cluster - * @param tableName name of the table to snapshot - * @throws IOException we fail to reach the master - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ - default void snapshotTable(String snapshotName, TableName tableName) - throws IOException, SnapshotCreationException, IllegalArgumentException { - snapshotTable(snapshotName, tableName, SnapshotType.FLUSH); - } - - /** - * Create typed snapshot of the table. - * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other - * snapshots stored on the cluster - * @param tableName name of the table to snapshot - * @param type type of snapshot to take - * @throws IOException we fail to reach the master - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ - default void snapshotTable(String snapshotName, TableName tableName, SnapshotType type) - throws IOException, SnapshotCreationException, IllegalArgumentException { - snapshotTable(new SnapshotDescription(snapshotName, tableName, type)); - } - - /** - * Create typed snapshot of the table. - * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other - * snapshots stored on the cluster - * @param tableName name of the table to snapshot - * @param type type of snapshot to take - * @param snapshotProps snapshot additional properties e.g. TTL - * @throws IOException we fail to reach the master - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ - default void snapshotTable(String snapshotName, TableName tableName, - SnapshotType type, Map snapshotProps) - throws IOException, SnapshotCreationException, IllegalArgumentException { - snapshot(new SnapshotDescription(snapshotName, tableName, type, snapshotProps)); - } - - /** - * Take a snapshot and wait for the server to complete that snapshot (blocking). - * @param snapshot snapshot to take - * @throws IOException we fail to reach the master - * @throws SnapshotCreationException if snapshot creation failed - * @throws IllegalArgumentException if the snapshot request is formatted incorrectly - */ - void snapshotTable(SnapshotDescription snapshot) - throws IOException, SnapshotCreationException, IllegalArgumentException; - /** * Take a snapshot without waiting for the server to complete that snapshot (asynchronous). * Snapshots are considered unique based on the name of the snapshot. Snapshots are taken diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java index b4618c5a6eef..08de9797a16a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AdminOverAsyncAdmin.java @@ -625,12 +625,6 @@ public void snapshot(SnapshotDescription snapshot) get(admin.snapshot(snapshot)); } - @Override - public void snapshotTable(SnapshotDescription snapshot) - throws IOException, SnapshotCreationException, IllegalArgumentException { - get(admin.snapshotTable(snapshot)); - } - @Override public Future snapshotAsync(SnapshotDescription snapshot) throws IOException, SnapshotCreationException { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index dcd0f4b9fc59..ced022b0c5c3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -825,19 +825,6 @@ default CompletableFuture snapshot(String snapshotName, TableName tableNam */ CompletableFuture snapshot(SnapshotDescription snapshot); - /** - * Take a snapshot and wait for the server to complete that snapshot asynchronously. - */ - default CompletableFuture snapshotTable(String snapshotName, TableName tableName) { - return snapshot(new SnapshotDescription(snapshotName, tableName, SnapshotType.FLUSH)); - } - - default CompletableFuture snapshotTable(String snapshotName, TableName tableName, - SnapshotType type) { - return snapshot(new SnapshotDescription(snapshotName, tableName, type)); - } - - CompletableFuture snapshotTable(SnapshotDescription snapshot); /** * Check the current state of the passed snapshot. There are three possible states: diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index d36a8cb3c925..f0f564be1281 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -476,11 +476,6 @@ public CompletableFuture snapshot(SnapshotDescription snapshot) { return wrap(rawAdmin.snapshot(snapshot)); } - @Override - public CompletableFuture snapshotTable(SnapshotDescription snapshot) { - return wrap(rawAdmin.snapshotTable(snapshot)); - } - @Override public CompletableFuture isSnapshotFinished(SnapshotDescription snapshot) { return wrap(rawAdmin.isSnapshotFinished(snapshot)); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 23c40621bb61..1bbc08b65270 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -265,8 +265,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotTableResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest; @@ -1873,68 +1871,58 @@ public CompletableFuture snapshot(SnapshotDescription snapshotDesc) { return failedFuture(e); } CompletableFuture future = new CompletableFuture<>(); - final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot).build(); - addListener(this. newMasterCaller() - .action((controller, stub) -> this. call(controller, - stub, request, (s, c, req, done) -> s.snapshot(c, req, done), - resp -> resp.getExpectedTimeout())) - .call(), (expectedTimeout, err) -> { + final SnapshotRequest request = + SnapshotRequest.newBuilder().setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup()) + .setNonce(ng.newNonce()).build(); + addListener(this. newMasterCaller() + .action((controller, stub) -> + this. call(controller, stub, + request, (s, c, req, done) -> s.snapshot(c, req, done), resp -> resp)) + .call(), (resp, err) -> { if (err != null) { future.completeExceptionally(err); return; } - TimerTask pollingTask = new TimerTask() { - int tries = 0; - long startTime = EnvironmentEdgeManager.currentTime(); - long endTime = startTime + expectedTimeout; - long maxPauseTime = expectedTimeout / maxAttempts; - - @Override - public void run(Timeout timeout) throws Exception { - if (EnvironmentEdgeManager.currentTime() < endTime) { - addListener(isSnapshotFinished(snapshotDesc), (done, err2) -> { - if (err2 != null) { - future.completeExceptionally(err2); - } else if (done) { - future.complete(null); - } else { - // retry again after pauseTime. - long pauseTime = - ConnectionUtils.getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries); - pauseTime = Math.min(pauseTime, maxPauseTime); - AsyncConnectionImpl.RETRY_TIMER.newTimeout(this, pauseTime, - TimeUnit.MILLISECONDS); - } - }); - } else { - future.completeExceptionally( - new SnapshotCreationException("Snapshot '" + snapshot.getName() + - "' wasn't completed in expectedTime:" + expectedTimeout + " ms", snapshotDesc)); + if (resp.hasProcId()) { + getProcedureResult(resp.getProcId(), future, 0); + addListener(future, new SnapshotProcedureBiConsumer(snapshotDesc.getTableName())); + } else { + long expectedTimeout = resp.getExpectedTimeout(); + TimerTask pollingTask = new TimerTask() { + int tries = 0; + long startTime = EnvironmentEdgeManager.currentTime(); + long endTime = startTime + expectedTimeout; + long maxPauseTime = expectedTimeout / maxAttempts; + + @Override + public void run(Timeout timeout) throws Exception { + if (EnvironmentEdgeManager.currentTime() < endTime) { + addListener(isSnapshotFinished(snapshotDesc), (done, err2) -> { + if (err2 != null) { + future.completeExceptionally(err2); + } else if (done) { + future.complete(null); + } else { + // retry again after pauseTime. + long pauseTime = ConnectionUtils + .getPauseTime(TimeUnit.NANOSECONDS.toMillis(pauseNs), ++tries); + pauseTime = Math.min(pauseTime, maxPauseTime); + AsyncConnectionImpl.RETRY_TIMER + .newTimeout(this, pauseTime, TimeUnit.MILLISECONDS); + } + }); + } else { + future.completeExceptionally(new SnapshotCreationException( + "Snapshot '" + snapshot.getName() + "' wasn't completed in expectedTime:" + + expectedTimeout + " ms", snapshotDesc)); + } } - } - }; - AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS); - }); + }; + AsyncConnectionImpl.RETRY_TIMER.newTimeout(pollingTask, 1, TimeUnit.MILLISECONDS); + }}); return future; } - @Override - public CompletableFuture snapshotTable(SnapshotDescription snapshotDesc) { - SnapshotProtos.SnapshotDescription snapshot = - ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotDesc); - try { - ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot); - } catch (IllegalArgumentException e) { - return failedFuture(e); - } - - SnapshotTableRequest snapshotTableRequest = SnapshotTableRequest.newBuilder() - .setSnapshot(snapshot).setNonceGroup(ng.getNonceGroup()).setNonce(ng.newNonce()).build(); - return this. procedureCall(snapshotTableRequest, - (s, c, req, done) -> s.snapshotTable(c, req, done), (resp) -> resp.getProcId(), - new SnapshotTableProcedureBiConsumer(TableName.valueOf(snapshot.getTable()))); - } - @Override public CompletableFuture isSnapshotFinished(SnapshotDescription snapshot) { return this @@ -2757,8 +2745,8 @@ String getOperationType() { } } - private static class SnapshotTableProcedureBiConsumer extends TableProcedureBiConsumer { - SnapshotTableProcedureBiConsumer(TableName tableName) { + private static class SnapshotProcedureBiConsumer extends TableProcedureBiConsumer { + SnapshotProcedureBiConsumer(TableName tableName) { super(tableName); } diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto index 6c7643f19b93..8ea977de3d28 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto @@ -437,20 +437,13 @@ message IsCleanerChoreEnabledResponse { message SnapshotRequest { required SnapshotDescription snapshot = 1; -} - -message SnapshotResponse { - required int64 expected_timeout = 1; -} - -message SnapshotTableRequest { - required SnapshotDescription snapshot = 1; optional uint64 nonce_group = 2 [default = 0]; optional uint64 nonce = 3 [default = 0]; } -message SnapshotTableResponse { - optional int64 proc_id = 1; +message SnapshotResponse { + required int64 expected_timeout = 1; + optional int64 proc_id = 2; } message GetCompletedSnapshotsRequest { @@ -942,11 +935,6 @@ service MasterService { */ rpc Snapshot(SnapshotRequest) returns(SnapshotResponse); - /** - * Create a snapshot for the given table. - */ - rpc SnapshotTable(SnapshotTableRequest) returns(SnapshotTableResponse); - /** * Get completed snapshots. * Returns a list of snapshot descriptors for completed snapshots diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 9979e94e96c3..fc57f4b8121f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; import org.apache.hadoop.hbase.ipc.PriorityFunction; import org.apache.hadoop.hbase.ipc.QosPriority; +import org.apache.hadoop.hbase.ipc.RpcCallContext; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; @@ -329,8 +330,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotTableRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotTableResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SplitTableRegionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest; @@ -1708,33 +1707,23 @@ public SnapshotResponse snapshot(RpcController controller, // get the snapshot information SnapshotDescription snapshot = SnapshotDescriptionUtils.validate( request.getSnapshot(), server.getConfiguration()); - server.snapshotManager.takeSnapshot(snapshot); - // send back the max amount of time the client should wait for the snapshot to complete long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(server.getConfiguration(), snapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME); - return SnapshotResponse.newBuilder().setExpectedTimeout(waitTime).build(); - } catch (ForeignException e) { - throw new ServiceException(e.getCause()); - } catch (IOException e) { - throw new ServiceException(e); - } - } - @Override - public MasterProtos.SnapshotTableResponse snapshotTable(RpcController controller, - SnapshotTableRequest request) throws ServiceException { - try { - server.checkInitialized(); - server.snapshotManager.checkSnapshotSupport(); - LOG.info(server.getClientIdAuditPrefix() + " snapshot request for:" + - ClientSnapshotDescriptionUtils.toString(request.getSnapshot())); + SnapshotResponse.Builder builder = SnapshotResponse.newBuilder().setExpectedTimeout(waitTime); - SnapshotDescription snapshot = SnapshotDescriptionUtils.validate( - request.getSnapshot(), server.getConfiguration()); - long procId = server.snapshotManager - .takeSnapshot(snapshot, request.getNonceGroup(), request.getNonce()); - return SnapshotTableResponse.newBuilder().setProcId(procId).build(); + // just to pass the unit tests for all 3.x versions, + // the minimum version maybe needs to be modified later + if (VersionInfoUtil.currentClientHasMinimumVersion(2, 10)) { + long nonceGroup = request.getNonceGroup(); + long nonce = request.getNonce(); + long procId = server.snapshotManager.takeSnapshot(snapshot, nonceGroup, nonce); + return builder.setProcId(procId).build(); + } else { + server.snapshotManager.takeSnapshot(snapshot); + return builder.build(); + } } catch (ForeignException e) { throw new ServiceException(e.getCause()); } catch (IOException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 531288c3627f..6d349723a9c5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -390,6 +390,15 @@ public boolean isSnapshotDone(SnapshotDescription expected) throws IOException { "No snapshot name passed in request, can't figure out which snapshot you want to check."); } + Long procId = snapshotToProcIdMap.get(expected); + if (procId != null) { + if (master.getMasterProcedureExecutor().isRunning()) { + return master.getMasterProcedureExecutor().isFinished(procId); + } else { + return false; + } + } + String ssString = ClientSnapshotDescriptionUtils.toString(expected); // check to see if the sentinel exists, @@ -656,28 +665,33 @@ public synchronized long takeSnapshot(SnapshotDescription snapshot, long nonceGroup, long nonce) throws IOException { this.takingSnapshotLock.readLock().lock(); try { - return MasterProcedureUtil.submitProcedure( - new MasterProcedureUtil.NonceProcedureRunnable(master, nonceGroup, nonce) { - @Override - protected void run() throws IOException { - sanityCheckBeforeSnapshot(snapshot, false); - - long procId = submitProcedure(new SnapshotProcedure( - master.getMasterProcedureExecutor().getEnvironment(), snapshot)); - - getMaster().getSnapshotManager().registerSnapshotProcedure(snapshot, procId); - } - - @Override - protected String getDescription() { - return "SnapshotTableProcedure"; - } - }); + return submitSnapshotProcedure(snapshot, nonceGroup, nonce); } finally { this.takingSnapshotLock.readLock().unlock(); } } + private long submitSnapshotProcedure(SnapshotDescription snapshot, + long nonceGroup, long nonce) throws IOException { + return MasterProcedureUtil.submitProcedure( + new MasterProcedureUtil.NonceProcedureRunnable(master, nonceGroup, nonce) { + @Override + protected void run() throws IOException { + sanityCheckBeforeSnapshot(snapshot, false); + + long procId = submitProcedure(new SnapshotProcedure( + master.getMasterProcedureExecutor().getEnvironment(), snapshot)); + + getMaster().getSnapshotManager().registerSnapshotProcedure(snapshot, procId); + } + + @Override + protected String getDescription() { + return "SnapshotProcedure"; + } + }); + } + private void takeSnapshotInternal(SnapshotDescription snapshot) throws IOException { TableDescriptor desc = sanityCheckBeforeSnapshot(snapshot, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedure.java index fd2c266d1a23..afb0eb3c42a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSnapshotProcedure.java @@ -119,7 +119,7 @@ public void setup() throws Exception { @Test public void testSimpleSnapshotTable() throws Exception { - TEST_UTIL.getAdmin().snapshotTable(snapshot); + TEST_UTIL.getAdmin().snapshot(snapshot); SnapshotTestingUtils.assertOneSnapshotThatMatches(TEST_UTIL.getAdmin(), snapshotProto); SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotProto, TABLE_NAME, CF); } @@ -221,7 +221,7 @@ public void testClientTakingTwoSnapshotOnSameTable() throws Exception { @Override public void run() { try { - TEST_UTIL.getAdmin().snapshotTable(snapshot); + TEST_UTIL.getAdmin().snapshot(snapshot); } catch (IOException e) { LOG.error("first client failed taking snapshot", e); fail("first client failed taking snapshot"); @@ -233,7 +233,7 @@ public void run() { // we don't allow different snapshot with same name SnapshotDescription snapshotWithSameName = new SnapshotDescription(SNAPSHOT_NAME, TABLE_NAME, SnapshotType.SKIPFLUSH); - TEST_UTIL.getAdmin().snapshotTable(snapshotWithSameName); + TEST_UTIL.getAdmin().snapshot(snapshotWithSameName); } @Test(expected = org.apache.hadoop.hbase.snapshot.SnapshotCreationException.class) @@ -242,7 +242,7 @@ public void testClientTakeSameSnapshotTwice() throws IOException, InterruptedExc @Override public void run() { try { - TEST_UTIL.getAdmin().snapshotTable(snapshot); + TEST_UTIL.getAdmin().snapshot(snapshot); } catch (IOException e) { LOG.error("first client failed taking snapshot", e); fail("first client failed taking snapshot"); @@ -251,7 +251,7 @@ public void run() { }; first.start(); Thread.sleep(1000); - TEST_UTIL.getAdmin().snapshotTable(snapshot); + TEST_UTIL.getAdmin().snapshot(snapshot); } @Test @@ -261,7 +261,7 @@ public void testTakeZkCoordinatedSnapshotAndProcedureCoordinatedSnapshotBoth() t @Override public void run() { try { - TEST_UTIL.getAdmin().snapshotTable(snapshot); + TEST_UTIL.getAdmin().snapshot(snapshot); } catch (IOException e) { LOG.error("procedure snapshot failed", e); fail("procedure snapshot failed"); @@ -284,7 +284,7 @@ public void run() { @Override public void run() { try { - TEST_UTIL.getAdmin().snapshot(snapshotOnSameTable); + master.getSnapshotManager().takeSnapshot(snapshotOnSameTableProto); } catch (IOException e) { LOG.error("zk snapshot failed", e); fail("zk snapshot failed"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java index c8ebd4199f53..c5911f2a4a87 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdmin.java @@ -526,12 +526,6 @@ public void snapshot(SnapshotDescription snapshot) admin.snapshot(snapshot); } - @Override - public void snapshotTable(SnapshotDescription snapshot) - throws IOException, SnapshotCreationException, IllegalArgumentException { - admin.snapshotTable(snapshot); - } - public Future snapshotAsync(SnapshotDescription snapshot) throws IOException, SnapshotCreationException { return admin.snapshotAsync(snapshot); diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java index c61a4af267c6..6d9e1f08c73b 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftAdmin.java @@ -845,11 +845,6 @@ public void snapshot(SnapshotDescription snapshot) { throw new NotImplementedException("snapshot not supported in ThriftAdmin"); } - @Override - public void snapshotTable(SnapshotDescription snapshot) { - throw new NotImplementedException("snapshot not supported in ThriftAdmin"); - } - @Override public Future snapshotAsync(SnapshotDescription snapshot) { throw new NotImplementedException("snapshotAsync not supported in ThriftAdmin");