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;
20
21 import java.io.IOException;
22 import java.util.List;
23 import java.util.Map;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.Path;
29 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
30 import org.apache.hadoop.hbase.HRegionInfo;
31 import org.apache.hadoop.hbase.TableName;
32 import org.apache.hadoop.hbase.backup.impl.BackupManager;
33 import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
34 import org.apache.hadoop.hbase.classification.InterfaceAudience;
35 import org.apache.hadoop.hbase.client.Connection;
36 import org.apache.hadoop.hbase.client.ConnectionFactory;
37 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
38 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
39 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
40 import org.apache.hadoop.hbase.util.Pair;
41
42
43
44
45 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
46 public class BackupObserver extends BaseRegionObserver {
47 private static final Log LOG = LogFactory.getLog(BackupObserver.class);
48 @Override
49 public boolean postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
50 List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths,
51 boolean hasLoaded) throws IOException {
52 Configuration cfg = ctx.getEnvironment().getConfiguration();
53 if (!hasLoaded) {
54
55 return hasLoaded;
56 }
57 if (finalPaths == null || !BackupManager.isBackupEnabled(cfg)) {
58 LOG.debug("skipping recording bulk load in postBulkLoadHFile");
59 return hasLoaded;
60 }
61 List<TableName> fullyBackedUpTables;
62 try (Connection connection = ConnectionFactory.createConnection(cfg);
63 BackupSystemTable tbl = new BackupSystemTable(connection)) {
64 fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
65 HRegionInfo info = ctx.getEnvironment().getRegionInfo();
66 TableName tableName = info.getTable();
67 if (!fullyBackedUpTables.contains(tableName)) {
68 LOG.debug(tableName + " has not gone thru full backup");
69 return hasLoaded;
70 }
71 tbl.writeOrigBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths);
72 return hasLoaded;
73 } catch (IOException ioe) {
74 LOG.error("Failed to get tables which have been fully backed up", ioe);
75 return false;
76 }
77 }
78 @Override
79 public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
80 final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
81 Configuration cfg = ctx.getEnvironment().getConfiguration();
82 if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
83 LOG.debug("skipping recording bulk load in preCommitStoreFile");
84 return;
85 }
86 List<TableName> fullyBackedUpTables;
87 try (Connection connection = ConnectionFactory.createConnection(cfg);
88 BackupSystemTable tbl = new BackupSystemTable(connection)) {
89 fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
90 HRegionInfo info = ctx.getEnvironment().getRegionInfo();
91 TableName tableName = info.getTable();
92 if (!fullyBackedUpTables.contains(tableName)) {
93 LOG.debug(tableName + " has not gone thru full backup");
94 return;
95 }
96 tbl.writeOrigBulkLoad(tableName, info.getEncodedNameAsBytes(), family, pairs);
97 return;
98 }
99 }
100 }