|
26 | 26 | import org.apache.hadoop.conf.Configuration;
|
27 | 27 | import org.apache.hadoop.fs.FileSystem;
|
28 | 28 | import org.apache.hadoop.fs.Path;
|
| 29 | +import org.apache.hadoop.hbase.HBaseIOException; |
29 | 30 | import org.apache.hadoop.hbase.TableName;
|
30 | 31 | import org.apache.hadoop.hbase.client.RegionInfo;
|
31 | 32 | import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
@@ -143,13 +144,18 @@ protected Flow executeFromState(MasterProcedureEnv env, SnapshotState state)
|
143 | 144 | setNextState(SnapshotState.SNAPSHOT_WRITE_SNAPSHOT_INFO);
|
144 | 145 | return Flow.HAS_MORE_STATE;
|
145 | 146 | case SNAPSHOT_WRITE_SNAPSHOT_INFO:
|
146 |
| - SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, workingDirFS); |
147 | 147 | TableState tableState =
|
148 | 148 | env.getMasterServices().getTableStateManager().getTableState(snapshotTable);
|
149 | 149 | if (tableState.isEnabled()) {
|
| 150 | + SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, workingDirFS); |
150 | 151 | setNextState(SnapshotState.SNAPSHOT_SNAPSHOT_ONLINE_REGIONS);
|
151 | 152 | } else if (tableState.isDisabled()) {
|
| 153 | + SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, workingDirFS); |
152 | 154 | setNextState(SnapshotState.SNAPSHOT_SNAPSHOT_CLOSED_REGIONS);
|
| 155 | + } else { |
| 156 | + setFailureState(new HBaseIOException( |
| 157 | + "Table " + snapshotTable + " is not in a valid state for snapshot: " + tableState)); |
| 158 | + return Flow.NO_MORE_STATE; |
153 | 159 | }
|
154 | 160 | return Flow.HAS_MORE_STATE;
|
155 | 161 | case SNAPSHOT_SNAPSHOT_ONLINE_REGIONS:
|
@@ -195,9 +201,7 @@ protected Flow executeFromState(MasterProcedureEnv env, SnapshotState state)
|
195 | 201 | } catch (ProcedureSuspendedException e) {
|
196 | 202 | throw e;
|
197 | 203 | } catch (Exception e) {
|
198 |
| - setFailure("master-snapshot", e); |
199 |
| - LOG.warn("unexpected exception while execute {}. Mark procedure Failed.", this, e); |
200 |
| - status.abort("Abort Snapshot " + snapshot.getName() + " on Table " + snapshotTable); |
| 204 | + setFailureState(e); |
201 | 205 | return Flow.NO_MORE_STATE;
|
202 | 206 | }
|
203 | 207 | }
|
@@ -236,6 +240,12 @@ protected SnapshotState getInitialState() {
|
236 | 240 | return SnapshotState.SNAPSHOT_PREPARE;
|
237 | 241 | }
|
238 | 242 |
|
| 243 | + private void setFailureState(Exception e) { |
| 244 | + setFailure("master-snapshot", e); |
| 245 | + LOG.warn("unexpected exception while execute {}. Mark procedure Failed.", this, e); |
| 246 | + status.abort("Abort Snapshot " + snapshot.getName() + " on Table " + snapshotTable); |
| 247 | + } |
| 248 | + |
239 | 249 | private void prepareSnapshot(MasterProcedureEnv env)
|
240 | 250 | throws ProcedureSuspendedException, IOException {
|
241 | 251 | if (isAnySplitOrMergeProcedureRunning(env)) {
|
|
0 commit comments