HBASE-18632 TestMultiParallel#testFlushCommitsWithAbort fails in master branch
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / backup / BackupObserver.java
blob595e862a2d06614afa8933692ff76b4f25a4ca38
1 /*
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing,
13 * software distributed under the License is distributed on an
14 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 * KIND, either express or implied. See the License for the
16 * specific language governing permissions and limitations
17 * under the License.
19 package org.apache.hadoop.hbase.backup;
21 import java.io.IOException;
22 import java.util.List;
23 import java.util.Map;
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.ObserverContext;
38 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
39 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
40 import org.apache.hadoop.hbase.util.Pair;
42 /**
43 * An Observer to facilitate backup operations
45 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
46 public class BackupObserver implements RegionObserver {
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 // there is no need to record state
55 return hasLoaded;
57 if (finalPaths == null || !BackupManager.isBackupEnabled(cfg)) {
58 LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
59 return hasLoaded;
61 try (Connection connection = ConnectionFactory.createConnection(cfg);
62 BackupSystemTable tbl = new BackupSystemTable(connection)) {
63 List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
64 HRegionInfo info = ctx.getEnvironment().getRegionInfo();
65 TableName tableName = info.getTable();
66 if (!fullyBackedUpTables.contains(tableName)) {
67 if (LOG.isTraceEnabled()) {
68 LOG.trace(tableName + " has not gone thru full backup");
70 return hasLoaded;
72 tbl.writePathsPostBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths);
73 return hasLoaded;
74 } catch (IOException ioe) {
75 LOG.error("Failed to get tables which have been fully backed up", ioe);
76 return false;
79 @Override
80 public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
81 final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
82 Configuration cfg = ctx.getEnvironment().getConfiguration();
83 if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
84 LOG.debug("skipping recording bulk load in preCommitStoreFile since backup is disabled");
85 return;
87 try (Connection connection = ConnectionFactory.createConnection(cfg);
88 BackupSystemTable tbl = new BackupSystemTable(connection)) {
89 List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
90 HRegionInfo info = ctx.getEnvironment().getRegionInfo();
91 TableName tableName = info.getTable();
92 if (!fullyBackedUpTables.contains(tableName)) {
93 if (LOG.isTraceEnabled()) {
94 LOG.trace(tableName + " has not gone thru full backup");
96 return;
98 tbl.writeFilesForBulkLoadPreCommit(tableName, info.getEncodedNameAsBytes(), family, pairs);
99 return;