1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.backup.master;
20
21 import java.io.IOException;
22 import java.io.InputStream;
23 import java.io.OutputStream;
24 import java.util.ArrayList;
25 import java.util.HashMap;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.Map.Entry;
29 import java.util.concurrent.atomic.AtomicBoolean;
30
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.fs.FileStatus;
35 import org.apache.hadoop.fs.FileSystem;
36 import org.apache.hadoop.fs.Path;
37 import org.apache.hadoop.hbase.HConstants;
38 import org.apache.hadoop.hbase.NotServingRegionException;
39 import org.apache.hadoop.hbase.TableName;
40 import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
41 import org.apache.hadoop.hbase.backup.BackupInfo;
42 import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
43 import org.apache.hadoop.hbase.backup.BackupType;
44 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
45 import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
46 import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
47 import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
48 import org.apache.hadoop.hbase.backup.BackupCopyService;
49 import org.apache.hadoop.hbase.backup.impl.BackupException;
50 import org.apache.hadoop.hbase.backup.impl.BackupManager;
51 import org.apache.hadoop.hbase.backup.impl.BackupManifest;
52 import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
53 import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
54 import org.apache.hadoop.hbase.classification.InterfaceAudience;
55 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
56 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
57 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
58 import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
59 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
60 import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
61 import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.FullTableBackupState;
62 import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp;
63 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
64 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
65 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
66 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
67 import org.apache.hadoop.hbase.util.FSUtils;
68
69 @InterfaceAudience.Private
70 public class FullTableBackupProcedure
71 extends StateMachineProcedure<MasterProcedureEnv, FullTableBackupState>
72 implements TableProcedureInterface {
73 private static final Log LOG = LogFactory.getLog(FullTableBackupProcedure.class);
74
75 private static final String SNAPSHOT_BACKUP_MAX_ATTEMPTS_KEY = "hbase.backup.snapshot.attempts.max";
76 private static final int DEFAULT_SNAPSHOT_BACKUP_MAX_ATTEMPTS = 10;
77
78 private static final String SNAPSHOT_BACKUP_ATTEMPTS_DELAY_KEY = "hbase.backup.snapshot.attempts.delay";
79 private static final int DEFAULT_SNAPSHOT_BACKUP_ATTEMPTS_DELAY = 10000;
80
81 private final AtomicBoolean aborted = new AtomicBoolean(false);
82 private Configuration conf;
83 private String backupId;
84 private List<TableName> tableList;
85 private String targetRootDir;
86 HashMap<String, Long> newTimestamps = null;
87
88 private BackupManager backupManager;
89 private BackupInfo backupInfo;
90
91 public FullTableBackupProcedure() {
92
93 }
94
95 public FullTableBackupProcedure(final MasterProcedureEnv env,
96 final String backupId, List<TableName> tableList, String targetRootDir, final int workers,
97 final long bandwidth) throws IOException {
98 backupManager = new BackupManager(env.getMasterConfiguration());
99 this.backupId = backupId;
100 this.tableList = tableList;
101 this.targetRootDir = targetRootDir;
102 backupInfo =
103 backupManager.createBackupInfo(backupId, BackupType.FULL, tableList,
104 targetRootDir, workers, bandwidth);
105 if (tableList == null || tableList.isEmpty()) {
106 this.tableList = new ArrayList<>(backupInfo.getTables());
107 }
108 }
109
110 @Override
111 public byte[] getResult() {
112 return backupId.getBytes();
113 }
114
115
116
117
118
119
120 static void beginBackup(BackupManager backupManager, BackupInfo backupInfo)
121 throws IOException {
122 backupManager.setBackupInfo(backupInfo);
123
124 long startTs = EnvironmentEdgeManager.currentTime();
125 backupInfo.setStartTs(startTs);
126
127 backupInfo.setState(BackupState.RUNNING);
128 LOG.info("Backup " + backupInfo.getBackupId() + " started at " + startTs + ".");
129
130 backupManager.updateBackupInfo(backupInfo);
131 if (LOG.isDebugEnabled()) {
132 LOG.debug("Backup session " + backupInfo.getBackupId() + " has been started.");
133 }
134 }
135
136 private static String getMessage(Exception e) {
137 String msg = e.getMessage();
138 if (msg == null || msg.equals("")) {
139 msg = e.getClass().getName();
140 }
141 return msg;
142 }
143
144
145
146
147
148
149 private static void deleteSnapshot(final MasterProcedureEnv env,
150 BackupInfo backupCtx, Configuration conf)
151 throws IOException {
152 LOG.debug("Trying to delete snapshot for full backup.");
153 for (String snapshotName : backupCtx.getSnapshotNames()) {
154 if (snapshotName == null) {
155 continue;
156 }
157 LOG.debug("Trying to delete snapshot: " + snapshotName);
158 HBaseProtos.SnapshotDescription.Builder builder =
159 HBaseProtos.SnapshotDescription.newBuilder();
160 builder.setName(snapshotName);
161 try {
162 env.getMasterServices().getSnapshotManager().deleteSnapshot(builder.build());
163 } catch (IOException ioe) {
164 LOG.debug("when deleting snapshot " + snapshotName, ioe);
165 }
166 LOG.debug("Deleting the snapshot " + snapshotName + " for backup "
167 + backupCtx.getBackupId() + " succeeded.");
168 }
169 }
170
171
172
173
174
175
176 private static void cleanupExportSnapshotLog(Configuration conf) throws IOException {
177 FileSystem fs = FSUtils.getCurrentFileSystem(conf);
178 Path stagingDir =
179 new Path(conf.get(BackupRestoreConstants.CONF_STAGING_ROOT, fs.getWorkingDirectory()
180 .toString()));
181 FileStatus[] files = FSUtils.listStatus(fs, stagingDir);
182 if (files == null) {
183 return;
184 }
185 for (FileStatus file : files) {
186 if (file.getPath().getName().startsWith("exportSnapshot-")) {
187 LOG.debug("Delete log files of exporting snapshot: " + file.getPath().getName());
188 if (FSUtils.delete(fs, file.getPath(), true) == false) {
189 LOG.warn("Can not delete " + file.getPath());
190 }
191 }
192 }
193 }
194
195
196
197
198
199 static void cleanupTargetDir(BackupInfo backupInfo, Configuration conf) {
200 try {
201
202
203 LOG.debug("Trying to cleanup up target dir. Current backup phase: "
204 + backupInfo.getPhase());
205 if (backupInfo.getPhase().equals(BackupPhase.SNAPSHOTCOPY)
206 || backupInfo.getPhase().equals(BackupPhase.INCREMENTAL_COPY)
207 || backupInfo.getPhase().equals(BackupPhase.STORE_MANIFEST)) {
208 FileSystem outputFs =
209 FileSystem.get(new Path(backupInfo.getTargetRootDir()).toUri(), conf);
210
211
212
213 for (TableName table : backupInfo.getTables()) {
214 Path targetDirPath =
215 new Path(HBackupFileSystem.getTableBackupDir(backupInfo.getTargetRootDir(),
216 backupInfo.getBackupId(), table));
217 if (outputFs.delete(targetDirPath, true)) {
218 LOG.info("Cleaning up uncompleted backup data at " + targetDirPath.toString()
219 + " done.");
220 } else {
221 LOG.info("No data has been copied to " + targetDirPath.toString() + ".");
222 }
223
224 Path tableDir = targetDirPath.getParent();
225 FileStatus[] backups = FSUtils.listStatus(outputFs, tableDir);
226 if (backups == null || backups.length == 0) {
227 outputFs.delete(tableDir, true);
228 LOG.debug(tableDir.toString() + " is empty, remove it.");
229 }
230 }
231 }
232
233 } catch (IOException e1) {
234 LOG.error("Cleaning up uncompleted backup data of " + backupInfo.getBackupId() + " at "
235 + backupInfo.getTargetRootDir() + " failed due to " + e1.getMessage() + ".");
236 }
237 }
238
239
240
241
242
243
244
245 static void failBackup(final MasterProcedureEnv env, BackupInfo backupInfo,
246 BackupManager backupManager, Exception e,
247 String msg, BackupType type, Configuration conf) throws IOException {
248 LOG.error(msg + getMessage(e), e);
249
250
251
252 backupInfo.setEndTs(EnvironmentEdgeManager.currentTime());
253
254
255 backupInfo.setFailedMsg(e.getMessage());
256
257
258 backupInfo.setState(BackupState.FAILED);
259
260
261 String backupFailedData =
262 "BackupId=" + backupInfo.getBackupId() + ",startts=" + backupInfo.getStartTs()
263 + ",failedts=" + backupInfo.getEndTs() + ",failedphase=" + backupInfo.getPhase()
264 + ",failedmessage=" + backupInfo.getFailedMsg();
265 LOG.error(backupFailedData);
266
267 backupManager.updateBackupInfo(backupInfo);
268
269
270
271 if (type == BackupType.FULL) {
272 deleteSnapshot(env, backupInfo, conf);
273 cleanupExportSnapshotLog(conf);
274 }
275
276
277
278
279 cleanupTargetDir(backupInfo, conf);
280
281 LOG.info("Backup " + backupInfo.getBackupId() + " failed.");
282 }
283
284
285
286
287
288
289 private void snapshotCopy(BackupInfo backupInfo) throws Exception {
290 LOG.info("Snapshot copy is starting.");
291
292
293 backupInfo.setPhase(BackupPhase.SNAPSHOTCOPY);
294
295
296
297 BackupCopyService copyService = BackupRestoreServerFactory.getBackupCopyService(conf);
298
299
300 float numOfSnapshots = backupInfo.getSnapshotNames().size();
301
302 LOG.debug("There are " + (int) numOfSnapshots + " snapshots to be copied.");
303
304 for (TableName table : backupInfo.getTables()) {
305
306
307
308 int res = 0;
309 String[] args = new String[4];
310 args[0] = "-snapshot";
311 args[1] = backupInfo.getSnapshotName(table);
312 args[2] = "-copy-to";
313 args[3] = backupInfo.getBackupStatus(table).getTargetDir();
314
315 LOG.debug("Copy snapshot " + args[1] + " to " + args[3]);
316 res = copyService.copy(backupInfo, backupManager, conf, BackupCopyService.Type.FULL, args);
317
318 if (res != 0) {
319 LOG.error("Exporting Snapshot " + args[1] + " failed with return code: " + res + ".");
320
321 throw new IOException("Failed of exporting snapshot " + args[1] + " to " + args[3]
322 + " with reason code " + res);
323 }
324
325 LOG.info("Snapshot copy " + args[1] + " finished.");
326 }
327 }
328
329
330
331
332
333
334
335
336 private static void addManifest(BackupInfo backupInfo, BackupManager backupManager,
337 BackupType type, Configuration conf) throws IOException, BackupException {
338
339 backupInfo.setPhase(BackupPhase.STORE_MANIFEST);
340
341 BackupManifest manifest;
342
343
344
345 for (TableName table : backupInfo.getTables()) {
346 manifest = new BackupManifest(backupInfo, table);
347 ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupInfo, table);
348 for (BackupImage image : ancestors) {
349 manifest.addDependentImage(image);
350 }
351
352 if (type == BackupType.INCREMENTAL) {
353
354 HashMap<TableName, HashMap<String, Long>> tableTimestampMap =
355 new HashMap<TableName, HashMap<String, Long>>();
356 tableTimestampMap.put(table, backupInfo.getIncrTimestampMap().get(table));
357 manifest.setIncrTimestampMap(tableTimestampMap);
358 ArrayList<BackupImage> ancestorss = backupManager.getAncestors(backupInfo);
359 for (BackupImage image : ancestorss) {
360 manifest.addDependentImage(image);
361 }
362 }
363 manifest.store(conf);
364 }
365
366
367
368
369 if (type == BackupType.INCREMENTAL) {
370 manifest = new BackupManifest(backupInfo);
371
372 manifest.setIncrTimestampMap(backupInfo.getIncrTimestampMap());
373 ArrayList<BackupImage> ancestors = backupManager.getAncestors(backupInfo);
374 for (BackupImage image : ancestors) {
375 manifest.addDependentImage(image);
376 }
377 manifest.store(conf);
378 }
379 }
380
381
382
383
384
385
386 private static String obtainBackupMetaDataStr(BackupInfo backupInfo) {
387 StringBuffer sb = new StringBuffer();
388 sb.append("type=" + backupInfo.getType() + ",tablelist=");
389 for (TableName table : backupInfo.getTables()) {
390 sb.append(table + ";");
391 }
392 if (sb.lastIndexOf(";") > 0) {
393 sb.delete(sb.lastIndexOf(";"), sb.lastIndexOf(";") + 1);
394 }
395 sb.append(",targetRootDir=" + backupInfo.getTargetRootDir());
396
397 return sb.toString();
398 }
399
400
401
402
403
404
405 private static void cleanupDistCpLog(BackupInfo backupInfo, Configuration conf)
406 throws IOException {
407 Path rootPath = new Path(backupInfo.getHLogTargetDir()).getParent();
408 FileSystem fs = FileSystem.get(rootPath.toUri(), conf);
409 FileStatus[] files = FSUtils.listStatus(fs, rootPath);
410 if (files == null) {
411 return;
412 }
413 for (FileStatus file : files) {
414 if (file.getPath().getName().startsWith("_distcp_logs")) {
415 LOG.debug("Delete log files of DistCp: " + file.getPath().getName());
416 FSUtils.delete(fs, file.getPath(), true);
417 }
418 }
419 }
420
421
422
423
424
425
426 static void completeBackup(final MasterProcedureEnv env, BackupInfo backupInfo,
427 BackupManager backupManager, BackupType type, Configuration conf) throws IOException {
428
429 backupInfo.setEndTs(EnvironmentEdgeManager.currentTime());
430
431 backupInfo.setState(BackupState.COMPLETE);
432 backupInfo.setProgress(100);
433
434 addManifest(backupInfo, backupManager, type, conf);
435
436
437
438 LOG.debug("in-fly convert code here, provided by future jira");
439
440
441 String backupCompleteData =
442 obtainBackupMetaDataStr(backupInfo) + ",startts=" + backupInfo.getStartTs()
443 + ",completets=" + backupInfo.getEndTs() + ",bytescopied="
444 + backupInfo.getTotalBytesCopied();
445 if (LOG.isDebugEnabled()) {
446 LOG.debug("Backup " + backupInfo.getBackupId() + " finished: " + backupCompleteData);
447 }
448 backupManager.updateBackupInfo(backupInfo);
449
450
451
452
453
454 if (type == BackupType.FULL) {
455 deleteSnapshot(env, backupInfo, conf);
456 cleanupExportSnapshotLog(conf);
457 } else if (type == BackupType.INCREMENTAL) {
458 cleanupDistCpLog(backupInfo, conf);
459 }
460
461 LOG.info("Backup " + backupInfo.getBackupId() + " completed.");
462 }
463
464
465
466
467
468
469 static SnapshotDescription wrapSnapshotDescription(TableName tableName, String snapshotName) {
470
471
472 HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
473 builder.setTable(tableName.getNameAsString());
474 builder.setName(snapshotName);
475 HBaseProtos.SnapshotDescription backupSnapshot = builder.build();
476
477 LOG.debug("Wrapped a SnapshotDescription " + backupSnapshot.getName()
478 + " from backupInfo to request snapshot for backup.");
479
480 return backupSnapshot;
481 }
482
483 @Override
484 protected Flow executeFromState(final MasterProcedureEnv env, final FullTableBackupState state) {
485 if (conf == null) {
486 conf = env.getMasterConfiguration();
487 }
488 if (backupManager == null) {
489 try {
490 backupManager = new BackupManager(env.getMasterConfiguration());
491 } catch (IOException ioe) {
492 setFailure("full backup", ioe);
493 return Flow.NO_MORE_STATE;
494 }
495 }
496 if (LOG.isTraceEnabled()) {
497 LOG.trace(this + " execute state=" + state);
498 }
499 try {
500 switch (state) {
501 case PRE_SNAPSHOT_TABLE:
502 beginBackup(backupManager, backupInfo);
503 String savedStartCode = null;
504 boolean firstBackup = false;
505
506
507 try {
508 savedStartCode = backupManager.readBackupStartCode();
509 firstBackup = savedStartCode == null || Long.parseLong(savedStartCode) == 0L;
510 if (firstBackup) {
511
512
513 backupManager.writeBackupStartCode(0L);
514 }
515
516
517
518
519
520 LOG.info("Execute roll log procedure for full backup ...");
521 MasterProcedureManager mpm = env.getMasterServices().getMasterProcedureManagerHost()
522 .getProcedureManager(LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE);
523 Map<String, String> props= new HashMap<String, String>();
524 props.put("backupRoot", backupInfo.getTargetRootDir());
525 long waitTime = MasterProcedureUtil.execProcedure(mpm,
526 LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
527 LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props);
528 MasterProcedureUtil.waitForProcedure(mpm,
529 LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_SIGNATURE,
530 LogRollMasterProcedureManager.ROLLLOG_PROCEDURE_NAME, props, waitTime,
531 conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
532 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER),
533 conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
534 HConstants.DEFAULT_HBASE_CLIENT_PAUSE));
535
536 newTimestamps = backupManager.readRegionServerLastLogRollResult();
537 if (firstBackup) {
538
539
540
541
542 List<String> logFiles = BackupServerUtil.getWALFilesOlderThan(conf, newTimestamps);
543 backupManager.recordWALFiles(logFiles);
544 }
545 } catch (BackupException e) {
546 setFailure("Failure in full-backup: pre-snapshot phase", e);
547
548 failBackup(env, backupInfo, backupManager, e, "Unexpected BackupException : ",
549 BackupType.FULL, conf);
550 return Flow.NO_MORE_STATE;
551 }
552 setNextState(FullTableBackupState.SNAPSHOT_TABLES);
553 break;
554 case SNAPSHOT_TABLES:
555 for (TableName tableName : tableList) {
556 String snapshotName = "snapshot_" + Long.toString(EnvironmentEdgeManager.currentTime())
557 + "_" + tableName.getNamespaceAsString() + "_" + tableName.getQualifierAsString();
558 HBaseProtos.SnapshotDescription backupSnapshot;
559
560
561 backupSnapshot = wrapSnapshotDescription(tableName,snapshotName);
562 try {
563 env.getMasterServices().getSnapshotManager().deleteSnapshot(backupSnapshot);
564 } catch (IOException e) {
565 LOG.debug("Unable to delete " + snapshotName, e);
566 }
567
568 snapshotTable(env, backupSnapshot);
569 backupInfo.setSnapshotName(tableName, backupSnapshot.getName());
570 }
571 setNextState(FullTableBackupState.SNAPSHOT_COPY);
572 break;
573 case SNAPSHOT_COPY:
574
575 LOG.debug("snapshot copy for " + backupId);
576 try {
577 this.snapshotCopy(backupInfo);
578 } catch (Exception e) {
579 setFailure("Failure in full-backup: snapshot copy phase" + backupId, e);
580
581 failBackup(env, backupInfo, backupManager, e, "Unexpected BackupException : ",
582 BackupType.FULL, conf);
583 return Flow.NO_MORE_STATE;
584 }
585
586 backupManager.addIncrementalBackupTableSet(backupInfo.getTables());
587 setNextState(FullTableBackupState.BACKUP_COMPLETE);
588 break;
589
590 case BACKUP_COMPLETE:
591
592
593 backupInfo.setState(BackupState.COMPLETE);
594
595
596 backupManager.writeRegionServerLogTimestamp(backupInfo.getTables(), newTimestamps);
597
598 HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
599 backupManager.readLogTimestampMap();
600
601 Long newStartCode =
602 BackupClientUtil.getMinValue(BackupServerUtil.getRSLogTimestampMins(newTableSetTimestampMap));
603 backupManager.writeBackupStartCode(newStartCode);
604
605
606 completeBackup(env, backupInfo, backupManager, BackupType.FULL, conf);
607 return Flow.NO_MORE_STATE;
608
609 default:
610 throw new UnsupportedOperationException("unhandled state=" + state);
611 }
612 } catch (IOException e) {
613 LOG.error("Backup failed in " + state);
614 setFailure("snapshot-table", e);
615 }
616 return Flow.HAS_MORE_STATE;
617 }
618
619 private void snapshotTable(final MasterProcedureEnv env, SnapshotDescription backupSnapshot)
620 throws IOException
621 {
622
623 int maxAttempts = env.getMasterConfiguration().getInt(SNAPSHOT_BACKUP_MAX_ATTEMPTS_KEY,
624 DEFAULT_SNAPSHOT_BACKUP_MAX_ATTEMPTS);
625 int delay = env.getMasterConfiguration().getInt(SNAPSHOT_BACKUP_ATTEMPTS_DELAY_KEY,
626 DEFAULT_SNAPSHOT_BACKUP_ATTEMPTS_DELAY);
627 int attempts = 0;
628
629 while (attempts++ < maxAttempts) {
630 try {
631 env.getMasterServices().getSnapshotManager().takeSnapshot(backupSnapshot);
632 long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(
633 env.getMasterConfiguration(),
634 backupSnapshot.getType(), SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
635 BackupServerUtil.waitForSnapshot(backupSnapshot, waitTime,
636 env.getMasterServices().getSnapshotManager(), env.getMasterConfiguration());
637 break;
638 } catch( Exception ee) {
639 LOG.warn("Snapshot attempt "+attempts +" failed for table "+backupSnapshot.getTable() +
640 ", sleeping for " + delay+"ms", ee);
641 if(attempts < maxAttempts) {
642 try {
643 Thread.sleep(delay);
644 } catch (InterruptedException e) {
645 Thread.currentThread().interrupt();
646 }
647 }
648 }
649 }
650 }
651 @Override
652 protected void rollbackState(final MasterProcedureEnv env, final FullTableBackupState state)
653 throws IOException {
654 if (state != FullTableBackupState.PRE_SNAPSHOT_TABLE) {
655 deleteSnapshot(env, backupInfo, conf);
656 cleanupExportSnapshotLog(conf);
657 }
658
659
660
661
662 if (state == FullTableBackupState.SNAPSHOT_COPY) {
663 cleanupTargetDir(backupInfo, conf);
664 }
665 }
666
667 @Override
668 protected FullTableBackupState getState(final int stateId) {
669 return FullTableBackupState.valueOf(stateId);
670 }
671
672 @Override
673 protected int getStateId(final FullTableBackupState state) {
674 return state.getNumber();
675 }
676
677 @Override
678 protected FullTableBackupState getInitialState() {
679 return FullTableBackupState.PRE_SNAPSHOT_TABLE;
680 }
681
682 @Override
683 protected void setNextState(final FullTableBackupState state) {
684 if (aborted.get()) {
685 setAbortFailure("backup-table", "abort requested");
686 } else {
687 super.setNextState(state);
688 }
689 }
690
691 @Override
692 public boolean abort(final MasterProcedureEnv env) {
693 aborted.set(true);
694 return true;
695 }
696
697 @Override
698 public void toStringClassDetails(StringBuilder sb) {
699 sb.append(getClass().getSimpleName());
700 sb.append(" (targetRootDir=");
701 sb.append(targetRootDir);
702 sb.append("; backupId=").append(backupId);
703 sb.append("; tables=");
704 int len = tableList.size();
705 for (int i = 0; i < len-1; i++) {
706 sb.append(tableList.get(i)).append(",");
707 }
708 if (len >= 1) sb.append(tableList.get(len-1));
709 sb.append(")");
710 }
711
712 BackupProtos.BackupProcContext toBackupInfo() {
713 BackupProtos.BackupProcContext.Builder ctxBuilder = BackupProtos.BackupProcContext.newBuilder();
714 ctxBuilder.setCtx(backupInfo.toProtosBackupInfo());
715 if (newTimestamps != null && !newTimestamps.isEmpty()) {
716 BackupProtos.ServerTimestamp.Builder tsBuilder = ServerTimestamp.newBuilder();
717 for (Entry<String, Long> entry : newTimestamps.entrySet()) {
718 tsBuilder.clear().setServer(entry.getKey()).setTimestamp(entry.getValue());
719 ctxBuilder.addServerTimestamp(tsBuilder.build());
720 }
721 }
722 return ctxBuilder.build();
723 }
724
725 @Override
726 public void serializeStateData(final OutputStream stream) throws IOException {
727 super.serializeStateData(stream);
728
729 BackupProtos.BackupProcContext backupProcCtx = toBackupInfo();
730 backupProcCtx.writeDelimitedTo(stream);
731 }
732
733 @Override
734 public void deserializeStateData(final InputStream stream) throws IOException {
735 super.deserializeStateData(stream);
736
737 BackupProtos.BackupProcContext proto =BackupProtos.BackupProcContext.parseDelimitedFrom(stream);
738 backupInfo = BackupInfo.fromProto(proto.getCtx());
739 backupId = backupInfo.getBackupId();
740 targetRootDir = backupInfo.getTargetRootDir();
741 tableList = backupInfo.getTableNames();
742 List<ServerTimestamp> svrTimestamps = proto.getServerTimestampList();
743 if (svrTimestamps != null && !svrTimestamps.isEmpty()) {
744 newTimestamps = new HashMap<>();
745 for (ServerTimestamp ts : svrTimestamps) {
746 newTimestamps.put(ts.getServer(), ts.getTimestamp());
747 }
748 }
749 }
750
751 @Override
752 public TableName getTableName() {
753 return TableName.BACKUP_TABLE_NAME;
754 }
755
756 @Override
757 public TableOperationType getTableOperationType() {
758 return TableOperationType.BACKUP;
759 }
760
761 @Override
762 protected boolean acquireLock(final MasterProcedureEnv env) {
763 if (!env.isInitialized() && !getTableName().isSystemTable()) {
764 return false;
765 }
766 return env.getProcedureQueue().tryAcquireTableWrite(getTableName(), "full backup");
767
768
769
770
771
772
773 }
774
775 @Override
776 protected void releaseLock(final MasterProcedureEnv env) {
777 env.getProcedureQueue().releaseTableWrite(getTableName());
778 }
779 }