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