001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *   http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing,
013 * software distributed under the License is distributed on an
014 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
015 * KIND, either express or implied.  See the License for the
016 * specific language governing permissions and limitations
017 * under the License.
018 */
019package org.apache.hadoop.hbase.backup;
020
021import java.io.IOException;
022import java.util.List;
023import java.util.Map;
024import java.util.Optional;
025
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.fs.Path;
028import org.apache.hadoop.hbase.HBaseInterfaceAudience;
029import org.apache.hadoop.hbase.TableName;
030import org.apache.hadoop.hbase.backup.impl.BackupManager;
031import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
032import org.apache.hadoop.hbase.client.Connection;
033import org.apache.hadoop.hbase.client.ConnectionFactory;
034import org.apache.hadoop.hbase.client.RegionInfo;
035import org.apache.hadoop.hbase.coprocessor.ObserverContext;
036import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
037import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
038import org.apache.hadoop.hbase.coprocessor.RegionObserver;
039import org.apache.hadoop.hbase.util.Pair;
040import org.apache.yetus.audience.InterfaceAudience;
041import org.slf4j.Logger;
042import org.slf4j.LoggerFactory;
043
044/**
045 * An Observer to facilitate backup operations
046 */
047@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
048public class BackupObserver implements RegionCoprocessor, RegionObserver {
049  private static final Logger LOG = LoggerFactory.getLogger(BackupObserver.class);
050
051  @Override
052  public Optional<RegionObserver> getRegionObserver() {
053    return Optional.of(this);
054  }
055
056  @Override
057  public void postBulkLoadHFile(ObserverContext<RegionCoprocessorEnvironment> ctx,
058    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths)
059        throws IOException {
060    Configuration cfg = ctx.getEnvironment().getConfiguration();
061    if (finalPaths == null) {
062      // there is no need to record state
063      return;
064    }
065    if (!BackupManager.isBackupEnabled(cfg)) {
066      LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
067      return;
068    }
069    try (Connection connection = ConnectionFactory.createConnection(cfg);
070        BackupSystemTable tbl = new BackupSystemTable(connection)) {
071      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
072      RegionInfo info = ctx.getEnvironment().getRegionInfo();
073      TableName tableName = info.getTable();
074      if (!fullyBackedUpTables.contains(tableName)) {
075        if (LOG.isTraceEnabled()) {
076          LOG.trace(tableName + " has not gone thru full backup");
077        }
078        return;
079      }
080      tbl.writePathsPostBulkLoad(tableName, info.getEncodedNameAsBytes(), finalPaths);
081    } catch (IOException ioe) {
082      LOG.error("Failed to get tables which have been fully backed up", ioe);
083    }
084  }
085  @Override
086  public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
087      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
088    Configuration cfg = ctx.getEnvironment().getConfiguration();
089    if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
090      LOG.debug("skipping recording bulk load in preCommitStoreFile since backup is disabled");
091      return;
092    }
093    try (Connection connection = ConnectionFactory.createConnection(cfg);
094        BackupSystemTable tbl = new BackupSystemTable(connection)) {
095      List<TableName> fullyBackedUpTables = tbl.getTablesForBackupType(BackupType.FULL);
096      RegionInfo info = ctx.getEnvironment().getRegionInfo();
097      TableName tableName = info.getTable();
098      if (!fullyBackedUpTables.contains(tableName)) {
099        if (LOG.isTraceEnabled()) {
100          LOG.trace(tableName + " has not gone thru full backup");
101        }
102        return;
103      }
104      tbl.writeFilesForBulkLoadPreCommit(tableName, info.getEncodedNameAsBytes(), family, pairs);
105      return;
106    }
107  }
108}