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
19 package org
.apache
.hadoop
.hbase
.backup
;
21 import java
.io
.IOException
;
22 import java
.util
.List
;
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
;
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);
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();
54 // there is no need to record state
57 if (finalPaths
== null || !BackupManager
.isBackupEnabled(cfg
)) {
58 LOG
.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
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");
72 tbl
.writePathsPostBulkLoad(tableName
, info
.getEncodedNameAsBytes(), finalPaths
);
74 } catch (IOException ioe
) {
75 LOG
.error("Failed to get tables which have been fully backed up", ioe
);
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");
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");
98 tbl
.writeFilesForBulkLoadPreCommit(tableName
, info
.getEncodedNameAsBytes(), family
, pairs
);