Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.backup.replication;

import org.apache.yetus.audience.InterfaceAudience;

@InterfaceAudience.Private
public class BackupConfigurationException extends Exception {
public BackupConfigurationException(String message) {
super(message);
}

public BackupConfigurationException(String message, Throwable cause) {
super(message, cause);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.backup.replication;

import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@InterfaceAudience.Private
public class BackupFileSystemManager {
private static final Logger LOG = LoggerFactory.getLogger(BackupFileSystemManager.class);

public static final String WALS_DIR = "WALs";
public static final String BULKLOAD_FILES_DIR = "bulk-load-files";
private final String peerId;
private final FileSystem backupFs;
private final Path backupRootDir;
private Path walsDir;
private Path bulkLoadFilesDir;

public BackupFileSystemManager(String peerId, Configuration conf, String backupRootDirStr)
throws IOException {
this.peerId = peerId;
this.backupRootDir = new Path(backupRootDirStr);
this.backupFs = FileSystem.get(backupRootDir.toUri(), conf);
initBackupDirectories();
}

private void initBackupDirectories() throws IOException {
LOG.info("{} Initializing backup directories under root: {}", Utils.logPeerId(peerId),
backupRootDir);
try {
walsDir = createDirectoryIfNotExists(WALS_DIR);
bulkLoadFilesDir = createDirectoryIfNotExists(BULKLOAD_FILES_DIR);
} catch (IOException e) {
LOG.error("{} Failed to initialize backup directories: {}", Utils.logPeerId(peerId),
e.getMessage(), e);
throw e;
}
}

private Path createDirectoryIfNotExists(String dirName) throws IOException {
Path dirPath = new Path(backupRootDir, dirName);
if (backupFs.exists(dirPath)) {
LOG.info("{} Directory already exists: {}", Utils.logPeerId(peerId), dirPath);
} else {
backupFs.mkdirs(dirPath);
LOG.info("{} Successfully created directory: {}", Utils.logPeerId(peerId), dirPath);
}
Comment on lines +63 to +68
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How many HBase instances are running this code? Isn't there a race condition here?
Is this run by master only?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will run on region servers. I believe we can simplify the logic by directly using backupFs.mkdirs(dirPath). It will create the directory if it doesn't exist, and simply return true if the directory already exists.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That would be better.

return dirPath;
}

public Path getWalsDir() {
return walsDir;
}

public Path getBulkLoadFilesDir() {
return bulkLoadFilesDir;
}

public FileSystem getBackupFs() {
return backupFs;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.backup.replication;

import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.yetus.audience.InterfaceAudience;

import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;

/**
* Processes bulk load files from Write-Ahead Log (WAL) entries for HBase replication.
* <p>
* This utility class extracts and constructs the file paths of bulk-loaded files based on WAL
* entries. It processes bulk load descriptors and their associated store descriptors to generate
* the paths for each bulk-loaded file.
* <p>
* The class is designed for scenarios where replicable bulk load operations need to be parsed and
* their file paths need to be determined programmatically.
* </p>
*/
@InterfaceAudience.Private
public final class BulkLoadProcessor {
private BulkLoadProcessor() {
}

public static List<Path> processBulkLoadFiles(TableName tableName, List<WAL.Entry> walEntries)
throws IOException {
List<Path> bulkLoadFilePaths = new ArrayList<>();
String namespace = tableName.getNamespaceAsString();
String table = tableName.getQualifierAsString();

for (WAL.Entry entry : walEntries) {
WALEdit edit = entry.getEdit();
for (Cell cell : edit.getCells()) {
if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
bulkLoadFilePaths.addAll(processBulkLoadDescriptor(cell, namespace, table));
}
}
}
return bulkLoadFilePaths;
}

private static List<Path> processBulkLoadDescriptor(Cell cell, String namespace, String table)
throws IOException {
List<Path> bulkLoadFilePaths = new ArrayList<>();
WALProtos.BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);

if (bld == null || !bld.getReplicate() || bld.getEncodedRegionName() == null) {
return bulkLoadFilePaths; // Skip if not replicable
}

String regionName = bld.getEncodedRegionName().toStringUtf8();
for (WALProtos.StoreDescriptor storeDescriptor : bld.getStoresList()) {
bulkLoadFilePaths
.addAll(processStoreDescriptor(storeDescriptor, namespace, table, regionName));
}

return bulkLoadFilePaths;
}

private static List<Path> processStoreDescriptor(WALProtos.StoreDescriptor storeDescriptor,
String namespace, String table, String regionName) {
List<Path> paths = new ArrayList<>();
String columnFamily = storeDescriptor.getFamilyName().toStringUtf8();

for (String storeFile : storeDescriptor.getStoreFileList()) {
paths.add(new Path(namespace,
new Path(table, new Path(regionName, new Path(columnFamily, storeFile)))));
}

return paths;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,182 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.backup.replication;

import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Manages the continuous backup process for HBase WAL entries and bulk load files.
* <p>
* This class is responsible for initializing backup components, processing WAL entries, staging
* them for backup, and committing the backup to the configured storage. It uses
* {@link BackupFileSystemManager} for handling file system operations and
* {@link ContinuousBackupStagingManager} for managing staging.
* </p>
*/
@InterfaceAudience.Private
public class ContinuousBackupManager {
private static final Logger LOG = LoggerFactory.getLogger(ContinuousBackupManager.class);
public static final String CONF_BACKUP_ROOT_DIR = "hbase.backup.root.dir";
public static final String CONF_BACKUP_MAX_WAL_SIZE = "hbase.backup.max.wal.size";
public static final long DEFAULT_MAX_WAL_SIZE = 128 * 1024 * 1024;
private final Configuration conf;
private final String peerId;
private final BackupFileSystemManager backupFileSystemManager;
private final ContinuousBackupStagingManager stagingManager;

/**
* Constructs a {@code ContinuousBackupManager} instance with the specified peer ID and
* configuration.
* @param peerId the unique identifier of the replication peer
* @param conf the HBase configuration object
* @throws BackupConfigurationException if the backup configuration is invalid
*/
public ContinuousBackupManager(String peerId, Configuration conf)
throws BackupConfigurationException {
this.peerId = peerId;
this.conf = conf;
String backupRootDirStr = conf.get(CONF_BACKUP_ROOT_DIR);
if (backupRootDirStr == null || backupRootDirStr.isEmpty()) {
String errorMsg = Utils.logPeerId(peerId)
+ " Backup root directory not specified. Set it using " + CONF_BACKUP_ROOT_DIR;
LOG.error(errorMsg);
throw new BackupConfigurationException(errorMsg);
}
LOG.debug("{} Backup root directory: {}", Utils.logPeerId(peerId), backupRootDirStr);

try {
this.backupFileSystemManager = new BackupFileSystemManager(peerId, conf, backupRootDirStr);
LOG.info("{} BackupFileSystemManager initialized successfully.", Utils.logPeerId(peerId));
} catch (IOException e) {
String errorMsg = Utils.logPeerId(peerId) + " Failed to initialize BackupFileSystemManager";
LOG.error(errorMsg, e);
throw new BackupConfigurationException(errorMsg, e);
}

try {
this.stagingManager = new ContinuousBackupStagingManager(conf, this);
LOG.info("{} ContinuousBackupStagingManager initialized successfully.",
Utils.logPeerId(peerId));
} catch (IOException e) {
String errorMsg = "Failed to initialize ContinuousBackupStagingManager";
LOG.error(errorMsg, e);
throw new BackupConfigurationException(errorMsg, e);
}
}

/**
* Backs up the provided WAL entries grouped by table.
* <p>
* The method processes WAL entries, identifies bulk load files, stages them, and prepares them
* for backup.
* </p>
* @param tableToEntriesMap a map of table names to WAL entries
* @throws IOException if an error occurs during the backup process
*/
public void backup(Map<TableName, List<WAL.Entry>> tableToEntriesMap) throws IOException {
LOG.debug("{} Starting backup process for {} table(s)", Utils.logPeerId(peerId),
tableToEntriesMap.size());

for (Map.Entry<TableName, List<WAL.Entry>> entry : tableToEntriesMap.entrySet()) {
TableName tableName = entry.getKey();
List<WAL.Entry> walEntries = entry.getValue();

LOG.debug("{} Processing {} WAL entries for table: {}", Utils.logPeerId(peerId),
walEntries.size(), tableName);

List<Path> bulkLoadFiles = BulkLoadProcessor.processBulkLoadFiles(tableName, walEntries);
LOG.debug("{} Identified {} bulk load file(s) for table: {}", Utils.logPeerId(peerId),
bulkLoadFiles.size(), tableName);

stagingManager.stageEntries(tableName, walEntries, bulkLoadFiles);
LOG.debug("{} Staged WAL entries and bulk load files for table: {}", Utils.logPeerId(peerId),
tableName);
}

LOG.debug("{} Backup process completed for all tables.", Utils.logPeerId(peerId));
}

/**
* Commits the backup for a given WAL file and its associated bulk load files.
* <p>
* This method copies the WAL file and bulk load files from the staging area to the configured
* backup directory.
* </p>
* @param sourceFs the source file system where the files are currently staged
* @param walFile the WAL file to back up
* @param bulkLoadFiles a list of bulk load files associated with the WAL file
* @throws IOException if an error occurs while committing the backup
*/
public void commitBackup(FileSystem sourceFs, Path walFile, List<Path> bulkLoadFiles)
throws IOException {
LOG.debug("{} Starting commit for WAL file: {}", Utils.logPeerId(peerId), walFile);

Path sourcePath = stagingManager.getWalFileStagingPath(walFile);
Path backupWalPath = new Path(backupFileSystemManager.getWalsDir(), walFile);

try {
FileUtil.copy(sourceFs, sourcePath, backupFileSystemManager.getBackupFs(), backupWalPath,
false, conf);
LOG.info("{} WAL file {} successfully backed up to {}", Utils.logPeerId(peerId), walFile,
backupWalPath);
} catch (IOException e) {
LOG.error("{} Failed to back up WAL file: {}", Utils.logPeerId(peerId), walFile, e);
throw e;
}

uploadBulkLoadFiles(sourceFs, bulkLoadFiles);
LOG.debug("{} Commit completed for WAL file: {}", Utils.logPeerId(peerId), walFile);
}

private void uploadBulkLoadFiles(FileSystem sourceFs, List<Path> bulkLoadFiles)
throws IOException {
for (Path file : bulkLoadFiles) {
Path sourcePath = stagingManager.getBulkloadFileStagingPath(file);
Path destPath = new Path(backupFileSystemManager.getBulkLoadFilesDir(), file);

try {
FileUtil.copy(sourceFs, sourcePath, backupFileSystemManager.getBackupFs(), destPath, false,
conf);
LOG.info("{} Bulk load file {} successfully backed up to {}", Utils.logPeerId(peerId), file,
destPath);
} catch (IOException e) {
LOG.error("{} Failed to back up bulk load file: {}", Utils.logPeerId(peerId), file, e);
throw e;
}
}
}

public void close() {
stagingManager.close();
}

public String getPeerId() {
return peerId;
}
}
Loading