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.ArrayList;
022import java.util.Collections;
023import java.util.List;
024import java.util.Map;
025import java.util.Optional;
026import java.util.Set;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.fs.Path;
029import org.apache.hadoop.hbase.HBaseInterfaceAudience;
030import org.apache.hadoop.hbase.TableName;
031import org.apache.hadoop.hbase.backup.impl.BackupManager;
032import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
033import org.apache.hadoop.hbase.client.Connection;
034import org.apache.hadoop.hbase.client.ConnectionFactory;
035import org.apache.hadoop.hbase.client.RegionInfo;
036import org.apache.hadoop.hbase.coprocessor.ObserverContext;
037import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
038import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
039import org.apache.hadoop.hbase.coprocessor.RegionObserver;
040import org.apache.hadoop.hbase.util.Pair;
041import org.apache.yetus.audience.InterfaceAudience;
042import org.slf4j.Logger;
043import org.slf4j.LoggerFactory;
044
045/**
046 * An Observer to facilitate backup operations
047 */
048@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
049public class BackupObserver implements RegionCoprocessor, RegionObserver {
050  private static final Logger LOG = LoggerFactory.getLogger(BackupObserver.class);
051
052  @Override
053  public Optional<RegionObserver> getRegionObserver() {
054    return Optional.of(this);
055  }
056
057  @Override
058  public void postBulkLoadHFile(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
059    List<Pair<byte[], String>> stagingFamilyPaths, Map<byte[], List<Path>> finalPaths)
060    throws IOException {
061    Configuration cfg = ctx.getEnvironment().getConfiguration();
062    if (finalPaths == null) {
063      // there is no need to record state
064      return;
065    }
066    if (!BackupManager.isBackupEnabled(cfg)) {
067      LOG.debug("skipping recording bulk load in postBulkLoadHFile since backup is disabled");
068      return;
069    }
070
071    registerBulkLoad(ctx, finalPaths);
072  }
073
074  @Override
075  public void preCommitStoreFile(final ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
076    final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
077    Configuration cfg = ctx.getEnvironment().getConfiguration();
078    if (pairs == null || pairs.isEmpty() || !BackupManager.isBackupEnabled(cfg)) {
079      LOG.debug("skipping recording bulk load in preCommitStoreFile since backup is disabled");
080      return;
081    }
082
083    List<Path> hfiles = new ArrayList<>(pairs.size());
084    for (Pair<Path, Path> pair : pairs) {
085      hfiles.add(pair.getSecond());
086    }
087    registerBulkLoad(ctx, Collections.singletonMap(family, hfiles));
088  }
089
090  private void registerBulkLoad(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
091    Map<byte[], List<Path>> cfToHFilePaths) throws IOException {
092    Configuration cfg = ctx.getEnvironment().getConfiguration();
093    RegionInfo region = ctx.getEnvironment().getRegionInfo();
094    TableName tableName = region.getTable();
095
096    try (Connection connection = ConnectionFactory.createConnection(cfg);
097      BackupSystemTable tbl = new BackupSystemTable(connection)) {
098      Set<TableName> fullyBackedUpTables = tbl.getTablesIncludedInBackups();
099
100      if (fullyBackedUpTables.contains(tableName)) {
101        tbl.registerBulkLoad(tableName, region.getEncodedNameAsBytes(), cfToHFilePaths);
102      } else {
103        if (LOG.isTraceEnabled()) {
104          LOG.trace("Table {} has not gone through full backup - skipping.", tableName);
105        }
106      }
107    }
108  }
109}